Skip to content

Commit a321cde

Browse files
abreisalamb
authored andcommitted
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 <andre@brg.rs> Signed-off-by: Andrew Lamb <andrew@nerdnetworks.org>
1 parent 4bacb4c commit a321cde

1 file changed

Lines changed: 26 additions & 19 deletions

File tree

rust/datafusion/src/execution/context.rs

Lines changed: 26 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -95,18 +95,19 @@ use parquet::file::properties::WriterProperties;
9595
/// # Ok(())
9696
/// # }
9797
/// ```
98+
#[derive(Clone)]
9899
pub struct ExecutionContext {
99100
/// Internal state for the context
100101
pub state: Arc<Mutex<ExecutionContextState>>,
101102
}
102103

103104
impl ExecutionContext {
104-
/// Create a new execution context using a default configuration.
105+
/// Creates a new execution context using a default configuration.
105106
pub fn new() -> Self {
106107
Self::with_config(ExecutionConfig::new())
107108
}
108109

109-
/// Create a new execution context using the provided configuration
110+
/// Creates a new execution context using the provided configuration.
110111
pub fn with_config(config: ExecutionConfig) -> Self {
111112
Self {
112113
state: Arc::new(Mutex::new(ExecutionContextState {
@@ -119,7 +120,7 @@ impl ExecutionContext {
119120
}
120121
}
121122

122-
/// of RecordBatch instances)
123+
/// Creates a dataframe that will execute a SQL query.
123124
pub fn sql(&mut self, sql: &str) -> Result<Arc<dyn DataFrame>> {
124125
let plan = self.create_logical_plan(sql)?;
125126
match plan {
@@ -156,8 +157,9 @@ impl ExecutionContext {
156157
}
157158
}
158159

159-
/// Creates a logical plan. This function is intended for internal use and should not be
160-
/// called directly.
160+
/// Creates a logical plan.
161+
///
162+
/// This function is intended for internal use and should not be called directly.
161163
pub fn create_logical_plan(&self, sql: &str) -> Result<LogicalPlan> {
162164
let statements = DFParser::parse_sql(sql)?;
163165

@@ -173,7 +175,7 @@ impl ExecutionContext {
173175
Ok(query_planner.statement_to_plan(&statements[0])?)
174176
}
175177

176-
/// Register variable
178+
/// Registers a variable provider within this context.
177179
pub fn register_variable(
178180
&mut self,
179181
variable_type: VarType,
@@ -186,7 +188,7 @@ impl ExecutionContext {
186188
.insert(variable_type, provider);
187189
}
188190

189-
/// Register a scalar UDF
191+
/// Registers a scalar UDF within this context.
190192
pub fn register_udf(&mut self, f: ScalarUDF) {
191193
self.state
192194
.lock()
@@ -195,7 +197,7 @@ impl ExecutionContext {
195197
.insert(f.name.clone(), Arc::new(f));
196198
}
197199

198-
/// Register a aggregate UDF
200+
/// Registers an aggregate UDF within this context.
199201
pub fn register_udaf(&mut self, f: AggregateUDF) {
200202
self.state
201203
.lock()
@@ -229,7 +231,7 @@ impl ExecutionContext {
229231
)))
230232
}
231233

232-
/// Creates a DataFrame for reading a custom TableProvider
234+
/// Creates a DataFrame for reading a custom TableProvider.
233235
pub fn read_table(
234236
&mut self,
235237
provider: Arc<dyn TableProvider + Send + Sync>,
@@ -248,7 +250,7 @@ impl ExecutionContext {
248250
)))
249251
}
250252

251-
/// Register a CSV data source so that it can be referenced from SQL statements
253+
/// Registers a CSV data source so that it can be referenced from SQL statements
252254
/// executed against this context.
253255
pub fn register_csv(
254256
&mut self,
@@ -260,7 +262,7 @@ impl ExecutionContext {
260262
Ok(())
261263
}
262264

263-
/// Register a Parquet data source so that it can be referenced from SQL statements
265+
/// Registers a Parquet data source so that it can be referenced from SQL statements
264266
/// executed against this context.
265267
pub fn register_parquet(&mut self, name: &str, filename: &str) -> Result<()> {
266268
let table = ParquetTable::try_new(
@@ -271,7 +273,7 @@ impl ExecutionContext {
271273
Ok(())
272274
}
273275

274-
/// Register a table using a custom TableProvider so that it can be referenced from SQL
276+
/// Registers a table using a custom TableProvider so that it can be referenced from SQL
275277
/// statements executed against this context.
276278
pub fn register_table(
277279
&mut self,
@@ -286,8 +288,9 @@ impl ExecutionContext {
286288
}
287289

288290
/// Retrieves a DataFrame representing a table previously registered by calling the
289-
/// register_table function. An Err result will be returned if no table has been
290-
/// registered with the provided name.
291+
/// register_table function.
292+
///
293+
/// Returns an error if no table has been registered with the provided name.
291294
pub fn table(&self, table_name: &str) -> Result<Arc<dyn DataFrame>> {
292295
match self.state.lock().unwrap().datasources.get(table_name) {
293296
Some(provider) => {
@@ -311,7 +314,11 @@ impl ExecutionContext {
311314
}
312315
}
313316

314-
/// The set of available tables. Use `table` to get a specific table.
317+
/// Returns the set of available tables.
318+
///
319+
/// Use [`table`] to get a specific table.
320+
///
321+
/// [`table`]: ExecutionContext::table
315322
pub fn tables(&self) -> HashSet<String> {
316323
self.state
317324
.lock()
@@ -322,7 +329,7 @@ impl ExecutionContext {
322329
.collect()
323330
}
324331

325-
/// Optimize the logical plan by applying optimizer rules
332+
/// Optimizes the logical plan by applying optimizer rules.
326333
pub fn optimize(&self, plan: &LogicalPlan) -> Result<LogicalPlan> {
327334
let optimizers = &self.state.lock().unwrap().config.optimizers;
328335

@@ -335,7 +342,7 @@ impl ExecutionContext {
335342
Ok(new_plan)
336343
}
337344

338-
/// Create a physical plan from a logical plan
345+
/// Creates a physical plan from a logical plan.
339346
pub fn create_physical_plan(
340347
&self,
341348
logical_plan: &LogicalPlan,
@@ -347,7 +354,7 @@ impl ExecutionContext {
347354
.create_physical_plan(logical_plan, &state)
348355
}
349356

350-
/// Execute a query and write the results to a partitioned CSV file
357+
/// Executes a query and writes the results to a partitioned CSV file.
351358
pub async fn write_csv(
352359
&self,
353360
plan: Arc<dyn ExecutionPlan>,
@@ -384,7 +391,7 @@ impl ExecutionContext {
384391
}
385392
}
386393

387-
/// Execute a query and write the results to a partitioned Parquet file
394+
/// Executes a query and writes the results to a partitioned Parquet file.
388395
pub async fn write_parquet(
389396
&self,
390397
plan: Arc<dyn ExecutionPlan>,

0 commit comments

Comments
 (0)