diff --git a/datafusion_ray/context.py b/datafusion_ray/context.py index e67d074..b853e5c 100644 --- a/datafusion_ray/context.py +++ b/datafusion_ray/context.py @@ -77,7 +77,9 @@ def _get_worker_inputs( plan_bytes = datafusion_ray.serialize_execution_plan(stage.get_execution_plan()) futures = [] opt = {} - opt["resources"] = {"worker": 1e-3} + # TODO not sure why we had this but my Ray cluster could not find suitable resource + # until I commented this out + # opt["resources"] = {"worker": 1e-3} opt["num_returns"] = output_partitions_count for part in range(concurrency): ids, inputs = _get_worker_inputs(part) @@ -93,7 +95,6 @@ def _get_worker_inputs( def execute_query_stage( query_stages: list[QueryStage], stage_id: int, - use_ray_shuffle: bool, ) -> tuple[int, list[ray.ObjectRef]]: """ Execute a query stage on the workers. @@ -106,7 +107,7 @@ def execute_query_stage( child_futures = [] for child_id in stage.get_child_stage_ids(): child_futures.append( - execute_query_stage.remote(query_stages, child_id, use_ray_shuffle) + execute_query_stage.remote(query_stages, child_id) ) # if the query stage has a single output partition then we need to execute for the output @@ -133,33 +134,28 @@ def _get_worker_inputs( ) -> tuple[list[tuple[int, int, int]], list[ray.ObjectRef]]: ids = [] futures = [] - if use_ray_shuffle: - for child_stage_id, child_futures in child_outputs: - for i, lst in enumerate(child_futures): - if isinstance(lst, list): - for j, f in enumerate(lst): - if concurrency == 1 or j == part: - # If concurrency is 1, pass in all shuffle partitions. Otherwise, - # only pass in the partitions that match the current worker partition. - ids.append((child_stage_id, i, j)) - futures.append(f) - elif concurrency == 1 or part == 0: - ids.append((child_stage_id, i, 0)) - futures.append(lst) + for child_stage_id, child_futures in child_outputs: + for i, lst in enumerate(child_futures): + if isinstance(lst, list): + for j, f in enumerate(lst): + if concurrency == 1 or j == part: + # If concurrency is 1, pass in all shuffle partitions. Otherwise, + # only pass in the partitions that match the current worker partition. + ids.append((child_stage_id, i, j)) + futures.append(f) + elif concurrency == 1 or part == 0: + ids.append((child_stage_id, i, 0)) + futures.append(lst) return ids, futures - # if we are using disk-based shuffle, wait until the child stages to finish - # writing the shuffle files to disk first. - if not use_ray_shuffle: - ray.get([f for _, lst in child_outputs for f in lst]) - # schedule the actual execution workers plan_bytes = datafusion_ray.serialize_execution_plan(stage.get_execution_plan()) futures = [] opt = {} - opt["resources"] = {"worker": 1e-3} - if use_ray_shuffle: - opt["num_returns"] = output_partitions_count + # TODO not sure why we had this but my Ray cluster could not find suitable resource + # until I commented this out + #opt["resources"] = {"worker": 1e-3} + opt["num_returns"] = output_partitions_count for part in range(concurrency): ids, inputs = _get_worker_inputs(part) futures.append( @@ -210,10 +206,9 @@ def execute_query_partition( class DatafusionRayContext: - def __init__(self, num_workers: int = 1, use_ray_shuffle: bool = False): - self.ctx = Context(num_workers, use_ray_shuffle) + def __init__(self, num_workers: int = 1): + self.ctx = Context(num_workers) self.num_workers = num_workers - self.use_ray_shuffle = use_ray_shuffle def register_csv(self, table_name: str, path: str, has_header: bool): self.ctx.register_csv(table_name, path, has_header) @@ -234,23 +229,7 @@ def sql(self, sql: str) -> pa.RecordBatch: graph = self.ctx.plan(sql) final_stage_id = graph.get_final_query_stage().id() - if self.use_ray_shuffle: - partitions = schedule_execution(graph, final_stage_id, True) - else: - # serialize the query stages and store in Ray object store - query_stages = [ - datafusion_ray.serialize_execution_plan( - graph.get_query_stage(i).get_execution_plan() - ) - for i in range(final_stage_id + 1) - ] - # schedule execution - future = execute_query_stage.remote( - query_stages, - final_stage_id, - self.use_ray_shuffle, - ) - _, partitions = ray.get(future) + partitions = schedule_execution(graph, final_stage_id, True) # assert len(partitions) == 1, len(partitions) result_set = ray.get(partitions[0]) return result_set diff --git a/datafusion_ray/main.py b/datafusion_ray/main.py index cfc08ec..a05f8d5 100644 --- a/datafusion_ray/main.py +++ b/datafusion_ray/main.py @@ -31,9 +31,9 @@ RESULTS_DIR = f"results-sf{SF}" -def setup_context(use_ray_shuffle: bool, num_workers: int = 2) -> DatafusionRayContext: +def setup_context(num_workers: int = 2) -> DatafusionRayContext: print(f"Using {num_workers} workers") - ctx = DatafusionRayContext(num_workers, use_ray_shuffle) + ctx = DatafusionRayContext(num_workers) for table in [ "customer", "lineitem", @@ -103,10 +103,9 @@ def compare(q: int): def tpch_bench(): ray.init(resources={"worker": 1}) num_workers = int(ray.cluster_resources().get("worker", 1)) * NUM_CPUS_PER_WORKER - use_ray_shuffle = False - ctx = setup_context(use_ray_shuffle, num_workers) + ctx = setup_context(num_workers) # t = tpch_timing(ctx, 11, print_result=True) - # print(f"query,{t},{use_ray_shuffle},{num_workers}") + # print(f"query,{t},{num_workers}") # return run_id = time.strftime("%Y-%m-%d-%H-%M-%S") with open(f"results-sf{SF}-{run_id}.csv", "w") as fout: diff --git a/docs/testing.md b/docs/testing.md new file mode 100644 index 0000000..bd08518 --- /dev/null +++ b/docs/testing.md @@ -0,0 +1,50 @@ +# Distributed Testing + +Install Ray on at least two nodes. + +https://docs.ray.io/en/latest/ray-overview/installation.html + +```shell +sudo apt install -y python3-pip python3.12-venv +python3 -m venv venv +source venv/bin/activate +pip3 install -U "ray[default]" +``` + +## Start Ray Head Node + +```shell +ray start --head --node-ip-address=10.0.0.23 --port=6379 --dashboard-host=0.0.0.0 +``` + +## Start Ray Worker Nodes(s) + +```shell +ray start --address=10.0.0.23:6379 --redis-password='5241590000000000' +``` + +## Install DataFusion Ray (on each node) + +Clone the repo with the version that you want to test. Run `maturin build --release` in the virtual env. + +```shell +curl --proto '=https' --tlsv1.2 -sSf https://sh.rustup.rs | sh +. "$HOME/.cargo/env" +``` + +```shell +pip3 install maturin +``` + +```shell +git clone https://github.com/apache/datafusion-ray.git +cd datafusion-ray +maturin develop --release +``` + +## Submit Job + +```shell +cd examples +RAY_ADDRESS='http://10.0.0.23:8265' ray job submit --working-dir `pwd` -- python3 tips.py +``` diff --git a/examples/tips.py b/examples/tips.py index 30117c6..e72425a 100644 --- a/examples/tips.py +++ b/examples/tips.py @@ -16,7 +16,6 @@ # under the License. import os -import pandas as pd import ray from datafusion_ray import DatafusionRayContext @@ -26,8 +25,11 @@ # Start a local cluster ray.init(resources={"worker": 1}) +# Connect to a cluster +# ray.init() + # Create a context and register a table -ctx = DatafusionRayContext(2, use_ray_shuffle=True) +ctx = DatafusionRayContext(2) # Register either a CSV or Parquet file # ctx.register_csv("tips", f"{SCRIPT_DIR}/tips.csv", True) ctx.register_parquet("tips", f"{SCRIPT_DIR}/tips.parquet") diff --git a/src/context.rs b/src/context.rs index 6821ba6..560f506 100644 --- a/src/context.rs +++ b/src/context.rs @@ -21,12 +21,10 @@ use crate::utils::wait_for_future; use datafusion::arrow::pyarrow::FromPyArrow; use datafusion::arrow::pyarrow::ToPyArrow; use datafusion::arrow::record_batch::RecordBatch; -use datafusion::config::Extensions; use datafusion::error::{DataFusionError, Result}; use datafusion::execution::context::TaskContext; use datafusion::execution::disk_manager::DiskManagerConfig; use datafusion::execution::memory_pool::FairSpillPool; -use datafusion::execution::options::ReadOptions; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::physical_plan::{displayable, ExecutionPlan}; use datafusion::prelude::*; @@ -47,13 +45,12 @@ type PyResultSet = Vec; #[pyclass(name = "Context", module = "datafusion_ray", subclass)] pub struct PyContext { pub(crate) ctx: SessionContext, - use_ray_shuffle: bool, } #[pymethods] impl PyContext { #[new] - pub fn new(target_partitions: usize, use_ray_shuffle: bool) -> Result { + pub fn new(target_partitions: usize) -> Result { let config = SessionConfig::default() .with_target_partitions(target_partitions) .with_batch_size(16 * 1024) @@ -67,11 +64,8 @@ impl PyContext { .with_memory_pool(Arc::new(FairSpillPool::new(mem_pool_size))) .with_disk_manager(DiskManagerConfig::new_specified(vec!["/tmp".into()])); let runtime = Arc::new(RuntimeEnv::new(runtime_config)?); - let ctx = SessionContext::with_config_rt(config, runtime); - Ok(Self { - ctx, - use_ray_shuffle, - }) + let ctx = SessionContext::new_with_config_rt(config, runtime); + Ok(Self { ctx }) } pub fn register_csv( @@ -94,9 +88,9 @@ impl PyContext { pub fn register_datalake_table( &self, - name: &str, - path: Vec, - py: Python, + _name: &str, + _path: Vec, + _py: Python, ) -> PyResult<()> { // let options = ParquetReadOptions::default(); // let listing_options = options.to_listing_options(&self.ctx.state().config()); @@ -119,7 +113,7 @@ impl PyContext { let df = wait_for_future(py, self.ctx.sql(sql))?; let plan = wait_for_future(py, df.create_physical_plan())?; - let graph = make_execution_graph(plan.clone(), self.use_ray_shuffle)?; + let graph = make_execution_graph(plan.clone())?; // debug logging let mut stages = graph.query_stages.values().collect::>(); diff --git a/src/planner.rs b/src/planner.rs index 7a3824d..93b04a6 100644 --- a/src/planner.rs +++ b/src/planner.rs @@ -18,7 +18,6 @@ use crate::query_stage::PyQueryStage; use crate::query_stage::QueryStage; use crate::shuffle::{RayShuffleReaderExec, RayShuffleWriterExec}; -use crate::shuffle::{ShuffleReaderExec, ShuffleWriterExec}; use datafusion::error::Result; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion::physical_plan::repartition::RepartitionExec; @@ -30,7 +29,6 @@ use pyo3::prelude::*; use std::collections::HashMap; use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::Arc; -use uuid::Uuid; #[pyclass(name = "ExecutionGraph", module = "datafusion_ray", subclass)] pub struct PyExecutionGraph { @@ -114,12 +112,9 @@ impl ExecutionGraph { } } -pub fn make_execution_graph( - plan: Arc, - use_ray_shuffle: bool, -) -> Result { +pub fn make_execution_graph(plan: Arc) -> Result { let mut graph = ExecutionGraph::new(); - let root = generate_query_stages(plan, &mut graph, use_ray_shuffle)?; + let root = generate_query_stages(plan, &mut graph)?; // We force the final stage to produce a single partition to return // to the driver. This might not suit ETL workloads. if root.properties().output_partitioning().partition_count() > 1 { @@ -136,13 +131,12 @@ pub fn make_execution_graph( fn generate_query_stages( plan: Arc, graph: &mut ExecutionGraph, - use_ray_shuffle: bool, ) -> Result> { // recurse down first let new_children: Vec> = plan .children() .into_iter() - .map(|x| generate_query_stages(x.clone(), graph, use_ray_shuffle)) + .map(|x| generate_query_stages(x.clone(), graph)) .collect::>>()?; let plan = with_new_children_if_necessary(plan, new_children)?; @@ -162,7 +156,6 @@ fn generate_query_stages( plan.children()[0].clone(), graph, partitioning_scheme.clone(), - use_ray_shuffle, ), } } else if plan @@ -180,7 +173,6 @@ fn generate_query_stages( coalesce_input.clone(), graph, partitioning_scheme.to_owned(), - use_ray_shuffle, )?; with_new_children_if_necessary(plan, vec![new_input]) } else { @@ -204,29 +196,16 @@ fn create_shuffle_exchange( plan: Arc, graph: &mut ExecutionGraph, partitioning_scheme: Partitioning, - use_ray_shuffle: bool, ) -> Result> { // introduce shuffle to produce one output partition let stage_id = graph.next_id(); - // create temp dir for stage shuffle files - let temp_dir = create_temp_dir(stage_id)?; - let shuffle_writer_input = plan.clone(); - let shuffle_writer: Arc = if use_ray_shuffle { - Arc::new(RayShuffleWriterExec::new( - stage_id, - shuffle_writer_input, - partitioning_scheme.clone(), - )) - } else { - Arc::new(ShuffleWriterExec::new( - stage_id, - shuffle_writer_input, - partitioning_scheme.clone(), - &temp_dir, - )) - }; + let shuffle_writer: Arc = Arc::new(RayShuffleWriterExec::new( + stage_id, + shuffle_writer_input, + partitioning_scheme.clone(), + )); debug!( "Created shuffle writer with output partitioning {:?}", @@ -235,28 +214,11 @@ fn create_shuffle_exchange( let stage_id = graph.add_query_stage(stage_id, shuffle_writer); // replace the plan with a shuffle reader - if use_ray_shuffle { - Ok(Arc::new(RayShuffleReaderExec::new( - stage_id, - plan.schema(), - partitioning_scheme, - ))) - } else { - Ok(Arc::new(ShuffleReaderExec::new( - stage_id, - plan.schema(), - partitioning_scheme, - &temp_dir, - ))) - } -} - -fn create_temp_dir(stage_id: usize) -> Result { - let uuid = Uuid::new_v4(); - let temp_dir = format!("/tmp/ray-sql-{uuid}-stage-{stage_id}"); - debug!("Creating temp shuffle dir: {temp_dir}"); - std::fs::create_dir(&temp_dir)?; - Ok(temp_dir) + Ok(Arc::new(RayShuffleReaderExec::new( + stage_id, + plan.schema(), + partitioning_scheme, + ))) } #[cfg(test)] @@ -390,7 +352,10 @@ mod test { async fn do_test(n: u8) -> TestResult<()> { let tpch_path_env_var = "TPCH_DATA_PATH"; - let data_path = env::var(tpch_path_env_var).expect(&format!("Environment variable {} not found", tpch_path_env_var)); + let data_path = env::var(tpch_path_env_var).expect(&format!( + "Environment variable {} not found", + tpch_path_env_var + )); let file = format!("testdata/queries/q{n}.sql"); let sql = fs::read_to_string(&file)?; @@ -424,7 +389,7 @@ mod test { )); output.push_str("DataFusion Ray Distributed Plan\n===========\n\n"); - let graph = make_execution_graph(plan, false)?; + let graph = make_execution_graph(plan)?; for id in 0..=graph.get_final_query_stage().id { let query_stage = graph.query_stages.get(&id).unwrap(); output.push_str(&format!( diff --git a/src/proto/datafusion_ray.proto b/src/proto/datafusion_ray.proto index f64dcaf..6a71005 100644 --- a/src/proto/datafusion_ray.proto +++ b/src/proto/datafusion_ray.proto @@ -10,35 +10,11 @@ import "datafusion.proto"; message RaySqlExecNode { oneof PlanType { - ShuffleReaderExecNode shuffle_reader = 1; - ShuffleWriterExecNode shuffle_writer = 2; RayShuffleReaderExecNode ray_shuffle_reader = 3; RayShuffleWriterExecNode ray_shuffle_writer = 4; } } -message ShuffleReaderExecNode { - // stage to read from - uint32 stage_id = 1; - // schema of the shuffle stage - datafusion.Schema schema = 2; - // this must match the output partitioning of the writer we are reading from - datafusion.PhysicalHashRepartition partitioning = 3; - // directory for shuffle files - string shuffle_dir = 4; -} - -message ShuffleWriterExecNode { - // stage that is writing the shuffle files - uint32 stage_id = 1; - // plan to execute - datafusion.PhysicalPlanNode plan = 2; - // output partitioning schema - datafusion.PhysicalHashRepartition partitioning = 3; - // directory for shuffle files - string shuffle_dir = 4; -} - message RayShuffleReaderExecNode { // stage to read from uint32 stage_id = 1; diff --git a/src/query_stage.rs b/src/query_stage.rs index 2a2f7af..bb9f375 100644 --- a/src/query_stage.rs +++ b/src/query_stage.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::shuffle::{RayShuffleReaderExec, ShuffleCodec, ShuffleReaderExec}; +use crate::shuffle::{RayShuffleReaderExec, ShuffleCodec}; use datafusion::error::Result; use datafusion::physical_plan::{ExecutionPlan, Partitioning}; use datafusion::prelude::SessionContext; @@ -110,9 +110,7 @@ impl QueryStage { } fn collect_child_stage_ids(plan: &dyn ExecutionPlan, ids: &mut Vec) { - if let Some(shuffle_reader) = plan.as_any().downcast_ref::() { - ids.push(shuffle_reader.stage_id); - } else if let Some(shuffle_reader) = plan.as_any().downcast_ref::() { + if let Some(shuffle_reader) = plan.as_any().downcast_ref::() { ids.push(shuffle_reader.stage_id); } else { for child_plan in plan.children() { diff --git a/src/shuffle/codec.rs b/src/shuffle/codec.rs index 9bb3f4b..ca20e92 100644 --- a/src/shuffle/codec.rs +++ b/src/shuffle/codec.rs @@ -16,27 +16,19 @@ // under the License. use crate::protobuf::ray_sql_exec_node::PlanType; -use crate::protobuf::{ - RayShuffleReaderExecNode, RayShuffleWriterExecNode, RaySqlExecNode, ShuffleReaderExecNode, - ShuffleWriterExecNode, -}; -use crate::shuffle::{ - RayShuffleReaderExec, RayShuffleWriterExec, ShuffleReaderExec, ShuffleWriterExec, -}; +use crate::protobuf::{RayShuffleReaderExecNode, RayShuffleWriterExecNode, RaySqlExecNode}; +use crate::shuffle::{RayShuffleReaderExec, RayShuffleWriterExec}; use datafusion::arrow::datatypes::SchemaRef; use datafusion::common::{DataFusionError, Result}; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::FunctionRegistry; -use datafusion::logical_expr::{AggregateUDF, ScalarUDF, WindowUDF}; use datafusion::physical_plan::{ExecutionPlan, Partitioning}; use datafusion_proto::physical_plan::from_proto::parse_protobuf_hash_partitioning; use datafusion_proto::physical_plan::to_proto::serialize_physical_expr; use datafusion_proto::physical_plan::PhysicalExtensionCodec; use datafusion_proto::physical_plan::{AsExecutionPlan, DefaultPhysicalExtensionCodec}; -use datafusion_proto::protobuf::{self, PhysicalExprNode}; -use datafusion_proto::protobuf::{PhysicalHashRepartition, PhysicalPlanNode}; +use datafusion_proto::protobuf::{self, PhysicalHashRepartition, PhysicalPlanNode}; use prost::Message; -use std::collections::HashSet; use std::sync::Arc; #[derive(Debug)] @@ -54,41 +46,6 @@ impl PhysicalExtensionCodec for ShuffleCodec { .map_err(|e| DataFusionError::Internal(format!("failed to decode plan: {e:?}")))?; let extension_codec = DefaultPhysicalExtensionCodec {}; match node.plan_type { - Some(PlanType::ShuffleReader(reader)) => { - let schema = reader.schema.as_ref().unwrap(); - let schema: SchemaRef = Arc::new(schema.try_into().unwrap()); - let hash_part = parse_protobuf_hash_partitioning( - reader.partitioning.as_ref(), - registry, - &schema, - &extension_codec, - )?; - Ok(Arc::new(ShuffleReaderExec::new( - reader.stage_id as usize, - schema, - hash_part.unwrap(), - &reader.shuffle_dir, - ))) - } - Some(PlanType::ShuffleWriter(writer)) => { - let plan = writer.plan.unwrap().try_into_physical_plan( - registry, - &RuntimeEnv::default(), - self, - )?; - let hash_part = parse_protobuf_hash_partitioning( - writer.partitioning.as_ref(), - registry, - plan.schema().as_ref(), - &extension_codec, - )?; - Ok(Arc::new(ShuffleWriterExec::new( - writer.stage_id as usize, - plan, - hash_part.unwrap(), - &writer.shuffle_dir, - ))) - } Some(PlanType::RayShuffleReader(reader)) => { let schema = reader.schema.as_ref().unwrap(); let schema: SchemaRef = Arc::new(schema.try_into().unwrap()); @@ -131,29 +88,7 @@ impl PhysicalExtensionCodec for ShuffleCodec { node: Arc, buf: &mut Vec, ) -> Result<(), DataFusionError> { - let plan = if let Some(reader) = node.as_any().downcast_ref::() { - let schema: protobuf::Schema = reader.schema().try_into().unwrap(); - let partitioning = - encode_partitioning_scheme(reader.properties().output_partitioning())?; - let reader = ShuffleReaderExecNode { - stage_id: reader.stage_id as u32, - schema: Some(schema), - partitioning: Some(partitioning), - shuffle_dir: reader.shuffle_dir.clone(), - }; - PlanType::ShuffleReader(reader) - } else if let Some(writer) = node.as_any().downcast_ref::() { - let plan = PhysicalPlanNode::try_from_physical_plan(writer.plan.clone(), self)?; - let partitioning = - encode_partitioning_scheme(writer.properties().output_partitioning())?; - let writer = ShuffleWriterExecNode { - stage_id: writer.stage_id as u32, - plan: Some(plan), - partitioning: Some(partitioning), - shuffle_dir: writer.shuffle_dir.clone(), - }; - PlanType::ShuffleWriter(writer) - } else if let Some(reader) = node.as_any().downcast_ref::() { + let plan = if let Some(reader) = node.as_any().downcast_ref::() { let schema: protobuf::Schema = reader.schema().try_into().unwrap(); let partitioning = encode_partitioning_scheme(reader.properties().output_partitioning())?; @@ -201,27 +136,3 @@ fn encode_partitioning_scheme(partitioning: &Partitioning) -> Result HashSet { - HashSet::new() - } - - fn udf(&self, name: &str) -> datafusion::common::Result> { - Err(DataFusionError::Plan(format!("Invalid UDF: {name}"))) - } - - fn udaf(&self, name: &str) -> datafusion::common::Result> { - Err(DataFusionError::Plan(format!("Invalid UDAF: {name}"))) - } - - fn udwf(&self, name: &str) -> datafusion::common::Result> { - Err(DataFusionError::Plan(format!("Invalid UDAWF: {name}"))) - } - - fn expr_planners(&self) -> Vec> { - todo!() - } -} diff --git a/src/shuffle/mod.rs b/src/shuffle/mod.rs index cfa3980..4d2709c 100644 --- a/src/shuffle/mod.rs +++ b/src/shuffle/mod.rs @@ -17,11 +17,7 @@ mod codec; mod ray_shuffle; -mod reader; -mod writer; pub use codec::ShuffleCodec; pub use ray_shuffle::RayShuffleReaderExec; pub use ray_shuffle::RayShuffleWriterExec; -pub use reader::ShuffleReaderExec; -pub use writer::ShuffleWriterExec; diff --git a/src/shuffle/reader.rs b/src/shuffle/reader.rs deleted file mode 100644 index 62b845b..0000000 --- a/src/shuffle/reader.rs +++ /dev/null @@ -1,191 +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. - -use crate::shuffle::ray_shuffle::CombinedRecordBatchStream; -use datafusion::arrow::datatypes::SchemaRef; -use datafusion::arrow::ipc::reader::FileReader; -use datafusion::arrow::record_batch::RecordBatch; -use datafusion::common::Statistics; -use datafusion::error::{DataFusionError, Result}; -use datafusion::execution::context::TaskContext; -use datafusion::physical_expr::expressions::UnKnownColumn; -use datafusion::physical_expr::{EquivalenceProperties, PhysicalSortExpr}; -use datafusion::physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, RecordBatchStream, - SendableRecordBatchStream, -}; -use futures::Stream; -use glob::glob; -use log::debug; -use std::any::Any; -use std::fmt::Formatter; -use std::fs::File; -use std::pin::Pin; -use std::sync::Arc; -use std::task::{Context, Poll}; - -#[derive(Debug)] -pub struct ShuffleReaderExec { - /// Query stage to read from - pub stage_id: usize, - /// The output schema of the query stage being read from - schema: SchemaRef, - - properties: PlanProperties, - /// Directory to read shuffle files from - pub shuffle_dir: String, -} - -impl ShuffleReaderExec { - pub fn new( - stage_id: usize, - schema: SchemaRef, - partitioning: Partitioning, - shuffle_dir: &str, - ) -> Self { - let partitioning = match partitioning { - Partitioning::Hash(expr, n) if expr.is_empty() => Partitioning::UnknownPartitioning(n), - Partitioning::Hash(expr, n) => { - // workaround for DataFusion bug https://github.com/apache/arrow-datafusion/issues/5184 - Partitioning::Hash( - expr.into_iter() - .filter(|e| e.as_any().downcast_ref::().is_none()) - .collect(), - n, - ) - } - _ => partitioning, - }; - - let properties = PlanProperties::new( - EquivalenceProperties::new(schema.clone()), - partitioning, - datafusion::physical_plan::ExecutionMode::Unbounded, - ); - - Self { - stage_id, - schema, - properties, - shuffle_dir: shuffle_dir.to_string(), - } - } -} - -impl ExecutionPlan for ShuffleReaderExec { - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> SchemaRef { - self.schema.clone() - } - - fn children(&self) -> Vec<&Arc> { - vec![] - } - - fn with_new_children( - self: Arc, - _: Vec>, - ) -> datafusion::common::Result> { - Ok(self) - } - - fn execute( - &self, - partition: usize, - _context: Arc, - ) -> datafusion::common::Result { - let pattern = format!( - "/{}/shuffle_{}_*_{partition}.arrow", - self.shuffle_dir, self.stage_id - ); - let mut streams: Vec = vec![]; - for entry in glob(&pattern).expect("Failed to read glob pattern") { - let file = entry.unwrap(); - debug!( - "ShuffleReaderExec partition {} reading from stage {} file {}", - partition, - self.stage_id, - file.display() - ); - let reader = FileReader::try_new(File::open(&file)?, None)?; - let stream = LocalShuffleStream::new(reader); - if self.schema != stream.schema() { - return Err(DataFusionError::Internal( - "Not all shuffle files have the same schema".to_string(), - )); - } - streams.push(Box::pin(stream)); - } - Ok(Box::pin(CombinedRecordBatchStream::new( - self.schema.clone(), - streams, - ))) - } - - fn statistics(&self) -> Result { - Ok(Statistics::new_unknown(&self.schema)) - } - - fn name(&self) -> &str { - "shuffle reader" - } - - fn properties(&self) -> &datafusion::physical_plan::PlanProperties { - &self.properties - } -} - -impl DisplayAs for ShuffleReaderExec { - fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { - write!( - f, - "ShuffleReaderExec(stage_id={}, input_partitioning={:?})", - self.stage_id, - self.properties().partitioning - ) - } -} - -struct LocalShuffleStream { - reader: FileReader, -} - -impl LocalShuffleStream { - pub fn new(reader: FileReader) -> Self { - LocalShuffleStream { reader } - } -} - -impl Stream for LocalShuffleStream { - type Item = datafusion::error::Result; - - fn poll_next(mut self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { - if let Some(batch) = self.reader.next() { - return Poll::Ready(Some(batch.map_err(|e| e.into()))); - } - Poll::Ready(None) - } -} - -impl RecordBatchStream for LocalShuffleStream { - fn schema(&self) -> SchemaRef { - self.reader.schema() - } -} diff --git a/src/shuffle/writer.rs b/src/shuffle/writer.rs deleted file mode 100644 index 9684406..0000000 --- a/src/shuffle/writer.rs +++ /dev/null @@ -1,310 +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. - -use datafusion::arrow::array::Int32Array; -use datafusion::arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use datafusion::arrow::ipc::writer::FileWriter; -use datafusion::arrow::record_batch::RecordBatch; -use datafusion::arrow::util::pretty::pretty_format_batches; -use datafusion::common::{Result, Statistics}; -use datafusion::execution::context::TaskContext; -use datafusion::physical_expr::expressions::UnKnownColumn; -use datafusion::physical_expr::{EquivalenceProperties, PhysicalSortExpr}; -use datafusion::physical_plan::common::IPCWriter; -use datafusion::physical_plan::memory::MemoryStream; -use datafusion::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricBuilder}; -use datafusion::physical_plan::repartition::BatchPartitioner; -use datafusion::physical_plan::stream::RecordBatchStreamAdapter; -use datafusion::physical_plan::{ - metrics, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, - RecordBatchStream, SendableRecordBatchStream, -}; -use datafusion_proto::protobuf::PartitionStats; -use futures::StreamExt; -use futures::TryStreamExt; -use log::debug; -use std::any::Any; -use std::fmt::Formatter; -use std::fs::File; -use std::path::Path; -use std::pin::Pin; -use std::sync::Arc; - -#[derive(Debug)] -pub struct ShuffleWriterExec { - pub stage_id: usize, - pub(crate) plan: Arc, - /// Output partitioning - properties: PlanProperties, - /// Directory to write shuffle files from - pub shuffle_dir: String, - /// Metrics - pub metrics: ExecutionPlanMetricsSet, -} - -impl ShuffleWriterExec { - pub fn new( - stage_id: usize, - plan: Arc, - partitioning: Partitioning, - shuffle_dir: &str, - ) -> Self { - let partitioning = match partitioning { - Partitioning::Hash(expr, n) if expr.is_empty() => Partitioning::UnknownPartitioning(n), - Partitioning::Hash(expr, n) => { - // workaround for DataFusion bug https://github.com/apache/arrow-datafusion/issues/5184 - Partitioning::Hash( - expr.into_iter() - .filter(|e| e.as_any().downcast_ref::().is_none()) - .collect(), - n, - ) - } - _ => partitioning, - }; - let properties = PlanProperties::new( - EquivalenceProperties::new(plan.schema()), - partitioning, - datafusion::physical_plan::ExecutionMode::Unbounded, - ); - - Self { - stage_id, - plan, - properties, - shuffle_dir: shuffle_dir.to_string(), - metrics: ExecutionPlanMetricsSet::new(), - } - } -} - -impl ExecutionPlan for ShuffleWriterExec { - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> SchemaRef { - self.plan.schema() - } - - fn children(&self) -> Vec<&Arc> { - vec![&self.plan] - } - - fn with_new_children( - self: Arc, - _: Vec>, - ) -> Result> { - unimplemented!() - } - - fn execute( - &self, - input_partition: usize, - context: Arc, - ) -> Result { - debug!( - "ShuffleWriterExec[stage={}].execute(input_partition={input_partition})", - self.stage_id - ); - - let mut stream = self.plan.execute(input_partition, context)?; - let write_time = - MetricBuilder::new(&self.metrics).subset_time("write_time", input_partition); - let repart_time = - MetricBuilder::new(&self.metrics).subset_time("repart_time", input_partition); - - let stage_id = self.stage_id; - let partitioning = self.properties().output_partitioning().to_owned(); - let partition_count = partitioning.partition_count(); - let shuffle_dir = self.shuffle_dir.clone(); - - let results = async move { - match &partitioning { - Partitioning::RoundRobinBatch(_) => { - unimplemented!() - } - Partitioning::UnknownPartitioning(_) => { - // stream the results from the query, preserving the input partitioning - let file = - format!("/{shuffle_dir}/shuffle_{stage_id}_{input_partition}_0.arrow"); - debug!("Executing query and writing results to {file}"); - let stats = write_stream_to_disk(&mut stream, &file, &write_time).await?; - debug!( - "Query completed. Shuffle write time: {}. Rows: {}.", - write_time, stats.num_rows - ); - } - Partitioning::Hash(_, _) => { - // we won't necessary produce output for every possible partition, so we - // create writers on demand - let mut writers: Vec> = vec![]; - for _ in 0..partition_count { - writers.push(None); - } - - let mut partitioner = - BatchPartitioner::try_new(partitioning.clone(), repart_time.clone())?; - - let mut rows = 0; - - while let Some(result) = stream.next().await { - let input_batch = result?; - rows += input_batch.num_rows(); - - debug!( - "ShuffleWriterExec[stage={}] writing batch:\n{}", - stage_id, - pretty_format_batches(&[input_batch.clone()])? - ); - - //write_metrics.input_rows.add(input_batch.num_rows()); - - partitioner.partition(input_batch, |output_partition, output_batch| { - match &mut writers[output_partition] { - Some(w) => { - w.write(&output_batch)?; - } - None => { - let path = format!( - "/{shuffle_dir}/shuffle_{stage_id}_{input_partition}_{output_partition}.arrow", - ); - let path = Path::new(&path); - debug!("ShuffleWriterExec[stage={}] Writing results to {:?}", stage_id, path); - - let mut writer = IPCWriter::new(path, stream.schema().as_ref())?; - - writer.write(&output_batch)?; - writers[output_partition] = Some(writer); - } - } - Ok(()) - })?; - } - - for (i, w) in writers.iter_mut().enumerate() { - match w { - Some(w) => { - w.finish()?; - debug!( - "ShuffleWriterExec[stage={}] Finished writing shuffle partition {} at {:?}. Batches: {}. Rows: {}. Bytes: {}.", - stage_id, - i, - w.path(), - w.num_batches, - w.num_rows, - w.num_bytes - ); - } - None => {} - } - } - debug!( - "ShuffleWriterExec[stage={}] Finished processing stream with {rows} rows", - stage_id - ); - } - } - - // create a dummy batch to return - later this could be metadata about the - // shuffle partitions that were written out - let schema = Arc::new(Schema::new(vec![ - Field::new("shuffle_repart_time", DataType::Int32, true), - Field::new("shuffle_write_time", DataType::Int32, true), - ])); - let arr_repart_time = Int32Array::from(vec![repart_time.value() as i32]); - let arr_write_time = Int32Array::from(vec![write_time.value() as i32]); - let batch = RecordBatch::try_new( - schema.clone(), - vec![Arc::new(arr_repart_time), Arc::new(arr_write_time)], - )?; - - // return as a stream - MemoryStream::try_new(vec![batch], schema, None) - }; - let schema = self.schema(); - - Ok(Box::pin(RecordBatchStreamAdapter::new( - schema, - futures::stream::once(results).try_flatten(), - ))) - } - - fn statistics(&self) -> Result { - Ok(Statistics::new_unknown(&self.schema())) - } - - fn name(&self) -> &str { - "shuffle writer" - } - - fn properties(&self) -> &datafusion::physical_plan::PlanProperties { - &self.properties - } -} - -impl DisplayAs for ShuffleWriterExec { - fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { - write!( - f, - "ShuffleWriterExec(stage_id={}, output_partitioning={:?})", - self.stage_id, - self.properties().partitioning - ) - } -} - -/// Stream data to disk in Arrow IPC format -pub async fn write_stream_to_disk( - stream: &mut Pin>, - path: &str, - disk_write_metric: &metrics::Time, -) -> Result { - let file = File::create(path).unwrap(); - - /*.map_err(|e| { - error!("Failed to create partition file at {}: {:?}", path, e); - BallistaError::IoError(e) - })?;*/ - - let mut num_rows = 0; - let mut num_batches = 0; - let mut num_bytes = 0; - let mut writer = FileWriter::try_new(file, stream.schema().as_ref())?; - - while let Some(result) = stream.next().await { - let batch = result?; - - let batch_size_bytes: usize = batch.get_array_memory_size(); - num_batches += 1; - num_rows += batch.num_rows(); - num_bytes += batch_size_bytes; - - let timer = disk_write_metric.timer(); - writer.write(&batch)?; - timer.done(); - } - let timer = disk_write_metric.timer(); - writer.finish()?; - timer.done(); - Ok(PartitionStats { - num_rows: num_rows as i64, - num_batches: num_batches as i64, - num_bytes: num_bytes as i64, - column_stats: vec![], - }) -} diff --git a/testdata/expected-plans/q1.txt b/testdata/expected-plans/q1.txt index 3b1f94a..4dc185b 100644 --- a/testdata/expected-plans/q1.txt +++ b/testdata/expected-plans/q1.txt @@ -27,7 +27,7 @@ DataFusion Ray Distributed Plan =========== Query Stage #0 (2 -> 2): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 2)) +RayShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 2)) AggregateExec: mode=Partial, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(*)] ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_2, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] CoalesceBatchesExec: target_batch_size=8192 @@ -35,14 +35,14 @@ ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "l_return ParquetExec: file_groups={ ... }, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], predicate=l_shipdate@10 <= 1998-09-24, pruning_predicate=CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_min@0 <= 1998-09-24 END, required_guarantees=[] Query Stage #1 (2 -> 2): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 2)) +RayShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 2)) SortExec: expr=[l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[true] ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, sum(lineitem.l_quantity)@2 as sum_qty, sum(lineitem.l_extendedprice)@3 as sum_base_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, avg(lineitem.l_quantity)@6 as avg_qty, avg(lineitem.l_extendedprice)@7 as avg_price, avg(lineitem.l_discount)@8 as avg_disc, count(*)@9 as count_order] AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(*)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 2)) + RayShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 2)) Query Stage #2 (2 -> 1): SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST] - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 2)) + RayShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 2)) diff --git a/testdata/expected-plans/q10.txt b/testdata/expected-plans/q10.txt index 23f582b..b853fb6 100644 --- a/testdata/expected-plans/q10.txt +++ b/testdata/expected-plans/q10.txt @@ -65,66 +65,66 @@ DataFusion Ray Distributed Plan =========== Query Stage #0 (1 -> 2): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name] Query Stage #1 (2 -> 2): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment] Query Stage #2 (2 -> 2): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 2)) +RayShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 2)) ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: o_orderdate@2 >= 1993-07-01 AND o_orderdate@2 < 1993-10-01 ParquetExec: file_groups={ ... }, projection=[o_orderkey, o_custkey, o_orderdate], predicate=o_orderdate@4 >= 1993-07-01 AND o_orderdate@4 < 1993-10-01, pruning_predicate=CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_max@0 >= 1993-07-01 END AND CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_min@3 < 1993-10-01 END, required_guarantees=[] Query Stage #3 (2 -> 2): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "o_orderkey", index: 7 }], 2)) +RayShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "o_orderkey", index: 7 }], 2)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, o_orderkey@7] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 2)) + RayShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 2)) Query Stage #4 (2 -> 2): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] CoalesceBatchesExec: target_batch_size=8192 FilterExec: l_returnflag@3 = R ParquetExec: file_groups={ ... }, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], predicate=l_returnflag@8 = R, pruning_predicate=CASE WHEN l_returnflag_null_count@2 = l_returnflag_row_count@3 THEN false ELSE l_returnflag_min@0 <= R AND R <= l_returnflag_max@1 END, required_guarantees=[l_returnflag in (R)] Query Stage #5 (2 -> 2): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "c_nationkey", index: 3 }], 2)) +RayShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "c_nationkey", index: 3 }], 2)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@9, l_discount@10] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "o_orderkey", index: 7 }], 2)) + RayShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "o_orderkey", index: 7 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) Query Stage #6 (2 -> 2): -ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }, Column { name: "c_name", index: 1 }, Column { name: "c_acctbal", index: 2 }, Column { name: "c_phone", index: 3 }, Column { name: "n_name", index: 4 }, Column { name: "c_address", index: 5 }, Column { name: "c_comment", index: 6 }], 2)) +RayShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }, Column { name: "c_name", index: 1 }, Column { name: "c_acctbal", index: 2 }, Column { name: "c_phone", index: 3 }, Column { name: "n_name", index: 4 }, Column { name: "c_address", index: 5 }, Column { name: "c_comment", index: 6 }], 2)) AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ProjectionExec: expr=[c_custkey@1 as c_custkey, c_name@2 as c_name, c_address@3 as c_address, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@7 as l_extendedprice, l_discount@8 as l_discount, n_name@0 as n_name] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_nationkey@0, c_nationkey@3)], projection=[n_name@1, c_custkey@2, c_name@3, c_address@4, c_phone@6, c_acctbal@7, c_comment@8, l_extendedprice@9, l_discount@10] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "c_nationkey", index: 3 }], 2)) + RayShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "c_nationkey", index: 3 }], 2)) Query Stage #7 (2 -> 2): -ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }, Column { name: "c_name", index: 1 }, Column { name: "c_acctbal", index: 3 }, Column { name: "c_phone", index: 6 }, Column { name: "n_name", index: 4 }, Column { name: "c_address", index: 5 }, Column { name: "c_comment", index: 7 }], 2)) +RayShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }, Column { name: "c_name", index: 1 }, Column { name: "c_acctbal", index: 3 }, Column { name: "c_phone", index: 6 }, Column { name: "n_name", index: 4 }, Column { name: "c_address", index: 5 }, Column { name: "c_comment", index: 7 }], 2)) SortExec: TopK(fetch=20), expr=[revenue@2 DESC], preserve_partitioning=[true] ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@7 as revenue, c_acctbal@2 as c_acctbal, n_name@4 as n_name, c_address@5 as c_address, c_phone@3 as c_phone, c_comment@6 as c_comment] AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@2 as c_acctbal, c_phone@3 as c_phone, n_name@4 as n_name, c_address@5 as c_address, c_comment@6 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }, Column { name: "c_name", index: 1 }, Column { name: "c_acctbal", index: 2 }, Column { name: "c_phone", index: 3 }, Column { name: "n_name", index: 4 }, Column { name: "c_address", index: 5 }, Column { name: "c_comment", index: 6 }], 2)) + RayShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }, Column { name: "c_name", index: 1 }, Column { name: "c_acctbal", index: 2 }, Column { name: "c_phone", index: 3 }, Column { name: "n_name", index: 4 }, Column { name: "c_address", index: 5 }, Column { name: "c_comment", index: 6 }], 2)) Query Stage #8 (1 -> 1): GlobalLimitExec: skip=0, fetch=20 SortPreservingMergeExec: [revenue@2 DESC], fetch=20 - ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }, Column { name: "c_name", index: 1 }, Column { name: "c_acctbal", index: 3 }, Column { name: "c_phone", index: 6 }, Column { name: "n_name", index: 4 }, Column { name: "c_address", index: 5 }, Column { name: "c_comment", index: 7 }], 2)) + RayShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }, Column { name: "c_name", index: 1 }, Column { name: "c_acctbal", index: 3 }, Column { name: "c_phone", index: 6 }, Column { name: "n_name", index: 4 }, Column { name: "c_address", index: 5 }, Column { name: "c_comment", index: 7 }], 2)) diff --git a/testdata/expected-plans/q11.txt b/testdata/expected-plans/q11.txt index 58319a0..a335871 100644 --- a/testdata/expected-plans/q11.txt +++ b/testdata/expected-plans/q11.txt @@ -89,89 +89,89 @@ DataFusion Ray Distributed Plan =========== Query Stage #0 (1 -> 2): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) ProjectionExec: expr=[n_nationkey@0 as n_nationkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: n_name@1 = ALGERIA ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name], predicate=n_name@1 = ALGERIA, pruning_predicate=CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= ALGERIA AND ALGERIA <= n_name_max@1 END, required_guarantees=[n_name in (ALGERIA)] Query Stage #1 (1 -> 2): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_nationkey] Query Stage #2 (2 -> 2): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[ps_suppkey, ps_availqty, ps_supplycost] Query Stage #3 (2 -> 2): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 2)) +RayShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 2)) ProjectionExec: expr=[ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost, s_nationkey@0 as s_nationkey] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_nationkey@1, ps_availqty@3, ps_supplycost@4] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 2)) Query Stage #4 (2 -> 1): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([], 2)) +RayShuffleWriterExec(stage_id=4, output_partitioning=Hash([], 2)) AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[ps_availqty@1, ps_supplycost@2] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 2)) + RayShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 2)) Query Stage #5 (1 -> 2): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) ProjectionExec: expr=[n_nationkey@0 as n_nationkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: n_name@1 = ALGERIA ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name], predicate=n_name@1 = ALGERIA, pruning_predicate=CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= ALGERIA AND ALGERIA <= n_name_max@1 END, required_guarantees=[n_name in (ALGERIA)] Query Stage #6 (1 -> 2): -ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_nationkey] Query Stage #7 (2 -> 2): -ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }], 2)) +RayShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }], 2)) ParquetExec: file_groups={ ... }, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost] Query Stage #8 (2 -> 2): -ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 2)) +RayShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 2)) ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_availqty@2 as ps_availqty, ps_supplycost@3 as ps_supplycost, s_nationkey@0 as s_nationkey] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@1)], projection=[s_nationkey@1, ps_partkey@2, ps_availqty@4, ps_supplycost@5] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }], 2)) + RayShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }], 2)) Query Stage #9 (2 -> 2): -ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[ps_partkey@1, ps_availqty@2, ps_supplycost@3] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 2)) + RayShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 2)) Query Stage #10 (2 -> 2): -ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) SortExec: expr=[value@1 DESC], preserve_partitioning=[true] ProjectionExec: expr=[ps_partkey@1 as ps_partkey, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@2 as value] NestedLoopJoinExec: join_type=Inner, filter=CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Decimal128(38, 15)) > sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)@1 ProjectionExec: expr=[CAST(CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] CoalescePartitionsExec - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([], 2)) + RayShuffleReaderExec(stage_id=4, input_partitioning=Hash([], 2)) AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) Query Stage #11 (2 -> 1): SortPreservingMergeExec: [value@1 DESC] - ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) diff --git a/testdata/expected-plans/q13.txt b/testdata/expected-plans/q13.txt index 428dd9f..ddb7050 100644 --- a/testdata/expected-plans/q13.txt +++ b/testdata/expected-plans/q13.txt @@ -42,37 +42,37 @@ DataFusion Ray Distributed Plan =========== Query Stage #0 (2 -> 2): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[c_custkey] Query Stage #1 (2 -> 2): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 2)) +RayShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 2)) ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: o_comment@2 NOT LIKE %express%requests% ParquetExec: file_groups={ ... }, projection=[o_orderkey, o_custkey, o_comment], predicate=o_comment@8 NOT LIKE %express%requests% Query Stage #2 (2 -> 2): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "c_count", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "c_count", index: 0 }], 2)) AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(*)] ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 2)) + RayShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 2)) Query Stage #3 (2 -> 2): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "c_count", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "c_count", index: 0 }], 2)) SortExec: expr=[custdist@1 DESC,c_count@0 DESC], preserve_partitioning=[true] ProjectionExec: expr=[c_count@0 as c_count, count(*)@1 as custdist] AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(*)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "c_count", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "c_count", index: 0 }], 2)) Query Stage #4 (2 -> 1): SortPreservingMergeExec: [custdist@1 DESC,c_count@0 DESC] - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "c_count", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "c_count", index: 0 }], 2)) diff --git a/testdata/expected-plans/q14.txt b/testdata/expected-plans/q14.txt index 383958c..f022fdc 100644 --- a/testdata/expected-plans/q14.txt +++ b/testdata/expected-plans/q14.txt @@ -35,30 +35,30 @@ DataFusion Ray Distributed Plan =========== Query Stage #0 (1 -> 2): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[p_partkey, p_type] Query Stage #1 (2 -> 2): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 2)) ProjectionExec: expr=[l_partkey@0 as l_partkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] CoalesceBatchesExec: target_batch_size=8192 FilterExec: l_shipdate@3 >= 1995-02-01 AND l_shipdate@3 < 1995-03-01 ParquetExec: file_groups={ ... }, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], predicate=l_shipdate@10 >= 1995-02-01 AND l_shipdate@10 < 1995-03-01, pruning_predicate=CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_max@0 >= 1995-02-01 END AND CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_min@3 < 1995-03-01 END, required_guarantees=[] Query Stage #2 (2 -> 1): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([], 2)) +RayShuffleWriterExec(stage_id=2, output_partitioning=Hash([], 2)) AggregateExec: mode=Partial, gby=[], aggr=[sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_2, p_type@0 as p_type] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_type@1, l_extendedprice@3, l_discount@4] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 2)) Query Stage #3 (1 -> 1): ProjectionExec: expr=[100 * CAST(sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END)@0 AS Float64) / CAST(sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 AS Float64) as promo_revenue] AggregateExec: mode=Final, gby=[], aggr=[sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] CoalescePartitionsExec - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([], 2)) + RayShuffleReaderExec(stage_id=2, input_partitioning=Hash([], 2)) diff --git a/testdata/expected-plans/q17.txt b/testdata/expected-plans/q17.txt index 89f1ec7..7b5aac9 100644 --- a/testdata/expected-plans/q17.txt +++ b/testdata/expected-plans/q17.txt @@ -49,23 +49,23 @@ DataFusion Ray Distributed Plan =========== Query Stage #0 (1 -> 2): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) ProjectionExec: expr=[p_partkey@0 as p_partkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: p_brand@1 = Brand#42 AND p_container@2 = LG BAG ParquetExec: file_groups={ ... }, projection=[p_partkey, p_brand, p_container], predicate=p_brand@3 = Brand#42 AND p_container@6 = LG BAG, pruning_predicate=CASE WHEN p_brand_null_count@2 = p_brand_row_count@3 THEN false ELSE p_brand_min@0 <= Brand#42 AND Brand#42 <= p_brand_max@1 END AND CASE WHEN p_container_null_count@6 = p_container_row_count@7 THEN false ELSE p_container_min@4 <= LG BAG AND LG BAG <= p_container_max@5 END, required_guarantees=[p_brand in (Brand#42), p_container in (LG BAG)] Query Stage #1 (2 -> 2): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[l_partkey, l_quantity, l_extendedprice] Query Stage #2 (2 -> 2): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 2)) AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] ParquetExec: file_groups={ ... }, projection=[l_partkey, l_quantity] Query Stage #3 (2 -> 1): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([], 2)) +RayShuffleWriterExec(stage_id=3, output_partitioning=Hash([], 2)) AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice)] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * avg(lineitem.l_quantity)@1, projection=[l_extendedprice@1] @@ -73,17 +73,17 @@ ShuffleWriterExec(stage_id=3, output_partitioning=Hash([], 2)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_partkey@0, l_quantity@2, l_extendedprice@3] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 2)) ProjectionExec: expr=[CAST(0.2 * CAST(avg(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * avg(lineitem.l_quantity), l_partkey@0 as l_partkey] AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 2)) Query Stage #4 (1 -> 1): ProjectionExec: expr=[CAST(sum(lineitem.l_extendedprice)@0 AS Float64) / 7 as avg_yearly] AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice)] CoalescePartitionsExec - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([], 2)) + RayShuffleReaderExec(stage_id=3, input_partitioning=Hash([], 2)) diff --git a/testdata/expected-plans/q18.txt b/testdata/expected-plans/q18.txt index 8547084..a9c18bf 100644 --- a/testdata/expected-plans/q18.txt +++ b/testdata/expected-plans/q18.txt @@ -58,33 +58,33 @@ DataFusion Ray Distributed Plan =========== Query Stage #0 (2 -> 2): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] ParquetExec: file_groups={ ... }, projection=[l_orderkey, l_quantity] Query Stage #1 (2 -> 2): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[c_custkey, c_name] Query Stage #2 (2 -> 2): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 2)) +RayShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 2)) ParquetExec: file_groups={ ... }, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate] Query Stage #3 (2 -> 2): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "o_orderkey", index: 2 }], 2)) +RayShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "o_orderkey", index: 2 }], 2)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 2)) + RayShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 2)) Query Stage #4 (2 -> 2): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[l_orderkey, l_quantity] Query Stage #5 (2 -> 2): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "c_name", index: 0 }, Column { name: "c_custkey", index: 1 }, Column { name: "o_orderkey", index: 2 }, Column { name: "o_orderdate", index: 3 }, Column { name: "o_totalprice", index: 4 }], 2)) +RayShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "c_name", index: 0 }, Column { name: "c_custkey", index: 1 }, Column { name: "o_orderkey", index: 2 }, Column { name: "o_orderdate", index: 3 }, Column { name: "o_totalprice", index: 4 }], 2)) AggregateExec: mode=Partial, gby=[c_name@1 as c_name, c_custkey@0 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@4 as o_orderdate, o_totalprice@3 as o_totalprice], aggr=[sum(lineitem.l_quantity)] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(l_orderkey@0, o_orderkey@2)] @@ -93,23 +93,23 @@ ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "c_name", FilterExec: sum(lineitem.l_quantity)@1 > Some(31300),21,2 AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@3, o_orderdate@4, l_quantity@6] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "o_orderkey", index: 2 }], 2)) + RayShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "o_orderkey", index: 2 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) Query Stage #6 (2 -> 2): -ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "c_name", index: 0 }, Column { name: "c_custkey", index: 1 }, Column { name: "o_orderkey", index: 2 }, Column { name: "o_orderdate", index: 3 }, Column { name: "o_totalprice", index: 4 }], 2)) +RayShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "c_name", index: 0 }, Column { name: "c_custkey", index: 1 }, Column { name: "o_orderkey", index: 2 }, Column { name: "o_orderdate", index: 3 }, Column { name: "o_totalprice", index: 4 }], 2)) SortExec: TopK(fetch=100), expr=[o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[true] AggregateExec: mode=FinalPartitioned, gby=[c_name@0 as c_name, c_custkey@1 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@3 as o_orderdate, o_totalprice@4 as o_totalprice], aggr=[sum(lineitem.l_quantity)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "c_name", index: 0 }, Column { name: "c_custkey", index: 1 }, Column { name: "o_orderkey", index: 2 }, Column { name: "o_orderdate", index: 3 }, Column { name: "o_totalprice", index: 4 }], 2)) + RayShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "c_name", index: 0 }, Column { name: "c_custkey", index: 1 }, Column { name: "o_orderkey", index: 2 }, Column { name: "o_orderdate", index: 3 }, Column { name: "o_totalprice", index: 4 }], 2)) Query Stage #7 (1 -> 1): GlobalLimitExec: skip=0, fetch=100 SortPreservingMergeExec: [o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST], fetch=100 - ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "c_name", index: 0 }, Column { name: "c_custkey", index: 1 }, Column { name: "o_orderkey", index: 2 }, Column { name: "o_orderdate", index: 3 }, Column { name: "o_totalprice", index: 4 }], 2)) + RayShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "c_name", index: 0 }, Column { name: "c_custkey", index: 1 }, Column { name: "o_orderkey", index: 2 }, Column { name: "o_orderdate", index: 3 }, Column { name: "o_totalprice", index: 4 }], 2)) diff --git a/testdata/expected-plans/q2.txt b/testdata/expected-plans/q2.txt index 7e8f91b..456c8da 100644 --- a/testdata/expected-plans/q2.txt +++ b/testdata/expected-plans/q2.txt @@ -130,138 +130,138 @@ DataFusion Ray Distributed Plan =========== Query Stage #0 (1 -> 2): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 2)) ProjectionExec: expr=[r_regionkey@0 as r_regionkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: r_name@1 = ASIA ParquetExec: file_groups={ ... }, projection=[r_regionkey, r_name], predicate=r_name@1 = ASIA, pruning_predicate=CASE WHEN r_name_null_count@2 = r_name_row_count@3 THEN false ELSE r_name_min@0 <= ASIA AND ASIA <= r_name_max@1 END, required_guarantees=[r_name in (ASIA)] Query Stage #1 (1 -> 2): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name, n_regionkey] Query Stage #2 (1 -> 2): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] Query Stage #3 (1 -> 2): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr] CoalesceBatchesExec: target_batch_size=8192 FilterExec: p_size@3 = 48 AND p_type@2 LIKE %TIN ParquetExec: file_groups={ ... }, projection=[p_partkey, p_mfgr, p_type, p_size], predicate=p_size@5 = 48 AND p_type@4 LIKE %TIN, pruning_predicate=CASE WHEN p_size_null_count@2 = p_size_row_count@3 THEN false ELSE p_size_min@0 <= 48 AND 48 <= p_size_max@1 END, required_guarantees=[p_size in (48)] Query Stage #4 (2 -> 2): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[ps_partkey, ps_suppkey, ps_supplycost] Query Stage #5 (2 -> 2): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "ps_suppkey", index: 2 }], 2)) +RayShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "ps_suppkey", index: 2 }], 2)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_suppkey@3, ps_supplycost@4] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) Query Stage #6 (2 -> 2): -ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "s_nationkey", index: 4 }], 2)) +RayShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "s_nationkey", index: 4 }], 2)) ProjectionExec: expr=[p_partkey@6 as p_partkey, p_mfgr@7 as p_mfgr, s_name@0 as s_name, s_address@1 as s_address, s_nationkey@2 as s_nationkey, s_phone@3 as s_phone, s_acctbal@4 as s_acctbal, s_comment@5 as s_comment, ps_supplycost@8 as ps_supplycost] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@2)], projection=[s_name@1, s_address@2, s_nationkey@3, s_phone@4, s_acctbal@5, s_comment@6, p_partkey@7, p_mfgr@8, ps_supplycost@10] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "ps_suppkey", index: 2 }], 2)) + RayShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "ps_suppkey", index: 2 }], 2)) Query Stage #7 (2 -> 2): -ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "n_regionkey", index: 9 }], 2)) +RayShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "n_regionkey", index: 9 }], 2)) ProjectionExec: expr=[p_partkey@2 as p_partkey, p_mfgr@3 as p_mfgr, s_name@4 as s_name, s_address@5 as s_address, s_phone@6 as s_phone, s_acctbal@7 as s_acctbal, s_comment@8 as s_comment, ps_supplycost@9 as ps_supplycost, n_name@0 as n_name, n_regionkey@1 as n_regionkey] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_nationkey@0, s_nationkey@4)], projection=[n_name@1, n_regionkey@2, p_partkey@3, p_mfgr@4, s_name@5, s_address@6, s_phone@8, s_acctbal@9, s_comment@10, ps_supplycost@11] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "s_nationkey", index: 4 }], 2)) + RayShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "s_nationkey", index: 4 }], 2)) Query Stage #8 (2 -> 2): -ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }, Column { name: "ps_supplycost", index: 7 }], 2)) +RayShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }, Column { name: "ps_supplycost", index: 7 }], 2)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(r_regionkey@0, n_regionkey@9)], projection=[p_partkey@1, p_mfgr@2, s_name@3, s_address@4, s_phone@5, s_acctbal@6, s_comment@7, ps_supplycost@8, n_name@9] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "n_regionkey", index: 9 }], 2)) + RayShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "n_regionkey", index: 9 }], 2)) Query Stage #9 (1 -> 2): -ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 2)) ProjectionExec: expr=[r_regionkey@0 as r_regionkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: r_name@1 = ASIA ParquetExec: file_groups={ ... }, projection=[r_regionkey, r_name], predicate=r_name@1 = ASIA, pruning_predicate=CASE WHEN r_name_null_count@2 = r_name_row_count@3 THEN false ELSE r_name_min@0 <= ASIA AND ASIA <= r_name_max@1 END, required_guarantees=[r_name in (ASIA)] Query Stage #10 (1 -> 2): -ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_regionkey] Query Stage #11 (1 -> 2): -ShuffleWriterExec(stage_id=11, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=11, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_nationkey] Query Stage #12 (2 -> 2): -ShuffleWriterExec(stage_id=12, output_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }], 2)) +RayShuffleWriterExec(stage_id=12, output_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }], 2)) ParquetExec: file_groups={ ... }, projection=[ps_partkey, ps_suppkey, ps_supplycost] Query Stage #13 (2 -> 2): -ShuffleWriterExec(stage_id=13, output_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 2)) +RayShuffleWriterExec(stage_id=13, output_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 2)) ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_supplycost@2 as ps_supplycost, s_nationkey@0 as s_nationkey] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@1)], projection=[s_nationkey@1, ps_partkey@2, ps_supplycost@4] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=11, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=11, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=12, input_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }], 2)) + RayShuffleReaderExec(stage_id=12, input_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }], 2)) Query Stage #14 (2 -> 2): -ShuffleWriterExec(stage_id=14, output_partitioning=Hash([Column { name: "n_regionkey", index: 2 }], 2)) +RayShuffleWriterExec(stage_id=14, output_partitioning=Hash([Column { name: "n_regionkey", index: 2 }], 2)) ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_supplycost@2 as ps_supplycost, n_regionkey@0 as n_regionkey] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_regionkey@1, ps_partkey@2, ps_supplycost@3] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=13, input_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 2)) + RayShuffleReaderExec(stage_id=13, input_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 2)) Query Stage #15 (2 -> 2): -ShuffleWriterExec(stage_id=15, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=15, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(r_regionkey@0, n_regionkey@2)], projection=[ps_partkey@1, ps_supplycost@2] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=14, input_partitioning=Hash([Column { name: "n_regionkey", index: 2 }], 2)) + RayShuffleReaderExec(stage_id=14, input_partitioning=Hash([Column { name: "n_regionkey", index: 2 }], 2)) Query Stage #16 (2 -> 2): -ShuffleWriterExec(stage_id=16, output_partitioning=Hash([Column { name: "ps_partkey", index: 1 }, Column { name: "min(partsupp.ps_supplycost)", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=16, output_partitioning=Hash([Column { name: "ps_partkey", index: 1 }, Column { name: "min(partsupp.ps_supplycost)", index: 0 }], 2)) ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=15, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=15, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) Query Stage #17 (2 -> 2): -ShuffleWriterExec(stage_id=17, output_partitioning=Hash([Column { name: "p_partkey", index: 3 }], 2)) +RayShuffleWriterExec(stage_id=17, output_partitioning=Hash([Column { name: "p_partkey", index: 3 }], 2)) SortExec: TopK(fetch=100), expr=[s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST], preserve_partitioning=[true] ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@7 as n_name, p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_address@3 as s_address, s_phone@4 as s_phone, s_comment@6 as s_comment] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, min(partsupp.ps_supplycost)@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, n_name@8] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }, Column { name: "ps_supplycost", index: 7 }], 2)) + RayShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }, Column { name: "ps_supplycost", index: 7 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=16, input_partitioning=Hash([Column { name: "ps_partkey", index: 1 }, Column { name: "min(partsupp.ps_supplycost)", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=16, input_partitioning=Hash([Column { name: "ps_partkey", index: 1 }, Column { name: "min(partsupp.ps_supplycost)", index: 0 }], 2)) Query Stage #18 (1 -> 1): GlobalLimitExec: skip=0, fetch=100 SortPreservingMergeExec: [s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST], fetch=100 - ShuffleReaderExec(stage_id=17, input_partitioning=Hash([Column { name: "p_partkey", index: 3 }], 2)) + RayShuffleReaderExec(stage_id=17, input_partitioning=Hash([Column { name: "p_partkey", index: 3 }], 2)) diff --git a/testdata/expected-plans/q20.txt b/testdata/expected-plans/q20.txt index 2d5f70f..946f7f3 100644 --- a/testdata/expected-plans/q20.txt +++ b/testdata/expected-plans/q20.txt @@ -80,47 +80,47 @@ DataFusion Ray Distributed Plan =========== Query Stage #0 (1 -> 2): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) ProjectionExec: expr=[n_nationkey@0 as n_nationkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: n_name@1 = KENYA ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name], predicate=n_name@1 = KENYA, pruning_predicate=CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= KENYA AND KENYA <= n_name_max@1 END, required_guarantees=[n_name in (KENYA)] Query Stage #1 (1 -> 2): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 2)) +RayShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 2)) ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_name, s_address, s_nationkey] Query Stage #2 (2 -> 2): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[s_suppkey@1, s_name@2, s_address@3] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 2)) + RayShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 2)) Query Stage #3 (1 -> 2): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) ProjectionExec: expr=[p_partkey@0 as p_partkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: p_name@1 LIKE blanched% ParquetExec: file_groups={ ... }, projection=[p_partkey, p_name], predicate=p_name@1 LIKE blanched% Query Stage #4 (2 -> 2): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[ps_partkey, ps_suppkey, ps_availqty] Query Stage #5 (2 -> 2): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }, Column { name: "ps_suppkey", index: 1 }], 2)) +RayShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }, Column { name: "ps_suppkey", index: 1 }], 2)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(p_partkey@0, ps_partkey@0)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) Query Stage #6 (2 -> 2): -ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "l_partkey", index: 0 }, Column { name: "l_suppkey", index: 1 }], 2)) +RayShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "l_partkey", index: 0 }, Column { name: "l_suppkey", index: 1 }], 2)) AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, l_quantity@2 as l_quantity] CoalesceBatchesExec: target_batch_size=8192 @@ -128,27 +128,27 @@ ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "l_partke ParquetExec: file_groups={ ... }, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], predicate=l_shipdate@10 >= 1993-01-01 AND l_shipdate@10 < 1994-01-01, pruning_predicate=CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_max@0 >= 1993-01-01 END AND CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_min@3 < 1994-01-01 END, required_guarantees=[] Query Stage #7 (2 -> 2): -ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * sum(lineitem.l_quantity)@1, projection=[ps_suppkey@1] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }, Column { name: "ps_suppkey", index: 1 }], 2)) + RayShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }, Column { name: "ps_suppkey", index: 1 }], 2)) ProjectionExec: expr=[0.5 * CAST(sum(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * sum(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "l_partkey", index: 0 }, Column { name: "l_suppkey", index: 1 }], 2)) + RayShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "l_partkey", index: 0 }, Column { name: "l_suppkey", index: 1 }], 2)) Query Stage #8 (2 -> 1): -ShuffleWriterExec(stage_id=8, output_partitioning=Hash([], 2)) +RayShuffleWriterExec(stage_id=8, output_partitioning=Hash([], 2)) SortExec: expr=[s_name@0 ASC NULLS LAST], preserve_partitioning=[true] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_name@1, s_address@2] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 2)) Query Stage #9 (2 -> 1): SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] - ShuffleReaderExec(stage_id=8, input_partitioning=Hash([], 2)) + RayShuffleReaderExec(stage_id=8, input_partitioning=Hash([], 2)) diff --git a/testdata/expected-plans/q21.txt b/testdata/expected-plans/q21.txt index de59694..6121007 100644 --- a/testdata/expected-plans/q21.txt +++ b/testdata/expected-plans/q21.txt @@ -98,92 +98,92 @@ DataFusion Ray Distributed Plan =========== Query Stage #0 (1 -> 2): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) ProjectionExec: expr=[n_nationkey@0 as n_nationkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: n_name@1 = ARGENTINA ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name], predicate=n_name@1 = ARGENTINA, pruning_predicate=CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= ARGENTINA AND ARGENTINA <= n_name_max@1 END, required_guarantees=[n_name in (ARGENTINA)] Query Stage #1 (2 -> 2): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) ProjectionExec: expr=[o_orderkey@0 as o_orderkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: o_orderstatus@1 = F ParquetExec: file_groups={ ... }, projection=[o_orderkey, o_orderstatus], predicate=o_orderstatus@2 = F, pruning_predicate=CASE WHEN o_orderstatus_null_count@2 = o_orderstatus_row_count@3 THEN false ELSE o_orderstatus_min@0 <= F AND F <= o_orderstatus_max@1 END, required_guarantees=[o_orderstatus in (F)] Query Stage #2 (1 -> 2): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_name, s_nationkey] Query Stage #3 (2 -> 2): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 2)) +RayShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 2)) ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: l_receiptdate@3 > l_commitdate@2 ParquetExec: file_groups={ ... }, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], predicate=l_receiptdate@12 > l_commitdate@11 Query Stage #4 (2 -> 2): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "l_orderkey", index: 2 }], 2)) +RayShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "l_orderkey", index: 2 }], 2)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 2)) + RayShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 2)) Query Stage #5 (2 -> 2): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "s_nationkey", index: 1 }], 2)) +RayShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "s_nationkey", index: 1 }], 2)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@2)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "l_orderkey", index: 2 }], 2)) + RayShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "l_orderkey", index: 2 }], 2)) Query Stage #6 (2 -> 2): -ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "l_orderkey", index: 1 }], 2)) +RayShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "l_orderkey", index: 1 }], 2)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_nationkey@0, s_nationkey@1)], projection=[s_name@1, l_orderkey@3, l_suppkey@4] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "s_nationkey", index: 1 }], 2)) + RayShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "s_nationkey", index: 1 }], 2)) Query Stage #7 (2 -> 2): -ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[l_orderkey, l_suppkey] Query Stage #8 (2 -> 2): -ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: l_receiptdate@3 > l_commitdate@2 ParquetExec: file_groups={ ... }, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], predicate=l_receiptdate@12 > l_commitdate@11 Query Stage #9 (2 -> 2): -ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "s_name", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "s_name", index: 0 }], 2)) AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[count(*)] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "l_orderkey", index: 1 }], 2)) + RayShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "l_orderkey", index: 1 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) Query Stage #10 (2 -> 2): -ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "s_name", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "s_name", index: 0 }], 2)) SortExec: TopK(fetch=100), expr=[numwait@1 DESC,s_name@0 ASC NULLS LAST], preserve_partitioning=[true] ProjectionExec: expr=[s_name@0 as s_name, count(*)@1 as numwait] AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[count(*)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "s_name", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "s_name", index: 0 }], 2)) Query Stage #11 (1 -> 1): GlobalLimitExec: skip=0, fetch=100 SortPreservingMergeExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST], fetch=100 - ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "s_name", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "s_name", index: 0 }], 2)) diff --git a/testdata/expected-plans/q22.txt b/testdata/expected-plans/q22.txt index 4b451e9..afebeaf 100644 --- a/testdata/expected-plans/q22.txt +++ b/testdata/expected-plans/q22.txt @@ -53,7 +53,7 @@ DataFusion Ray Distributed Plan =========== Query Stage #0 (2 -> 1): -ShuffleWriterExec(stage_id=0, output_partitioning=UnknownPartitioning(2)) +RayShuffleWriterExec(stage_id=0, output_partitioning=UnknownPartitioning(2)) AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] ProjectionExec: expr=[c_acctbal@1 as c_acctbal] CoalesceBatchesExec: target_batch_size=8192 @@ -61,39 +61,39 @@ ShuffleWriterExec(stage_id=0, output_partitioning=UnknownPartitioning(2)) ParquetExec: file_groups={ ... }]) AND c_acctbal@5 > Some(0),11,2, pruning_predicate=CASE WHEN c_acctbal_null_count@1 = c_acctbal_row_count@2 THEN false ELSE c_acctbal_max@0 > Some(0),11,2 END AND CASE WHEN c_acctbal_null_count@1 = c_acctbal_row_count@2 THEN false ELSE c_acctbal_max@0 > Some(0),11,2 END, required_guarantees=[] Query Stage #1 (2 -> 2): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: Use substr(c_phone@1, 1, 2) IN (SET) ([Literal { value: Utf8("24") }, Literal { value: Utf8("34") }, Literal { value: Utf8("16") }, Literal { value: Utf8("30") }, Literal { value: Utf8("33") }, Literal { value: Utf8("14") }, Literal { value: Utf8("13") }]) ParquetExec: file_groups={ ... }]) Query Stage #2 (2 -> 2): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_custkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_custkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[o_custkey] Query Stage #3 (2 -> 2): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "cntrycode", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "cntrycode", index: 0 }], 2)) AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[count(*), sum(custsale.c_acctbal)] ProjectionExec: expr=[substr(c_phone@1, 1, 2) as cntrycode, c_acctbal@2 as c_acctbal] NestedLoopJoinExec: join_type=Inner, filter=CAST(c_acctbal@0 AS Decimal128(15, 6)) > avg(customer.c_acctbal)@1 AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] CoalescePartitionsExec - ShuffleReaderExec(stage_id=0, input_partitioning=UnknownPartitioning(2)) + RayShuffleReaderExec(stage_id=0, input_partitioning=UnknownPartitioning(2)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "o_custkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "o_custkey", index: 0 }], 2)) Query Stage #4 (2 -> 2): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "cntrycode", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "cntrycode", index: 0 }], 2)) SortExec: expr=[cntrycode@0 ASC NULLS LAST], preserve_partitioning=[true] ProjectionExec: expr=[cntrycode@0 as cntrycode, count(*)@1 as numcust, sum(custsale.c_acctbal)@2 as totacctbal] AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[count(*), sum(custsale.c_acctbal)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "cntrycode", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "cntrycode", index: 0 }], 2)) Query Stage #5 (2 -> 1): SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "cntrycode", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "cntrycode", index: 0 }], 2)) diff --git a/testdata/expected-plans/q3.txt b/testdata/expected-plans/q3.txt index 7bb5c68..2a1026c 100644 --- a/testdata/expected-plans/q3.txt +++ b/testdata/expected-plans/q3.txt @@ -57,54 +57,54 @@ DataFusion Ray Distributed Plan =========== Query Stage #0 (2 -> 2): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) ProjectionExec: expr=[c_custkey@0 as c_custkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: c_mktsegment@1 = BUILDING ParquetExec: file_groups={ ... }, projection=[c_custkey, c_mktsegment], predicate=c_mktsegment@6 = BUILDING, pruning_predicate=CASE WHEN c_mktsegment_null_count@2 = c_mktsegment_row_count@3 THEN false ELSE c_mktsegment_min@0 <= BUILDING AND BUILDING <= c_mktsegment_max@1 END, required_guarantees=[c_mktsegment in (BUILDING)] Query Stage #1 (2 -> 2): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 2)) +RayShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 2)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: o_orderdate@2 < 1995-03-15 ParquetExec: file_groups={ ... }, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], predicate=o_orderdate@4 < 1995-03-15, pruning_predicate=CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_min@0 < 1995-03-15 END, required_guarantees=[] Query Stage #2 (2 -> 2): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 2)) + RayShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 2)) Query Stage #3 (2 -> 2): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] CoalesceBatchesExec: target_batch_size=8192 FilterExec: l_shipdate@3 > 1995-03-15 ParquetExec: file_groups={ ... }, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], predicate=l_shipdate@10 > 1995-03-15, pruning_predicate=CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_max@0 > 1995-03-15 END, required_guarantees=[] Query Stage #4 (2 -> 2): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderdate", index: 1 }, Column { name: "o_shippriority", index: 2 }], 2)) +RayShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderdate", index: 1 }, Column { name: "o_shippriority", index: 2 }], 2)) AggregateExec: mode=Partial, gby=[l_orderkey@2 as l_orderkey, o_orderdate@0 as o_orderdate, o_shippriority@1 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, o_shippriority@2, l_orderkey@3, l_extendedprice@4, l_discount@5] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) Query Stage #5 (2 -> 2): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderdate", index: 2 }, Column { name: "o_shippriority", index: 3 }], 2)) +RayShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderdate", index: 2 }, Column { name: "o_shippriority", index: 3 }], 2)) SortExec: TopK(fetch=10), expr=[revenue@1 DESC,o_orderdate@2 ASC NULLS LAST], preserve_partitioning=[true] ProjectionExec: expr=[l_orderkey@0 as l_orderkey, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority] AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderdate", index: 1 }, Column { name: "o_shippriority", index: 2 }], 2)) + RayShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderdate", index: 1 }, Column { name: "o_shippriority", index: 2 }], 2)) Query Stage #6 (1 -> 1): GlobalLimitExec: skip=0, fetch=10 SortPreservingMergeExec: [revenue@1 DESC,o_orderdate@2 ASC NULLS LAST], fetch=10 - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderdate", index: 2 }, Column { name: "o_shippriority", index: 3 }], 2)) + RayShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderdate", index: 2 }, Column { name: "o_shippriority", index: 3 }], 2)) diff --git a/testdata/expected-plans/q4.txt b/testdata/expected-plans/q4.txt index 3ba554e..cb2cc48 100644 --- a/testdata/expected-plans/q4.txt +++ b/testdata/expected-plans/q4.txt @@ -43,38 +43,38 @@ DataFusion Ray Distributed Plan =========== Query Stage #0 (2 -> 2): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_orderpriority@2 as o_orderpriority] CoalesceBatchesExec: target_batch_size=8192 FilterExec: o_orderdate@1 >= 1995-04-01 AND o_orderdate@1 < 1995-07-01 ParquetExec: file_groups={ ... }, projection=[o_orderkey, o_orderdate, o_orderpriority], predicate=o_orderdate@4 >= 1995-04-01 AND o_orderdate@4 < 1995-07-01, pruning_predicate=CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_max@0 >= 1995-04-01 END AND CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_min@3 < 1995-07-01 END, required_guarantees=[] Query Stage #1 (2 -> 2): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) ProjectionExec: expr=[l_orderkey@0 as l_orderkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: l_receiptdate@2 > l_commitdate@1 ParquetExec: file_groups={ ... }, projection=[l_orderkey, l_commitdate, l_receiptdate], predicate=l_receiptdate@12 > l_commitdate@11 Query Stage #2 (2 -> 2): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 2)) AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(*)] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) Query Stage #3 (2 -> 2): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 2)) SortExec: expr=[o_orderpriority@0 ASC NULLS LAST], preserve_partitioning=[true] ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, count(*)@1 as order_count] AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(*)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 2)) Query Stage #4 (2 -> 1): SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 2)) diff --git a/testdata/expected-plans/q5.txt b/testdata/expected-plans/q5.txt index 4306de6..f5ef8e3 100644 --- a/testdata/expected-plans/q5.txt +++ b/testdata/expected-plans/q5.txt @@ -86,92 +86,92 @@ DataFusion Ray Distributed Plan =========== Query Stage #0 (1 -> 2): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 2)) ProjectionExec: expr=[r_regionkey@0 as r_regionkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: r_name@1 = AFRICA ParquetExec: file_groups={ ... }, projection=[r_regionkey, r_name], predicate=r_name@1 = AFRICA, pruning_predicate=CASE WHEN r_name_null_count@2 = r_name_row_count@3 THEN false ELSE r_name_min@0 <= AFRICA AND AFRICA <= r_name_max@1 END, required_guarantees=[r_name in (AFRICA)] Query Stage #1 (1 -> 2): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name, n_regionkey] Query Stage #2 (1 -> 2): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }, Column { name: "s_nationkey", index: 1 }], 2)) +RayShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }, Column { name: "s_nationkey", index: 1 }], 2)) ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_nationkey] Query Stage #3 (2 -> 2): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[c_custkey, c_nationkey] Query Stage #4 (2 -> 2): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 2)) +RayShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 2)) ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01 ParquetExec: file_groups={ ... }, projection=[o_orderkey, o_custkey, o_orderdate], predicate=o_orderdate@4 >= 1994-01-01 AND o_orderdate@4 < 1995-01-01, pruning_predicate=CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_max@0 >= 1994-01-01 END AND CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_min@3 < 1995-01-01 END, required_guarantees=[] Query Stage #5 (2 -> 2): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "o_orderkey", index: 1 }], 2)) +RayShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "o_orderkey", index: 1 }], 2)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 2)) + RayShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 2)) Query Stage #6 (2 -> 2): -ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount] Query Stage #7 (2 -> 2): -ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "l_suppkey", index: 1 }, Column { name: "c_nationkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "l_suppkey", index: 1 }, Column { name: "c_nationkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)], projection=[c_nationkey@0, l_suppkey@3, l_extendedprice@4, l_discount@5] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "o_orderkey", index: 1 }], 2)) + RayShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "o_orderkey", index: 1 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) Query Stage #8 (2 -> 2): -ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 2)) +RayShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 2)) ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@0 as s_nationkey] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1), (s_nationkey@1, c_nationkey@0)], projection=[s_nationkey@1, l_extendedprice@4, l_discount@5] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }, Column { name: "s_nationkey", index: 1 }], 2)) + RayShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }, Column { name: "s_nationkey", index: 1 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "l_suppkey", index: 1 }, Column { name: "c_nationkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "l_suppkey", index: 1 }, Column { name: "c_nationkey", index: 0 }], 2)) Query Stage #9 (2 -> 2): -ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "n_regionkey", index: 3 }], 2)) +RayShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "n_regionkey", index: 3 }], 2)) ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, n_name@0 as n_name, n_regionkey@1 as n_regionkey] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_name@1, n_regionkey@2, l_extendedprice@3, l_discount@4] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 2)) + RayShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 2)) Query Stage #10 (2 -> 2): -ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "n_name", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "n_name", index: 0 }], 2)) AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(r_regionkey@0, n_regionkey@3)], projection=[l_extendedprice@1, l_discount@2, n_name@3] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "n_regionkey", index: 3 }], 2)) + RayShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "n_regionkey", index: 3 }], 2)) Query Stage #11 (2 -> 2): -ShuffleWriterExec(stage_id=11, output_partitioning=Hash([Column { name: "n_name", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=11, output_partitioning=Hash([Column { name: "n_name", index: 0 }], 2)) SortExec: expr=[revenue@1 DESC], preserve_partitioning=[true] ProjectionExec: expr=[n_name@0 as n_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue] AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "n_name", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "n_name", index: 0 }], 2)) Query Stage #12 (2 -> 1): SortPreservingMergeExec: [revenue@1 DESC] - ShuffleReaderExec(stage_id=11, input_partitioning=Hash([Column { name: "n_name", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=11, input_partitioning=Hash([Column { name: "n_name", index: 0 }], 2)) diff --git a/testdata/expected-plans/q6.txt b/testdata/expected-plans/q6.txt index 37544c1..70264bf 100644 --- a/testdata/expected-plans/q6.txt +++ b/testdata/expected-plans/q6.txt @@ -23,7 +23,7 @@ DataFusion Ray Distributed Plan =========== Query Stage #0 (2 -> 1): -ShuffleWriterExec(stage_id=0, output_partitioning=UnknownPartitioning(2)) +RayShuffleWriterExec(stage_id=0, output_partitioning=UnknownPartitioning(2)) AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] CoalesceBatchesExec: target_batch_size=8192 @@ -34,5 +34,5 @@ Query Stage #1 (1 -> 1): ProjectionExec: expr=[sum(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue] AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] CoalescePartitionsExec - ShuffleReaderExec(stage_id=0, input_partitioning=UnknownPartitioning(2)) + RayShuffleReaderExec(stage_id=0, input_partitioning=UnknownPartitioning(2)) diff --git a/testdata/expected-plans/q8.txt b/testdata/expected-plans/q8.txt index f25755b..7b24fe1 100644 --- a/testdata/expected-plans/q8.txt +++ b/testdata/expected-plans/q8.txt @@ -117,124 +117,124 @@ DataFusion Ray Distributed Plan =========== Query Stage #0 (1 -> 2): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 2)) ProjectionExec: expr=[r_regionkey@0 as r_regionkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: r_name@1 = MIDDLE EAST ParquetExec: file_groups={ ... }, projection=[r_regionkey, r_name], predicate=r_name@1 = MIDDLE EAST, pruning_predicate=CASE WHEN r_name_null_count@2 = r_name_row_count@3 THEN false ELSE r_name_min@0 <= MIDDLE EAST AND MIDDLE EAST <= r_name_max@1 END, required_guarantees=[r_name in (MIDDLE EAST)] Query Stage #1 (1 -> 2): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name] Query Stage #2 (1 -> 2): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_regionkey] Query Stage #3 (2 -> 2): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[c_custkey, c_nationkey] Query Stage #4 (2 -> 2): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 ParquetExec: file_groups={ ... }, projection=[o_orderkey, o_custkey, o_orderdate], predicate=o_orderdate@4 >= 1995-01-01 AND o_orderdate@4 <= 1996-12-31, pruning_predicate=CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_max@0 >= 1995-01-01 END AND CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_min@3 <= 1996-12-31 END, required_guarantees=[] Query Stage #5 (1 -> 2): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_nationkey] Query Stage #6 (1 -> 2): -ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) ProjectionExec: expr=[p_partkey@0 as p_partkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: p_type@1 = LARGE PLATED STEEL ParquetExec: file_groups={ ... }, projection=[p_partkey, p_type], predicate=p_type@4 = LARGE PLATED STEEL, pruning_predicate=CASE WHEN p_type_null_count@2 = p_type_row_count@3 THEN false ELSE p_type_min@0 <= LARGE PLATED STEEL AND LARGE PLATED STEEL <= p_type_max@1 END, required_guarantees=[p_type in (LARGE PLATED STEEL)] Query Stage #7 (2 -> 2): -ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "l_partkey", index: 1 }], 2)) +RayShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "l_partkey", index: 1 }], 2)) ParquetExec: file_groups={ ... }, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount] Query Stage #8 (2 -> 2): -ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 2)) +RayShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 2)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_suppkey@3, l_extendedprice@4, l_discount@5] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "l_partkey", index: 1 }], 2)) + RayShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "l_partkey", index: 1 }], 2)) Query Stage #9 (2 -> 2): -ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@0 as s_nationkey] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 2)) + RayShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 2)) Query Stage #10 (2 -> 2): -ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "o_custkey", index: 3 }], 2)) +RayShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "o_custkey", index: 3 }], 2)) ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@4 as s_nationkey, o_custkey@0 as o_custkey, o_orderdate@1 as o_orderdate] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_custkey@1, o_orderdate@2, l_extendedprice@4, l_discount@5, s_nationkey@6] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) Query Stage #11 (2 -> 2): -ShuffleWriterExec(stage_id=11, output_partitioning=Hash([Column { name: "c_nationkey", index: 4 }], 2)) +RayShuffleWriterExec(stage_id=11, output_partitioning=Hash([Column { name: "c_nationkey", index: 4 }], 2)) ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@3 as s_nationkey, o_orderdate@4 as o_orderdate, c_nationkey@0 as c_nationkey] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@3)], projection=[c_nationkey@1, l_extendedprice@2, l_discount@3, s_nationkey@4, o_orderdate@6] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "o_custkey", index: 3 }], 2)) + RayShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "o_custkey", index: 3 }], 2)) Query Stage #12 (2 -> 2): -ShuffleWriterExec(stage_id=12, output_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 2)) +RayShuffleWriterExec(stage_id=12, output_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 2)) ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@3 as s_nationkey, o_orderdate@4 as o_orderdate, n_regionkey@0 as n_regionkey] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_nationkey@0, c_nationkey@4)], projection=[n_regionkey@1, l_extendedprice@2, l_discount@3, s_nationkey@4, o_orderdate@5] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=11, input_partitioning=Hash([Column { name: "c_nationkey", index: 4 }], 2)) + RayShuffleReaderExec(stage_id=11, input_partitioning=Hash([Column { name: "c_nationkey", index: 4 }], 2)) Query Stage #13 (2 -> 2): -ShuffleWriterExec(stage_id=13, output_partitioning=Hash([Column { name: "n_regionkey", index: 3 }], 2)) +RayShuffleWriterExec(stage_id=13, output_partitioning=Hash([Column { name: "n_regionkey", index: 3 }], 2)) ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, o_orderdate@3 as o_orderdate, n_regionkey@4 as n_regionkey, n_name@0 as n_name] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_name@1, l_extendedprice@2, l_discount@3, o_orderdate@5, n_regionkey@6] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=12, input_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 2)) + RayShuffleReaderExec(stage_id=12, input_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 2)) Query Stage #14 (2 -> 2): -ShuffleWriterExec(stage_id=14, output_partitioning=Hash([Column { name: "o_year", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=14, output_partitioning=Hash([Column { name: "o_year", index: 0 }], 2)) AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = Utf8("IRAQ") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@3 as nation] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(r_regionkey@0, n_regionkey@3)], projection=[l_extendedprice@1, l_discount@2, o_orderdate@3, n_name@5] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=13, input_partitioning=Hash([Column { name: "n_regionkey", index: 3 }], 2)) + RayShuffleReaderExec(stage_id=13, input_partitioning=Hash([Column { name: "n_regionkey", index: 3 }], 2)) Query Stage #15 (2 -> 2): -ShuffleWriterExec(stage_id=15, output_partitioning=Hash([Column { name: "o_year", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=15, output_partitioning=Hash([Column { name: "o_year", index: 0 }], 2)) SortExec: expr=[o_year@0 ASC NULLS LAST], preserve_partitioning=[true] ProjectionExec: expr=[o_year@0 as o_year, sum(CASE WHEN all_nations.nation = Utf8("IRAQ") THEN all_nations.volume ELSE Int64(0) END)@1 / sum(all_nations.volume)@2 as mkt_share] AggregateExec: mode=FinalPartitioned, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = Utf8("IRAQ") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=14, input_partitioning=Hash([Column { name: "o_year", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=14, input_partitioning=Hash([Column { name: "o_year", index: 0 }], 2)) Query Stage #16 (2 -> 1): SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] - ShuffleReaderExec(stage_id=15, input_partitioning=Hash([Column { name: "o_year", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=15, input_partitioning=Hash([Column { name: "o_year", index: 0 }], 2)) diff --git a/testdata/expected-plans/q9.txt b/testdata/expected-plans/q9.txt index 20cfd92..25c45b5 100644 --- a/testdata/expected-plans/q9.txt +++ b/testdata/expected-plans/q9.txt @@ -84,91 +84,91 @@ DataFusion Ray Distributed Plan =========== Query Stage #0 (1 -> 2): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name] Query Stage #1 (2 -> 2): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[o_orderkey, o_orderdate] Query Stage #2 (2 -> 2): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }, Column { name: "ps_partkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }, Column { name: "ps_partkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[ps_partkey, ps_suppkey, ps_supplycost] Query Stage #3 (1 -> 2): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_nationkey] Query Stage #4 (1 -> 2): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) ProjectionExec: expr=[p_partkey@0 as p_partkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: p_name@1 LIKE %moccasin% ParquetExec: file_groups={ ... }, projection=[p_partkey, p_name], predicate=p_name@1 LIKE %moccasin% Query Stage #5 (2 -> 2): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "l_partkey", index: 1 }], 2)) +RayShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "l_partkey", index: 1 }], 2)) ParquetExec: file_groups={ ... }, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount] Query Stage #6 (2 -> 2): -ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "l_suppkey", index: 2 }], 2)) +RayShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "l_suppkey", index: 2 }], 2)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_partkey@2, l_suppkey@3, l_quantity@4, l_extendedprice@5, l_discount@6] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "l_partkey", index: 1 }], 2)) + RayShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "l_partkey", index: 1 }], 2)) Query Stage #7 (2 -> 2): -ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "l_suppkey", index: 2 }, Column { name: "l_partkey", index: 1 }], 2)) +RayShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "l_suppkey", index: 2 }, Column { name: "l_partkey", index: 1 }], 2)) ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_partkey@2 as l_partkey, l_suppkey@3 as l_suppkey, l_quantity@4 as l_quantity, l_extendedprice@5 as l_extendedprice, l_discount@6 as l_discount, s_nationkey@0 as s_nationkey] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@2)], projection=[s_nationkey@1, l_orderkey@2, l_partkey@3, l_suppkey@4, l_quantity@5, l_extendedprice@6, l_discount@7] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "l_suppkey", index: 2 }], 2)) + RayShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "l_suppkey", index: 2 }], 2)) Query Stage #8 (2 -> 2): -ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) +RayShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_quantity@2 as l_quantity, l_extendedprice@3 as l_extendedprice, l_discount@4 as l_discount, s_nationkey@5 as s_nationkey, ps_supplycost@0 as ps_supplycost] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, l_suppkey@2), (ps_partkey@0, l_partkey@1)], projection=[ps_supplycost@2, l_orderkey@3, l_quantity@6, l_extendedprice@7, l_discount@8, s_nationkey@9] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }, Column { name: "ps_partkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }, Column { name: "ps_partkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "l_suppkey", index: 2 }, Column { name: "l_partkey", index: 1 }], 2)) + RayShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "l_suppkey", index: 2 }, Column { name: "l_partkey", index: 1 }], 2)) Query Stage #9 (2 -> 2): -ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 2)) +RayShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 2)) ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@4 as s_nationkey, ps_supplycost@5 as ps_supplycost, o_orderdate@0 as o_orderdate] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@7] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) Query Stage #10 (2 -> 2): -ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "nation", index: 0 }, Column { name: "o_year", index: 1 }], 2)) +RayShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "nation", index: 0 }, Column { name: "o_year", index: 1 }], 2)) AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] ProjectionExec: expr=[n_name@0 as nation, date_part(YEAR, o_orderdate@5) as o_year, l_extendedprice@2 * (Some(1),20,0 - l_discount@3) - ps_supplycost@4 * l_quantity@1 as amount] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[n_name@1, l_quantity@2, l_extendedprice@3, l_discount@4, ps_supplycost@6, o_orderdate@7] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) + RayShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 2)) + RayShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 2)) Query Stage #11 (2 -> 2): -ShuffleWriterExec(stage_id=11, output_partitioning=Hash([Column { name: "nation", index: 0 }, Column { name: "o_year", index: 1 }], 2)) +RayShuffleWriterExec(stage_id=11, output_partitioning=Hash([Column { name: "nation", index: 0 }, Column { name: "o_year", index: 1 }], 2)) SortExec: expr=[nation@0 ASC NULLS LAST,o_year@1 DESC], preserve_partitioning=[true] ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, sum(profit.amount)@2 as sum_profit] AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "nation", index: 0 }, Column { name: "o_year", index: 1 }], 2)) + RayShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "nation", index: 0 }, Column { name: "o_year", index: 1 }], 2)) Query Stage #12 (2 -> 1): SortPreservingMergeExec: [nation@0 ASC NULLS LAST,o_year@1 DESC] - ShuffleReaderExec(stage_id=11, input_partitioning=Hash([Column { name: "nation", index: 0 }, Column { name: "o_year", index: 1 }], 2)) + RayShuffleReaderExec(stage_id=11, input_partitioning=Hash([Column { name: "nation", index: 0 }, Column { name: "o_year", index: 1 }], 2))