[BEAM-9891] TPC-DS module initialization, tables and queries stored (#12436)

* [BEAM-9891] TPC-DS module init, table schemas and queries stored
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/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/settings.gradle b/settings.gradle
index 8289f79..0220857 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -133,6 +133,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"