You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/03/03 10:45:44 UTC

[GitHub] [arrow-datafusion] yahoNanJing opened a new pull request #1912: Refactor the event channel

yahoNanJing opened a new pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912


   # Which issue does this PR close?
   
   <!--
   We generally require a GitHub issue to be filed for all bug fixes and enhancements and this helps us generate change logs for our releases. You can link an issue to this PR using the GitHub syntax. For example `Closes #123` indicates that this PR will close issue #123.
   -->
   
   Closes #1909.
   
    # Rationale for this change
   <!--
    Why are you proposing this change? If this is already explained clearly in the issue then this section is not needed.
    Explaining clearly why changes are proposed helps reviewers understand your changes and offer better suggestions for fixes.  
   -->
   
   Mentioned in #1909.
   
   # What changes are included in this PR?
   <!--
   There is no need to duplicate the description in the issue here but it is sometimes worth providing a summary of the individual changes in this PR.
   -->
   
   Introduce the following for event-based processing:
   - EventAction for common trait
   - EventLoop for common struct
   
   # Are there any user-facing changes?
   <!--
   If there are user-facing changes then we may require documentation to be updated before approving the PR.
   -->
   
   <!--
   If there are any breaking changes to public APIs, please add the `api change` label.
   -->
   It's blocked by #1911.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] thinkharderdev commented on pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
thinkharderdev commented on pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#issuecomment-1061288380


   > I am happy with this PR now other than the `unsafe` -- if @thinkharderdev or @liukun4515 is ok with the `unsafe` approach, I'll be happy to merge this PR. Otherwise I really suggest going with a `RwLock`
   
   I would also prefer we remove the `unsafe` block before merging. 


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] alamb commented on a change in pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#discussion_r820217955



##########
File path: ballista/rust/scheduler/src/scheduler_server/mod.rs
##########
@@ -0,0 +1,142 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::collections::HashMap;
+use std::sync::Arc;
+use std::time::{SystemTime, UNIX_EPOCH};
+
+use tokio::sync::RwLock;
+use tonic::transport::Channel;
+
+use ballista_core::config::{BallistaConfig, TaskSchedulingPolicy};
+use ballista_core::error::Result;
+use ballista_core::event_loop::EventLoop;
+use ballista_core::serde::protobuf::executor_grpc_client::ExecutorGrpcClient;
+
+use ballista_core::serde::{AsExecutionPlan, AsLogicalPlan, BallistaCodec};
+use datafusion::prelude::{ExecutionConfig, ExecutionContext};
+
+use crate::scheduler_server::event_loop::{
+    SchedulerServerEvent, SchedulerServerEventAction,
+};
+use crate::state::{ConfigBackendClient, SchedulerState};
+
+// include the generated protobuf source as a submodule
+#[allow(clippy::all)]
+pub mod externalscaler {
+    include!(concat!(env!("OUT_DIR"), "/externalscaler.rs"));
+}
+
+mod event_loop;
+mod external_scaler;
+mod grpc;
+mod task_scheduler;
+
+type ExecutorsClient = Arc<RwLock<HashMap<String, ExecutorGrpcClient<Channel>>>>;
+
+#[derive(Clone)]
+pub struct SchedulerServer<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> {
+    pub(crate) state: Arc<SchedulerState<T, U>>,
+    pub start_time: u128,
+    policy: TaskSchedulingPolicy,
+    executors_client: Option<ExecutorsClient>,
+    event_loop: Option<Arc<EventLoop<SchedulerServerEvent>>>,
+    ctx: Arc<RwLock<ExecutionContext>>,
+    codec: BallistaCodec<T, U>,
+}
+
+impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> SchedulerServer<T, U> {
+    pub fn new(
+        config: Arc<dyn ConfigBackendClient>,
+        namespace: String,
+        ctx: Arc<RwLock<ExecutionContext>>,
+        codec: BallistaCodec<T, U>,
+    ) -> Self {
+        SchedulerServer::new_with_policy(
+            config,
+            namespace,
+            TaskSchedulingPolicy::PullStaged,
+            ctx,
+            codec,
+        )
+    }
+
+    pub fn new_with_policy(
+        config: Arc<dyn ConfigBackendClient>,
+        namespace: String,
+        policy: TaskSchedulingPolicy,
+        ctx: Arc<RwLock<ExecutionContext>>,
+        codec: BallistaCodec<T, U>,
+    ) -> Self {
+        let state = Arc::new(SchedulerState::new(config, namespace, codec.clone()));
+
+        let (executors_client, event_loop) =
+            if matches!(policy, TaskSchedulingPolicy::PushStaged) {
+                let executors_client = Arc::new(RwLock::new(HashMap::new()));
+                let event_action: Arc<SchedulerServerEventAction<T, U>> =
+                    Arc::new(SchedulerServerEventAction::new(
+                        state.clone(),
+                        executors_client.clone(),
+                    ));
+                let event_loop =
+                    Arc::new(EventLoop::new("scheduler".to_owned(), 10000, event_action));
+                (Some(executors_client), Some(event_loop))
+            } else {
+                (None, None)
+            };
+        Self {
+            state,
+            start_time: SystemTime::now()
+                .duration_since(UNIX_EPOCH)
+                .unwrap()
+                .as_millis(),
+            policy,
+            executors_client,
+            event_loop,
+            ctx,
+            codec,
+        }
+    }
+
+    pub async fn init(&mut self) -> Result<()> {
+        {
+            // initialize state
+            let ctx = self.ctx.read().await;
+            self.state.init(&ctx).await?;
+        }
+
+        {
+            if let Some(event_loop) = self.event_loop.as_mut() {
+                // It's OK here, since we are sure the mutable reference only be used in the initialization
+                unsafe {

Review comment:
       The philosophy in Rust in general is to avoid `unsafe` as much as possible. Typically it `unsafe` is used for one of two reasons:
   1. Interfacing with other languages (like `C/C++`) that don't have the same lifetime system as Rust
   2. Performance where the overhead of doing things like array bounds checks is too great
   
   Neither of those situations seems to apply here. 
   
   I would personally suggest figuring out some way to avoid this `unsafe` block - @thinkharderdev  has one suggestion; 
   
   I don't know this code well enough to offer anything specific, but I would say another pattern for something like this is "Interior Mutability" which is a fancy rust way of saying use something like `Mutex`. Perhaps
   
   ```rust
   event_loop: Option<Arc<Mutex<EventLoop<SchedulerServerEvent>>>>,
   ```
   

##########
File path: ballista/rust/core/src/event_loop.rs
##########
@@ -0,0 +1,128 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::sync::atomic::{AtomicBool, Ordering};
+use std::sync::Arc;
+
+use async_trait::async_trait;
+use log::{error, info, warn};
+use tokio::sync::mpsc;
+
+use crate::error::{BallistaError, Result};
+
+#[async_trait]
+pub trait EventAction<E>: Send + Sync {
+    fn on_start(&self);
+
+    fn on_stop(&self);
+
+    async fn on_receive(&self, event: E) -> Result<Option<E>>;
+
+    fn on_error(&self, error: BallistaError);
+}
+
+pub struct EventLoop<E> {
+    name: String,
+    stopped: Arc<AtomicBool>,
+    buffer_size: usize,
+    action: Arc<dyn EventAction<E>>,
+    tx_event: Option<mpsc::Sender<E>>,
+}
+
+impl<E: Send + 'static> EventLoop<E> {
+    pub fn new(
+        name: String,
+        buffer_size: usize,
+        action: Arc<dyn EventAction<E>>,
+    ) -> Self {
+        Self {
+            name,
+            stopped: Arc::new(AtomicBool::new(false)),
+            buffer_size,
+            action,
+            tx_event: None,
+        }
+    }
+
+    fn run(&self, mut rx_event: mpsc::Receiver<E>) {
+        assert!(
+            self.tx_event.is_some(),
+            "The event sender should be initialized first!"
+        );
+        let tx_event = self.tx_event.as_ref().unwrap().clone();
+        let name = self.name.clone();
+        let stopped = self.stopped.clone();
+        let action = self.action.clone();
+        tokio::spawn(async move {
+            info!("Starting the event loop {}", name);
+            while !stopped.load(Ordering::SeqCst) {
+                let event = rx_event.recv().await.unwrap();

Review comment:
       In IOx and elsewhere instead of `panic` when the other end of a channel shuts down, we do something like
   
   ```rust
   info!("Event Channel closed, shutting down");
   return
   ```
   
   so it is clear from the logs what is happening and gives the process / task a chance at a clean shutdown
   

##########
File path: ballista/rust/scheduler/src/scheduler_server/event_loop.rs
##########
@@ -0,0 +1,138 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::sync::Arc;
+use std::time::Duration;
+
+use async_trait::async_trait;
+use log::{debug, warn};
+
+use ballista_core::error::{BallistaError, Result};
+use ballista_core::event_loop::EventAction;
+use ballista_core::serde::protobuf::{LaunchTaskParams, TaskDefinition};
+use ballista_core::serde::scheduler::ExecutorData;
+use ballista_core::serde::{AsExecutionPlan, AsLogicalPlan};
+
+use crate::scheduler_server::task_scheduler::TaskScheduler;
+use crate::scheduler_server::ExecutorsClient;
+use crate::state::SchedulerState;
+
+pub(crate) enum SchedulerServerEvent {
+    JobSubmitted(String),
+}
+
+pub(crate) struct SchedulerServerEventAction<
+    T: 'static + AsLogicalPlan,
+    U: 'static + AsExecutionPlan,
+> {
+    state: Arc<SchedulerState<T, U>>,
+    executors_client: ExecutorsClient,
+}
+
+impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan>
+    SchedulerServerEventAction<T, U>
+{
+    pub fn new(
+        state: Arc<SchedulerState<T, U>>,
+        executors_client: ExecutorsClient,
+    ) -> Self {
+        Self {
+            state,
+            executors_client,
+        }
+    }
+
+    async fn offer_resources(
+        &self,
+        job_id: String,
+    ) -> Result<Option<SchedulerServerEvent>> {
+        let mut available_executors = self.state.get_available_executors_data();
+        // In case of there's no enough resources, reschedule the tasks of the job
+        if available_executors.is_empty() {
+            // TODO Maybe it's better to use an exclusive runtime for this kind task scheduling
+            warn!("Not enough available executors for task running");
+            tokio::time::sleep(Duration::from_millis(100)).await;
+            return Ok(Some(SchedulerServerEvent::JobSubmitted(job_id)));
+        }
+
+        let (tasks_assigment, num_tasks) = self
+            .state
+            .fetch_tasks(&mut available_executors, &job_id)
+            .await?;
+        if num_tasks > 0 {
+            self.launch_tasks(&available_executors, tasks_assigment)
+                .await?;
+        }
+
+        Ok(None)
+    }
+
+    async fn launch_tasks(
+        &self,
+        executors: &[ExecutorData],
+        tasks_assigment: Vec<Vec<TaskDefinition>>,
+    ) -> Result<()> {
+        for (idx_executor, tasks) in tasks_assigment.into_iter().enumerate() {
+            if !tasks.is_empty() {
+                let executor_data = &executors[idx_executor];
+                debug!(
+                    "Start to launch tasks {:?} to executor {:?}",
+                    tasks, executor_data.executor_id
+                );
+                let mut client = {
+                    let clients = self.executors_client.read().await;
+                    clients.get(&executor_data.executor_id).unwrap().clone()
+                };
+                // Update the resources first
+                self.state.save_executor_data(executor_data.clone());
+                // TODO check whether launching task is successful or not
+                client.launch_task(LaunchTaskParams { task: tasks }).await?;
+            } else {
+                // Since the task assignment policy is round robin,
+                // if find tasks for one executor is empty, just break fast
+                break;
+            }
+        }
+
+        Ok(())
+    }
+}
+
+#[async_trait]
+impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan>
+    EventAction<SchedulerServerEvent> for SchedulerServerEventAction<T, U>
+{
+    // TODO
+    fn on_start(&self) {}

Review comment:
       I am not sure what the plan is here, but sometimes the `todo!()` macro gets used in situations so the TODO isn't silently forgotten. However, since `todo!()` panic's this may not be possible if the functions are called
   
   ```suggestion
       fn on_start(&self) {
         todo!();
       }
   ```
   
   




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yahoNanJing commented on a change in pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
yahoNanJing commented on a change in pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#discussion_r821256286



##########
File path: ballista/rust/scheduler/src/scheduler_server/mod.rs
##########
@@ -0,0 +1,142 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::collections::HashMap;
+use std::sync::Arc;
+use std::time::{SystemTime, UNIX_EPOCH};
+
+use tokio::sync::RwLock;
+use tonic::transport::Channel;
+
+use ballista_core::config::{BallistaConfig, TaskSchedulingPolicy};
+use ballista_core::error::Result;
+use ballista_core::event_loop::EventLoop;
+use ballista_core::serde::protobuf::executor_grpc_client::ExecutorGrpcClient;
+
+use ballista_core::serde::{AsExecutionPlan, AsLogicalPlan, BallistaCodec};
+use datafusion::prelude::{ExecutionConfig, ExecutionContext};
+
+use crate::scheduler_server::event_loop::{
+    SchedulerServerEvent, SchedulerServerEventAction,
+};
+use crate::state::{ConfigBackendClient, SchedulerState};
+
+// include the generated protobuf source as a submodule
+#[allow(clippy::all)]
+pub mod externalscaler {
+    include!(concat!(env!("OUT_DIR"), "/externalscaler.rs"));
+}
+
+mod event_loop;
+mod external_scaler;
+mod grpc;
+mod task_scheduler;
+
+type ExecutorsClient = Arc<RwLock<HashMap<String, ExecutorGrpcClient<Channel>>>>;
+
+#[derive(Clone)]
+pub struct SchedulerServer<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> {
+    pub(crate) state: Arc<SchedulerState<T, U>>,
+    pub start_time: u128,
+    policy: TaskSchedulingPolicy,
+    executors_client: Option<ExecutorsClient>,
+    event_loop: Option<Arc<EventLoop<SchedulerServerEvent>>>,
+    ctx: Arc<RwLock<ExecutionContext>>,
+    codec: BallistaCodec<T, U>,
+}
+
+impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> SchedulerServer<T, U> {
+    pub fn new(
+        config: Arc<dyn ConfigBackendClient>,
+        namespace: String,
+        ctx: Arc<RwLock<ExecutionContext>>,
+        codec: BallistaCodec<T, U>,
+    ) -> Self {
+        SchedulerServer::new_with_policy(
+            config,
+            namespace,
+            TaskSchedulingPolicy::PullStaged,
+            ctx,
+            codec,
+        )
+    }
+
+    pub fn new_with_policy(
+        config: Arc<dyn ConfigBackendClient>,
+        namespace: String,
+        policy: TaskSchedulingPolicy,
+        ctx: Arc<RwLock<ExecutionContext>>,
+        codec: BallistaCodec<T, U>,
+    ) -> Self {
+        let state = Arc::new(SchedulerState::new(config, namespace, codec.clone()));
+
+        let (executors_client, event_loop) =
+            if matches!(policy, TaskSchedulingPolicy::PushStaged) {
+                let executors_client = Arc::new(RwLock::new(HashMap::new()));
+                let event_action: Arc<SchedulerServerEventAction<T, U>> =
+                    Arc::new(SchedulerServerEventAction::new(
+                        state.clone(),
+                        executors_client.clone(),
+                    ));
+                let event_loop =
+                    Arc::new(EventLoop::new("scheduler".to_owned(), 10000, event_action));
+                (Some(executors_client), Some(event_loop))
+            } else {
+                (None, None)
+            };
+        Self {
+            state,
+            start_time: SystemTime::now()
+                .duration_since(UNIX_EPOCH)
+                .unwrap()
+                .as_millis(),
+            policy,
+            executors_client,
+            event_loop,
+            ctx,
+            codec,
+        }
+    }
+
+    pub async fn init(&mut self) -> Result<()> {
+        {
+            // initialize state
+            let ctx = self.ctx.read().await;
+            self.state.init(&ctx).await?;
+        }
+
+        {
+            if let Some(event_loop) = self.event_loop.as_mut() {
+                // It's OK here, since we are sure the mutable reference only be used in the initialization
+                unsafe {

Review comment:
       Thanks @alamb and @thinkharderdev. I think to avoid unsafe here, the only way is to use RwLock or Mutex. Although I still think it's not necessary, I will change it to use RwLock to avoid unsafe.




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yahoNanJing commented on pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
yahoNanJing commented on pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#issuecomment-1060193956


   > Thanks @yahoNanJing -- I skimmed this PR and the basic idea of breaking the system down into smaller modules I think is a great direction.
   > 
   > One thing I wanted to mention that might help with review / merge speed in the future is to break such refactoring PRs into two parts:
   > 
   > 1. A PR that just moves code around / breaks up files but doesn't change behavior
   > 2. The PR that changes behavior
   > 
   > The reason is that often moving files around causes large diffs in github but if there is no change in behavior can be straightforward to review / merge
   > 
   > The changes in behavior may take some more thought / comment and can be harder to understand in a larger diff.
   > 
   > Also, a larger PR has a greater chance to accumulate conflicts so if behavior changes get delayed in review, that can also cause issues
   
   Thanks @alamb. The background of doing this refactoring is mainly for achieving the final goal of #1704. Without these refactoring, all of the complex things will messed up in one or two files, which is really hard for maintenance. 
   
   For the step by step refactoring, every PR will be tested and verified without changing the behavior.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] alamb commented on pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
alamb commented on pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#issuecomment-1061163830


   I am happy with this PR now other than the `unsafe` -- if @thinkharderdev or @liukun4515  is ok with the `unsafe` approach, I'll be happy to merge this PR. Otherwise I really suggest going with a `RwLock`


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] alamb merged pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
alamb merged pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912


   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] alamb commented on pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
alamb commented on pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#issuecomment-1062930250


   Thanks again @yahoNanJing 


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yahoNanJing commented on a change in pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
yahoNanJing commented on a change in pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#discussion_r820368031



##########
File path: ballista/rust/scheduler/src/scheduler_server/mod.rs
##########
@@ -0,0 +1,142 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::collections::HashMap;
+use std::sync::Arc;
+use std::time::{SystemTime, UNIX_EPOCH};
+
+use tokio::sync::RwLock;
+use tonic::transport::Channel;
+
+use ballista_core::config::{BallistaConfig, TaskSchedulingPolicy};
+use ballista_core::error::Result;
+use ballista_core::event_loop::EventLoop;
+use ballista_core::serde::protobuf::executor_grpc_client::ExecutorGrpcClient;
+
+use ballista_core::serde::{AsExecutionPlan, AsLogicalPlan, BallistaCodec};
+use datafusion::prelude::{ExecutionConfig, ExecutionContext};
+
+use crate::scheduler_server::event_loop::{
+    SchedulerServerEvent, SchedulerServerEventAction,
+};
+use crate::state::{ConfigBackendClient, SchedulerState};
+
+// include the generated protobuf source as a submodule
+#[allow(clippy::all)]
+pub mod externalscaler {
+    include!(concat!(env!("OUT_DIR"), "/externalscaler.rs"));
+}
+
+mod event_loop;
+mod external_scaler;
+mod grpc;
+mod task_scheduler;
+
+type ExecutorsClient = Arc<RwLock<HashMap<String, ExecutorGrpcClient<Channel>>>>;
+
+#[derive(Clone)]
+pub struct SchedulerServer<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> {
+    pub(crate) state: Arc<SchedulerState<T, U>>,
+    pub start_time: u128,
+    policy: TaskSchedulingPolicy,
+    executors_client: Option<ExecutorsClient>,
+    event_loop: Option<Arc<EventLoop<SchedulerServerEvent>>>,
+    ctx: Arc<RwLock<ExecutionContext>>,
+    codec: BallistaCodec<T, U>,
+}
+
+impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> SchedulerServer<T, U> {
+    pub fn new(
+        config: Arc<dyn ConfigBackendClient>,
+        namespace: String,
+        ctx: Arc<RwLock<ExecutionContext>>,
+        codec: BallistaCodec<T, U>,
+    ) -> Self {
+        SchedulerServer::new_with_policy(
+            config,
+            namespace,
+            TaskSchedulingPolicy::PullStaged,
+            ctx,
+            codec,
+        )
+    }
+
+    pub fn new_with_policy(
+        config: Arc<dyn ConfigBackendClient>,
+        namespace: String,
+        policy: TaskSchedulingPolicy,
+        ctx: Arc<RwLock<ExecutionContext>>,
+        codec: BallistaCodec<T, U>,
+    ) -> Self {
+        let state = Arc::new(SchedulerState::new(config, namespace, codec.clone()));
+
+        let (executors_client, event_loop) =
+            if matches!(policy, TaskSchedulingPolicy::PushStaged) {
+                let executors_client = Arc::new(RwLock::new(HashMap::new()));
+                let event_action: Arc<SchedulerServerEventAction<T, U>> =
+                    Arc::new(SchedulerServerEventAction::new(
+                        state.clone(),
+                        executors_client.clone(),
+                    ));
+                let event_loop =
+                    Arc::new(EventLoop::new("scheduler".to_owned(), 10000, event_action));
+                (Some(executors_client), Some(event_loop))
+            } else {
+                (None, None)
+            };
+        Self {
+            state,
+            start_time: SystemTime::now()
+                .duration_since(UNIX_EPOCH)
+                .unwrap()
+                .as_millis(),
+            policy,
+            executors_client,
+            event_loop,
+            ctx,
+            codec,
+        }
+    }
+
+    pub async fn init(&mut self) -> Result<()> {
+        {
+            // initialize state
+            let ctx = self.ctx.read().await;
+            self.state.init(&ctx).await?;
+        }
+
+        {
+            if let Some(event_loop) = self.event_loop.as_mut() {
+                // It's OK here, since we are sure the mutable reference only be used in the initialization
+                unsafe {

Review comment:
       Hi @alamb, the reason I prefer not to add Mutex here is that the mutation only happens in the initialization phase. In other places, only it's only for reading. And we can avoid the overhead of locks.
   
   @thinkharderdev, this PR is for providing some encapsulation for the channel processing and extracting some common behaviors in the EventLoop. 




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] alamb commented on a change in pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#discussion_r820643364



##########
File path: ballista/rust/scheduler/src/scheduler_server/mod.rs
##########
@@ -0,0 +1,142 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::collections::HashMap;
+use std::sync::Arc;
+use std::time::{SystemTime, UNIX_EPOCH};
+
+use tokio::sync::RwLock;
+use tonic::transport::Channel;
+
+use ballista_core::config::{BallistaConfig, TaskSchedulingPolicy};
+use ballista_core::error::Result;
+use ballista_core::event_loop::EventLoop;
+use ballista_core::serde::protobuf::executor_grpc_client::ExecutorGrpcClient;
+
+use ballista_core::serde::{AsExecutionPlan, AsLogicalPlan, BallistaCodec};
+use datafusion::prelude::{ExecutionConfig, ExecutionContext};
+
+use crate::scheduler_server::event_loop::{
+    SchedulerServerEvent, SchedulerServerEventAction,
+};
+use crate::state::{ConfigBackendClient, SchedulerState};
+
+// include the generated protobuf source as a submodule
+#[allow(clippy::all)]
+pub mod externalscaler {
+    include!(concat!(env!("OUT_DIR"), "/externalscaler.rs"));
+}
+
+mod event_loop;
+mod external_scaler;
+mod grpc;
+mod task_scheduler;
+
+type ExecutorsClient = Arc<RwLock<HashMap<String, ExecutorGrpcClient<Channel>>>>;
+
+#[derive(Clone)]
+pub struct SchedulerServer<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> {
+    pub(crate) state: Arc<SchedulerState<T, U>>,
+    pub start_time: u128,
+    policy: TaskSchedulingPolicy,
+    executors_client: Option<ExecutorsClient>,
+    event_loop: Option<Arc<EventLoop<SchedulerServerEvent>>>,
+    ctx: Arc<RwLock<ExecutionContext>>,
+    codec: BallistaCodec<T, U>,
+}
+
+impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> SchedulerServer<T, U> {
+    pub fn new(
+        config: Arc<dyn ConfigBackendClient>,
+        namespace: String,
+        ctx: Arc<RwLock<ExecutionContext>>,
+        codec: BallistaCodec<T, U>,
+    ) -> Self {
+        SchedulerServer::new_with_policy(
+            config,
+            namespace,
+            TaskSchedulingPolicy::PullStaged,
+            ctx,
+            codec,
+        )
+    }
+
+    pub fn new_with_policy(
+        config: Arc<dyn ConfigBackendClient>,
+        namespace: String,
+        policy: TaskSchedulingPolicy,
+        ctx: Arc<RwLock<ExecutionContext>>,
+        codec: BallistaCodec<T, U>,
+    ) -> Self {
+        let state = Arc::new(SchedulerState::new(config, namespace, codec.clone()));
+
+        let (executors_client, event_loop) =
+            if matches!(policy, TaskSchedulingPolicy::PushStaged) {
+                let executors_client = Arc::new(RwLock::new(HashMap::new()));
+                let event_action: Arc<SchedulerServerEventAction<T, U>> =
+                    Arc::new(SchedulerServerEventAction::new(
+                        state.clone(),
+                        executors_client.clone(),
+                    ));
+                let event_loop =
+                    Arc::new(EventLoop::new("scheduler".to_owned(), 10000, event_action));
+                (Some(executors_client), Some(event_loop))
+            } else {
+                (None, None)
+            };
+        Self {
+            state,
+            start_time: SystemTime::now()
+                .duration_since(UNIX_EPOCH)
+                .unwrap()
+                .as_millis(),
+            policy,
+            executors_client,
+            event_loop,
+            ctx,
+            codec,
+        }
+    }
+
+    pub async fn init(&mut self) -> Result<()> {
+        {
+            // initialize state
+            let ctx = self.ctx.read().await;
+            self.state.init(&ctx).await?;
+        }
+
+        {
+            if let Some(event_loop) = self.event_loop.as_mut() {
+                // It's OK here, since we are sure the mutable reference only be used in the initialization
+                unsafe {

Review comment:
       My personal recommendation is go with the `safe` approach (even if it seems slower at first as it may require locks). My rationale is:
   
   I have spent countless hours tracking down strange, subtle and hard to reproduce bugs related to memory corruption when working on C/C++ systems. My Rust experience has largely been free of such pain and I think the avoidance of that pain a key advantage of Rust and makes some of the painful parts of rust (like the borrow checker) worthwhile. While the `unsafe` during initialization is fine now, if the code changes over time there may be a real chance of race conditions or other situations not currently anticipated
   
   I also think the overhead of taking a mutex is likely to be low, and you could make it lower still by using using a `RwLock` instead. 
   
   Even better would be if you can refactor the code so that it is clear when the initialization has occurred so you need neither mutex nor unsafe, perhaps as @thinkharderdev  is suggesting
   
   I like to think in most matters I am pretty pragmatic, but the use of `unsafe` is something I do feel quite strongly about. 




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] alamb edited a comment on pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
alamb edited a comment on pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#issuecomment-1062930250


   Thanks again @yahoNanJing and to @thinkharderdev and @liukun4515  for the reviews


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yahoNanJing commented on a change in pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
yahoNanJing commented on a change in pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#discussion_r820329876



##########
File path: ballista/rust/core/src/event_loop.rs
##########
@@ -0,0 +1,128 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::sync::atomic::{AtomicBool, Ordering};
+use std::sync::Arc;
+
+use async_trait::async_trait;
+use log::{error, info, warn};
+use tokio::sync::mpsc;
+
+use crate::error::{BallistaError, Result};
+
+#[async_trait]
+pub trait EventAction<E>: Send + Sync {
+    fn on_start(&self);
+
+    fn on_stop(&self);
+
+    async fn on_receive(&self, event: E) -> Result<Option<E>>;
+
+    fn on_error(&self, error: BallistaError);
+}
+
+pub struct EventLoop<E> {
+    name: String,
+    stopped: Arc<AtomicBool>,
+    buffer_size: usize,
+    action: Arc<dyn EventAction<E>>,
+    tx_event: Option<mpsc::Sender<E>>,
+}
+
+impl<E: Send + 'static> EventLoop<E> {
+    pub fn new(
+        name: String,
+        buffer_size: usize,
+        action: Arc<dyn EventAction<E>>,
+    ) -> Self {
+        Self {
+            name,
+            stopped: Arc::new(AtomicBool::new(false)),
+            buffer_size,
+            action,
+            tx_event: None,
+        }
+    }
+
+    fn run(&self, mut rx_event: mpsc::Receiver<E>) {
+        assert!(
+            self.tx_event.is_some(),
+            "The event sender should be initialized first!"
+        );
+        let tx_event = self.tx_event.as_ref().unwrap().clone();
+        let name = self.name.clone();
+        let stopped = self.stopped.clone();
+        let action = self.action.clone();
+        tokio::spawn(async move {
+            info!("Starting the event loop {}", name);
+            while !stopped.load(Ordering::SeqCst) {
+                let event = rx_event.recv().await.unwrap();

Review comment:
       Thanks @alamb. I'll add the informative log.




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] thinkharderdev commented on a change in pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
thinkharderdev commented on a change in pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#discussion_r820111007



##########
File path: ballista/rust/scheduler/src/scheduler_server/mod.rs
##########
@@ -0,0 +1,142 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::collections::HashMap;
+use std::sync::Arc;
+use std::time::{SystemTime, UNIX_EPOCH};
+
+use tokio::sync::RwLock;
+use tonic::transport::Channel;
+
+use ballista_core::config::{BallistaConfig, TaskSchedulingPolicy};
+use ballista_core::error::Result;
+use ballista_core::event_loop::EventLoop;
+use ballista_core::serde::protobuf::executor_grpc_client::ExecutorGrpcClient;
+
+use ballista_core::serde::{AsExecutionPlan, AsLogicalPlan, BallistaCodec};
+use datafusion::prelude::{ExecutionConfig, ExecutionContext};
+
+use crate::scheduler_server::event_loop::{
+    SchedulerServerEvent, SchedulerServerEventAction,
+};
+use crate::state::{ConfigBackendClient, SchedulerState};
+
+// include the generated protobuf source as a submodule
+#[allow(clippy::all)]
+pub mod externalscaler {
+    include!(concat!(env!("OUT_DIR"), "/externalscaler.rs"));
+}
+
+mod event_loop;
+mod external_scaler;
+mod grpc;
+mod task_scheduler;
+
+type ExecutorsClient = Arc<RwLock<HashMap<String, ExecutorGrpcClient<Channel>>>>;
+
+#[derive(Clone)]
+pub struct SchedulerServer<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> {
+    pub(crate) state: Arc<SchedulerState<T, U>>,
+    pub start_time: u128,
+    policy: TaskSchedulingPolicy,
+    executors_client: Option<ExecutorsClient>,
+    event_loop: Option<Arc<EventLoop<SchedulerServerEvent>>>,
+    ctx: Arc<RwLock<ExecutionContext>>,
+    codec: BallistaCodec<T, U>,
+}
+
+impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> SchedulerServer<T, U> {
+    pub fn new(
+        config: Arc<dyn ConfigBackendClient>,
+        namespace: String,
+        ctx: Arc<RwLock<ExecutionContext>>,
+        codec: BallistaCodec<T, U>,
+    ) -> Self {
+        SchedulerServer::new_with_policy(
+            config,
+            namespace,
+            TaskSchedulingPolicy::PullStaged,
+            ctx,
+            codec,
+        )
+    }
+
+    pub fn new_with_policy(
+        config: Arc<dyn ConfigBackendClient>,
+        namespace: String,
+        policy: TaskSchedulingPolicy,
+        ctx: Arc<RwLock<ExecutionContext>>,
+        codec: BallistaCodec<T, U>,
+    ) -> Self {
+        let state = Arc::new(SchedulerState::new(config, namespace, codec.clone()));
+
+        let (executors_client, event_loop) =
+            if matches!(policy, TaskSchedulingPolicy::PushStaged) {
+                let executors_client = Arc::new(RwLock::new(HashMap::new()));
+                let event_action: Arc<SchedulerServerEventAction<T, U>> =
+                    Arc::new(SchedulerServerEventAction::new(
+                        state.clone(),
+                        executors_client.clone(),
+                    ));
+                let event_loop =
+                    Arc::new(EventLoop::new("scheduler".to_owned(), 10000, event_action));
+                (Some(executors_client), Some(event_loop))
+            } else {
+                (None, None)
+            };
+        Self {
+            state,
+            start_time: SystemTime::now()
+                .duration_since(UNIX_EPOCH)
+                .unwrap()
+                .as_millis(),
+            policy,
+            executors_client,
+            event_loop,
+            ctx,
+            codec,
+        }
+    }
+
+    pub async fn init(&mut self) -> Result<()> {
+        {
+            // initialize state
+            let ctx = self.ctx.read().await;
+            self.state.init(&ctx).await?;
+        }
+
+        {
+            if let Some(event_loop) = self.event_loop.as_mut() {
+                // It's OK here, since we are sure the mutable reference only be used in the initialization
+                unsafe {

Review comment:
       I see. In that case, does the `EventLoop` need to be passed around, or can we just pass a `Sender` to different producers? In that case, the `EventLoop` could still be owned and multiple producers could hold a clone of the event loops `Sender`. 
   
   Maybe @alamb could comment on how much we should care to avoiding `unsafe`. 




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] alamb commented on a change in pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#discussion_r820643364



##########
File path: ballista/rust/scheduler/src/scheduler_server/mod.rs
##########
@@ -0,0 +1,142 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::collections::HashMap;
+use std::sync::Arc;
+use std::time::{SystemTime, UNIX_EPOCH};
+
+use tokio::sync::RwLock;
+use tonic::transport::Channel;
+
+use ballista_core::config::{BallistaConfig, TaskSchedulingPolicy};
+use ballista_core::error::Result;
+use ballista_core::event_loop::EventLoop;
+use ballista_core::serde::protobuf::executor_grpc_client::ExecutorGrpcClient;
+
+use ballista_core::serde::{AsExecutionPlan, AsLogicalPlan, BallistaCodec};
+use datafusion::prelude::{ExecutionConfig, ExecutionContext};
+
+use crate::scheduler_server::event_loop::{
+    SchedulerServerEvent, SchedulerServerEventAction,
+};
+use crate::state::{ConfigBackendClient, SchedulerState};
+
+// include the generated protobuf source as a submodule
+#[allow(clippy::all)]
+pub mod externalscaler {
+    include!(concat!(env!("OUT_DIR"), "/externalscaler.rs"));
+}
+
+mod event_loop;
+mod external_scaler;
+mod grpc;
+mod task_scheduler;
+
+type ExecutorsClient = Arc<RwLock<HashMap<String, ExecutorGrpcClient<Channel>>>>;
+
+#[derive(Clone)]
+pub struct SchedulerServer<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> {
+    pub(crate) state: Arc<SchedulerState<T, U>>,
+    pub start_time: u128,
+    policy: TaskSchedulingPolicy,
+    executors_client: Option<ExecutorsClient>,
+    event_loop: Option<Arc<EventLoop<SchedulerServerEvent>>>,
+    ctx: Arc<RwLock<ExecutionContext>>,
+    codec: BallistaCodec<T, U>,
+}
+
+impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> SchedulerServer<T, U> {
+    pub fn new(
+        config: Arc<dyn ConfigBackendClient>,
+        namespace: String,
+        ctx: Arc<RwLock<ExecutionContext>>,
+        codec: BallistaCodec<T, U>,
+    ) -> Self {
+        SchedulerServer::new_with_policy(
+            config,
+            namespace,
+            TaskSchedulingPolicy::PullStaged,
+            ctx,
+            codec,
+        )
+    }
+
+    pub fn new_with_policy(
+        config: Arc<dyn ConfigBackendClient>,
+        namespace: String,
+        policy: TaskSchedulingPolicy,
+        ctx: Arc<RwLock<ExecutionContext>>,
+        codec: BallistaCodec<T, U>,
+    ) -> Self {
+        let state = Arc::new(SchedulerState::new(config, namespace, codec.clone()));
+
+        let (executors_client, event_loop) =
+            if matches!(policy, TaskSchedulingPolicy::PushStaged) {
+                let executors_client = Arc::new(RwLock::new(HashMap::new()));
+                let event_action: Arc<SchedulerServerEventAction<T, U>> =
+                    Arc::new(SchedulerServerEventAction::new(
+                        state.clone(),
+                        executors_client.clone(),
+                    ));
+                let event_loop =
+                    Arc::new(EventLoop::new("scheduler".to_owned(), 10000, event_action));
+                (Some(executors_client), Some(event_loop))
+            } else {
+                (None, None)
+            };
+        Self {
+            state,
+            start_time: SystemTime::now()
+                .duration_since(UNIX_EPOCH)
+                .unwrap()
+                .as_millis(),
+            policy,
+            executors_client,
+            event_loop,
+            ctx,
+            codec,
+        }
+    }
+
+    pub async fn init(&mut self) -> Result<()> {
+        {
+            // initialize state
+            let ctx = self.ctx.read().await;
+            self.state.init(&ctx).await?;
+        }
+
+        {
+            if let Some(event_loop) = self.event_loop.as_mut() {
+                // It's OK here, since we are sure the mutable reference only be used in the initialization
+                unsafe {

Review comment:
       My personal recommendation is go with the `safe` approach (even if it seems slower at first as it may require locks). My rationale is:
   
   I have spent countless hours tracking down strange, subtle and hard to reproduce bugs related to memory corruption when working on C/C++ systems. My Rust experience has largely been free of such pain and I think the avoidance of that pain a key advantage of Rust and makes some of the painful parts of rust (like the borrow checker) worthwhile.
   
   I also think the overhead of taking a mutex is likely to be low, and you could make it lower still by using using a `RwLock` instead. 
   
   Even better would be if you can refactor the code so that it is clear when the initialization has occured so you need neither mutex nor unsafe, perhaps as @thinkharderdev  is suggesting
   
   I like to think in most matters I am pretty pragmatic, but the use of `unsafe` is something I do feel quite strongly about. 




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yahoNanJing commented on a change in pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
yahoNanJing commented on a change in pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#discussion_r819974810



##########
File path: ballista/rust/scheduler/src/scheduler_server/mod.rs
##########
@@ -0,0 +1,142 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::collections::HashMap;
+use std::sync::Arc;
+use std::time::{SystemTime, UNIX_EPOCH};
+
+use tokio::sync::RwLock;
+use tonic::transport::Channel;
+
+use ballista_core::config::{BallistaConfig, TaskSchedulingPolicy};
+use ballista_core::error::Result;
+use ballista_core::event_loop::EventLoop;
+use ballista_core::serde::protobuf::executor_grpc_client::ExecutorGrpcClient;
+
+use ballista_core::serde::{AsExecutionPlan, AsLogicalPlan, BallistaCodec};
+use datafusion::prelude::{ExecutionConfig, ExecutionContext};
+
+use crate::scheduler_server::event_loop::{
+    SchedulerServerEvent, SchedulerServerEventAction,
+};
+use crate::state::{ConfigBackendClient, SchedulerState};
+
+// include the generated protobuf source as a submodule
+#[allow(clippy::all)]
+pub mod externalscaler {
+    include!(concat!(env!("OUT_DIR"), "/externalscaler.rs"));
+}
+
+mod event_loop;
+mod external_scaler;
+mod grpc;
+mod task_scheduler;
+
+type ExecutorsClient = Arc<RwLock<HashMap<String, ExecutorGrpcClient<Channel>>>>;
+
+#[derive(Clone)]
+pub struct SchedulerServer<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> {
+    pub(crate) state: Arc<SchedulerState<T, U>>,
+    pub start_time: u128,
+    policy: TaskSchedulingPolicy,
+    executors_client: Option<ExecutorsClient>,
+    event_loop: Option<Arc<EventLoop<SchedulerServerEvent>>>,
+    ctx: Arc<RwLock<ExecutionContext>>,
+    codec: BallistaCodec<T, U>,
+}
+
+impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> SchedulerServer<T, U> {
+    pub fn new(
+        config: Arc<dyn ConfigBackendClient>,
+        namespace: String,
+        ctx: Arc<RwLock<ExecutionContext>>,
+        codec: BallistaCodec<T, U>,
+    ) -> Self {
+        SchedulerServer::new_with_policy(
+            config,
+            namespace,
+            TaskSchedulingPolicy::PullStaged,
+            ctx,
+            codec,
+        )
+    }
+
+    pub fn new_with_policy(
+        config: Arc<dyn ConfigBackendClient>,
+        namespace: String,
+        policy: TaskSchedulingPolicy,
+        ctx: Arc<RwLock<ExecutionContext>>,
+        codec: BallistaCodec<T, U>,
+    ) -> Self {
+        let state = Arc::new(SchedulerState::new(config, namespace, codec.clone()));
+
+        let (executors_client, event_loop) =
+            if matches!(policy, TaskSchedulingPolicy::PushStaged) {
+                let executors_client = Arc::new(RwLock::new(HashMap::new()));
+                let event_action: Arc<SchedulerServerEventAction<T, U>> =
+                    Arc::new(SchedulerServerEventAction::new(
+                        state.clone(),
+                        executors_client.clone(),
+                    ));
+                let event_loop =
+                    Arc::new(EventLoop::new("scheduler".to_owned(), 10000, event_action));
+                (Some(executors_client), Some(event_loop))
+            } else {
+                (None, None)
+            };
+        Self {
+            state,
+            start_time: SystemTime::now()
+                .duration_since(UNIX_EPOCH)
+                .unwrap()
+                .as_millis(),
+            policy,
+            executors_client,
+            event_loop,
+            ctx,
+            codec,
+        }
+    }
+
+    pub async fn init(&mut self) -> Result<()> {
+        {
+            // initialize state
+            let ctx = self.ctx.read().await;
+            self.state.init(&ctx).await?;
+        }
+
+        {
+            if let Some(event_loop) = self.event_loop.as_mut() {
+                // It's OK here, since we are sure the mutable reference only be used in the initialization
+                unsafe {

Review comment:
       Actually, in this PR the EventLoop don't need Arc. However, from the whole processing figure in #1704, the event loop will be used in somewhere else. The related PR is under progress.




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yahoNanJing commented on pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
yahoNanJing commented on pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#issuecomment-1060173782


   > Thanks @yahoNanJing -- I skimmed this PR and the basic idea of breaking the system down into smaller modules I think is a great direction.
   > 
   > One thing I wanted to mention that might help with review / merge speed in the future is to break such refactoring PRs into two parts:
   > 
   > 1. A PR that just moves code around / breaks up files but doesn't change behavior
   > 2. The PR that changes behavior
   > 
   > The reason is that often moving files around causes large diffs in github but if there is no change in behavior can be straightforward to review / merge
   > 
   > The changes in behavior may take some more thought / comment and can be harder to understand in a larger diff.
   > 
   > Also, a larger PR has a greater chance to accumulate conflicts so if behavior changes get delayed in review, that can also cause issues
   
   Thanks @alamb. The background of doing this refactoring is mainly for achieving the final goal of #1704. Without these refactoring, all of the complex things will messed up in one or two files, which is really hard for maintenance. 
   
   For the step by step refactoring, every PR will be tested and verified without changing the behavior.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] liukun4515 commented on pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
liukun4515 commented on pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#issuecomment-1059951963


   > Thanks @yahoNanJing -- I skimmed this PR and the basic idea of breaking the system down into smaller modules I think is a great direction.
   > 
   > One thing I wanted to mention that might help with review / merge speed in the future is to break such refactoring PRs into two parts:
   > 
   > 1. A PR that just moves code around / breaks up files but doesn't change behavior
   > 2. The PR that changes behavior
   > 
   > The reason is that often moving files around causes large diffs in github but if there is no change in behavior can be straightforward to review / merge
   > 
   > The changes in behavior may take some more thought / comment and can be harder to understand in a larger diff.
   > 
   > Also, a larger PR has a greater chance to accumulate conflicts so if behavior changes get delayed in review, that can also cause issues
   
   I think we can review this pull request after rebase.
   @alamb 


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yahoNanJing commented on a change in pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
yahoNanJing commented on a change in pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#discussion_r820380702



##########
File path: ballista/rust/scheduler/src/scheduler_server/event_loop.rs
##########
@@ -0,0 +1,138 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::sync::Arc;
+use std::time::Duration;
+
+use async_trait::async_trait;
+use log::{debug, warn};
+
+use ballista_core::error::{BallistaError, Result};
+use ballista_core::event_loop::EventAction;
+use ballista_core::serde::protobuf::{LaunchTaskParams, TaskDefinition};
+use ballista_core::serde::scheduler::ExecutorData;
+use ballista_core::serde::{AsExecutionPlan, AsLogicalPlan};
+
+use crate::scheduler_server::task_scheduler::TaskScheduler;
+use crate::scheduler_server::ExecutorsClient;
+use crate::state::SchedulerState;
+
+pub(crate) enum SchedulerServerEvent {
+    JobSubmitted(String),
+}
+
+pub(crate) struct SchedulerServerEventAction<
+    T: 'static + AsLogicalPlan,
+    U: 'static + AsExecutionPlan,
+> {
+    state: Arc<SchedulerState<T, U>>,
+    executors_client: ExecutorsClient,
+}
+
+impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan>
+    SchedulerServerEventAction<T, U>
+{
+    pub fn new(
+        state: Arc<SchedulerState<T, U>>,
+        executors_client: ExecutorsClient,
+    ) -> Self {
+        Self {
+            state,
+            executors_client,
+        }
+    }
+
+    async fn offer_resources(
+        &self,
+        job_id: String,
+    ) -> Result<Option<SchedulerServerEvent>> {
+        let mut available_executors = self.state.get_available_executors_data();
+        // In case of there's no enough resources, reschedule the tasks of the job
+        if available_executors.is_empty() {
+            // TODO Maybe it's better to use an exclusive runtime for this kind task scheduling
+            warn!("Not enough available executors for task running");
+            tokio::time::sleep(Duration::from_millis(100)).await;
+            return Ok(Some(SchedulerServerEvent::JobSubmitted(job_id)));
+        }
+
+        let (tasks_assigment, num_tasks) = self
+            .state
+            .fetch_tasks(&mut available_executors, &job_id)
+            .await?;
+        if num_tasks > 0 {
+            self.launch_tasks(&available_executors, tasks_assigment)
+                .await?;
+        }
+
+        Ok(None)
+    }
+
+    async fn launch_tasks(
+        &self,
+        executors: &[ExecutorData],
+        tasks_assigment: Vec<Vec<TaskDefinition>>,
+    ) -> Result<()> {
+        for (idx_executor, tasks) in tasks_assigment.into_iter().enumerate() {
+            if !tasks.is_empty() {
+                let executor_data = &executors[idx_executor];
+                debug!(
+                    "Start to launch tasks {:?} to executor {:?}",
+                    tasks, executor_data.executor_id
+                );
+                let mut client = {
+                    let clients = self.executors_client.read().await;
+                    clients.get(&executor_data.executor_id).unwrap().clone()
+                };
+                // Update the resources first
+                self.state.save_executor_data(executor_data.clone());
+                // TODO check whether launching task is successful or not
+                client.launch_task(LaunchTaskParams { task: tasks }).await?;
+            } else {
+                // Since the task assignment policy is round robin,
+                // if find tasks for one executor is empty, just break fast
+                break;
+            }
+        }
+
+        Ok(())
+    }
+}
+
+#[async_trait]
+impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan>
+    EventAction<SchedulerServerEvent> for SchedulerServerEventAction<T, U>
+{
+    // TODO
+    fn on_start(&self) {}

Review comment:
       Agree. I'll change it to be comments.




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] thinkharderdev commented on a change in pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
thinkharderdev commented on a change in pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#discussion_r820585110



##########
File path: ballista/rust/scheduler/src/scheduler_server/mod.rs
##########
@@ -0,0 +1,142 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::collections::HashMap;
+use std::sync::Arc;
+use std::time::{SystemTime, UNIX_EPOCH};
+
+use tokio::sync::RwLock;
+use tonic::transport::Channel;
+
+use ballista_core::config::{BallistaConfig, TaskSchedulingPolicy};
+use ballista_core::error::Result;
+use ballista_core::event_loop::EventLoop;
+use ballista_core::serde::protobuf::executor_grpc_client::ExecutorGrpcClient;
+
+use ballista_core::serde::{AsExecutionPlan, AsLogicalPlan, BallistaCodec};
+use datafusion::prelude::{ExecutionConfig, ExecutionContext};
+
+use crate::scheduler_server::event_loop::{
+    SchedulerServerEvent, SchedulerServerEventAction,
+};
+use crate::state::{ConfigBackendClient, SchedulerState};
+
+// include the generated protobuf source as a submodule
+#[allow(clippy::all)]
+pub mod externalscaler {
+    include!(concat!(env!("OUT_DIR"), "/externalscaler.rs"));
+}
+
+mod event_loop;
+mod external_scaler;
+mod grpc;
+mod task_scheduler;
+
+type ExecutorsClient = Arc<RwLock<HashMap<String, ExecutorGrpcClient<Channel>>>>;
+
+#[derive(Clone)]
+pub struct SchedulerServer<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> {
+    pub(crate) state: Arc<SchedulerState<T, U>>,
+    pub start_time: u128,
+    policy: TaskSchedulingPolicy,
+    executors_client: Option<ExecutorsClient>,
+    event_loop: Option<Arc<EventLoop<SchedulerServerEvent>>>,
+    ctx: Arc<RwLock<ExecutionContext>>,
+    codec: BallistaCodec<T, U>,
+}
+
+impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> SchedulerServer<T, U> {
+    pub fn new(
+        config: Arc<dyn ConfigBackendClient>,
+        namespace: String,
+        ctx: Arc<RwLock<ExecutionContext>>,
+        codec: BallistaCodec<T, U>,
+    ) -> Self {
+        SchedulerServer::new_with_policy(
+            config,
+            namespace,
+            TaskSchedulingPolicy::PullStaged,
+            ctx,
+            codec,
+        )
+    }
+
+    pub fn new_with_policy(
+        config: Arc<dyn ConfigBackendClient>,
+        namespace: String,
+        policy: TaskSchedulingPolicy,
+        ctx: Arc<RwLock<ExecutionContext>>,
+        codec: BallistaCodec<T, U>,
+    ) -> Self {
+        let state = Arc::new(SchedulerState::new(config, namespace, codec.clone()));
+
+        let (executors_client, event_loop) =
+            if matches!(policy, TaskSchedulingPolicy::PushStaged) {
+                let executors_client = Arc::new(RwLock::new(HashMap::new()));
+                let event_action: Arc<SchedulerServerEventAction<T, U>> =
+                    Arc::new(SchedulerServerEventAction::new(
+                        state.clone(),
+                        executors_client.clone(),
+                    ));
+                let event_loop =
+                    Arc::new(EventLoop::new("scheduler".to_owned(), 10000, event_action));
+                (Some(executors_client), Some(event_loop))
+            } else {
+                (None, None)
+            };
+        Self {
+            state,
+            start_time: SystemTime::now()
+                .duration_since(UNIX_EPOCH)
+                .unwrap()
+                .as_millis(),
+            policy,
+            executors_client,
+            event_loop,
+            ctx,
+            codec,
+        }
+    }
+
+    pub async fn init(&mut self) -> Result<()> {
+        {
+            // initialize state
+            let ctx = self.ctx.read().await;
+            self.state.init(&ctx).await?;
+        }
+
+        {
+            if let Some(event_loop) = self.event_loop.as_mut() {
+                // It's OK here, since we are sure the mutable reference only be used in the initialization
+                unsafe {

Review comment:
       Right, I think it's a great idea to encapsulate the event loop, but I think we can avoid the `unsafe` and the overhead of a `Mutex` or `RwLock` by decoupling the producer and consumer interface. If we want to encapsulate the error handling around sending an event on the underlying channel then perhaps we can create an explicit `EventProducer` like:
   
   ```rust
   pub struct EventProducer<E> {
     tx_event: mpsc::Sender<E>
   }
   
   impl<E> EventProducer<E> {
         pub async fn post_event(&self, event: E) -> Result<()> {
           if let Some(tx_event) = self.tx_event.as_ref() {
               tx_event.send(event).await.map_err(|e| {
                   BallistaError::General(format!("Fail to send event due to {}", e))
               })?
           } else {
               warn!("Event sender not exist!!!");
           }
   
           Ok(())
       }
   } 
   
   impl<E> EventLoop<E> {
     pub fn producer(&self) -> Result<EventProducer<E>> {
       EventPrdocuer::new(self.tx_event.cloned().ok_or(...)?)
     }
   }
   ```




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] thinkharderdev commented on a change in pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
thinkharderdev commented on a change in pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#discussion_r819833301



##########
File path: ballista/rust/core/src/event_loop.rs
##########
@@ -0,0 +1,128 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::sync::atomic::{AtomicBool, Ordering};
+use std::sync::Arc;
+
+use async_trait::async_trait;
+use log::{error, info, warn};
+use tokio::sync::mpsc;
+
+use crate::error::{BallistaError, Result};
+
+#[async_trait]
+pub trait EventAction<E>: Send + Sync {
+    fn on_start(&self);
+
+    fn on_stop(&self);
+
+    async fn on_receive(&self, event: E) -> Result<Option<E>>;
+
+    fn on_error(&self, error: BallistaError);
+}
+
+pub struct EventLoop<E> {
+    name: String,
+    stopped: Arc<AtomicBool>,
+    buffer_size: usize,
+    action: Arc<dyn EventAction<E>>,
+    tx_event: Option<mpsc::Sender<E>>,
+}
+
+impl<E: Send + 'static> EventLoop<E> {
+    pub fn new(
+        name: String,
+        buffer_size: usize,
+        action: Arc<dyn EventAction<E>>,
+    ) -> Self {
+        Self {
+            name,
+            stopped: Arc::new(AtomicBool::new(false)),
+            buffer_size,
+            action,
+            tx_event: None,
+        }
+    }
+
+    fn run(&self, mut rx_event: mpsc::Receiver<E>) {
+        assert!(
+            self.tx_event.is_some(),
+            "The event sender should be initialized first!"
+        );
+        let tx_event = self.tx_event.as_ref().unwrap().clone();
+        let name = self.name.clone();
+        let stopped = self.stopped.clone();
+        let action = self.action.clone();
+        tokio::spawn(async move {
+            info!("Starting the event loop {}", name);
+            while !stopped.load(Ordering::SeqCst) {
+                let event = rx_event.recv().await.unwrap();

Review comment:
       So I understand, if the channel is closed, this will panic and the task will die? 

##########
File path: ballista/rust/scheduler/src/scheduler_server/mod.rs
##########
@@ -0,0 +1,142 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::collections::HashMap;
+use std::sync::Arc;
+use std::time::{SystemTime, UNIX_EPOCH};
+
+use tokio::sync::RwLock;
+use tonic::transport::Channel;
+
+use ballista_core::config::{BallistaConfig, TaskSchedulingPolicy};
+use ballista_core::error::Result;
+use ballista_core::event_loop::EventLoop;
+use ballista_core::serde::protobuf::executor_grpc_client::ExecutorGrpcClient;
+
+use ballista_core::serde::{AsExecutionPlan, AsLogicalPlan, BallistaCodec};
+use datafusion::prelude::{ExecutionConfig, ExecutionContext};
+
+use crate::scheduler_server::event_loop::{
+    SchedulerServerEvent, SchedulerServerEventAction,
+};
+use crate::state::{ConfigBackendClient, SchedulerState};
+
+// include the generated protobuf source as a submodule
+#[allow(clippy::all)]
+pub mod externalscaler {
+    include!(concat!(env!("OUT_DIR"), "/externalscaler.rs"));
+}
+
+mod event_loop;
+mod external_scaler;
+mod grpc;
+mod task_scheduler;
+
+type ExecutorsClient = Arc<RwLock<HashMap<String, ExecutorGrpcClient<Channel>>>>;
+
+#[derive(Clone)]
+pub struct SchedulerServer<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> {
+    pub(crate) state: Arc<SchedulerState<T, U>>,
+    pub start_time: u128,
+    policy: TaskSchedulingPolicy,
+    executors_client: Option<ExecutorsClient>,
+    event_loop: Option<Arc<EventLoop<SchedulerServerEvent>>>,
+    ctx: Arc<RwLock<ExecutionContext>>,
+    codec: BallistaCodec<T, U>,
+}
+
+impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> SchedulerServer<T, U> {
+    pub fn new(
+        config: Arc<dyn ConfigBackendClient>,
+        namespace: String,
+        ctx: Arc<RwLock<ExecutionContext>>,
+        codec: BallistaCodec<T, U>,
+    ) -> Self {
+        SchedulerServer::new_with_policy(
+            config,
+            namespace,
+            TaskSchedulingPolicy::PullStaged,
+            ctx,
+            codec,
+        )
+    }
+
+    pub fn new_with_policy(
+        config: Arc<dyn ConfigBackendClient>,
+        namespace: String,
+        policy: TaskSchedulingPolicy,
+        ctx: Arc<RwLock<ExecutionContext>>,
+        codec: BallistaCodec<T, U>,
+    ) -> Self {
+        let state = Arc::new(SchedulerState::new(config, namespace, codec.clone()));
+
+        let (executors_client, event_loop) =
+            if matches!(policy, TaskSchedulingPolicy::PushStaged) {
+                let executors_client = Arc::new(RwLock::new(HashMap::new()));
+                let event_action: Arc<SchedulerServerEventAction<T, U>> =
+                    Arc::new(SchedulerServerEventAction::new(
+                        state.clone(),
+                        executors_client.clone(),
+                    ));
+                let event_loop =
+                    Arc::new(EventLoop::new("scheduler".to_owned(), 10000, event_action));
+                (Some(executors_client), Some(event_loop))
+            } else {
+                (None, None)
+            };
+        Self {
+            state,
+            start_time: SystemTime::now()
+                .duration_since(UNIX_EPOCH)
+                .unwrap()
+                .as_millis(),
+            policy,
+            executors_client,
+            event_loop,
+            ctx,
+            codec,
+        }
+    }
+
+    pub async fn init(&mut self) -> Result<()> {
+        {
+            // initialize state
+            let ctx = self.ctx.read().await;
+            self.state.init(&ctx).await?;
+        }
+
+        {
+            if let Some(event_loop) = self.event_loop.as_mut() {
+                // It's OK here, since we are sure the mutable reference only be used in the initialization
+                unsafe {

Review comment:
       Can we avoid this by just not wrapping the `EventLoop` in an `Arc`?




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yahoNanJing commented on a change in pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
yahoNanJing commented on a change in pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#discussion_r819974010



##########
File path: ballista/rust/core/src/event_loop.rs
##########
@@ -0,0 +1,128 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::sync::atomic::{AtomicBool, Ordering};
+use std::sync::Arc;
+
+use async_trait::async_trait;
+use log::{error, info, warn};
+use tokio::sync::mpsc;
+
+use crate::error::{BallistaError, Result};
+
+#[async_trait]
+pub trait EventAction<E>: Send + Sync {
+    fn on_start(&self);
+
+    fn on_stop(&self);
+
+    async fn on_receive(&self, event: E) -> Result<Option<E>>;
+
+    fn on_error(&self, error: BallistaError);
+}
+
+pub struct EventLoop<E> {
+    name: String,
+    stopped: Arc<AtomicBool>,
+    buffer_size: usize,
+    action: Arc<dyn EventAction<E>>,
+    tx_event: Option<mpsc::Sender<E>>,
+}
+
+impl<E: Send + 'static> EventLoop<E> {
+    pub fn new(
+        name: String,
+        buffer_size: usize,
+        action: Arc<dyn EventAction<E>>,
+    ) -> Self {
+        Self {
+            name,
+            stopped: Arc::new(AtomicBool::new(false)),
+            buffer_size,
+            action,
+            tx_event: None,
+        }
+    }
+
+    fn run(&self, mut rx_event: mpsc::Receiver<E>) {
+        assert!(
+            self.tx_event.is_some(),
+            "The event sender should be initialized first!"
+        );
+        let tx_event = self.tx_event.as_ref().unwrap().clone();
+        let name = self.name.clone();
+        let stopped = self.stopped.clone();
+        let action = self.action.clone();
+        tokio::spawn(async move {
+            info!("Starting the event loop {}", name);
+            while !stopped.load(Ordering::SeqCst) {
+                let event = rx_event.recv().await.unwrap();

Review comment:
       Yes. In general, the channel won't be closed until the scheduler is down. The dangling task should be handled by another way, like speculative task execution.




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yahoNanJing commented on a change in pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
yahoNanJing commented on a change in pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#discussion_r821260098



##########
File path: ballista/rust/scheduler/src/scheduler_server/mod.rs
##########
@@ -0,0 +1,142 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::collections::HashMap;
+use std::sync::Arc;
+use std::time::{SystemTime, UNIX_EPOCH};
+
+use tokio::sync::RwLock;
+use tonic::transport::Channel;
+
+use ballista_core::config::{BallistaConfig, TaskSchedulingPolicy};
+use ballista_core::error::Result;
+use ballista_core::event_loop::EventLoop;
+use ballista_core::serde::protobuf::executor_grpc_client::ExecutorGrpcClient;
+
+use ballista_core::serde::{AsExecutionPlan, AsLogicalPlan, BallistaCodec};
+use datafusion::prelude::{ExecutionConfig, ExecutionContext};
+
+use crate::scheduler_server::event_loop::{
+    SchedulerServerEvent, SchedulerServerEventAction,
+};
+use crate::state::{ConfigBackendClient, SchedulerState};
+
+// include the generated protobuf source as a submodule
+#[allow(clippy::all)]
+pub mod externalscaler {
+    include!(concat!(env!("OUT_DIR"), "/externalscaler.rs"));
+}
+
+mod event_loop;
+mod external_scaler;
+mod grpc;
+mod task_scheduler;
+
+type ExecutorsClient = Arc<RwLock<HashMap<String, ExecutorGrpcClient<Channel>>>>;
+
+#[derive(Clone)]
+pub struct SchedulerServer<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> {
+    pub(crate) state: Arc<SchedulerState<T, U>>,
+    pub start_time: u128,
+    policy: TaskSchedulingPolicy,
+    executors_client: Option<ExecutorsClient>,
+    event_loop: Option<Arc<EventLoop<SchedulerServerEvent>>>,
+    ctx: Arc<RwLock<ExecutionContext>>,
+    codec: BallistaCodec<T, U>,
+}
+
+impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> SchedulerServer<T, U> {
+    pub fn new(
+        config: Arc<dyn ConfigBackendClient>,
+        namespace: String,
+        ctx: Arc<RwLock<ExecutionContext>>,
+        codec: BallistaCodec<T, U>,
+    ) -> Self {
+        SchedulerServer::new_with_policy(
+            config,
+            namespace,
+            TaskSchedulingPolicy::PullStaged,
+            ctx,
+            codec,
+        )
+    }
+
+    pub fn new_with_policy(
+        config: Arc<dyn ConfigBackendClient>,
+        namespace: String,
+        policy: TaskSchedulingPolicy,
+        ctx: Arc<RwLock<ExecutionContext>>,
+        codec: BallistaCodec<T, U>,
+    ) -> Self {
+        let state = Arc::new(SchedulerState::new(config, namespace, codec.clone()));
+
+        let (executors_client, event_loop) =
+            if matches!(policy, TaskSchedulingPolicy::PushStaged) {
+                let executors_client = Arc::new(RwLock::new(HashMap::new()));
+                let event_action: Arc<SchedulerServerEventAction<T, U>> =
+                    Arc::new(SchedulerServerEventAction::new(
+                        state.clone(),
+                        executors_client.clone(),
+                    ));
+                let event_loop =
+                    Arc::new(EventLoop::new("scheduler".to_owned(), 10000, event_action));
+                (Some(executors_client), Some(event_loop))
+            } else {
+                (None, None)
+            };
+        Self {
+            state,
+            start_time: SystemTime::now()
+                .duration_since(UNIX_EPOCH)
+                .unwrap()
+                .as_millis(),
+            policy,
+            executors_client,
+            event_loop,
+            ctx,
+            codec,
+        }
+    }
+
+    pub async fn init(&mut self) -> Result<()> {
+        {
+            // initialize state
+            let ctx = self.ctx.read().await;
+            self.state.init(&ctx).await?;
+        }
+
+        {
+            if let Some(event_loop) = self.event_loop.as_mut() {
+                // It's OK here, since we are sure the mutable reference only be used in the initialization
+                unsafe {

Review comment:
       Thanks @thinkharderdev and @alamb. I'll refine the code according to the @thinkharderdev's suggestion.




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yahoNanJing commented on pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
yahoNanJing commented on pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#issuecomment-1061373907


   Hi @thinkharderdev and @alamb, could you help review the latest commit for fixing unsafe.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] alamb commented on a change in pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#discussion_r820638651



##########
File path: ballista/rust/core/src/event_loop.rs
##########
@@ -70,20 +70,24 @@ impl<E: Send + 'static> EventLoop<E> {
         tokio::spawn(async move {
             info!("Starting the event loop {}", name);
             while !stopped.load(Ordering::SeqCst) {
-                let event = rx_event.recv().await.unwrap();
-                match action.on_receive(event).await {
-                    Ok(Some(event)) => {
-                        if let Err(e) = tx_event.send(event).await {
-                            let msg = format!("Fail to send event due to {}", e);
-                            error!("{}", msg);
-                            action.on_error(BallistaError::General(msg));
+                if let Some(event) = rx_event.recv().await {
+                    match action.on_receive(event).await {
+                        Ok(Some(event)) => {
+                            if let Err(e) = tx_event.send(event).await {
+                                let msg = format!("Fail to send event due to {}", e);
+                                error!("{}", msg);
+                                action.on_error(BallistaError::General(msg));
+                            }
                         }
+                        Err(e) => {
+                            error!("Fail to process event due to {}", e);
+                            action.on_error(e);
+                        }
+                        _ => {}
                     }
-                    Err(e) => {
-                        error!("Fail to process event due to {}", e);
-                        action.on_error(e);
-                    }
-                    _ => {}
+                } else {
+                    info!("Event Channel closed, shutting down");

Review comment:
       👍 




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yahoNanJing commented on a change in pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
yahoNanJing commented on a change in pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#discussion_r820380702



##########
File path: ballista/rust/scheduler/src/scheduler_server/event_loop.rs
##########
@@ -0,0 +1,138 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::sync::Arc;
+use std::time::Duration;
+
+use async_trait::async_trait;
+use log::{debug, warn};
+
+use ballista_core::error::{BallistaError, Result};
+use ballista_core::event_loop::EventAction;
+use ballista_core::serde::protobuf::{LaunchTaskParams, TaskDefinition};
+use ballista_core::serde::scheduler::ExecutorData;
+use ballista_core::serde::{AsExecutionPlan, AsLogicalPlan};
+
+use crate::scheduler_server::task_scheduler::TaskScheduler;
+use crate::scheduler_server::ExecutorsClient;
+use crate::state::SchedulerState;
+
+pub(crate) enum SchedulerServerEvent {
+    JobSubmitted(String),
+}
+
+pub(crate) struct SchedulerServerEventAction<
+    T: 'static + AsLogicalPlan,
+    U: 'static + AsExecutionPlan,
+> {
+    state: Arc<SchedulerState<T, U>>,
+    executors_client: ExecutorsClient,
+}
+
+impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan>
+    SchedulerServerEventAction<T, U>
+{
+    pub fn new(
+        state: Arc<SchedulerState<T, U>>,
+        executors_client: ExecutorsClient,
+    ) -> Self {
+        Self {
+            state,
+            executors_client,
+        }
+    }
+
+    async fn offer_resources(
+        &self,
+        job_id: String,
+    ) -> Result<Option<SchedulerServerEvent>> {
+        let mut available_executors = self.state.get_available_executors_data();
+        // In case of there's no enough resources, reschedule the tasks of the job
+        if available_executors.is_empty() {
+            // TODO Maybe it's better to use an exclusive runtime for this kind task scheduling
+            warn!("Not enough available executors for task running");
+            tokio::time::sleep(Duration::from_millis(100)).await;
+            return Ok(Some(SchedulerServerEvent::JobSubmitted(job_id)));
+        }
+
+        let (tasks_assigment, num_tasks) = self
+            .state
+            .fetch_tasks(&mut available_executors, &job_id)
+            .await?;
+        if num_tasks > 0 {
+            self.launch_tasks(&available_executors, tasks_assigment)
+                .await?;
+        }
+
+        Ok(None)
+    }
+
+    async fn launch_tasks(
+        &self,
+        executors: &[ExecutorData],
+        tasks_assigment: Vec<Vec<TaskDefinition>>,
+    ) -> Result<()> {
+        for (idx_executor, tasks) in tasks_assigment.into_iter().enumerate() {
+            if !tasks.is_empty() {
+                let executor_data = &executors[idx_executor];
+                debug!(
+                    "Start to launch tasks {:?} to executor {:?}",
+                    tasks, executor_data.executor_id
+                );
+                let mut client = {
+                    let clients = self.executors_client.read().await;
+                    clients.get(&executor_data.executor_id).unwrap().clone()
+                };
+                // Update the resources first
+                self.state.save_executor_data(executor_data.clone());
+                // TODO check whether launching task is successful or not
+                client.launch_task(LaunchTaskParams { task: tasks }).await?;
+            } else {
+                // Since the task assignment policy is round robin,
+                // if find tasks for one executor is empty, just break fast
+                break;
+            }
+        }
+
+        Ok(())
+    }
+}
+
+#[async_trait]
+impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan>
+    EventAction<SchedulerServerEvent> for SchedulerServerEventAction<T, U>
+{
+    // TODO
+    fn on_start(&self) {}

Review comment:
       Here, the reason not to use todo!() is to avoid panic. The reason to add TODO comments is for future error handling or other things to make the whole system more robust. If it makes confusing, we can just remove them currently.




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yahoNanJing commented on a change in pull request #1912: Refactor the event channel

Posted by GitBox <gi...@apache.org>.
yahoNanJing commented on a change in pull request #1912:
URL: https://github.com/apache/arrow-datafusion/pull/1912#discussion_r821256286



##########
File path: ballista/rust/scheduler/src/scheduler_server/mod.rs
##########
@@ -0,0 +1,142 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::collections::HashMap;
+use std::sync::Arc;
+use std::time::{SystemTime, UNIX_EPOCH};
+
+use tokio::sync::RwLock;
+use tonic::transport::Channel;
+
+use ballista_core::config::{BallistaConfig, TaskSchedulingPolicy};
+use ballista_core::error::Result;
+use ballista_core::event_loop::EventLoop;
+use ballista_core::serde::protobuf::executor_grpc_client::ExecutorGrpcClient;
+
+use ballista_core::serde::{AsExecutionPlan, AsLogicalPlan, BallistaCodec};
+use datafusion::prelude::{ExecutionConfig, ExecutionContext};
+
+use crate::scheduler_server::event_loop::{
+    SchedulerServerEvent, SchedulerServerEventAction,
+};
+use crate::state::{ConfigBackendClient, SchedulerState};
+
+// include the generated protobuf source as a submodule
+#[allow(clippy::all)]
+pub mod externalscaler {
+    include!(concat!(env!("OUT_DIR"), "/externalscaler.rs"));
+}
+
+mod event_loop;
+mod external_scaler;
+mod grpc;
+mod task_scheduler;
+
+type ExecutorsClient = Arc<RwLock<HashMap<String, ExecutorGrpcClient<Channel>>>>;
+
+#[derive(Clone)]
+pub struct SchedulerServer<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> {
+    pub(crate) state: Arc<SchedulerState<T, U>>,
+    pub start_time: u128,
+    policy: TaskSchedulingPolicy,
+    executors_client: Option<ExecutorsClient>,
+    event_loop: Option<Arc<EventLoop<SchedulerServerEvent>>>,
+    ctx: Arc<RwLock<ExecutionContext>>,
+    codec: BallistaCodec<T, U>,
+}
+
+impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> SchedulerServer<T, U> {
+    pub fn new(
+        config: Arc<dyn ConfigBackendClient>,
+        namespace: String,
+        ctx: Arc<RwLock<ExecutionContext>>,
+        codec: BallistaCodec<T, U>,
+    ) -> Self {
+        SchedulerServer::new_with_policy(
+            config,
+            namespace,
+            TaskSchedulingPolicy::PullStaged,
+            ctx,
+            codec,
+        )
+    }
+
+    pub fn new_with_policy(
+        config: Arc<dyn ConfigBackendClient>,
+        namespace: String,
+        policy: TaskSchedulingPolicy,
+        ctx: Arc<RwLock<ExecutionContext>>,
+        codec: BallistaCodec<T, U>,
+    ) -> Self {
+        let state = Arc::new(SchedulerState::new(config, namespace, codec.clone()));
+
+        let (executors_client, event_loop) =
+            if matches!(policy, TaskSchedulingPolicy::PushStaged) {
+                let executors_client = Arc::new(RwLock::new(HashMap::new()));
+                let event_action: Arc<SchedulerServerEventAction<T, U>> =
+                    Arc::new(SchedulerServerEventAction::new(
+                        state.clone(),
+                        executors_client.clone(),
+                    ));
+                let event_loop =
+                    Arc::new(EventLoop::new("scheduler".to_owned(), 10000, event_action));
+                (Some(executors_client), Some(event_loop))
+            } else {
+                (None, None)
+            };
+        Self {
+            state,
+            start_time: SystemTime::now()
+                .duration_since(UNIX_EPOCH)
+                .unwrap()
+                .as_millis(),
+            policy,
+            executors_client,
+            event_loop,
+            ctx,
+            codec,
+        }
+    }
+
+    pub async fn init(&mut self) -> Result<()> {
+        {
+            // initialize state
+            let ctx = self.ctx.read().await;
+            self.state.init(&ctx).await?;
+        }
+
+        {
+            if let Some(event_loop) = self.event_loop.as_mut() {
+                // It's OK here, since we are sure the mutable reference only be used in the initialization
+                unsafe {

Review comment:
       Thanks @alamb and @thinkharderdev. I think to avoid unsafe here, the only way is to use RwLock or Mutex. Although I still think it's not necessary, I will change it to use RwLock to avoid unsafe.




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org