From 331631f5c9404abf7a7114b9f190db8761334db8 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 16 Feb 2026 14:58:42 -0700 Subject: [PATCH 01/24] Consolidate TPC benchmark scripts into single Python runner Replace 9 per-engine shell scripts with a single `run.py` that loads per-engine TOML config files. This eliminates duplicated Spark conf boilerplate and makes it easier to add new engines or modify shared settings. Usage: `python3 run.py --engine comet --benchmark tpch [--dry-run]` Also moves benchmarks from `dev/benchmarks/` to `benchmarks/tpc/` and updates all documentation references. Co-Authored-By: Claude Opus 4.6 --- {dev/benchmarks => benchmarks/tpc}/.gitignore | 0 {dev/benchmarks => benchmarks/tpc}/README.md | 39 +- .../tpc}/create-iceberg-tpch.py | 0 .../tpc}/drop-caches.sh | 0 benchmarks/tpc/engines/blaze.toml | 36 ++ benchmarks/tpc/engines/comet-iceberg.toml | 49 +++ benchmarks/tpc/engines/comet.toml | 35 ++ benchmarks/tpc/engines/gluten.toml | 34 ++ benchmarks/tpc/engines/spark.toml | 19 + .../tpc}/generate-comparison.py | 0 benchmarks/tpc/run.py | 378 ++++++++++++++++++ .../benchmarks => benchmarks/tpc}/tpcbench.py | 0 dev/benchmarks/blaze-tpcds.sh | 53 --- dev/benchmarks/blaze-tpch.sh | 53 --- dev/benchmarks/comet-tpcds.sh | 53 --- dev/benchmarks/comet-tpch-iceberg.sh | 114 ------ dev/benchmarks/comet-tpch.sh | 55 --- dev/benchmarks/gluten-tpcds.sh | 53 --- dev/benchmarks/gluten-tpch.sh | 53 --- dev/benchmarks/spark-tpcds.sh | 45 --- dev/benchmarks/spark-tpch.sh | 46 --- docs/source/about/gluten_comparison.md | 2 +- .../benchmark-results/tpc-ds.md | 2 +- .../benchmark-results/tpc-h.md | 2 +- docs/source/contributor-guide/benchmarking.md | 2 +- .../contributor-guide/benchmarking_aws_ec2.md | 15 +- 26 files changed, 600 insertions(+), 538 deletions(-) rename {dev/benchmarks => benchmarks/tpc}/.gitignore (100%) rename {dev/benchmarks => benchmarks/tpc}/README.md (79%) rename {dev/benchmarks => benchmarks/tpc}/create-iceberg-tpch.py (100%) rename {dev/benchmarks => benchmarks/tpc}/drop-caches.sh (100%) create mode 100644 benchmarks/tpc/engines/blaze.toml create mode 100644 benchmarks/tpc/engines/comet-iceberg.toml create mode 100644 benchmarks/tpc/engines/comet.toml create mode 100644 benchmarks/tpc/engines/gluten.toml create mode 100644 benchmarks/tpc/engines/spark.toml rename {dev/benchmarks => benchmarks/tpc}/generate-comparison.py (100%) create mode 100755 benchmarks/tpc/run.py rename {dev/benchmarks => benchmarks/tpc}/tpcbench.py (100%) delete mode 100755 dev/benchmarks/blaze-tpcds.sh delete mode 100755 dev/benchmarks/blaze-tpch.sh delete mode 100755 dev/benchmarks/comet-tpcds.sh delete mode 100755 dev/benchmarks/comet-tpch-iceberg.sh delete mode 100755 dev/benchmarks/comet-tpch.sh delete mode 100755 dev/benchmarks/gluten-tpcds.sh delete mode 100755 dev/benchmarks/gluten-tpch.sh delete mode 100755 dev/benchmarks/spark-tpcds.sh delete mode 100755 dev/benchmarks/spark-tpch.sh diff --git a/dev/benchmarks/.gitignore b/benchmarks/tpc/.gitignore similarity index 100% rename from dev/benchmarks/.gitignore rename to benchmarks/tpc/.gitignore diff --git a/dev/benchmarks/README.md b/benchmarks/tpc/README.md similarity index 79% rename from dev/benchmarks/README.md rename to benchmarks/tpc/README.md index b3ea674199..48df9c63d1 100644 --- a/dev/benchmarks/README.md +++ b/benchmarks/tpc/README.md @@ -26,6 +26,26 @@ For full instructions on running these benchmarks on an EC2 instance, see the [C [Comet Benchmarking on EC2 Guide]: https://datafusion.apache.org/comet/contributor-guide/benchmarking_aws_ec2.html +## Usage + +All benchmarks are run via `run.py`: + +``` +python3 run.py --engine --benchmark [options] +``` + +| Option | Description | +| --------------- | ---------------------------------------- | +| `--engine` | Engine name (matches a TOML file in `engines/`) | +| `--benchmark` | `tpch` or `tpcds` | +| `--iterations` | Number of iterations (default: 1) | +| `--output` | Output directory (default: `.`) | +| `--query` | Run a single query number | +| `--no-restart` | Skip Spark master/worker restart | +| `--dry-run` | Print the spark-submit command without executing | + +Available engines: `spark`, `comet`, `comet-iceberg`, `gluten`, `blaze` + ## Example usage Set Spark environment variables: @@ -47,7 +67,7 @@ Run Spark benchmark: ```shell export JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64 sudo ./drop-caches.sh -./spark-tpch.sh +python3 run.py --engine spark --benchmark tpch ``` Run Comet benchmark: @@ -56,7 +76,7 @@ Run Comet benchmark: export JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64 export COMET_JAR=/opt/comet/comet-spark-spark3.5_2.12-0.10.0.jar sudo ./drop-caches.sh -./comet-tpch.sh +python3 run.py --engine comet --benchmark tpch ``` Run Gluten benchmark: @@ -65,7 +85,13 @@ Run Gluten benchmark: export JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64 export GLUTEN_JAR=/opt/gluten/gluten-velox-bundle-spark3.5_2.12-linux_amd64-1.4.0.jar sudo ./drop-caches.sh -./gluten-tpch.sh +python3 run.py --engine gluten --benchmark tpch +``` + +Preview a command without running it: + +```shell +python3 run.py --engine comet --benchmark tpch --dry-run ``` Generating charts: @@ -74,6 +100,11 @@ Generating charts: python3 generate-comparison.py --benchmark tpch --labels "Spark 3.5.3" "Comet 0.9.0" "Gluten 1.4.0" --title "TPC-H @ 100 GB (single executor, 8 cores, local Parquet files)" spark-tpch-1752338506381.json comet-tpch-1752337818039.json gluten-tpch-1752337474344.json ``` +## Engine Configuration + +Each engine is defined by a TOML file in `engines/`. The config specifies JARs, Spark conf overrides, +required environment variables, and optional defaults/exports. See existing files for examples. + ## Iceberg Benchmarking Comet includes native Iceberg support via iceberg-rust integration. This enables benchmarking TPC-H queries @@ -124,7 +155,7 @@ export ICEBERG_JAR=/path/to/iceberg-spark-runtime-3.5_2.12-1.8.1.jar export ICEBERG_WAREHOUSE=/mnt/bigdata/iceberg-warehouse export TPCH_QUERIES=/mnt/bigdata/tpch/queries/ sudo ./drop-caches.sh -./comet-tpch-iceberg.sh +python3 run.py --engine comet-iceberg --benchmark tpch ``` The benchmark uses `spark.comet.scan.icebergNative.enabled=true` to enable Comet's native iceberg-rust diff --git a/dev/benchmarks/create-iceberg-tpch.py b/benchmarks/tpc/create-iceberg-tpch.py similarity index 100% rename from dev/benchmarks/create-iceberg-tpch.py rename to benchmarks/tpc/create-iceberg-tpch.py diff --git a/dev/benchmarks/drop-caches.sh b/benchmarks/tpc/drop-caches.sh similarity index 100% rename from dev/benchmarks/drop-caches.sh rename to benchmarks/tpc/drop-caches.sh diff --git a/benchmarks/tpc/engines/blaze.toml b/benchmarks/tpc/engines/blaze.toml new file mode 100644 index 0000000000..7298a8f3ab --- /dev/null +++ b/benchmarks/tpc/engines/blaze.toml @@ -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. + +[engine] +name = "blaze" + +[env] +required = ["BLAZE_JAR"] + +[spark_submit] +jars = ["$BLAZE_JAR"] +driver_class_path = ["$BLAZE_JAR"] + +[spark_conf] +"spark.driver.extraClassPath" = "$BLAZE_JAR" +"spark.executor.extraClassPath" = "$BLAZE_JAR" +"spark.sql.extensions" = "org.apache.spark.sql.blaze.BlazeSparkSessionExtension" +"spark.shuffle.manager" = "org.apache.spark.sql.execution.blaze.shuffle.BlazeShuffleManager" +"spark.blaze.enable" = "true" +"spark.blaze.forceShuffledHashJoin" = "true" +"spark.executor.memoryOverhead" = "16g" +"spark.memory.offHeap.enabled" = "false" diff --git a/benchmarks/tpc/engines/comet-iceberg.toml b/benchmarks/tpc/engines/comet-iceberg.toml new file mode 100644 index 0000000000..ebabfce7b4 --- /dev/null +++ b/benchmarks/tpc/engines/comet-iceberg.toml @@ -0,0 +1,49 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +[engine] +name = "comet-iceberg" + +[env] +required = ["COMET_JAR", "ICEBERG_JAR", "ICEBERG_WAREHOUSE", "TPCH_QUERIES"] + +[env.defaults] +ICEBERG_CATALOG = "local" +ICEBERG_DATABASE = "tpch" + +[spark_submit] +jars = ["$COMET_JAR", "$ICEBERG_JAR"] +driver_class_path = ["$COMET_JAR", "$ICEBERG_JAR"] + +[spark_conf] +"spark.driver.extraClassPath" = "$COMET_JAR:$ICEBERG_JAR" +"spark.executor.extraClassPath" = "$COMET_JAR:$ICEBERG_JAR" +"spark.plugins" = "org.apache.spark.CometPlugin" +"spark.shuffle.manager" = "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager" +"spark.comet.exec.replaceSortMergeJoin" = "true" +"spark.comet.expression.Cast.allowIncompatible" = "true" +"spark.comet.enabled" = "true" +"spark.comet.exec.enabled" = "true" +"spark.comet.scan.icebergNative.enabled" = "true" +"spark.comet.explainFallback.enabled" = "true" +"spark.sql.catalog.${ICEBERG_CATALOG}" = "org.apache.iceberg.spark.SparkCatalog" +"spark.sql.catalog.${ICEBERG_CATALOG}.type" = "hadoop" +"spark.sql.catalog.${ICEBERG_CATALOG}.warehouse" = "$ICEBERG_WAREHOUSE" +"spark.sql.defaultCatalog" = "${ICEBERG_CATALOG}" + +[tpcbench_args] +use_iceberg = true diff --git a/benchmarks/tpc/engines/comet.toml b/benchmarks/tpc/engines/comet.toml new file mode 100644 index 0000000000..8e19165ebf --- /dev/null +++ b/benchmarks/tpc/engines/comet.toml @@ -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. + +[engine] +name = "comet" + +[env] +required = ["COMET_JAR"] + +[spark_submit] +jars = ["$COMET_JAR"] +driver_class_path = ["$COMET_JAR"] + +[spark_conf] +"spark.driver.extraClassPath" = "$COMET_JAR" +"spark.executor.extraClassPath" = "$COMET_JAR" +"spark.plugins" = "org.apache.spark.CometPlugin" +"spark.shuffle.manager" = "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager" +"spark.comet.scan.impl" = "native_datafusion" +"spark.comet.exec.replaceSortMergeJoin" = "true" +"spark.comet.expression.Cast.allowIncompatible" = "true" diff --git a/benchmarks/tpc/engines/gluten.toml b/benchmarks/tpc/engines/gluten.toml new file mode 100644 index 0000000000..20165788c2 --- /dev/null +++ b/benchmarks/tpc/engines/gluten.toml @@ -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. + +[engine] +name = "gluten" + +[env] +required = ["GLUTEN_JAR"] +exports = { TZ = "UTC" } + +[spark_submit] +jars = ["$GLUTEN_JAR"] + +[spark_conf] +"spark.plugins" = "org.apache.gluten.GlutenPlugin" +"spark.driver.extraClassPath" = "${GLUTEN_JAR}" +"spark.executor.extraClassPath" = "${GLUTEN_JAR}" +"spark.gluten.sql.columnar.forceShuffledHashJoin" = "true" +"spark.shuffle.manager" = "org.apache.spark.shuffle.sort.ColumnarShuffleManager" +"spark.sql.session.timeZone" = "UTC" diff --git a/benchmarks/tpc/engines/spark.toml b/benchmarks/tpc/engines/spark.toml new file mode 100644 index 0000000000..c02e7a6ad2 --- /dev/null +++ b/benchmarks/tpc/engines/spark.toml @@ -0,0 +1,19 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +[engine] +name = "spark" diff --git a/dev/benchmarks/generate-comparison.py b/benchmarks/tpc/generate-comparison.py similarity index 100% rename from dev/benchmarks/generate-comparison.py rename to benchmarks/tpc/generate-comparison.py diff --git a/benchmarks/tpc/run.py b/benchmarks/tpc/run.py new file mode 100755 index 0000000000..122f90f821 --- /dev/null +++ b/benchmarks/tpc/run.py @@ -0,0 +1,378 @@ +#!/usr/bin/env python3 +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +"""Consolidated TPC benchmark runner for Spark-based engines. + +Usage: + python3 run.py --engine comet --benchmark tpch + python3 run.py --engine blaze --benchmark tpcds --iterations 3 + python3 run.py --engine comet-iceberg --benchmark tpch + python3 run.py --engine comet --benchmark tpch --dry-run + python3 run.py --engine spark --benchmark tpch --no-restart +""" + +import argparse +import os +import re +import subprocess +import sys + +# --------------------------------------------------------------------------- +# TOML loading – prefer stdlib tomllib (3.11+), else minimal fallback +# --------------------------------------------------------------------------- + +try: + import tomllib # Python 3.11+ + + def load_toml(path): + with open(path, "rb") as f: + return tomllib.load(f) + +except ModuleNotFoundError: + + def _parse_toml(text): + """Minimal TOML parser supporting tables, quoted-key strings, plain + strings, arrays of strings, booleans, and comments. Sufficient for + the engine config files used by this runner.""" + root = {} + current = root + for line in text.splitlines(): + line = line.strip() + if not line or line.startswith("#"): + continue + # Table header: [env.defaults] or [spark_conf] + m = re.match(r"^\[([^\]]+)\]$", line) + if m: + keys = m.group(1).split(".") + current = root + for k in keys: + current = current.setdefault(k, {}) + continue + # Key = value + m = re.match(r'^("(?:[^"\\]|\\.)*"|[A-Za-z0-9_.]+)\s*=\s*(.+)$', line) + if not m: + continue + raw_key, raw_val = m.group(1), m.group(2).strip() + key = raw_key.strip('"') + val = _parse_value(raw_val) + current[key] = val + return root + + def _parse_value(raw): + if raw == "true": + return True + if raw == "false": + return False + if raw.startswith('"') and raw.endswith('"'): + return raw[1:-1] + if raw.startswith("["): + # Simple array of strings + items = [] + for m in re.finditer(r'"((?:[^"\\]|\\.)*)"', raw): + items.append(m.group(1)) + return items + if raw.startswith("{"): + # Inline table: { KEY = "VAL", ... } + tbl = {} + for m in re.finditer(r'([A-Za-z0-9_]+)\s*=\s*"((?:[^"\\]|\\.)*)"', raw): + tbl[m.group(1)] = m.group(2) + return tbl + return raw + + def load_toml(path): + with open(path, "r") as f: + return _parse_toml(f.read()) + + +# --------------------------------------------------------------------------- +# Common Spark configuration (shared across all engines) +# --------------------------------------------------------------------------- + +COMMON_SPARK_CONF = { + "spark.driver.memory": "8G", + "spark.executor.memory": "16g", + "spark.memory.offHeap.enabled": "true", + "spark.memory.offHeap.size": "16g", + "spark.eventLog.enabled": "true", + "spark.hadoop.fs.s3a.impl": "org.apache.hadoop.fs.s3a.S3AFileSystem", + "spark.hadoop.fs.s3a.aws.credentials.provider": "com.amazonaws.auth.DefaultAWSCredentialsProviderChain", +} + +# --------------------------------------------------------------------------- +# Benchmark profiles +# --------------------------------------------------------------------------- + +BENCHMARK_PROFILES = { + "tpch": { + "executor_instances": "1", + "executor_cores": "8", + "max_cores": "8", + "data_env": "TPCH_DATA", + "queries_env": "TPCH_QUERIES", + "format": "parquet", + }, + "tpcds": { + "executor_instances": "2", + "executor_cores": "8", + "max_cores": "16", + "data_env": "TPCDS_DATA", + "queries_env": "TPCDS_QUERIES", + "format": None, # omit --format for TPC-DS + }, +} + +# --------------------------------------------------------------------------- +# Helpers +# --------------------------------------------------------------------------- + + +def resolve_env(value): + """Expand $VAR and ${VAR} references using os.environ.""" + if not isinstance(value, str): + return value + return re.sub( + r"\$\{([^}]+)\}|\$([A-Za-z_][A-Za-z0-9_]*)", + lambda m: os.environ.get(m.group(1) or m.group(2), ""), + value, + ) + + +def resolve_env_in_list(lst): + return [resolve_env(v) for v in lst] + + +def load_engine_config(engine_name): + """Load and return the TOML config for the given engine.""" + script_dir = os.path.dirname(os.path.abspath(__file__)) + config_path = os.path.join(script_dir, "engines", f"{engine_name}.toml") + if not os.path.exists(config_path): + available = sorted( + f.removesuffix(".toml") + for f in os.listdir(os.path.join(script_dir, "engines")) + if f.endswith(".toml") + ) + print(f"Error: Unknown engine '{engine_name}'", file=sys.stderr) + print(f"Available engines: {', '.join(available)}", file=sys.stderr) + sys.exit(1) + return load_toml(config_path) + + +def apply_env_defaults(config): + """Set environment variable defaults from [env.defaults].""" + defaults = config.get("env", {}).get("defaults", {}) + for key, val in defaults.items(): + if key not in os.environ: + os.environ[key] = val + + +def apply_env_exports(config): + """Export environment variables from [env.exports].""" + exports = config.get("env", {}).get("exports", {}) + for key, val in exports.items(): + os.environ[key] = val + + +def check_required_env(config): + """Validate that required environment variables are set.""" + required = config.get("env", {}).get("required", []) + missing = [v for v in required if not os.environ.get(v)] + if missing: + print( + f"Error: Required environment variable(s) not set: {', '.join(missing)}", + file=sys.stderr, + ) + sys.exit(1) + + +def check_common_env(): + """Validate SPARK_HOME and SPARK_MASTER are set.""" + for var in ("SPARK_HOME", "SPARK_MASTER"): + if not os.environ.get(var): + print(f"Error: {var} is not set", file=sys.stderr) + sys.exit(1) + + +def build_spark_submit_cmd(config, benchmark, args): + """Build the spark-submit command list.""" + spark_home = os.environ["SPARK_HOME"] + spark_master = os.environ["SPARK_MASTER"] + profile = BENCHMARK_PROFILES[benchmark] + + cmd = [os.path.join(spark_home, "bin", "spark-submit")] + cmd += ["--master", spark_master] + + # --jars + jars = config.get("spark_submit", {}).get("jars", []) + if jars: + cmd += ["--jars", ",".join(resolve_env_in_list(jars))] + + # --driver-class-path + driver_cp = config.get("spark_submit", {}).get("driver_class_path", []) + if driver_cp: + cmd += ["--driver-class-path", ":".join(resolve_env_in_list(driver_cp))] + + # Merge spark confs: common + benchmark profile + engine overrides + conf = dict(COMMON_SPARK_CONF) + conf["spark.executor.instances"] = profile["executor_instances"] + conf["spark.executor.cores"] = profile["executor_cores"] + conf["spark.cores.max"] = profile["max_cores"] + + engine_conf = config.get("spark_conf", {}) + for key, val in engine_conf.items(): + if isinstance(val, bool): + val = "true" if val else "false" + conf[resolve_env(key)] = resolve_env(str(val)) + + for key, val in sorted(conf.items()): + cmd += ["--conf", f"{key}={val}"] + + # tpcbench.py path + cmd.append("tpcbench.py") + + # tpcbench args + engine_name = config.get("engine", {}).get("name", args.engine) + cmd += ["--name", engine_name] + cmd += ["--benchmark", benchmark] + + use_iceberg = config.get("tpcbench_args", {}).get("use_iceberg", False) + if use_iceberg: + cmd += ["--catalog", resolve_env("${ICEBERG_CATALOG}")] + cmd += ["--database", resolve_env("${ICEBERG_DATABASE}")] + else: + data_var = profile["data_env"] + data_val = os.environ.get(data_var, "") + cmd += ["--data", data_val] + + queries_var = profile["queries_env"] + queries_val = os.environ.get(queries_var, "") + cmd += ["--queries", queries_val] + + cmd += ["--output", args.output] + cmd += ["--iterations", str(args.iterations)] + + if args.query is not None: + cmd += ["--query", str(args.query)] + + if profile["format"] and not use_iceberg: + cmd += ["--format", profile["format"]] + + return cmd + + +def restart_spark(): + """Stop and start Spark master and worker.""" + spark_home = os.environ["SPARK_HOME"] + sbin = os.path.join(spark_home, "sbin") + spark_master = os.environ["SPARK_MASTER"] + + # Stop (ignore errors) + subprocess.run( + [os.path.join(sbin, "stop-master.sh")], + stdout=subprocess.DEVNULL, + stderr=subprocess.DEVNULL, + ) + subprocess.run( + [os.path.join(sbin, "stop-worker.sh")], + stdout=subprocess.DEVNULL, + stderr=subprocess.DEVNULL, + ) + + # Start (check errors) + r = subprocess.run([os.path.join(sbin, "start-master.sh")]) + if r.returncode != 0: + print("Error: Failed to start Spark master", file=sys.stderr) + sys.exit(1) + + r = subprocess.run([os.path.join(sbin, "start-worker.sh"), spark_master]) + if r.returncode != 0: + print("Error: Failed to start Spark worker", file=sys.stderr) + sys.exit(1) + + +def main(): + parser = argparse.ArgumentParser( + description="Consolidated TPC benchmark runner for Spark-based engines." + ) + parser.add_argument( + "--engine", + required=True, + help="Engine name (matches a TOML file in engines/)", + ) + parser.add_argument( + "--benchmark", + required=True, + choices=["tpch", "tpcds"], + help="Benchmark to run", + ) + parser.add_argument( + "--iterations", type=int, default=1, help="Number of iterations (default: 1)" + ) + parser.add_argument( + "--output", default=".", help="Output directory (default: .)" + ) + parser.add_argument( + "--query", type=int, default=None, help="Run a single query number" + ) + parser.add_argument( + "--no-restart", + action="store_true", + help="Skip Spark master/worker restart", + ) + parser.add_argument( + "--dry-run", + action="store_true", + help="Print the spark-submit command without executing", + ) + args = parser.parse_args() + + config = load_engine_config(args.engine) + + # Apply env defaults and exports before validation + apply_env_defaults(config) + apply_env_exports(config) + + check_common_env() + check_required_env(config) + + # Restart Spark unless --no-restart or --dry-run + if not args.no_restart and not args.dry_run: + restart_spark() + + cmd = build_spark_submit_cmd(config, args.benchmark, args) + + if args.dry_run: + # Group paired arguments (e.g. --conf key=value) on one line + parts = [] + i = 0 + while i < len(cmd): + token = cmd[i] + if token.startswith("--") and i + 1 < len(cmd) and not cmd[i + 1].startswith("--"): + parts.append(f"{token} {cmd[i + 1]}") + i += 2 + else: + parts.append(token) + i += 1 + print(" \\\n ".join(parts)) + else: + r = subprocess.run(cmd) + sys.exit(r.returncode) + + +if __name__ == "__main__": + main() diff --git a/dev/benchmarks/tpcbench.py b/benchmarks/tpc/tpcbench.py similarity index 100% rename from dev/benchmarks/tpcbench.py rename to benchmarks/tpc/tpcbench.py diff --git a/dev/benchmarks/blaze-tpcds.sh b/dev/benchmarks/blaze-tpcds.sh deleted file mode 100755 index 90a4a48468..0000000000 --- a/dev/benchmarks/blaze-tpcds.sh +++ /dev/null @@ -1,53 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -# - -$SPARK_HOME/sbin/stop-master.sh -$SPARK_HOME/sbin/stop-worker.sh - -$SPARK_HOME/sbin/start-master.sh -$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER - -$SPARK_HOME/bin/spark-submit \ - --master $SPARK_MASTER \ - --jars $BLAZE_JAR \ - --driver-class-path $BLAZE_JAR \ - --conf spark.driver.memory=8G \ - --conf spark.executor.instances=2 \ - --conf spark.executor.cores=8 \ - --conf spark.cores.max=16 \ - --conf spark.executor.memory=16g \ - --conf spark.executor.memoryOverhead=16g \ - --conf spark.memory.offHeap.enabled=false \ - --conf spark.eventLog.enabled=true \ - --conf spark.driver.extraClassPath=$BLAZE_JAR \ - --conf spark.executor.extraClassPath=$BLAZE_JAR \ - --conf spark.sql.extensions=org.apache.spark.sql.blaze.BlazeSparkSessionExtension \ - --conf spark.shuffle.manager=org.apache.spark.sql.execution.blaze.shuffle.BlazeShuffleManager \ - --conf spark.blaze.enable=true \ - --conf spark.blaze.forceShuffledHashJoin=true \ - --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ - --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ - tpcbench.py \ - --name blaze \ - --benchmark tpcds \ - --data $TPCDS_DATA \ - --queries $TPCDS_QUERIES \ - --output . \ - --iterations 1 diff --git a/dev/benchmarks/blaze-tpch.sh b/dev/benchmarks/blaze-tpch.sh deleted file mode 100755 index 2c6878737d..0000000000 --- a/dev/benchmarks/blaze-tpch.sh +++ /dev/null @@ -1,53 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -# - -$SPARK_HOME/sbin/stop-master.sh -$SPARK_HOME/sbin/stop-worker.sh - -$SPARK_HOME/sbin/start-master.sh -$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER - -$SPARK_HOME/bin/spark-submit \ - --master $SPARK_MASTER \ - --jars $BLAZE_JAR \ - --driver-class-path $BLAZE_JAR \ - --conf spark.driver.memory=8G \ - --conf spark.executor.instances=1 \ - --conf spark.executor.cores=8 \ - --conf spark.cores.max=8 \ - --conf spark.executor.memory=16g \ - --conf spark.executor.memoryOverhead=16g \ - --conf spark.memory.offHeap.enabled=false \ - --conf spark.eventLog.enabled=true \ - --conf spark.driver.extraClassPath=$BLAZE_JAR \ - --conf spark.executor.extraClassPath=$BLAZE_JAR \ - --conf spark.sql.extensions=org.apache.spark.sql.blaze.BlazeSparkSessionExtension \ - --conf spark.shuffle.manager=org.apache.spark.sql.execution.blaze.shuffle.BlazeShuffleManager \ - --conf spark.blaze.enable=true \ - --conf spark.blaze.forceShuffledHashJoin=true \ - --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ - --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ - tpcbench.py \ - --name blaze \ - --benchmark tpch \ - --data $TPCH_DATA \ - --queries $TPCH_QUERIES \ - --output . \ - --iterations 1 diff --git a/dev/benchmarks/comet-tpcds.sh b/dev/benchmarks/comet-tpcds.sh deleted file mode 100755 index b55b27188c..0000000000 --- a/dev/benchmarks/comet-tpcds.sh +++ /dev/null @@ -1,53 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -# - -$SPARK_HOME/sbin/stop-master.sh -$SPARK_HOME/sbin/stop-worker.sh - -$SPARK_HOME/sbin/start-master.sh -$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER - -$SPARK_HOME/bin/spark-submit \ - --master $SPARK_MASTER \ - --jars $COMET_JAR \ - --driver-class-path $COMET_JAR \ - --conf spark.driver.memory=8G \ - --conf spark.executor.instances=2 \ - --conf spark.executor.cores=8 \ - --conf spark.cores.max=16 \ - --conf spark.executor.memory=16g \ - --conf spark.memory.offHeap.enabled=true \ - --conf spark.memory.offHeap.size=16g \ - --conf spark.eventLog.enabled=true \ - --conf spark.driver.extraClassPath=$COMET_JAR \ - --conf spark.executor.extraClassPath=$COMET_JAR \ - --conf spark.plugins=org.apache.spark.CometPlugin \ - --conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \ - --conf spark.comet.scan.impl=native_datafusion \ - --conf spark.comet.expression.Cast.allowIncompatible=true \ - --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ - --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ - tpcbench.py \ - --name comet \ - --benchmark tpcds \ - --data $TPCDS_DATA \ - --queries $TPCDS_QUERIES \ - --output . \ - --iterations 1 diff --git a/dev/benchmarks/comet-tpch-iceberg.sh b/dev/benchmarks/comet-tpch-iceberg.sh deleted file mode 100755 index 7907125c82..0000000000 --- a/dev/benchmarks/comet-tpch-iceberg.sh +++ /dev/null @@ -1,114 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -# - -# TPC-H benchmark using Iceberg tables with Comet's native iceberg-rust integration. -# -# Required environment variables: -# SPARK_HOME - Path to Spark installation -# SPARK_MASTER - Spark master URL (e.g., spark://localhost:7077) -# COMET_JAR - Path to Comet JAR -# ICEBERG_JAR - Path to Iceberg Spark runtime JAR -# ICEBERG_WAREHOUSE - Path to Iceberg warehouse directory -# TPCH_QUERIES - Path to TPC-H query files -# -# Optional: -# ICEBERG_CATALOG - Catalog name (default: local) -# ICEBERG_DATABASE - Database name (default: tpch) -# -# Setup (run once to create Iceberg tables from Parquet): -# $SPARK_HOME/bin/spark-submit \ -# --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.8.1 \ -# --conf spark.sql.catalog.local=org.apache.iceberg.spark.SparkCatalog \ -# --conf spark.sql.catalog.local.type=hadoop \ -# --conf spark.sql.catalog.local.warehouse=$ICEBERG_WAREHOUSE \ -# create-iceberg-tpch.py \ -# --parquet-path $TPCH_DATA \ -# --catalog local \ -# --database tpch - -set -e - -# Defaults -ICEBERG_CATALOG=${ICEBERG_CATALOG:-local} -ICEBERG_DATABASE=${ICEBERG_DATABASE:-tpch} - -# Validate required variables -if [ -z "$SPARK_HOME" ]; then - echo "Error: SPARK_HOME is not set" - exit 1 -fi -if [ -z "$COMET_JAR" ]; then - echo "Error: COMET_JAR is not set" - exit 1 -fi -if [ -z "$ICEBERG_JAR" ]; then - echo "Error: ICEBERG_JAR is not set" - echo "Download from: https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-spark-runtime-3.5_2.12/1.8.1/" - exit 1 -fi -if [ -z "$ICEBERG_WAREHOUSE" ]; then - echo "Error: ICEBERG_WAREHOUSE is not set" - exit 1 -fi -if [ -z "$TPCH_QUERIES" ]; then - echo "Error: TPCH_QUERIES is not set" - exit 1 -fi - -$SPARK_HOME/sbin/stop-master.sh 2>/dev/null || true -$SPARK_HOME/sbin/stop-worker.sh 2>/dev/null || true - -$SPARK_HOME/sbin/start-master.sh -$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER - -$SPARK_HOME/bin/spark-submit \ - --master $SPARK_MASTER \ - --jars $COMET_JAR,$ICEBERG_JAR \ - --driver-class-path $COMET_JAR:$ICEBERG_JAR \ - --conf spark.driver.memory=8G \ - --conf spark.executor.instances=1 \ - --conf spark.executor.cores=8 \ - --conf spark.cores.max=8 \ - --conf spark.executor.memory=16g \ - --conf spark.memory.offHeap.enabled=true \ - --conf spark.memory.offHeap.size=16g \ - --conf spark.eventLog.enabled=true \ - --conf spark.driver.extraClassPath=$COMET_JAR:$ICEBERG_JAR \ - --conf spark.executor.extraClassPath=$COMET_JAR:$ICEBERG_JAR \ - --conf spark.plugins=org.apache.spark.CometPlugin \ - --conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \ - --conf spark.comet.exec.replaceSortMergeJoin=true \ - --conf spark.comet.expression.Cast.allowIncompatible=true \ - --conf spark.comet.enabled=true \ - --conf spark.comet.exec.enabled=true \ - --conf spark.comet.scan.icebergNative.enabled=true \ - --conf spark.comet.explainFallback.enabled=true \ - --conf spark.sql.catalog.${ICEBERG_CATALOG}=org.apache.iceberg.spark.SparkCatalog \ - --conf spark.sql.catalog.${ICEBERG_CATALOG}.type=hadoop \ - --conf spark.sql.catalog.${ICEBERG_CATALOG}.warehouse=$ICEBERG_WAREHOUSE \ - --conf spark.sql.defaultCatalog=${ICEBERG_CATALOG} \ - tpcbench.py \ - --name comet-iceberg \ - --benchmark tpch \ - --catalog $ICEBERG_CATALOG \ - --database $ICEBERG_DATABASE \ - --queries $TPCH_QUERIES \ - --output . \ - --iterations 1 diff --git a/dev/benchmarks/comet-tpch.sh b/dev/benchmarks/comet-tpch.sh deleted file mode 100755 index a748a02319..0000000000 --- a/dev/benchmarks/comet-tpch.sh +++ /dev/null @@ -1,55 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -# - -$SPARK_HOME/sbin/stop-master.sh -$SPARK_HOME/sbin/stop-worker.sh - -$SPARK_HOME/sbin/start-master.sh -$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER - -$SPARK_HOME/bin/spark-submit \ - --master $SPARK_MASTER \ - --jars $COMET_JAR \ - --driver-class-path $COMET_JAR \ - --conf spark.driver.memory=8G \ - --conf spark.executor.instances=1 \ - --conf spark.executor.cores=8 \ - --conf spark.cores.max=8 \ - --conf spark.executor.memory=16g \ - --conf spark.memory.offHeap.enabled=true \ - --conf spark.memory.offHeap.size=16g \ - --conf spark.eventLog.enabled=true \ - --conf spark.driver.extraClassPath=$COMET_JAR \ - --conf spark.executor.extraClassPath=$COMET_JAR \ - --conf spark.plugins=org.apache.spark.CometPlugin \ - --conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \ - --conf spark.comet.scan.impl=native_datafusion \ - --conf spark.comet.exec.replaceSortMergeJoin=true \ - --conf spark.comet.expression.Cast.allowIncompatible=true \ - --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ - --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ - tpcbench.py \ - --name comet \ - --benchmark tpch \ - --data $TPCH_DATA \ - --queries $TPCH_QUERIES \ - --output . \ - --iterations 1 \ - --format parquet diff --git a/dev/benchmarks/gluten-tpcds.sh b/dev/benchmarks/gluten-tpcds.sh deleted file mode 100755 index 7c475c79c0..0000000000 --- a/dev/benchmarks/gluten-tpcds.sh +++ /dev/null @@ -1,53 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -# -export TZ=UTC - -$SPARK_HOME/sbin/stop-master.sh -$SPARK_HOME/sbin/stop-worker.sh - -$SPARK_HOME/sbin/start-master.sh -$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER - -$SPARK_HOME/bin/spark-submit \ - --master $SPARK_MASTER \ - --conf spark.driver.memory=8G \ - --conf spark.executor.instances=2 \ - --conf spark.executor.memory=16G \ - --conf spark.executor.cores=8 \ - --conf spark.cores.max=16 \ - --conf spark.eventLog.enabled=true \ - --jars $GLUTEN_JAR \ - --conf spark.plugins=org.apache.gluten.GlutenPlugin \ - --conf spark.driver.extraClassPath=${GLUTEN_JAR} \ - --conf spark.executor.extraClassPath=${GLUTEN_JAR} \ - --conf spark.memory.offHeap.enabled=true \ - --conf spark.memory.offHeap.size=16g \ - --conf spark.gluten.sql.columnar.forceShuffledHashJoin=true \ - --conf spark.shuffle.manager=org.apache.spark.shuffle.sort.ColumnarShuffleManager \ - --conf spark.sql.session.timeZone=UTC \ - --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ - --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ - tpcbench.py \ - --name gluten \ - --benchmark tpcds \ - --data $TPCDS_DATA \ - --queries $TPCDS_QUERIES \ - --output . \ - --iterations 1 diff --git a/dev/benchmarks/gluten-tpch.sh b/dev/benchmarks/gluten-tpch.sh deleted file mode 100755 index 46c3ed7527..0000000000 --- a/dev/benchmarks/gluten-tpch.sh +++ /dev/null @@ -1,53 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -# -export TZ=UTC - -$SPARK_HOME/sbin/stop-master.sh -$SPARK_HOME/sbin/stop-worker.sh - -$SPARK_HOME/sbin/start-master.sh -$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER - -$SPARK_HOME/bin/spark-submit \ - --master $SPARK_MASTER \ - --conf spark.driver.memory=8G \ - --conf spark.executor.instances=1 \ - --conf spark.executor.memory=16G \ - --conf spark.executor.cores=8 \ - --conf spark.cores.max=8 \ - --conf spark.eventLog.enabled=true \ - --jars $GLUTEN_JAR \ - --conf spark.plugins=org.apache.gluten.GlutenPlugin \ - --conf spark.driver.extraClassPath=${GLUTEN_JAR} \ - --conf spark.executor.extraClassPath=${GLUTEN_JAR} \ - --conf spark.memory.offHeap.enabled=true \ - --conf spark.memory.offHeap.size=16g \ - --conf spark.gluten.sql.columnar.forceShuffledHashJoin=true \ - --conf spark.shuffle.manager=org.apache.spark.shuffle.sort.ColumnarShuffleManager \ - --conf spark.sql.session.timeZone=UTC \ - --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ - --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ - tpcbench.py \ - --name gluten \ - --benchmark tpch \ - --data $TPCH_DATA \ - --queries $TPCH_QUERIES \ - --output . \ - --iterations 1 diff --git a/dev/benchmarks/spark-tpcds.sh b/dev/benchmarks/spark-tpcds.sh deleted file mode 100755 index dad079ba23..0000000000 --- a/dev/benchmarks/spark-tpcds.sh +++ /dev/null @@ -1,45 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -# - -$SPARK_HOME/sbin/stop-master.sh -$SPARK_HOME/sbin/stop-worker.sh - -$SPARK_HOME/sbin/start-master.sh -$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER - -$SPARK_HOME/bin/spark-submit \ - --master $SPARK_MASTER \ - --conf spark.driver.memory=8G \ - --conf spark.executor.instances=2 \ - --conf spark.executor.cores=8 \ - --conf spark.cores.max=16 \ - --conf spark.executor.memory=16g \ - --conf spark.memory.offHeap.enabled=true \ - --conf spark.memory.offHeap.size=16g \ - --conf spark.eventLog.enabled=true \ - --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ - --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ - tpcbench.py \ - --name spark \ - --benchmark tpcds \ - --data $TPCDS_DATA \ - --queries $TPCDS_QUERIES \ - --output . \ - --iterations 1 diff --git a/dev/benchmarks/spark-tpch.sh b/dev/benchmarks/spark-tpch.sh deleted file mode 100755 index ae359f049f..0000000000 --- a/dev/benchmarks/spark-tpch.sh +++ /dev/null @@ -1,46 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -# - -$SPARK_HOME/sbin/stop-master.sh -$SPARK_HOME/sbin/stop-worker.sh - -$SPARK_HOME/sbin/start-master.sh -$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER - -$SPARK_HOME/bin/spark-submit \ - --master $SPARK_MASTER \ - --conf spark.driver.memory=8G \ - --conf spark.executor.instances=1 \ - --conf spark.executor.cores=8 \ - --conf spark.cores.max=8 \ - --conf spark.executor.memory=16g \ - --conf spark.memory.offHeap.enabled=true \ - --conf spark.memory.offHeap.size=16g \ - --conf spark.eventLog.enabled=true \ - --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ - --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ - tpcbench.py \ - --name spark \ - --benchmark tpch \ - --data $TPCH_DATA \ - --queries $TPCH_QUERIES \ - --output . \ - --iterations 1 \ - --format parquet diff --git a/docs/source/about/gluten_comparison.md b/docs/source/about/gluten_comparison.md index 492479bb9d..40c6c2741a 100644 --- a/docs/source/about/gluten_comparison.md +++ b/docs/source/about/gluten_comparison.md @@ -86,7 +86,7 @@ on your existing Spark jobs. ![tpch_allqueries_comet_gluten.png](/_static/images/tpch_allqueries_comet_gluten.png) -The scripts that were used to generate these results can be found [here](https://github.com/apache/datafusion-comet/tree/main/dev/benchmarks). +The scripts that were used to generate these results can be found [here](https://github.com/apache/datafusion-comet/tree/main/benchmarks/tpc). ## Ease of Development & Contributing diff --git a/docs/source/contributor-guide/benchmark-results/tpc-ds.md b/docs/source/contributor-guide/benchmark-results/tpc-ds.md index 66ff2e51a7..bea254cc08 100644 --- a/docs/source/contributor-guide/benchmark-results/tpc-ds.md +++ b/docs/source/contributor-guide/benchmark-results/tpc-ds.md @@ -46,4 +46,4 @@ The raw results of these benchmarks in JSON format is available here: - [Spark](spark-3.5.3-tpcds.json) - [Comet](comet-0.11.0-tpcds.json) -The scripts that were used to generate these results can be found [here](https://github.com/apache/datafusion-comet/tree/main/dev/benchmarks). +The scripts that were used to generate these results can be found [here](https://github.com/apache/datafusion-comet/tree/main/benchmarks/tpc). diff --git a/docs/source/contributor-guide/benchmark-results/tpc-h.md b/docs/source/contributor-guide/benchmark-results/tpc-h.md index 4424d9eac7..2170426c05 100644 --- a/docs/source/contributor-guide/benchmark-results/tpc-h.md +++ b/docs/source/contributor-guide/benchmark-results/tpc-h.md @@ -46,4 +46,4 @@ The raw results of these benchmarks in JSON format is available here: - [Spark](spark-3.5.3-tpch.json) - [Comet](comet-0.11.0-tpch.json) -The scripts that were used to generate these results can be found [here](https://github.com/apache/datafusion-comet/tree/main/dev/benchmarks). +The scripts that were used to generate these results can be found [here](https://github.com/apache/datafusion-comet/tree/main/benchmarks/tpc). diff --git a/docs/source/contributor-guide/benchmarking.md b/docs/source/contributor-guide/benchmarking.md index 768089c955..ce98fc4be6 100644 --- a/docs/source/contributor-guide/benchmarking.md +++ b/docs/source/contributor-guide/benchmarking.md @@ -21,7 +21,7 @@ under the License. To track progress on performance, we regularly run benchmarks derived from TPC-H and TPC-DS. -The benchmarking scripts are contained at [https://github.com/apache/datafusion-comet/tree/main/dev/benchmarks](https://github.com/apache/datafusion-comet/tree/main/dev/benchmarks). +The benchmarking scripts are contained at [https://github.com/apache/datafusion-comet/tree/main/benchmarks/tpc](https://github.com/apache/datafusion-comet/tree/main/benchmarks/tpc). Data generation scripts are available in the [DataFusion Benchmarks](https://github.com/apache/datafusion-benchmarks) GitHub repository. diff --git a/docs/source/contributor-guide/benchmarking_aws_ec2.md b/docs/source/contributor-guide/benchmarking_aws_ec2.md index 922b0379fe..81f15d64ea 100644 --- a/docs/source/contributor-guide/benchmarking_aws_ec2.md +++ b/docs/source/contributor-guide/benchmarking_aws_ec2.md @@ -109,23 +109,23 @@ export COMET_JAR=/home/ec2-user/datafusion-comet/spark/target/comet-spark-spark3 ## Run Benchmarks -Use the scripts in `dev/benchmarks` in the Comet repository. +Use the scripts in `benchmarks/tpc` in the Comet repository. ```shell -cd dev/benchmarks +cd benchmarks/tpc export TPCH_QUERIES=/home/ec2-user/datafusion-benchmarks/tpch/queries/ ``` Run Spark benchmark: ```shell -./spark-tpch.sh +python3 run.py --engine spark --benchmark tpch ``` Run Comet benchmark: ```shell -./comet-tpch.sh +python3 run.py --engine comet --benchmark tpch ``` ## Running Benchmarks with S3 @@ -164,4 +164,9 @@ Modify the scripts to add the following configurations. --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ ``` -Now run the `spark-tpch.sh` and `comet-tpch.sh` scripts. +Now run the benchmarks: + +```shell +python3 run.py --engine spark --benchmark tpch +python3 run.py --engine comet --benchmark tpch +``` From 93b9069d0d3c3274f5b8ee2c32e9d0465b9caa1b Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 16 Feb 2026 15:10:10 -0700 Subject: [PATCH 02/24] Add TPC-DS support for Iceberg benchmarking - Rename create-iceberg-tpch.py to create-iceberg-tables.py with --benchmark flag supporting both tpch and tpcds table sets - Remove hardcoded TPCH_QUERIES from comet-iceberg.toml required env vars - Remove hardcoded ICEBERG_DATABASE default of "tpch" from comet-iceberg.toml - Add check_benchmark_env() in run.py to validate benchmark-specific env vars and default ICEBERG_DATABASE to the benchmark name - Update README with TPC-DS Iceberg table creation examples Co-Authored-By: Claude Opus 4.6 --- benchmarks/tpc/README.md | 30 ++++- benchmarks/tpc/create-iceberg-tables.py | 150 ++++++++++++++++++++++ benchmarks/tpc/create-iceberg-tpch.py | 88 ------------- benchmarks/tpc/engines/comet-iceberg.toml | 3 +- benchmarks/tpc/run.py | 24 ++++ 5 files changed, 199 insertions(+), 96 deletions(-) create mode 100644 benchmarks/tpc/create-iceberg-tables.py delete mode 100644 benchmarks/tpc/create-iceberg-tpch.py diff --git a/benchmarks/tpc/README.md b/benchmarks/tpc/README.md index 48df9c63d1..36752668bd 100644 --- a/benchmarks/tpc/README.md +++ b/benchmarks/tpc/README.md @@ -121,14 +121,15 @@ export ICEBERG_JAR=/path/to/iceberg-spark-runtime-3.5_2.12-1.8.1.jar Note: Table creation uses `--packages` which auto-downloads the dependency. -### Create Iceberg TPC-H tables +### Create Iceberg tables -Convert existing Parquet TPC-H data to Iceberg format: +Convert existing Parquet data to Iceberg format using `create-iceberg-tables.py`: ```shell export ICEBERG_WAREHOUSE=/mnt/bigdata/iceberg-warehouse export ICEBERG_CATALOG=${ICEBERG_CATALOG:-local} +# TPC-H $SPARK_HOME/bin/spark-submit \ --master $SPARK_MASTER \ --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.8.1 \ @@ -140,10 +141,27 @@ $SPARK_HOME/bin/spark-submit \ --conf spark.sql.catalog.${ICEBERG_CATALOG}=org.apache.iceberg.spark.SparkCatalog \ --conf spark.sql.catalog.${ICEBERG_CATALOG}.type=hadoop \ --conf spark.sql.catalog.${ICEBERG_CATALOG}.warehouse=$ICEBERG_WAREHOUSE \ - create-iceberg-tpch.py \ + create-iceberg-tables.py \ + --benchmark tpch \ --parquet-path $TPCH_DATA \ - --catalog $ICEBERG_CATALOG \ - --database tpch + --catalog $ICEBERG_CATALOG + +# TPC-DS +$SPARK_HOME/bin/spark-submit \ + --master $SPARK_MASTER \ + --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.8.1 \ + --conf spark.driver.memory=8G \ + --conf spark.executor.instances=2 \ + --conf spark.executor.cores=8 \ + --conf spark.cores.max=16 \ + --conf spark.executor.memory=16g \ + --conf spark.sql.catalog.${ICEBERG_CATALOG}=org.apache.iceberg.spark.SparkCatalog \ + --conf spark.sql.catalog.${ICEBERG_CATALOG}.type=hadoop \ + --conf spark.sql.catalog.${ICEBERG_CATALOG}.warehouse=$ICEBERG_WAREHOUSE \ + create-iceberg-tables.py \ + --benchmark tpcds \ + --parquet-path $TPCDS_DATA \ + --catalog $ICEBERG_CATALOG ``` ### Run Iceberg benchmark @@ -167,7 +185,7 @@ physical plan output. | Environment Variable | Default | Description | | -------------------- | ---------- | ----------------------------------- | | `ICEBERG_CATALOG` | `local` | Iceberg catalog name | -| `ICEBERG_DATABASE` | `tpch` | Database containing TPC-H tables | +| `ICEBERG_DATABASE` | `tpch` | Database containing TPC tables | | `ICEBERG_WAREHOUSE` | (required) | Path to Iceberg warehouse directory | ### Comparing Parquet vs Iceberg performance diff --git a/benchmarks/tpc/create-iceberg-tables.py b/benchmarks/tpc/create-iceberg-tables.py new file mode 100644 index 0000000000..56002fe3b5 --- /dev/null +++ b/benchmarks/tpc/create-iceberg-tables.py @@ -0,0 +1,150 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +""" +Convert TPC-H or TPC-DS Parquet data to Iceberg tables. + +Usage: + spark-submit \ + --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.8.1 \ + --conf spark.sql.catalog.local=org.apache.iceberg.spark.SparkCatalog \ + --conf spark.sql.catalog.local.type=hadoop \ + --conf spark.sql.catalog.local.warehouse=/path/to/iceberg-warehouse \ + create-iceberg-tables.py \ + --benchmark tpch \ + --parquet-path /path/to/tpch/parquet \ + --catalog local \ + --database tpch + + spark-submit \ + --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.8.1 \ + --conf spark.sql.catalog.local=org.apache.iceberg.spark.SparkCatalog \ + --conf spark.sql.catalog.local.type=hadoop \ + --conf spark.sql.catalog.local.warehouse=/path/to/iceberg-warehouse \ + create-iceberg-tables.py \ + --benchmark tpcds \ + --parquet-path /path/to/tpcds/parquet \ + --catalog local \ + --database tpcds +""" + +import argparse +from pyspark.sql import SparkSession +import time + +TPCH_TABLES = [ + "customer", + "lineitem", + "nation", + "orders", + "part", + "partsupp", + "region", + "supplier", +] + +TPCDS_TABLES = [ + "call_center", + "catalog_page", + "catalog_returns", + "catalog_sales", + "customer", + "customer_address", + "customer_demographics", + "date_dim", + "time_dim", + "household_demographics", + "income_band", + "inventory", + "item", + "promotion", + "reason", + "ship_mode", + "store", + "store_returns", + "store_sales", + "warehouse", + "web_page", + "web_returns", + "web_sales", + "web_site", +] + +BENCHMARK_TABLES = { + "tpch": TPCH_TABLES, + "tpcds": TPCDS_TABLES, +} + + +def main(benchmark: str, parquet_path: str, catalog: str, database: str): + table_names = BENCHMARK_TABLES[benchmark] + + spark = SparkSession.builder \ + .appName(f"Create Iceberg {benchmark.upper()} Tables") \ + .getOrCreate() + + # Create database if it doesn't exist + spark.sql(f"CREATE DATABASE IF NOT EXISTS {catalog}.{database}") + + for table in table_names: + parquet_table_path = f"{parquet_path}/{table}.parquet" + iceberg_table = f"{catalog}.{database}.{table}" + + print(f"Converting {parquet_table_path} -> {iceberg_table}") + start_time = time.time() + + # Drop table if exists to allow re-running + spark.sql(f"DROP TABLE IF EXISTS {iceberg_table}") + + # Read parquet and write as Iceberg + df = spark.read.parquet(parquet_table_path) + df.writeTo(iceberg_table).using("iceberg").create() + + row_count = spark.table(iceberg_table).count() + elapsed = time.time() - start_time + print(f" Created {iceberg_table} with {row_count} rows in {elapsed:.2f}s") + + print(f"\nAll {benchmark.upper()} tables created successfully!") + print(f"Tables available at: {catalog}.{database}.*") + + spark.stop() + + +if __name__ == "__main__": + parser = argparse.ArgumentParser( + description="Convert TPC-H or TPC-DS Parquet data to Iceberg tables" + ) + parser.add_argument( + "--benchmark", required=True, choices=["tpch", "tpcds"], + help="Benchmark whose tables to convert (tpch or tpcds)" + ) + parser.add_argument( + "--parquet-path", required=True, + help="Path to Parquet data directory" + ) + parser.add_argument( + "--catalog", required=True, + help="Iceberg catalog name (e.g., 'local')" + ) + parser.add_argument( + "--database", default=None, + help="Database name to create tables in (defaults to benchmark name)" + ) + args = parser.parse_args() + + database = args.database if args.database else args.benchmark + main(args.benchmark, args.parquet_path, args.catalog, database) diff --git a/benchmarks/tpc/create-iceberg-tpch.py b/benchmarks/tpc/create-iceberg-tpch.py deleted file mode 100644 index 44f0f63a2e..0000000000 --- a/benchmarks/tpc/create-iceberg-tpch.py +++ /dev/null @@ -1,88 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -""" -Convert TPC-H Parquet data to Iceberg tables. - -Usage: - spark-submit \ - --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.8.1 \ - --conf spark.sql.catalog.local=org.apache.iceberg.spark.SparkCatalog \ - --conf spark.sql.catalog.local.type=hadoop \ - --conf spark.sql.catalog.local.warehouse=/path/to/iceberg-warehouse \ - create-iceberg-tpch.py \ - --parquet-path /path/to/tpch/parquet \ - --catalog local \ - --database tpch -""" - -import argparse -from pyspark.sql import SparkSession -import time - - -def main(parquet_path: str, catalog: str, database: str): - spark = SparkSession.builder \ - .appName("Create Iceberg TPC-H Tables") \ - .getOrCreate() - - table_names = [ - "customer", - "lineitem", - "nation", - "orders", - "part", - "partsupp", - "region", - "supplier" - ] - - # Create database if it doesn't exist - spark.sql(f"CREATE DATABASE IF NOT EXISTS {catalog}.{database}") - - for table in table_names: - parquet_table_path = f"{parquet_path}/{table}.parquet" - iceberg_table = f"{catalog}.{database}.{table}" - - print(f"Converting {parquet_table_path} -> {iceberg_table}") - start_time = time.time() - - # Drop table if exists to allow re-running - spark.sql(f"DROP TABLE IF EXISTS {iceberg_table}") - - # Read parquet and write as Iceberg - df = spark.read.parquet(parquet_table_path) - df.writeTo(iceberg_table).using("iceberg").create() - - row_count = spark.table(iceberg_table).count() - elapsed = time.time() - start_time - print(f" Created {iceberg_table} with {row_count} rows in {elapsed:.2f}s") - - print("\nAll TPC-H tables created successfully!") - print(f"Tables available at: {catalog}.{database}.*") - - spark.stop() - - -if __name__ == "__main__": - parser = argparse.ArgumentParser(description="Convert TPC-H Parquet data to Iceberg tables") - parser.add_argument("--parquet-path", required=True, help="Path to TPC-H Parquet data directory") - parser.add_argument("--catalog", required=True, help="Iceberg catalog name (e.g., 'local')") - parser.add_argument("--database", default="tpch", help="Database name to create tables in") - args = parser.parse_args() - - main(args.parquet_path, args.catalog, args.database) diff --git a/benchmarks/tpc/engines/comet-iceberg.toml b/benchmarks/tpc/engines/comet-iceberg.toml index ebabfce7b4..2e01270f13 100644 --- a/benchmarks/tpc/engines/comet-iceberg.toml +++ b/benchmarks/tpc/engines/comet-iceberg.toml @@ -19,11 +19,10 @@ name = "comet-iceberg" [env] -required = ["COMET_JAR", "ICEBERG_JAR", "ICEBERG_WAREHOUSE", "TPCH_QUERIES"] +required = ["COMET_JAR", "ICEBERG_JAR", "ICEBERG_WAREHOUSE"] [env.defaults] ICEBERG_CATALOG = "local" -ICEBERG_DATABASE = "tpch" [spark_submit] jars = ["$COMET_JAR", "$ICEBERG_JAR"] diff --git a/benchmarks/tpc/run.py b/benchmarks/tpc/run.py index 122f90f821..d2c65ce012 100755 --- a/benchmarks/tpc/run.py +++ b/benchmarks/tpc/run.py @@ -208,6 +208,29 @@ def check_common_env(): sys.exit(1) +def check_benchmark_env(config, benchmark): + """Validate benchmark-specific environment variables.""" + profile = BENCHMARK_PROFILES[benchmark] + use_iceberg = config.get("tpcbench_args", {}).get("use_iceberg", False) + + required = [profile["queries_env"]] + if not use_iceberg: + required.append(profile["data_env"]) + + missing = [v for v in required if not os.environ.get(v)] + if missing: + print( + f"Error: Required environment variable(s) not set for " + f"{benchmark}: {', '.join(missing)}", + file=sys.stderr, + ) + sys.exit(1) + + # Default ICEBERG_DATABASE to the benchmark name if not already set + if use_iceberg and not os.environ.get("ICEBERG_DATABASE"): + os.environ["ICEBERG_DATABASE"] = benchmark + + def build_spark_submit_cmd(config, benchmark, args): """Build the spark-submit command list.""" spark_home = os.environ["SPARK_HOME"] @@ -349,6 +372,7 @@ def main(): check_common_env() check_required_env(config) + check_benchmark_env(config, args.benchmark) # Restart Spark unless --no-restart or --dry-run if not args.no_restart and not args.dry_run: From 49be49442ff3688ecc7506ec9d8416e2bfc997b8 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 16 Feb 2026 15:12:24 -0700 Subject: [PATCH 03/24] Remove blaze engine configuration and references Co-Authored-By: Claude Opus 4.6 --- benchmarks/tpc/README.md | 2 +- benchmarks/tpc/engines/blaze.toml | 36 ------------------------------- benchmarks/tpc/run.py | 2 +- 3 files changed, 2 insertions(+), 38 deletions(-) delete mode 100644 benchmarks/tpc/engines/blaze.toml diff --git a/benchmarks/tpc/README.md b/benchmarks/tpc/README.md index 36752668bd..374f240523 100644 --- a/benchmarks/tpc/README.md +++ b/benchmarks/tpc/README.md @@ -44,7 +44,7 @@ python3 run.py --engine --benchmark [options] | `--no-restart` | Skip Spark master/worker restart | | `--dry-run` | Print the spark-submit command without executing | -Available engines: `spark`, `comet`, `comet-iceberg`, `gluten`, `blaze` +Available engines: `spark`, `comet`, `comet-iceberg`, `gluten` ## Example usage diff --git a/benchmarks/tpc/engines/blaze.toml b/benchmarks/tpc/engines/blaze.toml deleted file mode 100644 index 7298a8f3ab..0000000000 --- a/benchmarks/tpc/engines/blaze.toml +++ /dev/null @@ -1,36 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -[engine] -name = "blaze" - -[env] -required = ["BLAZE_JAR"] - -[spark_submit] -jars = ["$BLAZE_JAR"] -driver_class_path = ["$BLAZE_JAR"] - -[spark_conf] -"spark.driver.extraClassPath" = "$BLAZE_JAR" -"spark.executor.extraClassPath" = "$BLAZE_JAR" -"spark.sql.extensions" = "org.apache.spark.sql.blaze.BlazeSparkSessionExtension" -"spark.shuffle.manager" = "org.apache.spark.sql.execution.blaze.shuffle.BlazeShuffleManager" -"spark.blaze.enable" = "true" -"spark.blaze.forceShuffledHashJoin" = "true" -"spark.executor.memoryOverhead" = "16g" -"spark.memory.offHeap.enabled" = "false" diff --git a/benchmarks/tpc/run.py b/benchmarks/tpc/run.py index d2c65ce012..41a2d5fdaf 100755 --- a/benchmarks/tpc/run.py +++ b/benchmarks/tpc/run.py @@ -21,7 +21,7 @@ Usage: python3 run.py --engine comet --benchmark tpch - python3 run.py --engine blaze --benchmark tpcds --iterations 3 + python3 run.py --engine comet --benchmark tpcds --iterations 3 python3 run.py --engine comet-iceberg --benchmark tpch python3 run.py --engine comet --benchmark tpch --dry-run python3 run.py --engine spark --benchmark tpch --no-restart From 233f919ba1a2a90fdf9cca84717d8466eee286c5 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 16 Feb 2026 15:24:16 -0700 Subject: [PATCH 04/24] Format README.md with prettier Co-Authored-By: Claude Opus 4.6 --- benchmarks/tpc/README.md | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/benchmarks/tpc/README.md b/benchmarks/tpc/README.md index 374f240523..fbdb64fef6 100644 --- a/benchmarks/tpc/README.md +++ b/benchmarks/tpc/README.md @@ -34,15 +34,15 @@ All benchmarks are run via `run.py`: python3 run.py --engine --benchmark [options] ``` -| Option | Description | -| --------------- | ---------------------------------------- | -| `--engine` | Engine name (matches a TOML file in `engines/`) | -| `--benchmark` | `tpch` or `tpcds` | -| `--iterations` | Number of iterations (default: 1) | -| `--output` | Output directory (default: `.`) | -| `--query` | Run a single query number | -| `--no-restart` | Skip Spark master/worker restart | -| `--dry-run` | Print the spark-submit command without executing | +| Option | Description | +| -------------- | ------------------------------------------------ | +| `--engine` | Engine name (matches a TOML file in `engines/`) | +| `--benchmark` | `tpch` or `tpcds` | +| `--iterations` | Number of iterations (default: 1) | +| `--output` | Output directory (default: `.`) | +| `--query` | Run a single query number | +| `--no-restart` | Skip Spark master/worker restart | +| `--dry-run` | Print the spark-submit command without executing | Available engines: `spark`, `comet`, `comet-iceberg`, `gluten` From fe0308bb85cafd26b56956b91a52eafe0de12a73 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 17 Feb 2026 06:22:07 -0700 Subject: [PATCH 05/24] Move Iceberg catalog config into create-iceberg-tables.py The script now configures the Iceberg catalog via SparkSession.builder instead of requiring --conf flags on the spark-submit command line. This adds --warehouse as a required CLI arg, makes --catalog optional (default: local), and validates paths with clear error messages before starting Spark. Co-Authored-By: Claude Opus 4.6 --- benchmarks/tpc/README.md | 29 ++++++-------- benchmarks/tpc/create-iceberg-tables.py | 53 +++++++++++++++++-------- 2 files changed, 50 insertions(+), 32 deletions(-) diff --git a/benchmarks/tpc/README.md b/benchmarks/tpc/README.md index fbdb64fef6..e2b0e8bcde 100644 --- a/benchmarks/tpc/README.md +++ b/benchmarks/tpc/README.md @@ -123,11 +123,12 @@ Note: Table creation uses `--packages` which auto-downloads the dependency. ### Create Iceberg tables -Convert existing Parquet data to Iceberg format using `create-iceberg-tables.py`: +Convert existing Parquet data to Iceberg format using `create-iceberg-tables.py`. +The script configures the Iceberg catalog automatically -- no `--conf` flags needed. ```shell export ICEBERG_WAREHOUSE=/mnt/bigdata/iceberg-warehouse -export ICEBERG_CATALOG=${ICEBERG_CATALOG:-local} +mkdir -p $ICEBERG_WAREHOUSE # TPC-H $SPARK_HOME/bin/spark-submit \ @@ -138,13 +139,10 @@ $SPARK_HOME/bin/spark-submit \ --conf spark.executor.cores=8 \ --conf spark.cores.max=8 \ --conf spark.executor.memory=16g \ - --conf spark.sql.catalog.${ICEBERG_CATALOG}=org.apache.iceberg.spark.SparkCatalog \ - --conf spark.sql.catalog.${ICEBERG_CATALOG}.type=hadoop \ - --conf spark.sql.catalog.${ICEBERG_CATALOG}.warehouse=$ICEBERG_WAREHOUSE \ create-iceberg-tables.py \ --benchmark tpch \ --parquet-path $TPCH_DATA \ - --catalog $ICEBERG_CATALOG + --warehouse $ICEBERG_WAREHOUSE # TPC-DS $SPARK_HOME/bin/spark-submit \ @@ -155,13 +153,10 @@ $SPARK_HOME/bin/spark-submit \ --conf spark.executor.cores=8 \ --conf spark.cores.max=16 \ --conf spark.executor.memory=16g \ - --conf spark.sql.catalog.${ICEBERG_CATALOG}=org.apache.iceberg.spark.SparkCatalog \ - --conf spark.sql.catalog.${ICEBERG_CATALOG}.type=hadoop \ - --conf spark.sql.catalog.${ICEBERG_CATALOG}.warehouse=$ICEBERG_WAREHOUSE \ create-iceberg-tables.py \ --benchmark tpcds \ --parquet-path $TPCDS_DATA \ - --catalog $ICEBERG_CATALOG + --warehouse $ICEBERG_WAREHOUSE ``` ### Run Iceberg benchmark @@ -180,13 +175,15 @@ The benchmark uses `spark.comet.scan.icebergNative.enabled=true` to enable Comet integration. Verify native scanning is active by checking for `CometIcebergNativeScanExec` in the physical plan output. -### Iceberg-specific options +### create-iceberg-tables.py options -| Environment Variable | Default | Description | -| -------------------- | ---------- | ----------------------------------- | -| `ICEBERG_CATALOG` | `local` | Iceberg catalog name | -| `ICEBERG_DATABASE` | `tpch` | Database containing TPC tables | -| `ICEBERG_WAREHOUSE` | (required) | Path to Iceberg warehouse directory | +| Option | Required | Default | Description | +| ---------------- | -------- | ---------------- | ----------------------------------- | +| `--benchmark` | Yes | | `tpch` or `tpcds` | +| `--parquet-path` | Yes | | Path to source Parquet data | +| `--warehouse` | Yes | | Path to Iceberg warehouse directory | +| `--catalog` | No | `local` | Iceberg catalog name | +| `--database` | No | benchmark name | Database name for the tables | ### Comparing Parquet vs Iceberg performance diff --git a/benchmarks/tpc/create-iceberg-tables.py b/benchmarks/tpc/create-iceberg-tables.py index 56002fe3b5..219969bda7 100644 --- a/benchmarks/tpc/create-iceberg-tables.py +++ b/benchmarks/tpc/create-iceberg-tables.py @@ -21,28 +21,22 @@ Usage: spark-submit \ --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.8.1 \ - --conf spark.sql.catalog.local=org.apache.iceberg.spark.SparkCatalog \ - --conf spark.sql.catalog.local.type=hadoop \ - --conf spark.sql.catalog.local.warehouse=/path/to/iceberg-warehouse \ create-iceberg-tables.py \ --benchmark tpch \ --parquet-path /path/to/tpch/parquet \ - --catalog local \ - --database tpch + --warehouse /path/to/iceberg-warehouse spark-submit \ --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.8.1 \ - --conf spark.sql.catalog.local=org.apache.iceberg.spark.SparkCatalog \ - --conf spark.sql.catalog.local.type=hadoop \ - --conf spark.sql.catalog.local.warehouse=/path/to/iceberg-warehouse \ create-iceberg-tables.py \ --benchmark tpcds \ --parquet-path /path/to/tpcds/parquet \ - --catalog local \ - --database tpcds + --warehouse /path/to/iceberg-warehouse """ import argparse +import os +import sys from pyspark.sql import SparkSession import time @@ -90,15 +84,38 @@ } -def main(benchmark: str, parquet_path: str, catalog: str, database: str): +def main(benchmark: str, parquet_path: str, warehouse: str, catalog: str, database: str): table_names = BENCHMARK_TABLES[benchmark] + # Validate paths before starting Spark + errors = [] + if not os.path.isdir(parquet_path): + errors.append(f"Error: --parquet-path '{parquet_path}' does not exist or is not a directory") + if not os.path.isdir(warehouse): + errors.append(f"Error: --warehouse '{warehouse}' does not exist or is not a directory. " + "Create it with: mkdir -p " + warehouse) + if errors: + for e in errors: + print(e, file=sys.stderr) + sys.exit(1) + spark = SparkSession.builder \ .appName(f"Create Iceberg {benchmark.upper()} Tables") \ + .config(f"spark.sql.catalog.{catalog}", "org.apache.iceberg.spark.SparkCatalog") \ + .config(f"spark.sql.catalog.{catalog}.type", "hadoop") \ + .config(f"spark.sql.catalog.{catalog}.warehouse", warehouse) \ .getOrCreate() - # Create database if it doesn't exist - spark.sql(f"CREATE DATABASE IF NOT EXISTS {catalog}.{database}") + # Set the Iceberg catalog as the current catalog so that + # namespace operations are routed correctly + spark.sql(f"USE {catalog}") + + # Create namespace if it doesn't exist + try: + spark.sql(f"CREATE NAMESPACE IF NOT EXISTS {database}") + except Exception: + # Namespace may already exist + pass for table in table_names: parquet_table_path = f"{parquet_path}/{table}.parquet" @@ -137,8 +154,12 @@ def main(benchmark: str, parquet_path: str, catalog: str, database: str): help="Path to Parquet data directory" ) parser.add_argument( - "--catalog", required=True, - help="Iceberg catalog name (e.g., 'local')" + "--warehouse", required=True, + help="Path to Iceberg warehouse directory" + ) + parser.add_argument( + "--catalog", default="local", + help="Iceberg catalog name (default: 'local')" ) parser.add_argument( "--database", default=None, @@ -147,4 +168,4 @@ def main(benchmark: str, parquet_path: str, catalog: str, database: str): args = parser.parse_args() database = args.database if args.database else args.benchmark - main(args.benchmark, args.parquet_path, args.catalog, database) + main(args.benchmark, args.parquet_path, args.warehouse, args.catalog, database) From b4f2af02901de1cec7b72a6bd5e7c627633a75ec Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 17 Feb 2026 06:33:25 -0700 Subject: [PATCH 06/24] prettier --- benchmarks/tpc/README.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/benchmarks/tpc/README.md b/benchmarks/tpc/README.md index e2b0e8bcde..779ad1753a 100644 --- a/benchmarks/tpc/README.md +++ b/benchmarks/tpc/README.md @@ -177,13 +177,13 @@ physical plan output. ### create-iceberg-tables.py options -| Option | Required | Default | Description | -| ---------------- | -------- | ---------------- | ----------------------------------- | -| `--benchmark` | Yes | | `tpch` or `tpcds` | -| `--parquet-path` | Yes | | Path to source Parquet data | -| `--warehouse` | Yes | | Path to Iceberg warehouse directory | -| `--catalog` | No | `local` | Iceberg catalog name | -| `--database` | No | benchmark name | Database name for the tables | +| Option | Required | Default | Description | +| ---------------- | -------- | -------------- | ----------------------------------- | +| `--benchmark` | Yes | | `tpch` or `tpcds` | +| `--parquet-path` | Yes | | Path to source Parquet data | +| `--warehouse` | Yes | | Path to Iceberg warehouse directory | +| `--catalog` | No | `local` | Iceberg catalog name | +| `--database` | No | benchmark name | Database name for the tables | ### Comparing Parquet vs Iceberg performance From 3b9ea4c05f53c1417c0f0f464ff665b4d3fcc228 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 16 Feb 2026 15:21:08 -0700 Subject: [PATCH 07/24] Add Docker Compose support for TPC benchmarks Provides a containerized Spark standalone cluster for running TPC-H and TPC-DS benchmarks. Includes a Dockerfile with Java 8+17 support, a three-service Compose file (master, worker, bench runner), a memory-constrained overlay with cgroup metrics collection, and README documentation. Co-Authored-By: Claude Opus 4.6 --- benchmarks/tpc/README.md | 59 ++++++++++ benchmarks/tpc/infra/docker/Dockerfile | 55 ++++++++++ .../tpc/infra/docker/collect-metrics.sh | 103 ++++++++++++++++++ .../docker/docker-compose.constrained.yml | 48 ++++++++ .../tpc/infra/docker/docker-compose.yml | 92 ++++++++++++++++ 5 files changed, 357 insertions(+) create mode 100644 benchmarks/tpc/infra/docker/Dockerfile create mode 100755 benchmarks/tpc/infra/docker/collect-metrics.sh create mode 100644 benchmarks/tpc/infra/docker/docker-compose.constrained.yml create mode 100644 benchmarks/tpc/infra/docker/docker-compose.yml diff --git a/benchmarks/tpc/README.md b/benchmarks/tpc/README.md index 779ad1753a..2ce95e1172 100644 --- a/benchmarks/tpc/README.md +++ b/benchmarks/tpc/README.md @@ -185,6 +185,65 @@ physical plan output. | `--catalog` | No | `local` | Iceberg catalog name | | `--database` | No | benchmark name | Database name for the tables | +## Running with Docker + +A Docker Compose setup is provided in `infra/docker/` for running benchmarks in an isolated +Spark standalone cluster. + +### Build the image + +From the repository root: + +```shell +docker build -t comet-bench -f benchmarks/tpc/infra/docker/Dockerfile . +``` + +### Start the cluster + +Set environment variables pointing to your host paths, then start the Spark master and worker: + +```shell +export DATA_DIR=/mnt/bigdata/tpch/sf100 +export QUERIES_DIR=/mnt/bigdata/tpch/queries +export RESULTS_DIR=/tmp/bench-results +export ENGINE_JARS_DIR=/opt/engine-jars + +docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml up -d +``` + +### Run benchmarks + +Use `docker compose run` to execute benchmarks. Pass `--no-restart` since the cluster is +already managed by Compose: + +```shell +docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml \ + run bench python3 /opt/benchmarks/run.py \ + --engine comet --benchmark tpch --no-restart +``` + +For Gluten (requires Java 8), override `JAVA_HOME`: + +```shell +docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml \ + run -e JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64 \ + bench python3 /opt/benchmarks/run.py \ + --engine gluten --benchmark tpch --no-restart +``` + +### Memory-constrained benchmarks + +Apply the constrained overlay to enforce hard memory limits and collect cgroup metrics: + +```shell +docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml \ + -f benchmarks/tpc/infra/docker/docker-compose.constrained.yml up -d +``` + +Metrics are written to `$RESULTS_DIR/container-metrics.csv`. Configure limits via environment +variables: `WORKER_MEM_LIMIT` (default: 6g), `BENCH_MEM_LIMIT` (default: 10g), +`METRICS_INTERVAL` (default: 1 second). + ### Comparing Parquet vs Iceberg performance Run both benchmarks and compare: diff --git a/benchmarks/tpc/infra/docker/Dockerfile b/benchmarks/tpc/infra/docker/Dockerfile new file mode 100644 index 0000000000..cc87ddcaa2 --- /dev/null +++ b/benchmarks/tpc/infra/docker/Dockerfile @@ -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. + +# Benchmark image for running TPC-H and TPC-DS benchmarks across engines +# (Spark, Comet, Gluten). +# +# Build (from repository root): +# docker build -t comet-bench -f benchmarks/tpc/infra/docker/Dockerfile . + +ARG SPARK_IMAGE=apache/spark:3.5.2-python3 +FROM ${SPARK_IMAGE} + +USER root + +# Install Java 8 (Gluten) and Java 17 (Comet) plus Python 3. +RUN apt-get update \ + && apt-get install -y --no-install-recommends \ + openjdk-8-jdk-headless \ + openjdk-17-jdk-headless \ + python3 python3-pip procps \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* + +# Default to Java 17 (override with JAVA_HOME at runtime for Gluten). +ENV JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64 + +# Copy the benchmark scripts into the image. +COPY benchmarks/tpc/run.py /opt/benchmarks/run.py +COPY benchmarks/tpc/tpcbench.py /opt/benchmarks/tpcbench.py +COPY benchmarks/tpc/engines /opt/benchmarks/engines +COPY benchmarks/tpc/create-iceberg-tables.py /opt/benchmarks/create-iceberg-tables.py +COPY benchmarks/tpc/generate-comparison.py /opt/benchmarks/generate-comparison.py + +# Copy the metrics collector script. +COPY benchmarks/tpc/infra/docker/collect-metrics.sh /opt/benchmarks/collect-metrics.sh +RUN chmod +x /opt/benchmarks/collect-metrics.sh + +# Engine JARs are bind-mounted or copied in at runtime via --jars. +# Data and query paths are also bind-mounted. + +WORKDIR /opt/benchmarks + +USER ${spark_uid} diff --git a/benchmarks/tpc/infra/docker/collect-metrics.sh b/benchmarks/tpc/infra/docker/collect-metrics.sh new file mode 100755 index 0000000000..fd9c1d848f --- /dev/null +++ b/benchmarks/tpc/infra/docker/collect-metrics.sh @@ -0,0 +1,103 @@ +#!/bin/sh +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Container-level memory metrics collector. +# +# Polls cgroup memory stats at a fixed interval and writes a CSV with +# columns: timestamp, memory_usage_bytes, memory_limit_bytes, rss_bytes, +# cache_bytes, swap_bytes. +# +# Works with both cgroup v1 and v2. +# +# Usage: +# collect-metrics.sh [INTERVAL_SECS] [OUTPUT_CSV] +# +# Defaults: interval=1, output=/results/container-metrics.csv + +set -e + +INTERVAL="${1:-1}" +OUTPUT="${2:-/results/container-metrics.csv}" + +# Detect cgroup version +if [ -f /sys/fs/cgroup/memory/memory.usage_in_bytes ]; then + CGROUP_VERSION=1 +elif [ -f /sys/fs/cgroup/memory.current ]; then + CGROUP_VERSION=2 +else + echo "Warning: cannot detect cgroup memory files; polling disabled" >&2 + # Still write a header so downstream tools don't break on a missing file. + echo "timestamp_ms,memory_usage_bytes,memory_limit_bytes,rss_bytes,cache_bytes,swap_bytes" > "$OUTPUT" + # Sleep forever so the container stays up (compose expects it to keep running). + exec sleep infinity +fi + +# ---- helpers ---- + +read_file() { + # Return the contents of a file, or "0" if it doesn't exist. + if [ -f "$1" ]; then cat "$1"; else echo "0"; fi +} + +read_stat() { + # Extract a named field from memory.stat (cgroup v1 format: "key value"). + grep "^$1 " "$2" 2>/dev/null | awk '{print $2}' || echo "0" +} + +poll_v1() { + local usage limit rss cache swap + usage=$(read_file /sys/fs/cgroup/memory/memory.usage_in_bytes) + limit=$(read_file /sys/fs/cgroup/memory/memory.limit_in_bytes) + local stat=/sys/fs/cgroup/memory/memory.stat + rss=$(read_stat total_rss "$stat") + cache=$(read_stat total_cache "$stat") + swap=$(read_file /sys/fs/cgroup/memory/memory.memsw.usage_in_bytes) + # swap file reports memory+swap; subtract memory to get swap only + if [ "$swap" != "0" ]; then + swap=$((swap - usage)) + [ "$swap" -lt 0 ] && swap=0 + fi + echo "$usage,$limit,$rss,$cache,$swap" +} + +poll_v2() { + local usage limit rss cache swap + usage=$(read_file /sys/fs/cgroup/memory.current) + limit=$(read_file /sys/fs/cgroup/memory.max) + [ "$limit" = "max" ] && limit=0 + local stat=/sys/fs/cgroup/memory.stat + rss=$(read_stat anon "$stat") + cache=$(read_stat file "$stat") + swap=$(read_file /sys/fs/cgroup/memory.swap.current) + echo "$usage,$limit,$rss,$cache,$swap" +} + +# ---- main loop ---- + +echo "timestamp_ms,memory_usage_bytes,memory_limit_bytes,rss_bytes,cache_bytes,swap_bytes" > "$OUTPUT" +echo "Collecting container memory metrics every ${INTERVAL}s -> ${OUTPUT} (cgroup v${CGROUP_VERSION})" >&2 + +while true; do + ts=$(date +%s%3N 2>/dev/null || python3 -c 'import time; print(int(time.time()*1000))') + if [ "$CGROUP_VERSION" = "1" ]; then + vals=$(poll_v1) + else + vals=$(poll_v2) + fi + echo "${ts},${vals}" >> "$OUTPUT" + sleep "$INTERVAL" +done diff --git a/benchmarks/tpc/infra/docker/docker-compose.constrained.yml b/benchmarks/tpc/infra/docker/docker-compose.constrained.yml new file mode 100644 index 0000000000..eff730d0e3 --- /dev/null +++ b/benchmarks/tpc/infra/docker/docker-compose.constrained.yml @@ -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. + +# Constrained memory overlay. +# +# Apply on top of docker-compose.yml to enforce hard memory limits and +# enable the metrics-collector sidecar: +# +# docker compose -f docker-compose.yml -f docker-compose.constrained.yml up -d +# +# Environment variables: +# WORKER_MEM_LIMIT - Hard memory limit for the worker (default: 6g) +# BENCH_MEM_LIMIT - Hard memory limit for the bench runner (default: 10g) +# METRICS_INTERVAL - Collection interval in seconds (default: 1) + +services: + spark-worker: + mem_limit: ${WORKER_MEM_LIMIT:-6g} + memswap_limit: ${WORKER_MEM_LIMIT:-6g} # same as mem_limit → no swap + + bench: + mem_limit: ${BENCH_MEM_LIMIT:-10g} + memswap_limit: ${BENCH_MEM_LIMIT:-10g} + + metrics-collector: + image: ${BENCH_IMAGE:-comet-bench} + container_name: metrics-collector + pid: "service:spark-worker" # share PID namespace with worker + command: + - /opt/benchmarks/collect-metrics.sh + - "${METRICS_INTERVAL:-1}" + - /results/container-metrics.csv + volumes: + - ${RESULTS_DIR:-/tmp/bench-results}:/results + depends_on: + - spark-worker diff --git a/benchmarks/tpc/infra/docker/docker-compose.yml b/benchmarks/tpc/infra/docker/docker-compose.yml new file mode 100644 index 0000000000..2309d1c4d3 --- /dev/null +++ b/benchmarks/tpc/infra/docker/docker-compose.yml @@ -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. + +# Spark standalone cluster for TPC benchmarks. +# +# Usage: +# docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml up -d +# +# Override with constrained memory limits: +# docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml \ +# -f benchmarks/tpc/infra/docker/docker-compose.constrained.yml up -d +# +# Environment variables (set in .env or export before running): +# BENCH_IMAGE - Docker image to use (default: comet-bench) +# DATA_DIR - Host path to TPC data (default: /tmp/tpc-data) +# QUERIES_DIR - Host path to query SQL files (default: /tmp/tpc-queries) +# RESULTS_DIR - Host path for results output (default: /tmp/bench-results) +# ENGINE_JARS_DIR - Host path containing engine JARs (default: /tmp/engine-jars) + +services: + spark-master: + image: ${BENCH_IMAGE:-comet-bench} + container_name: spark-master + hostname: spark-master + command: /opt/spark/sbin/start-master.sh --host spark-master + ports: + - "7077:7077" + - "8080:8080" + volumes: + - ${DATA_DIR:-/tmp/tpc-data}:/data:ro + - ${QUERIES_DIR:-/tmp/tpc-queries}:/queries:ro + - ${RESULTS_DIR:-/tmp/bench-results}:/results + - ${ENGINE_JARS_DIR:-/tmp/engine-jars}:/jars:ro + environment: + - SPARK_MASTER_HOST=spark-master + - SPARK_NO_DAEMONIZE=true + + spark-worker: + image: ${BENCH_IMAGE:-comet-bench} + container_name: spark-worker + hostname: spark-worker + depends_on: + - spark-master + command: /opt/spark/sbin/start-worker.sh spark://spark-master:7077 + ports: + - "8081:8081" + volumes: + - ${DATA_DIR:-/tmp/tpc-data}:/data:ro + - ${QUERIES_DIR:-/tmp/tpc-queries}:/queries:ro + - ${RESULTS_DIR:-/tmp/bench-results}:/results + - ${ENGINE_JARS_DIR:-/tmp/engine-jars}:/jars:ro + environment: + - SPARK_WORKER_CORES=${WORKER_CORES:-8} + - SPARK_WORKER_MEMORY=${WORKER_MEMORY:-16g} + - SPARK_NO_DAEMONIZE=true + + bench: + image: ${BENCH_IMAGE:-comet-bench} + container_name: bench-runner + depends_on: + - spark-master + - spark-worker + # Override 'command' to run a specific benchmark, e.g.: + # docker compose run bench python3 /opt/benchmarks/run.py \ + # --engine comet --benchmark tpch --no-restart + command: ["echo", "Use 'docker compose run bench python3 /opt/benchmarks/run.py ...' to run benchmarks"] + volumes: + - ${DATA_DIR:-/tmp/tpc-data}:/data:ro + - ${QUERIES_DIR:-/tmp/tpc-queries}:/queries:ro + - ${RESULTS_DIR:-/tmp/bench-results}:/results + - ${ENGINE_JARS_DIR:-/tmp/engine-jars}:/jars:ro + environment: + - SPARK_HOME=/opt/spark + - SPARK_MASTER=spark://spark-master:7077 + - COMET_JAR=/jars/comet.jar + - GLUTEN_JAR=/jars/gluten.jar + - TPCH_DATA=/data + - TPCH_QUERIES=/queries + - TPCDS_DATA=/data + - TPCDS_QUERIES=/queries From 1cad3be5001adf922b17fef3e7e726f72e6e22ec Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 16 Feb 2026 15:42:43 -0700 Subject: [PATCH 08/24] Add Spark REST API executor metrics profiling Ports SparkMetricsProfiler from unified-benchmark-runner branch to collect executor memory metrics via the Spark REST API during benchmark runs. Co-Authored-By: Claude Opus 4.6 --- benchmarks/tpc/README.md | 20 ++- benchmarks/tpc/infra/docker/Dockerfile | 1 + benchmarks/tpc/profiling.py | 179 +++++++++++++++++++++++++ benchmarks/tpc/run.py | 15 +++ benchmarks/tpc/tpcbench.py | 27 +++- 5 files changed, 238 insertions(+), 4 deletions(-) create mode 100644 benchmarks/tpc/profiling.py diff --git a/benchmarks/tpc/README.md b/benchmarks/tpc/README.md index 2ce95e1172..91feddc081 100644 --- a/benchmarks/tpc/README.md +++ b/benchmarks/tpc/README.md @@ -41,8 +41,10 @@ python3 run.py --engine --benchmark [options] | `--iterations` | Number of iterations (default: 1) | | `--output` | Output directory (default: `.`) | | `--query` | Run a single query number | -| `--no-restart` | Skip Spark master/worker restart | -| `--dry-run` | Print the spark-submit command without executing | +| `--no-restart` | Skip Spark master/worker restart | +| `--dry-run` | Print the spark-submit command without executing | +| `--profile` | Enable executor metrics profiling via Spark REST API | +| `--profile-interval` | Profiling poll interval in seconds (default: 2.0) | Available engines: `spark`, `comet`, `comet-iceberg`, `gluten` @@ -100,6 +102,20 @@ Generating charts: python3 generate-comparison.py --benchmark tpch --labels "Spark 3.5.3" "Comet 0.9.0" "Gluten 1.4.0" --title "TPC-H @ 100 GB (single executor, 8 cores, local Parquet files)" spark-tpch-1752338506381.json comet-tpch-1752337818039.json gluten-tpch-1752337474344.json ``` +## Profiling + +Use `--profile` to collect executor memory metrics from the Spark REST API during the benchmark run. +A background thread polls `/api/v1/applications/{appId}/executors` at a configurable interval and +writes the time-series data to a CSV file alongside the benchmark results. + +```shell +python3 run.py --engine comet --benchmark tpch --profile +python3 run.py --engine comet --benchmark tpch --profile --profile-interval 1.0 +``` + +The output CSV is written to `{output}/{name}-{benchmark}-metrics.csv` and contains per-executor +columns including `memoryUsed`, `maxMemory`, heap/off-heap storage metrics, and peak memory metrics. + ## Engine Configuration Each engine is defined by a TOML file in `engines/`. The config specifies JARs, Spark conf overrides, diff --git a/benchmarks/tpc/infra/docker/Dockerfile b/benchmarks/tpc/infra/docker/Dockerfile index cc87ddcaa2..dd12445bdd 100644 --- a/benchmarks/tpc/infra/docker/Dockerfile +++ b/benchmarks/tpc/infra/docker/Dockerfile @@ -39,6 +39,7 @@ ENV JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64 # Copy the benchmark scripts into the image. COPY benchmarks/tpc/run.py /opt/benchmarks/run.py COPY benchmarks/tpc/tpcbench.py /opt/benchmarks/tpcbench.py +COPY benchmarks/tpc/profiling.py /opt/benchmarks/profiling.py COPY benchmarks/tpc/engines /opt/benchmarks/engines COPY benchmarks/tpc/create-iceberg-tables.py /opt/benchmarks/create-iceberg-tables.py COPY benchmarks/tpc/generate-comparison.py /opt/benchmarks/generate-comparison.py diff --git a/benchmarks/tpc/profiling.py b/benchmarks/tpc/profiling.py new file mode 100644 index 0000000000..98aa56d493 --- /dev/null +++ b/benchmarks/tpc/profiling.py @@ -0,0 +1,179 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +""" +Level 1 profiling hooks: JVM metrics via the Spark REST API. + +Polls ``/api/v1/applications/{appId}/executors`` at a configurable interval +and records executor memory metrics as a time-series CSV alongside the +benchmark results. + +Usage:: + + profiler = SparkMetricsProfiler(spark, interval_secs=2) + profiler.start() + # ... run benchmark ... + profiler.stop() + profiler.write_csv("/path/to/output/metrics.csv") +""" + +import csv +import threading +import time +from typing import Any, Dict, List, Optional + +from pyspark.sql import SparkSession + +try: + from urllib.request import urlopen + import json as _json + + def _fetch_json(url: str) -> Any: + with urlopen(url, timeout=5) as resp: + return _json.loads(resp.read().decode()) +except ImportError: + _fetch_json = None # type: ignore[assignment] + + +# Metrics we extract per executor from the REST API response +_EXECUTOR_METRICS = [ + "memoryUsed", + "maxMemory", + "totalOnHeapStorageMemory", + "usedOnHeapStorageMemory", + "totalOffHeapStorageMemory", + "usedOffHeapStorageMemory", +] + +# Metrics nested under peakMemoryMetrics (if available) +_PEAK_MEMORY_METRICS = [ + "JVMHeapMemory", + "JVMOffHeapMemory", + "OnHeapExecutionMemory", + "OffHeapExecutionMemory", + "OnHeapStorageMemory", + "OffHeapStorageMemory", + "OnHeapUnifiedMemory", + "OffHeapUnifiedMemory", + "ProcessTreeJVMRSSMemory", +] + + +class SparkMetricsProfiler: + """Periodically polls executor metrics from the Spark REST API.""" + + def __init__( + self, + spark: SparkSession, + interval_secs: float = 2.0, + ): + self._spark = spark + self._interval = interval_secs + self._samples: List[Dict[str, Any]] = [] + self._stop_event = threading.Event() + self._thread: Optional[threading.Thread] = None + self._start_time: float = 0.0 + + @property + def samples(self) -> List[Dict[str, Any]]: + """Return collected samples (each is a flat dict).""" + return list(self._samples) + + def _ui_url(self) -> Optional[str]: + """Return the Spark UI base URL, or None if unavailable.""" + url = self._spark.sparkContext.uiWebUrl + if url: + return url.rstrip("/") + return None + + def _app_id(self) -> str: + return self._spark.sparkContext.applicationId + + def _poll_once(self) -> None: + """Fetch executor metrics and append a timestamped sample.""" + base = self._ui_url() + if base is None or _fetch_json is None: + return + + url = f"{base}/api/v1/applications/{self._app_id()}/executors" + try: + executors = _fetch_json(url) + except Exception: + return + + elapsed = time.time() - self._start_time + for exc in executors: + row: Dict[str, Any] = { + "elapsed_secs": round(elapsed, 2), + "executor_id": exc.get("id", ""), + "is_active": exc.get("isActive", True), + } + for key in _EXECUTOR_METRICS: + row[key] = exc.get(key, 0) + + peak = exc.get("peakMemoryMetrics", {}) + for key in _PEAK_MEMORY_METRICS: + row[f"peak_{key}"] = peak.get(key, 0) + + self._samples.append(row) + + def _run(self) -> None: + """Background polling loop.""" + while not self._stop_event.is_set(): + self._poll_once() + self._stop_event.wait(self._interval) + + def start(self) -> None: + """Start background polling thread.""" + if self._thread is not None: + return + self._start_time = time.time() + self._stop_event.clear() + self._thread = threading.Thread( + target=self._run, name="spark-metrics-profiler", daemon=True + ) + self._thread.start() + print( + f"Profiler started (interval={self._interval}s, " + f"ui={self._ui_url()})" + ) + + def stop(self) -> None: + """Stop the polling thread and collect a final sample.""" + if self._thread is None: + return + self._stop_event.set() + self._thread.join(timeout=self._interval + 2) + self._thread = None + # One last poll to capture final state + self._poll_once() + print(f"Profiler stopped ({len(self._samples)} samples collected)") + + def write_csv(self, path: str) -> str: + """Write collected samples to a CSV file. Returns the path.""" + if not self._samples: + print("Profiler: no samples to write") + return path + + fieldnames = list(self._samples[0].keys()) + with open(path, "w", newline="") as f: + writer = csv.DictWriter(f, fieldnames=fieldnames) + writer.writeheader() + for row in self._samples: + writer.writerow(row) + print(f"Profiler: wrote {len(self._samples)} samples to {path}") + return path diff --git a/benchmarks/tpc/run.py b/benchmarks/tpc/run.py index 41a2d5fdaf..205615915e 100755 --- a/benchmarks/tpc/run.py +++ b/benchmarks/tpc/run.py @@ -295,6 +295,10 @@ def build_spark_submit_cmd(config, benchmark, args): if profile["format"] and not use_iceberg: cmd += ["--format", profile["format"]] + if args.profile: + cmd += ["--profile"] + cmd += ["--profile-interval", str(args.profile_interval)] + return cmd @@ -362,6 +366,17 @@ def main(): action="store_true", help="Print the spark-submit command without executing", ) + parser.add_argument( + "--profile", + action="store_true", + help="Enable executor metrics profiling via Spark REST API", + ) + parser.add_argument( + "--profile-interval", + type=float, + default=2.0, + help="Profiling poll interval in seconds (default: 2.0)", + ) args = parser.parse_args() config = load_engine_config(args.engine) diff --git a/benchmarks/tpc/tpcbench.py b/benchmarks/tpc/tpcbench.py index 400ccd175a..9be64e2e78 100644 --- a/benchmarks/tpc/tpcbench.py +++ b/benchmarks/tpc/tpcbench.py @@ -57,7 +57,9 @@ def main( format: str, query_num: int = None, write_path: str = None, - options: Dict[str, str] = None + options: Dict[str, str] = None, + profile: bool = False, + profile_interval: float = 2.0, ): if options is None: options = {} @@ -66,6 +68,12 @@ def main( .appName(f"{name} benchmark derived from {benchmark}") \ .getOrCreate() + profiler = None + if profile: + from profiling import SparkMetricsProfiler + profiler = SparkMetricsProfiler(spark, interval_secs=profile_interval) + profiler.start() + # Define tables for each benchmark if benchmark == "tpch": num_queries = 22 @@ -176,6 +184,11 @@ def main( with open(results_path, "w") as f: f.write(result_str) + if profiler is not None: + profiler.stop() + metrics_path = f"{output}/{name}-{benchmark}-metrics.csv" + profiler.write_csv(metrics_path) + spark.stop() @@ -239,6 +252,14 @@ def main( "--write", help="Path to save query results as Parquet" ) + parser.add_argument( + "--profile", action="store_true", + help="Enable executor metrics profiling via Spark REST API" + ) + parser.add_argument( + "--profile-interval", type=float, default=2.0, + help="Profiling poll interval in seconds (default: 2.0)" + ) args = parser.parse_args() main( @@ -253,5 +274,7 @@ def main( args.format, args.query, args.write, - args.options + args.options, + args.profile, + args.profile_interval, ) From 1cea3b68fe80ab5551be2eba0988a70d060f2c31 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 17 Feb 2026 10:37:41 -0700 Subject: [PATCH 09/24] Use 2-worker Docker Compose topology with constrained memory Use a fixed 2-worker setup so shuffles go through the network stack, better reflecting real cluster behavior. Merge the constrained memory overlay into the main compose file and use YAML anchors to avoid duplication. Update TPC-H to use 2 executors to match. Co-Authored-By: Claude Opus 4.6 --- benchmarks/tpc/README.md | 25 +++--- .../docker/docker-compose.constrained.yml | 48 ---------- .../tpc/infra/docker/docker-compose.yml | 89 ++++++++++++++----- benchmarks/tpc/run.py | 4 +- 4 files changed, 81 insertions(+), 85 deletions(-) delete mode 100644 benchmarks/tpc/infra/docker/docker-compose.constrained.yml diff --git a/benchmarks/tpc/README.md b/benchmarks/tpc/README.md index 91feddc081..db04cb6c8e 100644 --- a/benchmarks/tpc/README.md +++ b/benchmarks/tpc/README.md @@ -151,9 +151,9 @@ $SPARK_HOME/bin/spark-submit \ --master $SPARK_MASTER \ --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.8.1 \ --conf spark.driver.memory=8G \ - --conf spark.executor.instances=1 \ + --conf spark.executor.instances=2 \ --conf spark.executor.cores=8 \ - --conf spark.cores.max=8 \ + --conf spark.cores.max=16 \ --conf spark.executor.memory=16g \ create-iceberg-tables.py \ --benchmark tpch \ @@ -216,13 +216,14 @@ docker build -t comet-bench -f benchmarks/tpc/infra/docker/Dockerfile . ### Start the cluster -Set environment variables pointing to your host paths, then start the Spark master and worker: +Set environment variables pointing to your host paths, then start the Spark master and +two workers: ```shell export DATA_DIR=/mnt/bigdata/tpch/sf100 export QUERIES_DIR=/mnt/bigdata/tpch/queries export RESULTS_DIR=/tmp/bench-results -export ENGINE_JARS_DIR=/opt/engine-jars +export ENGINE_JARS_DIR=/opt/comet docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml up -d ``` @@ -247,19 +248,15 @@ docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml \ --engine gluten --benchmark tpch --no-restart ``` -### Memory-constrained benchmarks - -Apply the constrained overlay to enforce hard memory limits and collect cgroup metrics: - -```shell -docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml \ - -f benchmarks/tpc/infra/docker/docker-compose.constrained.yml up -d -``` +### Memory limits and metrics -Metrics are written to `$RESULTS_DIR/container-metrics.csv`. Configure limits via environment -variables: `WORKER_MEM_LIMIT` (default: 6g), `BENCH_MEM_LIMIT` (default: 10g), +Hard memory limits are enforced on all worker and bench containers. A metrics-collector +sidecar runs alongside each worker to collect cgroup metrics. Configure via environment +variables: `WORKER_MEM_LIMIT` (default: 32g per worker), `BENCH_MEM_LIMIT` (default: 10g), `METRICS_INTERVAL` (default: 1 second). +Metrics are written to `$RESULTS_DIR/container-metrics-worker-{1,2}.csv`. + ### Comparing Parquet vs Iceberg performance Run both benchmarks and compare: diff --git a/benchmarks/tpc/infra/docker/docker-compose.constrained.yml b/benchmarks/tpc/infra/docker/docker-compose.constrained.yml deleted file mode 100644 index eff730d0e3..0000000000 --- a/benchmarks/tpc/infra/docker/docker-compose.constrained.yml +++ /dev/null @@ -1,48 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -# Constrained memory overlay. -# -# Apply on top of docker-compose.yml to enforce hard memory limits and -# enable the metrics-collector sidecar: -# -# docker compose -f docker-compose.yml -f docker-compose.constrained.yml up -d -# -# Environment variables: -# WORKER_MEM_LIMIT - Hard memory limit for the worker (default: 6g) -# BENCH_MEM_LIMIT - Hard memory limit for the bench runner (default: 10g) -# METRICS_INTERVAL - Collection interval in seconds (default: 1) - -services: - spark-worker: - mem_limit: ${WORKER_MEM_LIMIT:-6g} - memswap_limit: ${WORKER_MEM_LIMIT:-6g} # same as mem_limit → no swap - - bench: - mem_limit: ${BENCH_MEM_LIMIT:-10g} - memswap_limit: ${BENCH_MEM_LIMIT:-10g} - - metrics-collector: - image: ${BENCH_IMAGE:-comet-bench} - container_name: metrics-collector - pid: "service:spark-worker" # share PID namespace with worker - command: - - /opt/benchmarks/collect-metrics.sh - - "${METRICS_INTERVAL:-1}" - - /results/container-metrics.csv - volumes: - - ${RESULTS_DIR:-/tmp/bench-results}:/results - depends_on: - - spark-worker diff --git a/benchmarks/tpc/infra/docker/docker-compose.yml b/benchmarks/tpc/infra/docker/docker-compose.yml index 2309d1c4d3..359b46e345 100644 --- a/benchmarks/tpc/infra/docker/docker-compose.yml +++ b/benchmarks/tpc/infra/docker/docker-compose.yml @@ -15,19 +15,46 @@ # Spark standalone cluster for TPC benchmarks. # +# Two workers are used so that shuffles go through the network stack, +# which better reflects real cluster behavior. +# # Usage: # docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml up -d # -# Override with constrained memory limits: -# docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml \ -# -f benchmarks/tpc/infra/docker/docker-compose.constrained.yml up -d -# # Environment variables (set in .env or export before running): # BENCH_IMAGE - Docker image to use (default: comet-bench) # DATA_DIR - Host path to TPC data (default: /tmp/tpc-data) # QUERIES_DIR - Host path to query SQL files (default: /tmp/tpc-queries) # RESULTS_DIR - Host path for results output (default: /tmp/bench-results) # ENGINE_JARS_DIR - Host path containing engine JARs (default: /tmp/engine-jars) +# WORKER_MEM_LIMIT - Hard memory limit per worker container (default: 32g) +# BENCH_MEM_LIMIT - Hard memory limit for the bench runner (default: 10g) +# METRICS_INTERVAL - Metrics collection interval in seconds (default: 1) + +x-worker: &worker + image: ${BENCH_IMAGE:-comet-bench} + depends_on: + - spark-master + command: /opt/spark/sbin/start-worker.sh spark://spark-master:7077 + volumes: + - ${DATA_DIR:-/tmp/tpc-data}:/data:ro + - ${QUERIES_DIR:-/tmp/tpc-queries}:/queries:ro + - ${RESULTS_DIR:-/tmp/bench-results}:/results + - ${ENGINE_JARS_DIR:-/tmp/engine-jars}:/jars:ro + environment: + - SPARK_WORKER_CORES=${WORKER_CORES:-8} + - SPARK_WORKER_MEMORY=${WORKER_MEMORY:-16g} + - SPARK_NO_DAEMONIZE=true + mem_limit: ${WORKER_MEM_LIMIT:-32g} + memswap_limit: ${WORKER_MEM_LIMIT:-32g} + +x-metrics-collector: &metrics-collector + image: ${BENCH_IMAGE:-comet-bench} + command: + - /opt/benchmarks/collect-metrics.sh + - "${METRICS_INTERVAL:-1}" + volumes: + - ${RESULTS_DIR:-/tmp/bench-results}:/results services: spark-master: @@ -47,31 +74,27 @@ services: - SPARK_MASTER_HOST=spark-master - SPARK_NO_DAEMONIZE=true - spark-worker: - image: ${BENCH_IMAGE:-comet-bench} - container_name: spark-worker - hostname: spark-worker - depends_on: - - spark-master - command: /opt/spark/sbin/start-worker.sh spark://spark-master:7077 + spark-worker-1: + <<: *worker + container_name: spark-worker-1 + hostname: spark-worker-1 ports: - "8081:8081" - volumes: - - ${DATA_DIR:-/tmp/tpc-data}:/data:ro - - ${QUERIES_DIR:-/tmp/tpc-queries}:/queries:ro - - ${RESULTS_DIR:-/tmp/bench-results}:/results - - ${ENGINE_JARS_DIR:-/tmp/engine-jars}:/jars:ro - environment: - - SPARK_WORKER_CORES=${WORKER_CORES:-8} - - SPARK_WORKER_MEMORY=${WORKER_MEMORY:-16g} - - SPARK_NO_DAEMONIZE=true + + spark-worker-2: + <<: *worker + container_name: spark-worker-2 + hostname: spark-worker-2 + ports: + - "8082:8081" bench: image: ${BENCH_IMAGE:-comet-bench} container_name: bench-runner depends_on: - spark-master - - spark-worker + - spark-worker-1 + - spark-worker-2 # Override 'command' to run a specific benchmark, e.g.: # docker compose run bench python3 /opt/benchmarks/run.py \ # --engine comet --benchmark tpch --no-restart @@ -90,3 +113,27 @@ services: - TPCH_QUERIES=/queries - TPCDS_DATA=/data - TPCDS_QUERIES=/queries + mem_limit: ${BENCH_MEM_LIMIT:-10g} + memswap_limit: ${BENCH_MEM_LIMIT:-10g} + + metrics-collector-1: + <<: *metrics-collector + container_name: metrics-collector-1 + pid: "service:spark-worker-1" + command: + - /opt/benchmarks/collect-metrics.sh + - "${METRICS_INTERVAL:-1}" + - /results/container-metrics-worker-1.csv + depends_on: + - spark-worker-1 + + metrics-collector-2: + <<: *metrics-collector + container_name: metrics-collector-2 + pid: "service:spark-worker-2" + command: + - /opt/benchmarks/collect-metrics.sh + - "${METRICS_INTERVAL:-1}" + - /results/container-metrics-worker-2.csv + depends_on: + - spark-worker-2 diff --git a/benchmarks/tpc/run.py b/benchmarks/tpc/run.py index 205615915e..7f76ccfed0 100755 --- a/benchmarks/tpc/run.py +++ b/benchmarks/tpc/run.py @@ -120,9 +120,9 @@ def load_toml(path): BENCHMARK_PROFILES = { "tpch": { - "executor_instances": "1", + "executor_instances": "2", "executor_cores": "8", - "max_cores": "8", + "max_cores": "16", "data_env": "TPCH_DATA", "queries_env": "TPCH_QUERIES", "format": "parquet", From d88eddad3f363817552d62c169fe2bf3c6ed9436 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 17 Feb 2026 10:40:19 -0700 Subject: [PATCH 10/24] Mount engine JARs individually instead of a shared directory Replace ENGINE_JARS_DIR with individual COMET_JAR, GLUTEN_JAR, and ICEBERG_JAR env vars pointing to host paths. Each JAR is mounted into the container at a fixed path, making it easy to switch between JAR versions by changing the path and restarting. Co-Authored-By: Claude Opus 4.6 --- benchmarks/tpc/README.md | 6 +++- .../tpc/infra/docker/docker-compose.yml | 32 +++++++++---------- 2 files changed, 21 insertions(+), 17 deletions(-) diff --git a/benchmarks/tpc/README.md b/benchmarks/tpc/README.md index db04cb6c8e..02f7bd354c 100644 --- a/benchmarks/tpc/README.md +++ b/benchmarks/tpc/README.md @@ -223,11 +223,15 @@ two workers: export DATA_DIR=/mnt/bigdata/tpch/sf100 export QUERIES_DIR=/mnt/bigdata/tpch/queries export RESULTS_DIR=/tmp/bench-results -export ENGINE_JARS_DIR=/opt/comet +export COMET_JAR=/opt/comet/comet-spark-spark3.5_2.12-0.10.0.jar docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml up -d ``` +Set `COMET_JAR`, `GLUTEN_JAR`, or `ICEBERG_JAR` to the host path of the engine JAR you +want to use. Each JAR is mounted individually into the container, so you can easily switch +between versions by changing the path and restarting. + ### Run benchmarks Use `docker compose run` to execute benchmarks. Pass `--no-restart` since the cluster is diff --git a/benchmarks/tpc/infra/docker/docker-compose.yml b/benchmarks/tpc/infra/docker/docker-compose.yml index 359b46e345..67dd0c7260 100644 --- a/benchmarks/tpc/infra/docker/docker-compose.yml +++ b/benchmarks/tpc/infra/docker/docker-compose.yml @@ -19,6 +19,7 @@ # which better reflects real cluster behavior. # # Usage: +# export COMET_JAR=/path/to/comet-spark-0.10.0.jar # docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml up -d # # Environment variables (set in .env or export before running): @@ -26,21 +27,27 @@ # DATA_DIR - Host path to TPC data (default: /tmp/tpc-data) # QUERIES_DIR - Host path to query SQL files (default: /tmp/tpc-queries) # RESULTS_DIR - Host path for results output (default: /tmp/bench-results) -# ENGINE_JARS_DIR - Host path containing engine JARs (default: /tmp/engine-jars) +# COMET_JAR - Host path to Comet JAR +# GLUTEN_JAR - Host path to Gluten JAR +# ICEBERG_JAR - Host path to Iceberg Spark runtime JAR # WORKER_MEM_LIMIT - Hard memory limit per worker container (default: 32g) # BENCH_MEM_LIMIT - Hard memory limit for the bench runner (default: 10g) # METRICS_INTERVAL - Metrics collection interval in seconds (default: 1) +x-volumes: &volumes + - ${DATA_DIR:-/tmp/tpc-data}:/data:ro + - ${QUERIES_DIR:-/tmp/tpc-queries}:/queries:ro + - ${RESULTS_DIR:-/tmp/bench-results}:/results + - ${COMET_JAR:-/dev/null}:/jars/comet.jar:ro + - ${GLUTEN_JAR:-/dev/null}:/jars/gluten.jar:ro + - ${ICEBERG_JAR:-/dev/null}:/jars/iceberg.jar:ro + x-worker: &worker image: ${BENCH_IMAGE:-comet-bench} depends_on: - spark-master command: /opt/spark/sbin/start-worker.sh spark://spark-master:7077 - volumes: - - ${DATA_DIR:-/tmp/tpc-data}:/data:ro - - ${QUERIES_DIR:-/tmp/tpc-queries}:/queries:ro - - ${RESULTS_DIR:-/tmp/bench-results}:/results - - ${ENGINE_JARS_DIR:-/tmp/engine-jars}:/jars:ro + volumes: *volumes environment: - SPARK_WORKER_CORES=${WORKER_CORES:-8} - SPARK_WORKER_MEMORY=${WORKER_MEMORY:-16g} @@ -65,11 +72,7 @@ services: ports: - "7077:7077" - "8080:8080" - volumes: - - ${DATA_DIR:-/tmp/tpc-data}:/data:ro - - ${QUERIES_DIR:-/tmp/tpc-queries}:/queries:ro - - ${RESULTS_DIR:-/tmp/bench-results}:/results - - ${ENGINE_JARS_DIR:-/tmp/engine-jars}:/jars:ro + volumes: *volumes environment: - SPARK_MASTER_HOST=spark-master - SPARK_NO_DAEMONIZE=true @@ -99,16 +102,13 @@ services: # docker compose run bench python3 /opt/benchmarks/run.py \ # --engine comet --benchmark tpch --no-restart command: ["echo", "Use 'docker compose run bench python3 /opt/benchmarks/run.py ...' to run benchmarks"] - volumes: - - ${DATA_DIR:-/tmp/tpc-data}:/data:ro - - ${QUERIES_DIR:-/tmp/tpc-queries}:/queries:ro - - ${RESULTS_DIR:-/tmp/bench-results}:/results - - ${ENGINE_JARS_DIR:-/tmp/engine-jars}:/jars:ro + volumes: *volumes environment: - SPARK_HOME=/opt/spark - SPARK_MASTER=spark://spark-master:7077 - COMET_JAR=/jars/comet.jar - GLUTEN_JAR=/jars/gluten.jar + - ICEBERG_JAR=/jars/iceberg.jar - TPCH_DATA=/data - TPCH_QUERIES=/queries - TPCDS_DATA=/data From 2ab23e28953cab869a4de563951c6513f119a39f Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 17 Feb 2026 11:50:15 -0700 Subject: [PATCH 11/24] Add visualize-metrics.py for benchmark memory charts Add a matplotlib script that generates four PNG charts from the JVM and cgroup metrics collected during TPC benchmark runs: - jvm_memory_usage.png: peak JVM heap/off-heap per executor over time - jvm_peak_memory.png: grouped bar chart of peak memory breakdown - cgroup_memory.png: container memory usage and RSS per worker - combined_memory.png: dual-axis overlay of JVM peaks and cgroup usage Also fix cgroup metrics collection: move the collector from a separate sidecar container (which could only see its own cgroup) into the worker container itself, so it reads the worker's actual memory stats. Add timestamp_ms to the profiling CSV output so the visualization script can automatically align JVM and cgroup timelines. Co-Authored-By: Claude Opus 4.6 --- benchmarks/tpc/infra/docker/Dockerfile | 1 + .../tpc/infra/docker/docker-compose.yml | 40 +- benchmarks/tpc/profiling.py | 5 +- benchmarks/tpc/visualize-metrics.py | 434 ++++++++++++++++++ 4 files changed, 449 insertions(+), 31 deletions(-) create mode 100644 benchmarks/tpc/visualize-metrics.py diff --git a/benchmarks/tpc/infra/docker/Dockerfile b/benchmarks/tpc/infra/docker/Dockerfile index dd12445bdd..e7d587365d 100644 --- a/benchmarks/tpc/infra/docker/Dockerfile +++ b/benchmarks/tpc/infra/docker/Dockerfile @@ -43,6 +43,7 @@ COPY benchmarks/tpc/profiling.py /opt/benchmarks/profiling.py COPY benchmarks/tpc/engines /opt/benchmarks/engines COPY benchmarks/tpc/create-iceberg-tables.py /opt/benchmarks/create-iceberg-tables.py COPY benchmarks/tpc/generate-comparison.py /opt/benchmarks/generate-comparison.py +COPY benchmarks/tpc/visualize-metrics.py /opt/benchmarks/visualize-metrics.py # Copy the metrics collector script. COPY benchmarks/tpc/infra/docker/collect-metrics.sh /opt/benchmarks/collect-metrics.sh diff --git a/benchmarks/tpc/infra/docker/docker-compose.yml b/benchmarks/tpc/infra/docker/docker-compose.yml index 67dd0c7260..c71a2bb07c 100644 --- a/benchmarks/tpc/infra/docker/docker-compose.yml +++ b/benchmarks/tpc/infra/docker/docker-compose.yml @@ -46,7 +46,16 @@ x-worker: &worker image: ${BENCH_IMAGE:-comet-bench} depends_on: - spark-master - command: /opt/spark/sbin/start-worker.sh spark://spark-master:7077 + # The metrics collector runs inside the worker container so that it + # reads the worker's own cgroup memory stats (a separate sidecar + # container would only see its own cgroup, not the worker's). + command: + - sh + - -c + - >- + /opt/benchmarks/collect-metrics.sh ${METRICS_INTERVAL:-1} + /results/container-metrics-$${HOSTNAME}.csv & + exec /opt/spark/sbin/start-worker.sh spark://spark-master:7077 volumes: *volumes environment: - SPARK_WORKER_CORES=${WORKER_CORES:-8} @@ -55,14 +64,6 @@ x-worker: &worker mem_limit: ${WORKER_MEM_LIMIT:-32g} memswap_limit: ${WORKER_MEM_LIMIT:-32g} -x-metrics-collector: &metrics-collector - image: ${BENCH_IMAGE:-comet-bench} - command: - - /opt/benchmarks/collect-metrics.sh - - "${METRICS_INTERVAL:-1}" - volumes: - - ${RESULTS_DIR:-/tmp/bench-results}:/results - services: spark-master: image: ${BENCH_IMAGE:-comet-bench} @@ -116,24 +117,3 @@ services: mem_limit: ${BENCH_MEM_LIMIT:-10g} memswap_limit: ${BENCH_MEM_LIMIT:-10g} - metrics-collector-1: - <<: *metrics-collector - container_name: metrics-collector-1 - pid: "service:spark-worker-1" - command: - - /opt/benchmarks/collect-metrics.sh - - "${METRICS_INTERVAL:-1}" - - /results/container-metrics-worker-1.csv - depends_on: - - spark-worker-1 - - metrics-collector-2: - <<: *metrics-collector - container_name: metrics-collector-2 - pid: "service:spark-worker-2" - command: - - /opt/benchmarks/collect-metrics.sh - - "${METRICS_INTERVAL:-1}" - - /results/container-metrics-worker-2.csv - depends_on: - - spark-worker-2 diff --git a/benchmarks/tpc/profiling.py b/benchmarks/tpc/profiling.py index 98aa56d493..603cffe568 100644 --- a/benchmarks/tpc/profiling.py +++ b/benchmarks/tpc/profiling.py @@ -115,9 +115,12 @@ def _poll_once(self) -> None: except Exception: return - elapsed = time.time() - self._start_time + now = time.time() + elapsed = now - self._start_time + timestamp_ms = int(now * 1000) for exc in executors: row: Dict[str, Any] = { + "timestamp_ms": timestamp_ms, "elapsed_secs": round(elapsed, 2), "executor_id": exc.get("id", ""), "is_active": exc.get("isActive", True), diff --git a/benchmarks/tpc/visualize-metrics.py b/benchmarks/tpc/visualize-metrics.py new file mode 100644 index 0000000000..4672153cbd --- /dev/null +++ b/benchmarks/tpc/visualize-metrics.py @@ -0,0 +1,434 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import argparse +import csv +import os +from collections import defaultdict + +import matplotlib +matplotlib.use('Agg') +import matplotlib.pyplot as plt + + +# Stable colors: driver=blue, executor-0/worker-1=orange, executor-1/worker-2=green +ENTITY_COLORS = { + 'driver': 'tab:blue', + '0': 'tab:orange', + '1': 'tab:green', + '2': 'tab:red', + '3': 'tab:purple', + 'worker-1': 'tab:orange', + 'worker-2': 'tab:green', +} + +DEFAULT_COLORS = ['tab:orange', 'tab:green', 'tab:red', 'tab:purple', 'tab:brown', 'tab:pink'] + + +def color_for(entity_id): + """Return a stable color for the given entity identifier.""" + if entity_id in ENTITY_COLORS: + return ENTITY_COLORS[entity_id] + # Hash-based fallback for unknown entities + idx = hash(entity_id) % len(DEFAULT_COLORS) + return DEFAULT_COLORS[idx] + + +def auto_unit(max_bytes): + """Pick MB or GB and return (divisor, label).""" + if max_bytes > 1e9: + return 1e9, 'GB' + return 1e6, 'MB' + + +def load_jvm_metrics(path): + """Load JVM metrics CSV and return data grouped by executor_id.""" + data = defaultdict(lambda: defaultdict(list)) + with open(path, newline='') as f: + reader = csv.DictReader(f) + for row in reader: + eid = row['executor_id'] + for key, val in row.items(): + if key == 'executor_id': + data[eid][key].append(val) + elif key == 'is_active': + data[eid][key].append(val == 'True') + else: + data[eid][key].append(float(val)) + return data + + +def load_cgroup_metrics(path, label): + """Load a cgroup metrics CSV and return dict with lists of values.""" + data = {'label': label} + with open(path, newline='') as f: + reader = csv.DictReader(f) + rows = list(reader) + if not rows: + return data + timestamps = [float(r['timestamp_ms']) for r in rows] + data['timestamp_ms'] = timestamps + min_ts = min(timestamps) + data['elapsed_secs'] = [(ts - min_ts) / 1000.0 for ts in timestamps] + for key in ['memory_usage_bytes', 'memory_limit_bytes', 'rss_bytes', 'cache_bytes', 'swap_bytes']: + if key in rows[0]: + data[key] = [float(r[key]) for r in rows] + return data + + +def generate_jvm_memory_usage(jvm_data, output_dir, title): + """Chart 1: Peak-so-far JVM memory per executor over time (driver excluded). + + The Spark REST API only exposes monotonically-increasing peak counters + (not current usage), so the lines show the running peak at each poll. + """ + peak_series = [ + ('peak_JVMHeapMemory', 'Heap', 'solid'), + ('peak_JVMOffHeapMemory', 'OffHeap', 'dashed'), + ('peak_OffHeapExecutionMemory', 'OffHeapExec', 'dotted'), + ] + + executors = {eid: s for eid, s in jvm_data.items() if eid != 'driver'} + if not executors: + print(' Skipped jvm_memory_usage.png (no executor data)') + return + + # Check whether there is any non-zero peak data + all_values = [] + for series in executors.values(): + for field, _, _ in peak_series: + all_values.extend(series.get(field, [])) + if not all_values or max(all_values) == 0: + print(' Skipped jvm_memory_usage.png (all peak values are zero)') + return + + divisor, unit = auto_unit(max(all_values)) + + fig, ax = plt.subplots(figsize=(14, 6)) + for eid, series in sorted(executors.items()): + c = color_for(eid) + for field, label, ls in peak_series: + vals = series.get(field, []) + if vals and max(vals) > 0: + ax.plot(series['elapsed_secs'], + [v / divisor for v in vals], + color=c, linestyle=ls, linewidth=1.5, + label=f'executor {eid} {label}') + + ax.set_title(f'{title} — JVM Peak Memory Over Time' if title else 'JVM Peak Memory Over Time') + ax.set_xlabel('Elapsed Time (seconds)') + ax.set_ylabel(f'Peak Memory ({unit})') + ax.legend(fontsize=8) + ax.yaxis.grid(True) + plt.tight_layout() + plt.savefig(os.path.join(output_dir, 'jvm_memory_usage.png'), format='png') + plt.close(fig) + print(f' Created jvm_memory_usage.png') + + +def generate_jvm_peak_memory(jvm_data, output_dir, title): + """Chart 2: Peak memory breakdown per executor, excluding driver (grouped bar).""" + peak_fields = [ + 'peak_JVMHeapMemory', + 'peak_JVMOffHeapMemory', + 'peak_OnHeapExecutionMemory', + 'peak_OffHeapExecutionMemory', + ] + + executor_data = {eid: s for eid, s in jvm_data.items() if eid != 'driver'} + + # Check if all peak values are zero — skip chart if so + all_zero = True + for eid, series in executor_data.items(): + for field in peak_fields: + if field in series and max(series[field]) > 0: + all_zero = False + break + if not all_zero: + break + if all_zero: + print(' Skipped jvm_peak_memory.png (all peak values are zero)') + return + + executors = sorted(executor_data.keys()) + # Use the max of each peak field per executor + peak_values = {} + all_vals = [] + for eid in executors: + peak_values[eid] = {} + for field in peak_fields: + val = max(executor_data[eid].get(field, [0])) + peak_values[eid][field] = val + all_vals.append(val) + + divisor, unit = auto_unit(max(all_vals)) if all_vals else (1e6, 'MB') + + import numpy as np + x = np.arange(len(executors)) + n_fields = len(peak_fields) + bar_width = 0.8 / n_fields + field_colors = ['tab:blue', 'tab:orange', 'tab:green', 'tab:red'] + + fig, ax = plt.subplots(figsize=(max(8, len(executors) * 2), 6)) + for i, field in enumerate(peak_fields): + vals = [peak_values[eid][field] / divisor for eid in executors] + short_label = field.replace('peak_', '') + ax.bar(x + i * bar_width, vals, bar_width, label=short_label, color=field_colors[i]) + + ax.set_title(f'{title} — JVM Peak Memory' if title else 'JVM Peak Memory') + ax.set_xlabel('Executor') + ax.set_ylabel(f'Peak Memory ({unit})') + ax.set_xticks(x + bar_width * (n_fields - 1) / 2) + ax.set_xticklabels([f'executor {eid}' for eid in executors]) + ax.legend(fontsize=8) + ax.yaxis.grid(True) + plt.tight_layout() + plt.savefig(os.path.join(output_dir, 'jvm_peak_memory.png'), format='png') + plt.close(fig) + print(f' Created jvm_peak_memory.png') + + +def _jvm_time_range(jvm_data, cgroup_datasets): + """Compute the x-axis limit from JVM data using absolute timestamps. + + Returns the max elapsed time (in seconds, relative to the global time-zero + across both data sources) that the JVM profiler was active, or None if + absolute timestamps are not available. + """ + exec_data = {eid: s for eid, s in jvm_data.items() if eid != 'driver'} + has_jvm_ts = any('timestamp_ms' in s for s in exec_data.values()) + has_cg_ts = any('timestamp_ms' in ds for ds in cgroup_datasets) + if not (has_jvm_ts and has_cg_ts): + return None + all_abs = [] + for s in exec_data.values(): + all_abs.extend(s['timestamp_ms']) + for ds in cgroup_datasets: + all_abs.extend(ds.get('timestamp_ms', [])) + t_zero = min(all_abs) + jvm_max = 0 + for s in exec_data.values(): + jvm_max = max(jvm_max, max(s['timestamp_ms'])) + return (jvm_max - t_zero) / 1000.0 + + +def generate_cgroup_memory(cgroup_datasets, jvm_data, output_dir, title): + """Chart 3: cgroup memory_usage_bytes (solid) and rss_bytes (dashed) per worker.""" + x_max = _jvm_time_range(jvm_data, cgroup_datasets) + + # Convert cgroup timestamps to the same shared time-zero as the combined chart + exec_data = {eid: s for eid, s in jvm_data.items() if eid != 'driver'} + has_jvm_ts = any('timestamp_ms' in s for s in exec_data.values()) + has_cg_ts = any('timestamp_ms' in ds for ds in cgroup_datasets) + if has_jvm_ts and has_cg_ts: + all_abs = [] + for s in exec_data.values(): + all_abs.extend(s['timestamp_ms']) + for ds in cgroup_datasets: + all_abs.extend(ds.get('timestamp_ms', [])) + t_zero = min(all_abs) + def elapsed_for(ds): + return [(t - t_zero) / 1000.0 for t in ds['timestamp_ms']] + else: + def elapsed_for(ds): + return ds.get('elapsed_secs', []) + + fig, ax = plt.subplots(figsize=(14, 6)) + + all_values = [] + for ds in cgroup_datasets: + all_values.extend(ds.get('memory_usage_bytes', [])) + all_values.extend(ds.get('rss_bytes', [])) + + divisor, unit = auto_unit(max(all_values)) if all_values else (1e6, 'MB') + + for ds in cgroup_datasets: + label = ds['label'] + c = color_for(label) + elapsed = elapsed_for(ds) + if 'memory_usage_bytes' in ds: + ax.plot(elapsed, + [v / divisor for v in ds['memory_usage_bytes']], + color=c, linewidth=1.5, label=f'{label} usage') + if 'rss_bytes' in ds: + ax.plot(elapsed, + [v / divisor for v in ds['rss_bytes']], + color=c, linewidth=1.5, linestyle='--', label=f'{label} RSS') + + if x_max is not None: + ax.set_xlim(left=0, right=x_max * 1.02) + + ax.set_title(f'{title} — Container Memory (cgroup)' if title else 'Container Memory (cgroup)') + ax.set_xlabel('Elapsed Time (seconds)') + ax.set_ylabel(f'Memory ({unit})') + ax.legend(fontsize=8) + ax.yaxis.grid(True) + plt.tight_layout() + plt.savefig(os.path.join(output_dir, 'cgroup_memory.png'), format='png') + plt.close(fig) + print(f' Created cgroup_memory.png') + + +def generate_combined_memory(jvm_data, cgroup_datasets, cgroup_offset, output_dir, title): + """Chart 4: Dual-axis — JVM memoryUsed per executor (left) + cgroup usage per worker (right). + + Aligns both data sources onto a shared elapsed-time axis. When the JVM CSV + contains ``timestamp_ms`` (absolute epoch), the script computes a common + time-zero from the earliest timestamp across *both* sources. Otherwise it + falls back to ``elapsed_secs`` with the manual ``--cgroup-offset``. + """ + fig, ax1 = plt.subplots(figsize=(14, 6)) + ax2 = ax1.twinx() + + # --- Determine shared time-zero from absolute timestamps when available --- + exec_data = {eid: s for eid, s in jvm_data.items() if eid != 'driver'} + has_jvm_ts = any('timestamp_ms' in series for series in exec_data.values()) + has_cg_ts = any('timestamp_ms' in ds for ds in cgroup_datasets) + + if has_jvm_ts and has_cg_ts: + # Gather all absolute timestamps to find the global minimum + all_abs = [] + for series in exec_data.values(): + all_abs.extend(series['timestamp_ms']) + for ds in cgroup_datasets: + all_abs.extend(ds.get('timestamp_ms', [])) + t_zero = min(all_abs) + + def jvm_elapsed(series): + return [(t - t_zero) / 1000.0 for t in series['timestamp_ms']] + + def cg_elapsed(ds): + return [(t - t_zero) / 1000.0 for t in ds['timestamp_ms']] + else: + # Fallback: use elapsed_secs with manual offset + def jvm_elapsed(series): + return series['elapsed_secs'] + + def cg_elapsed(ds): + elapsed = ds.get('elapsed_secs', []) + if cgroup_offset != 0: + return [t + cgroup_offset for t in elapsed] + return elapsed + + # --- JVM: per-executor peak memory over time (driver excluded) --- + executor_data = {eid: s for eid, s in jvm_data.items() if eid != 'driver'} + jvm_peak_fields = [ + ('peak_JVMHeapMemory', 'Heap', 'solid'), + ('peak_OffHeapExecutionMemory', 'OffHeapExec', 'dotted'), + ] + all_jvm_vals = [] + for series in executor_data.values(): + for field, _, _ in jvm_peak_fields: + all_jvm_vals.extend(series.get(field, [])) + all_cg_vals = [] + for ds in cgroup_datasets: + all_cg_vals.extend(ds.get('memory_usage_bytes', [])) + + jvm_div, jvm_unit = auto_unit(max(all_jvm_vals)) if all_jvm_vals and max(all_jvm_vals) > 0 else (1e6, 'MB') + cg_div, cg_unit = auto_unit(max(all_cg_vals)) if all_cg_vals else (1e6, 'MB') + + for eid, series in sorted(executor_data.items()): + c = color_for(eid) + for field, label, ls in jvm_peak_fields: + vals = series.get(field, []) + if vals and max(vals) > 0: + ax1.plot(jvm_elapsed(series), + [v / jvm_div for v in vals], + color=c, linestyle=ls, linewidth=1.5, + label=f'executor {eid} {label}') + + ax1.set_xlabel('Elapsed Time (seconds)') + ax1.set_ylabel(f'JVM Peak Memory ({jvm_unit})') + ax1.tick_params(axis='y') + + # --- Cgroup: usage per worker --- + for ds in cgroup_datasets: + label = ds['label'] + c = color_for(label) + if 'memory_usage_bytes' in ds: + ax2.plot(cg_elapsed(ds), + [v / cg_div for v in ds['memory_usage_bytes']], + color=c, linewidth=1.5, linestyle='--', label=f'{label} cgroup usage') + + ax2.set_ylabel(f'Container Memory ({cg_unit})') + ax2.tick_params(axis='y') + + # Truncate x-axis to JVM time range + x_max = _jvm_time_range(jvm_data, cgroup_datasets) + if x_max is not None: + ax1.set_xlim(left=0, right=x_max * 1.02) + + # Combine legends from both axes + lines1, labels1 = ax1.get_legend_handles_labels() + lines2, labels2 = ax2.get_legend_handles_labels() + ax1.legend(lines1 + lines2, labels1 + labels2, fontsize=8, loc='upper left') + + ax1.set_title(f'{title} — Combined Memory Overview' if title else 'Combined Memory Overview') + ax1.yaxis.grid(True) + plt.tight_layout() + plt.savefig(os.path.join(output_dir, 'combined_memory.png'), format='png') + plt.close(fig) + print(f' Created combined_memory.png') + + +def main(): + parser = argparse.ArgumentParser( + description='Visualize TPC benchmark memory metrics as time-series charts.') + parser.add_argument('--jvm-metrics', type=str, required=True, + help='Path to JVM metrics CSV (comet-tpch-metrics.csv)') + parser.add_argument('--cgroup-metrics', type=str, nargs='+', default=None, + help='Paths to cgroup metrics CSVs (container-metrics-worker-*.csv)') + parser.add_argument('--output-dir', type=str, default='.', + help='Directory to write chart PNGs (default: current directory)') + parser.add_argument('--title', type=str, default='', + help='Title prefix for charts') + parser.add_argument('--cgroup-offset', type=float, default=0, + help='Seconds to shift cgroup timestamps for alignment with JVM data (default: 0)') + args = parser.parse_args() + + os.makedirs(args.output_dir, exist_ok=True) + + print(f'Loading JVM metrics from {args.jvm_metrics}') + jvm_data = load_jvm_metrics(args.jvm_metrics) + print(f' Found executors: {", ".join(sorted(jvm_data.keys()))}') + + # Compute the JVM time window so cgroup charts can be truncated to match. + jvm_max_elapsed = 0 + for eid, series in jvm_data.items(): + if eid != 'driver' and series['elapsed_secs']: + jvm_max_elapsed = max(jvm_max_elapsed, max(series['elapsed_secs'])) + + generate_jvm_memory_usage(jvm_data, args.output_dir, args.title) + generate_jvm_peak_memory(jvm_data, args.output_dir, args.title) + + if args.cgroup_metrics: + cgroup_datasets = [] + for i, path in enumerate(args.cgroup_metrics, start=1): + label = f'worker-{i}' + print(f'Loading cgroup metrics from {path} (as {label})') + cgroup_datasets.append(load_cgroup_metrics(path, label)) + + generate_cgroup_memory(cgroup_datasets, jvm_data, + args.output_dir, args.title) + generate_combined_memory(jvm_data, cgroup_datasets, args.cgroup_offset, + args.output_dir, args.title) + + print(f'Done. Charts written to {args.output_dir}') + + +if __name__ == '__main__': + main() From 9a502f39ca6038379db2a74dc080d535ec661b50 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 17 Feb 2026 12:33:03 -0700 Subject: [PATCH 12/24] Improve visualize-metrics charts and add JAVA_HOME compose override Generate per-worker/executor charts instead of combined ones for better readability. Use distinct colors per series type (Heap, OffHeap, OffHeapExec, cgroup usage/RSS). Add JVMOffHeapMemory to combined chart. Trim chart x-axis to the JVM profiler window on both ends. Add JAVA_HOME environment variable to all compose services to support engines that require a different JDK (e.g. Gluten with Java 8). Co-Authored-By: Claude Opus 4.6 --- .../tpc/infra/docker/docker-compose.yml | 3 + benchmarks/tpc/visualize-metrics.py | 362 +++++++++--------- 2 files changed, 175 insertions(+), 190 deletions(-) diff --git a/benchmarks/tpc/infra/docker/docker-compose.yml b/benchmarks/tpc/infra/docker/docker-compose.yml index c71a2bb07c..4b8eb06332 100644 --- a/benchmarks/tpc/infra/docker/docker-compose.yml +++ b/benchmarks/tpc/infra/docker/docker-compose.yml @@ -58,6 +58,7 @@ x-worker: &worker exec /opt/spark/sbin/start-worker.sh spark://spark-master:7077 volumes: *volumes environment: + - JAVA_HOME=${JAVA_HOME:-/usr/lib/jvm/java-17-openjdk-amd64} - SPARK_WORKER_CORES=${WORKER_CORES:-8} - SPARK_WORKER_MEMORY=${WORKER_MEMORY:-16g} - SPARK_NO_DAEMONIZE=true @@ -75,6 +76,7 @@ services: - "8080:8080" volumes: *volumes environment: + - JAVA_HOME=${JAVA_HOME:-/usr/lib/jvm/java-17-openjdk-amd64} - SPARK_MASTER_HOST=spark-master - SPARK_NO_DAEMONIZE=true @@ -105,6 +107,7 @@ services: command: ["echo", "Use 'docker compose run bench python3 /opt/benchmarks/run.py ...' to run benchmarks"] volumes: *volumes environment: + - JAVA_HOME=${JAVA_HOME:-/usr/lib/jvm/java-17-openjdk-amd64} - SPARK_HOME=/opt/spark - SPARK_MASTER=spark://spark-master:7077 - COMET_JAR=/jars/comet.jar diff --git a/benchmarks/tpc/visualize-metrics.py b/benchmarks/tpc/visualize-metrics.py index 4672153cbd..2645b9607f 100644 --- a/benchmarks/tpc/visualize-metrics.py +++ b/benchmarks/tpc/visualize-metrics.py @@ -38,6 +38,19 @@ DEFAULT_COLORS = ['tab:orange', 'tab:green', 'tab:red', 'tab:purple', 'tab:brown', 'tab:pink'] +PEAK_SERIES = [ + ('peak_JVMHeapMemory', 'Heap', 'solid', 'tab:blue'), + ('peak_JVMOffHeapMemory', 'OffHeap', 'solid', 'tab:orange'), + ('peak_OffHeapExecutionMemory', 'OffHeapExec', 'solid', 'tab:red'), +] + +PEAK_BAR_FIELDS = [ + 'peak_JVMHeapMemory', + 'peak_JVMOffHeapMemory', + 'peak_OnHeapExecutionMemory', + 'peak_OffHeapExecutionMemory', +] + def color_for(entity_id): """Return a stable color for the given entity identifier.""" @@ -90,71 +103,90 @@ def load_cgroup_metrics(path, label): return data -def generate_jvm_memory_usage(jvm_data, output_dir, title): - """Chart 1: Peak-so-far JVM memory per executor over time (driver excluded). +def _shared_time_zero(jvm_data, cgroup_datasets): + """Compute the global time-zero from absolute timestamps across both sources. - The Spark REST API only exposes monotonically-increasing peak counters - (not current usage), so the lines show the running peak at each poll. + Returns ``t_zero`` (ms) or ``None`` if absolute timestamps are unavailable. """ - peak_series = [ - ('peak_JVMHeapMemory', 'Heap', 'solid'), - ('peak_JVMOffHeapMemory', 'OffHeap', 'dashed'), - ('peak_OffHeapExecutionMemory', 'OffHeapExec', 'dotted'), - ] + exec_data = {eid: s for eid, s in jvm_data.items() if eid != 'driver'} + has_jvm_ts = any('timestamp_ms' in s for s in exec_data.values()) + has_cg_ts = any('timestamp_ms' in ds for ds in cgroup_datasets) + if not (has_jvm_ts and has_cg_ts): + return None + all_abs = [] + for s in exec_data.values(): + all_abs.extend(s['timestamp_ms']) + for ds in cgroup_datasets: + all_abs.extend(ds.get('timestamp_ms', [])) + return min(all_abs) + + +def _jvm_time_range(jvm_data, cgroup_datasets): + """Compute the x-axis limits from JVM data using absolute timestamps. + + Returns ``(x_min, x_max)`` — the elapsed-time window (in seconds, relative + to the global time-zero across both data sources) during which the JVM + profiler was active, or ``None`` if absolute timestamps are not available. + """ + t_zero = _shared_time_zero(jvm_data, cgroup_datasets) + if t_zero is None: + return None + exec_data = {eid: s for eid, s in jvm_data.items() if eid != 'driver'} + jvm_min = float('inf') + jvm_max = 0 + for s in exec_data.values(): + jvm_min = min(jvm_min, min(s['timestamp_ms'])) + jvm_max = max(jvm_max, max(s['timestamp_ms'])) + return ((jvm_min - t_zero) / 1000.0, (jvm_max - t_zero) / 1000.0) + +def generate_jvm_memory_usage(jvm_data, output_dir, title): + """Generate one JVM peak-memory time-series chart per executor (driver excluded).""" executors = {eid: s for eid, s in jvm_data.items() if eid != 'driver'} if not executors: - print(' Skipped jvm_memory_usage.png (no executor data)') + print(' Skipped jvm_memory (no executor data)') return - # Check whether there is any non-zero peak data - all_values = [] - for series in executors.values(): - for field, _, _ in peak_series: + for eid, series in sorted(executors.items()): + all_values = [] + for field, _, _, _ in PEAK_SERIES: all_values.extend(series.get(field, [])) - if not all_values or max(all_values) == 0: - print(' Skipped jvm_memory_usage.png (all peak values are zero)') - return + if not all_values or max(all_values) == 0: + print(f' Skipped jvm_memory_executor_{eid}.png (all peak values are zero)') + continue - divisor, unit = auto_unit(max(all_values)) + divisor, unit = auto_unit(max(all_values)) - fig, ax = plt.subplots(figsize=(14, 6)) - for eid, series in sorted(executors.items()): - c = color_for(eid) - for field, label, ls in peak_series: + fig, ax = plt.subplots(figsize=(14, 6)) + for field, label, ls, color in PEAK_SERIES: vals = series.get(field, []) if vals and max(vals) > 0: ax.plot(series['elapsed_secs'], [v / divisor for v in vals], - color=c, linestyle=ls, linewidth=1.5, - label=f'executor {eid} {label}') - - ax.set_title(f'{title} — JVM Peak Memory Over Time' if title else 'JVM Peak Memory Over Time') - ax.set_xlabel('Elapsed Time (seconds)') - ax.set_ylabel(f'Peak Memory ({unit})') - ax.legend(fontsize=8) - ax.yaxis.grid(True) - plt.tight_layout() - plt.savefig(os.path.join(output_dir, 'jvm_memory_usage.png'), format='png') - plt.close(fig) - print(f' Created jvm_memory_usage.png') + color=color, linestyle=ls, linewidth=1.5, + label=label) + + suffix = f' — Executor {eid} JVM Peak Memory' if title else f'Executor {eid} JVM Peak Memory' + ax.set_title(f'{title}{suffix}') + ax.set_xlabel('Elapsed Time (seconds)') + ax.set_ylabel(f'Peak Memory ({unit})') + ax.legend(fontsize=8) + ax.yaxis.grid(True) + plt.tight_layout() + fname = f'jvm_memory_executor_{eid}.png' + plt.savefig(os.path.join(output_dir, fname), format='png') + plt.close(fig) + print(f' Created {fname}') def generate_jvm_peak_memory(jvm_data, output_dir, title): - """Chart 2: Peak memory breakdown per executor, excluding driver (grouped bar).""" - peak_fields = [ - 'peak_JVMHeapMemory', - 'peak_JVMOffHeapMemory', - 'peak_OnHeapExecutionMemory', - 'peak_OffHeapExecutionMemory', - ] - + """Peak memory breakdown per executor, excluding driver (grouped bar).""" executor_data = {eid: s for eid, s in jvm_data.items() if eid != 'driver'} # Check if all peak values are zero — skip chart if so all_zero = True for eid, series in executor_data.items(): - for field in peak_fields: + for field in PEAK_BAR_FIELDS: if field in series and max(series[field]) > 0: all_zero = False break @@ -170,7 +202,7 @@ def generate_jvm_peak_memory(jvm_data, output_dir, title): all_vals = [] for eid in executors: peak_values[eid] = {} - for field in peak_fields: + for field in PEAK_BAR_FIELDS: val = max(executor_data[eid].get(field, [0])) peak_values[eid][field] = val all_vals.append(val) @@ -179,12 +211,12 @@ def generate_jvm_peak_memory(jvm_data, output_dir, title): import numpy as np x = np.arange(len(executors)) - n_fields = len(peak_fields) + n_fields = len(PEAK_BAR_FIELDS) bar_width = 0.8 / n_fields field_colors = ['tab:blue', 'tab:orange', 'tab:green', 'tab:red'] fig, ax = plt.subplots(figsize=(max(8, len(executors) * 2), 6)) - for i, field in enumerate(peak_fields): + for i, field in enumerate(PEAK_BAR_FIELDS): vals = [peak_values[eid][field] / divisor for eid in executors] short_label = field.replace('peak_', '') ax.bar(x + i * bar_width, vals, bar_width, label=short_label, color=field_colors[i]) @@ -202,187 +234,143 @@ def generate_jvm_peak_memory(jvm_data, output_dir, title): print(f' Created jvm_peak_memory.png') -def _jvm_time_range(jvm_data, cgroup_datasets): - """Compute the x-axis limit from JVM data using absolute timestamps. - - Returns the max elapsed time (in seconds, relative to the global time-zero - across both data sources) that the JVM profiler was active, or None if - absolute timestamps are not available. - """ - exec_data = {eid: s for eid, s in jvm_data.items() if eid != 'driver'} - has_jvm_ts = any('timestamp_ms' in s for s in exec_data.values()) - has_cg_ts = any('timestamp_ms' in ds for ds in cgroup_datasets) - if not (has_jvm_ts and has_cg_ts): - return None - all_abs = [] - for s in exec_data.values(): - all_abs.extend(s['timestamp_ms']) - for ds in cgroup_datasets: - all_abs.extend(ds.get('timestamp_ms', [])) - t_zero = min(all_abs) - jvm_max = 0 - for s in exec_data.values(): - jvm_max = max(jvm_max, max(s['timestamp_ms'])) - return (jvm_max - t_zero) / 1000.0 - - def generate_cgroup_memory(cgroup_datasets, jvm_data, output_dir, title): - """Chart 3: cgroup memory_usage_bytes (solid) and rss_bytes (dashed) per worker.""" - x_max = _jvm_time_range(jvm_data, cgroup_datasets) + """Generate one cgroup memory chart per worker.""" + jvm_range = _jvm_time_range(jvm_data, cgroup_datasets) + t_zero = _shared_time_zero(jvm_data, cgroup_datasets) - # Convert cgroup timestamps to the same shared time-zero as the combined chart - exec_data = {eid: s for eid, s in jvm_data.items() if eid != 'driver'} - has_jvm_ts = any('timestamp_ms' in s for s in exec_data.values()) - has_cg_ts = any('timestamp_ms' in ds for ds in cgroup_datasets) - if has_jvm_ts and has_cg_ts: - all_abs = [] - for s in exec_data.values(): - all_abs.extend(s['timestamp_ms']) - for ds in cgroup_datasets: - all_abs.extend(ds.get('timestamp_ms', [])) - t_zero = min(all_abs) - def elapsed_for(ds): - return [(t - t_zero) / 1000.0 for t in ds['timestamp_ms']] - else: - def elapsed_for(ds): - return ds.get('elapsed_secs', []) + for ds in cgroup_datasets: + label = ds['label'] + c = color_for(label) - fig, ax = plt.subplots(figsize=(14, 6)) + if t_zero is not None and 'timestamp_ms' in ds: + elapsed = [(t - t_zero) / 1000.0 for t in ds['timestamp_ms']] + else: + elapsed = ds.get('elapsed_secs', []) - all_values = [] - for ds in cgroup_datasets: + all_values = [] all_values.extend(ds.get('memory_usage_bytes', [])) all_values.extend(ds.get('rss_bytes', [])) + if not all_values: + continue - divisor, unit = auto_unit(max(all_values)) if all_values else (1e6, 'MB') + divisor, unit = auto_unit(max(all_values)) - for ds in cgroup_datasets: - label = ds['label'] - c = color_for(label) - elapsed = elapsed_for(ds) + fig, ax = plt.subplots(figsize=(14, 6)) if 'memory_usage_bytes' in ds: ax.plot(elapsed, [v / divisor for v in ds['memory_usage_bytes']], - color=c, linewidth=1.5, label=f'{label} usage') + color='tab:blue', linewidth=1.5, label='usage') if 'rss_bytes' in ds: ax.plot(elapsed, [v / divisor for v in ds['rss_bytes']], - color=c, linewidth=1.5, linestyle='--', label=f'{label} RSS') + color='tab:orange', linewidth=1.5, label='RSS') - if x_max is not None: - ax.set_xlim(left=0, right=x_max * 1.02) + if jvm_range is not None: + x_min, x_max = jvm_range + ax.set_xlim(left=x_min, right=x_max * 1.02) - ax.set_title(f'{title} — Container Memory (cgroup)' if title else 'Container Memory (cgroup)') - ax.set_xlabel('Elapsed Time (seconds)') - ax.set_ylabel(f'Memory ({unit})') - ax.legend(fontsize=8) - ax.yaxis.grid(True) - plt.tight_layout() - plt.savefig(os.path.join(output_dir, 'cgroup_memory.png'), format='png') - plt.close(fig) - print(f' Created cgroup_memory.png') + suffix = f' — {label} Container Memory (cgroup)' + ax.set_title(f'{title}{suffix}' if title else suffix.lstrip(' — ')) + ax.set_xlabel('Elapsed Time (seconds)') + ax.set_ylabel(f'Memory ({unit})') + ax.legend(fontsize=8) + ax.yaxis.grid(True) + plt.tight_layout() + fname = f'cgroup_memory_{label}.png' + plt.savefig(os.path.join(output_dir, fname), format='png') + plt.close(fig) + print(f' Created {fname}') def generate_combined_memory(jvm_data, cgroup_datasets, cgroup_offset, output_dir, title): - """Chart 4: Dual-axis — JVM memoryUsed per executor (left) + cgroup usage per worker (right). - - Aligns both data sources onto a shared elapsed-time axis. When the JVM CSV - contains ``timestamp_ms`` (absolute epoch), the script computes a common - time-zero from the earliest timestamp across *both* sources. Otherwise it - falls back to ``elapsed_secs`` with the manual ``--cgroup-offset``. - """ - fig, ax1 = plt.subplots(figsize=(14, 6)) - ax2 = ax1.twinx() - - # --- Determine shared time-zero from absolute timestamps when available --- + """Generate one combined dual-axis chart per worker, pairing executor N with worker-(N+1).""" exec_data = {eid: s for eid, s in jvm_data.items() if eid != 'driver'} - has_jvm_ts = any('timestamp_ms' in series for series in exec_data.values()) - has_cg_ts = any('timestamp_ms' in ds for ds in cgroup_datasets) - - if has_jvm_ts and has_cg_ts: - # Gather all absolute timestamps to find the global minimum - all_abs = [] - for series in exec_data.values(): - all_abs.extend(series['timestamp_ms']) - for ds in cgroup_datasets: - all_abs.extend(ds.get('timestamp_ms', [])) - t_zero = min(all_abs) + t_zero = _shared_time_zero(jvm_data, cgroup_datasets) + jvm_range = _jvm_time_range(jvm_data, cgroup_datasets) + # Build elapsed-time helpers + if t_zero is not None: def jvm_elapsed(series): return [(t - t_zero) / 1000.0 for t in series['timestamp_ms']] - def cg_elapsed(ds): return [(t - t_zero) / 1000.0 for t in ds['timestamp_ms']] else: - # Fallback: use elapsed_secs with manual offset def jvm_elapsed(series): return series['elapsed_secs'] - def cg_elapsed(ds): elapsed = ds.get('elapsed_secs', []) if cgroup_offset != 0: return [t + cgroup_offset for t in elapsed] return elapsed - # --- JVM: per-executor peak memory over time (driver excluded) --- - executor_data = {eid: s for eid, s in jvm_data.items() if eid != 'driver'} - jvm_peak_fields = [ - ('peak_JVMHeapMemory', 'Heap', 'solid'), - ('peak_OffHeapExecutionMemory', 'OffHeapExec', 'dotted'), - ] - all_jvm_vals = [] - for series in executor_data.values(): - for field, _, _ in jvm_peak_fields: - all_jvm_vals.extend(series.get(field, [])) - all_cg_vals = [] - for ds in cgroup_datasets: - all_cg_vals.extend(ds.get('memory_usage_bytes', [])) - - jvm_div, jvm_unit = auto_unit(max(all_jvm_vals)) if all_jvm_vals and max(all_jvm_vals) > 0 else (1e6, 'MB') - cg_div, cg_unit = auto_unit(max(all_cg_vals)) if all_cg_vals else (1e6, 'MB') + # Pair executor IDs with cgroup datasets by index: + # sorted executors ['0','1'] map to cgroup_datasets [worker-1, worker-2] + sorted_eids = sorted(exec_data.keys()) - for eid, series in sorted(executor_data.items()): - c = color_for(eid) - for field, label, ls in jvm_peak_fields: - vals = series.get(field, []) - if vals and max(vals) > 0: - ax1.plot(jvm_elapsed(series), - [v / jvm_div for v in vals], - color=c, linestyle=ls, linewidth=1.5, - label=f'executor {eid} {label}') - - ax1.set_xlabel('Elapsed Time (seconds)') - ax1.set_ylabel(f'JVM Peak Memory ({jvm_unit})') - ax1.tick_params(axis='y') - - # --- Cgroup: usage per worker --- - for ds in cgroup_datasets: + for idx, ds in enumerate(cgroup_datasets): label = ds['label'] - c = color_for(label) + eid = sorted_eids[idx] if idx < len(sorted_eids) else None + + fig, ax1 = plt.subplots(figsize=(14, 6)) + ax2 = ax1.twinx() + + # --- JVM: peak memory for this executor --- + if eid is not None: + series = exec_data[eid] + c = color_for(eid) + all_jvm_vals = [] + for field, _, _, _ in PEAK_SERIES: + all_jvm_vals.extend(series.get(field, [])) + jvm_div, jvm_unit = auto_unit(max(all_jvm_vals)) if all_jvm_vals and max(all_jvm_vals) > 0 else (1e6, 'MB') + + for field, flabel, ls, color in PEAK_SERIES: + vals = series.get(field, []) + if vals and max(vals) > 0: + ax1.plot(jvm_elapsed(series), + [v / jvm_div for v in vals], + color=color, linestyle=ls, linewidth=1.5, + label=f'{flabel}') + else: + jvm_unit = 'MB' + + ax1.set_xlabel('Elapsed Time (seconds)') + ax1.set_ylabel(f'JVM Peak Memory ({jvm_unit})') + ax1.tick_params(axis='y') + + # --- Cgroup: usage for this worker --- + all_cg_vals = ds.get('memory_usage_bytes', []) + cg_div, cg_unit = auto_unit(max(all_cg_vals)) if all_cg_vals else (1e6, 'MB') if 'memory_usage_bytes' in ds: ax2.plot(cg_elapsed(ds), [v / cg_div for v in ds['memory_usage_bytes']], - color=c, linewidth=1.5, linestyle='--', label=f'{label} cgroup usage') + color='tab:purple', linewidth=1.5, linestyle='--', label='cgroup usage') + if 'rss_bytes' in ds: + ax2.plot(cg_elapsed(ds), + [v / cg_div for v in ds['rss_bytes']], + color='tab:brown', linewidth=1, linestyle='--', label='cgroup RSS') - ax2.set_ylabel(f'Container Memory ({cg_unit})') - ax2.tick_params(axis='y') + ax2.set_ylabel(f'Container Memory ({cg_unit})') + ax2.tick_params(axis='y') - # Truncate x-axis to JVM time range - x_max = _jvm_time_range(jvm_data, cgroup_datasets) - if x_max is not None: - ax1.set_xlim(left=0, right=x_max * 1.02) + if jvm_range is not None: + x_min, x_max = jvm_range + ax1.set_xlim(left=x_min, right=x_max * 1.02) - # Combine legends from both axes - lines1, labels1 = ax1.get_legend_handles_labels() - lines2, labels2 = ax2.get_legend_handles_labels() - ax1.legend(lines1 + lines2, labels1 + labels2, fontsize=8, loc='upper left') + # Combine legends from both axes + lines1, labels1 = ax1.get_legend_handles_labels() + lines2, labels2 = ax2.get_legend_handles_labels() + ax1.legend(lines1 + lines2, labels1 + labels2, fontsize=8, loc='upper left') - ax1.set_title(f'{title} — Combined Memory Overview' if title else 'Combined Memory Overview') - ax1.yaxis.grid(True) - plt.tight_layout() - plt.savefig(os.path.join(output_dir, 'combined_memory.png'), format='png') - plt.close(fig) - print(f' Created combined_memory.png') + eid_label = f'executor {eid} / {label}' if eid else label + suffix = f' — {eid_label} Combined Memory' + ax1.set_title(f'{title}{suffix}' if title else suffix.lstrip(' — ')) + ax1.yaxis.grid(True) + plt.tight_layout() + fname = f'combined_memory_{label}.png' + plt.savefig(os.path.join(output_dir, fname), format='png') + plt.close(fig) + print(f' Created {fname}') def main(): @@ -406,12 +394,6 @@ def main(): jvm_data = load_jvm_metrics(args.jvm_metrics) print(f' Found executors: {", ".join(sorted(jvm_data.keys()))}') - # Compute the JVM time window so cgroup charts can be truncated to match. - jvm_max_elapsed = 0 - for eid, series in jvm_data.items(): - if eid != 'driver' and series['elapsed_secs']: - jvm_max_elapsed = max(jvm_max_elapsed, max(series['elapsed_secs'])) - generate_jvm_memory_usage(jvm_data, args.output_dir, args.title) generate_jvm_peak_memory(jvm_data, args.output_dir, args.title) From 769b2a7d6fdef4e20fc91cadb6ec31508ba59ee8 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 17 Feb 2026 12:42:31 -0700 Subject: [PATCH 13/24] Snapshot cgroup metrics per engine run during profiled benchmarks The cgroup collector runs continuously and its CSV gets overwritten on cluster restart, so earlier engine data is lost. The new snapshot_cgroup_metrics() method filters the raw container-metrics CSVs to the profiler's start/stop time window and writes per-engine snapshots (e.g. comet-tpch-container-metrics-spark-worker-1.csv) alongside the existing JVM metrics CSV. Co-Authored-By: Claude Opus 4.6 --- benchmarks/tpc/profiling.py | 64 +++++++++++++++++++++++++++++++++++++ benchmarks/tpc/tpcbench.py | 1 + 2 files changed, 65 insertions(+) diff --git a/benchmarks/tpc/profiling.py b/benchmarks/tpc/profiling.py index 603cffe568..23a8a7f34e 100644 --- a/benchmarks/tpc/profiling.py +++ b/benchmarks/tpc/profiling.py @@ -32,6 +32,8 @@ """ import csv +import glob +import os import threading import time from typing import Any, Dict, List, Optional @@ -87,6 +89,7 @@ def __init__( self._stop_event = threading.Event() self._thread: Optional[threading.Thread] = None self._start_time: float = 0.0 + self._stop_time: float = 0.0 @property def samples(self) -> List[Dict[str, Any]]: @@ -164,6 +167,7 @@ def stop(self) -> None: self._thread = None # One last poll to capture final state self._poll_once() + self._stop_time = time.time() print(f"Profiler stopped ({len(self._samples)} samples collected)") def write_csv(self, path: str) -> str: @@ -180,3 +184,63 @@ def write_csv(self, path: str) -> str: writer.writerow(row) print(f"Profiler: wrote {len(self._samples)} samples to {path}") return path + + def snapshot_cgroup_metrics( + self, output_dir: str, name: str, benchmark: str + ) -> List[str]: + """Filter cgroup CSVs to the profiling time window and write snapshots. + + Looks for ``container-metrics-*.csv`` in *output_dir*, keeps only + rows whose ``timestamp_ms`` falls within the profiler's start/stop + window, and writes each filtered file as + ``{name}-{benchmark}-container-metrics-{label}.csv``. + + Returns the list of snapshot file paths written. + """ + start_ms = int(self._start_time * 1000) + stop_ms = int(self._stop_time * 1000) if self._stop_time else int( + time.time() * 1000 + ) + + source_files = sorted( + glob.glob(os.path.join(output_dir, "container-metrics-*.csv")) + ) + if not source_files: + print("Profiler: no container-metrics CSVs found to snapshot") + return [] + + written: List[str] = [] + for src in source_files: + # Extract label, e.g. "spark-worker-1" from + # "container-metrics-spark-worker-1.csv" + basename = os.path.basename(src) + label = basename.replace("container-metrics-", "").replace( + ".csv", "" + ) + dest = os.path.join( + output_dir, + f"{name}-{benchmark}-container-metrics-{label}.csv", + ) + + with open(src, "r", newline="") as fin: + reader = csv.DictReader(fin) + fieldnames = reader.fieldnames + if not fieldnames: + continue + rows = [ + row + for row in reader + if start_ms <= int(row["timestamp_ms"]) <= stop_ms + ] + + with open(dest, "w", newline="") as fout: + writer = csv.DictWriter(fout, fieldnames=fieldnames) + writer.writeheader() + writer.writerows(rows) + + print( + f"Profiler: snapshot {len(rows)} cgroup rows -> {dest}" + ) + written.append(dest) + + return written diff --git a/benchmarks/tpc/tpcbench.py b/benchmarks/tpc/tpcbench.py index 9be64e2e78..1fb53ef85d 100644 --- a/benchmarks/tpc/tpcbench.py +++ b/benchmarks/tpc/tpcbench.py @@ -188,6 +188,7 @@ def main( profiler.stop() metrics_path = f"{output}/{name}-{benchmark}-metrics.csv" profiler.write_csv(metrics_path) + profiler.snapshot_cgroup_metrics(output, name, benchmark) spark.stop() From f6b07c58712519e0a5d61e92199c9be50fced475 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 17 Feb 2026 13:06:24 -0700 Subject: [PATCH 14/24] Update Docker benchmark README with operational lessons - Gluten requires restarting the entire cluster with JAVA_HOME set to Java 8 (not just the bench container) - Use --output /results so output lands alongside cgroup CSVs - mkdir -p /tmp/spark-events needed in ephemeral bench container - Document cgroup snapshot output files produced by --profile - Add visualize-metrics.py usage example for chart generation Co-Authored-By: Claude Opus 4.6 --- benchmarks/tpc/README.md | 54 ++++++++++++++++++++++++++++++++++------ 1 file changed, 46 insertions(+), 8 deletions(-) diff --git a/benchmarks/tpc/README.md b/benchmarks/tpc/README.md index 02f7bd354c..485f3476ba 100644 --- a/benchmarks/tpc/README.md +++ b/benchmarks/tpc/README.md @@ -235,23 +235,39 @@ between versions by changing the path and restarting. ### Run benchmarks Use `docker compose run` to execute benchmarks. Pass `--no-restart` since the cluster is -already managed by Compose: +already managed by Compose, and `--output /results` so that output files land in the +mounted results directory (alongside cgroup metrics CSVs): ```shell docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml \ - run bench python3 /opt/benchmarks/run.py \ - --engine comet --benchmark tpch --no-restart + run bench sh -c 'mkdir -p /tmp/spark-events && \ + python3 /opt/benchmarks/run.py \ + --engine comet --benchmark tpch --output /results --no-restart' ``` -For Gluten (requires Java 8), override `JAVA_HOME`: +> **Note:** The `mkdir -p /tmp/spark-events` is needed because the common Spark +> config enables event logging. The bench container is ephemeral so this directory +> does not persist between runs. + +For Gluten (requires Java 8), you must restart the **entire cluster** with `JAVA_HOME` +set so that all services (master, workers, and bench) use Java 8: ```shell +export JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64 +docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml down +docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml up -d + docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml \ - run -e JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64 \ - bench python3 /opt/benchmarks/run.py \ - --engine gluten --benchmark tpch --no-restart + run bench sh -c 'mkdir -p /tmp/spark-events && \ + python3 /opt/benchmarks/run.py \ + --engine gluten --benchmark tpch --output /results --no-restart' ``` +> **Important:** Only passing `-e JAVA_HOME=...` to the `bench` container is not +> sufficient -- the workers also need Java 8 or Gluten will fail at runtime with +> `sun.misc.Unsafe` errors. Switch back to Java 17 (or unset `JAVA_HOME`) and +> restart the cluster before running Comet or Spark benchmarks. + ### Memory limits and metrics Hard memory limits are enforced on all worker and bench containers. A metrics-collector @@ -259,7 +275,29 @@ sidecar runs alongside each worker to collect cgroup metrics. Configure via envi variables: `WORKER_MEM_LIMIT` (default: 32g per worker), `BENCH_MEM_LIMIT` (default: 10g), `METRICS_INTERVAL` (default: 1 second). -Metrics are written to `$RESULTS_DIR/container-metrics-worker-{1,2}.csv`. +Raw cgroup metrics are continuously written to +`$RESULTS_DIR/container-metrics-spark-worker-{1,2}.csv`. These files are overwritten each +time the cluster restarts. + +When `--profile` is used, the profiler automatically snapshots the cgroup data for the +benchmark time window, producing per-engine files: + +- `{name}-{benchmark}-metrics.csv` -- JVM executor metrics +- `{name}-{benchmark}-container-metrics-spark-worker-1.csv` -- cgroup snapshot for worker 1 +- `{name}-{benchmark}-container-metrics-spark-worker-2.csv` -- cgroup snapshot for worker 2 + +This ensures each engine run has its own paired JVM + cgroup dataset even when multiple +engines are benchmarked on the same cluster. + +Use `visualize-metrics.py` to generate memory charts from these files: + +```shell +python3 visualize-metrics.py \ + --jvm-metrics /tmp/bench-results/comet-tpch-metrics.csv \ + --cgroup-metrics /tmp/bench-results/comet-tpch-container-metrics-spark-worker-1.csv \ + /tmp/bench-results/comet-tpch-container-metrics-spark-worker-2.csv \ + --output-dir /tmp/comet-charts --title "Comet TPC-H" +``` ### Comparing Parquet vs Iceberg performance From 39e20eec54ffc0bda6c19644c47676bc9cac238d Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 17 Feb 2026 13:24:55 -0700 Subject: [PATCH 15/24] Use single log-scale axis for combined memory charts Replace the dual-axis linear chart with a single y-axis log scale so all JVM and cgroup memory series are directly comparable. Co-Authored-By: Claude Opus 4.6 --- benchmarks/tpc/visualize-metrics.py | 59 +++++++++++------------------ 1 file changed, 23 insertions(+), 36 deletions(-) diff --git a/benchmarks/tpc/visualize-metrics.py b/benchmarks/tpc/visualize-metrics.py index 2645b9607f..4859265135 100644 --- a/benchmarks/tpc/visualize-metrics.py +++ b/benchmarks/tpc/visualize-metrics.py @@ -284,7 +284,7 @@ def generate_cgroup_memory(cgroup_datasets, jvm_data, output_dir, title): def generate_combined_memory(jvm_data, cgroup_datasets, cgroup_offset, output_dir, title): - """Generate one combined dual-axis chart per worker, pairing executor N with worker-(N+1).""" + """Generate one combined log-scale chart per worker, pairing executor N with worker-(N+1).""" exec_data = {eid: s for eid, s in jvm_data.items() if eid != 'driver'} t_zero = _shared_time_zero(jvm_data, cgroup_datasets) jvm_range = _jvm_time_range(jvm_data, cgroup_datasets) @@ -312,60 +312,47 @@ def cg_elapsed(ds): label = ds['label'] eid = sorted_eids[idx] if idx < len(sorted_eids) else None - fig, ax1 = plt.subplots(figsize=(14, 6)) - ax2 = ax1.twinx() + fig, ax = plt.subplots(figsize=(14, 6)) + + # All series plotted in GB on a single log-scale axis + divisor = 1e9 + unit = 'GB' # --- JVM: peak memory for this executor --- if eid is not None: series = exec_data[eid] - c = color_for(eid) - all_jvm_vals = [] - for field, _, _, _ in PEAK_SERIES: - all_jvm_vals.extend(series.get(field, [])) - jvm_div, jvm_unit = auto_unit(max(all_jvm_vals)) if all_jvm_vals and max(all_jvm_vals) > 0 else (1e6, 'MB') - for field, flabel, ls, color in PEAK_SERIES: vals = series.get(field, []) if vals and max(vals) > 0: - ax1.plot(jvm_elapsed(series), - [v / jvm_div for v in vals], - color=color, linestyle=ls, linewidth=1.5, - label=f'{flabel}') - else: - jvm_unit = 'MB' - - ax1.set_xlabel('Elapsed Time (seconds)') - ax1.set_ylabel(f'JVM Peak Memory ({jvm_unit})') - ax1.tick_params(axis='y') + ax.plot(jvm_elapsed(series), + [v / divisor for v in vals], + color=color, linestyle=ls, linewidth=1.5, + label=f'JVM {flabel}') # --- Cgroup: usage for this worker --- - all_cg_vals = ds.get('memory_usage_bytes', []) - cg_div, cg_unit = auto_unit(max(all_cg_vals)) if all_cg_vals else (1e6, 'MB') if 'memory_usage_bytes' in ds: - ax2.plot(cg_elapsed(ds), - [v / cg_div for v in ds['memory_usage_bytes']], - color='tab:purple', linewidth=1.5, linestyle='--', label='cgroup usage') + ax.plot(cg_elapsed(ds), + [v / divisor for v in ds['memory_usage_bytes']], + color='tab:purple', linewidth=1.5, linestyle='--', label='cgroup usage') if 'rss_bytes' in ds: - ax2.plot(cg_elapsed(ds), - [v / cg_div for v in ds['rss_bytes']], - color='tab:brown', linewidth=1, linestyle='--', label='cgroup RSS') + ax.plot(cg_elapsed(ds), + [v / divisor for v in ds['rss_bytes']], + color='tab:brown', linewidth=1, linestyle='--', label='cgroup RSS') - ax2.set_ylabel(f'Container Memory ({cg_unit})') - ax2.tick_params(axis='y') + ax.set_yscale('log') + ax.set_xlabel('Elapsed Time (seconds)') + ax.set_ylabel(f'Memory ({unit}, log scale)') if jvm_range is not None: x_min, x_max = jvm_range - ax1.set_xlim(left=x_min, right=x_max * 1.02) + ax.set_xlim(left=x_min, right=x_max * 1.02) - # Combine legends from both axes - lines1, labels1 = ax1.get_legend_handles_labels() - lines2, labels2 = ax2.get_legend_handles_labels() - ax1.legend(lines1 + lines2, labels1 + labels2, fontsize=8, loc='upper left') + ax.legend(fontsize=8, loc='upper left') eid_label = f'executor {eid} / {label}' if eid else label suffix = f' — {eid_label} Combined Memory' - ax1.set_title(f'{title}{suffix}' if title else suffix.lstrip(' — ')) - ax1.yaxis.grid(True) + ax.set_title(f'{title}{suffix}' if title else suffix.lstrip(' — ')) + ax.yaxis.grid(True, which='both') plt.tight_layout() fname = f'combined_memory_{label}.png' plt.savefig(os.path.join(output_dir, fname), format='png') From e56555095d6acef2d17fc524d72b48fa48195ae1 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 18 Feb 2026 09:08:58 -0700 Subject: [PATCH 16/24] Bundle TPC-H/DS query files in the repository Copies 22 TPC-H and 99 TPC-DS SQL query files (from SQLBench-H/DS, derived under the TPC Fair Use Policy) into benchmarks/tpc/queries/ so that benchmarks are self-contained. Removes all TPCH_QUERIES/TPCDS_QUERIES env var configuration from run.py, tpcbench.py, docker-compose.yml, and the README. Adds a requirements.txt with pyspark==3.5.2 and venv setup instructions. Excludes query files from both Maven and release RAT checks. Co-Authored-By: Claude Opus 4.6 --- .gitignore | 1 + benchmarks/tpc/README.md | 18 +- benchmarks/tpc/infra/docker/Dockerfile | 1 + .../tpc/infra/docker/docker-compose.yml | 4 - benchmarks/tpc/queries/tpcds/q1.sql | 26 +++ benchmarks/tpc/queries/tpcds/q10.sql | 60 +++++ benchmarks/tpc/queries/tpcds/q11.sql | 82 +++++++ benchmarks/tpc/queries/tpcds/q12.sql | 35 +++ benchmarks/tpc/queries/tpcds/q13.sql | 53 +++++ benchmarks/tpc/queries/tpcds/q14.sql | 211 +++++++++++++++++ benchmarks/tpc/queries/tpcds/q15.sql | 21 ++ benchmarks/tpc/queries/tpcds/q16.sql | 32 +++ benchmarks/tpc/queries/tpcds/q17.sql | 46 ++++ benchmarks/tpc/queries/tpcds/q18.sql | 35 +++ benchmarks/tpc/queries/tpcds/q19.sql | 26 +++ benchmarks/tpc/queries/tpcds/q2.sql | 61 +++++ benchmarks/tpc/queries/tpcds/q20.sql | 31 +++ benchmarks/tpc/queries/tpcds/q21.sql | 31 +++ benchmarks/tpc/queries/tpcds/q22.sql | 21 ++ benchmarks/tpc/queries/tpcds/q23.sql | 108 +++++++++ benchmarks/tpc/queries/tpcds/q24.sql | 108 +++++++++ benchmarks/tpc/queries/tpcds/q25.sql | 49 ++++ benchmarks/tpc/queries/tpcds/q26.sql | 22 ++ benchmarks/tpc/queries/tpcds/q27.sql | 24 ++ benchmarks/tpc/queries/tpcds/q28.sql | 54 +++++ benchmarks/tpc/queries/tpcds/q29.sql | 48 ++++ benchmarks/tpc/queries/tpcds/q3.sql | 22 ++ benchmarks/tpc/queries/tpcds/q30.sql | 32 +++ benchmarks/tpc/queries/tpcds/q31.sql | 53 +++++ benchmarks/tpc/queries/tpcds/q32.sql | 29 +++ benchmarks/tpc/queries/tpcds/q33.sql | 76 ++++++ benchmarks/tpc/queries/tpcds/q34.sql | 32 +++ benchmarks/tpc/queries/tpcds/q35.sql | 59 +++++ benchmarks/tpc/queries/tpcds/q36.sql | 31 +++ benchmarks/tpc/queries/tpcds/q37.sql | 18 ++ benchmarks/tpc/queries/tpcds/q38.sql | 24 ++ benchmarks/tpc/queries/tpcds/q39.sql | 55 +++++ benchmarks/tpc/queries/tpcds/q4.sql | 117 ++++++++++ benchmarks/tpc/queries/tpcds/q40.sql | 29 +++ benchmarks/tpc/queries/tpcds/q41.sql | 53 +++++ benchmarks/tpc/queries/tpcds/q42.sql | 23 ++ benchmarks/tpc/queries/tpcds/q43.sql | 20 ++ benchmarks/tpc/queries/tpcds/q44.sql | 36 +++ benchmarks/tpc/queries/tpcds/q45.sql | 21 ++ benchmarks/tpc/queries/tpcds/q46.sql | 36 +++ benchmarks/tpc/queries/tpcds/q47.sql | 52 +++++ benchmarks/tpc/queries/tpcds/q48.sql | 68 ++++++ benchmarks/tpc/queries/tpcds/q49.sql | 130 +++++++++++ benchmarks/tpc/queries/tpcds/q5.sql | 129 ++++++++++ benchmarks/tpc/queries/tpcds/q50.sql | 60 +++++ benchmarks/tpc/queries/tpcds/q51.sql | 46 ++++ benchmarks/tpc/queries/tpcds/q52.sql | 23 ++ benchmarks/tpc/queries/tpcds/q53.sql | 29 +++ benchmarks/tpc/queries/tpcds/q54.sql | 57 +++++ benchmarks/tpc/queries/tpcds/q55.sql | 15 ++ benchmarks/tpc/queries/tpcds/q56.sql | 70 ++++++ benchmarks/tpc/queries/tpcds/q57.sql | 49 ++++ benchmarks/tpc/queries/tpcds/q58.sql | 66 ++++++ benchmarks/tpc/queries/tpcds/q59.sql | 45 ++++ benchmarks/tpc/queries/tpcds/q6.sql | 27 +++ benchmarks/tpc/queries/tpcds/q60.sql | 79 +++++++ benchmarks/tpc/queries/tpcds/q61.sql | 45 ++++ benchmarks/tpc/queries/tpcds/q62.sql | 36 +++ benchmarks/tpc/queries/tpcds/q63.sql | 30 +++ benchmarks/tpc/queries/tpcds/q64.sql | 122 ++++++++++ benchmarks/tpc/queries/tpcds/q65.sql | 30 +++ benchmarks/tpc/queries/tpcds/q66.sql | 221 ++++++++++++++++++ benchmarks/tpc/queries/tpcds/q67.sql | 45 ++++ benchmarks/tpc/queries/tpcds/q68.sql | 43 ++++ benchmarks/tpc/queries/tpcds/q69.sql | 48 ++++ benchmarks/tpc/queries/tpcds/q7.sql | 22 ++ benchmarks/tpc/queries/tpcds/q70.sql | 39 ++++ benchmarks/tpc/queries/tpcds/q71.sql | 41 ++++ benchmarks/tpc/queries/tpcds/q72.sql | 30 +++ benchmarks/tpc/queries/tpcds/q73.sql | 29 +++ benchmarks/tpc/queries/tpcds/q74.sql | 62 +++++ benchmarks/tpc/queries/tpcds/q75.sql | 71 ++++++ benchmarks/tpc/queries/tpcds/q76.sql | 25 ++ benchmarks/tpc/queries/tpcds/q77.sql | 109 +++++++++ benchmarks/tpc/queries/tpcds/q78.sql | 59 +++++ benchmarks/tpc/queries/tpcds/q79.sql | 24 ++ benchmarks/tpc/queries/tpcds/q8.sql | 109 +++++++++ benchmarks/tpc/queries/tpcds/q80.sql | 97 ++++++++ benchmarks/tpc/queries/tpcds/q81.sql | 32 +++ benchmarks/tpc/queries/tpcds/q82.sql | 18 ++ benchmarks/tpc/queries/tpcds/q83.sql | 68 ++++++ benchmarks/tpc/queries/tpcds/q84.sql | 22 ++ benchmarks/tpc/queries/tpcds/q85.sql | 85 +++++++ benchmarks/tpc/queries/tpcds/q86.sql | 27 +++ benchmarks/tpc/queries/tpcds/q87.sql | 24 ++ benchmarks/tpc/queries/tpcds/q88.sql | 95 ++++++++ benchmarks/tpc/queries/tpcds/q89.sql | 29 +++ benchmarks/tpc/queries/tpcds/q9.sql | 52 +++++ benchmarks/tpc/queries/tpcds/q90.sql | 23 ++ benchmarks/tpc/queries/tpcds/q91.sql | 32 +++ benchmarks/tpc/queries/tpcds/q92.sql | 31 +++ benchmarks/tpc/queries/tpcds/q93.sql | 19 ++ benchmarks/tpc/queries/tpcds/q94.sql | 30 +++ benchmarks/tpc/queries/tpcds/q95.sql | 33 +++ benchmarks/tpc/queries/tpcds/q96.sql | 17 ++ benchmarks/tpc/queries/tpcds/q97.sql | 26 +++ benchmarks/tpc/queries/tpcds/q98.sql | 34 +++ benchmarks/tpc/queries/tpcds/q99.sql | 36 +++ benchmarks/tpc/queries/tpch/q1.sql | 23 ++ benchmarks/tpc/queries/tpch/q10.sql | 33 +++ benchmarks/tpc/queries/tpch/q11.sql | 29 +++ benchmarks/tpc/queries/tpch/q12.sql | 30 +++ benchmarks/tpc/queries/tpch/q13.sql | 22 ++ benchmarks/tpc/queries/tpch/q14.sql | 15 ++ benchmarks/tpc/queries/tpch/q15.sql | 33 +++ benchmarks/tpc/queries/tpch/q16.sql | 32 +++ benchmarks/tpc/queries/tpch/q17.sql | 19 ++ benchmarks/tpc/queries/tpch/q18.sql | 34 +++ benchmarks/tpc/queries/tpch/q19.sql | 37 +++ benchmarks/tpc/queries/tpch/q2.sql | 45 ++++ benchmarks/tpc/queries/tpch/q20.sql | 39 ++++ benchmarks/tpc/queries/tpch/q21.sql | 41 ++++ benchmarks/tpc/queries/tpch/q22.sql | 39 ++++ benchmarks/tpc/queries/tpch/q3.sql | 24 ++ benchmarks/tpc/queries/tpch/q4.sql | 23 ++ benchmarks/tpc/queries/tpch/q5.sql | 26 +++ benchmarks/tpc/queries/tpch/q6.sql | 11 + benchmarks/tpc/queries/tpch/q7.sql | 41 ++++ benchmarks/tpc/queries/tpch/q8.sql | 39 ++++ benchmarks/tpc/queries/tpch/q9.sql | 34 +++ benchmarks/tpc/requirements.txt | 1 + benchmarks/tpc/run.py | 8 +- benchmarks/tpc/tpcbench.py | 12 +- dev/release/rat_exclude_files.txt | 3 + pom.xml | 2 + 130 files changed, 5662 insertions(+), 22 deletions(-) create mode 100644 benchmarks/tpc/queries/tpcds/q1.sql create mode 100644 benchmarks/tpc/queries/tpcds/q10.sql create mode 100644 benchmarks/tpc/queries/tpcds/q11.sql create mode 100644 benchmarks/tpc/queries/tpcds/q12.sql create mode 100644 benchmarks/tpc/queries/tpcds/q13.sql create mode 100644 benchmarks/tpc/queries/tpcds/q14.sql create mode 100644 benchmarks/tpc/queries/tpcds/q15.sql create mode 100644 benchmarks/tpc/queries/tpcds/q16.sql create mode 100644 benchmarks/tpc/queries/tpcds/q17.sql create mode 100644 benchmarks/tpc/queries/tpcds/q18.sql create mode 100644 benchmarks/tpc/queries/tpcds/q19.sql create mode 100644 benchmarks/tpc/queries/tpcds/q2.sql create mode 100644 benchmarks/tpc/queries/tpcds/q20.sql create mode 100644 benchmarks/tpc/queries/tpcds/q21.sql create mode 100644 benchmarks/tpc/queries/tpcds/q22.sql create mode 100644 benchmarks/tpc/queries/tpcds/q23.sql create mode 100644 benchmarks/tpc/queries/tpcds/q24.sql create mode 100644 benchmarks/tpc/queries/tpcds/q25.sql create mode 100644 benchmarks/tpc/queries/tpcds/q26.sql create mode 100644 benchmarks/tpc/queries/tpcds/q27.sql create mode 100644 benchmarks/tpc/queries/tpcds/q28.sql create mode 100644 benchmarks/tpc/queries/tpcds/q29.sql create mode 100644 benchmarks/tpc/queries/tpcds/q3.sql create mode 100644 benchmarks/tpc/queries/tpcds/q30.sql create mode 100644 benchmarks/tpc/queries/tpcds/q31.sql create mode 100644 benchmarks/tpc/queries/tpcds/q32.sql create mode 100644 benchmarks/tpc/queries/tpcds/q33.sql create mode 100644 benchmarks/tpc/queries/tpcds/q34.sql create mode 100644 benchmarks/tpc/queries/tpcds/q35.sql create mode 100644 benchmarks/tpc/queries/tpcds/q36.sql create mode 100644 benchmarks/tpc/queries/tpcds/q37.sql create mode 100644 benchmarks/tpc/queries/tpcds/q38.sql create mode 100644 benchmarks/tpc/queries/tpcds/q39.sql create mode 100644 benchmarks/tpc/queries/tpcds/q4.sql create mode 100644 benchmarks/tpc/queries/tpcds/q40.sql create mode 100644 benchmarks/tpc/queries/tpcds/q41.sql create mode 100644 benchmarks/tpc/queries/tpcds/q42.sql create mode 100644 benchmarks/tpc/queries/tpcds/q43.sql create mode 100644 benchmarks/tpc/queries/tpcds/q44.sql create mode 100644 benchmarks/tpc/queries/tpcds/q45.sql create mode 100644 benchmarks/tpc/queries/tpcds/q46.sql create mode 100644 benchmarks/tpc/queries/tpcds/q47.sql create mode 100644 benchmarks/tpc/queries/tpcds/q48.sql create mode 100644 benchmarks/tpc/queries/tpcds/q49.sql create mode 100644 benchmarks/tpc/queries/tpcds/q5.sql create mode 100644 benchmarks/tpc/queries/tpcds/q50.sql create mode 100644 benchmarks/tpc/queries/tpcds/q51.sql create mode 100644 benchmarks/tpc/queries/tpcds/q52.sql create mode 100644 benchmarks/tpc/queries/tpcds/q53.sql create mode 100644 benchmarks/tpc/queries/tpcds/q54.sql create mode 100644 benchmarks/tpc/queries/tpcds/q55.sql create mode 100644 benchmarks/tpc/queries/tpcds/q56.sql create mode 100644 benchmarks/tpc/queries/tpcds/q57.sql create mode 100644 benchmarks/tpc/queries/tpcds/q58.sql create mode 100644 benchmarks/tpc/queries/tpcds/q59.sql create mode 100644 benchmarks/tpc/queries/tpcds/q6.sql create mode 100644 benchmarks/tpc/queries/tpcds/q60.sql create mode 100644 benchmarks/tpc/queries/tpcds/q61.sql create mode 100644 benchmarks/tpc/queries/tpcds/q62.sql create mode 100644 benchmarks/tpc/queries/tpcds/q63.sql create mode 100644 benchmarks/tpc/queries/tpcds/q64.sql create mode 100644 benchmarks/tpc/queries/tpcds/q65.sql create mode 100644 benchmarks/tpc/queries/tpcds/q66.sql create mode 100644 benchmarks/tpc/queries/tpcds/q67.sql create mode 100644 benchmarks/tpc/queries/tpcds/q68.sql create mode 100644 benchmarks/tpc/queries/tpcds/q69.sql create mode 100644 benchmarks/tpc/queries/tpcds/q7.sql create mode 100644 benchmarks/tpc/queries/tpcds/q70.sql create mode 100644 benchmarks/tpc/queries/tpcds/q71.sql create mode 100644 benchmarks/tpc/queries/tpcds/q72.sql create mode 100644 benchmarks/tpc/queries/tpcds/q73.sql create mode 100644 benchmarks/tpc/queries/tpcds/q74.sql create mode 100644 benchmarks/tpc/queries/tpcds/q75.sql create mode 100644 benchmarks/tpc/queries/tpcds/q76.sql create mode 100644 benchmarks/tpc/queries/tpcds/q77.sql create mode 100644 benchmarks/tpc/queries/tpcds/q78.sql create mode 100644 benchmarks/tpc/queries/tpcds/q79.sql create mode 100644 benchmarks/tpc/queries/tpcds/q8.sql create mode 100644 benchmarks/tpc/queries/tpcds/q80.sql create mode 100644 benchmarks/tpc/queries/tpcds/q81.sql create mode 100644 benchmarks/tpc/queries/tpcds/q82.sql create mode 100644 benchmarks/tpc/queries/tpcds/q83.sql create mode 100644 benchmarks/tpc/queries/tpcds/q84.sql create mode 100644 benchmarks/tpc/queries/tpcds/q85.sql create mode 100644 benchmarks/tpc/queries/tpcds/q86.sql create mode 100644 benchmarks/tpc/queries/tpcds/q87.sql create mode 100644 benchmarks/tpc/queries/tpcds/q88.sql create mode 100644 benchmarks/tpc/queries/tpcds/q89.sql create mode 100644 benchmarks/tpc/queries/tpcds/q9.sql create mode 100644 benchmarks/tpc/queries/tpcds/q90.sql create mode 100644 benchmarks/tpc/queries/tpcds/q91.sql create mode 100644 benchmarks/tpc/queries/tpcds/q92.sql create mode 100644 benchmarks/tpc/queries/tpcds/q93.sql create mode 100644 benchmarks/tpc/queries/tpcds/q94.sql create mode 100644 benchmarks/tpc/queries/tpcds/q95.sql create mode 100644 benchmarks/tpc/queries/tpcds/q96.sql create mode 100644 benchmarks/tpc/queries/tpcds/q97.sql create mode 100644 benchmarks/tpc/queries/tpcds/q98.sql create mode 100644 benchmarks/tpc/queries/tpcds/q99.sql create mode 100644 benchmarks/tpc/queries/tpch/q1.sql create mode 100644 benchmarks/tpc/queries/tpch/q10.sql create mode 100644 benchmarks/tpc/queries/tpch/q11.sql create mode 100644 benchmarks/tpc/queries/tpch/q12.sql create mode 100644 benchmarks/tpc/queries/tpch/q13.sql create mode 100644 benchmarks/tpc/queries/tpch/q14.sql create mode 100644 benchmarks/tpc/queries/tpch/q15.sql create mode 100644 benchmarks/tpc/queries/tpch/q16.sql create mode 100644 benchmarks/tpc/queries/tpch/q17.sql create mode 100644 benchmarks/tpc/queries/tpch/q18.sql create mode 100644 benchmarks/tpc/queries/tpch/q19.sql create mode 100644 benchmarks/tpc/queries/tpch/q2.sql create mode 100644 benchmarks/tpc/queries/tpch/q20.sql create mode 100644 benchmarks/tpc/queries/tpch/q21.sql create mode 100644 benchmarks/tpc/queries/tpch/q22.sql create mode 100644 benchmarks/tpc/queries/tpch/q3.sql create mode 100644 benchmarks/tpc/queries/tpch/q4.sql create mode 100644 benchmarks/tpc/queries/tpch/q5.sql create mode 100644 benchmarks/tpc/queries/tpch/q6.sql create mode 100644 benchmarks/tpc/queries/tpch/q7.sql create mode 100644 benchmarks/tpc/queries/tpch/q8.sql create mode 100644 benchmarks/tpc/queries/tpch/q9.sql create mode 100644 benchmarks/tpc/requirements.txt diff --git a/.gitignore b/.gitignore index 05b37627bd..4675567ef8 100644 --- a/.gitignore +++ b/.gitignore @@ -17,6 +17,7 @@ filtered_rat.txt dev/dist apache-rat-*.jar venv +.venv dev/release/comet-rm/workdir spark/benchmarks .DS_Store diff --git a/benchmarks/tpc/README.md b/benchmarks/tpc/README.md index 485f3476ba..7dc9440e11 100644 --- a/benchmarks/tpc/README.md +++ b/benchmarks/tpc/README.md @@ -26,6 +26,19 @@ For full instructions on running these benchmarks on an EC2 instance, see the [C [Comet Benchmarking on EC2 Guide]: https://datafusion.apache.org/comet/contributor-guide/benchmarking_aws_ec2.html +## Setup + +TPC queries are bundled in `benchmarks/tpc/queries/` (derived from TPC-H/DS under the TPC Fair Use Policy). + +Create a virtual environment and install dependencies: + +```shell +cd benchmarks/tpc +python3 -m venv .venv +source .venv/bin/activate +pip install -r requirements.txt +``` + ## Usage All benchmarks are run via `run.py`: @@ -57,10 +70,9 @@ export SPARK_HOME=/opt/spark-3.5.3-bin-hadoop3/ export SPARK_MASTER=spark://yourhostname:7077 ``` -Set path to queries and data: +Set path to data (TPC queries are bundled in `benchmarks/tpc/queries/`): ```shell -export TPCH_QUERIES=/mnt/bigdata/tpch/queries/ export TPCH_DATA=/mnt/bigdata/tpch/sf100/ ``` @@ -182,7 +194,6 @@ export JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64 export COMET_JAR=/opt/comet/comet-spark-spark3.5_2.12-0.10.0.jar export ICEBERG_JAR=/path/to/iceberg-spark-runtime-3.5_2.12-1.8.1.jar export ICEBERG_WAREHOUSE=/mnt/bigdata/iceberg-warehouse -export TPCH_QUERIES=/mnt/bigdata/tpch/queries/ sudo ./drop-caches.sh python3 run.py --engine comet-iceberg --benchmark tpch ``` @@ -221,7 +232,6 @@ two workers: ```shell export DATA_DIR=/mnt/bigdata/tpch/sf100 -export QUERIES_DIR=/mnt/bigdata/tpch/queries export RESULTS_DIR=/tmp/bench-results export COMET_JAR=/opt/comet/comet-spark-spark3.5_2.12-0.10.0.jar diff --git a/benchmarks/tpc/infra/docker/Dockerfile b/benchmarks/tpc/infra/docker/Dockerfile index e7d587365d..12aaf3ee5a 100644 --- a/benchmarks/tpc/infra/docker/Dockerfile +++ b/benchmarks/tpc/infra/docker/Dockerfile @@ -41,6 +41,7 @@ COPY benchmarks/tpc/run.py /opt/benchmarks/run.py COPY benchmarks/tpc/tpcbench.py /opt/benchmarks/tpcbench.py COPY benchmarks/tpc/profiling.py /opt/benchmarks/profiling.py COPY benchmarks/tpc/engines /opt/benchmarks/engines +COPY benchmarks/tpc/queries /opt/benchmarks/queries COPY benchmarks/tpc/create-iceberg-tables.py /opt/benchmarks/create-iceberg-tables.py COPY benchmarks/tpc/generate-comparison.py /opt/benchmarks/generate-comparison.py COPY benchmarks/tpc/visualize-metrics.py /opt/benchmarks/visualize-metrics.py diff --git a/benchmarks/tpc/infra/docker/docker-compose.yml b/benchmarks/tpc/infra/docker/docker-compose.yml index 4b8eb06332..3e0acaccde 100644 --- a/benchmarks/tpc/infra/docker/docker-compose.yml +++ b/benchmarks/tpc/infra/docker/docker-compose.yml @@ -25,7 +25,6 @@ # Environment variables (set in .env or export before running): # BENCH_IMAGE - Docker image to use (default: comet-bench) # DATA_DIR - Host path to TPC data (default: /tmp/tpc-data) -# QUERIES_DIR - Host path to query SQL files (default: /tmp/tpc-queries) # RESULTS_DIR - Host path for results output (default: /tmp/bench-results) # COMET_JAR - Host path to Comet JAR # GLUTEN_JAR - Host path to Gluten JAR @@ -36,7 +35,6 @@ x-volumes: &volumes - ${DATA_DIR:-/tmp/tpc-data}:/data:ro - - ${QUERIES_DIR:-/tmp/tpc-queries}:/queries:ro - ${RESULTS_DIR:-/tmp/bench-results}:/results - ${COMET_JAR:-/dev/null}:/jars/comet.jar:ro - ${GLUTEN_JAR:-/dev/null}:/jars/gluten.jar:ro @@ -114,9 +112,7 @@ services: - GLUTEN_JAR=/jars/gluten.jar - ICEBERG_JAR=/jars/iceberg.jar - TPCH_DATA=/data - - TPCH_QUERIES=/queries - TPCDS_DATA=/data - - TPCDS_QUERIES=/queries mem_limit: ${BENCH_MEM_LIMIT:-10g} memswap_limit: ${BENCH_MEM_LIMIT:-10g} diff --git a/benchmarks/tpc/queries/tpcds/q1.sql b/benchmarks/tpc/queries/tpcds/q1.sql new file mode 100644 index 0000000000..31bd9eb045 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q1.sql @@ -0,0 +1,26 @@ +-- CometBench-DS query 1 derived from TPC-DS query 1 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with customer_total_return as +(select sr_customer_sk as ctr_customer_sk +,sr_store_sk as ctr_store_sk +,sum(SR_RETURN_AMT_INC_TAX) as ctr_total_return +from store_returns +,date_dim +where sr_returned_date_sk = d_date_sk +and d_year =1999 +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/benchmarks/tpc/queries/tpcds/q10.sql b/benchmarks/tpc/queries/tpcds/q10.sql new file mode 100644 index 0000000000..9ddfefb25d --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q10.sql @@ -0,0 +1,60 @@ +-- CometBench-DS query 10 derived from TPC-DS query 10 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 ('Clinton County','Platte County','Franklin County','Louisa County','Harmon 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 3 and 3+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 3 ANd 3+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 3 and 3+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/benchmarks/tpc/queries/tpcds/q11.sql b/benchmarks/tpc/queries/tpcds/q11.sql new file mode 100644 index 0000000000..abc1e14ae4 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q11.sql @@ -0,0 +1,82 @@ +-- CometBench-DS query 11 derived from TPC-DS query 11 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 1999 + and t_s_secyear.dyear = 1999+1 + and t_w_firstyear.dyear = 1999 + and t_w_secyear.dyear = 1999+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/benchmarks/tpc/queries/tpcds/q12.sql b/benchmarks/tpc/queries/tpcds/q12.sql new file mode 100644 index 0000000000..e44b5de0b6 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q12.sql @@ -0,0 +1,35 @@ +-- CometBench-DS query 12 derived from TPC-DS query 12 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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', 'Books', 'Women') + and ws_sold_date_sk = d_date_sk + and d_date between cast('2002-03-22' as date) + and (cast('2002-03-22' 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/benchmarks/tpc/queries/tpcds/q13.sql b/benchmarks/tpc/queries/tpcds/q13.sql new file mode 100644 index 0000000000..9cf55cc15d --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q13.sql @@ -0,0 +1,53 @@ +-- CometBench-DS query 13 derived from TPC-DS query 13 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 'U' + and cd_education_status = '4 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 = 'Unknown' + 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 = 'D' + and cd_education_status = '2 yr 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', 'MI', '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 ('NC', 'NY', 'TX') + and ss_net_profit between 150 and 300 + ) or + (ss_addr_sk = ca_address_sk + and ca_country = 'United States' + and ca_state in ('CA', 'NE', 'TN') + and ss_net_profit between 50 and 250 + )) +; + diff --git a/benchmarks/tpc/queries/tpcds/q14.sql b/benchmarks/tpc/queries/tpcds/q14.sql new file mode 100644 index 0000000000..f242c1eaa6 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q14.sql @@ -0,0 +1,211 @@ +-- CometBench-DS query 14 derived from TPC-DS query 14 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 1999 AND 1999 + 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 1999 AND 1999 + 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 1999 AND 1999 + 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 1999 and 1999 + 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 1999 and 1999 + 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 1999 and 1999 + 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 = 1999+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 = 1999+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 = 1999+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 1999 AND 1999 + 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 1999 AND 1999 + 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 1999 AND 1999 + 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 1999 and 1999 + 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 1999 and 1999 + 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 1999 and 1999 + 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 = 1999 + 1 + and d_moy = 12 + and d_dom = 14) + 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 = 1999 + and d_moy = 12 + and d_dom = 14) + 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/benchmarks/tpc/queries/tpcds/q15.sql b/benchmarks/tpc/queries/tpcds/q15.sql new file mode 100644 index 0000000000..37f0d2791f --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q15.sql @@ -0,0 +1,21 @@ +-- CometBench-DS query 15 derived from TPC-DS query 15 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 2002 + group by ca_zip + order by ca_zip + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q16.sql b/benchmarks/tpc/queries/tpcds/q16.sql new file mode 100644 index 0000000000..2f5db58ff3 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q16.sql @@ -0,0 +1,32 @@ +-- CometBench-DS query 16 derived from TPC-DS query 16 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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-5-01' and + (cast('1999-5-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 = 'ID' +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/benchmarks/tpc/queries/tpcds/q17.sql b/benchmarks/tpc/queries/tpcds/q17.sql new file mode 100644 index 0000000000..37602e2a8a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q17.sql @@ -0,0 +1,46 @@ +-- CometBench-DS query 17 derived from TPC-DS query 17 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = '1999Q1' + 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 ('1999Q1','1999Q2','1999Q3') + 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 ('1999Q1','1999Q2','1999Q3') + 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/benchmarks/tpc/queries/tpcds/q18.sql b/benchmarks/tpc/queries/tpcds/q18.sql new file mode 100644 index 0000000000..fa89d2db86 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q18.sql @@ -0,0 +1,35 @@ +-- CometBench-DS query 18 derived from TPC-DS query 18 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 'Primary' and + c_current_cdemo_sk = cd2.cd_demo_sk and + c_current_addr_sk = ca_address_sk and + c_birth_month in (1,2,9,5,11,3) and + d_year = 1998 and + ca_state in ('MS','NE','IA' + ,'MI','GA','NY','CO') + 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/benchmarks/tpc/queries/tpcds/q19.sql b/benchmarks/tpc/queries/tpcds/q19.sql new file mode 100644 index 0000000000..54cc69519f --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q19.sql @@ -0,0 +1,26 @@ +-- CometBench-DS query 19 derived from TPC-DS query 19 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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=8 + 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/benchmarks/tpc/queries/tpcds/q2.sql b/benchmarks/tpc/queries/tpcds/q2.sql new file mode 100644 index 0000000000..77b0dccf8c --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q2.sql @@ -0,0 +1,61 @@ +-- CometBench-DS query 2 derived from TPC-DS query 2 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 2000) 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 = 2000+1) z + where d_week_seq1=d_week_seq2-53 + order by d_week_seq1; + diff --git a/benchmarks/tpc/queries/tpcds/q20.sql b/benchmarks/tpc/queries/tpcds/q20.sql new file mode 100644 index 0000000000..12e61a06fa --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q20.sql @@ -0,0 +1,31 @@ +-- CometBench-DS query 20 derived from TPC-DS query 20 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 ('Children', 'Sports', 'Music') + and cs_sold_date_sk = d_date_sk + and d_date between cast('2002-04-01' as date) + and (cast('2002-04-01' 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/benchmarks/tpc/queries/tpcds/q21.sql b/benchmarks/tpc/queries/tpcds/q21.sql new file mode 100644 index 0000000000..b7c2a5eb2f --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q21.sql @@ -0,0 +1,31 @@ +-- CometBench-DS query 21 derived from TPC-DS query 21 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select * + from(select w_warehouse_name + ,i_item_id + ,sum(case when (cast(d_date as date) < cast ('2000-05-19' as date)) + then inv_quantity_on_hand + else 0 end) as inv_before + ,sum(case when (cast(d_date as date) >= cast ('2000-05-19' 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 ('2000-05-19' as date) - 30 days) + and (cast ('2000-05-19' 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/benchmarks/tpc/queries/tpcds/q22.sql b/benchmarks/tpc/queries/tpcds/q22.sql new file mode 100644 index 0000000000..416c4879c5 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q22.sql @@ -0,0 +1,21 @@ +-- CometBench-DS query 22 derived from TPC-DS query 22 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 1201 and 1201 + 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/benchmarks/tpc/queries/tpcds/q23.sql b/benchmarks/tpc/queries/tpcds/q23.sql new file mode 100644 index 0000000000..220bf5bf72 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q23.sql @@ -0,0 +1,108 @@ +-- CometBench-DS query 23 derived from TPC-DS query 23 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 (2000,2000+1,2000+2,2000+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 (2000,2000+1,2000+2,2000+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 = 2000 + and d_moy = 3 + 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 = 2000 + and d_moy = 3 + 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 (2000,2000 + 1,2000 + 2,2000 + 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 (2000,2000+1,2000+2,2000+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 = 2000 + and d_moy = 3 + 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 = 2000 + and d_moy = 3 + 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/benchmarks/tpc/queries/tpcds/q24.sql b/benchmarks/tpc/queries/tpcds/q24.sql new file mode 100644 index 0000000000..119f86267b --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q24.sql @@ -0,0 +1,108 @@ +-- CometBench-DS query 24 derived from TPC-DS query 24 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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_net_profit) 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=10 +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_net_profit) 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 = 10 +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 = 'green' +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/benchmarks/tpc/queries/tpcds/q25.sql b/benchmarks/tpc/queries/tpcds/q25.sql new file mode 100644 index 0000000000..eda5d6f724 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q25.sql @@ -0,0 +1,49 @@ +-- CometBench-DS query 25 derived from TPC-DS query 25 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + ,min(ss_net_profit) as store_sales_profit + ,min(sr_net_loss) as store_returns_loss + ,min(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 = 2002 + 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 = 2002 + 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 = 2002 + 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/benchmarks/tpc/queries/tpcds/q26.sql b/benchmarks/tpc/queries/tpcds/q26.sql new file mode 100644 index 0000000000..44cc067b3a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q26.sql @@ -0,0 +1,22 @@ +-- CometBench-DS query 26 derived from TPC-DS query 26 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 'M' and + cd_education_status = '4 yr Degree' and + (p_channel_email = 'N' or p_channel_event = 'N') and + d_year = 2000 + group by i_item_id + order by i_item_id + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q27.sql b/benchmarks/tpc/queries/tpcds/q27.sql new file mode 100644 index 0000000000..d141d56a8a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q27.sql @@ -0,0 +1,24 @@ +-- CometBench-DS query 27 derived from TPC-DS query 27 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 'M' and + cd_marital_status = 'U' and + cd_education_status = 'Secondary' and + d_year = 2000 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/benchmarks/tpc/queries/tpcds/q28.sql b/benchmarks/tpc/queries/tpcds/q28.sql new file mode 100644 index 0000000000..a34b079dea --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q28.sql @@ -0,0 +1,54 @@ +-- CometBench-DS query 28 derived from TPC-DS query 28 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 28 and 28+10 + or ss_coupon_amt between 12573 and 12573+1000 + or ss_wholesale_cost between 33 and 33+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 143 and 143+10 + or ss_coupon_amt between 5562 and 5562+1000 + or ss_wholesale_cost between 45 and 45+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 159 and 159+10 + or ss_coupon_amt between 2807 and 2807+1000 + or ss_wholesale_cost between 24 and 24+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 24 and 24+10 + or ss_coupon_amt between 3706 and 3706+1000 + or ss_wholesale_cost between 46 and 46+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 76 and 76+10 + or ss_coupon_amt between 2096 and 2096+1000 + or ss_wholesale_cost between 50 and 50+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 169 and 169+10 + or ss_coupon_amt between 10672 and 10672+1000 + or ss_wholesale_cost between 58 and 58+20)) B6 + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q29.sql b/benchmarks/tpc/queries/tpcds/q29.sql new file mode 100644 index 0000000000..3d6a7f8d0b --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q29.sql @@ -0,0 +1,48 @@ +-- CometBench-DS query 29 derived from TPC-DS query 29 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + ,stddev_samp(ss_quantity) as store_sales_quantity + ,stddev_samp(sr_return_quantity) as store_returns_quantity + ,stddev_samp(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/benchmarks/tpc/queries/tpcds/q3.sql b/benchmarks/tpc/queries/tpcds/q3.sql new file mode 100644 index 0000000000..2890c02e85 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q3.sql @@ -0,0 +1,22 @@ +-- CometBench-DS query 3 derived from TPC-DS query 3 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select dt.d_year + ,item.i_brand_id brand_id + ,item.i_brand brand + ,sum(ss_net_profit) 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 = 445 + 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/benchmarks/tpc/queries/tpcds/q30.sql b/benchmarks/tpc/queries/tpcds/q30.sql new file mode 100644 index 0000000000..658653381f --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q30.sql @@ -0,0 +1,32 @@ +-- CometBench-DS query 30 derived from TPC-DS query 30 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 =2000 + 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 = 'KS' + 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/benchmarks/tpc/queries/tpcds/q31.sql b/benchmarks/tpc/queries/tpcds/q31.sql new file mode 100644 index 0000000000..509a86fa43 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q31.sql @@ -0,0 +1,53 @@ +-- CometBench-DS query 31 derived from TPC-DS query 31 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 1999 + and ss1.ca_county = ss2.ca_county + and ss2.d_qoy = 2 + and ss2.d_year = 1999 + and ss2.ca_county = ss3.ca_county + and ss3.d_qoy = 3 + and ss3.d_year = 1999 + and ss1.ca_county = ws1.ca_county + and ws1.d_qoy = 1 + and ws1.d_year = 1999 + and ws1.ca_county = ws2.ca_county + and ws2.d_qoy = 2 + and ws2.d_year = 1999 + and ws1.ca_county = ws3.ca_county + and ws3.d_qoy = 3 + and ws3.d_year =1999 + 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.ca_county; + diff --git a/benchmarks/tpc/queries/tpcds/q32.sql b/benchmarks/tpc/queries/tpcds/q32.sql new file mode 100644 index 0000000000..058756465c --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q32.sql @@ -0,0 +1,29 @@ +-- CometBench-DS query 32 derived from TPC-DS query 32 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select sum(cs_ext_discount_amt) as "excess discount amount" +from + catalog_sales + ,item + ,date_dim +where +i_manufact_id = 283 +and i_item_sk = cs_item_sk +and d_date between '1999-02-22' and + (cast('1999-02-22' 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 '1999-02-22' and + (cast('1999-02-22' as date) + 90 days) + and d_date_sk = cs_sold_date_sk + ) + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q33.sql b/benchmarks/tpc/queries/tpcds/q33.sql new file mode 100644 index 0000000000..97f1f0fbf8 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q33.sql @@ -0,0 +1,76 @@ +-- CometBench-DS query 33 derived from TPC-DS query 33 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 4 + 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 = 4 + 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 = 4 + 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/benchmarks/tpc/queries/tpcds/q34.sql b/benchmarks/tpc/queries/tpcds/q34.sql new file mode 100644 index 0000000000..2c772481ef --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q34.sql @@ -0,0 +1,32 @@ +-- CometBench-DS query 34 derived from TPC-DS query 34 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = '501-1000' 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 (2000,2000+1,2000+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/benchmarks/tpc/queries/tpcds/q35.sql b/benchmarks/tpc/queries/tpcds/q35.sql new file mode 100644 index 0000000000..5876bb94ad --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q35.sql @@ -0,0 +1,59 @@ +-- CometBench-DS query 35 derived from TPC-DS query 35 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + ca_state, + cd_gender, + cd_marital_status, + cd_dep_count, + count(*) cnt1, + min(cd_dep_count), + max(cd_dep_count), + avg(cd_dep_count), + cd_dep_employed_count, + count(*) cnt2, + min(cd_dep_employed_count), + max(cd_dep_employed_count), + avg(cd_dep_employed_count), + cd_dep_college_count, + count(*) cnt3, + min(cd_dep_college_count), + max(cd_dep_college_count), + avg(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 = 2000 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 = 2000 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 = 2000 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/benchmarks/tpc/queries/tpcds/q36.sql b/benchmarks/tpc/queries/tpcds/q36.sql new file mode 100644 index 0000000000..0b5d84adcf --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q36.sql @@ -0,0 +1,31 @@ +-- CometBench-DS query 36 derived from TPC-DS query 36 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 2001 + 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/benchmarks/tpc/queries/tpcds/q37.sql b/benchmarks/tpc/queries/tpcds/q37.sql new file mode 100644 index 0000000000..4cc1d66d29 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q37.sql @@ -0,0 +1,18 @@ +-- CometBench-DS query 37 derived from TPC-DS query 37 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select i_item_id + ,i_item_desc + ,i_current_price + from item, inventory, date_dim, catalog_sales + where i_current_price between 26 and 26 + 30 + and inv_item_sk = i_item_sk + and d_date_sk=inv_date_sk + and d_date between cast('2001-06-09' as date) and (cast('2001-06-09' as date) + 60 days) + and i_manufact_id in (744,884,722,693) + 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/benchmarks/tpc/queries/tpcds/q38.sql b/benchmarks/tpc/queries/tpcds/q38.sql new file mode 100644 index 0000000000..563be4e807 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q38.sql @@ -0,0 +1,24 @@ +-- CometBench-DS query 38 derived from TPC-DS query 38 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 1190 and 1190 + 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 1190 and 1190 + 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 1190 and 1190 + 11 +) hot_cust + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q39.sql b/benchmarks/tpc/queries/tpcds/q39.sql new file mode 100644 index 0000000000..41d981501c --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q39.sql @@ -0,0 +1,55 @@ +-- CometBench-DS query 39 derived from TPC-DS query 39 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 =2001 + 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=1 + and inv2.d_moy=1+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 =2001 + 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=1 + and inv2.d_moy=1+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/benchmarks/tpc/queries/tpcds/q4.sql b/benchmarks/tpc/queries/tpcds/q4.sql new file mode 100644 index 0000000000..8edacf2f9d --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q4.sql @@ -0,0 +1,117 @@ +-- CometBench-DS query 4 derived from TPC-DS query 4 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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/benchmarks/tpc/queries/tpcds/q40.sql b/benchmarks/tpc/queries/tpcds/q40.sql new file mode 100644 index 0000000000..f1bd48aa7a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q40.sql @@ -0,0 +1,29 @@ +-- CometBench-DS query 40 derived from TPC-DS query 40 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + w_state + ,i_item_id + ,sum(case when (cast(d_date as date) < cast ('2002-05-18' 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 ('2002-05-18' 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 ('2002-05-18' as date) - 30 days) + and (cast ('2002-05-18' as date) + 30 days) + group by + w_state,i_item_id + order by w_state,i_item_id + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q41.sql b/benchmarks/tpc/queries/tpcds/q41.sql new file mode 100644 index 0000000000..df2db88833 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q41.sql @@ -0,0 +1,53 @@ +-- CometBench-DS query 41 derived from TPC-DS query 41 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select distinct(i_product_name) + from item i1 + where i_manufact_id between 668 and 668+40 + and (select count(*) as item_cnt + from item + where (i_manufact = i1.i_manufact and + ((i_category = 'Women' and + (i_color = 'cream' or i_color = 'ghost') and + (i_units = 'Ton' or i_units = 'Gross') and + (i_size = 'economy' or i_size = 'small') + ) or + (i_category = 'Women' and + (i_color = 'midnight' or i_color = 'burlywood') and + (i_units = 'Tsp' or i_units = 'Bundle') and + (i_size = 'medium' or i_size = 'extra large') + ) or + (i_category = 'Men' and + (i_color = 'lavender' or i_color = 'azure') and + (i_units = 'Each' or i_units = 'Lb') and + (i_size = 'large' or i_size = 'N/A') + ) or + (i_category = 'Men' and + (i_color = 'chocolate' or i_color = 'steel') and + (i_units = 'N/A' or i_units = 'Dozen') and + (i_size = 'economy' or i_size = 'small') + ))) or + (i_manufact = i1.i_manufact and + ((i_category = 'Women' and + (i_color = 'floral' or i_color = 'royal') and + (i_units = 'Unknown' or i_units = 'Tbl') and + (i_size = 'economy' or i_size = 'small') + ) or + (i_category = 'Women' and + (i_color = 'navy' or i_color = 'forest') and + (i_units = 'Bunch' or i_units = 'Dram') and + (i_size = 'medium' or i_size = 'extra large') + ) or + (i_category = 'Men' and + (i_color = 'cyan' or i_color = 'indian') and + (i_units = 'Carton' or i_units = 'Cup') and + (i_size = 'large' or i_size = 'N/A') + ) or + (i_category = 'Men' and + (i_color = 'coral' or i_color = 'pale') and + (i_units = 'Pallet' or i_units = 'Gram') and + (i_size = 'economy' or i_size = 'small') + )))) > 0 + order by i_product_name + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q42.sql b/benchmarks/tpc/queries/tpcds/q42.sql new file mode 100644 index 0000000000..3c565534a9 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q42.sql @@ -0,0 +1,23 @@ +-- CometBench-DS query 42 derived from TPC-DS query 42 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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=11 + 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/benchmarks/tpc/queries/tpcds/q43.sql b/benchmarks/tpc/queries/tpcds/q43.sql new file mode 100644 index 0000000000..1aaf123c9a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q43.sql @@ -0,0 +1,20 @@ +-- CometBench-DS query 43 derived from TPC-DS query 43 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 2000 + 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/benchmarks/tpc/queries/tpcds/q44.sql b/benchmarks/tpc/queries/tpcds/q44.sql new file mode 100644 index 0000000000..f7e474249f --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q44.sql @@ -0,0 +1,36 @@ +-- CometBench-DS query 44 derived from TPC-DS query 44 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 6 + 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 = 6 + 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 = 6 + 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 = 6 + 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/benchmarks/tpc/queries/tpcds/q45.sql b/benchmarks/tpc/queries/tpcds/q45.sql new file mode 100644 index 0000000000..dd77ce6cb3 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q45.sql @@ -0,0 +1,21 @@ +-- CometBench-DS query 45 derived from TPC-DS query 45 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select ca_zip, ca_city, 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_city + order by ca_zip, ca_city + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q46.sql b/benchmarks/tpc/queries/tpcds/q46.sql new file mode 100644 index 0000000000..1321f134ef --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q46.sql @@ -0,0 +1,36 @@ +-- CometBench-DS query 46 derived from TPC-DS query 46 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 3 or + household_demographics.hd_vehicle_count= 1) + 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/benchmarks/tpc/queries/tpcds/q47.sql b/benchmarks/tpc/queries/tpcds/q47.sql new file mode 100644 index 0000000000..5b29007c03 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q47.sql @@ -0,0 +1,52 @@ +-- CometBench-DS query 47 derived from TPC-DS query 47 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 2001 or + ( d_year = 2001-1 and d_moy =12) or + ( d_year = 2001+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.s_store_name, v1.s_company_name + ,v1.d_year + ,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 = 2001 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/benchmarks/tpc/queries/tpcds/q48.sql b/benchmarks/tpc/queries/tpcds/q48.sql new file mode 100644 index 0000000000..cf08721423 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q48.sql @@ -0,0 +1,68 @@ +-- CometBench-DS query 48 derived from TPC-DS query 48 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 2001 + and + ( + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'W' + and + cd_education_status = '2 yr Degree' + and + ss_sales_price between 100.00 and 150.00 + ) + or + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'S' + and + cd_education_status = 'Advanced Degree' + and + ss_sales_price between 50.00 and 100.00 + ) + or + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'D' + and + cd_education_status = 'Primary' + 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 ('IL', 'KY', 'OR') + and ss_net_profit between 0 and 2000 + ) + or + (ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('VA', 'FL', 'AL') + and ss_net_profit between 150 and 3000 + ) + or + (ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('OK', 'IA', 'TX') + and ss_net_profit between 50 and 25000 + ) + ) +; + diff --git a/benchmarks/tpc/queries/tpcds/q49.sql b/benchmarks/tpc/queries/tpcds/q49.sql new file mode 100644 index 0000000000..3e37517bc4 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q49.sql @@ -0,0 +1,130 @@ +-- CometBench-DS query 49 derived from TPC-DS query 49 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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/benchmarks/tpc/queries/tpcds/q5.sql b/benchmarks/tpc/queries/tpcds/q5.sql new file mode 100644 index 0000000000..6b750b4c76 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q5.sql @@ -0,0 +1,129 @@ +-- CometBench-DS query 5 derived from TPC-DS query 5 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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('2001-08-04' as date) + and (cast('2001-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('2001-08-04' as date) + and (cast('2001-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('2001-08-04' as date) + and (cast('2001-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/benchmarks/tpc/queries/tpcds/q50.sql b/benchmarks/tpc/queries/tpcds/q50.sql new file mode 100644 index 0000000000..0fd43d1130 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q50.sql @@ -0,0 +1,60 @@ +-- CometBench-DS query 50 derived from TPC-DS query 50 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 2002 +and d2.d_moy = 8 +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/benchmarks/tpc/queries/tpcds/q51.sql b/benchmarks/tpc/queries/tpcds/q51.sql new file mode 100644 index 0000000000..55ed6c462f --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q51.sql @@ -0,0 +1,46 @@ +-- CometBench-DS query 51 derived from TPC-DS query 51 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 1215 and 1215+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 1215 and 1215+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/benchmarks/tpc/queries/tpcds/q52.sql b/benchmarks/tpc/queries/tpcds/q52.sql new file mode 100644 index 0000000000..fc133fa7a5 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q52.sql @@ -0,0 +1,23 @@ +-- CometBench-DS query 52 derived from TPC-DS query 52 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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=11 + and dt.d_year=2000 + 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/benchmarks/tpc/queries/tpcds/q53.sql b/benchmarks/tpc/queries/tpcds/q53.sql new file mode 100644 index 0000000000..cff50f65e8 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q53.sql @@ -0,0 +1,29 @@ +-- CometBench-DS query 53 derived from TPC-DS query 53 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 (1197,1197+1,1197+2,1197+3,1197+4,1197+5,1197+6,1197+7,1197+8,1197+9,1197+10,1197+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/benchmarks/tpc/queries/tpcds/q54.sql b/benchmarks/tpc/queries/tpcds/q54.sql new file mode 100644 index 0000000000..1316ba2399 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q54.sql @@ -0,0 +1,57 @@ +-- CometBench-DS query 54 derived from TPC-DS query 54 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 'Men' + and i_class = 'shirts' + and c_customer_sk = cs_or_ws_sales.customer_sk + and d_moy = 4 + and d_year = 1998 + ) + , 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 = 1998 and d_moy = 4) + and (select distinct d_month_seq+3 + from date_dim where d_year = 1998 and d_moy = 4) + 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/benchmarks/tpc/queries/tpcds/q55.sql b/benchmarks/tpc/queries/tpcds/q55.sql new file mode 100644 index 0000000000..b448279ded --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q55.sql @@ -0,0 +1,15 @@ +-- CometBench-DS query 55 derived from TPC-DS query 55 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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=20 + and d_moy=12 + and d_year=1998 + group by i_brand, i_brand_id + order by ext_price desc, i_brand_id + LIMIT 100 ; + diff --git a/benchmarks/tpc/queries/tpcds/q56.sql b/benchmarks/tpc/queries/tpcds/q56.sql new file mode 100644 index 0000000000..4fb34dd3e4 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q56.sql @@ -0,0 +1,70 @@ +-- CometBench-DS query 56 derived from TPC-DS query 56 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 ('powder','goldenrod','bisque')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 1998 + and d_moy = 5 + and ss_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + 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 ('powder','goldenrod','bisque')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 1998 + and d_moy = 5 + and cs_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + 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 ('powder','goldenrod','bisque')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 1998 + and d_moy = 5 + and ws_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + 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/benchmarks/tpc/queries/tpcds/q57.sql b/benchmarks/tpc/queries/tpcds/q57.sql new file mode 100644 index 0000000000..3dc568848a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q57.sql @@ -0,0 +1,49 @@ +-- CometBench-DS query 57 derived from TPC-DS query 57 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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, psum + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q58.sql b/benchmarks/tpc/queries/tpcds/q58.sql new file mode 100644 index 0000000000..3243c0ac8a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q58.sql @@ -0,0 +1,66 @@ +-- CometBench-DS query 58 derived from TPC-DS query 58 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = '2000-02-12')) + 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 = '2000-02-12')) + 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 = '2000-02-12')) + 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/benchmarks/tpc/queries/tpcds/q59.sql b/benchmarks/tpc/queries/tpcds/q59.sql new file mode 100644 index 0000000000..cd92847be6 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q59.sql @@ -0,0 +1,45 @@ +-- CometBench-DS query 59 derived from TPC-DS query 59 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 1206 and 1206 + 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 1206+ 12 and 1206 + 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/benchmarks/tpc/queries/tpcds/q6.sql b/benchmarks/tpc/queries/tpcds/q6.sql new file mode 100644 index 0000000000..9e5a5ba8e3 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q6.sql @@ -0,0 +1,27 @@ +-- CometBench-DS query 6 derived from TPC-DS query 6 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 1998 + and d_moy = 3 ) + 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/benchmarks/tpc/queries/tpcds/q60.sql b/benchmarks/tpc/queries/tpcds/q60.sql new file mode 100644 index 0000000000..cf1f885524 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q60.sql @@ -0,0 +1,79 @@ +-- CometBench-DS query 60 derived from TPC-DS query 60 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 ('Shoes')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 2001 + and d_moy = 10 + 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 ('Shoes')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 2001 + and d_moy = 10 + 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 ('Shoes')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 2001 + and d_moy = 10 + 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/benchmarks/tpc/queries/tpcds/q61.sql b/benchmarks/tpc/queries/tpcds/q61.sql new file mode 100644 index 0000000000..ef2e89b971 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q61.sql @@ -0,0 +1,45 @@ +-- CometBench-DS query 61 derived from TPC-DS query 61 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = -6 + and i_category = 'Sports' + and (p_channel_dmail = 'Y' or p_channel_email = 'Y' or p_channel_tv = 'Y') + and s_gmt_offset = -6 + and d_year = 2002 + 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 = -6 + and i_category = 'Sports' + and s_gmt_offset = -6 + and d_year = 2002 + and d_moy = 11) all_sales +order by promotions, total + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q62.sql b/benchmarks/tpc/queries/tpcds/q62.sql new file mode 100644 index 0000000000..15a0815270 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q62.sql @@ -0,0 +1,36 @@ +-- CometBench-DS query 62 derived from TPC-DS query 62 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 1217 and 1217 + 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/benchmarks/tpc/queries/tpcds/q63.sql b/benchmarks/tpc/queries/tpcds/q63.sql new file mode 100644 index 0000000000..e903fca6a0 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q63.sql @@ -0,0 +1,30 @@ +-- CometBench-DS query 63 derived from TPC-DS query 63 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 (1181,1181+1,1181+2,1181+3,1181+4,1181+5,1181+6,1181+7,1181+8,1181+9,1181+10,1181+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/benchmarks/tpc/queries/tpcds/q64.sql b/benchmarks/tpc/queries/tpcds/q64.sql new file mode 100644 index 0000000000..bc8f1676fc --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q64.sql @@ -0,0 +1,122 @@ +-- CometBench-DS query 64 derived from TPC-DS query 64 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 ('light','cyan','burnished','green','almond','smoke') and + i_current_price between 22 and 22 + 10 and + i_current_price between 22 + 1 and 22 + 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 = 2001 and + cs2.syear = 2001 + 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/benchmarks/tpc/queries/tpcds/q65.sql b/benchmarks/tpc/queries/tpcds/q65.sql new file mode 100644 index 0000000000..d00f247199 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q65.sql @@ -0,0 +1,30 @@ +-- CometBench-DS query 65 derived from TPC-DS query 65 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 1186 and 1186+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 1186 and 1186+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/benchmarks/tpc/queries/tpcds/q66.sql b/benchmarks/tpc/queries/tpcds/q66.sql new file mode 100644 index 0000000000..9fb527c2d5 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q66.sql @@ -0,0 +1,221 @@ +-- CometBench-DS query 66 derived from TPC-DS query 66 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 + ,'FEDEX' || ',' || 'GERMA' as ship_carriers + ,d_year as year + ,sum(case when d_moy = 1 + then ws_ext_list_price* ws_quantity else 0 end) as jan_sales + ,sum(case when d_moy = 2 + then ws_ext_list_price* ws_quantity else 0 end) as feb_sales + ,sum(case when d_moy = 3 + then ws_ext_list_price* ws_quantity else 0 end) as mar_sales + ,sum(case when d_moy = 4 + then ws_ext_list_price* ws_quantity else 0 end) as apr_sales + ,sum(case when d_moy = 5 + then ws_ext_list_price* ws_quantity else 0 end) as may_sales + ,sum(case when d_moy = 6 + then ws_ext_list_price* ws_quantity else 0 end) as jun_sales + ,sum(case when d_moy = 7 + then ws_ext_list_price* ws_quantity else 0 end) as jul_sales + ,sum(case when d_moy = 8 + then ws_ext_list_price* ws_quantity else 0 end) as aug_sales + ,sum(case when d_moy = 9 + then ws_ext_list_price* ws_quantity else 0 end) as sep_sales + ,sum(case when d_moy = 10 + then ws_ext_list_price* ws_quantity else 0 end) as oct_sales + ,sum(case when d_moy = 11 + then ws_ext_list_price* ws_quantity else 0 end) as nov_sales + ,sum(case when d_moy = 12 + then ws_ext_list_price* ws_quantity else 0 end) as dec_sales + ,sum(case when d_moy = 1 + then ws_net_profit * ws_quantity else 0 end) as jan_net + ,sum(case when d_moy = 2 + then ws_net_profit * ws_quantity else 0 end) as feb_net + ,sum(case when d_moy = 3 + then ws_net_profit * ws_quantity else 0 end) as mar_net + ,sum(case when d_moy = 4 + then ws_net_profit * ws_quantity else 0 end) as apr_net + ,sum(case when d_moy = 5 + then ws_net_profit * ws_quantity else 0 end) as may_net + ,sum(case when d_moy = 6 + then ws_net_profit * ws_quantity else 0 end) as jun_net + ,sum(case when d_moy = 7 + then ws_net_profit * ws_quantity else 0 end) as jul_net + ,sum(case when d_moy = 8 + then ws_net_profit * ws_quantity else 0 end) as aug_net + ,sum(case when d_moy = 9 + then ws_net_profit * ws_quantity else 0 end) as sep_net + ,sum(case when d_moy = 10 + then ws_net_profit * ws_quantity else 0 end) as oct_net + ,sum(case when d_moy = 11 + then ws_net_profit * ws_quantity else 0 end) as nov_net + ,sum(case when d_moy = 12 + then ws_net_profit * 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 = 2001 + and t_time between 19072 and 19072+28800 + and sm_carrier in ('FEDEX','GERMA') + 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 + ,'FEDEX' || ',' || 'GERMA' as ship_carriers + ,d_year as year + ,sum(case when d_moy = 1 + then cs_sales_price* cs_quantity else 0 end) as jan_sales + ,sum(case when d_moy = 2 + then cs_sales_price* cs_quantity else 0 end) as feb_sales + ,sum(case when d_moy = 3 + then cs_sales_price* cs_quantity else 0 end) as mar_sales + ,sum(case when d_moy = 4 + then cs_sales_price* cs_quantity else 0 end) as apr_sales + ,sum(case when d_moy = 5 + then cs_sales_price* cs_quantity else 0 end) as may_sales + ,sum(case when d_moy = 6 + then cs_sales_price* cs_quantity else 0 end) as jun_sales + ,sum(case when d_moy = 7 + then cs_sales_price* cs_quantity else 0 end) as jul_sales + ,sum(case when d_moy = 8 + then cs_sales_price* cs_quantity else 0 end) as aug_sales + ,sum(case when d_moy = 9 + then cs_sales_price* cs_quantity else 0 end) as sep_sales + ,sum(case when d_moy = 10 + then cs_sales_price* cs_quantity else 0 end) as oct_sales + ,sum(case when d_moy = 11 + then cs_sales_price* cs_quantity else 0 end) as nov_sales + ,sum(case when d_moy = 12 + then cs_sales_price* cs_quantity else 0 end) as dec_sales + ,sum(case when d_moy = 1 + then cs_net_paid * cs_quantity else 0 end) as jan_net + ,sum(case when d_moy = 2 + then cs_net_paid * cs_quantity else 0 end) as feb_net + ,sum(case when d_moy = 3 + then cs_net_paid * cs_quantity else 0 end) as mar_net + ,sum(case when d_moy = 4 + then cs_net_paid * cs_quantity else 0 end) as apr_net + ,sum(case when d_moy = 5 + then cs_net_paid * cs_quantity else 0 end) as may_net + ,sum(case when d_moy = 6 + then cs_net_paid * cs_quantity else 0 end) as jun_net + ,sum(case when d_moy = 7 + then cs_net_paid * cs_quantity else 0 end) as jul_net + ,sum(case when d_moy = 8 + then cs_net_paid * cs_quantity else 0 end) as aug_net + ,sum(case when d_moy = 9 + then cs_net_paid * cs_quantity else 0 end) as sep_net + ,sum(case when d_moy = 10 + then cs_net_paid * cs_quantity else 0 end) as oct_net + ,sum(case when d_moy = 11 + then cs_net_paid * cs_quantity else 0 end) as nov_net + ,sum(case when d_moy = 12 + then cs_net_paid * 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 = 2001 + and t_time between 19072 AND 19072+28800 + and sm_carrier in ('FEDEX','GERMA') + 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/benchmarks/tpc/queries/tpcds/q67.sql b/benchmarks/tpc/queries/tpcds/q67.sql new file mode 100644 index 0000000000..50b7e125bd --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q67.sql @@ -0,0 +1,45 @@ +-- CometBench-DS query 67 derived from TPC-DS query 67 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 1194 and 1194+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/benchmarks/tpc/queries/tpcds/q68.sql b/benchmarks/tpc/queries/tpcds/q68.sql new file mode 100644 index 0000000000..fb69c8d045 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q68.sql @@ -0,0 +1,43 @@ +-- CometBench-DS query 68 derived from TPC-DS query 68 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 8 or + household_demographics.hd_vehicle_count= 3) + and date_dim.d_year in (2000,2000+1,2000+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/benchmarks/tpc/queries/tpcds/q69.sql b/benchmarks/tpc/queries/tpcds/q69.sql new file mode 100644 index 0000000000..d9bb0ff013 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q69.sql @@ -0,0 +1,48 @@ +-- CometBench-DS query 69 derived from TPC-DS query 69 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 ('IN','VA','MS') 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 2 and 2+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 = 2002 and + d_moy between 2 and 2+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 = 2002 and + d_moy between 2 and 2+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/benchmarks/tpc/queries/tpcds/q7.sql b/benchmarks/tpc/queries/tpcds/q7.sql new file mode 100644 index 0000000000..02e55dddf7 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q7.sql @@ -0,0 +1,22 @@ +-- CometBench-DS query 7 derived from TPC-DS query 7 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 'M' and + cd_marital_status = 'M' and + cd_education_status = '4 yr Degree' and + (p_channel_email = 'N' or p_channel_event = 'N') and + d_year = 2001 + group by i_item_id + order by i_item_id + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q70.sql b/benchmarks/tpc/queries/tpcds/q70.sql new file mode 100644 index 0000000000..a5b67fae2a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q70.sql @@ -0,0 +1,39 @@ +-- CometBench-DS query 70 derived from TPC-DS query 70 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 1180 and 1180+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 1180 and 1180+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/benchmarks/tpc/queries/tpcds/q71.sql b/benchmarks/tpc/queries/tpcds/q71.sql new file mode 100644 index 0000000000..04ce711a31 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q71.sql @@ -0,0 +1,41 @@ +-- CometBench-DS query 71 derived from TPC-DS query 71 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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=11 + and d_year=2001 + 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=11 + and d_year=2001 + 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=11 + and d_year=2001 + ) 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/benchmarks/tpc/queries/tpcds/q72.sql b/benchmarks/tpc/queries/tpcds/q72.sql new file mode 100644 index 0000000000..d89d62ce9e --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q72.sql @@ -0,0 +1,30 @@ +-- CometBench-DS query 72 derived from TPC-DS query 72 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = '501-1000' + and d1.d_year = 1999 + and cd_marital_status = 'S' +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/benchmarks/tpc/queries/tpcds/q73.sql b/benchmarks/tpc/queries/tpcds/q73.sql new file mode 100644 index 0000000000..b21caa6e8a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q73.sql @@ -0,0 +1,29 @@ +-- CometBench-DS query 73 derived from TPC-DS query 73 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = '1001-5000' or + household_demographics.hd_buy_potential = '5001-10000') + 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 (1999,1999+1,1999+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/benchmarks/tpc/queries/tpcds/q74.sql b/benchmarks/tpc/queries/tpcds/q74.sql new file mode 100644 index 0000000000..04af4b3fd0 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q74.sql @@ -0,0 +1,62 @@ +-- CometBench-DS query 74 derived from TPC-DS query 74 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 + ,stddev_samp(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 + ,stddev_samp(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 3,2,1 + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q75.sql b/benchmarks/tpc/queries/tpcds/q75.sql new file mode 100644 index 0000000000..8e739147d0 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q75.sql @@ -0,0 +1,71 @@ +-- CometBench-DS query 75 derived from TPC-DS query 75 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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='Shoes' + 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='Shoes' + 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='Shoes') 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=2000 + AND prev_yr.d_year=2000-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/benchmarks/tpc/queries/tpcds/q76.sql b/benchmarks/tpc/queries/tpcds/q76.sql new file mode 100644 index 0000000000..98fc8ddb2a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q76.sql @@ -0,0 +1,25 @@ +-- CometBench-DS query 76 derived from TPC-DS query 76 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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_customer_sk' col_name, d_year, d_qoy, i_category, ss_ext_sales_price ext_sales_price + FROM store_sales, item, date_dim + WHERE ss_customer_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_ship_hdemo_sk' col_name, d_year, d_qoy, i_category, ws_ext_sales_price ext_sales_price + FROM web_sales, item, date_dim + WHERE ws_ship_hdemo_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_bill_customer_sk' col_name, d_year, d_qoy, i_category, cs_ext_sales_price ext_sales_price + FROM catalog_sales, item, date_dim + WHERE cs_bill_customer_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/benchmarks/tpc/queries/tpcds/q77.sql b/benchmarks/tpc/queries/tpcds/q77.sql new file mode 100644 index 0000000000..fac28a26a6 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q77.sql @@ -0,0 +1,109 @@ +-- CometBench-DS query 77 derived from TPC-DS query 77 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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('2001-08-11' as date) + and (cast('2001-08-11' 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('2001-08-11' as date) + and (cast('2001-08-11' 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('2001-08-11' as date) + and (cast('2001-08-11' 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('2001-08-11' as date) + and (cast('2001-08-11' 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('2001-08-11' as date) + and (cast('2001-08-11' 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('2001-08-11' as date) + and (cast('2001-08-11' 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/benchmarks/tpc/queries/tpcds/q78.sql b/benchmarks/tpc/queries/tpcds/q78.sql new file mode 100644 index 0000000000..2cb7a97da2 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q78.sql @@ -0,0 +1,59 @@ +-- CometBench-DS query 78 derived from TPC-DS query 78 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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_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=2001 +order by + 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/benchmarks/tpc/queries/tpcds/q79.sql b/benchmarks/tpc/queries/tpcds/q79.sql new file mode 100644 index 0000000000..3680d08400 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q79.sql @@ -0,0 +1,24 @@ +-- CometBench-DS query 79 derived from TPC-DS query 79 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 0 or household_demographics.hd_vehicle_count > 4) + and date_dim.d_dow = 1 + and date_dim.d_year in (1999,1999+1,1999+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/benchmarks/tpc/queries/tpcds/q8.sql b/benchmarks/tpc/queries/tpcds/q8.sql new file mode 100644 index 0000000000..63235faf45 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q8.sql @@ -0,0 +1,109 @@ +-- CometBench-DS query 8 derived from TPC-DS query 8 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 ( + '19100','41548','51640','49699','88329','55986', + '85119','19510','61020','95452','26235', + '51102','16733','42819','27823','90192', + '31905','28865','62197','23750','81398', + '95288','45114','82060','12313','25218', + '64386','46400','77230','69271','43672', + '36521','34217','13017','27936','42766', + '59233','26060','27477','39981','93402', + '74270','13932','51731','71642','17710', + '85156','21679','70840','67191','39214', + '35273','27293','17128','15458','31615', + '60706','67657','54092','32775','14683', + '32206','62543','43053','11297','58216', + '49410','14710','24501','79057','77038', + '91286','32334','46298','18326','67213', + '65382','40315','56115','80162','55956', + '81583','73588','32513','62880','12201', + '11592','17014','83832','61796','57872', + '78829','69912','48524','22016','26905', + '48511','92168','63051','25748','89786', + '98827','86404','53029','37524','14039', + '50078','34487','70142','18697','40129', + '60642','42810','62667','57183','46414', + '58463','71211','46364','34851','54884', + '25382','25239','74126','21568','84204', + '13607','82518','32982','36953','86001', + '79278','21745','64444','35199','83181', + '73255','86177','98043','90392','13882', + '47084','17859','89526','42072','20233', + '52745','75000','22044','77013','24182', + '52554','56138','43440','86100','48791', + '21883','17096','15965','31196','74903', + '19810','35763','92020','55176','54433', + '68063','71919','44384','16612','32109', + '28207','14762','89933','10930','27616', + '56809','14244','22733','33177','29784', + '74968','37887','11299','34692','85843', + '83663','95421','19323','17406','69264', + '28341','50150','79121','73974','92917', + '21229','32254','97408','46011','37169', + '18146','27296','62927','68812','47734', + '86572','12620','80252','50173','27261', + '29534','23488','42184','23695','45868', + '12910','23429','29052','63228','30731', + '15747','25827','22332','62349','56661', + '44652','51862','57007','22773','40361', + '65238','19327','17282','44708','35484', + '34064','11148','92729','22995','18833', + '77528','48917','17256','93166','68576', + '71096','56499','35096','80551','82424', + '17700','32748','78969','46820','57725', + '46179','54677','98097','62869','83959', + '66728','19716','48326','27420','53458', + '69056','84216','36688','63957','41469', + '66843','18024','81950','21911','58387', + '58103','19813','34581','55347','17171', + '35914','75043','75088','80541','26802', + '28849','22356','57721','77084','46385', + '59255','29308','65885','70673','13306', + '68788','87335','40987','31654','67560', + '92309','78116','65961','45018','16548', + '67092','21818','33716','49449','86150', + '12156','27574','43201','50977','52839', + '33234','86611','71494','17823','57172', + '59869','34086','51052','11320','39717', + '79604','24672','70555','38378','91135', + '15567','21606','74994','77168','38607', + '27384','68328','88944','40203','37893', + '42726','83549','48739','55652','27543', + '23109','98908','28831','45011','47525', + '43870','79404','35780','42136','49317', + '14574','99586','21107','14302','83882', + '81272','92552','14916','87533','86518', + '17862','30741','96288','57886','30304', + '24201','79457','36728','49833','35182', + '20108','39858','10804','47042','20439', + '54708','59027','82499','75311','26548', + '53406','92060','41152','60446','33129', + '43979','16903','60319','35550','33887', + '25463','40343','20726','44429') + 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 = 2000 + 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/benchmarks/tpc/queries/tpcds/q80.sql b/benchmarks/tpc/queries/tpcds/q80.sql new file mode 100644 index 0000000000..484aa4b28f --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q80.sql @@ -0,0 +1,97 @@ +-- CometBench-DS query 80 derived from TPC-DS query 80 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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('2002-08-04' as date) + and (cast('2002-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('2002-08-04' as date) + and (cast('2002-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('2002-08-04' as date) + and (cast('2002-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/benchmarks/tpc/queries/tpcds/q81.sql b/benchmarks/tpc/queries/tpcds/q81.sql new file mode 100644 index 0000000000..a768764099 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q81.sql @@ -0,0 +1,32 @@ +-- CometBench-DS query 81 derived from TPC-DS query 81 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 'TX' + 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/benchmarks/tpc/queries/tpcds/q82.sql b/benchmarks/tpc/queries/tpcds/q82.sql new file mode 100644 index 0000000000..c967802b33 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q82.sql @@ -0,0 +1,18 @@ +-- CometBench-DS query 82 derived from TPC-DS query 82 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select i_item_id + ,i_item_desc + ,i_current_price + from item, inventory, date_dim, store_sales + where i_current_price between 69 and 69+30 + and inv_item_sk = i_item_sk + and d_date_sk=inv_date_sk + and d_date between cast('1998-06-06' as date) and (cast('1998-06-06' as date) + 60 days) + and i_manufact_id in (105,513,180,137) + 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/benchmarks/tpc/queries/tpcds/q83.sql b/benchmarks/tpc/queries/tpcds/q83.sql new file mode 100644 index 0000000000..85f25c7eea --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q83.sql @@ -0,0 +1,68 @@ +-- CometBench-DS query 83 derived from TPC-DS query 83 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 ('2000-04-29','2000-09-09','2000-11-02'))) + 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 ('2000-04-29','2000-09-09','2000-11-02'))) + 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 ('2000-04-29','2000-09-09','2000-11-02'))) + 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/benchmarks/tpc/queries/tpcds/q84.sql b/benchmarks/tpc/queries/tpcds/q84.sql new file mode 100644 index 0000000000..89fa1e44df --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q84.sql @@ -0,0 +1,22 @@ +-- CometBench-DS query 84 derived from TPC-DS query 84 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 'White Oak' + and c_current_addr_sk = ca_address_sk + and ib_lower_bound >= 45626 + and ib_upper_bound <= 45626 + 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/benchmarks/tpc/queries/tpcds/q85.sql b/benchmarks/tpc/queries/tpcds/q85.sql new file mode 100644 index 0000000000..54e53149c5 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q85.sql @@ -0,0 +1,85 @@ +-- CometBench-DS query 85 derived from TPC-DS query 85 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 2001 + 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 = '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 100.00 and 150.00 + ) + or + ( + cd1.cd_marital_status = 'U' + and + cd1.cd_marital_status = cd2.cd_marital_status + and + cd1.cd_education_status = 'Unknown' + and + cd1.cd_education_status = cd2.cd_education_status + and + ws_sales_price between 50.00 and 100.00 + ) + or + ( + cd1.cd_marital_status = 'M' + 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 ('SC', 'IN', 'VA') + and ws_net_profit between 100 and 200 + ) + or + ( + ca_country = 'United States' + and + ca_state in ('WA', 'KS', 'KY') + and ws_net_profit between 150 and 300 + ) + or + ( + ca_country = 'United States' + and + ca_state in ('SD', 'WI', 'NE') + 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/benchmarks/tpc/queries/tpcds/q86.sql b/benchmarks/tpc/queries/tpcds/q86.sql new file mode 100644 index 0000000000..c8d2d08b84 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q86.sql @@ -0,0 +1,27 @@ +-- CometBench-DS query 86 derived from TPC-DS query 86 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 1205 and 1205+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/benchmarks/tpc/queries/tpcds/q87.sql b/benchmarks/tpc/queries/tpcds/q87.sql new file mode 100644 index 0000000000..86e50e9ba6 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q87.sql @@ -0,0 +1,24 @@ +-- CometBench-DS query 87 derived from TPC-DS query 87 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 1189 and 1189+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 1189 and 1189+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 1189 and 1189+11) +) cool_cust +; + diff --git a/benchmarks/tpc/queries/tpcds/q88.sql b/benchmarks/tpc/queries/tpcds/q88.sql new file mode 100644 index 0000000000..d663b2738a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q88.sql @@ -0,0 +1,95 @@ +-- CometBench-DS query 88 derived from TPC-DS query 88 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+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 = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+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 = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+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 = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+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 = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+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 = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+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 = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+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 = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+2)) + and store.s_store_name = 'ese') s8 +; + diff --git a/benchmarks/tpc/queries/tpcds/q89.sql b/benchmarks/tpc/queries/tpcds/q89.sql new file mode 100644 index 0000000000..a9359808f0 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q89.sql @@ -0,0 +1,29 @@ +-- CometBench-DS query 89 derived from TPC-DS query 89 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 (2001) and + ((i_category in ('Children','Jewelry','Home') and + i_class in ('infants','birdal','flatware') + ) + or (i_category in ('Electronics','Music','Books') and + i_class in ('audio','classical','science') + )) +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/benchmarks/tpc/queries/tpcds/q9.sql b/benchmarks/tpc/queries/tpcds/q9.sql new file mode 100644 index 0000000000..7dab30e028 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q9.sql @@ -0,0 +1,52 @@ +-- CometBench-DS query 9 derived from TPC-DS query 9 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select case when (select count(*) + from store_sales + where ss_quantity between 1 and 20) > 31002 + 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) > 588 + 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) > 2456 + 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) > 21645 + 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) > 20553 + 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/benchmarks/tpc/queries/tpcds/q90.sql b/benchmarks/tpc/queries/tpcds/q90.sql new file mode 100644 index 0000000000..6c296c313d --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q90.sql @@ -0,0 +1,23 @@ +-- CometBench-DS query 90 derived from TPC-DS query 90 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 9 and 9+1 + and household_demographics.hd_dep_count = 2 + 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 15 and 15+1 + and household_demographics.hd_dep_count = 2 + and web_page.wp_char_count between 5000 and 5200) pt + order by am_pm_ratio + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q91.sql b/benchmarks/tpc/queries/tpcds/q91.sql new file mode 100644 index 0000000000..a9ebbc090a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q91.sql @@ -0,0 +1,32 @@ +-- CometBench-DS query 91 derived from TPC-DS query 91 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 = 2002 +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 'Unknown%' +and ca_gmt_offset = -6 +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/benchmarks/tpc/queries/tpcds/q92.sql b/benchmarks/tpc/queries/tpcds/q92.sql new file mode 100644 index 0000000000..2a858fc114 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q92.sql @@ -0,0 +1,31 @@ +-- CometBench-DS query 92 derived from TPC-DS query 92 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + sum(ws_ext_discount_amt) as "Excess Discount Amount" +from + web_sales + ,item + ,date_dim +where +i_manufact_id = 914 +and i_item_sk = ws_item_sk +and d_date between '2001-01-25' and + (cast('2001-01-25' 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 '2001-01-25' and + (cast('2001-01-25' as date) + 90 days) + and d_date_sk = ws_sold_date_sk + ) +order by sum(ws_ext_discount_amt) + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q93.sql b/benchmarks/tpc/queries/tpcds/q93.sql new file mode 100644 index 0000000000..e09acd8505 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q93.sql @@ -0,0 +1,19 @@ +-- CometBench-DS query 93 derived from TPC-DS query 93 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 get it on time') t + group by ss_customer_sk + order by sumsales, ss_customer_sk + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q94.sql b/benchmarks/tpc/queries/tpcds/q94.sql new file mode 100644 index 0000000000..ba9a1d9022 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q94.sql @@ -0,0 +1,30 @@ +-- CometBench-DS query 94 derived from TPC-DS query 94 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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-4-01' and + (cast('1999-4-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 = 'WI' +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/benchmarks/tpc/queries/tpcds/q95.sql b/benchmarks/tpc/queries/tpcds/q95.sql new file mode 100644 index 0000000000..01c3c8ccfe --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q95.sql @@ -0,0 +1,33 @@ +-- CometBench-DS query 95 derived from TPC-DS query 95 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 '2002-5-01' and + (cast('2002-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 = 'MA' +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/benchmarks/tpc/queries/tpcds/q96.sql b/benchmarks/tpc/queries/tpcds/q96.sql new file mode 100644 index 0000000000..9642a924e1 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q96.sql @@ -0,0 +1,17 @@ +-- CometBench-DS query 96 derived from TPC-DS query 96 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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/benchmarks/tpc/queries/tpcds/q97.sql b/benchmarks/tpc/queries/tpcds/q97.sql new file mode 100644 index 0000000000..10b29bd126 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q97.sql @@ -0,0 +1,26 @@ +-- CometBench-DS query 97 derived from TPC-DS query 97 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 1211 and 1211 + 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 1211 and 1211 + 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/benchmarks/tpc/queries/tpcds/q98.sql b/benchmarks/tpc/queries/tpcds/q98.sql new file mode 100644 index 0000000000..c364ca900b --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q98.sql @@ -0,0 +1,34 @@ +-- CometBench-DS query 98 derived from TPC-DS query 98 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 ('Shoes', 'Music', 'Men') + and ss_sold_date_sk = d_date_sk + and d_date between cast('2000-01-05' as date) + and (cast('2000-01-05' 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/benchmarks/tpc/queries/tpcds/q99.sql b/benchmarks/tpc/queries/tpcds/q99.sql new file mode 100644 index 0000000000..b4713752b7 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q99.sql @@ -0,0 +1,36 @@ +-- CometBench-DS query 99 derived from TPC-DS query 99 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +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 1188 and 1188 + 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/benchmarks/tpc/queries/tpch/q1.sql b/benchmarks/tpc/queries/tpch/q1.sql new file mode 100644 index 0000000000..f24b0e0615 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q1.sql @@ -0,0 +1,23 @@ +-- CometBench-H query 1 derived from TPC-H query 1 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + l_returnflag, + l_linestatus, + sum(l_quantity) as sum_qty, + sum(l_extendedprice) as sum_base_price, + sum(l_extendedprice * (1 - l_discount)) as sum_disc_price, + sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge, + avg(l_quantity) as avg_qty, + avg(l_extendedprice) as avg_price, + avg(l_discount) as avg_disc, + count(*) as count_order +from + lineitem +where + l_shipdate <= date '1998-12-01' - interval '68 days' +group by + l_returnflag, + l_linestatus +order by + l_returnflag, + l_linestatus; diff --git a/benchmarks/tpc/queries/tpch/q10.sql b/benchmarks/tpc/queries/tpch/q10.sql new file mode 100644 index 0000000000..8475ae5fa2 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q10.sql @@ -0,0 +1,33 @@ +-- CometBench-H query 10 derived from TPC-H query 10 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + c_custkey, + c_name, + sum(l_extendedprice * (1 - l_discount)) as revenue, + c_acctbal, + n_name, + c_address, + c_phone, + c_comment +from + customer, + orders, + lineitem, + nation +where + c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate >= date '1993-07-01' + and o_orderdate < date '1993-07-01' + interval '3' month + and l_returnflag = 'R' + and c_nationkey = n_nationkey +group by + c_custkey, + c_name, + c_acctbal, + c_phone, + n_name, + c_address, + c_comment +order by + revenue desc limit 20; diff --git a/benchmarks/tpc/queries/tpch/q11.sql b/benchmarks/tpc/queries/tpch/q11.sql new file mode 100644 index 0000000000..314b1e6fc7 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q11.sql @@ -0,0 +1,29 @@ +-- CometBench-H query 11 derived from TPC-H query 11 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + ps_partkey, + sum(ps_supplycost * ps_availqty) as value +from + partsupp, + supplier, + nation +where + ps_suppkey = s_suppkey + and s_nationkey = n_nationkey + and n_name = 'ALGERIA' +group by + ps_partkey having + sum(ps_supplycost * ps_availqty) > ( + select + sum(ps_supplycost * ps_availqty) * 0.0001000000 + from + partsupp, + supplier, + nation + where + ps_suppkey = s_suppkey + and s_nationkey = n_nationkey + and n_name = 'ALGERIA' + ) +order by + value desc; diff --git a/benchmarks/tpc/queries/tpch/q12.sql b/benchmarks/tpc/queries/tpch/q12.sql new file mode 100644 index 0000000000..f1abd806aa --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q12.sql @@ -0,0 +1,30 @@ +-- CometBench-H query 12 derived from TPC-H query 12 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + l_shipmode, + sum(case + when o_orderpriority = '1-URGENT' + or o_orderpriority = '2-HIGH' + then 1 + else 0 + end) as high_line_count, + sum(case + when o_orderpriority <> '1-URGENT' + and o_orderpriority <> '2-HIGH' + then 1 + else 0 + end) as low_line_count +from + orders, + lineitem +where + o_orderkey = l_orderkey + and l_shipmode in ('FOB', 'SHIP') + and l_commitdate < l_receiptdate + and l_shipdate < l_commitdate + and l_receiptdate >= date '1995-01-01' + and l_receiptdate < date '1995-01-01' + interval '1' year +group by + l_shipmode +order by + l_shipmode; diff --git a/benchmarks/tpc/queries/tpch/q13.sql b/benchmarks/tpc/queries/tpch/q13.sql new file mode 100644 index 0000000000..03c3f05818 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q13.sql @@ -0,0 +1,22 @@ +-- CometBench-H query 13 derived from TPC-H query 13 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + c_count, + count(*) as custdist +from + ( + select + c_custkey, + count(o_orderkey) + from + customer left outer join orders on + c_custkey = o_custkey + and o_comment not like '%express%requests%' + group by + c_custkey + ) as c_orders (c_custkey, c_count) +group by + c_count +order by + custdist desc, + c_count desc; diff --git a/benchmarks/tpc/queries/tpch/q14.sql b/benchmarks/tpc/queries/tpch/q14.sql new file mode 100644 index 0000000000..9f7b52e548 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q14.sql @@ -0,0 +1,15 @@ +-- CometBench-H query 14 derived from TPC-H query 14 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + 100.00 * sum(case + when p_type like 'PROMO%' + then l_extendedprice * (1 - l_discount) + else 0 + end) / sum(l_extendedprice * (1 - l_discount)) as promo_revenue +from + lineitem, + part +where + l_partkey = p_partkey + and l_shipdate >= date '1995-02-01' + and l_shipdate < date '1995-02-01' + interval '1' month; diff --git a/benchmarks/tpc/queries/tpch/q15.sql b/benchmarks/tpc/queries/tpch/q15.sql new file mode 100644 index 0000000000..dda91adf7b --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q15.sql @@ -0,0 +1,33 @@ +-- CometBench-H query 15 derived from TPC-H query 15 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +create view revenue0 (supplier_no, total_revenue) as + select + l_suppkey, + sum(l_extendedprice * (1 - l_discount)) + from + lineitem + where + l_shipdate >= date '1996-08-01' + and l_shipdate < date '1996-08-01' + interval '3' month + group by + l_suppkey; +select + s_suppkey, + s_name, + s_address, + s_phone, + total_revenue +from + supplier, + revenue0 +where + s_suppkey = supplier_no + and total_revenue = ( + select + max(total_revenue) + from + revenue0 + ) +order by + s_suppkey; +drop view revenue0; diff --git a/benchmarks/tpc/queries/tpch/q16.sql b/benchmarks/tpc/queries/tpch/q16.sql new file mode 100644 index 0000000000..6f6b023acb --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q16.sql @@ -0,0 +1,32 @@ +-- CometBench-H query 16 derived from TPC-H query 16 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + p_brand, + p_type, + p_size, + count(distinct ps_suppkey) as supplier_cnt +from + partsupp, + part +where + p_partkey = ps_partkey + and p_brand <> 'Brand#14' + and p_type not like 'SMALL PLATED%' + and p_size in (14, 6, 5, 31, 49, 15, 41, 47) + and ps_suppkey not in ( + select + s_suppkey + from + supplier + where + s_comment like '%Customer%Complaints%' + ) +group by + p_brand, + p_type, + p_size +order by + supplier_cnt desc, + p_brand, + p_type, + p_size; diff --git a/benchmarks/tpc/queries/tpch/q17.sql b/benchmarks/tpc/queries/tpch/q17.sql new file mode 100644 index 0000000000..8b90e18002 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q17.sql @@ -0,0 +1,19 @@ +-- CometBench-H query 17 derived from TPC-H query 17 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + sum(l_extendedprice) / 7.0 as avg_yearly +from + lineitem, + part +where + p_partkey = l_partkey + and p_brand = 'Brand#42' + and p_container = 'LG BAG' + and l_quantity < ( + select + 0.2 * avg(l_quantity) + from + lineitem + where + l_partkey = p_partkey + ); diff --git a/benchmarks/tpc/queries/tpch/q18.sql b/benchmarks/tpc/queries/tpch/q18.sql new file mode 100644 index 0000000000..e0f733e088 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q18.sql @@ -0,0 +1,34 @@ +-- CometBench-H query 18 derived from TPC-H query 18 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + c_name, + c_custkey, + o_orderkey, + o_orderdate, + o_totalprice, + sum(l_quantity) +from + customer, + orders, + lineitem +where + o_orderkey in ( + select + l_orderkey + from + lineitem + group by + l_orderkey having + sum(l_quantity) > 313 + ) + and c_custkey = o_custkey + and o_orderkey = l_orderkey +group by + c_name, + c_custkey, + o_orderkey, + o_orderdate, + o_totalprice +order by + o_totalprice desc, + o_orderdate limit 100; diff --git a/benchmarks/tpc/queries/tpch/q19.sql b/benchmarks/tpc/queries/tpch/q19.sql new file mode 100644 index 0000000000..8777b2c887 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q19.sql @@ -0,0 +1,37 @@ +-- CometBench-H query 19 derived from TPC-H query 19 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + sum(l_extendedprice* (1 - l_discount)) as revenue +from + lineitem, + part +where + ( + p_partkey = l_partkey + and p_brand = 'Brand#21' + and p_container in ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG') + and l_quantity >= 8 and l_quantity <= 8 + 10 + and p_size between 1 and 5 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ) + or + ( + p_partkey = l_partkey + and p_brand = 'Brand#13' + and p_container in ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK') + and l_quantity >= 20 and l_quantity <= 20 + 10 + and p_size between 1 and 10 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ) + or + ( + p_partkey = l_partkey + and p_brand = 'Brand#52' + and p_container in ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG') + and l_quantity >= 30 and l_quantity <= 30 + 10 + and p_size between 1 and 15 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ); diff --git a/benchmarks/tpc/queries/tpch/q2.sql b/benchmarks/tpc/queries/tpch/q2.sql new file mode 100644 index 0000000000..947413fcf5 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q2.sql @@ -0,0 +1,45 @@ +-- CometBench-H query 2 derived from TPC-H query 2 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + s_acctbal, + s_name, + n_name, + p_partkey, + p_mfgr, + s_address, + s_phone, + s_comment +from + part, + supplier, + partsupp, + nation, + region +where + p_partkey = ps_partkey + and s_suppkey = ps_suppkey + and p_size = 48 + and p_type like '%TIN' + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'ASIA' + and ps_supplycost = ( + select + min(ps_supplycost) + from + partsupp, + supplier, + nation, + region + where + p_partkey = ps_partkey + and s_suppkey = ps_suppkey + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'ASIA' + ) +order by + s_acctbal desc, + n_name, + s_name, + p_partkey limit 100; diff --git a/benchmarks/tpc/queries/tpch/q20.sql b/benchmarks/tpc/queries/tpch/q20.sql new file mode 100644 index 0000000000..d83bfcc696 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q20.sql @@ -0,0 +1,39 @@ +-- CometBench-H query 20 derived from TPC-H query 20 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + s_name, + s_address +from + supplier, + nation +where + s_suppkey in ( + select + ps_suppkey + from + partsupp + where + ps_partkey in ( + select + p_partkey + from + part + where + p_name like 'blanched%' + ) + and ps_availqty > ( + select + 0.5 * sum(l_quantity) + from + lineitem + where + l_partkey = ps_partkey + and l_suppkey = ps_suppkey + and l_shipdate >= date '1993-01-01' + and l_shipdate < date '1993-01-01' + interval '1' year + ) + ) + and s_nationkey = n_nationkey + and n_name = 'KENYA' +order by + s_name; diff --git a/benchmarks/tpc/queries/tpch/q21.sql b/benchmarks/tpc/queries/tpch/q21.sql new file mode 100644 index 0000000000..ce6ade49a3 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q21.sql @@ -0,0 +1,41 @@ +-- CometBench-H query 21 derived from TPC-H query 21 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + s_name, + count(*) as numwait +from + supplier, + lineitem l1, + orders, + nation +where + s_suppkey = l1.l_suppkey + and o_orderkey = l1.l_orderkey + and o_orderstatus = 'F' + and l1.l_receiptdate > l1.l_commitdate + and exists ( + select + * + from + lineitem l2 + where + l2.l_orderkey = l1.l_orderkey + and l2.l_suppkey <> l1.l_suppkey + ) + and not exists ( + select + * + from + lineitem l3 + where + l3.l_orderkey = l1.l_orderkey + and l3.l_suppkey <> l1.l_suppkey + and l3.l_receiptdate > l3.l_commitdate + ) + and s_nationkey = n_nationkey + and n_name = 'ARGENTINA' +group by + s_name +order by + numwait desc, + s_name limit 100; diff --git a/benchmarks/tpc/queries/tpch/q22.sql b/benchmarks/tpc/queries/tpch/q22.sql new file mode 100644 index 0000000000..012f1f698a --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q22.sql @@ -0,0 +1,39 @@ +-- CometBench-H query 22 derived from TPC-H query 22 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + cntrycode, + count(*) as numcust, + sum(c_acctbal) as totacctbal +from + ( + select + substring(c_phone from 1 for 2) as cntrycode, + c_acctbal + from + customer + where + substring(c_phone from 1 for 2) in + ('24', '34', '16', '30', '33', '14', '13') + and c_acctbal > ( + select + avg(c_acctbal) + from + customer + where + c_acctbal > 0.00 + and substring(c_phone from 1 for 2) in + ('24', '34', '16', '30', '33', '14', '13') + ) + and not exists ( + select + * + from + orders + where + o_custkey = c_custkey + ) + ) as custsale +group by + cntrycode +order by + cntrycode; diff --git a/benchmarks/tpc/queries/tpch/q3.sql b/benchmarks/tpc/queries/tpch/q3.sql new file mode 100644 index 0000000000..e277402d88 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q3.sql @@ -0,0 +1,24 @@ +-- CometBench-H query 3 derived from TPC-H query 3 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + l_orderkey, + sum(l_extendedprice * (1 - l_discount)) as revenue, + o_orderdate, + o_shippriority +from + customer, + orders, + lineitem +where + c_mktsegment = 'BUILDING' + and c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate < date '1995-03-15' + and l_shipdate > date '1995-03-15' +group by + l_orderkey, + o_orderdate, + o_shippriority +order by + revenue desc, + o_orderdate limit 10; diff --git a/benchmarks/tpc/queries/tpch/q4.sql b/benchmarks/tpc/queries/tpch/q4.sql new file mode 100644 index 0000000000..814ea8a2ec --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q4.sql @@ -0,0 +1,23 @@ +-- CometBench-H query 4 derived from TPC-H query 4 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + o_orderpriority, + count(*) as order_count +from + orders +where + o_orderdate >= date '1995-04-01' + and o_orderdate < date '1995-04-01' + interval '3' month + and exists ( + select + * + from + lineitem + where + l_orderkey = o_orderkey + and l_commitdate < l_receiptdate + ) +group by + o_orderpriority +order by + o_orderpriority; diff --git a/benchmarks/tpc/queries/tpch/q5.sql b/benchmarks/tpc/queries/tpch/q5.sql new file mode 100644 index 0000000000..5c462c397b --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q5.sql @@ -0,0 +1,26 @@ +-- CometBench-H query 5 derived from TPC-H query 5 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + n_name, + sum(l_extendedprice * (1 - l_discount)) as revenue +from + customer, + orders, + lineitem, + supplier, + nation, + region +where + c_custkey = o_custkey + and l_orderkey = o_orderkey + and l_suppkey = s_suppkey + and c_nationkey = s_nationkey + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'AFRICA' + and o_orderdate >= date '1994-01-01' + and o_orderdate < date '1994-01-01' + interval '1' year +group by + n_name +order by + revenue desc; diff --git a/benchmarks/tpc/queries/tpch/q6.sql b/benchmarks/tpc/queries/tpch/q6.sql new file mode 100644 index 0000000000..22b92f4e91 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q6.sql @@ -0,0 +1,11 @@ +-- CometBench-H query 6 derived from TPC-H query 6 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + sum(l_extendedprice * l_discount) as revenue +from + lineitem +where + l_shipdate >= date '1994-01-01' + and l_shipdate < date '1994-01-01' + interval '1' year + and l_discount between 0.04 - 0.01 and 0.04 + 0.01 + and l_quantity < 24; diff --git a/benchmarks/tpc/queries/tpch/q7.sql b/benchmarks/tpc/queries/tpch/q7.sql new file mode 100644 index 0000000000..26b4078ffc --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q7.sql @@ -0,0 +1,41 @@ +-- CometBench-H query 7 derived from TPC-H query 7 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + supp_nation, + cust_nation, + l_year, + sum(volume) as revenue +from + ( + select + n1.n_name as supp_nation, + n2.n_name as cust_nation, + extract(year from l_shipdate) as l_year, + l_extendedprice * (1 - l_discount) as volume + from + supplier, + lineitem, + orders, + customer, + nation n1, + nation n2 + where + s_suppkey = l_suppkey + and o_orderkey = l_orderkey + and c_custkey = o_custkey + and s_nationkey = n1.n_nationkey + and c_nationkey = n2.n_nationkey + and ( + (n1.n_name = 'GERMANY' and n2.n_name = 'IRAQ') + or (n1.n_name = 'IRAQ' and n2.n_name = 'GERMANY') + ) + and l_shipdate between date '1995-01-01' and date '1996-12-31' + ) as shipping +group by + supp_nation, + cust_nation, + l_year +order by + supp_nation, + cust_nation, + l_year; diff --git a/benchmarks/tpc/queries/tpch/q8.sql b/benchmarks/tpc/queries/tpch/q8.sql new file mode 100644 index 0000000000..c0ac2ecf16 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q8.sql @@ -0,0 +1,39 @@ +-- CometBench-H query 8 derived from TPC-H query 8 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + o_year, + sum(case + when nation = 'IRAQ' then volume + else 0 + end) / sum(volume) as mkt_share +from + ( + select + extract(year from o_orderdate) as o_year, + l_extendedprice * (1 - l_discount) as volume, + n2.n_name as nation + from + part, + supplier, + lineitem, + orders, + customer, + nation n1, + nation n2, + region + where + p_partkey = l_partkey + and s_suppkey = l_suppkey + and l_orderkey = o_orderkey + and o_custkey = c_custkey + and c_nationkey = n1.n_nationkey + and n1.n_regionkey = r_regionkey + and r_name = 'MIDDLE EAST' + and s_nationkey = n2.n_nationkey + and o_orderdate between date '1995-01-01' and date '1996-12-31' + and p_type = 'LARGE PLATED STEEL' + ) as all_nations +group by + o_year +order by + o_year; diff --git a/benchmarks/tpc/queries/tpch/q9.sql b/benchmarks/tpc/queries/tpch/q9.sql new file mode 100644 index 0000000000..ab525c371e --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q9.sql @@ -0,0 +1,34 @@ +-- CometBench-H query 9 derived from TPC-H query 9 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + nation, + o_year, + sum(amount) as sum_profit +from + ( + select + n_name as nation, + extract(year from o_orderdate) as o_year, + l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount + from + part, + supplier, + lineitem, + partsupp, + orders, + nation + where + s_suppkey = l_suppkey + and ps_suppkey = l_suppkey + and ps_partkey = l_partkey + and p_partkey = l_partkey + and o_orderkey = l_orderkey + and s_nationkey = n_nationkey + and p_name like '%moccasin%' + ) as profit +group by + nation, + o_year +order by + nation, + o_year desc; diff --git a/benchmarks/tpc/requirements.txt b/benchmarks/tpc/requirements.txt new file mode 100644 index 0000000000..48f63d718f --- /dev/null +++ b/benchmarks/tpc/requirements.txt @@ -0,0 +1 @@ +pyspark==3.5.2 diff --git a/benchmarks/tpc/run.py b/benchmarks/tpc/run.py index 7f76ccfed0..6cd10636ba 100755 --- a/benchmarks/tpc/run.py +++ b/benchmarks/tpc/run.py @@ -124,7 +124,6 @@ def load_toml(path): "executor_cores": "8", "max_cores": "16", "data_env": "TPCH_DATA", - "queries_env": "TPCH_QUERIES", "format": "parquet", }, "tpcds": { @@ -132,7 +131,6 @@ def load_toml(path): "executor_cores": "8", "max_cores": "16", "data_env": "TPCDS_DATA", - "queries_env": "TPCDS_QUERIES", "format": None, # omit --format for TPC-DS }, } @@ -213,7 +211,7 @@ def check_benchmark_env(config, benchmark): profile = BENCHMARK_PROFILES[benchmark] use_iceberg = config.get("tpcbench_args", {}).get("use_iceberg", False) - required = [profile["queries_env"]] + required = [] if not use_iceberg: required.append(profile["data_env"]) @@ -282,10 +280,6 @@ def build_spark_submit_cmd(config, benchmark, args): data_val = os.environ.get(data_var, "") cmd += ["--data", data_val] - queries_var = profile["queries_env"] - queries_val = os.environ.get(queries_var, "") - cmd += ["--queries", queries_val] - cmd += ["--output", args.output] cmd += ["--iterations", str(args.iterations)] diff --git a/benchmarks/tpc/tpcbench.py b/benchmarks/tpc/tpcbench.py index 1fb53ef85d..747f2d351f 100644 --- a/benchmarks/tpc/tpcbench.py +++ b/benchmarks/tpc/tpcbench.py @@ -26,6 +26,7 @@ import argparse from datetime import datetime import json +import os from pyspark.sql import SparkSession import time from typing import Dict @@ -50,7 +51,6 @@ def main( data_path: str, catalog: str, database: str, - query_path: str, iterations: int, output: str, name: str, @@ -64,6 +64,10 @@ def main( if options is None: options = {} + query_path = os.path.join( + os.path.dirname(os.path.abspath(__file__)), "queries", benchmark + ) + spark = SparkSession.builder \ .appName(f"{name} benchmark derived from {benchmark}") \ .getOrCreate() @@ -112,7 +116,6 @@ def main( results = { 'engine': 'datafusion-comet', 'benchmark': benchmark, - 'query_path': query_path, 'spark_conf': conf_dict, } if using_iceberg: @@ -229,10 +232,6 @@ def main( help="Database containing TPC tables (only used with --catalog)" ) - parser.add_argument( - "--queries", required=True, - help="Path to query SQL files" - ) parser.add_argument( "--iterations", type=int, default=1, help="Number of iterations" @@ -268,7 +267,6 @@ def main( args.data, args.catalog, args.database, - args.queries, args.iterations, args.output, args.name, diff --git a/dev/release/rat_exclude_files.txt b/dev/release/rat_exclude_files.txt index 4ea10c1dff..6553aafc5b 100644 --- a/dev/release/rat_exclude_files.txt +++ b/dev/release/rat_exclude_files.txt @@ -28,3 +28,6 @@ spark/src/test/resources/tpch-extended/q*.sql spark/src/test/resources/test-data/*.csv spark/src/test/resources/test-data/*.ndjson spark/inspections/CometTPC*results.txt +benchmarks/tpc/queries/tpch/q*.sql +benchmarks/tpc/queries/tpcds/q*.sql +benchmarks/tpc/requirements.txt diff --git a/pom.xml b/pom.xml index 1b33fc4757..c5923196ba 100644 --- a/pom.xml +++ b/pom.xml @@ -1069,6 +1069,8 @@ under the License. dev/release/rat_exclude_files.txt dev/release/requirements.txt native/proto/src/generated/** + benchmarks/tpc/queries/** + benchmarks/tpc/requirements.txt From 618717538f02d0cdc4906bdf2ee5975d657ce8d3 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 18 Feb 2026 09:26:26 -0700 Subject: [PATCH 17/24] Support both macOS (arm64) and Linux (amd64) in Docker benchmarks Create arch-agnostic Java symlinks in the Dockerfile using TARGETARCH so the image works on both platforms. Rename JAVA_HOME to BENCH_JAVA_HOME in docker-compose.yml to prevent the host's JAVA_HOME from leaking into containers. Support both table.parquet and table directory layouts in tpcbench.py. Co-Authored-By: Claude Opus 4.6 --- benchmarks/tpc/README.md | 21 +++++++++++++++---- benchmarks/tpc/infra/docker/Dockerfile | 6 +++++- .../tpc/infra/docker/docker-compose.yml | 8 ++++--- benchmarks/tpc/tpcbench.py | 3 +++ 4 files changed, 30 insertions(+), 8 deletions(-) diff --git a/benchmarks/tpc/README.md b/benchmarks/tpc/README.md index 7dc9440e11..d9ceab6f60 100644 --- a/benchmarks/tpc/README.md +++ b/benchmarks/tpc/README.md @@ -215,7 +215,8 @@ physical plan output. ## Running with Docker A Docker Compose setup is provided in `infra/docker/` for running benchmarks in an isolated -Spark standalone cluster. +Spark standalone cluster. The Docker image supports both **Linux (amd64)** and **macOS (arm64)** +via architecture-agnostic Java symlinks created at build time. ### Build the image @@ -225,6 +226,18 @@ From the repository root: docker build -t comet-bench -f benchmarks/tpc/infra/docker/Dockerfile . ``` +### Platform notes + +**macOS (Apple Silicon):** Docker Desktop is required. You may need to add your data +directory (e.g. `/opt`) to Docker Desktop > Settings > Resources > File Sharing before +mounting host volumes. + +**Linux (amd64):** No special configuration is needed. + +The Docker image auto-detects the container architecture (amd64/arm64) and sets up +arch-agnostic Java symlinks. The compose file uses `BENCH_JAVA_HOME` (not `JAVA_HOME`) +to avoid inheriting the host's Java path into the container. + ### Start the cluster Set environment variables pointing to your host paths, then start the Spark master and @@ -263,7 +276,7 @@ For Gluten (requires Java 8), you must restart the **entire cluster** with `JAVA set so that all services (master, workers, and bench) use Java 8: ```shell -export JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64 +export BENCH_JAVA_HOME=/usr/lib/jvm/java-8-openjdk docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml down docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml up -d @@ -275,8 +288,8 @@ docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml \ > **Important:** Only passing `-e JAVA_HOME=...` to the `bench` container is not > sufficient -- the workers also need Java 8 or Gluten will fail at runtime with -> `sun.misc.Unsafe` errors. Switch back to Java 17 (or unset `JAVA_HOME`) and -> restart the cluster before running Comet or Spark benchmarks. +> `sun.misc.Unsafe` errors. Unset `BENCH_JAVA_HOME` (or switch it back to Java 17) +> and restart the cluster before running Comet or Spark benchmarks. ### Memory limits and metrics diff --git a/benchmarks/tpc/infra/docker/Dockerfile b/benchmarks/tpc/infra/docker/Dockerfile index 12aaf3ee5a..774f39cf60 100644 --- a/benchmarks/tpc/infra/docker/Dockerfile +++ b/benchmarks/tpc/infra/docker/Dockerfile @@ -34,7 +34,11 @@ RUN apt-get update \ && rm -rf /var/lib/apt/lists/* # Default to Java 17 (override with JAVA_HOME at runtime for Gluten). -ENV JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64 +# Detect architecture (amd64 or arm64) so the image works on both Linux and macOS. +ARG TARGETARCH +RUN ln -s /usr/lib/jvm/java-17-openjdk-${TARGETARCH} /usr/lib/jvm/java-17-openjdk && \ + ln -s /usr/lib/jvm/java-8-openjdk-${TARGETARCH} /usr/lib/jvm/java-8-openjdk +ENV JAVA_HOME=/usr/lib/jvm/java-17-openjdk # Copy the benchmark scripts into the image. COPY benchmarks/tpc/run.py /opt/benchmarks/run.py diff --git a/benchmarks/tpc/infra/docker/docker-compose.yml b/benchmarks/tpc/infra/docker/docker-compose.yml index 3e0acaccde..03528f8a0c 100644 --- a/benchmarks/tpc/infra/docker/docker-compose.yml +++ b/benchmarks/tpc/infra/docker/docker-compose.yml @@ -32,6 +32,8 @@ # WORKER_MEM_LIMIT - Hard memory limit per worker container (default: 32g) # BENCH_MEM_LIMIT - Hard memory limit for the bench runner (default: 10g) # METRICS_INTERVAL - Metrics collection interval in seconds (default: 1) +# BENCH_JAVA_HOME - Java home inside container (default: /usr/lib/jvm/java-17-openjdk) +# Set to /usr/lib/jvm/java-8-openjdk for Gluten x-volumes: &volumes - ${DATA_DIR:-/tmp/tpc-data}:/data:ro @@ -56,7 +58,7 @@ x-worker: &worker exec /opt/spark/sbin/start-worker.sh spark://spark-master:7077 volumes: *volumes environment: - - JAVA_HOME=${JAVA_HOME:-/usr/lib/jvm/java-17-openjdk-amd64} + - JAVA_HOME=${BENCH_JAVA_HOME:-/usr/lib/jvm/java-17-openjdk} - SPARK_WORKER_CORES=${WORKER_CORES:-8} - SPARK_WORKER_MEMORY=${WORKER_MEMORY:-16g} - SPARK_NO_DAEMONIZE=true @@ -74,7 +76,7 @@ services: - "8080:8080" volumes: *volumes environment: - - JAVA_HOME=${JAVA_HOME:-/usr/lib/jvm/java-17-openjdk-amd64} + - JAVA_HOME=${BENCH_JAVA_HOME:-/usr/lib/jvm/java-17-openjdk} - SPARK_MASTER_HOST=spark-master - SPARK_NO_DAEMONIZE=true @@ -105,7 +107,7 @@ services: command: ["echo", "Use 'docker compose run bench python3 /opt/benchmarks/run.py ...' to run benchmarks"] volumes: *volumes environment: - - JAVA_HOME=${JAVA_HOME:-/usr/lib/jvm/java-17-openjdk-amd64} + - JAVA_HOME=${BENCH_JAVA_HOME:-/usr/lib/jvm/java-17-openjdk} - SPARK_HOME=/opt/spark - SPARK_MASTER=spark://spark-master:7077 - COMET_JAR=/jars/comet.jar diff --git a/benchmarks/tpc/tpcbench.py b/benchmarks/tpc/tpcbench.py index 747f2d351f..1ff54ed3c7 100644 --- a/benchmarks/tpc/tpcbench.py +++ b/benchmarks/tpc/tpcbench.py @@ -106,7 +106,10 @@ def main( print(f"Registering table {table} from {source}") df = spark.table(source) else: + # Support both "customer/" and "customer.parquet/" layouts source = f"{data_path}/{table}.{format}" + if not os.path.exists(source): + source = f"{data_path}/{table}" print(f"Registering table {table} from {source}") df = spark.read.format(format).options(**options).load(source) df.createOrReplaceTempView(table) From 0018f75e7e89839fd186e45bfe43adc07f2a6c9f Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 18 Feb 2026 09:32:26 -0700 Subject: [PATCH 18/24] Improve Docker benchmark setup for macOS Add Docker Desktop memory instructions to README (need >= 48 GB for the default config). Mount Spark logs and work directories to $RESULTS_DIR so executor stderr survives container restarts. Expose port 4040 for the Spark Application UI on the bench container. Co-Authored-By: Claude Opus 4.6 --- benchmarks/tpc/README.md | 17 ++++++++++++----- benchmarks/tpc/infra/docker/docker-compose.yml | 4 ++++ 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/benchmarks/tpc/README.md b/benchmarks/tpc/README.md index d9ceab6f60..dcbd44b083 100644 --- a/benchmarks/tpc/README.md +++ b/benchmarks/tpc/README.md @@ -228,11 +228,18 @@ docker build -t comet-bench -f benchmarks/tpc/infra/docker/Dockerfile . ### Platform notes -**macOS (Apple Silicon):** Docker Desktop is required. You may need to add your data -directory (e.g. `/opt`) to Docker Desktop > Settings > Resources > File Sharing before -mounting host volumes. - -**Linux (amd64):** No special configuration is needed. +**macOS (Apple Silicon):** Docker Desktop is required. + +- **Memory:** Docker Desktop defaults to a small memory allocation (often 8 GB) which + is not enough for Spark benchmarks. Go to **Docker Desktop > Settings > Resources > + Memory** and increase it to at least 48 GB (each worker requests 16 GB for its executor + plus overhead, and the driver needs 8 GB). Without enough memory, executors will be + OOM-killed (exit code 137). +- **File Sharing:** You may need to add your data directory (e.g. `/opt`) to + **Docker Desktop > Settings > Resources > File Sharing** before mounting host volumes. + +**Linux (amd64):** No special configuration is needed. Docker uses cgroup memory limits +directly without a VM layer. The Docker image auto-detects the container architecture (amd64/arm64) and sets up arch-agnostic Java symlinks. The compose file uses `BENCH_JAVA_HOME` (not `JAVA_HOME`) diff --git a/benchmarks/tpc/infra/docker/docker-compose.yml b/benchmarks/tpc/infra/docker/docker-compose.yml index 03528f8a0c..708ea7e835 100644 --- a/benchmarks/tpc/infra/docker/docker-compose.yml +++ b/benchmarks/tpc/infra/docker/docker-compose.yml @@ -41,6 +41,8 @@ x-volumes: &volumes - ${COMET_JAR:-/dev/null}:/jars/comet.jar:ro - ${GLUTEN_JAR:-/dev/null}:/jars/gluten.jar:ro - ${ICEBERG_JAR:-/dev/null}:/jars/iceberg.jar:ro + - ${RESULTS_DIR:-/tmp/bench-results}/logs:/opt/spark/logs + - ${RESULTS_DIR:-/tmp/bench-results}/work:/opt/spark/work x-worker: &worker image: ${BENCH_IMAGE:-comet-bench} @@ -105,6 +107,8 @@ services: # docker compose run bench python3 /opt/benchmarks/run.py \ # --engine comet --benchmark tpch --no-restart command: ["echo", "Use 'docker compose run bench python3 /opt/benchmarks/run.py ...' to run benchmarks"] + ports: + - "4040:4040" volumes: *volumes environment: - JAVA_HOME=${BENCH_JAVA_HOME:-/usr/lib/jvm/java-17-openjdk} From 96ca4e644e165e0a6908f3e98bc69b2b41781954 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 18 Feb 2026 09:54:10 -0700 Subject: [PATCH 19/24] Declare spark_uid ARG to silence Docker build warning Co-Authored-By: Claude Opus 4.6 --- benchmarks/tpc/infra/docker/Dockerfile | 2 ++ 1 file changed, 2 insertions(+) diff --git a/benchmarks/tpc/infra/docker/Dockerfile b/benchmarks/tpc/infra/docker/Dockerfile index 774f39cf60..202ec66de6 100644 --- a/benchmarks/tpc/infra/docker/Dockerfile +++ b/benchmarks/tpc/infra/docker/Dockerfile @@ -59,4 +59,6 @@ RUN chmod +x /opt/benchmarks/collect-metrics.sh WORKDIR /opt/benchmarks +# Defined in the base apache/spark image. +ARG spark_uid USER ${spark_uid} From 7c81c6e344d65d366a60333e078f5f421fc21d63 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 18 Feb 2026 09:55:09 -0700 Subject: [PATCH 20/24] Document Docker Spark UI ports and internal hostname caveat Co-Authored-By: Claude Opus 4.6 --- benchmarks/tpc/README.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/benchmarks/tpc/README.md b/benchmarks/tpc/README.md index dcbd44b083..cdb13431de 100644 --- a/benchmarks/tpc/README.md +++ b/benchmarks/tpc/README.md @@ -279,6 +279,19 @@ docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml \ > config enables event logging. The bench container is ephemeral so this directory > does not persist between runs. +The following UIs are available during a benchmark run: + +| UI | URL | +| -------------------- | ---------------------- | +| Spark Master | http://localhost:8080 | +| Worker 1 | http://localhost:8081 | +| Worker 2 | http://localhost:8082 | +| Spark Application | http://localhost:4040 | + +> **Note:** The Master UI links to the Application UI using the container's internal +> hostname, which is not reachable from the host. Use `http://localhost:4040` directly +> to access the Application UI. + For Gluten (requires Java 8), you must restart the **entire cluster** with `JAVA_HOME` set so that all services (master, workers, and bench) use Java 8: From 512133f183d934e3f160718967ef727abdd24ea9 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 18 Feb 2026 11:10:22 -0700 Subject: [PATCH 21/24] Document platform-specific Docker image builds The Comet JAR contains native libraries for a specific OS/arch. On macOS (Apple Silicon), build for linux/amd64 with --platform to match the standard release JARs. Co-Authored-By: Claude Opus 4.6 --- benchmarks/tpc/README.md | 22 +++++++++++++++++++++- 1 file changed, 21 insertions(+), 1 deletion(-) diff --git a/benchmarks/tpc/README.md b/benchmarks/tpc/README.md index cdb13431de..ab542c6940 100644 --- a/benchmarks/tpc/README.md +++ b/benchmarks/tpc/README.md @@ -220,12 +220,32 @@ via architecture-agnostic Java symlinks created at build time. ### Build the image -From the repository root: +The image must be built for the correct platform to match the native libraries in the +engine JARs (e.g. Comet bundles `libcomet.so` for a specific OS/arch). + +**Linux (amd64):** ```shell docker build -t comet-bench -f benchmarks/tpc/infra/docker/Dockerfile . ``` +**macOS (Apple Silicon):** Build for `linux/amd64` and use a Comet JAR that contains +`linux/amd64` native libraries (the standard release JARs). Docker Desktop runs +amd64 images via Rosetta emulation: + +```shell +docker build --platform linux/amd64 -t comet-bench \ + -f benchmarks/tpc/infra/docker/Dockerfile . +``` + +If you have a Comet JAR with `linux/aarch64` native libraries (e.g. cross-compiled), +you can build a native arm64 image instead: + +```shell +docker build --platform linux/arm64 -t comet-bench \ + -f benchmarks/tpc/infra/docker/Dockerfile . +``` + ### Platform notes **macOS (Apple Silicon):** Docker Desktop is required. From e60799b1e0028f7ee6b94cab5437c940a13369bd Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 18 Feb 2026 11:11:57 -0700 Subject: [PATCH 22/24] Add Dockerfile for building Comet JAR with Linux native libraries On macOS, host-built Comet JARs contain darwin native libraries that don't work inside Linux Docker containers. Dockerfile.build-comet compiles the Comet JAR inside a Linux container, producing a JAR with the correct linux native libraries for the container architecture. Co-Authored-By: Claude Opus 4.6 --- benchmarks/tpc/README.md | 21 ++++--- .../tpc/infra/docker/Dockerfile.build-comet | 56 +++++++++++++++++++ 2 files changed, 68 insertions(+), 9 deletions(-) create mode 100644 benchmarks/tpc/infra/docker/Dockerfile.build-comet diff --git a/benchmarks/tpc/README.md b/benchmarks/tpc/README.md index ab542c6940..4cdbd93e89 100644 --- a/benchmarks/tpc/README.md +++ b/benchmarks/tpc/README.md @@ -229,21 +229,24 @@ engine JARs (e.g. Comet bundles `libcomet.so` for a specific OS/arch). docker build -t comet-bench -f benchmarks/tpc/infra/docker/Dockerfile . ``` -**macOS (Apple Silicon):** Build for `linux/amd64` and use a Comet JAR that contains -`linux/amd64` native libraries (the standard release JARs). Docker Desktop runs -amd64 images via Rosetta emulation: +**macOS (Apple Silicon):** Engine JARs contain platform-specific native libraries. A +Comet JAR built on macOS includes `darwin/aarch64` libraries which won't work inside +Linux containers. You need a JAR with Linux native libraries. + +Build a Comet JAR with Linux native libraries using the provided build Dockerfile: ```shell -docker build --platform linux/amd64 -t comet-bench \ - -f benchmarks/tpc/infra/docker/Dockerfile . +mkdir -p output +docker build -t comet-builder \ + -f benchmarks/tpc/infra/docker/Dockerfile.build-comet . +docker run --rm -v $(pwd)/output:/output comet-builder +export COMET_JAR=$(pwd)/output/comet-spark-spark3.5_2.12-*.jar ``` -If you have a Comet JAR with `linux/aarch64` native libraries (e.g. cross-compiled), -you can build a native arm64 image instead: +Then build the benchmark image (the architecture is auto-detected): ```shell -docker build --platform linux/arm64 -t comet-bench \ - -f benchmarks/tpc/infra/docker/Dockerfile . +docker build -t comet-bench -f benchmarks/tpc/infra/docker/Dockerfile . ``` ### Platform notes diff --git a/benchmarks/tpc/infra/docker/Dockerfile.build-comet b/benchmarks/tpc/infra/docker/Dockerfile.build-comet new file mode 100644 index 0000000000..fe770c0e61 --- /dev/null +++ b/benchmarks/tpc/infra/docker/Dockerfile.build-comet @@ -0,0 +1,56 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Build a Comet JAR with native libraries for the current platform. +# +# This is useful on macOS (Apple Silicon) where the host-built JAR contains +# darwin/aarch64 native libraries but Docker containers need linux/aarch64. +# +# Usage (from repository root): +# docker build -t comet-builder -f benchmarks/tpc/infra/docker/Dockerfile.build-comet . +# docker run --rm -v $(pwd)/output:/output comet-builder +# +# The JAR is copied to ./output/ on the host. + +FROM ubuntu:22.04 AS builder + +ENV DEBIAN_FRONTEND=noninteractive + +# Install build dependencies: Java 17, Maven wrapper prerequisites, Rust toolchain. +RUN apt-get update && apt-get install -y --no-install-recommends \ + openjdk-17-jdk-headless \ + curl ca-certificates git build-essential pkg-config \ + libssl-dev protobuf-compiler \ + && apt-get clean && rm -rf /var/lib/apt/lists/* + +ENV JAVA_HOME=/usr/lib/jvm/java-17-openjdk-${TARGETARCH:-amd64} + +# Install Rust. +RUN curl --proto '=https' --tlsv1.2 -sSf https://sh.rustup.rs | sh -s -- -y +ENV PATH="/root/.cargo/bin:${PATH}" + +WORKDIR /build + +# Copy the full source tree. +COPY . . + +# Build native code + package the JAR (skip tests). +RUN make release + +# The entrypoint copies the built JAR to /output (bind-mounted from host). +RUN mkdir -p /output +CMD cp spark/target/comet-spark-spark3.5_2.12-*-SNAPSHOT.jar /output/ && \ + echo "Comet JAR copied to /output/" && \ + ls -lh /output/*.jar From 51a3660e1f3b4764b46221c37777d4e8b8b20c02 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 18 Feb 2026 13:15:44 -0700 Subject: [PATCH 23/24] Fix Comet build Dockerfile for GLIBC and GCC compatibility Use Ubuntu 20.04 (GLIBC 2.31) to match the apache/spark base image. Install GCC 11 from ubuntu-toolchain-r PPA to work around the memcmp bug in GCC 9 (GCC #95189) that breaks aws-lc-sys. Remove static port 4040 from the bench service in docker-compose.yml to avoid conflicts (use -p 4040:4040 with docker compose run instead). Co-Authored-By: Claude Opus 4.6 --- .../tpc/infra/docker/Dockerfile.build-comet | 38 ++++++++++++++----- .../tpc/infra/docker/docker-compose.yml | 2 - 2 files changed, 29 insertions(+), 11 deletions(-) diff --git a/benchmarks/tpc/infra/docker/Dockerfile.build-comet b/benchmarks/tpc/infra/docker/Dockerfile.build-comet index fe770c0e61..7c0c5e4cd2 100644 --- a/benchmarks/tpc/infra/docker/Dockerfile.build-comet +++ b/benchmarks/tpc/infra/docker/Dockerfile.build-comet @@ -24,18 +24,40 @@ # # The JAR is copied to ./output/ on the host. -FROM ubuntu:22.04 AS builder +# Use Ubuntu 20.04 to match the GLIBC version (2.31) in apache/spark images. +FROM ubuntu:20.04 AS builder +ARG TARGETARCH ENV DEBIAN_FRONTEND=noninteractive -# Install build dependencies: Java 17, Maven wrapper prerequisites, Rust toolchain. +# Install build dependencies: Java 17, Maven wrapper prerequisites, GCC 11. +# Ubuntu 20.04's default GCC 9 has a memcmp bug (GCC #95189) that breaks aws-lc-sys. RUN apt-get update && apt-get install -y --no-install-recommends \ openjdk-17-jdk-headless \ - curl ca-certificates git build-essential pkg-config \ - libssl-dev protobuf-compiler \ + curl ca-certificates git pkg-config \ + libssl-dev unzip software-properties-common \ + && add-apt-repository -y ppa:ubuntu-toolchain-r/test \ + && apt-get update \ + && apt-get install -y --no-install-recommends gcc-11 g++-11 make \ + && update-alternatives --install /usr/bin/gcc gcc /usr/bin/gcc-11 110 \ + && update-alternatives --install /usr/bin/g++ g++ /usr/bin/g++-11 110 \ + && update-alternatives --install /usr/bin/cc cc /usr/bin/gcc-11 110 \ && apt-get clean && rm -rf /var/lib/apt/lists/* -ENV JAVA_HOME=/usr/lib/jvm/java-17-openjdk-${TARGETARCH:-amd64} +# Install protoc 25.x (Ubuntu 22.04's protoc is too old for proto3 optional fields). +ARG PROTOC_VERSION=25.6 +RUN ARCH=$(uname -m) && \ + if [ "$ARCH" = "aarch64" ]; then PROTOC_ARCH="linux-aarch_64"; \ + else PROTOC_ARCH="linux-x86_64"; fi && \ + curl -sLO "https://github.com/protocolbuffers/protobuf/releases/download/v${PROTOC_VERSION}/protoc-${PROTOC_VERSION}-${PROTOC_ARCH}.zip" && \ + unzip -o "protoc-${PROTOC_VERSION}-${PROTOC_ARCH}.zip" -d /usr/local bin/protoc && \ + rm "protoc-${PROTOC_VERSION}-${PROTOC_ARCH}.zip" && \ + protoc --version + +# Set JAVA_HOME and LD_LIBRARY_PATH so the Rust build can find libjvm. +RUN ln -s /usr/lib/jvm/java-17-openjdk-${TARGETARCH} /usr/lib/jvm/java-17-openjdk +ENV JAVA_HOME=/usr/lib/jvm/java-17-openjdk +ENV LD_LIBRARY_PATH=${JAVA_HOME}/lib/server:${LD_LIBRARY_PATH} # Install Rust. RUN curl --proto '=https' --tlsv1.2 -sSf https://sh.rustup.rs | sh -s -- -y @@ -47,10 +69,8 @@ WORKDIR /build COPY . . # Build native code + package the JAR (skip tests). -RUN make release +RUN make release-nogit # The entrypoint copies the built JAR to /output (bind-mounted from host). RUN mkdir -p /output -CMD cp spark/target/comet-spark-spark3.5_2.12-*-SNAPSHOT.jar /output/ && \ - echo "Comet JAR copied to /output/" && \ - ls -lh /output/*.jar +CMD ["sh", "-c", "cp spark/target/comet-spark-spark3.5_2.12-*-SNAPSHOT.jar /output/ && echo 'Comet JAR copied to /output/' && ls -lh /output/*.jar"] diff --git a/benchmarks/tpc/infra/docker/docker-compose.yml b/benchmarks/tpc/infra/docker/docker-compose.yml index 708ea7e835..a8a8d807c1 100644 --- a/benchmarks/tpc/infra/docker/docker-compose.yml +++ b/benchmarks/tpc/infra/docker/docker-compose.yml @@ -107,8 +107,6 @@ services: # docker compose run bench python3 /opt/benchmarks/run.py \ # --engine comet --benchmark tpch --no-restart command: ["echo", "Use 'docker compose run bench python3 /opt/benchmarks/run.py ...' to run benchmarks"] - ports: - - "4040:4040" volumes: *volumes environment: - JAVA_HOME=${BENCH_JAVA_HOME:-/usr/lib/jvm/java-17-openjdk} From 52440438127796ab3aee014f3623acf2f165de9a Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 18 Feb 2026 13:18:56 -0700 Subject: [PATCH 24/24] Use --rm flag in docker compose run examples to prevent port conflicts Add --rm to all docker compose run commands in the README to auto-remove containers on exit, and add -p 4040:4040 to expose the Spark Application UI. Also add output/ to .gitignore for Comet build artifacts. Co-Authored-By: Claude Opus 4.6 --- .gitignore | 1 + benchmarks/tpc/README.md | 16 ++++++++++------ 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/.gitignore b/.gitignore index 4675567ef8..15cac247ed 100644 --- a/.gitignore +++ b/.gitignore @@ -23,3 +23,4 @@ spark/benchmarks .DS_Store comet-event-trace.json __pycache__ +output diff --git a/benchmarks/tpc/README.md b/benchmarks/tpc/README.md index 4cdbd93e89..e8bb0c3a15 100644 --- a/benchmarks/tpc/README.md +++ b/benchmarks/tpc/README.md @@ -287,13 +287,16 @@ between versions by changing the path and restarting. ### Run benchmarks -Use `docker compose run` to execute benchmarks. Pass `--no-restart` since the cluster is -already managed by Compose, and `--output /results` so that output files land in the -mounted results directory (alongside cgroup metrics CSVs): +Use `docker compose run --rm` to execute benchmarks. The `--rm` flag removes the +container when it exits, preventing port conflicts on subsequent runs. Pass +`--no-restart` since the cluster is already managed by Compose, and `--output /results` +so that output files land in the mounted results directory (alongside cgroup metrics +CSVs): ```shell docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml \ - run bench sh -c 'mkdir -p /tmp/spark-events && \ + run --rm -p 4040:4040 bench \ + sh -c 'mkdir -p /tmp/spark-events && \ python3 /opt/benchmarks/run.py \ --engine comet --benchmark tpch --output /results --no-restart' ``` @@ -302,7 +305,8 @@ docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml \ > config enables event logging. The bench container is ephemeral so this directory > does not persist between runs. -The following UIs are available during a benchmark run: +The `-p 4040:4040` flag exposes the Spark Application UI on the host. The following +UIs are available during a benchmark run: | UI | URL | | -------------------- | ---------------------- | @@ -324,7 +328,7 @@ docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml down docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml up -d docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml \ - run bench sh -c 'mkdir -p /tmp/spark-events && \ + run --rm bench sh -c 'mkdir -p /tmp/spark-events && \ python3 /opt/benchmarks/run.py \ --engine gluten --benchmark tpch --output /results --no-restart' ```