Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[EPIC] Support TPC-DS benchmarks #4763

Open
2 of 4 tasks
andygrove opened this issue Dec 28, 2022 · 7 comments
Open
2 of 4 tasks

[EPIC] Support TPC-DS benchmarks #4763

andygrove opened this issue Dec 28, 2022 · 7 comments
Assignees
Labels
enhancement New feature or request

Comments

@andygrove
Copy link
Member

andygrove commented Dec 28, 2022

Is your feature request related to a problem or challenge? Please describe what you are trying to do.
I would like to be able to run all TPC-DS queries with DataFusion, but some are not yet supported.

Old description:

I am testing with [SQLBench-DS](https://github.com/sql-benchmarks/sqlbench-ds) and I am seeing some failures. Many of these affect multiple queries but I have just listed a single example query here for each type of error.

- https://github.com/apache/arrow-datafusion/issues/4794
- https://github.com/apache/arrow-datafusion/issues/123
- `At least two values are needed to calculate variance` (q17)
- `The type of Int32 = Int64 of binary physical should be same` (q72)
- `physical plan is not yet implemented for GROUPING aggregate function` (q27)
- `Projections require unique expression names but the expression "MAX(customer_demographics.cd_dep_count)" at position 6 and "MAX(customer_demographics.cd_dep_count)" at position 7 have the same name. Consider aliasing ("AS") one of them.` (q35)
- `The function Stddev does not support inputs of type Decimal128(7, 2).` (q74)

Describe the solution you'd like
Support all the queries.

Describe alternatives you've considered
N/A

Additional context
N/A

@andygrove andygrove added the enhancement New feature or request label Dec 28, 2022
@andygrove andygrove changed the title Support TPC-DS benchmarks [EPIC] Support TPC-DS benchmarks Dec 28, 2022
@andygrove andygrove self-assigned this Jul 10, 2024
@andygrove
Copy link
Member Author

andygrove commented Jul 10, 2024

I just tried running TPC-DS with DataFusion 39 and these are the failing queries currently:

Executing query 10 from ../../tpcds/queries-spark/q10.sql
Fail: This feature is not implemented: Physical plan does not support logical expression Exists(Exists { subquery: <subquery>, negated: false })

Executing query 11 from ../../tpcds/queries-spark/q11.sql

Executing query 27 from ../../tpcds/queries-spark/q27.sql
Fail: External error: This feature is not implemented: physical plan is not yet implemented for GROUPING aggregate function

Executing query 35 from ../../tpcds/queries-spark/q35.sql
Fail: Error during planning: Projections require unique expression names but the expression "STDDEV(customer_demographics.cd_dep_count)" at position 6 and "STDDEV(customer_demographics.cd_dep_count)" at position 7 have the same name. Consider aliasing ("AS") one of them.

Executing query 36 from ../../tpcds/queries-spark/q36.sql
Fail: External error: External error: This feature is not implemented: physical plan is not yet implemented for GROUPING aggregate function

Executing query 41 from ../../tpcds/queries-spark/q41.sql
Fail: check_analyzed_plan
caused by
Error during planning: Correlated column is not allowed in predicate: item.i_manufact = outer_ref(i1.i_manufact) AND (item.i_category = Utf8("Women") AND (item.i_color = Utf8("cream") OR item.i_color = Utf8("ghost")) AND (item.i_units = Utf8("Ton") OR item.i_units = Utf8("Gross")) AND (item.i_size = Utf8("economy") OR item.i_size = Utf8("small")) OR item.i_category = Utf8("Women") AND (item.i_color = Utf8("midnight") OR item.i_color = Utf8("burlywood")) AND (item.i_units = Utf8("Tsp") OR item.i_units = Utf8("Bundle")) AND (item.i_size = Utf8("medium") OR item.i_size = Utf8("extra large")) OR item.i_category = Utf8("Men") AND (item.i_color = Utf8("lavender") OR item.i_color = Utf8("azure")) AND (item.i_units = Utf8("Each") OR item.i_units = Utf8("Lb")) AND (item.i_size = Utf8("large") OR item.i_size = Utf8("N/A")) OR item.i_category = Utf8("Men") AND (item.i_color = Utf8("chocolate") OR item.i_color = Utf8("steel")) AND (item.i_units = Utf8("N/A") OR item.i_units = Utf8("Dozen")) AND (item.i_size = Utf8("economy") OR item.i_size = Utf8("small"))) OR item.i_manufact = outer_ref(i1.i_manufact) AND (item.i_category = Utf8("Women") AND (item.i_color = Utf8("floral") OR item.i_color = Utf8("royal")) AND (item.i_units = Utf8("Unknown") OR item.i_units = Utf8("Tbl")) AND (item.i_size = Utf8("economy") OR item.i_size = Utf8("small")) OR item.i_category = Utf8("Women") AND (item.i_color = Utf8("navy") OR item.i_color = Utf8("forest")) AND (item.i_units = Utf8("Bunch") OR item.i_units = Utf8("Dram")) AND (item.i_size = Utf8("medium") OR item.i_size = Utf8("extra large")) OR item.i_category = Utf8("Men") AND (item.i_color = Utf8("cyan") OR item.i_color = Utf8("indian")) AND (item.i_units = Utf8("Carton") OR item.i_units = Utf8("Cup")) AND (item.i_size = Utf8("large") OR item.i_size = Utf8("N/A")) OR item.i_category = Utf8("Men") AND (item.i_color = Utf8("coral") OR item.i_color = Utf8("pale")) AND (item.i_units = Utf8("Pallet") OR item.i_units = Utf8("Gram")) AND (item.i_size = Utf8("economy") OR item.i_size = Utf8("small")))

Executing query 45 from ../../tpcds/queries-spark/q45.sql
Fail: This feature is not implemented: Physical plan does not support logical expression InSubquery(InSubquery { expr: Column(Column { relation: Some(Bare { table: "item" }), name: "i_item_id" }), subquery: <subquery>, negated: false })

Executing query 70 from ../../tpcds/queries-spark/q70.sql
Fail: External error: External error: This feature is not implemented: physical plan is not yet implemented for GROUPING aggregate function

Executing query 72 from ../../tpcds/queries-spark/q72.sql
Fail: type_coercion
caused by
Error during planning: Cannot coerce arithmetic expression Date32 + Int64 to valid types

Executing query 74 from ../../tpcds/queries-spark/q74.sql
Fail: Error during planning: No function matches the given name and argument types 'STDDEV(Decimal128(7, 2))'. You might need to add explicit type casts.
	Candidate functions:
	STDDEV(Int8/Int16/Int32/Int64/UInt8/UInt16/UInt32/UInt64/Float32/Float64)

Executing query 75 from ../../tpcds/queries-spark/q75.sql
Fail: External error: External error: External error: External error: Arrow error: Invalid argument error: RowConverter column schema mismatch, expected Int32 got Int64

Executing query 86 from ../../tpcds/queries-spark/q86.sql
Fail: External error: External error: This feature is not implemented: physical plan is not yet implemented for GROUPING aggregate function

eejbyfeldt added a commit to eejbyfeldt/datafusion that referenced this issue Oct 22, 2024
This patch makes it so that rules the configure an `apply_order` will
also include subqueries in their traversel.

This is a step twoards being able to run TPC-DS q41 (apache#4763) which has
an expressions that needs simplification before we can decorrelate the
subquery.

This closes apache#3770 and maybe apache#2480
eejbyfeldt added a commit to eejbyfeldt/datafusion that referenced this issue Oct 22, 2024
This patch makes it so that rules the configure an `apply_order` will
also include subqueries in their traversel.

This is a step twoards being able to run TPC-DS q41 (apache#4763) which has
an expressions that needs simplification before we can decorrelate the
subquery.

This closes apache#3770 and maybe apache#2480
@Dandandan
Copy link
Contributor

I think q41 is the latest one that has to be supported, is that right @eejbyfeldt ?

@Dandandan
Copy link
Contributor

Actually, some of those errors seem to be runtime errors, so we need to check against some dataset as well.
I'll create some issues for testing this.

Dandandan pushed a commit that referenced this issue Oct 23, 2024
This patch makes it so that rules the configure an `apply_order` will
also include subqueries in their traversel.

This is a step twoards being able to run TPC-DS q41 (#4763) which has
an expressions that needs simplification before we can decorrelate the
subquery.

This closes #3770 and maybe #2480
@eejbyfeldt
Copy link
Contributor

If I try to run the version of the queries we have in the datafusion repo against scala factor 1. Only q41 produces a hard error. But there are likely issues with the results for some of them.

I'll create some issues for testing this.

Sounds great! In the long run it would be nice to have it part of CI like we do for TPC-H.

@Dandandan
Copy link
Contributor

Cool, that sounds promising @eejbyfeldt thanks for the hard work on supporting them!

@Dandandan
Copy link
Contributor

@andygrove @eejbyfeldt I updated the epic with tasks to support / test / verify the queries.

@onursatici
Copy link
Contributor

this problem with q35, taken from the description above:

- `Projections require unique expression names but the expression "MAX(customer_demographics.cd_dep_count)" at position 6 and "MAX(customer_demographics.cd_dep_count)" at position 7 have the same name. Consider aliasing ("AS") one of them.` (q35)

seems to be some sort of bug in query generation for tpc-ds. For me the provided dsqgen binary creates the query 35 as something like:

select   
  ca_state,
  cd_gender,
  cd_marital_status,
  cd_dep_count,
  count(*) cnt1,
  max(cd_dep_count),
  stddev_samp(cd_dep_count),
  stddev_samp(cd_dep_count),
  cd_dep_employed_count,
  count(*) cnt2,
  max(cd_dep_employed_count),
  stddev_samp(cd_dep_employed_count),
  stddev_samp(cd_dep_employed_count),
  cd_dep_college_count,
  count(*) cnt3,
  max(cd_dep_college_count),
  stddev_samp(cd_dep_college_count),
  stddev_samp(cd_dep_college_count)
 from
  customer c,customer_address ca,customer_demographics
  ...

This has multiple columns that would resolve to the same name, like stddev_samp(cd_dep_count, violating the unique column name constraint enforced in datafusion logical plan builder.

I believe this is not intentional, as the query template goes like this:

 [_LIMITA] select [_LIMITB]  
  ca_state,
  cd_gender,
  cd_marital_status,
  cd_dep_count,
  count(*) cnt1,
  [AGGONE](cd_dep_count),
  [AGGTWO](cd_dep_count),
  [AGGTHREE](cd_dep_count),
  cd_dep_employed_count,
  count(*) cnt2,
  [AGGONE](cd_dep_employed_count),
  [AGGTWO](cd_dep_employed_count),
  [AGGTHREE](cd_dep_employed_count),
  cd_dep_college_count,
  count(*) cnt3,
  [AGGONE](cd_dep_college_count),
  [AGGTWO](cd_dep_college_count),
  [AGGTHREE](cd_dep_college_count)
 from
  customer c,customer_address ca,customer_demographics

latest spec mentions this about q35:

Qualification Substitution Parameters:
• YEAR.01 = 2002
• AGGONE = min
• AGGTWO = max
• AGGTHREE = avg

so all aggregates are different than each other, if the template was used with the substitutions listed in the spec, no duplicates should occur, and this query can be executed successfully with the main branch of datafusion as of now

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
enhancement New feature or request
Projects
None yet
Development

No branches or pull requests

4 participants