Skip to content
Open
Show file tree
Hide file tree
Changes from 4 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
29 changes: 28 additions & 1 deletion release/nightly_tests/dataset/tpch/common.py
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,34 @@
"column13": "l_shipinstruct",
"column14": "l_shipmode",
"column15": "l_comment",
}
},
"customer": {
"column00": "c_custkey",
"column01": "c_name",
"column02": "c_address",
"column03": "c_nationkey",
"column04": "c_phone",
"column05": "c_acctbal",
"column06": "c_mktsegment",
"column07": "c_comment",
},
"orders": {
"column00": "o_orderkey",
"column01": "o_custkey",
"column02": "o_orderstatus",
"column03": "o_totalprice",
"column04": "o_orderdate",
"column05": "o_orderpriority",
"column06": "o_clerk",
"column07": "o_shippriority",
"column08": "o_comment",
},
"nation": {
"column00": "n_nationkey",
"column01": "n_name",
"column02": "n_regionkey",
"column03": "n_comment",
},
}


Expand Down
93 changes: 93 additions & 0 deletions release/nightly_tests/dataset/tpch/tpch_q10.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,93 @@
import ray
from ray.data.aggregate import Sum
from ray.data.expressions import col
from common import parse_tpch_args, load_table, to_f64, run_tpch_benchmark


def main(args):
def benchmark_fn():
from datetime import datetime

# Load all required tables
customer = load_table("customer", args.sf)
orders = load_table("orders", args.sf)
lineitem = load_table("lineitem", args.sf)
nation = load_table("nation", args.sf)

# Q10 parameters
date = datetime(1993, 10, 1)
# Calculate end date (3 months later)
if date.month <= 9:
end_date = datetime(date.year, date.month + 3, date.day)
else:
end_date = datetime(date.year + 1, date.month + 3 - 12, date.day)

# Filter orders by date (3 months range)
orders_filtered = orders.filter(
expr=((col("o_orderdate") >= date) & (col("o_orderdate") < end_date))
)

# Filter lineitem by return flag
lineitem_filtered = lineitem.filter(expr=col("l_returnflag") == "R")

# Join lineitem with orders
lineitem_orders = lineitem_filtered.join(
orders_filtered,
join_type="inner",
num_partitions=100,
on=("l_orderkey",),
right_on=("o_orderkey",),
)

# Join with customer
ds = lineitem_orders.join(
customer,
join_type="inner",
num_partitions=100,
on=("o_custkey",),
right_on=("c_custkey",),
)

# Join with nation
ds = ds.join(
nation,
join_type="inner",
num_partitions=100,
on=("c_nationkey",),
right_on=("n_nationkey",),
)

# Calculate revenue
ds = ds.with_column(
"revenue",
to_f64(col("l_extendedprice")) * (1 - to_f64(col("l_discount"))),
)

# Aggregate by customer key, customer name, address, phone, account balance, nation, and region
_ = (
ds.groupby(
[
"c_custkey",
"c_name",
"c_acctbal",
"n_name",
"c_address",
"c_phone",
"c_comment",
]
)
.aggregate(Sum(on="revenue", alias_name="revenue"))
.sort(key="revenue", descending=True)
.materialize()
)

# Report arguments for the benchmark.
return vars(args)

run_tpch_benchmark("tpch_q10", benchmark_fn)


if __name__ == "__main__":
ray.init()
args = parse_tpch_args()
main(args)
71 changes: 71 additions & 0 deletions release/nightly_tests/dataset/tpch/tpch_q18.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
import ray
from ray.data.aggregate import Sum
from ray.data.expressions import col
from common import parse_tpch_args, load_table, run_tpch_benchmark


def main(args):
def benchmark_fn():

# Load all required tables
customer = load_table("customer", args.sf)
orders = load_table("orders", args.sf)
lineitem = load_table("lineitem", args.sf)

# Q18 parameters
quantity = 300

# Calculate total quantity per order
lineitem_quantity = lineitem.groupby("l_orderkey").aggregate(
Sum(on="l_quantity", alias_name="total_quantity")
)

# Filter orders with total quantity > threshold
large_orders = lineitem_quantity.filter(expr=col("total_quantity") > quantity)

# Join lineitem with large orders
lineitem_large = lineitem.join(
large_orders,
join_type="inner",
num_partitions=100,
on="l_orderkey",
)

# Join with orders
lineitem_orders = lineitem_large.join(
orders,
join_type="inner",
num_partitions=100,
on=("l_orderkey",),
right_on=("o_orderkey",),
)

# Join with customer
ds = lineitem_orders.join(
customer,
join_type="inner",
num_partitions=100,
on=("o_custkey",),
right_on=("c_custkey",),
)

# Aggregate by customer name, customer key, order key, and order date
_ = (
ds.groupby(
["c_name", "c_custkey", "o_orderkey", "o_orderdate", "o_totalprice"]
)
.aggregate(Sum(on="l_quantity", alias_name="sum_quantity"))
.sort(key=["o_totalprice", "o_orderdate"], descending=[True, False])
.materialize()
)

# Report arguments for the benchmark.
return vars(args)

run_tpch_benchmark("tpch_q18", benchmark_fn)


if __name__ == "__main__":
ray.init()
args = parse_tpch_args()
main(args)
68 changes: 68 additions & 0 deletions release/nightly_tests/dataset/tpch/tpch_q3.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
import ray
from ray.data.aggregate import Sum
from ray.data.expressions import col
from common import parse_tpch_args, load_table, to_f64, run_tpch_benchmark


def main(args):
def benchmark_fn():
from datetime import datetime

# Load all required tables
customer = load_table("customer", args.sf)
orders = load_table("orders", args.sf)
lineitem = load_table("lineitem", args.sf)

# Q3 parameters
date = datetime(1995, 3, 15)
segment = "BUILDING"

# Filter customer by segment
customer_filtered = customer.filter(expr=col("c_mktsegment") == segment)

# Filter orders by date
orders_filtered = orders.filter(expr=col("o_orderdate") < date)

# Join orders with customer
orders_customer = orders_filtered.join(
customer_filtered,
join_type="inner",
num_partitions=100,
on=("o_custkey",),
right_on=("c_custkey",),
)

# Join with lineitem and filter by ship date
lineitem_filtered = lineitem.filter(expr=col("l_shipdate") > date)
ds = lineitem_filtered.join(
orders_customer,
join_type="inner",
num_partitions=100,
on=("l_orderkey",),
right_on=("o_orderkey",),
)

# Calculate revenue
ds = ds.with_column(
"revenue",
to_f64(col("l_extendedprice")) * (1 - to_f64(col("l_discount"))),
)

# Aggregate by order key, order date, and ship priority
_ = (
ds.groupby(["l_orderkey", "o_orderdate", "o_shippriority"])
.aggregate(Sum(on="revenue", alias_name="revenue"))
.sort(key=["revenue", "o_orderdate"], descending=[True, False])
.materialize()
)

# Report arguments for the benchmark.
return vars(args)

run_tpch_benchmark("tpch_q3", benchmark_fn)


if __name__ == "__main__":
ray.init()
args = parse_tpch_args()
main(args)
39 changes: 39 additions & 0 deletions release/release_data_tests.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -737,6 +737,19 @@
timeout: 5400
script: python tpch/tpch_q1.py --sf 100

- name: "tpch_q3_{{scaling}}"
python: "3.10"
matrix:
setup:
scaling: [fixed_size, autoscaling]

cluster:
cluster_compute: "{{scaling}}_all_to_all_compute.yaml"

run:
timeout: 5400
script: python tpch/tpch_q3.py --sf 100

- name: "tpch_q6_{{scaling}}"
python: "3.10"
matrix:
Expand All @@ -750,6 +763,32 @@
timeout: 5400
script: python tpch/tpch_q6.py --sf 100

- name: "tpch_q10_{{scaling}}"
python: "3.10"
matrix:
setup:
scaling: [fixed_size, autoscaling]

cluster:
cluster_compute: "{{scaling}}_all_to_all_compute.yaml"

run:
timeout: 5400
script: python tpch/tpch_q10.py --sf 100

- name: "tpch_q18_{{scaling}}"
python: "3.10"
matrix:
setup:
scaling: [fixed_size, autoscaling]

cluster:
cluster_compute: "{{scaling}}_all_to_all_compute.yaml"

run:
timeout: 5400
script: python tpch/tpch_q18.py --sf 100

#################################################
# Cross-AZ RPC fault tolerance test
#################################################
Expand Down