Merge pull request #12325: [BEAM-10543] Upgrade Kafka cross-language python tests

diff --git a/CHANGES.md b/CHANGES.md
index 04f3094..66dc68c 100644
--- a/CHANGES.md
+++ b/CHANGES.md
@@ -64,6 +64,7 @@
 * Add cross-language support to Java's JdbcIO, now available in the Python module `apache_beam.io.external.jdbc` ([BEAM-10135](https://issues.apache.org/jira/browse/BEAM-10135), [BEAM-10136](https://issues.apache.org/jira/browse/BEAM-10136)).
 * Add support of AWS SDK v2 for KinesisIO.Read (Java) ([BEAM-9702](https://issues.apache.org/jira/browse/BEAM-9702)).
 * Support for X source added (Java/Python) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)).
+* Add streaming support to SnowflakeIO in Java SDK ([BEAM-9896](https://issues.apache.org/jira/browse/BEAM-9896  ))
 
 ## New Features / Improvements
 
@@ -100,6 +101,7 @@
   reading data by exporting to JSON files. This has small differences in behavior for Time and Date-related fields. See
   Pydoc for more information.
 * Add dispositions for SnowflakeIO.write ([BEAM-10343](https://issues.apache.org/jira/browse/BEAM-10343))
+* Add cross-language support to SnowflakeIO.Read([BEAM-9897](https://issues.apache.org/jira/browse/BEAM-9897)).
 
 ## New Features / Improvements
 
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 3622764..82541ef 100644
--- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
+++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
@@ -391,7 +391,7 @@
     def gax_version = "1.54.0"
     def generated_grpc_ga_version = "1.85.1"
     def google_auth_version = "0.19.0"
-    def google_clients_version = "1.30.9"
+    def google_clients_version = "1.30.10"
     def google_cloud_bigdataoss_version = "2.1.3"
     def google_cloud_core_version = "1.92.2"
     def google_cloud_pubsublite_version = "0.1.6"
@@ -469,13 +469,13 @@
         google_api_client_jackson2                  : "com.google.api-client:google-api-client-jackson2:$google_clients_version",
         google_api_client_java6                     : "com.google.api-client:google-api-client-java6:$google_clients_version",
         google_api_common                           : "com.google.api:api-common:1.8.1",
-        google_api_services_bigquery                : "com.google.apis:google-api-services-bigquery:v2-rev20191211-$google_clients_version",
-        google_api_services_clouddebugger           : "com.google.apis:google-api-services-clouddebugger:v2-rev20200313-$google_clients_version",
-        google_api_services_cloudresourcemanager    : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20200311-$google_clients_version",
-        google_api_services_dataflow                : "com.google.apis:google-api-services-dataflow:v1b3-rev20200305-$google_clients_version",
-        google_api_services_healthcare              : "com.google.apis:google-api-services-healthcare:v1beta1-rev20200525-$google_clients_version",
-        google_api_services_pubsub                  : "com.google.apis:google-api-services-pubsub:v1-rev20200312-$google_clients_version",
-        google_api_services_storage                 : "com.google.apis:google-api-services-storage:v1-rev20200226-$google_clients_version",
+        google_api_services_bigquery                : "com.google.apis:google-api-services-bigquery:v2-rev20200719-$google_clients_version",
+        google_api_services_clouddebugger           : "com.google.apis:google-api-services-clouddebugger:v2-rev20200501-$google_clients_version",
+        google_api_services_cloudresourcemanager    : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20200720-$google_clients_version",
+        google_api_services_dataflow                : "com.google.apis:google-api-services-dataflow:v1b3-rev20200713-$google_clients_version",
+        google_api_services_healthcare              : "com.google.apis:google-api-services-healthcare:v1beta1-rev20200713-$google_clients_version",
+        google_api_services_pubsub                  : "com.google.apis:google-api-services-pubsub:v1-rev20200713-$google_clients_version",
+        google_api_services_storage                 : "com.google.apis:google-api-services-storage:v1-rev20200611-$google_clients_version",
         google_auth_library_credentials             : "com.google.auth:google-auth-library-credentials:$google_auth_version",
         google_auth_library_oauth2_http             : "com.google.auth:google-auth-library-oauth2-http:$google_auth_version",
         google_cloud_bigquery                       : "com.google.cloud:google-cloud-bigquery:1.108.0",
diff --git a/examples/java/build.gradle b/examples/java/build.gradle
index 0c91573..eaad35f 100644
--- a/examples/java/build.gradle
+++ b/examples/java/build.gradle
@@ -52,6 +52,7 @@
   compile project(path: ":sdks:java:core", configuration: "shadow")
   compile project(":sdks:java:extensions:google-cloud-platform-core")
   compile project(":sdks:java:io:google-cloud-platform")
+  compile project(":sdks:java:extensions:ml")
   compile library.java.avro
   compile library.java.bigdataoss_util
   compile library.java.google_api_client
@@ -60,6 +61,7 @@
   compile library.java.google_auth_library_credentials
   compile library.java.google_auth_library_oauth2_http
   compile library.java.google_cloud_datastore_v1_proto_client
+  compile library.java.google_code_gson
   compile library.java.google_http_client
   compile library.java.joda_time
   compile library.java.proto_google_cloud_datastore_v1
@@ -67,6 +69,7 @@
   compile library.java.slf4j_jdk14
   runtime project(path: ":runners:direct-java", configuration: "shadow")
   testCompile project(":sdks:java:io:google-cloud-platform")
+  testCompile project(":sdks:java:extensions:ml")
   testCompile library.java.hamcrest_core
   testCompile library.java.hamcrest_library
   testCompile library.java.junit
diff --git a/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java b/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java
index 3393e6d..cb55475 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java
@@ -22,6 +22,14 @@
 import com.google.api.services.bigquery.model.TableRow;
 import com.google.api.services.bigquery.model.TableSchema;
 import com.google.api.services.bigquery.model.TimePartitioning;
+import com.google.cloud.language.v1.AnnotateTextRequest;
+import com.google.cloud.language.v1.AnnotateTextResponse;
+import com.google.cloud.language.v1.Document;
+import com.google.cloud.language.v1.Entity;
+import com.google.cloud.language.v1.Sentence;
+import com.google.cloud.language.v1.Token;
+import com.google.gson.Gson;
+import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Base64;
@@ -30,12 +38,14 @@
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.AvroCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.DefaultCoder;
 import org.apache.beam.sdk.coders.DoubleCoder;
+import org.apache.beam.sdk.extensions.ml.AnnotateText;
 import org.apache.beam.sdk.io.Compression;
 import org.apache.beam.sdk.io.FileIO;
 import org.apache.beam.sdk.io.GenerateSequence;
@@ -63,6 +73,7 @@
 import org.apache.beam.sdk.transforms.MapElements;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.PeriodicImpulse;
+import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.transforms.Sum;
 import org.apache.beam.sdk.transforms.View;
 import org.apache.beam.sdk.transforms.Watch;
@@ -984,4 +995,167 @@
       return result;
     }
   }
+
+  public static class NaturalLanguageIntegration {
+    private static final SerializableFunction<AnnotateTextResponse, List<Map<String, List<String>>>>
+        // [START NlpAnalyzeDependencyTree]
+        analyzeDependencyTree =
+            (SerializableFunction<AnnotateTextResponse, List<Map<String, List<String>>>>)
+                response -> {
+                  List<Map<String, List<String>>> adjacencyLists = new ArrayList<>();
+                  int index = 0;
+                  for (Sentence s : response.getSentencesList()) {
+                    Map<String, List<String>> adjacencyMap = new HashMap<>();
+                    int sentenceBegin = s.getText().getBeginOffset();
+                    int sentenceEnd = sentenceBegin + s.getText().getContent().length() - 1;
+                    while (index < response.getTokensCount()
+                        && response.getTokens(index).getText().getBeginOffset() <= sentenceEnd) {
+                      Token token = response.getTokensList().get(index);
+                      int headTokenIndex = token.getDependencyEdge().getHeadTokenIndex();
+                      String headTokenContent =
+                          response.getTokens(headTokenIndex).getText().getContent();
+                      List<String> adjacencyList =
+                          adjacencyMap.getOrDefault(headTokenContent, new ArrayList<>());
+                      adjacencyList.add(token.getText().getContent());
+                      adjacencyMap.put(headTokenContent, adjacencyList);
+                      index++;
+                    }
+                    adjacencyLists.add(adjacencyMap);
+                  }
+                  return adjacencyLists;
+                };
+    // [END NlpAnalyzeDependencyTree]
+
+    private static final SerializableFunction<? super AnnotateTextResponse, TextSentiments>
+        // [START NlpExtractSentiments]
+        extractSentiments =
+        (SerializableFunction<AnnotateTextResponse, TextSentiments>)
+            annotateTextResponse -> {
+              TextSentiments sentiments = new TextSentiments();
+              sentiments.setDocumentSentiment(
+                  annotateTextResponse.getDocumentSentiment().getMagnitude());
+              Map<String, Float> sentenceSentimentsMap =
+                  annotateTextResponse.getSentencesList().stream()
+                      .collect(
+                          Collectors.toMap(
+                              (Sentence s) -> s.getText().getContent(),
+                              (Sentence s) -> s.getSentiment().getMagnitude()));
+              sentiments.setSentenceSentiments(sentenceSentimentsMap);
+              return sentiments;
+            };
+    // [END NlpExtractSentiments]
+
+    private static final SerializableFunction<? super AnnotateTextResponse, Map<String, String>>
+        // [START NlpExtractEntities]
+        extractEntities =
+        (SerializableFunction<AnnotateTextResponse, Map<String, String>>)
+            annotateTextResponse ->
+                annotateTextResponse.getEntitiesList().stream()
+                    .collect(
+                        Collectors.toMap(Entity::getName, (Entity e) -> e.getType().toString()));
+    // [END NlpExtractEntities]
+
+    private static final SerializableFunction<? super Map<String, String>, String>
+        mapEntitiesToJson =
+            (SerializableFunction<Map<String, String>, String>)
+                item -> {
+                  StringBuilder builder = new StringBuilder("[");
+                  builder.append(
+                      item.entrySet().stream()
+                          .map(
+                              entry -> "{\"" + entry.getKey() + "\": \"" + entry.getValue() + "\"}")
+                          .collect(Collectors.joining(",")));
+                  builder.append("]");
+                  return builder.toString();
+                };
+
+    private static final SerializableFunction<List<Map<String, List<String>>>, String>
+        mapDependencyTreesToJson =
+            (SerializableFunction<List<Map<String, List<String>>>, String>)
+                tree -> {
+                  Gson gson = new Gson();
+                  return gson.toJson(tree);
+                };
+
+    public static void main(Pipeline p) {
+      // [START NlpAnalyzeText]
+      AnnotateTextRequest.Features features =
+          AnnotateTextRequest.Features.newBuilder()
+              .setExtractEntities(true)
+              .setExtractDocumentSentiment(true)
+              .setExtractEntitySentiment(true)
+              .setExtractSyntax(true)
+              .build();
+      AnnotateText annotateText = AnnotateText.newBuilder().setFeatures(features).build();
+
+      PCollection<AnnotateTextResponse> responses =
+          p.apply(
+                  Create.of(
+                      "My experience so far has been fantastic, "
+                          + "I\'d really recommend this product."))
+              .apply(
+                  MapElements.into(TypeDescriptor.of(Document.class))
+                      .via(
+                          (SerializableFunction<String, Document>)
+                              input ->
+                                  Document.newBuilder()
+                                      .setContent(input)
+                                      .setType(Document.Type.PLAIN_TEXT)
+                                      .build()))
+              .apply(annotateText);
+
+      responses
+          .apply(MapElements.into(TypeDescriptor.of(TextSentiments.class)).via(extractSentiments))
+          .apply(
+              MapElements.into(TypeDescriptors.strings())
+                  .via((SerializableFunction<TextSentiments, String>) TextSentiments::toJson))
+          .apply(TextIO.write().to("sentiments.txt"));
+
+      responses
+          .apply(
+              MapElements.into(
+                      TypeDescriptors.maps(TypeDescriptors.strings(), TypeDescriptors.strings()))
+                  .via(extractEntities))
+          .apply(MapElements.into(TypeDescriptors.strings()).via(mapEntitiesToJson))
+          .apply(TextIO.write().to("entities.txt"));
+
+      responses
+          .apply(
+              MapElements.into(
+                      TypeDescriptors.lists(
+                          TypeDescriptors.maps(
+                              TypeDescriptors.strings(),
+                              TypeDescriptors.lists(TypeDescriptors.strings()))))
+                  .via(analyzeDependencyTree))
+          .apply(MapElements.into(TypeDescriptors.strings()).via(mapDependencyTreesToJson))
+          .apply(TextIO.write().to("adjacency_list.txt"));
+      // [END NlpAnalyzeText]
+    }
+
+    private static class TextSentiments implements Serializable {
+      private Float documentSentiment;
+      private Map<String, Float> sentenceSentiments;
+
+      public void setSentenceSentiments(Map<String, Float> sentenceSentiments) {
+        this.sentenceSentiments = sentenceSentiments;
+      }
+
+      public Float getDocumentSentiment() {
+        return documentSentiment;
+      }
+
+      public void setDocumentSentiment(Float documentSentiment) {
+        this.documentSentiment = documentSentiment;
+      }
+
+      public Map<String, Float> getSentenceSentiments() {
+        return sentenceSentiments;
+      }
+
+      public String toJson() {
+        Gson gson = new Gson();
+        return gson.toJson(this);
+      }
+    }
+  }
 }
diff --git a/learning/katas/go/core_transforms/section-info.yaml b/learning/katas/go/core_transforms/section-info.yaml
index 32ae3aa..bcdab5d 100644
--- a/learning/katas/go/core_transforms/section-info.yaml
+++ b/learning/katas/go/core_transforms/section-info.yaml
@@ -28,3 +28,4 @@
 - additional_outputs
 - branching
 - composite
+- windowing
diff --git a/learning/katas/go/core_transforms/windowing/lesson-info.yaml b/learning/katas/go/core_transforms/windowing/lesson-info.yaml
new file mode 100644
index 0000000..2315b4c
--- /dev/null
+++ b/learning/katas/go/core_transforms/windowing/lesson-info.yaml
@@ -0,0 +1,21 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#  http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+content:
+- windowing
diff --git a/learning/katas/go/core_transforms/windowing/lesson-remote-info.yaml b/learning/katas/go/core_transforms/windowing/lesson-remote-info.yaml
new file mode 100644
index 0000000..7a20a3a
--- /dev/null
+++ b/learning/katas/go/core_transforms/windowing/lesson-remote-info.yaml
@@ -0,0 +1,3 @@
+id: 387853
+update_date: Thu, 06 Aug 2020 17:53:20 UTC
+unit: 377026
diff --git a/learning/katas/go/core_transforms/windowing/windowing/cmd/main.go b/learning/katas/go/core_transforms/windowing/windowing/cmd/main.go
new file mode 100644
index 0000000..e1cfcf0
--- /dev/null
+++ b/learning/katas/go/core_transforms/windowing/windowing/cmd/main.go
@@ -0,0 +1,48 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements.  See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package main
+
+import (
+	"beam.apache.org/learning/katas/core_transforms/windowing/windowing/pkg/common"
+	"beam.apache.org/learning/katas/core_transforms/windowing/windowing/pkg/task"
+	"context"
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/go/pkg/beam/x/beamx"
+	"github.com/apache/beam/sdks/go/pkg/beam/x/debug"
+)
+
+func main() {
+	ctx := context.Background()
+
+	p, s := beam.NewPipelineWithRoot()
+
+	input := common.CreateLines(s)
+
+	result := task.ApplyTransform(s, input)
+
+	output := beam.ParDo(s, func(commit task.Commit) string {
+		return commit.String()
+	}, result)
+
+	debug.Print(s, output)
+
+	err := beamx.Run(ctx, p)
+
+	if err != nil {
+		log.Exitf(context.Background(), "Failed to execute job: %v", err)
+	}
+}
diff --git a/learning/katas/go/core_transforms/windowing/windowing/pkg/common/input.go b/learning/katas/go/core_transforms/windowing/windowing/pkg/common/input.go
new file mode 100644
index 0000000..3353e2b
--- /dev/null
+++ b/learning/katas/go/core_transforms/windowing/windowing/pkg/common/input.go
@@ -0,0 +1,42 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements.  See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package common
+
+import (
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+	"time"
+)
+
+var (
+	lines = map[time.Time]string{
+		time.Date(2020, 7, 31, 15, 52, 5, 0, time.UTC):  "3c6c45924a Remove trailing whitespace from README",
+		time.Date(2020, 7, 31, 15, 59, 40, 0, time.UTC): "a52be99b62 Merge pull request #12443 from KevinGG/whitespace",
+		time.Date(2020, 7, 31, 16, 7, 36, 0, time.UTC):  "7c1772d13f Merge pull request #12439 from ibzib/beam-9199-1",
+		time.Date(2020, 7, 31, 16, 35, 41, 0, time.UTC): "d971ba13b8 Widen ranges for GCP libraries (#12198)",
+		time.Date(2020, 8, 1, 0, 7, 25, 0, time.UTC):    "875620111b Enable all Jenkins jobs triggering for committers (#12407)",
+	}
+)
+
+func CreateLines(s beam.Scope) beam.PCollection {
+	return beam.ParDo(s, timestampFn, beam.Impulse(s))
+}
+
+func timestampFn(_ []byte, emit func(beam.EventTime, string)) {
+	for timestamp, line := range lines {
+		emit(mtime.FromTime(timestamp), line)
+	}
+}
diff --git a/learning/katas/go/core_transforms/windowing/windowing/pkg/task/task.go b/learning/katas/go/core_transforms/windowing/windowing/pkg/task/task.go
new file mode 100644
index 0000000..1838578
--- /dev/null
+++ b/learning/katas/go/core_transforms/windowing/windowing/pkg/task/task.go
@@ -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 task
+
+import (
+	"fmt"
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/window"
+	"time"
+)
+
+func ApplyTransform(s beam.Scope, input beam.PCollection) beam.PCollection {
+	windowed := beam.WindowInto(s, window.NewFixedWindows(time.Hour), input)
+	return beam.ParDo(s, timestampFn, windowed)
+}
+
+func timestampFn(iw beam.Window, et beam.EventTime, line string) Commit {
+	return Commit{
+		MaxTimestampWindow: toTime(iw.MaxTimestamp()),
+		EventTimestamp:     toTime(et),
+		Line:               line,
+	}
+}
+
+func toTime(et beam.EventTime) time.Time {
+	return time.Unix(0, et.Milliseconds() * int64(time.Millisecond))
+}
+
+type Commit struct {
+	MaxTimestampWindow time.Time
+	EventTimestamp     time.Time
+	Line               string
+}
+
+func (c Commit) String() string {
+	return fmt.Sprintf("Window ending at: %v contains timestamp: %v for commit: \"%s\"",
+		c.MaxTimestampWindow.Format(time.Kitchen),
+		c.EventTimestamp.Format(time.Kitchen),
+		c.Line)
+}
+
diff --git a/learning/katas/go/core_transforms/windowing/windowing/task-info.yaml b/learning/katas/go/core_transforms/windowing/windowing/task-info.yaml
new file mode 100644
index 0000000..b0e38f7
--- /dev/null
+++ b/learning/katas/go/core_transforms/windowing/windowing/task-info.yaml
@@ -0,0 +1,42 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#  http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+type: edu
+files:
+- name: test/task_test.go
+  visible: false
+- name: cmd/main.go
+  visible: true
+- name: pkg/common/input.go
+  visible: true
+- name: pkg/task/task.go
+  visible: true
+  placeholders:
+  - offset: 1030
+    length: 60
+    placeholder_text: TODO()
+  - offset: 1099
+    length: 36
+    placeholder_text: TODO()
+  - offset: 1156
+    length: 46
+    placeholder_text: TODO()
+  - offset: 1221
+    length: 121
+    placeholder_text: TODO()
diff --git a/learning/katas/go/core_transforms/windowing/windowing/task-remote-info.yaml b/learning/katas/go/core_transforms/windowing/windowing/task-remote-info.yaml
new file mode 100644
index 0000000..235e663
--- /dev/null
+++ b/learning/katas/go/core_transforms/windowing/windowing/task-remote-info.yaml
@@ -0,0 +1,2 @@
+id: 1464828
+update_date: Thu, 06 Aug 2020 17:53:26 UTC
diff --git a/learning/katas/go/core_transforms/windowing/windowing/task.md b/learning/katas/go/core_transforms/windowing/windowing/task.md
new file mode 100644
index 0000000..22444a6
--- /dev/null
+++ b/learning/katas/go/core_transforms/windowing/windowing/task.md
@@ -0,0 +1,77 @@
+<!--
+  ~ 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.
+  -->
+
+# Windowing
+
+This lesson introduces the concept of windowed PCollection elements.  A window is a view into a fixed beginning and 
+fixed end to a set of data.  In the beam model, windowing subdivides a PCollection according to the
+timestamps of its individual elements.  An element can be a part of one or more windows.
+
+A DoFn can request timestamp and windowing information about the element it is processing.  All the previous lessons 
+had this information available as well.  This lesson makes use of these parameters.  The simple dataset 
+has five git commit messages and their timestamps from the 
+[Apache Beam public repository](https://github.com/apache/beam).  Timestamps have been applied to this PCollection
+input according to the date and time of these messages.
+
+**Kata:** This lesson challenges you to apply an hourly fixed window to a PCollection.  You are then to 
+apply a ParDo to that hourly fixed windowed PCollection to produce a PCollection of a Commit struct.  The
+Commit struct is provided for you.  You are encouraged to run the pipeline at cmd/main.go of this task 
+to visualize the windowing and timestamps.
+
+<div class="hint">
+    Use <a href="https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam#ParDo">
+    beam.ParDo</a>
+    with a <a href="https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam#hdr-DoFns">
+    DoFn</a> to accomplish this lesson.
+</div>
+
+<div class="hint">
+    Use <a href="https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam#WindowInto">
+    beam.WindowInto</a>
+    with <a href="https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam/core/graph/window#NewFixedWindows">
+    window.NewFixedWindows(time.Hour)</a>
+    on your PCollection input to apply an hourly windowing strategy to each element.
+</div>
+
+<div class="hint">
+    To access <a href="https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam#Window">
+    beam.Window</a>
+    and <a href="https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam#EventTime">
+    beam.EventTime</a> in your DoFn, add the parameters in the set order.
+
+```
+func doFn(iw beam.Window, et beam.EventTime, element X) Y {
+    // do something with iw, et and element to return Y
+}
+```
+</div>
+
+<div class="hint">
+    The Commit struct provided for you has a MaxTimestampWindow property that can be set from
+    <a href="https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam#Window">
+    beam.Window</a>'s MaxTimestamp().
+</div>
+
+<div class="hint">
+    Refer to the Beam Programming Guide for additional information about 
+    <a href="https://beam.apache.org/documentation/programming-guide/#other-dofn-parameters">
+    additional DoFn parameters</a> and
+    <a href="https://beam.apache.org/documentation/programming-guide/#windowing">
+    windowing</a>.
+</div>
+
diff --git a/learning/katas/go/core_transforms/windowing/windowing/test/task_test.go b/learning/katas/go/core_transforms/windowing/windowing/test/task_test.go
new file mode 100644
index 0000000..8d4a2e2
--- /dev/null
+++ b/learning/katas/go/core_transforms/windowing/windowing/test/task_test.go
@@ -0,0 +1,72 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements.  See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package test
+
+import (
+	"beam.apache.org/learning/katas/core_transforms/windowing/windowing/pkg/common"
+	"beam.apache.org/learning/katas/core_transforms/windowing/windowing/pkg/task"
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/ptest"
+	"github.com/google/go-cmp/cmp"
+	"testing"
+	"time"
+)
+
+func TestApplyTransform(t *testing.T) {
+	p, s := beam.NewPipelineWithRoot()
+	tests := []struct {
+		input beam.PCollection
+		want []interface{}
+	}{
+		{
+			input: common.CreateLines(s),
+			want: []interface{}{
+				task.Commit{
+					MaxTimestampWindow: time.Unix(1596211199, 0),
+					EventTimestamp:     time.Unix(1596210725, 0),
+					Line:               "3c6c45924a Remove trailing whitespace from README",
+				},
+				task.Commit{
+					MaxTimestampWindow: time.Unix(1596211199, 0),
+					EventTimestamp:     time.Unix(1596211180, 0),
+					Line:               "a52be99b62 Merge pull request #12443 from KevinGG/whitespace",
+				},
+				task.Commit{
+					MaxTimestampWindow: time.Unix(1596214799, 0),
+					EventTimestamp:     time.Unix(1596211656, 0),
+					Line:               "7c1772d13f Merge pull request #12439 from ibzib/beam-9199-1",
+				},
+				task.Commit{
+					MaxTimestampWindow: time.Unix(1596214799, 0),
+					EventTimestamp:     time.Unix(1596213341, 0),
+					Line:               "d971ba13b8 Widen ranges for GCP libraries (#12198)",
+				},
+				task.Commit{
+					MaxTimestampWindow: time.Unix(1596243599, 0),
+					EventTimestamp:     time.Unix(1596240445, 0),
+					Line:               "875620111b Enable all Jenkins jobs triggering for committers (#12407)",
+				},
+			},
+		},
+	}
+	for _, tt := range tests {
+		got := task.ApplyTransform(s, tt.input)
+		cmp.Equal(got, tt.want)
+		if err := ptest.Run(p); err != nil {
+			t.Error(err)
+		}
+	}
+}
diff --git a/learning/katas/go/course-remote-info.yaml b/learning/katas/go/course-remote-info.yaml
index 90e7821..e944389 100644
--- a/learning/katas/go/course-remote-info.yaml
+++ b/learning/katas/go/course-remote-info.yaml
@@ -1,2 +1,2 @@
 id: 70387
-update_date: Mon, 27 Jul 2020 20:44:48 UTC
+update_date: Wed, 29 Jul 2020 20:42:36 UTC
diff --git a/model/fn-execution/src/main/resources/org/apache/beam/model/fnexecution/v1/standard_coders.yaml b/model/fn-execution/src/main/resources/org/apache/beam/model/fnexecution/v1/standard_coders.yaml
index 80e4eb8..08b823d 100644
--- a/model/fn-execution/src/main/resources/org/apache/beam/model/fnexecution/v1/standard_coders.yaml
+++ b/model/fn-execution/src/main/resources/org/apache/beam/model/fnexecution/v1/standard_coders.yaml
@@ -347,7 +347,6 @@
   urn: "beam:coder:row:v1"
   # str: string, i32: int32, f64: float64, arr: array[string]
   payload: "\n\t\n\x03str\x1a\x02\x10\x07\n\t\n\x03i32\x1a\x02\x10\x03\n\t\n\x03f64\x1a\x02\x10\x06\n\r\n\x03arr\x1a\x06\x1a\x04\n\x02\x10\x07\x12$4e5e554c-d4c1-4a5d-b5e1-f3293a6b9f05"
-nested: false
 examples:
     "\u0004\u0000\u0003foo\u00a9\u0046\u003f\u00b9\u0099\u0099\u0099\u0099\u0099\u009a\0\0\0\u0003\u0003foo\u0003bar\u0003baz": {str: "foo", i32: 9001, f64: "0.1", arr: ["foo", "bar", "baz"]}
 
@@ -357,7 +356,6 @@
   urn: "beam:coder:row:v1"
   # str: nullable string, i32: nullable int32, f64: nullable float64
   payload: "\n\x0b\n\x03str\x1a\x04\x08\x01\x10\x07\n\x0b\n\x03i32\x1a\x04\x08\x01\x10\x03\n\x0b\n\x03f64\x1a\x04\x08\x01\x10\x06\x12$b20c6545-57af-4bc8-b2a9-51ace21c7393"
-nested: false
 examples:
   "\u0003\u0001\u0007": {str: null, i32: null, f64: null}
   "\u0003\u0001\u0004\u0003foo\u00a9\u0046": {str: "foo", i32: 9001, f64: null}
@@ -380,7 +378,33 @@
   urn: "beam:coder:row:v1"
   # f_bool: boolean, f_bytes: nullable bytes
   payload: "\n\x0c\n\x06f_bool\x1a\x02\x10\x08\n\x0f\n\x07f_bytes\x1a\x04\x08\x01\x10\t\x12$eea1b747-7571-43d3-aafa-9255afdceafb"
-nested: false
 examples:
   "\x02\x01\x02\x01": {f_bool: True, f_bytes: null}
   "\x02\x00\x00\x04ab\x00c": {f_bool: False, f_bytes: "ab\0c"}
+
+---
+
+# Binary data generated with the python SDK:
+#
+# import typing
+# import apache_beam as beam
+# class Test(typing.NamedTuple):
+#   f_map: typing.Mapping[str,int]
+# schema = beam.typehints.schemas.named_tuple_to_schema(Test)
+# coder = beam.coders.row_coder.RowCoder(schema)
+# print("payload = %s" % schema.SerializeToString())
+# examples = (Test(f_map={}),
+#             Test(f_map={"foo": 9001, "bar": 9223372036854775807}),
+#             Test(f_map={"everything": None, "is": None, "null!": None, "¯\_(ツ)_/¯": None}))
+# for example in examples:
+#   print("example = %s" % coder.encode(example))
+coder:
+  urn: "beam:coder:row:v1"
+  # f_map: map<str, nullable int64>
+  payload: "\n\x15\n\x05f_map\x1a\x0c*\n\n\x02\x10\x07\x12\x04\x08\x01\x10\x04\x12$d8c8f969-14e6-457f-a8b5-62a1aec7f1cd"
+  # map ordering is non-deterministic
+  non_deterministic: True
+examples:
+  "\x01\x00\x00\x00\x00\x00": {f_map: {}}
+  "\x01\x00\x00\x00\x00\x02\x03foo\x01\xa9F\x03bar\x01\xff\xff\xff\xff\xff\xff\xff\xff\x7f": {f_map: {"foo": 9001, "bar": 9223372036854775807}}
+  "\x01\x00\x00\x00\x00\x04\neverything\x00\x02is\x00\x05null!\x00\r\xc2\xaf\\_(\xe3\x83\x84)_/\xc2\xaf\x00": {f_map: {"everything":null, "is": null, "null!": null, "¯\\_(ツ)_/¯": null}}
diff --git a/model/pipeline/src/main/proto/beam_runner_api.proto b/model/pipeline/src/main/proto/beam_runner_api.proto
index 3623790..8b1ce6b 100644
--- a/model/pipeline/src/main/proto/beam_runner_api.proto
+++ b/model/pipeline/src/main/proto/beam_runner_api.proto
@@ -855,10 +855,21 @@
     //     BOOLEAN:   beam:coder:bool:v1
     //     BYTES:     beam:coder:bytes:v1
     //   ArrayType:   beam:coder:iterable:v1 (always has a known length)
-    //   MapType:     not yet a standard coder (BEAM-7996)
+    //   MapType:     not a standard coder, specification defined below.
     //   RowType:     beam:coder:row:v1
     //   LogicalType: Uses the coder for its representation.
     //
+    // The MapType is encoded by:
+    //   - An INT32 representing the size of the map (N)
+    //   - Followed by N interleaved keys and values, encoded with their
+    //     corresponding coder.
+    //
+    // Nullable types in container types (ArrayType, MapType) are encoded by:
+    //   - A one byte null indicator, 0x00 for null values, or 0x01 for present
+    //     values.
+    //   - For present values the null indicator is followed by the value
+    //     encoded with it's corresponding coder.
+    //
     // The payload for RowCoder is an instance of Schema.
     // Components: None
     // Experimental.
diff --git a/model/pipeline/src/main/proto/schema.proto b/model/pipeline/src/main/proto/schema.proto
index dcf75ca..bffa5f1 100644
--- a/model/pipeline/src/main/proto/schema.proto
+++ b/model/pipeline/src/main/proto/schema.proto
@@ -19,6 +19,9 @@
 // ** Experimental **
 // Protocol Buffers describing Beam Schemas, a portable representation for
 // complex types.
+//
+// The primary application of Schema is as the payload for the standard coder
+// "beam:coder:row:v1", defined in beam_runner_api.proto
 
 syntax = "proto3";
 
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CommonCoderTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CommonCoderTest.java
index 10221b8..2c4090e 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CommonCoderTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CommonCoderTest.java
@@ -21,7 +21,6 @@
 import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects.firstNonNull;
 import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
 import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList.toImmutableList;
-import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap.toImmutableMap;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.hasItem;
 import static org.hamcrest.Matchers.instanceOf;
@@ -43,6 +42,7 @@
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -340,6 +340,10 @@
   }
 
   private static Object parseField(Object value, Schema.FieldType fieldType) {
+    if (value == null) {
+      return null;
+    }
+
     switch (fieldType.getTypeName()) {
       case BYTE:
         return ((Number) value).byteValue();
@@ -366,14 +370,18 @@
                 .map((element) -> parseField(element, fieldType.getCollectionElementType()))
                 .collect(toImmutableList());
       case MAP:
-        Map<Object, Object> kvMap = (Map<Object, Object>) value;
-        return kvMap.entrySet().stream()
-            .collect(
-                toImmutableMap(
-                    (pair) -> parseField(pair.getKey(), fieldType.getMapKeyType()),
-                    (pair) -> parseField(pair.getValue(), fieldType.getMapValueType())));
+        Map<Object, Object> kvMap = new HashMap<>();
+        ((Map<Object, Object>) value)
+            .entrySet().stream()
+                .forEach(
+                    (entry) ->
+                        kvMap.put(
+                            parseField(entry.getKey(), fieldType.getMapKeyType()),
+                            parseField(entry.getValue(), fieldType.getMapValueType())));
+        return kvMap;
       case ROW:
-        Map<String, Object> rowMap = (Map<String, Object>) value;
+        // Clone map so we don't mutate the underlying value
+        Map<String, Object> rowMap = new HashMap<>((Map<String, Object>) value);
         Schema schema = fieldType.getRowSchema();
         Row.Builder row = Row.withSchema(schema);
         for (Schema.Field field : schema.getFields()) {
diff --git a/sdks/go/pkg/beam/core/graph/coder/map.go b/sdks/go/pkg/beam/core/graph/coder/map.go
new file mode 100644
index 0000000..4e5dc2c
--- /dev/null
+++ b/sdks/go/pkg/beam/core/graph/coder/map.go
@@ -0,0 +1,102 @@
+// 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 coder
+
+import (
+	"io"
+	"reflect"
+)
+
+// TODO(lostluck): 2020.08.04 export these for use for others?
+
+// mapDecoder produces a decoder for the beam schema map encoding.
+func mapDecoder(rt reflect.Type, decodeToKey, decodeToElem func(reflect.Value, io.Reader) error) func(reflect.Value, io.Reader) error {
+	return func(ret reflect.Value, r io.Reader) error {
+		// (1) Read count prefixed encoded data
+		size, err := DecodeInt32(r)
+		if err != nil {
+			return err
+		}
+		n := int(size)
+		ret.Set(reflect.MakeMapWithSize(rt, n))
+		for i := 0; i < n; i++ {
+			rvk := reflect.New(rt.Key()).Elem()
+			if err := decodeToKey(rvk, r); err != nil {
+				return err
+			}
+			rvv := reflect.New(rt.Elem()).Elem()
+			if err := decodeToElem(rvv, r); err != nil {
+				return err
+			}
+			ret.SetMapIndex(rvk, rvv)
+		}
+		return nil
+	}
+}
+
+// containerNilDecoder handles when a value is nillable for map or iterable components.
+// Nillable types have an extra byte prefixing them indicating nil status.
+func containerNilDecoder(decodeToElem func(reflect.Value, io.Reader) error) func(reflect.Value, io.Reader) error {
+	return func(ret reflect.Value, r io.Reader) error {
+		hasValue, err := DecodeBool(r)
+		if err != nil {
+			return err
+		}
+		if !hasValue {
+			return nil
+		}
+		rv := reflect.New(ret.Type().Elem())
+		if err := decodeToElem(rv.Elem(), r); err != nil {
+			return err
+		}
+		ret.Set(rv)
+		return nil
+	}
+}
+
+// mapEncoder reflectively encodes a map or array type using the beam map encoding.
+func mapEncoder(rt reflect.Type, encodeKey, encodeValue func(reflect.Value, io.Writer) error) func(reflect.Value, io.Writer) error {
+	return func(rv reflect.Value, w io.Writer) error {
+		size := rv.Len()
+		if err := EncodeInt32((int32)(size), w); err != nil {
+			return err
+		}
+		iter := rv.MapRange()
+		for iter.Next() {
+			if err := encodeKey(iter.Key(), w); err != nil {
+				return err
+			}
+			if err := encodeValue(iter.Value(), w); err != nil {
+				return err
+			}
+		}
+		return nil
+	}
+}
+
+// containerNilEncoder handles when a value is nillable for map or iterable components.
+// Nillable types have an extra byte prefixing them indicating nil status.
+func containerNilEncoder(encodeElem func(reflect.Value, io.Writer) error) func(reflect.Value, io.Writer) error {
+	return func(rv reflect.Value, w io.Writer) error {
+		if rv.IsNil() {
+			return EncodeBool(false, w)
+		}
+		if err := EncodeBool(true, w); err != nil {
+			return err
+		}
+		return encodeElem(rv.Elem(), w)
+	}
+}
diff --git a/sdks/go/pkg/beam/core/graph/coder/map_test.go b/sdks/go/pkg/beam/core/graph/coder/map_test.go
new file mode 100644
index 0000000..0b825c2
--- /dev/null
+++ b/sdks/go/pkg/beam/core/graph/coder/map_test.go
@@ -0,0 +1,98 @@
+// 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 coder
+
+import (
+	"bytes"
+	"fmt"
+	"io"
+	"reflect"
+	"testing"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	"github.com/google/go-cmp/cmp"
+)
+
+func TestEncodeDecodeMap(t *testing.T) {
+	byteEnc := containerEncoderForType(reflectx.Uint8)
+	byteDec := containerDecoderForType(reflectx.Uint8)
+	bytePtrEnc := containerEncoderForType(reflect.PtrTo(reflectx.Uint8))
+	bytePtrDec := containerDecoderForType(reflect.PtrTo(reflectx.Uint8))
+
+	ptrByte := byte(42)
+
+	tests := []struct {
+		v          interface{}
+		encK, encV func(reflect.Value, io.Writer) error
+		decK, decV func(reflect.Value, io.Reader) error
+		encoded    []byte
+		decodeOnly bool
+	}{
+		{
+			v:       map[byte]byte{10: 42},
+			encK:    byteEnc,
+			encV:    byteEnc,
+			decK:    byteDec,
+			decV:    byteDec,
+			encoded: []byte{0, 0, 0, 1, 10, 42},
+		}, {
+			v:       map[byte]*byte{10: &ptrByte},
+			encK:    byteEnc,
+			encV:    bytePtrEnc,
+			decK:    byteDec,
+			decV:    bytePtrDec,
+			encoded: []byte{0, 0, 0, 1, 10, 1, 42},
+		}, {
+			v:          map[byte]*byte{10: &ptrByte, 23: nil, 53: nil},
+			encK:       byteEnc,
+			encV:       bytePtrEnc,
+			decK:       byteDec,
+			decV:       bytePtrDec,
+			encoded:    []byte{0, 0, 0, 3, 10, 1, 42, 23, 0, 53, 0},
+			decodeOnly: true,
+		},
+	}
+	for _, test := range tests {
+		test := test
+		if !test.decodeOnly {
+			t.Run(fmt.Sprintf("encode %q", test.v), func(t *testing.T) {
+				var buf bytes.Buffer
+				err := mapEncoder(reflect.TypeOf(test.v), test.encK, test.encV)(reflect.ValueOf(test.v), &buf)
+				if err != nil {
+					t.Fatalf("mapEncoder(%q) = %v", test.v, err)
+				}
+				if d := cmp.Diff(test.encoded, buf.Bytes()); d != "" {
+					t.Errorf("mapEncoder(%q) = %v, want %v diff(-want,+got):\n %v", test.v, buf.Bytes(), test.encoded, d)
+				}
+			})
+		}
+		t.Run(fmt.Sprintf("decode %v", test.v), func(t *testing.T) {
+			buf := bytes.NewBuffer(test.encoded)
+			rt := reflect.TypeOf(test.v)
+			var dec func(reflect.Value, io.Reader) error
+			dec = mapDecoder(rt, test.decK, test.decV)
+			rv := reflect.New(rt).Elem()
+			err := dec(rv, buf)
+			if err != nil {
+				t.Fatalf("mapDecoder(%q) = %v", test.encoded, err)
+			}
+			got := rv.Interface()
+			if d := cmp.Diff(test.v, got); d != "" {
+				t.Errorf("mapDecoder(%q) = %q, want %v diff(-want,+got):\n %v", test.encoded, got, test.v, d)
+			}
+		})
+	}
+}
diff --git a/sdks/go/pkg/beam/core/graph/coder/row.go b/sdks/go/pkg/beam/core/graph/coder/row.go
index aac34ac..00b4c26 100644
--- a/sdks/go/pkg/beam/core/graph/coder/row.go
+++ b/sdks/go/pkg/beam/core/graph/coder/row.go
@@ -152,21 +152,26 @@
 				return nil
 			}
 		}
-		decf := decoderForSingleTypeReflect(t.Elem())
-		sdec := iterableDecoderForSlice(t, decf)
-		return func(rv reflect.Value, r io.Reader) error {
-			return sdec(rv, r)
-		}
+		decf := containerDecoderForType(t.Elem())
+		return iterableDecoderForSlice(t, decf)
 	case reflect.Array:
-		decf := decoderForSingleTypeReflect(t.Elem())
-		sdec := iterableDecoderForArray(t, decf)
-		return func(rv reflect.Value, r io.Reader) error {
-			return sdec(rv, r)
-		}
+		decf := containerDecoderForType(t.Elem())
+		return iterableDecoderForArray(t, decf)
+	case reflect.Map:
+		decK := containerDecoderForType(t.Key())
+		decV := containerDecoderForType(t.Elem())
+		return mapDecoder(t, decK, decV)
 	}
 	panic(fmt.Sprintf("unimplemented type to decode: %v", t))
 }
 
+func containerDecoderForType(t reflect.Type) func(reflect.Value, io.Reader) error {
+	if t.Kind() == reflect.Ptr {
+		return containerNilDecoder(decoderForSingleTypeReflect(t.Elem()))
+	}
+	return decoderForSingleTypeReflect(t)
+}
+
 type typeDecoderReflect struct {
 	typ    reflect.Type
 	fields []func(reflect.Value, io.Reader) error
@@ -270,15 +275,26 @@
 				return EncodeBytes(rv.Bytes(), w)
 			}
 		}
-		encf := encoderForSingleTypeReflect(t.Elem())
+		encf := containerEncoderForType(t.Elem())
 		return iterableEncoder(t, encf)
 	case reflect.Array:
-		encf := encoderForSingleTypeReflect(t.Elem())
+		encf := containerEncoderForType(t.Elem())
 		return iterableEncoder(t, encf)
+	case reflect.Map:
+		encK := containerEncoderForType(t.Key())
+		encV := containerEncoderForType(t.Elem())
+		return mapEncoder(t, encK, encV)
 	}
 	panic(fmt.Sprintf("unimplemented type to encode: %v", t))
 }
 
+func containerEncoderForType(t reflect.Type) func(reflect.Value, io.Writer) error {
+	if t.Kind() == reflect.Ptr {
+		return containerNilEncoder(encoderForSingleTypeReflect(t.Elem()))
+	}
+	return encoderForSingleTypeReflect(t)
+}
+
 type typeEncoderReflect struct {
 	fields []func(reflect.Value, io.Writer) error
 }
diff --git a/sdks/go/pkg/beam/core/graph/coder/row_test.go b/sdks/go/pkg/beam/core/graph/coder/row_test.go
index f1089b8..38b7c5d 100644
--- a/sdks/go/pkg/beam/core/graph/coder/row_test.go
+++ b/sdks/go/pkg/beam/core/graph/coder/row_test.go
@@ -78,16 +78,18 @@
 			V12 [0]int
 			V13 [2]int
 			V14 []int
-			// V15 map[string]int // not yet a standard coder (BEAM-7996)
+			V15 map[string]int
 			V16 float32
 			V17 float64
 			V18 []byte
+			V19 [2]*int
+			V20 map[*string]*int
 		}{},
 	}, {
 		want: struct {
 			V00 bool
-			V01 byte
-			V02 uint8
+			V01 byte  // unsupported by spec (same as uint8)
+			V02 uint8 // unsupported by spec
 			V03 int16
 			//	V04 uint16 // unsupported by spec
 			V05 int32
@@ -100,10 +102,13 @@
 			V12 [0]int
 			V13 [2]int
 			V14 []int
-			// V15 map[string]int // not yet a standard coder (BEAM-7996) (encoding unspecified)
+			V15 map[string]int
 			V16 float32
 			V17 float64
 			V18 []byte
+			V19 [2]*int
+			V20 map[string]*int
+			V21 []*int
 		}{
 			V00: true,
 			V01: 1,
@@ -117,9 +122,16 @@
 			V12: [0]int{},
 			V13: [2]int{72, 908},
 			V14: []int{12, 9326, 641346, 6},
+			V15: map[string]int{"pants": 42},
 			V16: 3.14169,
 			V17: 2.6e100,
 			V18: []byte{21, 17, 65, 255, 0, 16},
+			V19: [2]*int{nil, &num},
+			V20: map[string]*int{
+				"notnil": &num,
+				"nil":    nil,
+			},
+			V21: []*int{nil, &num, nil},
 		},
 		// TODO add custom types such as protocol buffers.
 	},
diff --git a/sdks/go/pkg/beam/core/runtime/graphx/schema/schema.go b/sdks/go/pkg/beam/core/runtime/graphx/schema/schema.go
new file mode 100644
index 0000000..ac724b3
--- /dev/null
+++ b/sdks/go/pkg/beam/core/runtime/graphx/schema/schema.go
@@ -0,0 +1,269 @@
+// 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 schema contains utility functions for relating Go types and Beam Schemas.
+//
+// Not all Go types can be converted to schemas. This is Go is more expressive than
+// Beam schemas. Just as not all Go types can be serialized, similarly,
+// not all Beam Schemas will have a conversion to Go types, until the correct
+// mechanism exists in the SDK to handle them.
+//
+// While efforts will be made to have conversions be reversable, this will not
+// be possible in all instances. Eg. Go arrays as fields will be converted to
+// Beam Arrays, but a Beam Array type will map by default to a Go slice.
+package schema
+
+import (
+	"fmt"
+	"reflect"
+	"strings"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	"github.com/apache/beam/sdks/go/pkg/beam/internal/errors"
+	pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+)
+
+// FromType returns a Beam Schema of the passed in type.
+// Returns an error if the type cannot be converted to a Schema.
+func FromType(ot reflect.Type) (*pipepb.Schema, error) {
+	t := ot // keep the original type for errors.
+	// The top level schema for a pointer to struct and the struct is the same.
+	if t.Kind() == reflect.Ptr {
+		t = t.Elem()
+	}
+	if t.Kind() != reflect.Struct {
+		return nil, errors.Errorf("cannot convert %v to schema. FromType only converts structs to schemas", ot)
+	}
+	return structToSchema(t)
+}
+
+func structToSchema(t reflect.Type) (*pipepb.Schema, error) {
+	fields := make([]*pipepb.Field, 0, t.NumField())
+	for i := 0; i < t.NumField(); i++ {
+		f, err := structFieldToField(t.Field(i))
+		if err != nil {
+			return nil, errors.Wrapf(err, "cannot convert field %v to schema", t.Field(i).Name)
+		}
+		fields = append(fields, f)
+	}
+	return &pipepb.Schema{
+		Fields: fields,
+	}, nil
+}
+
+func structFieldToField(sf reflect.StructField) (*pipepb.Field, error) {
+	name := sf.Name
+	if tag := sf.Tag.Get("beam"); tag != "" {
+		name, _ = parseTag(tag)
+	}
+	ftype, err := reflectTypeToFieldType(sf.Type)
+	if err != nil {
+		return nil, err
+	}
+	return &pipepb.Field{
+		Name: name,
+		Type: ftype,
+	}, nil
+}
+
+func reflectTypeToFieldType(ot reflect.Type) (*pipepb.FieldType, error) {
+	var isPtr bool
+	t := ot
+	if t.Kind() == reflect.Ptr {
+		isPtr = true
+		t = t.Elem()
+	}
+	switch t.Kind() {
+	case reflect.Map:
+		kt, err := reflectTypeToFieldType(t.Key())
+		if err != nil {
+			return nil, errors.Wrapf(err, "unable to convert key of %v to schema field", ot)
+		}
+		vt, err := reflectTypeToFieldType(t.Elem())
+		if err != nil {
+			return nil, errors.Wrapf(err, "unable to convert value of %v to schema field", ot)
+		}
+		return &pipepb.FieldType{
+			Nullable: isPtr,
+			TypeInfo: &pipepb.FieldType_MapType{
+				MapType: &pipepb.MapType{
+					KeyType:   kt,
+					ValueType: vt,
+				},
+			},
+		}, nil
+	case reflect.Struct:
+		sch, err := structToSchema(t)
+		if err != nil {
+			return nil, errors.Wrapf(err, "unable to convert %v to schema field", ot)
+		}
+		return &pipepb.FieldType{
+			Nullable: isPtr,
+			TypeInfo: &pipepb.FieldType_RowType{
+				RowType: &pipepb.RowType{
+					Schema: sch,
+				},
+			},
+		}, nil
+	case reflect.Slice, reflect.Array:
+		// Special handling for []byte
+		if t == reflectx.ByteSlice {
+			return &pipepb.FieldType{
+				Nullable: isPtr,
+				TypeInfo: &pipepb.FieldType_AtomicType{
+					AtomicType: pipepb.AtomicType_BYTES,
+				},
+			}, nil
+		}
+		vt, err := reflectTypeToFieldType(t.Elem())
+		if err != nil {
+			return nil, errors.Wrapf(err, "unable to convert element type of %v to schema field", ot)
+		}
+		return &pipepb.FieldType{
+			Nullable: isPtr,
+			TypeInfo: &pipepb.FieldType_ArrayType{
+				ArrayType: &pipepb.ArrayType{
+					ElementType: vt,
+				},
+			},
+		}, nil
+	case reflect.Interface, reflect.Chan, reflect.UnsafePointer, reflect.Complex128, reflect.Complex64:
+		return nil, errors.Errorf("unable to convert unsupported type %v to schema", ot)
+	default: // must be an atomic type
+		if enum, ok := reflectTypeToAtomicTypeMap[t.Kind()]; ok {
+			return &pipepb.FieldType{
+				Nullable: isPtr,
+				TypeInfo: &pipepb.FieldType_AtomicType{
+					AtomicType: enum,
+				},
+			}, nil
+		}
+		return nil, errors.Errorf("unable to map %v to pipepb.AtomicType", t)
+	}
+}
+
+var reflectTypeToAtomicTypeMap = map[reflect.Kind]pipepb.AtomicType{
+	reflect.Uint8:   pipepb.AtomicType_BYTE,
+	reflect.Int16:   pipepb.AtomicType_INT16,
+	reflect.Int32:   pipepb.AtomicType_INT32,
+	reflect.Int64:   pipepb.AtomicType_INT64,
+	reflect.Int:     pipepb.AtomicType_INT64,
+	reflect.Float32: pipepb.AtomicType_FLOAT,
+	reflect.Float64: pipepb.AtomicType_DOUBLE,
+	reflect.String:  pipepb.AtomicType_STRING,
+	reflect.Bool:    pipepb.AtomicType_BOOLEAN,
+}
+
+// ToType returns a Go type of the passed in Schema.
+// Types returned by ToType are always of Struct kind.
+// Returns an error if the Schema cannot be converted to a type.
+func ToType(s *pipepb.Schema) (reflect.Type, error) {
+	fields := make([]reflect.StructField, 0, len(s.GetFields()))
+	for _, sf := range s.GetFields() {
+		rf, err := fieldToStructField(sf)
+		if err != nil {
+			return nil, errors.Wrapf(err, "cannot convert schema field %v to field", sf.GetName())
+		}
+		fields = append(fields, rf)
+	}
+	return reflect.StructOf(fields), nil
+}
+
+func fieldToStructField(sf *pipepb.Field) (reflect.StructField, error) {
+	name := sf.GetName()
+	rt, err := fieldTypeToReflectType(sf.GetType())
+	if err != nil {
+		return reflect.StructField{}, err
+	}
+	return reflect.StructField{
+		Name: strings.ToUpper(name[:1]) + name[1:], // Go field name must be capitalized for export and encoding.
+		Type: rt,
+		Tag:  reflect.StructTag(fmt.Sprintf("beam:\"%s\"", name)),
+	}, nil
+}
+
+var atomicTypeToReflectType = map[pipepb.AtomicType]reflect.Type{
+	pipepb.AtomicType_BYTE:    reflectx.Uint8,
+	pipepb.AtomicType_INT16:   reflectx.Int16,
+	pipepb.AtomicType_INT32:   reflectx.Int32,
+	pipepb.AtomicType_INT64:   reflectx.Int64,
+	pipepb.AtomicType_FLOAT:   reflectx.Float32,
+	pipepb.AtomicType_DOUBLE:  reflectx.Float64,
+	pipepb.AtomicType_STRING:  reflectx.String,
+	pipepb.AtomicType_BOOLEAN: reflectx.Bool,
+	pipepb.AtomicType_BYTES:   reflectx.ByteSlice,
+}
+
+func fieldTypeToReflectType(sft *pipepb.FieldType) (reflect.Type, error) {
+	var t reflect.Type
+	switch sft.GetTypeInfo().(type) {
+	case *pipepb.FieldType_AtomicType:
+		var ok bool
+		if t, ok = atomicTypeToReflectType[sft.GetAtomicType()]; !ok {
+			return nil, errors.Errorf("unknown atomic type: %v", sft.GetAtomicType())
+		}
+	case *pipepb.FieldType_ArrayType:
+		rt, err := fieldTypeToReflectType(sft.GetArrayType().GetElementType())
+		if err != nil {
+			return nil, errors.Wrap(err, "unable to convert array element type")
+		}
+		t = reflect.SliceOf(rt)
+	case *pipepb.FieldType_MapType:
+		kt, err := fieldTypeToReflectType(sft.GetMapType().GetKeyType())
+		if err != nil {
+			return nil, errors.Wrap(err, "unable to convert map key type")
+		}
+		vt, err := fieldTypeToReflectType(sft.GetMapType().GetValueType())
+		if err != nil {
+			return nil, errors.Wrap(err, "unable to convert map value type")
+		}
+		t = reflect.MapOf(kt, vt) // Panics for invalid map keys (slices/iterables)
+	case *pipepb.FieldType_RowType:
+		rt, err := ToType(sft.GetRowType().GetSchema())
+		if err != nil {
+			return nil, errors.Wrapf(err, "unable to convert row type: %v", sft.GetRowType().GetSchema().GetId())
+		}
+		t = rt
+	// case *pipepb.FieldType_IterableType:
+	// TODO(BEAM-9615): handle IterableTypes.
+
+	// case *pipepb.FieldType_LogicalType:
+	// TODO(BEAM-9615): handle LogicalTypes types.
+
+	// Logical Types are for things that have more specialized user representation already, or
+	// things like Time or protocol buffers.
+	// They would be encoded with the schema encoding.
+
+	default:
+		return nil, errors.Errorf("unknown fieldtype: %T", sft.GetTypeInfo())
+	}
+	if sft.GetNullable() {
+		return reflect.PtrTo(t), nil
+	}
+	return t, nil
+}
+
+// parseTag splits a struct field's beam tag into its name and
+// comma-separated options.
+func parseTag(tag string) (string, options) {
+	if idx := strings.Index(tag, ","); idx != -1 {
+		return tag[:idx], options(tag[idx+1:])
+	}
+	return tag, options("")
+}
+
+type options string
+
+// TODO(BEAM-9615): implement looking up specific options from the tags.
diff --git a/sdks/go/pkg/beam/core/runtime/graphx/schema/schema_test.go b/sdks/go/pkg/beam/core/runtime/graphx/schema/schema_test.go
new file mode 100644
index 0000000..f6b5ff7
--- /dev/null
+++ b/sdks/go/pkg/beam/core/runtime/graphx/schema/schema_test.go
@@ -0,0 +1,182 @@
+// 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 schema
+
+import (
+	"fmt"
+	"reflect"
+	"testing"
+
+	pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+	"github.com/google/go-cmp/cmp"
+)
+
+func TestSchemaConversion(t *testing.T) {
+	tests := []struct {
+		st *pipepb.Schema
+		rt reflect.Type
+	}{
+		{
+			st: &pipepb.Schema{
+				Fields: []*pipepb.Field{
+					&pipepb.Field{
+						Name: "firstField",
+						Type: &pipepb.FieldType{
+							TypeInfo: &pipepb.FieldType_AtomicType{
+								AtomicType: pipepb.AtomicType_INT32,
+							},
+						},
+					},
+				},
+			},
+			rt: reflect.TypeOf(struct {
+				FirstField int32 `beam:"firstField"`
+			}{}),
+		}, {
+			st: &pipepb.Schema{
+				Fields: []*pipepb.Field{
+					&pipepb.Field{
+						Name: "stringField",
+						Type: &pipepb.FieldType{
+							TypeInfo: &pipepb.FieldType_AtomicType{
+								AtomicType: pipepb.AtomicType_STRING,
+							},
+						},
+					},
+					&pipepb.Field{
+						Name: "intPtrField",
+						Type: &pipepb.FieldType{
+							Nullable: true,
+							TypeInfo: &pipepb.FieldType_AtomicType{
+								AtomicType: pipepb.AtomicType_INT32,
+							},
+						},
+					},
+				},
+			},
+			rt: reflect.TypeOf(struct {
+				StringField string `beam:"stringField"`
+				IntPtrField *int32 `beam:"intPtrField"`
+			}{}),
+		}, {
+			st: &pipepb.Schema{
+				Fields: []*pipepb.Field{
+					&pipepb.Field{
+						Name: "cypher",
+						Type: &pipepb.FieldType{
+							TypeInfo: &pipepb.FieldType_MapType{
+								MapType: &pipepb.MapType{
+									KeyType: &pipepb.FieldType{
+										TypeInfo: &pipepb.FieldType_AtomicType{
+											AtomicType: pipepb.AtomicType_BOOLEAN,
+										},
+									},
+									ValueType: &pipepb.FieldType{
+										TypeInfo: &pipepb.FieldType_AtomicType{
+											AtomicType: pipepb.AtomicType_FLOAT,
+										},
+									},
+								},
+							},
+						},
+					},
+				},
+			},
+			rt: reflect.TypeOf(struct {
+				Cypher map[bool]float32 `beam:"cypher"`
+			}{}),
+		}, {
+			st: &pipepb.Schema{
+				Fields: []*pipepb.Field{
+					&pipepb.Field{
+						Name: "wrapper",
+						Type: &pipepb.FieldType{
+							TypeInfo: &pipepb.FieldType_RowType{
+								RowType: &pipepb.RowType{
+									Schema: &pipepb.Schema{
+										Fields: []*pipepb.Field{{
+											Name: "threshold",
+											Type: &pipepb.FieldType{
+												TypeInfo: &pipepb.FieldType_AtomicType{
+													AtomicType: pipepb.AtomicType_DOUBLE,
+												},
+											},
+										},
+										},
+									},
+								},
+							},
+						},
+					},
+				},
+			},
+			rt: reflect.TypeOf(struct {
+				Wrapper struct {
+					Threshold float64 `beam:"threshold"`
+				} `beam:"wrapper"`
+			}{}),
+		}, {
+			st: &pipepb.Schema{
+				Fields: []*pipepb.Field{
+					&pipepb.Field{
+						Name: "payloads",
+						Type: &pipepb.FieldType{
+							TypeInfo: &pipepb.FieldType_ArrayType{
+								ArrayType: &pipepb.ArrayType{
+									ElementType: &pipepb.FieldType{
+										TypeInfo: &pipepb.FieldType_AtomicType{
+											AtomicType: pipepb.AtomicType_BYTES,
+										},
+									},
+								},
+							},
+						},
+					},
+				},
+			},
+			rt: reflect.TypeOf(struct {
+				Payloads [][]byte `beam:"payloads"`
+			}{}),
+		},
+	}
+
+	for _, test := range tests {
+		test := test
+		t.Run(fmt.Sprintf("%v", test.rt), func(t *testing.T) {
+			{
+				got, err := ToType(test.st)
+				if err != nil {
+					t.Fatalf("error ToType(%v) = %v", test.st, err)
+				}
+
+				if d := cmp.Diff(reflect.New(test.rt).Elem().Interface(), reflect.New(got).Elem().Interface()); d != "" {
+					t.Errorf("diff (-want, +got): %v", d)
+				}
+			}
+			{
+				got, err := FromType(test.rt)
+				if err != nil {
+					t.Fatalf("error FromType(%v) = %v", test.rt, err)
+				}
+
+				if d := cmp.Diff(test.st, got); d != "" {
+					t.Errorf("diff (-want, +got): %v", d)
+				}
+
+			}
+		})
+	}
+}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/DateTime.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/DateTime.java
new file mode 100644
index 0000000..148c726
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/DateTime.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.schemas.logicaltypes;
+
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.values.Row;
+
+/**
+ * A datetime without a time-zone.
+ *
+ * <p>It cannot represent an instant on the time-line without additional information such as an
+ * offset or time-zone.
+ *
+ * <p>Its input type is a {@link LocalDateTime}, and base type is a {@link Row} containing Date
+ * field and Time field. Date field is the same as the base type of {@link Date}, which is a Long
+ * that represents incrementing count of days where day 0 is 1970-01-01 (ISO). Time field is the
+ * same as the base type of {@link Time}, which is a Long that represents a count of time in
+ * nanoseconds.
+ */
+public class DateTime implements Schema.LogicalType<LocalDateTime, Row> {
+  public static final String DATE_FIELD_NAME = "Date";
+  public static final String TIME_FIELD_NAME = "Time";
+  public static final Schema DATETIME_SCHEMA =
+      Schema.builder().addInt64Field(DATE_FIELD_NAME).addInt64Field(TIME_FIELD_NAME).build();
+
+  @Override
+  public String getIdentifier() {
+    return "beam:logical_type:datetime:v1";
+  }
+
+  // unused
+  @Override
+  public Schema.FieldType getArgumentType() {
+    return Schema.FieldType.STRING;
+  }
+
+  // unused
+  @Override
+  public String getArgument() {
+    return "";
+  }
+
+  @Override
+  public Schema.FieldType getBaseType() {
+    return Schema.FieldType.row(DATETIME_SCHEMA);
+  }
+
+  @Override
+  public Row toBaseType(LocalDateTime input) {
+    return input == null
+        ? null
+        : Row.withSchema(DATETIME_SCHEMA)
+            .addValues(input.toLocalDate().toEpochDay(), input.toLocalTime().toNanoOfDay())
+            .build();
+  }
+
+  @Override
+  public LocalDateTime toInputType(Row base) {
+    return base == null
+        ? null
+        : LocalDateTime.of(
+            LocalDate.ofEpochDay(base.getInt64(DATE_FIELD_NAME)),
+            LocalTime.ofNanoOfDay(base.getInt64(TIME_FIELD_NAME)));
+  }
+}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/SqlTypes.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/SqlTypes.java
index d22b77a..ef6a68a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/SqlTypes.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/SqlTypes.java
@@ -18,8 +18,10 @@
 package org.apache.beam.sdk.schemas.logicaltypes;
 
 import java.time.LocalDate;
+import java.time.LocalDateTime;
 import java.time.LocalTime;
 import org.apache.beam.sdk.schemas.Schema.LogicalType;
+import org.apache.beam.sdk.values.Row;
 
 /** Beam {@link org.apache.beam.sdk.schemas.Schema.LogicalType}s corresponding to SQL data types. */
 public class SqlTypes {
@@ -31,4 +33,7 @@
 
   /** Beam LogicalType corresponding to ZetaSQL/CalciteSQL TIME type. */
   public static final LogicalType<LocalTime, Long> TIME = new Time();
+
+  /** Beam LogicalType corresponding to ZetaSQL DATETIME type. */
+  public static final LogicalType<LocalDateTime, Row> DATETIME = new DateTime();
 }
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 4604a00..42efbbd 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
@@ -39,7 +39,7 @@
           .put("BOOL", FieldType.BOOLEAN)
           .put("BYTES", FieldType.BYTES)
           .put("DATE", FieldType.logicalType(SqlTypes.DATE))
-          .put("DATETIME", FieldType.DATETIME)
+          .put("DATETIME", FieldType.logicalType(SqlTypes.DATETIME))
           .put("DOUBLE", FieldType.DOUBLE)
           .put("FLOAT", FieldType.DOUBLE)
           .put("FLOAT64", FieldType.DOUBLE)
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 3ef4d9f..b4a9d7e 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
@@ -124,7 +124,7 @@
 
   @Override
   public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
-    return CalciteUtils.sqlTypeWithAutoCast(typeFactory, method.getReturnType());
+    return CalciteUtils.sqlTypeWithAutoCast(typeFactory, method.getGenericReturnType());
   }
 
   @Override
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 1d234b0..4452422 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
@@ -48,6 +48,8 @@
     switch (typeName) {
       case TIME:
         return 6; // support microsecond time precision
+      case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+        return 6; // support microsecond datetime precision
       default:
         return super.getMaxPrecision(typeName);
     }
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 b6d606b..d9ad401 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
@@ -27,6 +27,7 @@
 import java.lang.reflect.Type;
 import java.math.BigDecimal;
 import java.time.LocalDate;
+import java.time.LocalDateTime;
 import java.time.LocalTime;
 import java.util.AbstractList;
 import java.util.AbstractMap;
@@ -35,13 +36,14 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.TimeZone;
 import org.apache.beam.sdk.extensions.sql.impl.BeamSqlPipelineOptions;
 import org.apache.beam.sdk.extensions.sql.impl.planner.BeamJavaTypeFactory;
 import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
 import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils.CharType;
 import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils.TimeWithLocalTzType;
-import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils.TimestampWithLocalTzType;
 import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.logicaltypes.DateTime;
 import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.PTransform;
@@ -92,6 +94,7 @@
 public class BeamCalcRel extends AbstractBeamCalcRel {
 
   private static final long NANOS_PER_MILLISECOND = 1000000L;
+  private static final long MILLIS_PER_DAY = 86400000L;
 
   private static final ParameterExpression outputSchemaParam =
       Expressions.parameter(Schema.class, "outputSchema");
@@ -344,6 +347,18 @@
         valueDateTime = Expressions.unbox(valueDateTime);
       }
       valueDateTime = Expressions.call(LocalDate.class, "ofEpochDay", valueDateTime);
+    } else if (CalciteUtils.TIMESTAMP_WITH_LOCAL_TZ.typesEqual(toType)
+        || CalciteUtils.NULLABLE_TIMESTAMP_WITH_LOCAL_TZ.typesEqual(toType)) {
+      // Convert TimeStamp_With_Local_TimeZone to LocalDateTime
+      Expression dateValue =
+          Expressions.divide(valueDateTime, Expressions.constant(MILLIS_PER_DAY));
+      Expression date = Expressions.call(LocalDate.class, "ofEpochDay", dateValue);
+      Expression timeValue =
+          Expressions.multiply(
+              Expressions.modulo(valueDateTime, Expressions.constant(MILLIS_PER_DAY)),
+              Expressions.constant(NANOS_PER_MILLISECOND));
+      Expression time = Expressions.call(LocalTime.class, "ofNanoOfDay", timeValue);
+      valueDateTime = Expressions.call(LocalDateTime.class, "of", date, time);
     } else {
       throw new UnsupportedOperationException("Unknown DateTime type " + toType);
     }
@@ -385,7 +400,7 @@
             .put(SqlTypes.DATE.getIdentifier(), Long.class)
             .put(SqlTypes.TIME.getIdentifier(), Long.class)
             .put(TimeWithLocalTzType.IDENTIFIER, ReadableInstant.class)
-            .put(TimestampWithLocalTzType.IDENTIFIER, ReadableInstant.class)
+            .put(SqlTypes.DATETIME.getIdentifier(), Row.class)
             .put(CharType.IDENTIFIER, String.class)
             .build();
 
@@ -442,6 +457,16 @@
               value, Expressions.divide(value, Expressions.constant(NANOS_PER_MILLISECOND)));
         } else if (SqlTypes.DATE.getIdentifier().equals(logicalId)) {
           return value;
+        } else if (SqlTypes.DATETIME.getIdentifier().equals(logicalId)) {
+          Expression dateValue =
+              Expressions.call(value, "getInt64", Expressions.constant(DateTime.DATE_FIELD_NAME));
+          Expression timeValue =
+              Expressions.call(value, "getInt64", Expressions.constant(DateTime.TIME_FIELD_NAME));
+          Expression returnValue =
+              Expressions.add(
+                  Expressions.multiply(dateValue, Expressions.constant(MILLIS_PER_DAY)),
+                  Expressions.divide(timeValue, Expressions.constant(NANOS_PER_MILLISECOND)));
+          return nullOr(value, returnValue);
         } else if (!CharType.IDENTIFIER.equals(logicalId)) {
           throw new UnsupportedOperationException(
               "Unknown LogicalType " + type.getLogicalType().getIdentifier());
@@ -563,6 +588,8 @@
           || name.equals(DataContext.Variable.CURRENT_TIMESTAMP.camelName)
           || name.equals(DataContext.Variable.LOCAL_TIMESTAMP.camelName)) {
         return System.currentTimeMillis();
+      } else if (name.equals(Variable.TIME_ZONE.camelName)) {
+        return TimeZone.getDefault();
       }
       return null;
     }
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 acb4ee1..6ded492 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
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.utils;
 
+import java.lang.reflect.ParameterizedType;
 import java.lang.reflect.Type;
 import java.util.Date;
 import java.util.Map;
@@ -53,15 +54,6 @@
     }
   }
 
-  /** A LogicalType corresponding to TIMESTAMP_WITH_LOCAL_TIME_ZONE. */
-  public static class TimestampWithLocalTzType extends PassThroughLogicalType<Instant> {
-    public static final String IDENTIFIER = "SqlTimestampWithLocalTzType";
-
-    public TimestampWithLocalTzType() {
-      super(IDENTIFIER, FieldType.STRING, "", FieldType.DATETIME);
-    }
-  }
-
   /** A LogicalType corresponding to CHAR. */
   public static class CharType extends PassThroughLogicalType<String> {
     public static final String IDENTIFIER = "SqlCharType";
@@ -82,7 +74,7 @@
       return logicalId.equals(SqlTypes.DATE.getIdentifier())
           || logicalId.equals(SqlTypes.TIME.getIdentifier())
           || logicalId.equals(TimeWithLocalTzType.IDENTIFIER)
-          || logicalId.equals(TimestampWithLocalTzType.IDENTIFIER);
+          || logicalId.equals(SqlTypes.DATETIME.getIdentifier());
     }
     return false;
   }
@@ -121,8 +113,9 @@
       FieldType.logicalType(new TimeWithLocalTzType());
   public static final FieldType TIMESTAMP = FieldType.DATETIME;
   public static final FieldType NULLABLE_TIMESTAMP = FieldType.DATETIME.withNullable(true);
-  public static final FieldType TIMESTAMP_WITH_LOCAL_TZ =
-      FieldType.logicalType(new TimestampWithLocalTzType());
+  public static final FieldType TIMESTAMP_WITH_LOCAL_TZ = FieldType.logicalType(SqlTypes.DATETIME);
+  public static final FieldType NULLABLE_TIMESTAMP_WITH_LOCAL_TZ =
+      FieldType.logicalType(SqlTypes.DATETIME).withNullable(true);
 
   private static final BiMap<FieldType, SqlTypeName> BEAM_TO_CALCITE_TYPE_MAPPING =
       ImmutableBiMap.<FieldType, SqlTypeName>builder()
@@ -283,18 +276,26 @@
 
   /**
    * SQL-Java type mapping, with specified Beam rules: <br>
-   * 1. redirect {@link AbstractInstant} to {@link Date} so Calcite can recognize it.
+   * 1. redirect {@link AbstractInstant} to {@link Date} so Calcite can recognize it. <br>
+   * 2. For a list, the component type is needed to create a Sql array type.
    *
-   * @param rawType
-   * @return
+   * @param type
+   * @return Calcite RelDataType
    */
-  public static RelDataType sqlTypeWithAutoCast(RelDataTypeFactory typeFactory, Type rawType) {
+  public static RelDataType sqlTypeWithAutoCast(RelDataTypeFactory typeFactory, Type type) {
     // For Joda time types, return SQL type for java.util.Date.
-    if (rawType instanceof Class && AbstractInstant.class.isAssignableFrom((Class<?>) rawType)) {
+    if (type instanceof Class && AbstractInstant.class.isAssignableFrom((Class<?>) type)) {
       return typeFactory.createJavaType(Date.class);
-    } else if (rawType instanceof Class && ByteString.class.isAssignableFrom((Class<?>) rawType)) {
+    } else if (type instanceof Class && ByteString.class.isAssignableFrom((Class<?>) type)) {
       return typeFactory.createJavaType(byte[].class);
+    } else if (type instanceof ParameterizedType
+        && java.util.List.class.isAssignableFrom(
+            (Class<?>) ((ParameterizedType) type).getRawType())) {
+      ParameterizedType parameterizedType = (ParameterizedType) type;
+      Class<?> genericType = (Class<?>) parameterizedType.getActualTypeArguments()[0];
+      RelDataType collectionElementType = typeFactory.createJavaType(genericType);
+      return typeFactory.createArrayType(collectionElementType, UNLIMITED_ARRAY_SIZE);
     }
-    return typeFactory.createJavaType((Class) rawType);
+    return typeFactory.createJavaType((Class) type);
   }
 }
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 7aa5032..f107dc3 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
@@ -38,6 +38,7 @@
 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.checkerframework.checker.nullness.qual.Nullable;
 
 public class BeamSqlUnparseContext extends SqlImplementor.SimpleContext {
@@ -69,8 +70,12 @@
   public SqlNode toSql(RexProgram program, RexNode rex) {
     if (rex.getKind().equals(SqlKind.LITERAL)) {
       final RexLiteral literal = (RexLiteral) rex;
-      SqlTypeFamily family = literal.getTypeName().getFamily();
-      if (SqlTypeFamily.BINARY.equals(family)) {
+      SqlTypeName name = literal.getTypeName();
+      SqlTypeFamily family = name.getFamily();
+      if (SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE.equals(name)) {
+        TimestampString timestampString = literal.getValueAs(TimestampString.class);
+        return new SqlDateTimeLiteral(timestampString, POS);
+      } else if (SqlTypeFamily.BINARY.equals(family)) {
         ByteString byteString = literal.getValueAs(ByteString.class);
         BitString bitString = BitString.createFromHexString(byteString.toString(16));
         return new SqlByteStringLiteral(bitString, POS);
@@ -92,6 +97,21 @@
     return super.toSql(program, rex);
   }
 
+  private static class SqlDateTimeLiteral extends SqlLiteral {
+
+    private final TimestampString timestampString;
+
+    SqlDateTimeLiteral(TimestampString timestampString, SqlParserPos pos) {
+      super(timestampString, SqlTypeName.TIMESTAMP, pos);
+      this.timestampString = timestampString;
+    }
+
+    @Override
+    public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+      writer.literal("DATETIME '" + timestampString.toString() + "'");
+    }
+  }
+
   private static class SqlByteStringLiteral extends SqlLiteral {
 
     SqlByteStringLiteral(BitString bytes, SqlParserPos pos) {
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 1b1641f..0519798 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
@@ -18,6 +18,7 @@
 package org.apache.beam.sdk.extensions.sql;
 
 import java.time.LocalDate;
+import java.time.LocalDateTime;
 import java.time.LocalTime;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -368,22 +369,135 @@
   }
 
   @Test
-  public void testNullDatetimeFields() {
+  public void testDatetimeFields() {
     Instant current = new Instant(1561671380000L); // Long value corresponds to 27/06/2019
 
     Schema dateTimeFieldSchema =
         Schema.builder()
             .addField("dateTimeField", FieldType.DATETIME)
             .addNullableField("nullableDateTimeField", FieldType.DATETIME)
-            .addField("timeTypeField", FieldType.logicalType(SqlTypes.TIME))
-            .addNullableField("nullableTimeTypeField", FieldType.logicalType(SqlTypes.TIME))
+            .build();
+
+    Row dateTimeRow = Row.withSchema(dateTimeFieldSchema).addValues(current, null).build();
+
+    PCollection<Row> outputRow =
+        pipeline
+            .apply(Create.of(dateTimeRow))
+            .setRowSchema(dateTimeFieldSchema)
+            .apply(
+                SqlTransform.query(
+                    "select EXTRACT(YEAR from dateTimeField) as yyyy, "
+                        + " EXTRACT(YEAR from nullableDateTimeField) as year_with_null, "
+                        + " EXTRACT(MONTH from dateTimeField) as mm, "
+                        + " EXTRACT(MONTH from nullableDateTimeField) as month_with_null "
+                        + " from PCOLLECTION"));
+
+    Schema outputRowSchema =
+        Schema.builder()
+            .addField("yyyy", FieldType.INT64)
+            .addNullableField("year_with_null", FieldType.INT64)
+            .addField("mm", FieldType.INT64)
+            .addNullableField("month_with_null", FieldType.INT64)
+            .build();
+
+    PAssert.that(outputRow)
+        .containsInAnyOrder(
+            Row.withSchema(outputRowSchema).addValues(2019L, null, 06L, null).build());
+
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(2));
+  }
+
+  @Test
+  public void testSqlLogicalTypeDateFields() {
+    Schema dateTimeFieldSchema =
+        Schema.builder()
             .addField("dateTypeField", FieldType.logicalType(SqlTypes.DATE))
             .addNullableField("nullableDateTypeField", FieldType.logicalType(SqlTypes.DATE))
             .build();
 
+    Row dateRow =
+        Row.withSchema(dateTimeFieldSchema).addValues(LocalDate.of(2019, 6, 27), null).build();
+
+    PCollection<Row> outputRow =
+        pipeline
+            .apply(Create.of(dateRow))
+            .setRowSchema(dateTimeFieldSchema)
+            .apply(
+                SqlTransform.query(
+                    "select EXTRACT(DAY from dateTypeField) as dd, "
+                        + " EXTRACT(DAY from nullableDateTypeField) as day_with_null, "
+                        + " dateTypeField + interval '1' day as date_with_day_added, "
+                        + " nullableDateTypeField + interval '1' day as day_added_with_null "
+                        + " from PCOLLECTION"));
+
+    Schema outputRowSchema =
+        Schema.builder()
+            .addField("dd", FieldType.INT64)
+            .addNullableField("day_with_null", FieldType.INT64)
+            .addField("date_with_day_added", FieldType.logicalType(SqlTypes.DATE))
+            .addNullableField("day_added_with_null", FieldType.logicalType(SqlTypes.DATE))
+            .build();
+
+    PAssert.that(outputRow)
+        .containsInAnyOrder(
+            Row.withSchema(outputRowSchema)
+                .addValues(27L, null, LocalDate.of(2019, 6, 28), null)
+                .build());
+
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(2));
+  }
+
+  @Test
+  public void testSqlLogicalTypeTimeFields() {
+    Schema dateTimeFieldSchema =
+        Schema.builder()
+            .addField("timeTypeField", FieldType.logicalType(SqlTypes.TIME))
+            .addNullableField("nullableTimeTypeField", FieldType.logicalType(SqlTypes.TIME))
+            .build();
+
+    Row timeRow =
+        Row.withSchema(dateTimeFieldSchema).addValues(LocalTime.of(1, 0, 0), null).build();
+
+    PCollection<Row> outputRow =
+        pipeline
+            .apply(Create.of(timeRow))
+            .setRowSchema(dateTimeFieldSchema)
+            .apply(
+                SqlTransform.query(
+                    "select timeTypeField + interval '1' hour as time_with_hour_added, "
+                        + " nullableTimeTypeField + interval '1' hour as hour_added_with_null, "
+                        + " timeTypeField - INTERVAL '60' SECOND as time_with_seconds_added, "
+                        + " nullableTimeTypeField - INTERVAL '60' SECOND as seconds_added_with_null "
+                        + " from PCOLLECTION"));
+
+    Schema outputRowSchema =
+        Schema.builder()
+            .addField("time_with_hour_added", FieldType.logicalType(SqlTypes.TIME))
+            .addNullableField("hour_added_with_null", FieldType.logicalType(SqlTypes.TIME))
+            .addField("time_with_seconds_added", FieldType.logicalType(SqlTypes.TIME))
+            .addNullableField("seconds_added_with_null", FieldType.logicalType(SqlTypes.TIME))
+            .build();
+
+    PAssert.that(outputRow)
+        .containsInAnyOrder(
+            Row.withSchema(outputRowSchema)
+                .addValues(LocalTime.of(2, 0, 0), null, LocalTime.of(0, 59, 0), null)
+                .build());
+
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(2));
+  }
+
+  @Test
+  public void testSqlLogicalTypeDatetimeFields() {
+    Schema dateTimeFieldSchema =
+        Schema.builder()
+            .addField("dateTimeField", FieldType.logicalType(SqlTypes.DATETIME))
+            .addNullableField("nullableDateTimeField", FieldType.logicalType(SqlTypes.DATETIME))
+            .build();
+
     Row dateTimeRow =
         Row.withSchema(dateTimeFieldSchema)
-            .addValues(current, null, LocalTime.of(1, 0, 0), null, LocalDate.of(2019, 6, 27), null)
+            .addValues(LocalDateTime.of(2008, 12, 25, 15, 30, 0), null)
             .build();
 
     PCollection<Row> outputRow =
@@ -396,14 +510,14 @@
                         + " EXTRACT(YEAR from nullableDateTimeField) as year_with_null, "
                         + " EXTRACT(MONTH from dateTimeField) as mm, "
                         + " EXTRACT(MONTH from nullableDateTimeField) as month_with_null, "
-                        + " timeTypeField + interval '1' hour as time_with_hour_added, "
-                        + " nullableTimeTypeField + interval '1' hour as hour_added_with_null, "
-                        + " timeTypeField - INTERVAL '60' SECOND as time_with_seconds_added, "
-                        + " nullableTimeTypeField - INTERVAL '60' SECOND as seconds_added_with_null, "
-                        + " EXTRACT(DAY from dateTypeField) as dd, "
-                        + " EXTRACT(DAY from nullableDateTypeField) as day_with_null, "
-                        + " dateTypeField + interval '1' day as date_with_day_added, "
-                        + " nullableDateTypeField + interval '1' day as day_added_with_null "
+                        + " dateTimeField + interval '1' hour as time_with_hour_added, "
+                        + " nullableDateTimeField + interval '1' hour as hour_added_with_null, "
+                        + " dateTimeField - INTERVAL '60' SECOND as time_with_seconds_added, "
+                        + " nullableDateTimeField - INTERVAL '60' SECOND as seconds_added_with_null, "
+                        + " EXTRACT(DAY from dateTimeField) as dd, "
+                        + " EXTRACT(DAY from nullableDateTimeField) as day_with_null, "
+                        + " dateTimeField + interval '1' day as date_with_day_added, "
+                        + " nullableDateTimeField + interval '1' day as day_added_with_null "
                         + " from PCOLLECTION"));
 
     Schema outputRowSchema =
@@ -412,31 +526,31 @@
             .addNullableField("year_with_null", FieldType.INT64)
             .addField("mm", FieldType.INT64)
             .addNullableField("month_with_null", FieldType.INT64)
-            .addField("time_with_hour_added", FieldType.logicalType(SqlTypes.TIME))
-            .addNullableField("hour_added_with_null", FieldType.logicalType(SqlTypes.TIME))
-            .addField("time_with_seconds_added", FieldType.logicalType(SqlTypes.TIME))
-            .addNullableField("seconds_added_with_null", FieldType.logicalType(SqlTypes.TIME))
+            .addField("time_with_hour_added", FieldType.logicalType(SqlTypes.DATETIME))
+            .addNullableField("hour_added_with_null", FieldType.logicalType(SqlTypes.DATETIME))
+            .addField("time_with_seconds_added", FieldType.logicalType(SqlTypes.DATETIME))
+            .addNullableField("seconds_added_with_null", FieldType.logicalType(SqlTypes.DATETIME))
             .addField("dd", FieldType.INT64)
             .addNullableField("day_with_null", FieldType.INT64)
-            .addField("date_with_day_added", FieldType.logicalType(SqlTypes.DATE))
-            .addNullableField("day_added_with_null", FieldType.logicalType(SqlTypes.DATE))
+            .addField("date_with_day_added", FieldType.logicalType(SqlTypes.DATETIME))
+            .addNullableField("day_added_with_null", FieldType.logicalType(SqlTypes.DATETIME))
             .build();
 
     PAssert.that(outputRow)
         .containsInAnyOrder(
             Row.withSchema(outputRowSchema)
                 .addValues(
-                    2019L,
+                    2008L,
                     null,
-                    06L,
+                    12L,
                     null,
-                    LocalTime.of(2, 0, 0),
+                    LocalDateTime.of(2008, 12, 25, 16, 30, 0),
                     null,
-                    LocalTime.of(0, 59, 0),
+                    LocalDateTime.of(2008, 12, 25, 15, 29, 0),
                     null,
-                    27L,
+                    25L,
                     null,
-                    LocalDate.of(2019, 6, 28),
+                    LocalDateTime.of(2008, 12, 26, 15, 30, 0),
                     null)
                 .build());
 
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 75e8a08..c2afc5d 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
@@ -23,6 +23,7 @@
 
 import com.google.auto.service.AutoService;
 import java.sql.Timestamp;
+import java.util.Arrays;
 import java.util.Map;
 import java.util.stream.IntStream;
 import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteTable;
@@ -30,6 +31,7 @@
 import org.apache.beam.sdk.extensions.sql.meta.provider.UdfUdafProvider;
 import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestBoundedTable;
 import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
 import org.apache.beam.sdk.transforms.SerializableFunction;
@@ -101,6 +103,29 @@
     pipeline.run().waitUntilFinish();
   }
 
+  @Test
+  public void testListUdf() throws Exception {
+    Schema resultType1 = Schema.builder().addArrayField("array_field", FieldType.INT64).build();
+    Row row1 = Row.withSchema(resultType1).addValue(Arrays.asList(1L)).build();
+    String sql1 = "SELECT test_array(1)";
+    PCollection<Row> result1 =
+        boundedInput1.apply(
+            "testArrayUdf",
+            SqlTransform.query(sql1).registerUdf("test_array", TestReturnTypeList.class));
+    PAssert.that(result1).containsInAnyOrder(row1);
+
+    Schema resultType2 = Schema.builder().addInt32Field("int_field").build();
+    Row row2 = Row.withSchema(resultType2).addValue(3).build();
+    String sql2 = "select array_length(ARRAY[1, 2, 3])";
+    PCollection<Row> result2 =
+        boundedInput1.apply(
+            "testArrayUdf2",
+            SqlTransform.query(sql2).registerUdf("array_length", TestListLength.class));
+    PAssert.that(result2).containsInAnyOrder(row2);
+
+    pipeline.run().waitUntilFinish();
+  }
+
   /** Test that an indirect subclass of a {@link CombineFn} works as a UDAF. BEAM-3777 */
   @Test
   public void testUdafMultiLevelDescendent() {
@@ -347,6 +372,20 @@
     }
   }
 
+  /** A UDF to test support of array as return type. */
+  public static final class TestReturnTypeList implements BeamSqlUdf {
+    public static java.util.List<Long> eval(Long i) {
+      return Arrays.asList(i);
+    }
+  }
+
+  /** A UDF to test support of array as argument type. */
+  public static final class TestListLength implements BeamSqlUdf {
+    public static Integer eval(java.util.List<Long> i) {
+      return i.size();
+    }
+  }
+
   /**
    * UDF to test support for {@link
    * org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.TableMacro}.
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 f0854bc..d4819fc 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
@@ -19,6 +19,7 @@
 
 import java.math.BigDecimal;
 import java.time.LocalDate;
+import java.time.LocalDateTime;
 import java.time.LocalTime;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
@@ -51,6 +52,7 @@
             .add("col_string_varchar", SqlTypeName.VARCHAR)
             .add("col_time", SqlTypeName.TIME)
             .add("col_date", SqlTypeName.DATE)
+            .add("col_timestamp_with_local_time_zone", SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE)
             .add("col_timestamp", SqlTypeName.TIMESTAMP)
             .add("col_boolean", SqlTypeName.BOOLEAN)
             .build();
@@ -70,6 +72,7 @@
                 "hello",
                 LocalTime.now(),
                 LocalDate.now(),
+                LocalDateTime.now(),
                 DateTime.now().toInstant(),
                 true)
             .build();
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 cc25380..5186099 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
@@ -20,11 +20,13 @@
 import com.google.zetasql.CivilTimeEncoder;
 import com.google.zetasql.Value;
 import io.grpc.Status;
+import java.time.LocalDateTime;
 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_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.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;
@@ -161,10 +163,17 @@
     }
   }
 
-  public static TimeString convertTimeValueToTimeString(Value value) {
-    LocalTime localTime = CivilTimeEncoder.decodePacked64TimeNanosAsJavaTime(value.getTimeValue());
-    return new TimeString(localTime.getHour(), localTime.getMinute(), localTime.getSecond())
-        .withNanos(localTime.getNano());
+  public static TimestampString convertDateTimeValueToTimeStampString(Value value) {
+    LocalDateTime dateTime =
+        CivilTimeEncoder.decodePacked96DatetimeNanosAsJavaTime(value.getDatetimeValue());
+    return new TimestampString(
+            dateTime.getYear(),
+            dateTime.getMonthValue(),
+            dateTime.getDayOfMonth(),
+            dateTime.getHour(),
+            dateTime.getMinute(),
+            dateTime.getSecond())
+        .withNanos(dateTime.getNano());
   }
 
   // dates are represented as an int32 value, indicating the offset
@@ -174,6 +183,12 @@
     return DateString.fromDaysSinceEpoch(value.getDateValue());
   }
 
+  public static TimeString convertTimeValueToTimeString(Value value) {
+    LocalTime localTime = CivilTimeEncoder.decodePacked64TimeNanosAsJavaTime(value.getTimeValue());
+    return new TimeString(localTime.getHour(), localTime.getMinute(), localTime.getSecond())
+        .withNanos(localTime.getNano());
+  }
+
   public static Value parseDateToValue(String dateString) {
     DateTime dateTime = parseDate(dateString);
     return Value.createDateValue((int) (dateTime.getMillis() / MILLIS_PER_DAY));
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SupportedZetaSqlBuiltinFunctions.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SupportedZetaSqlBuiltinFunctions.java
index 6ccfb26..eb8d51f 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SupportedZetaSqlBuiltinFunctions.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SupportedZetaSqlBuiltinFunctions.java
@@ -197,23 +197,23 @@
 
           // Time functions
           FunctionSignatureId.FN_CURRENT_DATE, // current_date
-          // FunctionSignatureId.FN_CURRENT_DATETIME, // current_datetime
+          FunctionSignatureId.FN_CURRENT_DATETIME, // current_datetime
           FunctionSignatureId.FN_CURRENT_TIME, // current_time
           FunctionSignatureId.FN_CURRENT_TIMESTAMP, // current_timestamp
           FunctionSignatureId.FN_DATE_ADD_DATE, // date_add
-          // FunctionSignatureId.FN_DATETIME_ADD, // datetime_add
+          FunctionSignatureId.FN_DATETIME_ADD, // datetime_add
           FunctionSignatureId.FN_TIME_ADD, // time_add
           FunctionSignatureId.FN_TIMESTAMP_ADD, // timestamp_add
           FunctionSignatureId.FN_DATE_DIFF_DATE, // date_diff
-          // FunctionSignatureId.FN_DATETIME_DIFF, // datetime_diff
+          FunctionSignatureId.FN_DATETIME_DIFF, // datetime_diff
           FunctionSignatureId.FN_TIME_DIFF, // time_diff
           FunctionSignatureId.FN_TIMESTAMP_DIFF, // timestamp_diff
           FunctionSignatureId.FN_DATE_SUB_DATE, // date_sub
-          // FunctionSignatureId.FN_DATETIME_SUB, // datetime_sub
+          FunctionSignatureId.FN_DATETIME_SUB, // datetime_sub
           FunctionSignatureId.FN_TIME_SUB, // time_sub
           FunctionSignatureId.FN_TIMESTAMP_SUB, // timestamp_sub
           FunctionSignatureId.FN_DATE_TRUNC_DATE, // date_trunc
-          // FunctionSignatureId.FN_DATETIME_TRUNC, // datetime_trunc
+          FunctionSignatureId.FN_DATETIME_TRUNC, // datetime_trunc
           FunctionSignatureId.FN_TIME_TRUNC, // time_trunc
           FunctionSignatureId.FN_TIMESTAMP_TRUNC, // timestamp_trunc
           FunctionSignatureId.FN_DATE_FROM_UNIX_DATE, // date_from_unix_date
@@ -234,19 +234,18 @@
           FunctionSignatureId.FN_UNIX_MILLIS_FROM_TIMESTAMP,
           // FunctionSignatureId.FN_UNIX_MICROS_FROM_TIMESTAMP,
           FunctionSignatureId.FN_DATE_FROM_TIMESTAMP, // date
-          // FunctionSignatureId.FN_DATE_FROM_DATETIME, // date
+          FunctionSignatureId.FN_DATE_FROM_DATETIME, // date
           FunctionSignatureId.FN_DATE_FROM_YEAR_MONTH_DAY, // date
           FunctionSignatureId.FN_TIMESTAMP_FROM_STRING, // timestamp
           FunctionSignatureId.FN_TIMESTAMP_FROM_DATE, // timestamp
-          // FunctionSignatureId.FN_TIMESTAMP_FROM_DATETIME, // timestamp
+          FunctionSignatureId.FN_TIMESTAMP_FROM_DATETIME, // timestamp
           FunctionSignatureId.FN_TIME_FROM_HOUR_MINUTE_SECOND, // time
           FunctionSignatureId.FN_TIME_FROM_TIMESTAMP, // time
-          // FunctionSignatureId.FN_TIME_FROM_DATETIME, // time
-          // FunctionSignatureId.FN_DATETIME_FROM_DATE_AND_TIME, // datetime
-          // FunctionSignatureId.FN_DATETIME_FROM_YEAR_MONTH_DAY_HOUR_MINUTE_SECOND, // datetime
-          // FunctionSignatureId.FN_DATETIME_FROM_TIMESTAMP, // datetime
-          // FunctionSignatureId.FN_DATETIME_FROM_DATE, // datetime
-
+          FunctionSignatureId.FN_TIME_FROM_DATETIME, // time
+          FunctionSignatureId.FN_DATETIME_FROM_DATE_AND_TIME, // datetime
+          FunctionSignatureId.FN_DATETIME_FROM_YEAR_MONTH_DAY_HOUR_MINUTE_SECOND, // datetime
+          FunctionSignatureId.FN_DATETIME_FROM_TIMESTAMP, // datetime
+          FunctionSignatureId.FN_DATETIME_FROM_DATE, // datetime
           FunctionSignatureId.FN_STRING_FROM_TIMESTAMP, // string
 
           // Signatures for extracting date parts, taking a date/timestamp
@@ -258,23 +257,24 @@
 
           // Signatures specific to extracting the DATE date part from a DATETIME or a
           // TIMESTAMP.
-          // FunctionSignatureId.FN_EXTRACT_DATE_FROM_DATETIME, // $extract_date
+          FunctionSignatureId.FN_EXTRACT_DATE_FROM_DATETIME, // $extract_date
           FunctionSignatureId.FN_EXTRACT_DATE_FROM_TIMESTAMP, // $extract_date
 
           // Signatures specific to extracting the TIME date part from a DATETIME or a
           // TIMESTAMP.
-          // FunctionSignatureId.FN_EXTRACT_TIME_FROM_DATETIME, // $extract_time
+          FunctionSignatureId.FN_EXTRACT_TIME_FROM_DATETIME, // $extract_time
           FunctionSignatureId.FN_EXTRACT_TIME_FROM_TIMESTAMP, // $extract_time
 
           // Signature specific to extracting the DATETIME date part from a TIMESTAMP.
-          // FunctionSignatureId.FN_EXTRACT_DATETIME_FROM_TIMESTAMP, // $extract_datetime
+          FunctionSignatureId.FN_EXTRACT_DATETIME_FROM_TIMESTAMP, // $extract_datetime
 
+          // Signature for formatting and parsing
           FunctionSignatureId.FN_FORMAT_DATE, // format_date
-          // FunctionSignatureId.FN_FORMAT_DATETIME, // format_datetime
+          FunctionSignatureId.FN_FORMAT_DATETIME, // format_datetime
           FunctionSignatureId.FN_FORMAT_TIME, // format_time
           FunctionSignatureId.FN_FORMAT_TIMESTAMP, // format_timestamp
           FunctionSignatureId.FN_PARSE_DATE, // parse_date
-          // FunctionSignatureId.FN_PARSE_DATETIME, // parse_datetime
+          FunctionSignatureId.FN_PARSE_DATETIME, // parse_datetime
           FunctionSignatureId.FN_PARSE_TIME, // parse_time
           FunctionSignatureId.FN_PARSE_TIMESTAMP, // parse_timestamp
 
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlBeamTranslationUtils.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlBeamTranslationUtils.java
index 073aa41..dbab34a 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlBeamTranslationUtils.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlBeamTranslationUtils.java
@@ -28,6 +28,7 @@
 import com.google.zetasql.ZetaSQLType.TypeKind;
 import java.math.BigDecimal;
 import java.time.LocalDate;
+import java.time.LocalDateTime;
 import java.time.LocalTime;
 import java.util.ArrayList;
 import java.util.List;
@@ -36,6 +37,7 @@
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.schemas.Schema.Field;
 import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.logicaltypes.DateTime;
 import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes;
 import org.apache.beam.sdk.values.Row;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.math.LongMath;
@@ -45,7 +47,6 @@
  * Utility methods for ZetaSQL <=> Beam translation.
  *
  * <p>Unsupported ZetaSQL types: INT32, UINT32, UINT64, FLOAT, ENUM, PROTO, GEOGRAPHY
- * TODO[BEAM-10238]: support ZetaSQL types: TIME, DATETIME, NUMERIC
  */
 @Internal
 public final class ZetaSqlBeamTranslationUtils {
@@ -106,6 +107,9 @@
     } else if (SqlTypes.TIME.getIdentifier().equals(identifier)) {
       // Time type
       return TypeFactory.createSimpleType(TypeKind.TYPE_TIME);
+    } else if (SqlTypes.DATETIME.getIdentifier().equals(identifier)) {
+      // DateTime type
+      return TypeFactory.createSimpleType(TypeKind.TYPE_DATETIME);
     } else {
       throw new UnsupportedOperationException("Unknown Beam logical type: " + identifier);
     }
@@ -184,6 +188,20 @@
       } else { // input type
         return Value.createTimeValue(CivilTimeEncoder.encodePacked64TimeNanos((LocalTime) object));
       }
+    } else if (SqlTypes.DATETIME.getIdentifier().equals(identifier)) {
+      // DateTime value
+      LocalDateTime datetime;
+      if (object instanceof Row) { // base type
+        datetime =
+            LocalDateTime.of(
+                LocalDate.ofEpochDay(((Row) object).getInt64(DateTime.DATE_FIELD_NAME)),
+                LocalTime.ofNanoOfDay(((Row) object).getInt64(DateTime.TIME_FIELD_NAME)));
+      } else { // input type
+        datetime = (LocalDateTime) object;
+      }
+      // TODO[BEAM-10611]: Create ZetaSQL Value.createDatetimeValue(LocalDateTime) function
+      return Value.createDatetimeValue(
+          CivilTimeEncoder.encodePacked64DatetimeSeconds(datetime), datetime.getNano());
     } else {
       throw new UnsupportedOperationException("Unknown Beam logical type: " + identifier);
     }
@@ -208,6 +226,8 @@
         return FieldType.logicalType(SqlTypes.DATE).withNullable(true);
       case TYPE_TIME:
         return FieldType.logicalType(SqlTypes.TIME).withNullable(true);
+      case TYPE_DATETIME:
+        return FieldType.logicalType(SqlTypes.DATETIME).withNullable(true);
       case TYPE_TIMESTAMP:
         return FieldType.DATETIME.withNullable(true);
       case TYPE_ARRAY:
@@ -314,6 +334,9 @@
     } else if (SqlTypes.TIME.getIdentifier().equals(identifier)) {
       // Time value
       return CivilTimeEncoder.decodePacked64TimeNanosAsJavaTime(value.getTimeValue());
+    } else if (SqlTypes.DATETIME.getIdentifier().equals(identifier)) {
+      // DateTime value
+      return CivilTimeEncoder.decodePacked96DatetimeNanosAsJavaTime(value.getDatetimeValue());
     } else {
       throw new UnsupportedOperationException("Unknown Beam logical type: " + identifier);
     }
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 d8394ab..81bc142 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
@@ -20,6 +20,7 @@
 import static com.google.zetasql.ZetaSQLType.TypeKind.TYPE_BOOL;
 import static com.google.zetasql.ZetaSQLType.TypeKind.TYPE_BYTES;
 import static com.google.zetasql.ZetaSQLType.TypeKind.TYPE_DATE;
+import static com.google.zetasql.ZetaSQLType.TypeKind.TYPE_DATETIME;
 import static com.google.zetasql.ZetaSQLType.TypeKind.TYPE_DOUBLE;
 import static com.google.zetasql.ZetaSQLType.TypeKind.TYPE_INT64;
 import static com.google.zetasql.ZetaSQLType.TypeKind.TYPE_NUMERIC;
@@ -46,7 +47,6 @@
  * Utility methods for ZetaSQL <=> Calcite translation.
  *
  * <p>Unsupported ZetaSQL types: INT32, UINT32, UINT64, FLOAT, ENUM, PROTO, GEOGRAPHY
- * TODO[BEAM-10238]: support ZetaSQL types: TIME, DATETIME, NUMERIC
  */
 @Internal
 public final class ZetaSqlCalciteTranslationUtils {
@@ -72,6 +72,8 @@
         return TypeFactory.createSimpleType(TYPE_DATE);
       case TIME:
         return TypeFactory.createSimpleType(TYPE_TIME);
+      case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+        return TypeFactory.createSimpleType(TYPE_DATETIME);
       case TIMESTAMP:
         return TypeFactory.createSimpleType(TYPE_TIMESTAMP);
       case ARRAY:
@@ -107,6 +109,8 @@
         return SqlTypeName.DATE;
       case TYPE_TIME:
         return SqlTypeName.TIME;
+      case TYPE_DATETIME:
+        return SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE;
       case TYPE_TIMESTAMP:
         // TODO: handle timestamp with time zone.
         return SqlTypeName.TIMESTAMP;
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 19f18cd..fd5651f 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
@@ -24,6 +24,7 @@
 import static com.google.zetasql.ZetaSQLType.TypeKind.TYPE_INT64;
 import static com.google.zetasql.ZetaSQLType.TypeKind.TYPE_STRING;
 import static com.google.zetasql.ZetaSQLType.TypeKind.TYPE_TIMESTAMP;
+import static org.apache.beam.sdk.extensions.sql.zetasql.DateTimeUtils.convertDateTimeValueToTimeStampString;
 import static org.apache.beam.sdk.extensions.sql.zetasql.DateTimeUtils.convertDateValueToDateString;
 import static org.apache.beam.sdk.extensions.sql.zetasql.DateTimeUtils.convertTimeValueToTimeString;
 import static org.apache.beam.sdk.extensions.sql.zetasql.DateTimeUtils.safeMicrosToMillis;
@@ -545,7 +546,7 @@
       case TYPE_TIMESTAMP:
       case TYPE_DATE:
       case TYPE_TIME:
-        // case TYPE_DATETIME:
+      case TYPE_DATETIME:
       case TYPE_BYTES:
       case TYPE_ARRAY:
       case TYPE_STRUCT:
@@ -709,7 +710,7 @@
       case TYPE_TIMESTAMP:
       case TYPE_DATE:
       case TYPE_TIME:
-        // case TYPE_DATETIME:
+      case TYPE_DATETIME:
       case TYPE_BYTES:
         ret = convertSimpleValueToRexNode(type.getKind(), value);
         break;
@@ -792,9 +793,7 @@
               rexBuilder()
                   .makeCall(
                       SqlOperators.createZetaSqlFunction(wrapperFun, returnType.getSqlTypeName()),
-                      ImmutableList.of(
-                          rexBuilder()
-                              .makeApproxLiteral(new BigDecimal(Math.random()), returnType)));
+                      rexBuilder().makeApproxLiteral(new BigDecimal(Math.random()), returnType));
           ;
         } else {
           ret =
@@ -823,12 +822,11 @@
                     SqlOperators.createZetaSqlFunction(
                         BeamBigQuerySqlDialect.NUMERIC_LITERAL_FUNCTION,
                         ZetaSqlCalciteTranslationUtils.toCalciteTypeName(kind)),
-                    ImmutableList.of(
-                        rexBuilder()
-                            .makeExactLiteral(
-                                value.getNumericValue(),
-                                ZetaSqlCalciteTranslationUtils.toSimpleRelDataType(
-                                    kind, rexBuilder()))));
+                    rexBuilder()
+                        .makeExactLiteral(
+                            value.getNumericValue(),
+                            ZetaSqlCalciteTranslationUtils.toSimpleRelDataType(
+                                kind, rexBuilder())));
         break;
       case TYPE_TIMESTAMP:
         ret =
@@ -850,6 +848,15 @@
         // TODO: Doing micro to mills truncation, need to throw exception.
         ret = rexBuilder().makeLiteral(convertTimeValueToTimeString(value), timeType, false);
         break;
+      case TYPE_DATETIME:
+        ret =
+            rexBuilder()
+                .makeTimestampWithLocalTimeZoneLiteral(
+                    convertDateTimeValueToTimeStampString(value),
+                    typeFactory()
+                        .getTypeSystem()
+                        .getMaxPrecision(SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE));
+        break;
       case TYPE_BYTES:
         ret = rexBuilder().makeBinaryLiteral(new ByteString(value.getBytesValue().toByteArray()));
         break;
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 25f40ed..9137b94 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,17 +17,13 @@
  */
 package org.apache.beam.sdk.extensions.sql.zetasql.translation;
 
-import static com.google.zetasql.ZetaSQLType.TypeKind.TYPE_DATETIME;
 import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkNotNull;
 
-import com.google.zetasql.ZetaSQLType.TypeKind;
-import com.google.zetasql.resolvedast.ResolvedColumn;
 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.com.google.common.collect.ImmutableSet;
 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;
@@ -44,16 +40,12 @@
 /** Converts table scan. */
 class TableScanConverter extends RelConverter<ResolvedTableScan> {
 
-  private static final ImmutableSet<TypeKind> UNSUPPORTED_DATA_TYPES =
-      ImmutableSet.of(TYPE_DATETIME);
-
   TableScanConverter(ConversionContext context) {
     super(context);
   }
 
   @Override
   public RelNode convert(ResolvedTableScan zetaNode, List<RelNode> inputs) {
-    checkTableScanSchema(zetaNode.getColumnList());
 
     List<String> tablePath = getTablePath(zetaNode.getTable());
 
@@ -115,15 +107,4 @@
       }
     };
   }
-
-  private void checkTableScanSchema(List<ResolvedColumn> columnList) {
-    if (columnList != null) {
-      for (ResolvedColumn resolvedColumn : columnList) {
-        if (UNSUPPORTED_DATA_TYPES.contains(resolvedColumn.getType().getKind())) {
-          throw new UnsupportedOperationException(
-              "Does not support " + UNSUPPORTED_DATA_TYPES + " types in source tables");
-        }
-      }
-    }
-  }
 }
diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/TestInput.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/TestInput.java
index a761d9f..2edb4d0 100644
--- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/TestInput.java
+++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/TestInput.java
@@ -21,6 +21,7 @@
 
 import java.nio.charset.StandardCharsets;
 import java.time.LocalDate;
+import java.time.LocalDateTime;
 import java.time.LocalTime;
 import java.util.Arrays;
 import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestBoundedTable;
@@ -225,47 +226,59 @@
   public static final TestBoundedTable TABLE_EMPTY =
       TestBoundedTable.of(Schema.builder().addInt64Field("ColId").addStringField("Value").build());
 
-  private static final Schema TABLE_WTH_MAP_SCHEMA =
+  private static final Schema TABLE_WITH_MAP_SCHEMA =
       Schema.builder()
           .addMapField("map_field", FieldType.STRING, FieldType.STRING)
           .addRowField("row_field", structSchema)
           .build();
   public static final TestBoundedTable TABLE_WITH_MAP =
-      TestBoundedTable.of(TABLE_WTH_MAP_SCHEMA)
+      TestBoundedTable.of(TABLE_WITH_MAP_SCHEMA)
           .addRows(
               ImmutableMap.of("MAP_KEY_1", "MAP_VALUE_1"),
               Row.withSchema(structSchema).addValues(1L, "data1").build());
 
-  private static final Schema TABLE_WTH_DATE_SCHEMA =
+  private static final Schema TABLE_WITH_DATE_SCHEMA =
       Schema.builder()
           .addLogicalTypeField("date_field", SqlTypes.DATE)
           .addStringField("str_field")
           .build();
 
   public static final TestBoundedTable TABLE_WITH_DATE =
-      TestBoundedTable.of(TABLE_WTH_DATE_SCHEMA)
+      TestBoundedTable.of(TABLE_WITH_DATE_SCHEMA)
           .addRows(LocalDate.of(2008, 12, 25), "s")
           .addRows(LocalDate.of(2020, 4, 7), "s");
 
-  private static final Schema TABLE_WTH_TIME_SCHEMA =
+  private static final Schema TABLE_WITH_TIME_SCHEMA =
       Schema.builder()
           .addLogicalTypeField("time_field", SqlTypes.TIME)
           .addStringField("str_field")
           .build();
 
   public static final TestBoundedTable TABLE_WITH_TIME =
-      TestBoundedTable.of(TABLE_WTH_TIME_SCHEMA)
+      TestBoundedTable.of(TABLE_WITH_TIME_SCHEMA)
           .addRows(LocalTime.of(15, 30, 0), "s")
           .addRows(LocalTime.of(23, 35, 59), "s");
 
-  private static final Schema TABLE_WTH_NUMERIC_SCHEMA =
+  private static final Schema TABLE_WITH_NUMERIC_SCHEMA =
       Schema.builder().addDecimalField("numeric_field").addStringField("str_field").build();
+
   public static final TestBoundedTable TABLE_WITH_NUMERIC =
-      TestBoundedTable.of(TABLE_WTH_NUMERIC_SCHEMA)
+      TestBoundedTable.of(TABLE_WITH_NUMERIC_SCHEMA)
           .addRows(ZetaSqlTypesUtils.bigDecimalAsNumeric("123.4567"), "str1")
           .addRows(ZetaSqlTypesUtils.bigDecimalAsNumeric("765.4321"), "str2")
           .addRows(ZetaSqlTypesUtils.bigDecimalAsNumeric("-555.5555"), "str3");
 
+  private static final Schema TABLE_WITH_DATETIME_SCHEMA =
+      Schema.builder()
+          .addLogicalTypeField("datetime_field", SqlTypes.DATETIME)
+          .addStringField("str_field")
+          .build();
+
+  public static final TestBoundedTable TABLE_WITH_DATETIME =
+      TestBoundedTable.of(TABLE_WITH_DATETIME_SCHEMA)
+          .addRows(LocalDateTime.of(2008, 12, 25, 15, 30, 0).withNano(123456000), "s")
+          .addRows(LocalDateTime.of(2012, 10, 6, 11, 45, 0).withNano(987654000), "s");
+
   private static byte[] stringToBytes(String s) {
     return s.getBytes(StandardCharsets.UTF_8);
   }
diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlBeamTranslationUtilsTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlBeamTranslationUtilsTest.java
index 0510590..7b450fb 100644
--- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlBeamTranslationUtilsTest.java
+++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlBeamTranslationUtilsTest.java
@@ -28,6 +28,7 @@
 import com.google.zetasql.Value;
 import com.google.zetasql.ZetaSQLType.TypeKind;
 import java.time.LocalDate;
+import java.time.LocalDateTime;
 import java.time.LocalTime;
 import java.util.Arrays;
 import org.apache.beam.sdk.schemas.Schema;
@@ -54,12 +55,12 @@
           .addField("f_string", FieldType.STRING)
           .addField("f_bytes", FieldType.BYTES)
           .addLogicalTypeField("f_date", SqlTypes.DATE)
-          // .addLogicalTypeField("f_datetime", SqlTypes.DATETIME)
+          .addLogicalTypeField("f_datetime", SqlTypes.DATETIME)
           .addLogicalTypeField("f_time", SqlTypes.TIME)
           .addField("f_timestamp", FieldType.DATETIME)
           .addArrayField("f_array", FieldType.DOUBLE)
           .addRowField("f_struct", TEST_INNER_SCHEMA)
-          // .addLogicalTypeField("f_numeric", SqlTypes.NUMERIC)
+          .addField("f_numeric", FieldType.DECIMAL)
           .addNullableField("f_null", FieldType.INT64)
           .build();
 
@@ -83,10 +84,12 @@
               new StructField("f_string", TypeFactory.createSimpleType(TypeKind.TYPE_STRING)),
               new StructField("f_bytes", TypeFactory.createSimpleType(TypeKind.TYPE_BYTES)),
               new StructField("f_date", TypeFactory.createSimpleType(TypeKind.TYPE_DATE)),
+              new StructField("f_datetime", TypeFactory.createSimpleType(TypeKind.TYPE_DATETIME)),
               new StructField("f_time", TypeFactory.createSimpleType(TypeKind.TYPE_TIME)),
               new StructField("f_timestamp", TypeFactory.createSimpleType(TypeKind.TYPE_TIMESTAMP)),
               new StructField("f_array", TEST_INNER_ARRAY_TYPE),
               new StructField("f_struct", TEST_INNER_STRUCT_TYPE),
+              new StructField("f_numeric", TypeFactory.createSimpleType(TypeKind.TYPE_NUMERIC)),
               new StructField("f_null", TypeFactory.createSimpleType(TypeKind.TYPE_INT64))));
 
   private static final Row TEST_ROW =
@@ -97,10 +100,12 @@
           .addValue("Hello")
           .addValue(new byte[] {0x11, 0x22})
           .addValue(LocalDate.of(2020, 6, 4))
+          .addValue(LocalDateTime.of(2008, 12, 25, 15, 30, 0))
           .addValue(LocalTime.of(15, 30, 45))
           .addValue(Instant.ofEpochMilli(12345678L))
           .addArray(3.0, 6.5)
           .addValue(Row.withSchema(TEST_INNER_SCHEMA).addValues(0L, "world").build())
+          .addValue(ZetaSqlTypesUtils.bigDecimalAsNumeric("12346"))
           .addValue(null)
           .build();
 
@@ -114,6 +119,10 @@
               Value.createStringValue("Hello"),
               Value.createBytesValue(ByteString.copyFrom(new byte[] {0x11, 0x22})),
               Value.createDateValue((int) LocalDate.of(2020, 6, 4).toEpochDay()),
+              Value.createDatetimeValue(
+                  CivilTimeEncoder.encodePacked64DatetimeSeconds(
+                      LocalDateTime.of(2008, 12, 25, 15, 30, 0)),
+                  LocalDateTime.of(2008, 12, 25, 15, 30, 0).getNano()),
               Value.createTimeValue(
                   CivilTimeEncoder.encodePacked64TimeNanos(LocalTime.of(15, 30, 45))),
               Value.createTimestampValueFromUnixMicros(12345678000L),
@@ -123,6 +132,7 @@
               Value.createStructValue(
                   TEST_INNER_STRUCT_TYPE,
                   Arrays.asList(Value.createInt64Value(0L), Value.createStringValue("world"))),
+              Value.createNumericValue(ZetaSqlTypesUtils.bigDecimalAsNumeric("12346")),
               Value.createNullValue(TypeFactory.createSimpleType(TypeKind.TYPE_INT64))));
 
   @Test
diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlDialectSpecTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlDialectSpecTest.java
index e9c51c7..d148012 100644
--- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlDialectSpecTest.java
+++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlDialectSpecTest.java
@@ -2942,7 +2942,7 @@
   }
 
   @Test
-  @Ignore("BEAM-9515")
+  @Ignore("[BEAM-9515] ArrayScanToUncollectConverter Unnest does not support sub-queries")
   public void testUNNESTExpression() {
     String sql = "SELECT * FROM UNNEST(ARRAY(SELECT Value FROM KeyValue));";
     ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
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 6d9ba67..483a9c1 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
@@ -61,6 +61,7 @@
     testBoundedTableMap.put("table_with_date", TestInput.TABLE_WITH_DATE);
     testBoundedTableMap.put("table_with_time", TestInput.TABLE_WITH_TIME);
     testBoundedTableMap.put("table_with_numeric", TestInput.TABLE_WITH_NUMERIC);
+    testBoundedTableMap.put("table_with_datetime", TestInput.TABLE_WITH_DATETIME);
     testBoundedTableMap.put(
         "table_with_struct_ts_string", TestInput.TABLE_WITH_STRUCT_TIMESTAMP_STRING);
     testBoundedTableMap.put("streaming_sql_test_table_a", TestInput.STREAMING_SQL_TABLE_A);
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 6789d63..109ca1e 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
@@ -23,9 +23,11 @@
 import static org.apache.beam.sdk.extensions.sql.zetasql.DateTimeUtils.parseTimestampWithTimeZone;
 import static org.apache.beam.sdk.extensions.sql.zetasql.DateTimeUtils.parseTimestampWithUTCTimeZone;
 
+import com.google.zetasql.CivilTimeEncoder;
 import com.google.zetasql.Value;
 import com.google.zetasql.ZetaSQLType.TypeKind;
 import java.time.LocalDate;
+import java.time.LocalDateTime;
 import java.time.LocalTime;
 import org.apache.beam.sdk.extensions.sql.impl.BeamSqlPipelineOptions;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
@@ -40,7 +42,6 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 import org.joda.time.Duration;
 import org.junit.Before;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
@@ -148,7 +149,7 @@
             + "  EXTRACT(ISOYEAR FROM date) AS isoyear,\n"
             + "  EXTRACT(YEAR FROM date) AS year,\n"
             + "  EXTRACT(ISOWEEK FROM date) AS isoweek,\n"
-            // TODO[BEAM-9178]: Add tests for DATE_TRUNC and EXTRACT with "week with weekday" date
+            // TODO[BEAM-10606]: Add tests for DATE_TRUNC and EXTRACT with "week with weekday" date
             //  parts once they are supported
             // + "  EXTRACT(WEEK FROM date) AS week,\n"
             + "  EXTRACT(MONTH FROM date) AS month,\n"
@@ -219,6 +220,22 @@
   }
 
   @Test
+  public void testDateFromDateTime() {
+    String sql = "SELECT DATE(DATETIME '2008-12-25 15:30:00.123456')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addLogicalTypeField("f_date", SqlTypes.DATE).build())
+                .addValues(LocalDate.of(2008, 12, 25))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
   public void testDateAdd() {
     String sql =
         "SELECT "
@@ -580,6 +597,22 @@
   }
 
   @Test
+  public void testTimeFromDateTime() {
+    String sql = "SELECT TIME(DATETIME '2008-12-25 15:30:00.123456')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addLogicalTypeField("f_time", SqlTypes.TIME).build())
+                .addValues(LocalTime.of(15, 30, 0, 123456000))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
   public void testTimeAdd() {
     String sql =
         "SELECT "
@@ -753,13 +786,420 @@
   /////////////////////////////////////////////////////////////////////////////
 
   @Test
-  @Ignore("Does not support Datetime literal.")
-  public void testDatetimeLiteral() {
-    String sql = "SELECT DATETIME '2018-01-01 05:30:00.334'";
+  public void testDateTimeLiteral() {
+    String sql = "SELECT DATETIME '2008-12-25 15:30:00.123456'";
+
     ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
-    thrown.expect(RuntimeException.class);
-    thrown.expectMessage("Unsupported ResolvedLiteral type: DATETIME");
-    zetaSQLQueryPlanner.convertToBeamRel(sql);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder().addLogicalTypeField("f_datetime", SqlTypes.DATETIME).build())
+                .addValues(LocalDateTime.of(2008, 12, 25, 15, 30, 0).withNano(123456000))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testDateTimeColumn() {
+    String sql = "SELECT FORMAT_DATETIME('%D %T %E6S', datetime_field) FROM table_with_datetime";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addStringField("f_datetime_str").build())
+                .addValues("12/25/08 15:30:00 00.123456")
+                .build(),
+            Row.withSchema(Schema.builder().addStringField("f_datetime_str").build())
+                .addValues("10/06/12 11:45:00 00.987654")
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testGroupByDateTime() {
+    String sql = "SELECT datetime_field, COUNT(*) FROM table_with_datetime GROUP BY datetime_field";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    final Schema schema =
+        Schema.builder()
+            .addLogicalTypeField("datetime_field", SqlTypes.DATETIME)
+            .addInt64Field("count")
+            .build();
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(schema)
+                .addValues(LocalDateTime.of(2008, 12, 25, 15, 30, 0).withNano(123456000), 1L)
+                .build(),
+            Row.withSchema(schema)
+                .addValues(LocalDateTime.of(2012, 10, 6, 11, 45, 0).withNano(987654000), 1L)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testAggregateOnDateTime() {
+    String sql = "SELECT MAX(datetime_field) FROM table_with_datetime GROUP BY str_field";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addLogicalTypeField("datetime_field", SqlTypes.DATETIME)
+                        .build())
+                .addValues(LocalDateTime.of(2012, 10, 6, 11, 45, 0).withNano(987654000))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  // TODO[BEAM-9166]: Add a test for CURRENT_DATETIME function ("SELECT CURRENT_DATETIME()")
+
+  @Test
+  public void testExtractFromDateTime() {
+    String sql =
+        "SELECT "
+            + "EXTRACT(YEAR FROM DATETIME '2008-12-25 15:30:00') as year, "
+            + "EXTRACT(QUARTER FROM DATETIME '2008-12-25 15:30:00') as quarter, "
+            + "EXTRACT(MONTH FROM DATETIME '2008-12-25 15:30:00') as month, "
+            // TODO[BEAM-10606]: Add tests for DATETIME_TRUNC and EXTRACT with "week with weekday"
+            //  date parts once they are supported
+            // + "EXTRACT(WEEK FROM DATETIME '2008-12-25 15:30:00') as week, "
+            + "EXTRACT(DAY FROM DATETIME '2008-12-25 15:30:00') as day, "
+            + "EXTRACT(DAYOFWEEK FROM DATETIME '2008-12-25 15:30:00') as dayofweek, "
+            + "EXTRACT(DAYOFYEAR FROM DATETIME '2008-12-25 15:30:00') as dayofyear, "
+            + "EXTRACT(HOUR FROM DATETIME '2008-12-25 15:30:00.123456') as hour, "
+            + "EXTRACT(MINUTE FROM DATETIME '2008-12-25 15:30:00.123456') as minute, "
+            + "EXTRACT(SECOND FROM DATETIME '2008-12-25 15:30:00.123456') as second, "
+            + "EXTRACT(MILLISECOND FROM DATETIME '2008-12-25 15:30:00.123456') as millisecond, "
+            + "EXTRACT(MICROSECOND FROM DATETIME '2008-12-25 15:30:00.123456') as microsecond, "
+            + "EXTRACT(DATE FROM DATETIME '2008-12-25 15:30:00.123456') as date, "
+            + "EXTRACT(TIME FROM DATETIME '2008-12-25 15:30:00.123456') as time ";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    final Schema schema =
+        Schema.builder()
+            .addInt64Field("year")
+            .addInt64Field("quarter")
+            .addInt64Field("month")
+            // .addInt64Field("week")
+            .addInt64Field("day")
+            .addInt64Field("dayofweek")
+            .addInt64Field("dayofyear")
+            .addInt64Field("hour")
+            .addInt64Field("minute")
+            .addInt64Field("second")
+            .addInt64Field("millisecond")
+            .addInt64Field("microsecond")
+            .addLogicalTypeField("date", SqlTypes.DATE)
+            .addLogicalTypeField("time", SqlTypes.TIME)
+            .build();
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(schema)
+                .addValues(
+                    2008L,
+                    4L,
+                    12L,
+                    // 52L,
+                    25L,
+                    5L,
+                    360L,
+                    15L,
+                    30L,
+                    0L,
+                    123L,
+                    123456L,
+                    LocalDate.of(2008, 12, 25),
+                    LocalTime.of(15, 30, 0, 123456000))
+                .build());
+
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testDateTimeFromDateAndTime() {
+    String sql = "SELECT DATETIME(DATE '2008-12-25', TIME '15:30:00.123456')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder().addLogicalTypeField("f_datetime", SqlTypes.DATETIME).build())
+                .addValues(LocalDateTime.of(2008, 12, 25, 15, 30, 0).withNano(123456000))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testDateTimeFromDate() {
+    String sql = "SELECT DATETIME(DATE '2008-12-25')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder().addLogicalTypeField("f_datetime", SqlTypes.DATETIME).build())
+                .addValues(LocalDateTime.of(2008, 12, 25, 0, 0, 0))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testDateTimeFromYearMonthDayHourMinuteSecond() {
+    String sql = "SELECT DATETIME(2008, 12, 25, 15, 30, 0)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder().addLogicalTypeField("f_datetime", SqlTypes.DATETIME).build())
+                .addValues(LocalDateTime.of(2008, 12, 25, 15, 30, 0))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testDateTimeFromTimestamp() {
+    String sql = "SELECT DATETIME(TIMESTAMP '2008-12-25 15:30:00+08', 'America/Los_Angeles')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder().addLogicalTypeField("f_datetime", SqlTypes.DATETIME).build())
+                .addValues(LocalDateTime.of(2008, 12, 24, 23, 30, 0))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testDateTimeAdd() {
+    String sql =
+        "SELECT "
+            + "DATETIME_ADD(DATETIME '2008-12-25 15:30:00', INTERVAL 10 MICROSECOND), "
+            + "DATETIME_ADD(DATETIME '2008-12-25 15:30:00', INTERVAL 10 MILLISECOND), "
+            + "DATETIME_ADD(DATETIME '2008-12-25 15:30:00', INTERVAL 10 SECOND), "
+            + "DATETIME_ADD(DATETIME '2008-12-25 15:30:00', INTERVAL 10 MINUTE), "
+            + "DATETIME_ADD(DATETIME '2008-12-25 15:30:00', INTERVAL 10 HOUR), "
+            + "DATETIME_ADD(DATETIME '2008-12-25 15:30:00', INTERVAL 10 DAY), "
+            + "DATETIME_ADD(DATETIME '2008-12-25 15:30:00', INTERVAL 10 MONTH), "
+            + "DATETIME_ADD(DATETIME '2008-12-25 15:30:00', INTERVAL 10 QUARTER), "
+            + "DATETIME_ADD(DATETIME '2008-12-25 15:30:00', INTERVAL 10 YEAR) ";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addLogicalTypeField("f_time1", SqlTypes.DATETIME)
+                        .addLogicalTypeField("f_time2", SqlTypes.DATETIME)
+                        .addLogicalTypeField("f_time3", SqlTypes.DATETIME)
+                        .addLogicalTypeField("f_time4", SqlTypes.DATETIME)
+                        .addLogicalTypeField("f_time5", SqlTypes.DATETIME)
+                        .addLogicalTypeField("f_time6", SqlTypes.DATETIME)
+                        .addLogicalTypeField("f_time7", SqlTypes.DATETIME)
+                        .addLogicalTypeField("f_time8", SqlTypes.DATETIME)
+                        .addLogicalTypeField("f_time9", SqlTypes.DATETIME)
+                        .build())
+                .addValues(
+                    LocalDateTime.of(2008, 12, 25, 15, 30, 0).withNano(10000),
+                    LocalDateTime.of(2008, 12, 25, 15, 30, 0).withNano(10000000),
+                    LocalDateTime.of(2008, 12, 25, 15, 30, 10),
+                    LocalDateTime.of(2008, 12, 25, 15, 40, 0),
+                    LocalDateTime.of(2008, 12, 26, 1, 30, 0),
+                    LocalDateTime.of(2009, 1, 4, 15, 30, 0),
+                    LocalDateTime.of(2009, 10, 25, 15, 30, 0),
+                    LocalDateTime.of(2011, 6, 25, 15, 30, 0),
+                    LocalDateTime.of(2018, 12, 25, 15, 30, 0))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testDateTimeAddWithParameter() {
+    String sql = "SELECT DATETIME_ADD(@p0, INTERVAL @p1 HOUR)";
+
+    LocalDateTime datetime = LocalDateTime.of(2008, 12, 25, 15, 30, 00).withNano(123456000);
+    ImmutableMap<String, Value> params =
+        ImmutableMap.of(
+            "p0",
+                Value.createDatetimeValue(
+                    CivilTimeEncoder.encodePacked64DatetimeSeconds(datetime), datetime.getNano()),
+            "p1", Value.createInt64Value(3L));
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql, params);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder().addLogicalTypeField("f_datetime", SqlTypes.DATETIME).build())
+                .addValues(LocalDateTime.of(2008, 12, 25, 18, 30, 00).withNano(123456000))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testDateTimeSub() {
+    String sql =
+        "SELECT "
+            + "DATETIME_SUB(DATETIME '2008-12-25 15:30:00', INTERVAL 10 MICROSECOND), "
+            + "DATETIME_SUB(DATETIME '2008-12-25 15:30:00', INTERVAL 10 MILLISECOND), "
+            + "DATETIME_SUB(DATETIME '2008-12-25 15:30:00', INTERVAL 10 SECOND), "
+            + "DATETIME_SUB(DATETIME '2008-12-25 15:30:00', INTERVAL 10 MINUTE), "
+            + "DATETIME_SUB(DATETIME '2008-12-25 15:30:00', INTERVAL 10 HOUR), "
+            + "DATETIME_SUB(DATETIME '2008-12-25 15:30:00', INTERVAL 10 DAY), "
+            + "DATETIME_SUB(DATETIME '2008-12-25 15:30:00', INTERVAL 10 MONTH), "
+            + "DATETIME_SUB(DATETIME '2008-12-25 15:30:00', INTERVAL 10 QUARTER), "
+            + "DATETIME_SUB(DATETIME '2008-12-25 15:30:00', INTERVAL 10 YEAR) ";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addLogicalTypeField("f_time1", SqlTypes.DATETIME)
+                        .addLogicalTypeField("f_time2", SqlTypes.DATETIME)
+                        .addLogicalTypeField("f_time3", SqlTypes.DATETIME)
+                        .addLogicalTypeField("f_time4", SqlTypes.DATETIME)
+                        .addLogicalTypeField("f_time5", SqlTypes.DATETIME)
+                        .addLogicalTypeField("f_time6", SqlTypes.DATETIME)
+                        .addLogicalTypeField("f_time7", SqlTypes.DATETIME)
+                        .addLogicalTypeField("f_time8", SqlTypes.DATETIME)
+                        .addLogicalTypeField("f_time9", SqlTypes.DATETIME)
+                        .build())
+                .addValues(
+                    LocalDateTime.of(2008, 12, 25, 15, 29, 59).withNano(999990000),
+                    LocalDateTime.of(2008, 12, 25, 15, 29, 59).withNano(990000000),
+                    LocalDateTime.of(2008, 12, 25, 15, 29, 50),
+                    LocalDateTime.of(2008, 12, 25, 15, 20, 0),
+                    LocalDateTime.of(2008, 12, 25, 5, 30, 0),
+                    LocalDateTime.of(2008, 12, 15, 15, 30, 0),
+                    LocalDateTime.of(2008, 2, 25, 15, 30, 0),
+                    LocalDateTime.of(2006, 6, 25, 15, 30, 0),
+                    LocalDateTime.of(1998, 12, 25, 15, 30, 0))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testDateTimeDiff() {
+    String sql =
+        "SELECT DATETIME_DIFF(DATETIME '2008-12-25 15:30:00', DATETIME '2008-10-25 15:30:00', DAY)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addInt64Field("f_datetime_diff").build())
+                .addValues(61L)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testDateTimeDiffNegativeResult() {
+    String sql =
+        "SELECT DATETIME_DIFF(DATETIME '2008-10-25 15:30:00', DATETIME '2008-12-25 15:30:00', DAY)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addInt64Field("f_datetime_diff").build())
+                .addValues(-61L)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testDateTimeTrunc() {
+    String sql = "SELECT DATETIME_TRUNC(DATETIME '2008-12-25 15:30:00', HOUR)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addLogicalTypeField("f_datetime_trunc", SqlTypes.DATETIME)
+                        .build())
+                .addValues(LocalDateTime.of(2008, 12, 25, 15, 0, 0))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testFormatDateTime() {
+    String sql = "SELECT FORMAT_DATETIME('%D %T %E6S', DATETIME '2008-12-25 15:30:00.123456')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addStringField("f_datetime_str").build())
+                .addValues("12/25/08 15:30:00 00.123456")
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testParseDateTime() {
+    String sql = "SELECT PARSE_DATETIME('%Y-%m-%d %H:%M:%E6S', '2008-12-25 15:30:00.123456')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder().addLogicalTypeField("f_datetime", SqlTypes.DATETIME).build())
+                .addValues(LocalDateTime.of(2008, 12, 25, 15, 30, 0).withNano(123456000))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
   }
 
   /////////////////////////////////////////////////////////////////////////////
@@ -846,7 +1286,7 @@
             + "  EXTRACT(ISOYEAR FROM timestamp) AS isoyear,\n"
             + "  EXTRACT(YEAR FROM timestamp) AS year,\n"
             + "  EXTRACT(ISOWEEK FROM timestamp) AS isoweek,\n"
-            // TODO[BEAM-9178]: Add tests for TIMESTAMP_TRUNC and EXTRACT with "week with weekday"
+            // TODO[BEAM-10606]: Add tests for TIMESTAMP_TRUNC and EXTRACT with "week with weekday"
             //  date parts once they are supported
             // + "  EXTRACT(WEEK FROM timestamp) AS week,\n"
             + "  EXTRACT(MONTH FROM timestamp) AS month,\n"
@@ -926,6 +1366,23 @@
   }
 
   @Test
+  public void testExtractDateTimeFromTimestamp() {
+    String sql = "SELECT EXTRACT(DATETIME FROM TIMESTAMP '2017-05-26 12:34:56')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder().addLogicalTypeField("datetime", SqlTypes.DATETIME).build())
+                .addValues(LocalDateTime.of(2017, 5, 26, 12, 34, 56))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
   public void testExtractFromTimestampAtTimeZone() {
     String sql =
         "WITH Timestamps AS (\n"
@@ -1028,6 +1485,45 @@
   }
 
   @Test
+  public void testTimestampFromDateTime() {
+    String sql = "SELECT TIMESTAMP(DATETIME '2008-12-25 15:30:00')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addDateTimeField("f_timestamp").build())
+                .addValues(parseTimestampWithTimeZone("2008-12-25 15:30:00+00"))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  // test default timezone works properly in query execution stage
+  public void testTimestampFromDateTimeWithDefaultTimezoneSet() {
+    String sql = "SELECT TIMESTAMP(DATETIME '2008-12-25 15:30:00')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    zetaSQLQueryPlanner.setDefaultTimezone("Asia/Shanghai");
+    pipeline
+        .getOptions()
+        .as(BeamSqlPipelineOptions.class)
+        .setZetaSqlDefaultTimezone("Asia/Shanghai");
+
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addDateTimeField("f_timestamp").build())
+                .addValues(parseTimestampWithTimeZone("2008-12-25 15:30:00+08"))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
   public void testTimestampAdd() {
     String sql =
         "SELECT "
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClient.java
index 9521443..3f449d2 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClient.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClient.java
@@ -129,7 +129,7 @@
           new PubsubMessage().encodeData(outgoingMessage.message().getData().toByteArray());
       pubsubMessage.setAttributes(getMessageAttributes(outgoingMessage));
       if (!outgoingMessage.message().getOrderingKey().isEmpty()) {
-        pubsubMessage.put("orderingKey", outgoingMessage.message().getOrderingKey());
+        pubsubMessage.setOrderingKey(outgoingMessage.message().getOrderingKey());
       }
       pubsubMessages.add(pubsubMessage);
     }
@@ -156,6 +156,7 @@
   }
 
   @Override
+  @SuppressWarnings("ProtoFieldNullComparison")
   public List<IncomingMessage> pull(
       long requestTimeMsSinceEpoch,
       SubscriptionPath subscription,
@@ -207,8 +208,12 @@
           com.google.pubsub.v1.PubsubMessage.newBuilder();
       protoMessage.setData(ByteString.copyFrom(elementBytes));
       protoMessage.putAllAttributes(attributes);
-      protoMessage.setOrderingKey(
-          (String) pubsubMessage.getUnknownKeys().getOrDefault("orderingKey", ""));
+      // PubsubMessage uses `null` to represent no ordering key where we want a default of "".
+      if (pubsubMessage.getOrderingKey() != null) {
+        protoMessage.setOrderingKey(pubsubMessage.getOrderingKey());
+      } else {
+        protoMessage.setOrderingKey("");
+      }
       incomingMessages.add(
           IncomingMessage.of(
               protoMessage.build(),
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClientTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClientTest.java
index aad9729..22c1cb1 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClientTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClientTest.java
@@ -101,7 +101,7 @@
             .setAttributes(
                 ImmutableMap.of(
                     TIMESTAMP_ATTRIBUTE, String.valueOf(MESSAGE_TIME), ID_ATTRIBUTE, RECORD_ID))
-            .set("orderingKey", ORDERING_KEY);
+            .setOrderingKey(ORDERING_KEY);
     ReceivedMessage expectedReceivedMessage =
         new ReceivedMessage().setMessage(expectedPubsubMessage).setAckId(ACK_ID);
     PullResponse expectedResponse =
diff --git a/sdks/java/io/snowflake/build.gradle b/sdks/java/io/snowflake/build.gradle
index 32ad7af..ea8204d 100644
--- a/sdks/java/io/snowflake/build.gradle
+++ b/sdks/java/io/snowflake/build.gradle
@@ -22,6 +22,7 @@
   automaticModuleName: 'org.apache.beam.sdk.io.snowflake')
 provideIntegrationTestingDependencies()
 enableJavaPerformanceTesting()
+
 description = "Apache Beam :: SDKs :: Java :: IO :: Snowflake"
 ext.summary = "IO to read and write on Snowflake."
 dependencies {
@@ -31,6 +32,7 @@
   compile library.java.slf4j_api
   compile group: 'net.snowflake', name: 'snowflake-jdbc', version: '3.12.7'
   compile group: 'com.opencsv', name: 'opencsv', version: '5.0'
+  compile 'net.snowflake:snowflake-ingest-sdk:0.9.9'
   testCompile project(path: ":sdks:java:core", configuration: "shadowTest")
   testCompile project(path: ":sdks:java:io:common", configuration: "testRuntime")
   testCompile project(path: ":sdks:java:testing:test-utils", configuration: "testRuntime")
diff --git a/sdks/java/io/snowflake/expansion-service/build.gradle b/sdks/java/io/snowflake/expansion-service/build.gradle
new file mode 100644
index 0000000..8a6ea6c
--- /dev/null
+++ b/sdks/java/io/snowflake/expansion-service/build.gradle
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+apply plugin: 'org.apache.beam.module'
+apply plugin: 'application'
+mainClassName = "org.apache.beam.sdk.expansion.service.ExpansionService"
+
+applyJavaNature(enableChecker:false,
+  automaticModuleName: 'org.apache.beam.sdk.io.expansion.service',
+  exportJavadoc: false,
+  validateShadowJar: false,
+  shadowClosure: {},
+)
+
+description = "Apache Beam :: SDKs :: Java :: IO :: Snowflake ::Expansion Service"
+  ext.summary = "Expansion service serving Snowflake IO"
+
+dependencies {
+  compile project(":sdks:java:expansion-service")
+  compile project(":sdks:java:io:snowflake")
+  runtime library.java.slf4j_jdk14
+}
+
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeIO.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeIO.java
index 845f137..a2c0a7e 100644
--- a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeIO.java
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeIO.java
@@ -37,6 +37,8 @@
 import java.util.stream.Collectors;
 import javax.sql.DataSource;
 import net.snowflake.client.jdbc.SnowflakeBasicDataSource;
+import net.snowflake.ingest.SimpleIngestManager;
+import net.snowflake.ingest.connection.HistoryResponse;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.ListCoder;
@@ -51,13 +53,18 @@
 import org.apache.beam.sdk.io.snowflake.credentials.KeyPairSnowflakeCredentials;
 import org.apache.beam.sdk.io.snowflake.credentials.OAuthTokenSnowflakeCredentials;
 import org.apache.beam.sdk.io.snowflake.credentials.SnowflakeCredentials;
+import org.apache.beam.sdk.io.snowflake.credentials.SnowflakeCredentialsFactory;
 import org.apache.beam.sdk.io.snowflake.credentials.UsernamePasswordSnowflakeCredentials;
 import org.apache.beam.sdk.io.snowflake.data.SnowflakeTableSchema;
 import org.apache.beam.sdk.io.snowflake.enums.CreateDisposition;
+import org.apache.beam.sdk.io.snowflake.enums.StreamingLogLevel;
 import org.apache.beam.sdk.io.snowflake.enums.WriteDisposition;
+import org.apache.beam.sdk.io.snowflake.services.SnowflakeBatchServiceConfig;
+import org.apache.beam.sdk.io.snowflake.services.SnowflakeBatchServiceImpl;
 import org.apache.beam.sdk.io.snowflake.services.SnowflakeService;
-import org.apache.beam.sdk.io.snowflake.services.SnowflakeServiceConfig;
-import org.apache.beam.sdk.io.snowflake.services.SnowflakeServiceImpl;
+import org.apache.beam.sdk.io.snowflake.services.SnowflakeStreamingServiceConfig;
+import org.apache.beam.sdk.io.snowflake.services.SnowflakeStreamingServiceImpl;
+import org.apache.beam.sdk.options.ValueProvider;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.DoFn;
@@ -73,12 +80,20 @@
 import org.apache.beam.sdk.transforms.Wait;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.transforms.display.HasDisplayData;
+import org.apache.beam.sdk.transforms.windowing.AfterFirst;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.PDone;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Splitter;
 import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Duration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -92,9 +107,9 @@
  *
  * <p>To configure SnowflakeIO to read/write from your Snowflake instance, you have to provide a
  * {@link DataSourceConfiguration} using {@link
- * DataSourceConfiguration#create(SnowflakeCredentials)}, where {@link SnowflakeCredentials might be
- * created using {@link org.apache.beam.sdk.io.snowflake.credentials.SnowflakeCredentialsFactory}}.
- * Additionally one of {@link DataSourceConfiguration#withServerName(String)} or {@link
+ * DataSourceConfiguration#create(SnowflakeCredentials)}, where {@link SnowflakeCredentials} might
+ * be created using {@link SnowflakeCredentialsFactory }. Additionally one of {@link
+ * DataSourceConfiguration#withServerName(String)} or {@link
  * DataSourceConfiguration#withUrl(String)} must be used to tell SnowflakeIO which instance to use.
  * <br>
  * There are also other options available to configure connection to Snowflake:
@@ -147,8 +162,8 @@
  *
  * <h3>Writing to Snowflake</h3>
  *
- * <p>SnowflakeIO.Write supports writing records into a database. It writes a {@link PCollection<T>}
- * to the database by converting each T into a {@link Object[]} via a user-provided {@link
+ * <p>SnowflakeIO.Write supports writing records into a database. It writes a {@link PCollection} to
+ * the database by converting each T into a {@link Object[]} via a user-provided {@link
  * UserDataMapper}.
  *
  * <p>For example
@@ -171,7 +186,13 @@
   private static final Logger LOG = LoggerFactory.getLogger(SnowflakeIO.class);
 
   private static final String CSV_QUOTE_CHAR = "'";
-  private static final String WRITE_TMP_PATH = "data";
+
+  static final int DEFAULT_FLUSH_ROW_LIMIT = 10000;
+  static final int DEFAULT_STREAMING_SHARDS_NUMBER = 1;
+  static final int DEFAULT_BATCH_SHARDS_NUMBER = 0;
+  static final Duration DEFAULT_FLUSH_TIME_LIMIT = Duration.millis(30000); // 30 seconds
+  static final Duration DEFAULT_STREAMING_LOGS_MAX_SLEEP = Duration.standardMinutes(2);
+  static final Duration DEFAULT_SLEEP_STREAMING_LOGS = Duration.standardSeconds(5000);
 
   /**
    * Read data from Snowflake.
@@ -182,6 +203,7 @@
   public static <T> Read<T> read(SnowflakeService snowflakeService) {
     return new AutoValue_SnowflakeIO_Read.Builder<T>()
         .setSnowflakeService(snowflakeService)
+        .setQuotationMark(CSV_QUOTE_CHAR)
         .build();
   }
 
@@ -191,7 +213,7 @@
    * @param <T> Type of the data to be read.
    */
   public static <T> Read<T> read() {
-    return read(new SnowflakeServiceImpl());
+    return read(new SnowflakeBatchServiceImpl());
   }
 
   /**
@@ -226,6 +248,10 @@
         .setFileNameTemplate("output")
         .setCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
         .setWriteDisposition(WriteDisposition.APPEND)
+        .setFlushTimeLimit(DEFAULT_FLUSH_TIME_LIMIT)
+        .setShardsNumber(DEFAULT_BATCH_SHARDS_NUMBER)
+        .setFlushRowLimit(DEFAULT_FLUSH_ROW_LIMIT)
+        .setQuotationMark(CSV_QUOTE_CHAR)
         .build();
   }
 
@@ -249,6 +275,9 @@
 
     abstract @Nullable SnowflakeService getSnowflakeService();
 
+    @Nullable
+    abstract String getQuotationMark();
+
     abstract Builder<T> toBuilder();
 
     @AutoValue.Builder
@@ -270,13 +299,15 @@
 
       abstract Builder<T> setSnowflakeService(SnowflakeService snowflakeService);
 
+      abstract Builder<T> setQuotationMark(String quotationMark);
+
       abstract Read<T> build();
     }
 
     /**
      * Setting information about Snowflake server.
      *
-     * @param config - An instance of {@link DataSourceConfiguration}.
+     * @param config An instance of {@link DataSourceConfiguration}.
      */
     public Read<T> withDataSourceConfiguration(final DataSourceConfiguration config) {
       return withDataSourceProviderFn(new DataSourceProviderFromDataSourceConfiguration(config));
@@ -295,7 +326,7 @@
     /**
      * A query to be executed in Snowflake.
      *
-     * @param query - String with query.
+     * @param query String with query.
      */
     public Read<T> fromQuery(String query) {
       return toBuilder().setQuery(query).build();
@@ -304,7 +335,7 @@
     /**
      * A table name to be read in Snowflake.
      *
-     * @param table - String with the name of the table.
+     * @param table String with the name of the table.
      */
     public Read<T> fromTable(String table) {
       return toBuilder().setTable(table).build();
@@ -313,9 +344,12 @@
     /**
      * Name of the cloud bucket (GCS by now) to use as tmp location of CSVs during COPY statement.
      *
-     * @param stagingBucketName - String with the name of the bucket.
+     * @param stagingBucketName String with the name of the bucket.
      */
     public Read<T> withStagingBucketName(String stagingBucketName) {
+      checkArgument(
+          stagingBucketName.endsWith("/"),
+          "stagingBucketName must be a cloud storage path ending with /");
       return toBuilder().setStagingBucketName(stagingBucketName).build();
     }
 
@@ -324,7 +358,7 @@
      * https://docs.snowflake.com/en/sql-reference/sql/create-storage-integration.html for
      * reference.
      *
-     * @param integrationName - String with the name of the Storage Integration.
+     * @param integrationName String with the name of the Storage Integration.
      */
     public Read<T> withStorageIntegrationName(String integrationName) {
       return toBuilder().setStorageIntegrationName(integrationName).build();
@@ -333,7 +367,7 @@
     /**
      * User-defined function mapping CSV lines into user data.
      *
-     * @param csvMapper - an instance of {@link CsvMapper}.
+     * @param csvMapper an instance of {@link CsvMapper}.
      */
     public Read<T> withCsvMapper(CsvMapper<T> csvMapper) {
       return toBuilder().setCsvMapper(csvMapper).build();
@@ -342,21 +376,29 @@
     /**
      * A Coder to be used by the output PCollection generated by the source.
      *
-     * @param coder - an instance of {@link Coder}.
+     * @param coder an instance of {@link Coder}.
      */
     public Read<T> withCoder(Coder<T> coder) {
       return toBuilder().setCoder(coder).build();
     }
 
+    /**
+     * Sets Snowflake-specific quotations around strings.
+     *
+     * @param quotationMark with possible single quote {@code '}, double quote {@code "} or nothing.
+     *     Default value is single quotation {@code '}.
+     * @return
+     */
+    public Read<T> withQuotationMark(String quotationMark) {
+      return toBuilder().setQuotationMark(quotationMark).build();
+    }
+
     @Override
     public PCollection<T> expand(PBegin input) {
       checkArguments();
 
-      String tmpDirName = makeTmpDirName();
-      String stagingBucketDir = String.format("%s/%s/", getStagingBucketName(), tmpDirName);
-
       PCollection<Void> emptyCollection = input.apply(Create.of((Void) null));
-
+      String tmpDirName = makeTmpDirName();
       PCollection<T> output =
           emptyCollection
               .apply(
@@ -366,20 +408,22 @@
                           getQuery(),
                           getTable(),
                           getStorageIntegrationName(),
-                          stagingBucketDir,
-                          getSnowflakeService())))
+                          getStagingBucketName(),
+                          tmpDirName,
+                          getSnowflakeService(),
+                          getQuotationMark())))
               .apply(Reshuffle.viaRandomKey())
               .apply(FileIO.matchAll())
               .apply(FileIO.readMatches())
               .apply(readFiles())
-              .apply(ParDo.of(new MapCsvToStringArrayFn()))
+              .apply(ParDo.of(new MapCsvToStringArrayFn(getQuotationMark())))
               .apply(ParDo.of(new MapStringArrayToUserDataFn<>(getCsvMapper())));
 
       output.setCoder(getCoder());
 
       emptyCollection
           .apply(Wait.on(output))
-          .apply(ParDo.of(new CleanTmpFilesFromGcsFn(stagingBucketDir)));
+          .apply(ParDo.of(new CleanTmpFilesFromGcsFn(getStagingBucketName(), tmpDirName)));
       return output;
     }
 
@@ -387,8 +431,9 @@
       // Either table or query is required. If query is present, it's being used, table is used
       // otherwise
 
-      checkArgument(getStorageIntegrationName() != null, "withStorageIntegrationName is required");
-      checkArgument(getStagingBucketName() != null, "withStagingBucketName is required");
+      checkArgument(
+          getStorageIntegrationName() != null, "withStorageIntegrationName() is required");
+      checkArgument(getStagingBucketName() != null, "withStagingBucketName() is required");
 
       checkArgument(
           getQuery() != null || getTable() != null, "fromTable() or fromQuery() is required");
@@ -414,9 +459,13 @@
       private final SerializableFunction<Void, DataSource> dataSourceProviderFn;
       private final String query;
       private final String table;
+      private final String database;
+      private final String schema;
+      private final String tmpDirName;
       private final String storageIntegrationName;
       private final String stagingBucketDir;
       private final SnowflakeService snowflakeService;
+      private final String quotationMark;
 
       private CopyIntoStageFn(
           SerializableFunction<Void, DataSource> dataSourceProviderFn,
@@ -424,23 +473,44 @@
           String table,
           String storageIntegrationName,
           String stagingBucketDir,
-          SnowflakeService snowflakeService) {
+          String tmpDirName,
+          SnowflakeService snowflakeService,
+          String quotationMark) {
         this.dataSourceProviderFn = dataSourceProviderFn;
         this.query = query;
         this.table = table;
         this.storageIntegrationName = storageIntegrationName;
-        this.stagingBucketDir =
-            String.format(
-                "%s/run_%s/", stagingBucketDir, UUID.randomUUID().toString().subSequence(0, 8));
         this.snowflakeService = snowflakeService;
+        this.quotationMark = quotationMark;
+        this.stagingBucketDir = stagingBucketDir;
+        this.tmpDirName = tmpDirName;
+        DataSourceProviderFromDataSourceConfiguration
+            dataSourceProviderFromDataSourceConfiguration =
+                (DataSourceProviderFromDataSourceConfiguration) this.dataSourceProviderFn;
+        DataSourceConfiguration config = dataSourceProviderFromDataSourceConfiguration.getConfig();
+
+        this.database = config.getDatabase();
+        this.schema = config.getSchema();
       }
 
       @ProcessElement
       public void processElement(ProcessContext context) throws Exception {
 
-        SnowflakeServiceConfig config =
-            new SnowflakeServiceConfig(
-                dataSourceProviderFn, table, query, storageIntegrationName, stagingBucketDir);
+        String stagingBucketRunDir =
+            String.format(
+                "%s/%s/run_%s/",
+                stagingBucketDir, tmpDirName, UUID.randomUUID().toString().subSequence(0, 8));
+
+        SnowflakeBatchServiceConfig config =
+            new SnowflakeBatchServiceConfig(
+                dataSourceProviderFn,
+                database,
+                schema,
+                table,
+                query,
+                storageIntegrationName,
+                stagingBucketRunDir,
+                quotationMark);
 
         String output = snowflakeService.read(config);
 
@@ -448,11 +518,21 @@
       }
     }
 
+    /**
+     * Parses {@code String} from incoming data in {@link PCollection} to have proper format for CSV
+     * files.
+     */
     public static class MapCsvToStringArrayFn extends DoFn<String, String[]> {
+      private String quoteChar;
+
+      public MapCsvToStringArrayFn(String quoteChar) {
+        this.quoteChar = quoteChar;
+      }
+
       @ProcessElement
       public void processElement(ProcessContext c) throws IOException {
         String csvLine = c.element();
-        CSVParser parser = new CSVParserBuilder().withQuoteChar(CSV_QUOTE_CHAR.charAt(0)).build();
+        CSVParser parser = new CSVParserBuilder().withQuoteChar(quoteChar.charAt(0)).build();
         String[] parts = parser.parseLine(csvLine);
         c.output(parts);
       }
@@ -471,16 +551,25 @@
       }
     }
 
+    /** Removes temporary staged files after reading. */
     public static class CleanTmpFilesFromGcsFn extends DoFn<Object, Object> {
       private final String stagingBucketDir;
+      private final String tmpDirName;
 
-      public CleanTmpFilesFromGcsFn(String stagingBucketDir) {
+      /**
+       * Created object that will remove temp files from stage.
+       *
+       * @param stagingBucketDir bucket and directory where temporary files are saved
+       * @param tmpDirName temporary directory created on bucket where files were saved
+       */
+      public CleanTmpFilesFromGcsFn(String stagingBucketDir, String tmpDirName) {
         this.stagingBucketDir = stagingBucketDir;
+        this.tmpDirName = tmpDirName;
       }
 
       @ProcessElement
       public void processElement(ProcessContext c) throws IOException {
-        String combinedPath = stagingBucketDir + "/**";
+        String combinedPath = String.format("%s/%s/**", stagingBucketDir, tmpDirName);
         List<ResourceId> paths =
             FileSystems.match(combinedPath).metadata().stream()
                 .map(metadata -> metadata.resourceId())
@@ -521,10 +610,18 @@
 
     abstract @Nullable String getStagingBucketName();
 
-    abstract @Nullable String getQuery();
+    abstract @Nullable ValueProvider<String> getSnowPipe();
+
+    abstract @Nullable Integer getFlushRowLimit();
+
+    abstract @Nullable Integer getShardsNumber();
+
+    abstract @Nullable Duration getFlushTimeLimit();
 
     abstract @Nullable String getFileNameTemplate();
 
+    abstract @Nullable String getQuery();
+
     abstract @Nullable WriteDisposition getWriteDisposition();
 
     abstract @Nullable CreateDisposition getCreateDisposition();
@@ -535,6 +632,12 @@
 
     abstract @Nullable SnowflakeService getSnowflakeService();
 
+    @Nullable
+    abstract String getQuotationMark();
+
+    @Nullable
+    abstract StreamingLogLevel getDebugMode();
+
     abstract Builder<T> toBuilder();
 
     @AutoValue.Builder
@@ -550,6 +653,14 @@
 
       abstract Builder<T> setQuery(String query);
 
+      abstract Builder<T> setSnowPipe(ValueProvider<String> snowPipe);
+
+      abstract Builder<T> setFlushRowLimit(Integer rowsCount);
+
+      abstract Builder<T> setShardsNumber(Integer shardsNumber);
+
+      abstract Builder<T> setFlushTimeLimit(Duration triggeringFrequency);
+
       abstract Builder<T> setFileNameTemplate(String fileNameTemplate);
 
       abstract Builder<T> setUserDataMapper(UserDataMapper userDataMapper);
@@ -562,13 +673,17 @@
 
       abstract Builder<T> setSnowflakeService(SnowflakeService snowflakeService);
 
+      abstract Builder<T> setQuotationMark(String quotationMark);
+
+      abstract Builder<T> setDebugMode(StreamingLogLevel debugLevel);
+
       abstract Write<T> build();
     }
 
     /**
      * Setting information about Snowflake server.
      *
-     * @param config - An instance of {@link DataSourceConfiguration}.
+     * @param config An instance of {@link DataSourceConfiguration}.
      */
     public Write<T> withDataSourceConfiguration(final DataSourceConfiguration config) {
       return withDataSourceProviderFn(new DataSourceProviderFromDataSourceConfiguration(config));
@@ -587,7 +702,7 @@
     /**
      * A table name to be written in Snowflake.
      *
-     * @param table - String with the name of the table.
+     * @param table String with the name of the table.
      */
     public Write<T> to(String table) {
       return toBuilder().setTable(table).build();
@@ -596,9 +711,12 @@
     /**
      * Name of the cloud bucket (GCS by now) to use as tmp location of CSVs during COPY statement.
      *
-     * @param stagingBucketName - String with the name of the bucket.
+     * @param stagingBucketName String with the name of the bucket.
      */
     public Write<T> withStagingBucketName(String stagingBucketName) {
+      checkArgument(
+          stagingBucketName.endsWith("/"),
+          "stagingBucketName must be a cloud storage path ending with /");
       return toBuilder().setStagingBucketName(stagingBucketName).build();
     }
 
@@ -607,7 +725,7 @@
      * https://docs.snowflake.com/en/sql-reference/sql/create-storage-integration.html for
      * reference.
      *
-     * @param integrationName - String with the name of the Storage Integration.
+     * @param integrationName String with the name of the Storage Integration.
      */
     public Write<T> withStorageIntegrationName(String integrationName) {
       return toBuilder().setStorageIntegrationName(integrationName).build();
@@ -616,7 +734,7 @@
     /**
      * A query to be executed in Snowflake.
      *
-     * @param query - String with query.
+     * @param query String with query.
      */
     public Write<T> withQueryTransformation(String query) {
       return toBuilder().setQuery(query).build();
@@ -625,7 +743,7 @@
     /**
      * A template name for files saved to GCP.
      *
-     * @param fileNameTemplate - String with template name for files.
+     * @param fileNameTemplate String with template name for files.
      */
     public Write<T> withFileNameTemplate(String fileNameTemplate) {
       return toBuilder().setFileNameTemplate(fileNameTemplate).build();
@@ -634,16 +752,96 @@
     /**
      * User-defined function mapping user data into CSV lines.
      *
-     * @param userDataMapper - an instance of {@link UserDataMapper}.
+     * @param userDataMapper an instance of {@link UserDataMapper}.
      */
     public Write<T> withUserDataMapper(UserDataMapper userDataMapper) {
       return toBuilder().setUserDataMapper(userDataMapper).build();
     }
 
     /**
+     * Sets duration how often staged files will be created and then how often ingested by Snowflake
+     * during streaming.
+     *
+     * @param triggeringFrequency time for triggering frequency in {@link Duration} type.
+     * @return
+     */
+    public Write<T> withFlushTimeLimit(Duration triggeringFrequency) {
+      return toBuilder().setFlushTimeLimit(triggeringFrequency).build();
+    }
+
+    /**
+     * Sets name of <a
+     * href="https://docs.snowflake.com/en/user-guide/data-load-snowpipe-intro.html">SnowPipe</a>
+     * which can be created in Snowflake dashboard or cli:
+     *
+     * <pre>{@code
+     * CREATE snowPipeName AS COPY INTO your_table from @yourstage;
+     * }</pre>
+     *
+     * <p>The stage in <a
+     * href="https://docs.snowflake.com/en/sql-reference/sql/copy-into-table.html">COPY</a>
+     * statement should be pointing to the cloud <a
+     * href="https://docs.snowflake.com/en/sql-reference/sql/create-storage-integration.html">integration</a>
+     * with the valid bucket url, ex. for GCS:
+     *
+     * <pre>{@code
+     * CREATE STAGE yourstage
+     * URL = 'gcs://yourbucket/path/'
+     * STORAGE_INTEGRATION = your_integration;
+     * }</pre>
+     *
+     * <pre>{@code
+     * CREATE STORAGE INTEGRATION your_integration
+     *   TYPE = EXTERNAL_STAGE
+     *   STORAGE_PROVIDER = GCS
+     *   ENABLED = TRUE
+     *   STORAGE_ALLOWED_LOCATIONS = ('gcs://yourbucket/path/')
+     * }</pre>
+     *
+     * @param snowPipe name of created SnowPipe in Snowflake dashboard.
+     * @return
+     */
+    public Write<T> withSnowPipe(String snowPipe) {
+      return toBuilder().setSnowPipe(ValueProvider.StaticValueProvider.of(snowPipe)).build();
+    }
+
+    /**
+     * Same as {@code withSnowPipe(String}, but with a {@link ValueProvider}.
+     *
+     * @param snowPipe name of created SnowPipe in Snowflake dashboard.
+     * @return
+     */
+    public Write<T> withSnowPipe(ValueProvider<String> snowPipe) {
+      return toBuilder().setSnowPipe(snowPipe).build();
+    }
+
+    /**
+     * Number of shards that are created per window.
+     *
+     * @param shardsNumber defined number of shards or 1 by default.
+     * @return
+     */
+    public Write<T> withShardsNumber(Integer shardsNumber) {
+      return toBuilder().setShardsNumber(shardsNumber).build();
+    }
+
+    /**
+     * Sets number of row limit that will be saved to the staged file and then loaded to Snowflake.
+     * If the number of rows will be lower than the limit it will be loaded with current number of
+     * rows after certain time specified by setting {@code withFlushTimeLimit(Duration
+     * triggeringFrequency)}
+     *
+     * @param rowsCount Number of rows that will be in one file staged for loading. Default: 10000.
+     * @return
+     */
+    public Write<T> withFlushRowLimit(Integer rowsCount) {
+      return toBuilder().setFlushRowLimit(rowsCount).build();
+    }
+
+    /**
      * A disposition to be used during writing to table phase.
      *
-     * @param writeDisposition - an instance of {@link WriteDisposition}.
+     * @param writeDisposition an instance of {@link WriteDisposition}.
      */
     public Write<T> withWriteDisposition(WriteDisposition writeDisposition) {
       return toBuilder().setWriteDisposition(writeDisposition).build();
@@ -668,28 +866,56 @@
     }
 
     /**
-     * A snowflake service which is supposed to be used. Note: Currently we have {@link
-     * SnowflakeServiceImpl} with corresponding {@link FakeSnowflakeServiceImpl} used for testing.
+     * A snowflake service {@link SnowflakeService} implementation which is supposed to be used.
      *
-     * @param snowflakeService - an instance of {@link SnowflakeService}.
+     * @param snowflakeService an instance of {@link SnowflakeService}.
      */
     public Write<T> withSnowflakeService(SnowflakeService snowflakeService) {
       return toBuilder().setSnowflakeService(snowflakeService).build();
     }
 
+    /**
+     * Sets Snowflake-specific quotations around strings.
+     *
+     * @param quotationMark with possible single quote {@code '}, double quote {@code "} or nothing.
+     *     Default value is single quotation {@code '}.
+     * @return
+     */
+    public Write<T> withQuotationMark(String quotationMark) {
+      return toBuilder().setQuotationMark(quotationMark).build();
+    }
+
+    /**
+     * The option to verbose info (or only errors) of loaded files while streaming. It is not set by
+     * default because it may influence performance. For details: <a
+     * href="https://docs.snowflake.com/en/user-guide/data-load-snowpipe-rest-apis.html#endpoint-insertreport">insert
+     * report REST API.</a>
+     *
+     * @param debugLevel error or info debug level from enum {@link StreamingLogLevel}
+     * @return
+     */
+    public Write<T> withDebugMode(StreamingLogLevel debugLevel) {
+      return toBuilder().setDebugMode(debugLevel).build();
+    }
+
     @Override
     public PDone expand(PCollection<T> input) {
-      checkArguments();
+      checkArguments(input);
 
-      String stagingBucketDir = String.format("%s/%s/", getStagingBucketName(), WRITE_TMP_PATH);
+      PCollection out;
 
-      PCollection<String> out = write(input, stagingBucketDir);
+      if (getSnowPipe() != null) {
+        out = writeStream(input, getStagingBucketName());
+      } else {
+        out = writeBatch(input, getStagingBucketName());
+      }
+
       out.setCoder(StringUtf8Coder.of());
 
       return PDone.in(out.getPipeline());
     }
 
-    private void checkArguments() {
+    private void checkArguments(PCollection<T> input) {
       checkArgument(getStagingBucketName() != null, "withStagingBucketName is required");
 
       checkArgument(getUserDataMapper() != null, "withUserDataMapper() is required");
@@ -698,14 +924,65 @@
           (getDataSourceProviderFn() != null),
           "withDataSourceConfiguration() or withDataSourceProviderFn() is required");
 
-      checkArgument(getTable() != null, "to() is required");
+      if (input.isBounded() == PCollection.IsBounded.UNBOUNDED) {
+        checkArgument(
+            getSnowPipe() != null,
+            "in streaming (unbounded) write it is required to specify SnowPipe name via withSnowPipe() method.");
+      } else {
+        checkArgument(
+            getTable() != null,
+            "in batch writing it is required to specify destination table name via to() method.");
+      }
     }
 
-    private PCollection<String> write(PCollection<T> input, String stagingBucketDir) {
+    private PCollection<T> writeStream(PCollection<T> input, String stagingBucketDir) {
       SnowflakeService snowflakeService =
-          getSnowflakeService() != null ? getSnowflakeService() : new SnowflakeServiceImpl();
+          getSnowflakeService() != null
+              ? getSnowflakeService()
+              : new SnowflakeStreamingServiceImpl();
 
-      PCollection<String> files = writeFiles(input, stagingBucketDir);
+      /* Ensure that files will be created after specific record count or duration specified */
+      PCollection<T> inputInGlobalWindow =
+          input.apply(
+              "Rewindow Into Global",
+              Window.<T>into(new GlobalWindows())
+                  .triggering(
+                      Repeatedly.forever(
+                          AfterFirst.of(
+                              AfterProcessingTime.pastFirstElementInPane()
+                                  .plusDelayOf(getFlushTimeLimit()),
+                              AfterPane.elementCountAtLeast(getFlushRowLimit()))))
+                  .discardingFiredPanes());
+
+      int shards = (getShardsNumber() > 0) ? getShardsNumber() : DEFAULT_STREAMING_SHARDS_NUMBER;
+      PCollection files = writeFiles(inputInGlobalWindow, stagingBucketDir, shards);
+
+      /* Ensuring that files will be ingested after flush time */
+      files =
+          (PCollection)
+              files.apply(
+                  "Apply User Trigger",
+                  Window.<T>into(new GlobalWindows())
+                      .triggering(
+                          Repeatedly.forever(
+                              AfterProcessingTime.pastFirstElementInPane()
+                                  .plusDelayOf(getFlushTimeLimit())))
+                      .discardingFiredPanes());
+      files =
+          (PCollection)
+              files.apply(
+                  "Create list of files for loading via SnowPipe",
+                  Combine.globally(new Concatenate()).withoutDefaults());
+
+      return (PCollection)
+          files.apply("Stream files to table", streamToTable(snowflakeService, stagingBucketDir));
+    }
+
+    private PCollection writeBatch(PCollection input, String stagingBucketDir) {
+      SnowflakeService snowflakeService =
+          getSnowflakeService() != null ? getSnowflakeService() : new SnowflakeBatchServiceImpl();
+
+      PCollection<String> files = writeBatchFiles(input, stagingBucketDir);
 
       // Combining PCollection of files as a side input into one list of files
       ListCoder<String> coder = ListCoder.of(StringUtf8Coder.of());
@@ -721,7 +998,12 @@
           files.apply("Copy files to table", copyToTable(snowflakeService, stagingBucketDir));
     }
 
-    private PCollection<String> writeFiles(PCollection<T> input, String stagingBucketDir) {
+    private PCollection writeBatchFiles(PCollection<T> input, String outputDirectory) {
+      return writeFiles(input, outputDirectory, DEFAULT_BATCH_SHARDS_NUMBER);
+    }
+
+    private PCollection<String> writeFiles(
+        PCollection<T> input, String stagingBucketDir, int numShards) {
 
       PCollection<String> mappedUserData =
           input
@@ -733,7 +1015,9 @@
                           return getUserDataMapper().mapRow(element);
                         }
                       }))
-              .apply("Map Objects array to CSV lines", ParDo.of(new MapObjectsArrayToCsvFn()))
+              .apply(
+                  "Map Objects array to CSV lines",
+                  ParDo.of(new MapObjectsArrayToCsvFn(getQuotationMark())))
               .setCoder(StringUtf8Coder.of());
 
       WriteFilesResult filesResult =
@@ -742,8 +1026,9 @@
               FileIO.<String>write()
                   .via(TextIO.sink())
                   .to(stagingBucketDir)
-                  .withPrefix(getFileNameTemplate())
+                  .withPrefix(UUID.randomUUID().toString().subSequence(0, 8).toString())
                   .withSuffix(".csv")
+                  .withNumShards(numShards)
                   .withCompression(Compression.GZIP));
 
       return (PCollection)
@@ -764,10 +1049,25 @@
               getCreateDisposition(),
               getWriteDisposition(),
               getTableSchema(),
+              snowflakeService,
+              getQuotationMark()));
+    }
+
+    protected PTransform streamToTable(SnowflakeService snowflakeService, String stagingBucketDir) {
+      return ParDo.of(
+          new StreamToTableFn(
+              getDataSourceProviderFn(),
+              getSnowPipe(),
+              stagingBucketDir,
+              getDebugMode(),
               snowflakeService));
     }
   }
 
+  /**
+   * Combines list of {@code String} to provide one {@code String} with paths where files were
+   * staged for write.
+   */
   public static class Concatenate extends Combine.CombineFn<String, List<String>, List<String>> {
     @Override
     public List<String> createAccumulator() {
@@ -801,6 +1101,11 @@
    * <p>Adds Snowflake-specific quotations around strings.
    */
   private static class MapObjectsArrayToCsvFn extends DoFn<Object[], String> {
+    private String quotationMark;
+
+    public MapObjectsArrayToCsvFn(String quotationMark) {
+      this.quotationMark = quotationMark;
+    }
 
     @ProcessElement
     public void processElement(ProcessContext context) {
@@ -820,7 +1125,7 @@
     }
 
     private String quoteField(String field) {
-      return quoteField(field, CSV_QUOTE_CHAR);
+      return quoteField(field, this.quotationMark);
     }
 
     private String quoteField(String field, String quotation) {
@@ -830,11 +1135,14 @@
 
   private static class CopyToTableFn<ParameterT, OutputT> extends DoFn<ParameterT, OutputT> {
     private final SerializableFunction<Void, DataSource> dataSourceProviderFn;
+    private final String database;
+    private final String schema;
     private final String table;
     private final String query;
     private final SnowflakeTableSchema tableSchema;
     private final String stagingBucketDir;
     private final String storageIntegrationName;
+    private final String quotationMark;
     private final WriteDisposition writeDisposition;
     private final CreateDisposition createDisposition;
     private final SnowflakeService snowflakeService;
@@ -848,35 +1156,156 @@
         CreateDisposition createDisposition,
         WriteDisposition writeDisposition,
         SnowflakeTableSchema tableSchema,
-        SnowflakeService snowflakeService) {
+        SnowflakeService snowflakeService,
+        String quotationMark) {
       this.dataSourceProviderFn = dataSourceProviderFn;
-      this.table = table;
       this.query = query;
+      this.table = table;
       this.stagingBucketDir = stagingBucketDir;
       this.storageIntegrationName = storageIntegrationName;
       this.writeDisposition = writeDisposition;
       this.createDisposition = createDisposition;
       this.tableSchema = tableSchema;
       this.snowflakeService = snowflakeService;
+      this.quotationMark = quotationMark;
+
+      DataSourceProviderFromDataSourceConfiguration dataSourceProviderFromDataSourceConfiguration =
+          (DataSourceProviderFromDataSourceConfiguration) this.dataSourceProviderFn;
+      DataSourceConfiguration config = dataSourceProviderFromDataSourceConfiguration.getConfig();
+
+      this.database = config.getDatabase();
+      this.schema = config.getSchema();
     }
 
     @ProcessElement
     public void processElement(ProcessContext context) throws Exception {
-      SnowflakeServiceConfig config =
-          new SnowflakeServiceConfig(
+      SnowflakeBatchServiceConfig config =
+          new SnowflakeBatchServiceConfig(
               dataSourceProviderFn,
               (List<String>) context.element(),
+              database,
+              schema,
               table,
               query,
               tableSchema,
               createDisposition,
               writeDisposition,
               storageIntegrationName,
-              stagingBucketDir);
+              stagingBucketDir,
+              quotationMark);
       snowflakeService.write(config);
     }
   }
 
+  /** Custom DoFn that streams data to Snowflake table. */
+  private static class StreamToTableFn<ParameterT, OutputT> extends DoFn<ParameterT, OutputT> {
+    private final SerializableFunction<Void, DataSource> dataSourceProviderFn;
+    private final String stagingBucketDir;
+    private final ValueProvider<String> snowPipe;
+    private final StreamingLogLevel debugMode;
+    private final SnowflakeService snowflakeService;
+    private transient SimpleIngestManager ingestManager;
+
+    private transient DataSource dataSource;
+    ArrayList<String> trackedFilesNames;
+
+    StreamToTableFn(
+        SerializableFunction<Void, DataSource> dataSourceProviderFn,
+        ValueProvider<String> snowPipe,
+        String stagingBucketDir,
+        StreamingLogLevel debugMode,
+        SnowflakeService snowflakeService) {
+      this.dataSourceProviderFn = dataSourceProviderFn;
+      this.stagingBucketDir = stagingBucketDir;
+      this.snowPipe = snowPipe;
+      this.debugMode = debugMode;
+      this.snowflakeService = snowflakeService;
+      trackedFilesNames = new ArrayList<>();
+    }
+
+    @Setup
+    public void setup() throws Exception {
+      dataSource = dataSourceProviderFn.apply(null);
+
+      DataSourceProviderFromDataSourceConfiguration dataSourceProviderFromDataSourceConfiguration =
+          (DataSourceProviderFromDataSourceConfiguration) this.dataSourceProviderFn;
+      DataSourceConfiguration config = dataSourceProviderFromDataSourceConfiguration.getConfig();
+
+      checkArgument(config.getPrivateKey() != null, "KeyPair is required for authentication");
+
+      String hostName = config.getServerName();
+      List<String> path = Splitter.on('.').splitToList(hostName);
+      String account = path.get(0);
+      String username = config.getUsername();
+      PrivateKey privateKey = config.getPrivateKey();
+      String schema = config.getSchema();
+      String database = config.getDatabase();
+      String snowPipeName = String.format("%s.%s.%s", database, schema, snowPipe.get());
+
+      this.ingestManager =
+          new SimpleIngestManager(
+              account, username, snowPipeName, privateKey, "https", hostName, 443);
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext context) throws Exception {
+      List<String> filesList = (List<String>) context.element();
+
+      if (debugMode != null) {
+        trackedFilesNames.addAll(filesList);
+      }
+      SnowflakeStreamingServiceConfig config =
+          new SnowflakeStreamingServiceConfig(filesList, this.stagingBucketDir, this.ingestManager);
+      snowflakeService.write(config);
+    }
+
+    @FinishBundle
+    public void finishBundle() throws Exception {
+      if (debugMode != null) {
+        String beginMark = null;
+        Duration currentSleep = Duration.ZERO;
+
+        while (currentSleep.isShorterThan(DEFAULT_STREAMING_LOGS_MAX_SLEEP)
+            && trackedFilesNames.size() > 0) {
+          Thread.sleep(DEFAULT_SLEEP_STREAMING_LOGS.getMillis());
+          currentSleep = currentSleep.plus(DEFAULT_SLEEP_STREAMING_LOGS);
+          HistoryResponse response = ingestManager.getHistory(null, null, beginMark);
+
+          if (response != null && response.getNextBeginMark() != null) {
+            beginMark = response.getNextBeginMark();
+          }
+          if (response != null && response.files != null) {
+            response.files.forEach(
+                entry -> {
+                  if (entry.getPath() != null && entry.isComplete()) {
+                    String responseFileName =
+                        String.format("'%s%s'", entry.getStageLocation(), entry.getPath())
+                            .toLowerCase()
+                            .replace("gcs://", "gs://");
+                    if (trackedFilesNames.contains(responseFileName)) {
+                      trackedFilesNames.remove(responseFileName);
+
+                      if (entry.getErrorsSeen() > 0) {
+                        LOG.error(String.format("Snowflake SnowPipe ERROR: %s", entry.toString()));
+                      } else if (entry.getErrorsSeen() == 0
+                          && debugMode.equals(StreamingLogLevel.INFO)) {
+                        LOG.info(String.format("Snowflake SnowPipe INFO: %s", entry.toString()));
+                      }
+                    }
+                  }
+                });
+          }
+        }
+        trackedFilesNames.forEach(
+            file -> LOG.info(String.format("File %s was not found in ingest history", file)));
+      }
+    }
+  }
+
+  private static String getValueOrNull(ValueProvider<String> valueProvider) {
+    return valueProvider != null ? valueProvider.get() : null;
+  }
+
   /**
    * A POJO describing a {@link DataSource}, providing all properties allowing to create a {@link
    * DataSource}.
@@ -954,7 +1383,7 @@
     /**
      * Creates {@link DataSourceConfiguration} from existing instance of {@link DataSource}.
      *
-     * @param dataSource - an instance of {@link DataSource}.
+     * @param dataSource an instance of {@link DataSource}.
      */
     public static DataSourceConfiguration create(DataSource dataSource) {
       checkArgument(dataSource instanceof Serializable, "dataSource must be Serializable");
@@ -967,7 +1396,7 @@
     /**
      * Creates {@link DataSourceConfiguration} from instance of {@link SnowflakeCredentials}.
      *
-     * @param credentials - an instance of {@link SnowflakeCredentials}.
+     * @param credentials an instance of {@link SnowflakeCredentials}.
      */
     public static DataSourceConfiguration create(SnowflakeCredentials credentials) {
       if (credentials instanceof UsernamePasswordSnowflakeCredentials) {
@@ -998,7 +1427,7 @@
      *
      * <p>Either withUrl or withServerName is required.
      *
-     * @param url - String with URL of the Snowflake server.
+     * @param url String with URL of the Snowflake server.
      */
     public DataSourceConfiguration withUrl(String url) {
       checkArgument(
@@ -1013,7 +1442,7 @@
     /**
      * Sets database to use.
      *
-     * @param database - String with database name.
+     * @param database String with database name.
      */
     public DataSourceConfiguration withDatabase(String database) {
       return builder().setDatabase(database).build();
@@ -1022,7 +1451,7 @@
     /**
      * Sets Snowflake Warehouse to use.
      *
-     * @param warehouse - String with warehouse name.
+     * @param warehouse String with warehouse name.
      */
     public DataSourceConfiguration withWarehouse(String warehouse) {
       return builder().setWarehouse(warehouse).build();
@@ -1031,7 +1460,7 @@
     /**
      * Sets schema to use when connecting to Snowflake.
      *
-     * @param schema - String with schema name.
+     * @param schema String with schema name.
      */
     public DataSourceConfiguration withSchema(String schema) {
       return builder().setSchema(schema).build();
@@ -1043,7 +1472,7 @@
      *
      * <p>Either withServerName or withUrl is required.
      *
-     * @param serverName - String with server name.
+     * @param serverName String with server name.
      */
     public DataSourceConfiguration withServerName(String serverName) {
       checkArgument(
@@ -1055,7 +1484,7 @@
     /**
      * Sets port number to use to connect to Snowflake.
      *
-     * @param portNumber - Integer with port number.
+     * @param portNumber Integer with port number.
      */
     public DataSourceConfiguration withPortNumber(Integer portNumber) {
       return builder().setPortNumber(portNumber).build();
@@ -1064,16 +1493,16 @@
     /**
      * Sets user's role to be used when running queries on Snowflake.
      *
-     * @param role - String with role name.
+     * @param role String with role name.
      */
     public DataSourceConfiguration withRole(String role) {
       return builder().setRole(role).build();
     }
 
     /**
-     * Sets loginTimeout that will be used in {@link SnowflakeBasicDataSource:setLoginTimeout}.
+     * Sets loginTimeout that will be used in {@link SnowflakeBasicDataSource#setLoginTimeout}.
      *
-     * @param loginTimeout - Integer with timeout value.
+     * @param loginTimeout Integer with timeout value.
      */
     public DataSourceConfiguration withLoginTimeout(Integer loginTimeout) {
       return builder().setLoginTimeout(loginTimeout).build();
@@ -1156,6 +1585,7 @@
     }
   }
 
+  /** Wraps {@link DataSourceConfiguration} to provide DataSource. */
   public static class DataSourceProviderFromDataSourceConfiguration
       implements SerializableFunction<Void, DataSource>, HasDisplayData {
     private static final ConcurrentHashMap<DataSourceConfiguration, DataSource> instances =
@@ -1188,5 +1618,9 @@
     public void populateDisplayData(DisplayData.Builder builder) {
       config.populateDisplayData(builder);
     }
+
+    public DataSourceConfiguration getConfig() {
+      return this.config;
+    }
   }
 }
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakePipelineOptions.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakePipelineOptions.java
index bf91e0c..201f71c 100644
--- a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakePipelineOptions.java
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakePipelineOptions.java
@@ -20,9 +20,11 @@
 import org.apache.beam.sdk.options.Default;
 import org.apache.beam.sdk.options.Description;
 import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.StreamingOptions;
 import org.apache.beam.sdk.options.Validation;
+import org.apache.beam.sdk.options.ValueProvider;
 
-public interface SnowflakePipelineOptions extends PipelineOptions {
+public interface SnowflakePipelineOptions extends PipelineOptions, StreamingOptions {
   String BASIC_CONNECTION_INFO_VALIDATION_GROUP = "BASIC_CONNECTION_INFO_GROUP";
   String AUTH_VALIDATION_GROUP = "AUTH_VALIDATION_GROUP";
 
@@ -120,4 +122,9 @@
   String getStorageIntegrationName();
 
   void setStorageIntegrationName(String storageIntegrationName);
+
+  @Description("SnowPipe name. Optional.")
+  ValueProvider<String> getSnowPipe();
+
+  void setSnowPipe(ValueProvider<String> snowPipe);
 }
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/SnowflakeCredentialsFactory.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/SnowflakeCredentialsFactory.java
index 3876c2f..2b45dc1 100644
--- a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/SnowflakeCredentialsFactory.java
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/SnowflakeCredentialsFactory.java
@@ -18,38 +18,52 @@
 package org.apache.beam.sdk.io.snowflake.credentials;
 
 import org.apache.beam.sdk.io.snowflake.SnowflakePipelineOptions;
+import org.apache.beam.sdk.io.snowflake.crosslanguage.SnowflakeReadRegistrar;
 
 /**
  * Factory class for creating implementations of {@link SnowflakeCredentials} from {@link
  * SnowflakePipelineOptions}.
  */
 public class SnowflakeCredentialsFactory {
-  public static SnowflakeCredentials of(SnowflakePipelineOptions options) {
-    if (oauthOptionsAvailable(options)) {
-      return new OAuthTokenSnowflakeCredentials(options.getOauthToken());
-    } else if (usernamePasswordOptionsAvailable(options)) {
-      return new UsernamePasswordSnowflakeCredentials(options.getUsername(), options.getPassword());
-    } else if (keyPairOptionsAvailable(options)) {
+  public static SnowflakeCredentials of(SnowflakePipelineOptions o) {
+    if (oauthOptionsAvailable(o.getOauthToken())) {
+      return new OAuthTokenSnowflakeCredentials(o.getOauthToken());
+    } else if (usernamePasswordOptionsAvailable(o.getUsername(), o.getPassword())) {
+      return new UsernamePasswordSnowflakeCredentials(o.getUsername(), o.getPassword());
+    } else if (keyPairOptionsAvailable(
+        o.getUsername(), o.getPrivateKeyPath(), o.getPrivateKeyPassphrase())) {
       return new KeyPairSnowflakeCredentials(
-          options.getUsername(), options.getPrivateKeyPath(), options.getPrivateKeyPassphrase());
+          o.getUsername(), o.getPrivateKeyPath(), o.getPrivateKeyPassphrase());
     }
     throw new RuntimeException("Can't get credentials from Options");
   }
 
-  private static boolean oauthOptionsAvailable(SnowflakePipelineOptions options) {
-    return options.getOauthToken() != null && !options.getOauthToken().isEmpty();
+  public static SnowflakeCredentials of(SnowflakeReadRegistrar.ReadConfiguration c) {
+    if (oauthOptionsAvailable(c.getOAuthToken())) {
+      return new OAuthTokenSnowflakeCredentials(c.getOAuthToken());
+    } else if (usernamePasswordOptionsAvailable(c.getUsername(), c.getPassword())) {
+      return new UsernamePasswordSnowflakeCredentials(c.getUsername(), c.getPassword());
+    } else if (keyPairOptionsAvailable(
+        c.getUsername(), c.getPrivateKeyPath(), c.getPrivateKeyPassphrase())) {
+      return new KeyPairSnowflakeCredentials(
+          c.getUsername(), c.getPrivateKeyPath(), c.getPrivateKeyPassphrase());
+    }
+    throw new RuntimeException("Can't get credentials from Options");
   }
 
-  private static boolean usernamePasswordOptionsAvailable(SnowflakePipelineOptions options) {
-    return options.getUsername() != null
-        && !options.getUsername().isEmpty()
-        && !options.getPassword().isEmpty();
+  private static boolean oauthOptionsAvailable(String token) {
+    return token != null && !token.isEmpty();
   }
 
-  private static boolean keyPairOptionsAvailable(SnowflakePipelineOptions options) {
-    return options.getUsername() != null
-        && !options.getUsername().isEmpty()
-        && !options.getPrivateKeyPath().isEmpty()
-        && !options.getPrivateKeyPassphrase().isEmpty();
+  private static boolean usernamePasswordOptionsAvailable(String username, String password) {
+    return username != null && !username.isEmpty() && !password.isEmpty();
+  }
+
+  private static boolean keyPairOptionsAvailable(
+      String username, String privateKeyPath, String privateKeyPassphrase) {
+    return username != null
+        && !username.isEmpty()
+        && !privateKeyPath.isEmpty()
+        && !privateKeyPassphrase.isEmpty();
   }
 }
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/crosslanguage/Configuration.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/crosslanguage/Configuration.java
new file mode 100644
index 0000000..38162ae
--- /dev/null
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/crosslanguage/Configuration.java
@@ -0,0 +1,130 @@
+/*
+ * 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.snowflake.crosslanguage;
+
+/** Parameters abstract class to expose the transforms to an external SDK. */
+public abstract class Configuration {
+  private String serverName;
+  private String username;
+  private String password;
+  private String privateKeyPath;
+  private String privateKeyPassphrase;
+  private String oAuthToken;
+  private String database;
+  private String schema;
+  private String table;
+  private String query;
+  private String stagingBucketName;
+  private String storageIntegrationName;
+
+  public String getServerName() {
+    return serverName;
+  }
+
+  public void setServerName(String serverName) {
+    this.serverName = serverName;
+  }
+
+  public String getUsername() {
+    return username;
+  }
+
+  public void setUsername(String username) {
+    this.username = username;
+  }
+
+  public String getPassword() {
+    return password;
+  }
+
+  public void setPassword(String password) {
+    this.password = password;
+  }
+
+  public String getPrivateKeyPath() {
+    return privateKeyPath;
+  }
+
+  public void setPrivateKeyPath(String privateKeyPath) {
+    this.privateKeyPath = privateKeyPath;
+  }
+
+  public String getPrivateKeyPassphrase() {
+    return privateKeyPassphrase;
+  }
+
+  public void setPrivateKeyPassphrase(String privateKeyPassphrase) {
+    this.privateKeyPassphrase = privateKeyPassphrase;
+  }
+
+  public String getOAuthToken() {
+    return oAuthToken;
+  }
+
+  public void setOAuthToken(String oAuthToken) {
+    this.oAuthToken = oAuthToken;
+  }
+
+  public String getDatabase() {
+    return database;
+  }
+
+  public void setDatabase(String database) {
+    this.database = database;
+  }
+
+  public String getSchema() {
+    return schema;
+  }
+
+  public void setSchema(String schema) {
+    this.schema = schema;
+  }
+
+  public String getTable() {
+    return table;
+  }
+
+  public void setTable(String table) {
+    this.table = table;
+  }
+
+  public String getQuery() {
+    return query;
+  }
+
+  public void setQuery(String query) {
+    this.query = query;
+  }
+
+  public String getStagingBucketName() {
+    return stagingBucketName;
+  }
+
+  public void setStagingBucketName(String stagingBucketName) {
+    this.stagingBucketName = stagingBucketName;
+  }
+
+  public String getStorageIntegrationName() {
+    return storageIntegrationName;
+  }
+
+  public void setStorageIntegrationName(String storageIntegrationName) {
+    this.storageIntegrationName = storageIntegrationName;
+  }
+}
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/crosslanguage/SnowflakeReadRegistrar.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/crosslanguage/SnowflakeReadRegistrar.java
new file mode 100644
index 0000000..1e7be0f
--- /dev/null
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/crosslanguage/SnowflakeReadRegistrar.java
@@ -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.
+ */
+package org.apache.beam.sdk.io.snowflake.crosslanguage;
+
+import com.google.auto.service.AutoService;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.util.Map;
+import javax.sql.DataSource;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.expansion.ExternalTransformRegistrar;
+import org.apache.beam.sdk.io.snowflake.SnowflakeIO;
+import org.apache.beam.sdk.io.snowflake.credentials.SnowflakeCredentials;
+import org.apache.beam.sdk.io.snowflake.credentials.SnowflakeCredentialsFactory;
+import org.apache.beam.sdk.transforms.ExternalTransformBuilder;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+/** Exposes {@link SnowflakeIO.Read} as an external transform for cross-language usage. */
+@Experimental
+@AutoService(ExternalTransformRegistrar.class)
+public final class SnowflakeReadRegistrar implements ExternalTransformRegistrar {
+
+  public static final String URN = "beam:external:java:snowflake:read:v1";
+
+  @Override
+  public Map<String, Class<? extends ExternalTransformBuilder<?, ?, ?>>> knownBuilders() {
+    return ImmutableMap.of(URN, ReadBuilder.class);
+  }
+
+  /** Parameters class to expose the transform to an external SDK. */
+  public static class ReadConfiguration extends Configuration {}
+
+  public static class ReadBuilder
+      implements ExternalTransformBuilder<ReadConfiguration, PBegin, PCollection<byte[]>> {
+    public ReadBuilder() {}
+
+    @Override
+    public PTransform<PBegin, PCollection<byte[]>> buildExternal(ReadConfiguration c) {
+      SnowflakeCredentials credentials = SnowflakeCredentialsFactory.of(c);
+
+      SerializableFunction<Void, DataSource> dataSourceSerializableFunction =
+          SnowflakeIO.DataSourceProviderFromDataSourceConfiguration.of(
+              SnowflakeIO.DataSourceConfiguration.create(credentials)
+                  .withServerName(c.getServerName())
+                  .withDatabase(c.getDatabase())
+                  .withSchema(c.getSchema()));
+
+      return SnowflakeIO.<byte[]>read()
+          .withStorageIntegrationName(c.getStorageIntegrationName())
+          .withStagingBucketName(c.getStagingBucketName())
+          .withDataSourceProviderFn(dataSourceSerializableFunction)
+          .withCsvMapper(CsvMapper.getCsvMapper())
+          .withCoder(ByteArrayCoder.of())
+          .fromTable(c.getTable())
+          .fromQuery(c.getQuery());
+    }
+  }
+
+  private static class CsvMapper implements Serializable {
+
+    public static SnowflakeIO.CsvMapper getCsvMapper() {
+      return (SnowflakeIO.CsvMapper<byte[]>)
+          parts -> {
+            String partsCSV = String.join(",", parts);
+
+            return partsCSV.getBytes(Charset.defaultCharset());
+          };
+    }
+  }
+}
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/crosslanguage/package-info.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/crosslanguage/package-info.java
new file mode 100644
index 0000000..7e24ee9
--- /dev/null
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/crosslanguage/package-info.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** Cross-language for SnowflakeIO. */
+@Experimental(Kind.PORTABILITY)
+@DefaultAnnotation(NonNull.class)
+package org.apache.beam.sdk.io.snowflake.crosslanguage;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Experimental.Kind;
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/enums/StreamingLogLevel.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/enums/StreamingLogLevel.java
new file mode 100644
index 0000000..f547b07
--- /dev/null
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/enums/StreamingLogLevel.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io.snowflake.enums;
+
+public enum StreamingLogLevel {
+  INFO,
+  ERROR
+}
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/ServiceConfig.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/ServiceConfig.java
index 09e1368..1826ce9 100644
--- a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/ServiceConfig.java
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/ServiceConfig.java
@@ -17,4 +17,8 @@
  */
 package org.apache.beam.sdk.io.snowflake.services;
 
+/**
+ * Configuration abstract class for {@link SnowflakeService} that gives parameters for write and
+ * read (batch and streaming).
+ */
 public abstract class ServiceConfig {}
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeServiceConfig.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeBatchServiceConfig.java
similarity index 60%
rename from sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeServiceConfig.java
rename to sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeBatchServiceConfig.java
index fc68a00..726e9d7 100644
--- a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeServiceConfig.java
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeBatchServiceConfig.java
@@ -24,81 +24,120 @@
 import org.apache.beam.sdk.io.snowflake.enums.WriteDisposition;
 import org.apache.beam.sdk.transforms.SerializableFunction;
 
-public class SnowflakeServiceConfig extends ServiceConfig {
-  private SerializableFunction<Void, DataSource> dataSourceProviderFn;
+/** Class for preparing configuration for batch write and read. */
+public class SnowflakeBatchServiceConfig extends ServiceConfig {
+  private final SerializableFunction<Void, DataSource> dataSourceProviderFn;
 
-  private String table;
-  private String query;
-  private String storageIntegrationName;
+  private final String database;
+  private final String schema;
+  private final String table;
+  private final String query;
+  private final String storageIntegrationName;
   private List<String> filesList;
-
   private WriteDisposition writeDisposition;
   private CreateDisposition createDisposition;
   private SnowflakeTableSchema tableSchema;
-  private String stagingBucketDir;
+  private final String stagingBucketDir;
+  private final String quotationMark;
 
-  public SnowflakeServiceConfig(
+  /** Creating a batch configuration for reading. */
+  public SnowflakeBatchServiceConfig(
       SerializableFunction<Void, DataSource> dataSourceProviderFn,
+      String database,
+      String schema,
       String table,
       String query,
-      String storageIntegration,
-      String stagingBucketDir) {
+      String storageIntegrationName,
+      String stagingBucketDir,
+      String quotationMark) {
     this.dataSourceProviderFn = dataSourceProviderFn;
+    this.database = database;
+    this.schema = schema;
     this.table = table;
     this.query = query;
-    this.storageIntegrationName = storageIntegration;
+    this.storageIntegrationName = storageIntegrationName;
     this.stagingBucketDir = stagingBucketDir;
+    this.quotationMark = quotationMark;
   }
 
-  public SnowflakeServiceConfig(
+  /** Creating a batch configuration for writing. */
+  public SnowflakeBatchServiceConfig(
       SerializableFunction<Void, DataSource> dataSourceProviderFn,
       List<String> filesList,
+      String database,
+      String schema,
       String table,
       String query,
       SnowflakeTableSchema tableSchema,
       CreateDisposition createDisposition,
       WriteDisposition writeDisposition,
       String storageIntegrationName,
-      String stagingBucketDir) {
+      String stagingBucketDir,
+      String quotationMark) {
     this.dataSourceProviderFn = dataSourceProviderFn;
     this.filesList = filesList;
+    this.database = database;
+    this.schema = schema;
     this.table = table;
     this.query = query;
-    this.tableSchema = tableSchema;
     this.writeDisposition = writeDisposition;
     this.createDisposition = createDisposition;
+    this.tableSchema = tableSchema;
     this.storageIntegrationName = storageIntegrationName;
     this.stagingBucketDir = stagingBucketDir;
+    this.quotationMark = quotationMark;
   }
 
+  /** Getting a DataSource provider function for connection credentials. */
   public SerializableFunction<Void, DataSource> getDataSourceProviderFn() {
     return dataSourceProviderFn;
   }
 
+  /** Getting a table as a source of reading or destination to write. */
   public String getTable() {
     return table;
   }
 
+  /** Getting a query which can be source for reading. */
   public String getQuery() {
     return query;
   }
 
+  /** Getting Snowflake integration which is used in COPY statement. */
   public String getStorageIntegrationName() {
     return storageIntegrationName;
   }
 
+  /** Getting directory where files are staged. */
   public String getStagingBucketDir() {
     return stagingBucketDir;
   }
 
+  /** Getting list of names of staged files. */
   public List<String> getFilesList() {
     return filesList;
   }
 
+  /** Getting disposition how write data to table, see: {@link WriteDisposition}. */
   public WriteDisposition getWriteDisposition() {
     return writeDisposition;
   }
 
+  /** Getting a character that will surround {@code String} in staged CSV files. */
+  public String getQuotationMark() {
+    return quotationMark;
+  }
+
+  /** Getting a Snowflake database. */
+  public String getDatabase() {
+    return database;
+  }
+
+  /** Getting a schema of a Snowflake table. */
+  public String getSchema() {
+    return schema;
+  }
+
   public CreateDisposition getCreateDisposition() {
     return createDisposition;
   }
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeServiceImpl.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeBatchServiceImpl.java
similarity index 79%
rename from sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeServiceImpl.java
rename to sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeBatchServiceImpl.java
index 36e9f3b..2b73bd5 100644
--- a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeServiceImpl.java
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeBatchServiceImpl.java
@@ -19,6 +19,8 @@
 
 import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
 
+import java.math.BigInteger;
+import java.nio.charset.Charset;
 import java.sql.Connection;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
@@ -35,23 +37,36 @@
 import org.slf4j.LoggerFactory;
 
 /** Implemenation of {@link SnowflakeService} used in production. */
-public class SnowflakeServiceImpl implements SnowflakeService<SnowflakeServiceConfig> {
-  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeServiceImpl.class);
+public class SnowflakeBatchServiceImpl implements SnowflakeService<SnowflakeBatchServiceConfig> {
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeBatchServiceImpl.class);
   private static final String SNOWFLAKE_GCS_PREFIX = "gcs://";
   private static final String GCS_PREFIX = "gs://";
 
+  /** Writing data to Snowflake in batch mode. */
   @Override
-  public void write(SnowflakeServiceConfig config) throws Exception {
+  public void write(SnowflakeBatchServiceConfig config) throws Exception {
     copyToTable(config);
   }
 
+  /** Reading data from Snowflake tables in batch processing. */
   @Override
-  public String read(SnowflakeServiceConfig config) throws Exception {
+  public String read(SnowflakeBatchServiceConfig config) throws Exception {
     return copyIntoStage(config);
   }
 
-  public String copyIntoStage(SnowflakeServiceConfig config) throws SQLException {
+  /**
+   * Copies data from specified table to stage (bucket and directory). Uses Snowflake's <a
+   * href="https://docs.snowflake.com/en/sql-reference/sql/copy-into-location.html">COPY method</a>.
+   * All the details needed for COPY are inside passed configuration.
+   *
+   * @param config object with configuration to perform COPY query.
+   * @return destination where files were copied into
+   * @throws SQLException in case COPY query failed.
+   */
+  private String copyIntoStage(SnowflakeBatchServiceConfig config) throws SQLException {
     SerializableFunction<Void, DataSource> dataSourceProviderFn = config.getDataSourceProviderFn();
+    String database = config.getDatabase();
+    String schema = config.getSchema();
     String table = config.getTable();
     String query = config.getQuery();
     String storageIntegrationName = config.getStorageIntegrationName();
@@ -62,7 +77,7 @@
       // Query must be surrounded with brackets
       source = String.format("(%s)", query);
     } else {
-      source = table;
+      source = getTablePath(database, schema, table);
     }
 
     String copyQuery =
@@ -71,17 +86,31 @@
             getProperBucketDir(stagingBucketDir),
             source,
             storageIntegrationName,
-            CSV_QUOTE_CHAR_FOR_COPY);
+            getASCIICharRepresentation(config.getQuotationMark()));
 
     runStatement(copyQuery, getConnection(dataSourceProviderFn), null);
 
     return stagingBucketDir.concat("*");
   }
 
-  public void copyToTable(SnowflakeServiceConfig config) throws SQLException {
+  private String getASCIICharRepresentation(String input) {
+    return String.format("0x%x", new BigInteger(1, input.getBytes(Charset.defaultCharset())));
+  }
+
+  /**
+   * Copies staged data from bucket directory to table. Uses Snowflake's <a
+   * href="https://docs.snowflake.com/en/sql-reference/sql/copy-into-table.html">COPY method</a>.
+   * All the details needed for COPY are inside passed configuration.
+   *
+   * @param config object with configuration to perform COPY query.
+   * @throws SQLException
+   */
+  private void copyToTable(SnowflakeBatchServiceConfig config) throws SQLException {
 
     SerializableFunction<Void, DataSource> dataSourceProviderFn = config.getDataSourceProviderFn();
     List<String> filesList = config.getFilesList();
+    String database = config.getDatabase();
+    String schema = config.getSchema();
     String table = config.getTable();
     String query = config.getQuery();
     SnowflakeTableSchema tableSchema = config.getTableSchema();
@@ -110,28 +139,29 @@
       query =
           String.format(
               "COPY INTO %s FROM %s FILES=(%s) FILE_FORMAT=(TYPE=CSV FIELD_OPTIONALLY_ENCLOSED_BY='%s' COMPRESSION=GZIP) STORAGE_INTEGRATION=%s;",
-              table,
+              getTablePath(database, schema, table),
               getProperBucketDir(source),
               files,
-              CSV_QUOTE_CHAR_FOR_COPY,
+              getASCIICharRepresentation(config.getQuotationMark()),
               storageIntegrationName);
     } else {
       query =
           String.format(
               "COPY INTO %s FROM %s FILES=(%s) FILE_FORMAT=(TYPE=CSV FIELD_OPTIONALLY_ENCLOSED_BY='%s' COMPRESSION=GZIP);",
-              table, source, files, CSV_QUOTE_CHAR_FOR_COPY);
+              table, source, files, getASCIICharRepresentation(config.getQuotationMark()));
     }
 
     runStatement(query, dataSource.getConnection(), null);
   }
 
-  private void truncateTable(DataSource dataSource, String table) throws SQLException {
-    String query = String.format("TRUNCATE %s;", table);
+  private void truncateTable(DataSource dataSource, String tablePath) throws SQLException {
+    String query = String.format("TRUNCATE %s;", tablePath);
     runConnectionWithStatement(dataSource, query, null);
   }
 
-  private static void checkIfTableIsEmpty(DataSource dataSource, String table) throws SQLException {
-    String selectQuery = String.format("SELECT count(*) FROM %s LIMIT 1;", table);
+  private static void checkIfTableIsEmpty(DataSource dataSource, String tablePath)
+      throws SQLException {
+    String selectQuery = String.format("SELECT count(*) FROM %s LIMIT 1;", tablePath);
     runConnectionWithStatement(
         dataSource,
         selectQuery,
@@ -275,4 +305,8 @@
     }
     return bucketDir;
   }
+
+  private String getTablePath(String database, String schema, String table) {
+    return String.format("%s.%s.%s", database, schema, table);
+  }
 }
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeService.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeService.java
index 16cd3c6..192856e 100644
--- a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeService.java
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeService.java
@@ -21,8 +21,6 @@
 
 /** Interface which defines common methods for interacting with Snowflake. */
 public interface SnowflakeService<T extends ServiceConfig> extends Serializable {
-  String CSV_QUOTE_CHAR_FOR_COPY = "''";
-
   String read(T config) throws Exception;
 
   void write(T config) throws Exception;
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeStreamingServiceConfig.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeStreamingServiceConfig.java
new file mode 100644
index 0000000..7039c89
--- /dev/null
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeStreamingServiceConfig.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.snowflake.services;
+
+import java.util.List;
+import net.snowflake.ingest.SimpleIngestManager;
+
+/** Class for preparing configuration for streaming write. */
+public class SnowflakeStreamingServiceConfig extends ServiceConfig {
+  private final SimpleIngestManager ingestManager;
+  private final List<String> filesList;
+  private final String stagingBucketDir;
+
+  /**
+   * Constructor to create configuration for streaming write.
+   *
+   * @param filesList list of strings of staged files' names.
+   * @param stagingBucketDir name of a bucket and directory inside where files are staged and awaits
+   *     for being loaded to Snowflake.
+   * @param ingestManager instance of {@link SimpleIngestManager}.
+   */
+  public SnowflakeStreamingServiceConfig(
+      List<String> filesList, String stagingBucketDir, SimpleIngestManager ingestManager) {
+    this.filesList = filesList;
+    this.stagingBucketDir = stagingBucketDir;
+    this.ingestManager = ingestManager;
+  }
+
+  /**
+   * Getter for ingest manager which serves API to load data in streaming mode and retrieve a report
+   * about loaded data.
+   *
+   * @return instance of {@link SimpleIngestManager}.
+   */
+  public SimpleIngestManager getIngestManager() {
+    return ingestManager;
+  }
+
+  /**
+   * Getter for a list of staged files which are will be loaded to Snowflake.
+   *
+   * @return list of strings of staged files' names.
+   */
+  public List<String> getFilesList() {
+    return filesList;
+  }
+
+  /**
+   * Getter for a bucket name with directory where files were staged and waiting for loading.
+   *
+   * @return name of a bucket and directory inside in form {@code gs://mybucket/dir/}
+   */
+  public String getStagingBucketDir() {
+    return stagingBucketDir;
+  }
+}
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeStreamingServiceImpl.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeStreamingServiceImpl.java
new file mode 100644
index 0000000..8d555ed
--- /dev/null
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeStreamingServiceImpl.java
@@ -0,0 +1,72 @@
+/*
+ * 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.snowflake.services;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import net.snowflake.ingest.SimpleIngestManager;
+import net.snowflake.ingest.connection.IngestResponseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Implemenation of {@link SnowflakeService} used in production. */
+public class SnowflakeStreamingServiceImpl
+    implements SnowflakeService<SnowflakeStreamingServiceConfig> {
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeStreamingServiceImpl.class);
+  private transient SimpleIngestManager ingestManager;
+
+  /** Writing data to Snowflake in streaming mode. */
+  @Override
+  public void write(SnowflakeStreamingServiceConfig config) throws Exception {
+    ingest(config);
+  }
+
+  /** Reading data from Snowflake in streaming mode is not supported. */
+  @Override
+  public String read(SnowflakeStreamingServiceConfig config) throws Exception {
+    throw new UnsupportedOperationException("Not supported by SnowflakeIO.");
+  }
+
+  /**
+   * SnowPipe is processing files from stage in streaming mode.
+   *
+   * @param config configuration object containing parameters for writing files to Snowflake
+   * @throws IngestResponseException REST API response error
+   * @throws IOException Snowflake problem while streaming
+   * @throws URISyntaxException creating request error
+   */
+  private void ingest(SnowflakeStreamingServiceConfig config)
+      throws IngestResponseException, IOException, URISyntaxException {
+    List<String> filesList = config.getFilesList();
+    String stagingBucketDir = config.getStagingBucketDir();
+    ingestManager = config.getIngestManager();
+
+    Set<String> files =
+        filesList.stream()
+            .map(e -> e.replaceAll(String.valueOf(stagingBucketDir), ""))
+            .map(e -> e.replaceAll("'", ""))
+            .collect(Collectors.toSet());
+
+    if (!files.isEmpty()) {
+      this.ingestManager.ingestFiles(SimpleIngestManager.wrapFilepaths(files), null);
+    }
+  }
+}
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeServiceImpl.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeBatchServiceImpl.java
similarity index 89%
rename from sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeServiceImpl.java
rename to sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeBatchServiceImpl.java
index 7ff097d..90ee4b9 100644
--- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeServiceImpl.java
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeBatchServiceImpl.java
@@ -28,23 +28,24 @@
 import org.apache.beam.sdk.io.snowflake.data.SnowflakeTableSchema;
 import org.apache.beam.sdk.io.snowflake.enums.CreateDisposition;
 import org.apache.beam.sdk.io.snowflake.enums.WriteDisposition;
+import org.apache.beam.sdk.io.snowflake.services.SnowflakeBatchServiceConfig;
 import org.apache.beam.sdk.io.snowflake.services.SnowflakeService;
-import org.apache.beam.sdk.io.snowflake.services.SnowflakeServiceConfig;
 
 /** Fake implementation of {@link SnowflakeService} used in tests. */
-public class FakeSnowflakeServiceImpl implements SnowflakeService<SnowflakeServiceConfig> {
+public class FakeSnowflakeBatchServiceImpl
+    implements SnowflakeService<SnowflakeBatchServiceConfig> {
 
   @Override
-  public void write(SnowflakeServiceConfig config) throws Exception {
+  public void write(SnowflakeBatchServiceConfig config) throws Exception {
     copyToTable(config);
   }
 
   @Override
-  public String read(SnowflakeServiceConfig config) throws Exception {
+  public String read(SnowflakeBatchServiceConfig config) throws Exception {
     return copyIntoStage(config);
   }
 
-  public String copyIntoStage(SnowflakeServiceConfig config) throws SQLException {
+  public String copyIntoStage(SnowflakeBatchServiceConfig config) throws SQLException {
     String table = config.getTable();
     String query = config.getQuery();
 
@@ -60,7 +61,7 @@
     return String.format("./%s/*", stagingBucketDir);
   }
 
-  public void copyToTable(SnowflakeServiceConfig config) throws SQLException {
+  public void copyToTable(SnowflakeBatchServiceConfig config) throws SQLException {
     List<String> filesList = config.getFilesList();
     String table = config.getTable();
     SnowflakeTableSchema tableSchema = config.getTableSchema();
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeIngestManager.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeIngestManager.java
new file mode 100644
index 0000000..e144ae7
--- /dev/null
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeIngestManager.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io.snowflake.test;
+
+import java.util.List;
+
+public class FakeSnowflakeIngestManager {
+  // Only for testing purposes
+  private String table = "TEST_TABLE";
+
+  public FakeSnowflakeIngestManager() {}
+
+  public void ingestFiles(List<String> rows) {
+    FakeSnowflakeDatabase.createTableWithElements(this.table, rows);
+  }
+
+  public String getTable() {
+    return this.table;
+  }
+}
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeStreamingServiceImpl.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeStreamingServiceImpl.java
new file mode 100644
index 0000000..362eb5e
--- /dev/null
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeStreamingServiceImpl.java
@@ -0,0 +1,73 @@
+/*
+ * 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.snowflake.test;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.zip.GZIPInputStream;
+import org.apache.beam.sdk.io.snowflake.services.SnowflakeService;
+import org.apache.beam.sdk.io.snowflake.services.SnowflakeStreamingServiceConfig;
+
+/** Fake implementation of {@link SnowflakeService} used in tests. */
+public class FakeSnowflakeStreamingServiceImpl
+    implements SnowflakeService<SnowflakeStreamingServiceConfig> {
+  private FakeSnowflakeIngestManager snowflakeIngestManager;
+
+  @Override
+  public void write(SnowflakeStreamingServiceConfig config) throws Exception {
+    snowflakeIngestManager = new FakeSnowflakeIngestManager();
+    ingest(config);
+  }
+
+  @Override
+  public String read(SnowflakeStreamingServiceConfig config) throws Exception {
+    throw new UnsupportedOperationException("Streaming read is not supported in SnowflakeIO.");
+  }
+
+  public void ingest(SnowflakeStreamingServiceConfig config) {
+    List<String> rows = new ArrayList<>();
+    List<String> filesList = config.getFilesList();
+    for (String file : filesList) {
+      rows.addAll(readGZIPFile(file.replace("'", "")));
+    }
+
+    snowflakeIngestManager.ingestFiles(rows);
+  }
+
+  private List<String> readGZIPFile(String file) {
+    List<String> lines = new ArrayList<>();
+    try {
+      GZIPInputStream gzip = new GZIPInputStream(new FileInputStream(file));
+      BufferedReader br = new BufferedReader(new InputStreamReader(gzip, Charset.defaultCharset()));
+
+      String line;
+      while ((line = br.readLine()) != null) {
+        lines.add(line);
+      }
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to read file", e);
+    }
+
+    return lines;
+  }
+}
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/TestUtils.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/TestUtils.java
index 05c80c5..41eac70 100644
--- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/TestUtils.java
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/TestUtils.java
@@ -138,6 +138,10 @@
     return (SnowflakeIO.UserDataMapper<String[]>) recordLine -> recordLine;
   }
 
+  public static SnowflakeIO.UserDataMapper<String> getStringCsvMapper() {
+    return (SnowflakeIO.UserDataMapper<String>) recordLine -> new String[] {recordLine};
+  }
+
   public static class ParseToKv extends DoFn<Long, KV<String, Long>> {
     @ProcessElement
     public void processElement(ProcessContext c) {
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/read/SnowflakeIOReadTest.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/read/SnowflakeIOReadTest.java
index 6016a66..b844760 100644
--- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/read/SnowflakeIOReadTest.java
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/read/SnowflakeIOReadTest.java
@@ -28,13 +28,15 @@
 import org.apache.beam.sdk.io.snowflake.SnowflakeIO;
 import org.apache.beam.sdk.io.snowflake.services.SnowflakeService;
 import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeBasicDataSource;
+import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeBatchServiceImpl;
 import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeDatabase;
-import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeServiceImpl;
+import org.apache.beam.sdk.io.snowflake.test.TestUtils;
 import org.apache.beam.sdk.io.snowflake.test.unit.TestPipelineOptions;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
@@ -46,6 +48,7 @@
 public class SnowflakeIOReadTest implements Serializable {
   public static final String FAKE_TABLE = "FAKE_TABLE";
   public static final String FAKE_QUERY = "SELECT * FROM FAKE_TABLE";
+  public static final String BUCKET_NAME = "BUCKET/";
 
   private static final TestPipelineOptions options =
       TestPipeline.testingPipelineOptions().as(TestPipelineOptions.class);;
@@ -71,19 +74,24 @@
 
     options.setServerName("NULL.snowflakecomputing.com");
     options.setStorageIntegrationName("STORAGE_INTEGRATION");
-    options.setStagingBucketName("BUCKET");
+    options.setStagingBucketName(BUCKET_NAME);
 
     dataSourceConfiguration =
         SnowflakeIO.DataSourceConfiguration.create(new FakeSnowflakeBasicDataSource())
             .withServerName(options.getServerName());
 
-    snowflakeService = new FakeSnowflakeServiceImpl();
+    snowflakeService = new FakeSnowflakeBatchServiceImpl();
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    TestUtils.removeTempDir(BUCKET_NAME);
   }
 
   @Test
   public void testConfigIsMissingStagingBucketName() {
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("withStagingBucketName is required");
+    thrown.expectMessage("withStagingBucketName() is required");
 
     pipeline.apply(
         SnowflakeIO.<GenericRecord>read(snowflakeService)
@@ -99,7 +107,7 @@
   @Test
   public void testConfigIsMissingStorageIntegration() {
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("withStorageIntegrationName is required");
+    thrown.expectMessage("withStorageIntegrationName() is required");
 
     pipeline.apply(
         SnowflakeIO.<GenericRecord>read(snowflakeService)
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/CreateDispositionTest.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/CreateDispositionTest.java
index 4b0f728..2dcd88b 100644
--- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/CreateDispositionTest.java
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/CreateDispositionTest.java
@@ -32,8 +32,8 @@
 import org.apache.beam.sdk.io.snowflake.enums.CreateDisposition;
 import org.apache.beam.sdk.io.snowflake.services.SnowflakeService;
 import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeBasicDataSource;
+import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeBatchServiceImpl;
 import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeDatabase;
-import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeServiceImpl;
 import org.apache.beam.sdk.io.snowflake.test.TestUtils;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.TestPipeline;
@@ -73,7 +73,7 @@
     stagingBucketName = options.getStagingBucketName();
     storageIntegrationName = options.getStorageIntegrationName();
 
-    snowflakeService = new FakeSnowflakeServiceImpl();
+    snowflakeService = new FakeSnowflakeBatchServiceImpl();
     testData = LongStream.range(0, 100).boxed().collect(Collectors.toList());
 
     dc =
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/QueryDispositionLocationTest.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/QueryDispositionLocationTest.java
index 2825c25..5c06dfe 100644
--- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/QueryDispositionLocationTest.java
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/QueryDispositionLocationTest.java
@@ -29,8 +29,8 @@
 import org.apache.beam.sdk.io.snowflake.enums.WriteDisposition;
 import org.apache.beam.sdk.io.snowflake.services.SnowflakeService;
 import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeBasicDataSource;
+import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeBatchServiceImpl;
 import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeDatabase;
-import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeServiceImpl;
 import org.apache.beam.sdk.io.snowflake.test.TestUtils;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.TestPipeline;
@@ -47,7 +47,7 @@
 @RunWith(JUnit4.class)
 public class QueryDispositionLocationTest {
   private static final String FAKE_TABLE = "FAKE_TABLE";
-  private static final String BUCKET_NAME = "BUCKET";
+  private static final String BUCKET_NAME = "BUCKET/";
 
   @Rule public final transient TestPipeline pipeline = TestPipeline.create();
   @Rule public ExpectedException exceptionRule = ExpectedException.none();
@@ -63,7 +63,7 @@
     PipelineOptionsFactory.register(SnowflakePipelineOptions.class);
     options = TestPipeline.testingPipelineOptions().as(SnowflakePipelineOptions.class);
 
-    snowflakeService = new FakeSnowflakeServiceImpl();
+    snowflakeService = new FakeSnowflakeBatchServiceImpl();
     testData = LongStream.range(0, 100).boxed().collect(Collectors.toList());
   }
 
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/SchemaDispositionTest.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/SchemaDispositionTest.java
index ac0af04..fe8e98e 100644
--- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/SchemaDispositionTest.java
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/SchemaDispositionTest.java
@@ -38,8 +38,8 @@
 import org.apache.beam.sdk.io.snowflake.enums.CreateDisposition;
 import org.apache.beam.sdk.io.snowflake.services.SnowflakeService;
 import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeBasicDataSource;
+import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeBatchServiceImpl;
 import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeDatabase;
-import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeServiceImpl;
 import org.apache.beam.sdk.io.snowflake.test.TestUtils;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.TestPipeline;
@@ -78,7 +78,7 @@
     stagingBucketName = options.getStagingBucketName();
     storageIntegrationName = options.getStorageIntegrationName();
 
-    snowflakeService = new FakeSnowflakeServiceImpl();
+    snowflakeService = new FakeSnowflakeBatchServiceImpl();
 
     dc =
         SnowflakeIO.DataSourceConfiguration.create(new FakeSnowflakeBasicDataSource())
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/SnowflakeIOWriteTest.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/SnowflakeIOWriteTest.java
index e73760b..95798c5 100644
--- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/SnowflakeIOWriteTest.java
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/SnowflakeIOWriteTest.java
@@ -20,6 +20,7 @@
 import static org.junit.Assert.assertTrue;
 
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.stream.Collectors;
 import java.util.stream.LongStream;
@@ -28,8 +29,8 @@
 import org.apache.beam.sdk.io.snowflake.SnowflakePipelineOptions;
 import org.apache.beam.sdk.io.snowflake.services.SnowflakeService;
 import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeBasicDataSource;
+import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeBatchServiceImpl;
 import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeDatabase;
-import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeServiceImpl;
 import org.apache.beam.sdk.io.snowflake.test.TestUtils;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.TestPipeline;
@@ -48,7 +49,7 @@
 @RunWith(JUnit4.class)
 public class SnowflakeIOWriteTest {
   private static final String FAKE_TABLE = "FAKE_TABLE";
-  private static final String BUCKET_NAME = "BUCKET";
+  private static final String BUCKET_NAME = "BUCKET/";
 
   @Rule public final transient TestPipeline pipeline = TestPipeline.create();
 
@@ -59,11 +60,21 @@
 
   private static SnowflakeService snowflakeService;
   private static List<Long> testData;
+  private static List<String> testDataInStrings;
 
   @BeforeClass
   public static void setupAll() {
-    snowflakeService = new FakeSnowflakeServiceImpl();
+    snowflakeService = new FakeSnowflakeBatchServiceImpl();
     testData = LongStream.range(0, 100).boxed().collect(Collectors.toList());
+
+    testDataInStrings = new ArrayList<>();
+    testDataInStrings.add("First row");
+    testDataInStrings.add("Second row with 'single' quotation");
+    testDataInStrings.add("Second row with single one ' quotation");
+    testDataInStrings.add("Second row with single twice '' quotation");
+    testDataInStrings.add("Third row with \"double\" quotation");
+    testDataInStrings.add("Third row with double one \" quotation");
+    testDataInStrings.add("Third row with double twice \"\" quotation");
   }
 
   @Before
@@ -147,7 +158,7 @@
 
     List<String> actualData = FakeSnowflakeDatabase.getElements(FAKE_TABLE);
     List<String> testDataInStrings =
-        testData.stream().map(e -> e.toString()).collect(Collectors.toList());
+        testData.stream().map(Object::toString).collect(Collectors.toList());
     assertTrue(TestUtils.areListsEqual(testDataInStrings, actualData));
   }
 
@@ -174,4 +185,55 @@
 
     assertTrue(TestUtils.areListsEqual(testData, actualData));
   }
+
+  @Test
+  public void writeToExternalWithDoubleQuotation() throws SnowflakeSQLException {
+
+    pipeline
+        .apply(Create.of(testDataInStrings))
+        .apply(
+            "Write SnowflakeIO",
+            SnowflakeIO.<String>write()
+                .withDataSourceConfiguration(dc)
+                .withUserDataMapper(TestUtils.getStringCsvMapper())
+                .to(FAKE_TABLE)
+                .withStagingBucketName(options.getStagingBucketName())
+                .withStorageIntegrationName(options.getStorageIntegrationName())
+                .withSnowflakeService(snowflakeService)
+                .withQuotationMark("\""));
+
+    pipeline.run(options).waitUntilFinish();
+
+    List<String> actualData = FakeSnowflakeDatabase.getElements(FAKE_TABLE);
+    List<String> escapedTestData =
+        testDataInStrings.stream()
+            .map(e -> e.replace("'", "''"))
+            .map(e -> String.format("\"%s\"", e))
+            .collect(Collectors.toList());
+    assertTrue(TestUtils.areListsEqual(escapedTestData, actualData));
+  }
+
+  @Test
+  public void writeToExternalWithBlankQuotation() throws SnowflakeSQLException {
+    pipeline
+        .apply(Create.of(testDataInStrings))
+        .apply(
+            "Write SnowflakeIO",
+            SnowflakeIO.<String>write()
+                .withDataSourceConfiguration(dc)
+                .withUserDataMapper(TestUtils.getStringCsvMapper())
+                .to(FAKE_TABLE)
+                .withStagingBucketName(options.getStagingBucketName())
+                .withStorageIntegrationName(options.getStorageIntegrationName())
+                .withSnowflakeService(snowflakeService)
+                .withQuotationMark(""));
+
+    pipeline.run(options).waitUntilFinish();
+
+    List<String> actualData = FakeSnowflakeDatabase.getElements(FAKE_TABLE);
+
+    List<String> escapedTestData =
+        testDataInStrings.stream().map(e -> e.replace("'", "''")).collect(Collectors.toList());
+    assertTrue(TestUtils.areListsEqual(escapedTestData, actualData));
+  }
 }
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/StreamingWriteTest.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/StreamingWriteTest.java
new file mode 100644
index 0000000..3f8ffe5
--- /dev/null
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/StreamingWriteTest.java
@@ -0,0 +1,321 @@
+/*
+ * 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.snowflake.test.unit.write;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+
+import java.io.IOException;
+import java.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+import net.snowflake.client.jdbc.SnowflakeSQLException;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.snowflake.SnowflakeIO;
+import org.apache.beam.sdk.io.snowflake.SnowflakePipelineOptions;
+import org.apache.beam.sdk.io.snowflake.credentials.SnowflakeCredentialsFactory;
+import org.apache.beam.sdk.io.snowflake.services.SnowflakeService;
+import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeBasicDataSource;
+import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeDatabase;
+import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeStreamingServiceImpl;
+import org.apache.beam.sdk.io.snowflake.test.TestUtils;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestStream;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.hamcrest.MatcherAssert;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@RunWith(JUnit4.class)
+public class StreamingWriteTest {
+  private static final Logger LOG = LoggerFactory.getLogger(StreamingWriteTest.class);
+  private static final String FAKE_TABLE = "TEST_TABLE";
+  private static final String STAGING_BUCKET_NAME = "BUCKET/";
+  private static final String STORAGE_INTEGRATION_NAME = "STORAGE_INTEGRATION";
+  private static final String SNOW_PIPE = "Snowpipe";
+  private static final Instant START_TIME = new Instant(0);
+
+  @Rule public final transient TestPipeline pipeline = TestPipeline.create();
+
+  @Rule public ExpectedException exceptionRule = ExpectedException.none();
+  private static SnowflakeIO.DataSourceConfiguration dataSourceConfiguration;
+  private static SnowflakeService snowflakeService;
+  private static SnowflakePipelineOptions options;
+  private static List<Long> testData;
+
+  private static final List<String> SENTENCES =
+      Arrays.asList(
+          "Snowflake window 1 1",
+          "Snowflake window 1 2",
+          "Snowflake window 1 3",
+          "Snowflake window 1 4",
+          "Snowflake window 2 1",
+          "Snowflake window 2 2");
+
+  private static final List<String> FIRST_WIN_WORDS = SENTENCES.subList(0, 4);
+  private static final List<String> SECOND_WIN_WORDS = SENTENCES.subList(4, 6);
+  private static final Duration WINDOW_DURATION = Duration.standardMinutes(1);
+
+  @BeforeClass
+  public static void setup() {
+    snowflakeService = new FakeSnowflakeStreamingServiceImpl();
+
+    PipelineOptionsFactory.register(SnowflakePipelineOptions.class);
+    options = TestPipeline.testingPipelineOptions().as(SnowflakePipelineOptions.class);
+    options.setUsername("username");
+
+    options.setServerName("NULL.snowflakecomputing.com");
+
+    testData = LongStream.range(0, 100).boxed().collect(Collectors.toList());
+
+    FakeSnowflakeDatabase.createTable(FAKE_TABLE);
+    dataSourceConfiguration =
+        SnowflakeIO.DataSourceConfiguration.create(new FakeSnowflakeBasicDataSource())
+            .withServerName(options.getServerName())
+            .withoutValidation()
+            .withSchema("PUBLIC")
+            .withDatabase("DATABASE")
+            .withWarehouse("WAREHOUSE");
+  }
+
+  @After
+  public void tearDown() {
+    TestUtils.removeTempDir(STAGING_BUCKET_NAME);
+  }
+
+  @Test
+  public void streamWriteWithOAuthFails() {
+    options.setOauthToken("token");
+    dataSourceConfiguration =
+        SnowflakeIO.DataSourceConfiguration.create(SnowflakeCredentialsFactory.of(options))
+            .withoutValidation()
+            .withServerName(options.getServerName())
+            .withSchema("PUBLIC")
+            .withDatabase("DATABASE")
+            .withWarehouse("WAREHOUSE");
+
+    exceptionRule.expectMessage("KeyPair is required for authentication");
+
+    pipeline
+        .apply(Create.of(testData))
+        .apply(
+            SnowflakeIO.<Long>write()
+                .withDataSourceConfiguration(dataSourceConfiguration)
+                .to(FAKE_TABLE)
+                .withStagingBucketName(STAGING_BUCKET_NAME)
+                .withStorageIntegrationName(STORAGE_INTEGRATION_NAME)
+                .withSnowPipe(SNOW_PIPE)
+                .withUserDataMapper(TestUtils.getLongCsvMapper())
+                .withSnowflakeService(snowflakeService));
+
+    pipeline.run(options);
+  }
+
+  @Test
+  public void streamWriteWithUserPasswordFails() {
+    options.setPassword("password");
+    dataSourceConfiguration =
+        SnowflakeIO.DataSourceConfiguration.create(SnowflakeCredentialsFactory.of(options))
+            .withoutValidation()
+            .withServerName(options.getServerName())
+            .withSchema("PUBLIC")
+            .withDatabase("DATABASE")
+            .withWarehouse("WAREHOUSE");
+
+    exceptionRule.expectMessage("KeyPair is required for authentication");
+
+    pipeline
+        .apply(Create.of(testData))
+        .apply(
+            SnowflakeIO.<Long>write()
+                .withDataSourceConfiguration(dataSourceConfiguration)
+                .to(FAKE_TABLE)
+                .withStagingBucketName(STAGING_BUCKET_NAME)
+                .withStorageIntegrationName(STORAGE_INTEGRATION_NAME)
+                .withSnowPipe(SNOW_PIPE)
+                .withUserDataMapper(TestUtils.getLongCsvMapper())
+                .withSnowflakeService(snowflakeService));
+
+    pipeline.run(options);
+  }
+
+  @Test
+  public void streamWriteWithKey() throws SnowflakeSQLException {
+    String quotationMark = "'";
+    options.setPrivateKeyPath(TestUtils.getPrivateKeyPath(getClass()));
+    options.setPrivateKeyPassphrase(TestUtils.getPrivateKeyPassphrase());
+
+    TestStream<String> stringsStream =
+        TestStream.create(StringUtf8Coder.of())
+            .advanceWatermarkTo(START_TIME)
+            .addElements(event(FIRST_WIN_WORDS.get(0), 2L))
+            .advanceWatermarkTo(START_TIME.plus(Duration.standardSeconds(27L)))
+            .addElements(
+                event(FIRST_WIN_WORDS.get(1), 25L),
+                event(FIRST_WIN_WORDS.get(2), 18L),
+                event(FIRST_WIN_WORDS.get(3), 26L))
+            .advanceWatermarkTo(START_TIME.plus(Duration.standardSeconds(65L)))
+            // This are late elements after window ends so they should not be saved
+            .addElements(event(SECOND_WIN_WORDS.get(0), 67L), event(SECOND_WIN_WORDS.get(1), 68L))
+            .advanceWatermarkToInfinity();
+
+    dataSourceConfiguration =
+        SnowflakeIO.DataSourceConfiguration.create(SnowflakeCredentialsFactory.of(options))
+            .withServerName(options.getServerName())
+            .withoutValidation()
+            .withSchema("PUBLIC")
+            .withDatabase("DATABASE")
+            .withWarehouse("WAREHOUSE");
+
+    pipeline
+        .apply(stringsStream)
+        .apply(Window.into(FixedWindows.of(WINDOW_DURATION)))
+        .apply(
+            SnowflakeIO.<String>write()
+                .withDataSourceConfiguration(dataSourceConfiguration)
+                .withStagingBucketName(STAGING_BUCKET_NAME)
+                .withStorageIntegrationName(STORAGE_INTEGRATION_NAME)
+                .withSnowPipe(SNOW_PIPE)
+                .withFlushRowLimit(4)
+                .withFlushTimeLimit(WINDOW_DURATION)
+                .withUserDataMapper(TestUtils.getStringCsvMapper())
+                .withSnowflakeService(snowflakeService));
+
+    pipeline.run(options).waitUntilFinish();
+
+    List<String> actualDataFirstWin =
+        parseResults(FakeSnowflakeDatabase.getElements(String.format(FAKE_TABLE)), quotationMark);
+
+    Map<String, List<String>> mapOfResults = getMapOfFilesAndResults();
+
+    String firstFileKey = "0";
+    List<String> filesResult = parseResults(mapOfResults.get(firstFileKey), quotationMark);
+
+    int amountOfCreatedFiles = 2;
+    MatcherAssert.assertThat(mapOfResults.size(), equalTo(amountOfCreatedFiles));
+    MatcherAssert.assertThat(filesResult, equalTo(FIRST_WIN_WORDS));
+    MatcherAssert.assertThat(actualDataFirstWin, equalTo(SENTENCES));
+  }
+
+  @Test
+  public void streamWriteWithDoubleQuotation() throws SnowflakeSQLException {
+    String quotationMark = "\"";
+    options.setPrivateKeyPath(TestUtils.getPrivateKeyPath(getClass()));
+    options.setPrivateKeyPassphrase(TestUtils.getPrivateKeyPassphrase());
+
+    TestStream<String> stringsStream =
+        TestStream.create(StringUtf8Coder.of())
+            .advanceWatermarkTo(START_TIME)
+            .addElements(event(FIRST_WIN_WORDS.get(0), 2L))
+            .advanceWatermarkTo(START_TIME.plus(Duration.standardSeconds(27L)))
+            .addElements(
+                event(FIRST_WIN_WORDS.get(1), 25L),
+                event(FIRST_WIN_WORDS.get(2), 18L),
+                event(FIRST_WIN_WORDS.get(3), 26L))
+            .advanceWatermarkTo(START_TIME.plus(Duration.standardSeconds(65L)))
+            // This are late elements after window ends so they should not be saved
+            .addElements(event(SECOND_WIN_WORDS.get(0), 67L), event(SECOND_WIN_WORDS.get(1), 68L))
+            .advanceWatermarkToInfinity();
+
+    dataSourceConfiguration =
+        SnowflakeIO.DataSourceConfiguration.create(SnowflakeCredentialsFactory.of(options))
+            .withServerName(options.getServerName())
+            .withoutValidation()
+            .withSchema("PUBLIC")
+            .withDatabase("DATABASE")
+            .withWarehouse("WAREHOUSE");
+
+    pipeline
+        .apply(stringsStream)
+        .apply(Window.into(FixedWindows.of(WINDOW_DURATION)))
+        .apply(
+            SnowflakeIO.<String>write()
+                .withDataSourceConfiguration(dataSourceConfiguration)
+                .withStagingBucketName(STAGING_BUCKET_NAME)
+                .withStorageIntegrationName(STORAGE_INTEGRATION_NAME)
+                .withSnowPipe(SNOW_PIPE)
+                .withFlushRowLimit(4)
+                .withQuotationMark(quotationMark)
+                .withFlushTimeLimit(WINDOW_DURATION)
+                .withUserDataMapper(TestUtils.getStringCsvMapper())
+                .withSnowflakeService(snowflakeService));
+
+    pipeline.run(options).waitUntilFinish();
+
+    List<String> actualDataFirstWin =
+        parseResults(FakeSnowflakeDatabase.getElements(String.format(FAKE_TABLE)), quotationMark);
+
+    Map<String, List<String>> mapOfResults = getMapOfFilesAndResults();
+
+    String firstFileKey = "0";
+    List<String> filesResult = parseResults(mapOfResults.get(firstFileKey), quotationMark);
+
+    int amountOfCreatedFiles = 2;
+    MatcherAssert.assertThat(mapOfResults.size(), equalTo(amountOfCreatedFiles));
+    MatcherAssert.assertThat(filesResult, equalTo(FIRST_WIN_WORDS));
+    MatcherAssert.assertThat(actualDataFirstWin, equalTo(SENTENCES));
+  }
+
+  private List<String> parseResults(List<String> resultsList, String quotationMark) {
+    return resultsList.stream()
+        .map(s -> s.replaceAll(quotationMark, ""))
+        .collect(Collectors.toList());
+  }
+
+  private Map<String, List<String>> getMapOfFilesAndResults() {
+    return new HashMap<>(getFiles(Paths.get(STAGING_BUCKET_NAME)));
+  }
+
+  private Map<String, List<String>> getFiles(Path file) {
+    Map<String, List<String>> fileNames = new HashMap<>();
+    try (DirectoryStream<Path> paths = Files.newDirectoryStream(file, "*.gz")) {
+
+      paths.forEach(
+          path -> {
+            String key = path.getFileName().toString().split("-", -1)[1];
+            fileNames.put(key, TestUtils.readGZIPFile(path.toString()));
+          });
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to retrieve files", e);
+    }
+    return fileNames;
+  }
+
+  private TimestampedValue<String> event(String word, Long timestamp) {
+    return TimestampedValue.of(word, START_TIME.plus(new Duration(timestamp)));
+  }
+}
diff --git a/sdks/python/apache_beam/coders/coder_impl.py b/sdks/python/apache_beam/coders/coder_impl.py
index 8fdc4a4..54c63e7 100644
--- a/sdks/python/apache_beam/coders/coder_impl.py
+++ b/sdks/python/apache_beam/coders/coder_impl.py
@@ -530,6 +530,92 @@
     return 1
 
 
+class MapCoderImpl(StreamCoderImpl):
+  """For internal use only; no backwards-compatibility guarantees.
+
+  Note this implementation always uses nested context when encoding keys
+  and values. This differs from Java's MapCoder, which uses
+  nested=False if possible for the last value encoded.
+
+  This difference is acceptable because MapCoder is not standard. It is only
+  used in a standard context by RowCoder which always uses nested context for
+  attribute values.
+
+  A coder for typing.Mapping objects."""
+  def __init__(
+      self,
+      key_coder,  # type: CoderImpl
+      value_coder  # type: CoderImpl
+  ):
+    self._key_coder = key_coder
+    self._value_coder = value_coder
+
+  def encode_to_stream(self, dict_value, out, nested):
+    out.write_bigendian_int32(len(dict_value))
+    for key, value in dict_value.items():
+      # Note this implementation always uses nested context when encoding keys
+      # and values which differs from Java. See note in docstring.
+      self._key_coder.encode_to_stream(key, out, True)
+      self._value_coder.encode_to_stream(value, out, True)
+
+  def decode_from_stream(self, in_stream, nested):
+    size = in_stream.read_bigendian_int32()
+    result = {}
+    for _ in range(size):
+      # Note this implementation always uses nested context when encoding keys
+      # and values which differs from Java. See note in docstring.
+      key = self._key_coder.decode_from_stream(in_stream, True)
+      value = self._value_coder.decode_from_stream(in_stream, True)
+      result[key] = value
+
+    return result
+
+  def estimate_size(self, unused_value, nested=False):
+    estimate = 4  # 4 bytes for int32 size prefix
+    for key, value in unused_value.items():
+      estimate += self._key_coder.estimate_size(key, True)
+      estimate += self._value_coder.estimate_size(value, True)
+    return estimate
+
+
+class NullableCoderImpl(StreamCoderImpl):
+  """For internal use only; no backwards-compatibility guarantees.
+
+  A coder for typing.Optional objects."""
+
+  ENCODE_NULL = 0
+  ENCODE_PRESENT = 1
+
+  def __init__(
+      self,
+      value_coder  # type: CoderImpl
+  ):
+    self._value_coder = value_coder
+
+  def encode_to_stream(self, value, out, nested):
+    if value is None:
+      out.write_byte(self.ENCODE_NULL)
+    else:
+      out.write_byte(self.ENCODE_PRESENT)
+      self._value_coder.encode_to_stream(value, out, nested)
+
+  def decode_from_stream(self, in_stream, nested):
+    null_indicator = in_stream.read_byte()
+    if null_indicator == self.ENCODE_NULL:
+      return None
+    elif null_indicator == self.ENCODE_PRESENT:
+      return self._value_coder.decode_from_stream(in_stream, nested)
+    else:
+      raise ValueError(
+          "Encountered unexpected value for null indicator: '%s'" %
+          null_indicator)
+
+  def estimate_size(self, unused_value, nested=False):
+    return 1 + (
+        self._value_coder.estimate_size(unused_value)
+        if unused_value is not None else 0)
+
+
 class FloatCoderImpl(StreamCoderImpl):
   """For internal use only; no backwards-compatibility guarantees."""
   def encode_to_stream(self, value, out, nested):
diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py
index 399a46d..b6aca0a 100644
--- a/sdks/python/apache_beam/coders/coders.py
+++ b/sdks/python/apache_beam/coders/coders.py
@@ -83,6 +83,8 @@
     'FastPrimitivesCoder',
     'FloatCoder',
     'IterableCoder',
+    'MapCoder',
+    'NullableCoder',
     'PickleCoder',
     'ProtoCoder',
     'SingletonCoder',
@@ -520,6 +522,57 @@
 Coder.register_structured_urn(common_urns.coders.BOOL.urn, BooleanCoder)
 
 
+class MapCoder(FastCoder):
+  def __init__(self, key_coder, value_coder):
+    # type: (Coder, Coder) -> None
+    self._key_coder = key_coder
+    self._value_coder = value_coder
+
+  def _create_impl(self):
+    return coder_impl.MapCoderImpl(
+        self._key_coder.get_impl(), self._value_coder.get_impl())
+
+  def to_type_hint(self):
+    return typehints.Dict[self._key_coder.to_type_hint(),
+                          self._value_coder.to_type_hint()]
+
+  def is_deterministic(self):
+    # () -> bool
+    # Map ordering is non-deterministic
+    return False
+
+  def __eq__(self, other):
+    return (
+        type(self) == type(other) and self._key_coder == other._key_coder and
+        self._value_coder == other._value_coder)
+
+  def __hash__(self):
+    return hash(type(self)) + hash(self._key_coder) + hash(self._value_coder)
+
+
+class NullableCoder(FastCoder):
+  def __init__(self, value_coder):
+    # type: (Coder) -> None
+    self._value_coder = value_coder
+
+  def _create_impl(self):
+    return coder_impl.NullableCoderImpl(self._value_coder.get_impl())
+
+  def to_type_hint(self):
+    return typehints.Optional[self._value_coder.to_type_hint()]
+
+  def is_deterministic(self):
+    # () -> bool
+    return self._value_coder.is_deterministic()
+
+  def __eq__(self, other):
+    return (
+        type(self) == type(other) and self._value_coder == other._value_coder)
+
+  def __hash__(self):
+    return hash(type(self)) + hash(self._value_coder)
+
+
 class VarIntCoder(FastCoder):
   """Variable-length integer coder."""
   def _create_impl(self):
diff --git a/sdks/python/apache_beam/coders/coders_test_common.py b/sdks/python/apache_beam/coders/coders_test_common.py
index f1a771a..e1ce23b 100644
--- a/sdks/python/apache_beam/coders/coders_test_common.py
+++ b/sdks/python/apache_beam/coders/coders_test_common.py
@@ -82,8 +82,8 @@
         coders.ToBytesCoder
     ])
     cls.seen_nested -= set([coders.ProtoCoder, CustomCoder])
-    assert not standard - cls.seen
-    assert not cls.seen_nested - standard
+    assert not standard - cls.seen, str(standard - cls.seen)
+    assert not cls.seen_nested - standard, str(cls.seen_nested - standard)
 
   @classmethod
   def _observe(cls, coder):
@@ -560,6 +560,16 @@
         context=context,
         test_size_estimation=False)
 
+  def test_nullable_coder(self):
+    self.check_coder(coders.NullableCoder(coders.VarIntCoder()), None, 2 * 64)
+
+  def test_map_coder(self):
+    self.check_coder(
+        coders.MapCoder(coders.VarIntCoder(), coders.StrUtf8Coder()), {
+            1: "one", 300: "three hundred"
+        }, {}, {i: str(i)
+                for i in range(5000)})
+
 
 if __name__ == '__main__':
   logging.getLogger().setLevel(logging.INFO)
diff --git a/sdks/python/apache_beam/coders/row_coder.py b/sdks/python/apache_beam/coders/row_coder.py
index 3ad880f..02c7f06 100644
--- a/sdks/python/apache_beam/coders/row_coder.py
+++ b/sdks/python/apache_beam/coders/row_coder.py
@@ -30,6 +30,8 @@
 from apache_beam.coders.coders import FastCoder
 from apache_beam.coders.coders import FloatCoder
 from apache_beam.coders.coders import IterableCoder
+from apache_beam.coders.coders import MapCoder
+from apache_beam.coders.coders import NullableCoder
 from apache_beam.coders.coders import StrUtf8Coder
 from apache_beam.coders.coders import TupleCoder
 from apache_beam.coders.coders import VarIntCoder
@@ -58,8 +60,9 @@
         to encode/decode.
     """
     self.schema = schema
+    # Use non-null coders because null values are represented separately
     self.components = [
-        RowCoder.coder_from_type(field.type) for field in self.schema.fields
+        _nonnull_coder_from_type(field.type) for field in self.schema.fields
     ]
 
   def _create_impl(self):
@@ -102,32 +105,6 @@
     # type: (bytes) -> RowCoder
     return RowCoder(proto_utils.parse_Bytes(payload, schema_pb2.Schema))
 
-  @staticmethod
-  def coder_from_type(field_type):
-    type_info = field_type.WhichOneof("type_info")
-    if type_info == "atomic_type":
-      if field_type.atomic_type in (schema_pb2.INT32, schema_pb2.INT64):
-        return VarIntCoder()
-      elif field_type.atomic_type == schema_pb2.DOUBLE:
-        return FloatCoder()
-      elif field_type.atomic_type == schema_pb2.STRING:
-        return StrUtf8Coder()
-      elif field_type.atomic_type == schema_pb2.BOOLEAN:
-        return BooleanCoder()
-      elif field_type.atomic_type == schema_pb2.BYTES:
-        return BytesCoder()
-    elif type_info == "array_type":
-      return IterableCoder(
-          RowCoder.coder_from_type(field_type.array_type.element_type))
-    elif type_info == "row_type":
-      return RowCoder(field_type.row_type.schema)
-
-    # The Java SDK supports several more types, but the coders are not yet
-    # standard, and are not implemented in Python.
-    raise ValueError(
-        "Encountered a type that is not currently supported by RowCoder: %s" %
-        field_type)
-
   def __reduce__(self):
     # when pickling, use bytes representation of the schema. schema_pb2.Schema
     # objects cannot be pickled.
@@ -137,6 +114,43 @@
 typecoders.registry.register_coder(row_type.RowTypeConstraint, RowCoder)
 
 
+def _coder_from_type(field_type):
+  coder = _nonnull_coder_from_type(field_type)
+  if field_type.nullable:
+    return NullableCoder(coder)
+  else:
+    return coder
+
+
+def _nonnull_coder_from_type(field_type):
+  type_info = field_type.WhichOneof("type_info")
+  if type_info == "atomic_type":
+    if field_type.atomic_type in (schema_pb2.INT32, schema_pb2.INT64):
+      return VarIntCoder()
+    elif field_type.atomic_type == schema_pb2.DOUBLE:
+      return FloatCoder()
+    elif field_type.atomic_type == schema_pb2.STRING:
+      return StrUtf8Coder()
+    elif field_type.atomic_type == schema_pb2.BOOLEAN:
+      return BooleanCoder()
+    elif field_type.atomic_type == schema_pb2.BYTES:
+      return BytesCoder()
+  elif type_info == "array_type":
+    return IterableCoder(_coder_from_type(field_type.array_type.element_type))
+  elif type_info == "map_type":
+    return MapCoder(
+        _coder_from_type(field_type.map_type.key_type),
+        _coder_from_type(field_type.map_type.value_type))
+  elif type_info == "row_type":
+    return RowCoder(field_type.row_type.schema)
+
+  # The Java SDK supports several more types, but the coders are not yet
+  # standard, and are not implemented in Python.
+  raise ValueError(
+      "Encountered a type that is not currently supported by RowCoder: %s" %
+      field_type)
+
+
 class RowCoderImpl(StreamCoderImpl):
   """For internal use only; no backwards-compatibility guarantees."""
   SIZE_CODER = VarIntCoder().get_impl()
diff --git a/sdks/python/apache_beam/coders/row_coder_test.py b/sdks/python/apache_beam/coders/row_coder_test.py
index 65b1024..4277e58 100644
--- a/sdks/python/apache_beam/coders/row_coder_test.py
+++ b/sdks/python/apache_beam/coders/row_coder_test.py
@@ -45,14 +45,15 @@
         ("aliases", typing.List[unicode]),
         ("knows_javascript", bool),
         # TODO(BEAM-7372): Use bytes instead of ByteString
-        ("payload", typing.Optional[typing.ByteString])
+        ("payload", typing.Optional[typing.ByteString]),
+        ("custom_metadata", typing.Mapping[unicode, int])
     ])
 
 coders_registry.register_coder(Person, RowCoder)
 
 
 class RowCoderTest(unittest.TestCase):
-  JON_SNOW = Person("Jon Snow", 23, None, ["crow", "wildling"], False, None)
+  JON_SNOW = Person("Jon Snow", 23, None, ["crow", "wildling"], False, None, {})
   PEOPLE = [
       JON_SNOW,
       Person(
@@ -60,8 +61,9 @@
           25,
           "Westeros", ["Mother of Dragons"],
           False,
-          None),
-      Person("Michael Bluth", 30, None, [], True, b"I've made a huge mistake")
+          None, {"dragons": 3}),
+      Person(
+          "Michael Bluth", 30, None, [], True, b"I've made a huge mistake", {})
   ]
 
   def test_create_row_coder_from_named_tuple(self):
@@ -102,6 +104,15 @@
                 name="payload",
                 type=schema_pb2.FieldType(
                     atomic_type=schema_pb2.BYTES, nullable=True)),
+            schema_pb2.Field(
+                name="custom_metadata",
+                type=schema_pb2.FieldType(
+                    map_type=schema_pb2.MapType(
+                        key_type=schema_pb2.FieldType(
+                            atomic_type=schema_pb2.STRING),
+                        value_type=schema_pb2.FieldType(
+                            atomic_type=schema_pb2.INT64),
+                    ))),
         ])
     coder = RowCoder(schema)
 
diff --git a/sdks/python/apache_beam/coders/standard_coders_test.py b/sdks/python/apache_beam/coders/standard_coders_test.py
index 4a13895..df88978 100644
--- a/sdks/python/apache_beam/coders/standard_coders_test.py
+++ b/sdks/python/apache_beam/coders/standard_coders_test.py
@@ -29,6 +29,7 @@
 import sys
 import unittest
 from builtins import map
+from copy import deepcopy
 from typing import Dict
 from typing import Tuple
 
@@ -78,6 +79,13 @@
 
 def value_parser_from_schema(schema):
   def attribute_parser_from_type(type_):
+    parser = nonnull_attribute_parser_from_type(type_)
+    if type_.nullable:
+      return lambda x: None if x is None else parser(x)
+    else:
+      return parser
+
+  def nonnull_attribute_parser_from_type(type_):
     # TODO: This should be exhaustive
     type_info = type_.WhichOneof("type_info")
     if type_info == "atomic_type":
@@ -89,8 +97,8 @@
       element_parser = attribute_parser_from_type(type_.array_type.element_type)
       return lambda x: list(map(element_parser, x))
     elif type_info == "map_type":
-      key_parser = attribute_parser_from_type(type_.array_type.key_type)
-      value_parser = attribute_parser_from_type(type_.array_type.value_type)
+      key_parser = attribute_parser_from_type(type_.map_type.key_type)
+      value_parser = attribute_parser_from_type(type_.map_type.value_type)
       return lambda x: dict(
           (key_parser(k), value_parser(v)) for k, v in x.items())
 
@@ -101,6 +109,7 @@
 
   def value_parser(x):
     result = []
+    x = deepcopy(x)
     for name, parser in parsers:
       value = x.pop(name)
       result.append(None if value is None else parser(value))
diff --git a/sdks/python/apache_beam/dataframe/frame_base.py b/sdks/python/apache_beam/dataframe/frame_base.py
index 7780cd3..0b1c296 100644
--- a/sdks/python/apache_beam/dataframe/frame_base.py
+++ b/sdks/python/apache_beam/dataframe/frame_base.py
@@ -16,7 +16,9 @@
 
 from __future__ import absolute_import
 
+import functools
 import inspect
+import sys
 from typing import Any
 from typing import Callable
 from typing import Dict
@@ -29,6 +31,18 @@
 from apache_beam.dataframe import expressions
 from apache_beam.dataframe import partitionings
 
+# pylint: disable=deprecated-method
+if sys.version_info < (3, ):
+  _getargspec = inspect.getargspec
+
+  def _unwrap(func):
+    while hasattr(func, '__wrapped__'):
+      func = func.__wrapped__
+    return func
+else:
+  _getargspec = inspect.getfullargspec
+  _unwrap = inspect.unwrap
+
 
 class DeferredBase(object):
 
@@ -146,8 +160,7 @@
         value = kwargs[key]
       else:
         try:
-          # pylint: disable=deprecated-method
-          ix = inspect.getargspec(func).args.index(key)
+          ix = _getargspec(func).args.index(key)
         except ValueError:
           # TODO: fix for delegation?
           continue
@@ -226,6 +239,68 @@
   return wrapper
 
 
+def maybe_inplace(func):
+  @functools.wraps(func)
+  def wrapper(self, inplace=False, **kwargs):
+    result = func(self, **kwargs)
+    if inplace:
+      self._expr = result._expr
+    else:
+      return result
+
+  return wrapper
+
+
+def args_to_kwargs(base_type):
+  def wrap(func):
+    arg_names = _getargspec(_unwrap(getattr(base_type, func.__name__))).args
+
+    @functools.wraps(func)
+    def wrapper(*args, **kwargs):
+      for name, value in zip(arg_names, args):
+        if name in kwargs:
+          raise TypeError(
+              "%s() got multiple values for argument '%s'" %
+              (func.__name__, name))
+        kwargs[name] = value
+      return func(**kwargs)
+
+    return wrapper
+
+  return wrap
+
+
+def populate_defaults(base_type):
+  def wrap(func):
+    base_argspec = _getargspec(_unwrap(getattr(base_type, func.__name__)))
+    if not base_argspec.defaults:
+      return func
+
+    arg_to_default = dict(
+        zip(
+            base_argspec.args[-len(base_argspec.defaults):],
+            base_argspec.defaults))
+
+    unwrapped_func = _unwrap(func)
+    # args that do not have defaults in func, but do have defaults in base
+    func_argspec = _getargspec(unwrapped_func)
+    num_non_defaults = len(func_argspec.args) - len(func_argspec.defaults or ())
+    defaults_to_populate = set(
+        func_argspec.args[:num_non_defaults]).intersection(
+            arg_to_default.keys())
+
+    @functools.wraps(func)
+    def wrapper(**kwargs):
+      for name in defaults_to_populate:
+        if name not in kwargs:
+          kwargs[name] = arg_to_default[name]
+      return func(**kwargs)
+
+    return wrapper
+
+  return wrap
+
+
 class WontImplementError(NotImplementedError):
   """An subclass of NotImplementedError to raise indicating that implementing
   the given method is infeasible.
diff --git a/sdks/python/apache_beam/dataframe/frame_base_test.py b/sdks/python/apache_beam/dataframe/frame_base_test.py
index 392272c..b527da0 100644
--- a/sdks/python/apache_beam/dataframe/frame_base_test.py
+++ b/sdks/python/apache_beam/dataframe/frame_base_test.py
@@ -41,6 +41,59 @@
     self.assertTrue(sub(x, b)._expr.evaluate_at(session).equals(a - b))
     self.assertTrue(sub(a, y)._expr.evaluate_at(session).equals(a - b))
 
+  def test_maybe_inplace(self):
+    @frame_base.maybe_inplace
+    def add_one(frame):
+      return frame + 1
+
+    frames.DeferredSeries.add_one = add_one
+    original_expr = expressions.PlaceholderExpression(pd.Series([1, 2, 3]))
+    x = frames.DeferredSeries(original_expr)
+    x.add_one()
+    self.assertIs(x._expr, original_expr)
+    x.add_one(inplace=False)
+    self.assertIs(x._expr, original_expr)
+    x.add_one(inplace=True)
+    self.assertIsNot(x._expr, original_expr)
+
+  def test_args_to_kwargs(self):
+    class Base(object):
+      def func(self, a=1, b=2, c=3):
+        pass
+
+    class Proxy(object):
+      @frame_base.args_to_kwargs(Base)
+      def func(self, **kwargs):
+        return kwargs
+
+    proxy = Proxy()
+    # pylint: disable=too-many-function-args
+    self.assertEqual(proxy.func(), {})
+    self.assertEqual(proxy.func(100), {'a': 100})
+    self.assertEqual(proxy.func(2, 4, 6), {'a': 2, 'b': 4, 'c': 6})
+    self.assertEqual(proxy.func(2, c=6), {'a': 2, 'c': 6})
+    self.assertEqual(proxy.func(c=6, a=2), {'a': 2, 'c': 6})
+
+  def test_args_to_kwargs_populates_defaults(self):
+    class Base(object):
+      def func(self, a=1, b=2, c=3):
+        pass
+
+    class Proxy(object):
+      @frame_base.args_to_kwargs(Base)
+      @frame_base.populate_defaults(Base)
+      def func(self, a, c=1000, **kwargs):
+        return dict(kwargs, a=a, c=c)
+
+    proxy = Proxy()
+    # pylint: disable=too-many-function-args
+    self.assertEqual(proxy.func(), {'a': 1, 'c': 1000})
+    self.assertEqual(proxy.func(100), {'a': 100, 'c': 1000})
+    self.assertEqual(proxy.func(2, 4, 6), {'a': 2, 'b': 4, 'c': 6})
+    self.assertEqual(proxy.func(2, c=6), {'a': 2, 'c': 6})
+    self.assertEqual(proxy.func(c=6, a=2), {'a': 2, 'c': 6})
+    self.assertEqual(proxy.func(c=6), {'a': 1, 'c': 6})
+
 
 if __name__ == '__main__':
   unittest.main()
diff --git a/sdks/python/apache_beam/dataframe/frames.py b/sdks/python/apache_beam/dataframe/frames.py
index 89e9154..9e2e97a 100644
--- a/sdks/python/apache_beam/dataframe/frames.py
+++ b/sdks/python/apache_beam/dataframe/frames.py
@@ -54,29 +54,20 @@
       'order-sensitive')
   diff = frame_base.wont_implement_method('order-sensitive')
 
-  def replace(
-      self,
-      to_replace=None,
-      value=None,
-      inplace=False,
-      limit=None,
-      *args,
-      **kwargs):
+  @frame_base.args_to_kwargs(pd.Series)
+  @frame_base.populate_defaults(pd.Series)
+  @frame_base.maybe_inplace
+  def replace(self, limit, **kwargs):
     if limit is None:
       requires_partition_by = partitionings.Nothing()
     else:
       requires_partition_by = partitionings.Singleton()
-    result = frame_base.DeferredFrame.wrap(
+    return frame_base.DeferredFrame.wrap(
         expressions.ComputedExpression(
             'replace',
-            lambda df: df.replace(
-                to_replace, value, False, limit, *args, **kwargs), [self._expr],
+            lambda df: df.replace(limit=limit, **kwargs), [self._expr],
             preserves_partition_by=partitionings.Singleton(),
             requires_partition_by=requires_partition_by))
-    if inplace:
-      self._expr = result._expr
-    else:
-      return result
 
   def unstack(self, *args, **kwargs):
     raise frame_base.WontImplementError('non-deferred column values')
@@ -159,14 +150,15 @@
   def loc(self):
     return _DeferredLoc(self)
 
-  def aggregate(self, *args, **kwargs):
-    if 'axis' in kwargs and kwargs['axis'] is None:
-      return self.agg(*args, **dict(kwargs, axis=1)).agg(
-          *args, **dict(kwargs, axis=0))
+  @frame_base.args_to_kwargs(pd.DataFrame)
+  @frame_base.populate_defaults(pd.DataFrame)
+  def aggregate(self, axis, **kwargs):
+    if axis is None:
+      return self.agg(axis=1, **kwargs).agg(axis=0, **kwargs)
     return frame_base.DeferredFrame.wrap(
         expressions.ComputedExpression(
             'aggregate',
-            lambda df: df.agg(*args, **kwargs),
+            lambda df: df.agg(axis=axis, **kwargs),
             [self._expr],
             # TODO(robertwb): Sub-aggregate when possible.
             requires_partition_by=partitionings.Singleton()))
@@ -188,32 +180,22 @@
   min = frame_base._associative_agg_method('min')
   mode = frame_base._agg_method('mode')
 
-  def dropna(
-      self,
-      axis=0,
-      how='any',
-      thresh=None,
-      subset=None,
-      inplace=False,
-      *args,
-      **kwargs):
+  @frame_base.args_to_kwargs(pd.DataFrame)
+  @frame_base.populate_defaults(pd.DataFrame)
+  @frame_base.maybe_inplace
+  def dropna(self, axis, **kwargs):
     # TODO(robertwb): This is a common pattern. Generalize?
     if axis == 1 or axis == 'columns':
       requires_partition_by = partitionings.Singleton()
     else:
       requires_partition_by = partitionings.Nothing()
-    result = frame_base.DeferredFrame.wrap(
+    return frame_base.DeferredFrame.wrap(
         expressions.ComputedExpression(
             'dropna',
-            lambda df: df.dropna(
-                axis, how, thresh, subset, False, *args, **kwargs),
+            lambda df: df.dropna(axis=axis, **kwargs),
             [self._expr],
             preserves_partition_by=partitionings.Singleton(),
             requires_partition_by=requires_partition_by))
-    if inplace:
-      self._expr = result._expr
-    else:
-      return result
 
   items = itertuples = iterrows = iteritems = frame_base.wont_implement_method(
       'non-lazy')
@@ -223,13 +205,15 @@
 
   prod = product = frame_base._associative_agg_method('prod')
 
-  def quantile(self, q=0.5, axis=0, *args, **kwargs):
-    if axis != 0:
+  @frame_base.args_to_kwargs(pd.DataFrame)
+  @frame_base.populate_defaults(pd.DataFrame)
+  def quantile(self, axis, **kwargs):
+    if axis == 1 or axis == 'columns':
       raise frame_base.WontImplementError('non-deferred column values')
     return frame_base.DeferredFrame.wrap(
         expressions.ComputedExpression(
             'quantile',
-            lambda df: df.quantile(q, axis, *args, **kwargs),
+            lambda df: df.quantile(axis=axis, **kwargs),
             [self._expr],
             #TODO(robertwb): Approximate quantiles?
             requires_partition_by=partitionings.Singleton(),
@@ -237,28 +221,26 @@
 
   query = frame_base._elementwise_method('query')
 
-  def replace(self, to_replace=None,
-      value=None,
-      inplace=False,
-      limit=None, *args, **kwargs):
+  @frame_base.args_to_kwargs(pd.DataFrame)
+  @frame_base.populate_defaults(pd.DataFrame)
+  @frame_base.maybe_inplace
+  def replace(self, limit, **kwargs):
     if limit is None:
       requires_partition_by = partitionings.Nothing()
     else:
       requires_partition_by = partitionings.Singleton()
-    result = frame_base.DeferredFrame.wrap(
+    return frame_base.DeferredFrame.wrap(
         expressions.ComputedExpression(
             'replace',
-            lambda df: df.replace(
-                to_replace, value, False, limit, *args, **kwargs),
+            lambda df: df.replace(limit=limit, **kwargs),
             [self._expr],
             preserves_partition_by=partitionings.Singleton(),
             requires_partition_by=requires_partition_by))
-    if inplace:
-      self._expr = result._expr
-    else:
-      return result
 
-  def reset_index(self, level=None, drop=False, inplace=False, *args, **kwargs):
+  @frame_base.args_to_kwargs(pd.DataFrame)
+  @frame_base.populate_defaults(pd.DataFrame)
+  @frame_base.maybe_inplace
+  def reset_index(self, level, **kwargs):
     if level is not None and not isinstance(level, (tuple, list)):
       level = [level]
     if level is None or len(level) == len(self._expr.proxy().index.levels):
@@ -266,22 +248,20 @@
       requires_partition_by = partitionings.Singleton()
     else:
       requires_partition_by = partitionings.Nothing()
-    result = frame_base.DeferredFrame.wrap(
+    return frame_base.DeferredFrame.wrap(
         expressions.ComputedExpression(
             'reset_index',
-            lambda df: df.reset_index(level, drop, False, *args, **kwargs),
+            lambda df: df.reset_index(level=level, **kwargs),
             [self._expr],
             preserves_partition_by=partitionings.Singleton(),
             requires_partition_by=requires_partition_by))
-    if inplace:
-      self._expr = result._expr
-    else:
-      return result
 
   round = frame_base._elementwise_method('round')
   select_dtypes = frame_base._elementwise_method('select_dtypes')
 
-  def shift(self, periods=1, freq=None, axis=0, *args, **kwargs):
+  @frame_base.args_to_kwargs(pd.DataFrame)
+  @frame_base.populate_defaults(pd.DataFrame)
+  def shift(self, axis, **kwargs):
     if axis == 1 or axis == 'columns':
       requires_partition_by = partitionings.Nothing()
     else:
@@ -289,7 +269,7 @@
     return frame_base.DeferredFrame.wrap(
         expressions.ComputedExpression(
             'shift',
-            lambda df: df.shift(periods, freq, axis, *args, **kwargs),
+            lambda df: df.shift(axis=axis, **kwargs),
             [self._expr],
             preserves_partition_by=partitionings.Singleton(),
             requires_partition_by=requires_partition_by))
@@ -298,24 +278,21 @@
   def shape(self):
     raise frame_base.WontImplementError('scalar value')
 
-  def sort_values(
-      self, by, axis=0, ascending=True, inplace=False, *args, **kwargs):
+  @frame_base.args_to_kwargs(pd.DataFrame)
+  @frame_base.populate_defaults(pd.DataFrame)
+  @frame_base.maybe_inplace
+  def sort_values(self, axis, **kwargs):
     if axis == 1 or axis == 'columns':
       requires_partition_by = partitionings.Nothing()
     else:
       requires_partition_by = partitionings.Singleton()
-    result = frame_base.DeferredFrame.wrap(
+    return frame_base.DeferredFrame.wrap(
         expressions.ComputedExpression(
             'sort_values',
-            lambda df: df.sort_values(
-                by, axis, ascending, False, *args, **kwargs),
+            lambda df: df.sort_values(axis=axis, **kwargs),
             [self._expr],
             preserves_partition_by=partitionings.Singleton(),
             requires_partition_by=requires_partition_by))
-    if inplace:
-      self._expr = result._expr
-    else:
-      return result
 
   stack = frame_base._elementwise_method('stack')
 
diff --git a/sdks/python/apache_beam/examples/snippets/snippets_test_py3.py b/sdks/python/apache_beam/examples/snippets/snippets_test_py3.py
index 0f0b668..5eb1d4b 100644
--- a/sdks/python/apache_beam/examples/snippets/snippets_test_py3.py
+++ b/sdks/python/apache_beam/examples/snippets/snippets_test_py3.py
@@ -96,6 +96,18 @@
       ids = numbers | 'to_id' >> beam.Map(my_fn)
       # [END type_hints_map_annotations]
 
+    # Example using an annotated PTransform.
+    with self.assertRaises(typehints.TypeCheckError):
+      # [START type_hints_ptransforms]
+      from apache_beam.pvalue import PCollection
+
+      class IntToStr(beam.PTransform):
+        def expand(self, pcoll: PCollection[int]) -> PCollection[str]:
+          return pcoll | beam.Map(lambda elem: str(elem))
+
+      ids = numbers | 'convert to str' >> IntToStr()
+      # [END type_hints_ptransforms]
+
 
 if __name__ == '__main__':
   logging.getLogger().setLevel(logging.INFO)
diff --git a/sdks/python/apache_beam/examples/wordcount.py b/sdks/python/apache_beam/examples/wordcount.py
index 6732568..aa07802 100644
--- a/sdks/python/apache_beam/examples/wordcount.py
+++ b/sdks/python/apache_beam/examples/wordcount.py
@@ -30,24 +30,12 @@
 import apache_beam as beam
 from apache_beam.io import ReadFromText
 from apache_beam.io import WriteToText
-from apache_beam.metrics import Metrics
-from apache_beam.metrics.metric import MetricsFilter
 from apache_beam.options.pipeline_options import PipelineOptions
 from apache_beam.options.pipeline_options import SetupOptions
 
 
 class WordExtractingDoFn(beam.DoFn):
   """Parse each line of input text into words."""
-  def __init__(self):
-    # TODO(BEAM-6158): Revert the workaround once we can pickle super() on py3.
-    # super(WordExtractingDoFn, self).__init__()
-    beam.DoFn.__init__(self)
-    self.words_counter = Metrics.counter(self.__class__, 'words')
-    self.word_lengths_counter = Metrics.counter(self.__class__, 'word_lengths')
-    self.word_lengths_dist = Metrics.distribution(
-        self.__class__, 'word_len_dist')
-    self.empty_line_counter = Metrics.counter(self.__class__, 'empty_lines')
-
   def process(self, element):
     """Returns an iterator over the words of this element.
 
@@ -59,15 +47,7 @@
     Returns:
       The processed element.
     """
-    text_line = element.strip()
-    if not text_line:
-      self.empty_line_counter.inc(1)
-    words = re.findall(r'[\w\']+', text_line, re.UNICODE)
-    for w in words:
-      self.words_counter.inc()
-      self.word_lengths_counter.inc(len(w))
-      self.word_lengths_dist.update(len(w))
-    return words
+    return re.findall(r'[\w\']+', element, re.UNICODE)
 
 
 def run(argv=None, save_main_session=True):
@@ -89,52 +69,29 @@
   # workflow rely on global context (e.g., a module imported at module level).
   pipeline_options = PipelineOptions(pipeline_args)
   pipeline_options.view_as(SetupOptions).save_main_session = save_main_session
-  p = beam.Pipeline(options=pipeline_options)
 
-  # Read the text file[pattern] into a PCollection.
-  lines = p | 'read' >> ReadFromText(known_args.input)
+  # The pipeline will be run on exiting the with block.
+  with beam.Pipeline(options=pipeline_options) as p:
 
-  # Count the occurrences of each word.
-  def count_ones(word_ones):
-    (word, ones) = word_ones
-    return (word, sum(ones))
+    # Read the text file[pattern] into a PCollection.
+    lines = p | 'Read' >> ReadFromText(known_args.input)
 
-  counts = (
-      lines
-      | 'split' >>
-      (beam.ParDo(WordExtractingDoFn()).with_output_types(unicode))
-      | 'pair_with_one' >> beam.Map(lambda x: (x, 1))
-      | 'group' >> beam.GroupByKey()
-      | 'count' >> beam.Map(count_ones))
+    counts = (
+        lines
+        | 'Split' >>
+        (beam.ParDo(WordExtractingDoFn()).with_output_types(unicode))
+        | 'PairWIthOne' >> beam.Map(lambda x: (x, 1))
+        | 'GroupAndSum' >> beam.CombinePerKey(sum))
 
-  # Format the counts into a PCollection of strings.
-  def format_result(word_count):
-    (word, count) = word_count
-    return '%s: %d' % (word, count)
+    # Format the counts into a PCollection of strings.
+    def format_result(word, count):
+      return '%s: %d' % (word, count)
 
-  output = counts | 'format' >> beam.Map(format_result)
+    output = counts | 'Format' >> beam.MapTuple(format_result)
 
-  # Write the output using a "Write" transform that has side effects.
-  # pylint: disable=expression-not-assigned
-  output | 'write' >> WriteToText(known_args.output)
-
-  result = p.run()
-  result.wait_until_finish()
-
-  # Do not query metrics when creating a template which doesn't run
-  if (not hasattr(result, 'has_job')  # direct runner
-      or result.has_job):  # not just a template creation
-    empty_lines_filter = MetricsFilter().with_name('empty_lines')
-    query_result = result.metrics().query(empty_lines_filter)
-    if query_result['counters']:
-      empty_lines_counter = query_result['counters'][0]
-      logging.info('number of empty lines: %d', empty_lines_counter.result)
-
-    word_lengths_filter = MetricsFilter().with_name('word_len_dist')
-    query_result = result.metrics().query(word_lengths_filter)
-    if query_result['distributions']:
-      word_lengths_dist = query_result['distributions'][0]
-      logging.info('average word length: %d', word_lengths_dist.result.mean)
+    # Write the output using a "Write" transform that has side effects.
+    # pylint: disable=expression-not-assigned
+    output | 'Write' >> WriteToText(known_args.output)
 
 
 if __name__ == '__main__':
diff --git a/sdks/python/apache_beam/examples/wordcount_with_metrics.py b/sdks/python/apache_beam/examples/wordcount_with_metrics.py
new file mode 100644
index 0000000..6732568
--- /dev/null
+++ b/sdks/python/apache_beam/examples/wordcount_with_metrics.py
@@ -0,0 +1,142 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""A word-counting workflow."""
+
+# pytype: skip-file
+
+from __future__ import absolute_import
+
+import argparse
+import logging
+import re
+
+from past.builtins import unicode
+
+import apache_beam as beam
+from apache_beam.io import ReadFromText
+from apache_beam.io import WriteToText
+from apache_beam.metrics import Metrics
+from apache_beam.metrics.metric import MetricsFilter
+from apache_beam.options.pipeline_options import PipelineOptions
+from apache_beam.options.pipeline_options import SetupOptions
+
+
+class WordExtractingDoFn(beam.DoFn):
+  """Parse each line of input text into words."""
+  def __init__(self):
+    # TODO(BEAM-6158): Revert the workaround once we can pickle super() on py3.
+    # super(WordExtractingDoFn, self).__init__()
+    beam.DoFn.__init__(self)
+    self.words_counter = Metrics.counter(self.__class__, 'words')
+    self.word_lengths_counter = Metrics.counter(self.__class__, 'word_lengths')
+    self.word_lengths_dist = Metrics.distribution(
+        self.__class__, 'word_len_dist')
+    self.empty_line_counter = Metrics.counter(self.__class__, 'empty_lines')
+
+  def process(self, element):
+    """Returns an iterator over the words of this element.
+
+    The element is a line of text.  If the line is blank, note that, too.
+
+    Args:
+      element: the element being processed
+
+    Returns:
+      The processed element.
+    """
+    text_line = element.strip()
+    if not text_line:
+      self.empty_line_counter.inc(1)
+    words = re.findall(r'[\w\']+', text_line, re.UNICODE)
+    for w in words:
+      self.words_counter.inc()
+      self.word_lengths_counter.inc(len(w))
+      self.word_lengths_dist.update(len(w))
+    return words
+
+
+def run(argv=None, save_main_session=True):
+  """Main entry point; defines and runs the wordcount pipeline."""
+  parser = argparse.ArgumentParser()
+  parser.add_argument(
+      '--input',
+      dest='input',
+      default='gs://dataflow-samples/shakespeare/kinglear.txt',
+      help='Input file to process.')
+  parser.add_argument(
+      '--output',
+      dest='output',
+      required=True,
+      help='Output file to write results to.')
+  known_args, pipeline_args = parser.parse_known_args(argv)
+
+  # We use the save_main_session option because one or more DoFn's in this
+  # workflow rely on global context (e.g., a module imported at module level).
+  pipeline_options = PipelineOptions(pipeline_args)
+  pipeline_options.view_as(SetupOptions).save_main_session = save_main_session
+  p = beam.Pipeline(options=pipeline_options)
+
+  # Read the text file[pattern] into a PCollection.
+  lines = p | 'read' >> ReadFromText(known_args.input)
+
+  # Count the occurrences of each word.
+  def count_ones(word_ones):
+    (word, ones) = word_ones
+    return (word, sum(ones))
+
+  counts = (
+      lines
+      | 'split' >>
+      (beam.ParDo(WordExtractingDoFn()).with_output_types(unicode))
+      | 'pair_with_one' >> beam.Map(lambda x: (x, 1))
+      | 'group' >> beam.GroupByKey()
+      | 'count' >> beam.Map(count_ones))
+
+  # Format the counts into a PCollection of strings.
+  def format_result(word_count):
+    (word, count) = word_count
+    return '%s: %d' % (word, count)
+
+  output = counts | 'format' >> beam.Map(format_result)
+
+  # Write the output using a "Write" transform that has side effects.
+  # pylint: disable=expression-not-assigned
+  output | 'write' >> WriteToText(known_args.output)
+
+  result = p.run()
+  result.wait_until_finish()
+
+  # Do not query metrics when creating a template which doesn't run
+  if (not hasattr(result, 'has_job')  # direct runner
+      or result.has_job):  # not just a template creation
+    empty_lines_filter = MetricsFilter().with_name('empty_lines')
+    query_result = result.metrics().query(empty_lines_filter)
+    if query_result['counters']:
+      empty_lines_counter = query_result['counters'][0]
+      logging.info('number of empty lines: %d', empty_lines_counter.result)
+
+    word_lengths_filter = MetricsFilter().with_name('word_len_dist')
+    query_result = result.metrics().query(word_lengths_filter)
+    if query_result['distributions']:
+      word_lengths_dist = query_result['distributions'][0]
+      logging.info('average word length: %d', word_lengths_dist.result.mean)
+
+
+if __name__ == '__main__':
+  logging.getLogger().setLevel(logging.INFO)
+  run()
diff --git a/sdks/python/apache_beam/io/external/snowflake.py b/sdks/python/apache_beam/io/external/snowflake.py
new file mode 100644
index 0000000..e7ffa6a
--- /dev/null
+++ b/sdks/python/apache_beam/io/external/snowflake.py
@@ -0,0 +1,204 @@
+#
+# 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.
+#
+
+"""Snowflake transforms tested against Flink portable runner.
+
+  **Setup**
+
+  Transforms provided in this module are cross-language transforms
+  implemented in the Beam Java SDK. During the pipeline construction, Python SDK
+  will connect to a Java expansion service to expand these transforms.
+  To facilitate this, a small amount of setup is needed before using these
+  transforms in a Beam Python pipeline.
+
+  There are several ways to setup cross-language Snowflake transforms.
+
+  * Option 1: use the default expansion service
+  * Option 2: specify a custom expansion service
+
+  See below for details regarding each of these options.
+
+  *Option 1: Use the default expansion service*
+
+  This is the recommended and easiest setup option for using Python Snowflake
+  transforms.This option requires following pre-requisites
+  before running the Beam pipeline.
+
+  * Install Java runtime in the computer from where the pipeline is constructed
+    and make sure that 'java' command is available.
+
+  In this option, Python SDK will either download (for released Beam version) or
+  build (when running from a Beam Git clone) a expansion service jar and use
+  that to expand transforms. Currently Snowflake transforms use the
+  'beam-sdks-java-io-expansion-service' jar for this purpose.
+
+  *Option 2: specify a custom expansion service*
+
+  In this option, you startup your own expansion service and provide that as
+  a parameter when using the transforms provided in this module.
+
+  This option requires following pre-requisites before running the Beam
+  pipeline.
+
+  * Startup your own expansion service.
+  * Update your pipeline to provide the expansion service address when
+    initiating Snowflake transforms provided in this module.
+
+  Flink Users can use the built-in Expansion Service of the Flink Runner's
+  Job Server. If you start Flink's Job Server, the expansion service will be
+  started on port 8097. For a different address, please set the
+  expansion_service parameter.
+
+  **More information**
+
+  For more information regarding cross-language transforms see:
+  - https://beam.apache.org/roadmap/portability/
+
+  For more information specific to Flink runner see:
+  - https://beam.apache.org/documentation/runners/flink/
+"""
+
+# pytype: skip-file
+
+from __future__ import absolute_import
+
+import typing
+
+from past.builtins import unicode
+
+import apache_beam as beam
+from apache_beam.transforms.external import BeamJarExpansionService
+from apache_beam.transforms.external import ExternalTransform
+from apache_beam.transforms.external import NamedTupleBasedPayloadBuilder
+
+ReadFromSnowflakeSchema = typing.NamedTuple(
+    'ReadFromSnowflakeSchema',
+    [
+        ('server_name', unicode),
+        ('schema', unicode),
+        ('database', unicode),
+        ('staging_bucket_name', unicode),
+        ('storage_integration_name', unicode),
+        ('username', typing.Optional[unicode]),
+        ('password', typing.Optional[unicode]),
+        ('private_key_path', typing.Optional[unicode]),
+        ('private_key_passphrase', typing.Optional[unicode]),
+        ('o_auth_token', typing.Optional[unicode]),
+        ('table', typing.Optional[unicode]),
+        ('query', typing.Optional[unicode]),
+    ])
+
+
+def default_io_expansion_service():
+  return BeamJarExpansionService(
+      'sdks:java:io:snowflake:expansion-service:shadowJar')
+
+
+class ReadFromSnowflake(beam.PTransform):
+  """
+    An external PTransform which reads from Snowflake.
+  """
+
+  URN = 'beam:external:java:snowflake:read:v1'
+
+  def __init__(
+      self,
+      server_name,
+      schema,
+      database,
+      staging_bucket_name,
+      storage_integration_name,
+      csv_mapper,
+      username=None,
+      password=None,
+      private_key_path=None,
+      private_key_passphrase=None,
+      o_auth_token=None,
+      table=None,
+      query=None,
+      expansion_service=None):
+    """
+    Initializes a read operation from Snowflake.
+
+    Required parameters:
+
+    :param server_name: full Snowflake server name with the following format
+         account.region.gcp.snowflakecomputing.com.
+    :param schema: name of the Snowflake schema in the database to use.
+    :param database: name of the Snowflake database to use.
+    :param staging_bucket_name: name of the Google Cloud Storage bucket.::
+        Bucket will be used as a temporary location for storing CSV files.
+        Those temporary directories will be named
+        'sf_copy_csv_DATE_TIME_RANDOMSUFFIX'
+        and they will be removed automatically once Read operation finishes.
+    :param storage_integration_name: is the name of storage integration
+        object created according to Snowflake documentation.
+    :param csv_mapper: specifies a function which must translate
+        user-defined object to array of strings.
+        SnowflakeIO uses a COPY INTO <location> statement to move data from
+        a Snowflake table to Google Cloud Storage as CSV files.These files
+        are then downloaded via FileIO and processed line by line.
+        Each line is split into an array of Strings using the OpenCSV
+        The csv_mapper function job is to give the user the possibility to
+        convert the array of Strings to a user-defined type,
+        ie. GenericRecord for Avro or Parquet files, or custom objects.
+        Example:
+        def csv_mapper(strings_array)
+        return User(strings_array[0], int(strings_array[1])))
+    :param table: specifies a Snowflake table name.
+    :param query: specifies a Snowflake custom SQL query.
+    :param expansion_service: specifies URL of expansion service.
+
+    Authentication parameters:
+
+    :param username: specifies username for
+        username/password authentication method.
+    :param password: specifies password for
+        username/password authentication method.
+    :param private_key_path: specifies a private key file for
+        key/ pair authentication method.
+    :param private_key_passphrase: specifies password for
+        key/ pair authentication method.
+    :param o_auth_token: specifies access token for
+        OAuth authentication method.
+    """
+    self.params = ReadFromSnowflakeSchema(
+        server_name=server_name,
+        schema=schema,
+        database=database,
+        staging_bucket_name=staging_bucket_name,
+        storage_integration_name=storage_integration_name,
+        username=username,
+        password=password,
+        private_key_path=private_key_path,
+        private_key_passphrase=private_key_passphrase,
+        o_auth_token=o_auth_token,
+        table=table,
+        query=query)
+    self.csv_mapper = csv_mapper
+    self.expansion_service = expansion_service or default_io_expansion_service()
+
+  def expand(self, pbegin):
+    return (
+        pbegin
+        | ExternalTransform(
+            self.URN,
+            NamedTupleBasedPayloadBuilder(self.params),
+            self.expansion_service,
+        )
+        | 'CSV to array mapper' >> beam.Map(lambda csv: csv.split(b','))
+        | 'CSV mapper' >> beam.Map(self.csv_mapper))
diff --git a/sdks/python/apache_beam/transforms/ptransform.py b/sdks/python/apache_beam/transforms/ptransform.py
index 2ab121e..4a0e80f 100644
--- a/sdks/python/apache_beam/transforms/ptransform.py
+++ b/sdks/python/apache_beam/transforms/ptransform.py
@@ -68,13 +68,16 @@
 from apache_beam.internal import pickler
 from apache_beam.internal import util
 from apache_beam.portability import python_urns
+from apache_beam.pvalue import DoOutputsTuple
 from apache_beam.transforms.display import DisplayDataItem
 from apache_beam.transforms.display import HasDisplayData
 from apache_beam.typehints import native_type_compatibility
 from apache_beam.typehints import typehints
+from apache_beam.typehints.decorators import IOTypeHints
 from apache_beam.typehints.decorators import TypeCheckError
 from apache_beam.typehints.decorators import WithTypeHints
 from apache_beam.typehints.decorators import get_signature
+from apache_beam.typehints.decorators import get_type_hints
 from apache_beam.typehints.decorators import getcallargs_forhints
 from apache_beam.typehints.trivial_inference import instance_to_type
 from apache_beam.typehints.typehints import validate_composite_type_param
@@ -350,6 +353,14 @@
     # type: () -> str
     return self.__class__.__name__
 
+  def default_type_hints(self):
+    fn_type_hints = IOTypeHints.from_callable(self.expand)
+    if fn_type_hints is not None:
+      fn_type_hints = fn_type_hints.strip_pcoll()
+
+    # Prefer class decorator type hints for backwards compatibility.
+    return get_type_hints(self.__class__).with_defaults(fn_type_hints)
+
   def with_input_types(self, input_type_hint):
     """Annotates the input type of a :class:`PTransform` with a type-hint.
 
@@ -419,6 +430,8 @@
     root_hint = (
         arg_hints[0] if len(arg_hints) == 1 else arg_hints or kwarg_hints)
     for context, pvalue_, hint in _ZipPValues().visit(pvalueish, root_hint):
+      if isinstance(pvalue_, DoOutputsTuple):
+        continue
       if pvalue_.element_type is None:
         # TODO(robertwb): It's a bug that we ever get here. (typecheck)
         continue
diff --git a/sdks/python/apache_beam/typehints/decorators.py b/sdks/python/apache_beam/typehints/decorators.py
index dad0a31..4cd7681 100644
--- a/sdks/python/apache_beam/typehints/decorators.py
+++ b/sdks/python/apache_beam/typehints/decorators.py
@@ -105,6 +105,7 @@
 from typing import Optional
 from typing import Tuple
 from typing import TypeVar
+from typing import Union
 
 from apache_beam.typehints import native_type_compatibility
 from apache_beam.typehints import typehints
@@ -378,6 +379,75 @@
         self.output_types and len(self.output_types[0]) == 1 and
         not self.output_types[1])
 
+  def strip_pcoll(self):
+    from apache_beam.pipeline import Pipeline
+    from apache_beam.pvalue import PBegin
+    from apache_beam.pvalue import PDone
+
+    return self.strip_pcoll_helper(self.input_types,
+                                   self._has_input_types,
+                                   'input_types',
+                                    [Pipeline, PBegin],
+                                   'This input type hint will be ignored '
+                                   'and not used for type-checking purposes. '
+                                   'Typically, input type hints for a '
+                                   'PTransform are single (or nested) types '
+                                   'wrapped by a PCollection, or PBegin.',
+                                   'strip_pcoll_input()').\
+                strip_pcoll_helper(self.output_types,
+                                   self.has_simple_output_type,
+                                   'output_types',
+                                   [PDone, None],
+                                   'This output type hint will be ignored '
+                                   'and not used for type-checking purposes. '
+                                   'Typically, output type hints for a '
+                                   'PTransform are single (or nested) types '
+                                   'wrapped by a PCollection, PDone, or None.',
+                                   'strip_pcoll_output()')
+
+  def strip_pcoll_helper(
+      self,
+      my_type,            # type: any
+      has_my_type,        # type: Callable[[], bool]
+      my_key,             # type: str
+      special_containers,   # type: List[Union[PBegin, PDone, PCollection]]
+      error_str,          # type: str
+      source_str          # type: str
+      ):
+    # type: (...) -> IOTypeHints
+    from apache_beam.pvalue import PCollection
+
+    if not has_my_type() or not my_type or len(my_type[0]) != 1:
+      return self
+
+    my_type = my_type[0][0]
+
+    if isinstance(my_type, typehints.AnyTypeConstraint):
+      return self
+
+    special_containers += [PCollection]
+    kwarg_dict = {}
+
+    if (my_type not in special_containers and
+        getattr(my_type, '__origin__', None) != PCollection):
+      logging.warning(error_str + ' Got: %s instead.' % my_type)
+      kwarg_dict[my_key] = None
+      return self._replace(
+          origin=self._make_origin([self], tb=False, msg=[source_str]),
+          **kwarg_dict)
+
+    if (getattr(my_type, '__args__', -1) in [-1, None] or
+        len(my_type.__args__) == 0):
+      # e.g. PCollection (or PBegin/PDone)
+      kwarg_dict[my_key] = ((typehints.Any, ), {})
+    else:
+      # e.g. PCollection[type]
+      kwarg_dict[my_key] = ((convert_to_beam_type(my_type.__args__[0]), ), {})
+
+    return self._replace(
+        origin=self._make_origin([self], tb=False, msg=[source_str]),
+        **kwarg_dict)
+
   def strip_iterable(self):
     # type: () -> IOTypeHints
 
diff --git a/sdks/python/apache_beam/typehints/schemas.py b/sdks/python/apache_beam/typehints/schemas.py
index 0cc513f..cb4cf01 100644
--- a/sdks/python/apache_beam/typehints/schemas.py
+++ b/sdks/python/apache_beam/typehints/schemas.py
@@ -178,6 +178,11 @@
     return schema_pb2.FieldType(
         array_type=schema_pb2.ArrayType(element_type=element_type))
 
+  elif _safe_issubclass(type_, Mapping):
+    key_type, value_type = map(typing_to_runner_api, _get_args(type_))
+    return schema_pb2.FieldType(
+        map_type=schema_pb2.MapType(key_type=key_type, value_type=value_type))
+
   raise ValueError("Unsupported type: %s" % type_)
 
 
diff --git a/sdks/python/apache_beam/typehints/typed_pipeline_test_py3.py b/sdks/python/apache_beam/typehints/typed_pipeline_test_py3.py
index 2016871..e12930d 100644
--- a/sdks/python/apache_beam/typehints/typed_pipeline_test_py3.py
+++ b/sdks/python/apache_beam/typehints/typed_pipeline_test_py3.py
@@ -22,6 +22,7 @@
 
 from __future__ import absolute_import
 
+import typing
 import unittest
 
 import apache_beam as beam
@@ -257,6 +258,135 @@
     result = [1, 2, 3] | beam.FlatMap(fn) | beam.Map(fn2)
     self.assertCountEqual([4, 6], result)
 
+  def test_typed_ptransform_with_no_error(self):
+    class StrToInt(beam.PTransform):
+      def expand(
+          self,
+          pcoll: beam.pvalue.PCollection[str]) -> beam.pvalue.PCollection[int]:
+        return pcoll | beam.Map(lambda x: int(x))
+
+    class IntToStr(beam.PTransform):
+      def expand(
+          self,
+          pcoll: beam.pvalue.PCollection[int]) -> beam.pvalue.PCollection[str]:
+        return pcoll | beam.Map(lambda x: str(x))
+
+    _ = ['1', '2', '3'] | StrToInt() | IntToStr()
+
+  def test_typed_ptransform_with_bad_typehints(self):
+    class StrToInt(beam.PTransform):
+      def expand(
+          self,
+          pcoll: beam.pvalue.PCollection[str]) -> beam.pvalue.PCollection[int]:
+        return pcoll | beam.Map(lambda x: int(x))
+
+    class IntToStr(beam.PTransform):
+      def expand(
+          self,
+          pcoll: beam.pvalue.PCollection[str]) -> beam.pvalue.PCollection[str]:
+        return pcoll | beam.Map(lambda x: str(x))
+
+    with self.assertRaisesRegex(typehints.TypeCheckError,
+                                "Input type hint violation at IntToStr: "
+                                "expected <class 'str'>, got <class 'int'>"):
+      _ = ['1', '2', '3'] | StrToInt() | IntToStr()
+
+  def test_typed_ptransform_with_bad_input(self):
+    class StrToInt(beam.PTransform):
+      def expand(
+          self,
+          pcoll: beam.pvalue.PCollection[str]) -> beam.pvalue.PCollection[int]:
+        return pcoll | beam.Map(lambda x: int(x))
+
+    class IntToStr(beam.PTransform):
+      def expand(
+          self,
+          pcoll: beam.pvalue.PCollection[int]) -> beam.pvalue.PCollection[str]:
+        return pcoll | beam.Map(lambda x: str(x))
+
+    with self.assertRaisesRegex(typehints.TypeCheckError,
+                                "Input type hint violation at StrToInt: "
+                                "expected <class 'str'>, got <class 'int'>"):
+      # Feed integers to a PTransform that expects strings
+      _ = [1, 2, 3] | StrToInt() | IntToStr()
+
+  def test_typed_ptransform_with_partial_typehints(self):
+    class StrToInt(beam.PTransform):
+      def expand(self, pcoll) -> beam.pvalue.PCollection[int]:
+        return pcoll | beam.Map(lambda x: int(x))
+
+    class IntToStr(beam.PTransform):
+      def expand(
+          self,
+          pcoll: beam.pvalue.PCollection[int]) -> beam.pvalue.PCollection[str]:
+        return pcoll | beam.Map(lambda x: str(x))
+
+    # Feed integers to a PTransform that should expect strings
+    # but has no typehints so it expects any
+    _ = [1, 2, 3] | StrToInt() | IntToStr()
+
+  def test_typed_ptransform_with_bare_wrappers(self):
+    class StrToInt(beam.PTransform):
+      def expand(
+          self, pcoll: beam.pvalue.PCollection) -> beam.pvalue.PCollection:
+        return pcoll | beam.Map(lambda x: int(x))
+
+    class IntToStr(beam.PTransform):
+      def expand(
+          self,
+          pcoll: beam.pvalue.PCollection[int]) -> beam.pvalue.PCollection[str]:
+        return pcoll | beam.Map(lambda x: str(x))
+
+    _ = [1, 2, 3] | StrToInt() | IntToStr()
+
+  def test_typed_ptransform_with_no_typehints(self):
+    class StrToInt(beam.PTransform):
+      def expand(self, pcoll):
+        return pcoll | beam.Map(lambda x: int(x))
+
+    class IntToStr(beam.PTransform):
+      def expand(
+          self,
+          pcoll: beam.pvalue.PCollection[int]) -> beam.pvalue.PCollection[str]:
+        return pcoll | beam.Map(lambda x: str(x))
+
+    # Feed integers to a PTransform that should expect strings
+    # but has no typehints so it expects any
+    _ = [1, 2, 3] | StrToInt() | IntToStr()
+
+  def test_typed_ptransform_with_generic_annotations(self):
+    T = typing.TypeVar('T')
+
+    class IntToInt(beam.PTransform):
+      def expand(
+          self,
+          pcoll: beam.pvalue.PCollection[T]) -> beam.pvalue.PCollection[T]:
+        return pcoll | beam.Map(lambda x: x)
+
+    class IntToStr(beam.PTransform):
+      def expand(
+          self,
+          pcoll: beam.pvalue.PCollection[T]) -> beam.pvalue.PCollection[str]:
+        return pcoll | beam.Map(lambda x: str(x))
+
+    _ = [1, 2, 3] | IntToInt() | IntToStr()
+
+  def test_typed_ptransform_with_do_outputs_tuple_compiles(self):
+    class MyDoFn(beam.DoFn):
+      def process(self, element: int, *args, **kwargs):
+        if element % 2:
+          yield beam.pvalue.TaggedOutput('odd', 1)
+        else:
+          yield beam.pvalue.TaggedOutput('even', 1)
+
+    class MyPTransform(beam.PTransform):
+      def expand(self, pcoll: beam.pvalue.PCollection[int]):
+        return pcoll | beam.ParDo(MyDoFn()).with_outputs('odd', 'even')
+
+    # This test fails if you remove the following line from ptransform.py
+    # if isinstance(pvalue_, DoOutputsTuple): continue
+    _ = [1, 2, 3] | MyPTransform()
+
 
 class AnnotationsTest(unittest.TestCase):
   def test_pardo_dofn(self):
diff --git a/sdks/python/apache_beam/typehints/typehints_test_py3.py b/sdks/python/apache_beam/typehints/typehints_test_py3.py
index a7c23f0..5a36330 100644
--- a/sdks/python/apache_beam/typehints/typehints_test_py3.py
+++ b/sdks/python/apache_beam/typehints/typehints_test_py3.py
@@ -23,11 +23,19 @@
 from __future__ import absolute_import
 from __future__ import print_function
 
+import typing
 import unittest
 
+import apache_beam.typehints.typehints as typehints
+from apache_beam import Map
+from apache_beam import PTransform
+from apache_beam.pvalue import PBegin
+from apache_beam.pvalue import PCollection
+from apache_beam.pvalue import PDone
 from apache_beam.transforms.core import DoFn
 from apache_beam.typehints import KV
 from apache_beam.typehints import Iterable
+from apache_beam.typehints.typehints import Any
 
 
 class TestParDoAnnotations(unittest.TestCase):
@@ -46,11 +54,221 @@
       def process(self, element: int) -> Iterable[str]:
         pass
 
-    print(MyDoFn().get_type_hints())
     th = MyDoFn().get_type_hints()
     self.assertEqual(th.input_types, ((int, ), {}))
     self.assertEqual(th.output_types, ((str, ), {}))
 
 
+class TestPTransformAnnotations(unittest.TestCase):
+  def test_pep484_annotations(self):
+    class MyPTransform(PTransform):
+      def expand(self, pcoll: PCollection[int]) -> PCollection[str]:
+        return pcoll | Map(lambda num: str(num))
+
+    th = MyPTransform().get_type_hints()
+    self.assertEqual(th.input_types, ((int, ), {}))
+    self.assertEqual(th.output_types, ((str, ), {}))
+
+  def test_annotations_without_input_pcollection_wrapper(self):
+    class MyPTransform(PTransform):
+      def expand(self, pcoll: int) -> PCollection[str]:
+        return pcoll | Map(lambda num: str(num))
+
+    error_str = (
+        r'This input type hint will be ignored and not used for '
+        r'type-checking purposes. Typically, input type hints for a '
+        r'PTransform are single (or nested) types wrapped by a '
+        r'PCollection, or PBegin. Got: {} instead.'.format(int))
+
+    with self.assertLogs(level='WARN') as log:
+      MyPTransform().get_type_hints()
+      self.assertIn(error_str, log.output[0])
+
+  def test_annotations_without_output_pcollection_wrapper(self):
+    class MyPTransform(PTransform):
+      def expand(self, pcoll: PCollection[int]) -> str:
+        return pcoll | Map(lambda num: str(num))
+
+    error_str = (
+        r'This output type hint will be ignored and not used for '
+        r'type-checking purposes. Typically, output type hints for a '
+        r'PTransform are single (or nested) types wrapped by a '
+        r'PCollection, PDone, or None. Got: {} instead.'.format(str))
+
+    with self.assertLogs(level='WARN') as log:
+      th = MyPTransform().get_type_hints()
+      self.assertIn(error_str, log.output[0])
+      self.assertEqual(th.input_types, ((int, ), {}))
+      self.assertEqual(th.output_types, None)
+
+  def test_annotations_without_input_internal_type(self):
+    class MyPTransform(PTransform):
+      def expand(self, pcoll: PCollection) -> PCollection[str]:
+        return pcoll | Map(lambda num: str(num))
+
+    th = MyPTransform().get_type_hints()
+    self.assertEqual(th.input_types, ((Any, ), {}))
+    self.assertEqual(th.output_types, ((str, ), {}))
+
+  def test_annotations_without_output_internal_type(self):
+    class MyPTransform(PTransform):
+      def expand(self, pcoll: PCollection[int]) -> PCollection:
+        return pcoll | Map(lambda num: str(num))
+
+    th = MyPTransform().get_type_hints()
+    self.assertEqual(th.input_types, ((int, ), {}))
+    self.assertEqual(th.output_types, ((Any, ), {}))
+
+  def test_annotations_without_any_internal_type(self):
+    class MyPTransform(PTransform):
+      def expand(self, pcoll: PCollection) -> PCollection:
+        return pcoll | Map(lambda num: str(num))
+
+    th = MyPTransform().get_type_hints()
+    self.assertEqual(th.input_types, ((Any, ), {}))
+    self.assertEqual(th.output_types, ((Any, ), {}))
+
+  def test_annotations_without_input_typehint(self):
+    class MyPTransform(PTransform):
+      def expand(self, pcoll) -> PCollection[str]:
+        return pcoll | Map(lambda num: str(num))
+
+    th = MyPTransform().get_type_hints()
+    self.assertEqual(th.input_types, ((Any, ), {}))
+    self.assertEqual(th.output_types, ((str, ), {}))
+
+  def test_annotations_without_output_typehint(self):
+    class MyPTransform(PTransform):
+      def expand(self, pcoll: PCollection[int]):
+        return pcoll | Map(lambda num: str(num))
+
+    th = MyPTransform().get_type_hints()
+    self.assertEqual(th.input_types, ((int, ), {}))
+    self.assertEqual(th.output_types, ((Any, ), {}))
+
+  def test_annotations_without_any_typehints(self):
+    class MyPTransform(PTransform):
+      def expand(self, pcoll):
+        return pcoll | Map(lambda num: str(num))
+
+    th = MyPTransform().get_type_hints()
+    self.assertEqual(th.input_types, None)
+    self.assertEqual(th.output_types, None)
+
+  def test_annotations_with_pbegin(self):
+    class MyPTransform(PTransform):
+      def expand(self, pcoll: PBegin):
+        return pcoll | Map(lambda num: str(num))
+
+    th = MyPTransform().get_type_hints()
+    self.assertEqual(th.input_types, ((Any, ), {}))
+    self.assertEqual(th.output_types, ((Any, ), {}))
+
+  def test_annotations_with_pdone(self):
+    class MyPTransform(PTransform):
+      def expand(self, pcoll) -> PDone:
+        return pcoll | Map(lambda num: str(num))
+
+    th = MyPTransform().get_type_hints()
+    self.assertEqual(th.input_types, ((Any, ), {}))
+    self.assertEqual(th.output_types, ((Any, ), {}))
+
+  def test_annotations_with_none_input(self):
+    class MyPTransform(PTransform):
+      def expand(self, pcoll: None) -> PCollection[str]:
+        return pcoll | Map(lambda num: str(num))
+
+    error_str = (
+        r'This input type hint will be ignored and not used for '
+        r'type-checking purposes. Typically, input type hints for a '
+        r'PTransform are single (or nested) types wrapped by a '
+        r'PCollection, or PBegin. Got: {} instead.'.format(None))
+
+    with self.assertLogs(level='WARN') as log:
+      th = MyPTransform().get_type_hints()
+      self.assertIn(error_str, log.output[0])
+      self.assertEqual(th.input_types, None)
+      self.assertEqual(th.output_types, ((str, ), {}))
+
+  def test_annotations_with_none_output(self):
+    class MyPTransform(PTransform):
+      def expand(self, pcoll) -> None:
+        return pcoll | Map(lambda num: str(num))
+
+    th = MyPTransform().get_type_hints()
+    self.assertEqual(th.input_types, ((Any, ), {}))
+    self.assertEqual(th.output_types, ((Any, ), {}))
+
+  def test_annotations_with_arbitrary_output(self):
+    class MyPTransform(PTransform):
+      def expand(self, pcoll) -> str:
+        return pcoll | Map(lambda num: str(num))
+
+    th = MyPTransform().get_type_hints()
+    self.assertEqual(th.input_types, ((Any, ), {}))
+    self.assertEqual(th.output_types, None)
+
+  def test_annotations_with_arbitrary_input_and_output(self):
+    class MyPTransform(PTransform):
+      def expand(self, pcoll: int) -> str:
+        return pcoll | Map(lambda num: str(num))
+
+    input_error_str = (
+        r'This input type hint will be ignored and not used for '
+        r'type-checking purposes. Typically, input type hints for a '
+        r'PTransform are single (or nested) types wrapped by a '
+        r'PCollection, or PBegin. Got: {} instead.'.format(int))
+
+    output_error_str = (
+        r'This output type hint will be ignored and not used for '
+        r'type-checking purposes. Typically, output type hints for a '
+        r'PTransform are single (or nested) types wrapped by a '
+        r'PCollection, PDone, or None. Got: {} instead.'.format(str))
+
+    with self.assertLogs(level='WARN') as log:
+      th = MyPTransform().get_type_hints()
+      self.assertIn(input_error_str, log.output[0])
+      self.assertIn(output_error_str, log.output[1])
+      self.assertEqual(th.input_types, None)
+      self.assertEqual(th.output_types, None)
+
+  def test_typing_module_annotations_are_converted_to_beam_annotations(self):
+    class MyPTransform(PTransform):
+      def expand(
+          self, pcoll: PCollection[typing.Dict[str, str]]
+      ) -> PCollection[typing.Dict[str, str]]:
+        return pcoll
+
+    th = MyPTransform().get_type_hints()
+    self.assertEqual(th.input_types, ((typehints.Dict[str, str], ), {}))
+    self.assertEqual(th.input_types, ((typehints.Dict[str, str], ), {}))
+
+  def test_nested_typing_annotations_are_converted_to_beam_annotations(self):
+    class MyPTransform(PTransform):
+      def expand(self, pcoll:
+         PCollection[typing.Union[int, typing.Any, typing.Dict[str, float]]]) \
+      -> PCollection[typing.Union[int, typing.Any, typing.Dict[str, float]]]:
+        return pcoll
+
+    th = MyPTransform().get_type_hints()
+    self.assertEqual(
+        th.input_types,
+        ((typehints.Union[int, typehints.Any, typehints.Dict[str,
+                                                             float]], ), {}))
+    self.assertEqual(
+        th.input_types,
+        ((typehints.Union[int, typehints.Any, typehints.Dict[str,
+                                                             float]], ), {}))
+
+  def test_mixed_annotations_are_converted_to_beam_annotations(self):
+    class MyPTransform(PTransform):
+      def expand(self, pcoll: typing.Any) -> typehints.Any:
+        return pcoll
+
+    th = MyPTransform().get_type_hints()
+    self.assertEqual(th.input_types, ((typehints.Any, ), {}))
+    self.assertEqual(th.input_types, ((typehints.Any, ), {}))
+
+
 if __name__ == '__main__':
   unittest.main()
diff --git a/settings.gradle b/settings.gradle
index 0220857..638c216 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -120,6 +120,7 @@
 include ":sdks:java:io:redis"
 include ":sdks:java:io:solr"
 include ":sdks:java:io:snowflake"
+include ":sdks:java:io:snowflake:expansion-service"
 include ":sdks:java:io:splunk"
 include ":sdks:java:io:thrift"
 include ":sdks:java:io:tika"
diff --git a/website/www/site/content/en/documentation/io/built-in/snowflake.md b/website/www/site/content/en/documentation/io/built-in/snowflake.md
new file mode 100644
index 0000000..078a373
--- /dev/null
+++ b/website/www/site/content/en/documentation/io/built-in/snowflake.md
@@ -0,0 +1,364 @@
+---
+title: "Apache Snowflake I/O connector"
+---
+<!--
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+
+[Built-in I/O Transforms](/documentation/io/built-in/)
+
+# Snowflake I/O
+Pipeline options and general information about using and running Snowflake IO.
+
+## Authentication
+All authentication methods available for the Snowflake JDBC Driver are possible to use with the IO transforms:
+
+- Username and password
+- Key pair
+- OAuth token
+
+Passing credentials is done via Pipeline options.
+
+Passing credentials is done via Pipeline options used to instantiate `SnowflakeIO.DataSourceConfiguration`:
+{{< highlight java >}}
+SnowflakePipelineOptions options = PipelineOptionsFactory
+        .fromArgs(args)
+        .withValidation()
+        .as(SnowflakePipelineOptions.class);
+SnowflakeCredentials credentials = SnowflakeCredentialsFactory.of(options);
+
+SnowflakeIO.DataSourceConfiguration.create(credentials)
+        .(other DataSourceConfiguration options)
+{{< /highlight >}}
+### Username and password 
+To use username/password authentication in SnowflakeIO, invoke your pipeline with the following Pipeline options:
+{{< highlight >}}
+--username=<USERNAME> --password=<PASSWORD>
+{{< /highlight >}}
+### Key pair
+To use this authentication method, you must first generate a key pair and associate the public key with the Snowflake user that will connect using the IO transform. For instructions, see the [Snowflake documentation](https://docs.snowflake.com/en/user-guide/jdbc-configure.html).
+
+To use key pair authentication with SnowflakeIO, invoke your pipeline with following Pipeline options:
+{{< highlight >}}
+--username=<USERNAME> --privateKeyPath=<PATH_TO_P8_FILE> --privateKeyPassphrase=<PASSWORD_FOR_KEY>
+{{< /highlight >}}
+
+### OAuth token
+SnowflakeIO also supports OAuth token.
+
+**IMPORTANT**: SnowflakeIO requires a valid OAuth access token. It will neither be able to refresh the token nor obtain it using a web-based flow. For information on configuring an OAuth integration and obtaining the token, see the  [Snowflake documentation](https://docs.snowflake.com/en/user-guide/oauth-intro.html).
+
+Once you have the token, invoke your pipeline with following Pipeline Options:
+{{< highlight >}}
+--oauthToken=<TOKEN>
+{{< /highlight >}}
+## DataSource Configuration
+DataSource configuration is required in both read and write object for configuring Snowflake connection properties for IO purposes.
+### General usage
+Create the DataSource configuration:
+{{< highlight java >}}
+ SnowflakeIO.DataSourceConfiguration
+            .create(SnowflakeCredentialsFactory.of(options))
+            .withUrl(options.getUrl())
+            .withServerName(options.getServerName())
+            .withDatabase(options.getDatabase())
+            .withWarehouse(options.getWarehouse())
+            .withSchema(options.getSchema());
+{{< /highlight >}}
+Where parameters can be:
+
+- ` .withUrl(...)`
+  - JDBC-like URL for your Snowflake account, including account name and region, without any parameters.
+  - Example: `.withUrl("jdbc:snowflake://account.snowflakecomputing.com")`
+- `.withServerName(...)`
+  - Server Name - full server name with account, zone and domain.
+  - Example: `.withServerName("account.snowflakecomputing.com")`
+- `.withDatabase(...)`
+  - Name of the Snowflake database to use. 
+  - Example: `.withDatabase("MY_DATABASE")`
+- `.withWarehouse(...)`
+  - Name of the Snowflake warehouse to use. This parameter is optional. If no warehouse name is specified, the default warehouse for the user is used.
+  - Example: `.withWarehouse("MY_WAREHOUSE")`
+- `.withSchema(...)`
+  - Name of the schema in the database to use. This parameter is optional.
+  - Example: `.withSchema("PUBLIC")`
+
+
+**Note** - either `.withUrl(...)` or `.withServerName(...)` **is required**.
+
+## Pipeline options
+Use Beam’s [Pipeline options](https://beam.apache.org/releases/javadoc/2.17.0/org/apache/beam/sdk/options/PipelineOptions.html) to set options via the command line.
+### Snowflake Pipeline options
+Snowflake IO library supports following options that can be passed via the [command line](https://beam.apache.org/documentation/io/built-in/snowflake/#running-main-command-with-pipeline-options) by default when a Pipeline uses them:
+
+`--url` Snowflake's JDBC-like url including account name and region without any parameters.
+
+`--serverName` Full server name with account, zone and domain.
+
+`--username` Required for username/password and Private Key authentication.
+
+`--oauthToken` Required for OAuth authentication only.
+
+`--password` Required for username/password authentication only.
+
+`--privateKeyPath` Path to Private Key file. Required for Private Key authentication only.
+
+`--privateKeyPassphrase` Private Key's passphrase. Required for Private Key authentication only.
+
+`--stagingBucketName` External bucket path ending with `/`. I.e. `gs://bucket/`. Sub-directories are allowed.
+
+`--storageIntegrationName` Storage integration name
+
+`--warehouse` Warehouse to use. Optional.
+
+`--database` Database name to connect to. Optional.
+
+`--schema` Schema to use. Optional.
+
+`--table` Table to use. Optional.
+
+`--query` Query to use. Optional.
+
+`--role` Role to use. Optional.
+
+`--authenticator` Authenticator to use. Optional.
+
+`--portNumber` Port number. Optional.
+
+`--loginTimeout` Login timeout. Optional.
+
+## Running pipelines on Dataflow
+By default, pipelines are run on [Direct Runner](https://beam.apache.org/documentation/runners/direct/) on your local machine. To run a pipeline on [Google Dataflow](https://cloud.google.com/dataflow/), you must provide the following Pipeline options:
+
+- `--runner=DataflowRunner`
+  - The Dataflow’s specific runner.
+
+- `--project=<GCS PROJECT>`
+  - Name of the Google Cloud Platform project.
+
+- `--stagingBucketName=<GCS BUCKET NAME>`
+  - Google Cloud Services bucket where the Beam files will be staged.
+
+- `--maxNumWorkers=5`
+  - (optional) Maximum number of workers.
+
+- `--appName=<JOB NAME>`
+  - (optional) Prefix for the job name in the Dataflow Dashboard.
+
+More pipeline options for Dataflow can be found [here](https://beam.apache.org/releases/javadoc/2.17.0/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.html).
+
+**Note**: To properly authenticate with Google Cloud, please use [gcloud](https://cloud.google.com/sdk/gcloud/) or follow the [Google Cloud documentation](https://cloud.google.com/docs/authentication/).
+
+**Important**: Please acknowledge [Google Dataflow pricing](Important: Please acknowledge Google Dataflow pricing).
+
+## Writing to Snowflake tables
+One of the functions of SnowflakeIO is writing to Snowflake tables. This transformation enables you to finish the Beam pipeline with an output operation that sends the user's [PCollection](https://beam.apache.org/releases/javadoc/2.17.0/org/apache/beam/sdk/values/PCollection.html) to your Snowflake database.
+### Batch write (from a bounded source)
+The basic .`write()` operation usage is as follows:
+{{< highlight java >}}
+data.apply(
+   SnowflakeIO.<type>write()
+       .withDataSourceConfiguration(dc)
+       .to("MY_TABLE")
+       .withStagingBucketName("BUCKET NAME")
+       .withStorageIntegrationName("STORAGE INTEGRATION NAME")
+       .withUserDataMapper(mapper)
+)
+{{< /highlight >}}
+Replace type with the data type of the PCollection object to write; for example, SnowflakeIO.<String> for an input PCollection of Strings.
+
+All the below parameters are required:
+
+- `.withDataSourceConfiguration()` Accepts a DatasourceConfiguration object.
+
+- `.to()` Accepts the target Snowflake table name.
+
+- `.withStagingBucketName()` Accepts a cloud bucket path ended with slash.
+ -Example: `.withStagingBucketName("gs://mybucket/my/dir/")`
+
+- `.withStorageIntegrationName()` Accepts a name of a Snowflake storage integration object created according to Snowflake documentationt. Example:
+{{< highlight >}}
+CREATE OR REPLACE STORAGE INTEGRATION test_integration 
+TYPE = EXTERNAL_STAGE 
+STORAGE_PROVIDER = GCS 
+ENABLED = TRUE 
+STORAGE_ALLOWED_LOCATIONS = ('gcs://bucket/');
+{{< /highlight >}}
+Then:
+{{< highlight >}}
+.withStorageIntegrationName(test_integration)
+{{< /highlight >}}
+
+- `.withUserDataMapper()` Accepts the UserDataMapper function that will map a user's PCollection to an array of String values `(String[])`.
+
+**Note**:
+SnowflakeIO uses COPY statements behind the scenes to write (using [COPY to table](https://docs.snowflake.net/manuals/sql-reference/sql/copy-into-table.html)). StagingBucketName will be used to save CSV files which will end up in Snowflake. Those CSV files will be saved under the “stagingBucketName” path.
+
+### UserDataMapper function
+The UserDataMapper function is required to map data from a PCollection to an array of String values before the `write()` operation saves the data to temporary .csv files. For example:
+{{< highlight java >}}
+public static SnowflakeIO.UserDataMapper<Long> getCsvMapper() {
+    return (SnowflakeIO.UserDataMapper<Long>) recordLine -> new String[] {recordLine.toString()};
+}
+{{< /highlight >}}
+
+### Additional write options
+#### Transformation query
+The `.withQueryTransformation()` option for the `write()` operation accepts a SQL query as a String value, which will be performed while transfering data staged in CSV files directly to the target Snowflake table. For information about the transformation SQL syntax,  see the [Snowflake Documentation](https://docs.snowflake.net/manuals/sql-reference/sql/copy-into-table.html#transformation-parameters).
+
+Usage:
+{{< highlight java >}}
+String query = "SELECT t.$1 from YOUR_TABLE;";
+data.apply(
+   SnowflakeIO.<~>write()
+       .withDataSourceConfiguration(dc)
+       .to("MY_TABLE")
+       .withStagingBucketName("BUCKET NAME")
+       .withStorageIntegrationName("STORAGE INTEGRATION NAME")
+       .withUserDataMapper(mapper)
+       .withQueryTransformation(query)
+)
+{{< /highlight >}}
+
+#### Write disposition
+Define the write behaviour based on the table where data will be written to by specifying the `.withWriteDisposition(...)` option for the `write()` operation. The following values are supported:
+
+- APPEND - Default behaviour. Written data is added to the existing rows in the table,
+
+- EMPTY - The target table must be empty;  otherwise, the write operation fails,
+
+- TRUNCATE - The write operation deletes all rows from the target table before writing to it.
+
+Example of usage:
+{{< highlight java >}}
+data.apply(
+   SnowflakeIO.<~>write()
+       .withDataSourceConfiguration(dc)
+       .to("MY_TABLE")
+       .withStagingBucketName("BUCKET NAME")
+       .withStorageIntegrationName("STORAGE INTEGRATION NAME")
+       .withUserDataMapper(mapper)
+       .withWriteDisposition(TRUNCATE)
+)
+{{< /highlight >}}
+
+#### Create disposition
+The `.withCreateDisposition()` option defines the behavior of the write operation if the target table does not exist . The following values are supported:
+
+- CREATE_IF_NEEDED - default behaviour. The write operation checks whether the specified target table exists; if it does not, the write operation attempts to create the table Specify the schema for the target table using the `.withTableSchema()` option.
+
+- CREATE_NEVER -  The write operation fails if the target table does not exist.
+
+Usage:
+{{< highlight java >}}
+data.apply(
+   SnowflakeIO.<~>write()
+       .withDataSourceConfiguration(dc)
+       .to("MY_TABLE")
+       .withStagingBucketName("BUCKET NAME")
+       .withStorageIntegrationName("STORAGE INTEGRATION NAME")
+       .withUserDataMapper(mapper)
+       .withCreateDisposition(CREATE_NEVER)
+)
+{{< /highlight >}}
+
+#### Table schema disposition
+When the `.withCreateDisposition()` .option is set to `CREATE_IF_NEEDED`, the `.withTableSchema()` option enables specifying the schema for the created target table. 
+A table schema is a list of `SFColumn` objects with name and type corresponding to column type for each column in the table. 
+
+Usage:
+{{< highlight java >}}
+SFTableSchema tableSchema =
+    new SFTableSchema(
+        SFColumn.of("my_date", new SFDate(), true),
+        new SFColumn("id", new SFNumber()),
+        SFColumn.of("name", new SFText(), true));
+
+data.apply(
+   SnowflakeIO.<~>write()
+       .withDataSourceConfiguration(dc)
+       .to("MY_TABLE")
+       .withStagingBucketName("BUCKET NAME")
+       .withStorageIntegrationName("STORAGE INTEGRATION NAME")
+       .withUserDataMapper(mapper)
+       .withTableSchema(tableSchema)
+)
+{{< /highlight >}}
+## Reading from Snowflake
+One of the functions of SnowflakeIO is reading Snowflake tables - either full tables via table name or custom data via query. Output of the read transform is a [PCollection](https://beam.apache.org/releases/javadoc/2.17.0/org/apache/beam/sdk/values/PCollection.html) of user-defined data type.
+
+### General usage
+
+The basic `.read()` operation usage:
+{{< highlight java >}}
+PCollection<USER_DATA_TYPE> items = pipeline.apply(
+   SnowflakeIO.<USER_DATA_TYPE>read()
+       .withDataSourceConfiguration(dc)
+       .fromTable("MY_TABLE") // or .fromQuery("QUERY")
+       .withStagingBucketName("BUCKET NAME")
+       .withStorageIntegrationName("STORAGE INTEGRATION NAME")
+       .withCsvMapper(mapper)
+       .withCoder(coder));
+)
+{{< /highlight >}}
+Where all below parameters are required:
+
+- `.withDataSourceConfiguration(...)`
+  - Accepts a DataSourceConfiguration object.
+
+- `.fromTable(...) or .fromQuery(...)`
+  - Specifies a Snowflake table name or custom SQL query.
+
+- `.withStagingBucketName()`
+  - Accepts a cloud bucket name.
+
+-  `.withStorageIntegrationName()`
+  - Accepts a name of a Snowflake storage integration object created according to Snowflake documentation. Example:
+{{< highlight >}}
+CREATE OR REPLACE STORAGE INTEGRATION test_integration 
+TYPE = EXTERNAL_STAGE 
+STORAGE_PROVIDER = GCS 
+ENABLED = TRUE 
+STORAGE_ALLOWED_LOCATIONS = ('gcs://bucket/');
+{{< /highlight >}}
+Then:
+{{< highlight >}}
+.withStorageIntegrationName(test_integration)
+{{< /highlight >}}
+
+- `.withCsvMapper(mapper)`
+  - Accepts a [CSVMapper](https://beam.apache.org/documentation/io/built-in/snowflake/#csvmapper) instance for mapping String[] to USER_DATA_TYPE.
+- `.withCoder(coder)`
+  - Accepts the [Coder](https://beam.apache.org/releases/javadoc/2.0.0/org/apache/beam/sdk/coders/Coder.html) for USER_DATA_TYPE.
+
+**Note**:
+SnowflakeIO uses COPY statements behind the scenes to read (using [COPY to location](https://docs.snowflake.net/manuals/sql-reference/sql/copy-into-location.html)) files staged in cloud storage.StagingBucketName will be used as a temporary location for storing CSV files. Those temporary directories will be named `sf_copy_csv_DATE_TIME_RANDOMSUFFIX` and they will be removed automatically once Read operation finishes.
+
+### CSVMapper
+SnowflakeIO uses a [COPY INTO <location>](https://docs.snowflake.net/manuals/sql-reference/sql/copy-into-location.html) statement to move data from a Snowflake table to Google Cloud Storage as CSV files. These files are then downloaded via [FileIO](https://beam.apache.org/releases/javadoc/2.3.0/index.html?org/apache/beam/sdk/io/FileIO.html) and processed line by line. Each line is split into an array of Strings using the [OpenCSV](http://opencsv.sourceforge.net/) library. 
+
+The CSVMapper’s job is to give the user the possibility to convert the array of Strings to a user-defined type, ie. GenericRecord for Avro or Parquet files, or custom POJO.
+
+Example implementation of CsvMapper for GenericRecord:
+{{< highlight java >}}
+static SnowflakeIO.CsvMapper<GenericRecord> getCsvMapper() {
+   return (SnowflakeIO.CsvMapper<GenericRecord>)
+           parts -> {
+               return new GenericRecordBuilder(PARQUET_SCHEMA)
+                       .set("ID", Long.valueOf(parts[0]))
+                       .set("NAME", parts[1])
+                       [...]
+                       .build();
+           };
+}
+{{< /highlight >}}
\ No newline at end of file
diff --git a/website/www/site/content/en/documentation/patterns/ai-platform.md b/website/www/site/content/en/documentation/patterns/ai-platform.md
index 905f895..b2a7b10 100644
--- a/website/www/site/content/en/documentation/patterns/ai-platform.md
+++ b/website/www/site/content/en/documentation/patterns/ai-platform.md
@@ -35,7 +35,7 @@
 {{< /highlight >}}
 
 {{< highlight java >}}
-// Java examples will be available on Beam 2.23 release.
+{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" NlpAnalyzeText >}}
 {{< /highlight >}}
 
 
@@ -79,7 +79,7 @@
 {{< /highlight >}}
 
 {{< highlight java >}}
-// Java examples will be available on Beam 2.23 release.
+{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" NlpExtractSentiments >}}
 {{< /highlight >}}
 
 The snippet loops over `sentences` and, for each sentence, extracts the sentiment score. 
@@ -99,7 +99,7 @@
 {{< /highlight >}}
 
 {{< highlight java >}}
-// Java examples will be available on Beam 2.23 release.
+{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" NlpExtractEntities >}}
 {{< /highlight >}}
 
 Entities can be found in `entities` attribute. Just like before, `entities` is a sequence, that's why list comprehension is a viable choice. The most tricky part is interpreting the types of entities. Natural Language API defines entity types as enum. In a response object, entity types are returned as integers. That's why a user has to instantiate `naturallanguageml.enums.Entity.Type` to access a human-readable name.
@@ -119,7 +119,7 @@
 {{< /highlight >}}
 
 {{< highlight java >}}
-// Java examples will be available on Beam 2.23 release.
+{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" NlpAnalyzeDependencyTree >}}
 {{< /highlight >}}
 
 The output is below. For better readability, indexes are replaced by text which they refer to:
diff --git a/website/www/site/content/en/documentation/sdks/python-type-safety.md b/website/www/site/content/en/documentation/sdks/python-type-safety.md
index 074460c..755f795 100644
--- a/website/www/site/content/en/documentation/sdks/python-type-safety.md
+++ b/website/www/site/content/en/documentation/sdks/python-type-safety.md
@@ -71,7 +71,7 @@
 Using Annotations has the added benefit of allowing use of a static type checker (such as mypy) to additionally type check your code.
 If you already use a type checker, using annotations instead of decorators reduces code duplication.
 However, annotations do not cover all the use cases that decorators and inline declarations do.
-Two such are the `expand` of a composite transform and lambda functions.
+For instance, they do not work for lambda functions.
 
 ### Declaring Type Hints Using Type Annotations
 
@@ -82,6 +82,7 @@
 Annotations are currently supported on:
 
  - `process()` methods on `DoFn` subclasses.
+ - `expand()` methods on `PTransform` subclasses.
  - Functions passed to: `ParDo`, `Map`, `FlatMap`, `Filter`.
 
 The following code declares an `int` input and a `str` output type hint on the `to_id` transform, using annotations on `my_fn`.
@@ -90,6 +91,15 @@
 {{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test_py3.py" type_hints_map_annotations >}}
 {{< /highlight >}}
 
+The following code demonstrates how to use annotations on `PTransform` subclasses. 
+A valid annotation is a `PCollection` that wraps an internal (nested) type, `PBegin`, `PDone`, or `None`. 
+The following code declares typehints on a custom PTransform, that takes a `PCollection[int]` input
+and outputs a `PCollection[str]`, using annotations.
+
+{{< highlight py >}}
+{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test_py3.py" type_hints_ptransforms >}}
+{{< /highlight >}}
+
 The following code declares `int` input and output type hints on `filter_evens`, using annotations on `FilterEvensDoFn.process`.
 Since `process` returns a generator, the output type for a DoFn producing a `PCollection[int]` is annotated as `Iterable[int]` (`Generator[int, None, None]` would also work here).
 Beam will remove the outer iterable of the return type on the `DoFn.process` method and functions passed to `FlatMap` to deduce the element type of resulting PCollection .
@@ -182,6 +192,7 @@
 * `Iterable[T]`
 * `Iterator[T]`
 * `Generator[T]`
+* `PCollection[T]`
 
 **Note:** The `Tuple[T, U]` type hint is a tuple with a fixed number of heterogeneously typed elements, while the `Tuple[T, ...]` type hint is a tuple with a variable of homogeneously typed elements.
 
diff --git a/website/www/site/content/en/roadmap/connectors-multi-sdk.md b/website/www/site/content/en/roadmap/connectors-multi-sdk.md
index 464ad83..3b13f59 100644
--- a/website/www/site/content/en/roadmap/connectors-multi-sdk.md
+++ b/website/www/site/content/en/roadmap/connectors-multi-sdk.md
@@ -80,6 +80,7 @@
 * Java KafkaIO - completed - [BEAM-7029](https://issues.apache.org/jira/browse/BEAM-7029)
 * Java KinesisIO - In progress - [BEAM-10137](https://issues.apache.org/jira/browse/BEAM-10137), [BEAM-10138](https://issues.apache.org/jira/browse/BEAM-10138)
 * Java PubSubIO - In progress - [BEAM-7738](https://issues.apache.org/jira/browse/BEAM-7738)
+* Java SnowflakeIO - In progress - [BEAM-9897](https://issues.apache.org/jira/browse/BEAM-9897), [BEAM-9898](https://issues.apache.org/jira/browse/BEAM-9898)
 * Java SpannerIO - In progress - [BEAM-10139](https://issues.apache.org/jira/browse/BEAM-10139), [BEAM-10140](https://issues.apache.org/jira/browse/BEAM-10140)
 * Java SQL - completed - [BEAM-8603](https://issues.apache.org/jira/browse/BEAM-8603)
 
diff --git a/website/www/site/data/io_matrix.yaml b/website/www/site/data/io_matrix.yaml
index 5923a86..84637c6 100644
--- a/website/www/site/data/io_matrix.yaml
+++ b/website/www/site/data/io_matrix.yaml
@@ -305,6 +305,7 @@
             url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.datastore.v1new.datastoreio.html
       - transform: SnowflakeIO
         description: Experimental Transforms for reading from and writing to [Snowflake](https://www.snowflake.com/).
+        docs: /documentation/io/built-in/snowflake
         implementations:
           - language: java
             name: org.apache.beam.sdk.io.snowflake.SnowflakeIO
diff --git a/website/www/site/layouts/partials/section-menu/en/documentation.html b/website/www/site/layouts/partials/section-menu/en/documentation.html
index c8d0da9..fd35a21bd 100644
--- a/website/www/site/layouts/partials/section-menu/en/documentation.html
+++ b/website/www/site/layouts/partials/section-menu/en/documentation.html
@@ -74,6 +74,7 @@
             <li><a href="/documentation/io/built-in/hadoop/">Hadoop Input/Output Format IO</a></li>
             <li><a href="/documentation/io/built-in/hcatalog/">HCatalog IO</a></li>
             <li><a href="/documentation/io/built-in/google-bigquery/">Google BigQuery I/O connector</a></li>
+            <li><a href="/documentation/io/built-in/snowflake/">Snowflake I/O connector</a></li>
           </ul>
        </li>