@@ -128,6 +128,9 @@ where
128128/// the state of the connection between a user and an instance of the
129129/// DataFusion engine.
130130///
131+ /// See examples below for how to use the `SessionContext` to execute queries
132+ /// and how to configure the session.
133+ ///
131134/// # Overview
132135///
133136/// [`SessionContext`] provides the following functionality:
@@ -200,7 +203,38 @@ where
200203/// # }
201204/// ```
202205///
203- /// # `SessionContext`, `SessionState`, and `TaskContext`
206+ /// # Example: Configuring `SessionContext`
207+ ///
208+ /// The `SessionContext` can be configured by creating a [`SessionState`] using
209+ /// [`SessionStateBuilder`]:
210+ ///
211+ /// ```
212+ /// # use std::sync::Arc;
213+ /// # use datafusion::prelude::*;
214+ /// # use datafusion::execution::SessionStateBuilder;
215+ /// # use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv};
216+ /// // Configure a 4k batch size
217+ /// let config = SessionConfig::new() .with_batch_size(4 * 1024);
218+ ///
219+ /// // configure a memory limit of 1GB with 20% slop
220+ /// let runtime_env = RuntimeEnv::new(
221+ /// RuntimeConfig::new()
222+ /// .with_memory_limit(1024 * 1024 * 1024, 0.80)
223+ /// ).unwrap();
224+ ///
225+ /// // Create a SessionState using the config and runtime_env
226+ /// let state = SessionStateBuilder::new()
227+ /// .with_config(config)
228+ /// .with_runtime_env(Arc::new(runtime_env))
229+ /// // include support for built in functions and configurations
230+ /// .with_default_features()
231+ /// .build();
232+ ///
233+ /// // Create a SessionContext
234+ /// let ctx = SessionContext::from(state);
235+ /// ```
236+ ///
237+ /// # Relationship between `SessionContext`, `SessionState`, and `TaskContext`
204238///
205239/// The state required to optimize, and evaluate queries is
206240/// broken into three levels to allow tailoring
@@ -1427,6 +1461,12 @@ impl From<&SessionContext> for TaskContext {
14271461 }
14281462}
14291463
1464+ impl From < SessionState > for SessionContext {
1465+ fn from ( state : SessionState ) -> Self {
1466+ Self :: new_with_state ( state)
1467+ }
1468+ }
1469+
14301470/// A planner used to add extensions to DataFusion logical and physical plans.
14311471#[ async_trait]
14321472pub trait QueryPlanner {
0 commit comments