datafusion-dist 0.4.0

A distributed streaming execution library for Apache DataFusion
Documentation
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
use std::{
    collections::{HashMap, HashSet},
    sync::Arc,
    time::Duration,
};

use backon::{ExponentialBuilder, Retryable};
use futures::future::join_all;
use log::{debug, error, warn};
use parking_lot::Mutex;
use tokio::sync::mpsc::{Receiver, Sender};

use crate::{
    DistError, DistResult, JobId,
    cluster::{DistCluster, NodeId},
    config::DistConfig,
    network::{DistNetwork, StageInfo},
    planner::StageId,
    runtime::{StageState, cleanup_stages},
};

#[derive(Debug, Clone)]
pub enum Event {
    CheckJobCompleted(JobId),
    CleanupJob(JobId),
    ReceivedStage0Tasks(Vec<StageId>),
}

const MAX_BATCH_SIZE: usize = 1024;
const EVENT_SEND_TIMEOUT: Duration = Duration::from_secs(300);
const CHECK_JOB_RETRY_MAX_DELAY: Duration = Duration::from_secs(10);
const CHECK_JOB_RETRY_MAX_TIMES: usize = 3;

fn job_check_retry_strategy() -> ExponentialBuilder {
    ExponentialBuilder::default()
        .with_max_delay(CHECK_JOB_RETRY_MAX_DELAY)
        .with_max_times(CHECK_JOB_RETRY_MAX_TIMES)
        .with_jitter()
}

pub async fn send_event_with_timeout(sender: &Sender<Event>, event: Event) -> DistResult<()> {
    tokio::time::timeout(EVENT_SEND_TIMEOUT, sender.send(event))
        .await
        .map_err(|_| {
            DistError::internal(format!(
                "Timed out sending event after {}s",
                EVENT_SEND_TIMEOUT.as_secs()
            ))
        })?
        .map_err(|e| DistError::internal(format!("Failed to send event: {e}")))
}

/// Merge duplicate events in a batch.
///
/// - `CheckJobCompleted(JobId)` and `CleanupJob(JobId)` are deduplicated by
///   `job_id`: only the first occurrence for a given job is kept.
/// - All non-empty `ReceivedStage0Tasks(Vec<StageId>)` are concatenated into a
///   single event, preserving batch order.
/// - Empty `ReceivedStage0Tasks` vectors are silently skipped.
fn merge_events(events: &mut Vec<Event>) -> Vec<Event> {
    let mut merged: Vec<Event> = Vec::with_capacity(events.len());
    let mut seen_check_jobs = HashSet::with_capacity(events.len());
    let mut seen_cleanup_jobs = HashSet::with_capacity(events.len());
    let mut stage0_ids = Vec::new();

    for event in events.drain(..) {
        match event {
            Event::CheckJobCompleted(job_id) => {
                if seen_check_jobs.insert(job_id.clone()) {
                    merged.push(Event::CheckJobCompleted(job_id));
                }
            }
            Event::CleanupJob(job_id) => {
                if seen_cleanup_jobs.insert(job_id.clone()) {
                    merged.push(Event::CleanupJob(job_id));
                }
            }
            Event::ReceivedStage0Tasks(mut ids) => {
                if !ids.is_empty() {
                    stage0_ids.append(&mut ids);
                }
            }
        }
    }

    if !stage0_ids.is_empty() {
        merged.push(Event::ReceivedStage0Tasks(stage0_ids));
    }

    merged
}

pub fn start_event_handler(mut handler: EventHandler) {
    tokio::spawn(async move {
        handler.start().await;
    });
}

pub struct EventHandler {
    pub config: Arc<DistConfig>,
    pub cluster: Arc<dyn DistCluster>,
    pub network: Arc<dyn DistNetwork>,
    pub local_stages: Arc<Mutex<HashMap<StageId, StageState>>>,
    pub sender: Sender<Event>,
    pub receiver: Receiver<Event>,
}

impl EventHandler {
    pub async fn start(&mut self) {
        let mut batch = Vec::with_capacity(MAX_BATCH_SIZE);
        loop {
            batch.clear();
            let received = self.receiver.recv_many(&mut batch, MAX_BATCH_SIZE).await;
            if received == 0 {
                break;
            }
            debug!("Received batch of {received} events, merging duplicates");
            let merged = merge_events(&mut batch);
            debug!("Merged into {} events", merged.len());
            self.handle_events(merged).await;
        }
    }

    async fn handle_events(&self, events: Vec<Event>) {
        let mut check_job_ids = Vec::new();
        let mut cleanup_job_ids = Vec::new();
        let mut all_stage0_ids = Vec::new();

        for event in events {
            debug!("Handling event: {event:?}");
            match event {
                Event::CheckJobCompleted(job_id) => check_job_ids.push(job_id),
                Event::CleanupJob(job_id) => cleanup_job_ids.push(job_id),
                Event::ReceivedStage0Tasks(stage0_ids) => all_stage0_ids.extend(stage0_ids),
            }
        }

        if !check_job_ids.is_empty() {
            let cluster = self.cluster.clone();
            let network = self.network.clone();
            let local_stages = self.local_stages.clone();
            let sender = self.sender.clone();
            tokio::spawn(async move {
                handle_check_jobs_completed(
                    &cluster,
                    &network,
                    &local_stages,
                    &sender,
                    check_job_ids.clone(),
                )
                .await;
            });
        }

        if !cleanup_job_ids.is_empty() {
            let cluster = self.cluster.clone();
            let network = self.network.clone();
            let local_stages = self.local_stages.clone();
            tokio::spawn(async move {
                if let Err(e) =
                    cleanup_jobs(&cluster, &network, &local_stages, cleanup_job_ids.clone()).await
                {
                    error!("Failed to cleanup jobs {cleanup_job_ids:?}: {e}");
                }
            });
        }

        if !all_stage0_ids.is_empty() {
            let local_stages = self.local_stages.clone();
            let stage0_task_poll_timeout = self.config.stage0_task_poll_timeout;
            let sender = self.sender.clone();
            tokio::spawn(async move {
                wait_stage0_tasks_polling(
                    &local_stages,
                    stage0_task_poll_timeout,
                    &sender,
                    all_stage0_ids,
                )
                .await
            });
        }
    }
}

async fn handle_check_jobs_completed(
    cluster: &Arc<dyn DistCluster>,
    network: &Arc<dyn DistNetwork>,
    local_stages: &Arc<Mutex<HashMap<StageId, StageState>>>,
    sender: &Sender<Event>,
    job_ids: Vec<JobId>,
) {
    match (|| async { check_jobs_completed(cluster, network, local_stages, job_ids.clone()).await })
        .retry(job_check_retry_strategy())
        .await
    {
        Ok(completed_map) => {
            for (job_id, completed) in completed_map {
                if completed {
                    debug!("Job {job_id} completed, remove it from cluster");
                    if let Err(e) =
                        send_event_with_timeout(sender, Event::CleanupJob(job_id.clone())).await
                    {
                        error!("Failed to send cleanup job event for job {job_id}: {e}");
                    }
                }
            }
        }
        Err(err) => {
            error!("Failed to check jobs {job_ids:?} completed: {err}");
        }
    }
}

pub async fn check_jobs_completed(
    cluster: &Arc<dyn DistCluster>,
    network: &Arc<dyn DistNetwork>,
    local_stages: &Arc<Mutex<HashMap<StageId, StageState>>>,
    job_ids: Vec<JobId>,
) -> DistResult<HashMap<JobId, bool>> {
    if job_ids.is_empty() {
        return Ok(HashMap::new());
    }

    // Get alive nodes for validation
    let alive_nodes = cluster
        .alive_nodes()
        .await?
        .keys()
        .cloned()
        .collect::<HashSet<_>>();

    // Determine target nodes from job_task_distribution if available
    let target_nodes_by_job = {
        let guard = local_stages.lock();
        job_ids
            .iter()
            .cloned()
            .map(|job_id| {
                let target_nodes = guard
                    .values()
                    .find(|stage| stage.stage_id.job_id == job_id)
                    .map(|stage| {
                        stage
                            .job_task_distribution
                            .values()
                            .cloned()
                            .collect::<HashSet<_>>()
                    });
                (job_id, target_nodes)
            })
            .collect::<Vec<_>>()
    };

    let mut completed_map = HashMap::with_capacity(job_ids.len());

    let mut jobs_by_node: HashMap<NodeId, Vec<JobId>> = HashMap::new();
    for (job_id, target_nodes) in target_nodes_by_job {
        match target_nodes {
            Some(nodes) if nodes.is_subset(&alive_nodes) => {
                for node_id in nodes {
                    jobs_by_node
                        .entry(node_id)
                        .or_default()
                        .push(job_id.clone());
                }
            }
            Some(nodes) => {
                let missing: Vec<_> = nodes.difference(&alive_nodes).collect();
                warn!(
                    "Job {job_id} is polluted: task nodes {missing:?} are not alive, treat as completed"
                );
                completed_map.insert(job_id, true);
            }
            None => {
                warn!(
                    "No job_task_distribution found for job {job_id}, skipping remote status check"
                );
            }
        }
    }

    let mut all_job_statuses = HashMap::new();

    if let Some(local_job_ids) = jobs_by_node.remove(&network.local_node()) {
        let local_job_statuses = local_jobs(local_stages, Some(&local_job_ids));
        all_job_statuses.extend(local_job_statuses);
    }

    let mut futures = Vec::new();
    for (node_id, job_ids) in jobs_by_node {
        let network = network.clone();
        futures.push(async move {
            network
                .get_jobs(node_id.clone(), Some(job_ids.clone()))
                .await
        });
    }

    for remote_status in join_all(futures).await {
        let remote_status = remote_status?;
        for (stage_id, remote_stage_info) in remote_status {
            all_job_statuses
                .entry(stage_id)
                .and_modify(|existing| {
                    existing.merge(&remote_stage_info);
                })
                .or_insert(remote_stage_info);
        }
    }

    for job_id in job_ids {
        if completed_map.contains_key(&job_id) {
            continue;
        }

        let stage0 = StageId {
            job_id: job_id.clone(),
            stage: 0,
        };

        let job_completed = match all_job_statuses.get(&stage0) {
            Some(stage0_info) => stage0_info.assigned_partitions.iter().all(|partition| {
                stage0_info
                    .task_set_infos
                    .iter()
                    .any(|ts| ts.dropped_partitions.contains_key(partition))
            }),
            None => true,
        };
        completed_map.insert(job_id, job_completed);
    }

    Ok(completed_map)
}

pub fn local_jobs(
    stages: &Arc<Mutex<HashMap<StageId, StageState>>>,
    job_ids: Option<&Vec<JobId>>,
) -> HashMap<StageId, StageInfo> {
    let guard = stages.lock();

    let mut result = HashMap::new();
    for (stage_id, stage_state) in guard.iter() {
        if job_ids.is_none_or(|job_ids| job_ids.contains(&stage_id.job_id)) {
            let stage_info = StageInfo::from_stage_state(stage_state);
            result.insert(stage_id.clone(), stage_info);
        }
    }

    result
}

pub async fn cleanup_jobs(
    cluster: &Arc<dyn DistCluster>,
    network: &Arc<dyn DistNetwork>,
    local_stages: &Arc<Mutex<HashMap<StageId, StageState>>>,
    job_ids: Vec<JobId>,
) -> DistResult<()> {
    let alive_nodes: HashSet<NodeId> = cluster.alive_nodes().await?.keys().cloned().collect();

    let target_nodes_by_job = {
        let guard = local_stages.lock();
        job_ids
            .iter()
            .cloned()
            .map(|job_id| {
                let target_nodes = guard
                    .values()
                    .find(|stage| stage.stage_id.job_id == job_id)
                    .map(|stage| {
                        stage
                            .job_task_distribution
                            .values()
                            .cloned()
                            .collect::<HashSet<_>>()
                    });
                (job_id, target_nodes)
            })
            .collect::<Vec<_>>()
    };

    let mut jobs_by_node: HashMap<NodeId, Vec<JobId>> = HashMap::new();
    for (job_id, target_nodes) in target_nodes_by_job {
        let nodes_to_clean: HashSet<NodeId> = match target_nodes {
            Some(nodes) if nodes.is_subset(&alive_nodes) => nodes,
            Some(nodes) => {
                let missing: Vec<_> = nodes.difference(&alive_nodes).collect();
                warn!("Job {job_id} is polluted: task nodes {missing:?} are not alive");
                nodes
                    .into_iter()
                    .filter(|n| alive_nodes.contains(n))
                    .collect()
            }
            None => alive_nodes.clone(),
        };

        for node_id in nodes_to_clean {
            jobs_by_node
                .entry(node_id)
                .or_default()
                .push(job_id.clone());
        }
    }

    if let Some(local_job_ids) = jobs_by_node.remove(&network.local_node()) {
        let local_job_ids: HashSet<JobId> = local_job_ids.into_iter().collect();
        cleanup_stages(&mut local_stages.lock(), |stage_id| {
            local_job_ids.contains(&stage_id.job_id)
        });
    }

    let mut futures = Vec::new();
    for (node_id, job_ids) in jobs_by_node {
        if !job_ids.is_empty() {
            let network = network.clone();
            futures
                .push(async move { network.cleanup_jobs(node_id.clone(), job_ids.clone()).await });
        }
    }

    for res in join_all(futures).await {
        res?;
    }
    Ok(())
}

async fn wait_stage0_tasks_polling(
    local_stages: &Arc<Mutex<HashMap<StageId, StageState>>>,
    stage0_task_poll_timeout: Duration,
    sender: &Sender<Event>,
    stage0_ids: Vec<StageId>,
) {
    tokio::time::sleep(stage0_task_poll_timeout).await;

    let mut timeout_job_ids = HashSet::new();
    {
        let stages_guard = local_stages.lock();
        for stage_id in stage0_ids {
            if let Some(stage) = stages_guard.get(&stage_id)
                && stage.never_executed()
            {
                debug!("Found stage0 {stage_id} never polled until timeout");
                timeout_job_ids.insert(stage_id.job_id.clone());
            }
        }
        drop(stages_guard);
    }

    for job_id in timeout_job_ids {
        if let Err(e) = send_event_with_timeout(sender, Event::CleanupJob(job_id.clone())).await {
            error!("Failed to send CleanupJob event for job {job_id}: {e}");
        }
    }
}