Merge pull request #12467 [BEAM-7996] Add map & nil encoding to Go SDK.

diff --git a/CHANGES.md b/CHANGES.md
index 0d29362..66dc68c 100644
--- a/CHANGES.md
+++ b/CHANGES.md
@@ -58,9 +58,13 @@
 
 * New overloads for BigtableIO.Read.withKeyRange() and BigtableIO.Read.withRowFilter()
   methods that take ValueProvider as a parameter (Java) ([BEAM-10283](https://issues.apache.org/jira/browse/BEAM-10283)).
+* The WriteToBigQuery transform (Python) in Dataflow Batch no longer relies on BigQuerySink by default. It relies on 
+  a new, fully-featured transform based on file loads into BigQuery. To revert the behavior to the old implementation,
+  you may use `--experiments=use_legacy_bq_sink`.
 * 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
 
@@ -97,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/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/SplittableParDoExpander.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/SplittableParDoExpander.java
index 181ddab..4e8088f 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/SplittableParDoExpander.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/SplittableParDoExpander.java
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.runners.core.construction.graph;
 
+import com.google.auto.value.AutoValue;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Map;
@@ -66,7 +67,7 @@
    * information is available to the runner if it chooses to inspect it.
    */
   public static TransformReplacement createSizedReplacement() {
-    return SizedReplacement.INSTANCE;
+    return SizedReplacement.builder().setDrain(false).build();
   }
 
   /**
@@ -91,13 +92,25 @@
    * .
    */
   public static TransformReplacement createTruncateReplacement() {
-    return TruncateReplacement.INSTANCE;
+    return SizedReplacement.builder().setDrain(true).build();
   }
 
   /** See {@link #createSizedReplacement()} for details. */
-  private static class SizedReplacement implements TransformReplacement {
+  @AutoValue
+  abstract static class SizedReplacement implements TransformReplacement {
 
-    private static final SizedReplacement INSTANCE = new SizedReplacement();
+    static Builder builder() {
+      return new AutoValue_SplittableParDoExpander_SizedReplacement.Builder();
+    }
+
+    abstract boolean isDrain();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setDrain(boolean isDrain);
+
+      abstract SizedReplacement build();
+    }
 
     @Override
     public MessageWithComponents getReplacement(
@@ -209,14 +222,71 @@
           splitAndSize.setEnvironmentId(splittableParDo.getEnvironmentId());
           rval.getComponentsBuilder().putTransforms(splitAndSizeId, splitAndSize.build());
         }
+        PTransform.Builder newCompositeRoot =
+            splittableParDo
+                .toBuilder()
+                // Clear the original splittable ParDo spec and add all the new transforms as
+                // children.
+                .clearSpec()
+                .addAllSubtransforms(Arrays.asList(pairWithRestrictionId, splitAndSizeId));
 
         String processSizedElementsAndRestrictionsId =
             generateUniqueId(
                 transformId + "/ProcessSizedElementsAndRestrictions",
                 existingComponents::containsTransforms);
+        String processSizedElementsInputPCollectionId = splitAndSizeOutId;
+        if (isDrain()) {
+          String truncateAndSizeCoderId =
+              generateUniqueId(
+                  mainInputPCollection.getCoderId() + "/TruncateAndSize",
+                  existingComponents::containsCoders);
+          rval.getComponentsBuilder()
+              .putCoders(
+                  truncateAndSizeCoderId,
+                  ModelCoders.kvCoder(
+                      splitAndSizeOutCoderId, getOrAddDoubleCoder(existingComponents, rval)));
+          String truncateAndSizeOutId =
+              generateUniqueId(
+                  mainInputPCollectionId + "/TruncateAndSize",
+                  existingComponents::containsPcollections);
+
+          rval.getComponentsBuilder()
+              .putPcollections(
+                  truncateAndSizeOutId,
+                  PCollection.newBuilder()
+                      .setCoderId(truncateAndSizeCoderId)
+                      .setIsBounded(mainInputPCollection.getIsBounded())
+                      .setWindowingStrategyId(mainInputPCollection.getWindowingStrategyId())
+                      .setUniqueName(
+                          generateUniquePCollectonName(
+                              mainInputPCollection.getUniqueName() + "/TruncateAndSize",
+                              existingComponents))
+                      .build());
+          String truncateAndSizeId =
+              generateUniqueId(
+                  transformId + "/TruncateAndSize", existingComponents::containsTransforms);
+          {
+            PTransform.Builder truncateAndSize = PTransform.newBuilder();
+            truncateAndSize.putInputs(mainInputName, splitAndSizeOutId);
+            truncateAndSize.putAllInputs(sideInputs);
+            truncateAndSize.putOutputs("out", truncateAndSizeOutId);
+            truncateAndSize.setUniqueName(
+                generateUniquePCollectonName(
+                    splittableParDo.getUniqueName() + "/TruncateAndSize", existingComponents));
+            truncateAndSize.setSpec(
+                FunctionSpec.newBuilder()
+                    .setUrn(PTransformTranslation.SPLITTABLE_TRUNCATE_SIZED_RESTRICTION_URN)
+                    .setPayload(splittableParDo.getSpec().getPayload()));
+            truncateAndSize.setEnvironmentId(splittableParDo.getEnvironmentId());
+            rval.getComponentsBuilder().putTransforms(truncateAndSizeId, truncateAndSize.build());
+          }
+          newCompositeRoot.addSubtransforms(truncateAndSizeId);
+          processSizedElementsInputPCollectionId = truncateAndSizeOutId;
+        }
         {
           PTransform.Builder processSizedElementsAndRestrictions = PTransform.newBuilder();
-          processSizedElementsAndRestrictions.putInputs(mainInputName, splitAndSizeOutId);
+          processSizedElementsAndRestrictions.putInputs(
+              mainInputName, processSizedElementsInputPCollectionId);
           processSizedElementsAndRestrictions.putAllInputs(sideInputs);
           processSizedElementsAndRestrictions.putAllOutputs(splittableParDo.getOutputsMap());
           processSizedElementsAndRestrictions.setUniqueName(
@@ -234,20 +304,8 @@
                   processSizedElementsAndRestrictionsId,
                   processSizedElementsAndRestrictions.build());
         }
-
-        PTransform.Builder newCompositeRoot =
-            splittableParDo
-                .toBuilder()
-                // Clear the original splittable ParDo spec and add all the new transforms as
-                // children.
-                .clearSpec()
-                .addAllSubtransforms(
-                    Arrays.asList(
-                        pairWithRestrictionId,
-                        splitAndSizeId,
-                        processSizedElementsAndRestrictionsId));
+        newCompositeRoot.addSubtransforms(processSizedElementsAndRestrictionsId);
         rval.setPtransform(newCompositeRoot);
-
         return rval.build();
       } catch (IOException e) {
         throw new RuntimeException("Unable to perform expansion for transform " + transformId, e);
@@ -298,211 +356,4 @@
     }
     return prefix + i;
   }
-
-  /** See {@link #createTruncateReplacement} for details. */
-  private static class TruncateReplacement implements TransformReplacement {
-    private static final TruncateReplacement INSTANCE = new TruncateReplacement();
-
-    @Override
-    public MessageWithComponents getReplacement(
-        String transformId, ComponentsOrBuilder existingComponents) {
-      try {
-        MessageWithComponents.Builder rval = MessageWithComponents.newBuilder();
-
-        PTransform splittableParDo = existingComponents.getTransformsOrThrow(transformId);
-        ParDoPayload payload = ParDoPayload.parseFrom(splittableParDo.getSpec().getPayload());
-        // Only perform the expansion if this is a splittable DoFn.
-        if (payload.getRestrictionCoderId() == null || payload.getRestrictionCoderId().isEmpty()) {
-          return null;
-        }
-
-        String mainInputName = ParDoTranslation.getMainInputName(splittableParDo);
-        String mainInputPCollectionId = splittableParDo.getInputsOrThrow(mainInputName);
-        PCollection mainInputPCollection =
-            existingComponents.getPcollectionsOrThrow(mainInputPCollectionId);
-        Map<String, String> sideInputs =
-            Maps.filterKeys(
-                splittableParDo.getInputsMap(), input -> payload.containsSideInputs(input));
-
-        String pairWithRestrictionOutCoderId =
-            generateUniqueId(
-                mainInputPCollection.getCoderId() + "/PairWithRestriction",
-                existingComponents::containsCoders);
-        rval.getComponentsBuilder()
-            .putCoders(
-                pairWithRestrictionOutCoderId,
-                ModelCoders.kvCoder(
-                    mainInputPCollection.getCoderId(), payload.getRestrictionCoderId()));
-
-        String pairWithRestrictionOutId =
-            generateUniqueId(
-                mainInputPCollectionId + "/PairWithRestriction",
-                existingComponents::containsPcollections);
-        rval.getComponentsBuilder()
-            .putPcollections(
-                pairWithRestrictionOutId,
-                PCollection.newBuilder()
-                    .setCoderId(pairWithRestrictionOutCoderId)
-                    .setIsBounded(mainInputPCollection.getIsBounded())
-                    .setWindowingStrategyId(mainInputPCollection.getWindowingStrategyId())
-                    .setUniqueName(
-                        generateUniquePCollectonName(
-                            mainInputPCollection.getUniqueName() + "/PairWithRestriction",
-                            existingComponents))
-                    .build());
-
-        String splitAndSizeOutCoderId =
-            generateUniqueId(
-                mainInputPCollection.getCoderId() + "/SplitAndSize",
-                existingComponents::containsCoders);
-        rval.getComponentsBuilder()
-            .putCoders(
-                splitAndSizeOutCoderId,
-                ModelCoders.kvCoder(
-                    pairWithRestrictionOutCoderId, getOrAddDoubleCoder(existingComponents, rval)));
-
-        String splitAndSizeOutId =
-            generateUniqueId(
-                mainInputPCollectionId + "/SplitAndSize", existingComponents::containsPcollections);
-        rval.getComponentsBuilder()
-            .putPcollections(
-                splitAndSizeOutId,
-                PCollection.newBuilder()
-                    .setCoderId(splitAndSizeOutCoderId)
-                    .setIsBounded(mainInputPCollection.getIsBounded())
-                    .setWindowingStrategyId(mainInputPCollection.getWindowingStrategyId())
-                    .setUniqueName(
-                        generateUniquePCollectonName(
-                            mainInputPCollection.getUniqueName() + "/SplitAndSize",
-                            existingComponents))
-                    .build());
-
-        String truncateAndSizeCoderId =
-            generateUniqueId(
-                mainInputPCollection.getCoderId() + "/TruncateAndSize",
-                existingComponents::containsCoders);
-        rval.getComponentsBuilder()
-            .putCoders(
-                truncateAndSizeCoderId,
-                ModelCoders.kvCoder(
-                    splitAndSizeOutCoderId, getOrAddDoubleCoder(existingComponents, rval)));
-        String truncateAndSizeOutId =
-            generateUniqueId(
-                mainInputPCollectionId + "/TruncateAndSize",
-                existingComponents::containsPcollections);
-
-        rval.getComponentsBuilder()
-            .putPcollections(
-                truncateAndSizeOutId,
-                PCollection.newBuilder()
-                    .setCoderId(truncateAndSizeCoderId)
-                    .setIsBounded(mainInputPCollection.getIsBounded())
-                    .setWindowingStrategyId(mainInputPCollection.getWindowingStrategyId())
-                    .setUniqueName(
-                        generateUniquePCollectonName(
-                            mainInputPCollection.getUniqueName() + "/TruncateAndSize",
-                            existingComponents))
-                    .build());
-
-        String pairWithRestrictionId =
-            generateUniqueId(
-                transformId + "/PairWithRestriction", existingComponents::containsTransforms);
-        {
-          PTransform.Builder pairWithRestriction = PTransform.newBuilder();
-          pairWithRestriction.putAllInputs(splittableParDo.getInputsMap());
-          pairWithRestriction.putOutputs("out", pairWithRestrictionOutId);
-          pairWithRestriction.setUniqueName(
-              generateUniquePCollectonName(
-                  splittableParDo.getUniqueName() + "/PairWithRestriction", existingComponents));
-          pairWithRestriction.setSpec(
-              FunctionSpec.newBuilder()
-                  .setUrn(PTransformTranslation.SPLITTABLE_PAIR_WITH_RESTRICTION_URN)
-                  .setPayload(splittableParDo.getSpec().getPayload()));
-          pairWithRestriction.setEnvironmentId(splittableParDo.getEnvironmentId());
-          rval.getComponentsBuilder()
-              .putTransforms(pairWithRestrictionId, pairWithRestriction.build());
-        }
-
-        String splitAndSizeId =
-            generateUniqueId(transformId + "/SplitAndSize", existingComponents::containsTransforms);
-        {
-          PTransform.Builder splitAndSize = PTransform.newBuilder();
-          splitAndSize.putInputs(mainInputName, pairWithRestrictionOutId);
-          splitAndSize.putAllInputs(sideInputs);
-          splitAndSize.putOutputs("out", splitAndSizeOutId);
-          splitAndSize.setUniqueName(
-              generateUniquePCollectonName(
-                  splittableParDo.getUniqueName() + "/SplitAndSize", existingComponents));
-          splitAndSize.setSpec(
-              FunctionSpec.newBuilder()
-                  .setUrn(PTransformTranslation.SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN)
-                  .setPayload(splittableParDo.getSpec().getPayload()));
-          splitAndSize.setEnvironmentId(splittableParDo.getEnvironmentId());
-          rval.getComponentsBuilder().putTransforms(splitAndSizeId, splitAndSize.build());
-        }
-
-        String truncateAndSizeId =
-            generateUniqueId(
-                transformId + "/TruncateAndSize", existingComponents::containsTransforms);
-        {
-          PTransform.Builder truncateAndSize = PTransform.newBuilder();
-          truncateAndSize.putInputs(mainInputName, splitAndSizeOutId);
-          truncateAndSize.putAllInputs(sideInputs);
-          truncateAndSize.putOutputs("out", truncateAndSizeOutId);
-          truncateAndSize.setUniqueName(
-              generateUniquePCollectonName(
-                  splittableParDo.getUniqueName() + "/TruncateAndSize", existingComponents));
-          truncateAndSize.setSpec(
-              FunctionSpec.newBuilder()
-                  .setUrn(PTransformTranslation.SPLITTABLE_TRUNCATE_SIZED_RESTRICTION_URN)
-                  .setPayload(splittableParDo.getSpec().getPayload()));
-          truncateAndSize.setEnvironmentId(splittableParDo.getEnvironmentId());
-          rval.getComponentsBuilder().putTransforms(truncateAndSizeId, truncateAndSize.build());
-        }
-
-        String processSizedElementsAndRestrictionsId =
-            generateUniqueId(
-                transformId + "/ProcessSizedElementsAndRestrictions",
-                existingComponents::containsTransforms);
-        {
-          PTransform.Builder processSizedElementsAndRestrictions = PTransform.newBuilder();
-          processSizedElementsAndRestrictions.putInputs(mainInputName, truncateAndSizeOutId);
-          processSizedElementsAndRestrictions.putAllInputs(sideInputs);
-          processSizedElementsAndRestrictions.putAllOutputs(splittableParDo.getOutputsMap());
-          processSizedElementsAndRestrictions.setUniqueName(
-              generateUniquePCollectonName(
-                  splittableParDo.getUniqueName() + "/ProcessSizedElementsAndRestrictions",
-                  existingComponents));
-          processSizedElementsAndRestrictions.setSpec(
-              FunctionSpec.newBuilder()
-                  .setUrn(
-                      PTransformTranslation.SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN)
-                  .setPayload(splittableParDo.getSpec().getPayload()));
-          processSizedElementsAndRestrictions.setEnvironmentId(splittableParDo.getEnvironmentId());
-          rval.getComponentsBuilder()
-              .putTransforms(
-                  processSizedElementsAndRestrictionsId,
-                  processSizedElementsAndRestrictions.build());
-        }
-
-        PTransform.Builder newCompositeRoot =
-            splittableParDo
-                .toBuilder()
-                // Clear the original splittable ParDo spec and add all the new transforms as
-                // children.
-                .clearSpec()
-                .addAllSubtransforms(
-                    Arrays.asList(
-                        pairWithRestrictionId,
-                        splitAndSizeId,
-                        truncateAndSizeId,
-                        processSizedElementsAndRestrictionsId));
-        rval.setPtransform(newCompositeRoot);
-
-        return rval.build();
-      } catch (IOException e) {
-        throw new RuntimeException("Unable to perform expansion for transform " + transformId, e);
-      }
-    }
-  }
 }
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
index 988921e..f7c74c0 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
@@ -337,26 +337,30 @@
      */
     @Override
     public State waitUntilFinish(Duration duration) {
-      State startState = this.state;
-      if (!startState.isTerminal()) {
-        try {
-          state = executor.waitUntilFinish(duration);
-        } catch (UserCodeException uce) {
-          // Emulates the behavior of Pipeline#run(), where a stack trace caused by a
-          // UserCodeException is truncated and replaced with the stack starting at the call to
-          // waitToFinish
-          throw new Pipeline.PipelineExecutionException(uce.getCause());
-        } catch (Exception e) {
-          if (e instanceof InterruptedException) {
-            Thread.currentThread().interrupt();
-          }
-          if (e instanceof RuntimeException) {
-            throw (RuntimeException) e;
-          }
-          throw new RuntimeException(e);
-        }
+      if (this.state.isTerminal()) {
+        return this.state;
       }
-      return this.state;
+      final State endState;
+      try {
+        endState = executor.waitUntilFinish(duration);
+      } catch (UserCodeException uce) {
+        // Emulates the behavior of Pipeline#run(), where a stack trace caused by a
+        // UserCodeException is truncated and replaced with the stack starting at the call to
+        // waitToFinish
+        throw new Pipeline.PipelineExecutionException(uce.getCause());
+      } catch (Exception e) {
+        if (e instanceof InterruptedException) {
+          Thread.currentThread().interrupt();
+        }
+        if (e instanceof RuntimeException) {
+          throw (RuntimeException) e;
+        }
+        throw new RuntimeException(e);
+      }
+      if (endState != null) {
+        this.state = endState;
+      }
+      return endState;
     }
   }
 
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
index 8054a07..fbcf0c0 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
@@ -328,8 +328,9 @@
     // The pipeline should never complete;
     assertThat(result.getState(), is(State.RUNNING));
     // Must time out, otherwise this test will never complete
-    result.waitUntilFinish(Duration.millis(1L));
-    assertEquals(null, result.getState());
+    assertEquals(null, result.waitUntilFinish(Duration.millis(1L)));
+    // Ensure multiple calls complete
+    assertEquals(null, result.waitUntilFinish(Duration.millis(1L)));
   }
 
   private static final AtomicLong TEARDOWN_CALL = new AtomicLong(-1);
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/KeyedTimerData.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/KeyedTimerData.java
index a7e364d..46dc95a 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/KeyedTimerData.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/KeyedTimerData.java
@@ -99,6 +99,18 @@
   }
 
   @Override
+  public String toString() {
+    return "KeyedTimerData{"
+        + "key="
+        + key
+        + ", keyBytes="
+        + Arrays.toString(keyBytes)
+        + ", timerData="
+        + timerData
+        + '}';
+  }
+
+  @Override
   public boolean equals(@Nullable Object o) {
     if (this == o) {
       return true;
diff --git a/sdks/go/pkg/beam/core/runtime/exec/datasource.go b/sdks/go/pkg/beam/core/runtime/exec/datasource.go
index 054e34f..ad7b601 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/datasource.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/datasource.go
@@ -16,6 +16,7 @@
 package exec
 
 import (
+	"bytes"
 	"context"
 	"fmt"
 	"io"
@@ -25,7 +26,6 @@
 	"time"
 
 	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder"
-	"github.com/apache/beam/sdks/go/pkg/beam/core/sdf"
 	"github.com/apache/beam/sdks/go/pkg/beam/core/util/ioutilx"
 	"github.com/apache/beam/sdks/go/pkg/beam/internal/errors"
 	"github.com/apache/beam/sdks/go/pkg/beam/log"
@@ -47,20 +47,24 @@
 	splitIdx  int64
 	start     time.Time
 
-	// rt is non-nil if this DataSource feeds directly to a splittable unit,
-	// and receives the current restriction tracker being processed.
-	rt chan sdf.RTracker
+	// su is non-nil if this DataSource feeds directly to a splittable unit,
+	// and receives that splittable unit when it is available for splitting.
+	// While the splittable unit is received, it is blocked from processing
+	// new elements, so it must be sent back through the channel once the
+	// DataSource is finished using it.
+	su chan SplittableUnit
 
 	mu sync.Mutex
 }
 
-// Initializes the rt channel from the following unit when applicable.
+// InitSplittable initializes the SplittableUnit channel from the output unit,
+// if it provides one.
 func (n *DataSource) InitSplittable() {
 	if n.Out == nil {
 		return
 	}
 	if u, ok := n.Out.(*ProcessSizedElementsAndRestrictions); ok == true {
-		n.rt = u.Rt
+		n.su = u.SU
 	}
 }
 
@@ -284,16 +288,21 @@
 
 // Split takes a sorted set of potential split indices and a fraction of the
 // remainder to split at, selects and actuates a split on an appropriate split
-// index, and returns the selected split index if successful or an error when
-// unsuccessful.
+// index, and returns the selected split index in a SplitResult if successful or
+// an error when unsuccessful.
+//
+// If the following transform is splittable, and the split indices and fraction
+// allow for splitting on the currently processing element, then a sub-element
+// split is performed, and the appropriate information is returned in the
+// SplitResult.
 //
 // The bufSize param specifies the estimated number of elements that will be
 // sent to this DataSource, and is used to be able to perform accurate splits
 // even if the DataSource has not yet received all its elements. A bufSize of
 // 0 or less indicates that its unknown, and so uses the current known size.
-func (n *DataSource) Split(splits []int64, frac float64, bufSize int64) (int64, error) {
+func (n *DataSource) Split(splits []int64, frac float64, bufSize int64) (SplitResult, error) {
 	if n == nil {
-		return 0, fmt.Errorf("failed to split at requested splits: {%v}, DataSource not initialized", splits)
+		return SplitResult{}, fmt.Errorf("failed to split at requested splits: {%v}, DataSource not initialized", splits)
 	}
 	if frac > 1.0 {
 		frac = 1.0
@@ -302,31 +311,83 @@
 	}
 
 	n.mu.Lock()
+	defer n.mu.Unlock()
+
 	var currProg float64 // Current element progress.
-	if n.index < 0 {     // Progress is at the end of the non-existant -1st element.
+	var su SplittableUnit
+	if n.index < 0 { // Progress is at the end of the non-existant -1st element.
 		currProg = 1.0
-	} else if n.rt == nil { // If this isn't sub-element splittable, estimate some progress.
+	} else if n.su == nil { // If this isn't sub-element splittable, estimate some progress.
 		currProg = 0.5
 	} else { // If this is sub-element splittable, get progress of the current element.
-		rt := <-n.rt
-		d, r := rt.GetProgress()
-		currProg = d / (d + r)
-		n.rt <- rt
+
+		select {
+		case su = <-n.su:
+			// If an element is processing, we'll get a splittable unit.
+			if su == nil {
+				return SplitResult{}, fmt.Errorf("failed to split: splittable unit was nil")
+			}
+			defer func() {
+				n.su <- su
+			}()
+			currProg = su.GetProgress()
+		case <-time.After(500 * time.Millisecond):
+			// Otherwise, the current element hasn't started processing yet
+			// or has already finished. By adding a short timeout, we avoid
+			// the first possibility, and can assume progress is at max.
+			currProg = 1.0
+		}
 	}
 	// Size to split within is the minimum of bufSize or splitIdx so we avoid
 	// including elements we already know won't be processed.
 	if bufSize <= 0 || n.splitIdx < bufSize {
 		bufSize = n.splitIdx
 	}
-	s, _, err := splitHelper(n.index, bufSize, currProg, splits, frac, false)
+	s, f, err := splitHelper(n.index, bufSize, currProg, splits, frac, su != nil)
 	if err != nil {
-		n.mu.Unlock()
-		return 0, err
+		return SplitResult{}, err
 	}
-	n.splitIdx = s
-	fs := n.splitIdx
-	n.mu.Unlock()
-	return fs, nil
+
+	// No fraction returned, perform channel split.
+	if f < 0 {
+		n.splitIdx = s
+		return SplitResult{PI: s - 1, RI: s}, nil
+	}
+	// Otherwise, perform a sub-element split.
+	fr := f / (1.0 - currProg)
+	p, r, err := su.Split(fr)
+	if err != nil {
+		return SplitResult{}, err
+	}
+
+	if p == nil || r == nil { // Unsuccessful split.
+		// Fallback to channel split, so split at next elm, not current.
+		n.splitIdx = s + 1
+		return SplitResult{PI: s, RI: s + 1}, nil
+	}
+
+	// TODO(BEAM-10579) Eventually encode elements with the splittable
+	// unit's input coder instead of the DataSource's coder.
+	wc := MakeWindowEncoder(n.Coder.Window)
+	ec := MakeElementEncoder(coder.SkipW(n.Coder))
+	pEnc, err := encodeElm(p, wc, ec)
+	if err != nil {
+		return SplitResult{}, err
+	}
+	rEnc, err := encodeElm(r, wc, ec)
+	if err != nil {
+		return SplitResult{}, err
+	}
+	n.splitIdx = s + 1 // In a sub-element split, s is currIdx.
+	res := SplitResult{
+		PI:   s - 1,
+		RI:   s + 1,
+		PS:   pEnc,
+		RS:   rEnc,
+		TId:  su.GetTransformId(),
+		InId: su.GetInputId(),
+	}
+	return res, nil
 }
 
 // splitHelper is a helper function that finds a split point in a range.
@@ -348,7 +409,8 @@
 // Returns the element index to split at (first element of residual), and the
 // fraction within that element to split, iff the split point is the current
 // element, the splittable param is set to true, and both the element being
-// split and the following element are valid split points.
+// split and the following element are valid split points. If there is no
+// fraction, returns -1.
 func splitHelper(
 	currIdx, endIdx int64,
 	currProg float64,
@@ -377,7 +439,7 @@
 		if splitIdx < safeStart {
 			splitIdx = safeStart
 		}
-		return splitIdx, 0.0, nil
+		return splitIdx, -1.0, nil
 	}
 
 	// Cases where we have to find a valid split point.
@@ -419,10 +481,21 @@
 		}
 	}
 	if bestS != -1 {
-		return bestS, 0.0, nil
+		return bestS, -1.0, nil
 	}
 
-	return -1, 0.0, fmt.Errorf("failed to split DataSource (at index: %v) at requested splits: {%v}", currIdx, splits)
+	return -1, -1.0, fmt.Errorf("failed to split DataSource (at index: %v) at requested splits: {%v}", currIdx, splits)
+}
+
+func encodeElm(elm *FullValue, wc WindowEncoder, ec ElementEncoder) ([]byte, error) {
+	var b bytes.Buffer
+	if err := EncodeWindowedValueHeader(wc, elm.Windows, elm.Timestamp, &b); err != nil {
+		return nil, err
+	}
+	if err := ec.Encode(elm, &b); err != nil {
+		return nil, err
+	}
+	return b.Bytes(), nil
 }
 
 type concatReStream struct {
diff --git a/sdks/go/pkg/beam/core/runtime/exec/datasource_test.go b/sdks/go/pkg/beam/core/runtime/exec/datasource_test.go
index ff6126e..4089a1c 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/datasource_test.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/datasource_test.go
@@ -25,6 +25,7 @@
 	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder"
 	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
 	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/window"
+	"github.com/apache/beam/sdks/go/pkg/beam/internal/errors"
 )
 
 func TestDataSource_PerElement(t *testing.T) {
@@ -278,11 +279,17 @@
 			runOnRoots(ctx, t, p, "StartBundle", func(root Root, ctx context.Context) error { return root.StartBundle(ctx, "1", dc) })
 
 			// SDK never splits on 0, so check that every test.
-			if splitIdx, err := p.Split(SplitPoints{Splits: []int64{0, test.splitIdx}}); err != nil {
+			splitRes, err := p.Split(SplitPoints{Splits: []int64{0, test.splitIdx}})
+			if err != nil {
 				t.Fatalf("error in Split: %v", err)
-			} else if got, want := splitIdx, test.splitIdx; got != want {
+			}
+			if got, want := splitRes.RI, test.splitIdx; got != want {
 				t.Fatalf("error in Split: got splitIdx = %v, want %v ", got, want)
 			}
+			if got, want := splitRes.PI, test.splitIdx-1; got != want {
+				t.Fatalf("error in Split: got primary index = %v, want %v ", got, want)
+			}
+
 			runOnRoots(ctx, t, p, "Process", Root.Process)
 			runOnRoots(ctx, t, p, "FinishBundle", Root.FinishBundle)
 
@@ -342,10 +349,15 @@
 					<-blockedCh
 					// Validate that we do not split on the element we're blocking on index.
 					// The first valid split is at test.splitIdx.
-					if splitIdx, err := source.Split([]int64{0, 1, 2, 3, 4, 5}, -1, 0); err != nil {
+					if splitRes, err := source.Split([]int64{0, 1, 2, 3, 4, 5}, -1, 0); err != nil {
 						t.Errorf("error in Split: %v", err)
-					} else if got, want := splitIdx, test.splitIdx; got != want {
-						t.Errorf("error in Split: got splitIdx = %v, want %v ", got, want)
+					} else {
+						if got, want := splitRes.RI, test.splitIdx; got != want {
+							t.Errorf("error in Split: got splitIdx = %v, want %v ", got, want)
+						}
+						if got, want := splitRes.PI, test.splitIdx-1; got != want {
+							t.Errorf("error in Split: got primary index = %v, want %v ", got, want)
+						}
 					}
 					// Validate that our progress is where we expect it to be. (test.splitIdx - 1)
 					if got, want := source.Progress().Count, test.splitIdx-1; got != want {
@@ -412,17 +424,120 @@
 
 		// SDK never splits on 0, so check that every test.
 		sp := SplitPoints{Splits: test.splitPts, Frac: test.frac, BufSize: test.bufSize}
-		if splitIdx, err := p.Split(sp); err != nil {
+		splitRes, err := p.Split(sp)
+		if err != nil {
 			t.Fatalf("error in Split: %v", err)
-		} else if got, want := splitIdx, test.splitIdx; got != want {
+		}
+		if got, want := splitRes.RI, test.splitIdx; got != want {
 			t.Fatalf("error in Split: got splitIdx = %v, want %v ", got, want)
 		}
+		if got, want := splitRes.PI, test.splitIdx-1; got != want {
+			t.Fatalf("error in Split: got primary index = %v, want %v ", got, want)
+		}
 		runOnRoots(ctx, t, p, "Process", Root.Process)
 		runOnRoots(ctx, t, p, "FinishBundle", Root.FinishBundle)
 
 		validateSource(t, out, source, makeValues(test.expected...))
 	})
 
+	// Test splitting on sub-elements works when available.
+	t.Run("subElement", func(t *testing.T) {
+		// Each test will process up to an element, then split at different
+		// fractions and check that a sub-element split either was, or was not
+		// performed.
+		const blockOn int64 = 3 // Should leave 2 elements unprocessed, including blocked element.
+		numElms := int64(len(elements))
+		tests := []struct {
+			fraction float64
+			splitIdx int64
+			isSubElm bool
+		}{
+			{fraction: 0.0, splitIdx: blockOn + 1, isSubElm: true},
+			{fraction: 0.01, splitIdx: blockOn + 1, isSubElm: true},
+			{fraction: 0.49, splitIdx: blockOn + 1, isSubElm: true},  // Should be just within current element.
+			{fraction: 0.51, splitIdx: blockOn + 1, isSubElm: false}, // Should be just past current element.
+			{fraction: 0.99, splitIdx: numElms, isSubElm: false},
+		}
+		for _, test := range tests {
+			test := test
+			name := fmt.Sprintf("withFraction_%v", test.fraction)
+			t.Run(name, func(t *testing.T) {
+				source, out, pr := initSourceTest(name)
+				unblockCh, blockedCh := make(chan struct{}), make(chan struct{}, 1)
+				// Block on the one less than the desired split,
+				// so the desired split is the first valid split.
+				blocker := &BlockingNode{
+					UID: 3,
+					Block: func(elm *FullValue) bool {
+						if source.index == blockOn {
+							// Signal to call Split
+							blockedCh <- struct{}{}
+							return true
+						}
+						return false
+					},
+					Unblock: unblockCh,
+					Out:     out,
+				}
+				source.Out = blocker
+
+				splittableCh := make(chan SplittableUnit, 1)
+				source.su = splittableCh
+				splittableCh <- &TestSplittableUnit{elm: elements[blockOn]}
+
+				go func() {
+					// Wait to call Split until the DoFn is blocked at the desired element.
+					<-blockedCh
+					// Validate that we either do or do not perform a sub-element split with the
+					// given fraction.
+					if splitRes, err := source.Split([]int64{0, 1, 2, 3, 4, 5}, test.fraction, int64(len(elements))); err != nil {
+						t.Errorf("error in Split: %v", err)
+					} else {
+						// For sub-element splits, check sub-element split only results.
+						isSubElm := splitRes.RS != nil && splitRes.PS != nil
+						if isSubElm != test.isSubElm {
+							t.Errorf("error in Split: got sub-element split = %t, want %t", isSubElm, test.isSubElm)
+						}
+						if isSubElm {
+							if got, want := splitRes.TId, testTransformId; got != want {
+								t.Errorf("error in Split: got incorrect Transform Id = %v, want %v", got, want)
+							}
+							if got, want := splitRes.InId, testInputId; got != want {
+								t.Errorf("error in Split: got incorrect Input Id = %v, want %v", got, want)
+							}
+						}
+
+						// Check that split indices are correct, for both sub-element and channel splits.
+						var wantPI, wantRI = test.splitIdx - 1, test.splitIdx
+						if isSubElm {
+							// In sub-element splits, primary index is expected to be one element
+							// before the current (split) element.
+							wantPI--
+						}
+						if splitRes.PI != wantPI || splitRes.RI != wantRI {
+							t.Errorf("error in Split: got split indices of (primary, residual) = (%d, %d), want (%d, %d)",
+								splitRes.PI, splitRes.RI, wantPI, wantRI)
+						}
+					}
+					// Validate that our progress is where we expect it to be. (blockOn)
+					if got, want := source.Progress().Count, blockOn; got != want {
+						t.Errorf("error in Progress: got finished processing Count = %v, want %v ", got, want)
+					}
+					unblockCh <- struct{}{}
+				}()
+
+				constructAndExecutePlanWithContext(t, []Unit{out, blocker, source}, DataContext{
+					Data: &TestDataManager{R: pr},
+				})
+
+				validateSource(t, out, source, makeValues(elements[:test.splitIdx]...))
+				if got, want := source.Progress().Count, test.splitIdx; got != want {
+					t.Fatalf("progress didn't match split: got %v, want %v", got, want)
+				}
+			})
+		}
+	})
+
 	// Test expects splitting errors, but for processing to be successful.
 	t.Run("errors", func(t *testing.T) {
 		source, out, pr := initSourceTest("noSplitsUntilStarted")
@@ -471,6 +586,39 @@
 	})
 }
 
+const testTransformId = "transform_id"
+const testInputId = "input_id"
+
+// TestSplittableUnit is an implementation of the SplittableUnit interface
+// for DataSource tests.
+type TestSplittableUnit struct {
+	elm interface{} // The element to split.
+}
+
+// Split checks the input fraction for correctness, but otherwise always returns
+// a successful split. The split elements are just copies of the original.
+func (n *TestSplittableUnit) Split(f float64) (*FullValue, *FullValue, error) {
+	if f > 1.0 || f < 0.0 {
+		return nil, nil, errors.Errorf("Error")
+	}
+	return &FullValue{Elm: n.elm}, &FullValue{Elm: n.elm}, nil
+}
+
+// GetProgress always returns 0, to keep tests consistent.
+func (n *TestSplittableUnit) GetProgress() float64 {
+	return 0
+}
+
+// GetTransformId returns a constant transform ID that can be tested for.
+func (n *TestSplittableUnit) GetTransformId() string {
+	return testTransformId
+}
+
+// GetInputId returns a constant input ID that can be tested for.
+func (n *TestSplittableUnit) GetInputId() string {
+	return testInputId
+}
+
 func floatEquals(a, b, epsilon float64) bool {
 	return math.Abs(a-b) < epsilon
 }
@@ -501,7 +649,7 @@
 		for _, test := range tests {
 			test := test
 			t.Run(fmt.Sprintf("(%v of [%v, %v])", test.frac, test.curr, test.size), func(t *testing.T) {
-				wantFrac := 0.0
+				wantFrac := -1.0
 				got, gotFrac, err := splitHelper(test.curr, test.size, 0.0, nil, test.frac, false)
 				if err != nil {
 					t.Fatalf("error in splitHelper: %v", err)
@@ -533,7 +681,7 @@
 		for _, test := range tests {
 			test := test
 			t.Run(fmt.Sprintf("(%v of [%v, %v])", test.frac, float64(test.curr)+test.currProg, test.size), func(t *testing.T) {
-				wantFrac := 0.0
+				wantFrac := -1.0
 				got, gotFrac, err := splitHelper(test.curr, test.size, test.currProg, nil, test.frac, false)
 				if err != nil {
 					t.Fatalf("error in splitHelper: %v", err)
@@ -572,7 +720,7 @@
 		for _, test := range tests {
 			test := test
 			t.Run(fmt.Sprintf("(%v of [%v, %v], splits = %v)", test.frac, test.curr, test.size, test.splits), func(t *testing.T) {
-				wantFrac := 0.0
+				wantFrac := -1.0
 				got, gotFrac, err := splitHelper(test.curr, test.size, 0.0, test.splits, test.frac, false)
 				if test.err {
 					if err == nil {
@@ -602,10 +750,10 @@
 			wantFrac   float64
 		}{
 			// Split between future elements at element boundaries.
-			{curr: 0, currProg: 0, size: 4, frac: 0.51, want: 2},
-			{curr: 0, currProg: 0, size: 4, frac: 0.49, want: 2},
-			{curr: 0, currProg: 0, size: 4, frac: 0.26, want: 1},
-			{curr: 0, currProg: 0, size: 4, frac: 0.25, want: 1},
+			{curr: 0, currProg: 0, size: 4, frac: 0.51, want: 2, wantFrac: -1.0},
+			{curr: 0, currProg: 0, size: 4, frac: 0.49, want: 2, wantFrac: -1.0},
+			{curr: 0, currProg: 0, size: 4, frac: 0.26, want: 1, wantFrac: -1.0},
+			{curr: 0, currProg: 0, size: 4, frac: 0.25, want: 1, wantFrac: -1.0},
 
 			// If the split falls inside the first, splittable element, split there.
 			{curr: 0, currProg: 0, size: 4, frac: 0.20, want: 0, wantFrac: 0.8},
@@ -613,11 +761,11 @@
 			{curr: 0, currProg: 0, size: 4, frac: 0.125, want: 0, wantFrac: 0.5},
 			// Here we are far enough into the first element that splitting at 0.2 of the
 			// remainder falls outside the first element.
-			{curr: 0, currProg: 0.5, size: 4, frac: 0.2, want: 1},
+			{curr: 0, currProg: 0.5, size: 4, frac: 0.2, want: 1, wantFrac: -1.0},
 
 			// Verify the above logic when we are partially through the stream.
-			{curr: 2, currProg: 0, size: 4, frac: 0.6, want: 3},
-			{curr: 2, currProg: 0.9, size: 4, frac: 0.6, want: 4},
+			{curr: 2, currProg: 0, size: 4, frac: 0.6, want: 3, wantFrac: -1.0},
+			{curr: 2, currProg: 0.9, size: 4, frac: 0.6, want: 4, wantFrac: -1.0},
 			{curr: 2, currProg: 0.5, size: 4, frac: 0.2, want: 2, wantFrac: 0.8},
 		}
 		for _, test := range tests {
@@ -649,12 +797,12 @@
 			// This is where we would like to split, when all split points are available.
 			{curr: 2, currProg: 0, size: 5, frac: 0.2, splits: []int64{1, 2, 3, 4, 5}, want: 2, wantFrac: 0.6},
 			// We can't split element at index 2, because 3 is not a split point.
-			{curr: 2, currProg: 0, size: 5, frac: 0.2, splits: []int64{1, 2, 4, 5}, want: 4},
+			{curr: 2, currProg: 0, size: 5, frac: 0.2, splits: []int64{1, 2, 4, 5}, want: 4, wantFrac: -1.0},
 			// We can't even split element at index 4 as above, because 4 is also not a
 			// split point.
-			{curr: 2, currProg: 0, size: 5, frac: 0.2, splits: []int64{1, 2, 5}, want: 5},
+			{curr: 2, currProg: 0, size: 5, frac: 0.2, splits: []int64{1, 2, 5}, want: 5, wantFrac: -1.0},
 			// We can't split element at index 2, because 2 is not a split point.
-			{curr: 2, currProg: 0, size: 5, frac: 0.2, splits: []int64{1, 3, 4, 5}, want: 3},
+			{curr: 2, currProg: 0, size: 5, frac: 0.2, splits: []int64{1, 3, 4, 5}, want: 3, wantFrac: -1.0},
 		}
 		for _, test := range tests {
 			test := test
diff --git a/sdks/go/pkg/beam/core/runtime/exec/dynsplit_test.go b/sdks/go/pkg/beam/core/runtime/exec/dynsplit_test.go
new file mode 100644
index 0000000..3b17a0a
--- /dev/null
+++ b/sdks/go/pkg/beam/core/runtime/exec/dynsplit_test.go
@@ -0,0 +1,456 @@
+// 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 exec
+
+import (
+	"bytes"
+	"context"
+	"io"
+	"reflect"
+	"sync"
+	"testing"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/window"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/typex"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	"github.com/apache/beam/sdks/go/pkg/beam/internal/errors"
+	"github.com/apache/beam/sdks/go/pkg/beam/io/rtrackers/offsetrange"
+	"github.com/google/go-cmp/cmp"
+	"github.com/google/go-cmp/cmp/cmpopts"
+)
+
+// TestDynamicSplit tests that a dynamic split of an in-progress SDF succeeds
+// with valid input. It coordinates the two threads (processing and splitting)
+// to test what happens if operations happen in various orders. The test then
+// validates that the output of the SDF is correct according to the split.
+func TestDynamicSplit(t *testing.T) {
+	tests := []struct {
+		name string
+		// driver is a function determining how the processing and splitting
+		// threads are created and coordinated.
+		driver func(*Plan, DataContext, *splitTestSdf) (error, splitResult)
+	}{
+		{
+			// Complete a split before beginning processing.
+			name:   "Simple",
+			driver: nonBlockingDriver,
+		},
+		{
+			// Try claiming while blocked on a split.
+			name:   "BlockOnSplit",
+			driver: splitBlockingDriver,
+		},
+		{
+			// Try splitting while blocked on a claim.
+			name:   "BlockOnClaim",
+			driver: claimBlockingDriver,
+		},
+	}
+	for _, test := range tests {
+		test := test
+		t.Run(test.name, func(t *testing.T) {
+			// Create pipeline.
+			sdf := newSplitTestSdf()
+			dfn, err := graph.NewDoFn(sdf, graph.NumMainInputs(graph.MainSingle))
+			if err != nil {
+				t.Fatalf("invalid function: %v", err)
+			}
+			cdr := createSplitTestInCoder()
+			plan, out := createSdfPlan(t, t.Name(), dfn, cdr)
+
+			// Create thread to send element to pipeline.
+			pr, pw := io.Pipe()
+			elm := createElm()
+			go writeElm(elm, cdr, pw)
+			dc := DataContext{Data: &TestDataManager{R: pr}}
+
+			// Call driver to coordinate processing & splitting threads.
+			procRes, splitRes := test.driver(plan, dc, sdf)
+
+			// Validate we get a valid split result, aside from split elements.
+			if splitRes.err != nil {
+				t.Fatalf("Plan.Split failed: %v", splitRes.err)
+			}
+			wantSplit := SplitResult{
+				PI:   -1,
+				RI:   1,
+				PS:   nil,
+				RS:   nil,
+				TId:  testTransformId,
+				InId: indexToInputId(0),
+			}
+			if diff := cmp.Diff(splitRes.split, wantSplit, cmpopts.IgnoreFields(SplitResult{}, "PS", "RS")); diff != "" {
+				t.Errorf("Incorrect split result (ignoring split elements): %v", diff)
+			}
+
+			// Validate split elements are encoded correctly by decoding them
+			// with the input coder to the path.
+			// TODO(BEAM-10579) Switch to using splittable unit's input coder
+			// once that is implemented.
+			p, err := decodeDynSplitElm(splitRes.split.PS, cdr)
+			if err != nil {
+				t.Errorf("Failed decoding primary element split: %v", err)
+			}
+			_, err = decodeDynSplitElm(splitRes.split.RS, cdr)
+			if err != nil {
+				t.Errorf("Failed decoding residual element split: %v", err)
+			}
+
+			// Validate SDF output. Make sure each restriction matches the split result.
+			if err := procRes; err != nil {
+				t.Fatal(err)
+			}
+			pRest := p.Elm.(*FullValue).Elm2.(offsetrange.Restriction)
+			if got, want := len(out.Elements), int(pRest.End-pRest.Start); got != want {
+				t.Errorf("Unexpected number of elements: got: %v, want: %v", got, want)
+			}
+			for i, fv := range out.Elements {
+				rest := fv.Elm.(offsetrange.Restriction)
+				if got, want := rest, pRest; !cmp.Equal(got, want) {
+					t.Errorf("Output element %v had incorrect restriction: got: %v, want: %v", i, got, want)
+				}
+			}
+		})
+	}
+}
+
+// nonBlockingDriver performs a split before starting processing, so no thread
+// is forced to wait on a mutex.
+func nonBlockingDriver(plan *Plan, dc DataContext, sdf *splitTestSdf) (procRes error, splitRes splitResult) {
+	// Begin processing pipeline.
+	procResCh := make(chan error)
+	go processPlan(plan, dc, procResCh)
+	rt := <-sdf.rt // Tracker is created first, retrieve that.
+
+	// Complete a split before unblocking processing.
+	splitResCh := make(chan splitResult)
+	go splitPlan(plan, splitResCh)
+	<-rt.split
+	<-rt.blockSplit
+	splitRes = <-splitResCh
+
+	// Unblock and finishing processing.
+	<-sdf.proc
+	<-rt.claim
+	<-rt.blockClaim
+	<-rt.endClaim
+	procRes = <-procResCh
+
+	return procRes, splitRes
+}
+
+// splitBlockingDriver blocks on a split request so that the SDF attempts to
+// claim while the split is occurring.
+func splitBlockingDriver(plan *Plan, dc DataContext, sdf *splitTestSdf) (procRes error, splitRes splitResult) {
+	// Begin processing pipeline.
+	procResCh := make(chan error)
+	go processPlan(plan, dc, procResCh)
+	rt := <-sdf.rt // Tracker is created first, retrieve that.
+
+	// Start a split, but block on it so it holds the mutex.
+	splitResCh := make(chan splitResult)
+	go splitPlan(plan, splitResCh)
+	<-rt.split
+
+	// Start processing and start a claim, that'll be waiting for the mutex.
+	<-sdf.proc
+	<-rt.claim
+
+	// Unblock and finish splitting and free the mutex.
+	<-rt.blockSplit
+	splitRes = <-splitResCh
+
+	// Unblock and finish claiming and processing.
+	<-rt.blockClaim
+	<-rt.endClaim
+	procRes = <-procResCh
+
+	return procRes, splitRes
+}
+
+// claimBlockingDriver blocks on a claim request so that the SDF attempts to
+// split while the claim is occurring.
+func claimBlockingDriver(plan *Plan, dc DataContext, sdf *splitTestSdf) (procRes error, splitRes splitResult) {
+	// Begin processing pipeline.
+	procResCh := make(chan error)
+	go processPlan(plan, dc, procResCh)
+	rt := <-sdf.rt // Tracker is created first, retrieve that.
+
+	// Start a claim, but block on it so it holds the mutex.
+	<-sdf.proc
+	<-rt.claim
+
+	// Start a split that'll be waiting for the mutex.
+	splitResCh := make(chan splitResult)
+	go splitPlan(plan, splitResCh)
+	<-rt.split
+
+	// Unblock the claim, freeing the mutex (but not finishing processing yet).
+	<-rt.blockClaim
+
+	// Finish splitting, allowing processing to finish.
+	<-rt.blockSplit
+	splitRes = <-splitResCh
+	<-rt.endClaim // Delay the claim end so we don't process too much before splitting.
+	procRes = <-procResCh
+
+	return procRes, splitRes
+}
+
+// createElm creates the element for our test pipeline.
+func createElm() *FullValue {
+	return &FullValue{
+		Elm: &FullValue{
+			Elm:  20,
+			Elm2: offsetrange.Restriction{Start: 0, End: 20},
+		},
+		Elm2: float64(20),
+	}
+}
+
+// createSplitTestInCoder outputs the coder for inputs to our test pipeline,
+// (in particular, the DataSource transform of the pipeline). For the specific
+// element this is a coder for, see createElm.
+func createSplitTestInCoder() *coder.Coder {
+	restT := reflect.TypeOf((*offsetrange.Restriction)(nil)).Elem()
+	restCdr := coder.LookupCustomCoder(restT)
+
+	cdr := coder.NewW(
+		coder.NewKV([]*coder.Coder{
+			coder.NewKV([]*coder.Coder{
+				intCoder(reflectx.Int),
+				{Kind: coder.Custom, T: typex.New(restT), Custom: restCdr},
+			}),
+			coder.NewDouble(),
+		}),
+		coder.NewGlobalWindow())
+	return cdr
+}
+
+// createSdfPlan creates a plan containing the test pipeline, which is
+// DataSource -> SDF.ProcessSizedElementsAndRestrictions -> CaptureNode.
+func createSdfPlan(t *testing.T, name string, fn *graph.DoFn, cdr *coder.Coder) (*Plan, *CaptureNode) {
+	out := &CaptureNode{UID: 0}
+	n := &ParDo{UID: 1, Fn: fn, Out: []Node{out}}
+	sdf := &ProcessSizedElementsAndRestrictions{PDo: n, TfId: testTransformId}
+	ds := &DataSource{
+		UID:   2,
+		SID:   StreamID{PtransformID: "DataSource"},
+		Name:  "name",
+		Coder: cdr,
+		Out:   sdf,
+	}
+	units := []Unit{ds, sdf, out}
+
+	p, err := NewPlan(name+"_plan", units)
+	if err != nil {
+		t.Fatalf("NewPlan failed: %v", err)
+	}
+	return p, out
+}
+
+// writeElm is meant to be the goroutine for feeding an element to the
+// DataSourc of the test pipeline.
+func writeElm(elm *FullValue, cdr *coder.Coder, pw *io.PipeWriter) {
+	wc := MakeWindowEncoder(cdr.Window)
+	ec := MakeElementEncoder(coder.SkipW(cdr))
+	if err := EncodeWindowedValueHeader(wc, window.SingleGlobalWindow, mtime.ZeroTimestamp, pw); err != nil {
+		panic("err")
+	}
+	if err := ec.Encode(elm, pw); err != nil {
+		panic("err")
+	}
+	if err := pw.Close(); err != nil {
+		panic("err")
+	}
+}
+
+func decodeDynSplitElm(elm []byte, cdr *coder.Coder) (*FullValue, error) {
+	wd := MakeWindowDecoder(cdr.Window)
+	ed := MakeElementDecoder(coder.SkipW(cdr))
+	b := bytes.NewBuffer(elm)
+	w, t, err := DecodeWindowedValueHeader(wd, b)
+	if err != nil {
+		return nil, err
+	}
+	e, err := ed.Decode(b)
+	if err != nil {
+		return nil, err
+	}
+	e.Windows = w
+	e.Timestamp = t
+	return e, nil
+}
+
+// processPlan is meant to be the goroutine representing the thread processing
+// the SDF.
+func processPlan(plan *Plan, dc DataContext, result chan error) {
+	if err := plan.Execute(context.Background(), plan.ID()+"_execute", dc); err != nil {
+		result <- errors.Wrap(err, "Plan.Execute failed")
+	}
+	if err := plan.Down(context.Background()); err != nil {
+		result <- errors.Wrap(err, "Plan.Down failed")
+	}
+	result <- nil
+}
+
+type splitResult struct {
+	split SplitResult
+	err   error
+}
+
+// splitPlan is meant to be the goroutine representing the thread handling a
+// split request for the SDF.
+func splitPlan(plan *Plan, result chan splitResult) {
+	split, err := plan.Split(SplitPoints{Frac: 0.5, BufSize: 1})
+	result <- splitResult{split: split, err: err}
+}
+
+// splitTestRTracker adds signals needed to coordinate splitting and claiming
+// over multiple threads for this test. Semantically, this tracker is an
+// offset range tracker representing a range of integers to output.
+type splitTestRTracker struct {
+	mu sync.Mutex // Lock on accessing underlying tracker.
+	rt *offsetrange.Tracker
+
+	// Send signals when starting a claim, blocking a claim, and ending a claim.
+	claim      chan struct{}
+	blockClaim chan struct{}
+	endClaim   chan struct{}
+	blockInd   int64 // Only send signals when claiming a specific position.
+
+	// Send signals when starting a split, and blocking a split. Important note:
+	// the spot to use these in this test is dependent on the first operation
+	// taking place on a split, which may not necessarily be TrySplit.
+	split      chan struct{}
+	blockSplit chan struct{}
+}
+
+func newSplitTestRTracker(rest offsetrange.Restriction) *splitTestRTracker {
+	return &splitTestRTracker{
+		rt:         offsetrange.NewTracker(rest),
+		claim:      make(chan struct{}, 1),
+		blockClaim: make(chan struct{}),
+		endClaim:   make(chan struct{}),
+		blockInd:   rest.Start,
+		split:      make(chan struct{}, 1),
+		blockSplit: make(chan struct{}),
+	}
+}
+
+func (rt *splitTestRTracker) TryClaim(pos interface{}) bool {
+	i := pos.(int64)
+	if i == rt.blockInd {
+		rt.claim <- struct{}{}
+	}
+
+	rt.mu.Lock()
+	if i == rt.blockInd {
+		rt.blockClaim <- struct{}{}
+	}
+	result := rt.rt.TryClaim(pos)
+	rt.mu.Unlock()
+
+	if i == rt.blockInd {
+		rt.endClaim <- struct{}{}
+	}
+	return result
+}
+
+func (rt *splitTestRTracker) GetError() error {
+	rt.mu.Lock()
+	defer rt.mu.Unlock()
+	return rt.rt.GetError()
+}
+
+func (rt *splitTestRTracker) TrySplit(fraction float64) (interface{}, interface{}, error) {
+	rt.mu.Lock()
+	defer rt.mu.Unlock()
+	rt.blockSplit <- struct{}{}
+	return rt.rt.TrySplit(fraction)
+}
+
+func (rt *splitTestRTracker) GetProgress() (float64, float64) {
+	// Note: Currently, GetProgress is called first in a split and blocks if
+	// TryClaim is being called.
+	rt.split <- struct{}{}
+
+	rt.mu.Lock()
+	defer rt.mu.Unlock()
+	return rt.rt.GetProgress()
+}
+
+func (rt *splitTestRTracker) IsDone() bool {
+	rt.mu.Lock()
+	defer rt.mu.Unlock()
+	return rt.rt.IsDone()
+}
+
+func (rt *splitTestRTracker) GetRestriction() interface{} {
+	rt.mu.Lock()
+	defer rt.mu.Unlock()
+	return rt.rt.GetRestriction()
+}
+
+// splitTestSdf has signals needed to control processing behavior over multiple
+// threads. The actual behavior is to accept an integer N as the element and
+// output each element in the range of [0, N).
+type splitTestSdf struct {
+	proc chan struct{}
+	rt   chan *splitTestRTracker // Used to provide created trackers to the test code.
+}
+
+func newSplitTestSdf() *splitTestSdf {
+	return &splitTestSdf{
+		proc: make(chan struct{}),
+		rt:   make(chan *splitTestRTracker),
+	}
+}
+
+func (fn *splitTestSdf) ProcessElement(rt *splitTestRTracker, _ int, emit func(offsetrange.Restriction, int)) {
+	i := rt.GetRestriction().(offsetrange.Restriction).Start
+	fn.proc <- struct{}{}
+
+	for rt.TryClaim(i) == true {
+		rest := rt.GetRestriction().(offsetrange.Restriction)
+		emit(rest, int(i))
+		i++
+	}
+}
+
+func (fn *splitTestSdf) CreateInitialRestriction(i int) offsetrange.Restriction {
+	return offsetrange.Restriction{
+		Start: 0,
+		End:   int64(i),
+	}
+}
+
+func (fn *splitTestSdf) SplitRestriction(_ int, rest offsetrange.Restriction) []offsetrange.Restriction {
+	return []offsetrange.Restriction{rest}
+}
+
+func (fn *splitTestSdf) RestrictionSize(_ int, rest offsetrange.Restriction) float64 {
+	return rest.Size()
+}
+
+func (fn *splitTestSdf) CreateTracker(rest offsetrange.Restriction) *splitTestRTracker {
+	rt := newSplitTestRTracker(rest)
+	fn.rt <- rt
+	return rt
+}
diff --git a/sdks/go/pkg/beam/core/runtime/exec/plan.go b/sdks/go/pkg/beam/core/runtime/exec/plan.go
index 53891d6..23ce1c7 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/plan.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/plan.go
@@ -207,13 +207,28 @@
 	BufSize int64
 }
 
+// SplitResult contains the result of performing a split on a Plan.
+type SplitResult struct {
+	// Indices are always included, for both channel and sub-element splits.
+	PI int64 // Primary index, last element of the primary.
+	RI int64 // Residual index, first element of the residual.
+
+	// Extra information included for sub-element splits. If PS and RS are
+	// present then a sub-element split occurred.
+	PS   []byte // Primary split. If an element is split, this is the encoded primary.
+	RS   []byte // Residual split. If an element is split, this is the encoded residual.
+	TId  string // Transform ID of the transform receiving the split elements.
+	InId string // Input ID of the input the split elements are received from.
+}
+
 // Split takes a set of potential split indexes, and if successful returns
-// the split index of the first element of the residual, on which processing
-// will be halted.
+// the split result.
 // Returns an error when unable to split.
-func (p *Plan) Split(s SplitPoints) (int64, error) {
+func (p *Plan) Split(s SplitPoints) (SplitResult, error) {
+	// TODO: When bundles with multiple sources, are supported, perform splits
+	// on all sources.
 	if p.source != nil {
 		return p.source.Split(s.Splits, s.Frac, s.BufSize)
 	}
-	return 0, fmt.Errorf("failed to split at requested splits: {%v}, Source not initialized", s)
+	return SplitResult{}, fmt.Errorf("failed to split at requested splits: {%v}, Source not initialized", s)
 }
diff --git a/sdks/go/pkg/beam/core/runtime/exec/sdf.go b/sdks/go/pkg/beam/core/runtime/exec/sdf.go
index c24e34a..29b40c8 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/sdf.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/sdf.go
@@ -43,7 +43,7 @@
 }
 
 // Up performs one-time setup for this executor.
-func (n *PairWithRestriction) Up(ctx context.Context) error {
+func (n *PairWithRestriction) Up(_ context.Context) error {
 	fn := (*graph.SplittableDoFn)(n.Fn).CreateInitialRestrictionFn()
 	var err error
 	if n.inv, err = newCreateInitialRestrictionInvoker(fn); err != nil {
@@ -88,7 +88,7 @@
 }
 
 // Down currently does nothing.
-func (n *PairWithRestriction) Down(ctx context.Context) error {
+func (n *PairWithRestriction) Down(_ context.Context) error {
 	return nil
 }
 
@@ -117,7 +117,7 @@
 }
 
 // Up performs one-time setup for this executor.
-func (n *SplitAndSizeRestrictions) Up(ctx context.Context) error {
+func (n *SplitAndSizeRestrictions) Up(_ context.Context) error {
 	fn := (*graph.SplittableDoFn)(n.Fn).SplitRestrictionFn()
 	var err error
 	if n.splitInv, err = newSplitRestrictionInvoker(fn); err != nil {
@@ -200,7 +200,7 @@
 }
 
 // Down currently does nothing.
-func (n *SplitAndSizeRestrictions) Down(ctx context.Context) error {
+func (n *SplitAndSizeRestrictions) Down(_ context.Context) error {
 	return nil
 }
 
@@ -215,14 +215,27 @@
 // changes to support the SDF's method signatures and the expected structure
 // of the FullValue being received.
 type ProcessSizedElementsAndRestrictions struct {
-	PDo *ParDo
+	PDo     *ParDo
+	TfId    string // Transform ID. Needed for splitting.
+	ctInv   *ctInvoker
+	sizeInv *rsInvoker
 
-	inv *ctInvoker
+	// SU is a buffered channel for indicating when this unit is splittable.
+	// When this unit is processing an element, it sends a SplittableUnit
+	// interface through the channel. That interface can be received on other
+	// threads and used to perform splitting or other related operation.
+	//
+	// This channel should be received on in a non-blocking manner, to avoid
+	// hanging if no element is processing.
+	//
+	// Receiving the SplittableUnit prevents the current element from finishing
+	// processing, so the element does not unexpectedly change during a split.
+	// Therefore, receivers of the SplittableUnit must send it back through the
+	// channel once finished with it, or it will block indefinitely.
+	SU chan SplittableUnit
 
-	// Rt allows this unit to send out restriction trackers being processed.
-	// Receivers of the tracker do not own it, and must send it back through the
-	// same channel once finished with it.
-	Rt chan sdf.RTracker
+	elm *FullValue   // Currently processing element.
+	rt  sdf.RTracker // Currently processing element's restriction tracker.
 }
 
 // ID calls the ParDo's ID method.
@@ -234,10 +247,14 @@
 func (n *ProcessSizedElementsAndRestrictions) Up(ctx context.Context) error {
 	fn := (*graph.SplittableDoFn)(n.PDo.Fn).CreateTrackerFn()
 	var err error
-	if n.inv, err = newCreateTrackerInvoker(fn); err != nil {
+	if n.ctInv, err = newCreateTrackerInvoker(fn); err != nil {
 		return errors.WithContextf(err, "%v", n)
 	}
-	n.Rt = make(chan sdf.RTracker, 1)
+	fn = (*graph.SplittableDoFn)(n.PDo.Fn).RestrictionSizeFn()
+	if n.sizeInv, err = newRestrictionSizeInvoker(fn); err != nil {
+		return errors.WithContextf(err, "%v", n)
+	}
+	n.SU = make(chan SplittableUnit, 1)
 	return n.PDo.Up(ctx)
 }
 
@@ -268,15 +285,22 @@
 // and processes each element using the underlying ParDo and adding the
 // restriction tracker to the normal invocation. Sizing information is present
 // but currently ignored. Output is forwarded to the underlying ParDo's outputs.
-func (n *ProcessSizedElementsAndRestrictions) ProcessElement(ctx context.Context, elm *FullValue, values ...ReStream) error {
+func (n *ProcessSizedElementsAndRestrictions) ProcessElement(_ context.Context, elm *FullValue, values ...ReStream) error {
 	if n.PDo.status != Active {
 		err := errors.Errorf("invalid status %v, want Active", n.PDo.status)
 		return errors.WithContextf(err, "%v", n)
 	}
 
 	rest := elm.Elm.(*FullValue).Elm2
-	rt := n.inv.Invoke(rest)
-	n.Rt <- rt
+	rt := n.ctInv.Invoke(rest)
+
+	n.rt = rt
+	n.elm = elm
+	n.SU <- n
+	defer func() {
+		<-n.SU
+	}()
+
 	mainIn := &MainInput{
 		Values:   values,
 		RTracker: rt,
@@ -303,20 +327,18 @@
 		}
 	}
 
-	err := n.PDo.processMainInput(mainIn)
-	<-n.Rt
-	return err
+	return n.PDo.processMainInput(mainIn)
 }
 
 // FinishBundle resets the invokers and then calls the ParDo's FinishBundle method.
 func (n *ProcessSizedElementsAndRestrictions) FinishBundle(ctx context.Context) error {
-	n.inv.Reset()
+	n.ctInv.Reset()
+	n.sizeInv.Reset()
 	return n.PDo.FinishBundle(ctx)
 }
 
-// Down closes open channels and calls the ParDo's Down method.
+// Down calls the ParDo's Down method.
 func (n *ProcessSizedElementsAndRestrictions) Down(ctx context.Context) error {
-	close(n.Rt)
 	return n.PDo.Down(ctx)
 }
 
@@ -325,6 +347,104 @@
 	return fmt.Sprintf("SDF.ProcessSizedElementsAndRestrictions[%v] UID:%v Out:%v", path.Base(n.PDo.Fn.Name()), n.PDo.ID(), IDs(n.PDo.Out...))
 }
 
+// SplittableUnit is an interface that defines sub-element splitting operations
+// for a unit, and provides access to them on other threads.
+type SplittableUnit interface {
+	// Split performs a split on a fraction of a currently processing element
+	// and returns the primary and residual elements resulting from it, or an
+	// error if the split failed.
+	Split(fraction float64) (primary, residual *FullValue, err error)
+
+	// GetProgress returns the fraction of progress the current element has
+	// made in processing. (ex. 0.0 means no progress, and 1.0 means fully
+	// processed.)
+	GetProgress() float64
+
+	// GetTransformId returns the transform ID of the splittable unit.
+	GetTransformId() string
+
+	// GetInputId returns the local input ID of the input that the element being
+	// split was received from.
+	GetInputId() string
+}
+
+// Split splits the currently processing element using its restriction tracker.
+// Then it returns an element for primary and residual, following the expected
+// input structure to this unit, including updating the size of the split
+// elements.
+func (n *ProcessSizedElementsAndRestrictions) Split(f float64) (*FullValue, *FullValue, error) {
+	addContext := func(err error) error {
+		return errors.WithContext(err, "Attempting split in ProcessSizedElementsAndRestrictions")
+	}
+
+	// Check that the restriction tracker is in a state where it can be split.
+	if n.rt == nil {
+		return nil, nil, addContext(errors.New("Restriction tracker missing."))
+	}
+	if err := n.rt.GetError(); err != nil {
+		return nil, nil, addContext(err)
+	}
+	if n.rt.IsDone() { // Not an error, but not splittable.
+		return nil, nil, nil
+	}
+
+	p, r, err := n.rt.TrySplit(f)
+	if err != nil {
+		return nil, nil, addContext(err)
+	}
+	if r == nil { // If r is nil then the split failed/returned an empty residual.
+		return nil, nil, nil
+	}
+
+	var pfv, rfv *FullValue
+	var pSize, rSize float64
+	elm := n.elm.Elm.(*FullValue).Elm
+	if fv, ok := elm.(*FullValue); ok {
+		pSize = n.sizeInv.Invoke(fv, p)
+		rSize = n.sizeInv.Invoke(fv, r)
+	} else {
+		fv := &FullValue{Elm: elm}
+		pSize = n.sizeInv.Invoke(fv, p)
+		rSize = n.sizeInv.Invoke(fv, r)
+	}
+	pfv = &FullValue{
+		Elm: &FullValue{
+			Elm:  elm,
+			Elm2: p,
+		},
+		Elm2:      pSize,
+		Timestamp: n.elm.Timestamp,
+		Windows:   n.elm.Windows,
+	}
+	rfv = &FullValue{
+		Elm: &FullValue{
+			Elm:  elm,
+			Elm2: r,
+		},
+		Elm2:      rSize,
+		Timestamp: n.elm.Timestamp,
+		Windows:   n.elm.Windows,
+	}
+	return pfv, rfv, nil
+}
+
+// GetProgress returns the current restriction tracker's progress as a fraction.
+func (n *ProcessSizedElementsAndRestrictions) GetProgress() float64 {
+	d, r := n.rt.GetProgress()
+	return d / (d + r)
+}
+
+// GetTransformId returns this transform's transform ID.
+func (n *ProcessSizedElementsAndRestrictions) GetTransformId() string {
+	return n.TfId
+}
+
+// GetInputId returns the main input ID, since main input elements are being
+// split.
+func (n *ProcessSizedElementsAndRestrictions) GetInputId() string {
+	return indexToInputId(0)
+}
+
 // SdfFallback is an executor used when an SDF isn't expanded into steps by the
 // runner, indicating that the runner doesn't support splitting. It executes all
 // the SDF steps together in one unit.
@@ -370,7 +490,7 @@
 // restrictions, and then creating restriction trackers and processing each
 // restriction with the underlying ParDo. This executor skips the sizing step
 // because sizing information is unnecessary for unexpanded SDFs.
-func (n *SdfFallback) ProcessElement(ctx context.Context, elm *FullValue, values ...ReStream) error {
+func (n *SdfFallback) ProcessElement(_ context.Context, elm *FullValue, values ...ReStream) error {
 	if n.PDo.status != Active {
 		err := errors.Errorf("invalid status %v, want Active", n.PDo.status)
 		return errors.WithContextf(err, "%v", n)
diff --git a/sdks/go/pkg/beam/core/runtime/exec/sdf_invokers_test.go b/sdks/go/pkg/beam/core/runtime/exec/sdf_invokers_test.go
index 5e3abd6..7dbe3e8 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/sdf_invokers_test.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/sdf_invokers_test.go
@@ -16,9 +16,10 @@
 package exec
 
 import (
+	"testing"
+
 	"github.com/apache/beam/sdks/go/pkg/beam/core/graph"
 	"github.com/google/go-cmp/cmp"
-	"testing"
 )
 
 // TestInvokes runs tests on each SDF method invoker, using the SDFs defined
diff --git a/sdks/go/pkg/beam/core/runtime/exec/sdf_test.go b/sdks/go/pkg/beam/core/runtime/exec/sdf_test.go
index e746f85..fb85620 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/sdf_test.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/sdf_test.go
@@ -16,11 +16,13 @@
 package exec
 
 import (
+	"context"
+	"testing"
+
 	"github.com/apache/beam/sdks/go/pkg/beam/core/graph"
 	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/window"
 	"github.com/apache/beam/sdks/go/pkg/beam/core/typex"
 	"github.com/google/go-cmp/cmp"
-	"testing"
 )
 
 // testTimestamp is a constant used to check that timestamps are retained.
@@ -386,3 +388,145 @@
 		}
 	})
 }
+
+// TestAsSplittableUnit tests ProcessSizedElementsAndRestrictions' implementation
+// of the SplittableUnit interface.
+func TestAsSplittableUnit(t *testing.T) {
+	dfn, err := graph.NewDoFn(&VetSdf{}, graph.NumMainInputs(graph.MainSingle))
+	if err != nil {
+		t.Fatalf("invalid function: %v", err)
+	}
+	kvdfn, err := graph.NewDoFn(&VetKvSdf{}, graph.NumMainInputs(graph.MainKv))
+	if err != nil {
+		t.Fatalf("invalid function: %v", err)
+	}
+
+	// Test that Split returns properly structured results and calls Split on
+	// the restriction tracker.
+	t.Run("Split", func(t *testing.T) {
+		tests := []struct {
+			name         string
+			fn           *graph.DoFn
+			in           FullValue
+			wantPrimary  FullValue
+			wantResidual FullValue
+		}{
+			{
+				name: "SingleElem",
+				fn:   dfn,
+				in: FullValue{
+					Elm: &FullValue{
+						Elm:  1,
+						Elm2: &VetRestriction{ID: "Sdf"},
+					},
+					Elm2:      1.0,
+					Timestamp: testTimestamp,
+					Windows:   testWindows,
+				},
+				wantPrimary: FullValue{
+					Elm: &FullValue{
+						Elm:  1,
+						Elm2: &VetRestriction{ID: "Sdf.1", RestSize: true, Val: 1},
+					},
+					Elm2:      1.0,
+					Timestamp: testTimestamp,
+					Windows:   testWindows,
+				},
+				wantResidual: FullValue{
+					Elm: &FullValue{
+						Elm:  1,
+						Elm2: &VetRestriction{ID: "Sdf.2", RestSize: true, Val: 1},
+					},
+					Elm2:      1.0,
+					Timestamp: testTimestamp,
+					Windows:   testWindows,
+				},
+			},
+			{
+				name: "KvElem",
+				fn:   kvdfn,
+				in: FullValue{
+					Elm: &FullValue{
+						Elm: &FullValue{
+							Elm:  1,
+							Elm2: 2,
+						},
+						Elm2: &VetRestriction{ID: "KvSdf"},
+					},
+					Elm2:      3.0,
+					Timestamp: testTimestamp,
+					Windows:   testWindows,
+				},
+				wantPrimary: FullValue{
+					Elm: &FullValue{
+						Elm: &FullValue{
+							Elm:  1,
+							Elm2: 2,
+						},
+						Elm2: &VetRestriction{ID: "KvSdf.1", RestSize: true, Key: 1, Val: 2},
+					},
+					Elm2:      3.0,
+					Timestamp: testTimestamp,
+					Windows:   testWindows,
+				},
+				wantResidual: FullValue{
+					Elm: &FullValue{
+						Elm: &FullValue{
+							Elm:  1,
+							Elm2: 2,
+						},
+						Elm2: &VetRestriction{ID: "KvSdf.2", RestSize: true, Key: 1, Val: 2},
+					},
+					Elm2:      3.0,
+					Timestamp: testTimestamp,
+					Windows:   testWindows,
+				},
+			},
+		}
+		for _, test := range tests {
+			test := test
+			t.Run(test.name, func(t *testing.T) {
+				// Setup, create transforms, inputs, and desired outputs.
+				n := &ParDo{UID: 1, Fn: test.fn, Out: []Node{}}
+				node := &ProcessSizedElementsAndRestrictions{PDo: n}
+				node.rt = &SplittableUnitRTracker{
+					VetRTracker: VetRTracker{Rest: test.in.Elm.(*FullValue).Elm2.(*VetRestriction)},
+				}
+				node.elm = &test.in
+
+				// Call from SplittableUnit and check results.
+				su := SplittableUnit(node)
+				frac := 0.5
+				if err := node.Up(context.Background()); err != nil {
+					t.Fatalf("ProcessSizedElementsAndRestrictions.Up() failed: %v", err)
+				}
+				gotPrimary, gotResidual, err := su.Split(frac)
+				if err != nil {
+					t.Fatalf("SplittableUnit.Split(%v) failed: %v", frac, err)
+				}
+				if diff := cmp.Diff(gotPrimary, &test.wantPrimary); diff != "" {
+					t.Errorf("SplittableUnit.Split(%v) has incorrect primary: %v", frac, diff)
+				}
+				if diff := cmp.Diff(gotResidual, &test.wantResidual); diff != "" {
+					t.Errorf("SplittableUnit.Split(%v) has incorrect residual: %v", frac, diff)
+				}
+			})
+		}
+	})
+}
+
+// SplittableUnitRTracker is a VetRTracker with some added behavior needed for
+// TestAsSplittableUnit.
+type SplittableUnitRTracker struct {
+	VetRTracker
+}
+
+func (rt *SplittableUnitRTracker) IsDone() bool { return false }
+
+func (rt *SplittableUnitRTracker) TrySplit(_ float64) (interface{}, interface{}, error) {
+	rest1 := rt.Rest.copy()
+	rest1.ID += ".1"
+	rest2 := rt.Rest.copy()
+	rest2.ID += ".2"
+	return &rest1, &rest2, nil
+}
diff --git a/sdks/go/pkg/beam/core/runtime/exec/translate.go b/sdks/go/pkg/beam/core/runtime/exec/translate.go
index caa5a22..fbbdab3 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/translate.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/translate.go
@@ -418,7 +418,7 @@
 					}
 					u = n
 					if urn == urnProcessSizedElementsAndRestrictions {
-						u = &ProcessSizedElementsAndRestrictions{PDo: n}
+						u = &ProcessSizedElementsAndRestrictions{PDo: n, TfId: id.to}
 					} else if dofn.IsSplittable() {
 						u = &SdfFallback{PDo: n}
 					}
@@ -580,7 +580,7 @@
 	var unordered []string
 
 	for key := range m {
-		if i, err := strconv.Atoi(strings.TrimPrefix(key, "i")); strings.HasPrefix(key, "i") && err == nil {
+		if i, err := inputIdToIndex(key); err == nil {
 			if i < len(m) {
 				ordered[i] = key
 				continue
@@ -605,6 +605,24 @@
 	return ret
 }
 
+// inputIdToIndex converts a local input ID for a transform into an index. Use
+// this to avoid relying on format details for input IDs.
+//
+// Currently, expects IDs in the format "iN" where N is the index. If the ID is
+// in an invalid form, returns an error.
+func inputIdToIndex(id string) (int, error) {
+	if !strings.HasPrefix(id, "i") {
+		return 0, errors.New("invalid input ID format")
+	}
+	return strconv.Atoi(strings.TrimPrefix(id, "i"))
+}
+
+// inputIdToIndex converts an index into a local input ID for a transform. Use
+// this to avoid relying on format details for input IDs.
+func indexToInputId(i int) string {
+	return "i" + strconv.Itoa(i)
+}
+
 func unmarshalPort(data []byte) (Port, string, error) {
 	var port fnpb.RemoteGrpcPort
 	if err := proto.Unmarshal(data, &port); err != nil {
diff --git a/sdks/go/pkg/beam/core/runtime/harness/harness.go b/sdks/go/pkg/beam/core/runtime/harness/harness.go
index 4437529..3aa8d43 100644
--- a/sdks/go/pkg/beam/core/runtime/harness/harness.go
+++ b/sdks/go/pkg/beam/core/runtime/harness/harness.go
@@ -315,7 +315,7 @@
 		if ds == nil {
 			return fail(ctx, instID, "failed to split: desired splits for root of %v was empty.", ref)
 		}
-		split, err := plan.Split(exec.SplitPoints{
+		sr, err := plan.Split(exec.SplitPoints{
 			Splits:  ds.GetAllowedSplitPoints(),
 			Frac:    ds.GetFractionOfRemainder(),
 			BufSize: ds.GetEstimatedInputElements(),
@@ -329,12 +329,22 @@
 			InstructionId: string(instID),
 			Response: &fnpb.InstructionResponse_ProcessBundleSplit{
 				ProcessBundleSplit: &fnpb.ProcessBundleSplitResponse{
-					ChannelSplits: []*fnpb.ProcessBundleSplitResponse_ChannelSplit{
-						&fnpb.ProcessBundleSplitResponse_ChannelSplit{
-							LastPrimaryElement:   split - 1,
-							FirstResidualElement: split,
+					PrimaryRoots: []*fnpb.BundleApplication{{
+						TransformId: sr.TId,
+						InputId:     sr.InId,
+						Element:     sr.PS,
+					}},
+					ResidualRoots: []*fnpb.DelayedBundleApplication{{
+						Application: &fnpb.BundleApplication{
+							TransformId: sr.TId,
+							InputId:     sr.InId,
+							Element:     sr.RS,
 						},
-					},
+					}},
+					ChannelSplits: []*fnpb.ProcessBundleSplitResponse_ChannelSplit{{
+						LastPrimaryElement:   sr.PI,
+						FirstResidualElement: sr.RI,
+					}},
 				},
 			},
 		}
diff --git a/sdks/go/pkg/beam/core/sdf/sdf.go b/sdks/go/pkg/beam/core/sdf/sdf.go
index 3d663d5..1de3f5c 100644
--- a/sdks/go/pkg/beam/core/sdf/sdf.go
+++ b/sdks/go/pkg/beam/core/sdf/sdf.go
@@ -64,8 +64,9 @@
 	//
 	// This method modifies the underlying restriction in the RTracker to reflect the primary. It
 	// then returns a copy of the newly modified restriction as a primary, and returns a new
-	// restriction for the residual. If the split would produce an empty residual (i.e. the only
-	// split point is the end of the restriction), then the returned residual is nil.
+	// restriction for the residual. If the split would produce an empty residual (either because
+	// the only split point is the end of the restriction, or the split failed for some recoverable
+	// reason), then this function returns nil as the residual.
 	//
 	// If an error is returned, some catastrophic failure occurred and the entire bundle will fail.
 	TrySplit(fraction float64) (primary, residual interface{}, err error)
@@ -77,8 +78,8 @@
 
 	// IsDone returns a boolean indicating whether all blocks inside the restriction have been
 	// claimed. This method is called by the SDK Harness to validate that a splittable DoFn has
-	// correctly processed all work in a restriction before finishing. If this method returns false
-	// then GetError is expected to return a non-nil error.
+	// correctly processed all work in a restriction before finishing. If this method still returns
+	// false after processing, then GetError is expected to return a non-nil error.
 	IsDone() bool
 
 	// GetRestriction returns the restriction this tracker is tracking, or nil if the restriction
diff --git a/sdks/go/pkg/beam/io/rtrackers/offsetrange/offsetrange.go b/sdks/go/pkg/beam/io/rtrackers/offsetrange/offsetrange.go
index f01ba50..e140a7e 100644
--- a/sdks/go/pkg/beam/io/rtrackers/offsetrange/offsetrange.go
+++ b/sdks/go/pkg/beam/io/rtrackers/offsetrange/offsetrange.go
@@ -20,16 +20,39 @@
 package offsetrange
 
 import (
+	"bytes"
+	"encoding/binary"
 	"errors"
 	"math"
 	"reflect"
 
-	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime"
 )
 
 func init() {
-	beam.RegisterType(reflect.TypeOf((*Tracker)(nil)))
-	beam.RegisterType(reflect.TypeOf((*Restriction)(nil)))
+	runtime.RegisterType(reflect.TypeOf((*Tracker)(nil)))
+	runtime.RegisterType(reflect.TypeOf((*Restriction)(nil)).Elem())
+	runtime.RegisterFunction(restEnc)
+	runtime.RegisterFunction(restDec)
+	coder.RegisterCoder(reflect.TypeOf((*Restriction)(nil)).Elem(), restEnc, restDec)
+}
+
+func restEnc(in Restriction) ([]byte, error) {
+	buf := new(bytes.Buffer)
+	if err := binary.Write(buf, binary.BigEndian, in); err != nil {
+		return nil, err
+	}
+	return buf.Bytes(), nil
+}
+
+func restDec(in []byte) (Restriction, error) {
+	buf := bytes.NewBuffer(in)
+	rest := Restriction{}
+	if err := binary.Read(buf, binary.BigEndian, &rest); err != nil {
+		return rest, err
+	}
+	return rest, nil
 }
 
 // Restriction is an offset range restriction, which represents a range of
@@ -44,10 +67,10 @@
 //
 // Num should be greater than 0. Otherwise there is no way to split the
 // restriction and this function will return the original restriction.
-func (r *Restriction) EvenSplits(num int64) (splits []Restriction) {
+func (r Restriction) EvenSplits(num int64) (splits []Restriction) {
 	if num <= 1 {
 		// Don't split, just return original restriction.
-		return append(splits, *r)
+		return append(splits, r)
 	}
 
 	offset := r.Start
@@ -67,7 +90,7 @@
 }
 
 // Size returns the restriction's size as the difference between Start and End.
-func (r *Restriction) Size() float64 {
+func (r Restriction) Size() float64 {
 	return float64(r.End - r.Start)
 }
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/expansion/ExternalTransformRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/expansion/ExternalTransformRegistrar.java
index bbdd3a5..aa5288c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/expansion/ExternalTransformRegistrar.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/expansion/ExternalTransformRegistrar.java
@@ -17,10 +17,14 @@
  */
 package org.apache.beam.sdk.expansion;
 
+import java.lang.reflect.Constructor;
 import java.util.Map;
+import java.util.Map.Entry;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
 import org.apache.beam.sdk.transforms.ExternalTransformBuilder;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 
 /**
  * A registrar which contains a mapping from URNs to available {@link ExternalTransformBuilder}s.
@@ -29,6 +33,42 @@
 @Experimental(Kind.PORTABILITY)
 public interface ExternalTransformRegistrar {
 
-  /** A mapping from URN to an {@link ExternalTransformBuilder} class. */
-  Map<String, Class<? extends ExternalTransformBuilder<?, ?, ?>>> knownBuilders();
+  /**
+   * A mapping from URN to an {@link ExternalTransformBuilder} class.
+   *
+   * @deprecated Prefer implementing 'knownBuilderInstances'. This method will be removed in a
+   *     future version of Beam.
+   */
+  @Deprecated
+  default Map<String, Class<? extends ExternalTransformBuilder<?, ?, ?>>> knownBuilders() {
+    return ImmutableMap.<String, Class<? extends ExternalTransformBuilder<?, ?, ?>>>builder()
+        .build();
+  }
+
+  /** A mapping from URN to an {@link ExternalTransformBuilder} instance. */
+  default Map<String, ExternalTransformBuilder<?, ?, ?>> knownBuilderInstances() {
+    ImmutableMap.Builder builder = ImmutableMap.<String, ExternalTransformBuilder>builder();
+    Map<String, Class<? extends ExternalTransformBuilder<?, ?, ?>>> knownBuilders = knownBuilders();
+    for (Entry<String, Class<? extends ExternalTransformBuilder<?, ?, ?>>> knownBuilder :
+        knownBuilders.entrySet()) {
+      Preconditions.checkState(
+          ExternalTransformBuilder.class.isAssignableFrom(knownBuilder.getValue()),
+          "Provided identifier %s is not an ExternalTransformBuilder.",
+          knownBuilder.getValue().getName());
+      try {
+        Constructor<? extends ExternalTransformBuilder> constructor =
+            knownBuilder.getValue().getDeclaredConstructor();
+
+        constructor.setAccessible(true);
+        builder.put(knownBuilder.getKey(), constructor.newInstance());
+
+      } catch (RuntimeException e) {
+        throw e;
+      } catch (Exception e) {
+        throw new RuntimeException(
+            "Unable to instantiate ExternalTransformBuilder from constructor.");
+      }
+    }
+    return builder.build();
+  }
 }
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java
index 0995522..226924f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java
@@ -1273,7 +1273,7 @@
   public int indexOf(String fieldName) {
     Integer index = fieldIndices.get(fieldName);
     Preconditions.checkArgument(
-        index != null, String.format("Cannot find field %s in schema %s", fieldName, this));
+        index != null, "Cannot find field %s in schema %s", fieldName, this);
     return index;
   }
 
@@ -1285,7 +1285,7 @@
   /** Return the name of field by index. */
   public String nameOf(int fieldIndex) {
     String name = fieldIndices.inverse().get(fieldIndex);
-    Preconditions.checkArgument(name != null, String.format("Cannot find field %d", fieldIndex));
+    Preconditions.checkArgument(name != null, "Cannot find field %s", fieldIndex);
     return name;
   }
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTest.java
index 17518a6..05ed20d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTest.java
@@ -351,4 +351,24 @@
     Schema schema4 = Schema.builder().addInt32Field("foo").build();
     assertFalse(schema1.typesEqual(schema4)); // schema1 and schema4 differ by types
   }
+
+  @Test
+  public void testIllegalIndexOf() {
+    Schema schema = Schema.builder().addStringField("foo").build();
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Cannot find field bar in schema " + schema);
+
+    schema.indexOf("bar");
+  }
+
+  @Test
+  public void testIllegalNameOf() {
+    Schema schema = Schema.builder().addStringField("foo").build();
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Cannot find field 1");
+
+    schema.nameOf(1);
+  }
 }
diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java
index c20e8d2..09a42e3 100644
--- a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java
+++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java
@@ -22,7 +22,6 @@
 
 import com.google.auto.service.AutoService;
 import java.io.IOException;
-import java.lang.reflect.Constructor;
 import java.lang.reflect.Method;
 import java.util.ArrayDeque;
 import java.util.Collections;
@@ -107,17 +106,21 @@
           ImmutableMap.builder();
       for (ExternalTransformRegistrar registrar :
           ServiceLoader.load(ExternalTransformRegistrar.class)) {
-        for (Map.Entry<String, Class<? extends ExternalTransformBuilder<?, ?, ?>>> entry :
-            registrar.knownBuilders().entrySet()) {
+        for (Map.Entry<String, ExternalTransformBuilder<?, ?, ?>> entry :
+            registrar.knownBuilderInstances().entrySet()) {
           String urn = entry.getKey();
-          Class<? extends ExternalTransformBuilder<?, ?, ?>> builderClass = entry.getValue();
+          ExternalTransformBuilder builderInstance = entry.getValue();
           builder.put(
               urn,
               spec -> {
                 try {
                   ExternalTransforms.ExternalConfigurationPayload payload =
                       ExternalTransforms.ExternalConfigurationPayload.parseFrom(spec.getPayload());
-                  return translate(payload, builderClass);
+                  return builderInstance.buildExternal(
+                      payloadToConfig(
+                          payload,
+                          (Class<? extends ExternalTransformBuilder<?, ?, ?>>)
+                              builderInstance.getClass()));
                 } catch (Exception e) {
                   throw new RuntimeException(
                       String.format("Failed to build transform %s from spec %s", urn, spec), e);
@@ -125,21 +128,17 @@
               });
         }
       }
+
       return builder.build();
     }
 
-    private static PTransform<?, ?> translate(
+    Object payloadToConfig(
         ExternalTransforms.ExternalConfigurationPayload payload,
         Class<? extends ExternalTransformBuilder<?, ?, ?>> builderClass)
         throws Exception {
-      Preconditions.checkState(
-          ExternalTransformBuilder.class.isAssignableFrom(builderClass),
-          "Provided identifier %s is not an ExternalTransformBuilder.",
-          builderClass.getName());
-
       Object configObject = initConfiguration(builderClass);
       populateConfiguration(configObject, payload);
-      return buildTransform(builderClass, configObject);
+      return configObject;
     }
 
     private static Object initConfiguration(
@@ -239,28 +238,6 @@
 
       return coderBuilder.build();
     }
-
-    private static PTransform<?, ?> buildTransform(
-        Class<? extends ExternalTransformBuilder<?, ?, ?>> builderClass, Object configObject)
-        throws Exception {
-      Constructor<? extends ExternalTransformBuilder<?, ?, ?>> constructor =
-          builderClass.getDeclaredConstructor();
-      constructor.setAccessible(true);
-      ExternalTransformBuilder<?, ?, ?> externalTransformBuilder = constructor.newInstance();
-      Method buildMethod = builderClass.getMethod("buildExternal", configObject.getClass());
-      buildMethod.setAccessible(true);
-
-      PTransform<?, ?> transform =
-          (PTransform<?, ?>)
-              checkArgumentNotNull(
-                  buildMethod.invoke(externalTransformBuilder, configObject),
-                  "Invoking %s.%s(%s) returned null, violating its type.",
-                  builderClass.getCanonicalName(),
-                  "buildExternal",
-                  configObject);
-
-      return transform;
-    }
   }
 
   /**
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamTableUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamTableUtils.java
index c0ab5eb..c2a03df 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamTableUtils.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamTableUtils.java
@@ -151,10 +151,16 @@
         case INT32:
           return Integer.valueOf(raw);
         case INT64:
+          if (raw.equals("")) {
+            return null;
+          }
           return Long.valueOf(raw);
         case FLOAT:
           return Float.valueOf(raw);
         case DOUBLE:
+          if (raw.equals("")) {
+            return null;
+          }
           return Double.valueOf(raw);
         default:
           throw new UnsupportedOperationException(
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
index 3dee824..dfa2b37 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
@@ -19,6 +19,7 @@
 
 import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
 
+import com.google.api.client.googleapis.json.GoogleJsonError;
 import com.google.api.client.googleapis.json.GoogleJsonResponseException;
 import com.google.api.client.googleapis.services.AbstractGoogleClientRequest;
 import com.google.api.client.http.HttpRequestInitializer;
@@ -91,6 +92,7 @@
 import org.apache.beam.sdk.values.ValueInSingleWindow;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.joda.time.Duration;
 import org.slf4j.Logger;
@@ -119,6 +121,9 @@
   private static final FluentBackoff DEFAULT_BACKOFF_FACTORY =
       FluentBackoff.DEFAULT.withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF);
 
+  // The error code for quota exceeded error (https://cloud.google.com/bigquery/docs/error-messages)
+  private static final String QUOTA_EXCEEDED = "quotaExceeded";
+
   @Override
   public JobService getJobService(BigQueryOptions options) {
     return new JobServiceImpl(options);
@@ -813,6 +818,19 @@
                         try {
                           return insert.execute().getInsertErrors();
                         } catch (IOException e) {
+                          GoogleJsonError.ErrorInfo errorInfo = getErrorInfo(e);
+                          if (errorInfo == null) {
+                            throw e;
+                          }
+                          /**
+                           * TODO(BEAM-10584): Check for QUOTA_EXCEEDED error will be replaced by
+                           * ApiErrorExtractor.INSTANCE.quotaExceeded(e) after the next release of
+                           * GoogleCloudDataproc/hadoop-connectors
+                           */
+                          if (!ApiErrorExtractor.INSTANCE.rateLimited(e)
+                              && !errorInfo.getReason().equals(QUOTA_EXCEEDED)) {
+                            throw e;
+                          }
                           LOG.info(
                               String.format(
                                   "BigQuery insertAll error, retrying: %s",
@@ -922,6 +940,15 @@
           ignoreInsertIds);
     }
 
+    protected GoogleJsonError.ErrorInfo getErrorInfo(IOException e) {
+      if (!(e instanceof GoogleJsonResponseException)) {
+        return null;
+      }
+      GoogleJsonError jsonError = ((GoogleJsonResponseException) e).getDetails();
+      GoogleJsonError.ErrorInfo errorInfo = Iterables.getFirst(jsonError.getErrors(), null);
+      return errorInfo;
+    }
+
     @Override
     public Table patchTableDescription(
         TableReference tableReference, @Nullable String tableDescription)
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java
index 413acba..15bfc0a 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java
@@ -27,6 +27,7 @@
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
@@ -35,6 +36,7 @@
 import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo;
 import com.google.api.client.googleapis.json.GoogleJsonErrorContainer;
 import com.google.api.client.googleapis.json.GoogleJsonResponseException;
+import com.google.api.client.http.HttpResponseException;
 import com.google.api.client.http.LowLevelHttpResponse;
 import com.google.api.client.json.GenericJson;
 import com.google.api.client.json.Json;
@@ -110,7 +112,6 @@
             return response;
           }
         };
-
     // A mock transport that lets us mock the API responses.
     MockHttpTransport transport =
         new MockHttpTransport.Builder().setLowLevelHttpRequest(request).build();
@@ -715,11 +716,11 @@
   }
 
   /**
-   * Tests that {@link DatasetServiceImpl#insertAll} retries other non-rate-limited,
+   * Tests that {@link DatasetServiceImpl#insertAll} will not retry other non-rate-limited,
    * non-quota-exceeded attempts.
    */
   @Test
-  public void testInsertOtherRetry() throws Throwable {
+  public void testFailInsertOtherRetry() throws Exception {
     TableReference ref =
         new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table");
     List<ValueInSingleWindow<TableRow>> rows = new ArrayList<>();
@@ -733,26 +734,29 @@
     when(response.getContent())
         .thenReturn(toStream(errorWithReasonAndStatus("actually forbidden", 403)))
         .thenReturn(toStream(new TableDataInsertAllResponse()));
-
     DatasetServiceImpl dataService =
         new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create());
-    dataService.insertAll(
-        ref,
-        rows,
-        null,
-        BackOffAdapter.toGcpBackOff(TEST_BACKOFF.backoff()),
-        TEST_BACKOFF,
-        new MockSleeper(),
-        InsertRetryPolicy.alwaysRetry(),
-        null,
-        null,
-        false,
-        false,
-        false);
-    verify(response, times(2)).getStatusCode();
-    verify(response, times(2)).getContent();
-    verify(response, times(2)).getContentType();
-    expectedLogs.verifyInfo("BigQuery insertAll error, retrying:");
+    thrown.expect(RuntimeException.class);
+    thrown.expectMessage("actually forbidden");
+    try {
+      dataService.insertAll(
+          ref,
+          rows,
+          null,
+          BackOffAdapter.toGcpBackOff(TEST_BACKOFF.backoff()),
+          TEST_BACKOFF,
+          new MockSleeper(),
+          InsertRetryPolicy.alwaysRetry(),
+          null,
+          null,
+          false,
+          false,
+          false);
+    } finally {
+      verify(response, times(1)).getStatusCode();
+      verify(response, times(1)).getContent();
+      verify(response, times(1)).getContentType();
+    }
   }
 
   /**
@@ -919,6 +923,23 @@
   }
 
   @Test
+  public void testGetErrorInfo() throws IOException {
+    DatasetServiceImpl dataService =
+        new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create());
+    ErrorInfo info = new ErrorInfo();
+    List<ErrorInfo> infoList = new ArrayList<>();
+    infoList.add(info);
+    info.setReason("QuotaExceeded");
+    GoogleJsonError error = new GoogleJsonError();
+    error.setErrors(infoList);
+    HttpResponseException.Builder builder = mock(HttpResponseException.Builder.class);
+    IOException validException = new GoogleJsonResponseException(builder, error);
+    IOException invalidException = new IOException();
+    assertEquals(info.getReason(), dataService.getErrorInfo(validException).getReason());
+    assertNull(dataService.getErrorInfo(invalidException));
+  }
+
+  @Test
   public void testCreateTableSucceeds() throws IOException {
     TableReference ref =
         new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table");
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/java/testing/tpcds/build.gradle b/sdks/java/testing/tpcds/build.gradle
new file mode 100644
index 0000000..fa249bc93
--- /dev/null
+++ b/sdks/java/testing/tpcds/build.gradle
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+plugins {
+    id 'java'
+}
+
+description = "Apache Beam :: SDKs :: Java :: TPC-DS Benchark"
+
+version '2.24.0-SNAPSHOT'
+
+sourceCompatibility = 1.8
+
+repositories {
+    mavenCentral()
+}
+
+dependencies {
+    compile 'com.googlecode.json-simple:json-simple:1.1.1'
+    compile project(path: ":sdks:java:core", configuration: "shadow")
+    compile project(path: ":runners:google-cloud-dataflow-java")
+    compile project(":sdks:java:io:google-cloud-platform")
+    compile project(":sdks:java:extensions:sql")
+    compile group: 'com.google.auto.service', name: 'auto-service', version: '1.0-rc1'
+    testCompile group: 'junit', name: 'junit', version: '4.12'
+}
+
+// When running via Gradle, this property can be used to pass commandline arguments
+// to the tpcds run
+def tpcdsArgsProperty = "tpcds.args"
+
+task run(type: JavaExec) {
+    main = "org.apache.beam.sdk.tpcds.BeamTpcds"
+    classpath = sourceSets.main.runtimeClasspath
+    def tpcdsArgsStr = project.findProperty(tpcdsArgsProperty) ?: ""
+    args tpcdsArgsStr.split()
+}
diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/BeamTpcds.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/BeamTpcds.java
new file mode 100644
index 0000000..0e6e988
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/BeamTpcds.java
@@ -0,0 +1,129 @@
+/*
+ * 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.tpcds;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.extensions.sql.meta.provider.text.TextTableProvider;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore;
+import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import java.util.List;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+
+/**
+ * To execute this main() method, run the following example command from the command line.
+ *
+ * ./gradlew :sdks:java:testing:tpcds:run -Ptpcds.args="--dataSize=1G \
+ *         --queries=3,26,55 \
+ *         --tpcParallel=2 \
+ *         --project=apache-beam-testing \
+ *         --stagingLocation=gs://beamsql_tpcds_1/staging \
+ *         --tempLocation=gs://beamsql_tpcds_2/temp \
+ *         --runner=DataflowRunner \
+ *         --region=us-west1 \
+ *         --maxNumWorkers=10"
+ */
+public class BeamTpcds {
+    private static final String dataDirectory = "gs://beamsql_tpcds_1/data";
+    private static final String resultDirectory = "gs://beamsql_tpcds_1/tpcds_results";
+
+    private static String buildTableCreateStatement(String tableName) {
+        String createStatement = "CREATE EXTERNAL TABLE " + tableName + " (%s) TYPE text LOCATION '%s' TBLPROPERTIES '{\"format\":\"csv\", \"csvformat\": \"InformixUnload\"}'";
+        return createStatement;
+    }
+
+    private static String buildDataLocation(String dataSize, String tableName) {
+        String dataLocation = dataDirectory + "/" + dataSize + "/" + tableName + ".dat";
+        return dataLocation;
+    }
+
+    /** Register all tables into env, set their schemas, and set the locations where their corresponding data are stored. */
+    private static void registerAllTables(BeamSqlEnv env, String dataSize) throws Exception {
+        List<String> tableNames = TableSchemaJSONLoader.getAllTableNames();
+        for (String tableName : tableNames) {
+            String createStatement = buildTableCreateStatement(tableName);
+            String tableSchema = TableSchemaJSONLoader.parseTableSchema(tableName);
+            String dataLocation = buildDataLocation(dataSize, tableName);
+            env.executeDdl(String.format(createStatement, tableSchema, dataLocation));
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        InMemoryMetaStore inMemoryMetaStore = new InMemoryMetaStore();
+        inMemoryMetaStore.registerProvider(new TextTableProvider());
+
+        TpcdsOptions tpcdsOptions = PipelineOptionsFactory.fromArgs(args).withValidation().as(TpcdsOptions.class);
+
+        String dataSize = TpcdsParametersReader.getAndCheckDataSize(tpcdsOptions);
+        String[] queryNameArr = TpcdsParametersReader.getAndCheckQueryNameArray(tpcdsOptions);
+        int nThreads = TpcdsParametersReader.getAndCheckTpcParallel(tpcdsOptions);
+
+        // Using ExecutorService and CompletionService to fulfill multi-threading functionality
+        ExecutorService executor = Executors.newFixedThreadPool(nThreads);
+        CompletionService<PipelineResult> completion = new ExecutorCompletionService<>(executor);
+
+        BeamSqlEnv env =
+                BeamSqlEnv
+                        .builder(inMemoryMetaStore)
+                        .setPipelineOptions(tpcdsOptions)
+                        .build();
+
+        registerAllTables(env, dataSize);
+
+        // Make an array of pipelines, each pipeline is responsible for running a corresponding query.
+        Pipeline[] pipelines = new Pipeline[queryNameArr.length];
+
+        // Execute all queries, transform the each result into a PCollection<String>, write them into the txt file and store in a GCP directory.
+        for (int i = 0; i < queryNameArr.length; i++) {
+            // For each query, get a copy of pipelineOptions from command line arguments, cast tpcdsOptions as a DataflowPipelineOptions object to read and set required parameters for pipeline execution.
+            TpcdsOptions tpcdsOptionsCopy = PipelineOptionsFactory.fromArgs(args).withValidation().as(TpcdsOptions.class);
+            DataflowPipelineOptions dataflowPipelineOptionsCopy = tpcdsOptionsCopy.as(DataflowPipelineOptions.class);
+
+            // Set a unique job name using the time stamp so that multiple different pipelines can run together.
+            dataflowPipelineOptionsCopy.setJobName(queryNameArr[i] + "result" + System.currentTimeMillis());
+
+            pipelines[i] = Pipeline.create(dataflowPipelineOptionsCopy);
+            String queryString = QueryReader.readQuery(queryNameArr[i]);
+
+            // Query execution
+            PCollection<Row> rows = BeamSqlRelUtils.toPCollection(pipelines[i], env.parseQuery(queryString));
+
+            // Transform the result from PCollection<Row> into PCollection<String>, and write it to the location where results are stored.
+            PCollection<String> rowStrings = rows.apply(MapElements
+                    .into(TypeDescriptors.strings())
+                    .via((Row row) -> row.toString()));
+            rowStrings.apply(TextIO.write().to(resultDirectory + "/" + dataSize + "/" + pipelines[i].getOptions().getJobName()).withSuffix(".txt").withNumShards(1));
+
+            completion.submit(new TpcdsRun(pipelines[i]));
+        }
+
+        executor.shutdown();
+    }
+}
diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/QueryReader.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/QueryReader.java
new file mode 100644
index 0000000..1666c78
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/QueryReader.java
@@ -0,0 +1,59 @@
+/*
+ * 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.tpcds;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.util.Objects;
+
+/**
+ * The QueryReader reads query file (the file's extension is '.sql' and content doesn't end with a ';'), write the query as a string and return it.
+ */
+public class QueryReader {
+    /**
+     * Reads a query file (.sql), return the query as a string.
+     * @param queryFileName The name of the query file (such as "query1, query5...") which is stored in resource/queries directory
+     * @return The query string stored in this file.
+     * @throws Exception
+     */
+    public static String readQuery(String queryFileName) throws Exception {
+        // Prepare the file reader.
+        String queryFilePath = Objects.requireNonNull(QueryReader.class.getClassLoader().getResource("queries/" + queryFileName + ".sql")).getPath();
+        File queryFile = new File(queryFilePath);
+        FileReader fileReader = new FileReader(queryFile);
+        BufferedReader reader = new BufferedReader(fileReader);
+
+        // Read the file into stringBuilder.
+        StringBuilder stringBuilder = new StringBuilder();
+        String line;
+        String ls = System.getProperty("line.separator");
+        while ((line = reader.readLine()) != null) {
+            stringBuilder.append(line);
+            stringBuilder.append(ls);
+        }
+
+        // Delete the last new line separator.
+        stringBuilder.deleteCharAt(stringBuilder.length() - 1);
+        reader.close();
+
+        String queryString = stringBuilder.toString();
+
+        return queryString;
+    }
+}
diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoader.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoader.java
new file mode 100644
index 0000000..420386c
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoader.java
@@ -0,0 +1,112 @@
+/*
+ * 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.tpcds;
+
+import org.apache.beam.repackaged.core.org.apache.commons.compress.utils.FileNameUtils;
+import org.json.simple.JSONArray;
+import org.json.simple.JSONObject;
+import org.json.simple.parser.JSONParser;
+
+import java.io.File;
+import java.io.FileReader;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.ArrayList;
+
+
+/**
+ * TableSchemaJSONLoader can get all table's names from resource/schemas directory and parse a table's schema into a string.
+ */
+public class TableSchemaJSONLoader {
+    /**
+     * Read a table schema json file from resource/schemas directory, parse the file into a string which can be utilized by BeamSqlEnv.executeDdl method.
+     * @param tableName The name of the json file to be read (fo example: item, store_sales).
+     * @return A string that matches the format in BeamSqlEnv.executeDdl method, such as "d_date_sk bigint, d_date_id varchar"
+     * @throws Exception
+     */
+    public static String parseTableSchema(String tableName) throws Exception {
+        String tableFilePath = Objects.requireNonNull(TableSchemaJSONLoader.class.getClassLoader().getResource("schemas/" + tableName +".json")).getPath();
+
+        JSONObject jsonObject = (JSONObject) new JSONParser().parse(new FileReader(new File(tableFilePath)));
+        JSONArray jsonArray = (JSONArray) jsonObject.get("schema");
+
+        // Iterate each element in jsonArray to construct the schema string
+        StringBuilder schemaStringBuilder = new StringBuilder();
+
+        Iterator jsonArrIterator = jsonArray.iterator();
+        Iterator<Map.Entry> recordIterator;
+        while (jsonArrIterator.hasNext()) {
+            recordIterator = ((Map) jsonArrIterator.next()).entrySet().iterator();
+            while (recordIterator.hasNext()) {
+                Map.Entry pair = recordIterator.next();
+
+                if (pair.getKey().equals("type")) {
+                    // If the key of the pair is "type", make some modification before appending it to the schemaStringBuilder, then append a comma.
+                    String typeName = (String) pair.getValue();
+                    if (typeName.toLowerCase().equals("identifier") || typeName.toLowerCase().equals("integer")) {
+                        // Use long type to represent int, prevent overflow
+                        schemaStringBuilder.append("bigint");
+                    } else if (typeName.contains("decimal")) {
+                        // Currently Beam SQL doesn't handle "decimal" type properly, use "double" to replace it for now.
+                        schemaStringBuilder.append("double");
+                    } else {
+                        // Currently Beam SQL doesn't handle "date" type properly, use "varchar" replace it for now.
+                        schemaStringBuilder.append("varchar");
+                    }
+                    schemaStringBuilder.append(',');
+                } else {
+                    // If the key of the pair is "name", directly append it to the StringBuilder, then append a space.
+                    schemaStringBuilder.append((pair.getValue()));
+                    schemaStringBuilder.append(' ');
+                }
+            }
+        }
+
+        // Delete the last ',' in schema string
+        if (schemaStringBuilder.length() > 0) {
+            schemaStringBuilder.deleteCharAt(schemaStringBuilder.length() - 1);
+        }
+
+        String schemaString = schemaStringBuilder.toString();
+
+        return schemaString;
+    }
+
+    /**
+     * Get all tables' names. Tables are stored in resource/schemas directory in the form of json files, such as "item.json", "store_sales.json", they'll be converted to "item", "store_sales".
+     * @return The list of names of all tables.
+     */
+    public static List<String> getAllTableNames() {
+        String tableDirPath = Objects.requireNonNull(TableSchemaJSONLoader.class.getClassLoader().getResource("schemas")).getPath();
+        File tableDir = new File(tableDirPath);
+        File[] tableDirListing = tableDir.listFiles();
+
+        List<String> tableNames = new ArrayList<>();
+
+        if (tableDirListing != null) {
+            for (File file : tableDirListing) {
+                // Remove the .json extension in file name
+                tableNames.add(FileNameUtils.getBaseName((file.getName())));
+            }
+        }
+
+        return tableNames;
+    }
+}
diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsOptions.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsOptions.java
new file mode 100644
index 0000000..1c567dd
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsOptions.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.tpcds;
+
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+/** Options used to configure TPC-DS test */
+public interface TpcdsOptions extends PipelineOptions {
+    @Description("The size of TPC-DS data to run query on, user input should contain the unit, such as '1G', '10G'")
+    String getDataSize();
+
+    void setDataSize(String dataSize);
+
+    // Set the return type to be String since reading from the command line (user input will be like "1,2,55" which represent TPC-DS query1, query3, query55)
+    @Description("The queries numbers, read user input as string, numbers separated by commas")
+    String getQueries();
+
+    void setQueries(String queries);
+
+    @Description("The number of queries to run in parallel")
+    @Default.Integer(1)
+    Integer getTpcParallel();
+
+    void setTpcParallel(Integer parallelism);
+}
diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsOptionsRegistrar.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsOptionsRegistrar.java
new file mode 100644
index 0000000..d1ddc9d
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsOptionsRegistrar.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.tpcds;
+
+import com.google.auto.service.AutoService;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
+import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
+
+/** {@link AutoService} registrar for {@link TpcdsOptions}. */
+@AutoService(PipelineOptionsRegistrar.class)
+public class TpcdsOptionsRegistrar implements PipelineOptionsRegistrar{
+
+    @Override
+    public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
+        return ImmutableList.of(TpcdsOptions.class);
+    }
+}
diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsParametersReader.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsParametersReader.java
new file mode 100644
index 0000000..7f0e147
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsParametersReader.java
@@ -0,0 +1,108 @@
+/*
+ * 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.tpcds;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Get and check the TpcdsOptions' parameters, throw exceptions when user input is invalid
+ */
+public class TpcdsParametersReader {
+
+    /** The data sizes that have been supported. */
+    private static final Set<String> supportedDataSizes = Stream.of("1G", "10G").collect(Collectors.toCollection(HashSet::new));
+
+    /**
+     * Get and check dataSize entered by user. This dataSize has to have been supported.
+     *
+     * @param tpcdsOptions TpcdsOptions object constructed from user input
+     * @return The dateSize user entered, if it is contained in supportedDataSizes set.
+     * @throws Exception
+     */
+    public static String getAndCheckDataSize(TpcdsOptions tpcdsOptions) throws Exception {
+        String dataSize = tpcdsOptions.getDataSize();
+
+        if (!supportedDataSizes.contains(dataSize)) {
+            throw new Exception("The data size you entered has not been supported.");
+        }
+
+        return dataSize;
+    }
+
+    /**
+     * Get and check queries entered by user. This has to be a string of numbers separated by commas or "all" which means run all 99 queiries.
+     * All query numbers have to be between 1 and 99.
+     *
+     * @param tpcdsOptions TpcdsOptions object constructed from user input
+     * @return An array of query names, for example "1,2,7" will be output as "query1,query2,query7"
+     * @throws Exception
+     */
+    public static String[] getAndCheckQueryNameArray(TpcdsOptions tpcdsOptions) throws Exception {
+        String queryNums = tpcdsOptions.getQueries();
+
+        String[] queryNumArr;
+        if (queryNums.toLowerCase().equals("all")) {
+            // All 99 TPC-DS queries need to be executed.
+            queryNumArr = new String[99];
+            for (int i = 0; i < 99; i++) {
+                queryNumArr[i] = Integer.toString(i + 1);
+            }
+        } else {
+            // Split user input queryNums by spaces and commas, get an array of all query numbers.
+            queryNumArr = queryNums.split("[\\s,]+");
+
+            for (String queryNumStr : queryNumArr) {
+                try {
+                    int queryNum = Integer.parseInt(queryNumStr);
+                    if (queryNum < 1 || queryNum > 99) {
+                        throw new Exception("The queries you entered contains invalid query number, please provide integers between 1 and 99.");
+                    }
+                } catch (NumberFormatException e) {
+                    System.out.println("The queries you entered should be integers, please provide integers between 1 and 99.");
+                }
+            }
+        }
+
+        String[] queryNameArr = new String[queryNumArr.length];
+        for (int i = 0; i < queryNumArr.length; i++) {
+            queryNameArr[i] = "query" + queryNumArr[i];
+        }
+
+        return queryNameArr;
+    }
+
+    /**
+     * Get and check TpcParallel entered by user. This has to be an integer between 1 and 99.
+     *
+     * @param tpcdsOptions TpcdsOptions object constructed from user input.
+     * @return The TpcParallel user entered.
+     * @throws Exception
+     */
+    public static int getAndCheckTpcParallel(TpcdsOptions tpcdsOptions) throws Exception {
+        int nThreads = tpcdsOptions.getTpcParallel();
+
+        if (nThreads < 1 || nThreads > 99) {
+            throw new Exception("The TpcParallel your entered is invalid, please provide an integer between 1 and 99.");
+        }
+
+        return nThreads;
+    }
+}
diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsRun.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsRun.java
new file mode 100644
index 0000000..936c24f
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsRun.java
@@ -0,0 +1,40 @@
+/*
+ * 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.tpcds;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import java.util.concurrent.Callable;
+
+/**
+ * To fulfill multi-threaded execution
+ */
+public class TpcdsRun implements Callable<PipelineResult> {
+    private final Pipeline pipeline;
+
+    public TpcdsRun (Pipeline pipeline) {
+        this.pipeline = pipeline;
+    }
+
+    @Override
+    public PipelineResult call() {
+        PipelineResult pipelineResult = pipeline.run();
+        pipelineResult.waitUntilFinish();
+        return pipelineResult;
+    }
+}
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query1.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query1.sql
new file mode 100644
index 0000000..3cdf4ca
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query1.sql
@@ -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.
+
+with customer_total_return as
+(select sr_customer_sk as ctr_customer_sk
+,sr_store_sk as ctr_store_sk
+,sum(SR_FEE) as ctr_total_return
+from store_returns
+,date_dim
+where sr_returned_date_sk = d_date_sk
+and d_year =2000
+group by sr_customer_sk
+,sr_store_sk)
+ select  c_customer_id
+from customer_total_return ctr1
+,store
+,customer
+where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2
+from customer_total_return ctr2
+where ctr1.ctr_store_sk = ctr2.ctr_store_sk)
+and s_store_sk = ctr1.ctr_store_sk
+and s_state = 'TN'
+and ctr1.ctr_customer_sk = c_customer_sk
+order by c_customer_id
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query10.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query10.sql
new file mode 100644
index 0000000..d12ef0d
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query10.sql
@@ -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.
+
+select
+  cd_gender,
+  cd_marital_status,
+  cd_education_status,
+  count(*) cnt1,
+  cd_purchase_estimate,
+  count(*) cnt2,
+  cd_credit_rating,
+  count(*) cnt3,
+  cd_dep_count,
+  count(*) cnt4,
+  cd_dep_employed_count,
+  count(*) cnt5,
+  cd_dep_college_count,
+  count(*) cnt6
+ from
+  customer c,customer_address ca,customer_demographics
+ where
+  c.c_current_addr_sk = ca.ca_address_sk and
+  ca_county in ('Walker County','Richland County','Gaines County','Douglas County','Dona Ana County') and
+  cd_demo_sk = c.c_current_cdemo_sk and 
+  exists (select *
+          from store_sales,date_dim
+          where c.c_customer_sk = ss_customer_sk and
+                ss_sold_date_sk = d_date_sk and
+                d_year = 2002 and
+                d_moy between 4 and 4+3) and
+   (exists (select *
+            from web_sales,date_dim
+            where c.c_customer_sk = ws_bill_customer_sk and
+                  ws_sold_date_sk = d_date_sk and
+                  d_year = 2002 and
+                  d_moy between 4 ANd 4+3) or 
+    exists (select * 
+            from catalog_sales,date_dim
+            where c.c_customer_sk = cs_ship_customer_sk and
+                  cs_sold_date_sk = d_date_sk and
+                  d_year = 2002 and
+                  d_moy between 4 and 4+3))
+ group by cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating,
+          cd_dep_count,
+          cd_dep_employed_count,
+          cd_dep_college_count
+ order by cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating,
+          cd_dep_count,
+          cd_dep_employed_count,
+          cd_dep_college_count
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query11.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query11.sql
new file mode 100644
index 0000000..3955094
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query11.sql
@@ -0,0 +1,94 @@
+-- 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.
+
+with year_total as (
+ select c_customer_id customer_id
+       ,c_first_name customer_first_name
+       ,c_last_name customer_last_name
+       ,c_preferred_cust_flag customer_preferred_cust_flag
+       ,c_birth_country customer_birth_country
+       ,c_login customer_login
+       ,c_email_address customer_email_address
+       ,d_year dyear
+       ,sum(ss_ext_list_price-ss_ext_discount_amt) year_total
+       ,'s' sale_type
+ from customer
+     ,store_sales
+     ,date_dim
+ where c_customer_sk = ss_customer_sk
+   and ss_sold_date_sk = d_date_sk
+ group by c_customer_id
+         ,c_first_name
+         ,c_last_name
+         ,c_preferred_cust_flag 
+         ,c_birth_country
+         ,c_login
+         ,c_email_address
+         ,d_year 
+ union all
+ select c_customer_id customer_id
+       ,c_first_name customer_first_name
+       ,c_last_name customer_last_name
+       ,c_preferred_cust_flag customer_preferred_cust_flag
+       ,c_birth_country customer_birth_country
+       ,c_login customer_login
+       ,c_email_address customer_email_address
+       ,d_year dyear
+       ,sum(ws_ext_list_price-ws_ext_discount_amt) year_total
+       ,'w' sale_type
+ from customer
+     ,web_sales
+     ,date_dim
+ where c_customer_sk = ws_bill_customer_sk
+   and ws_sold_date_sk = d_date_sk
+ group by c_customer_id
+         ,c_first_name
+         ,c_last_name
+         ,c_preferred_cust_flag 
+         ,c_birth_country
+         ,c_login
+         ,c_email_address
+         ,d_year
+         )
+  select  
+                  t_s_secyear.customer_id
+                 ,t_s_secyear.customer_first_name
+                 ,t_s_secyear.customer_last_name
+                 ,t_s_secyear.customer_email_address
+ from year_total t_s_firstyear
+     ,year_total t_s_secyear
+     ,year_total t_w_firstyear
+     ,year_total t_w_secyear
+ where t_s_secyear.customer_id = t_s_firstyear.customer_id
+         and t_s_firstyear.customer_id = t_w_secyear.customer_id
+         and t_s_firstyear.customer_id = t_w_firstyear.customer_id
+         and t_s_firstyear.sale_type = 's'
+         and t_w_firstyear.sale_type = 'w'
+         and t_s_secyear.sale_type = 's'
+         and t_w_secyear.sale_type = 'w'
+         and t_s_firstyear.dyear = 2001
+         and t_s_secyear.dyear = 2001+1
+         and t_w_firstyear.dyear = 2001
+         and t_w_secyear.dyear = 2001+1
+         and t_s_firstyear.year_total > 0
+         and t_w_firstyear.year_total > 0
+         and case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else 0.0 end
+             > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else 0.0 end
+ order by t_s_secyear.customer_id
+         ,t_s_secyear.customer_first_name
+         ,t_s_secyear.customer_last_name
+         ,t_s_secyear.customer_email_address
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query12.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query12.sql
new file mode 100644
index 0000000..c015bff
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query12.sql
@@ -0,0 +1,47 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select  i_item_id
+      ,i_item_desc 
+      ,i_category 
+      ,i_class 
+      ,i_current_price
+      ,sum(ws_ext_sales_price) as itemrevenue 
+      ,sum(ws_ext_sales_price)*100/sum(sum(ws_ext_sales_price)) over
+          (partition by i_class) as revenueratio
+from	
+	web_sales
+    	,item 
+    	,date_dim
+where 
+	ws_item_sk = i_item_sk 
+  	and i_category in ('Jewelry', 'Sports', 'Books')
+  	and ws_sold_date_sk = d_date_sk
+	and d_date between cast('2001-01-12' as date) 
+				and (cast('2001-01-12' as date) + 30 days)
+group by 
+	i_item_id
+        ,i_item_desc 
+        ,i_category
+        ,i_class
+        ,i_current_price
+order by 
+	i_category
+        ,i_class
+        ,i_item_id
+        ,i_item_desc
+        ,revenueratio
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query13.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query13.sql
new file mode 100644
index 0000000..47fa265
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query13.sql
@@ -0,0 +1,64 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select avg(ss_quantity)
+       ,avg(ss_ext_sales_price)
+       ,avg(ss_ext_wholesale_cost)
+       ,sum(ss_ext_wholesale_cost)
+ from store_sales
+     ,store
+     ,customer_demographics
+     ,household_demographics
+     ,customer_address
+     ,date_dim
+ where s_store_sk = ss_store_sk
+ and  ss_sold_date_sk = d_date_sk and d_year = 2001
+ and((ss_hdemo_sk=hd_demo_sk
+  and cd_demo_sk = ss_cdemo_sk
+  and cd_marital_status = 'D'
+  and cd_education_status = '2 yr Degree'
+  and ss_sales_price between 100.00 and 150.00
+  and hd_dep_count = 3   
+     )or
+     (ss_hdemo_sk=hd_demo_sk
+  and cd_demo_sk = ss_cdemo_sk
+  and cd_marital_status = 'S'
+  and cd_education_status = 'Secondary'
+  and ss_sales_price between 50.00 and 100.00   
+  and hd_dep_count = 1
+     ) or 
+     (ss_hdemo_sk=hd_demo_sk
+  and cd_demo_sk = ss_cdemo_sk
+  and cd_marital_status = 'W'
+  and cd_education_status = 'Advanced Degree'
+  and ss_sales_price between 150.00 and 200.00 
+  and hd_dep_count = 1  
+     ))
+ and((ss_addr_sk = ca_address_sk
+  and ca_country = 'United States'
+  and ca_state in ('CO', 'IL', 'MN')
+  and ss_net_profit between 100 and 200  
+     ) or
+     (ss_addr_sk = ca_address_sk
+  and ca_country = 'United States'
+  and ca_state in ('OH', 'MT', 'NM')
+  and ss_net_profit between 150 and 300  
+     ) or
+     (ss_addr_sk = ca_address_sk
+  and ca_country = 'United States'
+  and ca_state in ('TX', 'MO', 'MI')
+  and ss_net_profit between 50 and 250  
+     ))
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query14.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query14.sql
new file mode 100644
index 0000000..8d9de3c
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query14.sql
@@ -0,0 +1,223 @@
+-- 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.
+
+with  cross_items as
+ (select i_item_sk ss_item_sk
+ from item,
+ (select iss.i_brand_id brand_id
+     ,iss.i_class_id class_id
+     ,iss.i_category_id category_id
+ from store_sales
+     ,item iss
+     ,date_dim d1
+ where ss_item_sk = iss.i_item_sk
+   and ss_sold_date_sk = d1.d_date_sk
+   and d1.d_year between 1998 AND 1998 + 2
+ intersect 
+ select ics.i_brand_id
+     ,ics.i_class_id
+     ,ics.i_category_id
+ from catalog_sales
+     ,item ics
+     ,date_dim d2
+ where cs_item_sk = ics.i_item_sk
+   and cs_sold_date_sk = d2.d_date_sk
+   and d2.d_year between 1998 AND 1998 + 2
+ intersect
+ select iws.i_brand_id
+     ,iws.i_class_id
+     ,iws.i_category_id
+ from web_sales
+     ,item iws
+     ,date_dim d3
+ where ws_item_sk = iws.i_item_sk
+   and ws_sold_date_sk = d3.d_date_sk
+   and d3.d_year between 1998 AND 1998 + 2)
+ where i_brand_id = brand_id
+      and i_class_id = class_id
+      and i_category_id = category_id
+),
+ avg_sales as
+ (select avg(quantity*list_price) average_sales
+  from (select ss_quantity quantity
+             ,ss_list_price list_price
+       from store_sales
+           ,date_dim
+       where ss_sold_date_sk = d_date_sk
+         and d_year between 1998 and 1998 + 2
+       union all 
+       select cs_quantity quantity 
+             ,cs_list_price list_price
+       from catalog_sales
+           ,date_dim
+       where cs_sold_date_sk = d_date_sk
+         and d_year between 1998 and 1998 + 2 
+       union all
+       select ws_quantity quantity
+             ,ws_list_price list_price
+       from web_sales
+           ,date_dim
+       where ws_sold_date_sk = d_date_sk
+         and d_year between 1998 and 1998 + 2) x)
+  select  channel, i_brand_id,i_class_id,i_category_id,sum(sales), sum(number_sales)
+ from(
+       select 'store' channel, i_brand_id,i_class_id
+             ,i_category_id,sum(ss_quantity*ss_list_price) sales
+             , count(*) number_sales
+       from store_sales
+           ,item
+           ,date_dim
+       where ss_item_sk in (select ss_item_sk from cross_items)
+         and ss_item_sk = i_item_sk
+         and ss_sold_date_sk = d_date_sk
+         and d_year = 1998+2 
+         and d_moy = 11
+       group by i_brand_id,i_class_id,i_category_id
+       having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales)
+       union all
+       select 'catalog' channel, i_brand_id,i_class_id,i_category_id, sum(cs_quantity*cs_list_price) sales, count(*) number_sales
+       from catalog_sales
+           ,item
+           ,date_dim
+       where cs_item_sk in (select ss_item_sk from cross_items)
+         and cs_item_sk = i_item_sk
+         and cs_sold_date_sk = d_date_sk
+         and d_year = 1998+2 
+         and d_moy = 11
+       group by i_brand_id,i_class_id,i_category_id
+       having sum(cs_quantity*cs_list_price) > (select average_sales from avg_sales)
+       union all
+       select 'web' channel, i_brand_id,i_class_id,i_category_id, sum(ws_quantity*ws_list_price) sales , count(*) number_sales
+       from web_sales
+           ,item
+           ,date_dim
+       where ws_item_sk in (select ss_item_sk from cross_items)
+         and ws_item_sk = i_item_sk
+         and ws_sold_date_sk = d_date_sk
+         and d_year = 1998+2
+         and d_moy = 11
+       group by i_brand_id,i_class_id,i_category_id
+       having sum(ws_quantity*ws_list_price) > (select average_sales from avg_sales)
+ ) y
+ group by rollup (channel, i_brand_id,i_class_id,i_category_id)
+ order by channel,i_brand_id,i_class_id,i_category_id
+ limit 100;
+with  cross_items as
+ (select i_item_sk ss_item_sk
+ from item,
+ (select iss.i_brand_id brand_id
+     ,iss.i_class_id class_id
+     ,iss.i_category_id category_id
+ from store_sales
+     ,item iss
+     ,date_dim d1
+ where ss_item_sk = iss.i_item_sk
+   and ss_sold_date_sk = d1.d_date_sk
+   and d1.d_year between 1998 AND 1998 + 2
+ intersect
+ select ics.i_brand_id
+     ,ics.i_class_id
+     ,ics.i_category_id
+ from catalog_sales
+     ,item ics
+     ,date_dim d2
+ where cs_item_sk = ics.i_item_sk
+   and cs_sold_date_sk = d2.d_date_sk
+   and d2.d_year between 1998 AND 1998 + 2
+ intersect
+ select iws.i_brand_id
+     ,iws.i_class_id
+     ,iws.i_category_id
+ from web_sales
+     ,item iws
+     ,date_dim d3
+ where ws_item_sk = iws.i_item_sk
+   and ws_sold_date_sk = d3.d_date_sk
+   and d3.d_year between 1998 AND 1998 + 2) x
+ where i_brand_id = brand_id
+      and i_class_id = class_id
+      and i_category_id = category_id
+),
+ avg_sales as
+(select avg(quantity*list_price) average_sales
+  from (select ss_quantity quantity
+             ,ss_list_price list_price
+       from store_sales
+           ,date_dim
+       where ss_sold_date_sk = d_date_sk
+         and d_year between 1998 and 1998 + 2
+       union all
+       select cs_quantity quantity
+             ,cs_list_price list_price
+       from catalog_sales
+           ,date_dim
+       where cs_sold_date_sk = d_date_sk
+         and d_year between 1998 and 1998 + 2
+       union all
+       select ws_quantity quantity
+             ,ws_list_price list_price
+       from web_sales
+           ,date_dim
+       where ws_sold_date_sk = d_date_sk
+         and d_year between 1998 and 1998 + 2) x)
+  select  this_year.channel ty_channel
+                           ,this_year.i_brand_id ty_brand
+                           ,this_year.i_class_id ty_class
+                           ,this_year.i_category_id ty_category
+                           ,this_year.sales ty_sales
+                           ,this_year.number_sales ty_number_sales
+                           ,last_year.channel ly_channel
+                           ,last_year.i_brand_id ly_brand
+                           ,last_year.i_class_id ly_class
+                           ,last_year.i_category_id ly_category
+                           ,last_year.sales ly_sales
+                           ,last_year.number_sales ly_number_sales 
+ from
+ (select 'store' channel, i_brand_id,i_class_id,i_category_id
+        ,sum(ss_quantity*ss_list_price) sales, count(*) number_sales
+ from store_sales 
+     ,item
+     ,date_dim
+ where ss_item_sk in (select ss_item_sk from cross_items)
+   and ss_item_sk = i_item_sk
+   and ss_sold_date_sk = d_date_sk
+   and d_week_seq = (select d_week_seq
+                     from date_dim
+                     where d_year = 1998 + 1
+                       and d_moy = 12
+                       and d_dom = 16)
+ group by i_brand_id,i_class_id,i_category_id
+ having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales)) this_year,
+ (select 'store' channel, i_brand_id,i_class_id
+        ,i_category_id, sum(ss_quantity*ss_list_price) sales, count(*) number_sales
+ from store_sales
+     ,item
+     ,date_dim
+ where ss_item_sk in (select ss_item_sk from cross_items)
+   and ss_item_sk = i_item_sk
+   and ss_sold_date_sk = d_date_sk
+   and d_week_seq = (select d_week_seq
+                     from date_dim
+                     where d_year = 1998
+                       and d_moy = 12
+                       and d_dom = 16)
+ group by i_brand_id,i_class_id,i_category_id
+ having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales)) last_year
+ where this_year.i_brand_id= last_year.i_brand_id
+   and this_year.i_class_id = last_year.i_class_id
+   and this_year.i_category_id = last_year.i_category_id
+ order by this_year.channel, this_year.i_brand_id, this_year.i_class_id, this_year.i_category_id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query15.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query15.sql
new file mode 100644
index 0000000..1ae0c37
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query15.sql
@@ -0,0 +1,33 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select  ca_zip
+       ,sum(cs_sales_price)
+ from catalog_sales
+     ,customer
+     ,customer_address
+     ,date_dim
+ where cs_bill_customer_sk = c_customer_sk
+ 	and c_current_addr_sk = ca_address_sk 
+ 	and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475',
+                                   '85392', '85460', '80348', '81792')
+ 	      or ca_state in ('CA','WA','GA')
+ 	      or cs_sales_price > 500)
+ 	and cs_sold_date_sk = d_date_sk
+ 	and d_qoy = 2 and d_year = 2000
+ group by ca_zip
+ order by ca_zip
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query16.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query16.sql
new file mode 100644
index 0000000..54b7164
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query16.sql
@@ -0,0 +1,44 @@
+-- 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.
+
+select
+   count(distinct cs_order_number) as "order count"
+  ,sum(cs_ext_ship_cost) as "total shipping cost"
+  ,sum(cs_net_profit) as "total net profit"
+from
+   catalog_sales cs1
+  ,date_dim
+  ,customer_address
+  ,call_center
+where
+    d_date between '1999-2-01' and 
+           (cast('1999-2-01' as date) + 60 days)
+and cs1.cs_ship_date_sk = d_date_sk
+and cs1.cs_ship_addr_sk = ca_address_sk
+and ca_state = 'IL'
+and cs1.cs_call_center_sk = cc_call_center_sk
+and cc_county in ('Williamson County','Williamson County','Williamson County','Williamson County',
+                  'Williamson County'
+)
+and exists (select *
+            from catalog_sales cs2
+            where cs1.cs_order_number = cs2.cs_order_number
+              and cs1.cs_warehouse_sk <> cs2.cs_warehouse_sk)
+and not exists(select *
+               from catalog_returns cr1
+               where cs1.cs_order_number = cr1.cr_order_number)
+order by count(distinct cs_order_number)
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query17.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query17.sql
new file mode 100644
index 0000000..19ae6b5
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query17.sql
@@ -0,0 +1,58 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select  i_item_id
+       ,i_item_desc
+       ,s_state
+       ,count(ss_quantity) as store_sales_quantitycount
+       ,avg(ss_quantity) as store_sales_quantityave
+       ,stddev_samp(ss_quantity) as store_sales_quantitystdev
+       ,stddev_samp(ss_quantity)/avg(ss_quantity) as store_sales_quantitycov
+       ,count(sr_return_quantity) as store_returns_quantitycount
+       ,avg(sr_return_quantity) as store_returns_quantityave
+       ,stddev_samp(sr_return_quantity) as store_returns_quantitystdev
+       ,stddev_samp(sr_return_quantity)/avg(sr_return_quantity) as store_returns_quantitycov
+       ,count(cs_quantity) as catalog_sales_quantitycount ,avg(cs_quantity) as catalog_sales_quantityave
+       ,stddev_samp(cs_quantity) as catalog_sales_quantitystdev
+       ,stddev_samp(cs_quantity)/avg(cs_quantity) as catalog_sales_quantitycov
+ from store_sales
+     ,store_returns
+     ,catalog_sales
+     ,date_dim d1
+     ,date_dim d2
+     ,date_dim d3
+     ,store
+     ,item
+ where d1.d_quarter_name = '1998Q1'
+   and d1.d_date_sk = ss_sold_date_sk
+   and i_item_sk = ss_item_sk
+   and s_store_sk = ss_store_sk
+   and ss_customer_sk = sr_customer_sk
+   and ss_item_sk = sr_item_sk
+   and ss_ticket_number = sr_ticket_number
+   and sr_returned_date_sk = d2.d_date_sk
+   and d2.d_quarter_name in ('1998Q1','1998Q2','1998Q3')
+   and sr_customer_sk = cs_bill_customer_sk
+   and sr_item_sk = cs_item_sk
+   and cs_sold_date_sk = d3.d_date_sk
+   and d3.d_quarter_name in ('1998Q1','1998Q2','1998Q3')
+ group by i_item_id
+         ,i_item_desc
+         ,s_state
+ order by i_item_id
+         ,i_item_desc
+         ,s_state
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query18.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query18.sql
new file mode 100644
index 0000000..0f03060
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query18.sql
@@ -0,0 +1,47 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select  i_item_id,
+        ca_country,
+        ca_state, 
+        ca_county,
+        avg( cast(cs_quantity as decimal(12,2))) agg1,
+        avg( cast(cs_list_price as decimal(12,2))) agg2,
+        avg( cast(cs_coupon_amt as decimal(12,2))) agg3,
+        avg( cast(cs_sales_price as decimal(12,2))) agg4,
+        avg( cast(cs_net_profit as decimal(12,2))) agg5,
+        avg( cast(c_birth_year as decimal(12,2))) agg6,
+        avg( cast(cd1.cd_dep_count as decimal(12,2))) agg7
+ from catalog_sales, customer_demographics cd1, 
+      customer_demographics cd2, customer, customer_address, date_dim, item
+ where cs_sold_date_sk = d_date_sk and
+       cs_item_sk = i_item_sk and
+       cs_bill_cdemo_sk = cd1.cd_demo_sk and
+       cs_bill_customer_sk = c_customer_sk and
+       cd1.cd_gender = 'M' and 
+       cd1.cd_education_status = 'College' and
+       c_current_cdemo_sk = cd2.cd_demo_sk and
+       c_current_addr_sk = ca_address_sk and
+       c_birth_month in (9,5,12,4,1,10) and
+       d_year = 2001 and
+       ca_state in ('ND','WI','AL'
+                   ,'NC','OK','MS','TN')
+ group by rollup (i_item_id, ca_country, ca_state, ca_county)
+ order by ca_country,
+        ca_state, 
+        ca_county,
+	i_item_id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query19.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query19.sql
new file mode 100644
index 0000000..7a85a10
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query19.sql
@@ -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.
+
+select  i_brand_id brand_id, i_brand brand, i_manufact_id, i_manufact,
+ 	sum(ss_ext_sales_price) ext_price
+ from date_dim, store_sales, item,customer,customer_address,store
+ where d_date_sk = ss_sold_date_sk
+   and ss_item_sk = i_item_sk
+   and i_manager_id=7
+   and d_moy=11
+   and d_year=1999
+   and ss_customer_sk = c_customer_sk 
+   and c_current_addr_sk = ca_address_sk
+   and substr(ca_zip,1,5) <> substr(s_zip,1,5) 
+   and ss_store_sk = s_store_sk 
+ group by i_brand
+      ,i_brand_id
+      ,i_manufact_id
+      ,i_manufact
+ order by ext_price desc
+         ,i_brand
+         ,i_brand_id
+         ,i_manufact_id
+         ,i_manufact
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query2.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query2.sql
new file mode 100644
index 0000000..9fddb0d
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query2.sql
@@ -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.
+
+with wscs as
+ (select sold_date_sk
+        ,sales_price
+  from (select ws_sold_date_sk sold_date_sk
+              ,ws_ext_sales_price sales_price
+        from web_sales 
+        union all
+        select cs_sold_date_sk sold_date_sk
+              ,cs_ext_sales_price sales_price
+        from catalog_sales)),
+ wswscs as 
+ (select d_week_seq,
+        sum(case when (d_day_name='Sunday') then sales_price else null end) sun_sales,
+        sum(case when (d_day_name='Monday') then sales_price else null end) mon_sales,
+        sum(case when (d_day_name='Tuesday') then sales_price else  null end) tue_sales,
+        sum(case when (d_day_name='Wednesday') then sales_price else null end) wed_sales,
+        sum(case when (d_day_name='Thursday') then sales_price else null end) thu_sales,
+        sum(case when (d_day_name='Friday') then sales_price else null end) fri_sales,
+        sum(case when (d_day_name='Saturday') then sales_price else null end) sat_sales
+ from wscs
+     ,date_dim
+ where d_date_sk = sold_date_sk
+ group by d_week_seq)
+ select d_week_seq1
+       ,round(sun_sales1/sun_sales2,2)
+       ,round(mon_sales1/mon_sales2,2)
+       ,round(tue_sales1/tue_sales2,2)
+       ,round(wed_sales1/wed_sales2,2)
+       ,round(thu_sales1/thu_sales2,2)
+       ,round(fri_sales1/fri_sales2,2)
+       ,round(sat_sales1/sat_sales2,2)
+ from
+ (select wswscs.d_week_seq d_week_seq1
+        ,sun_sales sun_sales1
+        ,mon_sales mon_sales1
+        ,tue_sales tue_sales1
+        ,wed_sales wed_sales1
+        ,thu_sales thu_sales1
+        ,fri_sales fri_sales1
+        ,sat_sales sat_sales1
+  from wswscs,date_dim 
+  where date_dim.d_week_seq = wswscs.d_week_seq and
+        d_year = 2001) y,
+ (select wswscs.d_week_seq d_week_seq2
+        ,sun_sales sun_sales2
+        ,mon_sales mon_sales2
+        ,tue_sales tue_sales2
+        ,wed_sales wed_sales2
+        ,thu_sales thu_sales2
+        ,fri_sales fri_sales2
+        ,sat_sales sat_sales2
+  from wswscs
+      ,date_dim 
+  where date_dim.d_week_seq = wswscs.d_week_seq and
+        d_year = 2001+1) z
+ where d_week_seq1=d_week_seq2-53
+ order by d_week_seq1
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query20.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query20.sql
new file mode 100644
index 0000000..95e960b
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query20.sql
@@ -0,0 +1,43 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select  i_item_id
+       ,i_item_desc 
+       ,i_category 
+       ,i_class 
+       ,i_current_price
+       ,sum(cs_ext_sales_price) as itemrevenue 
+       ,sum(cs_ext_sales_price)*100/sum(sum(cs_ext_sales_price)) over
+           (partition by i_class) as revenueratio
+ from	catalog_sales
+     ,item 
+     ,date_dim
+ where cs_item_sk = i_item_sk 
+   and i_category in ('Jewelry', 'Sports', 'Books')
+   and cs_sold_date_sk = d_date_sk
+ and d_date between cast('2001-01-12' as date) 
+ 				and (cast('2001-01-12' as date) + 30 days)
+ group by i_item_id
+         ,i_item_desc 
+         ,i_category
+         ,i_class
+         ,i_current_price
+ order by i_category
+         ,i_class
+         ,i_item_id
+         ,i_item_desc
+         ,revenueratio
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query21.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query21.sql
new file mode 100644
index 0000000..3ba811b
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query21.sql
@@ -0,0 +1,43 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select  *
+ from(select w_warehouse_name
+            ,i_item_id
+            ,sum(case when (cast(d_date as date) < cast ('1998-04-08' as date))
+	                then inv_quantity_on_hand 
+                      else 0 end) as inv_before
+            ,sum(case when (cast(d_date as date) >= cast ('1998-04-08' as date))
+                      then inv_quantity_on_hand 
+                      else 0 end) as inv_after
+   from inventory
+       ,warehouse
+       ,item
+       ,date_dim
+   where i_current_price between 0.99 and 1.49
+     and i_item_sk          = inv_item_sk
+     and inv_warehouse_sk   = w_warehouse_sk
+     and inv_date_sk    = d_date_sk
+     and d_date between (cast ('1998-04-08' as date) - 30 days)
+                    and (cast ('1998-04-08' as date) + 30 days)
+   group by w_warehouse_name, i_item_id) x
+ where (case when inv_before > 0 
+             then inv_after / inv_before 
+             else null
+             end) between 2.0/3.0 and 3.0/2.0
+ order by w_warehouse_name
+         ,i_item_id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query22.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query22.sql
new file mode 100644
index 0000000..e983b7b
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query22.sql
@@ -0,0 +1,33 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select  i_product_name
+             ,i_brand
+             ,i_class
+             ,i_category
+             ,avg(inv_quantity_on_hand) qoh
+       from inventory
+           ,date_dim
+           ,item
+       where inv_date_sk=d_date_sk
+              and inv_item_sk=i_item_sk
+              and d_month_seq between 1212 and 1212 + 11
+       group by rollup(i_product_name
+                       ,i_brand
+                       ,i_class
+                       ,i_category)
+order by qoh, i_product_name, i_brand, i_class, i_category
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query23.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query23.sql
new file mode 100644
index 0000000..0ee1dab
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query23.sql
@@ -0,0 +1,120 @@
+-- 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.
+
+with frequent_ss_items as
+ (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt
+  from store_sales
+      ,date_dim 
+      ,item
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = i_item_sk 
+    and d_year in (1999,1999+1,1999+2,1999+3)
+  group by substr(i_item_desc,1,30),i_item_sk,d_date
+  having count(*) >4),
+ max_store_sales as
+ (select max(csales) tpcds_cmax 
+  from (select c_customer_sk,sum(ss_quantity*ss_sales_price) csales
+        from store_sales
+            ,customer
+            ,date_dim 
+        where ss_customer_sk = c_customer_sk
+         and ss_sold_date_sk = d_date_sk
+         and d_year in (1999,1999+1,1999+2,1999+3) 
+        group by c_customer_sk)),
+ best_ss_customer as
+ (select c_customer_sk,sum(ss_quantity*ss_sales_price) ssales
+  from store_sales
+      ,customer
+  where ss_customer_sk = c_customer_sk
+  group by c_customer_sk
+  having sum(ss_quantity*ss_sales_price) > (95/100.0) * (select
+  *
+from
+ max_store_sales))
+  select  sum(sales)
+ from (select cs_quantity*cs_list_price sales
+       from catalog_sales
+           ,date_dim 
+       where d_year = 1999 
+         and d_moy = 1 
+         and cs_sold_date_sk = d_date_sk 
+         and cs_item_sk in (select item_sk from frequent_ss_items)
+         and cs_bill_customer_sk in (select c_customer_sk from best_ss_customer)
+      union all
+      select ws_quantity*ws_list_price sales
+       from web_sales 
+           ,date_dim 
+       where d_year = 1999 
+         and d_moy = 1 
+         and ws_sold_date_sk = d_date_sk 
+         and ws_item_sk in (select item_sk from frequent_ss_items)
+         and ws_bill_customer_sk in (select c_customer_sk from best_ss_customer)) 
+ limit 100;
+with frequent_ss_items as
+ (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt
+  from store_sales
+      ,date_dim
+      ,item
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = i_item_sk
+    and d_year in (1999,1999 + 1,1999 + 2,1999 + 3)
+  group by substr(i_item_desc,1,30),i_item_sk,d_date
+  having count(*) >4),
+ max_store_sales as
+ (select max(csales) tpcds_cmax
+  from (select c_customer_sk,sum(ss_quantity*ss_sales_price) csales
+        from store_sales
+            ,customer
+            ,date_dim 
+        where ss_customer_sk = c_customer_sk
+         and ss_sold_date_sk = d_date_sk
+         and d_year in (1999,1999+1,1999+2,1999+3)
+        group by c_customer_sk)),
+ best_ss_customer as
+ (select c_customer_sk,sum(ss_quantity*ss_sales_price) ssales
+  from store_sales
+      ,customer
+  where ss_customer_sk = c_customer_sk
+  group by c_customer_sk
+  having sum(ss_quantity*ss_sales_price) > (95/100.0) * (select
+  *
+ from max_store_sales))
+  select  c_last_name,c_first_name,sales
+ from (select c_last_name,c_first_name,sum(cs_quantity*cs_list_price) sales
+        from catalog_sales
+            ,customer
+            ,date_dim 
+        where d_year = 1999 
+         and d_moy = 1 
+         and cs_sold_date_sk = d_date_sk 
+         and cs_item_sk in (select item_sk from frequent_ss_items)
+         and cs_bill_customer_sk in (select c_customer_sk from best_ss_customer)
+         and cs_bill_customer_sk = c_customer_sk 
+       group by c_last_name,c_first_name
+      union all
+      select c_last_name,c_first_name,sum(ws_quantity*ws_list_price) sales
+       from web_sales
+           ,customer
+           ,date_dim 
+       where d_year = 1999 
+         and d_moy = 1 
+         and ws_sold_date_sk = d_date_sk 
+         and ws_item_sk in (select item_sk from frequent_ss_items)
+         and ws_bill_customer_sk in (select c_customer_sk from best_ss_customer)
+         and ws_bill_customer_sk = c_customer_sk
+       group by c_last_name,c_first_name) 
+     order by c_last_name,c_first_name,sales
+  limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query24.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query24.sql
new file mode 100644
index 0000000..3f45c4f
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query24.sql
@@ -0,0 +1,119 @@
+-- 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.
+
+with ssales as
+(select c_last_name
+      ,c_first_name
+      ,s_store_name
+      ,ca_state
+      ,s_state
+      ,i_color
+      ,i_current_price
+      ,i_manager_id
+      ,i_units
+      ,i_size
+      ,sum(ss_sales_price) netpaid
+from store_sales
+    ,store_returns
+    ,store
+    ,item
+    ,customer
+    ,customer_address
+where ss_ticket_number = sr_ticket_number
+  and ss_item_sk = sr_item_sk
+  and ss_customer_sk = c_customer_sk
+  and ss_item_sk = i_item_sk
+  and ss_store_sk = s_store_sk
+  and c_current_addr_sk = ca_address_sk
+  and c_birth_country <> upper(ca_country)
+  and s_zip = ca_zip
+and s_market_id=7
+group by c_last_name
+        ,c_first_name
+        ,s_store_name
+        ,ca_state
+        ,s_state
+        ,i_color
+        ,i_current_price
+        ,i_manager_id
+        ,i_units
+        ,i_size)
+select c_last_name
+      ,c_first_name
+      ,s_store_name
+      ,sum(netpaid) paid
+from ssales
+where i_color = 'orchid'
+group by c_last_name
+        ,c_first_name
+        ,s_store_name
+having sum(netpaid) > (select 0.05*avg(netpaid)
+                                 from ssales)
+order by c_last_name
+        ,c_first_name
+        ,s_store_name
+;
+with ssales as
+(select c_last_name
+      ,c_first_name
+      ,s_store_name
+      ,ca_state
+      ,s_state
+      ,i_color
+      ,i_current_price
+      ,i_manager_id
+      ,i_units
+      ,i_size
+      ,sum(ss_sales_price) netpaid
+from store_sales
+    ,store_returns
+    ,store
+    ,item
+    ,customer
+    ,customer_address
+where ss_ticket_number = sr_ticket_number
+  and ss_item_sk = sr_item_sk
+  and ss_customer_sk = c_customer_sk
+  and ss_item_sk = i_item_sk
+  and ss_store_sk = s_store_sk
+  and c_current_addr_sk = ca_address_sk
+  and c_birth_country <> upper(ca_country)
+  and s_zip = ca_zip
+  and s_market_id = 7
+group by c_last_name
+        ,c_first_name
+        ,s_store_name
+        ,ca_state
+        ,s_state
+        ,i_color
+        ,i_current_price
+        ,i_manager_id
+        ,i_units
+        ,i_size)
+select c_last_name
+      ,c_first_name
+      ,s_store_name
+      ,sum(netpaid) paid
+from ssales
+where i_color = 'chiffon'
+group by c_last_name
+        ,c_first_name
+        ,s_store_name
+having sum(netpaid) > (select 0.05*avg(netpaid)
+                           from ssales)
+order by c_last_name
+        ,c_first_name
+        ,s_store_name
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query25.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query25.sql
new file mode 100644
index 0000000..be825fd
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query25.sql
@@ -0,0 +1,61 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+ i_item_id
+ ,i_item_desc
+ ,s_store_id
+ ,s_store_name
+ ,sum(ss_net_profit) as store_sales_profit
+ ,sum(sr_net_loss) as store_returns_loss
+ ,sum(cs_net_profit) as catalog_sales_profit
+ from
+ store_sales
+ ,store_returns
+ ,catalog_sales
+ ,date_dim d1
+ ,date_dim d2
+ ,date_dim d3
+ ,store
+ ,item
+ where
+ d1.d_moy = 4
+ and d1.d_year = 2000
+ and d1.d_date_sk = ss_sold_date_sk
+ and i_item_sk = ss_item_sk
+ and s_store_sk = ss_store_sk
+ and ss_customer_sk = sr_customer_sk
+ and ss_item_sk = sr_item_sk
+ and ss_ticket_number = sr_ticket_number
+ and sr_returned_date_sk = d2.d_date_sk
+ and d2.d_moy               between 4 and  10
+ and d2.d_year              = 2000
+ and sr_customer_sk = cs_bill_customer_sk
+ and sr_item_sk = cs_item_sk
+ and cs_sold_date_sk = d3.d_date_sk
+ and d3.d_moy               between 4 and  10 
+ and d3.d_year              = 2000
+ group by
+ i_item_id
+ ,i_item_desc
+ ,s_store_id
+ ,s_store_name
+ order by
+ i_item_id
+ ,i_item_desc
+ ,s_store_id
+ ,s_store_name
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query26.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query26.sql
new file mode 100644
index 0000000..772d545
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query26.sql
@@ -0,0 +1,34 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select  i_item_id,
+        avg(cs_quantity) agg1,
+        avg(cs_list_price) agg2,
+        avg(cs_coupon_amt) agg3,
+        avg(cs_sales_price) agg4 
+ from catalog_sales, customer_demographics, date_dim, item, promotion
+ where cs_sold_date_sk = d_date_sk and
+       cs_item_sk = i_item_sk and
+       cs_bill_cdemo_sk = cd_demo_sk and
+       cs_promo_sk = p_promo_sk and
+       cd_gender = 'F' and 
+       cd_marital_status = 'W' and
+       cd_education_status = 'Primary' and
+       (p_channel_email = 'N' or p_channel_event = 'N') and
+       d_year = 1998 
+ group by i_item_id
+ order by i_item_id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query27.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query27.sql
new file mode 100644
index 0000000..37cf1f5
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query27.sql
@@ -0,0 +1,36 @@
+-- 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.
+
+select  i_item_id,
+        s_state, grouping(s_state) g_state,
+        avg(ss_quantity) agg1,
+        avg(ss_list_price) agg2,
+        avg(ss_coupon_amt) agg3,
+        avg(ss_sales_price) agg4
+ from store_sales, customer_demographics, date_dim, store, item
+ where ss_sold_date_sk = d_date_sk and
+       ss_item_sk = i_item_sk and
+       ss_store_sk = s_store_sk and
+       ss_cdemo_sk = cd_demo_sk and
+       cd_gender = 'F' and
+       cd_marital_status = 'W' and
+       cd_education_status = 'Primary' and
+       d_year = 1998 and
+       s_state in ('TN','TN', 'TN', 'TN', 'TN', 'TN')
+ group by rollup (i_item_id, s_state)
+ order by i_item_id
+         ,s_state
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query28.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query28.sql
new file mode 100644
index 0000000..afe2cf8
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query28.sql
@@ -0,0 +1,66 @@
+-- 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.
+
+select  *
+from (select avg(ss_list_price) B1_LP
+            ,count(ss_list_price) B1_CNT
+            ,count(distinct ss_list_price) B1_CNTD
+      from store_sales
+      where ss_quantity between 0 and 5
+        and (ss_list_price between 11 and 11+10 
+             or ss_coupon_amt between 460 and 460+1000
+             or ss_wholesale_cost between 14 and 14+20)) B1,
+     (select avg(ss_list_price) B2_LP
+            ,count(ss_list_price) B2_CNT
+            ,count(distinct ss_list_price) B2_CNTD
+      from store_sales
+      where ss_quantity between 6 and 10
+        and (ss_list_price between 91 and 91+10
+          or ss_coupon_amt between 1430 and 1430+1000
+          or ss_wholesale_cost between 32 and 32+20)) B2,
+     (select avg(ss_list_price) B3_LP
+            ,count(ss_list_price) B3_CNT
+            ,count(distinct ss_list_price) B3_CNTD
+      from store_sales
+      where ss_quantity between 11 and 15
+        and (ss_list_price between 66 and 66+10
+          or ss_coupon_amt between 920 and 920+1000
+          or ss_wholesale_cost between 4 and 4+20)) B3,
+     (select avg(ss_list_price) B4_LP
+            ,count(ss_list_price) B4_CNT
+            ,count(distinct ss_list_price) B4_CNTD
+      from store_sales
+      where ss_quantity between 16 and 20
+        and (ss_list_price between 142 and 142+10
+          or ss_coupon_amt between 3054 and 3054+1000
+          or ss_wholesale_cost between 80 and 80+20)) B4,
+     (select avg(ss_list_price) B5_LP
+            ,count(ss_list_price) B5_CNT
+            ,count(distinct ss_list_price) B5_CNTD
+      from store_sales
+      where ss_quantity between 21 and 25
+        and (ss_list_price between 135 and 135+10
+          or ss_coupon_amt between 14180 and 14180+1000
+          or ss_wholesale_cost between 38 and 38+20)) B5,
+     (select avg(ss_list_price) B6_LP
+            ,count(ss_list_price) B6_CNT
+            ,count(distinct ss_list_price) B6_CNTD
+      from store_sales
+      where ss_quantity between 26 and 30
+        and (ss_list_price between 28 and 28+10
+          or ss_coupon_amt between 2513 and 2513+1000
+          or ss_wholesale_cost between 42 and 42+20)) B6
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query29.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query29.sql
new file mode 100644
index 0000000..5db4817
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query29.sql
@@ -0,0 +1,60 @@
+-- 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.
+
+select
+     i_item_id
+    ,i_item_desc
+    ,s_store_id
+    ,s_store_name
+    ,sum(ss_quantity)        as store_sales_quantity
+    ,sum(sr_return_quantity) as store_returns_quantity
+    ,sum(cs_quantity)        as catalog_sales_quantity
+ from
+    store_sales
+   ,store_returns
+   ,catalog_sales
+   ,date_dim             d1
+   ,date_dim             d2
+   ,date_dim             d3
+   ,store
+   ,item
+ where
+     d1.d_moy               = 4 
+ and d1.d_year              = 1999
+ and d1.d_date_sk           = ss_sold_date_sk
+ and i_item_sk              = ss_item_sk
+ and s_store_sk             = ss_store_sk
+ and ss_customer_sk         = sr_customer_sk
+ and ss_item_sk             = sr_item_sk
+ and ss_ticket_number       = sr_ticket_number
+ and sr_returned_date_sk    = d2.d_date_sk
+ and d2.d_moy               between 4 and  4 + 3 
+ and d2.d_year              = 1999
+ and sr_customer_sk         = cs_bill_customer_sk
+ and sr_item_sk             = cs_item_sk
+ and cs_sold_date_sk        = d3.d_date_sk     
+ and d3.d_year              in (1999,1999+1,1999+2)
+ group by
+    i_item_id
+   ,i_item_desc
+   ,s_store_id
+   ,s_store_name
+ order by
+    i_item_id 
+   ,i_item_desc
+   ,s_store_id
+   ,s_store_name
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query3.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query3.sql
new file mode 100644
index 0000000..fa9025e
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query3.sql
@@ -0,0 +1,34 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select  dt.d_year
+       ,item.i_brand_id brand_id 
+       ,item.i_brand brand
+       ,sum(ss_ext_sales_price) sum_agg
+ from  date_dim dt 
+      ,store_sales
+      ,item
+ where dt.d_date_sk = store_sales.ss_sold_date_sk
+   and store_sales.ss_item_sk = item.i_item_sk
+   and item.i_manufact_id = 436
+   and dt.d_moy=12
+ group by dt.d_year
+      ,item.i_brand
+      ,item.i_brand_id
+ order by dt.d_year
+         ,sum_agg desc
+         ,brand_id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query30.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query30.sql
new file mode 100644
index 0000000..fabdf70
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query30.sql
@@ -0,0 +1,44 @@
+-- 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.
+
+with customer_total_return as
+ (select wr_returning_customer_sk as ctr_customer_sk
+        ,ca_state as ctr_state, 
+ 	sum(wr_return_amt) as ctr_total_return
+ from web_returns
+     ,date_dim
+     ,customer_address
+ where wr_returned_date_sk = d_date_sk 
+   and d_year =2002
+   and wr_returning_addr_sk = ca_address_sk 
+ group by wr_returning_customer_sk
+         ,ca_state)
+  select  c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag
+       ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address
+       ,c_last_review_date_sk,ctr_total_return
+ from customer_total_return ctr1
+     ,customer_address
+     ,customer
+ where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2
+ 			  from customer_total_return ctr2 
+                  	  where ctr1.ctr_state = ctr2.ctr_state)
+       and ca_address_sk = c_current_addr_sk
+       and ca_state = 'IL'
+       and ctr1.ctr_customer_sk = c_customer_sk
+ order by c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag
+                  ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address
+                  ,c_last_review_date_sk,ctr_total_return
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query31.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query31.sql
new file mode 100644
index 0000000..4217c55
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query31.sql
@@ -0,0 +1,65 @@
+-- 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.
+
+with ss as
+ (select ca_county,d_qoy, d_year,sum(ss_ext_sales_price) as store_sales
+ from store_sales,date_dim,customer_address
+ where ss_sold_date_sk = d_date_sk
+  and ss_addr_sk=ca_address_sk
+ group by ca_county,d_qoy, d_year),
+ ws as
+ (select ca_county,d_qoy, d_year,sum(ws_ext_sales_price) as web_sales
+ from web_sales,date_dim,customer_address
+ where ws_sold_date_sk = d_date_sk
+  and ws_bill_addr_sk=ca_address_sk
+ group by ca_county,d_qoy, d_year)
+ select 
+        ss1.ca_county
+       ,ss1.d_year
+       ,ws2.web_sales/ws1.web_sales web_q1_q2_increase
+       ,ss2.store_sales/ss1.store_sales store_q1_q2_increase
+       ,ws3.web_sales/ws2.web_sales web_q2_q3_increase
+       ,ss3.store_sales/ss2.store_sales store_q2_q3_increase
+ from
+        ss ss1
+       ,ss ss2
+       ,ss ss3
+       ,ws ws1
+       ,ws ws2
+       ,ws ws3
+ where
+    ss1.d_qoy = 1
+    and ss1.d_year = 2000
+    and ss1.ca_county = ss2.ca_county
+    and ss2.d_qoy = 2
+    and ss2.d_year = 2000
+ and ss2.ca_county = ss3.ca_county
+    and ss3.d_qoy = 3
+    and ss3.d_year = 2000
+    and ss1.ca_county = ws1.ca_county
+    and ws1.d_qoy = 1
+    and ws1.d_year = 2000
+    and ws1.ca_county = ws2.ca_county
+    and ws2.d_qoy = 2
+    and ws2.d_year = 2000
+    and ws1.ca_county = ws3.ca_county
+    and ws3.d_qoy = 3
+    and ws3.d_year =2000
+    and case when ws1.web_sales > 0 then ws2.web_sales/ws1.web_sales else null end 
+       > case when ss1.store_sales > 0 then ss2.store_sales/ss1.store_sales else null end
+    and case when ws2.web_sales > 0 then ws3.web_sales/ws2.web_sales else null end
+       > case when ss2.store_sales > 0 then ss3.store_sales/ss2.store_sales else null end
+ order by ss1.d_year
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query32.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query32.sql
new file mode 100644
index 0000000..70eb508
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query32.sql
@@ -0,0 +1,41 @@
+-- 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.
+
+select  sum(cs_ext_discount_amt)  as "excess discount amount"
+from 
+   catalog_sales 
+   ,item 
+   ,date_dim
+where
+i_manufact_id = 269
+and i_item_sk = cs_item_sk 
+and d_date between '1998-03-18' and 
+        (cast('1998-03-18' as date) + 90 days)
+and d_date_sk = cs_sold_date_sk 
+and cs_ext_discount_amt  
+     > ( 
+         select 
+            1.3 * avg(cs_ext_discount_amt) 
+         from 
+            catalog_sales 
+           ,date_dim
+         where 
+              cs_item_sk = i_item_sk 
+          and d_date between '1998-03-18' and
+                             (cast('1998-03-18' as date) + 90 days)
+          and d_date_sk = cs_sold_date_sk 
+      ) 
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query33.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query33.sql
new file mode 100644
index 0000000..bb845af
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query33.sql
@@ -0,0 +1,88 @@
+-- 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.
+
+with ss as (
+ select
+          i_manufact_id,sum(ss_ext_sales_price) total_sales
+ from
+ 	store_sales,
+ 	date_dim,
+         customer_address,
+         item
+ where
+         i_manufact_id in (select
+  i_manufact_id
+from
+ item
+where i_category in ('Books'))
+ and     ss_item_sk              = i_item_sk
+ and     ss_sold_date_sk         = d_date_sk
+ and     d_year                  = 1999
+ and     d_moy                   = 3
+ and     ss_addr_sk              = ca_address_sk
+ and     ca_gmt_offset           = -5 
+ group by i_manufact_id),
+ cs as (
+ select
+          i_manufact_id,sum(cs_ext_sales_price) total_sales
+ from
+ 	catalog_sales,
+ 	date_dim,
+         customer_address,
+         item
+ where
+         i_manufact_id               in (select
+  i_manufact_id
+from
+ item
+where i_category in ('Books'))
+ and     cs_item_sk              = i_item_sk
+ and     cs_sold_date_sk         = d_date_sk
+ and     d_year                  = 1999
+ and     d_moy                   = 3
+ and     cs_bill_addr_sk         = ca_address_sk
+ and     ca_gmt_offset           = -5 
+ group by i_manufact_id),
+ ws as (
+ select
+          i_manufact_id,sum(ws_ext_sales_price) total_sales
+ from
+ 	web_sales,
+ 	date_dim,
+         customer_address,
+         item
+ where
+         i_manufact_id               in (select
+  i_manufact_id
+from
+ item
+where i_category in ('Books'))
+ and     ws_item_sk              = i_item_sk
+ and     ws_sold_date_sk         = d_date_sk
+ and     d_year                  = 1999
+ and     d_moy                   = 3
+ and     ws_bill_addr_sk         = ca_address_sk
+ and     ca_gmt_offset           = -5
+ group by i_manufact_id)
+  select  i_manufact_id ,sum(total_sales) total_sales
+ from  (select * from ss 
+        union all
+        select * from cs 
+        union all
+        select * from ws) tmp1
+ group by i_manufact_id
+ order by total_sales
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query34.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query34.sql
new file mode 100644
index 0000000..b2c5283
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query34.sql
@@ -0,0 +1,44 @@
+-- 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.
+
+select c_last_name
+       ,c_first_name
+       ,c_salutation
+       ,c_preferred_cust_flag
+       ,ss_ticket_number
+       ,cnt from
+   (select ss_ticket_number
+          ,ss_customer_sk
+          ,count(*) cnt
+    from store_sales,date_dim,store,household_demographics
+    where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+    and store_sales.ss_store_sk = store.s_store_sk  
+    and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+    and (date_dim.d_dom between 1 and 3 or date_dim.d_dom between 25 and 28)
+    and (household_demographics.hd_buy_potential = '>10000' or
+         household_demographics.hd_buy_potential = 'Unknown')
+    and household_demographics.hd_vehicle_count > 0
+    and (case when household_demographics.hd_vehicle_count > 0 
+	then household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count 
+	else null 
+	end)  > 1.2
+    and date_dim.d_year in (1998,1998+1,1998+2)
+    and store.s_county in ('Williamson County','Williamson County','Williamson County','Williamson County',
+                           'Williamson County','Williamson County','Williamson County','Williamson County')
+    group by ss_ticket_number,ss_customer_sk) dn,customer
+    where ss_customer_sk = c_customer_sk
+      and cnt between 15 and 20
+    order by c_last_name,c_first_name,c_salutation,c_preferred_cust_flag desc, ss_ticket_number
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query35.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query35.sql
new file mode 100644
index 0000000..86ffd3b
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query35.sql
@@ -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.
+
+select
+  ca_state,
+  cd_gender,
+  cd_marital_status,
+  cd_dep_count,
+  count(*) cnt1,
+  avg(cd_dep_count),
+  max(cd_dep_count),
+  sum(cd_dep_count),
+  cd_dep_employed_count,
+  count(*) cnt2,
+  avg(cd_dep_employed_count),
+  max(cd_dep_employed_count),
+  sum(cd_dep_employed_count),
+  cd_dep_college_count,
+  count(*) cnt3,
+  avg(cd_dep_college_count),
+  max(cd_dep_college_count),
+  sum(cd_dep_college_count)
+ from
+  customer c,customer_address ca,customer_demographics
+ where
+  c.c_current_addr_sk = ca.ca_address_sk and
+  cd_demo_sk = c.c_current_cdemo_sk and 
+  exists (select *
+          from store_sales,date_dim
+          where c.c_customer_sk = ss_customer_sk and
+                ss_sold_date_sk = d_date_sk and
+                d_year = 1999 and
+                d_qoy < 4) and
+   (exists (select *
+            from web_sales,date_dim
+            where c.c_customer_sk = ws_bill_customer_sk and
+                  ws_sold_date_sk = d_date_sk and
+                  d_year = 1999 and
+                  d_qoy < 4) or 
+    exists (select * 
+            from catalog_sales,date_dim
+            where c.c_customer_sk = cs_ship_customer_sk and
+                  cs_sold_date_sk = d_date_sk and
+                  d_year = 1999 and
+                  d_qoy < 4))
+ group by ca_state,
+          cd_gender,
+          cd_marital_status,
+          cd_dep_count,
+          cd_dep_employed_count,
+          cd_dep_college_count
+ order by ca_state,
+          cd_gender,
+          cd_marital_status,
+          cd_dep_count,
+          cd_dep_employed_count,
+          cd_dep_college_count
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query36.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query36.sql
new file mode 100644
index 0000000..2436ef3
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query36.sql
@@ -0,0 +1,43 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+    sum(ss_net_profit)/sum(ss_ext_sales_price) as gross_margin
+   ,i_category
+   ,i_class
+   ,grouping(i_category)+grouping(i_class) as lochierarchy
+   ,rank() over (
+ 	partition by grouping(i_category)+grouping(i_class),
+ 	case when grouping(i_class) = 0 then i_category end 
+ 	order by sum(ss_net_profit)/sum(ss_ext_sales_price) asc) as rank_within_parent
+ from
+    store_sales
+   ,date_dim       d1
+   ,item
+   ,store
+ where
+    d1.d_year = 2000 
+ and d1.d_date_sk = ss_sold_date_sk
+ and i_item_sk  = ss_item_sk 
+ and s_store_sk  = ss_store_sk
+ and s_state in ('TN','TN','TN','TN',
+                 'TN','TN','TN','TN')
+ group by rollup(i_category,i_class)
+ order by
+   lochierarchy desc
+  ,case when lochierarchy = 0 then i_category end
+  ,rank_within_parent
+  limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query37.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query37.sql
new file mode 100644
index 0000000..24237b7
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query37.sql
@@ -0,0 +1,30 @@
+-- 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.
+
+select  i_item_id
+       ,i_item_desc
+       ,i_current_price
+ from item, inventory, date_dim, catalog_sales
+ where i_current_price between 22 and 22 + 30
+ and inv_item_sk = i_item_sk
+ and d_date_sk=inv_date_sk
+ and d_date between cast('2001-06-02' as date) and (cast('2001-06-02' as date) +  60 days)
+ and i_manufact_id in (678,964,918,849)
+ and inv_quantity_on_hand between 100 and 500
+ and cs_item_sk = i_item_sk
+ group by i_item_id,i_item_desc,i_current_price
+ order by i_item_id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query38.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query38.sql
new file mode 100644
index 0000000..3e781ad
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query38.sql
@@ -0,0 +1,36 @@
+-- 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.
+
+select  count(*) from (
+    select distinct c_last_name, c_first_name, d_date
+    from store_sales, date_dim, customer
+          where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+      and store_sales.ss_customer_sk = customer.c_customer_sk
+      and d_month_seq between 1212 and 1212 + 11
+  intersect
+    select distinct c_last_name, c_first_name, d_date
+    from catalog_sales, date_dim, customer
+          where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk
+      and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk
+      and d_month_seq between 1212 and 1212 + 11
+  intersect
+    select distinct c_last_name, c_first_name, d_date
+    from web_sales, date_dim, customer
+          where web_sales.ws_sold_date_sk = date_dim.d_date_sk
+      and web_sales.ws_bill_customer_sk = customer.c_customer_sk
+      and d_month_seq between 1212 and 1212 + 11
+) hot_cust
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query39.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query39.sql
new file mode 100644
index 0000000..aaed22a
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query39.sql
@@ -0,0 +1,66 @@
+-- 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.
+
+with inv as
+(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy
+       ,stdev,mean, case mean when 0 then null else stdev/mean end cov
+ from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy
+            ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean
+      from inventory
+          ,item
+          ,warehouse
+          ,date_dim
+      where inv_item_sk = i_item_sk
+        and inv_warehouse_sk = w_warehouse_sk
+        and inv_date_sk = d_date_sk
+        and d_year =1998
+      group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo
+ where case mean when 0 then 0 else stdev/mean end > 1)
+select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov
+        ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov
+from inv inv1,inv inv2
+where inv1.i_item_sk = inv2.i_item_sk
+  and inv1.w_warehouse_sk =  inv2.w_warehouse_sk
+  and inv1.d_moy=4
+  and inv2.d_moy=4+1
+order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov
+        ,inv2.d_moy,inv2.mean, inv2.cov
+;
+with inv as
+(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy
+       ,stdev,mean, case mean when 0 then null else stdev/mean end cov
+ from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy
+            ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean
+      from inventory
+          ,item
+          ,warehouse
+          ,date_dim
+      where inv_item_sk = i_item_sk
+        and inv_warehouse_sk = w_warehouse_sk
+        and inv_date_sk = d_date_sk
+        and d_year =1998
+      group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo
+ where case mean when 0 then 0 else stdev/mean end > 1)
+select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov
+        ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov
+from inv inv1,inv inv2
+where inv1.i_item_sk = inv2.i_item_sk
+  and inv1.w_warehouse_sk =  inv2.w_warehouse_sk
+  and inv1.d_moy=4
+  and inv2.d_moy=4+1
+  and inv1.cov > 1.5
+order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov
+        ,inv2.d_moy,inv2.mean, inv2.cov
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query4.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query4.sql
new file mode 100644
index 0000000..364c1a5
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query4.sql
@@ -0,0 +1,129 @@
+-- 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.
+
+with year_total as (
+ select c_customer_id customer_id
+       ,c_first_name customer_first_name
+       ,c_last_name customer_last_name
+       ,c_preferred_cust_flag customer_preferred_cust_flag
+       ,c_birth_country customer_birth_country
+       ,c_login customer_login
+       ,c_email_address customer_email_address
+       ,d_year dyear
+       ,sum(((ss_ext_list_price-ss_ext_wholesale_cost-ss_ext_discount_amt)+ss_ext_sales_price)/2) year_total
+       ,'s' sale_type
+ from customer
+     ,store_sales
+     ,date_dim
+ where c_customer_sk = ss_customer_sk
+   and ss_sold_date_sk = d_date_sk
+ group by c_customer_id
+         ,c_first_name
+         ,c_last_name
+         ,c_preferred_cust_flag
+         ,c_birth_country
+         ,c_login
+         ,c_email_address
+         ,d_year
+ union all
+ select c_customer_id customer_id
+       ,c_first_name customer_first_name
+       ,c_last_name customer_last_name
+       ,c_preferred_cust_flag customer_preferred_cust_flag
+       ,c_birth_country customer_birth_country
+       ,c_login customer_login
+       ,c_email_address customer_email_address
+       ,d_year dyear
+       ,sum((((cs_ext_list_price-cs_ext_wholesale_cost-cs_ext_discount_amt)+cs_ext_sales_price)/2) ) year_total
+       ,'c' sale_type
+ from customer
+     ,catalog_sales
+     ,date_dim
+ where c_customer_sk = cs_bill_customer_sk
+   and cs_sold_date_sk = d_date_sk
+ group by c_customer_id
+         ,c_first_name
+         ,c_last_name
+         ,c_preferred_cust_flag
+         ,c_birth_country
+         ,c_login
+         ,c_email_address
+         ,d_year
+union all
+ select c_customer_id customer_id
+       ,c_first_name customer_first_name
+       ,c_last_name customer_last_name
+       ,c_preferred_cust_flag customer_preferred_cust_flag
+       ,c_birth_country customer_birth_country
+       ,c_login customer_login
+       ,c_email_address customer_email_address
+       ,d_year dyear
+       ,sum((((ws_ext_list_price-ws_ext_wholesale_cost-ws_ext_discount_amt)+ws_ext_sales_price)/2) ) year_total
+       ,'w' sale_type
+ from customer
+     ,web_sales
+     ,date_dim
+ where c_customer_sk = ws_bill_customer_sk
+   and ws_sold_date_sk = d_date_sk
+ group by c_customer_id
+         ,c_first_name
+         ,c_last_name
+         ,c_preferred_cust_flag
+         ,c_birth_country
+         ,c_login
+         ,c_email_address
+         ,d_year
+         )
+  select  
+                  t_s_secyear.customer_id
+                 ,t_s_secyear.customer_first_name
+                 ,t_s_secyear.customer_last_name
+                 ,t_s_secyear.customer_email_address
+ from year_total t_s_firstyear
+     ,year_total t_s_secyear
+     ,year_total t_c_firstyear
+     ,year_total t_c_secyear
+     ,year_total t_w_firstyear
+     ,year_total t_w_secyear
+ where t_s_secyear.customer_id = t_s_firstyear.customer_id
+   and t_s_firstyear.customer_id = t_c_secyear.customer_id
+   and t_s_firstyear.customer_id = t_c_firstyear.customer_id
+   and t_s_firstyear.customer_id = t_w_firstyear.customer_id
+   and t_s_firstyear.customer_id = t_w_secyear.customer_id
+   and t_s_firstyear.sale_type = 's'
+   and t_c_firstyear.sale_type = 'c'
+   and t_w_firstyear.sale_type = 'w'
+   and t_s_secyear.sale_type = 's'
+   and t_c_secyear.sale_type = 'c'
+   and t_w_secyear.sale_type = 'w'
+   and t_s_firstyear.dyear =  2001
+   and t_s_secyear.dyear = 2001+1
+   and t_c_firstyear.dyear =  2001
+   and t_c_secyear.dyear =  2001+1
+   and t_w_firstyear.dyear = 2001
+   and t_w_secyear.dyear = 2001+1
+   and t_s_firstyear.year_total > 0
+   and t_c_firstyear.year_total > 0
+   and t_w_firstyear.year_total > 0
+   and case when t_c_firstyear.year_total > 0 then t_c_secyear.year_total / t_c_firstyear.year_total else null end
+           > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else null end
+   and case when t_c_firstyear.year_total > 0 then t_c_secyear.year_total / t_c_firstyear.year_total else null end
+           > case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else null end
+ order by t_s_secyear.customer_id
+         ,t_s_secyear.customer_first_name
+         ,t_s_secyear.customer_last_name
+         ,t_s_secyear.customer_email_address
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query40.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query40.sql
new file mode 100644
index 0000000..41a8cba
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query40.sql
@@ -0,0 +1,41 @@
+-- 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.
+
+select
+   w_state
+  ,i_item_id
+  ,sum(case when (cast(d_date as date) < cast ('1998-04-08' as date)) 
+ 		then cs_sales_price - coalesce(cr_refunded_cash,0) else 0 end) as sales_before
+  ,sum(case when (cast(d_date as date) >= cast ('1998-04-08' as date)) 
+ 		then cs_sales_price - coalesce(cr_refunded_cash,0) else 0 end) as sales_after
+ from
+   catalog_sales left outer join catalog_returns on
+       (cs_order_number = cr_order_number 
+        and cs_item_sk = cr_item_sk)
+  ,warehouse 
+  ,item
+  ,date_dim
+ where
+     i_current_price between 0.99 and 1.49
+ and i_item_sk          = cs_item_sk
+ and cs_warehouse_sk    = w_warehouse_sk 
+ and cs_sold_date_sk    = d_date_sk
+ and d_date between (cast ('1998-04-08' as date) - 30 days)
+                and (cast ('1998-04-08' as date) + 30 days) 
+ group by
+    w_state,i_item_id
+ order by w_state,i_item_id
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query41.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query41.sql
new file mode 100644
index 0000000..e42bef9
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query41.sql
@@ -0,0 +1,65 @@
+-- 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.
+
+select  distinct(i_product_name)
+ from item i1
+ where i_manufact_id between 742 and 742+40 
+   and (select count(*) as item_cnt
+        from item
+        where (i_manufact = i1.i_manufact and
+        ((i_category = 'Women' and 
+        (i_color = 'orchid' or i_color = 'papaya') and 
+        (i_units = 'Pound' or i_units = 'Lb') and
+        (i_size = 'petite' or i_size = 'medium')
+        ) or
+        (i_category = 'Women' and
+        (i_color = 'burlywood' or i_color = 'navy') and
+        (i_units = 'Bundle' or i_units = 'Each') and
+        (i_size = 'N/A' or i_size = 'extra large')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'bisque' or i_color = 'azure') and
+        (i_units = 'N/A' or i_units = 'Tsp') and
+        (i_size = 'small' or i_size = 'large')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'chocolate' or i_color = 'cornflower') and
+        (i_units = 'Bunch' or i_units = 'Gross') and
+        (i_size = 'petite' or i_size = 'medium')
+        ))) or
+       (i_manufact = i1.i_manufact and
+        ((i_category = 'Women' and 
+        (i_color = 'salmon' or i_color = 'midnight') and 
+        (i_units = 'Oz' or i_units = 'Box') and
+        (i_size = 'petite' or i_size = 'medium')
+        ) or
+        (i_category = 'Women' and
+        (i_color = 'snow' or i_color = 'steel') and
+        (i_units = 'Carton' or i_units = 'Tbl') and
+        (i_size = 'N/A' or i_size = 'extra large')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'purple' or i_color = 'gainsboro') and
+        (i_units = 'Dram' or i_units = 'Unknown') and
+        (i_size = 'small' or i_size = 'large')
+        ) or
+        (i_category = 'Men' and
+        (i_color = 'metallic' or i_color = 'forest') and
+        (i_units = 'Gram' or i_units = 'Ounce') and
+        (i_size = 'petite' or i_size = 'medium')
+        )))) > 0
+ order by i_product_name
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query42.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query42.sql
new file mode 100644
index 0000000..a7a8bc8
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query42.sql
@@ -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.
+
+select  dt.d_year
+ 	,item.i_category_id
+ 	,item.i_category
+ 	,sum(ss_ext_sales_price)
+ from 	date_dim dt
+ 	,store_sales
+ 	,item
+ where dt.d_date_sk = store_sales.ss_sold_date_sk
+ 	and store_sales.ss_item_sk = item.i_item_sk
+ 	and item.i_manager_id = 1  	
+ 	and dt.d_moy=12
+ 	and dt.d_year=1998
+ group by 	dt.d_year
+ 		,item.i_category_id
+ 		,item.i_category
+ order by       sum(ss_ext_sales_price) desc,dt.d_year
+ 		,item.i_category_id
+ 		,item.i_category
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query43.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query43.sql
new file mode 100644
index 0000000..db4db2b
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query43.sql
@@ -0,0 +1,32 @@
+-- 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.
+
+select  s_store_name, s_store_id,
+        sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales,
+        sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales,
+        sum(case when (d_day_name='Tuesday') then ss_sales_price else  null end) tue_sales,
+        sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales,
+        sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales,
+        sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales,
+        sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales
+ from date_dim, store_sales, store
+ where d_date_sk = ss_sold_date_sk and
+       s_store_sk = ss_store_sk and
+       s_gmt_offset = -5 and
+       d_year = 1998 
+ group by s_store_name, s_store_id
+ order by s_store_name, s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query44.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query44.sql
new file mode 100644
index 0000000..897ac5f
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query44.sql
@@ -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.
+
+select  asceding.rnk, i1.i_product_name best_performing, i2.i_product_name worst_performing
+from(select *
+     from (select item_sk,rank() over (order by rank_col asc) rnk
+           from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col 
+                 from store_sales ss1
+                 where ss_store_sk = 2
+                 group by ss_item_sk
+                 having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col
+                                                  from store_sales
+                                                  where ss_store_sk = 2
+                                                    and ss_hdemo_sk is null
+                                                  group by ss_store_sk))V1)V11
+     where rnk  < 11) asceding,
+    (select *
+     from (select item_sk,rank() over (order by rank_col desc) rnk
+           from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col
+                 from store_sales ss1
+                 where ss_store_sk = 2
+                 group by ss_item_sk
+                 having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col
+                                                  from store_sales
+                                                  where ss_store_sk = 2
+                                                    and ss_hdemo_sk is null
+                                                  group by ss_store_sk))V2)V21
+     where rnk  < 11) descending,
+item i1,
+item i2
+where asceding.rnk = descending.rnk 
+  and i1.i_item_sk=asceding.item_sk
+  and i2.i_item_sk=descending.item_sk
+order by asceding.rnk
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query45.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query45.sql
new file mode 100644
index 0000000..765456a
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query45.sql
@@ -0,0 +1,33 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select  ca_zip, ca_county, sum(ws_sales_price)
+ from web_sales, customer, customer_address, date_dim, item
+ where ws_bill_customer_sk = c_customer_sk
+ 	and c_current_addr_sk = ca_address_sk 
+ 	and ws_item_sk = i_item_sk 
+ 	and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', '85392', '85460', '80348', '81792')
+ 	      or 
+ 	      i_item_id in (select i_item_id
+                             from item
+                             where i_item_sk in (2, 3, 5, 7, 11, 13, 17, 19, 23, 29)
+                             )
+ 	    )
+ 	and ws_sold_date_sk = d_date_sk
+ 	and d_qoy = 2 and d_year = 2000
+ group by ca_zip, ca_county
+ order by ca_zip, ca_county
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query46.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query46.sql
new file mode 100644
index 0000000..f58de5e
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query46.sql
@@ -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.
+
+select  c_last_name
+       ,c_first_name
+       ,ca_city
+       ,bought_city
+       ,ss_ticket_number
+       ,amt,profit 
+ from
+   (select ss_ticket_number
+          ,ss_customer_sk
+          ,ca_city bought_city
+          ,sum(ss_coupon_amt) amt
+          ,sum(ss_net_profit) profit
+    from store_sales,date_dim,store,household_demographics,customer_address 
+    where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+    and store_sales.ss_store_sk = store.s_store_sk  
+    and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+    and store_sales.ss_addr_sk = customer_address.ca_address_sk
+    and (household_demographics.hd_dep_count = 5 or
+         household_demographics.hd_vehicle_count= 3)
+    and date_dim.d_dow in (6,0)
+    and date_dim.d_year in (1999,1999+1,1999+2) 
+    and store.s_city in ('Midway','Fairview','Fairview','Midway','Fairview') 
+    group by ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city) dn,customer,customer_address current_addr
+    where ss_customer_sk = c_customer_sk
+      and customer.c_current_addr_sk = current_addr.ca_address_sk
+      and current_addr.ca_city <> bought_city
+  order by c_last_name
+          ,c_first_name
+          ,ca_city
+          ,bought_city
+          ,ss_ticket_number
+  limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query47.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query47.sql
new file mode 100644
index 0000000..9d2e4ca
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query47.sql
@@ -0,0 +1,64 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with v1 as(
+ select i_category, i_brand,
+        s_store_name, s_company_name,
+        d_year, d_moy,
+        sum(ss_sales_price) sum_sales,
+        avg(sum(ss_sales_price)) over
+          (partition by i_category, i_brand,
+                     s_store_name, s_company_name, d_year)
+          avg_monthly_sales,
+        rank() over
+          (partition by i_category, i_brand,
+                     s_store_name, s_company_name
+           order by d_year, d_moy) rn
+ from item, store_sales, date_dim, store
+ where ss_item_sk = i_item_sk and
+       ss_sold_date_sk = d_date_sk and
+       ss_store_sk = s_store_sk and
+       (
+         d_year = 2000 or
+         ( d_year = 2000-1 and d_moy =12) or
+         ( d_year = 2000+1 and d_moy =1)
+       )
+ group by i_category, i_brand,
+          s_store_name, s_company_name,
+          d_year, d_moy),
+ v2 as(
+ select v1.i_category, v1.i_brand
+        ,v1.d_year, v1.d_moy
+        ,v1.avg_monthly_sales
+        ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum
+ from v1, v1 v1_lag, v1 v1_lead
+ where v1.i_category = v1_lag.i_category and
+       v1.i_category = v1_lead.i_category and
+       v1.i_brand = v1_lag.i_brand and
+       v1.i_brand = v1_lead.i_brand and
+       v1.s_store_name = v1_lag.s_store_name and
+       v1.s_store_name = v1_lead.s_store_name and
+       v1.s_company_name = v1_lag.s_company_name and
+       v1.s_company_name = v1_lead.s_company_name and
+       v1.rn = v1_lag.rn + 1 and
+       v1.rn = v1_lead.rn - 1)
+  select  *
+ from v2
+ where  d_year = 2000 and    
+        avg_monthly_sales > 0 and
+        case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1
+ order by sum_sales - avg_monthly_sales, nsum
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query48.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query48.sql
new file mode 100644
index 0000000..a924396
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query48.sql
@@ -0,0 +1,79 @@
+-- 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.
+
+select sum (ss_quantity)
+ from store_sales, store, customer_demographics, customer_address, date_dim
+ where s_store_sk = ss_store_sk
+ and  ss_sold_date_sk = d_date_sk and d_year = 1998
+ and  
+ (
+  (
+   cd_demo_sk = ss_cdemo_sk
+   and 
+   cd_marital_status = 'M'
+   and 
+   cd_education_status = '4 yr Degree'
+   and 
+   ss_sales_price between 100.00 and 150.00  
+   )
+ or
+  (
+  cd_demo_sk = ss_cdemo_sk
+   and 
+   cd_marital_status = 'D'
+   and 
+   cd_education_status = 'Primary'
+   and 
+   ss_sales_price between 50.00 and 100.00   
+  )
+ or 
+ (
+  cd_demo_sk = ss_cdemo_sk
+  and 
+   cd_marital_status = 'U'
+   and 
+   cd_education_status = 'Advanced Degree'
+   and 
+   ss_sales_price between 150.00 and 200.00  
+ )
+ )
+ and
+ (
+  (
+  ss_addr_sk = ca_address_sk
+  and
+  ca_country = 'United States'
+  and
+  ca_state in ('KY', 'GA', 'NM')
+  and ss_net_profit between 0 and 2000  
+  )
+ or
+  (ss_addr_sk = ca_address_sk
+  and
+  ca_country = 'United States'
+  and
+  ca_state in ('MT', 'OR', 'IN')
+  and ss_net_profit between 150 and 3000 
+  )
+ or
+  (ss_addr_sk = ca_address_sk
+  and
+  ca_country = 'United States'
+  and
+  ca_state in ('WI', 'MO', 'WV')
+  and ss_net_profit between 50 and 25000 
+  )
+ )
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query49.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query49.sql
new file mode 100644
index 0000000..2e29e15
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query49.sql
@@ -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.
+
+select  channel, item, return_ratio, return_rank, currency_rank from
+ (select
+ 'web' as channel
+ ,web.item
+ ,web.return_ratio
+ ,web.return_rank
+ ,web.currency_rank
+ from (
+ 	select 
+ 	 item
+ 	,return_ratio
+ 	,currency_ratio
+ 	,rank() over (order by return_ratio) as return_rank
+ 	,rank() over (order by currency_ratio) as currency_rank
+ 	from
+ 	(	select ws.ws_item_sk as item
+ 		,(cast(sum(coalesce(wr.wr_return_quantity,0)) as decimal(15,4))/
+ 		cast(sum(coalesce(ws.ws_quantity,0)) as decimal(15,4) )) as return_ratio
+ 		,(cast(sum(coalesce(wr.wr_return_amt,0)) as decimal(15,4))/
+ 		cast(sum(coalesce(ws.ws_net_paid,0)) as decimal(15,4) )) as currency_ratio
+ 		from 
+ 		 web_sales ws left outer join web_returns wr 
+ 			on (ws.ws_order_number = wr.wr_order_number and 
+ 			ws.ws_item_sk = wr.wr_item_sk)
+                 ,date_dim
+ 		where 
+ 			wr.wr_return_amt > 10000 
+ 			and ws.ws_net_profit > 1
+                         and ws.ws_net_paid > 0
+                         and ws.ws_quantity > 0
+                         and ws_sold_date_sk = d_date_sk
+                         and d_year = 2000
+                         and d_moy = 12
+ 		group by ws.ws_item_sk
+ 	) in_web
+ ) web
+ where 
+ (
+ web.return_rank <= 10
+ or
+ web.currency_rank <= 10
+ )
+ union
+ select 
+ 'catalog' as channel
+ ,catalog.item
+ ,catalog.return_ratio
+ ,catalog.return_rank
+ ,catalog.currency_rank
+ from (
+ 	select 
+ 	 item
+ 	,return_ratio
+ 	,currency_ratio
+ 	,rank() over (order by return_ratio) as return_rank
+ 	,rank() over (order by currency_ratio) as currency_rank
+ 	from
+ 	(	select 
+ 		cs.cs_item_sk as item
+ 		,(cast(sum(coalesce(cr.cr_return_quantity,0)) as decimal(15,4))/
+ 		cast(sum(coalesce(cs.cs_quantity,0)) as decimal(15,4) )) as return_ratio
+ 		,(cast(sum(coalesce(cr.cr_return_amount,0)) as decimal(15,4))/
+ 		cast(sum(coalesce(cs.cs_net_paid,0)) as decimal(15,4) )) as currency_ratio
+ 		from 
+ 		catalog_sales cs left outer join catalog_returns cr
+ 			on (cs.cs_order_number = cr.cr_order_number and 
+ 			cs.cs_item_sk = cr.cr_item_sk)
+                ,date_dim
+ 		where 
+ 			cr.cr_return_amount > 10000 
+ 			and cs.cs_net_profit > 1
+                         and cs.cs_net_paid > 0
+                         and cs.cs_quantity > 0
+                         and cs_sold_date_sk = d_date_sk
+                         and d_year = 2000
+                         and d_moy = 12
+                 group by cs.cs_item_sk
+ 	) in_cat
+ ) catalog
+ where 
+ (
+ catalog.return_rank <= 10
+ or
+ catalog.currency_rank <=10
+ )
+ union
+ select 
+ 'store' as channel
+ ,store.item
+ ,store.return_ratio
+ ,store.return_rank
+ ,store.currency_rank
+ from (
+ 	select 
+ 	 item
+ 	,return_ratio
+ 	,currency_ratio
+ 	,rank() over (order by return_ratio) as return_rank
+ 	,rank() over (order by currency_ratio) as currency_rank
+ 	from
+ 	(	select sts.ss_item_sk as item
+ 		,(cast(sum(coalesce(sr.sr_return_quantity,0)) as decimal(15,4))/cast(sum(coalesce(sts.ss_quantity,0)) as decimal(15,4) )) as return_ratio
+ 		,(cast(sum(coalesce(sr.sr_return_amt,0)) as decimal(15,4))/cast(sum(coalesce(sts.ss_net_paid,0)) as decimal(15,4) )) as currency_ratio
+ 		from 
+ 		store_sales sts left outer join store_returns sr
+ 			on (sts.ss_ticket_number = sr.sr_ticket_number and sts.ss_item_sk = sr.sr_item_sk)
+                ,date_dim
+ 		where 
+ 			sr.sr_return_amt > 10000 
+ 			and sts.ss_net_profit > 1
+                         and sts.ss_net_paid > 0 
+                         and sts.ss_quantity > 0
+                         and ss_sold_date_sk = d_date_sk
+                         and d_year = 2000
+                         and d_moy = 12
+ 		group by sts.ss_item_sk
+ 	) in_store
+ ) store
+ where  (
+ store.return_rank <= 10
+ or 
+ store.currency_rank <= 10
+ )
+ )
+ order by 1,4,5,2
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query5.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query5.sql
new file mode 100644
index 0000000..da2e30a
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query5.sql
@@ -0,0 +1,141 @@
+-- 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.
+
+with ssr as
+ (select s_store_id,
+        sum(sales_price) as sales,
+        sum(profit) as profit,
+        sum(return_amt) as returns,
+        sum(net_loss) as profit_loss
+ from
+  ( select  ss_store_sk as store_sk,
+            ss_sold_date_sk  as date_sk,
+            ss_ext_sales_price as sales_price,
+            ss_net_profit as profit,
+            cast(0 as decimal(7,2)) as return_amt,
+            cast(0 as decimal(7,2)) as net_loss
+    from store_sales
+    union all
+    select sr_store_sk as store_sk,
+           sr_returned_date_sk as date_sk,
+           cast(0 as decimal(7,2)) as sales_price,
+           cast(0 as decimal(7,2)) as profit,
+           sr_return_amt as return_amt,
+           sr_net_loss as net_loss
+    from store_returns
+   ) salesreturns,
+     date_dim,
+     store
+ where date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date) 
+                  and (cast('1998-08-04' as date) +  14 days)
+       and store_sk = s_store_sk
+ group by s_store_id)
+ ,
+ csr as
+ (select cp_catalog_page_id,
+        sum(sales_price) as sales,
+        sum(profit) as profit,
+        sum(return_amt) as returns,
+        sum(net_loss) as profit_loss
+ from
+  ( select  cs_catalog_page_sk as page_sk,
+            cs_sold_date_sk  as date_sk,
+            cs_ext_sales_price as sales_price,
+            cs_net_profit as profit,
+            cast(0 as decimal(7,2)) as return_amt,
+            cast(0 as decimal(7,2)) as net_loss
+    from catalog_sales
+    union all
+    select cr_catalog_page_sk as page_sk,
+           cr_returned_date_sk as date_sk,
+           cast(0 as decimal(7,2)) as sales_price,
+           cast(0 as decimal(7,2)) as profit,
+           cr_return_amount as return_amt,
+           cr_net_loss as net_loss
+    from catalog_returns
+   ) salesreturns,
+     date_dim,
+     catalog_page
+ where date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  14 days)
+       and page_sk = cp_catalog_page_sk
+ group by cp_catalog_page_id)
+ ,
+ wsr as
+ (select web_site_id,
+        sum(sales_price) as sales,
+        sum(profit) as profit,
+        sum(return_amt) as returns,
+        sum(net_loss) as profit_loss
+ from
+  ( select  ws_web_site_sk as wsr_web_site_sk,
+            ws_sold_date_sk  as date_sk,
+            ws_ext_sales_price as sales_price,
+            ws_net_profit as profit,
+            cast(0 as decimal(7,2)) as return_amt,
+            cast(0 as decimal(7,2)) as net_loss
+    from web_sales
+    union all
+    select ws_web_site_sk as wsr_web_site_sk,
+           wr_returned_date_sk as date_sk,
+           cast(0 as decimal(7,2)) as sales_price,
+           cast(0 as decimal(7,2)) as profit,
+           wr_return_amt as return_amt,
+           wr_net_loss as net_loss
+    from web_returns left outer join web_sales on
+         ( wr_item_sk = ws_item_sk
+           and wr_order_number = ws_order_number)
+   ) salesreturns,
+     date_dim,
+     web_site
+ where date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  14 days)
+       and wsr_web_site_sk = web_site_sk
+ group by web_site_id)
+  select  channel
+        , id
+        , sum(sales) as sales
+        , sum(returns) as returns
+        , sum(profit) as profit
+ from 
+ (select 'store channel' as channel
+        , 'store' || s_store_id as id
+        , sales
+        , returns
+        , (profit - profit_loss) as profit
+ from   ssr
+ union all
+ select 'catalog channel' as channel
+        , 'catalog_page' || cp_catalog_page_id as id
+        , sales
+        , returns
+        , (profit - profit_loss) as profit
+ from  csr
+ union all
+ select 'web channel' as channel
+        , 'web_site' || web_site_id as id
+        , sales
+        , returns
+        , (profit - profit_loss) as profit
+ from   wsr
+ ) x
+ group by rollup (channel, id)
+ order by channel
+         ,id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query50.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query50.sql
new file mode 100644
index 0000000..fc37add
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query50.sql
@@ -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.
+
+select
+   s_store_name
+  ,s_company_id
+  ,s_street_number
+  ,s_street_name
+  ,s_street_type
+  ,s_suite_number
+  ,s_city
+  ,s_county
+  ,s_state
+  ,s_zip
+  ,sum(case when (sr_returned_date_sk - ss_sold_date_sk <= 30 ) then 1 else 0 end)  as "30 days" 
+  ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 30) and 
+                 (sr_returned_date_sk - ss_sold_date_sk <= 60) then 1 else 0 end )  as "31-60 days" 
+  ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 60) and 
+                 (sr_returned_date_sk - ss_sold_date_sk <= 90) then 1 else 0 end)  as "61-90 days" 
+  ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 90) and
+                 (sr_returned_date_sk - ss_sold_date_sk <= 120) then 1 else 0 end)  as "91-120 days" 
+  ,sum(case when (sr_returned_date_sk - ss_sold_date_sk  > 120) then 1 else 0 end)  as ">120 days" 
+from
+   store_sales
+  ,store_returns
+  ,store
+  ,date_dim d1
+  ,date_dim d2
+where
+    d2.d_year = 2000
+and d2.d_moy  = 9
+and ss_ticket_number = sr_ticket_number
+and ss_item_sk = sr_item_sk
+and ss_sold_date_sk   = d1.d_date_sk
+and sr_returned_date_sk   = d2.d_date_sk
+and ss_customer_sk = sr_customer_sk
+and ss_store_sk = s_store_sk
+group by
+   s_store_name
+  ,s_company_id
+  ,s_street_number
+  ,s_street_name
+  ,s_street_type
+  ,s_suite_number
+  ,s_city
+  ,s_county
+  ,s_state
+  ,s_zip
+order by s_store_name
+        ,s_company_id
+        ,s_street_number
+        ,s_street_name
+        ,s_street_type
+        ,s_suite_number
+        ,s_city
+        ,s_county
+        ,s_state
+        ,s_zip
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query51.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query51.sql
new file mode 100644
index 0000000..49b6d1b
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query51.sql
@@ -0,0 +1,58 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+WITH web_v1 as (
+select
+  ws_item_sk item_sk, d_date,
+  sum(sum(ws_sales_price))
+      over (partition by ws_item_sk order by d_date rows between unbounded preceding and current row) cume_sales
+from web_sales
+    ,date_dim
+where ws_sold_date_sk=d_date_sk
+  and d_month_seq between 1212 and 1212+11
+  and ws_item_sk is not NULL
+group by ws_item_sk, d_date),
+store_v1 as (
+select
+  ss_item_sk item_sk, d_date,
+  sum(sum(ss_sales_price))
+      over (partition by ss_item_sk order by d_date rows between unbounded preceding and current row) cume_sales
+from store_sales
+    ,date_dim
+where ss_sold_date_sk=d_date_sk
+  and d_month_seq between 1212 and 1212+11
+  and ss_item_sk is not NULL
+group by ss_item_sk, d_date)
+ select  *
+from (select item_sk
+     ,d_date
+     ,web_sales
+     ,store_sales
+     ,max(web_sales)
+         over (partition by item_sk order by d_date rows between unbounded preceding and current row) web_cumulative
+     ,max(store_sales)
+         over (partition by item_sk order by d_date rows between unbounded preceding and current row) store_cumulative
+     from (select case when web.item_sk is not null then web.item_sk else store.item_sk end item_sk
+                 ,case when web.d_date is not null then web.d_date else store.d_date end d_date
+                 ,web.cume_sales web_sales
+                 ,store.cume_sales store_sales
+           from web_v1 web full outer join store_v1 store on (web.item_sk = store.item_sk
+                                                          and web.d_date = store.d_date)
+          )x )y
+where web_cumulative > store_cumulative
+order by item_sk
+        ,d_date
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query52.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query52.sql
new file mode 100644
index 0000000..5422e43
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query52.sql
@@ -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.
+
+select  dt.d_year
+ 	,item.i_brand_id brand_id
+ 	,item.i_brand brand
+ 	,sum(ss_ext_sales_price) ext_price
+ from date_dim dt
+     ,store_sales
+     ,item
+ where dt.d_date_sk = store_sales.ss_sold_date_sk
+    and store_sales.ss_item_sk = item.i_item_sk
+    and item.i_manager_id = 1
+    and dt.d_moy=12
+    and dt.d_year=1998
+ group by dt.d_year
+ 	,item.i_brand
+ 	,item.i_brand_id
+ order by dt.d_year
+ 	,ext_price desc
+ 	,brand_id
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query53.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query53.sql
new file mode 100644
index 0000000..5807a7b
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query53.sql
@@ -0,0 +1,41 @@
+-- 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.
+
+select  * from
+(select i_manufact_id,
+sum(ss_sales_price) sum_sales,
+avg(sum(ss_sales_price)) over (partition by i_manufact_id) avg_quarterly_sales
+from item, store_sales, date_dim, store
+where ss_item_sk = i_item_sk and
+ss_sold_date_sk = d_date_sk and
+ss_store_sk = s_store_sk and
+d_month_seq in (1212,1212+1,1212+2,1212+3,1212+4,1212+5,1212+6,1212+7,1212+8,1212+9,1212+10,1212+11) and
+((i_category in ('Books','Children','Electronics') and
+i_class in ('personal','portable','reference','self-help') and
+i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7',
+		'exportiunivamalg #9','scholaramalgamalg #9'))
+or(i_category in ('Women','Music','Men') and
+i_class in ('accessories','classical','fragrances','pants') and
+i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1',
+		'importoamalg #1')))
+group by i_manufact_id, d_qoy ) tmp1
+where case when avg_quarterly_sales > 0 
+	then abs (sum_sales - avg_quarterly_sales)/ avg_quarterly_sales 
+	else null end > 0.1
+order by avg_quarterly_sales,
+	 sum_sales,
+	 i_manufact_id
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query54.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query54.sql
new file mode 100644
index 0000000..93c5af7
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query54.sql
@@ -0,0 +1,69 @@
+-- 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.
+
+with my_customers as (
+ select distinct c_customer_sk
+        , c_current_addr_sk
+ from   
+        ( select cs_sold_date_sk sold_date_sk,
+                 cs_bill_customer_sk customer_sk,
+                 cs_item_sk item_sk
+          from   catalog_sales
+          union all
+          select ws_sold_date_sk sold_date_sk,
+                 ws_bill_customer_sk customer_sk,
+                 ws_item_sk item_sk
+          from   web_sales
+         ) cs_or_ws_sales,
+         item,
+         date_dim,
+         customer
+ where   sold_date_sk = d_date_sk
+         and item_sk = i_item_sk
+         and i_category = 'Jewelry'
+         and i_class = 'consignment'
+         and c_customer_sk = cs_or_ws_sales.customer_sk
+         and d_moy = 3
+         and d_year = 1999
+ )
+ , my_revenue as (
+ select c_customer_sk,
+        sum(ss_ext_sales_price) as revenue
+ from   my_customers,
+        store_sales,
+        customer_address,
+        store,
+        date_dim
+ where  c_current_addr_sk = ca_address_sk
+        and ca_county = s_county
+        and ca_state = s_state
+        and ss_sold_date_sk = d_date_sk
+        and c_customer_sk = ss_customer_sk
+        and d_month_seq between (select distinct d_month_seq+1
+                                 from   date_dim where d_year = 1999 and d_moy = 3)
+                           and  (select distinct d_month_seq+3
+                                 from   date_dim where d_year = 1999 and d_moy = 3)
+ group by c_customer_sk
+ )
+ , segments as
+ (select cast((revenue/50) as int) as segment
+  from   my_revenue
+ )
+  select  segment, count(*) as num_customers, segment*50 as segment_base
+ from segments
+ group by segment
+ order by segment, num_customers
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query55.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query55.sql
new file mode 100644
index 0000000..5ae4d34
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query55.sql
@@ -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.
+
+select  i_brand_id brand_id, i_brand brand,
+ 	sum(ss_ext_sales_price) ext_price
+ from date_dim, store_sales, item
+ where d_date_sk = ss_sold_date_sk
+ 	and ss_item_sk = i_item_sk
+ 	and i_manager_id=36
+ 	and d_moy=12
+ 	and d_year=2001
+ group by i_brand, i_brand_id
+ order by ext_price desc, i_brand_id
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query56.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query56.sql
new file mode 100644
index 0000000..0e76d59
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query56.sql
@@ -0,0 +1,82 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with ss as (
+ select i_item_id,sum(ss_ext_sales_price) total_sales
+ from
+ 	store_sales,
+ 	date_dim,
+         customer_address,
+         item
+ where i_item_id in (select
+     i_item_id
+from item
+where i_color in ('orchid','chiffon','lace'))
+ and     ss_item_sk              = i_item_sk
+ and     ss_sold_date_sk         = d_date_sk
+ and     d_year                  = 2000
+ and     d_moy                   = 1
+ and     ss_addr_sk              = ca_address_sk
+ and     ca_gmt_offset           = -8 
+ group by i_item_id),
+ cs as (
+ select i_item_id,sum(cs_ext_sales_price) total_sales
+ from
+ 	catalog_sales,
+ 	date_dim,
+         customer_address,
+         item
+ where
+         i_item_id               in (select
+  i_item_id
+from item
+where i_color in ('orchid','chiffon','lace'))
+ and     cs_item_sk              = i_item_sk
+ and     cs_sold_date_sk         = d_date_sk
+ and     d_year                  = 2000
+ and     d_moy                   = 1
+ and     cs_bill_addr_sk         = ca_address_sk
+ and     ca_gmt_offset           = -8 
+ group by i_item_id),
+ ws as (
+ select i_item_id,sum(ws_ext_sales_price) total_sales
+ from
+ 	web_sales,
+ 	date_dim,
+         customer_address,
+         item
+ where
+         i_item_id               in (select
+  i_item_id
+from item
+where i_color in ('orchid','chiffon','lace'))
+ and     ws_item_sk              = i_item_sk
+ and     ws_sold_date_sk         = d_date_sk
+ and     d_year                  = 2000
+ and     d_moy                   = 1
+ and     ws_bill_addr_sk         = ca_address_sk
+ and     ca_gmt_offset           = -8
+ group by i_item_id)
+  select  i_item_id ,sum(total_sales) total_sales
+ from  (select * from ss 
+        union all
+        select * from cs 
+        union all
+        select * from ws) tmp1
+ group by i_item_id
+ order by total_sales,
+          i_item_id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query57.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query57.sql
new file mode 100644
index 0000000..eb3b1fc
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query57.sql
@@ -0,0 +1,61 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with v1 as(
+ select i_category, i_brand,
+        cc_name,
+        d_year, d_moy,
+        sum(cs_sales_price) sum_sales,
+        avg(sum(cs_sales_price)) over
+          (partition by i_category, i_brand,
+                     cc_name, d_year)
+          avg_monthly_sales,
+        rank() over
+          (partition by i_category, i_brand,
+                     cc_name
+           order by d_year, d_moy) rn
+ from item, catalog_sales, date_dim, call_center
+ where cs_item_sk = i_item_sk and
+       cs_sold_date_sk = d_date_sk and
+       cc_call_center_sk= cs_call_center_sk and
+       (
+         d_year = 2000 or
+         ( d_year = 2000-1 and d_moy =12) or
+         ( d_year = 2000+1 and d_moy =1)
+       )
+ group by i_category, i_brand,
+          cc_name , d_year, d_moy),
+ v2 as(
+ select v1.cc_name
+        ,v1.d_year, v1.d_moy
+        ,v1.avg_monthly_sales
+        ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum
+ from v1, v1 v1_lag, v1 v1_lead
+ where v1.i_category = v1_lag.i_category and
+       v1.i_category = v1_lead.i_category and
+       v1.i_brand = v1_lag.i_brand and
+       v1.i_brand = v1_lead.i_brand and
+       v1. cc_name = v1_lag. cc_name and
+       v1. cc_name = v1_lead. cc_name and
+       v1.rn = v1_lag.rn + 1 and
+       v1.rn = v1_lead.rn - 1)
+  select  *
+ from v2
+ where  d_year = 2000 and
+        avg_monthly_sales > 0 and
+        case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1
+ order by sum_sales - avg_monthly_sales, nsum
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query58.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query58.sql
new file mode 100644
index 0000000..42366e6
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query58.sql
@@ -0,0 +1,78 @@
+-- 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.
+
+with ss_items as
+ (select i_item_id item_id
+        ,sum(ss_ext_sales_price) ss_item_rev 
+ from store_sales
+     ,item
+     ,date_dim
+ where ss_item_sk = i_item_sk
+   and d_date in (select d_date
+                  from date_dim
+                  where d_week_seq = (select d_week_seq 
+                                      from date_dim
+                                      where d_date = '1998-02-19'))
+   and ss_sold_date_sk   = d_date_sk
+ group by i_item_id),
+ cs_items as
+ (select i_item_id item_id
+        ,sum(cs_ext_sales_price) cs_item_rev
+  from catalog_sales
+      ,item
+      ,date_dim
+ where cs_item_sk = i_item_sk
+  and  d_date in (select d_date
+                  from date_dim
+                  where d_week_seq = (select d_week_seq 
+                                      from date_dim
+                                      where d_date = '1998-02-19'))
+  and  cs_sold_date_sk = d_date_sk
+ group by i_item_id),
+ ws_items as
+ (select i_item_id item_id
+        ,sum(ws_ext_sales_price) ws_item_rev
+  from web_sales
+      ,item
+      ,date_dim
+ where ws_item_sk = i_item_sk
+  and  d_date in (select d_date
+                  from date_dim
+                  where d_week_seq =(select d_week_seq 
+                                     from date_dim
+                                     where d_date = '1998-02-19'))
+  and ws_sold_date_sk   = d_date_sk
+ group by i_item_id)
+  select  ss_items.item_id
+       ,ss_item_rev
+       ,ss_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 ss_dev
+       ,cs_item_rev
+       ,cs_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 cs_dev
+       ,ws_item_rev
+       ,ws_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 ws_dev
+       ,(ss_item_rev+cs_item_rev+ws_item_rev)/3 average
+ from ss_items,cs_items,ws_items
+ where ss_items.item_id=cs_items.item_id
+   and ss_items.item_id=ws_items.item_id 
+   and ss_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev
+   and ss_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev
+   and cs_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev
+   and cs_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev
+   and ws_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev
+   and ws_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev
+ order by item_id
+         ,ss_item_rev
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query59.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query59.sql
new file mode 100644
index 0000000..462ef96
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query59.sql
@@ -0,0 +1,57 @@
+-- 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.
+
+with wss as
+ (select d_week_seq,
+        ss_store_sk,
+        sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales,
+        sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales,
+        sum(case when (d_day_name='Tuesday') then ss_sales_price else  null end) tue_sales,
+        sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales,
+        sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales,
+        sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales,
+        sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales
+ from store_sales,date_dim
+ where d_date_sk = ss_sold_date_sk
+ group by d_week_seq,ss_store_sk
+ )
+  select  s_store_name1,s_store_id1,d_week_seq1
+       ,sun_sales1/sun_sales2,mon_sales1/mon_sales2
+       ,tue_sales1/tue_sales2,wed_sales1/wed_sales2,thu_sales1/thu_sales2
+       ,fri_sales1/fri_sales2,sat_sales1/sat_sales2
+ from
+ (select s_store_name s_store_name1,wss.d_week_seq d_week_seq1
+        ,s_store_id s_store_id1,sun_sales sun_sales1
+        ,mon_sales mon_sales1,tue_sales tue_sales1
+        ,wed_sales wed_sales1,thu_sales thu_sales1
+        ,fri_sales fri_sales1,sat_sales sat_sales1
+  from wss,store,date_dim d
+  where d.d_week_seq = wss.d_week_seq and
+        ss_store_sk = s_store_sk and 
+        d_month_seq between 1185 and 1185 + 11) y,
+ (select s_store_name s_store_name2,wss.d_week_seq d_week_seq2
+        ,s_store_id s_store_id2,sun_sales sun_sales2
+        ,mon_sales mon_sales2,tue_sales tue_sales2
+        ,wed_sales wed_sales2,thu_sales thu_sales2
+        ,fri_sales fri_sales2,sat_sales sat_sales2
+  from wss,store,date_dim d
+  where d.d_week_seq = wss.d_week_seq and
+        ss_store_sk = s_store_sk and 
+        d_month_seq between 1185+ 12 and 1185 + 23) x
+ where s_store_id1=s_store_id2
+   and d_week_seq1=d_week_seq2-52
+ order by s_store_name1,s_store_id1,d_week_seq1
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query6.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query6.sql
new file mode 100644
index 0000000..389c61a
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query6.sql
@@ -0,0 +1,39 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select  a.ca_state state, count(*) cnt
+ from customer_address a
+     ,customer c
+     ,store_sales s
+     ,date_dim d
+     ,item i
+ where       a.ca_address_sk = c.c_current_addr_sk
+ 	and c.c_customer_sk = s.ss_customer_sk
+ 	and s.ss_sold_date_sk = d.d_date_sk
+ 	and s.ss_item_sk = i.i_item_sk
+ 	and d.d_month_seq = 
+ 	     (select distinct (d_month_seq)
+ 	      from date_dim
+               where d_year = 2000
+ 	        and d_moy = 2 )
+ 	and i.i_current_price > 1.2 * 
+             (select avg(j.i_current_price) 
+ 	     from item j 
+ 	     where j.i_category = i.i_category)
+ group by a.ca_state
+ having count(*) >= 10
+ order by cnt, a.ca_state 
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query60.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query60.sql
new file mode 100644
index 0000000..9f84287
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query60.sql
@@ -0,0 +1,91 @@
+-- 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.
+
+with ss as (
+ select
+          i_item_id,sum(ss_ext_sales_price) total_sales
+ from
+ 	store_sales,
+ 	date_dim,
+         customer_address,
+         item
+ where
+         i_item_id in (select
+  i_item_id
+from
+ item
+where i_category in ('Children'))
+ and     ss_item_sk              = i_item_sk
+ and     ss_sold_date_sk         = d_date_sk
+ and     d_year                  = 1999
+ and     d_moy                   = 9
+ and     ss_addr_sk              = ca_address_sk
+ and     ca_gmt_offset           = -6 
+ group by i_item_id),
+ cs as (
+ select
+          i_item_id,sum(cs_ext_sales_price) total_sales
+ from
+ 	catalog_sales,
+ 	date_dim,
+         customer_address,
+         item
+ where
+         i_item_id               in (select
+  i_item_id
+from
+ item
+where i_category in ('Children'))
+ and     cs_item_sk              = i_item_sk
+ and     cs_sold_date_sk         = d_date_sk
+ and     d_year                  = 1999
+ and     d_moy                   = 9
+ and     cs_bill_addr_sk         = ca_address_sk
+ and     ca_gmt_offset           = -6 
+ group by i_item_id),
+ ws as (
+ select
+          i_item_id,sum(ws_ext_sales_price) total_sales
+ from
+ 	web_sales,
+ 	date_dim,
+         customer_address,
+         item
+ where
+         i_item_id               in (select
+  i_item_id
+from
+ item
+where i_category in ('Children'))
+ and     ws_item_sk              = i_item_sk
+ and     ws_sold_date_sk         = d_date_sk
+ and     d_year                  = 1999
+ and     d_moy                   = 9
+ and     ws_bill_addr_sk         = ca_address_sk
+ and     ca_gmt_offset           = -6
+ group by i_item_id)
+  select   
+  i_item_id
+,sum(total_sales) total_sales
+ from  (select * from ss 
+        union all
+        select * from cs 
+        union all
+        select * from ws) tmp1
+ group by i_item_id
+ order by i_item_id
+      ,total_sales
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query61.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query61.sql
new file mode 100644
index 0000000..4e02adb
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query61.sql
@@ -0,0 +1,57 @@
+-- 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.
+
+select  promotions,total,cast(promotions as decimal(15,4))/cast(total as decimal(15,4))*100
+from
+  (select sum(ss_ext_sales_price) promotions
+   from  store_sales
+        ,store
+        ,promotion
+        ,date_dim
+        ,customer
+        ,customer_address 
+        ,item
+   where ss_sold_date_sk = d_date_sk
+   and   ss_store_sk = s_store_sk
+   and   ss_promo_sk = p_promo_sk
+   and   ss_customer_sk= c_customer_sk
+   and   ca_address_sk = c_current_addr_sk
+   and   ss_item_sk = i_item_sk 
+   and   ca_gmt_offset = -7
+   and   i_category = 'Books'
+   and   (p_channel_dmail = 'Y' or p_channel_email = 'Y' or p_channel_tv = 'Y')
+   and   s_gmt_offset = -7
+   and   d_year = 1999
+   and   d_moy  = 11) promotional_sales,
+  (select sum(ss_ext_sales_price) total
+   from  store_sales
+        ,store
+        ,date_dim
+        ,customer
+        ,customer_address
+        ,item
+   where ss_sold_date_sk = d_date_sk
+   and   ss_store_sk = s_store_sk
+   and   ss_customer_sk= c_customer_sk
+   and   ca_address_sk = c_current_addr_sk
+   and   ss_item_sk = i_item_sk
+   and   ca_gmt_offset = -7
+   and   i_category = 'Books'
+   and   s_gmt_offset = -7
+   and   d_year = 1999
+   and   d_moy  = 11) all_sales
+order by promotions, total
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query62.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query62.sql
new file mode 100644
index 0000000..f3f383e
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query62.sql
@@ -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.
+
+select
+   substr(w_warehouse_name,1,20)
+  ,sm_type
+  ,web_name
+  ,sum(case when (ws_ship_date_sk - ws_sold_date_sk <= 30 ) then 1 else 0 end)  as "30 days" 
+  ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 30) and 
+                 (ws_ship_date_sk - ws_sold_date_sk <= 60) then 1 else 0 end )  as "31-60 days" 
+  ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 60) and 
+                 (ws_ship_date_sk - ws_sold_date_sk <= 90) then 1 else 0 end)  as "61-90 days" 
+  ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 90) and
+                 (ws_ship_date_sk - ws_sold_date_sk <= 120) then 1 else 0 end)  as "91-120 days" 
+  ,sum(case when (ws_ship_date_sk - ws_sold_date_sk  > 120) then 1 else 0 end)  as ">120 days" 
+from
+   web_sales
+  ,warehouse
+  ,ship_mode
+  ,web_site
+  ,date_dim
+where
+    d_month_seq between 1212 and 1212 + 11
+and ws_ship_date_sk   = d_date_sk
+and ws_warehouse_sk   = w_warehouse_sk
+and ws_ship_mode_sk   = sm_ship_mode_sk
+and ws_web_site_sk    = web_site_sk
+group by
+   substr(w_warehouse_name,1,20)
+  ,sm_type
+  ,web_name
+order by substr(w_warehouse_name,1,20)
+        ,sm_type
+       ,web_name
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query63.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query63.sql
new file mode 100644
index 0000000..6e86c17c
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query63.sql
@@ -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.
+
+select  *
+from (select i_manager_id
+             ,sum(ss_sales_price) sum_sales
+             ,avg(sum(ss_sales_price)) over (partition by i_manager_id) avg_monthly_sales
+      from item
+          ,store_sales
+          ,date_dim
+          ,store
+      where ss_item_sk = i_item_sk
+        and ss_sold_date_sk = d_date_sk
+        and ss_store_sk = s_store_sk
+        and d_month_seq in (1212,1212+1,1212+2,1212+3,1212+4,1212+5,1212+6,1212+7,1212+8,1212+9,1212+10,1212+11)
+        and ((    i_category in ('Books','Children','Electronics')
+              and i_class in ('personal','portable','reference','self-help')
+              and i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7',
+		                  'exportiunivamalg #9','scholaramalgamalg #9'))
+           or(    i_category in ('Women','Music','Men')
+              and i_class in ('accessories','classical','fragrances','pants')
+              and i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1',
+		                 'importoamalg #1')))
+group by i_manager_id, d_moy) tmp1
+where case when avg_monthly_sales > 0 then abs (sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1
+order by i_manager_id
+        ,avg_monthly_sales
+        ,sum_sales
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query64.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query64.sql
new file mode 100644
index 0000000..c181e38
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query64.sql
@@ -0,0 +1,134 @@
+-- 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.
+
+with cs_ui as
+ (select cs_item_sk
+        ,sum(cs_ext_list_price) as sale,sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit) as refund
+  from catalog_sales
+      ,catalog_returns
+  where cs_item_sk = cr_item_sk
+    and cs_order_number = cr_order_number
+  group by cs_item_sk
+  having sum(cs_ext_list_price)>2*sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit)),
+cross_sales as
+ (select i_product_name product_name
+     ,i_item_sk item_sk
+     ,s_store_name store_name
+     ,s_zip store_zip
+     ,ad1.ca_street_number b_street_number
+     ,ad1.ca_street_name b_street_name
+     ,ad1.ca_city b_city
+     ,ad1.ca_zip b_zip
+     ,ad2.ca_street_number c_street_number
+     ,ad2.ca_street_name c_street_name
+     ,ad2.ca_city c_city
+     ,ad2.ca_zip c_zip
+     ,d1.d_year as syear
+     ,d2.d_year as fsyear
+     ,d3.d_year s2year
+     ,count(*) cnt
+     ,sum(ss_wholesale_cost) s1
+     ,sum(ss_list_price) s2
+     ,sum(ss_coupon_amt) s3
+  FROM   store_sales
+        ,store_returns
+        ,cs_ui
+        ,date_dim d1
+        ,date_dim d2
+        ,date_dim d3
+        ,store
+        ,customer
+        ,customer_demographics cd1
+        ,customer_demographics cd2
+        ,promotion
+        ,household_demographics hd1
+        ,household_demographics hd2
+        ,customer_address ad1
+        ,customer_address ad2
+        ,income_band ib1
+        ,income_band ib2
+        ,item
+  WHERE  ss_store_sk = s_store_sk AND
+         ss_sold_date_sk = d1.d_date_sk AND
+         ss_customer_sk = c_customer_sk AND
+         ss_cdemo_sk= cd1.cd_demo_sk AND
+         ss_hdemo_sk = hd1.hd_demo_sk AND
+         ss_addr_sk = ad1.ca_address_sk and
+         ss_item_sk = i_item_sk and
+         ss_item_sk = sr_item_sk and
+         ss_ticket_number = sr_ticket_number and
+         ss_item_sk = cs_ui.cs_item_sk and
+         c_current_cdemo_sk = cd2.cd_demo_sk AND
+         c_current_hdemo_sk = hd2.hd_demo_sk AND
+         c_current_addr_sk = ad2.ca_address_sk and
+         c_first_sales_date_sk = d2.d_date_sk and
+         c_first_shipto_date_sk = d3.d_date_sk and
+         ss_promo_sk = p_promo_sk and
+         hd1.hd_income_band_sk = ib1.ib_income_band_sk and
+         hd2.hd_income_band_sk = ib2.ib_income_band_sk and
+         cd1.cd_marital_status <> cd2.cd_marital_status and
+         i_color in ('maroon','burnished','dim','steel','navajo','chocolate') and
+         i_current_price between 35 and 35 + 10 and
+         i_current_price between 35 + 1 and 35 + 15
+group by i_product_name
+       ,i_item_sk
+       ,s_store_name
+       ,s_zip
+       ,ad1.ca_street_number
+       ,ad1.ca_street_name
+       ,ad1.ca_city
+       ,ad1.ca_zip
+       ,ad2.ca_street_number
+       ,ad2.ca_street_name
+       ,ad2.ca_city
+       ,ad2.ca_zip
+       ,d1.d_year
+       ,d2.d_year
+       ,d3.d_year
+)
+select cs1.product_name
+     ,cs1.store_name
+     ,cs1.store_zip
+     ,cs1.b_street_number
+     ,cs1.b_street_name
+     ,cs1.b_city
+     ,cs1.b_zip
+     ,cs1.c_street_number
+     ,cs1.c_street_name
+     ,cs1.c_city
+     ,cs1.c_zip
+     ,cs1.syear
+     ,cs1.cnt
+     ,cs1.s1 as s11
+     ,cs1.s2 as s21
+     ,cs1.s3 as s31
+     ,cs2.s1 as s12
+     ,cs2.s2 as s22
+     ,cs2.s3 as s32
+     ,cs2.syear
+     ,cs2.cnt
+from cross_sales cs1,cross_sales cs2
+where cs1.item_sk=cs2.item_sk and
+     cs1.syear = 2000 and
+     cs2.syear = 2000 + 1 and
+     cs2.cnt <= cs1.cnt and
+     cs1.store_name = cs2.store_name and
+     cs1.store_zip = cs2.store_zip
+order by cs1.product_name
+       ,cs1.store_name
+       ,cs2.cnt
+       ,cs1.s1
+       ,cs2.s1
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query65.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query65.sql
new file mode 100644
index 0000000..a106624
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query65.sql
@@ -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.
+
+select
+	s_store_name,
+	i_item_desc,
+	sc.revenue,
+	i_current_price,
+	i_wholesale_cost,
+	i_brand
+ from store, item,
+     (select ss_store_sk, avg(revenue) as ave
+ 	from
+ 	    (select  ss_store_sk, ss_item_sk, 
+ 		     sum(ss_sales_price) as revenue
+ 		from store_sales, date_dim
+ 		where ss_sold_date_sk = d_date_sk and d_month_seq between 1212 and 1212+11
+ 		group by ss_store_sk, ss_item_sk) sa
+ 	group by ss_store_sk) sb,
+     (select  ss_store_sk, ss_item_sk, sum(ss_sales_price) as revenue
+ 	from store_sales, date_dim
+ 	where ss_sold_date_sk = d_date_sk and d_month_seq between 1212 and 1212+11
+ 	group by ss_store_sk, ss_item_sk) sc
+ where sb.ss_store_sk = sc.ss_store_sk and 
+       sc.revenue <= 0.1 * sb.ave and
+       s_store_sk = sc.ss_store_sk and
+       i_item_sk = sc.ss_item_sk
+ order by s_store_name, i_item_desc
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query66.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query66.sql
new file mode 100644
index 0000000..f99b53b
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query66.sql
@@ -0,0 +1,233 @@
+-- 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.
+
+select
+         w_warehouse_name
+ 	,w_warehouse_sq_ft
+ 	,w_city
+ 	,w_county
+ 	,w_state
+ 	,w_country
+        ,ship_carriers
+        ,year
+ 	,sum(jan_sales) as jan_sales
+ 	,sum(feb_sales) as feb_sales
+ 	,sum(mar_sales) as mar_sales
+ 	,sum(apr_sales) as apr_sales
+ 	,sum(may_sales) as may_sales
+ 	,sum(jun_sales) as jun_sales
+ 	,sum(jul_sales) as jul_sales
+ 	,sum(aug_sales) as aug_sales
+ 	,sum(sep_sales) as sep_sales
+ 	,sum(oct_sales) as oct_sales
+ 	,sum(nov_sales) as nov_sales
+ 	,sum(dec_sales) as dec_sales
+ 	,sum(jan_sales/w_warehouse_sq_ft) as jan_sales_per_sq_foot
+ 	,sum(feb_sales/w_warehouse_sq_ft) as feb_sales_per_sq_foot
+ 	,sum(mar_sales/w_warehouse_sq_ft) as mar_sales_per_sq_foot
+ 	,sum(apr_sales/w_warehouse_sq_ft) as apr_sales_per_sq_foot
+ 	,sum(may_sales/w_warehouse_sq_ft) as may_sales_per_sq_foot
+ 	,sum(jun_sales/w_warehouse_sq_ft) as jun_sales_per_sq_foot
+ 	,sum(jul_sales/w_warehouse_sq_ft) as jul_sales_per_sq_foot
+ 	,sum(aug_sales/w_warehouse_sq_ft) as aug_sales_per_sq_foot
+ 	,sum(sep_sales/w_warehouse_sq_ft) as sep_sales_per_sq_foot
+ 	,sum(oct_sales/w_warehouse_sq_ft) as oct_sales_per_sq_foot
+ 	,sum(nov_sales/w_warehouse_sq_ft) as nov_sales_per_sq_foot
+ 	,sum(dec_sales/w_warehouse_sq_ft) as dec_sales_per_sq_foot
+ 	,sum(jan_net) as jan_net
+ 	,sum(feb_net) as feb_net
+ 	,sum(mar_net) as mar_net
+ 	,sum(apr_net) as apr_net
+ 	,sum(may_net) as may_net
+ 	,sum(jun_net) as jun_net
+ 	,sum(jul_net) as jul_net
+ 	,sum(aug_net) as aug_net
+ 	,sum(sep_net) as sep_net
+ 	,sum(oct_net) as oct_net
+ 	,sum(nov_net) as nov_net
+ 	,sum(dec_net) as dec_net
+ from (
+     select 
+ 	w_warehouse_name
+ 	,w_warehouse_sq_ft
+ 	,w_city
+ 	,w_county
+ 	,w_state
+ 	,w_country
+ 	,'DIAMOND' || ',' || 'AIRBORNE' as ship_carriers
+       ,d_year as year
+ 	,sum(case when d_moy = 1 
+ 		then ws_sales_price* ws_quantity else 0 end) as jan_sales
+ 	,sum(case when d_moy = 2 
+ 		then ws_sales_price* ws_quantity else 0 end) as feb_sales
+ 	,sum(case when d_moy = 3 
+ 		then ws_sales_price* ws_quantity else 0 end) as mar_sales
+ 	,sum(case when d_moy = 4 
+ 		then ws_sales_price* ws_quantity else 0 end) as apr_sales
+ 	,sum(case when d_moy = 5 
+ 		then ws_sales_price* ws_quantity else 0 end) as may_sales
+ 	,sum(case when d_moy = 6 
+ 		then ws_sales_price* ws_quantity else 0 end) as jun_sales
+ 	,sum(case when d_moy = 7 
+ 		then ws_sales_price* ws_quantity else 0 end) as jul_sales
+ 	,sum(case when d_moy = 8 
+ 		then ws_sales_price* ws_quantity else 0 end) as aug_sales
+ 	,sum(case when d_moy = 9 
+ 		then ws_sales_price* ws_quantity else 0 end) as sep_sales
+ 	,sum(case when d_moy = 10 
+ 		then ws_sales_price* ws_quantity else 0 end) as oct_sales
+ 	,sum(case when d_moy = 11
+ 		then ws_sales_price* ws_quantity else 0 end) as nov_sales
+ 	,sum(case when d_moy = 12
+ 		then ws_sales_price* ws_quantity else 0 end) as dec_sales
+ 	,sum(case when d_moy = 1 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as jan_net
+ 	,sum(case when d_moy = 2
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as feb_net
+ 	,sum(case when d_moy = 3 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as mar_net
+ 	,sum(case when d_moy = 4 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as apr_net
+ 	,sum(case when d_moy = 5 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as may_net
+ 	,sum(case when d_moy = 6 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as jun_net
+ 	,sum(case when d_moy = 7 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as jul_net
+ 	,sum(case when d_moy = 8 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as aug_net
+ 	,sum(case when d_moy = 9 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as sep_net
+ 	,sum(case when d_moy = 10 
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as oct_net
+ 	,sum(case when d_moy = 11
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as nov_net
+ 	,sum(case when d_moy = 12
+ 		then ws_net_paid_inc_tax * ws_quantity else 0 end) as dec_net
+     from
+          web_sales
+         ,warehouse
+         ,date_dim
+         ,time_dim
+ 	  ,ship_mode
+     where
+            ws_warehouse_sk =  w_warehouse_sk
+        and ws_sold_date_sk = d_date_sk
+        and ws_sold_time_sk = t_time_sk
+ 	and ws_ship_mode_sk = sm_ship_mode_sk
+        and d_year = 2002
+ 	and t_time between 49530 and 49530+28800 
+ 	and sm_carrier in ('DIAMOND','AIRBORNE')
+     group by 
+        w_warehouse_name
+ 	,w_warehouse_sq_ft
+ 	,w_city
+ 	,w_county
+ 	,w_state
+ 	,w_country
+       ,d_year
+ union all
+     select 
+ 	w_warehouse_name
+ 	,w_warehouse_sq_ft
+ 	,w_city
+ 	,w_county
+ 	,w_state
+ 	,w_country
+ 	,'DIAMOND' || ',' || 'AIRBORNE' as ship_carriers
+       ,d_year as year
+ 	,sum(case when d_moy = 1 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as jan_sales
+ 	,sum(case when d_moy = 2 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as feb_sales
+ 	,sum(case when d_moy = 3 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as mar_sales
+ 	,sum(case when d_moy = 4 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as apr_sales
+ 	,sum(case when d_moy = 5 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as may_sales
+ 	,sum(case when d_moy = 6 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as jun_sales
+ 	,sum(case when d_moy = 7 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as jul_sales
+ 	,sum(case when d_moy = 8 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as aug_sales
+ 	,sum(case when d_moy = 9 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as sep_sales
+ 	,sum(case when d_moy = 10 
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as oct_sales
+ 	,sum(case when d_moy = 11
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as nov_sales
+ 	,sum(case when d_moy = 12
+ 		then cs_ext_sales_price* cs_quantity else 0 end) as dec_sales
+ 	,sum(case when d_moy = 1 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jan_net
+ 	,sum(case when d_moy = 2 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as feb_net
+ 	,sum(case when d_moy = 3 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as mar_net
+ 	,sum(case when d_moy = 4 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as apr_net
+ 	,sum(case when d_moy = 5 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as may_net
+ 	,sum(case when d_moy = 6 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jun_net
+ 	,sum(case when d_moy = 7 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jul_net
+ 	,sum(case when d_moy = 8 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as aug_net
+ 	,sum(case when d_moy = 9 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as sep_net
+ 	,sum(case when d_moy = 10 
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as oct_net
+ 	,sum(case when d_moy = 11
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as nov_net
+ 	,sum(case when d_moy = 12
+ 		then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as dec_net
+     from
+          catalog_sales
+         ,warehouse
+         ,date_dim
+         ,time_dim
+ 	 ,ship_mode
+     where
+            cs_warehouse_sk =  w_warehouse_sk
+        and cs_sold_date_sk = d_date_sk
+        and cs_sold_time_sk = t_time_sk
+ 	and cs_ship_mode_sk = sm_ship_mode_sk
+        and d_year = 2002
+ 	and t_time between 49530 AND 49530+28800 
+ 	and sm_carrier in ('DIAMOND','AIRBORNE')
+     group by 
+        w_warehouse_name
+ 	,w_warehouse_sq_ft
+ 	,w_city
+ 	,w_county
+ 	,w_state
+ 	,w_country
+       ,d_year
+ ) x
+ group by 
+        w_warehouse_name
+ 	,w_warehouse_sq_ft
+ 	,w_city
+ 	,w_county
+ 	,w_state
+ 	,w_country
+ 	,ship_carriers
+       ,year
+ order by w_warehouse_name
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query67.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query67.sql
new file mode 100644
index 0000000..1d90fa2
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query67.sql
@@ -0,0 +1,57 @@
+-- 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.
+
+select  *
+from (select i_category
+            ,i_class
+            ,i_brand
+            ,i_product_name
+            ,d_year
+            ,d_qoy
+            ,d_moy
+            ,s_store_id
+            ,sumsales
+            ,rank() over (partition by i_category order by sumsales desc) rk
+      from (select i_category
+                  ,i_class
+                  ,i_brand
+                  ,i_product_name
+                  ,d_year
+                  ,d_qoy
+                  ,d_moy
+                  ,s_store_id
+                  ,sum(coalesce(ss_sales_price*ss_quantity,0)) sumsales
+            from store_sales
+                ,date_dim
+                ,store
+                ,item
+       where  ss_sold_date_sk=d_date_sk
+          and ss_item_sk=i_item_sk
+          and ss_store_sk = s_store_sk
+          and d_month_seq between 1212 and 1212+11
+       group by  rollup(i_category, i_class, i_brand, i_product_name, d_year, d_qoy, d_moy,s_store_id))dw1) dw2
+where rk <= 100
+order by i_category
+        ,i_class
+        ,i_brand
+        ,i_product_name
+        ,d_year
+        ,d_qoy
+        ,d_moy
+        ,s_store_id
+        ,sumsales
+        ,rk
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query68.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query68.sql
new file mode 100644
index 0000000..8ba3933
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query68.sql
@@ -0,0 +1,55 @@
+-- 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.
+
+select  c_last_name
+       ,c_first_name
+       ,ca_city
+       ,bought_city
+       ,ss_ticket_number
+       ,extended_price
+       ,extended_tax
+       ,list_price
+ from (select ss_ticket_number
+             ,ss_customer_sk
+             ,ca_city bought_city
+             ,sum(ss_ext_sales_price) extended_price 
+             ,sum(ss_ext_list_price) list_price
+             ,sum(ss_ext_tax) extended_tax 
+       from store_sales
+           ,date_dim
+           ,store
+           ,household_demographics
+           ,customer_address 
+       where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+         and store_sales.ss_store_sk = store.s_store_sk  
+        and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+        and store_sales.ss_addr_sk = customer_address.ca_address_sk
+        and date_dim.d_dom between 1 and 2 
+        and (household_demographics.hd_dep_count = 5 or
+             household_demographics.hd_vehicle_count= 3)
+        and date_dim.d_year in (1999,1999+1,1999+2)
+        and store.s_city in ('Midway','Fairview')
+       group by ss_ticket_number
+               ,ss_customer_sk
+               ,ss_addr_sk,ca_city) dn
+      ,customer
+      ,customer_address current_addr
+ where ss_customer_sk = c_customer_sk
+   and customer.c_current_addr_sk = current_addr.ca_address_sk
+   and current_addr.ca_city <> bought_city
+ order by c_last_name
+         ,ss_ticket_number
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query69.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query69.sql
new file mode 100644
index 0000000..103a056
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query69.sql
@@ -0,0 +1,60 @@
+-- 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.
+
+select
+  cd_gender,
+  cd_marital_status,
+  cd_education_status,
+  count(*) cnt1,
+  cd_purchase_estimate,
+  count(*) cnt2,
+  cd_credit_rating,
+  count(*) cnt3
+ from
+  customer c,customer_address ca,customer_demographics
+ where
+  c.c_current_addr_sk = ca.ca_address_sk and
+  ca_state in ('CO','IL','MN') and
+  cd_demo_sk = c.c_current_cdemo_sk and 
+  exists (select *
+          from store_sales,date_dim
+          where c.c_customer_sk = ss_customer_sk and
+                ss_sold_date_sk = d_date_sk and
+                d_year = 1999 and
+                d_moy between 1 and 1+2) and
+   (not exists (select *
+            from web_sales,date_dim
+            where c.c_customer_sk = ws_bill_customer_sk and
+                  ws_sold_date_sk = d_date_sk and
+                  d_year = 1999 and
+                  d_moy between 1 and 1+2) and
+    not exists (select * 
+            from catalog_sales,date_dim
+            where c.c_customer_sk = cs_ship_customer_sk and
+                  cs_sold_date_sk = d_date_sk and
+                  d_year = 1999 and
+                  d_moy between 1 and 1+2))
+ group by cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+ order by cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query7.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query7.sql
new file mode 100644
index 0000000..d190e7d
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query7.sql
@@ -0,0 +1,34 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select  i_item_id,
+        avg(ss_quantity) agg1,
+        avg(ss_list_price) agg2,
+        avg(ss_coupon_amt) agg3,
+        avg(ss_sales_price) agg4 
+ from store_sales, customer_demographics, date_dim, item, promotion
+ where ss_sold_date_sk = d_date_sk and
+       ss_item_sk = i_item_sk and
+       ss_cdemo_sk = cd_demo_sk and
+       ss_promo_sk = p_promo_sk and
+       cd_gender = 'F' and 
+       cd_marital_status = 'W' and
+       cd_education_status = 'Primary' and
+       (p_channel_email = 'N' or p_channel_event = 'N') and
+       d_year = 1998 
+ group by i_item_id
+ order by i_item_id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query70.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query70.sql
new file mode 100644
index 0000000..7ea3448
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query70.sql
@@ -0,0 +1,51 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+    sum(ss_net_profit) as total_sum
+   ,s_state
+   ,s_county
+   ,grouping(s_state)+grouping(s_county) as lochierarchy
+   ,rank() over (
+ 	partition by grouping(s_state)+grouping(s_county),
+ 	case when grouping(s_county) = 0 then s_state end 
+ 	order by sum(ss_net_profit) desc) as rank_within_parent
+ from
+    store_sales
+   ,date_dim       d1
+   ,store
+ where
+    d1.d_month_seq between 1212 and 1212+11
+ and d1.d_date_sk = ss_sold_date_sk
+ and s_store_sk  = ss_store_sk
+ and s_state in
+             ( select s_state
+               from  (select s_state as s_state,
+ 			    rank() over ( partition by s_state order by sum(ss_net_profit) desc) as ranking
+                      from   store_sales, store, date_dim
+                      where  d_month_seq between 1212 and 1212+11
+ 			    and d_date_sk = ss_sold_date_sk
+ 			    and s_store_sk  = ss_store_sk
+                      group by s_state
+                     ) tmp1 
+               where ranking <= 5
+             )
+ group by rollup(s_state,s_county)
+ order by
+   lochierarchy desc
+  ,case when lochierarchy = 0 then s_state end
+  ,rank_within_parent
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query71.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query71.sql
new file mode 100644
index 0000000..09eb27a
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query71.sql
@@ -0,0 +1,52 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select i_brand_id brand_id, i_brand brand,t_hour,t_minute,
+ 	sum(ext_price) ext_price
+ from item, (select ws_ext_sales_price as ext_price, 
+                        ws_sold_date_sk as sold_date_sk,
+                        ws_item_sk as sold_item_sk,
+                        ws_sold_time_sk as time_sk  
+                 from web_sales,date_dim
+                 where d_date_sk = ws_sold_date_sk
+                   and d_moy=12
+                   and d_year=2000
+                 union all
+                 select cs_ext_sales_price as ext_price,
+                        cs_sold_date_sk as sold_date_sk,
+                        cs_item_sk as sold_item_sk,
+                        cs_sold_time_sk as time_sk
+                 from catalog_sales,date_dim
+                 where d_date_sk = cs_sold_date_sk
+                   and d_moy=12
+                   and d_year=2000
+                 union all
+                 select ss_ext_sales_price as ext_price,
+                        ss_sold_date_sk as sold_date_sk,
+                        ss_item_sk as sold_item_sk,
+                        ss_sold_time_sk as time_sk
+                 from store_sales,date_dim
+                 where d_date_sk = ss_sold_date_sk
+                   and d_moy=12
+                   and d_year=2000
+                 ) tmp,time_dim
+ where
+   sold_item_sk = i_item_sk
+   and i_manager_id=1
+   and time_sk = t_time_sk
+   and (t_meal_time = 'breakfast' or t_meal_time = 'dinner')
+ group by i_brand, i_brand_id,t_hour,t_minute
+ order by ext_price desc, i_brand_id
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query72.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query72.sql
new file mode 100644
index 0000000..a1173dc
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query72.sql
@@ -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.
+
+select  i_item_desc
+      ,w_warehouse_name
+      ,d1.d_week_seq
+      ,sum(case when p_promo_sk is null then 1 else 0 end) no_promo
+      ,sum(case when p_promo_sk is not null then 1 else 0 end) promo
+      ,count(*) total_cnt
+from catalog_sales
+join inventory on (cs_item_sk = inv_item_sk)
+join warehouse on (w_warehouse_sk=inv_warehouse_sk)
+join item on (i_item_sk = cs_item_sk)
+join customer_demographics on (cs_bill_cdemo_sk = cd_demo_sk)
+join household_demographics on (cs_bill_hdemo_sk = hd_demo_sk)
+join date_dim d1 on (cs_sold_date_sk = d1.d_date_sk)
+join date_dim d2 on (inv_date_sk = d2.d_date_sk)
+join date_dim d3 on (cs_ship_date_sk = d3.d_date_sk)
+left outer join promotion on (cs_promo_sk=p_promo_sk)
+left outer join catalog_returns on (cr_item_sk = cs_item_sk and cr_order_number = cs_order_number)
+where d1.d_week_seq = d2.d_week_seq
+  and inv_quantity_on_hand < cs_quantity 
+  and d3.d_date > d1.d_date + 5
+  and hd_buy_potential = '1001-5000'
+  and d1.d_year = 2001
+  and cd_marital_status = 'M'
+group by i_item_desc,w_warehouse_name,d1.d_week_seq
+order by total_cnt desc, i_item_desc, w_warehouse_name, d_week_seq
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query73.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query73.sql
new file mode 100644
index 0000000..dba1bfa
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query73.sql
@@ -0,0 +1,41 @@
+-- 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.
+
+select c_last_name
+       ,c_first_name
+       ,c_salutation
+       ,c_preferred_cust_flag 
+       ,ss_ticket_number
+       ,cnt from
+   (select ss_ticket_number
+          ,ss_customer_sk
+          ,count(*) cnt
+    from store_sales,date_dim,store,household_demographics
+    where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+    and store_sales.ss_store_sk = store.s_store_sk  
+    and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+    and date_dim.d_dom between 1 and 2 
+    and (household_demographics.hd_buy_potential = '>10000' or
+         household_demographics.hd_buy_potential = 'Unknown')
+    and household_demographics.hd_vehicle_count > 0
+    and case when household_demographics.hd_vehicle_count > 0 then 
+             household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count else null end > 1
+    and date_dim.d_year in (1998,1998+1,1998+2)
+    and store.s_county in ('Williamson County','Williamson County','Williamson County','Williamson County')
+    group by ss_ticket_number,ss_customer_sk) dj,customer
+    where ss_customer_sk = c_customer_sk
+      and cnt between 1 and 5
+    order by cnt desc, c_last_name asc
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query74.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query74.sql
new file mode 100644
index 0000000..384d1e7
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query74.sql
@@ -0,0 +1,74 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with year_total as (
+ select c_customer_id customer_id
+       ,c_first_name customer_first_name
+       ,c_last_name customer_last_name
+       ,d_year as year
+       ,max(ss_net_paid) year_total
+       ,'s' sale_type
+ from customer
+     ,store_sales
+     ,date_dim
+ where c_customer_sk = ss_customer_sk
+   and ss_sold_date_sk = d_date_sk
+   and d_year in (2001,2001+1)
+ group by c_customer_id
+         ,c_first_name
+         ,c_last_name
+         ,d_year
+ union all
+ select c_customer_id customer_id
+       ,c_first_name customer_first_name
+       ,c_last_name customer_last_name
+       ,d_year as year
+       ,max(ws_net_paid) year_total
+       ,'w' sale_type
+ from customer
+     ,web_sales
+     ,date_dim
+ where c_customer_sk = ws_bill_customer_sk
+   and ws_sold_date_sk = d_date_sk
+   and d_year in (2001,2001+1)
+ group by c_customer_id
+         ,c_first_name
+         ,c_last_name
+         ,d_year
+         )
+  select 
+        t_s_secyear.customer_id, t_s_secyear.customer_first_name, t_s_secyear.customer_last_name
+ from year_total t_s_firstyear
+     ,year_total t_s_secyear
+     ,year_total t_w_firstyear
+     ,year_total t_w_secyear
+ where t_s_secyear.customer_id = t_s_firstyear.customer_id
+         and t_s_firstyear.customer_id = t_w_secyear.customer_id
+         and t_s_firstyear.customer_id = t_w_firstyear.customer_id
+         and t_s_firstyear.sale_type = 's'
+         and t_w_firstyear.sale_type = 'w'
+         and t_s_secyear.sale_type = 's'
+         and t_w_secyear.sale_type = 'w'
+         and t_s_firstyear.year = 2001
+         and t_s_secyear.year = 2001+1
+         and t_w_firstyear.year = 2001
+         and t_w_secyear.year = 2001+1
+         and t_s_firstyear.year_total > 0
+         and t_w_firstyear.year_total > 0
+         and case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else null end
+           > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else null end
+ order by 2,1,3
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query75.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query75.sql
new file mode 100644
index 0000000..6d9c689
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query75.sql
@@ -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.
+
+WITH all_sales AS (
+ SELECT d_year
+       ,i_brand_id
+       ,i_class_id
+       ,i_category_id
+       ,i_manufact_id
+       ,SUM(sales_cnt) AS sales_cnt
+       ,SUM(sales_amt) AS sales_amt
+ FROM (SELECT d_year
+             ,i_brand_id
+             ,i_class_id
+             ,i_category_id
+             ,i_manufact_id
+             ,cs_quantity - COALESCE(cr_return_quantity,0) AS sales_cnt
+             ,cs_ext_sales_price - COALESCE(cr_return_amount,0.0) AS sales_amt
+       FROM catalog_sales JOIN item ON i_item_sk=cs_item_sk
+                          JOIN date_dim ON d_date_sk=cs_sold_date_sk
+                          LEFT JOIN catalog_returns ON (cs_order_number=cr_order_number 
+                                                    AND cs_item_sk=cr_item_sk)
+       WHERE i_category='Sports'
+       UNION
+       SELECT d_year
+             ,i_brand_id
+             ,i_class_id
+             ,i_category_id
+             ,i_manufact_id
+             ,ss_quantity - COALESCE(sr_return_quantity,0) AS sales_cnt
+             ,ss_ext_sales_price - COALESCE(sr_return_amt,0.0) AS sales_amt
+       FROM store_sales JOIN item ON i_item_sk=ss_item_sk
+                        JOIN date_dim ON d_date_sk=ss_sold_date_sk
+                        LEFT JOIN store_returns ON (ss_ticket_number=sr_ticket_number 
+                                                AND ss_item_sk=sr_item_sk)
+       WHERE i_category='Sports'
+       UNION
+       SELECT d_year
+             ,i_brand_id
+             ,i_class_id
+             ,i_category_id
+             ,i_manufact_id
+             ,ws_quantity - COALESCE(wr_return_quantity,0) AS sales_cnt
+             ,ws_ext_sales_price - COALESCE(wr_return_amt,0.0) AS sales_amt
+       FROM web_sales JOIN item ON i_item_sk=ws_item_sk
+                      JOIN date_dim ON d_date_sk=ws_sold_date_sk
+                      LEFT JOIN web_returns ON (ws_order_number=wr_order_number 
+                                            AND ws_item_sk=wr_item_sk)
+       WHERE i_category='Sports') sales_detail
+ GROUP BY d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id)
+ SELECT  prev_yr.d_year AS prev_year
+                          ,curr_yr.d_year AS year
+                          ,curr_yr.i_brand_id
+                          ,curr_yr.i_class_id
+                          ,curr_yr.i_category_id
+                          ,curr_yr.i_manufact_id
+                          ,prev_yr.sales_cnt AS prev_yr_cnt
+                          ,curr_yr.sales_cnt AS curr_yr_cnt
+                          ,curr_yr.sales_cnt-prev_yr.sales_cnt AS sales_cnt_diff
+                          ,curr_yr.sales_amt-prev_yr.sales_amt AS sales_amt_diff
+ FROM all_sales curr_yr, all_sales prev_yr
+ WHERE curr_yr.i_brand_id=prev_yr.i_brand_id
+   AND curr_yr.i_class_id=prev_yr.i_class_id
+   AND curr_yr.i_category_id=prev_yr.i_category_id
+   AND curr_yr.i_manufact_id=prev_yr.i_manufact_id
+   AND curr_yr.d_year=2002
+   AND prev_yr.d_year=2002-1
+   AND CAST(curr_yr.sales_cnt AS DECIMAL(17,2))/CAST(prev_yr.sales_cnt AS DECIMAL(17,2))<0.9
+ ORDER BY sales_cnt_diff,sales_amt_diff
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query76.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query76.sql
new file mode 100644
index 0000000..8a87b00
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query76.sql
@@ -0,0 +1,37 @@
+-- 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.
+
+select  channel, col_name, d_year, d_qoy, i_category, COUNT(*) sales_cnt, SUM(ext_sales_price) sales_amt FROM (
+        SELECT 'store' as channel, 'ss_addr_sk' col_name, d_year, d_qoy, i_category, ss_ext_sales_price ext_sales_price
+         FROM store_sales, item, date_dim
+         WHERE ss_addr_sk IS NULL
+           AND ss_sold_date_sk=d_date_sk
+           AND ss_item_sk=i_item_sk
+        UNION ALL
+        SELECT 'web' as channel, 'ws_web_page_sk' col_name, d_year, d_qoy, i_category, ws_ext_sales_price ext_sales_price
+         FROM web_sales, item, date_dim
+         WHERE ws_web_page_sk IS NULL
+           AND ws_sold_date_sk=d_date_sk
+           AND ws_item_sk=i_item_sk
+        UNION ALL
+        SELECT 'catalog' as channel, 'cs_warehouse_sk' col_name, d_year, d_qoy, i_category, cs_ext_sales_price ext_sales_price
+         FROM catalog_sales, item, date_dim
+         WHERE cs_warehouse_sk IS NULL
+           AND cs_sold_date_sk=d_date_sk
+           AND cs_item_sk=i_item_sk) foo
+GROUP BY channel, col_name, d_year, d_qoy, i_category
+ORDER BY channel, col_name, d_year, d_qoy, i_category
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query77.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query77.sql
new file mode 100644
index 0000000..7257175
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query77.sql
@@ -0,0 +1,121 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with ss as
+ (select s_store_sk,
+         sum(ss_ext_sales_price) as sales,
+         sum(ss_net_profit) as profit
+ from store_sales,
+      date_dim,
+      store
+ where ss_sold_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date) 
+                  and (cast('1998-08-04' as date) +  30 days) 
+       and ss_store_sk = s_store_sk
+ group by s_store_sk)
+ ,
+ sr as
+ (select s_store_sk,
+         sum(sr_return_amt) as returns,
+         sum(sr_net_loss) as profit_loss
+ from store_returns,
+      date_dim,
+      store
+ where sr_returned_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  30 days)
+       and sr_store_sk = s_store_sk
+ group by s_store_sk), 
+ cs as
+ (select cs_call_center_sk,
+        sum(cs_ext_sales_price) as sales,
+        sum(cs_net_profit) as profit
+ from catalog_sales,
+      date_dim
+ where cs_sold_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  30 days)
+ group by cs_call_center_sk 
+ ), 
+ cr as
+ (select cr_call_center_sk,
+         sum(cr_return_amount) as returns,
+         sum(cr_net_loss) as profit_loss
+ from catalog_returns,
+      date_dim
+ where cr_returned_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  30 days)
+ group by cr_call_center_sk
+ ), 
+ ws as
+ ( select wp_web_page_sk,
+        sum(ws_ext_sales_price) as sales,
+        sum(ws_net_profit) as profit
+ from web_sales,
+      date_dim,
+      web_page
+ where ws_sold_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  30 days)
+       and ws_web_page_sk = wp_web_page_sk
+ group by wp_web_page_sk), 
+ wr as
+ (select wp_web_page_sk,
+        sum(wr_return_amt) as returns,
+        sum(wr_net_loss) as profit_loss
+ from web_returns,
+      date_dim,
+      web_page
+ where wr_returned_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  30 days)
+       and wr_web_page_sk = wp_web_page_sk
+ group by wp_web_page_sk)
+  select  channel
+        , id
+        , sum(sales) as sales
+        , sum(returns) as returns
+        , sum(profit) as profit
+ from 
+ (select 'store channel' as channel
+        , ss.s_store_sk as id
+        , sales
+        , coalesce(returns, 0) as returns
+        , (profit - coalesce(profit_loss,0)) as profit
+ from   ss left join sr
+        on  ss.s_store_sk = sr.s_store_sk
+ union all
+ select 'catalog channel' as channel
+        , cs_call_center_sk as id
+        , sales
+        , returns
+        , (profit - profit_loss) as profit
+ from  cs
+       , cr
+ union all
+ select 'web channel' as channel
+        , ws.wp_web_page_sk as id
+        , sales
+        , coalesce(returns, 0) returns
+        , (profit - coalesce(profit_loss,0)) as profit
+ from   ws left join wr
+        on  ws.wp_web_page_sk = wr.wp_web_page_sk
+ ) x
+ group by rollup (channel, id)
+ order by channel
+         ,id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query78.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query78.sql
new file mode 100644
index 0000000..e7bec2e
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query78.sql
@@ -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.
+
+with ws as
+  (select d_year AS ws_sold_year, ws_item_sk,
+    ws_bill_customer_sk ws_customer_sk,
+    sum(ws_quantity) ws_qty,
+    sum(ws_wholesale_cost) ws_wc,
+    sum(ws_sales_price) ws_sp
+   from web_sales
+   left join web_returns on wr_order_number=ws_order_number and ws_item_sk=wr_item_sk
+   join date_dim on ws_sold_date_sk = d_date_sk
+   where wr_order_number is null
+   group by d_year, ws_item_sk, ws_bill_customer_sk
+   ),
+cs as
+  (select d_year AS cs_sold_year, cs_item_sk,
+    cs_bill_customer_sk cs_customer_sk,
+    sum(cs_quantity) cs_qty,
+    sum(cs_wholesale_cost) cs_wc,
+    sum(cs_sales_price) cs_sp
+   from catalog_sales
+   left join catalog_returns on cr_order_number=cs_order_number and cs_item_sk=cr_item_sk
+   join date_dim on cs_sold_date_sk = d_date_sk
+   where cr_order_number is null
+   group by d_year, cs_item_sk, cs_bill_customer_sk
+   ),
+ss as
+  (select d_year AS ss_sold_year, ss_item_sk,
+    ss_customer_sk,
+    sum(ss_quantity) ss_qty,
+    sum(ss_wholesale_cost) ss_wc,
+    sum(ss_sales_price) ss_sp
+   from store_sales
+   left join store_returns on sr_ticket_number=ss_ticket_number and ss_item_sk=sr_item_sk
+   join date_dim on ss_sold_date_sk = d_date_sk
+   where sr_ticket_number is null
+   group by d_year, ss_item_sk, ss_customer_sk
+   )
+ select 
+ss_sold_year, ss_item_sk, ss_customer_sk,
+round(ss_qty/(coalesce(ws_qty,0)+coalesce(cs_qty,0)),2) ratio,
+ss_qty store_qty, ss_wc store_wholesale_cost, ss_sp store_sales_price,
+coalesce(ws_qty,0)+coalesce(cs_qty,0) other_chan_qty,
+coalesce(ws_wc,0)+coalesce(cs_wc,0) other_chan_wholesale_cost,
+coalesce(ws_sp,0)+coalesce(cs_sp,0) other_chan_sales_price
+from ss
+left join ws on (ws_sold_year=ss_sold_year and ws_item_sk=ss_item_sk and ws_customer_sk=ss_customer_sk)
+left join cs on (cs_sold_year=ss_sold_year and cs_item_sk=ss_item_sk and cs_customer_sk=ss_customer_sk)
+where (coalesce(ws_qty,0)>0 or coalesce(cs_qty, 0)>0) and ss_sold_year=2000
+order by 
+  ss_sold_year, ss_item_sk, ss_customer_sk,
+  ss_qty desc, ss_wc desc, ss_sp desc,
+  other_chan_qty,
+  other_chan_wholesale_cost,
+  other_chan_sales_price,
+  ratio
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query79.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query79.sql
new file mode 100644
index 0000000..0728226
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query79.sql
@@ -0,0 +1,36 @@
+-- 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.
+
+select
+  c_last_name,c_first_name,substr(s_city,1,30),ss_ticket_number,amt,profit
+  from
+   (select ss_ticket_number
+          ,ss_customer_sk
+          ,store.s_city
+          ,sum(ss_coupon_amt) amt
+          ,sum(ss_net_profit) profit
+    from store_sales,date_dim,store,household_demographics
+    where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+    and store_sales.ss_store_sk = store.s_store_sk  
+    and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+    and (household_demographics.hd_dep_count = 8 or household_demographics.hd_vehicle_count > 0)
+    and date_dim.d_dow = 1
+    and date_dim.d_year in (1998,1998+1,1998+2) 
+    and store.s_number_employees between 200 and 295
+    group by ss_ticket_number,ss_customer_sk,ss_addr_sk,store.s_city) ms,customer
+    where ss_customer_sk = c_customer_sk
+ order by c_last_name,c_first_name,substr(s_city,1,30), profit
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query8.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query8.sql
new file mode 100644
index 0000000..250c118
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query8.sql
@@ -0,0 +1,121 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select  s_store_name
+      ,sum(ss_net_profit)
+ from store_sales
+     ,date_dim
+     ,store,
+     (select ca_zip
+     from (
+      SELECT substr(ca_zip,1,5) ca_zip
+      FROM customer_address
+      WHERE substr(ca_zip,1,5) IN (
+                          '89436','30868','65085','22977','83927','77557',
+                          '58429','40697','80614','10502','32779',
+                          '91137','61265','98294','17921','18427',
+                          '21203','59362','87291','84093','21505',
+                          '17184','10866','67898','25797','28055',
+                          '18377','80332','74535','21757','29742',
+                          '90885','29898','17819','40811','25990',
+                          '47513','89531','91068','10391','18846',
+                          '99223','82637','41368','83658','86199',
+                          '81625','26696','89338','88425','32200',
+                          '81427','19053','77471','36610','99823',
+                          '43276','41249','48584','83550','82276',
+                          '18842','78890','14090','38123','40936',
+                          '34425','19850','43286','80072','79188',
+                          '54191','11395','50497','84861','90733',
+                          '21068','57666','37119','25004','57835',
+                          '70067','62878','95806','19303','18840',
+                          '19124','29785','16737','16022','49613',
+                          '89977','68310','60069','98360','48649',
+                          '39050','41793','25002','27413','39736',
+                          '47208','16515','94808','57648','15009',
+                          '80015','42961','63982','21744','71853',
+                          '81087','67468','34175','64008','20261',
+                          '11201','51799','48043','45645','61163',
+                          '48375','36447','57042','21218','41100',
+                          '89951','22745','35851','83326','61125',
+                          '78298','80752','49858','52940','96976',
+                          '63792','11376','53582','18717','90226',
+                          '50530','94203','99447','27670','96577',
+                          '57856','56372','16165','23427','54561',
+                          '28806','44439','22926','30123','61451',
+                          '92397','56979','92309','70873','13355',
+                          '21801','46346','37562','56458','28286',
+                          '47306','99555','69399','26234','47546',
+                          '49661','88601','35943','39936','25632',
+                          '24611','44166','56648','30379','59785',
+                          '11110','14329','93815','52226','71381',
+                          '13842','25612','63294','14664','21077',
+                          '82626','18799','60915','81020','56447',
+                          '76619','11433','13414','42548','92713',
+                          '70467','30884','47484','16072','38936',
+                          '13036','88376','45539','35901','19506',
+                          '65690','73957','71850','49231','14276',
+                          '20005','18384','76615','11635','38177',
+                          '55607','41369','95447','58581','58149',
+                          '91946','33790','76232','75692','95464',
+                          '22246','51061','56692','53121','77209',
+                          '15482','10688','14868','45907','73520',
+                          '72666','25734','17959','24677','66446',
+                          '94627','53535','15560','41967','69297',
+                          '11929','59403','33283','52232','57350',
+                          '43933','40921','36635','10827','71286',
+                          '19736','80619','25251','95042','15526',
+                          '36496','55854','49124','81980','35375',
+                          '49157','63512','28944','14946','36503',
+                          '54010','18767','23969','43905','66979',
+                          '33113','21286','58471','59080','13395',
+                          '79144','70373','67031','38360','26705',
+                          '50906','52406','26066','73146','15884',
+                          '31897','30045','61068','45550','92454',
+                          '13376','14354','19770','22928','97790',
+                          '50723','46081','30202','14410','20223',
+                          '88500','67298','13261','14172','81410',
+                          '93578','83583','46047','94167','82564',
+                          '21156','15799','86709','37931','74703',
+                          '83103','23054','70470','72008','49247',
+                          '91911','69998','20961','70070','63197',
+                          '54853','88191','91830','49521','19454',
+                          '81450','89091','62378','25683','61869',
+                          '51744','36580','85778','36871','48121',
+                          '28810','83712','45486','67393','26935',
+                          '42393','20132','55349','86057','21309',
+                          '80218','10094','11357','48819','39734',
+                          '40758','30432','21204','29467','30214',
+                          '61024','55307','74621','11622','68908',
+                          '33032','52868','99194','99900','84936',
+                          '69036','99149','45013','32895','59004',
+                          '32322','14933','32936','33562','72550',
+                          '27385','58049','58200','16808','21360',
+                          '32961','18586','79307','15492')
+     intersect
+      select ca_zip
+      from (SELECT substr(ca_zip,1,5) ca_zip,count(*) cnt
+            FROM customer_address, customer
+            WHERE ca_address_sk = c_current_addr_sk and
+                  c_preferred_cust_flag='Y'
+            group by ca_zip
+            having count(*) > 10)A1)A2) V1
+ where ss_store_sk = s_store_sk
+  and ss_sold_date_sk = d_date_sk
+  and d_qoy = 1 and d_year = 2002
+  and (substr(s_zip,1,2) = substr(V1.ca_zip,1,2))
+ group by s_store_name
+ order by s_store_name
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query80.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query80.sql
new file mode 100644
index 0000000..9c6e177
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query80.sql
@@ -0,0 +1,109 @@
+-- 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.
+
+with ssr as
+ (select  s_store_id as store_id,
+          sum(ss_ext_sales_price) as sales,
+          sum(coalesce(sr_return_amt, 0)) as returns,
+          sum(ss_net_profit - coalesce(sr_net_loss, 0)) as profit
+  from store_sales left outer join store_returns on
+         (ss_item_sk = sr_item_sk and ss_ticket_number = sr_ticket_number),
+     date_dim,
+     store,
+     item,
+     promotion
+ where ss_sold_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date) 
+                  and (cast('1998-08-04' as date) +  30 days)
+       and ss_store_sk = s_store_sk
+       and ss_item_sk = i_item_sk
+       and i_current_price > 50
+       and ss_promo_sk = p_promo_sk
+       and p_channel_tv = 'N'
+ group by s_store_id)
+ ,
+ csr as
+ (select  cp_catalog_page_id as catalog_page_id,
+          sum(cs_ext_sales_price) as sales,
+          sum(coalesce(cr_return_amount, 0)) as returns,
+          sum(cs_net_profit - coalesce(cr_net_loss, 0)) as profit
+  from catalog_sales left outer join catalog_returns on
+         (cs_item_sk = cr_item_sk and cs_order_number = cr_order_number),
+     date_dim,
+     catalog_page,
+     item,
+     promotion
+ where cs_sold_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  30 days)
+        and cs_catalog_page_sk = cp_catalog_page_sk
+       and cs_item_sk = i_item_sk
+       and i_current_price > 50
+       and cs_promo_sk = p_promo_sk
+       and p_channel_tv = 'N'
+group by cp_catalog_page_id)
+ ,
+ wsr as
+ (select  web_site_id,
+          sum(ws_ext_sales_price) as sales,
+          sum(coalesce(wr_return_amt, 0)) as returns,
+          sum(ws_net_profit - coalesce(wr_net_loss, 0)) as profit
+  from web_sales left outer join web_returns on
+         (ws_item_sk = wr_item_sk and ws_order_number = wr_order_number),
+     date_dim,
+     web_site,
+     item,
+     promotion
+ where ws_sold_date_sk = d_date_sk
+       and d_date between cast('1998-08-04' as date)
+                  and (cast('1998-08-04' as date) +  30 days)
+        and ws_web_site_sk = web_site_sk
+       and ws_item_sk = i_item_sk
+       and i_current_price > 50
+       and ws_promo_sk = p_promo_sk
+       and p_channel_tv = 'N'
+group by web_site_id)
+  select  channel
+        , id
+        , sum(sales) as sales
+        , sum(returns) as returns
+        , sum(profit) as profit
+ from 
+ (select 'store channel' as channel
+        , 'store' || store_id as id
+        , sales
+        , returns
+        , profit
+ from   ssr
+ union all
+ select 'catalog channel' as channel
+        , 'catalog_page' || catalog_page_id as id
+        , sales
+        , returns
+        , profit
+ from  csr
+ union all
+ select 'web channel' as channel
+        , 'web_site' || web_site_id as id
+        , sales
+        , returns
+        , profit
+ from   wsr
+ ) x
+ group by rollup (channel, id)
+ order by channel
+         ,id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query81.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query81.sql
new file mode 100644
index 0000000..71c49dc
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query81.sql
@@ -0,0 +1,44 @@
+-- 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.
+
+with customer_total_return as
+ (select cr_returning_customer_sk as ctr_customer_sk
+        ,ca_state as ctr_state, 
+ 	sum(cr_return_amt_inc_tax) as ctr_total_return
+ from catalog_returns
+     ,date_dim
+     ,customer_address
+ where cr_returned_date_sk = d_date_sk 
+   and d_year =1998
+   and cr_returning_addr_sk = ca_address_sk 
+ group by cr_returning_customer_sk
+         ,ca_state )
+  select  c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name
+                   ,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset
+                  ,ca_location_type,ctr_total_return
+ from customer_total_return ctr1
+     ,customer_address
+     ,customer
+ where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2
+ 			  from customer_total_return ctr2 
+                  	  where ctr1.ctr_state = ctr2.ctr_state)
+       and ca_address_sk = c_current_addr_sk
+       and ca_state = 'IL'
+       and ctr1.ctr_customer_sk = c_customer_sk
+ order by c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name
+                   ,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset
+                  ,ca_location_type,ctr_total_return
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query82.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query82.sql
new file mode 100644
index 0000000..f08cc17
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query82.sql
@@ -0,0 +1,30 @@
+-- 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.
+
+select  i_item_id
+       ,i_item_desc
+       ,i_current_price
+ from item, inventory, date_dim, store_sales
+ where i_current_price between 30 and 30+30
+ and inv_item_sk = i_item_sk
+ and d_date_sk=inv_date_sk
+ and d_date between cast('2002-05-30' as date) and (cast('2002-05-30' as date) +  60 days)
+ and i_manufact_id in (437,129,727,663)
+ and inv_quantity_on_hand between 100 and 500
+ and ss_item_sk = i_item_sk
+ group by i_item_id,i_item_desc,i_current_price
+ order by i_item_id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query83.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query83.sql
new file mode 100644
index 0000000..bc22723
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query83.sql
@@ -0,0 +1,80 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with sr_items as
+ (select i_item_id item_id,
+        sum(sr_return_quantity) sr_item_qty
+ from store_returns,
+      item,
+      date_dim
+ where sr_item_sk = i_item_sk
+ and   d_date    in 
+	(select d_date
+	from date_dim
+	where d_week_seq in 
+		(select d_week_seq
+		from date_dim
+	  where d_date in ('1998-01-02','1998-10-15','1998-11-10')))
+ and   sr_returned_date_sk   = d_date_sk
+ group by i_item_id),
+ cr_items as
+ (select i_item_id item_id,
+        sum(cr_return_quantity) cr_item_qty
+ from catalog_returns,
+      item,
+      date_dim
+ where cr_item_sk = i_item_sk
+ and   d_date    in 
+	(select d_date
+	from date_dim
+	where d_week_seq in 
+		(select d_week_seq
+		from date_dim
+	  where d_date in ('1998-01-02','1998-10-15','1998-11-10')))
+ and   cr_returned_date_sk   = d_date_sk
+ group by i_item_id),
+ wr_items as
+ (select i_item_id item_id,
+        sum(wr_return_quantity) wr_item_qty
+ from web_returns,
+      item,
+      date_dim
+ where wr_item_sk = i_item_sk
+ and   d_date    in 
+	(select d_date
+	from date_dim
+	where d_week_seq in 
+		(select d_week_seq
+		from date_dim
+		where d_date in ('1998-01-02','1998-10-15','1998-11-10')))
+ and   wr_returned_date_sk   = d_date_sk
+ group by i_item_id)
+  select  sr_items.item_id
+       ,sr_item_qty
+       ,sr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 sr_dev
+       ,cr_item_qty
+       ,cr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 cr_dev
+       ,wr_item_qty
+       ,wr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 wr_dev
+       ,(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 average
+ from sr_items
+     ,cr_items
+     ,wr_items
+ where sr_items.item_id=cr_items.item_id
+   and sr_items.item_id=wr_items.item_id 
+ order by sr_items.item_id
+         ,sr_item_qty
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query84.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query84.sql
new file mode 100644
index 0000000..2a519de
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query84.sql
@@ -0,0 +1,34 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select  c_customer_id as customer_id
+       , coalesce(c_last_name,'') || ', ' || coalesce(c_first_name,'') as customername
+ from customer
+     ,customer_address
+     ,customer_demographics
+     ,household_demographics
+     ,income_band
+     ,store_returns
+ where ca_city	        =  'Hopewell'
+   and c_current_addr_sk = ca_address_sk
+   and ib_lower_bound   >=  32287
+   and ib_upper_bound   <=  32287 + 50000
+   and ib_income_band_sk = hd_income_band_sk
+   and cd_demo_sk = c_current_cdemo_sk
+   and hd_demo_sk = c_current_hdemo_sk
+   and sr_cdemo_sk = cd_demo_sk
+ order by c_customer_id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query85.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query85.sql
new file mode 100644
index 0000000..dea9927
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query85.sql
@@ -0,0 +1,97 @@
+-- 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.
+
+select  substr(r_reason_desc,1,20)
+       ,avg(ws_quantity)
+       ,avg(wr_refunded_cash)
+       ,avg(wr_fee)
+ from web_sales, web_returns, web_page, customer_demographics cd1,
+      customer_demographics cd2, customer_address, date_dim, reason 
+ where ws_web_page_sk = wp_web_page_sk
+   and ws_item_sk = wr_item_sk
+   and ws_order_number = wr_order_number
+   and ws_sold_date_sk = d_date_sk and d_year = 1998
+   and cd1.cd_demo_sk = wr_refunded_cdemo_sk 
+   and cd2.cd_demo_sk = wr_returning_cdemo_sk
+   and ca_address_sk = wr_refunded_addr_sk
+   and r_reason_sk = wr_reason_sk
+   and
+   (
+    (
+     cd1.cd_marital_status = 'M'
+     and
+     cd1.cd_marital_status = cd2.cd_marital_status
+     and
+     cd1.cd_education_status = '4 yr Degree'
+     and 
+     cd1.cd_education_status = cd2.cd_education_status
+     and
+     ws_sales_price between 100.00 and 150.00
+    )
+   or
+    (
+     cd1.cd_marital_status = 'D'
+     and
+     cd1.cd_marital_status = cd2.cd_marital_status
+     and
+     cd1.cd_education_status = 'Primary' 
+     and
+     cd1.cd_education_status = cd2.cd_education_status
+     and
+     ws_sales_price between 50.00 and 100.00
+    )
+   or
+    (
+     cd1.cd_marital_status = 'U'
+     and
+     cd1.cd_marital_status = cd2.cd_marital_status
+     and
+     cd1.cd_education_status = 'Advanced Degree'
+     and
+     cd1.cd_education_status = cd2.cd_education_status
+     and
+     ws_sales_price between 150.00 and 200.00
+    )
+   )
+   and
+   (
+    (
+     ca_country = 'United States'
+     and
+     ca_state in ('KY', 'GA', 'NM')
+     and ws_net_profit between 100 and 200  
+    )
+    or
+    (
+     ca_country = 'United States'
+     and
+     ca_state in ('MT', 'OR', 'IN')
+     and ws_net_profit between 150 and 300  
+    )
+    or
+    (
+     ca_country = 'United States'
+     and
+     ca_state in ('WI', 'MO', 'WV')
+     and ws_net_profit between 50 and 250  
+    )
+   )
+group by r_reason_desc
+order by substr(r_reason_desc,1,20)
+        ,avg(ws_quantity)
+        ,avg(wr_refunded_cash)
+        ,avg(wr_fee)
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query86.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query86.sql
new file mode 100644
index 0000000..353732b
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query86.sql
@@ -0,0 +1,39 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+    sum(ws_net_paid) as total_sum
+   ,i_category
+   ,i_class
+   ,grouping(i_category)+grouping(i_class) as lochierarchy
+   ,rank() over (
+ 	partition by grouping(i_category)+grouping(i_class),
+ 	case when grouping(i_class) = 0 then i_category end 
+ 	order by sum(ws_net_paid) desc) as rank_within_parent
+ from
+    web_sales
+   ,date_dim       d1
+   ,item
+ where
+    d1.d_month_seq between 1212 and 1212+11
+ and d1.d_date_sk = ws_sold_date_sk
+ and i_item_sk  = ws_item_sk
+ group by rollup(i_category,i_class)
+ order by
+   lochierarchy desc,
+   case when lochierarchy = 0 then i_category end,
+   rank_within_parent
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query87.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query87.sql
new file mode 100644
index 0000000..12b0384
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query87.sql
@@ -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.
+
+select count(*)
+from ((select distinct c_last_name, c_first_name, d_date
+       from store_sales, date_dim, customer
+       where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+         and store_sales.ss_customer_sk = customer.c_customer_sk
+         and d_month_seq between 1212 and 1212+11)
+       except
+      (select distinct c_last_name, c_first_name, d_date
+       from catalog_sales, date_dim, customer
+       where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk
+         and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk
+         and d_month_seq between 1212 and 1212+11)
+       except
+      (select distinct c_last_name, c_first_name, d_date
+       from web_sales, date_dim, customer
+       where web_sales.ws_sold_date_sk = date_dim.d_date_sk
+         and web_sales.ws_bill_customer_sk = customer.c_customer_sk
+         and d_month_seq between 1212 and 1212+11)
+) cool_cust
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query88.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query88.sql
new file mode 100644
index 0000000..8ca2616
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query88.sql
@@ -0,0 +1,106 @@
+-- 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.
+
+select  *
+from
+ (select count(*) h8_30_to_9
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk   
+     and ss_hdemo_sk = household_demographics.hd_demo_sk 
+     and ss_store_sk = s_store_sk
+     and time_dim.t_hour = 8
+     and time_dim.t_minute >= 30
+     and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
+          (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+          (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) 
+     and store.s_store_name = 'ese') s1,
+ (select count(*) h9_to_9_30 
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk 
+     and time_dim.t_hour = 9 
+     and time_dim.t_minute < 30
+     and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
+          (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+          (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2))
+     and store.s_store_name = 'ese') s2,
+ (select count(*) h9_30_to_10 
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
+     and time_dim.t_hour = 9
+     and time_dim.t_minute >= 30
+     and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
+          (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+          (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2))
+     and store.s_store_name = 'ese') s3,
+ (select count(*) h10_to_10_30
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
+     and time_dim.t_hour = 10 
+     and time_dim.t_minute < 30
+     and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
+          (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+          (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2))
+     and store.s_store_name = 'ese') s4,
+ (select count(*) h10_30_to_11
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
+     and time_dim.t_hour = 10 
+     and time_dim.t_minute >= 30
+     and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
+          (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+          (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2))
+     and store.s_store_name = 'ese') s5,
+ (select count(*) h11_to_11_30
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk 
+     and time_dim.t_hour = 11
+     and time_dim.t_minute < 30
+     and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
+          (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+          (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2))
+     and store.s_store_name = 'ese') s6,
+ (select count(*) h11_30_to_12
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
+     and time_dim.t_hour = 11
+     and time_dim.t_minute >= 30
+     and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
+          (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+          (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2))
+     and store.s_store_name = 'ese') s7,
+ (select count(*) h12_to_12_30
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
+     and time_dim.t_hour = 12
+     and time_dim.t_minute < 30
+     and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
+          (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+          (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2))
+     and store.s_store_name = 'ese') s8
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query89.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query89.sql
new file mode 100644
index 0000000..217a22a
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query89.sql
@@ -0,0 +1,41 @@
+-- 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.
+
+select  *
+from(
+select i_category, i_class, i_brand,
+       s_store_name, s_company_name,
+       d_moy,
+       sum(ss_sales_price) sum_sales,
+       avg(sum(ss_sales_price)) over
+         (partition by i_category, i_brand, s_store_name, s_company_name)
+         avg_monthly_sales
+from item, store_sales, date_dim, store
+where ss_item_sk = i_item_sk and
+      ss_sold_date_sk = d_date_sk and
+      ss_store_sk = s_store_sk and
+      d_year in (2000) and
+        ((i_category in ('Home','Books','Electronics') and
+          i_class in ('wallpaper','parenting','musical')
+         )
+      or (i_category in ('Shoes','Jewelry','Men') and
+          i_class in ('womens','birdal','pants') 
+        ))
+group by i_category, i_class, i_brand,
+         s_store_name, s_company_name, d_moy) tmp1
+where case when (avg_monthly_sales <> 0) then (abs(sum_sales - avg_monthly_sales) / avg_monthly_sales) else null end > 0.1
+order by sum_sales - avg_monthly_sales, s_store_name
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query9.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query9.sql
new file mode 100644
index 0000000..ab72251
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query9.sql
@@ -0,0 +1,63 @@
+-- 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.
+
+select case when (select count(*)
+                  from store_sales 
+                  where ss_quantity between 1 and 20) > 25437
+            then (select avg(ss_ext_discount_amt) 
+                  from store_sales 
+                  where ss_quantity between 1 and 20) 
+            else (select avg(ss_net_profit)
+                  from store_sales
+                  where ss_quantity between 1 and 20) end bucket1 ,
+       case when (select count(*)
+                  from store_sales
+                  where ss_quantity between 21 and 40) > 22746
+            then (select avg(ss_ext_discount_amt)
+                  from store_sales
+                  where ss_quantity between 21 and 40) 
+            else (select avg(ss_net_profit)
+                  from store_sales
+                  where ss_quantity between 21 and 40) end bucket2,
+       case when (select count(*)
+                  from store_sales
+                  where ss_quantity between 41 and 60) > 9387
+            then (select avg(ss_ext_discount_amt)
+                  from store_sales
+                  where ss_quantity between 41 and 60)
+            else (select avg(ss_net_profit)
+                  from store_sales
+                  where ss_quantity between 41 and 60) end bucket3,
+       case when (select count(*)
+                  from store_sales
+                  where ss_quantity between 61 and 80) > 10098
+            then (select avg(ss_ext_discount_amt)
+                  from store_sales
+                  where ss_quantity between 61 and 80)
+            else (select avg(ss_net_profit)
+                  from store_sales
+                  where ss_quantity between 61 and 80) end bucket4,
+       case when (select count(*)
+                  from store_sales
+                  where ss_quantity between 81 and 100) > 18213
+            then (select avg(ss_ext_discount_amt)
+                  from store_sales
+                  where ss_quantity between 81 and 100)
+            else (select avg(ss_net_profit)
+                  from store_sales
+                  where ss_quantity between 81 and 100) end bucket5
+from reason
+where r_reason_sk = 1
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query90.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query90.sql
new file mode 100644
index 0000000..2dfa02a
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query90.sql
@@ -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.
+
+select  cast(amc as decimal(15,4))/cast(pmc as decimal(15,4)) am_pm_ratio
+ from ( select count(*) amc
+       from web_sales, household_demographics , time_dim, web_page
+       where ws_sold_time_sk = time_dim.t_time_sk
+         and ws_ship_hdemo_sk = household_demographics.hd_demo_sk
+         and ws_web_page_sk = web_page.wp_web_page_sk
+         and time_dim.t_hour between 6 and 6+1
+         and household_demographics.hd_dep_count = 8
+         and web_page.wp_char_count between 5000 and 5200) at,
+      ( select count(*) pmc
+       from web_sales, household_demographics , time_dim, web_page
+       where ws_sold_time_sk = time_dim.t_time_sk
+         and ws_ship_hdemo_sk = household_demographics.hd_demo_sk
+         and ws_web_page_sk = web_page.wp_web_page_sk
+         and time_dim.t_hour between 14 and 14+1
+         and household_demographics.hd_dep_count = 8
+         and web_page.wp_char_count between 5000 and 5200) pt
+ order by am_pm_ratio
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query91.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query91.sql
new file mode 100644
index 0000000..49d5fda
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query91.sql
@@ -0,0 +1,44 @@
+-- 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.
+
+select
+        cc_call_center_id Call_Center,
+        cc_name Call_Center_Name,
+        cc_manager Manager,
+        sum(cr_net_loss) Returns_Loss
+from
+        call_center,
+        catalog_returns,
+        date_dim,
+        customer,
+        customer_address,
+        customer_demographics,
+        household_demographics
+where
+        cr_call_center_sk       = cc_call_center_sk
+and     cr_returned_date_sk     = d_date_sk
+and     cr_returning_customer_sk= c_customer_sk
+and     cd_demo_sk              = c_current_cdemo_sk
+and     hd_demo_sk              = c_current_hdemo_sk
+and     ca_address_sk           = c_current_addr_sk
+and     d_year                  = 1999 
+and     d_moy                   = 11
+and     ( (cd_marital_status       = 'M' and cd_education_status     = 'Unknown')
+        or(cd_marital_status       = 'W' and cd_education_status     = 'Advanced Degree'))
+and     hd_buy_potential like '0-500%'
+and     ca_gmt_offset           = -7
+group by cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status
+order by sum(cr_net_loss) desc
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query92.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query92.sql
new file mode 100644
index 0000000..a7ce3a3
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query92.sql
@@ -0,0 +1,43 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+   sum(ws_ext_discount_amt)  as "Excess Discount Amount" 
+from 
+    web_sales 
+   ,item 
+   ,date_dim
+where
+i_manufact_id = 269
+and i_item_sk = ws_item_sk 
+and d_date between '1998-03-18' and 
+        (cast('1998-03-18' as date) + 90 days)
+and d_date_sk = ws_sold_date_sk 
+and ws_ext_discount_amt  
+     > ( 
+         SELECT 
+            1.3 * avg(ws_ext_discount_amt) 
+         FROM 
+            web_sales 
+           ,date_dim
+         WHERE 
+              ws_item_sk = i_item_sk 
+          and d_date between '1998-03-18' and
+                             (cast('1998-03-18' as date) + 90 days)
+          and d_date_sk = ws_sold_date_sk 
+      ) 
+order by sum(ws_ext_discount_amt)
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query93.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query93.sql
new file mode 100644
index 0000000..20aa0a8
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query93.sql
@@ -0,0 +1,31 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select  ss_customer_sk
+            ,sum(act_sales) sumsales
+      from (select ss_item_sk
+                  ,ss_ticket_number
+                  ,ss_customer_sk
+                  ,case when sr_return_quantity is not null then (ss_quantity-sr_return_quantity)*ss_sales_price
+                                                            else (ss_quantity*ss_sales_price) end act_sales
+            from store_sales left outer join store_returns on (sr_item_sk = ss_item_sk
+                                                               and sr_ticket_number = ss_ticket_number)
+                ,reason
+            where sr_reason_sk = r_reason_sk
+              and r_reason_desc = 'Did not like the warranty') t
+      group by ss_customer_sk
+      order by sumsales, ss_customer_sk
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query94.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query94.sql
new file mode 100644
index 0000000..dab63be
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query94.sql
@@ -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.
+
+select
+   count(distinct ws_order_number) as "order count"
+  ,sum(ws_ext_ship_cost) as "total shipping cost"
+  ,sum(ws_net_profit) as "total net profit"
+from
+   web_sales ws1
+  ,date_dim
+  ,customer_address
+  ,web_site
+where
+    d_date between '1999-5-01' and 
+           (cast('1999-5-01' as date) + 60 days)
+and ws1.ws_ship_date_sk = d_date_sk
+and ws1.ws_ship_addr_sk = ca_address_sk
+and ca_state = 'TX'
+and ws1.ws_web_site_sk = web_site_sk
+and web_company_name = 'pri'
+and exists (select *
+            from web_sales ws2
+            where ws1.ws_order_number = ws2.ws_order_number
+              and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk)
+and not exists(select *
+               from web_returns wr1
+               where ws1.ws_order_number = wr1.wr_order_number)
+order by count(distinct ws_order_number)
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query95.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query95.sql
new file mode 100644
index 0000000..b082826
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query95.sql
@@ -0,0 +1,45 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with ws_wh as
+(select ws1.ws_order_number,ws1.ws_warehouse_sk wh1,ws2.ws_warehouse_sk wh2
+ from web_sales ws1,web_sales ws2
+ where ws1.ws_order_number = ws2.ws_order_number
+   and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk)
+ select  
+   count(distinct ws_order_number) as "order count"
+  ,sum(ws_ext_ship_cost) as "total shipping cost"
+  ,sum(ws_net_profit) as "total net profit"
+from
+   web_sales ws1
+  ,date_dim
+  ,customer_address
+  ,web_site
+where
+    d_date between '1999-5-01' and 
+           (cast('1999-5-01' as date) + 60 days)
+and ws1.ws_ship_date_sk = d_date_sk
+and ws1.ws_ship_addr_sk = ca_address_sk
+and ca_state = 'TX'
+and ws1.ws_web_site_sk = web_site_sk
+and web_company_name = 'pri'
+and ws1.ws_order_number in (select ws_order_number
+                            from ws_wh)
+and ws1.ws_order_number in (select wr_order_number
+                            from web_returns,ws_wh
+                            where wr_order_number = ws_wh.ws_order_number)
+order by count(distinct ws_order_number)
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query96.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query96.sql
new file mode 100644
index 0000000..97cf08b
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query96.sql
@@ -0,0 +1,29 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select  count(*)
+from store_sales
+    ,household_demographics 
+    ,time_dim, store
+where ss_sold_time_sk = time_dim.t_time_sk   
+    and ss_hdemo_sk = household_demographics.hd_demo_sk 
+    and ss_store_sk = s_store_sk
+    and time_dim.t_hour = 8
+    and time_dim.t_minute >= 30
+    and household_demographics.hd_dep_count = 5
+    and store.s_store_name = 'ese'
+order by count(*)
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query97.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query97.sql
new file mode 100644
index 0000000..c2d51a7
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query97.sql
@@ -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.
+
+with ssci as (
+select ss_customer_sk customer_sk
+      ,ss_item_sk item_sk
+from store_sales,date_dim
+where ss_sold_date_sk = d_date_sk
+  and d_month_seq between 1212 and 1212 + 11
+group by ss_customer_sk
+        ,ss_item_sk),
+csci as(
+ select cs_bill_customer_sk customer_sk
+      ,cs_item_sk item_sk
+from catalog_sales,date_dim
+where cs_sold_date_sk = d_date_sk
+  and d_month_seq between 1212 and 1212 + 11
+group by cs_bill_customer_sk
+        ,cs_item_sk)
+ select  sum(case when ssci.customer_sk is not null and csci.customer_sk is null then 1 else 0 end) store_only
+      ,sum(case when ssci.customer_sk is null and csci.customer_sk is not null then 1 else 0 end) catalog_only
+      ,sum(case when ssci.customer_sk is not null and csci.customer_sk is not null then 1 else 0 end) store_and_catalog
+from ssci full outer join csci on (ssci.customer_sk=csci.customer_sk
+                               and ssci.item_sk = csci.item_sk)
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query98.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query98.sql
new file mode 100644
index 0000000..29d5757
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query98.sql
@@ -0,0 +1,46 @@
+-- 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.
+
+select i_item_id
+      ,i_item_desc 
+      ,i_category 
+      ,i_class 
+      ,i_current_price
+      ,sum(ss_ext_sales_price) as itemrevenue 
+      ,sum(ss_ext_sales_price)*100/sum(sum(ss_ext_sales_price)) over
+          (partition by i_class) as revenueratio
+from	
+	store_sales
+    	,item 
+    	,date_dim
+where 
+	ss_item_sk = i_item_sk 
+  	and i_category in ('Jewelry', 'Sports', 'Books')
+  	and ss_sold_date_sk = d_date_sk
+	and d_date between cast('2001-01-12' as date) 
+				and (cast('2001-01-12' as date) + 30 days)
+group by 
+	i_item_id
+        ,i_item_desc 
+        ,i_category
+        ,i_class
+        ,i_current_price
+order by 
+	i_category
+        ,i_class
+        ,i_item_id
+        ,i_item_desc
+        ,revenueratio
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query99.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query99.sql
new file mode 100644
index 0000000..de8e8ca
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query99.sql
@@ -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.
+
+select
+   substr(w_warehouse_name,1,20)
+  ,sm_type
+  ,cc_name
+  ,sum(case when (cs_ship_date_sk - cs_sold_date_sk <= 30 ) then 1 else 0 end)  as "30 days" 
+  ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 30) and 
+                 (cs_ship_date_sk - cs_sold_date_sk <= 60) then 1 else 0 end )  as "31-60 days" 
+  ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 60) and 
+                 (cs_ship_date_sk - cs_sold_date_sk <= 90) then 1 else 0 end)  as "61-90 days" 
+  ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 90) and
+                 (cs_ship_date_sk - cs_sold_date_sk <= 120) then 1 else 0 end)  as "91-120 days" 
+  ,sum(case when (cs_ship_date_sk - cs_sold_date_sk  > 120) then 1 else 0 end)  as ">120 days" 
+from
+   catalog_sales
+  ,warehouse
+  ,ship_mode
+  ,call_center
+  ,date_dim
+where
+    d_month_seq between 1212 and 1212 + 11
+and cs_ship_date_sk   = d_date_sk
+and cs_warehouse_sk   = w_warehouse_sk
+and cs_ship_mode_sk   = sm_ship_mode_sk
+and cs_call_center_sk = cc_call_center_sk
+group by
+   substr(w_warehouse_name,1,20)
+  ,sm_type
+  ,cc_name
+order by substr(w_warehouse_name,1,20)
+        ,sm_type
+        ,cc_name
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/call_center.json b/sdks/java/testing/tpcds/src/main/resources/schemas/call_center.json
new file mode 100644
index 0000000..ec95095
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/call_center.json
@@ -0,0 +1,33 @@
+{"schema": [
+  {"name":"cc_call_center_sk","type":"identifier"},
+  {"name":"cc_call_center_id","type":"char(16)"},
+  {"name":"cc_rec_start_date","type":"date"},
+  {"name":"cc_rec_end_date","type":"date"},
+  {"name":"cc_closed_date_sk","type":"integer"},
+  {"name":"cc_open_date_sk","type":"integer"},
+  {"name":"cc_name","type":"varchar(50)"},
+  {"name":"cc_class","type":"varchar(50)"},
+  {"name":"cc_employees","type":"integer"},
+  {"name":"cc_sq_ft","type":"integer"},
+  {"name":"cc_hours","type":"char(20)"},
+  {"name":"cc_manager","type":"varchar(40)"},
+  {"name":"cc_mkt_id","type":"integer"},
+  {"name":"cc_mkt_class","type":"char(50)"},
+  {"name":"cc_mkt_desc","type":"varchar(100)"},
+  {"name":"cc_market_manager","type":"varchar(40)"},
+  {"name":"cc_division","type":"integer"},
+  {"name":"cc_division_name","type":"varchar(50)"},
+  {"name":"cc_company","type":"integer"},
+  {"name":"cc_company_name","type":"char(50)"},
+  {"name":"cc_street_number","type":"char(10)"},
+  {"name":"cc_street_name","type":"varchar(60)"},
+  {"name":"cc_street_type","type":"char(15)"},
+  {"name":"cc_suite_number","type":"char(10)"},
+  {"name":"cc_city","type":"varchar(60)"},
+  {"name":"cc_county","type":"varchar(30)"},
+  {"name":"cc_state","type":"char(2)"},
+  {"name":"cc_zip","type":"char(10)"},
+  {"name":"cc_country","type":"varchar(20)"},
+  {"name":"cc_gmt_offset","type":"decimal(5,2)"},
+  {"name":"cc_tax_percentage","type":"decimal(5,2)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/catalog_page.json b/sdks/java/testing/tpcds/src/main/resources/schemas/catalog_page.json
new file mode 100644
index 0000000..15a549466
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/catalog_page.json
@@ -0,0 +1,11 @@
+{"schema": [
+  {"name":"cp_catalog_page_sk","type":"identifier"},
+  {"name":"cp_catalog_page_id","type":"char(16)"},
+  {"name":"cp_start_date_sk","type":"integer"},
+  {"name":"cp_end_date_sk","type":"integer"},
+  {"name":"cp_department","type":"varchar(50)"},
+  {"name":"cp_catalog_number","type":"integer"},
+  {"name":"cp_catalog_page_number","type":"integer"},
+  {"name":"cp_description","type":"varchar(100)"},
+  {"name":"cp_type","type":"varchar(100)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/catalog_returns.json b/sdks/java/testing/tpcds/src/main/resources/schemas/catalog_returns.json
new file mode 100644
index 0000000..d967a9e
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/catalog_returns.json
@@ -0,0 +1,29 @@
+{"schema": [
+  {"name":"cr_returned_date_sk","type":"identifier"},
+  {"name":"cr_returned_time_sk","type":"identifier"},
+  {"name":"cr_item_sk","type":"identifier"},
+  {"name":"cr_refunded_customer_sk","type":"identifier"},
+  {"name":"cr_refunded_cdemo_sk","type":"identifier"},
+  {"name":"cr_refunded_hdemo_sk","type":"identifier"},
+  {"name":"cr_refunded_addr_sk","type":"identifier"},
+  {"name":"cr_returning_customer_sk","type":"identifier"},
+  {"name":"cr_returning_cdemo_sk","type":"identifier"},
+  {"name":"cr_returning_hdemo_sk","type":"identifier"},
+  {"name":"cr_returning_addr_sk","type":"identifier"},
+  {"name":"cr_call_center_sk","type":"identifier"},
+  {"name":"cr_catalog_page_sk","type":"identifier"},
+  {"name":"cr_ship_mode_sk","type":"identifier"},
+  {"name":"cr_warehouse_sk","type":"identifier"},
+  {"name":"cr_reason_sk","type":"identifier"},
+  {"name":"cr_order_number","type":"identifier"},
+  {"name":"cr_return_quantity","type":"integer"},
+  {"name":"cr_return_amount","type":"decimal(7,2)"},
+  {"name":"cr_return_tax","type":"decimal(7,2)"},
+  {"name":"cr_return_amt_inc_tax","type":"decimal(7,2)"},
+  {"name":"cr_fee","type":"decimal(7,2)"},
+  {"name":"cr_return_ship_cost","type":"decimal(7,2)"},
+  {"name":"cr_refunded_cash","type":"decimal(7,2)"},
+  {"name":"cr_reversed_charge","type":"decimal(7,2)"},
+  {"name":"cr_store_credit","type":"decimal(7,2)"},
+  {"name":"cr_net_loss","type":"decimal(7,2)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/catalog_sales.json b/sdks/java/testing/tpcds/src/main/resources/schemas/catalog_sales.json
new file mode 100644
index 0000000..50fae92
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/catalog_sales.json
@@ -0,0 +1,36 @@
+{"schema": [
+  {"name":"cs_sold_date_sk","type":"identifier"},
+  {"name":"cs_sold_time_sk","type":"identifier"},
+  {"name":"cs_ship_date_sk","type":"identifier"},
+  {"name":"cs_bill_customer_sk","type":"identifier"},
+  {"name":"cs_bill_cdemo_sk","type":"identifier"},
+  {"name":"cs_bill_hdemo_sk","type":"identifier"},
+  {"name":"cs_bill_addr_sk","type":"identifier"},
+  {"name":"cs_ship_customer_sk","type":"identifier"},
+  {"name":"cs_ship_cdemo_sk","type":"identifier"},
+  {"name":"cs_ship_hdemo_sk","type":"identifier"},
+  {"name":"cs_ship_addr_sk","type":"identifier"},
+  {"name":"cs_call_center_sk","type":"identifier"},
+  {"name":"cs_catalog_page_sk","type":"identifier"},
+  {"name":"cs_ship_mode_sk","type":"identifier"},
+  {"name":"cs_warehouse_sk","type":"identifier"},
+  {"name":"cs_item_sk","type":"identifier"},
+  {"name":"cs_promo_sk","type":"identifier"},
+  {"name":"cs_order_number","type":"identifier"},
+  {"name":"cs_quantity","type":"integer"},
+  {"name":"cs_wholesale_cost","type":"decimal(7,2)"},
+  {"name":"cs_list_price","type":"decimal(7,2)"},
+  {"name":"cs_sales_price","type":"decimal(7,2)"},
+  {"name":"cs_ext_discount_amt","type":"decimal(7,2)"},
+  {"name":"cs_ext_sales_price","type":"decimal(7,2)"},
+  {"name":"cs_ext_wholesale_cost","type":"decimal(7,2)"},
+  {"name":"cs_ext_list_price","type":"decimal(7,2)"},
+  {"name":"cs_ext_tax","type":"decimal(7,2)"},
+  {"name":"cs_coupon_amt","type":"decimal(7,2)"},
+  {"name":"cs_ext_ship_cost","type":"decimal(7,2)"},
+  {"name":"cs_net_paid","type":"decimal(7,2)"},
+  {"name":"cs_net_paid_inc_tax","type":"decimal(7,2)"},
+  {"name":"cs_net_paid_inc_ship","type":"decimal(7,2)"},
+  {"name":"cs_net_paid_inc_ship_tax","type":"decimal(7,2)"},
+  {"name":"cs_net_profit","type":"decimal(7,2)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/customer.json b/sdks/java/testing/tpcds/src/main/resources/schemas/customer.json
new file mode 100644
index 0000000..fa1fcfb
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/customer.json
@@ -0,0 +1,20 @@
+{"schema": [
+  {"name":"c_customer_sk","type":"identifier"},
+  {"name":"c_customer_id","type":"char(16)"},
+  {"name":"c_current_cdemo_sk","type":"identifier"},
+  {"name":"c_current_hdemo_sk","type":"identifier"},
+  {"name":"c_current_addr_sk","type":"identifier"},
+  {"name":"c_first_shipto_date_sk","type":"identifier"},
+  {"name":"c_first_sales_date_sk","type":"identifier"},
+  {"name":"c_salutation","type":"char(10)"},
+  {"name":"c_first_name","type":"char(20)"},
+  {"name":"c_last_name","type":"char(30)"},
+  {"name":"c_preferred_cust_flag","type":"char(1)"},
+  {"name":"c_birth_day","type":"integer"},
+  {"name":"c_birth_month","type":"integer"},
+  {"name":"c_birth_year","type":"integer"},
+  {"name":"c_birth_country","type":"varchar(20)"},
+  {"name":"c_login","type":"char(13)"},
+  {"name":"c_email_address","type":"char(50)"},
+  {"name":"c_last_review_date_sk","type":"identifier"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/customer_address.json b/sdks/java/testing/tpcds/src/main/resources/schemas/customer_address.json
new file mode 100644
index 0000000..a37bdbe
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/customer_address.json
@@ -0,0 +1,15 @@
+{"schema": [
+  {"name":"ca_address_sk","type":"identifier"},
+  {"name":"ca_address_id","type":"char(16)"},
+  {"name":"ca_street_number","type":"char(10)"},
+  {"name":"ca_street_name","type":"varchar(60)"},
+  {"name":"ca_street_type","type":"char(15)"},
+  {"name":"ca_suite_number","type":"char(10)"},
+  {"name":"ca_city","type":"varchar(60)"},
+  {"name":"ca_county","type":"varchar(30)"},
+  {"name":"ca_state","type":"char(2)"},
+  {"name":"ca_zip","type":"char(10)"},
+  {"name":"ca_country","type":"varchar(20)"},
+  {"name":"ca_gmt_offset","type":"decimal(5,2)"},
+  {"name":"ca_location_type","type":"char(20)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/customer_demographics.json b/sdks/java/testing/tpcds/src/main/resources/schemas/customer_demographics.json
new file mode 100644
index 0000000..2b2211b
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/customer_demographics.json
@@ -0,0 +1,11 @@
+{"schema": [
+  {"name":"cd_demo_sk","type":"identifier"},
+  {"name":"cd_gender","type":"char(1)"},
+  {"name":"cd_marital_status","type":"char(1)"},
+  {"name":"cd_education_status","type":"char(20)"},
+  {"name":"cd_purchase_estimate","type":"integer"},
+  {"name":"cd_credit_rating","type":"char(10)"},
+  {"name":"cd_dep_count","type":"integer"},
+  {"name":"cd_dep_employed_count","type":"integer"},
+  {"name":"cd_dep_college_count","type":"integer"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/date_dim.json b/sdks/java/testing/tpcds/src/main/resources/schemas/date_dim.json
new file mode 100644
index 0000000..287738f
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/date_dim.json
@@ -0,0 +1,30 @@
+{"schema": [
+  {"name":"d_date_sk","type":"identifier"},
+  {"name":"d_date_id","type":"char(16)"},
+  {"name":"d_date","type":"date"},
+  {"name":"d_month_seq","type":"integer"},
+  {"name":"d_week_seq","type":"integer"},
+  {"name":"d_quarter_seq","type":"integer"},
+  {"name":"d_year","type":"integer"},
+  {"name":"d_dow","type":"integer"},
+  {"name":"d_moy","type":"integer"},
+  {"name":"d_dom","type":"integer"},
+  {"name":"d_qoy","type":"integer"},
+  {"name":"d_fy_year","type":"integer"},
+  {"name":"d_fy_quarter_seq","type":"integer"},
+  {"name":"d_fy_week_seq","type":"integer"},
+  {"name":"d_day_name","type":"char(9)"},
+  {"name":"d_quarter_name","type":"char(6)"},
+  {"name":"d_holiday","type":"char(1)"},
+  {"name":"d_weekend","type":"char(1)"},
+  {"name":"d_following_holiday","type":"char(1)"},
+  {"name":"d_first_dom","type":"integer"},
+  {"name":"d_last_dom","type":"integer"},
+  {"name":"d_same_day_ly","type":"integer"},
+  {"name":"d_same_day_lq","type":"integer"},
+  {"name":"d_current_day","type":"char(1)"},
+  {"name":"d_current_week","type":"char(1)"},
+  {"name":"d_current_month","type":"char(1)"},
+  {"name":"d_current_quarter","type":"char(1)"},
+  {"name":"d_current_year","type":"char(1)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/household_demographics.json b/sdks/java/testing/tpcds/src/main/resources/schemas/household_demographics.json
new file mode 100644
index 0000000..a261ae9
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/household_demographics.json
@@ -0,0 +1,7 @@
+{"schema": [
+  {"name":"hd_demo_sk","type":"identifier"},
+  {"name":"hd_income_band_sk","type":"identifier"},
+  {"name":"hd_buy_potential","type":"char(15)"},
+  {"name":"hd_dep_count","type":"integer"},
+  {"name":"hd_vehicle_count","type":"integer"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/income_band.json b/sdks/java/testing/tpcds/src/main/resources/schemas/income_band.json
new file mode 100644
index 0000000..3066b27
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/income_band.json
@@ -0,0 +1,5 @@
+{"schema": [
+  {"name":"ib_income_band_sk","type":"identifier"},
+  {"name":"ib_lower_bound","type":"integer"},
+  {"name":"ib_upper_bound","type":"integer"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/inventory.json b/sdks/java/testing/tpcds/src/main/resources/schemas/inventory.json
new file mode 100644
index 0000000..ee786bc
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/inventory.json
@@ -0,0 +1,7 @@
+{"schema": [
+  {"name":"inv_date_sk","type":"identifier"},
+  {"name":"inv_item_sk","type":"identifier"},
+  {"name":"inv_warehouse_sk","type":"identifier"},
+  {"name":"inv_quantity_on_hand","type":"integer"}
+]}
+
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/item.json b/sdks/java/testing/tpcds/src/main/resources/schemas/item.json
new file mode 100644
index 0000000..593aff1
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/item.json
@@ -0,0 +1,24 @@
+{"schema": [
+  {"name":"i_item_sk","type":"identifier"},
+  {"name":"i_item_id","type":"char(16)"},
+  {"name":"i_rec_start_date","type":"date"},
+  {"name":"i_rec_end_date","type":"date"},
+  {"name":"i_item_desc","type":"varchar(200)"},
+  {"name":"i_current_price","type":"decimal(7,2)"},
+  {"name":"i_wholesale_cost","type":"decimal(7,2)"},
+  {"name":"i_brand_id","type":"integer"},
+  {"name":"i_brand","type":"char(50)"},
+  {"name":"i_class_id","type":"integer"},
+  {"name":"i_class","type":"char(50)"},
+  {"name":"i_category_id","type":"integer"},
+  {"name":"i_category","type":"char(50)"},
+  {"name":"i_manufact_id","type":"integer"},
+  {"name":"i_manufact","type":"char(50)"},
+  {"name":"i_size","type":"char(20)"},
+  {"name":"i_formulation","type":"char(20)"},
+  {"name":"i_color","type":"char(20)"},
+  {"name":"i_units","type":"char(10)"},
+  {"name":"i_container","type":"char(10)"},
+  {"name":"i_manager_id","type":"integer"},
+  {"name":"i_product_name","type":"char(50)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/promotion.json b/sdks/java/testing/tpcds/src/main/resources/schemas/promotion.json
new file mode 100644
index 0000000..28d57d4
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/promotion.json
@@ -0,0 +1,21 @@
+{"schema": [
+  {"name":"p_promo_sk","type":"identifier"},
+  {"name":"p_promo_id","type":"char(16)"},
+  {"name":"p_start_date_sk","type":"identifier"},
+  {"name":"p_end_date_sk","type":"identifier"},
+  {"name":"p_item_sk","type":"identifier"},
+  {"name":"p_cost","type":"decimal(15,2)"},
+  {"name":"p_response_target","type":"integer"},
+  {"name":"p_promo_name","type":"char(50)"},
+  {"name":"p_channel_dmail","type":"char(1)"},
+  {"name":"p_channel_email","type":"char(1)"},
+  {"name":"p_channel_catalog","type":"char(1)"},
+  {"name":"p_channel_tv","type":"char(1)"},
+  {"name":"p_channel_radio","type":"char(1)"},
+  {"name":"p_channel_press","type":"char(1)"},
+  {"name":"p_channel_event","type":"char(1)"},
+  {"name":"p_channel_demo","type":"char(1)"},
+  {"name":"p_channel_details","type":"varchar(100)"},
+  {"name":"p_purpose","type":"char(15)"},
+  {"name":"p_discount_active","type":"char(1)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/reason.json b/sdks/java/testing/tpcds/src/main/resources/schemas/reason.json
new file mode 100644
index 0000000..64b9723
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/reason.json
@@ -0,0 +1,5 @@
+{"schema": [
+  {"name":"r_reason_sk","type":"identifier"},
+  {"name":"r_reason_id","type":"char(16)"},
+  {"name":"r_reason_desc","type":"char(100)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/ship_mode.json b/sdks/java/testing/tpcds/src/main/resources/schemas/ship_mode.json
new file mode 100644
index 0000000..09c4873
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/ship_mode.json
@@ -0,0 +1,8 @@
+{"schema": [
+  {"name":"sm_ship_mode_sk","type":"identifier"},
+  {"name":"sm_ship_mode_id","type":"char(16)"},
+  {"name":"sm_type","type":"char(30)"},
+  {"name":"sm_code","type":"char(10)"},
+  {"name":"sm_carrier","type":"char(20)"},
+  {"name":"sm_contract","type":"char(20)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/store.json b/sdks/java/testing/tpcds/src/main/resources/schemas/store.json
new file mode 100644
index 0000000..3df8465
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/store.json
@@ -0,0 +1,31 @@
+{"schema": [
+  {"name":"s_store_sk","type":"identifier"},
+  {"name":"s_store_id","type":"char(16)"},
+  {"name":"s_rec_start_date","type":"date"},
+  {"name":"s_rec_end_date","type":"date"},
+  {"name":"s_closed_date_sk","type":"identifier"},
+  {"name":"s_store_name","type":"varchar(50)"},
+  {"name":"s_number_employees","type":"integer"},
+  {"name":"s_floor_space","type":"integer"},
+  {"name":"s_hours","type":"char(20)"},
+  {"name":"S_manager","type":"varchar(40)"},
+  {"name":"S_market_id","type":"integer"},
+  {"name":"S_geography_class","type":"varchar(100)"},
+  {"name":"S_market_desc","type":"varchar(100)"},
+  {"name":"s_market_manager","type":"varchar(40)"},
+  {"name":"s_division_id","type":"integer"},
+  {"name":"s_division_name","type":"varchar(50)"},
+  {"name":"s_company_id","type":"integer"},
+  {"name":"s_company_name","type":"varchar(50)"},
+  {"name":"s_street_number","type":"varchar(10)"},
+  {"name":"s_street_name","type":"varchar(60)"},
+  {"name":"s_street_type","type":"char(15)"},
+  {"name":"s_suite_number","type":"char(10)"},
+  {"name":"s_city","type":"varchar(60)"},
+  {"name":"s_county","type":"varchar(30)"},
+  {"name":"s_state","type":"char(2)"},
+  {"name":"s_zip","type":"char(10)"},
+  {"name":"s_country","type":"varchar(20)"},
+  {"name":"s_gmt_offset","type":"decimal(5,2)"},
+  {"name":"s_tax_percentage","type":"decimal(5,2)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/store_returns.json b/sdks/java/testing/tpcds/src/main/resources/schemas/store_returns.json
new file mode 100644
index 0000000..99b0db1
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/store_returns.json
@@ -0,0 +1,22 @@
+{"schema": [
+  {"name":"sr_returned_date_sk","type":"identifier"},
+  {"name":"sr_return_time_sk","type":"identifier"},
+  {"name":"sr_item_sk","type":"identifier"},
+  {"name":"sr_customer_sk","type":"identifier"},
+  {"name":"sr_cdemo_sk","type":"identifier"},
+  {"name":"sr_hdemo_sk","type":"identifier"},
+  {"name":"sr_addr_sk","type":"identifier"},
+  {"name":"sr_store_sk","type":"identifier"},
+  {"name":"sr_reason_sk","type":"identifier"},
+  {"name":"sr_ticket_number","type":"identifier"},
+  {"name":"sr_return_quantity","type":"integer"},
+  {"name":"sr_return_amt","type":"decimal(7,2)"},
+  {"name":"sr_return_tax","type":"decimal(7,2)"},
+  {"name":"sr_return_amt_inc_tax","type":"decimal(7,2)"},
+  {"name":"sr_fee","type":"decimal(7,2)"},
+  {"name":"sr_return_ship_cost","type":"decimal(7,2)"},
+  {"name":"sr_refunded_cash","type":"decimal(7,2)"},
+  {"name":"sr_reversed_charge","type":"decimal(7,2)"},
+  {"name":"sr_store_credit","type":"decimal(7,2)"},
+  {"name":"sr_net_loss","type":"decimal(7,2)"}
+]}
\ No newline at end of file
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/store_sales.json b/sdks/java/testing/tpcds/src/main/resources/schemas/store_sales.json
new file mode 100644
index 0000000..3b133d8
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/store_sales.json
@@ -0,0 +1,25 @@
+{"schema": [
+  {"name":"ss_sold_date_sk","type":"identifier"},
+  {"name":"ss_sold_time_sk","type":"identifier"},
+  {"name":"ss_item_sk","type":"identifier"},
+  {"name":"ss_customer_sk","type":"identifier"},
+  {"name":"ss_cdemo_sk","type":"identifier"},
+  {"name":"ss_hdemo_sk","type":"identifier"},
+  {"name":"ss_addr_sk","type":"identifier"},
+  {"name":"ss_store_sk","type":"identifier"},
+  {"name":"ss_promo_sk","type":"identifier"},
+  {"name":"ss_ticket_number","type":"identifier"},
+  {"name":"ss_quantity","type":"integer"},
+  {"name":"ss_wholesale_cost","type":"decimal(7,2)"},
+  {"name":"ss_list_price","type":"decimal(7,2)"},
+  {"name":"ss_sales_price","type":"decimal(7,2)"},
+  {"name":"ss_ext_discount_amt","type":"decimal(7,2)"},
+  {"name":"ss_ext_sales_price","type":"decimal(7,2)"},
+  {"name":"ss_ext_wholesale_cost","type":"decimal(7,2)"},
+  {"name":"ss_ext_list_price","type":"decimal(7,2)"},
+  {"name":"ss_ext_tax","type":"decimal(7,2)"},
+  {"name":"ss_coupon_amt","type":"decimal(7,2)"},
+  {"name":"ss_net_paid","type":"decimal(7,2)"},
+  {"name":"ss_net_paid_inc_tax","type":"decimal(7,2)"},
+  {"name":"ss_net_profit","type":"decimal(7,2)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/time_dim.json b/sdks/java/testing/tpcds/src/main/resources/schemas/time_dim.json
new file mode 100644
index 0000000..e1d51f6
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/time_dim.json
@@ -0,0 +1,12 @@
+{"schema": [
+  {"name":"t_time_sk","type":"Identifier"},
+  {"name":"t_time_id","type":"char(16)"},
+  {"name":"t_time","type":"Integer"},
+  {"name":"t_hour","type":"Integer"},
+  {"name":"t_minute","type":"Integer"},
+  {"name":"t_second","type":"Integer"},
+  {"name":"t_am_pm","type":"char(2)"},
+  {"name":"t_shift","type":"char(20)"},
+  {"name":"t_sub_shift","type":"char(20)"},
+  {"name":"t_meal_time","type":"char(20)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/warehouse.json b/sdks/java/testing/tpcds/src/main/resources/schemas/warehouse.json
new file mode 100644
index 0000000..e3126de
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/warehouse.json
@@ -0,0 +1,16 @@
+{"schema": [
+  {"name":"w_warehouse_sk","type":"identifier"},
+  {"name":"w_warehouse_id","type":"char(16)"},
+  {"name":"w_warehouse_name","type":"varchar(20)"},
+  {"name":"w_warehouse_sq_ft","type":"integer"},
+  {"name":"w_street_number","type":"char(10)"},
+  {"name":"w_street_name","type":"varchar(60)"},
+  {"name":"w_street_type","type":"char(15)"},
+  {"name":"w_suite_number","type":"char(10)"},
+  {"name":"w_city","type":"varchar(60)"},
+  {"name":"w_county","type":"varchar(30)"},
+  {"name":"w_state","type":"char(2)"},
+  {"name":"w_zip","type":"char(10)"},
+  {"name":"w_country","type":"varchar(20)"},
+  {"name":"w_gmt_offset","type":"decimal(5,2)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/web_page.json b/sdks/java/testing/tpcds/src/main/resources/schemas/web_page.json
new file mode 100644
index 0000000..4dc3436
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/web_page.json
@@ -0,0 +1,16 @@
+{"schema": [
+  {"name":"wp_web_page_sk","type":"identifier"},
+  {"name":"wp_web_page_id","type":"char(16)"},
+  {"name":"wp_rec_start_date","type":"date"},
+  {"name":"wp_rec_end_date","type":"date"},
+  {"name":"wp_creation_date_sk","type":"identifier"},
+  {"name":"wp_access_date_sk","type":"identifier"},
+  {"name":"wp_autogen_flag","type":"char(1)"},
+  {"name":"wp_customer_sk","type":"identifier"},
+  {"name":"wp_url","type":"varchar(100)"},
+  {"name":"wp_type","type":"char(50)"},
+  {"name":"wp_char_count","type":"integer"},
+  {"name":"wp_link_count","type":"integer"},
+  {"name":"wp_image_count","type":"integer"},
+  {"name":"wp_max_ad_count","type":"integer"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/web_returns.json b/sdks/java/testing/tpcds/src/main/resources/schemas/web_returns.json
new file mode 100644
index 0000000..101ef1c
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/web_returns.json
@@ -0,0 +1,26 @@
+{"schema": [
+  {"name":"wr_returned_date_sk","type":"identifier"},
+  {"name":"wr_returned_time_sk","type":"identifier"},
+  {"name":"wr_item_sk","type":"identifier"},
+  {"name":"wr_refunded_customer_sk","type":"identifier"},
+  {"name":"wr_refunded_cdemo_sk","type":"identifier"},
+  {"name":"wr_refunded_hdemo_sk","type":"identifier"},
+  {"name":"wr_refunded_addr_sk","type":"identifier"},
+  {"name":"wr_returning_customer_sk","type":"identifier"},
+  {"name":"wr_returning_cdemo_sk","type":"identifier"},
+  {"name":"wr_returning_hdemo_sk","type":"identifier"},
+  {"name":"wr_returning_addr_sk","type":"identifier"},
+  {"name":"wr_web_page_sk","type":"identifier"},
+  {"name":"wr_reason_sk","type":"identifier"},
+  {"name":"wr_order_number","type":"identifier"},
+  {"name":"wr_return_quantity","type":"integer"},
+  {"name":"wr_return_amt","type":"decimal(7,2)"},
+  {"name":"wr_return_tax","type":"decimal(7,2)"},
+  {"name":"wr_return_amt_inc_tax","type":"decimal(7,2)"},
+  {"name":"wr_fee","type":"decimal(7,2)"},
+  {"name":"wr_return_ship_cost","type":"decimal(7,2)"},
+  {"name":"wr_refunded_cash","type":"decimal(7,2)"},
+  {"name":"wr_reversed_charge","type":"decimal(7,2)"},
+  {"name":"wr_account_credit","type":"decimal(7,2)"},
+  {"name":"wr_net_loss","type":"decimal(7,2)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/web_sales.json b/sdks/java/testing/tpcds/src/main/resources/schemas/web_sales.json
new file mode 100644
index 0000000..2cbcdaa
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/web_sales.json
@@ -0,0 +1,36 @@
+{"schema": [
+  {"name":"ws_sold_date_sk","type":"identifier"},
+  {"name":"ws_sold_time_sk","type":"identifier"},
+  {"name":"ws_ship_date_sk","type":"identifier"},
+  {"name":"ws_item_sk","type":"identifier"},
+  {"name":"ws_bill_customer_sk","type":"identifier"},
+  {"name":"ws_bill_cdemo_sk","type":"identifier"},
+  {"name":"ws_bill_hdemo_sk","type":"identifier"},
+  {"name":"ws_bill_addr_sk","type":"identifier"},
+  {"name":"ws_ship_customer_sk","type":"identifier"},
+  {"name":"ws_ship_cdemo_sk","type":"identifier"},
+  {"name":"ws_ship_hdemo_sk","type":"identifier"},
+  {"name":"ws_ship_addr_sk","type":"identifier"},
+  {"name":"ws_web_page_sk","type":"identifier"},
+  {"name":"ws_web_site_sk","type":"identifier"},
+  {"name":"ws_ship_mode_sk","type":"identifier"},
+  {"name":"ws_warehouse_sk","type":"identifier"},
+  {"name":"ws_promo_sk","type":"identifier"},
+  {"name":"ws_order_number","type":"identifier"},
+  {"name":"ws_quantity","type":"integer"},
+  {"name":"ws_wholesale_cost","type":"decimal(7,2)"},
+  {"name":"ws_list_price","type":"decimal(7,2)"},
+  {"name":"ws_sales_price","type":"decimal(7,2)"},
+  {"name":"ws_ext_discount_amt","type":"decimal(7,2)"},
+  {"name":"ws_ext_sales_price","type":"decimal(7,2)"},
+  {"name":"ws_ext_wholesale_cost","type":"decimal(7,2)"},
+  {"name":"ws_ext_list_price","type":"decimal(7,2)"},
+  {"name":"ws_ext_tax","type":"decimal(7,2)"},
+  {"name":"ws_coupon_amt","type":"decimal(7,2)"},
+  {"name":"ws_ext_ship_cost","type":"decimal(7,2)"},
+  {"name":"ws_net_paid","type":"decimal(7,2)"},
+  {"name":"ws_net_paid_inc_tax","type":"decimal(7,2)"},
+  {"name":"ws_net_paid_inc_ship","type":"decimal(7,2)"},
+  {"name":"ws_net_paid_inc_ship_tax","type":"decimal(7,2)"},
+  {"name":"ws_net_profit","type":"decimal(7,2)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/web_site.json b/sdks/java/testing/tpcds/src/main/resources/schemas/web_site.json
new file mode 100644
index 0000000..7cecde0
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/web_site.json
@@ -0,0 +1,28 @@
+{"schema": [
+  {"name":"web_site_sk","type":"varchar(100)"},
+  {"name":"web_site_id","type":"char(16)"},
+  {"name":"web_rec_start_date","type":"date"},
+  {"name":"web_rec_end_date","type":"date"},
+  {"name":"web_name","type":"varchar(50)"},
+  {"name":"web_open_date_sk","type":"identifier"},
+  {"name":"web_close_date_sk","type":"identifier"},
+  {"name":"web_class","type":"varchar(50)"},
+  {"name":"web_manager","type":"varchar(40)"},
+  {"name":"web_mkt_id","type":"integer"},
+  {"name":"web_mkt_class","type":"varchar(50)"},
+  {"name":"web_mkt_desc","type":"varchar(100)"},
+  {"name":"web_market_manager","type":"varchar(40)"},
+  {"name":"web_company_id","type":"integer"},
+  {"name":"web_company_name","type":"char(50)"},
+  {"name":"web_street_number","type":"char(10)"},
+  {"name":"web_street_name","type":"varchar(60)"},
+  {"name":"web_street_type","type":"char(15)"},
+  {"name":"web_suite_number","type":"char(10)"},
+  {"name":"web_city","type":"varchar(60)"},
+  {"name":"web_county","type":"varchar(30)"},
+  {"name":"web_state","type":"char(2)"},
+  {"name":"web_zip","type":"char(10)"},
+  {"name":"web_country","type":"varchar(20)"},
+  {"name":"web_gmt_offset","type":"decimal(5,2)"},
+  {"name":"web_tax_percentage","type":"decimal(5,2)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/test/java/org/apache/beam/sdk/tpcds/QueryReaderTest.java b/sdks/java/testing/tpcds/src/test/java/org/apache/beam/sdk/tpcds/QueryReaderTest.java
new file mode 100644
index 0000000..5696410
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/test/java/org/apache/beam/sdk/tpcds/QueryReaderTest.java
@@ -0,0 +1,205 @@
+/*
+ * 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.tpcds;
+
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
+
+public class QueryReaderTest {
+    private final String headers = "-- Licensed to the Apache Software Foundation (ASF) under one\n" +
+            "-- or more contributor license agreements.  See the NOTICE file\n" +
+            "-- distributed with this work for additional information\n" +
+            "-- regarding copyright ownership.  The ASF licenses this file\n" +
+            "-- to you under the Apache License, Version 2.0 (the\n" +
+            "-- \"License\"); you may not use this file except in compliance\n" +
+            "-- with the License.  You may obtain a copy of the License at\n" +
+            "--\n" +
+            "--     http://www.apache.org/licenses/LICENSE-2.0\n" +
+            "--\n" +
+            "-- Unless required by applicable law or agreed to in writing, software\n" +
+            "-- distributed under the License is distributed on an \"AS IS\" BASIS,\n" +
+            "-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n" +
+            "-- See the License for the specific language governing permissions and\n" +
+            "-- limitations under the License.\n";
+
+    @Test
+    public void testQuery3String() throws Exception {
+        String query3String = QueryReader.readQuery("query3");
+        String expected = "select  dt.d_year \n" +
+                "       ,item.i_brand_id brand_id \n" +
+                "       ,item.i_brand brand\n" +
+                "       ,sum(ss_ext_sales_price) sum_agg\n" +
+                " from  date_dim dt \n" +
+                "      ,store_sales\n" +
+                "      ,item\n" +
+                " where dt.d_date_sk = store_sales.ss_sold_date_sk\n" +
+                "   and store_sales.ss_item_sk = item.i_item_sk\n" +
+                "   and item.i_manufact_id = 436\n" +
+                "   and dt.d_moy=12\n" +
+                " group by dt.d_year\n" +
+                "      ,item.i_brand\n" +
+                "      ,item.i_brand_id\n" +
+                " order by dt.d_year\n" +
+                "         ,sum_agg desc\n" +
+                "         ,brand_id\n" +
+                " limit 100";
+        String query3StringNoSpaces = query3String.replaceAll("\\s+", "");
+        String expectedNoSpaces = (headers + expected).replaceAll("\\s+", "");
+        assertEquals(expectedNoSpaces, query3StringNoSpaces);
+    }
+
+    @Test
+    public void testQuery4String() throws Exception {
+        String query4String = QueryReader.readQuery("query4");
+        String expected = "with year_total as (\n" +
+                " select c_customer_id customer_id\n" +
+                "       ,c_first_name customer_first_name\n" +
+                "       ,c_last_name customer_last_name\n" +
+                "       ,c_preferred_cust_flag customer_preferred_cust_flag\n" +
+                "       ,c_birth_country customer_birth_country\n" +
+                "       ,c_login customer_login\n" +
+                "       ,c_email_address customer_email_address\n" +
+                "       ,d_year dyear\n" +
+                "       ,sum(((ss_ext_list_price-ss_ext_wholesale_cost-ss_ext_discount_amt)+ss_ext_sales_price)/2) year_total\n" +
+                "       ,'s' sale_type\n" +
+                " from customer\n" +
+                "     ,store_sales\n" +
+                "     ,date_dim\n" +
+                " where c_customer_sk = ss_customer_sk\n" +
+                "   and ss_sold_date_sk = d_date_sk\n" +
+                " group by c_customer_id\n" +
+                "         ,c_first_name\n" +
+                "         ,c_last_name\n" +
+                "         ,c_preferred_cust_flag\n" +
+                "         ,c_birth_country\n" +
+                "         ,c_login\n" +
+                "         ,c_email_address\n" +
+                "         ,d_year\n" +
+                " union all\n" +
+                " select c_customer_id customer_id\n" +
+                "       ,c_first_name customer_first_name\n" +
+                "       ,c_last_name customer_last_name\n" +
+                "       ,c_preferred_cust_flag customer_preferred_cust_flag\n" +
+                "       ,c_birth_country customer_birth_country\n" +
+                "       ,c_login customer_login\n" +
+                "       ,c_email_address customer_email_address\n" +
+                "       ,d_year dyear\n" +
+                "       ,sum((((cs_ext_list_price-cs_ext_wholesale_cost-cs_ext_discount_amt)+cs_ext_sales_price)/2) ) year_total\n" +
+                "       ,'c' sale_type\n" +
+                " from customer\n" +
+                "     ,catalog_sales\n" +
+                "     ,date_dim\n" +
+                " where c_customer_sk = cs_bill_customer_sk\n" +
+                "   and cs_sold_date_sk = d_date_sk\n" +
+                " group by c_customer_id\n" +
+                "         ,c_first_name\n" +
+                "         ,c_last_name\n" +
+                "         ,c_preferred_cust_flag\n" +
+                "         ,c_birth_country\n" +
+                "         ,c_login\n" +
+                "         ,c_email_address\n" +
+                "         ,d_year\n" +
+                "union all\n" +
+                " select c_customer_id customer_id\n" +
+                "       ,c_first_name customer_first_name\n" +
+                "       ,c_last_name customer_last_name\n" +
+                "       ,c_preferred_cust_flag customer_preferred_cust_flag\n" +
+                "       ,c_birth_country customer_birth_country\n" +
+                "       ,c_login customer_login\n" +
+                "       ,c_email_address customer_email_address\n" +
+                "       ,d_year dyear\n" +
+                "       ,sum((((ws_ext_list_price-ws_ext_wholesale_cost-ws_ext_discount_amt)+ws_ext_sales_price)/2) ) year_total\n" +
+                "       ,'w' sale_type\n" +
+                " from customer\n" +
+                "     ,web_sales\n" +
+                "     ,date_dim\n" +
+                " where c_customer_sk = ws_bill_customer_sk\n" +
+                "   and ws_sold_date_sk = d_date_sk\n" +
+                " group by c_customer_id\n" +
+                "         ,c_first_name\n" +
+                "         ,c_last_name\n" +
+                "         ,c_preferred_cust_flag\n" +
+                "         ,c_birth_country\n" +
+                "         ,c_login\n" +
+                "         ,c_email_address\n" +
+                "         ,d_year\n" +
+                "         )\n" +
+                "  select  \n" +
+                "                  t_s_secyear.customer_id\n" +
+                "                 ,t_s_secyear.customer_first_name\n" +
+                "                 ,t_s_secyear.customer_last_name\n" +
+                "                 ,t_s_secyear.customer_email_address\n" +
+                " from year_total t_s_firstyear\n" +
+                "     ,year_total t_s_secyear\n" +
+                "     ,year_total t_c_firstyear\n" +
+                "     ,year_total t_c_secyear\n" +
+                "     ,year_total t_w_firstyear\n" +
+                "     ,year_total t_w_secyear\n" +
+                " where t_s_secyear.customer_id = t_s_firstyear.customer_id\n" +
+                "   and t_s_firstyear.customer_id = t_c_secyear.customer_id\n" +
+                "   and t_s_firstyear.customer_id = t_c_firstyear.customer_id\n" +
+                "   and t_s_firstyear.customer_id = t_w_firstyear.customer_id\n" +
+                "   and t_s_firstyear.customer_id = t_w_secyear.customer_id\n" +
+                "   and t_s_firstyear.sale_type = 's'\n" +
+                "   and t_c_firstyear.sale_type = 'c'\n" +
+                "   and t_w_firstyear.sale_type = 'w'\n" +
+                "   and t_s_secyear.sale_type = 's'\n" +
+                "   and t_c_secyear.sale_type = 'c'\n" +
+                "   and t_w_secyear.sale_type = 'w'\n" +
+                "   and t_s_firstyear.dyear =  2001\n" +
+                "   and t_s_secyear.dyear = 2001+1\n" +
+                "   and t_c_firstyear.dyear =  2001\n" +
+                "   and t_c_secyear.dyear =  2001+1\n" +
+                "   and t_w_firstyear.dyear = 2001\n" +
+                "   and t_w_secyear.dyear = 2001+1\n" +
+                "   and t_s_firstyear.year_total > 0\n" +
+                "   and t_c_firstyear.year_total > 0\n" +
+                "   and t_w_firstyear.year_total > 0\n" +
+                "   and case when t_c_firstyear.year_total > 0 then t_c_secyear.year_total / t_c_firstyear.year_total else null end\n" +
+                "           > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else null end\n" +
+                "   and case when t_c_firstyear.year_total > 0 then t_c_secyear.year_total / t_c_firstyear.year_total else null end\n" +
+                "           > case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else null end\n" +
+                " order by t_s_secyear.customer_id\n" +
+                "         ,t_s_secyear.customer_first_name\n" +
+                "         ,t_s_secyear.customer_last_name\n" +
+                "         ,t_s_secyear.customer_email_address\n" +
+                "limit 100";
+        String query4StringNoSpaces = query4String.replaceAll("\\s+", "");
+        String expectedNoSpaces = (headers + expected).replaceAll("\\s+", "");
+        assertEquals(expectedNoSpaces, query4StringNoSpaces);
+    }
+
+    @Test
+    public void testQuery55String() throws Exception {
+        String query55String = QueryReader.readQuery("query55");
+        String expected = "select  i_brand_id brand_id, i_brand brand,\n" +
+                " \tsum(ss_ext_sales_price) ext_price\n" +
+                " from date_dim, store_sales, item\n" +
+                " where d_date_sk = ss_sold_date_sk\n" +
+                " \tand ss_item_sk = i_item_sk\n" +
+                " \tand i_manager_id=36\n" +
+                " \tand d_moy=12\n" +
+                " \tand d_year=2001\n" +
+                " group by i_brand, i_brand_id\n" +
+                " order by ext_price desc, i_brand_id\n" +
+                "limit 100";
+        String query55StringNoSpaces = query55String.replaceAll("\\s+", "");
+        String expectedNoSpaces = (headers + expected).replaceAll("\\s+", "");
+        assertEquals(expectedNoSpaces, query55StringNoSpaces);
+    }
+}
diff --git a/sdks/java/testing/tpcds/src/test/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoaderTest.java b/sdks/java/testing/tpcds/src/test/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoaderTest.java
new file mode 100644
index 0000000..7748bee
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/test/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoaderTest.java
@@ -0,0 +1,151 @@
+/*
+ * 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.tpcds;
+
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+
+public class TableSchemaJSONLoaderTest {
+    @Test
+    public void testStoreReturnsTable() throws Exception {
+        String storeReturnsSchemaString = TableSchemaJSONLoader.parseTableSchema("store_returns");
+        String expected = "sr_returned_date_sk bigint,"
+                + "sr_return_time_sk bigint,"
+                + "sr_item_sk bigint,"
+                + "sr_customer_sk bigint,"
+                + "sr_cdemo_sk bigint,"
+                + "sr_hdemo_sk bigint,"
+                + "sr_addr_sk bigint,"
+                + "sr_store_sk bigint,"
+                + "sr_reason_sk bigint,"
+                + "sr_ticket_number bigint,"
+                + "sr_return_quantity bigint,"
+                + "sr_return_amt double,"
+                + "sr_return_tax double,"
+                + "sr_return_amt_inc_tax double,"
+                + "sr_fee double,"
+                + "sr_return_ship_cost double,"
+                + "sr_refunded_cash double,"
+                + "sr_reversed_charge double,"
+                + "sr_store_credit double,"
+                + "sr_net_loss double";
+        assertEquals(expected, storeReturnsSchemaString);
+    }
+
+    @Test
+    public void testItemTable() throws Exception {
+        String itemSchemaString = TableSchemaJSONLoader.parseTableSchema("item");
+        String expected = "i_item_sk bigint,"
+                + "i_item_id varchar,"
+                + "i_rec_start_date varchar,"
+                + "i_rec_end_date varchar,"
+                + "i_item_desc varchar,"
+                + "i_current_price double,"
+                + "i_wholesale_cost double,"
+                + "i_brand_id bigint,"
+                + "i_brand varchar,"
+                + "i_class_id bigint,"
+                + "i_class varchar,"
+                + "i_category_id bigint,"
+                + "i_category varchar,"
+                + "i_manufact_id bigint,"
+                + "i_manufact varchar,"
+                + "i_size varchar,"
+                + "i_formulation varchar,"
+                + "i_color varchar,"
+                + "i_units varchar,"
+                + "i_container varchar,"
+                + "i_manager_id bigint,"
+                + "i_product_name varchar";
+        assertEquals(expected, itemSchemaString);
+    }
+
+    @Test
+    public void testDateDimTable() throws Exception {
+        String dateDimSchemaString = TableSchemaJSONLoader.parseTableSchema("date_dim");
+        String expected = "d_date_sk bigint,"
+                + "d_date_id varchar,"
+                + "d_date varchar,"
+                + "d_month_seq bigint,"
+                + "d_week_seq bigint,"
+                + "d_quarter_seq bigint,"
+                + "d_year bigint,"
+                + "d_dow bigint,"
+                + "d_moy bigint,"
+                + "d_dom bigint,"
+                + "d_qoy bigint,"
+                + "d_fy_year bigint,"
+                + "d_fy_quarter_seq bigint,"
+                + "d_fy_week_seq bigint,"
+                + "d_day_name varchar,"
+                + "d_quarter_name varchar,"
+                + "d_holiday varchar,"
+                + "d_weekend varchar,"
+                + "d_following_holiday varchar,"
+                + "d_first_dom bigint,"
+                + "d_last_dom bigint,"
+                + "d_same_day_ly bigint,"
+                + "d_same_day_lq bigint,"
+                + "d_current_day varchar,"
+                + "d_current_week varchar,"
+                + "d_current_month varchar,"
+                + "d_current_quarter varchar,"
+                + "d_current_year varchar";
+        assertEquals(expected, dateDimSchemaString);
+    }
+
+    @Test
+    public void testWarehouseTable() throws Exception {
+        String warehouseSchemaString = TableSchemaJSONLoader.parseTableSchema("warehouse");
+        String expected = "w_warehouse_sk bigint,"
+                + "w_warehouse_id varchar,"
+                + "w_warehouse_name varchar,"
+                + "w_warehouse_sq_ft bigint,"
+                + "w_street_number varchar,"
+                + "w_street_name varchar,"
+                + "w_street_type varchar,"
+                + "w_suite_number varchar,"
+                + "w_city varchar,"
+                + "w_county varchar,"
+                + "w_state varchar,"
+                + "w_zip varchar,"
+                + "w_country varchar,"
+                + "w_gmt_offset double";
+        assertEquals(expected, warehouseSchemaString);
+    }
+
+    @Test
+    public void testGetAllTableNames() {
+        List<String> tableNames = TableSchemaJSONLoader.getAllTableNames();
+        Collections.sort(tableNames);
+        List<String> expectedTableNames = Arrays.asList("call_center", "catalog_page", "catalog_returns", "catalog_sales", "customer", "customer_address", "customer_demographics",
+                "date_dim", "household_demographics", "income_band", "inventory", "item", "promotion", "reason", "ship_mode", "store", "store_returns", "store_sales", "time_dim",
+                "warehouse", "web_page", "web_returns", "web_sales", "web_site");
+
+        assertEquals(expectedTableNames.size(), tableNames.size());
+
+        for (int i = 0; i < tableNames.size(); i++) {
+            assertEquals(expectedTableNames.get(i), tableNames.get(i));
+        }
+    }
+}
diff --git a/sdks/java/testing/tpcds/src/test/java/org/apache/beam/sdk/tpcds/TpcdsParametersReaderTest.java b/sdks/java/testing/tpcds/src/test/java/org/apache/beam/sdk/tpcds/TpcdsParametersReaderTest.java
new file mode 100644
index 0000000..3f8c951
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/test/java/org/apache/beam/sdk/tpcds/TpcdsParametersReaderTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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.tpcds;
+
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TpcdsParametersReaderTest {
+    private TpcdsOptions tpcdsOptions;
+    private TpcdsOptions tpcdsOptionsError;
+
+    @Before
+    public void initializeTpcdsOptions() {
+        tpcdsOptions = PipelineOptionsFactory.as(TpcdsOptions.class);
+        tpcdsOptionsError = PipelineOptionsFactory.as(TpcdsOptions.class);
+
+        tpcdsOptions.setDataSize("1G");
+        tpcdsOptions.setQueries("1,2,3");
+        tpcdsOptions.setTpcParallel(2);
+
+        tpcdsOptionsError.setDataSize("5G");
+        tpcdsOptionsError.setQueries("0,100");
+        tpcdsOptionsError.setTpcParallel(0);
+    }
+
+    @Test
+    public void testGetAndCheckDataSize() throws Exception {
+        String dataSize = TpcdsParametersReader.getAndCheckDataSize(tpcdsOptions);
+        String expected = "1G";
+        assertEquals(expected, dataSize);
+    }
+
+    @Test( expected = Exception.class)
+    public void testGetAndCheckDataSizeException() throws Exception {
+        TpcdsParametersReader.getAndCheckDataSize(tpcdsOptionsError);
+    }
+
+    @Test
+    public void testGetAndCheckQueries() throws Exception {
+        TpcdsOptions tpcdsOptionsAll = PipelineOptionsFactory.as(TpcdsOptions.class);
+        tpcdsOptionsAll.setQueries("all");
+        String[] queryNameArray = TpcdsParametersReader.getAndCheckQueryNameArray(tpcdsOptionsAll);
+        String[] expected = new String[99];
+        for (int i = 0; i < 99; i++) {
+            expected[i] = "query" + (i + 1);
+        }
+        Assert.assertArrayEquals(expected, queryNameArray);
+    }
+
+    @Test
+    public void testGetAndCheckAllQueries() throws Exception {
+        String[] queryNameArray = TpcdsParametersReader.getAndCheckQueryNameArray(tpcdsOptions);
+        String[] expected = {"query1", "query2", "query3"};
+        Assert.assertArrayEquals(expected, queryNameArray);
+    }
+
+    @Test( expected = Exception.class)
+    public void testGetAndCheckQueriesException() throws Exception {
+        TpcdsParametersReader.getAndCheckQueryNameArray(tpcdsOptionsError);
+    }
+
+    @Test
+    public void testGetAndCheckTpcParallel() throws Exception {
+        int nThreads = TpcdsParametersReader.getAndCheckTpcParallel(tpcdsOptions);
+        int expected = 2;
+        assertEquals(expected, nThreads);
+    }
+
+    @Test( expected = Exception.class)
+    public void ttestGetAndCheckTpcParallelException() throws Exception {
+        TpcdsParametersReader.getAndCheckTpcParallel(tpcdsOptionsError);
+    }
+}
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/internal/pickler.py b/sdks/python/apache_beam/internal/pickler.py
index 9b10955..c4bfb44 100644
--- a/sdks/python/apache_beam/internal/pickler.py
+++ b/sdks/python/apache_beam/internal/pickler.py
@@ -190,16 +190,15 @@
       if obj_id not in known_module_dicts:
         # Trigger loading of lazily loaded modules (such as pytest vendored
         # modules).
-        # This first pass over sys.modules needs to iterate on a copy of
-        # sys.modules since lazy loading modifies the dictionary, hence the use
-        # of list().
+        # This pass over sys.modules needs to iterate on a copy of sys.modules
+        # since lazy loading modifies the dictionary, hence the use of list().
         for m in list(sys.modules.values()):
           try:
             _ = m.__dict__
           except AttributeError:
             pass
 
-        for m in sys.modules.values():
+        for m in list(sys.modules.values()):
           try:
             if (m and m.__name__ != '__main__' and
                 isinstance(m, dill.dill.ModuleType)):
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/io/gcp/big_query_query_to_table_it_test.py b/sdks/python/apache_beam/io/gcp/big_query_query_to_table_it_test.py
index 1f39928..c2dc3cd 100644
--- a/sdks/python/apache_beam/io/gcp/big_query_query_to_table_it_test.py
+++ b/sdks/python/apache_beam/io/gcp/big_query_query_to_table_it_test.py
@@ -174,7 +174,6 @@
         'use_standard_sql': False,
         'wait_until_finish_duration': WAIT_UNTIL_FINISH_DURATION_MS,
         'on_success_matcher': all_of(*pipeline_verifiers),
-        'experiments': 'use_beam_bq_sink',
     }
     options = self.test_pipeline.get_full_options_as_args(**extra_opts)
     big_query_query_to_table_pipeline.run_bq_pipeline(options)
@@ -198,7 +197,6 @@
         'use_standard_sql': True,
         'wait_until_finish_duration': WAIT_UNTIL_FINISH_DURATION_MS,
         'on_success_matcher': all_of(*pipeline_verifiers),
-        'experiments': 'use_beam_bq_sink',
     }
     options = self.test_pipeline.get_full_options_as_args(**extra_opts)
     big_query_query_to_table_pipeline.run_bq_pipeline(options)
@@ -227,6 +225,7 @@
         'on_success_matcher': all_of(*pipeline_verifiers),
         'kms_key': kms_key,
         'native': True,
+        'experiments': 'use_legacy_bq_sink',
     }
     options = self.test_pipeline.get_full_options_as_args(**extra_opts)
     big_query_query_to_table_pipeline.run_bq_pipeline(options)
@@ -281,7 +280,6 @@
         'use_standard_sql': False,
         'wait_until_finish_duration': WAIT_UNTIL_FINISH_DURATION_MS,
         'on_success_matcher': all_of(*pipeline_verifiers),
-        'experiments': 'use_beam_bq_sink',
     }
     options = self.test_pipeline.get_full_options_as_args(**extra_opts)
     big_query_query_to_table_pipeline.run_bq_pipeline(options)
@@ -305,7 +303,8 @@
         'use_standard_sql': False,
         'native': True,
         'wait_until_finish_duration': WAIT_UNTIL_FINISH_DURATION_MS,
-        'on_success_matcher': all_of(*pipeline_verifiers)
+        'on_success_matcher': all_of(*pipeline_verifiers),
+        'experiments': 'use_legacy_bq_sink',
     }
     options = self.test_pipeline.get_full_options_as_args(**extra_opts)
     big_query_query_to_table_pipeline.run_bq_pipeline(options)
diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py
index 2656a84..d011394 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery.py
@@ -1488,9 +1488,7 @@
   def _compute_method(self, experiments, is_streaming_pipeline):
     # If the new BQ sink is not activated for experiment flags, then we use
     # streaming inserts by default (it gets overridden in dataflow_runner.py).
-    if 'use_beam_bq_sink' not in experiments:
-      return self.Method.STREAMING_INSERTS
-    elif self.method == self.Method.DEFAULT and is_streaming_pipeline:
+    if self.method == self.Method.DEFAULT and is_streaming_pipeline:
       return self.Method.STREAMING_INSERTS
     elif self.method == self.Method.DEFAULT and not is_streaming_pipeline:
       return self.Method.FILE_LOADS
diff --git a/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py b/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py
index f9e0212..fca7d9c 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py
@@ -673,8 +673,7 @@
     ]
 
     args = self.test_pipeline.get_full_options_as_args(
-        on_success_matcher=all_of(*pipeline_verifiers),
-        experiments='use_beam_bq_sink')
+        on_success_matcher=all_of(*pipeline_verifiers))
 
     with beam.Pipeline(argv=args) as p:
       input = p | beam.Create(_ELEMENTS, reshuffle=False)
@@ -733,9 +732,7 @@
         data=[(i, ) for i in range(100)])
 
     args = self.test_pipeline.get_full_options_as_args(
-        on_success_matcher=all_of(state_matcher, bq_matcher),
-        experiments='use_beam_bq_sink',
-        streaming=True)
+        on_success_matcher=all_of(state_matcher, bq_matcher), streaming=True)
     with beam.Pipeline(argv=args) as p:
       stream_source = (
           TestStream().advance_watermark_to(0).advance_processing_time(
@@ -790,8 +787,7 @@
             data=[])
     ]
 
-    args = self.test_pipeline.get_full_options_as_args(
-        experiments='use_beam_bq_sink')
+    args = self.test_pipeline.get_full_options_as_args()
 
     with self.assertRaises(Exception):
       # The pipeline below fails because neither a schema nor SCHEMA_AUTODETECT
diff --git a/sdks/python/apache_beam/io/gcp/bigquery_test.py b/sdks/python/apache_beam/io/gcp/bigquery_test.py
index 5c05978..b341dc6 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery_test.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery_test.py
@@ -563,8 +563,7 @@
     self.assertEqual(expected_dict_schema, dict_schema)
 
   def test_schema_autodetect_not_allowed_with_avro_file_loads(self):
-    with TestPipeline(
-        additional_pipeline_args=["--experiments=use_beam_bq_sink"]) as p:
+    with TestPipeline() as p:
       pc = p | beam.Impulse()
 
       with self.assertRaisesRegex(ValueError, '^A schema must be provided'):
@@ -593,8 +592,7 @@
     """
     FULL_OUTPUT_TABLE = 'test_project:output_table'
 
-    p = TestPipeline(
-        additional_pipeline_args=["--experiments=use_beam_bq_sink"])
+    p = TestPipeline()
 
     # Used for testing side input parameters.
     table_record_pcv = beam.pvalue.AsDict(
@@ -886,8 +884,7 @@
     ]
 
     args = self.test_pipeline.get_full_options_as_args(
-        on_success_matcher=hc.all_of(*pipeline_verifiers),
-        experiments='use_beam_bq_sink')
+        on_success_matcher=hc.all_of(*pipeline_verifiers))
 
     with beam.Pipeline(argv=args) as p:
       input = p | beam.Create([row for row in _ELEMENTS if 'language' in row])
@@ -967,8 +964,7 @@
       ]
 
     args = self.test_pipeline.get_full_options_as_args(
-        on_success_matcher=hc.all_of(*pipeline_verifiers),
-        experiments='use_beam_bq_sink')
+        on_success_matcher=hc.all_of(*pipeline_verifiers))
 
     with beam.Pipeline(argv=args) as p:
       if streaming:
@@ -1083,7 +1079,6 @@
     args = self.test_pipeline.get_full_options_as_args(
         on_success_matcher=hc.all_of(*matchers),
         wait_until_finish_duration=self.WAIT_UNTIL_FINISH_DURATION,
-        experiments='use_beam_bq_sink',
         streaming=True)
 
     def add_schema_info(element):
@@ -1182,7 +1177,6 @@
 
     args = self.test_pipeline.get_full_options_as_args(
         on_success_matcher=hc.all_of(*pipeline_verifiers),
-        experiments='use_beam_bq_sink',
     )
 
     with beam.Pipeline(argv=args) as p:
diff --git a/sdks/python/apache_beam/io/gcp/bigquery_write_it_test.py b/sdks/python/apache_beam/io/gcp/bigquery_write_it_test.py
index 9ea75ea..a5c1ce7 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery_write_it_test.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery_write_it_test.py
@@ -197,8 +197,7 @@
     ]
 
     args = self.test_pipeline.get_full_options_as_args(
-        on_success_matcher=hc.all_of(*pipeline_verifiers),
-        experiments='use_beam_bq_sink')
+        on_success_matcher=hc.all_of(*pipeline_verifiers))
 
     with beam.Pipeline(argv=args) as p:
       # pylint: disable=expression-not-assigned
diff --git a/sdks/python/apache_beam/io/gcp/dicomclient.py b/sdks/python/apache_beam/io/gcp/dicomclient.py
new file mode 100644
index 0000000..e38a310
--- /dev/null
+++ b/sdks/python/apache_beam/io/gcp/dicomclient.py
@@ -0,0 +1,128 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# pytype: skip-file
+
+from __future__ import absolute_import
+
+from google.auth import default
+from google.auth.transport import requests
+
+
+class DicomApiHttpClient:
+  """DICOM api client that talk to api via http request"""
+  healthcare_base_url = "https://healthcare.googleapis.com/v1"
+  session = None
+
+  def get_session(self, credential):
+    if self.session:
+      return self.session
+
+    # if the credential is not provided, use the default credential.
+    if not credential:
+      credential, _ = default()
+    new_seesion = requests.AuthorizedSession(credential)
+    self.session = new_seesion
+    return new_seesion
+
+  def qido_search(
+      self,
+      project_id,
+      region,
+      dataset_id,
+      dicom_store_id,
+      search_type,
+      params=None,
+      credential=None):
+    """function for searching a DICOM store"""
+
+    # sending request to the REST healthcare api.
+    api_endpoint = "{}/projects/{}/locations/{}".format(
+        self.healthcare_base_url, project_id, region)
+
+    # base of dicomweb path.
+    dicomweb_path = "{}/datasets/{}/dicomStores/{}/dicomWeb/{}".format(
+        api_endpoint, dataset_id, dicom_store_id, search_type)
+
+    # Make an authenticated API request
+    session = self.get_session(credential)
+    headers = {"Content-Type": "application/dicom+json; charset=utf-8"}
+    page_size = 500
+
+    if params and 'limit' in params:
+      page_size = params['limit']
+    elif params:
+      params['limit'] = page_size
+    else:
+      params = {'limit': page_size}
+
+    offset = 0
+    output = []
+    # iterate to get all the results
+    while True:
+      params['offset'] = offset
+      response = session.get(dicomweb_path, headers=headers, params=params)
+      response.raise_for_status()
+      status = response.status_code
+      if status != 200:
+        if offset == 0:
+          return [], status
+        params['offset'] = offset - 1
+        params['limit'] = 1
+        response = session.get(dicomweb_path, headers=headers, params=params)
+        response.raise_for_status()
+        check_status = response.status_code
+        if check_status == 200:
+          # if the number of results equals to page size
+          return output, check_status
+        else:
+          # something wrong with the request or server
+          return [], status
+      results = response.json()
+      output += results
+      if len(results) < page_size:
+        # got all the results, return
+        break
+      offset += len(results)
+
+    return output, status
+
+  def dicomweb_store_instance(
+      self,
+      project_id,
+      region,
+      dataset_id,
+      dicom_store_id,
+      dcm_file,
+      credential=None):
+    """function for storing an instance."""
+
+    api_endpoint = "{}/projects/{}/locations/{}".format(
+        self.healthcare_base_url, project_id, region)
+
+    dicomweb_path = "{}/datasets/{}/dicomStores/{}/dicomWeb/studies".format(
+        api_endpoint, dataset_id, dicom_store_id)
+
+    # Make an authenticated API request
+    session = self.get_session(credential)
+    content_type = "application/dicom"
+    headers = {"Content-Type": content_type}
+
+    response = session.post(dicomweb_path, data=dcm_file, headers=headers)
+    response.raise_for_status()
+
+    return None, response.status_code
diff --git a/sdks/python/apache_beam/io/gcp/dicomio.py b/sdks/python/apache_beam/io/gcp/dicomio.py
new file mode 100644
index 0000000..e33d99d
--- /dev/null
+++ b/sdks/python/apache_beam/io/gcp/dicomio.py
@@ -0,0 +1,579 @@
+#
+# 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.
+#
+
+"""DICOM IO connector
+This module implements several tools to facilitate the interaction between
+a Google Cloud Healthcare DICOM store and a Beam pipeline.
+
+For more details on DICOM store and API:
+https://cloud.google.com/healthcare/docs/how-tos/dicom
+
+The DICOM IO connector can be used to search metadata or write DICOM files
+to DICOM store.
+
+When used together with Google Pubsub message connector, the
+`FormatToQido` PTransform implemented in this module can be used
+to convert Pubsub messages to search requests.
+
+Since Traceability is crucial for healthcare
+API users, every input or error message will be recorded in the output of
+the DICOM IO connector. As a result, every PTransform in this module will
+return a PCollection of dict that encodes results and detailed error messages.
+
+Search instance's metadata (QIDO request)
+===================================================
+DicomSearch() wraps the QIDO request client and supports 3 levels of search.
+Users should specify the level by setting the 'search_type' entry in the input
+dict. They can also refine the search by adding tags to filter the results using
+the 'params' entry. Here is a sample usage:
+
+  with Pipeline() as p:
+    input_dict = p | beam.Create(
+      [{'project_id': 'abc123', 'type': 'instances',...},
+      {'project_id': 'dicom_go', 'type': 'series',...}])
+
+    results = input_dict | io.gcp.DicomSearch()
+    results | 'print successful search' >> beam.Map(
+    lambda x: print(x['result'] if x['success'] else None))
+
+    results | 'print failed search' >> beam.Map(
+    lambda x: print(x['result'] if not x['success'] else None))
+
+In the example above, successful qido search results and error messages for
+failed requests are printed. When used in real life, user can choose to filter
+those data and output them to wherever they want.
+
+Convert DICOM Pubsub message to Qido search request
+===================================================
+Healthcare API users might read messages from Pubsub to monitor the store
+operations (e.g. new file) in a DICOM storage. Pubsub message encode
+DICOM as a web store path as well as instance ids. If users are interested in
+getting new instance's metadata, they can use the `FormatToQido` transform
+to convert the message into Qido Search dict then use the `DicomSearch`
+transform. Here is a sample usage:
+
+  pipeline_options = PipelineOptions()
+  pipeline_options.view_as(StandardOptions).streaming = True
+  p =  beam.Pipeline(options=pipeline_options)
+  pubsub = p | beam.io.ReadStringFromPubsub(subscription='a_dicom_store')
+  results = pubsub | FormatToQido()
+  success = results | 'filter message' >> beam.Filter(lambda x: x['success'])
+  qido_dict = success | 'get qido request' >> beam.Map(lambda x: x['result'])
+  metadata = qido_dict | DicomSearch()
+
+In the example above, the pipeline is listening to a pubsub topic and waiting
+for messages from DICOM API. When a new DICOM file comes into the storage, the
+pipeline will receive a pubsub message, convert it to a Qido request dict and
+feed it to DicomSearch() PTransform. As a result, users can get the metadata for
+every new DICOM file. Note that not every pubsub message received is from DICOM
+API, so we to filter the results first.
+
+Store a DICOM file in a DICOM storage
+===================================================
+UploadToDicomStore() wraps store request API and users can use it to send a
+DICOM file to a DICOM store. It supports two types of input: 1.file data in
+byte[] 2.fileio object. Users should set the 'input_type' when initialzing
+this PTransform. Here are the examples:
+
+  with Pipeline() as p:
+    input_dict = {'project_id': 'abc123', 'type': 'instances',...}
+    path = "gcs://bucketname/something/a.dcm"
+    match = p | fileio.MatchFiles(path)
+    fileio_obj = match | fileio.ReadAll()
+    results = fileio_obj | UploadToDicomStore(input_dict, 'fileio')
+
+  with Pipeline() as p:
+    input_dict = {'project_id': 'abc123', 'type': 'instances',...}
+    f = open("abc.dcm", "rb")
+    dcm_file = f.read()
+    byte_file = p | 'create byte file' >> beam.Create([dcm_file])
+    results = byte_file | UploadToDicomStore(input_dict, 'bytes')
+
+The first example uses a PCollection of fileio objects as input.
+UploadToDicomStore will read DICOM files from the objects and send them
+to a DICOM storage.
+The second example uses a PCollection of byte[] as input. UploadToDicomStore
+will directly send those DICOM files to a DICOM storage.
+Users can also get the operation results in the output PCollection if they want
+to handle the failed store requests.
+"""
+
+# pytype: skip-file
+from __future__ import absolute_import
+
+from concurrent.futures import ThreadPoolExecutor
+from concurrent.futures import as_completed
+
+import apache_beam as beam
+from apache_beam.io.gcp.dicomclient import DicomApiHttpClient
+from apache_beam.transforms import PTransform
+
+
+class DicomSearch(PTransform):
+  """A PTransform used for retrieving DICOM instance metadata from Google
+    Cloud DICOM store. It takes a PCollection of dicts as input and return
+    a PCollection of dict as results:
+    INPUT:
+    The input dict represents DICOM web path parameters, which has the following
+    string keys and values:
+    {
+    'project_id': str,
+    'region': str,
+    'dataset_id': str,
+    'dicom_store_id': str,
+    'search_type': str,
+    'params': dict(str,str) (Optional),
+    }
+
+    Key-value pairs:
+      project_id: Id of the project in which the DICOM store is
+      located. (Required)
+      region: Region where the DICOM store resides. (Required)
+      dataset_id: Id of the dataset where DICOM store belongs to. (Required)
+      dicom_store_id: Id of the dicom store. (Required)
+      search_type: Which type of search it is, could only be one of the three
+      values: 'instances', 'series', or 'studies'. (Required)
+      params: A dict of str:str pairs used to refine QIDO search. (Optional)
+      Supported tags in three categories:
+      1.Studies:
+      * StudyInstanceUID,
+      * PatientName,
+      * PatientID,
+      * AccessionNumber,
+      * ReferringPhysicianName,
+      * StudyDate,
+      2.Series: all study level search terms and
+      * SeriesInstanceUID,
+      * Modality,
+      3.Instances: all study/series level search terms and
+      * SOPInstanceUID,
+
+      e.g. {"StudyInstanceUID":"1","SeriesInstanceUID":"2"}
+
+    OUTPUT:
+    The output dict wraps results as well as error messages:
+    {
+    'result': a list of dicts in JSON style.
+    'success': boolean value telling whether the operation is successful.
+    'input': detail ids and dicomweb path for this retrieval.
+    'status': status code from the server, used as error message.
+    }
+
+  """
+  def __init__(
+      self, buffer_size=8, max_workers=5, client=None, credential=None):
+    """Initializes DicomSearch.
+    Args:
+      buffer_size: # type: Int. Size of the request buffer.
+      max_workers: # type: Int. Maximum number of threads a worker can
+      create. If it is set to one, all the request will be processed
+      sequentially in a worker.
+      client: # type: object. If it is specified, all the Api calls will
+      made by this client instead of the default one (DicomApiHttpClient).
+      credential: # type: Google credential object, if it is specified, the
+      Http client will use it to create sessions instead of the default.
+    """
+    self.buffer_size = buffer_size
+    self.max_workers = max_workers
+    self.client = client or DicomApiHttpClient()
+    self.credential = credential
+
+  def expand(self, pcoll):
+    return pcoll | beam.ParDo(
+        _QidoReadFn(
+            self.buffer_size, self.max_workers, self.client, self.credential))
+
+
+class _QidoReadFn(beam.DoFn):
+  """A DoFn for executing every qido query request."""
+  def __init__(self, buffer_size, max_workers, client, credential=None):
+    self.buffer_size = buffer_size
+    self.max_workers = max_workers
+    self.client = client
+    self.credential = credential
+
+  def start_bundle(self):
+    self.buffer = []
+
+  def finish_bundle(self):
+    for item in self._flush():
+      yield item
+
+  def validate_element(self, element):
+    # Check if all required keys present.
+    required_keys = [
+        'project_id', 'region', 'dataset_id', 'dicom_store_id', 'search_type'
+    ]
+
+    for key in required_keys:
+      if key not in element:
+        error_message = 'Must have %s in the dict.' % (key)
+        return False, error_message
+
+    # Check if return type is correct.
+    if element['search_type'] in ['instances', "studies", "series"]:
+      return True, None
+    else:
+      error_message = (
+          'Search type can only be "studies", '
+          '"instances" or "series"')
+      return False, error_message
+
+  def process(
+      self,
+      element,
+      window=beam.DoFn.WindowParam,
+      timestamp=beam.DoFn.TimestampParam):
+    # Check if the element is valid
+    valid, error_message = self.validate_element(element)
+
+    if valid:
+      self.buffer.append((element, window, timestamp))
+      if len(self.buffer) >= self.buffer_size:
+        for item in self._flush():
+          yield item
+    else:
+      # Return this when the input dict dose not meet the requirements
+      out = {}
+      out['result'] = []
+      out['status'] = error_message
+      out['input'] = element
+      out['success'] = False
+      yield out
+
+  def make_request(self, element):
+    # Sending Qido request to DICOM Api
+    project_id = element['project_id']
+    region = element['region']
+    dataset_id = element['dataset_id']
+    dicom_store_id = element['dicom_store_id']
+    search_type = element['search_type']
+    params = element['params'] if 'params' in element else None
+
+    # Call qido search http client
+    result, status_code = self.client.qido_search(
+      project_id, region, dataset_id, dicom_store_id,
+      search_type, params, self.credential
+    )
+
+    out = {}
+    out['result'] = result
+    out['status'] = status_code
+    out['input'] = element
+    out['success'] = (status_code == 200)
+    return out
+
+  def process_buffer_element(self, buffer_element):
+    # Thread job runner - each thread makes a Qido search request
+    value = self.make_request(buffer_element[0])
+    windows = [buffer_element[1]]
+    timestamp = buffer_element[2]
+    return beam.utils.windowed_value.WindowedValue(
+        value=value, timestamp=timestamp, windows=windows)
+
+  def _flush(self):
+    # Create thread pool executor and process the buffered elements in paralllel
+    executor = ThreadPoolExecutor(max_workers=self.max_workers)
+    futures = [
+        executor.submit(self.process_buffer_element, ele) for ele in self.buffer
+    ]
+    self.buffer = []
+    for f in as_completed(futures):
+      yield f.result()
+
+
+class FormatToQido(PTransform):
+  """A PTransform for converting pubsub messages into search input dict.
+    Takes PCollection of string as input and returns a PCollection of dict as
+    results. Note that some pubsub messages may not be from DICOM API, which
+    will be recorded as failed conversions.
+    INPUT:
+    The input are normally strings from Pubsub topic:
+    "projects/PROJECT_ID/locations/LOCATION/datasets/DATASET_ID/
+    dicomStores/DICOM_STORE_ID/dicomWeb/studies/STUDY_UID/
+    series/SERIES_UID/instances/INSTANCE_UID"
+
+    OUTPUT:
+    The output dict encodes results as well as error messages:
+    {
+    'result': a dict representing instance level qido search request.
+    'success': boolean value telling whether the conversion is successful.
+    'input': input pubsub message string.
+    }
+
+  """
+  def __init__(self, credential=None):
+    """Initializes FormatToQido.
+    Args:
+      credential: # type: Google credential object, if it is specified, the
+      Http client will use it instead of the default one.
+    """
+    self.credential = credential
+
+  def expand(self, pcoll):
+    return pcoll | beam.ParDo(_ConvertStringToQido())
+
+
+class _ConvertStringToQido(beam.DoFn):
+  """A DoFn for converting pubsub string to qido search parameters."""
+  def process(self, element):
+    # Some constants for DICOM pubsub message
+    NUM_PUBSUB_STR_ENTRIES = 15
+    NUM_DICOM_WEBPATH_PARAMETERS = 5
+    NUM_TOTAL_PARAMETERS = 8
+    INDEX_PROJECT_ID = 1
+    INDEX_REGION = 3
+    INDEX_DATASET_ID = 5
+    INDEX_DICOMSTORE_ID = 7
+    INDEX_STUDY_ID = 10
+    INDEX_SERIE_ID = 12
+    INDEX_INSTANCE_ID = 14
+
+    entries = element.split('/')
+
+    # Output dict with error message, used when
+    # receiving invalid pubsub string.
+    error_dict = {}
+    error_dict['result'] = {}
+    error_dict['input'] = element
+    error_dict['success'] = False
+
+    if len(entries) != NUM_PUBSUB_STR_ENTRIES:
+      return [error_dict]
+
+    required_keys = [
+        'projects',
+        'locations',
+        'datasets',
+        'dicomStores',
+        'dicomWeb',
+        'studies',
+        'series',
+        'instances'
+    ]
+
+    # Check if the required keys present and
+    # the positions of those keys are correct
+    for i in range(NUM_DICOM_WEBPATH_PARAMETERS):
+      if required_keys[i] != entries[i * 2]:
+        return [error_dict]
+    for i in range(NUM_DICOM_WEBPATH_PARAMETERS, NUM_TOTAL_PARAMETERS):
+      if required_keys[i] != entries[i * 2 - 1]:
+        return [error_dict]
+
+    # Compose dicom webpath parameters for qido search
+    qido_dict = {}
+    qido_dict['project_id'] = entries[INDEX_PROJECT_ID]
+    qido_dict['region'] = entries[INDEX_REGION]
+    qido_dict['dataset_id'] = entries[INDEX_DATASET_ID]
+    qido_dict['dicom_store_id'] = entries[INDEX_DICOMSTORE_ID]
+    qido_dict['search_type'] = 'instances'
+
+    # Compose instance level params for qido search
+    params = {}
+    params['StudyInstanceUID'] = entries[INDEX_STUDY_ID]
+    params['SeriesInstanceUID'] = entries[INDEX_SERIE_ID]
+    params['SOPInstanceUID'] = entries[INDEX_INSTANCE_ID]
+    qido_dict['params'] = params
+
+    out = {}
+    out['result'] = qido_dict
+    out['input'] = element
+    out['success'] = True
+
+    return [out]
+
+
+class UploadToDicomStore(PTransform):
+  """A PTransform for storing instances to a DICOM store.
+    Takes PCollection of byte[] as input and return a PCollection of dict as
+    results. The inputs are normally DICOM file in bytes or str filename.
+    INPUT:
+    This PTransform supports two types of input:
+    1. Byte[]: representing dicom file.
+    2. Fileio object: stream file object.
+
+    OUTPUT:
+    The output dict encodes status as well as error messages:
+    {
+    'success': boolean value telling whether the store is successful.
+    'input': undeliverable data. Exactly the same as the input,
+    only set if the operation is failed.
+    'status': status code from the server, used as error messages.
+    }
+
+  """
+  def __init__(
+      self,
+      destination_dict,
+      input_type,
+      buffer_size=8,
+      max_workers=5,
+      client=None,
+      credential=None):
+    """Initializes UploadToDicomStore.
+    Args:
+      destination_dict: # type: python dict, encodes DICOM endpoint information:
+      {
+      'project_id': str,
+      'region': str,
+      'dataset_id': str,
+      'dicom_store_id': str,
+      }
+
+      Key-value pairs:
+      * project_id: Id of the project in which DICOM store locates. (Required)
+      * region: Region where the DICOM store resides. (Required)
+      * dataset_id: Id of the dataset where DICOM store belongs to. (Required)
+      * dicom_store_id: Id of the dicom store. (Required)
+
+      input_type: # type: string, could only be 'bytes' or 'fileio'
+      buffer_size: # type: Int. Size of the request buffer.
+      max_workers: # type: Int. Maximum number of threads a worker can
+      create. If it is set to one, all the request will be processed
+      sequentially in a worker.
+      client: # type: object. If it is specified, all the Api calls will
+      made by this client instead of the default one (DicomApiHttpClient).
+      credential: # type: Google credential object, if it is specified, the
+      Http client will use it instead of the default one.
+    """
+    self.destination_dict = destination_dict
+    # input_type pre-check
+    if input_type not in ['bytes', 'fileio']:
+      raise ValueError("input_type could only be 'bytes' or 'fileio'")
+    self.input_type = input_type
+    self.buffer_size = buffer_size
+    self.max_workers = max_workers
+    self.client = client
+    self.credential = credential
+
+  def expand(self, pcoll):
+    return pcoll | beam.ParDo(
+        _StoreInstance(
+            self.destination_dict,
+            self.input_type,
+            self.buffer_size,
+            self.max_workers,
+            self.client,
+            self.credential))
+
+
+class _StoreInstance(beam.DoFn):
+  """A DoFn read or fetch dicom files then push it to a dicom store."""
+  def __init__(
+      self,
+      destination_dict,
+      input_type,
+      buffer_size,
+      max_workers,
+      client,
+      credential=None):
+    # pre-check destination dict
+    required_keys = ['project_id', 'region', 'dataset_id', 'dicom_store_id']
+    for key in required_keys:
+      if key not in destination_dict:
+        raise ValueError('Must have %s in the dict.' % (key))
+    self.destination_dict = destination_dict
+    self.input_type = input_type
+    self.buffer_size = buffer_size
+    self.max_workers = max_workers
+    self.client = client
+    self.credential = credential
+
+  def start_bundle(self):
+    self.buffer = []
+
+  def finish_bundle(self):
+    for item in self._flush():
+      yield item
+
+  def process(
+      self,
+      element,
+      window=beam.DoFn.WindowParam,
+      timestamp=beam.DoFn.TimestampParam):
+    self.buffer.append((element, window, timestamp))
+    if len(self.buffer) >= self.buffer_size:
+      for item in self._flush():
+        yield item
+
+  def make_request(self, dicom_file):
+    # Send file to DICOM store and records the results.
+    project_id = self.destination_dict['project_id']
+    region = self.destination_dict['region']
+    dataset_id = self.destination_dict['dataset_id']
+    dicom_store_id = self.destination_dict['dicom_store_id']
+
+    # Feed the dicom file into store client
+    if self.client:
+      _, status_code = self.client.dicomweb_store_instance(
+        project_id, region, dataset_id, dicom_store_id, dicom_file,
+        self.credential
+      )
+    else:
+      _, status_code = DicomApiHttpClient().dicomweb_store_instance(
+        project_id, region, dataset_id, dicom_store_id, dicom_file,
+        self.credential
+      )
+
+    out = {}
+    out['status'] = status_code
+    out['success'] = (status_code == 200)
+    return out
+
+  def read_dicom_file(self, buffer_element):
+    # Read the file based on different input. If the read fails ,return
+    # an error dict which records input and error messages.
+    try:
+      if self.input_type == 'fileio':
+        f = buffer_element.open()
+        data = f.read()
+        f.close()
+        return True, data
+      else:
+        return True, buffer_element
+    except Exception as error_message:
+      error_out = {}
+      error_out['status'] = error_message
+      error_out['success'] = False
+      return False, error_out
+
+  def process_buffer_element(self, buffer_element):
+    # Thread job runner - each thread stores a DICOM file
+    success, read_result = self.read_dicom_file(buffer_element[0])
+    windows = [buffer_element[1]]
+    timestamp = buffer_element[2]
+    value = None
+    if success:
+      value = self.make_request(read_result)
+    else:
+      value = read_result
+    # save the undeliverable data
+    if not value['success']:
+      value['input'] = buffer_element[0]
+    return beam.utils.windowed_value.WindowedValue(
+        value=value, timestamp=timestamp, windows=windows)
+
+  def _flush(self):
+    # Create thread pool executor and process the buffered elements in paralllel
+    executor = ThreadPoolExecutor(max_workers=self.max_workers)
+    futures = [
+        executor.submit(self.process_buffer_element, ele) for ele in self.buffer
+    ]
+    self.buffer = []
+    for f in as_completed(futures):
+      yield f.result()
diff --git a/sdks/python/apache_beam/io/gcp/dicomio_test.py b/sdks/python/apache_beam/io/gcp/dicomio_test.py
new file mode 100644
index 0000000..2594e45
--- /dev/null
+++ b/sdks/python/apache_beam/io/gcp/dicomio_test.py
@@ -0,0 +1,468 @@
+# coding=utf-8
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""Unit tests for Dicom IO connectors."""
+
+# pytype: skip-file
+
+from __future__ import absolute_import
+
+import json
+import os
+import unittest
+
+# patches unittest.TestCase to be python3 compatible
+import future.tests.base  # pylint: disable=unused-import
+from mock import patch
+
+import apache_beam as beam
+from apache_beam.io import fileio
+from apache_beam.io.filebasedsink_test import _TestCaseWithTempDirCleanUp
+from apache_beam.io.filesystems import FileSystems
+from apache_beam.testing.test_pipeline import TestPipeline
+from apache_beam.testing.util import assert_that
+from apache_beam.testing.util import equal_to
+
+# Protect against environments where gcp library is not available.
+# pylint: disable=wrong-import-order, wrong-import-position
+try:
+  from apache_beam.io.gcp.dicomio import DicomSearch
+  from apache_beam.io.gcp.dicomio import FormatToQido
+  from apache_beam.io.gcp.dicomio import UploadToDicomStore
+except ImportError:
+  DicomSearch = None  # type: ignore
+# pylint: enable=wrong-import-order, wrong-import-position
+
+
+class FakeHttpClient():
+  # a fake http client that talks directly to a in-memory dicom store
+  def __init__(self):
+    # set 5 fake dicom instances
+    dicom_metadata = []
+    dicom_metadata.append({
+        'PatientName': 'Albert', 'Age': 21, 'TestResult': 'Positive'
+    })
+    dicom_metadata.append({
+        'PatientName': 'Albert', 'Age': 21, 'TestResult': 'Negative'
+    })
+    dicom_metadata.append({
+        'PatientName': 'Brian', 'Age': 20, 'TestResult': 'Positive'
+    })
+    dicom_metadata.append({
+        'PatientName': 'Colin', 'Age': 25, 'TestResult': 'Negative'
+    })
+    dicom_metadata.append({
+        'PatientName': 'Daniel', 'Age': 22, 'TestResult': 'Negative'
+    })
+    dicom_metadata.append({
+        'PatientName': 'Eric', 'Age': 50, 'TestResult': 'Negative'
+    })
+    self.dicom_metadata = dicom_metadata
+    # ids for this dicom sotre
+    self.project_id = 'test_project'
+    self.region = 'test_region'
+    self.dataset_id = 'test_dataset_id'
+    self.dicom_store_id = 'test_dicom_store_id'
+
+  def qido_search(
+      self,
+      project_id,
+      region,
+      dataset_id,
+      dicom_store_id,
+      search_type,
+      params=None,
+      credential=None):
+    # qido search function for this fake client
+    if project_id != self.project_id or region != self.region or \
+     dataset_id != self.dataset_id or dicom_store_id != self.dicom_store_id:
+      return [], 204
+    # only supports all instance search in this client
+    if not params:
+      return self.dicom_metadata, 200
+    # only supports patient name filter in this client
+    patient_name = params['PatientName']
+    out = []
+    for meta in self.dicom_metadata:
+      if meta['PatientName'] == patient_name:
+        out.append(meta)
+    return out, 200
+
+  def dicomweb_store_instance(
+      self,
+      project_id,
+      region,
+      dataset_id,
+      dicom_store_id,
+      dcm_file,
+      credential=None):
+    if project_id != self.project_id or region != self.region or \
+     dataset_id != self.dataset_id or dicom_store_id != self.dicom_store_id:
+      return [], 204
+    # convert the bytes file back to dict
+    string_array = dcm_file.decode('utf-8')
+    metadata_dict = json.loads(string_array)
+    self.dicom_metadata.append(metadata_dict)
+    return None, 200
+
+
+@unittest.skipIf(DicomSearch is None, 'GCP dependencies are not installed')
+class TestFormatToQido(unittest.TestCase):
+  valid_pubsub_string = (
+      "projects/PROJECT_ID/locations/LOCATION/datasets"
+      "/DATASET_ID/dicomStores/DICOM_STORE_ID/dicomWeb/"
+      "studies/STUDY_UID/series/SERIES_UID/instances/INSTANCE_UID")
+  expected_valid_pubsub_dict = {
+      'result': {
+          'project_id': 'PROJECT_ID',
+          'region': 'LOCATION',
+          'dataset_id': 'DATASET_ID',
+          'dicom_store_id': 'DICOM_STORE_ID',
+          'search_type': 'instances',
+          'params': {
+              'StudyInstanceUID': 'STUDY_UID',
+              'SeriesInstanceUID': 'SERIES_UID',
+              'SOPInstanceUID': 'INSTANCE_UID'
+          }
+      },
+      'input': valid_pubsub_string,
+      'success': True
+  }
+  invalid_pubsub_string = "this is not a valid pubsub message"
+  expected_invalid_pubsub_dict = {
+      'result': {},
+      'input': 'this is not a valid pubsub message',
+      'success': False
+  }
+
+  def test_normal_convert(self):
+    with TestPipeline() as p:
+      convert_result = (
+          p
+          | beam.Create([self.valid_pubsub_string])
+          | FormatToQido())
+      assert_that(convert_result, equal_to([self.expected_valid_pubsub_dict]))
+
+  def test_failed_convert(self):
+    with TestPipeline() as p:
+      convert_result = (
+          p
+          | beam.Create([self.invalid_pubsub_string])
+          | FormatToQido())
+      assert_that(convert_result, equal_to([self.expected_invalid_pubsub_dict]))
+
+
+@unittest.skipIf(DicomSearch is None, 'GCP dependencies are not installed')
+class TestDicomSearch(unittest.TestCase):
+  @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+  def test_successful_search(self, FakeClient):
+    input_dict = {}
+    input_dict['project_id'] = "test_project"
+    input_dict['region'] = "test_region"
+    input_dict['dataset_id'] = "test_dataset_id"
+    input_dict['dicom_store_id'] = "test_dicom_store_id"
+    input_dict['search_type'] = "instances"
+
+    fc = FakeHttpClient()
+    FakeClient.return_value = fc
+
+    expected_dict = {}
+    expected_dict['result'] = fc.dicom_metadata
+    expected_dict['status'] = 200
+    expected_dict['input'] = input_dict
+    expected_dict['success'] = True
+
+    with TestPipeline() as p:
+      results = (p | beam.Create([input_dict]) | DicomSearch())
+      assert_that(results, equal_to([expected_dict]))
+
+  @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+  def test_Qido_search_small_buffer_flush(self, FakeClient):
+    input_dict = {}
+    input_dict['project_id'] = "test_project"
+    input_dict['region'] = "test_region"
+    input_dict['dataset_id'] = "test_dataset_id"
+    input_dict['dicom_store_id'] = "test_dicom_store_id"
+    input_dict['search_type'] = "instances"
+
+    fc = FakeHttpClient()
+    FakeClient.return_value = fc
+
+    expected_dict = {}
+    expected_dict['result'] = fc.dicom_metadata
+    expected_dict['status'] = 200
+    expected_dict['input'] = input_dict
+    expected_dict['success'] = True
+
+    with TestPipeline() as p:
+      results = (p | beam.Create([input_dict] * 5) | DicomSearch(buffer_size=1))
+      assert_that(results, equal_to([expected_dict] * 5))
+
+  @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+  def test_param_dict_passing(self, FakeClient):
+    input_dict = {}
+    input_dict = {}
+    input_dict['project_id'] = "test_project"
+    input_dict['region'] = "test_region"
+    input_dict['dataset_id'] = "test_dataset_id"
+    input_dict['dicom_store_id'] = "test_dicom_store_id"
+    input_dict['search_type'] = "instances"
+    input_dict['params'] = {'PatientName': 'Brian'}
+
+    expected_dict = {}
+    expected_dict['result'] = [{
+        'PatientName': 'Brian', 'Age': 20, 'TestResult': 'Positive'
+    }]
+    expected_dict['status'] = 200
+    expected_dict['input'] = input_dict
+    expected_dict['success'] = True
+
+    fc = FakeHttpClient()
+    FakeClient.return_value = fc
+    with TestPipeline() as p:
+      results = (p | beam.Create([input_dict]) | DicomSearch())
+      assert_that(results, equal_to([expected_dict]))
+
+  @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+  def test_wrong_input_type(self, FakeClient):
+    input_dict = {}
+    input_dict['project_id'] = "test_project"
+    input_dict['region'] = "test_region"
+    input_dict['dataset_id'] = "test_dataset_id"
+    input_dict['dicom_store_id'] = "test_dicom_store_id"
+    input_dict['search_type'] = "not exist type"
+
+    expected_invalid_dict = {}
+    expected_invalid_dict['result'] = []
+    expected_invalid_dict[
+        'status'] = 'Search type can only be "studies", "instances" or "series"'
+    expected_invalid_dict['input'] = input_dict
+    expected_invalid_dict['success'] = False
+
+    fc = FakeHttpClient()
+    FakeClient.return_value = fc
+    with TestPipeline() as p:
+      results = (p | beam.Create([input_dict]) | DicomSearch())
+      assert_that(results, equal_to([expected_invalid_dict]))
+
+  @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+  def test_missing_parameters(self, FakeClient):
+    input_dict = {}
+    input_dict['project_id'] = "test_project"
+    input_dict['region'] = "test_region"
+
+    expected_invalid_dict = {}
+    expected_invalid_dict['result'] = []
+    expected_invalid_dict['status'] = 'Must have dataset_id in the dict.'
+    expected_invalid_dict['input'] = input_dict
+    expected_invalid_dict['success'] = False
+
+    fc = FakeHttpClient()
+    FakeClient.return_value = fc
+    with TestPipeline() as p:
+      results = (p | beam.Create([input_dict]) | DicomSearch())
+      assert_that(results, equal_to([expected_invalid_dict]))
+
+  @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+  def test_client_search_notfound(self, FakeClient):
+    input_dict = {}
+    # search instances in a not exist store
+    input_dict['project_id'] = "wrong_project"
+    input_dict['region'] = "wrong_region"
+    input_dict['dataset_id'] = "wrong_dataset_id"
+    input_dict['dicom_store_id'] = "wrong_dicom_store_id"
+    input_dict['search_type'] = "instances"
+
+    expected_invalid_dict = {}
+    expected_invalid_dict['result'] = []
+    expected_invalid_dict['status'] = 204
+    expected_invalid_dict['input'] = input_dict
+    expected_invalid_dict['success'] = False
+
+    fc = FakeHttpClient()
+    FakeClient.return_value = fc
+    with TestPipeline() as p:
+      results = (p | beam.Create([input_dict]) | DicomSearch())
+      assert_that(results, equal_to([expected_invalid_dict]))
+
+
+@unittest.skipIf(DicomSearch is None, 'GCP dependencies are not installed')
+class TestDicomStoreInstance(_TestCaseWithTempDirCleanUp):
+  @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+  def test_store_byte_file(self, FakeClient):
+    input_dict = {}
+    input_dict['project_id'] = "test_project"
+    input_dict['region'] = "test_region"
+    input_dict['dataset_id'] = "test_dataset_id"
+    input_dict['dicom_store_id'] = "test_dicom_store_id"
+
+    fc = FakeHttpClient()
+    FakeClient.return_value = fc
+
+    dict_input = {'PatientName': 'George', 'Age': 23, 'TestResult': 'Negative'}
+    str_input = json.dumps(dict_input)
+    bytes_input = bytes(str_input.encode("utf-8"))
+    with TestPipeline() as p:
+      results = (
+          p
+          | beam.Create([bytes_input])
+          | UploadToDicomStore(input_dict, 'bytes')
+          | beam.Map(lambda x: x['success']))
+      assert_that(results, equal_to([True]))
+    self.assertTrue(dict_input in fc.dicom_metadata)
+
+  @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+  def test_store_byte_file_small_buffer_flush(self, FakeClient):
+    input_dict = {}
+    input_dict['project_id'] = "test_project"
+    input_dict['region'] = "test_region"
+    input_dict['dataset_id'] = "test_dataset_id"
+    input_dict['dicom_store_id'] = "test_dicom_store_id"
+
+    fc = FakeHttpClient()
+    FakeClient.return_value = fc
+
+    dict_input_1 = {
+        'PatientName': 'George', 'Age': 23, 'TestResult': 'Negative'
+    }
+    str_input_1 = json.dumps(dict_input_1)
+    bytes_input_1 = bytes(str_input_1.encode("utf-8"))
+    dict_input_2 = {'PatientName': 'Peter', 'Age': 54, 'TestResult': 'Positive'}
+    str_input_2 = json.dumps(dict_input_2)
+    bytes_input_2 = bytes(str_input_2.encode("utf-8"))
+    dict_input_3 = {'PatientName': 'Zen', 'Age': 27, 'TestResult': 'Negative'}
+    str_input_3 = json.dumps(dict_input_3)
+    bytes_input_3 = bytes(str_input_3.encode("utf-8"))
+    with TestPipeline() as p:
+      results = (
+          p
+          | beam.Create([bytes_input_1, bytes_input_2, bytes_input_3])
+          | UploadToDicomStore(input_dict, 'bytes', buffer_size=1)
+          | beam.Map(lambda x: x['success']))
+      assert_that(results, equal_to([True] * 3))
+    self.assertTrue(dict_input_1 in fc.dicom_metadata)
+    self.assertTrue(dict_input_2 in fc.dicom_metadata)
+    self.assertTrue(dict_input_3 in fc.dicom_metadata)
+
+  @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+  def test_store_fileio_file(self, FakeClient):
+    input_dict = {}
+    input_dict['project_id'] = "test_project"
+    input_dict['region'] = "test_region"
+    input_dict['dataset_id'] = "test_dataset_id"
+    input_dict['dicom_store_id'] = "test_dicom_store_id"
+
+    fc = FakeHttpClient()
+    FakeClient.return_value = fc
+
+    dict_input = {'PatientName': 'George', 'Age': 23, 'TestResult': 'Negative'}
+    str_input = json.dumps(dict_input)
+    temp_dir = '%s%s' % (self._new_tempdir(), os.sep)
+    self._create_temp_file(dir=temp_dir, content=str_input)
+
+    with TestPipeline() as p:
+      results = (
+          p
+          | beam.Create([FileSystems.join(temp_dir, '*')])
+          | fileio.MatchAll()
+          | fileio.ReadMatches()
+          | UploadToDicomStore(input_dict, 'fileio')
+          | beam.Map(lambda x: x['success']))
+      assert_that(results, equal_to([True]))
+    self.assertTrue(dict_input in fc.dicom_metadata)
+
+  @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+  def test_store_fileio_file_small_buffer_flush(self, FakeClient):
+    input_dict = {}
+    input_dict['project_id'] = "test_project"
+    input_dict['region'] = "test_region"
+    input_dict['dataset_id'] = "test_dataset_id"
+    input_dict['dicom_store_id'] = "test_dicom_store_id"
+
+    fc = FakeHttpClient()
+    FakeClient.return_value = fc
+
+    temp_dir = '%s%s' % (self._new_tempdir(), os.sep)
+    dict_input_1 = {
+        'PatientName': 'George', 'Age': 23, 'TestResult': 'Negative'
+    }
+    str_input_1 = json.dumps(dict_input_1)
+    self._create_temp_file(dir=temp_dir, content=str_input_1)
+    dict_input_2 = {'PatientName': 'Peter', 'Age': 54, 'TestResult': 'Positive'}
+    str_input_2 = json.dumps(dict_input_2)
+    self._create_temp_file(dir=temp_dir, content=str_input_2)
+    dict_input_3 = {'PatientName': 'Zen', 'Age': 27, 'TestResult': 'Negative'}
+    str_input_3 = json.dumps(dict_input_3)
+    self._create_temp_file(dir=temp_dir, content=str_input_3)
+
+    with TestPipeline() as p:
+      results = (
+          p
+          | beam.Create([FileSystems.join(temp_dir, '*')])
+          | fileio.MatchAll()
+          | fileio.ReadMatches()
+          | UploadToDicomStore(input_dict, 'fileio', buffer_size=1)
+          | beam.Map(lambda x: x['success']))
+      assert_that(results, equal_to([True] * 3))
+    self.assertTrue(dict_input_1 in fc.dicom_metadata)
+    self.assertTrue(dict_input_2 in fc.dicom_metadata)
+    self.assertTrue(dict_input_3 in fc.dicom_metadata)
+
+  @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+  def test_destination_notfound(self, FakeClient):
+    input_dict = {}
+    # search instances in a not exist store
+    input_dict['project_id'] = "wrong_project"
+    input_dict['region'] = "wrong_region"
+    input_dict['dataset_id'] = "wrong_dataset_id"
+    input_dict['dicom_store_id'] = "wrong_dicom_store_id"
+
+    expected_invalid_dict = {}
+    expected_invalid_dict['status'] = 204
+    expected_invalid_dict['input'] = ''
+    expected_invalid_dict['success'] = False
+
+    fc = FakeHttpClient()
+    FakeClient.return_value = fc
+    with TestPipeline() as p:
+      results = (
+          p | beam.Create(['']) | UploadToDicomStore(input_dict, 'bytes'))
+      assert_that(results, equal_to([expected_invalid_dict]))
+
+  @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+  def test_missing_parameters(self, FakeClient):
+    input_dict = {}
+    input_dict['project_id'] = "test_project"
+    input_dict['region'] = "test_region"
+
+    expected_invalid_dict = {}
+    expected_invalid_dict['result'] = []
+    expected_invalid_dict['status'] = 'Must have dataset_id in the dict.'
+    expected_invalid_dict['input'] = input_dict
+    expected_invalid_dict['success'] = False
+
+    fc = FakeHttpClient()
+    FakeClient.return_value = fc
+    with self.assertRaisesRegex(ValueError,
+                                "Must have dataset_id in the dict."):
+      p = TestPipeline()
+      _ = (p | beam.Create(['']) | UploadToDicomStore(input_dict, 'bytes'))
+
+
+if __name__ == '__main__':
+  unittest.main()
diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py
index 8f2a9cf..dca3a39 100644
--- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py
+++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py
@@ -592,15 +592,13 @@
     return result
 
   def _maybe_add_unified_worker_missing_options(self, options):
-    # set default beam_fn_api and use_beam_bq_sink experiment if use unified
+    # set default beam_fn_api experiment if use unified
     # worker experiment flag exists, no-op otherwise.
     debug_options = options.view_as(DebugOptions)
     from apache_beam.runners.dataflow.internal import apiclient
     if apiclient._use_unified_worker(options):
       if not debug_options.lookup_experiment('beam_fn_api'):
         debug_options.add_experiment('beam_fn_api')
-      if not debug_options.lookup_experiment('use_beam_bq_sink'):
-        debug_options.add_experiment('use_beam_bq_sink')
 
   def _get_typehint_based_encoding(self, typehint, window_coder):
     """Returns an encoding based on a typehint object."""
diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py
index 1e4f6f9..0315af3 100644
--- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py
+++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py
@@ -698,6 +698,7 @@
   def test_write_bigquery_translation(self):
     runner = DataflowRunner()
 
+    self.default_properties.append('--experiments=use_legacy_bq_sink')
     with beam.Pipeline(runner=runner,
                        options=PipelineOptions(self.default_properties)) as p:
       # pylint: disable=expression-not-assigned
@@ -749,12 +750,13 @@
     """Tests that WriteToBigQuery cannot have any consumers if replaced."""
     runner = DataflowRunner()
 
-    with self.assertRaises(ValueError):
+    self.default_properties.append('--experiments=use_legacy_bq_sink')
+    with self.assertRaises(Exception):
       with beam.Pipeline(runner=runner,
                          options=PipelineOptions(self.default_properties)) as p:
         # pylint: disable=expression-not-assigned
         out = p | beam.Create([1]) | beam.io.WriteToBigQuery('some.table')
-        out['FailedRows'] | 'MyTransform' >> beam.Map(lambda _: _)
+        out['destination_file_pairs'] | 'MyTransform' >> beam.Map(lambda _: _)
 
 
 class CustomMergingWindowFn(window.WindowFn):
diff --git a/sdks/python/apache_beam/runners/dataflow/ptransform_overrides.py b/sdks/python/apache_beam/runners/dataflow/ptransform_overrides.py
index 75b1db0..5743f52 100644
--- a/sdks/python/apache_beam/runners/dataflow/ptransform_overrides.py
+++ b/sdks/python/apache_beam/runners/dataflow/ptransform_overrides.py
@@ -236,7 +236,10 @@
         self.visit_transform(transform_node)
 
       def visit_transform(self, transform_node):
-        if [o for o in self.outputs if o in transform_node.inputs]:
+        # Internal consumers of the outputs we're overriding are expected.
+        # We only error out on non-internal consumers.
+        if ('BigQueryBatchFileLoads' not in transform_node.full_label and
+            [o for o in self.outputs if o in transform_node.inputs]):
           raise ValueError(
               'WriteToBigQuery was being replaced with the native '
               'BigQuerySink, but the transform "{}" has an input which will be '
@@ -250,16 +253,13 @@
     # Imported here to avoid circular dependencies.
     # pylint: disable=wrong-import-order, wrong-import-position
     from apache_beam import io
-    from apache_beam.runners.dataflow.internal import apiclient
-
     transform = applied_ptransform.transform
     if (not isinstance(transform, io.WriteToBigQuery) or
         getattr(transform, 'override', False)):
       return False
 
-    use_fnapi = apiclient._use_fnapi(self.options)
     experiments = self.options.view_as(DebugOptions).experiments or []
-    if (use_fnapi or 'use_beam_bq_sink' in experiments):
+    if 'use_legacy_bq_sink' not in experiments:
       return False
 
     if transform.schema == io.gcp.bigquery.SCHEMA_AUTODETECT:
diff --git a/sdks/python/apache_beam/runners/interactive/background_caching_job.py b/sdks/python/apache_beam/runners/interactive/background_caching_job.py
index 117cd59..1b05285 100644
--- a/sdks/python/apache_beam/runners/interactive/background_caching_job.py
+++ b/sdks/python/apache_beam/runners/interactive/background_caching_job.py
@@ -291,7 +291,7 @@
             'data to start at the same time, all captured data has been '
             'cleared and a new segment of data will be recorded.')
 
-    ie.current_env().cleanup()
+    ie.current_env().cleanup(user_pipeline)
     ie.current_env().set_cached_source_signature(
         user_pipeline, current_signature)
   return is_changed
diff --git a/sdks/python/apache_beam/runners/interactive/background_caching_job_test.py b/sdks/python/apache_beam/runners/interactive/background_caching_job_test.py
index 45c65dd..803f6ce 100644
--- a/sdks/python/apache_beam/runners/interactive/background_caching_job_test.py
+++ b/sdks/python/apache_beam/runners/interactive/background_caching_job_test.py
@@ -91,8 +91,6 @@
     sys.version_info < (3, 6), 'The tests require at least Python 3.6 to work.')
 class BackgroundCachingJobTest(unittest.TestCase):
   def tearDown(self):
-    for _, job in ie.current_env()._background_caching_jobs.items():
-      job.cancel()
     ie.new_env()
 
   # TODO(BEAM-8335): remove the patches when there are appropriate test sources
@@ -302,9 +300,11 @@
   def test_determine_a_test_stream_service_running(self):
     pipeline = _build_an_empty_stream_pipeline()
     test_stream_service = TestStreamServiceController(reader=None)
+    test_stream_service.start()
     ie.current_env().set_test_stream_service_controller(
         pipeline, test_stream_service)
     self.assertTrue(bcj.is_a_test_stream_service_running(pipeline))
+    # the test_stream_service will be cleaned up on teardown.
 
   def test_stop_a_running_test_stream_service(self):
     pipeline = _build_an_empty_stream_pipeline()
diff --git a/sdks/python/apache_beam/runners/interactive/display/pcoll_visualization.py b/sdks/python/apache_beam/runners/interactive/display/pcoll_visualization.py
index d11cb24..ce7ec0e 100644
--- a/sdks/python/apache_beam/runners/interactive/display/pcoll_visualization.py
+++ b/sdks/python/apache_beam/runners/interactive/display/pcoll_visualization.py
@@ -47,7 +47,6 @@
   from IPython.core.display import Javascript  # pylint: disable=import-error
   from IPython.core.display import display  # pylint: disable=import-error
   from IPython.core.display import display_javascript  # pylint: disable=import-error
-  from IPython.core.display import update_display  # pylint: disable=import-error
   from facets_overview.generic_feature_statistics_generator import GenericFeatureStatisticsGenerator  # pylint: disable=import-error
   from timeloop import Timeloop  # pylint: disable=import-error
 
@@ -134,6 +133,10 @@
             <script>
               {script_in_jquery_with_datatable}
             </script>"""
+_NO_DATA_TEMPLATE = _CSS + """
+            <div id="no_data_{id}">No data to display.</div>"""
+_NO_DATA_REMOVAL_SCRIPT = """
+            $("#no_data_{id}").remove();"""
 
 
 def visualize(
@@ -382,16 +385,23 @@
     if update and not update._is_datatable_empty:
       display_javascript(Javascript(script_in_jquery_with_datatable))
     else:
-      html = _DATAFRAME_PAGINATION_TEMPLATE.format(
-          table_id=table_id,
-          script_in_jquery_with_datatable=script_in_jquery_with_datatable)
+      if data.empty:
+        html = _NO_DATA_TEMPLATE.format(id=table_id)
+      else:
+        html = _DATAFRAME_PAGINATION_TEMPLATE.format(
+            table_id=table_id,
+            script_in_jquery_with_datatable=script_in_jquery_with_datatable)
       if update:
         if not data.empty:
-          # Re-initialize a datatable to replace the existing empty datatable.
-          update_display(HTML(html), display_id=update._df_display_id)
+          # Initialize a datatable to replace the existing no data div.
+          display(
+              Javascript(
+                  ie._JQUERY_WITH_DATATABLE_TEMPLATE.format(
+                      customized_script=_NO_DATA_REMOVAL_SCRIPT.format(
+                          id=table_id))))
+          display(HTML(html), display_id=update._df_display_id)
           update._is_datatable_empty = False
       else:
-        # Initialize a datatable for the first time rendering.
         display(HTML(html), display_id=self._df_display_id)
         if not data.empty:
           self._is_datatable_empty = False
diff --git a/sdks/python/apache_beam/runners/interactive/interactive_environment.py b/sdks/python/apache_beam/runners/interactive/interactive_environment.py
index 1d28517..4363d17 100644
--- a/sdks/python/apache_beam/runners/interactive/interactive_environment.py
+++ b/sdks/python/apache_beam/runners/interactive/interactive_environment.py
@@ -251,18 +251,32 @@
     return self._inspector
 
   def cleanup(self, pipeline=None):
-    """Cleans up cached states for the given pipeline. Cleans up
-    for all pipelines if no specific pipeline is given."""
+    """Cleans up cached states for the given pipeline. Noop if the given
+    pipeline is absent from the environment. Cleans up for all pipelines
+    if no pipeline is specified."""
     if pipeline:
+      from apache_beam.runners.interactive import background_caching_job as bcj
+      bcj.attempt_to_cancel_background_caching_job(pipeline)
+      bcj.attempt_to_stop_test_stream_service(pipeline)
       cache_manager = self.get_cache_manager(pipeline)
       if cache_manager:
         cache_manager.cleanup()
     else:
+      for _, job in self._background_caching_jobs.items():
+        if job:
+          job.cancel()
+      for _, controller in self._test_stream_service_controllers.items():
+        if controller:
+          controller.stop()
       for _, cache_manager in self._cache_managers.items():
-        cache_manager.cleanup()
+        if cache_manager:
+          cache_manager.cleanup()
 
+    self.evict_background_caching_job(pipeline)
+    self.evict_test_stream_service_controller(pipeline)
     self.evict_computed_pcollections(pipeline)
     self.evict_cached_source_signature(pipeline)
+    self.evict_pipeline_result(pipeline)
 
   def watch(self, watchable):
     """Watches a watchable.
@@ -343,9 +357,13 @@
         'apache_beam.runners.runner.PipelineResult or its subclass')
     self._main_pipeline_results[str(id(pipeline))] = result
 
-  def evict_pipeline_result(self, pipeline):
-    """Evicts the tracking of given pipeline run. Noop if absent."""
-    return self._main_pipeline_results.pop(str(id(pipeline)), None)
+  def evict_pipeline_result(self, pipeline=None):
+    """Evicts the last run result of the given pipeline. Noop if the pipeline
+    is absent from the environment. If no pipeline is specified, evicts for all
+    pipelines."""
+    if pipeline:
+      return self._main_pipeline_results.pop(str(id(pipeline)), None)
+    self._main_pipeline_results.clear()
 
   def pipeline_result(self, pipeline):
     """Gets the pipeline run result. None if absent."""
@@ -364,16 +382,24 @@
     """Gets the background caching job started from the given pipeline."""
     return self._background_caching_jobs.get(str(id(pipeline)), None)
 
+  def evict_background_caching_job(self, pipeline=None):
+    """Evicts the background caching job started from the given pipeline. Noop
+    if the given pipeline is absent from the environment. If no pipeline is
+    specified, evicts for all pipelines."""
+    if pipeline:
+      return self._background_caching_jobs.pop(str(id(pipeline)), None)
+    self._background_caching_jobs.clear()
+
   def set_test_stream_service_controller(self, pipeline, controller):
     """Sets the test stream service controller that has started a gRPC server
-    serving the test stream for any job started from the given user-defined
+    serving the test stream for any job started from the given user defined
     pipeline.
     """
     self._test_stream_service_controllers[str(id(pipeline))] = controller
 
   def get_test_stream_service_controller(self, pipeline):
     """Gets the test stream service controller that has started a gRPC server
-    serving the test stream for any job started from the given user-defined
+    serving the test stream for any job started from the given user defined
     pipeline.
     """
     return self._test_stream_service_controllers.get(str(id(pipeline)), None)
@@ -381,9 +407,12 @@
   def evict_test_stream_service_controller(self, pipeline):
     """Evicts and pops the test stream service controller that has started a
     gRPC server serving the test stream for any job started from the given
-    user-defined pipeline.
+    user defined pipeline. Noop if the given pipeline is absent from the
+    environment. If no pipeline is specified, evicts for all pipelines.
     """
-    return self._test_stream_service_controllers.pop(str(id(pipeline)), None)
+    if pipeline:
+      return self._test_stream_service_controllers.pop(str(id(pipeline)), None)
+    self._test_stream_service_controllers.clear()
 
   def is_terminated(self, pipeline):
     """Queries if the most recent job (by executing the given pipeline) state
@@ -400,13 +429,15 @@
     return self._cached_source_signature.get(str(id(pipeline)), set())
 
   def evict_cached_source_signature(self, pipeline=None):
+    """Evicts the signature generated for each recorded source of the given
+    pipeline. Noop if the given pipeline is absent from the environment. If no
+    pipeline is specified, evicts for all pipelines."""
     if pipeline:
-      self._cached_source_signature.pop(str(id(pipeline)), None)
-    else:
-      self._cached_source_signature.clear()
+      return self._cached_source_signature.pop(str(id(pipeline)), None)
+    self._cached_source_signature.clear()
 
   def track_user_pipelines(self):
-    """Record references to all user-defined pipeline instances watched in
+    """Record references to all user defined pipeline instances watched in
     current environment.
 
     Current static global singleton interactive environment holds references to
@@ -416,11 +447,17 @@
     then handle them differently.
 
     This is invoked every time a PTransform is to be applied if the current
-    code execution is under ipython due to the possibility that any user-defined
+    code execution is under ipython due to the possibility that any user defined
     pipeline can be re-evaluated through notebook cell re-execution at any time.
 
     Each time this is invoked, it will check if there is a cache manager
     already created for each user defined pipeline. If not, create one for it.
+
+    If a pipeline is no longer watched due to re-execution while its
+    PCollections are still in watched scope, the pipeline becomes anonymous but
+    still accessible indirectly through references to its PCollections. This
+    function also clears up internal states for those anonymous pipelines once
+    all their PCollections are anonymous.
     """
     self._tracked_user_pipelines = set()
     for watching in self.watching():
@@ -428,6 +465,17 @@
         if isinstance(val, beam.pipeline.Pipeline):
           self._tracked_user_pipelines.add(val)
           _ = self.get_cache_manager(val, create_if_absent=True)
+    all_tracked_pipeline_ids = set(self._background_caching_jobs.keys()).union(
+        set(self._test_stream_service_controllers.keys()),
+        set(self._cache_managers.keys()),
+        {str(id(pcoll.pipeline))
+         for pcoll in self._computed_pcolls},
+        set(self._cached_source_signature.keys()),
+        set(self._main_pipeline_results.keys()))
+    inspectable_pipelines = self._inspector.inspectable_pipelines
+    for pipeline in all_tracked_pipeline_ids:
+      if pipeline not in inspectable_pipelines:
+        self.cleanup(pipeline)
 
   @property
   def tracked_user_pipelines(self):
diff --git a/sdks/python/apache_beam/runners/interactive/interactive_environment_test.py b/sdks/python/apache_beam/runners/interactive/interactive_environment_test.py
index 6f44dac..6650c63 100644
--- a/sdks/python/apache_beam/runners/interactive/interactive_environment_test.py
+++ b/sdks/python/apache_beam/runners/interactive/interactive_environment_test.py
@@ -236,14 +236,33 @@
   @patch(
       'apache_beam.runners.interactive.interactive_environment'
       '.InteractiveEnvironment.cleanup')
-  def test_cleanup_invoked_when_cache_manager_is_evicted(self, mocked_cleanup):
+  def test_track_user_pipeline_cleanup_non_inspectable_pipeline(
+      self, mocked_cleanup):
     ie._interactive_beam_env = None
     ie.new_env()
-    dummy_pipeline = 'dummy'
+    dummy_pipeline_1 = beam.Pipeline()
+    dummy_pipeline_2 = beam.Pipeline()
+    dummy_pipeline_3 = beam.Pipeline()
+    dummy_pipeline_4 = beam.Pipeline()
+    dummy_pcoll = dummy_pipeline_4 | beam.Create([1])
+    dummy_pipeline_5 = beam.Pipeline()
+    dummy_non_inspectable_pipeline = 'dummy'
+    ie.current_env().watch(locals())
+    from apache_beam.runners.interactive.background_caching_job import BackgroundCachingJob
+    ie.current_env().set_background_caching_job(
+        dummy_pipeline_1,
+        BackgroundCachingJob(
+            runner.PipelineResult(runner.PipelineState.DONE), limiters=[]))
+    ie.current_env().set_test_stream_service_controller(dummy_pipeline_2, None)
     ie.current_env().set_cache_manager(
-        cache.FileBasedCacheManager(), dummy_pipeline)
+        cache.FileBasedCacheManager(), dummy_pipeline_3)
+    ie.current_env().mark_pcollection_computed([dummy_pcoll])
+    ie.current_env().set_cached_source_signature(
+        dummy_non_inspectable_pipeline, None)
+    ie.current_env().set_pipeline_result(
+        dummy_pipeline_5, runner.PipelineResult(runner.PipelineState.RUNNING))
     mocked_cleanup.assert_not_called()
-    ie.current_env().evict_cache_manager(dummy_pipeline)
+    ie.current_env().track_user_pipelines()
     mocked_cleanup.assert_called_once()
 
 
diff --git a/sdks/python/apache_beam/runners/interactive/messaging/interactive_environment_inspector.py b/sdks/python/apache_beam/runners/interactive/messaging/interactive_environment_inspector.py
index 3bce182..a4a9f02 100644
--- a/sdks/python/apache_beam/runners/interactive/messaging/interactive_environment_inspector.py
+++ b/sdks/python/apache_beam/runners/interactive/messaging/interactive_environment_inspector.py
@@ -41,6 +41,7 @@
   def __init__(self):
     self._inspectables = {}
     self._anonymous = {}
+    self._inspectable_pipelines = set()
 
   @property
   def inspectables(self):
@@ -49,6 +50,20 @@
     self._inspectables = inspect()
     return self._inspectables
 
+  @property
+  def inspectable_pipelines(self):
+    """Returns a dictionary of all inspectable pipelines. The keys are
+    stringified id of pipeline instances.
+
+    This includes user defined pipeline assigned to variables and anonymous
+    pipelines with inspectable PCollections.
+    If a user defined pipeline is not within the returned dict, it can be
+    considered out of scope, and all resources and memory states related to it
+    should be released.
+    """
+    _ = self.list_inspectables()
+    return self._inspectable_pipelines
+
   @as_json
   def list_inspectables(self):
     """Lists inspectables in JSON format.
@@ -89,6 +104,8 @@
           pipeline_identifier = obfuscate(meta(pipelines[pipeline], pipeline))
           listing[pipeline_identifier]['pcolls'][identifier] = inspectable[
               'metadata']
+    self._inspectable_pipelines = dict(
+        (str(id(pipeline)), pipeline) for pipeline in pipelines)
     return listing
 
   def get_val(self, identifier):
diff --git a/sdks/python/apache_beam/runners/interactive/options/capture_control.py b/sdks/python/apache_beam/runners/interactive/options/capture_control.py
index 12e901f..ab877b5 100644
--- a/sdks/python/apache_beam/runners/interactive/options/capture_control.py
+++ b/sdks/python/apache_beam/runners/interactive/options/capture_control.py
@@ -29,7 +29,6 @@
 from datetime import timedelta
 
 from apache_beam.io.gcp.pubsub import ReadFromPubSub
-from apache_beam.runners.interactive import background_caching_job as bcj
 from apache_beam.runners.interactive import interactive_environment as ie
 from apache_beam.runners.interactive.options import capture_limiters
 
@@ -71,15 +70,7 @@
   runs, Interactive Beam will capture fresh data."""
   if ie.current_env().options.enable_capture_replay:
     _LOGGER.info(
-        'You have requested Interactive Beam to evict all captured '
+        'You have requested Interactive Beam to evict all recorded'
         'data that could be deterministically replayed among multiple '
         'pipeline runs.')
-  ie.current_env().track_user_pipelines()
-  if pipeline:
-    bcj.attempt_to_cancel_background_caching_job(pipeline)
-    bcj.attempt_to_stop_test_stream_service(pipeline)
-  else:
-    for user_pipeline in ie.current_env().tracked_user_pipelines:
-      bcj.attempt_to_cancel_background_caching_job(user_pipeline)
-      bcj.attempt_to_stop_test_stream_service(user_pipeline)
   ie.current_env().cleanup(pipeline)
diff --git a/sdks/python/apache_beam/runners/interactive/options/capture_limiters.py b/sdks/python/apache_beam/runners/interactive/options/capture_limiters.py
index a25aba0..2c84f80 100644
--- a/sdks/python/apache_beam/runners/interactive/options/capture_limiters.py
+++ b/sdks/python/apache_beam/runners/interactive/options/capture_limiters.py
@@ -24,6 +24,8 @@
 
 import threading
 
+from apache_beam.portability.api.beam_interactive_api_pb2 import TestStreamFileHeader
+from apache_beam.portability.api.beam_interactive_api_pb2 import TestStreamFileRecord
 from apache_beam.runners.interactive import interactive_environment as ie
 
 
@@ -36,6 +38,20 @@
     raise NotImplementedError
 
 
+class ElementLimiter(Limiter):
+  """A `Limiter` that limits reading from cache based on some property of an
+  element.
+  """
+  def update(self, e):
+    # type: (Any) -> None
+
+    """Update the internal state based on some property of an element.
+
+    This is executed on every element that is read from cache.
+    """
+    raise NotImplementedError
+
+
 class SizeLimiter(Limiter):
   """Limits the cache size to a specified byte limit."""
   def __init__(
@@ -71,3 +87,56 @@
 
   def is_triggered(self):
     return self._triggered
+
+
+class CountLimiter(ElementLimiter):
+  """Limits by counting the number of elements seen."""
+  def __init__(self, max_count):
+    self._max_count = max_count
+    self._count = 0
+
+  def update(self, e):
+    # A TestStreamFileRecord can contain many elements at once. If e is a file
+    # record, then count the number of elements in the bundle.
+    if isinstance(e, TestStreamFileRecord):
+      if not e.recorded_event.element_event:
+        return
+      self._count += len(e.recorded_event.element_event.elements)
+
+    # Otherwise, count everything else but the header of the file since it is
+    # not an element.
+    elif not isinstance(e, TestStreamFileHeader):
+      self._count += 1
+
+  def is_triggered(self):
+    return self._count >= self._max_count
+
+
+class ProcessingTimeLimiter(ElementLimiter):
+  """Limits by how long the ProcessingTime passed in the element stream.
+
+  This measures the duration from the first element in the stream. Each
+  subsequent element has a delta "advance_duration" that moves the internal
+  clock forward. This triggers when the duration from the internal clock and
+  the start exceeds the given duration.
+  """
+  def __init__(self, max_duration_secs):
+    """Initialize the ProcessingTimeLimiter."""
+    self._max_duration_us = max_duration_secs * 1e6
+    self._start_us = 0
+    self._cur_time_us = 0
+
+  def update(self, e):
+    # Only look at TestStreamFileRecords which hold the processing time.
+    if not isinstance(e, TestStreamFileRecord):
+      return
+
+    if not e.recorded_event.processing_time_event:
+      return
+
+    if self._start_us == 0:
+      self._start_us = e.recorded_event.processing_time_event.advance_duration
+    self._cur_time_us += e.recorded_event.processing_time_event.advance_duration
+
+  def is_triggered(self):
+    return self._cur_time_us - self._start_us >= self._max_duration_us
diff --git a/sdks/python/apache_beam/runners/interactive/options/capture_limiters_test.py b/sdks/python/apache_beam/runners/interactive/options/capture_limiters_test.py
new file mode 100644
index 0000000..850c56e2c
--- /dev/null
+++ b/sdks/python/apache_beam/runners/interactive/options/capture_limiters_test.py
@@ -0,0 +1,53 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#  http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+from __future__ import absolute_import
+
+import unittest
+
+from apache_beam.portability.api.beam_interactive_api_pb2 import TestStreamFileRecord
+from apache_beam.runners.interactive.options.capture_limiters import CountLimiter
+from apache_beam.runners.interactive.options.capture_limiters import ProcessingTimeLimiter
+
+
+class CaptureLimitersTest(unittest.TestCase):
+  def test_count_limiter(self):
+    limiter = CountLimiter(5)
+
+    for e in range(4):
+      limiter.update(e)
+
+    self.assertFalse(limiter.is_triggered())
+    limiter.update(5)
+    self.assertTrue(limiter.is_triggered())
+
+  def test_processing_time_limiter(self):
+    limiter = ProcessingTimeLimiter(max_duration_secs=2)
+
+    r = TestStreamFileRecord()
+    r.recorded_event.processing_time_event.advance_duration = int(1 * 1e6)
+    limiter.update(r)
+    self.assertFalse(limiter.is_triggered())
+
+    r = TestStreamFileRecord()
+    r.recorded_event.processing_time_event.advance_duration = int(2 * 1e6)
+    limiter.update(r)
+    self.assertTrue(limiter.is_triggered())
+
+
+if __name__ == '__main__':
+  unittest.main()
diff --git a/sdks/python/apache_beam/runners/portability/flink_runner.py b/sdks/python/apache_beam/runners/portability/flink_runner.py
index fddf25d..f886a7d 100644
--- a/sdks/python/apache_beam/runners/portability/flink_runner.py
+++ b/sdks/python/apache_beam/runners/portability/flink_runner.py
@@ -110,7 +110,7 @@
       return self._jar
     else:
       return self.path_to_beam_jar(
-          'runners:flink:%s:job-server:shadowJar' % self._flink_version)
+          ':runners:flink:%s:job-server:shadowJar' % self._flink_version)
 
   def java_arguments(
       self, job_port, artifact_port, expansion_port, artifacts_dir):
diff --git a/sdks/python/apache_beam/runners/portability/flink_runner_test.py b/sdks/python/apache_beam/runners/portability/flink_runner_test.py
index 6dd7634..eb83611 100644
--- a/sdks/python/apache_beam/runners/portability/flink_runner_test.py
+++ b/sdks/python/apache_beam/runners/portability/flink_runner_test.py
@@ -91,7 +91,7 @@
   flink_job_server_jar = (
       known_args.flink_job_server_jar or
       job_server.JavaJarJobServer.path_to_beam_jar(
-          'runners:flink:%s:job-server:shadowJar' %
+          ':runners:flink:%s:job-server:shadowJar' %
           FlinkRunnerOptions.PUBLISHED_FLINK_VERSIONS[-1]))
   streaming = known_args.streaming
   environment_type = known_args.environment_type.lower()
diff --git a/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py b/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py
index 4fa92cd..b10a75f 100644
--- a/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py
+++ b/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py
@@ -75,7 +75,7 @@
       url = self._executable_jar
     else:
       url = job_server.JavaJarJobServer.path_to_beam_jar(
-          'runners:flink:%s:job-server:shadowJar' % self.flink_version())
+          ':runners:flink:%s:job-server:shadowJar' % self.flink_version())
     return job_server.JavaJarJobServer.local_jar(url)
 
   def flink_version(self):
diff --git a/sdks/python/apache_beam/runners/portability/spark_runner.py b/sdks/python/apache_beam/runners/portability/spark_runner.py
index 73a3ec4..4619ea9 100644
--- a/sdks/python/apache_beam/runners/portability/spark_runner.py
+++ b/sdks/python/apache_beam/runners/portability/spark_runner.py
@@ -87,7 +87,7 @@
               self._jar)
       return self._jar
     else:
-      return self.path_to_beam_jar('runners:spark:job-server:shadowJar')
+      return self.path_to_beam_jar(':runners:spark:job-server:shadowJar')
 
   def java_arguments(
       self, job_port, artifact_port, expansion_port, artifacts_dir):
diff --git a/sdks/python/apache_beam/runners/portability/spark_runner_test.py b/sdks/python/apache_beam/runners/portability/spark_runner_test.py
index d0c2c4e..19967c1 100644
--- a/sdks/python/apache_beam/runners/portability/spark_runner_test.py
+++ b/sdks/python/apache_beam/runners/portability/spark_runner_test.py
@@ -61,7 +61,7 @@
   spark_job_server_jar = (
       known_args.spark_job_server_jar or
       job_server.JavaJarJobServer.path_to_beam_jar(
-          'runners:spark:job-server:shadowJar'))
+          ':runners:spark:job-server:shadowJar'))
   environment_type = known_args.environment_type.lower()
   environment_config = (
       known_args.environment_config if known_args.environment_config else None)
diff --git a/sdks/python/apache_beam/runners/portability/spark_uber_jar_job_server.py b/sdks/python/apache_beam/runners/portability/spark_uber_jar_job_server.py
index 252f70a..6b9e6fd 100644
--- a/sdks/python/apache_beam/runners/portability/spark_uber_jar_job_server.py
+++ b/sdks/python/apache_beam/runners/portability/spark_uber_jar_job_server.py
@@ -77,7 +77,7 @@
       url = self._executable_jar
     else:
       url = job_server.JavaJarJobServer.path_to_beam_jar(
-          'runners:spark:job-server:shadowJar')
+          ':runners:spark:job-server:shadowJar')
     return job_server.JavaJarJobServer.local_jar(url)
 
   def create_beam_job(self, job_id, job_name, pipeline, options):
diff --git a/sdks/python/apache_beam/testing/test_stream_service.py b/sdks/python/apache_beam/testing/test_stream_service.py
index d69465f..618e208 100644
--- a/sdks/python/apache_beam/testing/test_stream_service.py
+++ b/sdks/python/apache_beam/testing/test_stream_service.py
@@ -34,6 +34,8 @@
   """
   def __init__(self, reader, endpoint=None, exception_handler=None):
     self._server = grpc.server(ThreadPoolExecutor(max_workers=10))
+    self._server_started = False
+    self._server_stopped = False
 
     if endpoint:
       self.endpoint = endpoint
@@ -50,9 +52,18 @@
       self._exception_handler = lambda _: False
 
   def start(self):
+    # A server can only be started if never started and never stopped before.
+    if self._server_started or self._server_stopped:
+      return
+    self._server_started = True
     self._server.start()
 
   def stop(self):
+    # A server can only be stopped if already started and never stopped before.
+    if not self._server_started or self._server_stopped:
+      return
+    self._server_started = False
+    self._server_stopped = True
     self._server.stop(0)
     # This was introduced in grpcio 1.24 and might be gone in the future. Keep
     # this check in case the runtime is on a older, current or future grpcio.
diff --git a/sdks/python/apache_beam/testing/test_stream_service_test.py b/sdks/python/apache_beam/testing/test_stream_service_test.py
index 01b16a1..7a5b403 100644
--- a/sdks/python/apache_beam/testing/test_stream_service_test.py
+++ b/sdks/python/apache_beam/testing/test_stream_service_test.py
@@ -19,6 +19,7 @@
 
 from __future__ import absolute_import
 
+import sys
 import unittest
 
 import grpc
@@ -30,6 +31,13 @@
 from apache_beam.portability.api.beam_runner_api_pb2 import TestStreamPayload
 from apache_beam.testing.test_stream_service import TestStreamServiceController
 
+# TODO(BEAM-8288): clean up the work-around of nose tests using Python2 without
+# unittest.mock module.
+try:
+  from unittest.mock import patch
+except ImportError:
+  from mock import patch  # type: ignore[misc]
+
 # Nose automatically detects tests if they match a regex. Here, it mistakens
 # these protos as tests. For more info see the Nose docs at:
 # https://nose.readthedocs.io/en/latest/writing_tests.html
@@ -116,5 +124,78 @@
     self.assertEqual(events_b, expected_events)
 
 
+@unittest.skipIf(
+    sys.version_info < (3, 6), 'The tests require at least Python 3.6 to work.')
+class TestStreamServiceStartStopTest(unittest.TestCase):
+
+  # Weak internal use needs to be explicitly imported.
+  from grpc import _server
+
+  def setUp(self):
+    self.controller = TestStreamServiceController(
+        EventsReader(expected_key=[('full', EXPECTED_KEY)]))
+    self.assertFalse(self.controller._server_started)
+    self.assertFalse(self.controller._server_stopped)
+
+  def tearDown(self):
+    self.controller.stop()
+
+  def test_start_when_never_started(self):
+    with patch.object(self._server._Server,
+                      'start',
+                      wraps=self.controller._server.start) as mock_start:
+      self.controller.start()
+      mock_start.assert_called_once()
+      self.assertTrue(self.controller._server_started)
+      self.assertFalse(self.controller._server_stopped)
+
+  def test_start_noop_when_already_started(self):
+    with patch.object(self._server._Server,
+                      'start',
+                      wraps=self.controller._server.start) as mock_start:
+      self.controller.start()
+      mock_start.assert_called_once()
+      self.controller.start()
+      mock_start.assert_called_once()
+
+  def test_start_noop_when_already_stopped(self):
+    with patch.object(self._server._Server,
+                      'start',
+                      wraps=self.controller._server.start) as mock_start:
+      self.controller.start()
+      self.controller.stop()
+      mock_start.assert_called_once()
+      self.controller.start()
+      mock_start.assert_called_once()
+
+  def test_stop_noop_when_not_started(self):
+    with patch.object(self._server._Server,
+                      'stop',
+                      wraps=self.controller._server.stop) as mock_stop:
+      self.controller.stop()
+      mock_stop.assert_not_called()
+
+  def test_stop_when_already_started(self):
+    with patch.object(self._server._Server,
+                      'stop',
+                      wraps=self.controller._server.stop) as mock_stop:
+      self.controller.start()
+      mock_stop.assert_not_called()
+      self.controller.stop()
+      mock_stop.assert_called_once()
+      self.assertFalse(self.controller._server_started)
+      self.assertTrue(self.controller._server_stopped)
+
+  def test_stop_noop_when_already_stopped(self):
+    with patch.object(self._server._Server,
+                      'stop',
+                      wraps=self.controller._server.stop) as mock_stop:
+      self.controller.start()
+      self.controller.stop()
+      mock_stop.assert_called_once()
+      self.controller.stop()
+      mock_stop.assert_called_once()
+
+
 if __name__ == '__main__':
   unittest.main()
diff --git a/sdks/python/apache_beam/utils/shared.py b/sdks/python/apache_beam/utils/shared.py
index 31d4ed6..9f0ac94 100644
--- a/sdks/python/apache_beam/utils/shared.py
+++ b/sdks/python/apache_beam/utils/shared.py
@@ -102,10 +102,12 @@
   def __init__(self):
     self._lock = threading.Lock()
     self._ref = None
+    self._tag = None
 
   def acquire(
       self,
-      constructor_fn  # type: Callable[[], Any]
+      constructor_fn,  # type: Callable[[], Any]
+      tag=None  # type: Any
   ):
     # type: (...) -> Any
 
@@ -116,6 +118,9 @@
         present in the cache. This function should take no arguments. It should
         return an initialised object, or None if the object could not be
         initialised / constructed.
+      tag: an optional indentifier to store with the cached object. If
+        subsequent calls to acquire use different tags, the object will be
+        reloaded rather than returned from cache.
 
     Returns:
       An initialised object, either from a previous initialisation, or
@@ -124,7 +129,8 @@
     with self._lock:
       # self._ref is None if this is a new control block.
       # self._ref() is None if the weak reference was GCed.
-      if self._ref is None or self._ref() is None:
+      # self._tag != tag if user specifies a new identifier
+      if self._ref is None or self._ref() is None or self._tag != tag:
         result = constructor_fn()
         if result is None:
           return None
@@ -205,6 +211,7 @@
       self,
       key,  # type: Text
       constructor_fn,  # type: Callable[[], Any]
+      tag=None  # type: Any
   ):
     # type: (...) -> Any
 
@@ -216,6 +223,9 @@
         present in the cache. This function should take no arguments. It should
         return an initialised object, or None if the object could not be
         initialised / constructed.
+      tag: an optional indentifier to store with the cached object. If
+        subsequent calls to acquire use different tags, the object will be
+        reloaded rather than returned from cache.
 
     Returns:
       A reference to the initialised object, either from the cache, or
@@ -227,7 +237,7 @@
         control_block = _SharedControlBlock()
         self._cache_map[key] = control_block
 
-    result = control_block.acquire(constructor_fn)
+    result = control_block.acquire(constructor_fn, tag)
 
     # Because we release the lock in between, if we acquire multiple Shareds
     # in a short time, there's no guarantee as to which one will be kept alive.
@@ -266,7 +276,8 @@
 
   def acquire(
       self,
-      constructor_fn  # type: Callable[[], Any]
+      constructor_fn,  # type: Callable[[], Any]
+      tag=None  # type: Any
   ):
     # type: (...) -> Any
 
@@ -277,9 +288,12 @@
         present in the cache. This function should take no arguments. It should
         return an initialised object, or None if the object could not be
         initialised / constructed.
+      tag: an optional indentifier to store with the cached object. If
+        subsequent calls to acquire use different tags, the object will be
+        reloaded rather than returned from cache.
 
     Returns:
       A reference to an initialised object, either from the cache, or
       newly-constructed.
     """
-    return _shared_map.acquire(self._key, constructor_fn)
+    return _shared_map.acquire(self._key, constructor_fn, tag)
diff --git a/sdks/python/apache_beam/utils/shared_test.py b/sdks/python/apache_beam/utils/shared_test.py
index 35cdb26..28ba7c8 100644
--- a/sdks/python/apache_beam/utils/shared_test.py
+++ b/sdks/python/apache_beam/utils/shared_test.py
@@ -236,6 +236,28 @@
     self.assertEqual('sequence3', f3.get_name())
     self.assertEqual('sequence4', s3.get_name())
 
+  def testTagCacheEviction(self):
+    shared1 = shared.Shared()
+    shared2 = shared.Shared()
+
+    def acquire_fn_1():
+      return NamedObject('obj_1')
+
+    def acquire_fn_2():
+      return NamedObject('obj_2')
+
+    # with no tag, shared handle does not know when to evict objects
+    p1 = shared1.acquire(acquire_fn_1)
+    assert p1.get_name() == 'obj_1'
+    p2 = shared1.acquire(acquire_fn_2)
+    assert p2.get_name() == 'obj_1'
+
+    # cache eviction can be forced by specifying different tags
+    p1 = shared2.acquire(acquire_fn_1, tag='1')
+    assert p1.get_name() == 'obj_1'
+    p2 = shared2.acquire(acquire_fn_2, tag='2')
+    assert p2.get_name() == 'obj_2'
+
 
 if __name__ == '__main__':
   unittest.main()
diff --git a/sdks/python/apache_beam/utils/subprocess_server.py b/sdks/python/apache_beam/utils/subprocess_server.py
index 0d40181..018a837 100644
--- a/sdks/python/apache_beam/utils/subprocess_server.py
+++ b/sdks/python/apache_beam/utils/subprocess_server.py
@@ -291,11 +291,20 @@
     if port:
       return port
     else:
-      s = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
+      try:
+        s = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
+      except OSError as e:
+        # [Errno 97] Address family not supported by protocol
+        # Likely indicates we are in an IPv6-only environment (BEAM-10618). Try
+        # again with AF_INET6.
+        if e.errno == 97:
+          s = socket.socket(socket.AF_INET6, socket.SOCK_STREAM)
+        else:
+          raise e
+
       sockets.append(s)
       s.bind(('localhost', 0))
-      _, free_port = s.getsockname()
-      return free_port
+      return s.getsockname()[1]
 
   ports = list(map(find_free_port, ports))
   # Close sockets only now to avoid the same port to be chosen twice
diff --git a/sdks/python/apache_beam/utils/subprocess_server_test.py b/sdks/python/apache_beam/utils/subprocess_server_test.py
index 4663b0f..22640bc 100644
--- a/sdks/python/apache_beam/utils/subprocess_server_test.py
+++ b/sdks/python/apache_beam/utils/subprocess_server_test.py
@@ -51,12 +51,12 @@
         'https://repo.maven.apache.org/maven2/org/apache/beam/'
         'beam-sdks-java-fake/VERSION/beam-sdks-java-fake-VERSION.jar',
         subprocess_server.JavaJarServer.path_to_beam_jar(
-            'sdks:java:fake:fatJar', version='VERSION'))
+            ':sdks:java:fake:fatJar', version='VERSION'))
     self.assertEqual(
         'https://repo.maven.apache.org/maven2/org/apache/beam/'
         'beam-sdks-java-fake/VERSION/beam-sdks-java-fake-A-VERSION.jar',
         subprocess_server.JavaJarServer.path_to_beam_jar(
-            'sdks:java:fake:fatJar', appendix='A', version='VERSION'))
+            ':sdks:java:fake:fatJar', appendix='A', version='VERSION'))
 
   def test_gradle_jar_dev(self):
     with self.assertRaisesRegex(
@@ -69,7 +69,7 @@
                                'beam-sdks-java-fake-VERSION-SNAPSHOT.jar')) +
         ' not found.'):
       subprocess_server.JavaJarServer.path_to_beam_jar(
-          'sdks:java:fake:fatJar', version='VERSION.dev')
+          ':sdks:java:fake:fatJar', version='VERSION.dev')
     with self.assertRaisesRegex(
         Exception,
         re.escape(os.path.join('sdks',
@@ -80,7 +80,7 @@
                                'beam-sdks-java-fake-A-VERSION-SNAPSHOT.jar')) +
         ' not found.'):
       subprocess_server.JavaJarServer.path_to_beam_jar(
-          'sdks:java:fake:fatJar', appendix='A', version='VERSION.dev')
+          ':sdks:java:fake:fatJar', appendix='A', version='VERSION.dev')
 
   def test_beam_services(self):
     with subprocess_server.JavaJarServer.beam_services({':some:target': 'foo'}):
diff --git a/sdks/python/container/base_image_requirements.txt b/sdks/python/container/base_image_requirements.txt
index fd9163b..3484d38 100644
--- a/sdks/python/container/base_image_requirements.txt
+++ b/sdks/python/container/base_image_requirements.txt
@@ -45,7 +45,7 @@
 typing-extensions==3.7.4.1
 
 # GCP extra features
-google-api-core==1.20.0
+google-api-core==1.21.0
 google-apitools==0.5.28
 google-cloud-pubsub==1.0.2
 google-cloud-bigquery==1.26.1
diff --git a/sdks/python/setup.py b/sdks/python/setup.py
index cac23cd..652301f 100644
--- a/sdks/python/setup.py
+++ b/sdks/python/setup.py
@@ -201,6 +201,7 @@
 GCP_REQUIREMENTS = [
     'cachetools>=3.1.0,<4',
     'google-apitools>=0.5.31,<0.5.32',
+    'google-auth>=1.18.0,<2',
     'google-cloud-datastore>=1.7.1,<2',
     'google-cloud-pubsub>=0.39.0,<2',
     # GCP packages required by tests
diff --git a/settings.gradle b/settings.gradle
index 8289f79..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"
@@ -133,6 +134,7 @@
 include ":sdks:java:testing:nexmark"
 include ":sdks:java:testing:expansion-service"
 include ":sdks:java:testing:kafka-service"
+include ":sdks:java:testing:tpcds"
 include ":sdks:python"
 include ":sdks:python:apache_beam:testing:load_tests"
 include ":sdks:python:container"
diff --git a/website/www/site/content/en/documentation/dsls/sql/walkthrough.md b/website/www/site/content/en/documentation/dsls/sql/walkthrough.md
index a02e14c..59ecb82 100644
--- a/website/www/site/content/en/documentation/dsls/sql/walkthrough.md
+++ b/website/www/site/content/en/documentation/dsls/sql/walkthrough.md
@@ -129,7 +129,7 @@
           .builder()
           .addInt32Field("appId")
           .addInt32Field("reviewerId")
-          .withFloatField("rating")
+          .addFloatField("rating")
           .addDateTimeField("rowtime")
           .build();
     
diff --git a/website/www/site/content/en/documentation/transforms/python/aggregation/sample.md b/website/www/site/content/en/documentation/transforms/python/aggregation/sample.md
index 7d10eb0..33fc8f6 100644
--- a/website/www/site/content/en/documentation/transforms/python/aggregation/sample.md
+++ b/website/www/site/content/en/documentation/transforms/python/aggregation/sample.md
@@ -14,24 +14,62 @@
 See the License for the specific language governing permissions and
 limitations under the License.
 -->
-# Sample
-<table align="left">
-    <a target="_blank" class="button"
-        href="https://beam.apache.org/releases/pydoc/2.12.0/apache_beam.transforms.combiners.html?#apache_beam.transforms.combiners.Sample">
-      <img src="https://beam.apache.org/images/logos/sdks/python.png" width="20px" height="20px"
-           alt="Pydoc" />
-     Pydoc
-    </a>
-</table>
-<br><br>
 
+# Sample
+
+{{< localstorage language language-py >}}
+
+{{< button-pydoc path="apache_beam.transforms.combiners" class="Sample" >}}
 
 Transforms for taking samples of the elements in a collection, or
-samples of the values associated with each key in a collection of 
+samples of the values associated with each key in a collection of
 key-value pairs.
 
 ## Examples
-See [BEAM-7390](https://issues.apache.org/jira/browse/BEAM-7390) for updates. 
 
-## Related transforms 
-* [Top](/documentation/transforms/python/aggregation/top) finds the largest or smallest element.
\ No newline at end of file
+In the following example, we create a pipeline with a `PCollection`.
+Then, we get a random sample of elements in different ways.
+
+### Example 1: Sample elements from a PCollection
+
+We use `Sample.FixedSizeGlobally()` to get a fixed-size random sample of elements from the *entire* `PCollection`.
+
+{{< highlight py >}}
+{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/aggregation/sample.py" sample_fixed_size_globally >}}
+{{< /highlight >}}
+
+{{< paragraph class="notebook-skip" >}}
+Output:
+{{< /paragraph >}}
+
+{{< highlight class="notebook-skip" >}}
+{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/aggregation/sample_test.py" sample >}}
+{{< /highlight >}}
+
+{{< buttons-code-snippet
+  py="sdks/python/apache_beam/examples/snippets/transforms/aggregation/sample.py" >}}
+
+### Example 2: Sample elements for each key
+
+We use `Sample.FixedSizePerKey()` to get fixed-size random samples for each unique key in a `PCollection` of key-values.
+
+{{< highlight py >}}
+{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/aggregation/sample.py" sample_fixed_size_per_key >}}
+{{< /highlight >}}
+
+{{< paragraph class="notebook-skip" >}}
+Output:
+{{< /paragraph >}}
+
+{{< highlight class="notebook-skip" >}}
+{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/aggregation/sample_test.py" samples_per_key >}}
+{{< /highlight >}}
+
+{{< buttons-code-snippet
+  py="sdks/python/apache_beam/examples/snippets/transforms/aggregation/sample.py" >}}
+
+## Related transforms
+
+* [Top](/documentation/transforms/python/aggregation/top) finds the largest or smallest element.
+
+{{< button-pydoc path="apache_beam.transforms.combiners" class="Sample" >}}
diff --git a/website/www/site/content/en/get-started/quickstart-py.md b/website/www/site/content/en/get-started/quickstart-py.md
index 1c0afda..b2a503e 100644
--- a/website/www/site/content/en/get-started/quickstart-py.md
+++ b/website/www/site/content/en/get-started/quickstart-py.md
@@ -157,19 +157,16 @@
 python -m apache_beam.examples.wordcount --input /path/to/inputfile --output /path/to/write/counts
 {{< /highlight >}}
 
-{{< highlight class="runner-flink-local" >}}
-Currently, running wordcount.py on Flink requires a full download of the Beam source code.
-See https://beam.apache.org/roadmap/portability/#python-on-flink for more information.
-{{< /highlight >}}
-
-{{< highlight class="runner-flink-cluster" >}}
-Currently, running wordcount.py on Flink requires a full download of the Beam source code.
-See https://beam.apache.org/documentation/runners/flink/ for more information.
+{{< highlight class="runner-flink" >}}
+python -m apache_beam.examples.wordcount --input /path/to/inputfile \
+                                         --output /path/to/write/counts \
+                                         --runner FlinkRunner
 {{< /highlight >}}
 
 {{< highlight class="runner-spark" >}}
-Currently, running wordcount.py on Spark requires a full download of the Beam source code.
-See https://beam.apache.org/roadmap/portability/#python-on-spark for more information.
+python -m apache_beam.examples.wordcount --input /path/to/inputfile \
+                                         --output /path/to/write/counts \
+                                         --runner SparkRunner
 {{< /highlight >}}
 
 {{< highlight class="runner-dataflow" >}}
diff --git a/website/www/site/content/en/get-started/wordcount-example.md b/website/www/site/content/en/get-started/wordcount-example.md
index cd08f5a..9be441e 100644
--- a/website/www/site/content/en/get-started/wordcount-example.md
+++ b/website/www/site/content/en/get-started/wordcount-example.md
@@ -387,18 +387,20 @@
 {{< /highlight >}}
 
 {{< highlight class="runner-flink-local" >}}
-Currently, running wordcount.py on Flink requires a full download of the Beam source code.
-See https://beam.apache.org/roadmap/portability/#python-on-flink for more information.
+python -m apache_beam.examples.wordcount --input /path/to/inputfile \
+                                         --output /path/to/write/counts \
+                                         --runner FlinkRunner
 {{< /highlight >}}
 
 {{< highlight class="runner-flink-cluster" >}}
-Currently, running wordcount.py on Flink requires a full download of the Beam source code.
-See https://beam.apache.org/documentation/runners/flink/ for more information.
+# Running Beam Python on a distributed Flink cluster requires additional configuration.
+# See https://beam.apache.org/documentation/runners/flink/ for more information.
 {{< /highlight >}}
 
 {{< highlight class="runner-spark" >}}
-Currently, running wordcount.py on Spark requires a full download of the Beam source code.
-See https://beam.apache.org/roadmap/portability/#python-on-spark for more information.
+python -m apache_beam.examples.wordcount --input /path/to/inputfile \
+                                         --output /path/to/write/counts \
+                                         --runner SparkRunner
 {{< /highlight >}}
 
 {{< highlight class="runner-dataflow" >}}
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)