alamb commented on code in PR #4382:
URL: https://github.com/apache/arrow-datafusion/pull/4382#discussion_r1032789766


##########
datafusion/core/src/execution/context.rs:
##########
@@ -1844,55 +1836,52 @@ impl TaskContext {
         aggregate_functions: HashMap<String, Arc<AggregateUDF>>,
         runtime: Arc<RuntimeEnv>,
     ) -> Self {
+        let session_config = if task_props.is_empty() {
+            SessionConfig::new()
+        } else {
+            SessionConfig::new()
+                
.with_batch_size(task_props.get(OPT_BATCH_SIZE).unwrap().parse().unwrap())
+                .with_target_partitions(
+                    
task_props.get(TARGET_PARTITIONS).unwrap().parse().unwrap(),
+                )
+                .with_repartition_joins(
+                    
task_props.get(REPARTITION_JOINS).unwrap().parse().unwrap(),
+                )
+                .with_repartition_aggregations(
+                    task_props
+                        .get(REPARTITION_AGGREGATIONS)
+                        .unwrap()
+                        .parse()
+                        .unwrap(),
+                )
+                .with_repartition_windows(
+                    task_props
+                        .get(REPARTITION_WINDOWS)
+                        .unwrap()
+                        .parse()
+                        .unwrap(),
+                )
+                .with_parquet_pruning(
+                    task_props.get(PARQUET_PRUNING).unwrap().parse().unwrap(),
+                )
+                .with_collect_statistics(
+                    
task_props.get(COLLECT_STATISTICS).unwrap().parse().unwrap(),
+                )
+        };
+
         Self {
             task_id: Some(task_id),
             session_id,
-            properties: TaskProperties::KVPairs(task_props),
+            session_config,
             scalar_functions,
             aggregate_functions,
             runtime,
         }
     }
 
     /// Return the SessionConfig associated with the Task
-    pub fn session_config(&self) -> SessionConfig {

Review Comment:
   this conversion from string/value properties to SessionConfig is moved to 
`TaskContext::new`



##########
datafusion/core/src/execution/context.rs:
##########
@@ -1810,22 +1810,14 @@ impl FunctionRegistry for SessionState {
     }
 }
 
-/// Task Context Properties
-pub enum TaskProperties {
-    ///SessionConfig
-    SessionConfig(SessionConfig),

Review Comment:
   The core of the change is to just use `SessionConfig` everywhere, creating 
it in the constructor of `TaskContext` if needed



##########
datafusion/core/src/execution/context.rs:
##########
@@ -1914,39 +1903,22 @@ impl TaskContext {
 /// Create a new task context instance from SessionContext
 impl From<&SessionContext> for TaskContext {
     fn from(session: &SessionContext) -> Self {
-        let session_id = session.session_id.clone();
-        let (config, scalar_functions, aggregate_functions) = {
-            let session_state = session.state.read();
-            (
-                session_state.config.clone(),
-                session_state.scalar_functions.clone(),
-                session_state.aggregate_functions.clone(),
-            )
-        };
-        let runtime = session.runtime_env();
-        Self {
-            task_id: None,
-            session_id,
-            properties: TaskProperties::SessionConfig(config),
-            scalar_functions,
-            aggregate_functions,
-            runtime,
-        }
+        TaskContext::from(&*session.state.read())

Review Comment:
   this is a drive by clean up as the code was replicated in `impl 
From<&SessionState> for TaskContext {`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to