TPC Benchmark™ DS (TPC-DS) benchmark suite

What it is

From TPC-DS specification document:

“TPC-DS is a decision support benchmark that models several generally applicable aspects of a decision support system, including queries and data maintenance. The benchmark provides a representative evaluation of performance as a general purpose decision support system.”

In general, TPC-DS is:

Table schemas

Schema Overview

From TPC-DS specification document:

The TPC-DS schema models the sales and sales returns process for an organization that employs three primary sales channels: stores, catalogs, and the Internet. The schema includes seven fact tables:

  • A pair of fact tables focused on the product sales and returns for each of the three channels
  • A single fact table that models inventory for the catalog and internet sales channels.

In addition, the schema includes 17 dimension tables that are associated with all sales channels.

Tables

The queries

TPC-DS benchmark contains 99 distinct SQL-99 queries (including OLAP extensions). Each query answers a business question, which illustrates the business context in which the query could be used.

All queries are “templated” with random input parameters and used to compare completeness and performance of SQL implementations.

Input data

Input data source:

TPC-DS extension in Beam

Beam provides a simplified implementation of TPC-DS benchmark.

Reasons

There are several reasons to have TPC-DS benchmarks in Beam:

Queries

All TPC-DS queries in Beam are pre-generated and stored in the provided artifacts.

For the moment, 28 out of 103 SQL queries (99 + 4) successfully pass by running with Beam SQL transform since not all SQL-99 operations are supported.

Currently (as of Beam 2.40.0 release) supported queries are:

Tables

All TPC-DS table schemas are stored in the provided artifacts.

Input data

CSV and Parquet input data has been pre-generated and staged in the Google Cloud Storage bucket gs://beam-tpcds.

CSV datasets

Staged in gs://beam-tpcds/datasets/text/* bucket, spread by different data scale factors.

Parquet datasets

Staged in gs://beam-tpcds/datasets/parquet/nonpartitioned/ and gs://beam-tpcds/datasets/parquet/partitioned/, spreaded by different data scale factors.

For partitioned version, some large tables has been pre-partitioned by a date column into several files in the bucket.

Runtime

TPC-DS extension for Beam can only be run in Batch mode and supports these runners for the moment (not tested with other runners):

Benchmark launch configuration

The TPC-DS launcher accepts the --runner argument as usual for programs that use Beam PipelineOptions to manage their command line arguments. In addition to this, the necessary dependencies must be configured.

When running via Gradle, the following two parameters control the execution:

-P tpcds.args
    The command line to pass to the TPC-DS main program.

-P tpcds.runner
The Gradle project name of the runner, such as ":runners:spark:3" or
":runners:flink:1.17. The project names can be found in the root
    `settings.gradle.kts`.

Test data has to be generated before running a suite and stored to accessible file system. The query results will be written into output files.

Common configuration parameters

Scale factor size of input dataset (1GB / 10GB / 100GB / 1000GB):

--dataSize=<1GB|10GB|100GB|1000GB>

Path to input datasets directory:

--dataDirectory=<path to dir>

Path to results directory:

--resultsDirectory=<path to dir>

Format of input files:

--sourceType=<CSV|PARQUET>

Select queries to run (comma separated list of query numbers or all for all queries):

--queries=<1,2,...N|all>

Number of queries N to run in parallel:

--tpcParallel=N

Running TPC-DS

Here are some examples demonstrating how to run TPC-DS benchmarks on different runners.

Running suite on the SparkRunner (local) with Query3 against 1Gb dataset in Parquet format:

./gradlew :sdks:java:testing:tpcds:run \
    -Ptpcds.runner=":runners:spark:3" \
    -Ptpcds.args="
        --runner=SparkRunner
        --dataSize=1GB
        --sourceType=PARQUET
        --dataDirectory=gs://beam-tpcds/datasets/parquet/partitioned
        --resultsDirectory=/tmp/beam-tpcds/results/spark/
        --tpcParallel=1
        --queries=3"

Running suite on the FlinkRunner (local) with Query7 and Query10 in parallel against 10Gb dataset in CSV format:

./gradlew :sdks:java:testing:tpcds:run \
    -Ptpcds.runner=":runners:flink:1.13" \
    -Ptpcds.args="
        --runner=FlinkRunner
        --parallelism=2
        --dataSize=10GB
        --sourceType=CSV
        --dataDirectory=gs://beam-tpcds/datasets/csv
        --resultsDirectory=/tmp/beam-tpcds/results/flink/
        --tpcParallel=2
        --queries=7,10"

Running suite on the DataflowRunner with all queries against 100GB dataset in PARQUET format:

./gradlew :sdks:java:testing:tpcds:run \
    -Ptpcds.runner=":runners:google-cloud-dataflow-java" \
    -Ptpcds.args="
        --runner=DataflowRunner
        --region=<region_name>
        --project=<project_name>
        --numWorkers=4
        --maxNumWorkers=4
        --autoscalingAlgorithm=NONE
        --dataSize=100GB
        --sourceType=PARQUET
        --dataDirectory=gs://beam-tpcds/datasets/parquet/partitioned
        --resultsDirectory=/tmp/beam-tpcds/results/dataflow/
        --tpcParallel=4
        --queries=all"