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

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
33 changes: 16 additions & 17 deletions benchmarks/src/tpch/run.rs
Original file line number Diff line number Diff line change
Expand Up @@ -142,25 +142,25 @@ impl DistributedSessionBuilder for RunOpt {
&self,
ctx: DistributedSessionBuilderContext,
) -> Result<SessionState, DataFusionError> {
let mut builder = SessionStateBuilder::new().with_default_features();

let rt_builder = self.common.runtime_env_builder()?;
let config = self
.common
.config()?
.with_collect_statistics(!self.disable_statistics)
.with_target_partitions(self.partitions())
.with_collect_statistics(!self.disable_statistics);
let mut builder = SessionStateBuilder::new()
.with_runtime_env(rt_builder.build_arc()?)
.with_default_features()
.with_config(config)
.with_distributed_user_codec(InMemoryCacheExecCodec)
.with_distributed_channel_resolver(LocalHostChannelResolver::new(self.workers.clone()))
.with_distributed_option_extension_from_headers::<WarmingUpMarker>(&ctx.headers)?
.with_target_partitions(self.partitions());

let rt_builder = self.common.runtime_env_builder()?;
.with_distributed_execution(LocalHostChannelResolver::new(self.workers.clone()))
.with_distributed_option_extension_from_headers::<WarmingUpMarker>(&ctx.headers)?;

if self.mem_table {
builder = builder.with_physical_optimizer_rule(Arc::new(InMemoryDataSourceRule));
}
if !self.workers.is_empty() {
builder = builder
.with_physical_optimizer_rule(Arc::new(DistributedPhysicalOptimizerRule))
.with_distributed_network_coalesce_tasks(
self.coalesce_tasks.unwrap_or(self.workers.len()),
)
Expand All @@ -169,10 +169,7 @@ impl DistributedSessionBuilder for RunOpt {
);
}

Ok(builder
.with_config(config)
.with_runtime_env(rt_builder.build_arc()?)
.build())
Ok(builder.build())
}
}

Expand All @@ -196,7 +193,12 @@ impl RunOpt {
}

async fn run_local(mut self) -> Result<()> {
let state = self.build_session_state(Default::default()).await?;
let mut state = self.build_session_state(Default::default()).await?;
if self.mem_table {
state = SessionStateBuilder::from(state)
.with_distributed_option_extension(WarmingUpMarker::warming_up())?
.build();
}
let ctx = SessionContext::new_with_state(state);
self.register_tables(&ctx).await?;

Expand All @@ -218,9 +220,6 @@ impl RunOpt {
for query_id in query_range.clone() {
// put the WarmingUpMarker in the context, otherwise, queries will fail as the
// InMemoryCacheExec node will think they should already be warmed up.
let ctx = ctx
.clone()
.with_distributed_option_extension(WarmingUpMarker::warming_up())?;
for query in get_query_sql(query_id)? {
self.execute_query(&ctx, &query).await?;
}
Expand Down
8 changes: 3 additions & 5 deletions examples/in_memory_cluster.rs
Original file line number Diff line number Diff line change
Expand Up @@ -7,12 +7,11 @@ use datafusion::physical_plan::displayable;
use datafusion::prelude::{ParquetReadOptions, SessionContext};
use datafusion_distributed::{
ArrowFlightEndpoint, BoxCloneSyncChannel, ChannelResolver, DistributedExt,
DistributedPhysicalOptimizerRule, DistributedSessionBuilderContext, create_flight_client,
DistributedSessionBuilderContext, create_flight_client,
};
use futures::TryStreamExt;
use hyper_util::rt::TokioIo;
use std::error::Error;
use std::sync::Arc;
use structopt::StructOpt;
use tonic::transport::{Endpoint, Server};

Expand Down Expand Up @@ -41,8 +40,7 @@ async fn main() -> Result<(), Box<dyn Error>> {

let state = SessionStateBuilder::new()
.with_default_features()
.with_distributed_channel_resolver(InMemoryChannelResolver::new())
.with_physical_optimizer_rule(Arc::new(DistributedPhysicalOptimizerRule))
.with_distributed_execution(InMemoryChannelResolver::new())
.with_distributed_network_coalesce_tasks(args.network_shuffle_tasks)
.with_distributed_network_shuffle_tasks(args.network_coalesce_tasks)
.build();
Expand Down Expand Up @@ -107,7 +105,7 @@ impl InMemoryChannelResolver {
async move {
let builder = SessionStateBuilder::new()
.with_default_features()
.with_distributed_channel_resolver(this)
.with_distributed_execution(this)
.with_runtime_env(ctx.runtime_env.clone());
Ok(builder.build())
}
Expand Down
8 changes: 2 additions & 6 deletions examples/localhost_run.rs
Original file line number Diff line number Diff line change
Expand Up @@ -6,12 +6,9 @@ use datafusion::common::DataFusionError;
use datafusion::execution::SessionStateBuilder;
use datafusion::physical_plan::displayable;
use datafusion::prelude::{ParquetReadOptions, SessionContext};
use datafusion_distributed::{
BoxCloneSyncChannel, ChannelResolver, DistributedExt, DistributedPhysicalOptimizerRule,
};
use datafusion_distributed::{BoxCloneSyncChannel, ChannelResolver, DistributedExt};
use futures::TryStreamExt;
use std::error::Error;
use std::sync::Arc;
use structopt::StructOpt;
use tonic::transport::Channel;
use url::Url;
Expand Down Expand Up @@ -47,8 +44,7 @@ async fn main() -> Result<(), Box<dyn Error>> {

let state = SessionStateBuilder::new()
.with_default_features()
.with_distributed_channel_resolver(localhost_resolver)
.with_physical_optimizer_rule(Arc::new(DistributedPhysicalOptimizerRule))
.with_distributed_execution(localhost_resolver)
.with_distributed_network_coalesce_tasks(args.network_coalesce_tasks)
.with_distributed_network_shuffle_tasks(args.network_shuffle_tasks)
.build();
Expand Down
2 changes: 1 addition & 1 deletion examples/localhost_worker.rs
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ async fn main() -> Result<(), Box<dyn Error>> {
async move {
Ok(SessionStateBuilder::new()
.with_runtime_env(ctx.runtime_env)
.with_distributed_channel_resolver(local_host_resolver)
.with_distributed_execution(local_host_resolver)
.with_default_features()
.build())
}
Expand Down
Loading