Merge pull request #12304 from Borzoo/beam-10499-nicer-tostring
[BEAM-10499] Adds a descriptive toString to SamzaRunner KeyedTimerData
diff --git a/CHANGES.md b/CHANGES.md
index c2665e7..0d29362 100644
--- a/CHANGES.md
+++ b/CHANGES.md
@@ -66,6 +66,7 @@
* Shared library for simplifying management of large shared objects added to Python SDK. Example use case is sharing a large TF model object across threads ([BEAM-10417](https://issues.apache.org/jira/browse/BEAM-10417)).
* X feature added (Java/Python) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)).
+* OnTimerContext should not create a new one when processing each element/timer in FnApiDoFnRunner ([BEAM-9839](https://issues.apache.org/jira/browse/BEAM-9839))
## Breaking Changes
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/SplittableParDoExpander.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/SplittableParDoExpander.java
index 181ddab..4e8088f 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/SplittableParDoExpander.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/SplittableParDoExpander.java
@@ -17,6 +17,7 @@
*/
package org.apache.beam.runners.core.construction.graph;
+import com.google.auto.value.AutoValue;
import java.io.IOException;
import java.util.Arrays;
import java.util.Map;
@@ -66,7 +67,7 @@
* information is available to the runner if it chooses to inspect it.
*/
public static TransformReplacement createSizedReplacement() {
- return SizedReplacement.INSTANCE;
+ return SizedReplacement.builder().setDrain(false).build();
}
/**
@@ -91,13 +92,25 @@
* .
*/
public static TransformReplacement createTruncateReplacement() {
- return TruncateReplacement.INSTANCE;
+ return SizedReplacement.builder().setDrain(true).build();
}
/** See {@link #createSizedReplacement()} for details. */
- private static class SizedReplacement implements TransformReplacement {
+ @AutoValue
+ abstract static class SizedReplacement implements TransformReplacement {
- private static final SizedReplacement INSTANCE = new SizedReplacement();
+ static Builder builder() {
+ return new AutoValue_SplittableParDoExpander_SizedReplacement.Builder();
+ }
+
+ abstract boolean isDrain();
+
+ @AutoValue.Builder
+ abstract static class Builder {
+ abstract Builder setDrain(boolean isDrain);
+
+ abstract SizedReplacement build();
+ }
@Override
public MessageWithComponents getReplacement(
@@ -209,14 +222,71 @@
splitAndSize.setEnvironmentId(splittableParDo.getEnvironmentId());
rval.getComponentsBuilder().putTransforms(splitAndSizeId, splitAndSize.build());
}
+ PTransform.Builder newCompositeRoot =
+ splittableParDo
+ .toBuilder()
+ // Clear the original splittable ParDo spec and add all the new transforms as
+ // children.
+ .clearSpec()
+ .addAllSubtransforms(Arrays.asList(pairWithRestrictionId, splitAndSizeId));
String processSizedElementsAndRestrictionsId =
generateUniqueId(
transformId + "/ProcessSizedElementsAndRestrictions",
existingComponents::containsTransforms);
+ String processSizedElementsInputPCollectionId = splitAndSizeOutId;
+ if (isDrain()) {
+ String truncateAndSizeCoderId =
+ generateUniqueId(
+ mainInputPCollection.getCoderId() + "/TruncateAndSize",
+ existingComponents::containsCoders);
+ rval.getComponentsBuilder()
+ .putCoders(
+ truncateAndSizeCoderId,
+ ModelCoders.kvCoder(
+ splitAndSizeOutCoderId, getOrAddDoubleCoder(existingComponents, rval)));
+ String truncateAndSizeOutId =
+ generateUniqueId(
+ mainInputPCollectionId + "/TruncateAndSize",
+ existingComponents::containsPcollections);
+
+ rval.getComponentsBuilder()
+ .putPcollections(
+ truncateAndSizeOutId,
+ PCollection.newBuilder()
+ .setCoderId(truncateAndSizeCoderId)
+ .setIsBounded(mainInputPCollection.getIsBounded())
+ .setWindowingStrategyId(mainInputPCollection.getWindowingStrategyId())
+ .setUniqueName(
+ generateUniquePCollectonName(
+ mainInputPCollection.getUniqueName() + "/TruncateAndSize",
+ existingComponents))
+ .build());
+ String truncateAndSizeId =
+ generateUniqueId(
+ transformId + "/TruncateAndSize", existingComponents::containsTransforms);
+ {
+ PTransform.Builder truncateAndSize = PTransform.newBuilder();
+ truncateAndSize.putInputs(mainInputName, splitAndSizeOutId);
+ truncateAndSize.putAllInputs(sideInputs);
+ truncateAndSize.putOutputs("out", truncateAndSizeOutId);
+ truncateAndSize.setUniqueName(
+ generateUniquePCollectonName(
+ splittableParDo.getUniqueName() + "/TruncateAndSize", existingComponents));
+ truncateAndSize.setSpec(
+ FunctionSpec.newBuilder()
+ .setUrn(PTransformTranslation.SPLITTABLE_TRUNCATE_SIZED_RESTRICTION_URN)
+ .setPayload(splittableParDo.getSpec().getPayload()));
+ truncateAndSize.setEnvironmentId(splittableParDo.getEnvironmentId());
+ rval.getComponentsBuilder().putTransforms(truncateAndSizeId, truncateAndSize.build());
+ }
+ newCompositeRoot.addSubtransforms(truncateAndSizeId);
+ processSizedElementsInputPCollectionId = truncateAndSizeOutId;
+ }
{
PTransform.Builder processSizedElementsAndRestrictions = PTransform.newBuilder();
- processSizedElementsAndRestrictions.putInputs(mainInputName, splitAndSizeOutId);
+ processSizedElementsAndRestrictions.putInputs(
+ mainInputName, processSizedElementsInputPCollectionId);
processSizedElementsAndRestrictions.putAllInputs(sideInputs);
processSizedElementsAndRestrictions.putAllOutputs(splittableParDo.getOutputsMap());
processSizedElementsAndRestrictions.setUniqueName(
@@ -234,20 +304,8 @@
processSizedElementsAndRestrictionsId,
processSizedElementsAndRestrictions.build());
}
-
- PTransform.Builder newCompositeRoot =
- splittableParDo
- .toBuilder()
- // Clear the original splittable ParDo spec and add all the new transforms as
- // children.
- .clearSpec()
- .addAllSubtransforms(
- Arrays.asList(
- pairWithRestrictionId,
- splitAndSizeId,
- processSizedElementsAndRestrictionsId));
+ newCompositeRoot.addSubtransforms(processSizedElementsAndRestrictionsId);
rval.setPtransform(newCompositeRoot);
-
return rval.build();
} catch (IOException e) {
throw new RuntimeException("Unable to perform expansion for transform " + transformId, e);
@@ -298,211 +356,4 @@
}
return prefix + i;
}
-
- /** See {@link #createTruncateReplacement} for details. */
- private static class TruncateReplacement implements TransformReplacement {
- private static final TruncateReplacement INSTANCE = new TruncateReplacement();
-
- @Override
- public MessageWithComponents getReplacement(
- String transformId, ComponentsOrBuilder existingComponents) {
- try {
- MessageWithComponents.Builder rval = MessageWithComponents.newBuilder();
-
- PTransform splittableParDo = existingComponents.getTransformsOrThrow(transformId);
- ParDoPayload payload = ParDoPayload.parseFrom(splittableParDo.getSpec().getPayload());
- // Only perform the expansion if this is a splittable DoFn.
- if (payload.getRestrictionCoderId() == null || payload.getRestrictionCoderId().isEmpty()) {
- return null;
- }
-
- String mainInputName = ParDoTranslation.getMainInputName(splittableParDo);
- String mainInputPCollectionId = splittableParDo.getInputsOrThrow(mainInputName);
- PCollection mainInputPCollection =
- existingComponents.getPcollectionsOrThrow(mainInputPCollectionId);
- Map<String, String> sideInputs =
- Maps.filterKeys(
- splittableParDo.getInputsMap(), input -> payload.containsSideInputs(input));
-
- String pairWithRestrictionOutCoderId =
- generateUniqueId(
- mainInputPCollection.getCoderId() + "/PairWithRestriction",
- existingComponents::containsCoders);
- rval.getComponentsBuilder()
- .putCoders(
- pairWithRestrictionOutCoderId,
- ModelCoders.kvCoder(
- mainInputPCollection.getCoderId(), payload.getRestrictionCoderId()));
-
- String pairWithRestrictionOutId =
- generateUniqueId(
- mainInputPCollectionId + "/PairWithRestriction",
- existingComponents::containsPcollections);
- rval.getComponentsBuilder()
- .putPcollections(
- pairWithRestrictionOutId,
- PCollection.newBuilder()
- .setCoderId(pairWithRestrictionOutCoderId)
- .setIsBounded(mainInputPCollection.getIsBounded())
- .setWindowingStrategyId(mainInputPCollection.getWindowingStrategyId())
- .setUniqueName(
- generateUniquePCollectonName(
- mainInputPCollection.getUniqueName() + "/PairWithRestriction",
- existingComponents))
- .build());
-
- String splitAndSizeOutCoderId =
- generateUniqueId(
- mainInputPCollection.getCoderId() + "/SplitAndSize",
- existingComponents::containsCoders);
- rval.getComponentsBuilder()
- .putCoders(
- splitAndSizeOutCoderId,
- ModelCoders.kvCoder(
- pairWithRestrictionOutCoderId, getOrAddDoubleCoder(existingComponents, rval)));
-
- String splitAndSizeOutId =
- generateUniqueId(
- mainInputPCollectionId + "/SplitAndSize", existingComponents::containsPcollections);
- rval.getComponentsBuilder()
- .putPcollections(
- splitAndSizeOutId,
- PCollection.newBuilder()
- .setCoderId(splitAndSizeOutCoderId)
- .setIsBounded(mainInputPCollection.getIsBounded())
- .setWindowingStrategyId(mainInputPCollection.getWindowingStrategyId())
- .setUniqueName(
- generateUniquePCollectonName(
- mainInputPCollection.getUniqueName() + "/SplitAndSize",
- existingComponents))
- .build());
-
- String truncateAndSizeCoderId =
- generateUniqueId(
- mainInputPCollection.getCoderId() + "/TruncateAndSize",
- existingComponents::containsCoders);
- rval.getComponentsBuilder()
- .putCoders(
- truncateAndSizeCoderId,
- ModelCoders.kvCoder(
- splitAndSizeOutCoderId, getOrAddDoubleCoder(existingComponents, rval)));
- String truncateAndSizeOutId =
- generateUniqueId(
- mainInputPCollectionId + "/TruncateAndSize",
- existingComponents::containsPcollections);
-
- rval.getComponentsBuilder()
- .putPcollections(
- truncateAndSizeOutId,
- PCollection.newBuilder()
- .setCoderId(truncateAndSizeCoderId)
- .setIsBounded(mainInputPCollection.getIsBounded())
- .setWindowingStrategyId(mainInputPCollection.getWindowingStrategyId())
- .setUniqueName(
- generateUniquePCollectonName(
- mainInputPCollection.getUniqueName() + "/TruncateAndSize",
- existingComponents))
- .build());
-
- String pairWithRestrictionId =
- generateUniqueId(
- transformId + "/PairWithRestriction", existingComponents::containsTransforms);
- {
- PTransform.Builder pairWithRestriction = PTransform.newBuilder();
- pairWithRestriction.putAllInputs(splittableParDo.getInputsMap());
- pairWithRestriction.putOutputs("out", pairWithRestrictionOutId);
- pairWithRestriction.setUniqueName(
- generateUniquePCollectonName(
- splittableParDo.getUniqueName() + "/PairWithRestriction", existingComponents));
- pairWithRestriction.setSpec(
- FunctionSpec.newBuilder()
- .setUrn(PTransformTranslation.SPLITTABLE_PAIR_WITH_RESTRICTION_URN)
- .setPayload(splittableParDo.getSpec().getPayload()));
- pairWithRestriction.setEnvironmentId(splittableParDo.getEnvironmentId());
- rval.getComponentsBuilder()
- .putTransforms(pairWithRestrictionId, pairWithRestriction.build());
- }
-
- String splitAndSizeId =
- generateUniqueId(transformId + "/SplitAndSize", existingComponents::containsTransforms);
- {
- PTransform.Builder splitAndSize = PTransform.newBuilder();
- splitAndSize.putInputs(mainInputName, pairWithRestrictionOutId);
- splitAndSize.putAllInputs(sideInputs);
- splitAndSize.putOutputs("out", splitAndSizeOutId);
- splitAndSize.setUniqueName(
- generateUniquePCollectonName(
- splittableParDo.getUniqueName() + "/SplitAndSize", existingComponents));
- splitAndSize.setSpec(
- FunctionSpec.newBuilder()
- .setUrn(PTransformTranslation.SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN)
- .setPayload(splittableParDo.getSpec().getPayload()));
- splitAndSize.setEnvironmentId(splittableParDo.getEnvironmentId());
- rval.getComponentsBuilder().putTransforms(splitAndSizeId, splitAndSize.build());
- }
-
- String truncateAndSizeId =
- generateUniqueId(
- transformId + "/TruncateAndSize", existingComponents::containsTransforms);
- {
- PTransform.Builder truncateAndSize = PTransform.newBuilder();
- truncateAndSize.putInputs(mainInputName, splitAndSizeOutId);
- truncateAndSize.putAllInputs(sideInputs);
- truncateAndSize.putOutputs("out", truncateAndSizeOutId);
- truncateAndSize.setUniqueName(
- generateUniquePCollectonName(
- splittableParDo.getUniqueName() + "/TruncateAndSize", existingComponents));
- truncateAndSize.setSpec(
- FunctionSpec.newBuilder()
- .setUrn(PTransformTranslation.SPLITTABLE_TRUNCATE_SIZED_RESTRICTION_URN)
- .setPayload(splittableParDo.getSpec().getPayload()));
- truncateAndSize.setEnvironmentId(splittableParDo.getEnvironmentId());
- rval.getComponentsBuilder().putTransforms(truncateAndSizeId, truncateAndSize.build());
- }
-
- String processSizedElementsAndRestrictionsId =
- generateUniqueId(
- transformId + "/ProcessSizedElementsAndRestrictions",
- existingComponents::containsTransforms);
- {
- PTransform.Builder processSizedElementsAndRestrictions = PTransform.newBuilder();
- processSizedElementsAndRestrictions.putInputs(mainInputName, truncateAndSizeOutId);
- processSizedElementsAndRestrictions.putAllInputs(sideInputs);
- processSizedElementsAndRestrictions.putAllOutputs(splittableParDo.getOutputsMap());
- processSizedElementsAndRestrictions.setUniqueName(
- generateUniquePCollectonName(
- splittableParDo.getUniqueName() + "/ProcessSizedElementsAndRestrictions",
- existingComponents));
- processSizedElementsAndRestrictions.setSpec(
- FunctionSpec.newBuilder()
- .setUrn(
- PTransformTranslation.SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN)
- .setPayload(splittableParDo.getSpec().getPayload()));
- processSizedElementsAndRestrictions.setEnvironmentId(splittableParDo.getEnvironmentId());
- rval.getComponentsBuilder()
- .putTransforms(
- processSizedElementsAndRestrictionsId,
- processSizedElementsAndRestrictions.build());
- }
-
- PTransform.Builder newCompositeRoot =
- splittableParDo
- .toBuilder()
- // Clear the original splittable ParDo spec and add all the new transforms as
- // children.
- .clearSpec()
- .addAllSubtransforms(
- Arrays.asList(
- pairWithRestrictionId,
- splitAndSizeId,
- truncateAndSizeId,
- processSizedElementsAndRestrictionsId));
- rval.setPtransform(newCompositeRoot);
-
- return rval.build();
- } catch (IOException e) {
- throw new RuntimeException("Unable to perform expansion for transform " + transformId, e);
- }
- }
- }
}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java
index 0995522..226924f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java
@@ -1273,7 +1273,7 @@
public int indexOf(String fieldName) {
Integer index = fieldIndices.get(fieldName);
Preconditions.checkArgument(
- index != null, String.format("Cannot find field %s in schema %s", fieldName, this));
+ index != null, "Cannot find field %s in schema %s", fieldName, this);
return index;
}
@@ -1285,7 +1285,7 @@
/** Return the name of field by index. */
public String nameOf(int fieldIndex) {
String name = fieldIndices.inverse().get(fieldIndex);
- Preconditions.checkArgument(name != null, String.format("Cannot find field %d", fieldIndex));
+ Preconditions.checkArgument(name != null, "Cannot find field %s", fieldIndex);
return name;
}
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTest.java
index 17518a6..05ed20d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTest.java
@@ -351,4 +351,24 @@
Schema schema4 = Schema.builder().addInt32Field("foo").build();
assertFalse(schema1.typesEqual(schema4)); // schema1 and schema4 differ by types
}
+
+ @Test
+ public void testIllegalIndexOf() {
+ Schema schema = Schema.builder().addStringField("foo").build();
+
+ thrown.expect(IllegalArgumentException.class);
+ thrown.expectMessage("Cannot find field bar in schema " + schema);
+
+ schema.indexOf("bar");
+ }
+
+ @Test
+ public void testIllegalNameOf() {
+ Schema schema = Schema.builder().addStringField("foo").build();
+
+ thrown.expect(IllegalArgumentException.class);
+ thrown.expectMessage("Cannot find field 1");
+
+ schema.nameOf(1);
+ }
}
diff --git a/sdks/java/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/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java
index 702bcf4..20e915c 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java
@@ -230,7 +230,7 @@
private final DoFnInvoker<InputT, OutputT> doFnInvoker;
private final StartBundleArgumentProvider startBundleArgumentProvider;
private final ProcessBundleContextBase processContext;
- private OnTimerContext onTimerContext;
+ private final OnTimerContext<?> onTimerContext;
private final FinishBundleArgumentProvider finishBundleArgumentProvider;
/**
@@ -426,6 +426,7 @@
tagToSideInputSpecMap = tagToSideInputSpecMapBuilder.build();
this.splitListener = splitListener;
this.bundleFinalizer = bundleFinalizer;
+ this.onTimerContext = new OnTimerContext();
try {
this.mainInputId = ParDoTranslation.getMainInputName(pTransform);
@@ -1242,7 +1243,6 @@
String timerIdOrTimerFamilyId, TimeDomain timeDomain, Timer<K> timer) {
currentTimer = timer;
currentTimeDomain = timeDomain;
- onTimerContext = new OnTimerContext<>(timer.getUserKey());
// The timerIdOrTimerFamilyId contains either a timerId from timer declaration or timerFamilyId
// from timer family declaration.
String timerId =
@@ -2014,11 +2014,6 @@
/** Provides arguments for a {@link DoFnInvoker} for {@link DoFn.OnTimer @OnTimer}. */
private class OnTimerContext<K> extends BaseArgumentProvider<InputT, OutputT> {
- private final K key;
-
- public OnTimerContext(K key) {
- this.key = key;
- }
private class Context extends DoFn<InputT, OutputT>.OnTimerContext {
private Context() {
@@ -2119,7 +2114,7 @@
@Override
public K key() {
- return key;
+ return (K) currentTimer.getUserKey();
}
@Override
diff --git a/sdks/java/testing/tpcds/build.gradle b/sdks/java/testing/tpcds/build.gradle
new file mode 100644
index 0000000..fa249bc93
--- /dev/null
+++ b/sdks/java/testing/tpcds/build.gradle
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * License); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+plugins {
+ id 'java'
+}
+
+description = "Apache Beam :: SDKs :: Java :: TPC-DS Benchark"
+
+version '2.24.0-SNAPSHOT'
+
+sourceCompatibility = 1.8
+
+repositories {
+ mavenCentral()
+}
+
+dependencies {
+ compile 'com.googlecode.json-simple:json-simple:1.1.1'
+ compile project(path: ":sdks:java:core", configuration: "shadow")
+ compile project(path: ":runners:google-cloud-dataflow-java")
+ compile project(":sdks:java:io:google-cloud-platform")
+ compile project(":sdks:java:extensions:sql")
+ compile group: 'com.google.auto.service', name: 'auto-service', version: '1.0-rc1'
+ testCompile group: 'junit', name: 'junit', version: '4.12'
+}
+
+// When running via Gradle, this property can be used to pass commandline arguments
+// to the tpcds run
+def tpcdsArgsProperty = "tpcds.args"
+
+task run(type: JavaExec) {
+ main = "org.apache.beam.sdk.tpcds.BeamTpcds"
+ classpath = sourceSets.main.runtimeClasspath
+ def tpcdsArgsStr = project.findProperty(tpcdsArgsProperty) ?: ""
+ args tpcdsArgsStr.split()
+}
diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/BeamTpcds.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/BeamTpcds.java
new file mode 100644
index 0000000..0e6e988
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/BeamTpcds.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.tpcds;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.extensions.sql.meta.provider.text.TextTableProvider;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore;
+import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import java.util.List;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+
+/**
+ * To execute this main() method, run the following example command from the command line.
+ *
+ * ./gradlew :sdks:java:testing:tpcds:run -Ptpcds.args="--dataSize=1G \
+ * --queries=3,26,55 \
+ * --tpcParallel=2 \
+ * --project=apache-beam-testing \
+ * --stagingLocation=gs://beamsql_tpcds_1/staging \
+ * --tempLocation=gs://beamsql_tpcds_2/temp \
+ * --runner=DataflowRunner \
+ * --region=us-west1 \
+ * --maxNumWorkers=10"
+ */
+public class BeamTpcds {
+ private static final String dataDirectory = "gs://beamsql_tpcds_1/data";
+ private static final String resultDirectory = "gs://beamsql_tpcds_1/tpcds_results";
+
+ private static String buildTableCreateStatement(String tableName) {
+ String createStatement = "CREATE EXTERNAL TABLE " + tableName + " (%s) TYPE text LOCATION '%s' TBLPROPERTIES '{\"format\":\"csv\", \"csvformat\": \"InformixUnload\"}'";
+ return createStatement;
+ }
+
+ private static String buildDataLocation(String dataSize, String tableName) {
+ String dataLocation = dataDirectory + "/" + dataSize + "/" + tableName + ".dat";
+ return dataLocation;
+ }
+
+ /** Register all tables into env, set their schemas, and set the locations where their corresponding data are stored. */
+ private static void registerAllTables(BeamSqlEnv env, String dataSize) throws Exception {
+ List<String> tableNames = TableSchemaJSONLoader.getAllTableNames();
+ for (String tableName : tableNames) {
+ String createStatement = buildTableCreateStatement(tableName);
+ String tableSchema = TableSchemaJSONLoader.parseTableSchema(tableName);
+ String dataLocation = buildDataLocation(dataSize, tableName);
+ env.executeDdl(String.format(createStatement, tableSchema, dataLocation));
+ }
+ }
+
+ public static void main(String[] args) throws Exception {
+ InMemoryMetaStore inMemoryMetaStore = new InMemoryMetaStore();
+ inMemoryMetaStore.registerProvider(new TextTableProvider());
+
+ TpcdsOptions tpcdsOptions = PipelineOptionsFactory.fromArgs(args).withValidation().as(TpcdsOptions.class);
+
+ String dataSize = TpcdsParametersReader.getAndCheckDataSize(tpcdsOptions);
+ String[] queryNameArr = TpcdsParametersReader.getAndCheckQueryNameArray(tpcdsOptions);
+ int nThreads = TpcdsParametersReader.getAndCheckTpcParallel(tpcdsOptions);
+
+ // Using ExecutorService and CompletionService to fulfill multi-threading functionality
+ ExecutorService executor = Executors.newFixedThreadPool(nThreads);
+ CompletionService<PipelineResult> completion = new ExecutorCompletionService<>(executor);
+
+ BeamSqlEnv env =
+ BeamSqlEnv
+ .builder(inMemoryMetaStore)
+ .setPipelineOptions(tpcdsOptions)
+ .build();
+
+ registerAllTables(env, dataSize);
+
+ // Make an array of pipelines, each pipeline is responsible for running a corresponding query.
+ Pipeline[] pipelines = new Pipeline[queryNameArr.length];
+
+ // Execute all queries, transform the each result into a PCollection<String>, write them into the txt file and store in a GCP directory.
+ for (int i = 0; i < queryNameArr.length; i++) {
+ // For each query, get a copy of pipelineOptions from command line arguments, cast tpcdsOptions as a DataflowPipelineOptions object to read and set required parameters for pipeline execution.
+ TpcdsOptions tpcdsOptionsCopy = PipelineOptionsFactory.fromArgs(args).withValidation().as(TpcdsOptions.class);
+ DataflowPipelineOptions dataflowPipelineOptionsCopy = tpcdsOptionsCopy.as(DataflowPipelineOptions.class);
+
+ // Set a unique job name using the time stamp so that multiple different pipelines can run together.
+ dataflowPipelineOptionsCopy.setJobName(queryNameArr[i] + "result" + System.currentTimeMillis());
+
+ pipelines[i] = Pipeline.create(dataflowPipelineOptionsCopy);
+ String queryString = QueryReader.readQuery(queryNameArr[i]);
+
+ // Query execution
+ PCollection<Row> rows = BeamSqlRelUtils.toPCollection(pipelines[i], env.parseQuery(queryString));
+
+ // Transform the result from PCollection<Row> into PCollection<String>, and write it to the location where results are stored.
+ PCollection<String> rowStrings = rows.apply(MapElements
+ .into(TypeDescriptors.strings())
+ .via((Row row) -> row.toString()));
+ rowStrings.apply(TextIO.write().to(resultDirectory + "/" + dataSize + "/" + pipelines[i].getOptions().getJobName()).withSuffix(".txt").withNumShards(1));
+
+ completion.submit(new TpcdsRun(pipelines[i]));
+ }
+
+ executor.shutdown();
+ }
+}
diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/QueryReader.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/QueryReader.java
new file mode 100644
index 0000000..1666c78
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/QueryReader.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.tpcds;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.util.Objects;
+
+/**
+ * The QueryReader reads query file (the file's extension is '.sql' and content doesn't end with a ';'), write the query as a string and return it.
+ */
+public class QueryReader {
+ /**
+ * Reads a query file (.sql), return the query as a string.
+ * @param queryFileName The name of the query file (such as "query1, query5...") which is stored in resource/queries directory
+ * @return The query string stored in this file.
+ * @throws Exception
+ */
+ public static String readQuery(String queryFileName) throws Exception {
+ // Prepare the file reader.
+ String queryFilePath = Objects.requireNonNull(QueryReader.class.getClassLoader().getResource("queries/" + queryFileName + ".sql")).getPath();
+ File queryFile = new File(queryFilePath);
+ FileReader fileReader = new FileReader(queryFile);
+ BufferedReader reader = new BufferedReader(fileReader);
+
+ // Read the file into stringBuilder.
+ StringBuilder stringBuilder = new StringBuilder();
+ String line;
+ String ls = System.getProperty("line.separator");
+ while ((line = reader.readLine()) != null) {
+ stringBuilder.append(line);
+ stringBuilder.append(ls);
+ }
+
+ // Delete the last new line separator.
+ stringBuilder.deleteCharAt(stringBuilder.length() - 1);
+ reader.close();
+
+ String queryString = stringBuilder.toString();
+
+ return queryString;
+ }
+}
diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoader.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoader.java
new file mode 100644
index 0000000..420386c
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoader.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.tpcds;
+
+import org.apache.beam.repackaged.core.org.apache.commons.compress.utils.FileNameUtils;
+import org.json.simple.JSONArray;
+import org.json.simple.JSONObject;
+import org.json.simple.parser.JSONParser;
+
+import java.io.File;
+import java.io.FileReader;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.ArrayList;
+
+
+/**
+ * TableSchemaJSONLoader can get all table's names from resource/schemas directory and parse a table's schema into a string.
+ */
+public class TableSchemaJSONLoader {
+ /**
+ * Read a table schema json file from resource/schemas directory, parse the file into a string which can be utilized by BeamSqlEnv.executeDdl method.
+ * @param tableName The name of the json file to be read (fo example: item, store_sales).
+ * @return A string that matches the format in BeamSqlEnv.executeDdl method, such as "d_date_sk bigint, d_date_id varchar"
+ * @throws Exception
+ */
+ public static String parseTableSchema(String tableName) throws Exception {
+ String tableFilePath = Objects.requireNonNull(TableSchemaJSONLoader.class.getClassLoader().getResource("schemas/" + tableName +".json")).getPath();
+
+ JSONObject jsonObject = (JSONObject) new JSONParser().parse(new FileReader(new File(tableFilePath)));
+ JSONArray jsonArray = (JSONArray) jsonObject.get("schema");
+
+ // Iterate each element in jsonArray to construct the schema string
+ StringBuilder schemaStringBuilder = new StringBuilder();
+
+ Iterator jsonArrIterator = jsonArray.iterator();
+ Iterator<Map.Entry> recordIterator;
+ while (jsonArrIterator.hasNext()) {
+ recordIterator = ((Map) jsonArrIterator.next()).entrySet().iterator();
+ while (recordIterator.hasNext()) {
+ Map.Entry pair = recordIterator.next();
+
+ if (pair.getKey().equals("type")) {
+ // If the key of the pair is "type", make some modification before appending it to the schemaStringBuilder, then append a comma.
+ String typeName = (String) pair.getValue();
+ if (typeName.toLowerCase().equals("identifier") || typeName.toLowerCase().equals("integer")) {
+ // Use long type to represent int, prevent overflow
+ schemaStringBuilder.append("bigint");
+ } else if (typeName.contains("decimal")) {
+ // Currently Beam SQL doesn't handle "decimal" type properly, use "double" to replace it for now.
+ schemaStringBuilder.append("double");
+ } else {
+ // Currently Beam SQL doesn't handle "date" type properly, use "varchar" replace it for now.
+ schemaStringBuilder.append("varchar");
+ }
+ schemaStringBuilder.append(',');
+ } else {
+ // If the key of the pair is "name", directly append it to the StringBuilder, then append a space.
+ schemaStringBuilder.append((pair.getValue()));
+ schemaStringBuilder.append(' ');
+ }
+ }
+ }
+
+ // Delete the last ',' in schema string
+ if (schemaStringBuilder.length() > 0) {
+ schemaStringBuilder.deleteCharAt(schemaStringBuilder.length() - 1);
+ }
+
+ String schemaString = schemaStringBuilder.toString();
+
+ return schemaString;
+ }
+
+ /**
+ * Get all tables' names. Tables are stored in resource/schemas directory in the form of json files, such as "item.json", "store_sales.json", they'll be converted to "item", "store_sales".
+ * @return The list of names of all tables.
+ */
+ public static List<String> getAllTableNames() {
+ String tableDirPath = Objects.requireNonNull(TableSchemaJSONLoader.class.getClassLoader().getResource("schemas")).getPath();
+ File tableDir = new File(tableDirPath);
+ File[] tableDirListing = tableDir.listFiles();
+
+ List<String> tableNames = new ArrayList<>();
+
+ if (tableDirListing != null) {
+ for (File file : tableDirListing) {
+ // Remove the .json extension in file name
+ tableNames.add(FileNameUtils.getBaseName((file.getName())));
+ }
+ }
+
+ return tableNames;
+ }
+}
diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsOptions.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsOptions.java
new file mode 100644
index 0000000..1c567dd
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsOptions.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.tpcds;
+
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+/** Options used to configure TPC-DS test */
+public interface TpcdsOptions extends PipelineOptions {
+ @Description("The size of TPC-DS data to run query on, user input should contain the unit, such as '1G', '10G'")
+ String getDataSize();
+
+ void setDataSize(String dataSize);
+
+ // Set the return type to be String since reading from the command line (user input will be like "1,2,55" which represent TPC-DS query1, query3, query55)
+ @Description("The queries numbers, read user input as string, numbers separated by commas")
+ String getQueries();
+
+ void setQueries(String queries);
+
+ @Description("The number of queries to run in parallel")
+ @Default.Integer(1)
+ Integer getTpcParallel();
+
+ void setTpcParallel(Integer parallelism);
+}
diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsOptionsRegistrar.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsOptionsRegistrar.java
new file mode 100644
index 0000000..d1ddc9d
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsOptionsRegistrar.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.tpcds;
+
+import com.google.auto.service.AutoService;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
+import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
+
+/** {@link AutoService} registrar for {@link TpcdsOptions}. */
+@AutoService(PipelineOptionsRegistrar.class)
+public class TpcdsOptionsRegistrar implements PipelineOptionsRegistrar{
+
+ @Override
+ public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
+ return ImmutableList.of(TpcdsOptions.class);
+ }
+}
diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsParametersReader.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsParametersReader.java
new file mode 100644
index 0000000..7f0e147
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsParametersReader.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.tpcds;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Get and check the TpcdsOptions' parameters, throw exceptions when user input is invalid
+ */
+public class TpcdsParametersReader {
+
+ /** The data sizes that have been supported. */
+ private static final Set<String> supportedDataSizes = Stream.of("1G", "10G").collect(Collectors.toCollection(HashSet::new));
+
+ /**
+ * Get and check dataSize entered by user. This dataSize has to have been supported.
+ *
+ * @param tpcdsOptions TpcdsOptions object constructed from user input
+ * @return The dateSize user entered, if it is contained in supportedDataSizes set.
+ * @throws Exception
+ */
+ public static String getAndCheckDataSize(TpcdsOptions tpcdsOptions) throws Exception {
+ String dataSize = tpcdsOptions.getDataSize();
+
+ if (!supportedDataSizes.contains(dataSize)) {
+ throw new Exception("The data size you entered has not been supported.");
+ }
+
+ return dataSize;
+ }
+
+ /**
+ * Get and check queries entered by user. This has to be a string of numbers separated by commas or "all" which means run all 99 queiries.
+ * All query numbers have to be between 1 and 99.
+ *
+ * @param tpcdsOptions TpcdsOptions object constructed from user input
+ * @return An array of query names, for example "1,2,7" will be output as "query1,query2,query7"
+ * @throws Exception
+ */
+ public static String[] getAndCheckQueryNameArray(TpcdsOptions tpcdsOptions) throws Exception {
+ String queryNums = tpcdsOptions.getQueries();
+
+ String[] queryNumArr;
+ if (queryNums.toLowerCase().equals("all")) {
+ // All 99 TPC-DS queries need to be executed.
+ queryNumArr = new String[99];
+ for (int i = 0; i < 99; i++) {
+ queryNumArr[i] = Integer.toString(i + 1);
+ }
+ } else {
+ // Split user input queryNums by spaces and commas, get an array of all query numbers.
+ queryNumArr = queryNums.split("[\\s,]+");
+
+ for (String queryNumStr : queryNumArr) {
+ try {
+ int queryNum = Integer.parseInt(queryNumStr);
+ if (queryNum < 1 || queryNum > 99) {
+ throw new Exception("The queries you entered contains invalid query number, please provide integers between 1 and 99.");
+ }
+ } catch (NumberFormatException e) {
+ System.out.println("The queries you entered should be integers, please provide integers between 1 and 99.");
+ }
+ }
+ }
+
+ String[] queryNameArr = new String[queryNumArr.length];
+ for (int i = 0; i < queryNumArr.length; i++) {
+ queryNameArr[i] = "query" + queryNumArr[i];
+ }
+
+ return queryNameArr;
+ }
+
+ /**
+ * Get and check TpcParallel entered by user. This has to be an integer between 1 and 99.
+ *
+ * @param tpcdsOptions TpcdsOptions object constructed from user input.
+ * @return The TpcParallel user entered.
+ * @throws Exception
+ */
+ public static int getAndCheckTpcParallel(TpcdsOptions tpcdsOptions) throws Exception {
+ int nThreads = tpcdsOptions.getTpcParallel();
+
+ if (nThreads < 1 || nThreads > 99) {
+ throw new Exception("The TpcParallel your entered is invalid, please provide an integer between 1 and 99.");
+ }
+
+ return nThreads;
+ }
+}
diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsRun.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsRun.java
new file mode 100644
index 0000000..936c24f
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsRun.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.tpcds;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import java.util.concurrent.Callable;
+
+/**
+ * To fulfill multi-threaded execution
+ */
+public class TpcdsRun implements Callable<PipelineResult> {
+ private final Pipeline pipeline;
+
+ public TpcdsRun (Pipeline pipeline) {
+ this.pipeline = pipeline;
+ }
+
+ @Override
+ public PipelineResult call() {
+ PipelineResult pipelineResult = pipeline.run();
+ pipelineResult.waitUntilFinish();
+ return pipelineResult;
+ }
+}
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query1.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query1.sql
new file mode 100644
index 0000000..3cdf4ca
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query1.sql
@@ -0,0 +1,38 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with customer_total_return as
+(select sr_customer_sk as ctr_customer_sk
+,sr_store_sk as ctr_store_sk
+,sum(SR_FEE) as ctr_total_return
+from store_returns
+,date_dim
+where sr_returned_date_sk = d_date_sk
+and d_year =2000
+group by sr_customer_sk
+,sr_store_sk)
+ select c_customer_id
+from customer_total_return ctr1
+,store
+,customer
+where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2
+from customer_total_return ctr2
+where ctr1.ctr_store_sk = ctr2.ctr_store_sk)
+and s_store_sk = ctr1.ctr_store_sk
+and s_state = 'TN'
+and ctr1.ctr_customer_sk = c_customer_sk
+order by c_customer_id
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query10.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query10.sql
new file mode 100644
index 0000000..d12ef0d
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query10.sql
@@ -0,0 +1,72 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+ cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ count(*) cnt1,
+ cd_purchase_estimate,
+ count(*) cnt2,
+ cd_credit_rating,
+ count(*) cnt3,
+ cd_dep_count,
+ count(*) cnt4,
+ cd_dep_employed_count,
+ count(*) cnt5,
+ cd_dep_college_count,
+ count(*) cnt6
+ from
+ customer c,customer_address ca,customer_demographics
+ where
+ c.c_current_addr_sk = ca.ca_address_sk and
+ ca_county in ('Walker County','Richland County','Gaines County','Douglas County','Dona Ana County') and
+ cd_demo_sk = c.c_current_cdemo_sk and
+ exists (select *
+ from store_sales,date_dim
+ where c.c_customer_sk = ss_customer_sk and
+ ss_sold_date_sk = d_date_sk and
+ d_year = 2002 and
+ d_moy between 4 and 4+3) and
+ (exists (select *
+ from web_sales,date_dim
+ where c.c_customer_sk = ws_bill_customer_sk and
+ ws_sold_date_sk = d_date_sk and
+ d_year = 2002 and
+ d_moy between 4 ANd 4+3) or
+ exists (select *
+ from catalog_sales,date_dim
+ where c.c_customer_sk = cs_ship_customer_sk and
+ cs_sold_date_sk = d_date_sk and
+ d_year = 2002 and
+ d_moy between 4 and 4+3))
+ group by cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating,
+ cd_dep_count,
+ cd_dep_employed_count,
+ cd_dep_college_count
+ order by cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating,
+ cd_dep_count,
+ cd_dep_employed_count,
+ cd_dep_college_count
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query11.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query11.sql
new file mode 100644
index 0000000..3955094
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query11.sql
@@ -0,0 +1,94 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with year_total as (
+ select c_customer_id customer_id
+ ,c_first_name customer_first_name
+ ,c_last_name customer_last_name
+ ,c_preferred_cust_flag customer_preferred_cust_flag
+ ,c_birth_country customer_birth_country
+ ,c_login customer_login
+ ,c_email_address customer_email_address
+ ,d_year dyear
+ ,sum(ss_ext_list_price-ss_ext_discount_amt) year_total
+ ,'s' sale_type
+ from customer
+ ,store_sales
+ ,date_dim
+ where c_customer_sk = ss_customer_sk
+ and ss_sold_date_sk = d_date_sk
+ group by c_customer_id
+ ,c_first_name
+ ,c_last_name
+ ,c_preferred_cust_flag
+ ,c_birth_country
+ ,c_login
+ ,c_email_address
+ ,d_year
+ union all
+ select c_customer_id customer_id
+ ,c_first_name customer_first_name
+ ,c_last_name customer_last_name
+ ,c_preferred_cust_flag customer_preferred_cust_flag
+ ,c_birth_country customer_birth_country
+ ,c_login customer_login
+ ,c_email_address customer_email_address
+ ,d_year dyear
+ ,sum(ws_ext_list_price-ws_ext_discount_amt) year_total
+ ,'w' sale_type
+ from customer
+ ,web_sales
+ ,date_dim
+ where c_customer_sk = ws_bill_customer_sk
+ and ws_sold_date_sk = d_date_sk
+ group by c_customer_id
+ ,c_first_name
+ ,c_last_name
+ ,c_preferred_cust_flag
+ ,c_birth_country
+ ,c_login
+ ,c_email_address
+ ,d_year
+ )
+ select
+ t_s_secyear.customer_id
+ ,t_s_secyear.customer_first_name
+ ,t_s_secyear.customer_last_name
+ ,t_s_secyear.customer_email_address
+ from year_total t_s_firstyear
+ ,year_total t_s_secyear
+ ,year_total t_w_firstyear
+ ,year_total t_w_secyear
+ where t_s_secyear.customer_id = t_s_firstyear.customer_id
+ and t_s_firstyear.customer_id = t_w_secyear.customer_id
+ and t_s_firstyear.customer_id = t_w_firstyear.customer_id
+ and t_s_firstyear.sale_type = 's'
+ and t_w_firstyear.sale_type = 'w'
+ and t_s_secyear.sale_type = 's'
+ and t_w_secyear.sale_type = 'w'
+ and t_s_firstyear.dyear = 2001
+ and t_s_secyear.dyear = 2001+1
+ and t_w_firstyear.dyear = 2001
+ and t_w_secyear.dyear = 2001+1
+ and t_s_firstyear.year_total > 0
+ and t_w_firstyear.year_total > 0
+ and case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else 0.0 end
+ > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else 0.0 end
+ order by t_s_secyear.customer_id
+ ,t_s_secyear.customer_first_name
+ ,t_s_secyear.customer_last_name
+ ,t_s_secyear.customer_email_address
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query12.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query12.sql
new file mode 100644
index 0000000..c015bff
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query12.sql
@@ -0,0 +1,47 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select i_item_id
+ ,i_item_desc
+ ,i_category
+ ,i_class
+ ,i_current_price
+ ,sum(ws_ext_sales_price) as itemrevenue
+ ,sum(ws_ext_sales_price)*100/sum(sum(ws_ext_sales_price)) over
+ (partition by i_class) as revenueratio
+from
+ web_sales
+ ,item
+ ,date_dim
+where
+ ws_item_sk = i_item_sk
+ and i_category in ('Jewelry', 'Sports', 'Books')
+ and ws_sold_date_sk = d_date_sk
+ and d_date between cast('2001-01-12' as date)
+ and (cast('2001-01-12' as date) + 30 days)
+group by
+ i_item_id
+ ,i_item_desc
+ ,i_category
+ ,i_class
+ ,i_current_price
+order by
+ i_category
+ ,i_class
+ ,i_item_id
+ ,i_item_desc
+ ,revenueratio
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query13.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query13.sql
new file mode 100644
index 0000000..47fa265
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query13.sql
@@ -0,0 +1,64 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select avg(ss_quantity)
+ ,avg(ss_ext_sales_price)
+ ,avg(ss_ext_wholesale_cost)
+ ,sum(ss_ext_wholesale_cost)
+ from store_sales
+ ,store
+ ,customer_demographics
+ ,household_demographics
+ ,customer_address
+ ,date_dim
+ where s_store_sk = ss_store_sk
+ and ss_sold_date_sk = d_date_sk and d_year = 2001
+ and((ss_hdemo_sk=hd_demo_sk
+ and cd_demo_sk = ss_cdemo_sk
+ and cd_marital_status = 'D'
+ and cd_education_status = '2 yr Degree'
+ and ss_sales_price between 100.00 and 150.00
+ and hd_dep_count = 3
+ )or
+ (ss_hdemo_sk=hd_demo_sk
+ and cd_demo_sk = ss_cdemo_sk
+ and cd_marital_status = 'S'
+ and cd_education_status = 'Secondary'
+ and ss_sales_price between 50.00 and 100.00
+ and hd_dep_count = 1
+ ) or
+ (ss_hdemo_sk=hd_demo_sk
+ and cd_demo_sk = ss_cdemo_sk
+ and cd_marital_status = 'W'
+ and cd_education_status = 'Advanced Degree'
+ and ss_sales_price between 150.00 and 200.00
+ and hd_dep_count = 1
+ ))
+ and((ss_addr_sk = ca_address_sk
+ and ca_country = 'United States'
+ and ca_state in ('CO', 'IL', 'MN')
+ and ss_net_profit between 100 and 200
+ ) or
+ (ss_addr_sk = ca_address_sk
+ and ca_country = 'United States'
+ and ca_state in ('OH', 'MT', 'NM')
+ and ss_net_profit between 150 and 300
+ ) or
+ (ss_addr_sk = ca_address_sk
+ and ca_country = 'United States'
+ and ca_state in ('TX', 'MO', 'MI')
+ and ss_net_profit between 50 and 250
+ ))
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query14.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query14.sql
new file mode 100644
index 0000000..8d9de3c
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query14.sql
@@ -0,0 +1,223 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with cross_items as
+ (select i_item_sk ss_item_sk
+ from item,
+ (select iss.i_brand_id brand_id
+ ,iss.i_class_id class_id
+ ,iss.i_category_id category_id
+ from store_sales
+ ,item iss
+ ,date_dim d1
+ where ss_item_sk = iss.i_item_sk
+ and ss_sold_date_sk = d1.d_date_sk
+ and d1.d_year between 1998 AND 1998 + 2
+ intersect
+ select ics.i_brand_id
+ ,ics.i_class_id
+ ,ics.i_category_id
+ from catalog_sales
+ ,item ics
+ ,date_dim d2
+ where cs_item_sk = ics.i_item_sk
+ and cs_sold_date_sk = d2.d_date_sk
+ and d2.d_year between 1998 AND 1998 + 2
+ intersect
+ select iws.i_brand_id
+ ,iws.i_class_id
+ ,iws.i_category_id
+ from web_sales
+ ,item iws
+ ,date_dim d3
+ where ws_item_sk = iws.i_item_sk
+ and ws_sold_date_sk = d3.d_date_sk
+ and d3.d_year between 1998 AND 1998 + 2)
+ where i_brand_id = brand_id
+ and i_class_id = class_id
+ and i_category_id = category_id
+),
+ avg_sales as
+ (select avg(quantity*list_price) average_sales
+ from (select ss_quantity quantity
+ ,ss_list_price list_price
+ from store_sales
+ ,date_dim
+ where ss_sold_date_sk = d_date_sk
+ and d_year between 1998 and 1998 + 2
+ union all
+ select cs_quantity quantity
+ ,cs_list_price list_price
+ from catalog_sales
+ ,date_dim
+ where cs_sold_date_sk = d_date_sk
+ and d_year between 1998 and 1998 + 2
+ union all
+ select ws_quantity quantity
+ ,ws_list_price list_price
+ from web_sales
+ ,date_dim
+ where ws_sold_date_sk = d_date_sk
+ and d_year between 1998 and 1998 + 2) x)
+ select channel, i_brand_id,i_class_id,i_category_id,sum(sales), sum(number_sales)
+ from(
+ select 'store' channel, i_brand_id,i_class_id
+ ,i_category_id,sum(ss_quantity*ss_list_price) sales
+ , count(*) number_sales
+ from store_sales
+ ,item
+ ,date_dim
+ where ss_item_sk in (select ss_item_sk from cross_items)
+ and ss_item_sk = i_item_sk
+ and ss_sold_date_sk = d_date_sk
+ and d_year = 1998+2
+ and d_moy = 11
+ group by i_brand_id,i_class_id,i_category_id
+ having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales)
+ union all
+ select 'catalog' channel, i_brand_id,i_class_id,i_category_id, sum(cs_quantity*cs_list_price) sales, count(*) number_sales
+ from catalog_sales
+ ,item
+ ,date_dim
+ where cs_item_sk in (select ss_item_sk from cross_items)
+ and cs_item_sk = i_item_sk
+ and cs_sold_date_sk = d_date_sk
+ and d_year = 1998+2
+ and d_moy = 11
+ group by i_brand_id,i_class_id,i_category_id
+ having sum(cs_quantity*cs_list_price) > (select average_sales from avg_sales)
+ union all
+ select 'web' channel, i_brand_id,i_class_id,i_category_id, sum(ws_quantity*ws_list_price) sales , count(*) number_sales
+ from web_sales
+ ,item
+ ,date_dim
+ where ws_item_sk in (select ss_item_sk from cross_items)
+ and ws_item_sk = i_item_sk
+ and ws_sold_date_sk = d_date_sk
+ and d_year = 1998+2
+ and d_moy = 11
+ group by i_brand_id,i_class_id,i_category_id
+ having sum(ws_quantity*ws_list_price) > (select average_sales from avg_sales)
+ ) y
+ group by rollup (channel, i_brand_id,i_class_id,i_category_id)
+ order by channel,i_brand_id,i_class_id,i_category_id
+ limit 100;
+with cross_items as
+ (select i_item_sk ss_item_sk
+ from item,
+ (select iss.i_brand_id brand_id
+ ,iss.i_class_id class_id
+ ,iss.i_category_id category_id
+ from store_sales
+ ,item iss
+ ,date_dim d1
+ where ss_item_sk = iss.i_item_sk
+ and ss_sold_date_sk = d1.d_date_sk
+ and d1.d_year between 1998 AND 1998 + 2
+ intersect
+ select ics.i_brand_id
+ ,ics.i_class_id
+ ,ics.i_category_id
+ from catalog_sales
+ ,item ics
+ ,date_dim d2
+ where cs_item_sk = ics.i_item_sk
+ and cs_sold_date_sk = d2.d_date_sk
+ and d2.d_year between 1998 AND 1998 + 2
+ intersect
+ select iws.i_brand_id
+ ,iws.i_class_id
+ ,iws.i_category_id
+ from web_sales
+ ,item iws
+ ,date_dim d3
+ where ws_item_sk = iws.i_item_sk
+ and ws_sold_date_sk = d3.d_date_sk
+ and d3.d_year between 1998 AND 1998 + 2) x
+ where i_brand_id = brand_id
+ and i_class_id = class_id
+ and i_category_id = category_id
+),
+ avg_sales as
+(select avg(quantity*list_price) average_sales
+ from (select ss_quantity quantity
+ ,ss_list_price list_price
+ from store_sales
+ ,date_dim
+ where ss_sold_date_sk = d_date_sk
+ and d_year between 1998 and 1998 + 2
+ union all
+ select cs_quantity quantity
+ ,cs_list_price list_price
+ from catalog_sales
+ ,date_dim
+ where cs_sold_date_sk = d_date_sk
+ and d_year between 1998 and 1998 + 2
+ union all
+ select ws_quantity quantity
+ ,ws_list_price list_price
+ from web_sales
+ ,date_dim
+ where ws_sold_date_sk = d_date_sk
+ and d_year between 1998 and 1998 + 2) x)
+ select this_year.channel ty_channel
+ ,this_year.i_brand_id ty_brand
+ ,this_year.i_class_id ty_class
+ ,this_year.i_category_id ty_category
+ ,this_year.sales ty_sales
+ ,this_year.number_sales ty_number_sales
+ ,last_year.channel ly_channel
+ ,last_year.i_brand_id ly_brand
+ ,last_year.i_class_id ly_class
+ ,last_year.i_category_id ly_category
+ ,last_year.sales ly_sales
+ ,last_year.number_sales ly_number_sales
+ from
+ (select 'store' channel, i_brand_id,i_class_id,i_category_id
+ ,sum(ss_quantity*ss_list_price) sales, count(*) number_sales
+ from store_sales
+ ,item
+ ,date_dim
+ where ss_item_sk in (select ss_item_sk from cross_items)
+ and ss_item_sk = i_item_sk
+ and ss_sold_date_sk = d_date_sk
+ and d_week_seq = (select d_week_seq
+ from date_dim
+ where d_year = 1998 + 1
+ and d_moy = 12
+ and d_dom = 16)
+ group by i_brand_id,i_class_id,i_category_id
+ having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales)) this_year,
+ (select 'store' channel, i_brand_id,i_class_id
+ ,i_category_id, sum(ss_quantity*ss_list_price) sales, count(*) number_sales
+ from store_sales
+ ,item
+ ,date_dim
+ where ss_item_sk in (select ss_item_sk from cross_items)
+ and ss_item_sk = i_item_sk
+ and ss_sold_date_sk = d_date_sk
+ and d_week_seq = (select d_week_seq
+ from date_dim
+ where d_year = 1998
+ and d_moy = 12
+ and d_dom = 16)
+ group by i_brand_id,i_class_id,i_category_id
+ having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales)) last_year
+ where this_year.i_brand_id= last_year.i_brand_id
+ and this_year.i_class_id = last_year.i_class_id
+ and this_year.i_category_id = last_year.i_category_id
+ order by this_year.channel, this_year.i_brand_id, this_year.i_class_id, this_year.i_category_id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query15.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query15.sql
new file mode 100644
index 0000000..1ae0c37
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query15.sql
@@ -0,0 +1,33 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select ca_zip
+ ,sum(cs_sales_price)
+ from catalog_sales
+ ,customer
+ ,customer_address
+ ,date_dim
+ where cs_bill_customer_sk = c_customer_sk
+ and c_current_addr_sk = ca_address_sk
+ and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475',
+ '85392', '85460', '80348', '81792')
+ or ca_state in ('CA','WA','GA')
+ or cs_sales_price > 500)
+ and cs_sold_date_sk = d_date_sk
+ and d_qoy = 2 and d_year = 2000
+ group by ca_zip
+ order by ca_zip
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query16.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query16.sql
new file mode 100644
index 0000000..54b7164
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query16.sql
@@ -0,0 +1,44 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+ count(distinct cs_order_number) as "order count"
+ ,sum(cs_ext_ship_cost) as "total shipping cost"
+ ,sum(cs_net_profit) as "total net profit"
+from
+ catalog_sales cs1
+ ,date_dim
+ ,customer_address
+ ,call_center
+where
+ d_date between '1999-2-01' and
+ (cast('1999-2-01' as date) + 60 days)
+and cs1.cs_ship_date_sk = d_date_sk
+and cs1.cs_ship_addr_sk = ca_address_sk
+and ca_state = 'IL'
+and cs1.cs_call_center_sk = cc_call_center_sk
+and cc_county in ('Williamson County','Williamson County','Williamson County','Williamson County',
+ 'Williamson County'
+)
+and exists (select *
+ from catalog_sales cs2
+ where cs1.cs_order_number = cs2.cs_order_number
+ and cs1.cs_warehouse_sk <> cs2.cs_warehouse_sk)
+and not exists(select *
+ from catalog_returns cr1
+ where cs1.cs_order_number = cr1.cr_order_number)
+order by count(distinct cs_order_number)
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query17.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query17.sql
new file mode 100644
index 0000000..19ae6b5
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query17.sql
@@ -0,0 +1,58 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select i_item_id
+ ,i_item_desc
+ ,s_state
+ ,count(ss_quantity) as store_sales_quantitycount
+ ,avg(ss_quantity) as store_sales_quantityave
+ ,stddev_samp(ss_quantity) as store_sales_quantitystdev
+ ,stddev_samp(ss_quantity)/avg(ss_quantity) as store_sales_quantitycov
+ ,count(sr_return_quantity) as store_returns_quantitycount
+ ,avg(sr_return_quantity) as store_returns_quantityave
+ ,stddev_samp(sr_return_quantity) as store_returns_quantitystdev
+ ,stddev_samp(sr_return_quantity)/avg(sr_return_quantity) as store_returns_quantitycov
+ ,count(cs_quantity) as catalog_sales_quantitycount ,avg(cs_quantity) as catalog_sales_quantityave
+ ,stddev_samp(cs_quantity) as catalog_sales_quantitystdev
+ ,stddev_samp(cs_quantity)/avg(cs_quantity) as catalog_sales_quantitycov
+ from store_sales
+ ,store_returns
+ ,catalog_sales
+ ,date_dim d1
+ ,date_dim d2
+ ,date_dim d3
+ ,store
+ ,item
+ where d1.d_quarter_name = '1998Q1'
+ and d1.d_date_sk = ss_sold_date_sk
+ and i_item_sk = ss_item_sk
+ and s_store_sk = ss_store_sk
+ and ss_customer_sk = sr_customer_sk
+ and ss_item_sk = sr_item_sk
+ and ss_ticket_number = sr_ticket_number
+ and sr_returned_date_sk = d2.d_date_sk
+ and d2.d_quarter_name in ('1998Q1','1998Q2','1998Q3')
+ and sr_customer_sk = cs_bill_customer_sk
+ and sr_item_sk = cs_item_sk
+ and cs_sold_date_sk = d3.d_date_sk
+ and d3.d_quarter_name in ('1998Q1','1998Q2','1998Q3')
+ group by i_item_id
+ ,i_item_desc
+ ,s_state
+ order by i_item_id
+ ,i_item_desc
+ ,s_state
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query18.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query18.sql
new file mode 100644
index 0000000..0f03060
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query18.sql
@@ -0,0 +1,47 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select i_item_id,
+ ca_country,
+ ca_state,
+ ca_county,
+ avg( cast(cs_quantity as decimal(12,2))) agg1,
+ avg( cast(cs_list_price as decimal(12,2))) agg2,
+ avg( cast(cs_coupon_amt as decimal(12,2))) agg3,
+ avg( cast(cs_sales_price as decimal(12,2))) agg4,
+ avg( cast(cs_net_profit as decimal(12,2))) agg5,
+ avg( cast(c_birth_year as decimal(12,2))) agg6,
+ avg( cast(cd1.cd_dep_count as decimal(12,2))) agg7
+ from catalog_sales, customer_demographics cd1,
+ customer_demographics cd2, customer, customer_address, date_dim, item
+ where cs_sold_date_sk = d_date_sk and
+ cs_item_sk = i_item_sk and
+ cs_bill_cdemo_sk = cd1.cd_demo_sk and
+ cs_bill_customer_sk = c_customer_sk and
+ cd1.cd_gender = 'M' and
+ cd1.cd_education_status = 'College' and
+ c_current_cdemo_sk = cd2.cd_demo_sk and
+ c_current_addr_sk = ca_address_sk and
+ c_birth_month in (9,5,12,4,1,10) and
+ d_year = 2001 and
+ ca_state in ('ND','WI','AL'
+ ,'NC','OK','MS','TN')
+ group by rollup (i_item_id, ca_country, ca_state, ca_county)
+ order by ca_country,
+ ca_state,
+ ca_county,
+ i_item_id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query19.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query19.sql
new file mode 100644
index 0000000..7a85a10
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query19.sql
@@ -0,0 +1,38 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select i_brand_id brand_id, i_brand brand, i_manufact_id, i_manufact,
+ sum(ss_ext_sales_price) ext_price
+ from date_dim, store_sales, item,customer,customer_address,store
+ where d_date_sk = ss_sold_date_sk
+ and ss_item_sk = i_item_sk
+ and i_manager_id=7
+ and d_moy=11
+ and d_year=1999
+ and ss_customer_sk = c_customer_sk
+ and c_current_addr_sk = ca_address_sk
+ and substr(ca_zip,1,5) <> substr(s_zip,1,5)
+ and ss_store_sk = s_store_sk
+ group by i_brand
+ ,i_brand_id
+ ,i_manufact_id
+ ,i_manufact
+ order by ext_price desc
+ ,i_brand
+ ,i_brand_id
+ ,i_manufact_id
+ ,i_manufact
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query2.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query2.sql
new file mode 100644
index 0000000..9fddb0d
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query2.sql
@@ -0,0 +1,73 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with wscs as
+ (select sold_date_sk
+ ,sales_price
+ from (select ws_sold_date_sk sold_date_sk
+ ,ws_ext_sales_price sales_price
+ from web_sales
+ union all
+ select cs_sold_date_sk sold_date_sk
+ ,cs_ext_sales_price sales_price
+ from catalog_sales)),
+ wswscs as
+ (select d_week_seq,
+ sum(case when (d_day_name='Sunday') then sales_price else null end) sun_sales,
+ sum(case when (d_day_name='Monday') then sales_price else null end) mon_sales,
+ sum(case when (d_day_name='Tuesday') then sales_price else null end) tue_sales,
+ sum(case when (d_day_name='Wednesday') then sales_price else null end) wed_sales,
+ sum(case when (d_day_name='Thursday') then sales_price else null end) thu_sales,
+ sum(case when (d_day_name='Friday') then sales_price else null end) fri_sales,
+ sum(case when (d_day_name='Saturday') then sales_price else null end) sat_sales
+ from wscs
+ ,date_dim
+ where d_date_sk = sold_date_sk
+ group by d_week_seq)
+ select d_week_seq1
+ ,round(sun_sales1/sun_sales2,2)
+ ,round(mon_sales1/mon_sales2,2)
+ ,round(tue_sales1/tue_sales2,2)
+ ,round(wed_sales1/wed_sales2,2)
+ ,round(thu_sales1/thu_sales2,2)
+ ,round(fri_sales1/fri_sales2,2)
+ ,round(sat_sales1/sat_sales2,2)
+ from
+ (select wswscs.d_week_seq d_week_seq1
+ ,sun_sales sun_sales1
+ ,mon_sales mon_sales1
+ ,tue_sales tue_sales1
+ ,wed_sales wed_sales1
+ ,thu_sales thu_sales1
+ ,fri_sales fri_sales1
+ ,sat_sales sat_sales1
+ from wswscs,date_dim
+ where date_dim.d_week_seq = wswscs.d_week_seq and
+ d_year = 2001) y,
+ (select wswscs.d_week_seq d_week_seq2
+ ,sun_sales sun_sales2
+ ,mon_sales mon_sales2
+ ,tue_sales tue_sales2
+ ,wed_sales wed_sales2
+ ,thu_sales thu_sales2
+ ,fri_sales fri_sales2
+ ,sat_sales sat_sales2
+ from wswscs
+ ,date_dim
+ where date_dim.d_week_seq = wswscs.d_week_seq and
+ d_year = 2001+1) z
+ where d_week_seq1=d_week_seq2-53
+ order by d_week_seq1
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query20.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query20.sql
new file mode 100644
index 0000000..95e960b
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query20.sql
@@ -0,0 +1,43 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select i_item_id
+ ,i_item_desc
+ ,i_category
+ ,i_class
+ ,i_current_price
+ ,sum(cs_ext_sales_price) as itemrevenue
+ ,sum(cs_ext_sales_price)*100/sum(sum(cs_ext_sales_price)) over
+ (partition by i_class) as revenueratio
+ from catalog_sales
+ ,item
+ ,date_dim
+ where cs_item_sk = i_item_sk
+ and i_category in ('Jewelry', 'Sports', 'Books')
+ and cs_sold_date_sk = d_date_sk
+ and d_date between cast('2001-01-12' as date)
+ and (cast('2001-01-12' as date) + 30 days)
+ group by i_item_id
+ ,i_item_desc
+ ,i_category
+ ,i_class
+ ,i_current_price
+ order by i_category
+ ,i_class
+ ,i_item_id
+ ,i_item_desc
+ ,revenueratio
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query21.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query21.sql
new file mode 100644
index 0000000..3ba811b
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query21.sql
@@ -0,0 +1,43 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select *
+ from(select w_warehouse_name
+ ,i_item_id
+ ,sum(case when (cast(d_date as date) < cast ('1998-04-08' as date))
+ then inv_quantity_on_hand
+ else 0 end) as inv_before
+ ,sum(case when (cast(d_date as date) >= cast ('1998-04-08' as date))
+ then inv_quantity_on_hand
+ else 0 end) as inv_after
+ from inventory
+ ,warehouse
+ ,item
+ ,date_dim
+ where i_current_price between 0.99 and 1.49
+ and i_item_sk = inv_item_sk
+ and inv_warehouse_sk = w_warehouse_sk
+ and inv_date_sk = d_date_sk
+ and d_date between (cast ('1998-04-08' as date) - 30 days)
+ and (cast ('1998-04-08' as date) + 30 days)
+ group by w_warehouse_name, i_item_id) x
+ where (case when inv_before > 0
+ then inv_after / inv_before
+ else null
+ end) between 2.0/3.0 and 3.0/2.0
+ order by w_warehouse_name
+ ,i_item_id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query22.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query22.sql
new file mode 100644
index 0000000..e983b7b
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query22.sql
@@ -0,0 +1,33 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select i_product_name
+ ,i_brand
+ ,i_class
+ ,i_category
+ ,avg(inv_quantity_on_hand) qoh
+ from inventory
+ ,date_dim
+ ,item
+ where inv_date_sk=d_date_sk
+ and inv_item_sk=i_item_sk
+ and d_month_seq between 1212 and 1212 + 11
+ group by rollup(i_product_name
+ ,i_brand
+ ,i_class
+ ,i_category)
+order by qoh, i_product_name, i_brand, i_class, i_category
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query23.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query23.sql
new file mode 100644
index 0000000..0ee1dab
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query23.sql
@@ -0,0 +1,120 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with frequent_ss_items as
+ (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt
+ from store_sales
+ ,date_dim
+ ,item
+ where ss_sold_date_sk = d_date_sk
+ and ss_item_sk = i_item_sk
+ and d_year in (1999,1999+1,1999+2,1999+3)
+ group by substr(i_item_desc,1,30),i_item_sk,d_date
+ having count(*) >4),
+ max_store_sales as
+ (select max(csales) tpcds_cmax
+ from (select c_customer_sk,sum(ss_quantity*ss_sales_price) csales
+ from store_sales
+ ,customer
+ ,date_dim
+ where ss_customer_sk = c_customer_sk
+ and ss_sold_date_sk = d_date_sk
+ and d_year in (1999,1999+1,1999+2,1999+3)
+ group by c_customer_sk)),
+ best_ss_customer as
+ (select c_customer_sk,sum(ss_quantity*ss_sales_price) ssales
+ from store_sales
+ ,customer
+ where ss_customer_sk = c_customer_sk
+ group by c_customer_sk
+ having sum(ss_quantity*ss_sales_price) > (95/100.0) * (select
+ *
+from
+ max_store_sales))
+ select sum(sales)
+ from (select cs_quantity*cs_list_price sales
+ from catalog_sales
+ ,date_dim
+ where d_year = 1999
+ and d_moy = 1
+ and cs_sold_date_sk = d_date_sk
+ and cs_item_sk in (select item_sk from frequent_ss_items)
+ and cs_bill_customer_sk in (select c_customer_sk from best_ss_customer)
+ union all
+ select ws_quantity*ws_list_price sales
+ from web_sales
+ ,date_dim
+ where d_year = 1999
+ and d_moy = 1
+ and ws_sold_date_sk = d_date_sk
+ and ws_item_sk in (select item_sk from frequent_ss_items)
+ and ws_bill_customer_sk in (select c_customer_sk from best_ss_customer))
+ limit 100;
+with frequent_ss_items as
+ (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt
+ from store_sales
+ ,date_dim
+ ,item
+ where ss_sold_date_sk = d_date_sk
+ and ss_item_sk = i_item_sk
+ and d_year in (1999,1999 + 1,1999 + 2,1999 + 3)
+ group by substr(i_item_desc,1,30),i_item_sk,d_date
+ having count(*) >4),
+ max_store_sales as
+ (select max(csales) tpcds_cmax
+ from (select c_customer_sk,sum(ss_quantity*ss_sales_price) csales
+ from store_sales
+ ,customer
+ ,date_dim
+ where ss_customer_sk = c_customer_sk
+ and ss_sold_date_sk = d_date_sk
+ and d_year in (1999,1999+1,1999+2,1999+3)
+ group by c_customer_sk)),
+ best_ss_customer as
+ (select c_customer_sk,sum(ss_quantity*ss_sales_price) ssales
+ from store_sales
+ ,customer
+ where ss_customer_sk = c_customer_sk
+ group by c_customer_sk
+ having sum(ss_quantity*ss_sales_price) > (95/100.0) * (select
+ *
+ from max_store_sales))
+ select c_last_name,c_first_name,sales
+ from (select c_last_name,c_first_name,sum(cs_quantity*cs_list_price) sales
+ from catalog_sales
+ ,customer
+ ,date_dim
+ where d_year = 1999
+ and d_moy = 1
+ and cs_sold_date_sk = d_date_sk
+ and cs_item_sk in (select item_sk from frequent_ss_items)
+ and cs_bill_customer_sk in (select c_customer_sk from best_ss_customer)
+ and cs_bill_customer_sk = c_customer_sk
+ group by c_last_name,c_first_name
+ union all
+ select c_last_name,c_first_name,sum(ws_quantity*ws_list_price) sales
+ from web_sales
+ ,customer
+ ,date_dim
+ where d_year = 1999
+ and d_moy = 1
+ and ws_sold_date_sk = d_date_sk
+ and ws_item_sk in (select item_sk from frequent_ss_items)
+ and ws_bill_customer_sk in (select c_customer_sk from best_ss_customer)
+ and ws_bill_customer_sk = c_customer_sk
+ group by c_last_name,c_first_name)
+ order by c_last_name,c_first_name,sales
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query24.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query24.sql
new file mode 100644
index 0000000..3f45c4f
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query24.sql
@@ -0,0 +1,119 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with ssales as
+(select c_last_name
+ ,c_first_name
+ ,s_store_name
+ ,ca_state
+ ,s_state
+ ,i_color
+ ,i_current_price
+ ,i_manager_id
+ ,i_units
+ ,i_size
+ ,sum(ss_sales_price) netpaid
+from store_sales
+ ,store_returns
+ ,store
+ ,item
+ ,customer
+ ,customer_address
+where ss_ticket_number = sr_ticket_number
+ and ss_item_sk = sr_item_sk
+ and ss_customer_sk = c_customer_sk
+ and ss_item_sk = i_item_sk
+ and ss_store_sk = s_store_sk
+ and c_current_addr_sk = ca_address_sk
+ and c_birth_country <> upper(ca_country)
+ and s_zip = ca_zip
+and s_market_id=7
+group by c_last_name
+ ,c_first_name
+ ,s_store_name
+ ,ca_state
+ ,s_state
+ ,i_color
+ ,i_current_price
+ ,i_manager_id
+ ,i_units
+ ,i_size)
+select c_last_name
+ ,c_first_name
+ ,s_store_name
+ ,sum(netpaid) paid
+from ssales
+where i_color = 'orchid'
+group by c_last_name
+ ,c_first_name
+ ,s_store_name
+having sum(netpaid) > (select 0.05*avg(netpaid)
+ from ssales)
+order by c_last_name
+ ,c_first_name
+ ,s_store_name
+;
+with ssales as
+(select c_last_name
+ ,c_first_name
+ ,s_store_name
+ ,ca_state
+ ,s_state
+ ,i_color
+ ,i_current_price
+ ,i_manager_id
+ ,i_units
+ ,i_size
+ ,sum(ss_sales_price) netpaid
+from store_sales
+ ,store_returns
+ ,store
+ ,item
+ ,customer
+ ,customer_address
+where ss_ticket_number = sr_ticket_number
+ and ss_item_sk = sr_item_sk
+ and ss_customer_sk = c_customer_sk
+ and ss_item_sk = i_item_sk
+ and ss_store_sk = s_store_sk
+ and c_current_addr_sk = ca_address_sk
+ and c_birth_country <> upper(ca_country)
+ and s_zip = ca_zip
+ and s_market_id = 7
+group by c_last_name
+ ,c_first_name
+ ,s_store_name
+ ,ca_state
+ ,s_state
+ ,i_color
+ ,i_current_price
+ ,i_manager_id
+ ,i_units
+ ,i_size)
+select c_last_name
+ ,c_first_name
+ ,s_store_name
+ ,sum(netpaid) paid
+from ssales
+where i_color = 'chiffon'
+group by c_last_name
+ ,c_first_name
+ ,s_store_name
+having sum(netpaid) > (select 0.05*avg(netpaid)
+ from ssales)
+order by c_last_name
+ ,c_first_name
+ ,s_store_name
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query25.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query25.sql
new file mode 100644
index 0000000..be825fd
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query25.sql
@@ -0,0 +1,61 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+ i_item_id
+ ,i_item_desc
+ ,s_store_id
+ ,s_store_name
+ ,sum(ss_net_profit) as store_sales_profit
+ ,sum(sr_net_loss) as store_returns_loss
+ ,sum(cs_net_profit) as catalog_sales_profit
+ from
+ store_sales
+ ,store_returns
+ ,catalog_sales
+ ,date_dim d1
+ ,date_dim d2
+ ,date_dim d3
+ ,store
+ ,item
+ where
+ d1.d_moy = 4
+ and d1.d_year = 2000
+ and d1.d_date_sk = ss_sold_date_sk
+ and i_item_sk = ss_item_sk
+ and s_store_sk = ss_store_sk
+ and ss_customer_sk = sr_customer_sk
+ and ss_item_sk = sr_item_sk
+ and ss_ticket_number = sr_ticket_number
+ and sr_returned_date_sk = d2.d_date_sk
+ and d2.d_moy between 4 and 10
+ and d2.d_year = 2000
+ and sr_customer_sk = cs_bill_customer_sk
+ and sr_item_sk = cs_item_sk
+ and cs_sold_date_sk = d3.d_date_sk
+ and d3.d_moy between 4 and 10
+ and d3.d_year = 2000
+ group by
+ i_item_id
+ ,i_item_desc
+ ,s_store_id
+ ,s_store_name
+ order by
+ i_item_id
+ ,i_item_desc
+ ,s_store_id
+ ,s_store_name
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query26.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query26.sql
new file mode 100644
index 0000000..772d545
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query26.sql
@@ -0,0 +1,34 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select i_item_id,
+ avg(cs_quantity) agg1,
+ avg(cs_list_price) agg2,
+ avg(cs_coupon_amt) agg3,
+ avg(cs_sales_price) agg4
+ from catalog_sales, customer_demographics, date_dim, item, promotion
+ where cs_sold_date_sk = d_date_sk and
+ cs_item_sk = i_item_sk and
+ cs_bill_cdemo_sk = cd_demo_sk and
+ cs_promo_sk = p_promo_sk and
+ cd_gender = 'F' and
+ cd_marital_status = 'W' and
+ cd_education_status = 'Primary' and
+ (p_channel_email = 'N' or p_channel_event = 'N') and
+ d_year = 1998
+ group by i_item_id
+ order by i_item_id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query27.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query27.sql
new file mode 100644
index 0000000..37cf1f5
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query27.sql
@@ -0,0 +1,36 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select i_item_id,
+ s_state, grouping(s_state) g_state,
+ avg(ss_quantity) agg1,
+ avg(ss_list_price) agg2,
+ avg(ss_coupon_amt) agg3,
+ avg(ss_sales_price) agg4
+ from store_sales, customer_demographics, date_dim, store, item
+ where ss_sold_date_sk = d_date_sk and
+ ss_item_sk = i_item_sk and
+ ss_store_sk = s_store_sk and
+ ss_cdemo_sk = cd_demo_sk and
+ cd_gender = 'F' and
+ cd_marital_status = 'W' and
+ cd_education_status = 'Primary' and
+ d_year = 1998 and
+ s_state in ('TN','TN', 'TN', 'TN', 'TN', 'TN')
+ group by rollup (i_item_id, s_state)
+ order by i_item_id
+ ,s_state
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query28.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query28.sql
new file mode 100644
index 0000000..afe2cf8
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query28.sql
@@ -0,0 +1,66 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select *
+from (select avg(ss_list_price) B1_LP
+ ,count(ss_list_price) B1_CNT
+ ,count(distinct ss_list_price) B1_CNTD
+ from store_sales
+ where ss_quantity between 0 and 5
+ and (ss_list_price between 11 and 11+10
+ or ss_coupon_amt between 460 and 460+1000
+ or ss_wholesale_cost between 14 and 14+20)) B1,
+ (select avg(ss_list_price) B2_LP
+ ,count(ss_list_price) B2_CNT
+ ,count(distinct ss_list_price) B2_CNTD
+ from store_sales
+ where ss_quantity between 6 and 10
+ and (ss_list_price between 91 and 91+10
+ or ss_coupon_amt between 1430 and 1430+1000
+ or ss_wholesale_cost between 32 and 32+20)) B2,
+ (select avg(ss_list_price) B3_LP
+ ,count(ss_list_price) B3_CNT
+ ,count(distinct ss_list_price) B3_CNTD
+ from store_sales
+ where ss_quantity between 11 and 15
+ and (ss_list_price between 66 and 66+10
+ or ss_coupon_amt between 920 and 920+1000
+ or ss_wholesale_cost between 4 and 4+20)) B3,
+ (select avg(ss_list_price) B4_LP
+ ,count(ss_list_price) B4_CNT
+ ,count(distinct ss_list_price) B4_CNTD
+ from store_sales
+ where ss_quantity between 16 and 20
+ and (ss_list_price between 142 and 142+10
+ or ss_coupon_amt between 3054 and 3054+1000
+ or ss_wholesale_cost between 80 and 80+20)) B4,
+ (select avg(ss_list_price) B5_LP
+ ,count(ss_list_price) B5_CNT
+ ,count(distinct ss_list_price) B5_CNTD
+ from store_sales
+ where ss_quantity between 21 and 25
+ and (ss_list_price between 135 and 135+10
+ or ss_coupon_amt between 14180 and 14180+1000
+ or ss_wholesale_cost between 38 and 38+20)) B5,
+ (select avg(ss_list_price) B6_LP
+ ,count(ss_list_price) B6_CNT
+ ,count(distinct ss_list_price) B6_CNTD
+ from store_sales
+ where ss_quantity between 26 and 30
+ and (ss_list_price between 28 and 28+10
+ or ss_coupon_amt between 2513 and 2513+1000
+ or ss_wholesale_cost between 42 and 42+20)) B6
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query29.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query29.sql
new file mode 100644
index 0000000..5db4817
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query29.sql
@@ -0,0 +1,60 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+ i_item_id
+ ,i_item_desc
+ ,s_store_id
+ ,s_store_name
+ ,sum(ss_quantity) as store_sales_quantity
+ ,sum(sr_return_quantity) as store_returns_quantity
+ ,sum(cs_quantity) as catalog_sales_quantity
+ from
+ store_sales
+ ,store_returns
+ ,catalog_sales
+ ,date_dim d1
+ ,date_dim d2
+ ,date_dim d3
+ ,store
+ ,item
+ where
+ d1.d_moy = 4
+ and d1.d_year = 1999
+ and d1.d_date_sk = ss_sold_date_sk
+ and i_item_sk = ss_item_sk
+ and s_store_sk = ss_store_sk
+ and ss_customer_sk = sr_customer_sk
+ and ss_item_sk = sr_item_sk
+ and ss_ticket_number = sr_ticket_number
+ and sr_returned_date_sk = d2.d_date_sk
+ and d2.d_moy between 4 and 4 + 3
+ and d2.d_year = 1999
+ and sr_customer_sk = cs_bill_customer_sk
+ and sr_item_sk = cs_item_sk
+ and cs_sold_date_sk = d3.d_date_sk
+ and d3.d_year in (1999,1999+1,1999+2)
+ group by
+ i_item_id
+ ,i_item_desc
+ ,s_store_id
+ ,s_store_name
+ order by
+ i_item_id
+ ,i_item_desc
+ ,s_store_id
+ ,s_store_name
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query3.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query3.sql
new file mode 100644
index 0000000..fa9025e
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query3.sql
@@ -0,0 +1,34 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select dt.d_year
+ ,item.i_brand_id brand_id
+ ,item.i_brand brand
+ ,sum(ss_ext_sales_price) sum_agg
+ from date_dim dt
+ ,store_sales
+ ,item
+ where dt.d_date_sk = store_sales.ss_sold_date_sk
+ and store_sales.ss_item_sk = item.i_item_sk
+ and item.i_manufact_id = 436
+ and dt.d_moy=12
+ group by dt.d_year
+ ,item.i_brand
+ ,item.i_brand_id
+ order by dt.d_year
+ ,sum_agg desc
+ ,brand_id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query30.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query30.sql
new file mode 100644
index 0000000..fabdf70
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query30.sql
@@ -0,0 +1,44 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with customer_total_return as
+ (select wr_returning_customer_sk as ctr_customer_sk
+ ,ca_state as ctr_state,
+ sum(wr_return_amt) as ctr_total_return
+ from web_returns
+ ,date_dim
+ ,customer_address
+ where wr_returned_date_sk = d_date_sk
+ and d_year =2002
+ and wr_returning_addr_sk = ca_address_sk
+ group by wr_returning_customer_sk
+ ,ca_state)
+ select c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag
+ ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address
+ ,c_last_review_date_sk,ctr_total_return
+ from customer_total_return ctr1
+ ,customer_address
+ ,customer
+ where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2
+ from customer_total_return ctr2
+ where ctr1.ctr_state = ctr2.ctr_state)
+ and ca_address_sk = c_current_addr_sk
+ and ca_state = 'IL'
+ and ctr1.ctr_customer_sk = c_customer_sk
+ order by c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag
+ ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address
+ ,c_last_review_date_sk,ctr_total_return
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query31.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query31.sql
new file mode 100644
index 0000000..4217c55
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query31.sql
@@ -0,0 +1,65 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with ss as
+ (select ca_county,d_qoy, d_year,sum(ss_ext_sales_price) as store_sales
+ from store_sales,date_dim,customer_address
+ where ss_sold_date_sk = d_date_sk
+ and ss_addr_sk=ca_address_sk
+ group by ca_county,d_qoy, d_year),
+ ws as
+ (select ca_county,d_qoy, d_year,sum(ws_ext_sales_price) as web_sales
+ from web_sales,date_dim,customer_address
+ where ws_sold_date_sk = d_date_sk
+ and ws_bill_addr_sk=ca_address_sk
+ group by ca_county,d_qoy, d_year)
+ select
+ ss1.ca_county
+ ,ss1.d_year
+ ,ws2.web_sales/ws1.web_sales web_q1_q2_increase
+ ,ss2.store_sales/ss1.store_sales store_q1_q2_increase
+ ,ws3.web_sales/ws2.web_sales web_q2_q3_increase
+ ,ss3.store_sales/ss2.store_sales store_q2_q3_increase
+ from
+ ss ss1
+ ,ss ss2
+ ,ss ss3
+ ,ws ws1
+ ,ws ws2
+ ,ws ws3
+ where
+ ss1.d_qoy = 1
+ and ss1.d_year = 2000
+ and ss1.ca_county = ss2.ca_county
+ and ss2.d_qoy = 2
+ and ss2.d_year = 2000
+ and ss2.ca_county = ss3.ca_county
+ and ss3.d_qoy = 3
+ and ss3.d_year = 2000
+ and ss1.ca_county = ws1.ca_county
+ and ws1.d_qoy = 1
+ and ws1.d_year = 2000
+ and ws1.ca_county = ws2.ca_county
+ and ws2.d_qoy = 2
+ and ws2.d_year = 2000
+ and ws1.ca_county = ws3.ca_county
+ and ws3.d_qoy = 3
+ and ws3.d_year =2000
+ and case when ws1.web_sales > 0 then ws2.web_sales/ws1.web_sales else null end
+ > case when ss1.store_sales > 0 then ss2.store_sales/ss1.store_sales else null end
+ and case when ws2.web_sales > 0 then ws3.web_sales/ws2.web_sales else null end
+ > case when ss2.store_sales > 0 then ss3.store_sales/ss2.store_sales else null end
+ order by ss1.d_year
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query32.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query32.sql
new file mode 100644
index 0000000..70eb508
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query32.sql
@@ -0,0 +1,41 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select sum(cs_ext_discount_amt) as "excess discount amount"
+from
+ catalog_sales
+ ,item
+ ,date_dim
+where
+i_manufact_id = 269
+and i_item_sk = cs_item_sk
+and d_date between '1998-03-18' and
+ (cast('1998-03-18' as date) + 90 days)
+and d_date_sk = cs_sold_date_sk
+and cs_ext_discount_amt
+ > (
+ select
+ 1.3 * avg(cs_ext_discount_amt)
+ from
+ catalog_sales
+ ,date_dim
+ where
+ cs_item_sk = i_item_sk
+ and d_date between '1998-03-18' and
+ (cast('1998-03-18' as date) + 90 days)
+ and d_date_sk = cs_sold_date_sk
+ )
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query33.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query33.sql
new file mode 100644
index 0000000..bb845af
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query33.sql
@@ -0,0 +1,88 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with ss as (
+ select
+ i_manufact_id,sum(ss_ext_sales_price) total_sales
+ from
+ store_sales,
+ date_dim,
+ customer_address,
+ item
+ where
+ i_manufact_id in (select
+ i_manufact_id
+from
+ item
+where i_category in ('Books'))
+ and ss_item_sk = i_item_sk
+ and ss_sold_date_sk = d_date_sk
+ and d_year = 1999
+ and d_moy = 3
+ and ss_addr_sk = ca_address_sk
+ and ca_gmt_offset = -5
+ group by i_manufact_id),
+ cs as (
+ select
+ i_manufact_id,sum(cs_ext_sales_price) total_sales
+ from
+ catalog_sales,
+ date_dim,
+ customer_address,
+ item
+ where
+ i_manufact_id in (select
+ i_manufact_id
+from
+ item
+where i_category in ('Books'))
+ and cs_item_sk = i_item_sk
+ and cs_sold_date_sk = d_date_sk
+ and d_year = 1999
+ and d_moy = 3
+ and cs_bill_addr_sk = ca_address_sk
+ and ca_gmt_offset = -5
+ group by i_manufact_id),
+ ws as (
+ select
+ i_manufact_id,sum(ws_ext_sales_price) total_sales
+ from
+ web_sales,
+ date_dim,
+ customer_address,
+ item
+ where
+ i_manufact_id in (select
+ i_manufact_id
+from
+ item
+where i_category in ('Books'))
+ and ws_item_sk = i_item_sk
+ and ws_sold_date_sk = d_date_sk
+ and d_year = 1999
+ and d_moy = 3
+ and ws_bill_addr_sk = ca_address_sk
+ and ca_gmt_offset = -5
+ group by i_manufact_id)
+ select i_manufact_id ,sum(total_sales) total_sales
+ from (select * from ss
+ union all
+ select * from cs
+ union all
+ select * from ws) tmp1
+ group by i_manufact_id
+ order by total_sales
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query34.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query34.sql
new file mode 100644
index 0000000..b2c5283
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query34.sql
@@ -0,0 +1,44 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select c_last_name
+ ,c_first_name
+ ,c_salutation
+ ,c_preferred_cust_flag
+ ,ss_ticket_number
+ ,cnt from
+ (select ss_ticket_number
+ ,ss_customer_sk
+ ,count(*) cnt
+ from store_sales,date_dim,store,household_demographics
+ where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+ and store_sales.ss_store_sk = store.s_store_sk
+ and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+ and (date_dim.d_dom between 1 and 3 or date_dim.d_dom between 25 and 28)
+ and (household_demographics.hd_buy_potential = '>10000' or
+ household_demographics.hd_buy_potential = 'Unknown')
+ and household_demographics.hd_vehicle_count > 0
+ and (case when household_demographics.hd_vehicle_count > 0
+ then household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count
+ else null
+ end) > 1.2
+ and date_dim.d_year in (1998,1998+1,1998+2)
+ and store.s_county in ('Williamson County','Williamson County','Williamson County','Williamson County',
+ 'Williamson County','Williamson County','Williamson County','Williamson County')
+ group by ss_ticket_number,ss_customer_sk) dn,customer
+ where ss_customer_sk = c_customer_sk
+ and cnt between 15 and 20
+ order by c_last_name,c_first_name,c_salutation,c_preferred_cust_flag desc, ss_ticket_number
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query35.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query35.sql
new file mode 100644
index 0000000..86ffd3b
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query35.sql
@@ -0,0 +1,71 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+ ca_state,
+ cd_gender,
+ cd_marital_status,
+ cd_dep_count,
+ count(*) cnt1,
+ avg(cd_dep_count),
+ max(cd_dep_count),
+ sum(cd_dep_count),
+ cd_dep_employed_count,
+ count(*) cnt2,
+ avg(cd_dep_employed_count),
+ max(cd_dep_employed_count),
+ sum(cd_dep_employed_count),
+ cd_dep_college_count,
+ count(*) cnt3,
+ avg(cd_dep_college_count),
+ max(cd_dep_college_count),
+ sum(cd_dep_college_count)
+ from
+ customer c,customer_address ca,customer_demographics
+ where
+ c.c_current_addr_sk = ca.ca_address_sk and
+ cd_demo_sk = c.c_current_cdemo_sk and
+ exists (select *
+ from store_sales,date_dim
+ where c.c_customer_sk = ss_customer_sk and
+ ss_sold_date_sk = d_date_sk and
+ d_year = 1999 and
+ d_qoy < 4) and
+ (exists (select *
+ from web_sales,date_dim
+ where c.c_customer_sk = ws_bill_customer_sk and
+ ws_sold_date_sk = d_date_sk and
+ d_year = 1999 and
+ d_qoy < 4) or
+ exists (select *
+ from catalog_sales,date_dim
+ where c.c_customer_sk = cs_ship_customer_sk and
+ cs_sold_date_sk = d_date_sk and
+ d_year = 1999 and
+ d_qoy < 4))
+ group by ca_state,
+ cd_gender,
+ cd_marital_status,
+ cd_dep_count,
+ cd_dep_employed_count,
+ cd_dep_college_count
+ order by ca_state,
+ cd_gender,
+ cd_marital_status,
+ cd_dep_count,
+ cd_dep_employed_count,
+ cd_dep_college_count
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query36.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query36.sql
new file mode 100644
index 0000000..2436ef3
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query36.sql
@@ -0,0 +1,43 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+ sum(ss_net_profit)/sum(ss_ext_sales_price) as gross_margin
+ ,i_category
+ ,i_class
+ ,grouping(i_category)+grouping(i_class) as lochierarchy
+ ,rank() over (
+ partition by grouping(i_category)+grouping(i_class),
+ case when grouping(i_class) = 0 then i_category end
+ order by sum(ss_net_profit)/sum(ss_ext_sales_price) asc) as rank_within_parent
+ from
+ store_sales
+ ,date_dim d1
+ ,item
+ ,store
+ where
+ d1.d_year = 2000
+ and d1.d_date_sk = ss_sold_date_sk
+ and i_item_sk = ss_item_sk
+ and s_store_sk = ss_store_sk
+ and s_state in ('TN','TN','TN','TN',
+ 'TN','TN','TN','TN')
+ group by rollup(i_category,i_class)
+ order by
+ lochierarchy desc
+ ,case when lochierarchy = 0 then i_category end
+ ,rank_within_parent
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query37.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query37.sql
new file mode 100644
index 0000000..24237b7
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query37.sql
@@ -0,0 +1,30 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select i_item_id
+ ,i_item_desc
+ ,i_current_price
+ from item, inventory, date_dim, catalog_sales
+ where i_current_price between 22 and 22 + 30
+ and inv_item_sk = i_item_sk
+ and d_date_sk=inv_date_sk
+ and d_date between cast('2001-06-02' as date) and (cast('2001-06-02' as date) + 60 days)
+ and i_manufact_id in (678,964,918,849)
+ and inv_quantity_on_hand between 100 and 500
+ and cs_item_sk = i_item_sk
+ group by i_item_id,i_item_desc,i_current_price
+ order by i_item_id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query38.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query38.sql
new file mode 100644
index 0000000..3e781ad
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query38.sql
@@ -0,0 +1,36 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select count(*) from (
+ select distinct c_last_name, c_first_name, d_date
+ from store_sales, date_dim, customer
+ where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+ and store_sales.ss_customer_sk = customer.c_customer_sk
+ and d_month_seq between 1212 and 1212 + 11
+ intersect
+ select distinct c_last_name, c_first_name, d_date
+ from catalog_sales, date_dim, customer
+ where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk
+ and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk
+ and d_month_seq between 1212 and 1212 + 11
+ intersect
+ select distinct c_last_name, c_first_name, d_date
+ from web_sales, date_dim, customer
+ where web_sales.ws_sold_date_sk = date_dim.d_date_sk
+ and web_sales.ws_bill_customer_sk = customer.c_customer_sk
+ and d_month_seq between 1212 and 1212 + 11
+) hot_cust
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query39.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query39.sql
new file mode 100644
index 0000000..aaed22a
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query39.sql
@@ -0,0 +1,66 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with inv as
+(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy
+ ,stdev,mean, case mean when 0 then null else stdev/mean end cov
+ from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy
+ ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean
+ from inventory
+ ,item
+ ,warehouse
+ ,date_dim
+ where inv_item_sk = i_item_sk
+ and inv_warehouse_sk = w_warehouse_sk
+ and inv_date_sk = d_date_sk
+ and d_year =1998
+ group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo
+ where case mean when 0 then 0 else stdev/mean end > 1)
+select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov
+ ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov
+from inv inv1,inv inv2
+where inv1.i_item_sk = inv2.i_item_sk
+ and inv1.w_warehouse_sk = inv2.w_warehouse_sk
+ and inv1.d_moy=4
+ and inv2.d_moy=4+1
+order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov
+ ,inv2.d_moy,inv2.mean, inv2.cov
+;
+with inv as
+(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy
+ ,stdev,mean, case mean when 0 then null else stdev/mean end cov
+ from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy
+ ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean
+ from inventory
+ ,item
+ ,warehouse
+ ,date_dim
+ where inv_item_sk = i_item_sk
+ and inv_warehouse_sk = w_warehouse_sk
+ and inv_date_sk = d_date_sk
+ and d_year =1998
+ group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo
+ where case mean when 0 then 0 else stdev/mean end > 1)
+select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov
+ ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov
+from inv inv1,inv inv2
+where inv1.i_item_sk = inv2.i_item_sk
+ and inv1.w_warehouse_sk = inv2.w_warehouse_sk
+ and inv1.d_moy=4
+ and inv2.d_moy=4+1
+ and inv1.cov > 1.5
+order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov
+ ,inv2.d_moy,inv2.mean, inv2.cov
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query4.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query4.sql
new file mode 100644
index 0000000..364c1a5
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query4.sql
@@ -0,0 +1,129 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with year_total as (
+ select c_customer_id customer_id
+ ,c_first_name customer_first_name
+ ,c_last_name customer_last_name
+ ,c_preferred_cust_flag customer_preferred_cust_flag
+ ,c_birth_country customer_birth_country
+ ,c_login customer_login
+ ,c_email_address customer_email_address
+ ,d_year dyear
+ ,sum(((ss_ext_list_price-ss_ext_wholesale_cost-ss_ext_discount_amt)+ss_ext_sales_price)/2) year_total
+ ,'s' sale_type
+ from customer
+ ,store_sales
+ ,date_dim
+ where c_customer_sk = ss_customer_sk
+ and ss_sold_date_sk = d_date_sk
+ group by c_customer_id
+ ,c_first_name
+ ,c_last_name
+ ,c_preferred_cust_flag
+ ,c_birth_country
+ ,c_login
+ ,c_email_address
+ ,d_year
+ union all
+ select c_customer_id customer_id
+ ,c_first_name customer_first_name
+ ,c_last_name customer_last_name
+ ,c_preferred_cust_flag customer_preferred_cust_flag
+ ,c_birth_country customer_birth_country
+ ,c_login customer_login
+ ,c_email_address customer_email_address
+ ,d_year dyear
+ ,sum((((cs_ext_list_price-cs_ext_wholesale_cost-cs_ext_discount_amt)+cs_ext_sales_price)/2) ) year_total
+ ,'c' sale_type
+ from customer
+ ,catalog_sales
+ ,date_dim
+ where c_customer_sk = cs_bill_customer_sk
+ and cs_sold_date_sk = d_date_sk
+ group by c_customer_id
+ ,c_first_name
+ ,c_last_name
+ ,c_preferred_cust_flag
+ ,c_birth_country
+ ,c_login
+ ,c_email_address
+ ,d_year
+union all
+ select c_customer_id customer_id
+ ,c_first_name customer_first_name
+ ,c_last_name customer_last_name
+ ,c_preferred_cust_flag customer_preferred_cust_flag
+ ,c_birth_country customer_birth_country
+ ,c_login customer_login
+ ,c_email_address customer_email_address
+ ,d_year dyear
+ ,sum((((ws_ext_list_price-ws_ext_wholesale_cost-ws_ext_discount_amt)+ws_ext_sales_price)/2) ) year_total
+ ,'w' sale_type
+ from customer
+ ,web_sales
+ ,date_dim
+ where c_customer_sk = ws_bill_customer_sk
+ and ws_sold_date_sk = d_date_sk
+ group by c_customer_id
+ ,c_first_name
+ ,c_last_name
+ ,c_preferred_cust_flag
+ ,c_birth_country
+ ,c_login
+ ,c_email_address
+ ,d_year
+ )
+ select
+ t_s_secyear.customer_id
+ ,t_s_secyear.customer_first_name
+ ,t_s_secyear.customer_last_name
+ ,t_s_secyear.customer_email_address
+ from year_total t_s_firstyear
+ ,year_total t_s_secyear
+ ,year_total t_c_firstyear
+ ,year_total t_c_secyear
+ ,year_total t_w_firstyear
+ ,year_total t_w_secyear
+ where t_s_secyear.customer_id = t_s_firstyear.customer_id
+ and t_s_firstyear.customer_id = t_c_secyear.customer_id
+ and t_s_firstyear.customer_id = t_c_firstyear.customer_id
+ and t_s_firstyear.customer_id = t_w_firstyear.customer_id
+ and t_s_firstyear.customer_id = t_w_secyear.customer_id
+ and t_s_firstyear.sale_type = 's'
+ and t_c_firstyear.sale_type = 'c'
+ and t_w_firstyear.sale_type = 'w'
+ and t_s_secyear.sale_type = 's'
+ and t_c_secyear.sale_type = 'c'
+ and t_w_secyear.sale_type = 'w'
+ and t_s_firstyear.dyear = 2001
+ and t_s_secyear.dyear = 2001+1
+ and t_c_firstyear.dyear = 2001
+ and t_c_secyear.dyear = 2001+1
+ and t_w_firstyear.dyear = 2001
+ and t_w_secyear.dyear = 2001+1
+ and t_s_firstyear.year_total > 0
+ and t_c_firstyear.year_total > 0
+ and t_w_firstyear.year_total > 0
+ and case when t_c_firstyear.year_total > 0 then t_c_secyear.year_total / t_c_firstyear.year_total else null end
+ > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else null end
+ and case when t_c_firstyear.year_total > 0 then t_c_secyear.year_total / t_c_firstyear.year_total else null end
+ > case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else null end
+ order by t_s_secyear.customer_id
+ ,t_s_secyear.customer_first_name
+ ,t_s_secyear.customer_last_name
+ ,t_s_secyear.customer_email_address
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query40.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query40.sql
new file mode 100644
index 0000000..41a8cba
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query40.sql
@@ -0,0 +1,41 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+ w_state
+ ,i_item_id
+ ,sum(case when (cast(d_date as date) < cast ('1998-04-08' as date))
+ then cs_sales_price - coalesce(cr_refunded_cash,0) else 0 end) as sales_before
+ ,sum(case when (cast(d_date as date) >= cast ('1998-04-08' as date))
+ then cs_sales_price - coalesce(cr_refunded_cash,0) else 0 end) as sales_after
+ from
+ catalog_sales left outer join catalog_returns on
+ (cs_order_number = cr_order_number
+ and cs_item_sk = cr_item_sk)
+ ,warehouse
+ ,item
+ ,date_dim
+ where
+ i_current_price between 0.99 and 1.49
+ and i_item_sk = cs_item_sk
+ and cs_warehouse_sk = w_warehouse_sk
+ and cs_sold_date_sk = d_date_sk
+ and d_date between (cast ('1998-04-08' as date) - 30 days)
+ and (cast ('1998-04-08' as date) + 30 days)
+ group by
+ w_state,i_item_id
+ order by w_state,i_item_id
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query41.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query41.sql
new file mode 100644
index 0000000..e42bef9
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query41.sql
@@ -0,0 +1,65 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select distinct(i_product_name)
+ from item i1
+ where i_manufact_id between 742 and 742+40
+ and (select count(*) as item_cnt
+ from item
+ where (i_manufact = i1.i_manufact and
+ ((i_category = 'Women' and
+ (i_color = 'orchid' or i_color = 'papaya') and
+ (i_units = 'Pound' or i_units = 'Lb') and
+ (i_size = 'petite' or i_size = 'medium')
+ ) or
+ (i_category = 'Women' and
+ (i_color = 'burlywood' or i_color = 'navy') and
+ (i_units = 'Bundle' or i_units = 'Each') and
+ (i_size = 'N/A' or i_size = 'extra large')
+ ) or
+ (i_category = 'Men' and
+ (i_color = 'bisque' or i_color = 'azure') and
+ (i_units = 'N/A' or i_units = 'Tsp') and
+ (i_size = 'small' or i_size = 'large')
+ ) or
+ (i_category = 'Men' and
+ (i_color = 'chocolate' or i_color = 'cornflower') and
+ (i_units = 'Bunch' or i_units = 'Gross') and
+ (i_size = 'petite' or i_size = 'medium')
+ ))) or
+ (i_manufact = i1.i_manufact and
+ ((i_category = 'Women' and
+ (i_color = 'salmon' or i_color = 'midnight') and
+ (i_units = 'Oz' or i_units = 'Box') and
+ (i_size = 'petite' or i_size = 'medium')
+ ) or
+ (i_category = 'Women' and
+ (i_color = 'snow' or i_color = 'steel') and
+ (i_units = 'Carton' or i_units = 'Tbl') and
+ (i_size = 'N/A' or i_size = 'extra large')
+ ) or
+ (i_category = 'Men' and
+ (i_color = 'purple' or i_color = 'gainsboro') and
+ (i_units = 'Dram' or i_units = 'Unknown') and
+ (i_size = 'small' or i_size = 'large')
+ ) or
+ (i_category = 'Men' and
+ (i_color = 'metallic' or i_color = 'forest') and
+ (i_units = 'Gram' or i_units = 'Ounce') and
+ (i_size = 'petite' or i_size = 'medium')
+ )))) > 0
+ order by i_product_name
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query42.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query42.sql
new file mode 100644
index 0000000..a7a8bc8
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query42.sql
@@ -0,0 +1,35 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select dt.d_year
+ ,item.i_category_id
+ ,item.i_category
+ ,sum(ss_ext_sales_price)
+ from date_dim dt
+ ,store_sales
+ ,item
+ where dt.d_date_sk = store_sales.ss_sold_date_sk
+ and store_sales.ss_item_sk = item.i_item_sk
+ and item.i_manager_id = 1
+ and dt.d_moy=12
+ and dt.d_year=1998
+ group by dt.d_year
+ ,item.i_category_id
+ ,item.i_category
+ order by sum(ss_ext_sales_price) desc,dt.d_year
+ ,item.i_category_id
+ ,item.i_category
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query43.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query43.sql
new file mode 100644
index 0000000..db4db2b
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query43.sql
@@ -0,0 +1,32 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select s_store_name, s_store_id,
+ sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales,
+ sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales,
+ sum(case when (d_day_name='Tuesday') then ss_sales_price else null end) tue_sales,
+ sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales,
+ sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales,
+ sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales,
+ sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales
+ from date_dim, store_sales, store
+ where d_date_sk = ss_sold_date_sk and
+ s_store_sk = ss_store_sk and
+ s_gmt_offset = -5 and
+ d_year = 1998
+ group by s_store_name, s_store_id
+ order by s_store_name, s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query44.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query44.sql
new file mode 100644
index 0000000..897ac5f
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query44.sql
@@ -0,0 +1,48 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select asceding.rnk, i1.i_product_name best_performing, i2.i_product_name worst_performing
+from(select *
+ from (select item_sk,rank() over (order by rank_col asc) rnk
+ from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col
+ from store_sales ss1
+ where ss_store_sk = 2
+ group by ss_item_sk
+ having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col
+ from store_sales
+ where ss_store_sk = 2
+ and ss_hdemo_sk is null
+ group by ss_store_sk))V1)V11
+ where rnk < 11) asceding,
+ (select *
+ from (select item_sk,rank() over (order by rank_col desc) rnk
+ from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col
+ from store_sales ss1
+ where ss_store_sk = 2
+ group by ss_item_sk
+ having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col
+ from store_sales
+ where ss_store_sk = 2
+ and ss_hdemo_sk is null
+ group by ss_store_sk))V2)V21
+ where rnk < 11) descending,
+item i1,
+item i2
+where asceding.rnk = descending.rnk
+ and i1.i_item_sk=asceding.item_sk
+ and i2.i_item_sk=descending.item_sk
+order by asceding.rnk
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query45.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query45.sql
new file mode 100644
index 0000000..765456a
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query45.sql
@@ -0,0 +1,33 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select ca_zip, ca_county, sum(ws_sales_price)
+ from web_sales, customer, customer_address, date_dim, item
+ where ws_bill_customer_sk = c_customer_sk
+ and c_current_addr_sk = ca_address_sk
+ and ws_item_sk = i_item_sk
+ and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', '85392', '85460', '80348', '81792')
+ or
+ i_item_id in (select i_item_id
+ from item
+ where i_item_sk in (2, 3, 5, 7, 11, 13, 17, 19, 23, 29)
+ )
+ )
+ and ws_sold_date_sk = d_date_sk
+ and d_qoy = 2 and d_year = 2000
+ group by ca_zip, ca_county
+ order by ca_zip, ca_county
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query46.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query46.sql
new file mode 100644
index 0000000..f58de5e
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query46.sql
@@ -0,0 +1,48 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select c_last_name
+ ,c_first_name
+ ,ca_city
+ ,bought_city
+ ,ss_ticket_number
+ ,amt,profit
+ from
+ (select ss_ticket_number
+ ,ss_customer_sk
+ ,ca_city bought_city
+ ,sum(ss_coupon_amt) amt
+ ,sum(ss_net_profit) profit
+ from store_sales,date_dim,store,household_demographics,customer_address
+ where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+ and store_sales.ss_store_sk = store.s_store_sk
+ and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+ and store_sales.ss_addr_sk = customer_address.ca_address_sk
+ and (household_demographics.hd_dep_count = 5 or
+ household_demographics.hd_vehicle_count= 3)
+ and date_dim.d_dow in (6,0)
+ and date_dim.d_year in (1999,1999+1,1999+2)
+ and store.s_city in ('Midway','Fairview','Fairview','Midway','Fairview')
+ group by ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city) dn,customer,customer_address current_addr
+ where ss_customer_sk = c_customer_sk
+ and customer.c_current_addr_sk = current_addr.ca_address_sk
+ and current_addr.ca_city <> bought_city
+ order by c_last_name
+ ,c_first_name
+ ,ca_city
+ ,bought_city
+ ,ss_ticket_number
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query47.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query47.sql
new file mode 100644
index 0000000..9d2e4ca
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query47.sql
@@ -0,0 +1,64 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with v1 as(
+ select i_category, i_brand,
+ s_store_name, s_company_name,
+ d_year, d_moy,
+ sum(ss_sales_price) sum_sales,
+ avg(sum(ss_sales_price)) over
+ (partition by i_category, i_brand,
+ s_store_name, s_company_name, d_year)
+ avg_monthly_sales,
+ rank() over
+ (partition by i_category, i_brand,
+ s_store_name, s_company_name
+ order by d_year, d_moy) rn
+ from item, store_sales, date_dim, store
+ where ss_item_sk = i_item_sk and
+ ss_sold_date_sk = d_date_sk and
+ ss_store_sk = s_store_sk and
+ (
+ d_year = 2000 or
+ ( d_year = 2000-1 and d_moy =12) or
+ ( d_year = 2000+1 and d_moy =1)
+ )
+ group by i_category, i_brand,
+ s_store_name, s_company_name,
+ d_year, d_moy),
+ v2 as(
+ select v1.i_category, v1.i_brand
+ ,v1.d_year, v1.d_moy
+ ,v1.avg_monthly_sales
+ ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum
+ from v1, v1 v1_lag, v1 v1_lead
+ where v1.i_category = v1_lag.i_category and
+ v1.i_category = v1_lead.i_category and
+ v1.i_brand = v1_lag.i_brand and
+ v1.i_brand = v1_lead.i_brand and
+ v1.s_store_name = v1_lag.s_store_name and
+ v1.s_store_name = v1_lead.s_store_name and
+ v1.s_company_name = v1_lag.s_company_name and
+ v1.s_company_name = v1_lead.s_company_name and
+ v1.rn = v1_lag.rn + 1 and
+ v1.rn = v1_lead.rn - 1)
+ select *
+ from v2
+ where d_year = 2000 and
+ avg_monthly_sales > 0 and
+ case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1
+ order by sum_sales - avg_monthly_sales, nsum
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query48.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query48.sql
new file mode 100644
index 0000000..a924396
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query48.sql
@@ -0,0 +1,79 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select sum (ss_quantity)
+ from store_sales, store, customer_demographics, customer_address, date_dim
+ where s_store_sk = ss_store_sk
+ and ss_sold_date_sk = d_date_sk and d_year = 1998
+ and
+ (
+ (
+ cd_demo_sk = ss_cdemo_sk
+ and
+ cd_marital_status = 'M'
+ and
+ cd_education_status = '4 yr Degree'
+ and
+ ss_sales_price between 100.00 and 150.00
+ )
+ or
+ (
+ cd_demo_sk = ss_cdemo_sk
+ and
+ cd_marital_status = 'D'
+ and
+ cd_education_status = 'Primary'
+ and
+ ss_sales_price between 50.00 and 100.00
+ )
+ or
+ (
+ cd_demo_sk = ss_cdemo_sk
+ and
+ cd_marital_status = 'U'
+ and
+ cd_education_status = 'Advanced Degree'
+ and
+ ss_sales_price between 150.00 and 200.00
+ )
+ )
+ and
+ (
+ (
+ ss_addr_sk = ca_address_sk
+ and
+ ca_country = 'United States'
+ and
+ ca_state in ('KY', 'GA', 'NM')
+ and ss_net_profit between 0 and 2000
+ )
+ or
+ (ss_addr_sk = ca_address_sk
+ and
+ ca_country = 'United States'
+ and
+ ca_state in ('MT', 'OR', 'IN')
+ and ss_net_profit between 150 and 3000
+ )
+ or
+ (ss_addr_sk = ca_address_sk
+ and
+ ca_country = 'United States'
+ and
+ ca_state in ('WI', 'MO', 'WV')
+ and ss_net_profit between 50 and 25000
+ )
+ )
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query49.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query49.sql
new file mode 100644
index 0000000..2e29e15
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query49.sql
@@ -0,0 +1,142 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select channel, item, return_ratio, return_rank, currency_rank from
+ (select
+ 'web' as channel
+ ,web.item
+ ,web.return_ratio
+ ,web.return_rank
+ ,web.currency_rank
+ from (
+ select
+ item
+ ,return_ratio
+ ,currency_ratio
+ ,rank() over (order by return_ratio) as return_rank
+ ,rank() over (order by currency_ratio) as currency_rank
+ from
+ ( select ws.ws_item_sk as item
+ ,(cast(sum(coalesce(wr.wr_return_quantity,0)) as decimal(15,4))/
+ cast(sum(coalesce(ws.ws_quantity,0)) as decimal(15,4) )) as return_ratio
+ ,(cast(sum(coalesce(wr.wr_return_amt,0)) as decimal(15,4))/
+ cast(sum(coalesce(ws.ws_net_paid,0)) as decimal(15,4) )) as currency_ratio
+ from
+ web_sales ws left outer join web_returns wr
+ on (ws.ws_order_number = wr.wr_order_number and
+ ws.ws_item_sk = wr.wr_item_sk)
+ ,date_dim
+ where
+ wr.wr_return_amt > 10000
+ and ws.ws_net_profit > 1
+ and ws.ws_net_paid > 0
+ and ws.ws_quantity > 0
+ and ws_sold_date_sk = d_date_sk
+ and d_year = 2000
+ and d_moy = 12
+ group by ws.ws_item_sk
+ ) in_web
+ ) web
+ where
+ (
+ web.return_rank <= 10
+ or
+ web.currency_rank <= 10
+ )
+ union
+ select
+ 'catalog' as channel
+ ,catalog.item
+ ,catalog.return_ratio
+ ,catalog.return_rank
+ ,catalog.currency_rank
+ from (
+ select
+ item
+ ,return_ratio
+ ,currency_ratio
+ ,rank() over (order by return_ratio) as return_rank
+ ,rank() over (order by currency_ratio) as currency_rank
+ from
+ ( select
+ cs.cs_item_sk as item
+ ,(cast(sum(coalesce(cr.cr_return_quantity,0)) as decimal(15,4))/
+ cast(sum(coalesce(cs.cs_quantity,0)) as decimal(15,4) )) as return_ratio
+ ,(cast(sum(coalesce(cr.cr_return_amount,0)) as decimal(15,4))/
+ cast(sum(coalesce(cs.cs_net_paid,0)) as decimal(15,4) )) as currency_ratio
+ from
+ catalog_sales cs left outer join catalog_returns cr
+ on (cs.cs_order_number = cr.cr_order_number and
+ cs.cs_item_sk = cr.cr_item_sk)
+ ,date_dim
+ where
+ cr.cr_return_amount > 10000
+ and cs.cs_net_profit > 1
+ and cs.cs_net_paid > 0
+ and cs.cs_quantity > 0
+ and cs_sold_date_sk = d_date_sk
+ and d_year = 2000
+ and d_moy = 12
+ group by cs.cs_item_sk
+ ) in_cat
+ ) catalog
+ where
+ (
+ catalog.return_rank <= 10
+ or
+ catalog.currency_rank <=10
+ )
+ union
+ select
+ 'store' as channel
+ ,store.item
+ ,store.return_ratio
+ ,store.return_rank
+ ,store.currency_rank
+ from (
+ select
+ item
+ ,return_ratio
+ ,currency_ratio
+ ,rank() over (order by return_ratio) as return_rank
+ ,rank() over (order by currency_ratio) as currency_rank
+ from
+ ( select sts.ss_item_sk as item
+ ,(cast(sum(coalesce(sr.sr_return_quantity,0)) as decimal(15,4))/cast(sum(coalesce(sts.ss_quantity,0)) as decimal(15,4) )) as return_ratio
+ ,(cast(sum(coalesce(sr.sr_return_amt,0)) as decimal(15,4))/cast(sum(coalesce(sts.ss_net_paid,0)) as decimal(15,4) )) as currency_ratio
+ from
+ store_sales sts left outer join store_returns sr
+ on (sts.ss_ticket_number = sr.sr_ticket_number and sts.ss_item_sk = sr.sr_item_sk)
+ ,date_dim
+ where
+ sr.sr_return_amt > 10000
+ and sts.ss_net_profit > 1
+ and sts.ss_net_paid > 0
+ and sts.ss_quantity > 0
+ and ss_sold_date_sk = d_date_sk
+ and d_year = 2000
+ and d_moy = 12
+ group by sts.ss_item_sk
+ ) in_store
+ ) store
+ where (
+ store.return_rank <= 10
+ or
+ store.currency_rank <= 10
+ )
+ )
+ order by 1,4,5,2
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query5.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query5.sql
new file mode 100644
index 0000000..da2e30a
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query5.sql
@@ -0,0 +1,141 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with ssr as
+ (select s_store_id,
+ sum(sales_price) as sales,
+ sum(profit) as profit,
+ sum(return_amt) as returns,
+ sum(net_loss) as profit_loss
+ from
+ ( select ss_store_sk as store_sk,
+ ss_sold_date_sk as date_sk,
+ ss_ext_sales_price as sales_price,
+ ss_net_profit as profit,
+ cast(0 as decimal(7,2)) as return_amt,
+ cast(0 as decimal(7,2)) as net_loss
+ from store_sales
+ union all
+ select sr_store_sk as store_sk,
+ sr_returned_date_sk as date_sk,
+ cast(0 as decimal(7,2)) as sales_price,
+ cast(0 as decimal(7,2)) as profit,
+ sr_return_amt as return_amt,
+ sr_net_loss as net_loss
+ from store_returns
+ ) salesreturns,
+ date_dim,
+ store
+ where date_sk = d_date_sk
+ and d_date between cast('1998-08-04' as date)
+ and (cast('1998-08-04' as date) + 14 days)
+ and store_sk = s_store_sk
+ group by s_store_id)
+ ,
+ csr as
+ (select cp_catalog_page_id,
+ sum(sales_price) as sales,
+ sum(profit) as profit,
+ sum(return_amt) as returns,
+ sum(net_loss) as profit_loss
+ from
+ ( select cs_catalog_page_sk as page_sk,
+ cs_sold_date_sk as date_sk,
+ cs_ext_sales_price as sales_price,
+ cs_net_profit as profit,
+ cast(0 as decimal(7,2)) as return_amt,
+ cast(0 as decimal(7,2)) as net_loss
+ from catalog_sales
+ union all
+ select cr_catalog_page_sk as page_sk,
+ cr_returned_date_sk as date_sk,
+ cast(0 as decimal(7,2)) as sales_price,
+ cast(0 as decimal(7,2)) as profit,
+ cr_return_amount as return_amt,
+ cr_net_loss as net_loss
+ from catalog_returns
+ ) salesreturns,
+ date_dim,
+ catalog_page
+ where date_sk = d_date_sk
+ and d_date between cast('1998-08-04' as date)
+ and (cast('1998-08-04' as date) + 14 days)
+ and page_sk = cp_catalog_page_sk
+ group by cp_catalog_page_id)
+ ,
+ wsr as
+ (select web_site_id,
+ sum(sales_price) as sales,
+ sum(profit) as profit,
+ sum(return_amt) as returns,
+ sum(net_loss) as profit_loss
+ from
+ ( select ws_web_site_sk as wsr_web_site_sk,
+ ws_sold_date_sk as date_sk,
+ ws_ext_sales_price as sales_price,
+ ws_net_profit as profit,
+ cast(0 as decimal(7,2)) as return_amt,
+ cast(0 as decimal(7,2)) as net_loss
+ from web_sales
+ union all
+ select ws_web_site_sk as wsr_web_site_sk,
+ wr_returned_date_sk as date_sk,
+ cast(0 as decimal(7,2)) as sales_price,
+ cast(0 as decimal(7,2)) as profit,
+ wr_return_amt as return_amt,
+ wr_net_loss as net_loss
+ from web_returns left outer join web_sales on
+ ( wr_item_sk = ws_item_sk
+ and wr_order_number = ws_order_number)
+ ) salesreturns,
+ date_dim,
+ web_site
+ where date_sk = d_date_sk
+ and d_date between cast('1998-08-04' as date)
+ and (cast('1998-08-04' as date) + 14 days)
+ and wsr_web_site_sk = web_site_sk
+ group by web_site_id)
+ select channel
+ , id
+ , sum(sales) as sales
+ , sum(returns) as returns
+ , sum(profit) as profit
+ from
+ (select 'store channel' as channel
+ , 'store' || s_store_id as id
+ , sales
+ , returns
+ , (profit - profit_loss) as profit
+ from ssr
+ union all
+ select 'catalog channel' as channel
+ , 'catalog_page' || cp_catalog_page_id as id
+ , sales
+ , returns
+ , (profit - profit_loss) as profit
+ from csr
+ union all
+ select 'web channel' as channel
+ , 'web_site' || web_site_id as id
+ , sales
+ , returns
+ , (profit - profit_loss) as profit
+ from wsr
+ ) x
+ group by rollup (channel, id)
+ order by channel
+ ,id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query50.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query50.sql
new file mode 100644
index 0000000..fc37add
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query50.sql
@@ -0,0 +1,72 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+ s_store_name
+ ,s_company_id
+ ,s_street_number
+ ,s_street_name
+ ,s_street_type
+ ,s_suite_number
+ ,s_city
+ ,s_county
+ ,s_state
+ ,s_zip
+ ,sum(case when (sr_returned_date_sk - ss_sold_date_sk <= 30 ) then 1 else 0 end) as "30 days"
+ ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 30) and
+ (sr_returned_date_sk - ss_sold_date_sk <= 60) then 1 else 0 end ) as "31-60 days"
+ ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 60) and
+ (sr_returned_date_sk - ss_sold_date_sk <= 90) then 1 else 0 end) as "61-90 days"
+ ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 90) and
+ (sr_returned_date_sk - ss_sold_date_sk <= 120) then 1 else 0 end) as "91-120 days"
+ ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 120) then 1 else 0 end) as ">120 days"
+from
+ store_sales
+ ,store_returns
+ ,store
+ ,date_dim d1
+ ,date_dim d2
+where
+ d2.d_year = 2000
+and d2.d_moy = 9
+and ss_ticket_number = sr_ticket_number
+and ss_item_sk = sr_item_sk
+and ss_sold_date_sk = d1.d_date_sk
+and sr_returned_date_sk = d2.d_date_sk
+and ss_customer_sk = sr_customer_sk
+and ss_store_sk = s_store_sk
+group by
+ s_store_name
+ ,s_company_id
+ ,s_street_number
+ ,s_street_name
+ ,s_street_type
+ ,s_suite_number
+ ,s_city
+ ,s_county
+ ,s_state
+ ,s_zip
+order by s_store_name
+ ,s_company_id
+ ,s_street_number
+ ,s_street_name
+ ,s_street_type
+ ,s_suite_number
+ ,s_city
+ ,s_county
+ ,s_state
+ ,s_zip
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query51.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query51.sql
new file mode 100644
index 0000000..49b6d1b
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query51.sql
@@ -0,0 +1,58 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+WITH web_v1 as (
+select
+ ws_item_sk item_sk, d_date,
+ sum(sum(ws_sales_price))
+ over (partition by ws_item_sk order by d_date rows between unbounded preceding and current row) cume_sales
+from web_sales
+ ,date_dim
+where ws_sold_date_sk=d_date_sk
+ and d_month_seq between 1212 and 1212+11
+ and ws_item_sk is not NULL
+group by ws_item_sk, d_date),
+store_v1 as (
+select
+ ss_item_sk item_sk, d_date,
+ sum(sum(ss_sales_price))
+ over (partition by ss_item_sk order by d_date rows between unbounded preceding and current row) cume_sales
+from store_sales
+ ,date_dim
+where ss_sold_date_sk=d_date_sk
+ and d_month_seq between 1212 and 1212+11
+ and ss_item_sk is not NULL
+group by ss_item_sk, d_date)
+ select *
+from (select item_sk
+ ,d_date
+ ,web_sales
+ ,store_sales
+ ,max(web_sales)
+ over (partition by item_sk order by d_date rows between unbounded preceding and current row) web_cumulative
+ ,max(store_sales)
+ over (partition by item_sk order by d_date rows between unbounded preceding and current row) store_cumulative
+ from (select case when web.item_sk is not null then web.item_sk else store.item_sk end item_sk
+ ,case when web.d_date is not null then web.d_date else store.d_date end d_date
+ ,web.cume_sales web_sales
+ ,store.cume_sales store_sales
+ from web_v1 web full outer join store_v1 store on (web.item_sk = store.item_sk
+ and web.d_date = store.d_date)
+ )x )y
+where web_cumulative > store_cumulative
+order by item_sk
+ ,d_date
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query52.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query52.sql
new file mode 100644
index 0000000..5422e43
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query52.sql
@@ -0,0 +1,35 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select dt.d_year
+ ,item.i_brand_id brand_id
+ ,item.i_brand brand
+ ,sum(ss_ext_sales_price) ext_price
+ from date_dim dt
+ ,store_sales
+ ,item
+ where dt.d_date_sk = store_sales.ss_sold_date_sk
+ and store_sales.ss_item_sk = item.i_item_sk
+ and item.i_manager_id = 1
+ and dt.d_moy=12
+ and dt.d_year=1998
+ group by dt.d_year
+ ,item.i_brand
+ ,item.i_brand_id
+ order by dt.d_year
+ ,ext_price desc
+ ,brand_id
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query53.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query53.sql
new file mode 100644
index 0000000..5807a7b
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query53.sql
@@ -0,0 +1,41 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select * from
+(select i_manufact_id,
+sum(ss_sales_price) sum_sales,
+avg(sum(ss_sales_price)) over (partition by i_manufact_id) avg_quarterly_sales
+from item, store_sales, date_dim, store
+where ss_item_sk = i_item_sk and
+ss_sold_date_sk = d_date_sk and
+ss_store_sk = s_store_sk and
+d_month_seq in (1212,1212+1,1212+2,1212+3,1212+4,1212+5,1212+6,1212+7,1212+8,1212+9,1212+10,1212+11) and
+((i_category in ('Books','Children','Electronics') and
+i_class in ('personal','portable','reference','self-help') and
+i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7',
+ 'exportiunivamalg #9','scholaramalgamalg #9'))
+or(i_category in ('Women','Music','Men') and
+i_class in ('accessories','classical','fragrances','pants') and
+i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1',
+ 'importoamalg #1')))
+group by i_manufact_id, d_qoy ) tmp1
+where case when avg_quarterly_sales > 0
+ then abs (sum_sales - avg_quarterly_sales)/ avg_quarterly_sales
+ else null end > 0.1
+order by avg_quarterly_sales,
+ sum_sales,
+ i_manufact_id
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query54.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query54.sql
new file mode 100644
index 0000000..93c5af7
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query54.sql
@@ -0,0 +1,69 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with my_customers as (
+ select distinct c_customer_sk
+ , c_current_addr_sk
+ from
+ ( select cs_sold_date_sk sold_date_sk,
+ cs_bill_customer_sk customer_sk,
+ cs_item_sk item_sk
+ from catalog_sales
+ union all
+ select ws_sold_date_sk sold_date_sk,
+ ws_bill_customer_sk customer_sk,
+ ws_item_sk item_sk
+ from web_sales
+ ) cs_or_ws_sales,
+ item,
+ date_dim,
+ customer
+ where sold_date_sk = d_date_sk
+ and item_sk = i_item_sk
+ and i_category = 'Jewelry'
+ and i_class = 'consignment'
+ and c_customer_sk = cs_or_ws_sales.customer_sk
+ and d_moy = 3
+ and d_year = 1999
+ )
+ , my_revenue as (
+ select c_customer_sk,
+ sum(ss_ext_sales_price) as revenue
+ from my_customers,
+ store_sales,
+ customer_address,
+ store,
+ date_dim
+ where c_current_addr_sk = ca_address_sk
+ and ca_county = s_county
+ and ca_state = s_state
+ and ss_sold_date_sk = d_date_sk
+ and c_customer_sk = ss_customer_sk
+ and d_month_seq between (select distinct d_month_seq+1
+ from date_dim where d_year = 1999 and d_moy = 3)
+ and (select distinct d_month_seq+3
+ from date_dim where d_year = 1999 and d_moy = 3)
+ group by c_customer_sk
+ )
+ , segments as
+ (select cast((revenue/50) as int) as segment
+ from my_revenue
+ )
+ select segment, count(*) as num_customers, segment*50 as segment_base
+ from segments
+ group by segment
+ order by segment, num_customers
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query55.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query55.sql
new file mode 100644
index 0000000..5ae4d34
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query55.sql
@@ -0,0 +1,27 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select i_brand_id brand_id, i_brand brand,
+ sum(ss_ext_sales_price) ext_price
+ from date_dim, store_sales, item
+ where d_date_sk = ss_sold_date_sk
+ and ss_item_sk = i_item_sk
+ and i_manager_id=36
+ and d_moy=12
+ and d_year=2001
+ group by i_brand, i_brand_id
+ order by ext_price desc, i_brand_id
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query56.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query56.sql
new file mode 100644
index 0000000..0e76d59
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query56.sql
@@ -0,0 +1,82 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with ss as (
+ select i_item_id,sum(ss_ext_sales_price) total_sales
+ from
+ store_sales,
+ date_dim,
+ customer_address,
+ item
+ where i_item_id in (select
+ i_item_id
+from item
+where i_color in ('orchid','chiffon','lace'))
+ and ss_item_sk = i_item_sk
+ and ss_sold_date_sk = d_date_sk
+ and d_year = 2000
+ and d_moy = 1
+ and ss_addr_sk = ca_address_sk
+ and ca_gmt_offset = -8
+ group by i_item_id),
+ cs as (
+ select i_item_id,sum(cs_ext_sales_price) total_sales
+ from
+ catalog_sales,
+ date_dim,
+ customer_address,
+ item
+ where
+ i_item_id in (select
+ i_item_id
+from item
+where i_color in ('orchid','chiffon','lace'))
+ and cs_item_sk = i_item_sk
+ and cs_sold_date_sk = d_date_sk
+ and d_year = 2000
+ and d_moy = 1
+ and cs_bill_addr_sk = ca_address_sk
+ and ca_gmt_offset = -8
+ group by i_item_id),
+ ws as (
+ select i_item_id,sum(ws_ext_sales_price) total_sales
+ from
+ web_sales,
+ date_dim,
+ customer_address,
+ item
+ where
+ i_item_id in (select
+ i_item_id
+from item
+where i_color in ('orchid','chiffon','lace'))
+ and ws_item_sk = i_item_sk
+ and ws_sold_date_sk = d_date_sk
+ and d_year = 2000
+ and d_moy = 1
+ and ws_bill_addr_sk = ca_address_sk
+ and ca_gmt_offset = -8
+ group by i_item_id)
+ select i_item_id ,sum(total_sales) total_sales
+ from (select * from ss
+ union all
+ select * from cs
+ union all
+ select * from ws) tmp1
+ group by i_item_id
+ order by total_sales,
+ i_item_id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query57.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query57.sql
new file mode 100644
index 0000000..eb3b1fc
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query57.sql
@@ -0,0 +1,61 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with v1 as(
+ select i_category, i_brand,
+ cc_name,
+ d_year, d_moy,
+ sum(cs_sales_price) sum_sales,
+ avg(sum(cs_sales_price)) over
+ (partition by i_category, i_brand,
+ cc_name, d_year)
+ avg_monthly_sales,
+ rank() over
+ (partition by i_category, i_brand,
+ cc_name
+ order by d_year, d_moy) rn
+ from item, catalog_sales, date_dim, call_center
+ where cs_item_sk = i_item_sk and
+ cs_sold_date_sk = d_date_sk and
+ cc_call_center_sk= cs_call_center_sk and
+ (
+ d_year = 2000 or
+ ( d_year = 2000-1 and d_moy =12) or
+ ( d_year = 2000+1 and d_moy =1)
+ )
+ group by i_category, i_brand,
+ cc_name , d_year, d_moy),
+ v2 as(
+ select v1.cc_name
+ ,v1.d_year, v1.d_moy
+ ,v1.avg_monthly_sales
+ ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum
+ from v1, v1 v1_lag, v1 v1_lead
+ where v1.i_category = v1_lag.i_category and
+ v1.i_category = v1_lead.i_category and
+ v1.i_brand = v1_lag.i_brand and
+ v1.i_brand = v1_lead.i_brand and
+ v1. cc_name = v1_lag. cc_name and
+ v1. cc_name = v1_lead. cc_name and
+ v1.rn = v1_lag.rn + 1 and
+ v1.rn = v1_lead.rn - 1)
+ select *
+ from v2
+ where d_year = 2000 and
+ avg_monthly_sales > 0 and
+ case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1
+ order by sum_sales - avg_monthly_sales, nsum
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query58.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query58.sql
new file mode 100644
index 0000000..42366e6
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query58.sql
@@ -0,0 +1,78 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with ss_items as
+ (select i_item_id item_id
+ ,sum(ss_ext_sales_price) ss_item_rev
+ from store_sales
+ ,item
+ ,date_dim
+ where ss_item_sk = i_item_sk
+ and d_date in (select d_date
+ from date_dim
+ where d_week_seq = (select d_week_seq
+ from date_dim
+ where d_date = '1998-02-19'))
+ and ss_sold_date_sk = d_date_sk
+ group by i_item_id),
+ cs_items as
+ (select i_item_id item_id
+ ,sum(cs_ext_sales_price) cs_item_rev
+ from catalog_sales
+ ,item
+ ,date_dim
+ where cs_item_sk = i_item_sk
+ and d_date in (select d_date
+ from date_dim
+ where d_week_seq = (select d_week_seq
+ from date_dim
+ where d_date = '1998-02-19'))
+ and cs_sold_date_sk = d_date_sk
+ group by i_item_id),
+ ws_items as
+ (select i_item_id item_id
+ ,sum(ws_ext_sales_price) ws_item_rev
+ from web_sales
+ ,item
+ ,date_dim
+ where ws_item_sk = i_item_sk
+ and d_date in (select d_date
+ from date_dim
+ where d_week_seq =(select d_week_seq
+ from date_dim
+ where d_date = '1998-02-19'))
+ and ws_sold_date_sk = d_date_sk
+ group by i_item_id)
+ select ss_items.item_id
+ ,ss_item_rev
+ ,ss_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 ss_dev
+ ,cs_item_rev
+ ,cs_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 cs_dev
+ ,ws_item_rev
+ ,ws_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 ws_dev
+ ,(ss_item_rev+cs_item_rev+ws_item_rev)/3 average
+ from ss_items,cs_items,ws_items
+ where ss_items.item_id=cs_items.item_id
+ and ss_items.item_id=ws_items.item_id
+ and ss_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev
+ and ss_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev
+ and cs_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev
+ and cs_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev
+ and ws_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev
+ and ws_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev
+ order by item_id
+ ,ss_item_rev
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query59.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query59.sql
new file mode 100644
index 0000000..462ef96
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query59.sql
@@ -0,0 +1,57 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with wss as
+ (select d_week_seq,
+ ss_store_sk,
+ sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales,
+ sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales,
+ sum(case when (d_day_name='Tuesday') then ss_sales_price else null end) tue_sales,
+ sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales,
+ sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales,
+ sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales,
+ sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales
+ from store_sales,date_dim
+ where d_date_sk = ss_sold_date_sk
+ group by d_week_seq,ss_store_sk
+ )
+ select s_store_name1,s_store_id1,d_week_seq1
+ ,sun_sales1/sun_sales2,mon_sales1/mon_sales2
+ ,tue_sales1/tue_sales2,wed_sales1/wed_sales2,thu_sales1/thu_sales2
+ ,fri_sales1/fri_sales2,sat_sales1/sat_sales2
+ from
+ (select s_store_name s_store_name1,wss.d_week_seq d_week_seq1
+ ,s_store_id s_store_id1,sun_sales sun_sales1
+ ,mon_sales mon_sales1,tue_sales tue_sales1
+ ,wed_sales wed_sales1,thu_sales thu_sales1
+ ,fri_sales fri_sales1,sat_sales sat_sales1
+ from wss,store,date_dim d
+ where d.d_week_seq = wss.d_week_seq and
+ ss_store_sk = s_store_sk and
+ d_month_seq between 1185 and 1185 + 11) y,
+ (select s_store_name s_store_name2,wss.d_week_seq d_week_seq2
+ ,s_store_id s_store_id2,sun_sales sun_sales2
+ ,mon_sales mon_sales2,tue_sales tue_sales2
+ ,wed_sales wed_sales2,thu_sales thu_sales2
+ ,fri_sales fri_sales2,sat_sales sat_sales2
+ from wss,store,date_dim d
+ where d.d_week_seq = wss.d_week_seq and
+ ss_store_sk = s_store_sk and
+ d_month_seq between 1185+ 12 and 1185 + 23) x
+ where s_store_id1=s_store_id2
+ and d_week_seq1=d_week_seq2-52
+ order by s_store_name1,s_store_id1,d_week_seq1
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query6.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query6.sql
new file mode 100644
index 0000000..389c61a
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query6.sql
@@ -0,0 +1,39 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select a.ca_state state, count(*) cnt
+ from customer_address a
+ ,customer c
+ ,store_sales s
+ ,date_dim d
+ ,item i
+ where a.ca_address_sk = c.c_current_addr_sk
+ and c.c_customer_sk = s.ss_customer_sk
+ and s.ss_sold_date_sk = d.d_date_sk
+ and s.ss_item_sk = i.i_item_sk
+ and d.d_month_seq =
+ (select distinct (d_month_seq)
+ from date_dim
+ where d_year = 2000
+ and d_moy = 2 )
+ and i.i_current_price > 1.2 *
+ (select avg(j.i_current_price)
+ from item j
+ where j.i_category = i.i_category)
+ group by a.ca_state
+ having count(*) >= 10
+ order by cnt, a.ca_state
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query60.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query60.sql
new file mode 100644
index 0000000..9f84287
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query60.sql
@@ -0,0 +1,91 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with ss as (
+ select
+ i_item_id,sum(ss_ext_sales_price) total_sales
+ from
+ store_sales,
+ date_dim,
+ customer_address,
+ item
+ where
+ i_item_id in (select
+ i_item_id
+from
+ item
+where i_category in ('Children'))
+ and ss_item_sk = i_item_sk
+ and ss_sold_date_sk = d_date_sk
+ and d_year = 1999
+ and d_moy = 9
+ and ss_addr_sk = ca_address_sk
+ and ca_gmt_offset = -6
+ group by i_item_id),
+ cs as (
+ select
+ i_item_id,sum(cs_ext_sales_price) total_sales
+ from
+ catalog_sales,
+ date_dim,
+ customer_address,
+ item
+ where
+ i_item_id in (select
+ i_item_id
+from
+ item
+where i_category in ('Children'))
+ and cs_item_sk = i_item_sk
+ and cs_sold_date_sk = d_date_sk
+ and d_year = 1999
+ and d_moy = 9
+ and cs_bill_addr_sk = ca_address_sk
+ and ca_gmt_offset = -6
+ group by i_item_id),
+ ws as (
+ select
+ i_item_id,sum(ws_ext_sales_price) total_sales
+ from
+ web_sales,
+ date_dim,
+ customer_address,
+ item
+ where
+ i_item_id in (select
+ i_item_id
+from
+ item
+where i_category in ('Children'))
+ and ws_item_sk = i_item_sk
+ and ws_sold_date_sk = d_date_sk
+ and d_year = 1999
+ and d_moy = 9
+ and ws_bill_addr_sk = ca_address_sk
+ and ca_gmt_offset = -6
+ group by i_item_id)
+ select
+ i_item_id
+,sum(total_sales) total_sales
+ from (select * from ss
+ union all
+ select * from cs
+ union all
+ select * from ws) tmp1
+ group by i_item_id
+ order by i_item_id
+ ,total_sales
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query61.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query61.sql
new file mode 100644
index 0000000..4e02adb
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query61.sql
@@ -0,0 +1,57 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select promotions,total,cast(promotions as decimal(15,4))/cast(total as decimal(15,4))*100
+from
+ (select sum(ss_ext_sales_price) promotions
+ from store_sales
+ ,store
+ ,promotion
+ ,date_dim
+ ,customer
+ ,customer_address
+ ,item
+ where ss_sold_date_sk = d_date_sk
+ and ss_store_sk = s_store_sk
+ and ss_promo_sk = p_promo_sk
+ and ss_customer_sk= c_customer_sk
+ and ca_address_sk = c_current_addr_sk
+ and ss_item_sk = i_item_sk
+ and ca_gmt_offset = -7
+ and i_category = 'Books'
+ and (p_channel_dmail = 'Y' or p_channel_email = 'Y' or p_channel_tv = 'Y')
+ and s_gmt_offset = -7
+ and d_year = 1999
+ and d_moy = 11) promotional_sales,
+ (select sum(ss_ext_sales_price) total
+ from store_sales
+ ,store
+ ,date_dim
+ ,customer
+ ,customer_address
+ ,item
+ where ss_sold_date_sk = d_date_sk
+ and ss_store_sk = s_store_sk
+ and ss_customer_sk= c_customer_sk
+ and ca_address_sk = c_current_addr_sk
+ and ss_item_sk = i_item_sk
+ and ca_gmt_offset = -7
+ and i_category = 'Books'
+ and s_gmt_offset = -7
+ and d_year = 1999
+ and d_moy = 11) all_sales
+order by promotions, total
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query62.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query62.sql
new file mode 100644
index 0000000..f3f383e
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query62.sql
@@ -0,0 +1,48 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+ substr(w_warehouse_name,1,20)
+ ,sm_type
+ ,web_name
+ ,sum(case when (ws_ship_date_sk - ws_sold_date_sk <= 30 ) then 1 else 0 end) as "30 days"
+ ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 30) and
+ (ws_ship_date_sk - ws_sold_date_sk <= 60) then 1 else 0 end ) as "31-60 days"
+ ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 60) and
+ (ws_ship_date_sk - ws_sold_date_sk <= 90) then 1 else 0 end) as "61-90 days"
+ ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 90) and
+ (ws_ship_date_sk - ws_sold_date_sk <= 120) then 1 else 0 end) as "91-120 days"
+ ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 120) then 1 else 0 end) as ">120 days"
+from
+ web_sales
+ ,warehouse
+ ,ship_mode
+ ,web_site
+ ,date_dim
+where
+ d_month_seq between 1212 and 1212 + 11
+and ws_ship_date_sk = d_date_sk
+and ws_warehouse_sk = w_warehouse_sk
+and ws_ship_mode_sk = sm_ship_mode_sk
+and ws_web_site_sk = web_site_sk
+group by
+ substr(w_warehouse_name,1,20)
+ ,sm_type
+ ,web_name
+order by substr(w_warehouse_name,1,20)
+ ,sm_type
+ ,web_name
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query63.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query63.sql
new file mode 100644
index 0000000..6e86c17c
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query63.sql
@@ -0,0 +1,42 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select *
+from (select i_manager_id
+ ,sum(ss_sales_price) sum_sales
+ ,avg(sum(ss_sales_price)) over (partition by i_manager_id) avg_monthly_sales
+ from item
+ ,store_sales
+ ,date_dim
+ ,store
+ where ss_item_sk = i_item_sk
+ and ss_sold_date_sk = d_date_sk
+ and ss_store_sk = s_store_sk
+ and d_month_seq in (1212,1212+1,1212+2,1212+3,1212+4,1212+5,1212+6,1212+7,1212+8,1212+9,1212+10,1212+11)
+ and (( i_category in ('Books','Children','Electronics')
+ and i_class in ('personal','portable','reference','self-help')
+ and i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7',
+ 'exportiunivamalg #9','scholaramalgamalg #9'))
+ or( i_category in ('Women','Music','Men')
+ and i_class in ('accessories','classical','fragrances','pants')
+ and i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1',
+ 'importoamalg #1')))
+group by i_manager_id, d_moy) tmp1
+where case when avg_monthly_sales > 0 then abs (sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1
+order by i_manager_id
+ ,avg_monthly_sales
+ ,sum_sales
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query64.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query64.sql
new file mode 100644
index 0000000..c181e38
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query64.sql
@@ -0,0 +1,134 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with cs_ui as
+ (select cs_item_sk
+ ,sum(cs_ext_list_price) as sale,sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit) as refund
+ from catalog_sales
+ ,catalog_returns
+ where cs_item_sk = cr_item_sk
+ and cs_order_number = cr_order_number
+ group by cs_item_sk
+ having sum(cs_ext_list_price)>2*sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit)),
+cross_sales as
+ (select i_product_name product_name
+ ,i_item_sk item_sk
+ ,s_store_name store_name
+ ,s_zip store_zip
+ ,ad1.ca_street_number b_street_number
+ ,ad1.ca_street_name b_street_name
+ ,ad1.ca_city b_city
+ ,ad1.ca_zip b_zip
+ ,ad2.ca_street_number c_street_number
+ ,ad2.ca_street_name c_street_name
+ ,ad2.ca_city c_city
+ ,ad2.ca_zip c_zip
+ ,d1.d_year as syear
+ ,d2.d_year as fsyear
+ ,d3.d_year s2year
+ ,count(*) cnt
+ ,sum(ss_wholesale_cost) s1
+ ,sum(ss_list_price) s2
+ ,sum(ss_coupon_amt) s3
+ FROM store_sales
+ ,store_returns
+ ,cs_ui
+ ,date_dim d1
+ ,date_dim d2
+ ,date_dim d3
+ ,store
+ ,customer
+ ,customer_demographics cd1
+ ,customer_demographics cd2
+ ,promotion
+ ,household_demographics hd1
+ ,household_demographics hd2
+ ,customer_address ad1
+ ,customer_address ad2
+ ,income_band ib1
+ ,income_band ib2
+ ,item
+ WHERE ss_store_sk = s_store_sk AND
+ ss_sold_date_sk = d1.d_date_sk AND
+ ss_customer_sk = c_customer_sk AND
+ ss_cdemo_sk= cd1.cd_demo_sk AND
+ ss_hdemo_sk = hd1.hd_demo_sk AND
+ ss_addr_sk = ad1.ca_address_sk and
+ ss_item_sk = i_item_sk and
+ ss_item_sk = sr_item_sk and
+ ss_ticket_number = sr_ticket_number and
+ ss_item_sk = cs_ui.cs_item_sk and
+ c_current_cdemo_sk = cd2.cd_demo_sk AND
+ c_current_hdemo_sk = hd2.hd_demo_sk AND
+ c_current_addr_sk = ad2.ca_address_sk and
+ c_first_sales_date_sk = d2.d_date_sk and
+ c_first_shipto_date_sk = d3.d_date_sk and
+ ss_promo_sk = p_promo_sk and
+ hd1.hd_income_band_sk = ib1.ib_income_band_sk and
+ hd2.hd_income_band_sk = ib2.ib_income_band_sk and
+ cd1.cd_marital_status <> cd2.cd_marital_status and
+ i_color in ('maroon','burnished','dim','steel','navajo','chocolate') and
+ i_current_price between 35 and 35 + 10 and
+ i_current_price between 35 + 1 and 35 + 15
+group by i_product_name
+ ,i_item_sk
+ ,s_store_name
+ ,s_zip
+ ,ad1.ca_street_number
+ ,ad1.ca_street_name
+ ,ad1.ca_city
+ ,ad1.ca_zip
+ ,ad2.ca_street_number
+ ,ad2.ca_street_name
+ ,ad2.ca_city
+ ,ad2.ca_zip
+ ,d1.d_year
+ ,d2.d_year
+ ,d3.d_year
+)
+select cs1.product_name
+ ,cs1.store_name
+ ,cs1.store_zip
+ ,cs1.b_street_number
+ ,cs1.b_street_name
+ ,cs1.b_city
+ ,cs1.b_zip
+ ,cs1.c_street_number
+ ,cs1.c_street_name
+ ,cs1.c_city
+ ,cs1.c_zip
+ ,cs1.syear
+ ,cs1.cnt
+ ,cs1.s1 as s11
+ ,cs1.s2 as s21
+ ,cs1.s3 as s31
+ ,cs2.s1 as s12
+ ,cs2.s2 as s22
+ ,cs2.s3 as s32
+ ,cs2.syear
+ ,cs2.cnt
+from cross_sales cs1,cross_sales cs2
+where cs1.item_sk=cs2.item_sk and
+ cs1.syear = 2000 and
+ cs2.syear = 2000 + 1 and
+ cs2.cnt <= cs1.cnt and
+ cs1.store_name = cs2.store_name and
+ cs1.store_zip = cs2.store_zip
+order by cs1.product_name
+ ,cs1.store_name
+ ,cs2.cnt
+ ,cs1.s1
+ ,cs2.s1
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query65.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query65.sql
new file mode 100644
index 0000000..a106624
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query65.sql
@@ -0,0 +1,42 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+ s_store_name,
+ i_item_desc,
+ sc.revenue,
+ i_current_price,
+ i_wholesale_cost,
+ i_brand
+ from store, item,
+ (select ss_store_sk, avg(revenue) as ave
+ from
+ (select ss_store_sk, ss_item_sk,
+ sum(ss_sales_price) as revenue
+ from store_sales, date_dim
+ where ss_sold_date_sk = d_date_sk and d_month_seq between 1212 and 1212+11
+ group by ss_store_sk, ss_item_sk) sa
+ group by ss_store_sk) sb,
+ (select ss_store_sk, ss_item_sk, sum(ss_sales_price) as revenue
+ from store_sales, date_dim
+ where ss_sold_date_sk = d_date_sk and d_month_seq between 1212 and 1212+11
+ group by ss_store_sk, ss_item_sk) sc
+ where sb.ss_store_sk = sc.ss_store_sk and
+ sc.revenue <= 0.1 * sb.ave and
+ s_store_sk = sc.ss_store_sk and
+ i_item_sk = sc.ss_item_sk
+ order by s_store_name, i_item_desc
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query66.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query66.sql
new file mode 100644
index 0000000..f99b53b
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query66.sql
@@ -0,0 +1,233 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+ w_warehouse_name
+ ,w_warehouse_sq_ft
+ ,w_city
+ ,w_county
+ ,w_state
+ ,w_country
+ ,ship_carriers
+ ,year
+ ,sum(jan_sales) as jan_sales
+ ,sum(feb_sales) as feb_sales
+ ,sum(mar_sales) as mar_sales
+ ,sum(apr_sales) as apr_sales
+ ,sum(may_sales) as may_sales
+ ,sum(jun_sales) as jun_sales
+ ,sum(jul_sales) as jul_sales
+ ,sum(aug_sales) as aug_sales
+ ,sum(sep_sales) as sep_sales
+ ,sum(oct_sales) as oct_sales
+ ,sum(nov_sales) as nov_sales
+ ,sum(dec_sales) as dec_sales
+ ,sum(jan_sales/w_warehouse_sq_ft) as jan_sales_per_sq_foot
+ ,sum(feb_sales/w_warehouse_sq_ft) as feb_sales_per_sq_foot
+ ,sum(mar_sales/w_warehouse_sq_ft) as mar_sales_per_sq_foot
+ ,sum(apr_sales/w_warehouse_sq_ft) as apr_sales_per_sq_foot
+ ,sum(may_sales/w_warehouse_sq_ft) as may_sales_per_sq_foot
+ ,sum(jun_sales/w_warehouse_sq_ft) as jun_sales_per_sq_foot
+ ,sum(jul_sales/w_warehouse_sq_ft) as jul_sales_per_sq_foot
+ ,sum(aug_sales/w_warehouse_sq_ft) as aug_sales_per_sq_foot
+ ,sum(sep_sales/w_warehouse_sq_ft) as sep_sales_per_sq_foot
+ ,sum(oct_sales/w_warehouse_sq_ft) as oct_sales_per_sq_foot
+ ,sum(nov_sales/w_warehouse_sq_ft) as nov_sales_per_sq_foot
+ ,sum(dec_sales/w_warehouse_sq_ft) as dec_sales_per_sq_foot
+ ,sum(jan_net) as jan_net
+ ,sum(feb_net) as feb_net
+ ,sum(mar_net) as mar_net
+ ,sum(apr_net) as apr_net
+ ,sum(may_net) as may_net
+ ,sum(jun_net) as jun_net
+ ,sum(jul_net) as jul_net
+ ,sum(aug_net) as aug_net
+ ,sum(sep_net) as sep_net
+ ,sum(oct_net) as oct_net
+ ,sum(nov_net) as nov_net
+ ,sum(dec_net) as dec_net
+ from (
+ select
+ w_warehouse_name
+ ,w_warehouse_sq_ft
+ ,w_city
+ ,w_county
+ ,w_state
+ ,w_country
+ ,'DIAMOND' || ',' || 'AIRBORNE' as ship_carriers
+ ,d_year as year
+ ,sum(case when d_moy = 1
+ then ws_sales_price* ws_quantity else 0 end) as jan_sales
+ ,sum(case when d_moy = 2
+ then ws_sales_price* ws_quantity else 0 end) as feb_sales
+ ,sum(case when d_moy = 3
+ then ws_sales_price* ws_quantity else 0 end) as mar_sales
+ ,sum(case when d_moy = 4
+ then ws_sales_price* ws_quantity else 0 end) as apr_sales
+ ,sum(case when d_moy = 5
+ then ws_sales_price* ws_quantity else 0 end) as may_sales
+ ,sum(case when d_moy = 6
+ then ws_sales_price* ws_quantity else 0 end) as jun_sales
+ ,sum(case when d_moy = 7
+ then ws_sales_price* ws_quantity else 0 end) as jul_sales
+ ,sum(case when d_moy = 8
+ then ws_sales_price* ws_quantity else 0 end) as aug_sales
+ ,sum(case when d_moy = 9
+ then ws_sales_price* ws_quantity else 0 end) as sep_sales
+ ,sum(case when d_moy = 10
+ then ws_sales_price* ws_quantity else 0 end) as oct_sales
+ ,sum(case when d_moy = 11
+ then ws_sales_price* ws_quantity else 0 end) as nov_sales
+ ,sum(case when d_moy = 12
+ then ws_sales_price* ws_quantity else 0 end) as dec_sales
+ ,sum(case when d_moy = 1
+ then ws_net_paid_inc_tax * ws_quantity else 0 end) as jan_net
+ ,sum(case when d_moy = 2
+ then ws_net_paid_inc_tax * ws_quantity else 0 end) as feb_net
+ ,sum(case when d_moy = 3
+ then ws_net_paid_inc_tax * ws_quantity else 0 end) as mar_net
+ ,sum(case when d_moy = 4
+ then ws_net_paid_inc_tax * ws_quantity else 0 end) as apr_net
+ ,sum(case when d_moy = 5
+ then ws_net_paid_inc_tax * ws_quantity else 0 end) as may_net
+ ,sum(case when d_moy = 6
+ then ws_net_paid_inc_tax * ws_quantity else 0 end) as jun_net
+ ,sum(case when d_moy = 7
+ then ws_net_paid_inc_tax * ws_quantity else 0 end) as jul_net
+ ,sum(case when d_moy = 8
+ then ws_net_paid_inc_tax * ws_quantity else 0 end) as aug_net
+ ,sum(case when d_moy = 9
+ then ws_net_paid_inc_tax * ws_quantity else 0 end) as sep_net
+ ,sum(case when d_moy = 10
+ then ws_net_paid_inc_tax * ws_quantity else 0 end) as oct_net
+ ,sum(case when d_moy = 11
+ then ws_net_paid_inc_tax * ws_quantity else 0 end) as nov_net
+ ,sum(case when d_moy = 12
+ then ws_net_paid_inc_tax * ws_quantity else 0 end) as dec_net
+ from
+ web_sales
+ ,warehouse
+ ,date_dim
+ ,time_dim
+ ,ship_mode
+ where
+ ws_warehouse_sk = w_warehouse_sk
+ and ws_sold_date_sk = d_date_sk
+ and ws_sold_time_sk = t_time_sk
+ and ws_ship_mode_sk = sm_ship_mode_sk
+ and d_year = 2002
+ and t_time between 49530 and 49530+28800
+ and sm_carrier in ('DIAMOND','AIRBORNE')
+ group by
+ w_warehouse_name
+ ,w_warehouse_sq_ft
+ ,w_city
+ ,w_county
+ ,w_state
+ ,w_country
+ ,d_year
+ union all
+ select
+ w_warehouse_name
+ ,w_warehouse_sq_ft
+ ,w_city
+ ,w_county
+ ,w_state
+ ,w_country
+ ,'DIAMOND' || ',' || 'AIRBORNE' as ship_carriers
+ ,d_year as year
+ ,sum(case when d_moy = 1
+ then cs_ext_sales_price* cs_quantity else 0 end) as jan_sales
+ ,sum(case when d_moy = 2
+ then cs_ext_sales_price* cs_quantity else 0 end) as feb_sales
+ ,sum(case when d_moy = 3
+ then cs_ext_sales_price* cs_quantity else 0 end) as mar_sales
+ ,sum(case when d_moy = 4
+ then cs_ext_sales_price* cs_quantity else 0 end) as apr_sales
+ ,sum(case when d_moy = 5
+ then cs_ext_sales_price* cs_quantity else 0 end) as may_sales
+ ,sum(case when d_moy = 6
+ then cs_ext_sales_price* cs_quantity else 0 end) as jun_sales
+ ,sum(case when d_moy = 7
+ then cs_ext_sales_price* cs_quantity else 0 end) as jul_sales
+ ,sum(case when d_moy = 8
+ then cs_ext_sales_price* cs_quantity else 0 end) as aug_sales
+ ,sum(case when d_moy = 9
+ then cs_ext_sales_price* cs_quantity else 0 end) as sep_sales
+ ,sum(case when d_moy = 10
+ then cs_ext_sales_price* cs_quantity else 0 end) as oct_sales
+ ,sum(case when d_moy = 11
+ then cs_ext_sales_price* cs_quantity else 0 end) as nov_sales
+ ,sum(case when d_moy = 12
+ then cs_ext_sales_price* cs_quantity else 0 end) as dec_sales
+ ,sum(case when d_moy = 1
+ then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jan_net
+ ,sum(case when d_moy = 2
+ then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as feb_net
+ ,sum(case when d_moy = 3
+ then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as mar_net
+ ,sum(case when d_moy = 4
+ then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as apr_net
+ ,sum(case when d_moy = 5
+ then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as may_net
+ ,sum(case when d_moy = 6
+ then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jun_net
+ ,sum(case when d_moy = 7
+ then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jul_net
+ ,sum(case when d_moy = 8
+ then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as aug_net
+ ,sum(case when d_moy = 9
+ then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as sep_net
+ ,sum(case when d_moy = 10
+ then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as oct_net
+ ,sum(case when d_moy = 11
+ then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as nov_net
+ ,sum(case when d_moy = 12
+ then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as dec_net
+ from
+ catalog_sales
+ ,warehouse
+ ,date_dim
+ ,time_dim
+ ,ship_mode
+ where
+ cs_warehouse_sk = w_warehouse_sk
+ and cs_sold_date_sk = d_date_sk
+ and cs_sold_time_sk = t_time_sk
+ and cs_ship_mode_sk = sm_ship_mode_sk
+ and d_year = 2002
+ and t_time between 49530 AND 49530+28800
+ and sm_carrier in ('DIAMOND','AIRBORNE')
+ group by
+ w_warehouse_name
+ ,w_warehouse_sq_ft
+ ,w_city
+ ,w_county
+ ,w_state
+ ,w_country
+ ,d_year
+ ) x
+ group by
+ w_warehouse_name
+ ,w_warehouse_sq_ft
+ ,w_city
+ ,w_county
+ ,w_state
+ ,w_country
+ ,ship_carriers
+ ,year
+ order by w_warehouse_name
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query67.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query67.sql
new file mode 100644
index 0000000..1d90fa2
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query67.sql
@@ -0,0 +1,57 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select *
+from (select i_category
+ ,i_class
+ ,i_brand
+ ,i_product_name
+ ,d_year
+ ,d_qoy
+ ,d_moy
+ ,s_store_id
+ ,sumsales
+ ,rank() over (partition by i_category order by sumsales desc) rk
+ from (select i_category
+ ,i_class
+ ,i_brand
+ ,i_product_name
+ ,d_year
+ ,d_qoy
+ ,d_moy
+ ,s_store_id
+ ,sum(coalesce(ss_sales_price*ss_quantity,0)) sumsales
+ from store_sales
+ ,date_dim
+ ,store
+ ,item
+ where ss_sold_date_sk=d_date_sk
+ and ss_item_sk=i_item_sk
+ and ss_store_sk = s_store_sk
+ and d_month_seq between 1212 and 1212+11
+ group by rollup(i_category, i_class, i_brand, i_product_name, d_year, d_qoy, d_moy,s_store_id))dw1) dw2
+where rk <= 100
+order by i_category
+ ,i_class
+ ,i_brand
+ ,i_product_name
+ ,d_year
+ ,d_qoy
+ ,d_moy
+ ,s_store_id
+ ,sumsales
+ ,rk
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query68.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query68.sql
new file mode 100644
index 0000000..8ba3933
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query68.sql
@@ -0,0 +1,55 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select c_last_name
+ ,c_first_name
+ ,ca_city
+ ,bought_city
+ ,ss_ticket_number
+ ,extended_price
+ ,extended_tax
+ ,list_price
+ from (select ss_ticket_number
+ ,ss_customer_sk
+ ,ca_city bought_city
+ ,sum(ss_ext_sales_price) extended_price
+ ,sum(ss_ext_list_price) list_price
+ ,sum(ss_ext_tax) extended_tax
+ from store_sales
+ ,date_dim
+ ,store
+ ,household_demographics
+ ,customer_address
+ where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+ and store_sales.ss_store_sk = store.s_store_sk
+ and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+ and store_sales.ss_addr_sk = customer_address.ca_address_sk
+ and date_dim.d_dom between 1 and 2
+ and (household_demographics.hd_dep_count = 5 or
+ household_demographics.hd_vehicle_count= 3)
+ and date_dim.d_year in (1999,1999+1,1999+2)
+ and store.s_city in ('Midway','Fairview')
+ group by ss_ticket_number
+ ,ss_customer_sk
+ ,ss_addr_sk,ca_city) dn
+ ,customer
+ ,customer_address current_addr
+ where ss_customer_sk = c_customer_sk
+ and customer.c_current_addr_sk = current_addr.ca_address_sk
+ and current_addr.ca_city <> bought_city
+ order by c_last_name
+ ,ss_ticket_number
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query69.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query69.sql
new file mode 100644
index 0000000..103a056
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query69.sql
@@ -0,0 +1,60 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+ cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ count(*) cnt1,
+ cd_purchase_estimate,
+ count(*) cnt2,
+ cd_credit_rating,
+ count(*) cnt3
+ from
+ customer c,customer_address ca,customer_demographics
+ where
+ c.c_current_addr_sk = ca.ca_address_sk and
+ ca_state in ('CO','IL','MN') and
+ cd_demo_sk = c.c_current_cdemo_sk and
+ exists (select *
+ from store_sales,date_dim
+ where c.c_customer_sk = ss_customer_sk and
+ ss_sold_date_sk = d_date_sk and
+ d_year = 1999 and
+ d_moy between 1 and 1+2) and
+ (not exists (select *
+ from web_sales,date_dim
+ where c.c_customer_sk = ws_bill_customer_sk and
+ ws_sold_date_sk = d_date_sk and
+ d_year = 1999 and
+ d_moy between 1 and 1+2) and
+ not exists (select *
+ from catalog_sales,date_dim
+ where c.c_customer_sk = cs_ship_customer_sk and
+ cs_sold_date_sk = d_date_sk and
+ d_year = 1999 and
+ d_moy between 1 and 1+2))
+ group by cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+ order by cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query7.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query7.sql
new file mode 100644
index 0000000..d190e7d
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query7.sql
@@ -0,0 +1,34 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select i_item_id,
+ avg(ss_quantity) agg1,
+ avg(ss_list_price) agg2,
+ avg(ss_coupon_amt) agg3,
+ avg(ss_sales_price) agg4
+ from store_sales, customer_demographics, date_dim, item, promotion
+ where ss_sold_date_sk = d_date_sk and
+ ss_item_sk = i_item_sk and
+ ss_cdemo_sk = cd_demo_sk and
+ ss_promo_sk = p_promo_sk and
+ cd_gender = 'F' and
+ cd_marital_status = 'W' and
+ cd_education_status = 'Primary' and
+ (p_channel_email = 'N' or p_channel_event = 'N') and
+ d_year = 1998
+ group by i_item_id
+ order by i_item_id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query70.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query70.sql
new file mode 100644
index 0000000..7ea3448
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query70.sql
@@ -0,0 +1,51 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+ sum(ss_net_profit) as total_sum
+ ,s_state
+ ,s_county
+ ,grouping(s_state)+grouping(s_county) as lochierarchy
+ ,rank() over (
+ partition by grouping(s_state)+grouping(s_county),
+ case when grouping(s_county) = 0 then s_state end
+ order by sum(ss_net_profit) desc) as rank_within_parent
+ from
+ store_sales
+ ,date_dim d1
+ ,store
+ where
+ d1.d_month_seq between 1212 and 1212+11
+ and d1.d_date_sk = ss_sold_date_sk
+ and s_store_sk = ss_store_sk
+ and s_state in
+ ( select s_state
+ from (select s_state as s_state,
+ rank() over ( partition by s_state order by sum(ss_net_profit) desc) as ranking
+ from store_sales, store, date_dim
+ where d_month_seq between 1212 and 1212+11
+ and d_date_sk = ss_sold_date_sk
+ and s_store_sk = ss_store_sk
+ group by s_state
+ ) tmp1
+ where ranking <= 5
+ )
+ group by rollup(s_state,s_county)
+ order by
+ lochierarchy desc
+ ,case when lochierarchy = 0 then s_state end
+ ,rank_within_parent
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query71.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query71.sql
new file mode 100644
index 0000000..09eb27a
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query71.sql
@@ -0,0 +1,52 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select i_brand_id brand_id, i_brand brand,t_hour,t_minute,
+ sum(ext_price) ext_price
+ from item, (select ws_ext_sales_price as ext_price,
+ ws_sold_date_sk as sold_date_sk,
+ ws_item_sk as sold_item_sk,
+ ws_sold_time_sk as time_sk
+ from web_sales,date_dim
+ where d_date_sk = ws_sold_date_sk
+ and d_moy=12
+ and d_year=2000
+ union all
+ select cs_ext_sales_price as ext_price,
+ cs_sold_date_sk as sold_date_sk,
+ cs_item_sk as sold_item_sk,
+ cs_sold_time_sk as time_sk
+ from catalog_sales,date_dim
+ where d_date_sk = cs_sold_date_sk
+ and d_moy=12
+ and d_year=2000
+ union all
+ select ss_ext_sales_price as ext_price,
+ ss_sold_date_sk as sold_date_sk,
+ ss_item_sk as sold_item_sk,
+ ss_sold_time_sk as time_sk
+ from store_sales,date_dim
+ where d_date_sk = ss_sold_date_sk
+ and d_moy=12
+ and d_year=2000
+ ) tmp,time_dim
+ where
+ sold_item_sk = i_item_sk
+ and i_manager_id=1
+ and time_sk = t_time_sk
+ and (t_meal_time = 'breakfast' or t_meal_time = 'dinner')
+ group by i_brand, i_brand_id,t_hour,t_minute
+ order by ext_price desc, i_brand_id
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query72.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query72.sql
new file mode 100644
index 0000000..a1173dc
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query72.sql
@@ -0,0 +1,42 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select i_item_desc
+ ,w_warehouse_name
+ ,d1.d_week_seq
+ ,sum(case when p_promo_sk is null then 1 else 0 end) no_promo
+ ,sum(case when p_promo_sk is not null then 1 else 0 end) promo
+ ,count(*) total_cnt
+from catalog_sales
+join inventory on (cs_item_sk = inv_item_sk)
+join warehouse on (w_warehouse_sk=inv_warehouse_sk)
+join item on (i_item_sk = cs_item_sk)
+join customer_demographics on (cs_bill_cdemo_sk = cd_demo_sk)
+join household_demographics on (cs_bill_hdemo_sk = hd_demo_sk)
+join date_dim d1 on (cs_sold_date_sk = d1.d_date_sk)
+join date_dim d2 on (inv_date_sk = d2.d_date_sk)
+join date_dim d3 on (cs_ship_date_sk = d3.d_date_sk)
+left outer join promotion on (cs_promo_sk=p_promo_sk)
+left outer join catalog_returns on (cr_item_sk = cs_item_sk and cr_order_number = cs_order_number)
+where d1.d_week_seq = d2.d_week_seq
+ and inv_quantity_on_hand < cs_quantity
+ and d3.d_date > d1.d_date + 5
+ and hd_buy_potential = '1001-5000'
+ and d1.d_year = 2001
+ and cd_marital_status = 'M'
+group by i_item_desc,w_warehouse_name,d1.d_week_seq
+order by total_cnt desc, i_item_desc, w_warehouse_name, d_week_seq
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query73.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query73.sql
new file mode 100644
index 0000000..dba1bfa
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query73.sql
@@ -0,0 +1,41 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select c_last_name
+ ,c_first_name
+ ,c_salutation
+ ,c_preferred_cust_flag
+ ,ss_ticket_number
+ ,cnt from
+ (select ss_ticket_number
+ ,ss_customer_sk
+ ,count(*) cnt
+ from store_sales,date_dim,store,household_demographics
+ where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+ and store_sales.ss_store_sk = store.s_store_sk
+ and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+ and date_dim.d_dom between 1 and 2
+ and (household_demographics.hd_buy_potential = '>10000' or
+ household_demographics.hd_buy_potential = 'Unknown')
+ and household_demographics.hd_vehicle_count > 0
+ and case when household_demographics.hd_vehicle_count > 0 then
+ household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count else null end > 1
+ and date_dim.d_year in (1998,1998+1,1998+2)
+ and store.s_county in ('Williamson County','Williamson County','Williamson County','Williamson County')
+ group by ss_ticket_number,ss_customer_sk) dj,customer
+ where ss_customer_sk = c_customer_sk
+ and cnt between 1 and 5
+ order by cnt desc, c_last_name asc
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query74.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query74.sql
new file mode 100644
index 0000000..384d1e7
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query74.sql
@@ -0,0 +1,74 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with year_total as (
+ select c_customer_id customer_id
+ ,c_first_name customer_first_name
+ ,c_last_name customer_last_name
+ ,d_year as year
+ ,max(ss_net_paid) year_total
+ ,'s' sale_type
+ from customer
+ ,store_sales
+ ,date_dim
+ where c_customer_sk = ss_customer_sk
+ and ss_sold_date_sk = d_date_sk
+ and d_year in (2001,2001+1)
+ group by c_customer_id
+ ,c_first_name
+ ,c_last_name
+ ,d_year
+ union all
+ select c_customer_id customer_id
+ ,c_first_name customer_first_name
+ ,c_last_name customer_last_name
+ ,d_year as year
+ ,max(ws_net_paid) year_total
+ ,'w' sale_type
+ from customer
+ ,web_sales
+ ,date_dim
+ where c_customer_sk = ws_bill_customer_sk
+ and ws_sold_date_sk = d_date_sk
+ and d_year in (2001,2001+1)
+ group by c_customer_id
+ ,c_first_name
+ ,c_last_name
+ ,d_year
+ )
+ select
+ t_s_secyear.customer_id, t_s_secyear.customer_first_name, t_s_secyear.customer_last_name
+ from year_total t_s_firstyear
+ ,year_total t_s_secyear
+ ,year_total t_w_firstyear
+ ,year_total t_w_secyear
+ where t_s_secyear.customer_id = t_s_firstyear.customer_id
+ and t_s_firstyear.customer_id = t_w_secyear.customer_id
+ and t_s_firstyear.customer_id = t_w_firstyear.customer_id
+ and t_s_firstyear.sale_type = 's'
+ and t_w_firstyear.sale_type = 'w'
+ and t_s_secyear.sale_type = 's'
+ and t_w_secyear.sale_type = 'w'
+ and t_s_firstyear.year = 2001
+ and t_s_secyear.year = 2001+1
+ and t_w_firstyear.year = 2001
+ and t_w_secyear.year = 2001+1
+ and t_s_firstyear.year_total > 0
+ and t_w_firstyear.year_total > 0
+ and case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else null end
+ > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else null end
+ order by 2,1,3
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query75.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query75.sql
new file mode 100644
index 0000000..6d9c689
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query75.sql
@@ -0,0 +1,83 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+WITH all_sales AS (
+ SELECT d_year
+ ,i_brand_id
+ ,i_class_id
+ ,i_category_id
+ ,i_manufact_id
+ ,SUM(sales_cnt) AS sales_cnt
+ ,SUM(sales_amt) AS sales_amt
+ FROM (SELECT d_year
+ ,i_brand_id
+ ,i_class_id
+ ,i_category_id
+ ,i_manufact_id
+ ,cs_quantity - COALESCE(cr_return_quantity,0) AS sales_cnt
+ ,cs_ext_sales_price - COALESCE(cr_return_amount,0.0) AS sales_amt
+ FROM catalog_sales JOIN item ON i_item_sk=cs_item_sk
+ JOIN date_dim ON d_date_sk=cs_sold_date_sk
+ LEFT JOIN catalog_returns ON (cs_order_number=cr_order_number
+ AND cs_item_sk=cr_item_sk)
+ WHERE i_category='Sports'
+ UNION
+ SELECT d_year
+ ,i_brand_id
+ ,i_class_id
+ ,i_category_id
+ ,i_manufact_id
+ ,ss_quantity - COALESCE(sr_return_quantity,0) AS sales_cnt
+ ,ss_ext_sales_price - COALESCE(sr_return_amt,0.0) AS sales_amt
+ FROM store_sales JOIN item ON i_item_sk=ss_item_sk
+ JOIN date_dim ON d_date_sk=ss_sold_date_sk
+ LEFT JOIN store_returns ON (ss_ticket_number=sr_ticket_number
+ AND ss_item_sk=sr_item_sk)
+ WHERE i_category='Sports'
+ UNION
+ SELECT d_year
+ ,i_brand_id
+ ,i_class_id
+ ,i_category_id
+ ,i_manufact_id
+ ,ws_quantity - COALESCE(wr_return_quantity,0) AS sales_cnt
+ ,ws_ext_sales_price - COALESCE(wr_return_amt,0.0) AS sales_amt
+ FROM web_sales JOIN item ON i_item_sk=ws_item_sk
+ JOIN date_dim ON d_date_sk=ws_sold_date_sk
+ LEFT JOIN web_returns ON (ws_order_number=wr_order_number
+ AND ws_item_sk=wr_item_sk)
+ WHERE i_category='Sports') sales_detail
+ GROUP BY d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id)
+ SELECT prev_yr.d_year AS prev_year
+ ,curr_yr.d_year AS year
+ ,curr_yr.i_brand_id
+ ,curr_yr.i_class_id
+ ,curr_yr.i_category_id
+ ,curr_yr.i_manufact_id
+ ,prev_yr.sales_cnt AS prev_yr_cnt
+ ,curr_yr.sales_cnt AS curr_yr_cnt
+ ,curr_yr.sales_cnt-prev_yr.sales_cnt AS sales_cnt_diff
+ ,curr_yr.sales_amt-prev_yr.sales_amt AS sales_amt_diff
+ FROM all_sales curr_yr, all_sales prev_yr
+ WHERE curr_yr.i_brand_id=prev_yr.i_brand_id
+ AND curr_yr.i_class_id=prev_yr.i_class_id
+ AND curr_yr.i_category_id=prev_yr.i_category_id
+ AND curr_yr.i_manufact_id=prev_yr.i_manufact_id
+ AND curr_yr.d_year=2002
+ AND prev_yr.d_year=2002-1
+ AND CAST(curr_yr.sales_cnt AS DECIMAL(17,2))/CAST(prev_yr.sales_cnt AS DECIMAL(17,2))<0.9
+ ORDER BY sales_cnt_diff,sales_amt_diff
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query76.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query76.sql
new file mode 100644
index 0000000..8a87b00
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query76.sql
@@ -0,0 +1,37 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select channel, col_name, d_year, d_qoy, i_category, COUNT(*) sales_cnt, SUM(ext_sales_price) sales_amt FROM (
+ SELECT 'store' as channel, 'ss_addr_sk' col_name, d_year, d_qoy, i_category, ss_ext_sales_price ext_sales_price
+ FROM store_sales, item, date_dim
+ WHERE ss_addr_sk IS NULL
+ AND ss_sold_date_sk=d_date_sk
+ AND ss_item_sk=i_item_sk
+ UNION ALL
+ SELECT 'web' as channel, 'ws_web_page_sk' col_name, d_year, d_qoy, i_category, ws_ext_sales_price ext_sales_price
+ FROM web_sales, item, date_dim
+ WHERE ws_web_page_sk IS NULL
+ AND ws_sold_date_sk=d_date_sk
+ AND ws_item_sk=i_item_sk
+ UNION ALL
+ SELECT 'catalog' as channel, 'cs_warehouse_sk' col_name, d_year, d_qoy, i_category, cs_ext_sales_price ext_sales_price
+ FROM catalog_sales, item, date_dim
+ WHERE cs_warehouse_sk IS NULL
+ AND cs_sold_date_sk=d_date_sk
+ AND cs_item_sk=i_item_sk) foo
+GROUP BY channel, col_name, d_year, d_qoy, i_category
+ORDER BY channel, col_name, d_year, d_qoy, i_category
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query77.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query77.sql
new file mode 100644
index 0000000..7257175
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query77.sql
@@ -0,0 +1,121 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with ss as
+ (select s_store_sk,
+ sum(ss_ext_sales_price) as sales,
+ sum(ss_net_profit) as profit
+ from store_sales,
+ date_dim,
+ store
+ where ss_sold_date_sk = d_date_sk
+ and d_date between cast('1998-08-04' as date)
+ and (cast('1998-08-04' as date) + 30 days)
+ and ss_store_sk = s_store_sk
+ group by s_store_sk)
+ ,
+ sr as
+ (select s_store_sk,
+ sum(sr_return_amt) as returns,
+ sum(sr_net_loss) as profit_loss
+ from store_returns,
+ date_dim,
+ store
+ where sr_returned_date_sk = d_date_sk
+ and d_date between cast('1998-08-04' as date)
+ and (cast('1998-08-04' as date) + 30 days)
+ and sr_store_sk = s_store_sk
+ group by s_store_sk),
+ cs as
+ (select cs_call_center_sk,
+ sum(cs_ext_sales_price) as sales,
+ sum(cs_net_profit) as profit
+ from catalog_sales,
+ date_dim
+ where cs_sold_date_sk = d_date_sk
+ and d_date between cast('1998-08-04' as date)
+ and (cast('1998-08-04' as date) + 30 days)
+ group by cs_call_center_sk
+ ),
+ cr as
+ (select cr_call_center_sk,
+ sum(cr_return_amount) as returns,
+ sum(cr_net_loss) as profit_loss
+ from catalog_returns,
+ date_dim
+ where cr_returned_date_sk = d_date_sk
+ and d_date between cast('1998-08-04' as date)
+ and (cast('1998-08-04' as date) + 30 days)
+ group by cr_call_center_sk
+ ),
+ ws as
+ ( select wp_web_page_sk,
+ sum(ws_ext_sales_price) as sales,
+ sum(ws_net_profit) as profit
+ from web_sales,
+ date_dim,
+ web_page
+ where ws_sold_date_sk = d_date_sk
+ and d_date between cast('1998-08-04' as date)
+ and (cast('1998-08-04' as date) + 30 days)
+ and ws_web_page_sk = wp_web_page_sk
+ group by wp_web_page_sk),
+ wr as
+ (select wp_web_page_sk,
+ sum(wr_return_amt) as returns,
+ sum(wr_net_loss) as profit_loss
+ from web_returns,
+ date_dim,
+ web_page
+ where wr_returned_date_sk = d_date_sk
+ and d_date between cast('1998-08-04' as date)
+ and (cast('1998-08-04' as date) + 30 days)
+ and wr_web_page_sk = wp_web_page_sk
+ group by wp_web_page_sk)
+ select channel
+ , id
+ , sum(sales) as sales
+ , sum(returns) as returns
+ , sum(profit) as profit
+ from
+ (select 'store channel' as channel
+ , ss.s_store_sk as id
+ , sales
+ , coalesce(returns, 0) as returns
+ , (profit - coalesce(profit_loss,0)) as profit
+ from ss left join sr
+ on ss.s_store_sk = sr.s_store_sk
+ union all
+ select 'catalog channel' as channel
+ , cs_call_center_sk as id
+ , sales
+ , returns
+ , (profit - profit_loss) as profit
+ from cs
+ , cr
+ union all
+ select 'web channel' as channel
+ , ws.wp_web_page_sk as id
+ , sales
+ , coalesce(returns, 0) returns
+ , (profit - coalesce(profit_loss,0)) as profit
+ from ws left join wr
+ on ws.wp_web_page_sk = wr.wp_web_page_sk
+ ) x
+ group by rollup (channel, id)
+ order by channel
+ ,id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query78.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query78.sql
new file mode 100644
index 0000000..e7bec2e
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query78.sql
@@ -0,0 +1,71 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with ws as
+ (select d_year AS ws_sold_year, ws_item_sk,
+ ws_bill_customer_sk ws_customer_sk,
+ sum(ws_quantity) ws_qty,
+ sum(ws_wholesale_cost) ws_wc,
+ sum(ws_sales_price) ws_sp
+ from web_sales
+ left join web_returns on wr_order_number=ws_order_number and ws_item_sk=wr_item_sk
+ join date_dim on ws_sold_date_sk = d_date_sk
+ where wr_order_number is null
+ group by d_year, ws_item_sk, ws_bill_customer_sk
+ ),
+cs as
+ (select d_year AS cs_sold_year, cs_item_sk,
+ cs_bill_customer_sk cs_customer_sk,
+ sum(cs_quantity) cs_qty,
+ sum(cs_wholesale_cost) cs_wc,
+ sum(cs_sales_price) cs_sp
+ from catalog_sales
+ left join catalog_returns on cr_order_number=cs_order_number and cs_item_sk=cr_item_sk
+ join date_dim on cs_sold_date_sk = d_date_sk
+ where cr_order_number is null
+ group by d_year, cs_item_sk, cs_bill_customer_sk
+ ),
+ss as
+ (select d_year AS ss_sold_year, ss_item_sk,
+ ss_customer_sk,
+ sum(ss_quantity) ss_qty,
+ sum(ss_wholesale_cost) ss_wc,
+ sum(ss_sales_price) ss_sp
+ from store_sales
+ left join store_returns on sr_ticket_number=ss_ticket_number and ss_item_sk=sr_item_sk
+ join date_dim on ss_sold_date_sk = d_date_sk
+ where sr_ticket_number is null
+ group by d_year, ss_item_sk, ss_customer_sk
+ )
+ select
+ss_sold_year, ss_item_sk, ss_customer_sk,
+round(ss_qty/(coalesce(ws_qty,0)+coalesce(cs_qty,0)),2) ratio,
+ss_qty store_qty, ss_wc store_wholesale_cost, ss_sp store_sales_price,
+coalesce(ws_qty,0)+coalesce(cs_qty,0) other_chan_qty,
+coalesce(ws_wc,0)+coalesce(cs_wc,0) other_chan_wholesale_cost,
+coalesce(ws_sp,0)+coalesce(cs_sp,0) other_chan_sales_price
+from ss
+left join ws on (ws_sold_year=ss_sold_year and ws_item_sk=ss_item_sk and ws_customer_sk=ss_customer_sk)
+left join cs on (cs_sold_year=ss_sold_year and cs_item_sk=ss_item_sk and cs_customer_sk=ss_customer_sk)
+where (coalesce(ws_qty,0)>0 or coalesce(cs_qty, 0)>0) and ss_sold_year=2000
+order by
+ ss_sold_year, ss_item_sk, ss_customer_sk,
+ ss_qty desc, ss_wc desc, ss_sp desc,
+ other_chan_qty,
+ other_chan_wholesale_cost,
+ other_chan_sales_price,
+ ratio
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query79.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query79.sql
new file mode 100644
index 0000000..0728226
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query79.sql
@@ -0,0 +1,36 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+ c_last_name,c_first_name,substr(s_city,1,30),ss_ticket_number,amt,profit
+ from
+ (select ss_ticket_number
+ ,ss_customer_sk
+ ,store.s_city
+ ,sum(ss_coupon_amt) amt
+ ,sum(ss_net_profit) profit
+ from store_sales,date_dim,store,household_demographics
+ where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+ and store_sales.ss_store_sk = store.s_store_sk
+ and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+ and (household_demographics.hd_dep_count = 8 or household_demographics.hd_vehicle_count > 0)
+ and date_dim.d_dow = 1
+ and date_dim.d_year in (1998,1998+1,1998+2)
+ and store.s_number_employees between 200 and 295
+ group by ss_ticket_number,ss_customer_sk,ss_addr_sk,store.s_city) ms,customer
+ where ss_customer_sk = c_customer_sk
+ order by c_last_name,c_first_name,substr(s_city,1,30), profit
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query8.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query8.sql
new file mode 100644
index 0000000..250c118
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query8.sql
@@ -0,0 +1,121 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select s_store_name
+ ,sum(ss_net_profit)
+ from store_sales
+ ,date_dim
+ ,store,
+ (select ca_zip
+ from (
+ SELECT substr(ca_zip,1,5) ca_zip
+ FROM customer_address
+ WHERE substr(ca_zip,1,5) IN (
+ '89436','30868','65085','22977','83927','77557',
+ '58429','40697','80614','10502','32779',
+ '91137','61265','98294','17921','18427',
+ '21203','59362','87291','84093','21505',
+ '17184','10866','67898','25797','28055',
+ '18377','80332','74535','21757','29742',
+ '90885','29898','17819','40811','25990',
+ '47513','89531','91068','10391','18846',
+ '99223','82637','41368','83658','86199',
+ '81625','26696','89338','88425','32200',
+ '81427','19053','77471','36610','99823',
+ '43276','41249','48584','83550','82276',
+ '18842','78890','14090','38123','40936',
+ '34425','19850','43286','80072','79188',
+ '54191','11395','50497','84861','90733',
+ '21068','57666','37119','25004','57835',
+ '70067','62878','95806','19303','18840',
+ '19124','29785','16737','16022','49613',
+ '89977','68310','60069','98360','48649',
+ '39050','41793','25002','27413','39736',
+ '47208','16515','94808','57648','15009',
+ '80015','42961','63982','21744','71853',
+ '81087','67468','34175','64008','20261',
+ '11201','51799','48043','45645','61163',
+ '48375','36447','57042','21218','41100',
+ '89951','22745','35851','83326','61125',
+ '78298','80752','49858','52940','96976',
+ '63792','11376','53582','18717','90226',
+ '50530','94203','99447','27670','96577',
+ '57856','56372','16165','23427','54561',
+ '28806','44439','22926','30123','61451',
+ '92397','56979','92309','70873','13355',
+ '21801','46346','37562','56458','28286',
+ '47306','99555','69399','26234','47546',
+ '49661','88601','35943','39936','25632',
+ '24611','44166','56648','30379','59785',
+ '11110','14329','93815','52226','71381',
+ '13842','25612','63294','14664','21077',
+ '82626','18799','60915','81020','56447',
+ '76619','11433','13414','42548','92713',
+ '70467','30884','47484','16072','38936',
+ '13036','88376','45539','35901','19506',
+ '65690','73957','71850','49231','14276',
+ '20005','18384','76615','11635','38177',
+ '55607','41369','95447','58581','58149',
+ '91946','33790','76232','75692','95464',
+ '22246','51061','56692','53121','77209',
+ '15482','10688','14868','45907','73520',
+ '72666','25734','17959','24677','66446',
+ '94627','53535','15560','41967','69297',
+ '11929','59403','33283','52232','57350',
+ '43933','40921','36635','10827','71286',
+ '19736','80619','25251','95042','15526',
+ '36496','55854','49124','81980','35375',
+ '49157','63512','28944','14946','36503',
+ '54010','18767','23969','43905','66979',
+ '33113','21286','58471','59080','13395',
+ '79144','70373','67031','38360','26705',
+ '50906','52406','26066','73146','15884',
+ '31897','30045','61068','45550','92454',
+ '13376','14354','19770','22928','97790',
+ '50723','46081','30202','14410','20223',
+ '88500','67298','13261','14172','81410',
+ '93578','83583','46047','94167','82564',
+ '21156','15799','86709','37931','74703',
+ '83103','23054','70470','72008','49247',
+ '91911','69998','20961','70070','63197',
+ '54853','88191','91830','49521','19454',
+ '81450','89091','62378','25683','61869',
+ '51744','36580','85778','36871','48121',
+ '28810','83712','45486','67393','26935',
+ '42393','20132','55349','86057','21309',
+ '80218','10094','11357','48819','39734',
+ '40758','30432','21204','29467','30214',
+ '61024','55307','74621','11622','68908',
+ '33032','52868','99194','99900','84936',
+ '69036','99149','45013','32895','59004',
+ '32322','14933','32936','33562','72550',
+ '27385','58049','58200','16808','21360',
+ '32961','18586','79307','15492')
+ intersect
+ select ca_zip
+ from (SELECT substr(ca_zip,1,5) ca_zip,count(*) cnt
+ FROM customer_address, customer
+ WHERE ca_address_sk = c_current_addr_sk and
+ c_preferred_cust_flag='Y'
+ group by ca_zip
+ having count(*) > 10)A1)A2) V1
+ where ss_store_sk = s_store_sk
+ and ss_sold_date_sk = d_date_sk
+ and d_qoy = 1 and d_year = 2002
+ and (substr(s_zip,1,2) = substr(V1.ca_zip,1,2))
+ group by s_store_name
+ order by s_store_name
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query80.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query80.sql
new file mode 100644
index 0000000..9c6e177
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query80.sql
@@ -0,0 +1,109 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with ssr as
+ (select s_store_id as store_id,
+ sum(ss_ext_sales_price) as sales,
+ sum(coalesce(sr_return_amt, 0)) as returns,
+ sum(ss_net_profit - coalesce(sr_net_loss, 0)) as profit
+ from store_sales left outer join store_returns on
+ (ss_item_sk = sr_item_sk and ss_ticket_number = sr_ticket_number),
+ date_dim,
+ store,
+ item,
+ promotion
+ where ss_sold_date_sk = d_date_sk
+ and d_date between cast('1998-08-04' as date)
+ and (cast('1998-08-04' as date) + 30 days)
+ and ss_store_sk = s_store_sk
+ and ss_item_sk = i_item_sk
+ and i_current_price > 50
+ and ss_promo_sk = p_promo_sk
+ and p_channel_tv = 'N'
+ group by s_store_id)
+ ,
+ csr as
+ (select cp_catalog_page_id as catalog_page_id,
+ sum(cs_ext_sales_price) as sales,
+ sum(coalesce(cr_return_amount, 0)) as returns,
+ sum(cs_net_profit - coalesce(cr_net_loss, 0)) as profit
+ from catalog_sales left outer join catalog_returns on
+ (cs_item_sk = cr_item_sk and cs_order_number = cr_order_number),
+ date_dim,
+ catalog_page,
+ item,
+ promotion
+ where cs_sold_date_sk = d_date_sk
+ and d_date between cast('1998-08-04' as date)
+ and (cast('1998-08-04' as date) + 30 days)
+ and cs_catalog_page_sk = cp_catalog_page_sk
+ and cs_item_sk = i_item_sk
+ and i_current_price > 50
+ and cs_promo_sk = p_promo_sk
+ and p_channel_tv = 'N'
+group by cp_catalog_page_id)
+ ,
+ wsr as
+ (select web_site_id,
+ sum(ws_ext_sales_price) as sales,
+ sum(coalesce(wr_return_amt, 0)) as returns,
+ sum(ws_net_profit - coalesce(wr_net_loss, 0)) as profit
+ from web_sales left outer join web_returns on
+ (ws_item_sk = wr_item_sk and ws_order_number = wr_order_number),
+ date_dim,
+ web_site,
+ item,
+ promotion
+ where ws_sold_date_sk = d_date_sk
+ and d_date between cast('1998-08-04' as date)
+ and (cast('1998-08-04' as date) + 30 days)
+ and ws_web_site_sk = web_site_sk
+ and ws_item_sk = i_item_sk
+ and i_current_price > 50
+ and ws_promo_sk = p_promo_sk
+ and p_channel_tv = 'N'
+group by web_site_id)
+ select channel
+ , id
+ , sum(sales) as sales
+ , sum(returns) as returns
+ , sum(profit) as profit
+ from
+ (select 'store channel' as channel
+ , 'store' || store_id as id
+ , sales
+ , returns
+ , profit
+ from ssr
+ union all
+ select 'catalog channel' as channel
+ , 'catalog_page' || catalog_page_id as id
+ , sales
+ , returns
+ , profit
+ from csr
+ union all
+ select 'web channel' as channel
+ , 'web_site' || web_site_id as id
+ , sales
+ , returns
+ , profit
+ from wsr
+ ) x
+ group by rollup (channel, id)
+ order by channel
+ ,id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query81.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query81.sql
new file mode 100644
index 0000000..71c49dc
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query81.sql
@@ -0,0 +1,44 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with customer_total_return as
+ (select cr_returning_customer_sk as ctr_customer_sk
+ ,ca_state as ctr_state,
+ sum(cr_return_amt_inc_tax) as ctr_total_return
+ from catalog_returns
+ ,date_dim
+ ,customer_address
+ where cr_returned_date_sk = d_date_sk
+ and d_year =1998
+ and cr_returning_addr_sk = ca_address_sk
+ group by cr_returning_customer_sk
+ ,ca_state )
+ select c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name
+ ,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset
+ ,ca_location_type,ctr_total_return
+ from customer_total_return ctr1
+ ,customer_address
+ ,customer
+ where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2
+ from customer_total_return ctr2
+ where ctr1.ctr_state = ctr2.ctr_state)
+ and ca_address_sk = c_current_addr_sk
+ and ca_state = 'IL'
+ and ctr1.ctr_customer_sk = c_customer_sk
+ order by c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name
+ ,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset
+ ,ca_location_type,ctr_total_return
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query82.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query82.sql
new file mode 100644
index 0000000..f08cc17
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query82.sql
@@ -0,0 +1,30 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select i_item_id
+ ,i_item_desc
+ ,i_current_price
+ from item, inventory, date_dim, store_sales
+ where i_current_price between 30 and 30+30
+ and inv_item_sk = i_item_sk
+ and d_date_sk=inv_date_sk
+ and d_date between cast('2002-05-30' as date) and (cast('2002-05-30' as date) + 60 days)
+ and i_manufact_id in (437,129,727,663)
+ and inv_quantity_on_hand between 100 and 500
+ and ss_item_sk = i_item_sk
+ group by i_item_id,i_item_desc,i_current_price
+ order by i_item_id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query83.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query83.sql
new file mode 100644
index 0000000..bc22723
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query83.sql
@@ -0,0 +1,80 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with sr_items as
+ (select i_item_id item_id,
+ sum(sr_return_quantity) sr_item_qty
+ from store_returns,
+ item,
+ date_dim
+ where sr_item_sk = i_item_sk
+ and d_date in
+ (select d_date
+ from date_dim
+ where d_week_seq in
+ (select d_week_seq
+ from date_dim
+ where d_date in ('1998-01-02','1998-10-15','1998-11-10')))
+ and sr_returned_date_sk = d_date_sk
+ group by i_item_id),
+ cr_items as
+ (select i_item_id item_id,
+ sum(cr_return_quantity) cr_item_qty
+ from catalog_returns,
+ item,
+ date_dim
+ where cr_item_sk = i_item_sk
+ and d_date in
+ (select d_date
+ from date_dim
+ where d_week_seq in
+ (select d_week_seq
+ from date_dim
+ where d_date in ('1998-01-02','1998-10-15','1998-11-10')))
+ and cr_returned_date_sk = d_date_sk
+ group by i_item_id),
+ wr_items as
+ (select i_item_id item_id,
+ sum(wr_return_quantity) wr_item_qty
+ from web_returns,
+ item,
+ date_dim
+ where wr_item_sk = i_item_sk
+ and d_date in
+ (select d_date
+ from date_dim
+ where d_week_seq in
+ (select d_week_seq
+ from date_dim
+ where d_date in ('1998-01-02','1998-10-15','1998-11-10')))
+ and wr_returned_date_sk = d_date_sk
+ group by i_item_id)
+ select sr_items.item_id
+ ,sr_item_qty
+ ,sr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 sr_dev
+ ,cr_item_qty
+ ,cr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 cr_dev
+ ,wr_item_qty
+ ,wr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 wr_dev
+ ,(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 average
+ from sr_items
+ ,cr_items
+ ,wr_items
+ where sr_items.item_id=cr_items.item_id
+ and sr_items.item_id=wr_items.item_id
+ order by sr_items.item_id
+ ,sr_item_qty
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query84.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query84.sql
new file mode 100644
index 0000000..2a519de
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query84.sql
@@ -0,0 +1,34 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select c_customer_id as customer_id
+ , coalesce(c_last_name,'') || ', ' || coalesce(c_first_name,'') as customername
+ from customer
+ ,customer_address
+ ,customer_demographics
+ ,household_demographics
+ ,income_band
+ ,store_returns
+ where ca_city = 'Hopewell'
+ and c_current_addr_sk = ca_address_sk
+ and ib_lower_bound >= 32287
+ and ib_upper_bound <= 32287 + 50000
+ and ib_income_band_sk = hd_income_band_sk
+ and cd_demo_sk = c_current_cdemo_sk
+ and hd_demo_sk = c_current_hdemo_sk
+ and sr_cdemo_sk = cd_demo_sk
+ order by c_customer_id
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query85.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query85.sql
new file mode 100644
index 0000000..dea9927
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query85.sql
@@ -0,0 +1,97 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select substr(r_reason_desc,1,20)
+ ,avg(ws_quantity)
+ ,avg(wr_refunded_cash)
+ ,avg(wr_fee)
+ from web_sales, web_returns, web_page, customer_demographics cd1,
+ customer_demographics cd2, customer_address, date_dim, reason
+ where ws_web_page_sk = wp_web_page_sk
+ and ws_item_sk = wr_item_sk
+ and ws_order_number = wr_order_number
+ and ws_sold_date_sk = d_date_sk and d_year = 1998
+ and cd1.cd_demo_sk = wr_refunded_cdemo_sk
+ and cd2.cd_demo_sk = wr_returning_cdemo_sk
+ and ca_address_sk = wr_refunded_addr_sk
+ and r_reason_sk = wr_reason_sk
+ and
+ (
+ (
+ cd1.cd_marital_status = 'M'
+ and
+ cd1.cd_marital_status = cd2.cd_marital_status
+ and
+ cd1.cd_education_status = '4 yr Degree'
+ and
+ cd1.cd_education_status = cd2.cd_education_status
+ and
+ ws_sales_price between 100.00 and 150.00
+ )
+ or
+ (
+ cd1.cd_marital_status = 'D'
+ and
+ cd1.cd_marital_status = cd2.cd_marital_status
+ and
+ cd1.cd_education_status = 'Primary'
+ and
+ cd1.cd_education_status = cd2.cd_education_status
+ and
+ ws_sales_price between 50.00 and 100.00
+ )
+ or
+ (
+ cd1.cd_marital_status = 'U'
+ and
+ cd1.cd_marital_status = cd2.cd_marital_status
+ and
+ cd1.cd_education_status = 'Advanced Degree'
+ and
+ cd1.cd_education_status = cd2.cd_education_status
+ and
+ ws_sales_price between 150.00 and 200.00
+ )
+ )
+ and
+ (
+ (
+ ca_country = 'United States'
+ and
+ ca_state in ('KY', 'GA', 'NM')
+ and ws_net_profit between 100 and 200
+ )
+ or
+ (
+ ca_country = 'United States'
+ and
+ ca_state in ('MT', 'OR', 'IN')
+ and ws_net_profit between 150 and 300
+ )
+ or
+ (
+ ca_country = 'United States'
+ and
+ ca_state in ('WI', 'MO', 'WV')
+ and ws_net_profit between 50 and 250
+ )
+ )
+group by r_reason_desc
+order by substr(r_reason_desc,1,20)
+ ,avg(ws_quantity)
+ ,avg(wr_refunded_cash)
+ ,avg(wr_fee)
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query86.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query86.sql
new file mode 100644
index 0000000..353732b
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query86.sql
@@ -0,0 +1,39 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+ sum(ws_net_paid) as total_sum
+ ,i_category
+ ,i_class
+ ,grouping(i_category)+grouping(i_class) as lochierarchy
+ ,rank() over (
+ partition by grouping(i_category)+grouping(i_class),
+ case when grouping(i_class) = 0 then i_category end
+ order by sum(ws_net_paid) desc) as rank_within_parent
+ from
+ web_sales
+ ,date_dim d1
+ ,item
+ where
+ d1.d_month_seq between 1212 and 1212+11
+ and d1.d_date_sk = ws_sold_date_sk
+ and i_item_sk = ws_item_sk
+ group by rollup(i_category,i_class)
+ order by
+ lochierarchy desc,
+ case when lochierarchy = 0 then i_category end,
+ rank_within_parent
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query87.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query87.sql
new file mode 100644
index 0000000..12b0384
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query87.sql
@@ -0,0 +1,35 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select count(*)
+from ((select distinct c_last_name, c_first_name, d_date
+ from store_sales, date_dim, customer
+ where store_sales.ss_sold_date_sk = date_dim.d_date_sk
+ and store_sales.ss_customer_sk = customer.c_customer_sk
+ and d_month_seq between 1212 and 1212+11)
+ except
+ (select distinct c_last_name, c_first_name, d_date
+ from catalog_sales, date_dim, customer
+ where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk
+ and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk
+ and d_month_seq between 1212 and 1212+11)
+ except
+ (select distinct c_last_name, c_first_name, d_date
+ from web_sales, date_dim, customer
+ where web_sales.ws_sold_date_sk = date_dim.d_date_sk
+ and web_sales.ws_bill_customer_sk = customer.c_customer_sk
+ and d_month_seq between 1212 and 1212+11)
+) cool_cust
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query88.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query88.sql
new file mode 100644
index 0000000..8ca2616
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query88.sql
@@ -0,0 +1,106 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select *
+from
+ (select count(*) h8_30_to_9
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+ and ss_hdemo_sk = household_demographics.hd_demo_sk
+ and ss_store_sk = s_store_sk
+ and time_dim.t_hour = 8
+ and time_dim.t_minute >= 30
+ and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
+ (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+ (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2))
+ and store.s_store_name = 'ese') s1,
+ (select count(*) h9_to_9_30
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+ and ss_hdemo_sk = household_demographics.hd_demo_sk
+ and ss_store_sk = s_store_sk
+ and time_dim.t_hour = 9
+ and time_dim.t_minute < 30
+ and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
+ (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+ (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2))
+ and store.s_store_name = 'ese') s2,
+ (select count(*) h9_30_to_10
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+ and ss_hdemo_sk = household_demographics.hd_demo_sk
+ and ss_store_sk = s_store_sk
+ and time_dim.t_hour = 9
+ and time_dim.t_minute >= 30
+ and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
+ (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+ (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2))
+ and store.s_store_name = 'ese') s3,
+ (select count(*) h10_to_10_30
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+ and ss_hdemo_sk = household_demographics.hd_demo_sk
+ and ss_store_sk = s_store_sk
+ and time_dim.t_hour = 10
+ and time_dim.t_minute < 30
+ and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
+ (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+ (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2))
+ and store.s_store_name = 'ese') s4,
+ (select count(*) h10_30_to_11
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+ and ss_hdemo_sk = household_demographics.hd_demo_sk
+ and ss_store_sk = s_store_sk
+ and time_dim.t_hour = 10
+ and time_dim.t_minute >= 30
+ and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
+ (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+ (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2))
+ and store.s_store_name = 'ese') s5,
+ (select count(*) h11_to_11_30
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+ and ss_hdemo_sk = household_demographics.hd_demo_sk
+ and ss_store_sk = s_store_sk
+ and time_dim.t_hour = 11
+ and time_dim.t_minute < 30
+ and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
+ (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+ (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2))
+ and store.s_store_name = 'ese') s6,
+ (select count(*) h11_30_to_12
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+ and ss_hdemo_sk = household_demographics.hd_demo_sk
+ and ss_store_sk = s_store_sk
+ and time_dim.t_hour = 11
+ and time_dim.t_minute >= 30
+ and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
+ (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+ (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2))
+ and store.s_store_name = 'ese') s7,
+ (select count(*) h12_to_12_30
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+ and ss_hdemo_sk = household_demographics.hd_demo_sk
+ and ss_store_sk = s_store_sk
+ and time_dim.t_hour = 12
+ and time_dim.t_minute < 30
+ and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or
+ (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+ (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2))
+ and store.s_store_name = 'ese') s8
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query89.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query89.sql
new file mode 100644
index 0000000..217a22a
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query89.sql
@@ -0,0 +1,41 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select *
+from(
+select i_category, i_class, i_brand,
+ s_store_name, s_company_name,
+ d_moy,
+ sum(ss_sales_price) sum_sales,
+ avg(sum(ss_sales_price)) over
+ (partition by i_category, i_brand, s_store_name, s_company_name)
+ avg_monthly_sales
+from item, store_sales, date_dim, store
+where ss_item_sk = i_item_sk and
+ ss_sold_date_sk = d_date_sk and
+ ss_store_sk = s_store_sk and
+ d_year in (2000) and
+ ((i_category in ('Home','Books','Electronics') and
+ i_class in ('wallpaper','parenting','musical')
+ )
+ or (i_category in ('Shoes','Jewelry','Men') and
+ i_class in ('womens','birdal','pants')
+ ))
+group by i_category, i_class, i_brand,
+ s_store_name, s_company_name, d_moy) tmp1
+where case when (avg_monthly_sales <> 0) then (abs(sum_sales - avg_monthly_sales) / avg_monthly_sales) else null end > 0.1
+order by sum_sales - avg_monthly_sales, s_store_name
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query9.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query9.sql
new file mode 100644
index 0000000..ab72251
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query9.sql
@@ -0,0 +1,63 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select case when (select count(*)
+ from store_sales
+ where ss_quantity between 1 and 20) > 25437
+ then (select avg(ss_ext_discount_amt)
+ from store_sales
+ where ss_quantity between 1 and 20)
+ else (select avg(ss_net_profit)
+ from store_sales
+ where ss_quantity between 1 and 20) end bucket1 ,
+ case when (select count(*)
+ from store_sales
+ where ss_quantity between 21 and 40) > 22746
+ then (select avg(ss_ext_discount_amt)
+ from store_sales
+ where ss_quantity between 21 and 40)
+ else (select avg(ss_net_profit)
+ from store_sales
+ where ss_quantity between 21 and 40) end bucket2,
+ case when (select count(*)
+ from store_sales
+ where ss_quantity between 41 and 60) > 9387
+ then (select avg(ss_ext_discount_amt)
+ from store_sales
+ where ss_quantity between 41 and 60)
+ else (select avg(ss_net_profit)
+ from store_sales
+ where ss_quantity between 41 and 60) end bucket3,
+ case when (select count(*)
+ from store_sales
+ where ss_quantity between 61 and 80) > 10098
+ then (select avg(ss_ext_discount_amt)
+ from store_sales
+ where ss_quantity between 61 and 80)
+ else (select avg(ss_net_profit)
+ from store_sales
+ where ss_quantity between 61 and 80) end bucket4,
+ case when (select count(*)
+ from store_sales
+ where ss_quantity between 81 and 100) > 18213
+ then (select avg(ss_ext_discount_amt)
+ from store_sales
+ where ss_quantity between 81 and 100)
+ else (select avg(ss_net_profit)
+ from store_sales
+ where ss_quantity between 81 and 100) end bucket5
+from reason
+where r_reason_sk = 1
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query90.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query90.sql
new file mode 100644
index 0000000..2dfa02a
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query90.sql
@@ -0,0 +1,35 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select cast(amc as decimal(15,4))/cast(pmc as decimal(15,4)) am_pm_ratio
+ from ( select count(*) amc
+ from web_sales, household_demographics , time_dim, web_page
+ where ws_sold_time_sk = time_dim.t_time_sk
+ and ws_ship_hdemo_sk = household_demographics.hd_demo_sk
+ and ws_web_page_sk = web_page.wp_web_page_sk
+ and time_dim.t_hour between 6 and 6+1
+ and household_demographics.hd_dep_count = 8
+ and web_page.wp_char_count between 5000 and 5200) at,
+ ( select count(*) pmc
+ from web_sales, household_demographics , time_dim, web_page
+ where ws_sold_time_sk = time_dim.t_time_sk
+ and ws_ship_hdemo_sk = household_demographics.hd_demo_sk
+ and ws_web_page_sk = web_page.wp_web_page_sk
+ and time_dim.t_hour between 14 and 14+1
+ and household_demographics.hd_dep_count = 8
+ and web_page.wp_char_count between 5000 and 5200) pt
+ order by am_pm_ratio
+ limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query91.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query91.sql
new file mode 100644
index 0000000..49d5fda
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query91.sql
@@ -0,0 +1,44 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+ cc_call_center_id Call_Center,
+ cc_name Call_Center_Name,
+ cc_manager Manager,
+ sum(cr_net_loss) Returns_Loss
+from
+ call_center,
+ catalog_returns,
+ date_dim,
+ customer,
+ customer_address,
+ customer_demographics,
+ household_demographics
+where
+ cr_call_center_sk = cc_call_center_sk
+and cr_returned_date_sk = d_date_sk
+and cr_returning_customer_sk= c_customer_sk
+and cd_demo_sk = c_current_cdemo_sk
+and hd_demo_sk = c_current_hdemo_sk
+and ca_address_sk = c_current_addr_sk
+and d_year = 1999
+and d_moy = 11
+and ( (cd_marital_status = 'M' and cd_education_status = 'Unknown')
+ or(cd_marital_status = 'W' and cd_education_status = 'Advanced Degree'))
+and hd_buy_potential like '0-500%'
+and ca_gmt_offset = -7
+group by cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status
+order by sum(cr_net_loss) desc
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query92.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query92.sql
new file mode 100644
index 0000000..a7ce3a3
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query92.sql
@@ -0,0 +1,43 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+ sum(ws_ext_discount_amt) as "Excess Discount Amount"
+from
+ web_sales
+ ,item
+ ,date_dim
+where
+i_manufact_id = 269
+and i_item_sk = ws_item_sk
+and d_date between '1998-03-18' and
+ (cast('1998-03-18' as date) + 90 days)
+and d_date_sk = ws_sold_date_sk
+and ws_ext_discount_amt
+ > (
+ SELECT
+ 1.3 * avg(ws_ext_discount_amt)
+ FROM
+ web_sales
+ ,date_dim
+ WHERE
+ ws_item_sk = i_item_sk
+ and d_date between '1998-03-18' and
+ (cast('1998-03-18' as date) + 90 days)
+ and d_date_sk = ws_sold_date_sk
+ )
+order by sum(ws_ext_discount_amt)
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query93.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query93.sql
new file mode 100644
index 0000000..20aa0a8
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query93.sql
@@ -0,0 +1,31 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select ss_customer_sk
+ ,sum(act_sales) sumsales
+ from (select ss_item_sk
+ ,ss_ticket_number
+ ,ss_customer_sk
+ ,case when sr_return_quantity is not null then (ss_quantity-sr_return_quantity)*ss_sales_price
+ else (ss_quantity*ss_sales_price) end act_sales
+ from store_sales left outer join store_returns on (sr_item_sk = ss_item_sk
+ and sr_ticket_number = ss_ticket_number)
+ ,reason
+ where sr_reason_sk = r_reason_sk
+ and r_reason_desc = 'Did not like the warranty') t
+ group by ss_customer_sk
+ order by sumsales, ss_customer_sk
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query94.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query94.sql
new file mode 100644
index 0000000..dab63be
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query94.sql
@@ -0,0 +1,42 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+ count(distinct ws_order_number) as "order count"
+ ,sum(ws_ext_ship_cost) as "total shipping cost"
+ ,sum(ws_net_profit) as "total net profit"
+from
+ web_sales ws1
+ ,date_dim
+ ,customer_address
+ ,web_site
+where
+ d_date between '1999-5-01' and
+ (cast('1999-5-01' as date) + 60 days)
+and ws1.ws_ship_date_sk = d_date_sk
+and ws1.ws_ship_addr_sk = ca_address_sk
+and ca_state = 'TX'
+and ws1.ws_web_site_sk = web_site_sk
+and web_company_name = 'pri'
+and exists (select *
+ from web_sales ws2
+ where ws1.ws_order_number = ws2.ws_order_number
+ and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk)
+and not exists(select *
+ from web_returns wr1
+ where ws1.ws_order_number = wr1.wr_order_number)
+order by count(distinct ws_order_number)
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query95.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query95.sql
new file mode 100644
index 0000000..b082826
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query95.sql
@@ -0,0 +1,45 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with ws_wh as
+(select ws1.ws_order_number,ws1.ws_warehouse_sk wh1,ws2.ws_warehouse_sk wh2
+ from web_sales ws1,web_sales ws2
+ where ws1.ws_order_number = ws2.ws_order_number
+ and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk)
+ select
+ count(distinct ws_order_number) as "order count"
+ ,sum(ws_ext_ship_cost) as "total shipping cost"
+ ,sum(ws_net_profit) as "total net profit"
+from
+ web_sales ws1
+ ,date_dim
+ ,customer_address
+ ,web_site
+where
+ d_date between '1999-5-01' and
+ (cast('1999-5-01' as date) + 60 days)
+and ws1.ws_ship_date_sk = d_date_sk
+and ws1.ws_ship_addr_sk = ca_address_sk
+and ca_state = 'TX'
+and ws1.ws_web_site_sk = web_site_sk
+and web_company_name = 'pri'
+and ws1.ws_order_number in (select ws_order_number
+ from ws_wh)
+and ws1.ws_order_number in (select wr_order_number
+ from web_returns,ws_wh
+ where wr_order_number = ws_wh.ws_order_number)
+order by count(distinct ws_order_number)
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query96.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query96.sql
new file mode 100644
index 0000000..97cf08b
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query96.sql
@@ -0,0 +1,29 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select count(*)
+from store_sales
+ ,household_demographics
+ ,time_dim, store
+where ss_sold_time_sk = time_dim.t_time_sk
+ and ss_hdemo_sk = household_demographics.hd_demo_sk
+ and ss_store_sk = s_store_sk
+ and time_dim.t_hour = 8
+ and time_dim.t_minute >= 30
+ and household_demographics.hd_dep_count = 5
+ and store.s_store_name = 'ese'
+order by count(*)
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query97.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query97.sql
new file mode 100644
index 0000000..c2d51a7
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query97.sql
@@ -0,0 +1,38 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+with ssci as (
+select ss_customer_sk customer_sk
+ ,ss_item_sk item_sk
+from store_sales,date_dim
+where ss_sold_date_sk = d_date_sk
+ and d_month_seq between 1212 and 1212 + 11
+group by ss_customer_sk
+ ,ss_item_sk),
+csci as(
+ select cs_bill_customer_sk customer_sk
+ ,cs_item_sk item_sk
+from catalog_sales,date_dim
+where cs_sold_date_sk = d_date_sk
+ and d_month_seq between 1212 and 1212 + 11
+group by cs_bill_customer_sk
+ ,cs_item_sk)
+ select sum(case when ssci.customer_sk is not null and csci.customer_sk is null then 1 else 0 end) store_only
+ ,sum(case when ssci.customer_sk is null and csci.customer_sk is not null then 1 else 0 end) catalog_only
+ ,sum(case when ssci.customer_sk is not null and csci.customer_sk is not null then 1 else 0 end) store_and_catalog
+from ssci full outer join csci on (ssci.customer_sk=csci.customer_sk
+ and ssci.item_sk = csci.item_sk)
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query98.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query98.sql
new file mode 100644
index 0000000..29d5757
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query98.sql
@@ -0,0 +1,46 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select i_item_id
+ ,i_item_desc
+ ,i_category
+ ,i_class
+ ,i_current_price
+ ,sum(ss_ext_sales_price) as itemrevenue
+ ,sum(ss_ext_sales_price)*100/sum(sum(ss_ext_sales_price)) over
+ (partition by i_class) as revenueratio
+from
+ store_sales
+ ,item
+ ,date_dim
+where
+ ss_item_sk = i_item_sk
+ and i_category in ('Jewelry', 'Sports', 'Books')
+ and ss_sold_date_sk = d_date_sk
+ and d_date between cast('2001-01-12' as date)
+ and (cast('2001-01-12' as date) + 30 days)
+group by
+ i_item_id
+ ,i_item_desc
+ ,i_category
+ ,i_class
+ ,i_current_price
+order by
+ i_category
+ ,i_class
+ ,i_item_id
+ ,i_item_desc
+ ,revenueratio
diff --git a/sdks/java/testing/tpcds/src/main/resources/queries/query99.sql b/sdks/java/testing/tpcds/src/main/resources/queries/query99.sql
new file mode 100644
index 0000000..de8e8ca
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/queries/query99.sql
@@ -0,0 +1,48 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements. See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership. The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+select
+ substr(w_warehouse_name,1,20)
+ ,sm_type
+ ,cc_name
+ ,sum(case when (cs_ship_date_sk - cs_sold_date_sk <= 30 ) then 1 else 0 end) as "30 days"
+ ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 30) and
+ (cs_ship_date_sk - cs_sold_date_sk <= 60) then 1 else 0 end ) as "31-60 days"
+ ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 60) and
+ (cs_ship_date_sk - cs_sold_date_sk <= 90) then 1 else 0 end) as "61-90 days"
+ ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 90) and
+ (cs_ship_date_sk - cs_sold_date_sk <= 120) then 1 else 0 end) as "91-120 days"
+ ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 120) then 1 else 0 end) as ">120 days"
+from
+ catalog_sales
+ ,warehouse
+ ,ship_mode
+ ,call_center
+ ,date_dim
+where
+ d_month_seq between 1212 and 1212 + 11
+and cs_ship_date_sk = d_date_sk
+and cs_warehouse_sk = w_warehouse_sk
+and cs_ship_mode_sk = sm_ship_mode_sk
+and cs_call_center_sk = cc_call_center_sk
+group by
+ substr(w_warehouse_name,1,20)
+ ,sm_type
+ ,cc_name
+order by substr(w_warehouse_name,1,20)
+ ,sm_type
+ ,cc_name
+limit 100
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/call_center.json b/sdks/java/testing/tpcds/src/main/resources/schemas/call_center.json
new file mode 100644
index 0000000..ec95095
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/call_center.json
@@ -0,0 +1,33 @@
+{"schema": [
+ {"name":"cc_call_center_sk","type":"identifier"},
+ {"name":"cc_call_center_id","type":"char(16)"},
+ {"name":"cc_rec_start_date","type":"date"},
+ {"name":"cc_rec_end_date","type":"date"},
+ {"name":"cc_closed_date_sk","type":"integer"},
+ {"name":"cc_open_date_sk","type":"integer"},
+ {"name":"cc_name","type":"varchar(50)"},
+ {"name":"cc_class","type":"varchar(50)"},
+ {"name":"cc_employees","type":"integer"},
+ {"name":"cc_sq_ft","type":"integer"},
+ {"name":"cc_hours","type":"char(20)"},
+ {"name":"cc_manager","type":"varchar(40)"},
+ {"name":"cc_mkt_id","type":"integer"},
+ {"name":"cc_mkt_class","type":"char(50)"},
+ {"name":"cc_mkt_desc","type":"varchar(100)"},
+ {"name":"cc_market_manager","type":"varchar(40)"},
+ {"name":"cc_division","type":"integer"},
+ {"name":"cc_division_name","type":"varchar(50)"},
+ {"name":"cc_company","type":"integer"},
+ {"name":"cc_company_name","type":"char(50)"},
+ {"name":"cc_street_number","type":"char(10)"},
+ {"name":"cc_street_name","type":"varchar(60)"},
+ {"name":"cc_street_type","type":"char(15)"},
+ {"name":"cc_suite_number","type":"char(10)"},
+ {"name":"cc_city","type":"varchar(60)"},
+ {"name":"cc_county","type":"varchar(30)"},
+ {"name":"cc_state","type":"char(2)"},
+ {"name":"cc_zip","type":"char(10)"},
+ {"name":"cc_country","type":"varchar(20)"},
+ {"name":"cc_gmt_offset","type":"decimal(5,2)"},
+ {"name":"cc_tax_percentage","type":"decimal(5,2)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/catalog_page.json b/sdks/java/testing/tpcds/src/main/resources/schemas/catalog_page.json
new file mode 100644
index 0000000..15a549466
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/catalog_page.json
@@ -0,0 +1,11 @@
+{"schema": [
+ {"name":"cp_catalog_page_sk","type":"identifier"},
+ {"name":"cp_catalog_page_id","type":"char(16)"},
+ {"name":"cp_start_date_sk","type":"integer"},
+ {"name":"cp_end_date_sk","type":"integer"},
+ {"name":"cp_department","type":"varchar(50)"},
+ {"name":"cp_catalog_number","type":"integer"},
+ {"name":"cp_catalog_page_number","type":"integer"},
+ {"name":"cp_description","type":"varchar(100)"},
+ {"name":"cp_type","type":"varchar(100)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/catalog_returns.json b/sdks/java/testing/tpcds/src/main/resources/schemas/catalog_returns.json
new file mode 100644
index 0000000..d967a9e
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/catalog_returns.json
@@ -0,0 +1,29 @@
+{"schema": [
+ {"name":"cr_returned_date_sk","type":"identifier"},
+ {"name":"cr_returned_time_sk","type":"identifier"},
+ {"name":"cr_item_sk","type":"identifier"},
+ {"name":"cr_refunded_customer_sk","type":"identifier"},
+ {"name":"cr_refunded_cdemo_sk","type":"identifier"},
+ {"name":"cr_refunded_hdemo_sk","type":"identifier"},
+ {"name":"cr_refunded_addr_sk","type":"identifier"},
+ {"name":"cr_returning_customer_sk","type":"identifier"},
+ {"name":"cr_returning_cdemo_sk","type":"identifier"},
+ {"name":"cr_returning_hdemo_sk","type":"identifier"},
+ {"name":"cr_returning_addr_sk","type":"identifier"},
+ {"name":"cr_call_center_sk","type":"identifier"},
+ {"name":"cr_catalog_page_sk","type":"identifier"},
+ {"name":"cr_ship_mode_sk","type":"identifier"},
+ {"name":"cr_warehouse_sk","type":"identifier"},
+ {"name":"cr_reason_sk","type":"identifier"},
+ {"name":"cr_order_number","type":"identifier"},
+ {"name":"cr_return_quantity","type":"integer"},
+ {"name":"cr_return_amount","type":"decimal(7,2)"},
+ {"name":"cr_return_tax","type":"decimal(7,2)"},
+ {"name":"cr_return_amt_inc_tax","type":"decimal(7,2)"},
+ {"name":"cr_fee","type":"decimal(7,2)"},
+ {"name":"cr_return_ship_cost","type":"decimal(7,2)"},
+ {"name":"cr_refunded_cash","type":"decimal(7,2)"},
+ {"name":"cr_reversed_charge","type":"decimal(7,2)"},
+ {"name":"cr_store_credit","type":"decimal(7,2)"},
+ {"name":"cr_net_loss","type":"decimal(7,2)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/catalog_sales.json b/sdks/java/testing/tpcds/src/main/resources/schemas/catalog_sales.json
new file mode 100644
index 0000000..50fae92
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/catalog_sales.json
@@ -0,0 +1,36 @@
+{"schema": [
+ {"name":"cs_sold_date_sk","type":"identifier"},
+ {"name":"cs_sold_time_sk","type":"identifier"},
+ {"name":"cs_ship_date_sk","type":"identifier"},
+ {"name":"cs_bill_customer_sk","type":"identifier"},
+ {"name":"cs_bill_cdemo_sk","type":"identifier"},
+ {"name":"cs_bill_hdemo_sk","type":"identifier"},
+ {"name":"cs_bill_addr_sk","type":"identifier"},
+ {"name":"cs_ship_customer_sk","type":"identifier"},
+ {"name":"cs_ship_cdemo_sk","type":"identifier"},
+ {"name":"cs_ship_hdemo_sk","type":"identifier"},
+ {"name":"cs_ship_addr_sk","type":"identifier"},
+ {"name":"cs_call_center_sk","type":"identifier"},
+ {"name":"cs_catalog_page_sk","type":"identifier"},
+ {"name":"cs_ship_mode_sk","type":"identifier"},
+ {"name":"cs_warehouse_sk","type":"identifier"},
+ {"name":"cs_item_sk","type":"identifier"},
+ {"name":"cs_promo_sk","type":"identifier"},
+ {"name":"cs_order_number","type":"identifier"},
+ {"name":"cs_quantity","type":"integer"},
+ {"name":"cs_wholesale_cost","type":"decimal(7,2)"},
+ {"name":"cs_list_price","type":"decimal(7,2)"},
+ {"name":"cs_sales_price","type":"decimal(7,2)"},
+ {"name":"cs_ext_discount_amt","type":"decimal(7,2)"},
+ {"name":"cs_ext_sales_price","type":"decimal(7,2)"},
+ {"name":"cs_ext_wholesale_cost","type":"decimal(7,2)"},
+ {"name":"cs_ext_list_price","type":"decimal(7,2)"},
+ {"name":"cs_ext_tax","type":"decimal(7,2)"},
+ {"name":"cs_coupon_amt","type":"decimal(7,2)"},
+ {"name":"cs_ext_ship_cost","type":"decimal(7,2)"},
+ {"name":"cs_net_paid","type":"decimal(7,2)"},
+ {"name":"cs_net_paid_inc_tax","type":"decimal(7,2)"},
+ {"name":"cs_net_paid_inc_ship","type":"decimal(7,2)"},
+ {"name":"cs_net_paid_inc_ship_tax","type":"decimal(7,2)"},
+ {"name":"cs_net_profit","type":"decimal(7,2)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/customer.json b/sdks/java/testing/tpcds/src/main/resources/schemas/customer.json
new file mode 100644
index 0000000..fa1fcfb
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/customer.json
@@ -0,0 +1,20 @@
+{"schema": [
+ {"name":"c_customer_sk","type":"identifier"},
+ {"name":"c_customer_id","type":"char(16)"},
+ {"name":"c_current_cdemo_sk","type":"identifier"},
+ {"name":"c_current_hdemo_sk","type":"identifier"},
+ {"name":"c_current_addr_sk","type":"identifier"},
+ {"name":"c_first_shipto_date_sk","type":"identifier"},
+ {"name":"c_first_sales_date_sk","type":"identifier"},
+ {"name":"c_salutation","type":"char(10)"},
+ {"name":"c_first_name","type":"char(20)"},
+ {"name":"c_last_name","type":"char(30)"},
+ {"name":"c_preferred_cust_flag","type":"char(1)"},
+ {"name":"c_birth_day","type":"integer"},
+ {"name":"c_birth_month","type":"integer"},
+ {"name":"c_birth_year","type":"integer"},
+ {"name":"c_birth_country","type":"varchar(20)"},
+ {"name":"c_login","type":"char(13)"},
+ {"name":"c_email_address","type":"char(50)"},
+ {"name":"c_last_review_date_sk","type":"identifier"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/customer_address.json b/sdks/java/testing/tpcds/src/main/resources/schemas/customer_address.json
new file mode 100644
index 0000000..a37bdbe
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/customer_address.json
@@ -0,0 +1,15 @@
+{"schema": [
+ {"name":"ca_address_sk","type":"identifier"},
+ {"name":"ca_address_id","type":"char(16)"},
+ {"name":"ca_street_number","type":"char(10)"},
+ {"name":"ca_street_name","type":"varchar(60)"},
+ {"name":"ca_street_type","type":"char(15)"},
+ {"name":"ca_suite_number","type":"char(10)"},
+ {"name":"ca_city","type":"varchar(60)"},
+ {"name":"ca_county","type":"varchar(30)"},
+ {"name":"ca_state","type":"char(2)"},
+ {"name":"ca_zip","type":"char(10)"},
+ {"name":"ca_country","type":"varchar(20)"},
+ {"name":"ca_gmt_offset","type":"decimal(5,2)"},
+ {"name":"ca_location_type","type":"char(20)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/customer_demographics.json b/sdks/java/testing/tpcds/src/main/resources/schemas/customer_demographics.json
new file mode 100644
index 0000000..2b2211b
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/customer_demographics.json
@@ -0,0 +1,11 @@
+{"schema": [
+ {"name":"cd_demo_sk","type":"identifier"},
+ {"name":"cd_gender","type":"char(1)"},
+ {"name":"cd_marital_status","type":"char(1)"},
+ {"name":"cd_education_status","type":"char(20)"},
+ {"name":"cd_purchase_estimate","type":"integer"},
+ {"name":"cd_credit_rating","type":"char(10)"},
+ {"name":"cd_dep_count","type":"integer"},
+ {"name":"cd_dep_employed_count","type":"integer"},
+ {"name":"cd_dep_college_count","type":"integer"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/date_dim.json b/sdks/java/testing/tpcds/src/main/resources/schemas/date_dim.json
new file mode 100644
index 0000000..287738f
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/date_dim.json
@@ -0,0 +1,30 @@
+{"schema": [
+ {"name":"d_date_sk","type":"identifier"},
+ {"name":"d_date_id","type":"char(16)"},
+ {"name":"d_date","type":"date"},
+ {"name":"d_month_seq","type":"integer"},
+ {"name":"d_week_seq","type":"integer"},
+ {"name":"d_quarter_seq","type":"integer"},
+ {"name":"d_year","type":"integer"},
+ {"name":"d_dow","type":"integer"},
+ {"name":"d_moy","type":"integer"},
+ {"name":"d_dom","type":"integer"},
+ {"name":"d_qoy","type":"integer"},
+ {"name":"d_fy_year","type":"integer"},
+ {"name":"d_fy_quarter_seq","type":"integer"},
+ {"name":"d_fy_week_seq","type":"integer"},
+ {"name":"d_day_name","type":"char(9)"},
+ {"name":"d_quarter_name","type":"char(6)"},
+ {"name":"d_holiday","type":"char(1)"},
+ {"name":"d_weekend","type":"char(1)"},
+ {"name":"d_following_holiday","type":"char(1)"},
+ {"name":"d_first_dom","type":"integer"},
+ {"name":"d_last_dom","type":"integer"},
+ {"name":"d_same_day_ly","type":"integer"},
+ {"name":"d_same_day_lq","type":"integer"},
+ {"name":"d_current_day","type":"char(1)"},
+ {"name":"d_current_week","type":"char(1)"},
+ {"name":"d_current_month","type":"char(1)"},
+ {"name":"d_current_quarter","type":"char(1)"},
+ {"name":"d_current_year","type":"char(1)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/household_demographics.json b/sdks/java/testing/tpcds/src/main/resources/schemas/household_demographics.json
new file mode 100644
index 0000000..a261ae9
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/household_demographics.json
@@ -0,0 +1,7 @@
+{"schema": [
+ {"name":"hd_demo_sk","type":"identifier"},
+ {"name":"hd_income_band_sk","type":"identifier"},
+ {"name":"hd_buy_potential","type":"char(15)"},
+ {"name":"hd_dep_count","type":"integer"},
+ {"name":"hd_vehicle_count","type":"integer"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/income_band.json b/sdks/java/testing/tpcds/src/main/resources/schemas/income_band.json
new file mode 100644
index 0000000..3066b27
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/income_band.json
@@ -0,0 +1,5 @@
+{"schema": [
+ {"name":"ib_income_band_sk","type":"identifier"},
+ {"name":"ib_lower_bound","type":"integer"},
+ {"name":"ib_upper_bound","type":"integer"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/inventory.json b/sdks/java/testing/tpcds/src/main/resources/schemas/inventory.json
new file mode 100644
index 0000000..ee786bc
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/inventory.json
@@ -0,0 +1,7 @@
+{"schema": [
+ {"name":"inv_date_sk","type":"identifier"},
+ {"name":"inv_item_sk","type":"identifier"},
+ {"name":"inv_warehouse_sk","type":"identifier"},
+ {"name":"inv_quantity_on_hand","type":"integer"}
+]}
+
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/item.json b/sdks/java/testing/tpcds/src/main/resources/schemas/item.json
new file mode 100644
index 0000000..593aff1
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/item.json
@@ -0,0 +1,24 @@
+{"schema": [
+ {"name":"i_item_sk","type":"identifier"},
+ {"name":"i_item_id","type":"char(16)"},
+ {"name":"i_rec_start_date","type":"date"},
+ {"name":"i_rec_end_date","type":"date"},
+ {"name":"i_item_desc","type":"varchar(200)"},
+ {"name":"i_current_price","type":"decimal(7,2)"},
+ {"name":"i_wholesale_cost","type":"decimal(7,2)"},
+ {"name":"i_brand_id","type":"integer"},
+ {"name":"i_brand","type":"char(50)"},
+ {"name":"i_class_id","type":"integer"},
+ {"name":"i_class","type":"char(50)"},
+ {"name":"i_category_id","type":"integer"},
+ {"name":"i_category","type":"char(50)"},
+ {"name":"i_manufact_id","type":"integer"},
+ {"name":"i_manufact","type":"char(50)"},
+ {"name":"i_size","type":"char(20)"},
+ {"name":"i_formulation","type":"char(20)"},
+ {"name":"i_color","type":"char(20)"},
+ {"name":"i_units","type":"char(10)"},
+ {"name":"i_container","type":"char(10)"},
+ {"name":"i_manager_id","type":"integer"},
+ {"name":"i_product_name","type":"char(50)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/promotion.json b/sdks/java/testing/tpcds/src/main/resources/schemas/promotion.json
new file mode 100644
index 0000000..28d57d4
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/promotion.json
@@ -0,0 +1,21 @@
+{"schema": [
+ {"name":"p_promo_sk","type":"identifier"},
+ {"name":"p_promo_id","type":"char(16)"},
+ {"name":"p_start_date_sk","type":"identifier"},
+ {"name":"p_end_date_sk","type":"identifier"},
+ {"name":"p_item_sk","type":"identifier"},
+ {"name":"p_cost","type":"decimal(15,2)"},
+ {"name":"p_response_target","type":"integer"},
+ {"name":"p_promo_name","type":"char(50)"},
+ {"name":"p_channel_dmail","type":"char(1)"},
+ {"name":"p_channel_email","type":"char(1)"},
+ {"name":"p_channel_catalog","type":"char(1)"},
+ {"name":"p_channel_tv","type":"char(1)"},
+ {"name":"p_channel_radio","type":"char(1)"},
+ {"name":"p_channel_press","type":"char(1)"},
+ {"name":"p_channel_event","type":"char(1)"},
+ {"name":"p_channel_demo","type":"char(1)"},
+ {"name":"p_channel_details","type":"varchar(100)"},
+ {"name":"p_purpose","type":"char(15)"},
+ {"name":"p_discount_active","type":"char(1)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/reason.json b/sdks/java/testing/tpcds/src/main/resources/schemas/reason.json
new file mode 100644
index 0000000..64b9723
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/reason.json
@@ -0,0 +1,5 @@
+{"schema": [
+ {"name":"r_reason_sk","type":"identifier"},
+ {"name":"r_reason_id","type":"char(16)"},
+ {"name":"r_reason_desc","type":"char(100)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/ship_mode.json b/sdks/java/testing/tpcds/src/main/resources/schemas/ship_mode.json
new file mode 100644
index 0000000..09c4873
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/ship_mode.json
@@ -0,0 +1,8 @@
+{"schema": [
+ {"name":"sm_ship_mode_sk","type":"identifier"},
+ {"name":"sm_ship_mode_id","type":"char(16)"},
+ {"name":"sm_type","type":"char(30)"},
+ {"name":"sm_code","type":"char(10)"},
+ {"name":"sm_carrier","type":"char(20)"},
+ {"name":"sm_contract","type":"char(20)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/store.json b/sdks/java/testing/tpcds/src/main/resources/schemas/store.json
new file mode 100644
index 0000000..3df8465
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/store.json
@@ -0,0 +1,31 @@
+{"schema": [
+ {"name":"s_store_sk","type":"identifier"},
+ {"name":"s_store_id","type":"char(16)"},
+ {"name":"s_rec_start_date","type":"date"},
+ {"name":"s_rec_end_date","type":"date"},
+ {"name":"s_closed_date_sk","type":"identifier"},
+ {"name":"s_store_name","type":"varchar(50)"},
+ {"name":"s_number_employees","type":"integer"},
+ {"name":"s_floor_space","type":"integer"},
+ {"name":"s_hours","type":"char(20)"},
+ {"name":"S_manager","type":"varchar(40)"},
+ {"name":"S_market_id","type":"integer"},
+ {"name":"S_geography_class","type":"varchar(100)"},
+ {"name":"S_market_desc","type":"varchar(100)"},
+ {"name":"s_market_manager","type":"varchar(40)"},
+ {"name":"s_division_id","type":"integer"},
+ {"name":"s_division_name","type":"varchar(50)"},
+ {"name":"s_company_id","type":"integer"},
+ {"name":"s_company_name","type":"varchar(50)"},
+ {"name":"s_street_number","type":"varchar(10)"},
+ {"name":"s_street_name","type":"varchar(60)"},
+ {"name":"s_street_type","type":"char(15)"},
+ {"name":"s_suite_number","type":"char(10)"},
+ {"name":"s_city","type":"varchar(60)"},
+ {"name":"s_county","type":"varchar(30)"},
+ {"name":"s_state","type":"char(2)"},
+ {"name":"s_zip","type":"char(10)"},
+ {"name":"s_country","type":"varchar(20)"},
+ {"name":"s_gmt_offset","type":"decimal(5,2)"},
+ {"name":"s_tax_percentage","type":"decimal(5,2)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/store_returns.json b/sdks/java/testing/tpcds/src/main/resources/schemas/store_returns.json
new file mode 100644
index 0000000..99b0db1
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/store_returns.json
@@ -0,0 +1,22 @@
+{"schema": [
+ {"name":"sr_returned_date_sk","type":"identifier"},
+ {"name":"sr_return_time_sk","type":"identifier"},
+ {"name":"sr_item_sk","type":"identifier"},
+ {"name":"sr_customer_sk","type":"identifier"},
+ {"name":"sr_cdemo_sk","type":"identifier"},
+ {"name":"sr_hdemo_sk","type":"identifier"},
+ {"name":"sr_addr_sk","type":"identifier"},
+ {"name":"sr_store_sk","type":"identifier"},
+ {"name":"sr_reason_sk","type":"identifier"},
+ {"name":"sr_ticket_number","type":"identifier"},
+ {"name":"sr_return_quantity","type":"integer"},
+ {"name":"sr_return_amt","type":"decimal(7,2)"},
+ {"name":"sr_return_tax","type":"decimal(7,2)"},
+ {"name":"sr_return_amt_inc_tax","type":"decimal(7,2)"},
+ {"name":"sr_fee","type":"decimal(7,2)"},
+ {"name":"sr_return_ship_cost","type":"decimal(7,2)"},
+ {"name":"sr_refunded_cash","type":"decimal(7,2)"},
+ {"name":"sr_reversed_charge","type":"decimal(7,2)"},
+ {"name":"sr_store_credit","type":"decimal(7,2)"},
+ {"name":"sr_net_loss","type":"decimal(7,2)"}
+]}
\ No newline at end of file
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/store_sales.json b/sdks/java/testing/tpcds/src/main/resources/schemas/store_sales.json
new file mode 100644
index 0000000..3b133d8
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/store_sales.json
@@ -0,0 +1,25 @@
+{"schema": [
+ {"name":"ss_sold_date_sk","type":"identifier"},
+ {"name":"ss_sold_time_sk","type":"identifier"},
+ {"name":"ss_item_sk","type":"identifier"},
+ {"name":"ss_customer_sk","type":"identifier"},
+ {"name":"ss_cdemo_sk","type":"identifier"},
+ {"name":"ss_hdemo_sk","type":"identifier"},
+ {"name":"ss_addr_sk","type":"identifier"},
+ {"name":"ss_store_sk","type":"identifier"},
+ {"name":"ss_promo_sk","type":"identifier"},
+ {"name":"ss_ticket_number","type":"identifier"},
+ {"name":"ss_quantity","type":"integer"},
+ {"name":"ss_wholesale_cost","type":"decimal(7,2)"},
+ {"name":"ss_list_price","type":"decimal(7,2)"},
+ {"name":"ss_sales_price","type":"decimal(7,2)"},
+ {"name":"ss_ext_discount_amt","type":"decimal(7,2)"},
+ {"name":"ss_ext_sales_price","type":"decimal(7,2)"},
+ {"name":"ss_ext_wholesale_cost","type":"decimal(7,2)"},
+ {"name":"ss_ext_list_price","type":"decimal(7,2)"},
+ {"name":"ss_ext_tax","type":"decimal(7,2)"},
+ {"name":"ss_coupon_amt","type":"decimal(7,2)"},
+ {"name":"ss_net_paid","type":"decimal(7,2)"},
+ {"name":"ss_net_paid_inc_tax","type":"decimal(7,2)"},
+ {"name":"ss_net_profit","type":"decimal(7,2)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/time_dim.json b/sdks/java/testing/tpcds/src/main/resources/schemas/time_dim.json
new file mode 100644
index 0000000..e1d51f6
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/time_dim.json
@@ -0,0 +1,12 @@
+{"schema": [
+ {"name":"t_time_sk","type":"Identifier"},
+ {"name":"t_time_id","type":"char(16)"},
+ {"name":"t_time","type":"Integer"},
+ {"name":"t_hour","type":"Integer"},
+ {"name":"t_minute","type":"Integer"},
+ {"name":"t_second","type":"Integer"},
+ {"name":"t_am_pm","type":"char(2)"},
+ {"name":"t_shift","type":"char(20)"},
+ {"name":"t_sub_shift","type":"char(20)"},
+ {"name":"t_meal_time","type":"char(20)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/warehouse.json b/sdks/java/testing/tpcds/src/main/resources/schemas/warehouse.json
new file mode 100644
index 0000000..e3126de
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/warehouse.json
@@ -0,0 +1,16 @@
+{"schema": [
+ {"name":"w_warehouse_sk","type":"identifier"},
+ {"name":"w_warehouse_id","type":"char(16)"},
+ {"name":"w_warehouse_name","type":"varchar(20)"},
+ {"name":"w_warehouse_sq_ft","type":"integer"},
+ {"name":"w_street_number","type":"char(10)"},
+ {"name":"w_street_name","type":"varchar(60)"},
+ {"name":"w_street_type","type":"char(15)"},
+ {"name":"w_suite_number","type":"char(10)"},
+ {"name":"w_city","type":"varchar(60)"},
+ {"name":"w_county","type":"varchar(30)"},
+ {"name":"w_state","type":"char(2)"},
+ {"name":"w_zip","type":"char(10)"},
+ {"name":"w_country","type":"varchar(20)"},
+ {"name":"w_gmt_offset","type":"decimal(5,2)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/web_page.json b/sdks/java/testing/tpcds/src/main/resources/schemas/web_page.json
new file mode 100644
index 0000000..4dc3436
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/web_page.json
@@ -0,0 +1,16 @@
+{"schema": [
+ {"name":"wp_web_page_sk","type":"identifier"},
+ {"name":"wp_web_page_id","type":"char(16)"},
+ {"name":"wp_rec_start_date","type":"date"},
+ {"name":"wp_rec_end_date","type":"date"},
+ {"name":"wp_creation_date_sk","type":"identifier"},
+ {"name":"wp_access_date_sk","type":"identifier"},
+ {"name":"wp_autogen_flag","type":"char(1)"},
+ {"name":"wp_customer_sk","type":"identifier"},
+ {"name":"wp_url","type":"varchar(100)"},
+ {"name":"wp_type","type":"char(50)"},
+ {"name":"wp_char_count","type":"integer"},
+ {"name":"wp_link_count","type":"integer"},
+ {"name":"wp_image_count","type":"integer"},
+ {"name":"wp_max_ad_count","type":"integer"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/web_returns.json b/sdks/java/testing/tpcds/src/main/resources/schemas/web_returns.json
new file mode 100644
index 0000000..101ef1c
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/web_returns.json
@@ -0,0 +1,26 @@
+{"schema": [
+ {"name":"wr_returned_date_sk","type":"identifier"},
+ {"name":"wr_returned_time_sk","type":"identifier"},
+ {"name":"wr_item_sk","type":"identifier"},
+ {"name":"wr_refunded_customer_sk","type":"identifier"},
+ {"name":"wr_refunded_cdemo_sk","type":"identifier"},
+ {"name":"wr_refunded_hdemo_sk","type":"identifier"},
+ {"name":"wr_refunded_addr_sk","type":"identifier"},
+ {"name":"wr_returning_customer_sk","type":"identifier"},
+ {"name":"wr_returning_cdemo_sk","type":"identifier"},
+ {"name":"wr_returning_hdemo_sk","type":"identifier"},
+ {"name":"wr_returning_addr_sk","type":"identifier"},
+ {"name":"wr_web_page_sk","type":"identifier"},
+ {"name":"wr_reason_sk","type":"identifier"},
+ {"name":"wr_order_number","type":"identifier"},
+ {"name":"wr_return_quantity","type":"integer"},
+ {"name":"wr_return_amt","type":"decimal(7,2)"},
+ {"name":"wr_return_tax","type":"decimal(7,2)"},
+ {"name":"wr_return_amt_inc_tax","type":"decimal(7,2)"},
+ {"name":"wr_fee","type":"decimal(7,2)"},
+ {"name":"wr_return_ship_cost","type":"decimal(7,2)"},
+ {"name":"wr_refunded_cash","type":"decimal(7,2)"},
+ {"name":"wr_reversed_charge","type":"decimal(7,2)"},
+ {"name":"wr_account_credit","type":"decimal(7,2)"},
+ {"name":"wr_net_loss","type":"decimal(7,2)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/web_sales.json b/sdks/java/testing/tpcds/src/main/resources/schemas/web_sales.json
new file mode 100644
index 0000000..2cbcdaa
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/web_sales.json
@@ -0,0 +1,36 @@
+{"schema": [
+ {"name":"ws_sold_date_sk","type":"identifier"},
+ {"name":"ws_sold_time_sk","type":"identifier"},
+ {"name":"ws_ship_date_sk","type":"identifier"},
+ {"name":"ws_item_sk","type":"identifier"},
+ {"name":"ws_bill_customer_sk","type":"identifier"},
+ {"name":"ws_bill_cdemo_sk","type":"identifier"},
+ {"name":"ws_bill_hdemo_sk","type":"identifier"},
+ {"name":"ws_bill_addr_sk","type":"identifier"},
+ {"name":"ws_ship_customer_sk","type":"identifier"},
+ {"name":"ws_ship_cdemo_sk","type":"identifier"},
+ {"name":"ws_ship_hdemo_sk","type":"identifier"},
+ {"name":"ws_ship_addr_sk","type":"identifier"},
+ {"name":"ws_web_page_sk","type":"identifier"},
+ {"name":"ws_web_site_sk","type":"identifier"},
+ {"name":"ws_ship_mode_sk","type":"identifier"},
+ {"name":"ws_warehouse_sk","type":"identifier"},
+ {"name":"ws_promo_sk","type":"identifier"},
+ {"name":"ws_order_number","type":"identifier"},
+ {"name":"ws_quantity","type":"integer"},
+ {"name":"ws_wholesale_cost","type":"decimal(7,2)"},
+ {"name":"ws_list_price","type":"decimal(7,2)"},
+ {"name":"ws_sales_price","type":"decimal(7,2)"},
+ {"name":"ws_ext_discount_amt","type":"decimal(7,2)"},
+ {"name":"ws_ext_sales_price","type":"decimal(7,2)"},
+ {"name":"ws_ext_wholesale_cost","type":"decimal(7,2)"},
+ {"name":"ws_ext_list_price","type":"decimal(7,2)"},
+ {"name":"ws_ext_tax","type":"decimal(7,2)"},
+ {"name":"ws_coupon_amt","type":"decimal(7,2)"},
+ {"name":"ws_ext_ship_cost","type":"decimal(7,2)"},
+ {"name":"ws_net_paid","type":"decimal(7,2)"},
+ {"name":"ws_net_paid_inc_tax","type":"decimal(7,2)"},
+ {"name":"ws_net_paid_inc_ship","type":"decimal(7,2)"},
+ {"name":"ws_net_paid_inc_ship_tax","type":"decimal(7,2)"},
+ {"name":"ws_net_profit","type":"decimal(7,2)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/main/resources/schemas/web_site.json b/sdks/java/testing/tpcds/src/main/resources/schemas/web_site.json
new file mode 100644
index 0000000..7cecde0
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/main/resources/schemas/web_site.json
@@ -0,0 +1,28 @@
+{"schema": [
+ {"name":"web_site_sk","type":"varchar(100)"},
+ {"name":"web_site_id","type":"char(16)"},
+ {"name":"web_rec_start_date","type":"date"},
+ {"name":"web_rec_end_date","type":"date"},
+ {"name":"web_name","type":"varchar(50)"},
+ {"name":"web_open_date_sk","type":"identifier"},
+ {"name":"web_close_date_sk","type":"identifier"},
+ {"name":"web_class","type":"varchar(50)"},
+ {"name":"web_manager","type":"varchar(40)"},
+ {"name":"web_mkt_id","type":"integer"},
+ {"name":"web_mkt_class","type":"varchar(50)"},
+ {"name":"web_mkt_desc","type":"varchar(100)"},
+ {"name":"web_market_manager","type":"varchar(40)"},
+ {"name":"web_company_id","type":"integer"},
+ {"name":"web_company_name","type":"char(50)"},
+ {"name":"web_street_number","type":"char(10)"},
+ {"name":"web_street_name","type":"varchar(60)"},
+ {"name":"web_street_type","type":"char(15)"},
+ {"name":"web_suite_number","type":"char(10)"},
+ {"name":"web_city","type":"varchar(60)"},
+ {"name":"web_county","type":"varchar(30)"},
+ {"name":"web_state","type":"char(2)"},
+ {"name":"web_zip","type":"char(10)"},
+ {"name":"web_country","type":"varchar(20)"},
+ {"name":"web_gmt_offset","type":"decimal(5,2)"},
+ {"name":"web_tax_percentage","type":"decimal(5,2)"}
+]}
diff --git a/sdks/java/testing/tpcds/src/test/java/org/apache/beam/sdk/tpcds/QueryReaderTest.java b/sdks/java/testing/tpcds/src/test/java/org/apache/beam/sdk/tpcds/QueryReaderTest.java
new file mode 100644
index 0000000..5696410
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/test/java/org/apache/beam/sdk/tpcds/QueryReaderTest.java
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.tpcds;
+
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
+
+public class QueryReaderTest {
+ private final String headers = "-- Licensed to the Apache Software Foundation (ASF) under one\n" +
+ "-- or more contributor license agreements. See the NOTICE file\n" +
+ "-- distributed with this work for additional information\n" +
+ "-- regarding copyright ownership. The ASF licenses this file\n" +
+ "-- to you under the Apache License, Version 2.0 (the\n" +
+ "-- \"License\"); you may not use this file except in compliance\n" +
+ "-- with the License. You may obtain a copy of the License at\n" +
+ "--\n" +
+ "-- http://www.apache.org/licenses/LICENSE-2.0\n" +
+ "--\n" +
+ "-- Unless required by applicable law or agreed to in writing, software\n" +
+ "-- distributed under the License is distributed on an \"AS IS\" BASIS,\n" +
+ "-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n" +
+ "-- See the License for the specific language governing permissions and\n" +
+ "-- limitations under the License.\n";
+
+ @Test
+ public void testQuery3String() throws Exception {
+ String query3String = QueryReader.readQuery("query3");
+ String expected = "select dt.d_year \n" +
+ " ,item.i_brand_id brand_id \n" +
+ " ,item.i_brand brand\n" +
+ " ,sum(ss_ext_sales_price) sum_agg\n" +
+ " from date_dim dt \n" +
+ " ,store_sales\n" +
+ " ,item\n" +
+ " where dt.d_date_sk = store_sales.ss_sold_date_sk\n" +
+ " and store_sales.ss_item_sk = item.i_item_sk\n" +
+ " and item.i_manufact_id = 436\n" +
+ " and dt.d_moy=12\n" +
+ " group by dt.d_year\n" +
+ " ,item.i_brand\n" +
+ " ,item.i_brand_id\n" +
+ " order by dt.d_year\n" +
+ " ,sum_agg desc\n" +
+ " ,brand_id\n" +
+ " limit 100";
+ String query3StringNoSpaces = query3String.replaceAll("\\s+", "");
+ String expectedNoSpaces = (headers + expected).replaceAll("\\s+", "");
+ assertEquals(expectedNoSpaces, query3StringNoSpaces);
+ }
+
+ @Test
+ public void testQuery4String() throws Exception {
+ String query4String = QueryReader.readQuery("query4");
+ String expected = "with year_total as (\n" +
+ " select c_customer_id customer_id\n" +
+ " ,c_first_name customer_first_name\n" +
+ " ,c_last_name customer_last_name\n" +
+ " ,c_preferred_cust_flag customer_preferred_cust_flag\n" +
+ " ,c_birth_country customer_birth_country\n" +
+ " ,c_login customer_login\n" +
+ " ,c_email_address customer_email_address\n" +
+ " ,d_year dyear\n" +
+ " ,sum(((ss_ext_list_price-ss_ext_wholesale_cost-ss_ext_discount_amt)+ss_ext_sales_price)/2) year_total\n" +
+ " ,'s' sale_type\n" +
+ " from customer\n" +
+ " ,store_sales\n" +
+ " ,date_dim\n" +
+ " where c_customer_sk = ss_customer_sk\n" +
+ " and ss_sold_date_sk = d_date_sk\n" +
+ " group by c_customer_id\n" +
+ " ,c_first_name\n" +
+ " ,c_last_name\n" +
+ " ,c_preferred_cust_flag\n" +
+ " ,c_birth_country\n" +
+ " ,c_login\n" +
+ " ,c_email_address\n" +
+ " ,d_year\n" +
+ " union all\n" +
+ " select c_customer_id customer_id\n" +
+ " ,c_first_name customer_first_name\n" +
+ " ,c_last_name customer_last_name\n" +
+ " ,c_preferred_cust_flag customer_preferred_cust_flag\n" +
+ " ,c_birth_country customer_birth_country\n" +
+ " ,c_login customer_login\n" +
+ " ,c_email_address customer_email_address\n" +
+ " ,d_year dyear\n" +
+ " ,sum((((cs_ext_list_price-cs_ext_wholesale_cost-cs_ext_discount_amt)+cs_ext_sales_price)/2) ) year_total\n" +
+ " ,'c' sale_type\n" +
+ " from customer\n" +
+ " ,catalog_sales\n" +
+ " ,date_dim\n" +
+ " where c_customer_sk = cs_bill_customer_sk\n" +
+ " and cs_sold_date_sk = d_date_sk\n" +
+ " group by c_customer_id\n" +
+ " ,c_first_name\n" +
+ " ,c_last_name\n" +
+ " ,c_preferred_cust_flag\n" +
+ " ,c_birth_country\n" +
+ " ,c_login\n" +
+ " ,c_email_address\n" +
+ " ,d_year\n" +
+ "union all\n" +
+ " select c_customer_id customer_id\n" +
+ " ,c_first_name customer_first_name\n" +
+ " ,c_last_name customer_last_name\n" +
+ " ,c_preferred_cust_flag customer_preferred_cust_flag\n" +
+ " ,c_birth_country customer_birth_country\n" +
+ " ,c_login customer_login\n" +
+ " ,c_email_address customer_email_address\n" +
+ " ,d_year dyear\n" +
+ " ,sum((((ws_ext_list_price-ws_ext_wholesale_cost-ws_ext_discount_amt)+ws_ext_sales_price)/2) ) year_total\n" +
+ " ,'w' sale_type\n" +
+ " from customer\n" +
+ " ,web_sales\n" +
+ " ,date_dim\n" +
+ " where c_customer_sk = ws_bill_customer_sk\n" +
+ " and ws_sold_date_sk = d_date_sk\n" +
+ " group by c_customer_id\n" +
+ " ,c_first_name\n" +
+ " ,c_last_name\n" +
+ " ,c_preferred_cust_flag\n" +
+ " ,c_birth_country\n" +
+ " ,c_login\n" +
+ " ,c_email_address\n" +
+ " ,d_year\n" +
+ " )\n" +
+ " select \n" +
+ " t_s_secyear.customer_id\n" +
+ " ,t_s_secyear.customer_first_name\n" +
+ " ,t_s_secyear.customer_last_name\n" +
+ " ,t_s_secyear.customer_email_address\n" +
+ " from year_total t_s_firstyear\n" +
+ " ,year_total t_s_secyear\n" +
+ " ,year_total t_c_firstyear\n" +
+ " ,year_total t_c_secyear\n" +
+ " ,year_total t_w_firstyear\n" +
+ " ,year_total t_w_secyear\n" +
+ " where t_s_secyear.customer_id = t_s_firstyear.customer_id\n" +
+ " and t_s_firstyear.customer_id = t_c_secyear.customer_id\n" +
+ " and t_s_firstyear.customer_id = t_c_firstyear.customer_id\n" +
+ " and t_s_firstyear.customer_id = t_w_firstyear.customer_id\n" +
+ " and t_s_firstyear.customer_id = t_w_secyear.customer_id\n" +
+ " and t_s_firstyear.sale_type = 's'\n" +
+ " and t_c_firstyear.sale_type = 'c'\n" +
+ " and t_w_firstyear.sale_type = 'w'\n" +
+ " and t_s_secyear.sale_type = 's'\n" +
+ " and t_c_secyear.sale_type = 'c'\n" +
+ " and t_w_secyear.sale_type = 'w'\n" +
+ " and t_s_firstyear.dyear = 2001\n" +
+ " and t_s_secyear.dyear = 2001+1\n" +
+ " and t_c_firstyear.dyear = 2001\n" +
+ " and t_c_secyear.dyear = 2001+1\n" +
+ " and t_w_firstyear.dyear = 2001\n" +
+ " and t_w_secyear.dyear = 2001+1\n" +
+ " and t_s_firstyear.year_total > 0\n" +
+ " and t_c_firstyear.year_total > 0\n" +
+ " and t_w_firstyear.year_total > 0\n" +
+ " and case when t_c_firstyear.year_total > 0 then t_c_secyear.year_total / t_c_firstyear.year_total else null end\n" +
+ " > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else null end\n" +
+ " and case when t_c_firstyear.year_total > 0 then t_c_secyear.year_total / t_c_firstyear.year_total else null end\n" +
+ " > case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else null end\n" +
+ " order by t_s_secyear.customer_id\n" +
+ " ,t_s_secyear.customer_first_name\n" +
+ " ,t_s_secyear.customer_last_name\n" +
+ " ,t_s_secyear.customer_email_address\n" +
+ "limit 100";
+ String query4StringNoSpaces = query4String.replaceAll("\\s+", "");
+ String expectedNoSpaces = (headers + expected).replaceAll("\\s+", "");
+ assertEquals(expectedNoSpaces, query4StringNoSpaces);
+ }
+
+ @Test
+ public void testQuery55String() throws Exception {
+ String query55String = QueryReader.readQuery("query55");
+ String expected = "select i_brand_id brand_id, i_brand brand,\n" +
+ " \tsum(ss_ext_sales_price) ext_price\n" +
+ " from date_dim, store_sales, item\n" +
+ " where d_date_sk = ss_sold_date_sk\n" +
+ " \tand ss_item_sk = i_item_sk\n" +
+ " \tand i_manager_id=36\n" +
+ " \tand d_moy=12\n" +
+ " \tand d_year=2001\n" +
+ " group by i_brand, i_brand_id\n" +
+ " order by ext_price desc, i_brand_id\n" +
+ "limit 100";
+ String query55StringNoSpaces = query55String.replaceAll("\\s+", "");
+ String expectedNoSpaces = (headers + expected).replaceAll("\\s+", "");
+ assertEquals(expectedNoSpaces, query55StringNoSpaces);
+ }
+}
diff --git a/sdks/java/testing/tpcds/src/test/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoaderTest.java b/sdks/java/testing/tpcds/src/test/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoaderTest.java
new file mode 100644
index 0000000..7748bee
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/test/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoaderTest.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.tpcds;
+
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+
+public class TableSchemaJSONLoaderTest {
+ @Test
+ public void testStoreReturnsTable() throws Exception {
+ String storeReturnsSchemaString = TableSchemaJSONLoader.parseTableSchema("store_returns");
+ String expected = "sr_returned_date_sk bigint,"
+ + "sr_return_time_sk bigint,"
+ + "sr_item_sk bigint,"
+ + "sr_customer_sk bigint,"
+ + "sr_cdemo_sk bigint,"
+ + "sr_hdemo_sk bigint,"
+ + "sr_addr_sk bigint,"
+ + "sr_store_sk bigint,"
+ + "sr_reason_sk bigint,"
+ + "sr_ticket_number bigint,"
+ + "sr_return_quantity bigint,"
+ + "sr_return_amt double,"
+ + "sr_return_tax double,"
+ + "sr_return_amt_inc_tax double,"
+ + "sr_fee double,"
+ + "sr_return_ship_cost double,"
+ + "sr_refunded_cash double,"
+ + "sr_reversed_charge double,"
+ + "sr_store_credit double,"
+ + "sr_net_loss double";
+ assertEquals(expected, storeReturnsSchemaString);
+ }
+
+ @Test
+ public void testItemTable() throws Exception {
+ String itemSchemaString = TableSchemaJSONLoader.parseTableSchema("item");
+ String expected = "i_item_sk bigint,"
+ + "i_item_id varchar,"
+ + "i_rec_start_date varchar,"
+ + "i_rec_end_date varchar,"
+ + "i_item_desc varchar,"
+ + "i_current_price double,"
+ + "i_wholesale_cost double,"
+ + "i_brand_id bigint,"
+ + "i_brand varchar,"
+ + "i_class_id bigint,"
+ + "i_class varchar,"
+ + "i_category_id bigint,"
+ + "i_category varchar,"
+ + "i_manufact_id bigint,"
+ + "i_manufact varchar,"
+ + "i_size varchar,"
+ + "i_formulation varchar,"
+ + "i_color varchar,"
+ + "i_units varchar,"
+ + "i_container varchar,"
+ + "i_manager_id bigint,"
+ + "i_product_name varchar";
+ assertEquals(expected, itemSchemaString);
+ }
+
+ @Test
+ public void testDateDimTable() throws Exception {
+ String dateDimSchemaString = TableSchemaJSONLoader.parseTableSchema("date_dim");
+ String expected = "d_date_sk bigint,"
+ + "d_date_id varchar,"
+ + "d_date varchar,"
+ + "d_month_seq bigint,"
+ + "d_week_seq bigint,"
+ + "d_quarter_seq bigint,"
+ + "d_year bigint,"
+ + "d_dow bigint,"
+ + "d_moy bigint,"
+ + "d_dom bigint,"
+ + "d_qoy bigint,"
+ + "d_fy_year bigint,"
+ + "d_fy_quarter_seq bigint,"
+ + "d_fy_week_seq bigint,"
+ + "d_day_name varchar,"
+ + "d_quarter_name varchar,"
+ + "d_holiday varchar,"
+ + "d_weekend varchar,"
+ + "d_following_holiday varchar,"
+ + "d_first_dom bigint,"
+ + "d_last_dom bigint,"
+ + "d_same_day_ly bigint,"
+ + "d_same_day_lq bigint,"
+ + "d_current_day varchar,"
+ + "d_current_week varchar,"
+ + "d_current_month varchar,"
+ + "d_current_quarter varchar,"
+ + "d_current_year varchar";
+ assertEquals(expected, dateDimSchemaString);
+ }
+
+ @Test
+ public void testWarehouseTable() throws Exception {
+ String warehouseSchemaString = TableSchemaJSONLoader.parseTableSchema("warehouse");
+ String expected = "w_warehouse_sk bigint,"
+ + "w_warehouse_id varchar,"
+ + "w_warehouse_name varchar,"
+ + "w_warehouse_sq_ft bigint,"
+ + "w_street_number varchar,"
+ + "w_street_name varchar,"
+ + "w_street_type varchar,"
+ + "w_suite_number varchar,"
+ + "w_city varchar,"
+ + "w_county varchar,"
+ + "w_state varchar,"
+ + "w_zip varchar,"
+ + "w_country varchar,"
+ + "w_gmt_offset double";
+ assertEquals(expected, warehouseSchemaString);
+ }
+
+ @Test
+ public void testGetAllTableNames() {
+ List<String> tableNames = TableSchemaJSONLoader.getAllTableNames();
+ Collections.sort(tableNames);
+ List<String> expectedTableNames = Arrays.asList("call_center", "catalog_page", "catalog_returns", "catalog_sales", "customer", "customer_address", "customer_demographics",
+ "date_dim", "household_demographics", "income_band", "inventory", "item", "promotion", "reason", "ship_mode", "store", "store_returns", "store_sales", "time_dim",
+ "warehouse", "web_page", "web_returns", "web_sales", "web_site");
+
+ assertEquals(expectedTableNames.size(), tableNames.size());
+
+ for (int i = 0; i < tableNames.size(); i++) {
+ assertEquals(expectedTableNames.get(i), tableNames.get(i));
+ }
+ }
+}
diff --git a/sdks/java/testing/tpcds/src/test/java/org/apache/beam/sdk/tpcds/TpcdsParametersReaderTest.java b/sdks/java/testing/tpcds/src/test/java/org/apache/beam/sdk/tpcds/TpcdsParametersReaderTest.java
new file mode 100644
index 0000000..3f8c951
--- /dev/null
+++ b/sdks/java/testing/tpcds/src/test/java/org/apache/beam/sdk/tpcds/TpcdsParametersReaderTest.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.tpcds;
+
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TpcdsParametersReaderTest {
+ private TpcdsOptions tpcdsOptions;
+ private TpcdsOptions tpcdsOptionsError;
+
+ @Before
+ public void initializeTpcdsOptions() {
+ tpcdsOptions = PipelineOptionsFactory.as(TpcdsOptions.class);
+ tpcdsOptionsError = PipelineOptionsFactory.as(TpcdsOptions.class);
+
+ tpcdsOptions.setDataSize("1G");
+ tpcdsOptions.setQueries("1,2,3");
+ tpcdsOptions.setTpcParallel(2);
+
+ tpcdsOptionsError.setDataSize("5G");
+ tpcdsOptionsError.setQueries("0,100");
+ tpcdsOptionsError.setTpcParallel(0);
+ }
+
+ @Test
+ public void testGetAndCheckDataSize() throws Exception {
+ String dataSize = TpcdsParametersReader.getAndCheckDataSize(tpcdsOptions);
+ String expected = "1G";
+ assertEquals(expected, dataSize);
+ }
+
+ @Test( expected = Exception.class)
+ public void testGetAndCheckDataSizeException() throws Exception {
+ TpcdsParametersReader.getAndCheckDataSize(tpcdsOptionsError);
+ }
+
+ @Test
+ public void testGetAndCheckQueries() throws Exception {
+ TpcdsOptions tpcdsOptionsAll = PipelineOptionsFactory.as(TpcdsOptions.class);
+ tpcdsOptionsAll.setQueries("all");
+ String[] queryNameArray = TpcdsParametersReader.getAndCheckQueryNameArray(tpcdsOptionsAll);
+ String[] expected = new String[99];
+ for (int i = 0; i < 99; i++) {
+ expected[i] = "query" + (i + 1);
+ }
+ Assert.assertArrayEquals(expected, queryNameArray);
+ }
+
+ @Test
+ public void testGetAndCheckAllQueries() throws Exception {
+ String[] queryNameArray = TpcdsParametersReader.getAndCheckQueryNameArray(tpcdsOptions);
+ String[] expected = {"query1", "query2", "query3"};
+ Assert.assertArrayEquals(expected, queryNameArray);
+ }
+
+ @Test( expected = Exception.class)
+ public void testGetAndCheckQueriesException() throws Exception {
+ TpcdsParametersReader.getAndCheckQueryNameArray(tpcdsOptionsError);
+ }
+
+ @Test
+ public void testGetAndCheckTpcParallel() throws Exception {
+ int nThreads = TpcdsParametersReader.getAndCheckTpcParallel(tpcdsOptions);
+ int expected = 2;
+ assertEquals(expected, nThreads);
+ }
+
+ @Test( expected = Exception.class)
+ public void ttestGetAndCheckTpcParallelException() throws Exception {
+ TpcdsParametersReader.getAndCheckTpcParallel(tpcdsOptionsError);
+ }
+}
diff --git a/sdks/python/apache_beam/internal/pickler.py b/sdks/python/apache_beam/internal/pickler.py
index 9b10955..c4bfb44 100644
--- a/sdks/python/apache_beam/internal/pickler.py
+++ b/sdks/python/apache_beam/internal/pickler.py
@@ -190,16 +190,15 @@
if obj_id not in known_module_dicts:
# Trigger loading of lazily loaded modules (such as pytest vendored
# modules).
- # This first pass over sys.modules needs to iterate on a copy of
- # sys.modules since lazy loading modifies the dictionary, hence the use
- # of list().
+ # This pass over sys.modules needs to iterate on a copy of sys.modules
+ # since lazy loading modifies the dictionary, hence the use of list().
for m in list(sys.modules.values()):
try:
_ = m.__dict__
except AttributeError:
pass
- for m in sys.modules.values():
+ for m in list(sys.modules.values()):
try:
if (m and m.__name__ != '__main__' and
isinstance(m, dill.dill.ModuleType)):
diff --git a/sdks/python/apache_beam/io/gcp/dicomclient.py b/sdks/python/apache_beam/io/gcp/dicomclient.py
new file mode 100644
index 0000000..e38a310
--- /dev/null
+++ b/sdks/python/apache_beam/io/gcp/dicomclient.py
@@ -0,0 +1,128 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# pytype: skip-file
+
+from __future__ import absolute_import
+
+from google.auth import default
+from google.auth.transport import requests
+
+
+class DicomApiHttpClient:
+ """DICOM api client that talk to api via http request"""
+ healthcare_base_url = "https://healthcare.googleapis.com/v1"
+ session = None
+
+ def get_session(self, credential):
+ if self.session:
+ return self.session
+
+ # if the credential is not provided, use the default credential.
+ if not credential:
+ credential, _ = default()
+ new_seesion = requests.AuthorizedSession(credential)
+ self.session = new_seesion
+ return new_seesion
+
+ def qido_search(
+ self,
+ project_id,
+ region,
+ dataset_id,
+ dicom_store_id,
+ search_type,
+ params=None,
+ credential=None):
+ """function for searching a DICOM store"""
+
+ # sending request to the REST healthcare api.
+ api_endpoint = "{}/projects/{}/locations/{}".format(
+ self.healthcare_base_url, project_id, region)
+
+ # base of dicomweb path.
+ dicomweb_path = "{}/datasets/{}/dicomStores/{}/dicomWeb/{}".format(
+ api_endpoint, dataset_id, dicom_store_id, search_type)
+
+ # Make an authenticated API request
+ session = self.get_session(credential)
+ headers = {"Content-Type": "application/dicom+json; charset=utf-8"}
+ page_size = 500
+
+ if params and 'limit' in params:
+ page_size = params['limit']
+ elif params:
+ params['limit'] = page_size
+ else:
+ params = {'limit': page_size}
+
+ offset = 0
+ output = []
+ # iterate to get all the results
+ while True:
+ params['offset'] = offset
+ response = session.get(dicomweb_path, headers=headers, params=params)
+ response.raise_for_status()
+ status = response.status_code
+ if status != 200:
+ if offset == 0:
+ return [], status
+ params['offset'] = offset - 1
+ params['limit'] = 1
+ response = session.get(dicomweb_path, headers=headers, params=params)
+ response.raise_for_status()
+ check_status = response.status_code
+ if check_status == 200:
+ # if the number of results equals to page size
+ return output, check_status
+ else:
+ # something wrong with the request or server
+ return [], status
+ results = response.json()
+ output += results
+ if len(results) < page_size:
+ # got all the results, return
+ break
+ offset += len(results)
+
+ return output, status
+
+ def dicomweb_store_instance(
+ self,
+ project_id,
+ region,
+ dataset_id,
+ dicom_store_id,
+ dcm_file,
+ credential=None):
+ """function for storing an instance."""
+
+ api_endpoint = "{}/projects/{}/locations/{}".format(
+ self.healthcare_base_url, project_id, region)
+
+ dicomweb_path = "{}/datasets/{}/dicomStores/{}/dicomWeb/studies".format(
+ api_endpoint, dataset_id, dicom_store_id)
+
+ # Make an authenticated API request
+ session = self.get_session(credential)
+ content_type = "application/dicom"
+ headers = {"Content-Type": content_type}
+
+ response = session.post(dicomweb_path, data=dcm_file, headers=headers)
+ response.raise_for_status()
+
+ return None, response.status_code
diff --git a/sdks/python/apache_beam/io/gcp/dicomio.py b/sdks/python/apache_beam/io/gcp/dicomio.py
new file mode 100644
index 0000000..e33d99d
--- /dev/null
+++ b/sdks/python/apache_beam/io/gcp/dicomio.py
@@ -0,0 +1,579 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""DICOM IO connector
+This module implements several tools to facilitate the interaction between
+a Google Cloud Healthcare DICOM store and a Beam pipeline.
+
+For more details on DICOM store and API:
+https://cloud.google.com/healthcare/docs/how-tos/dicom
+
+The DICOM IO connector can be used to search metadata or write DICOM files
+to DICOM store.
+
+When used together with Google Pubsub message connector, the
+`FormatToQido` PTransform implemented in this module can be used
+to convert Pubsub messages to search requests.
+
+Since Traceability is crucial for healthcare
+API users, every input or error message will be recorded in the output of
+the DICOM IO connector. As a result, every PTransform in this module will
+return a PCollection of dict that encodes results and detailed error messages.
+
+Search instance's metadata (QIDO request)
+===================================================
+DicomSearch() wraps the QIDO request client and supports 3 levels of search.
+Users should specify the level by setting the 'search_type' entry in the input
+dict. They can also refine the search by adding tags to filter the results using
+the 'params' entry. Here is a sample usage:
+
+ with Pipeline() as p:
+ input_dict = p | beam.Create(
+ [{'project_id': 'abc123', 'type': 'instances',...},
+ {'project_id': 'dicom_go', 'type': 'series',...}])
+
+ results = input_dict | io.gcp.DicomSearch()
+ results | 'print successful search' >> beam.Map(
+ lambda x: print(x['result'] if x['success'] else None))
+
+ results | 'print failed search' >> beam.Map(
+ lambda x: print(x['result'] if not x['success'] else None))
+
+In the example above, successful qido search results and error messages for
+failed requests are printed. When used in real life, user can choose to filter
+those data and output them to wherever they want.
+
+Convert DICOM Pubsub message to Qido search request
+===================================================
+Healthcare API users might read messages from Pubsub to monitor the store
+operations (e.g. new file) in a DICOM storage. Pubsub message encode
+DICOM as a web store path as well as instance ids. If users are interested in
+getting new instance's metadata, they can use the `FormatToQido` transform
+to convert the message into Qido Search dict then use the `DicomSearch`
+transform. Here is a sample usage:
+
+ pipeline_options = PipelineOptions()
+ pipeline_options.view_as(StandardOptions).streaming = True
+ p = beam.Pipeline(options=pipeline_options)
+ pubsub = p | beam.io.ReadStringFromPubsub(subscription='a_dicom_store')
+ results = pubsub | FormatToQido()
+ success = results | 'filter message' >> beam.Filter(lambda x: x['success'])
+ qido_dict = success | 'get qido request' >> beam.Map(lambda x: x['result'])
+ metadata = qido_dict | DicomSearch()
+
+In the example above, the pipeline is listening to a pubsub topic and waiting
+for messages from DICOM API. When a new DICOM file comes into the storage, the
+pipeline will receive a pubsub message, convert it to a Qido request dict and
+feed it to DicomSearch() PTransform. As a result, users can get the metadata for
+every new DICOM file. Note that not every pubsub message received is from DICOM
+API, so we to filter the results first.
+
+Store a DICOM file in a DICOM storage
+===================================================
+UploadToDicomStore() wraps store request API and users can use it to send a
+DICOM file to a DICOM store. It supports two types of input: 1.file data in
+byte[] 2.fileio object. Users should set the 'input_type' when initialzing
+this PTransform. Here are the examples:
+
+ with Pipeline() as p:
+ input_dict = {'project_id': 'abc123', 'type': 'instances',...}
+ path = "gcs://bucketname/something/a.dcm"
+ match = p | fileio.MatchFiles(path)
+ fileio_obj = match | fileio.ReadAll()
+ results = fileio_obj | UploadToDicomStore(input_dict, 'fileio')
+
+ with Pipeline() as p:
+ input_dict = {'project_id': 'abc123', 'type': 'instances',...}
+ f = open("abc.dcm", "rb")
+ dcm_file = f.read()
+ byte_file = p | 'create byte file' >> beam.Create([dcm_file])
+ results = byte_file | UploadToDicomStore(input_dict, 'bytes')
+
+The first example uses a PCollection of fileio objects as input.
+UploadToDicomStore will read DICOM files from the objects and send them
+to a DICOM storage.
+The second example uses a PCollection of byte[] as input. UploadToDicomStore
+will directly send those DICOM files to a DICOM storage.
+Users can also get the operation results in the output PCollection if they want
+to handle the failed store requests.
+"""
+
+# pytype: skip-file
+from __future__ import absolute_import
+
+from concurrent.futures import ThreadPoolExecutor
+from concurrent.futures import as_completed
+
+import apache_beam as beam
+from apache_beam.io.gcp.dicomclient import DicomApiHttpClient
+from apache_beam.transforms import PTransform
+
+
+class DicomSearch(PTransform):
+ """A PTransform used for retrieving DICOM instance metadata from Google
+ Cloud DICOM store. It takes a PCollection of dicts as input and return
+ a PCollection of dict as results:
+ INPUT:
+ The input dict represents DICOM web path parameters, which has the following
+ string keys and values:
+ {
+ 'project_id': str,
+ 'region': str,
+ 'dataset_id': str,
+ 'dicom_store_id': str,
+ 'search_type': str,
+ 'params': dict(str,str) (Optional),
+ }
+
+ Key-value pairs:
+ project_id: Id of the project in which the DICOM store is
+ located. (Required)
+ region: Region where the DICOM store resides. (Required)
+ dataset_id: Id of the dataset where DICOM store belongs to. (Required)
+ dicom_store_id: Id of the dicom store. (Required)
+ search_type: Which type of search it is, could only be one of the three
+ values: 'instances', 'series', or 'studies'. (Required)
+ params: A dict of str:str pairs used to refine QIDO search. (Optional)
+ Supported tags in three categories:
+ 1.Studies:
+ * StudyInstanceUID,
+ * PatientName,
+ * PatientID,
+ * AccessionNumber,
+ * ReferringPhysicianName,
+ * StudyDate,
+ 2.Series: all study level search terms and
+ * SeriesInstanceUID,
+ * Modality,
+ 3.Instances: all study/series level search terms and
+ * SOPInstanceUID,
+
+ e.g. {"StudyInstanceUID":"1","SeriesInstanceUID":"2"}
+
+ OUTPUT:
+ The output dict wraps results as well as error messages:
+ {
+ 'result': a list of dicts in JSON style.
+ 'success': boolean value telling whether the operation is successful.
+ 'input': detail ids and dicomweb path for this retrieval.
+ 'status': status code from the server, used as error message.
+ }
+
+ """
+ def __init__(
+ self, buffer_size=8, max_workers=5, client=None, credential=None):
+ """Initializes DicomSearch.
+ Args:
+ buffer_size: # type: Int. Size of the request buffer.
+ max_workers: # type: Int. Maximum number of threads a worker can
+ create. If it is set to one, all the request will be processed
+ sequentially in a worker.
+ client: # type: object. If it is specified, all the Api calls will
+ made by this client instead of the default one (DicomApiHttpClient).
+ credential: # type: Google credential object, if it is specified, the
+ Http client will use it to create sessions instead of the default.
+ """
+ self.buffer_size = buffer_size
+ self.max_workers = max_workers
+ self.client = client or DicomApiHttpClient()
+ self.credential = credential
+
+ def expand(self, pcoll):
+ return pcoll | beam.ParDo(
+ _QidoReadFn(
+ self.buffer_size, self.max_workers, self.client, self.credential))
+
+
+class _QidoReadFn(beam.DoFn):
+ """A DoFn for executing every qido query request."""
+ def __init__(self, buffer_size, max_workers, client, credential=None):
+ self.buffer_size = buffer_size
+ self.max_workers = max_workers
+ self.client = client
+ self.credential = credential
+
+ def start_bundle(self):
+ self.buffer = []
+
+ def finish_bundle(self):
+ for item in self._flush():
+ yield item
+
+ def validate_element(self, element):
+ # Check if all required keys present.
+ required_keys = [
+ 'project_id', 'region', 'dataset_id', 'dicom_store_id', 'search_type'
+ ]
+
+ for key in required_keys:
+ if key not in element:
+ error_message = 'Must have %s in the dict.' % (key)
+ return False, error_message
+
+ # Check if return type is correct.
+ if element['search_type'] in ['instances', "studies", "series"]:
+ return True, None
+ else:
+ error_message = (
+ 'Search type can only be "studies", '
+ '"instances" or "series"')
+ return False, error_message
+
+ def process(
+ self,
+ element,
+ window=beam.DoFn.WindowParam,
+ timestamp=beam.DoFn.TimestampParam):
+ # Check if the element is valid
+ valid, error_message = self.validate_element(element)
+
+ if valid:
+ self.buffer.append((element, window, timestamp))
+ if len(self.buffer) >= self.buffer_size:
+ for item in self._flush():
+ yield item
+ else:
+ # Return this when the input dict dose not meet the requirements
+ out = {}
+ out['result'] = []
+ out['status'] = error_message
+ out['input'] = element
+ out['success'] = False
+ yield out
+
+ def make_request(self, element):
+ # Sending Qido request to DICOM Api
+ project_id = element['project_id']
+ region = element['region']
+ dataset_id = element['dataset_id']
+ dicom_store_id = element['dicom_store_id']
+ search_type = element['search_type']
+ params = element['params'] if 'params' in element else None
+
+ # Call qido search http client
+ result, status_code = self.client.qido_search(
+ project_id, region, dataset_id, dicom_store_id,
+ search_type, params, self.credential
+ )
+
+ out = {}
+ out['result'] = result
+ out['status'] = status_code
+ out['input'] = element
+ out['success'] = (status_code == 200)
+ return out
+
+ def process_buffer_element(self, buffer_element):
+ # Thread job runner - each thread makes a Qido search request
+ value = self.make_request(buffer_element[0])
+ windows = [buffer_element[1]]
+ timestamp = buffer_element[2]
+ return beam.utils.windowed_value.WindowedValue(
+ value=value, timestamp=timestamp, windows=windows)
+
+ def _flush(self):
+ # Create thread pool executor and process the buffered elements in paralllel
+ executor = ThreadPoolExecutor(max_workers=self.max_workers)
+ futures = [
+ executor.submit(self.process_buffer_element, ele) for ele in self.buffer
+ ]
+ self.buffer = []
+ for f in as_completed(futures):
+ yield f.result()
+
+
+class FormatToQido(PTransform):
+ """A PTransform for converting pubsub messages into search input dict.
+ Takes PCollection of string as input and returns a PCollection of dict as
+ results. Note that some pubsub messages may not be from DICOM API, which
+ will be recorded as failed conversions.
+ INPUT:
+ The input are normally strings from Pubsub topic:
+ "projects/PROJECT_ID/locations/LOCATION/datasets/DATASET_ID/
+ dicomStores/DICOM_STORE_ID/dicomWeb/studies/STUDY_UID/
+ series/SERIES_UID/instances/INSTANCE_UID"
+
+ OUTPUT:
+ The output dict encodes results as well as error messages:
+ {
+ 'result': a dict representing instance level qido search request.
+ 'success': boolean value telling whether the conversion is successful.
+ 'input': input pubsub message string.
+ }
+
+ """
+ def __init__(self, credential=None):
+ """Initializes FormatToQido.
+ Args:
+ credential: # type: Google credential object, if it is specified, the
+ Http client will use it instead of the default one.
+ """
+ self.credential = credential
+
+ def expand(self, pcoll):
+ return pcoll | beam.ParDo(_ConvertStringToQido())
+
+
+class _ConvertStringToQido(beam.DoFn):
+ """A DoFn for converting pubsub string to qido search parameters."""
+ def process(self, element):
+ # Some constants for DICOM pubsub message
+ NUM_PUBSUB_STR_ENTRIES = 15
+ NUM_DICOM_WEBPATH_PARAMETERS = 5
+ NUM_TOTAL_PARAMETERS = 8
+ INDEX_PROJECT_ID = 1
+ INDEX_REGION = 3
+ INDEX_DATASET_ID = 5
+ INDEX_DICOMSTORE_ID = 7
+ INDEX_STUDY_ID = 10
+ INDEX_SERIE_ID = 12
+ INDEX_INSTANCE_ID = 14
+
+ entries = element.split('/')
+
+ # Output dict with error message, used when
+ # receiving invalid pubsub string.
+ error_dict = {}
+ error_dict['result'] = {}
+ error_dict['input'] = element
+ error_dict['success'] = False
+
+ if len(entries) != NUM_PUBSUB_STR_ENTRIES:
+ return [error_dict]
+
+ required_keys = [
+ 'projects',
+ 'locations',
+ 'datasets',
+ 'dicomStores',
+ 'dicomWeb',
+ 'studies',
+ 'series',
+ 'instances'
+ ]
+
+ # Check if the required keys present and
+ # the positions of those keys are correct
+ for i in range(NUM_DICOM_WEBPATH_PARAMETERS):
+ if required_keys[i] != entries[i * 2]:
+ return [error_dict]
+ for i in range(NUM_DICOM_WEBPATH_PARAMETERS, NUM_TOTAL_PARAMETERS):
+ if required_keys[i] != entries[i * 2 - 1]:
+ return [error_dict]
+
+ # Compose dicom webpath parameters for qido search
+ qido_dict = {}
+ qido_dict['project_id'] = entries[INDEX_PROJECT_ID]
+ qido_dict['region'] = entries[INDEX_REGION]
+ qido_dict['dataset_id'] = entries[INDEX_DATASET_ID]
+ qido_dict['dicom_store_id'] = entries[INDEX_DICOMSTORE_ID]
+ qido_dict['search_type'] = 'instances'
+
+ # Compose instance level params for qido search
+ params = {}
+ params['StudyInstanceUID'] = entries[INDEX_STUDY_ID]
+ params['SeriesInstanceUID'] = entries[INDEX_SERIE_ID]
+ params['SOPInstanceUID'] = entries[INDEX_INSTANCE_ID]
+ qido_dict['params'] = params
+
+ out = {}
+ out['result'] = qido_dict
+ out['input'] = element
+ out['success'] = True
+
+ return [out]
+
+
+class UploadToDicomStore(PTransform):
+ """A PTransform for storing instances to a DICOM store.
+ Takes PCollection of byte[] as input and return a PCollection of dict as
+ results. The inputs are normally DICOM file in bytes or str filename.
+ INPUT:
+ This PTransform supports two types of input:
+ 1. Byte[]: representing dicom file.
+ 2. Fileio object: stream file object.
+
+ OUTPUT:
+ The output dict encodes status as well as error messages:
+ {
+ 'success': boolean value telling whether the store is successful.
+ 'input': undeliverable data. Exactly the same as the input,
+ only set if the operation is failed.
+ 'status': status code from the server, used as error messages.
+ }
+
+ """
+ def __init__(
+ self,
+ destination_dict,
+ input_type,
+ buffer_size=8,
+ max_workers=5,
+ client=None,
+ credential=None):
+ """Initializes UploadToDicomStore.
+ Args:
+ destination_dict: # type: python dict, encodes DICOM endpoint information:
+ {
+ 'project_id': str,
+ 'region': str,
+ 'dataset_id': str,
+ 'dicom_store_id': str,
+ }
+
+ Key-value pairs:
+ * project_id: Id of the project in which DICOM store locates. (Required)
+ * region: Region where the DICOM store resides. (Required)
+ * dataset_id: Id of the dataset where DICOM store belongs to. (Required)
+ * dicom_store_id: Id of the dicom store. (Required)
+
+ input_type: # type: string, could only be 'bytes' or 'fileio'
+ buffer_size: # type: Int. Size of the request buffer.
+ max_workers: # type: Int. Maximum number of threads a worker can
+ create. If it is set to one, all the request will be processed
+ sequentially in a worker.
+ client: # type: object. If it is specified, all the Api calls will
+ made by this client instead of the default one (DicomApiHttpClient).
+ credential: # type: Google credential object, if it is specified, the
+ Http client will use it instead of the default one.
+ """
+ self.destination_dict = destination_dict
+ # input_type pre-check
+ if input_type not in ['bytes', 'fileio']:
+ raise ValueError("input_type could only be 'bytes' or 'fileio'")
+ self.input_type = input_type
+ self.buffer_size = buffer_size
+ self.max_workers = max_workers
+ self.client = client
+ self.credential = credential
+
+ def expand(self, pcoll):
+ return pcoll | beam.ParDo(
+ _StoreInstance(
+ self.destination_dict,
+ self.input_type,
+ self.buffer_size,
+ self.max_workers,
+ self.client,
+ self.credential))
+
+
+class _StoreInstance(beam.DoFn):
+ """A DoFn read or fetch dicom files then push it to a dicom store."""
+ def __init__(
+ self,
+ destination_dict,
+ input_type,
+ buffer_size,
+ max_workers,
+ client,
+ credential=None):
+ # pre-check destination dict
+ required_keys = ['project_id', 'region', 'dataset_id', 'dicom_store_id']
+ for key in required_keys:
+ if key not in destination_dict:
+ raise ValueError('Must have %s in the dict.' % (key))
+ self.destination_dict = destination_dict
+ self.input_type = input_type
+ self.buffer_size = buffer_size
+ self.max_workers = max_workers
+ self.client = client
+ self.credential = credential
+
+ def start_bundle(self):
+ self.buffer = []
+
+ def finish_bundle(self):
+ for item in self._flush():
+ yield item
+
+ def process(
+ self,
+ element,
+ window=beam.DoFn.WindowParam,
+ timestamp=beam.DoFn.TimestampParam):
+ self.buffer.append((element, window, timestamp))
+ if len(self.buffer) >= self.buffer_size:
+ for item in self._flush():
+ yield item
+
+ def make_request(self, dicom_file):
+ # Send file to DICOM store and records the results.
+ project_id = self.destination_dict['project_id']
+ region = self.destination_dict['region']
+ dataset_id = self.destination_dict['dataset_id']
+ dicom_store_id = self.destination_dict['dicom_store_id']
+
+ # Feed the dicom file into store client
+ if self.client:
+ _, status_code = self.client.dicomweb_store_instance(
+ project_id, region, dataset_id, dicom_store_id, dicom_file,
+ self.credential
+ )
+ else:
+ _, status_code = DicomApiHttpClient().dicomweb_store_instance(
+ project_id, region, dataset_id, dicom_store_id, dicom_file,
+ self.credential
+ )
+
+ out = {}
+ out['status'] = status_code
+ out['success'] = (status_code == 200)
+ return out
+
+ def read_dicom_file(self, buffer_element):
+ # Read the file based on different input. If the read fails ,return
+ # an error dict which records input and error messages.
+ try:
+ if self.input_type == 'fileio':
+ f = buffer_element.open()
+ data = f.read()
+ f.close()
+ return True, data
+ else:
+ return True, buffer_element
+ except Exception as error_message:
+ error_out = {}
+ error_out['status'] = error_message
+ error_out['success'] = False
+ return False, error_out
+
+ def process_buffer_element(self, buffer_element):
+ # Thread job runner - each thread stores a DICOM file
+ success, read_result = self.read_dicom_file(buffer_element[0])
+ windows = [buffer_element[1]]
+ timestamp = buffer_element[2]
+ value = None
+ if success:
+ value = self.make_request(read_result)
+ else:
+ value = read_result
+ # save the undeliverable data
+ if not value['success']:
+ value['input'] = buffer_element[0]
+ return beam.utils.windowed_value.WindowedValue(
+ value=value, timestamp=timestamp, windows=windows)
+
+ def _flush(self):
+ # Create thread pool executor and process the buffered elements in paralllel
+ executor = ThreadPoolExecutor(max_workers=self.max_workers)
+ futures = [
+ executor.submit(self.process_buffer_element, ele) for ele in self.buffer
+ ]
+ self.buffer = []
+ for f in as_completed(futures):
+ yield f.result()
diff --git a/sdks/python/apache_beam/io/gcp/dicomio_test.py b/sdks/python/apache_beam/io/gcp/dicomio_test.py
new file mode 100644
index 0000000..2594e45
--- /dev/null
+++ b/sdks/python/apache_beam/io/gcp/dicomio_test.py
@@ -0,0 +1,468 @@
+# coding=utf-8
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""Unit tests for Dicom IO connectors."""
+
+# pytype: skip-file
+
+from __future__ import absolute_import
+
+import json
+import os
+import unittest
+
+# patches unittest.TestCase to be python3 compatible
+import future.tests.base # pylint: disable=unused-import
+from mock import patch
+
+import apache_beam as beam
+from apache_beam.io import fileio
+from apache_beam.io.filebasedsink_test import _TestCaseWithTempDirCleanUp
+from apache_beam.io.filesystems import FileSystems
+from apache_beam.testing.test_pipeline import TestPipeline
+from apache_beam.testing.util import assert_that
+from apache_beam.testing.util import equal_to
+
+# Protect against environments where gcp library is not available.
+# pylint: disable=wrong-import-order, wrong-import-position
+try:
+ from apache_beam.io.gcp.dicomio import DicomSearch
+ from apache_beam.io.gcp.dicomio import FormatToQido
+ from apache_beam.io.gcp.dicomio import UploadToDicomStore
+except ImportError:
+ DicomSearch = None # type: ignore
+# pylint: enable=wrong-import-order, wrong-import-position
+
+
+class FakeHttpClient():
+ # a fake http client that talks directly to a in-memory dicom store
+ def __init__(self):
+ # set 5 fake dicom instances
+ dicom_metadata = []
+ dicom_metadata.append({
+ 'PatientName': 'Albert', 'Age': 21, 'TestResult': 'Positive'
+ })
+ dicom_metadata.append({
+ 'PatientName': 'Albert', 'Age': 21, 'TestResult': 'Negative'
+ })
+ dicom_metadata.append({
+ 'PatientName': 'Brian', 'Age': 20, 'TestResult': 'Positive'
+ })
+ dicom_metadata.append({
+ 'PatientName': 'Colin', 'Age': 25, 'TestResult': 'Negative'
+ })
+ dicom_metadata.append({
+ 'PatientName': 'Daniel', 'Age': 22, 'TestResult': 'Negative'
+ })
+ dicom_metadata.append({
+ 'PatientName': 'Eric', 'Age': 50, 'TestResult': 'Negative'
+ })
+ self.dicom_metadata = dicom_metadata
+ # ids for this dicom sotre
+ self.project_id = 'test_project'
+ self.region = 'test_region'
+ self.dataset_id = 'test_dataset_id'
+ self.dicom_store_id = 'test_dicom_store_id'
+
+ def qido_search(
+ self,
+ project_id,
+ region,
+ dataset_id,
+ dicom_store_id,
+ search_type,
+ params=None,
+ credential=None):
+ # qido search function for this fake client
+ if project_id != self.project_id or region != self.region or \
+ dataset_id != self.dataset_id or dicom_store_id != self.dicom_store_id:
+ return [], 204
+ # only supports all instance search in this client
+ if not params:
+ return self.dicom_metadata, 200
+ # only supports patient name filter in this client
+ patient_name = params['PatientName']
+ out = []
+ for meta in self.dicom_metadata:
+ if meta['PatientName'] == patient_name:
+ out.append(meta)
+ return out, 200
+
+ def dicomweb_store_instance(
+ self,
+ project_id,
+ region,
+ dataset_id,
+ dicom_store_id,
+ dcm_file,
+ credential=None):
+ if project_id != self.project_id or region != self.region or \
+ dataset_id != self.dataset_id or dicom_store_id != self.dicom_store_id:
+ return [], 204
+ # convert the bytes file back to dict
+ string_array = dcm_file.decode('utf-8')
+ metadata_dict = json.loads(string_array)
+ self.dicom_metadata.append(metadata_dict)
+ return None, 200
+
+
+@unittest.skipIf(DicomSearch is None, 'GCP dependencies are not installed')
+class TestFormatToQido(unittest.TestCase):
+ valid_pubsub_string = (
+ "projects/PROJECT_ID/locations/LOCATION/datasets"
+ "/DATASET_ID/dicomStores/DICOM_STORE_ID/dicomWeb/"
+ "studies/STUDY_UID/series/SERIES_UID/instances/INSTANCE_UID")
+ expected_valid_pubsub_dict = {
+ 'result': {
+ 'project_id': 'PROJECT_ID',
+ 'region': 'LOCATION',
+ 'dataset_id': 'DATASET_ID',
+ 'dicom_store_id': 'DICOM_STORE_ID',
+ 'search_type': 'instances',
+ 'params': {
+ 'StudyInstanceUID': 'STUDY_UID',
+ 'SeriesInstanceUID': 'SERIES_UID',
+ 'SOPInstanceUID': 'INSTANCE_UID'
+ }
+ },
+ 'input': valid_pubsub_string,
+ 'success': True
+ }
+ invalid_pubsub_string = "this is not a valid pubsub message"
+ expected_invalid_pubsub_dict = {
+ 'result': {},
+ 'input': 'this is not a valid pubsub message',
+ 'success': False
+ }
+
+ def test_normal_convert(self):
+ with TestPipeline() as p:
+ convert_result = (
+ p
+ | beam.Create([self.valid_pubsub_string])
+ | FormatToQido())
+ assert_that(convert_result, equal_to([self.expected_valid_pubsub_dict]))
+
+ def test_failed_convert(self):
+ with TestPipeline() as p:
+ convert_result = (
+ p
+ | beam.Create([self.invalid_pubsub_string])
+ | FormatToQido())
+ assert_that(convert_result, equal_to([self.expected_invalid_pubsub_dict]))
+
+
+@unittest.skipIf(DicomSearch is None, 'GCP dependencies are not installed')
+class TestDicomSearch(unittest.TestCase):
+ @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+ def test_successful_search(self, FakeClient):
+ input_dict = {}
+ input_dict['project_id'] = "test_project"
+ input_dict['region'] = "test_region"
+ input_dict['dataset_id'] = "test_dataset_id"
+ input_dict['dicom_store_id'] = "test_dicom_store_id"
+ input_dict['search_type'] = "instances"
+
+ fc = FakeHttpClient()
+ FakeClient.return_value = fc
+
+ expected_dict = {}
+ expected_dict['result'] = fc.dicom_metadata
+ expected_dict['status'] = 200
+ expected_dict['input'] = input_dict
+ expected_dict['success'] = True
+
+ with TestPipeline() as p:
+ results = (p | beam.Create([input_dict]) | DicomSearch())
+ assert_that(results, equal_to([expected_dict]))
+
+ @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+ def test_Qido_search_small_buffer_flush(self, FakeClient):
+ input_dict = {}
+ input_dict['project_id'] = "test_project"
+ input_dict['region'] = "test_region"
+ input_dict['dataset_id'] = "test_dataset_id"
+ input_dict['dicom_store_id'] = "test_dicom_store_id"
+ input_dict['search_type'] = "instances"
+
+ fc = FakeHttpClient()
+ FakeClient.return_value = fc
+
+ expected_dict = {}
+ expected_dict['result'] = fc.dicom_metadata
+ expected_dict['status'] = 200
+ expected_dict['input'] = input_dict
+ expected_dict['success'] = True
+
+ with TestPipeline() as p:
+ results = (p | beam.Create([input_dict] * 5) | DicomSearch(buffer_size=1))
+ assert_that(results, equal_to([expected_dict] * 5))
+
+ @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+ def test_param_dict_passing(self, FakeClient):
+ input_dict = {}
+ input_dict = {}
+ input_dict['project_id'] = "test_project"
+ input_dict['region'] = "test_region"
+ input_dict['dataset_id'] = "test_dataset_id"
+ input_dict['dicom_store_id'] = "test_dicom_store_id"
+ input_dict['search_type'] = "instances"
+ input_dict['params'] = {'PatientName': 'Brian'}
+
+ expected_dict = {}
+ expected_dict['result'] = [{
+ 'PatientName': 'Brian', 'Age': 20, 'TestResult': 'Positive'
+ }]
+ expected_dict['status'] = 200
+ expected_dict['input'] = input_dict
+ expected_dict['success'] = True
+
+ fc = FakeHttpClient()
+ FakeClient.return_value = fc
+ with TestPipeline() as p:
+ results = (p | beam.Create([input_dict]) | DicomSearch())
+ assert_that(results, equal_to([expected_dict]))
+
+ @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+ def test_wrong_input_type(self, FakeClient):
+ input_dict = {}
+ input_dict['project_id'] = "test_project"
+ input_dict['region'] = "test_region"
+ input_dict['dataset_id'] = "test_dataset_id"
+ input_dict['dicom_store_id'] = "test_dicom_store_id"
+ input_dict['search_type'] = "not exist type"
+
+ expected_invalid_dict = {}
+ expected_invalid_dict['result'] = []
+ expected_invalid_dict[
+ 'status'] = 'Search type can only be "studies", "instances" or "series"'
+ expected_invalid_dict['input'] = input_dict
+ expected_invalid_dict['success'] = False
+
+ fc = FakeHttpClient()
+ FakeClient.return_value = fc
+ with TestPipeline() as p:
+ results = (p | beam.Create([input_dict]) | DicomSearch())
+ assert_that(results, equal_to([expected_invalid_dict]))
+
+ @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+ def test_missing_parameters(self, FakeClient):
+ input_dict = {}
+ input_dict['project_id'] = "test_project"
+ input_dict['region'] = "test_region"
+
+ expected_invalid_dict = {}
+ expected_invalid_dict['result'] = []
+ expected_invalid_dict['status'] = 'Must have dataset_id in the dict.'
+ expected_invalid_dict['input'] = input_dict
+ expected_invalid_dict['success'] = False
+
+ fc = FakeHttpClient()
+ FakeClient.return_value = fc
+ with TestPipeline() as p:
+ results = (p | beam.Create([input_dict]) | DicomSearch())
+ assert_that(results, equal_to([expected_invalid_dict]))
+
+ @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+ def test_client_search_notfound(self, FakeClient):
+ input_dict = {}
+ # search instances in a not exist store
+ input_dict['project_id'] = "wrong_project"
+ input_dict['region'] = "wrong_region"
+ input_dict['dataset_id'] = "wrong_dataset_id"
+ input_dict['dicom_store_id'] = "wrong_dicom_store_id"
+ input_dict['search_type'] = "instances"
+
+ expected_invalid_dict = {}
+ expected_invalid_dict['result'] = []
+ expected_invalid_dict['status'] = 204
+ expected_invalid_dict['input'] = input_dict
+ expected_invalid_dict['success'] = False
+
+ fc = FakeHttpClient()
+ FakeClient.return_value = fc
+ with TestPipeline() as p:
+ results = (p | beam.Create([input_dict]) | DicomSearch())
+ assert_that(results, equal_to([expected_invalid_dict]))
+
+
+@unittest.skipIf(DicomSearch is None, 'GCP dependencies are not installed')
+class TestDicomStoreInstance(_TestCaseWithTempDirCleanUp):
+ @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+ def test_store_byte_file(self, FakeClient):
+ input_dict = {}
+ input_dict['project_id'] = "test_project"
+ input_dict['region'] = "test_region"
+ input_dict['dataset_id'] = "test_dataset_id"
+ input_dict['dicom_store_id'] = "test_dicom_store_id"
+
+ fc = FakeHttpClient()
+ FakeClient.return_value = fc
+
+ dict_input = {'PatientName': 'George', 'Age': 23, 'TestResult': 'Negative'}
+ str_input = json.dumps(dict_input)
+ bytes_input = bytes(str_input.encode("utf-8"))
+ with TestPipeline() as p:
+ results = (
+ p
+ | beam.Create([bytes_input])
+ | UploadToDicomStore(input_dict, 'bytes')
+ | beam.Map(lambda x: x['success']))
+ assert_that(results, equal_to([True]))
+ self.assertTrue(dict_input in fc.dicom_metadata)
+
+ @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+ def test_store_byte_file_small_buffer_flush(self, FakeClient):
+ input_dict = {}
+ input_dict['project_id'] = "test_project"
+ input_dict['region'] = "test_region"
+ input_dict['dataset_id'] = "test_dataset_id"
+ input_dict['dicom_store_id'] = "test_dicom_store_id"
+
+ fc = FakeHttpClient()
+ FakeClient.return_value = fc
+
+ dict_input_1 = {
+ 'PatientName': 'George', 'Age': 23, 'TestResult': 'Negative'
+ }
+ str_input_1 = json.dumps(dict_input_1)
+ bytes_input_1 = bytes(str_input_1.encode("utf-8"))
+ dict_input_2 = {'PatientName': 'Peter', 'Age': 54, 'TestResult': 'Positive'}
+ str_input_2 = json.dumps(dict_input_2)
+ bytes_input_2 = bytes(str_input_2.encode("utf-8"))
+ dict_input_3 = {'PatientName': 'Zen', 'Age': 27, 'TestResult': 'Negative'}
+ str_input_3 = json.dumps(dict_input_3)
+ bytes_input_3 = bytes(str_input_3.encode("utf-8"))
+ with TestPipeline() as p:
+ results = (
+ p
+ | beam.Create([bytes_input_1, bytes_input_2, bytes_input_3])
+ | UploadToDicomStore(input_dict, 'bytes', buffer_size=1)
+ | beam.Map(lambda x: x['success']))
+ assert_that(results, equal_to([True] * 3))
+ self.assertTrue(dict_input_1 in fc.dicom_metadata)
+ self.assertTrue(dict_input_2 in fc.dicom_metadata)
+ self.assertTrue(dict_input_3 in fc.dicom_metadata)
+
+ @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+ def test_store_fileio_file(self, FakeClient):
+ input_dict = {}
+ input_dict['project_id'] = "test_project"
+ input_dict['region'] = "test_region"
+ input_dict['dataset_id'] = "test_dataset_id"
+ input_dict['dicom_store_id'] = "test_dicom_store_id"
+
+ fc = FakeHttpClient()
+ FakeClient.return_value = fc
+
+ dict_input = {'PatientName': 'George', 'Age': 23, 'TestResult': 'Negative'}
+ str_input = json.dumps(dict_input)
+ temp_dir = '%s%s' % (self._new_tempdir(), os.sep)
+ self._create_temp_file(dir=temp_dir, content=str_input)
+
+ with TestPipeline() as p:
+ results = (
+ p
+ | beam.Create([FileSystems.join(temp_dir, '*')])
+ | fileio.MatchAll()
+ | fileio.ReadMatches()
+ | UploadToDicomStore(input_dict, 'fileio')
+ | beam.Map(lambda x: x['success']))
+ assert_that(results, equal_to([True]))
+ self.assertTrue(dict_input in fc.dicom_metadata)
+
+ @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+ def test_store_fileio_file_small_buffer_flush(self, FakeClient):
+ input_dict = {}
+ input_dict['project_id'] = "test_project"
+ input_dict['region'] = "test_region"
+ input_dict['dataset_id'] = "test_dataset_id"
+ input_dict['dicom_store_id'] = "test_dicom_store_id"
+
+ fc = FakeHttpClient()
+ FakeClient.return_value = fc
+
+ temp_dir = '%s%s' % (self._new_tempdir(), os.sep)
+ dict_input_1 = {
+ 'PatientName': 'George', 'Age': 23, 'TestResult': 'Negative'
+ }
+ str_input_1 = json.dumps(dict_input_1)
+ self._create_temp_file(dir=temp_dir, content=str_input_1)
+ dict_input_2 = {'PatientName': 'Peter', 'Age': 54, 'TestResult': 'Positive'}
+ str_input_2 = json.dumps(dict_input_2)
+ self._create_temp_file(dir=temp_dir, content=str_input_2)
+ dict_input_3 = {'PatientName': 'Zen', 'Age': 27, 'TestResult': 'Negative'}
+ str_input_3 = json.dumps(dict_input_3)
+ self._create_temp_file(dir=temp_dir, content=str_input_3)
+
+ with TestPipeline() as p:
+ results = (
+ p
+ | beam.Create([FileSystems.join(temp_dir, '*')])
+ | fileio.MatchAll()
+ | fileio.ReadMatches()
+ | UploadToDicomStore(input_dict, 'fileio', buffer_size=1)
+ | beam.Map(lambda x: x['success']))
+ assert_that(results, equal_to([True] * 3))
+ self.assertTrue(dict_input_1 in fc.dicom_metadata)
+ self.assertTrue(dict_input_2 in fc.dicom_metadata)
+ self.assertTrue(dict_input_3 in fc.dicom_metadata)
+
+ @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+ def test_destination_notfound(self, FakeClient):
+ input_dict = {}
+ # search instances in a not exist store
+ input_dict['project_id'] = "wrong_project"
+ input_dict['region'] = "wrong_region"
+ input_dict['dataset_id'] = "wrong_dataset_id"
+ input_dict['dicom_store_id'] = "wrong_dicom_store_id"
+
+ expected_invalid_dict = {}
+ expected_invalid_dict['status'] = 204
+ expected_invalid_dict['input'] = ''
+ expected_invalid_dict['success'] = False
+
+ fc = FakeHttpClient()
+ FakeClient.return_value = fc
+ with TestPipeline() as p:
+ results = (
+ p | beam.Create(['']) | UploadToDicomStore(input_dict, 'bytes'))
+ assert_that(results, equal_to([expected_invalid_dict]))
+
+ @patch("apache_beam.io.gcp.dicomio.DicomApiHttpClient")
+ def test_missing_parameters(self, FakeClient):
+ input_dict = {}
+ input_dict['project_id'] = "test_project"
+ input_dict['region'] = "test_region"
+
+ expected_invalid_dict = {}
+ expected_invalid_dict['result'] = []
+ expected_invalid_dict['status'] = 'Must have dataset_id in the dict.'
+ expected_invalid_dict['input'] = input_dict
+ expected_invalid_dict['success'] = False
+
+ fc = FakeHttpClient()
+ FakeClient.return_value = fc
+ with self.assertRaisesRegex(ValueError,
+ "Must have dataset_id in the dict."):
+ p = TestPipeline()
+ _ = (p | beam.Create(['']) | UploadToDicomStore(input_dict, 'bytes'))
+
+
+if __name__ == '__main__':
+ unittest.main()
diff --git a/sdks/python/apache_beam/runners/interactive/background_caching_job.py b/sdks/python/apache_beam/runners/interactive/background_caching_job.py
index 117cd59..1b05285 100644
--- a/sdks/python/apache_beam/runners/interactive/background_caching_job.py
+++ b/sdks/python/apache_beam/runners/interactive/background_caching_job.py
@@ -291,7 +291,7 @@
'data to start at the same time, all captured data has been '
'cleared and a new segment of data will be recorded.')
- ie.current_env().cleanup()
+ ie.current_env().cleanup(user_pipeline)
ie.current_env().set_cached_source_signature(
user_pipeline, current_signature)
return is_changed
diff --git a/sdks/python/apache_beam/runners/interactive/background_caching_job_test.py b/sdks/python/apache_beam/runners/interactive/background_caching_job_test.py
index 45c65dd..803f6ce 100644
--- a/sdks/python/apache_beam/runners/interactive/background_caching_job_test.py
+++ b/sdks/python/apache_beam/runners/interactive/background_caching_job_test.py
@@ -91,8 +91,6 @@
sys.version_info < (3, 6), 'The tests require at least Python 3.6 to work.')
class BackgroundCachingJobTest(unittest.TestCase):
def tearDown(self):
- for _, job in ie.current_env()._background_caching_jobs.items():
- job.cancel()
ie.new_env()
# TODO(BEAM-8335): remove the patches when there are appropriate test sources
@@ -302,9 +300,11 @@
def test_determine_a_test_stream_service_running(self):
pipeline = _build_an_empty_stream_pipeline()
test_stream_service = TestStreamServiceController(reader=None)
+ test_stream_service.start()
ie.current_env().set_test_stream_service_controller(
pipeline, test_stream_service)
self.assertTrue(bcj.is_a_test_stream_service_running(pipeline))
+ # the test_stream_service will be cleaned up on teardown.
def test_stop_a_running_test_stream_service(self):
pipeline = _build_an_empty_stream_pipeline()
diff --git a/sdks/python/apache_beam/runners/interactive/display/pcoll_visualization.py b/sdks/python/apache_beam/runners/interactive/display/pcoll_visualization.py
index d11cb24..ce7ec0e 100644
--- a/sdks/python/apache_beam/runners/interactive/display/pcoll_visualization.py
+++ b/sdks/python/apache_beam/runners/interactive/display/pcoll_visualization.py
@@ -47,7 +47,6 @@
from IPython.core.display import Javascript # pylint: disable=import-error
from IPython.core.display import display # pylint: disable=import-error
from IPython.core.display import display_javascript # pylint: disable=import-error
- from IPython.core.display import update_display # pylint: disable=import-error
from facets_overview.generic_feature_statistics_generator import GenericFeatureStatisticsGenerator # pylint: disable=import-error
from timeloop import Timeloop # pylint: disable=import-error
@@ -134,6 +133,10 @@
<script>
{script_in_jquery_with_datatable}
</script>"""
+_NO_DATA_TEMPLATE = _CSS + """
+ <div id="no_data_{id}">No data to display.</div>"""
+_NO_DATA_REMOVAL_SCRIPT = """
+ $("#no_data_{id}").remove();"""
def visualize(
@@ -382,16 +385,23 @@
if update and not update._is_datatable_empty:
display_javascript(Javascript(script_in_jquery_with_datatable))
else:
- html = _DATAFRAME_PAGINATION_TEMPLATE.format(
- table_id=table_id,
- script_in_jquery_with_datatable=script_in_jquery_with_datatable)
+ if data.empty:
+ html = _NO_DATA_TEMPLATE.format(id=table_id)
+ else:
+ html = _DATAFRAME_PAGINATION_TEMPLATE.format(
+ table_id=table_id,
+ script_in_jquery_with_datatable=script_in_jquery_with_datatable)
if update:
if not data.empty:
- # Re-initialize a datatable to replace the existing empty datatable.
- update_display(HTML(html), display_id=update._df_display_id)
+ # Initialize a datatable to replace the existing no data div.
+ display(
+ Javascript(
+ ie._JQUERY_WITH_DATATABLE_TEMPLATE.format(
+ customized_script=_NO_DATA_REMOVAL_SCRIPT.format(
+ id=table_id))))
+ display(HTML(html), display_id=update._df_display_id)
update._is_datatable_empty = False
else:
- # Initialize a datatable for the first time rendering.
display(HTML(html), display_id=self._df_display_id)
if not data.empty:
self._is_datatable_empty = False
diff --git a/sdks/python/apache_beam/runners/interactive/interactive_environment.py b/sdks/python/apache_beam/runners/interactive/interactive_environment.py
index 1d28517..4363d17 100644
--- a/sdks/python/apache_beam/runners/interactive/interactive_environment.py
+++ b/sdks/python/apache_beam/runners/interactive/interactive_environment.py
@@ -251,18 +251,32 @@
return self._inspector
def cleanup(self, pipeline=None):
- """Cleans up cached states for the given pipeline. Cleans up
- for all pipelines if no specific pipeline is given."""
+ """Cleans up cached states for the given pipeline. Noop if the given
+ pipeline is absent from the environment. Cleans up for all pipelines
+ if no pipeline is specified."""
if pipeline:
+ from apache_beam.runners.interactive import background_caching_job as bcj
+ bcj.attempt_to_cancel_background_caching_job(pipeline)
+ bcj.attempt_to_stop_test_stream_service(pipeline)
cache_manager = self.get_cache_manager(pipeline)
if cache_manager:
cache_manager.cleanup()
else:
+ for _, job in self._background_caching_jobs.items():
+ if job:
+ job.cancel()
+ for _, controller in self._test_stream_service_controllers.items():
+ if controller:
+ controller.stop()
for _, cache_manager in self._cache_managers.items():
- cache_manager.cleanup()
+ if cache_manager:
+ cache_manager.cleanup()
+ self.evict_background_caching_job(pipeline)
+ self.evict_test_stream_service_controller(pipeline)
self.evict_computed_pcollections(pipeline)
self.evict_cached_source_signature(pipeline)
+ self.evict_pipeline_result(pipeline)
def watch(self, watchable):
"""Watches a watchable.
@@ -343,9 +357,13 @@
'apache_beam.runners.runner.PipelineResult or its subclass')
self._main_pipeline_results[str(id(pipeline))] = result
- def evict_pipeline_result(self, pipeline):
- """Evicts the tracking of given pipeline run. Noop if absent."""
- return self._main_pipeline_results.pop(str(id(pipeline)), None)
+ def evict_pipeline_result(self, pipeline=None):
+ """Evicts the last run result of the given pipeline. Noop if the pipeline
+ is absent from the environment. If no pipeline is specified, evicts for all
+ pipelines."""
+ if pipeline:
+ return self._main_pipeline_results.pop(str(id(pipeline)), None)
+ self._main_pipeline_results.clear()
def pipeline_result(self, pipeline):
"""Gets the pipeline run result. None if absent."""
@@ -364,16 +382,24 @@
"""Gets the background caching job started from the given pipeline."""
return self._background_caching_jobs.get(str(id(pipeline)), None)
+ def evict_background_caching_job(self, pipeline=None):
+ """Evicts the background caching job started from the given pipeline. Noop
+ if the given pipeline is absent from the environment. If no pipeline is
+ specified, evicts for all pipelines."""
+ if pipeline:
+ return self._background_caching_jobs.pop(str(id(pipeline)), None)
+ self._background_caching_jobs.clear()
+
def set_test_stream_service_controller(self, pipeline, controller):
"""Sets the test stream service controller that has started a gRPC server
- serving the test stream for any job started from the given user-defined
+ serving the test stream for any job started from the given user defined
pipeline.
"""
self._test_stream_service_controllers[str(id(pipeline))] = controller
def get_test_stream_service_controller(self, pipeline):
"""Gets the test stream service controller that has started a gRPC server
- serving the test stream for any job started from the given user-defined
+ serving the test stream for any job started from the given user defined
pipeline.
"""
return self._test_stream_service_controllers.get(str(id(pipeline)), None)
@@ -381,9 +407,12 @@
def evict_test_stream_service_controller(self, pipeline):
"""Evicts and pops the test stream service controller that has started a
gRPC server serving the test stream for any job started from the given
- user-defined pipeline.
+ user defined pipeline. Noop if the given pipeline is absent from the
+ environment. If no pipeline is specified, evicts for all pipelines.
"""
- return self._test_stream_service_controllers.pop(str(id(pipeline)), None)
+ if pipeline:
+ return self._test_stream_service_controllers.pop(str(id(pipeline)), None)
+ self._test_stream_service_controllers.clear()
def is_terminated(self, pipeline):
"""Queries if the most recent job (by executing the given pipeline) state
@@ -400,13 +429,15 @@
return self._cached_source_signature.get(str(id(pipeline)), set())
def evict_cached_source_signature(self, pipeline=None):
+ """Evicts the signature generated for each recorded source of the given
+ pipeline. Noop if the given pipeline is absent from the environment. If no
+ pipeline is specified, evicts for all pipelines."""
if pipeline:
- self._cached_source_signature.pop(str(id(pipeline)), None)
- else:
- self._cached_source_signature.clear()
+ return self._cached_source_signature.pop(str(id(pipeline)), None)
+ self._cached_source_signature.clear()
def track_user_pipelines(self):
- """Record references to all user-defined pipeline instances watched in
+ """Record references to all user defined pipeline instances watched in
current environment.
Current static global singleton interactive environment holds references to
@@ -416,11 +447,17 @@
then handle them differently.
This is invoked every time a PTransform is to be applied if the current
- code execution is under ipython due to the possibility that any user-defined
+ code execution is under ipython due to the possibility that any user defined
pipeline can be re-evaluated through notebook cell re-execution at any time.
Each time this is invoked, it will check if there is a cache manager
already created for each user defined pipeline. If not, create one for it.
+
+ If a pipeline is no longer watched due to re-execution while its
+ PCollections are still in watched scope, the pipeline becomes anonymous but
+ still accessible indirectly through references to its PCollections. This
+ function also clears up internal states for those anonymous pipelines once
+ all their PCollections are anonymous.
"""
self._tracked_user_pipelines = set()
for watching in self.watching():
@@ -428,6 +465,17 @@
if isinstance(val, beam.pipeline.Pipeline):
self._tracked_user_pipelines.add(val)
_ = self.get_cache_manager(val, create_if_absent=True)
+ all_tracked_pipeline_ids = set(self._background_caching_jobs.keys()).union(
+ set(self._test_stream_service_controllers.keys()),
+ set(self._cache_managers.keys()),
+ {str(id(pcoll.pipeline))
+ for pcoll in self._computed_pcolls},
+ set(self._cached_source_signature.keys()),
+ set(self._main_pipeline_results.keys()))
+ inspectable_pipelines = self._inspector.inspectable_pipelines
+ for pipeline in all_tracked_pipeline_ids:
+ if pipeline not in inspectable_pipelines:
+ self.cleanup(pipeline)
@property
def tracked_user_pipelines(self):
diff --git a/sdks/python/apache_beam/runners/interactive/interactive_environment_test.py b/sdks/python/apache_beam/runners/interactive/interactive_environment_test.py
index 6f44dac..6650c63 100644
--- a/sdks/python/apache_beam/runners/interactive/interactive_environment_test.py
+++ b/sdks/python/apache_beam/runners/interactive/interactive_environment_test.py
@@ -236,14 +236,33 @@
@patch(
'apache_beam.runners.interactive.interactive_environment'
'.InteractiveEnvironment.cleanup')
- def test_cleanup_invoked_when_cache_manager_is_evicted(self, mocked_cleanup):
+ def test_track_user_pipeline_cleanup_non_inspectable_pipeline(
+ self, mocked_cleanup):
ie._interactive_beam_env = None
ie.new_env()
- dummy_pipeline = 'dummy'
+ dummy_pipeline_1 = beam.Pipeline()
+ dummy_pipeline_2 = beam.Pipeline()
+ dummy_pipeline_3 = beam.Pipeline()
+ dummy_pipeline_4 = beam.Pipeline()
+ dummy_pcoll = dummy_pipeline_4 | beam.Create([1])
+ dummy_pipeline_5 = beam.Pipeline()
+ dummy_non_inspectable_pipeline = 'dummy'
+ ie.current_env().watch(locals())
+ from apache_beam.runners.interactive.background_caching_job import BackgroundCachingJob
+ ie.current_env().set_background_caching_job(
+ dummy_pipeline_1,
+ BackgroundCachingJob(
+ runner.PipelineResult(runner.PipelineState.DONE), limiters=[]))
+ ie.current_env().set_test_stream_service_controller(dummy_pipeline_2, None)
ie.current_env().set_cache_manager(
- cache.FileBasedCacheManager(), dummy_pipeline)
+ cache.FileBasedCacheManager(), dummy_pipeline_3)
+ ie.current_env().mark_pcollection_computed([dummy_pcoll])
+ ie.current_env().set_cached_source_signature(
+ dummy_non_inspectable_pipeline, None)
+ ie.current_env().set_pipeline_result(
+ dummy_pipeline_5, runner.PipelineResult(runner.PipelineState.RUNNING))
mocked_cleanup.assert_not_called()
- ie.current_env().evict_cache_manager(dummy_pipeline)
+ ie.current_env().track_user_pipelines()
mocked_cleanup.assert_called_once()
diff --git a/sdks/python/apache_beam/runners/interactive/messaging/interactive_environment_inspector.py b/sdks/python/apache_beam/runners/interactive/messaging/interactive_environment_inspector.py
index 3bce182..a4a9f02 100644
--- a/sdks/python/apache_beam/runners/interactive/messaging/interactive_environment_inspector.py
+++ b/sdks/python/apache_beam/runners/interactive/messaging/interactive_environment_inspector.py
@@ -41,6 +41,7 @@
def __init__(self):
self._inspectables = {}
self._anonymous = {}
+ self._inspectable_pipelines = set()
@property
def inspectables(self):
@@ -49,6 +50,20 @@
self._inspectables = inspect()
return self._inspectables
+ @property
+ def inspectable_pipelines(self):
+ """Returns a dictionary of all inspectable pipelines. The keys are
+ stringified id of pipeline instances.
+
+ This includes user defined pipeline assigned to variables and anonymous
+ pipelines with inspectable PCollections.
+ If a user defined pipeline is not within the returned dict, it can be
+ considered out of scope, and all resources and memory states related to it
+ should be released.
+ """
+ _ = self.list_inspectables()
+ return self._inspectable_pipelines
+
@as_json
def list_inspectables(self):
"""Lists inspectables in JSON format.
@@ -89,6 +104,8 @@
pipeline_identifier = obfuscate(meta(pipelines[pipeline], pipeline))
listing[pipeline_identifier]['pcolls'][identifier] = inspectable[
'metadata']
+ self._inspectable_pipelines = dict(
+ (str(id(pipeline)), pipeline) for pipeline in pipelines)
return listing
def get_val(self, identifier):
diff --git a/sdks/python/apache_beam/runners/interactive/options/capture_control.py b/sdks/python/apache_beam/runners/interactive/options/capture_control.py
index 12e901f..ab877b5 100644
--- a/sdks/python/apache_beam/runners/interactive/options/capture_control.py
+++ b/sdks/python/apache_beam/runners/interactive/options/capture_control.py
@@ -29,7 +29,6 @@
from datetime import timedelta
from apache_beam.io.gcp.pubsub import ReadFromPubSub
-from apache_beam.runners.interactive import background_caching_job as bcj
from apache_beam.runners.interactive import interactive_environment as ie
from apache_beam.runners.interactive.options import capture_limiters
@@ -71,15 +70,7 @@
runs, Interactive Beam will capture fresh data."""
if ie.current_env().options.enable_capture_replay:
_LOGGER.info(
- 'You have requested Interactive Beam to evict all captured '
+ 'You have requested Interactive Beam to evict all recorded'
'data that could be deterministically replayed among multiple '
'pipeline runs.')
- ie.current_env().track_user_pipelines()
- if pipeline:
- bcj.attempt_to_cancel_background_caching_job(pipeline)
- bcj.attempt_to_stop_test_stream_service(pipeline)
- else:
- for user_pipeline in ie.current_env().tracked_user_pipelines:
- bcj.attempt_to_cancel_background_caching_job(user_pipeline)
- bcj.attempt_to_stop_test_stream_service(user_pipeline)
ie.current_env().cleanup(pipeline)
diff --git a/sdks/python/setup.py b/sdks/python/setup.py
index cac23cd..652301f 100644
--- a/sdks/python/setup.py
+++ b/sdks/python/setup.py
@@ -201,6 +201,7 @@
GCP_REQUIREMENTS = [
'cachetools>=3.1.0,<4',
'google-apitools>=0.5.31,<0.5.32',
+ 'google-auth>=1.18.0,<2',
'google-cloud-datastore>=1.7.1,<2',
'google-cloud-pubsub>=0.39.0,<2',
# GCP packages required by tests
diff --git a/settings.gradle b/settings.gradle
index 8289f79..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"