|
| 1 | +use std::collections::HashMap; |
| 2 | +use std::sync::{Arc, RwLock}; |
| 3 | + |
| 4 | +use protocol::grpc::api::{ChainedOperator, FsProgram}; |
| 5 | +use tokio::sync::mpsc; |
| 6 | +use tracing::error; |
| 7 | + |
| 8 | +use crate::runtime::streaming::api::operator::ConstructedOperator; |
| 9 | +use crate::runtime::streaming::factory::OperatorFactory; |
| 10 | +use crate::runtime::streaming::job::edge_manager::EdgeManager; |
| 11 | +use crate::runtime::streaming::job::models::{PhysicalExecutionGraph, PhysicalPipeline, PipelineStatus}; |
| 12 | +use crate::runtime::streaming::job::pipeline_runner::{FusionOperatorChain, PipelineRunner}; |
| 13 | +use crate::runtime::streaming::memory::MemoryPool; |
| 14 | +use crate::runtime::streaming::protocol::control::{ControlCommand, StopMode}; |
| 15 | +use crate::runtime::streaming::storage::manager::TableManager; |
| 16 | + |
| 17 | +pub struct JobManager { |
| 18 | + active_jobs: Arc<RwLock<HashMap<String, PhysicalExecutionGraph>>>, |
| 19 | + operator_factory: Arc<OperatorFactory>, |
| 20 | + memory_pool: Arc<MemoryPool>, |
| 21 | + table_manager: Option<Arc<tokio::sync::Mutex<TableManager>>>, |
| 22 | +} |
| 23 | + |
| 24 | +impl JobManager { |
| 25 | + pub fn new( |
| 26 | + operator_factory: Arc<OperatorFactory>, |
| 27 | + max_memory_bytes: usize, |
| 28 | + table_manager: Option<Arc<tokio::sync::Mutex<TableManager>>>, |
| 29 | + ) -> Self { |
| 30 | + Self { |
| 31 | + active_jobs: Arc::new(RwLock::new(HashMap::new())), |
| 32 | + operator_factory, |
| 33 | + memory_pool: MemoryPool::new(max_memory_bytes), |
| 34 | + table_manager, |
| 35 | + } |
| 36 | + } |
| 37 | + |
| 38 | + /// 从逻辑计划点火物理线程 |
| 39 | + pub async fn submit_job(&self, program: FsProgram) -> anyhow::Result<String> { |
| 40 | + let job_id = format!("job-{}", chrono::Utc::now().timestamp_millis()); |
| 41 | + |
| 42 | + let mut edge_manager = EdgeManager::build(&program.nodes, &program.edges); |
| 43 | + let mut physical_pipelines = HashMap::new(); |
| 44 | + |
| 45 | + for node in &program.nodes { |
| 46 | + let pipe_id = node.node_index as u32; |
| 47 | + let (inbox, outboxes) = edge_manager.take_endpoints(pipe_id); |
| 48 | + let chain = self.create_chain(&node.operators)?; |
| 49 | + let (ctrl_tx, ctrl_rx) = mpsc::channel(64); |
| 50 | + let status = Arc::new(RwLock::new(PipelineStatus::Initializing)); |
| 51 | + |
| 52 | + let thread_status = status.clone(); |
| 53 | + let job_id_for_thread = job_id.clone(); |
| 54 | + let exit_job_id = job_id_for_thread.clone(); |
| 55 | + let registry_ptr = self.active_jobs.clone(); |
| 56 | + let memory_pool = self.memory_pool.clone(); |
| 57 | + let table_manager = self.table_manager.clone(); |
| 58 | + |
| 59 | + let handle = std::thread::Builder::new() |
| 60 | + .name(format!("Job-{}-Pipe-{}", job_id, pipe_id)) |
| 61 | + .spawn(move || { |
| 62 | + { |
| 63 | + let mut st = thread_status.write().unwrap(); |
| 64 | + *st = PipelineStatus::Running; |
| 65 | + } |
| 66 | + |
| 67 | + let rt = tokio::runtime::Builder::new_current_thread() |
| 68 | + .enable_all() |
| 69 | + .build() |
| 70 | + .expect("build current thread runtime"); |
| 71 | + |
| 72 | + let result = std::panic::catch_unwind(std::panic::AssertUnwindSafe(|| { |
| 73 | + rt.block_on(async move { |
| 74 | + let mut runner = PipelineRunner::new( |
| 75 | + pipe_id, |
| 76 | + chain, |
| 77 | + inbox, |
| 78 | + outboxes, |
| 79 | + ctrl_rx, |
| 80 | + job_id_for_thread.clone(), |
| 81 | + memory_pool, |
| 82 | + table_manager, |
| 83 | + ); |
| 84 | + runner.run().await |
| 85 | + }) |
| 86 | + })); |
| 87 | + |
| 88 | + Self::on_pipeline_exit(exit_job_id, pipe_id, result, thread_status, registry_ptr); |
| 89 | + })?; |
| 90 | + |
| 91 | + physical_pipelines.insert( |
| 92 | + pipe_id, |
| 93 | + PhysicalPipeline { |
| 94 | + pipeline_id: pipe_id, |
| 95 | + handle: Some(handle), |
| 96 | + status, |
| 97 | + control_tx: ctrl_tx, |
| 98 | + }, |
| 99 | + ); |
| 100 | + } |
| 101 | + |
| 102 | + let graph = PhysicalExecutionGraph { |
| 103 | + job_id: job_id.clone(), |
| 104 | + program, |
| 105 | + pipelines: physical_pipelines, |
| 106 | + start_time: std::time::Instant::now(), |
| 107 | + }; |
| 108 | + |
| 109 | + self.active_jobs.write().unwrap().insert(job_id.clone(), graph); |
| 110 | + Ok(job_id) |
| 111 | + } |
| 112 | + |
| 113 | + pub async fn stop_job(&self, job_id: &str, mode: StopMode) -> anyhow::Result<()> { |
| 114 | + let controllers = { |
| 115 | + let jobs = self.active_jobs.read().unwrap(); |
| 116 | + let graph = jobs |
| 117 | + .get(job_id) |
| 118 | + .ok_or_else(|| anyhow::anyhow!("job not found: {job_id}"))?; |
| 119 | + graph |
| 120 | + .pipelines |
| 121 | + .values() |
| 122 | + .map(|p| p.control_tx.clone()) |
| 123 | + .collect::<Vec<_>>() |
| 124 | + }; |
| 125 | + |
| 126 | + for tx in controllers { |
| 127 | + tx.send(ControlCommand::Stop { mode: mode.clone() }).await?; |
| 128 | + } |
| 129 | + Ok(()) |
| 130 | + } |
| 131 | + |
| 132 | + pub fn get_pipeline_statuses(&self, job_id: &str) -> Option<HashMap<u32, PipelineStatus>> { |
| 133 | + let jobs = self.active_jobs.read().unwrap(); |
| 134 | + let graph = jobs.get(job_id)?; |
| 135 | + Some( |
| 136 | + graph |
| 137 | + .pipelines |
| 138 | + .iter() |
| 139 | + .map(|(id, pipeline)| (*id, pipeline.status.read().unwrap().clone())) |
| 140 | + .collect(), |
| 141 | + ) |
| 142 | + } |
| 143 | + |
| 144 | + fn create_chain(&self, operators: &[ChainedOperator]) -> anyhow::Result<FusionOperatorChain> { |
| 145 | + let mut chain = Vec::with_capacity(operators.len()); |
| 146 | + for op in operators { |
| 147 | + match self |
| 148 | + .operator_factory |
| 149 | + .create_operator(&op.operator_name, &op.operator_config)? |
| 150 | + { |
| 151 | + ConstructedOperator::Operator(msg_op) => chain.push(msg_op), |
| 152 | + ConstructedOperator::Source(_) => { |
| 153 | + return Err(anyhow::anyhow!( |
| 154 | + "source operator '{}' cannot be used inside a physical pipeline chain", |
| 155 | + op.operator_name |
| 156 | + )); |
| 157 | + } |
| 158 | + } |
| 159 | + } |
| 160 | + Ok(FusionOperatorChain::new(chain)) |
| 161 | + } |
| 162 | + |
| 163 | + fn on_pipeline_exit( |
| 164 | + job_id: String, |
| 165 | + pipe_id: u32, |
| 166 | + result: std::thread::Result<anyhow::Result<()>>, |
| 167 | + status: Arc<RwLock<PipelineStatus>>, |
| 168 | + _registry: Arc<RwLock<HashMap<String, PhysicalExecutionGraph>>>, |
| 169 | + ) { |
| 170 | + let mut needs_abort = false; |
| 171 | + match result { |
| 172 | + Ok(Err(e)) => { |
| 173 | + *status.write().unwrap() = PipelineStatus::Failed { |
| 174 | + error: e.to_string(), |
| 175 | + is_panic: false, |
| 176 | + }; |
| 177 | + needs_abort = true; |
| 178 | + } |
| 179 | + Err(_) => { |
| 180 | + *status.write().unwrap() = PipelineStatus::Failed { |
| 181 | + error: "panic".into(), |
| 182 | + is_panic: true, |
| 183 | + }; |
| 184 | + needs_abort = true; |
| 185 | + } |
| 186 | + Ok(Ok(_)) => { |
| 187 | + *status.write().unwrap() = PipelineStatus::Finished; |
| 188 | + } |
| 189 | + } |
| 190 | + |
| 191 | + if needs_abort { |
| 192 | + error!( |
| 193 | + "Pipeline {}-{} failed. Initiating Job Abort.", |
| 194 | + job_id, pipe_id |
| 195 | + ); |
| 196 | + } |
| 197 | + } |
| 198 | +} |
0 commit comments