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/06 11:12:47 UTC

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

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