Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
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
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
108 changes: 108 additions & 0 deletions datafusion/proto/tests/cases/roundtrip_physical_plan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1780,3 +1780,111 @@ async fn test_tpch_part_in_list_query_with_real_parquet_data() -> Result<()> {

Ok(())
}

/// Helper function to create a SessionContext with all TPC-H tables registered as external tables
async fn tpch_context() -> Result<SessionContext> {
use datafusion_common::test_util::datafusion_test_data;

let ctx = SessionContext::new();
let test_data = datafusion_test_data();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

there is also parquet_test_data() which stores data on datafusion/parquet-testing


// TPC-H table names
let tables = [
"part", "supplier", "partsupp", "customer", "orders", "lineitem", "nation",
"region",
];

// Create external tables for all TPC-H tables
for table in &tables {
let table_sql = format!(
"CREATE EXTERNAL TABLE {table} STORED AS PARQUET LOCATION '{test_data}/tpch_{table}_small.parquet'"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we can also use register_parquet

);
ctx.sql(&table_sql).await.map_err(|e| {
DataFusionError::External(
format!("Failed to create {table} table: {e}").into(),
)
})?;
}

Ok(ctx)
}

/// Helper function to get TPC-H query SQL
fn get_tpch_query_sql(query: usize) -> Result<Vec<String>> {
use std::fs;

if !(1..=22).contains(&query) {
return Err(DataFusionError::External(
format!("Invalid TPC-H query number: {query}").into(),
));
}

let filename = format!("../../benchmarks/queries/q{query}.sql");
let contents = fs::read_to_string(&filename).map_err(|e| {
DataFusionError::External(
format!("Failed to read query file {filename}: {e}").into(),
)
})?;

Ok(contents
.split(';')
.map(|s| s.trim())
.filter(|s| !s.is_empty())
.map(|s| s.to_string())
.collect())
}

// Skip q16 until https://github.com/apache/datafusion/issues/16665 is fixed
#[tokio::test]
async fn test_serialize_deserialize_tpch_queries() -> Result<()> {
// Create context with TPC-H tables
let ctx = tpch_context().await?;

// repeat to run all 22 queries except q16 due to bug above
for query in 1..=22 {
if query == 16 {
continue;
}
// run all statements in the query
let sql = get_tpch_query_sql(query)?;
for stmt in sql {
let logical_plan = ctx.sql(&stmt).await?.into_unoptimized_plan();
let optimized_plan = ctx.state().optimize(&logical_plan)?;
let physical_plan = ctx.state().create_physical_plan(&optimized_plan).await?;

// serialize the physical plan
let codec = DefaultPhysicalExtensionCodec {};
let proto =
PhysicalPlanNode::try_from_physical_plan(physical_plan.clone(), &codec)?;

// deserialize the physical plan
let _deserialized_plan =
proto.try_into_physical_plan(&ctx, ctx.runtime_env().as_ref(), &codec)?;
}
}

Ok(())
}

// Only 4 queries pass: q3, q5, q10, q12
// The rest fails with 2 different reasons:
// - q16 fails at deserialization step: https://github.com/apache/datafusion/issues/16665
// - Other queries fails due to mismatch between the serialized and deserialized physical plans
Copy link
Contributor Author

@NGA-TRAN NGA-TRAN Jul 10, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@alamb : Let me know if you think a different bug ticket for this is necessary. I am happy to file it

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since I think @XiangpengHao says that #16744 does not fix them all, I do think we should file a separate bug to track the other failures. Who knows -- maybe even other people will fix them like @XiangpengHao did for #16665 ❤️

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Created #16772

// Ignore the test until the bugs are fixed
#[ignore]
#[tokio::test]
async fn test_round_trip_tpch_queries() -> Result<()> {
// Create context with TPC-H tables
let ctx = tpch_context().await?;

// repeat to run all 22 queries
for query in 1..=22 {
// run all statements in the query
let sql = get_tpch_query_sql(query)?;
for stmt in sql {
roundtrip_test_sql_with_context(&stmt, &ctx).await?;
}
}

Ok(())
}