From e34c74d4ca98562bcdf9e1af01c303766005b7a5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Marko=20Milenkovi=C4=87?= Date: Sat, 19 Oct 2024 07:26:46 +0100 Subject: [PATCH] update rustdocs --- ballista/client/src/extension.rs | 54 +++++++++++++++++++++----------- 1 file changed, 36 insertions(+), 18 deletions(-) diff --git a/ballista/client/src/extension.rs b/ballista/client/src/extension.rs index 1d314cf1d6..3da955d38d 100644 --- a/ballista/client/src/extension.rs +++ b/ballista/client/src/extension.rs @@ -25,8 +25,41 @@ use ballista_core::{ use datafusion::{error::DataFusionError, prelude::SessionContext}; use datafusion_proto::protobuf::LogicalPlanNode; -/// [SessionContext] extension which provides Ballista distribution -/// support to DataFusion +/// Module provides [SessionContextExt] which adds `standalone*` and `remote*` +/// methods to [SessionContext]. +/// +/// Provided methods set up [SessionContext] with [BallistaQueryPlanner](ballista_core::utils), which +/// handles running plans on Ballista clusters. +/// +///```no_run +/// use ballista::prelude::SessionContextExt; +/// use datafusion::prelude::SessionContext; +/// +/// # #[tokio::main] +/// # async fn main() -> datafusion::error::Result<()> { +/// let ctx: SessionContext = SessionContext::remote("localhost", 50050).await?; +/// # Ok(()) +/// # } +///``` +/// +/// [SessionContextExt::standalone()] provides an easy way to start up +/// local cluster. It is an optional feature which should be enabled +/// with `standalone` +/// +///```no_run +/// use ballista::prelude::SessionContextExt; +/// use datafusion::prelude::SessionContext; +/// +/// # #[tokio::main] +/// # async fn main() -> datafusion::error::Result<()> { +/// let ctx: SessionContext = SessionContext::standalone().await?; +/// # Ok(()) +/// # } +///``` +/// +/// There are still few limitations on query distribution, thus not all +/// [SessionContext] functionalities are supported. +/// #[async_trait::async_trait] pub trait SessionContextExt { /// Create a context for executing queries against a standalone Ballista scheduler instance @@ -41,30 +74,15 @@ pub trait SessionContextExt { #[cfg(feature = "standalone")] async fn standalone() -> datafusion::error::Result; - // To be added at the later stage - // #[cfg(feature = "standalone")] - // async fn standalone_with_state( - // config: &BallistaConfig, - // session_state: SessionState, - // ) -> datafusion::error::Result; - /// Create a context for executing queries against a remote Ballista scheduler instance async fn remote_with_config( host: &str, port: u16, config: &BallistaConfig, ) -> datafusion::error::Result; - + // TODO: it may make more sense to have Url instead of host, port tuple /// Create a context for executing queries against a remote Ballista scheduler instance async fn remote(host: &str, port: u16) -> datafusion::error::Result; - - // To be added at the later stage - // async fn remote_with_state( - // host: &str, - // port: u16, - // config: &BallistaConfig, - // session_state: SessionState, - // ) -> datafusion::error::Result; } #[async_trait::async_trait]