Skip to content

Conversation

@mingmwang
Copy link
Contributor

Which issue does this PR close?

Closes #1862.

Rationale for this change

What changes are included in this PR?

This is the part3 of the PR. It covers the below parts:

  1. Avoid using &mut` self in SessionContext methods, trivial UT fixes
  2. Remove the global SessionContext from Ballista SchedulerServer, add SessionContextRegistry to Scheduler to manage multiple running sessions.
  3. Remove the global SessionContext from Ballista Executor, implement FunctionRegistry for TaskContext.
  4. Ballista proto grpc API changes, add optional_session_id to ExecuteQueryParams, add session_id and props to TaskDefinition. Create new SessionContext for the first time of the query and reuse the existing SessionContext for ongoing queries for a users.
  5. Propagate the session configurations from Ballista Client to Ballista Scheduler and to Executors.

Are there any user-facing changes?

@mingmwang
Copy link
Contributor Author

Hi, @alamb @yjshen @andygrove @xudong963 @thinkharderdev @yahoNanJing

Please help to take a look.

Copy link
Contributor

@alamb alamb left a comment

Choose a reason for hiding this comment

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

I didn't review all the Ballista changes carefully, but I did review the changes to DataFusion and they look good to me.

Epic work @mingmwang

}

/// Default session builder using the provided configuration
pub fn default_session_builder(config: SessionConfig) -> SessionState {
Copy link
Contributor

Choose a reason for hiding this comment

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

Since there is already a SessionContext::with_config_rt maybe we could name this function SessionState::with_config and the existing SessionState::with_config to SessionState::with_config_rt for consistency

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done

}
}

impl FunctionRegistry for TaskContext {
Copy link
Contributor

Choose a reason for hiding this comment

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

👍

pub fn parse_expr(
proto: &protobuf::LogicalExprNode,
ctx: &SessionContext,
registry: &dyn FunctionRegistry,
Copy link
Contributor

Choose a reason for hiding this comment

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

👍

// /// Batch size when reading CSV or Parquet files
// #[structopt(short = "s", long = "batch-size", default_value = "8192")]
// batch_size: usize,
/// Batch size when reading CSV or Parquet files
Copy link
Contributor

Choose a reason for hiding this comment

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

👍

Copy link
Contributor

@thinkharderdev thinkharderdev left a comment

Choose a reason for hiding this comment

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

Awesome work!

@yahoNanJing
Copy link
Contributor

Thanks @mingmwang. I do agree to remove the SessionContext from the Ballista executor and to serialize session related things in the task definition.😄🐂

@alamb
Copy link
Contributor

alamb commented Mar 27, 2022

I'll plan to merge this PR on Monday unless we hear anything else -- this PR is likely to accumulate conflicts quickly.

In fact it looks like it has accumulated a few conflicts already which need to be resolved

@mingmwang
Copy link
Contributor Author

I'll plan to merge this PR on Monday unless we hear anything else -- this PR is likely to accumulate conflicts quickly.

In fact it looks like it has accumulated a few conflicts already which need to be resolved

I will resolve the conflicts today.

…figurations - Part 3

resolve review comments
@mingmwang
Copy link
Contributor Author

Rebased to master to resolve conflicts.

@alamb alamb merged commit 2d6addd into apache:master Mar 28, 2022
@alamb
Copy link
Contributor

alamb commented Mar 28, 2022

Thanks again @mingmwang and everyone else who helped review this PR!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Refactor ExecutionContext and related conf to support multi-tenancy configurations.

5 participants