You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by al...@apache.org on 2021/02/06 11:46:39 UTC
[arrow] branch master updated: ARROW-11484: [Rust][DataFusion]
Derive Clone for ExecutionContext
This is an automated email from the ASF dual-hosted git repository.
alamb pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git
The following commit(s) were added to refs/heads/master by this push:
new a321cde ARROW-11484: [Rust][DataFusion] Derive Clone for ExecutionContext
a321cde is described below
commit a321cdedb75b27b669995065ffe5596b3cfb3ae4
Author: Andre Braga Reis <an...@brg.rs>
AuthorDate: Sat Feb 6 06:45:51 2021 -0500
ARROW-11484: [Rust][DataFusion] Derive Clone for ExecutionContext
This derives `Clone` for `ExecutionContext`, which should be safe since it itself is an `Arc` wrapper.
I took the opportunity to freshen up a few doc comments along the way, writing them in the same style that Rust's stdlib uses.
Closes #9407 from abreis/context-clone
Authored-by: Andre Braga Reis <an...@brg.rs>
Signed-off-by: Andrew Lamb <an...@nerdnetworks.org>
---
rust/datafusion/src/execution/context.rs | 45 ++++++++++++++++++--------------
1 file changed, 26 insertions(+), 19 deletions(-)
diff --git a/rust/datafusion/src/execution/context.rs b/rust/datafusion/src/execution/context.rs
index e8185a8..4b8a4f7 100644
--- a/rust/datafusion/src/execution/context.rs
+++ b/rust/datafusion/src/execution/context.rs
@@ -95,18 +95,19 @@ use parquet::file::properties::WriterProperties;
/// # Ok(())
/// # }
/// ```
+#[derive(Clone)]
pub struct ExecutionContext {
/// Internal state for the context
pub state: Arc<Mutex<ExecutionContextState>>,
}
impl ExecutionContext {
- /// Create a new execution context using a default configuration.
+ /// Creates a new execution context using a default configuration.
pub fn new() -> Self {
Self::with_config(ExecutionConfig::new())
}
- /// Create a new execution context using the provided configuration
+ /// Creates a new execution context using the provided configuration.
pub fn with_config(config: ExecutionConfig) -> Self {
Self {
state: Arc::new(Mutex::new(ExecutionContextState {
@@ -119,7 +120,7 @@ impl ExecutionContext {
}
}
- /// of RecordBatch instances)
+ /// Creates a dataframe that will execute a SQL query.
pub fn sql(&mut self, sql: &str) -> Result<Arc<dyn DataFrame>> {
let plan = self.create_logical_plan(sql)?;
match plan {
@@ -156,8 +157,9 @@ impl ExecutionContext {
}
}
- /// Creates a logical plan. This function is intended for internal use and should not be
- /// called directly.
+ /// Creates a logical plan.
+ ///
+ /// This function is intended for internal use and should not be called directly.
pub fn create_logical_plan(&self, sql: &str) -> Result<LogicalPlan> {
let statements = DFParser::parse_sql(sql)?;
@@ -173,7 +175,7 @@ impl ExecutionContext {
Ok(query_planner.statement_to_plan(&statements[0])?)
}
- /// Register variable
+ /// Registers a variable provider within this context.
pub fn register_variable(
&mut self,
variable_type: VarType,
@@ -186,7 +188,7 @@ impl ExecutionContext {
.insert(variable_type, provider);
}
- /// Register a scalar UDF
+ /// Registers a scalar UDF within this context.
pub fn register_udf(&mut self, f: ScalarUDF) {
self.state
.lock()
@@ -195,7 +197,7 @@ impl ExecutionContext {
.insert(f.name.clone(), Arc::new(f));
}
- /// Register a aggregate UDF
+ /// Registers an aggregate UDF within this context.
pub fn register_udaf(&mut self, f: AggregateUDF) {
self.state
.lock()
@@ -229,7 +231,7 @@ impl ExecutionContext {
)))
}
- /// Creates a DataFrame for reading a custom TableProvider
+ /// Creates a DataFrame for reading a custom TableProvider.
pub fn read_table(
&mut self,
provider: Arc<dyn TableProvider + Send + Sync>,
@@ -248,7 +250,7 @@ impl ExecutionContext {
)))
}
- /// Register a CSV data source so that it can be referenced from SQL statements
+ /// Registers a CSV data source so that it can be referenced from SQL statements
/// executed against this context.
pub fn register_csv(
&mut self,
@@ -260,7 +262,7 @@ impl ExecutionContext {
Ok(())
}
- /// Register a Parquet data source so that it can be referenced from SQL statements
+ /// Registers a Parquet data source so that it can be referenced from SQL statements
/// executed against this context.
pub fn register_parquet(&mut self, name: &str, filename: &str) -> Result<()> {
let table = ParquetTable::try_new(
@@ -271,7 +273,7 @@ impl ExecutionContext {
Ok(())
}
- /// Register a table using a custom TableProvider so that it can be referenced from SQL
+ /// Registers a table using a custom TableProvider so that it can be referenced from SQL
/// statements executed against this context.
pub fn register_table(
&mut self,
@@ -286,8 +288,9 @@ impl ExecutionContext {
}
/// Retrieves a DataFrame representing a table previously registered by calling the
- /// register_table function. An Err result will be returned if no table has been
- /// registered with the provided name.
+ /// register_table function.
+ ///
+ /// Returns an error if no table has been registered with the provided name.
pub fn table(&self, table_name: &str) -> Result<Arc<dyn DataFrame>> {
match self.state.lock().unwrap().datasources.get(table_name) {
Some(provider) => {
@@ -311,7 +314,11 @@ impl ExecutionContext {
}
}
- /// The set of available tables. Use `table` to get a specific table.
+ /// Returns the set of available tables.
+ ///
+ /// Use [`table`] to get a specific table.
+ ///
+ /// [`table`]: ExecutionContext::table
pub fn tables(&self) -> HashSet<String> {
self.state
.lock()
@@ -322,7 +329,7 @@ impl ExecutionContext {
.collect()
}
- /// Optimize the logical plan by applying optimizer rules
+ /// Optimizes the logical plan by applying optimizer rules.
pub fn optimize(&self, plan: &LogicalPlan) -> Result<LogicalPlan> {
let optimizers = &self.state.lock().unwrap().config.optimizers;
@@ -335,7 +342,7 @@ impl ExecutionContext {
Ok(new_plan)
}
- /// Create a physical plan from a logical plan
+ /// Creates a physical plan from a logical plan.
pub fn create_physical_plan(
&self,
logical_plan: &LogicalPlan,
@@ -347,7 +354,7 @@ impl ExecutionContext {
.create_physical_plan(logical_plan, &state)
}
- /// Execute a query and write the results to a partitioned CSV file
+ /// Executes a query and writes the results to a partitioned CSV file.
pub async fn write_csv(
&self,
plan: Arc<dyn ExecutionPlan>,
@@ -384,7 +391,7 @@ impl ExecutionContext {
}
}
- /// Execute a query and write the results to a partitioned Parquet file
+ /// Executes a query and writes the results to a partitioned Parquet file.
pub async fn write_parquet(
&self,
plan: Arc<dyn ExecutionPlan>,