You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
delegation: add gc task to clean up abandoned contexts
This change addresses a TODO in the `StageDelegation` struct. Previously, it was possible for `StageContext`s to leak if a context was produced and not consumed (or if a waiter created a map entry and timed out without any corresponding producer call). This change introduces a GC task to clear contexts from the map if they have been there for longer than a certain TTL. The TTL proposed is 2 * `wait_timeout`.
When the `StageDelegation` is dropped, the GC task is terminated.
Testing
- Adds two unit tests (one for `add_delegate_info` and one for `wait_for_delegate_info`) which test that abandoned map entries are garbage collected appropriately.
use datafusion::common::{exec_datafusion_err, exec_err};
4
4
use datafusion::error::DataFusionError;
5
-
use std::time::Duration;
5
+
use std::ops::Add;
6
+
use std::sync::Arc;
7
+
use std::time::{Duration,SystemTime,UNIX_EPOCH};
6
8
use tokio::sync::oneshot;
9
+
use tokio::sync::Notify;
10
+
use tokio::time;
11
+
use tokio::time::Instant;
7
12
8
13
/// In each stage of the distributed plan, there will be N workers. All these workers
9
14
/// need to coordinate to pull data from the next stage, which will contain M workers.
@@ -30,20 +35,70 @@ use tokio::sync::oneshot;
30
35
/// On 2, the `wait_for_delegate_info` call will create an entry in the [DashMap] with a
31
36
/// [oneshot::Sender], and listen on the other end of the channel [oneshot::Receiver] for
32
37
/// the delegate to put something there.
38
+
///
39
+
/// It's possible for [StageContext] to "get lost" if `add_delegate_info` is called without
40
+
/// a corresponding call to `wait_for_delegate_info` or vice versa. In this case, a task will
41
+
/// reap any contexts that live for longer than the `gc_ttl`.
33
42
pubstructStageDelegation{
34
-
stage_targets:DashMap<(String,usize),Oneof>,
43
+
stage_targets:Arc<DashMap<(String,usize),Value>>,
35
44
wait_timeout:Duration,
45
+
46
+
/// notify is used to shut down the garbage collection task when the StageDelegation is dropped.
47
+
notify:Arc<Notify>,
36
48
}
37
49
38
50
implDefaultforStageDelegation{
39
51
fndefault() -> Self{
40
-
Self{
41
-
stage_targets:DashMap::default(),
52
+
let stage_targets = Arc::new(DashMap::default());
53
+
let notify = Arc::new(Notify::new());
54
+
55
+
let result = Self{
56
+
stage_targets: stage_targets.clone(),
42
57
wait_timeout:Duration::from_secs(5),
58
+
notify: notify.clone(),
59
+
};
60
+
61
+
// Run the GC task.
62
+
tokio::spawn(run_gc(
63
+
stage_targets.clone(),
64
+
notify.clone(),
65
+
Duration::from_secs(30),/* gc period */
66
+
));
67
+
68
+
result
69
+
}
70
+
}
71
+
72
+
constGC_PERIOD_SECONDS:usize = 30;
73
+
74
+
// run_gc will continuously clear expired entries from the map, checking every `period`. The
75
+
// function terminates if `shutdown` is signalled.
76
+
asyncfnrun_gc(
77
+
stage_targets:Arc<DashMap<(String,usize),Value>>,
78
+
shutdown:Arc<Notify>,
79
+
period:Duration,
80
+
){
81
+
loop{
82
+
tokio::select! {
83
+
_ = shutdown.notified() => {
84
+
break;
85
+
}
86
+
_ = tokio::time::sleep(period) => {
87
+
// Performance: This iterator is sharded, so it won't lock the whole map.
88
+
stage_targets.retain(|_key, value| {
89
+
value.expiry.gt(&Instant::now())
90
+
});
91
+
}
43
92
}
44
93
}
45
94
}
46
95
96
+
implDropforStageDelegation{
97
+
fndrop(&mutself){
98
+
self.notify.notify_one();
99
+
}
100
+
}
101
+
47
102
implStageDelegation{
48
103
/// Puts the [StageContext] info so that an actor can pick it up with `wait_for_delegate_info`.
49
104
///
@@ -57,9 +112,13 @@ impl StageDelegation {
57
112
actor_idx:usize,
58
113
next_stage_context:StageContext,
59
114
) -> Result<(),DataFusionError>{
115
+
let now = SystemTime::now()
116
+
.duration_since(UNIX_EPOCH)
117
+
.unwrap()
118
+
.as_secs();
60
119
let tx = matchself.stage_targets.entry((stage_id, actor_idx)){
61
-
Entry::Occupied(entry) => match entry.get(){
62
-
Oneof::Sender(_) => match entry.remove(){
120
+
Entry::Occupied(entry) => match entry.get().value{
121
+
Oneof::Sender(_) => match entry.remove().value{
63
122
Oneof::Sender(tx) => tx,
64
123
Oneof::Receiver(_) => unreachable!(),
65
124
},
@@ -69,17 +128,14 @@ impl StageDelegation {
69
128
},
70
129
Entry::Vacant(entry) => {
71
130
let(tx, rx) = oneshot::channel();
72
-
entry.insert(Oneof::Receiver(rx));
131
+
entry.insert(Value{
132
+
expiry:Instant::now().add(self.gc_ttl()),
133
+
value:Oneof::Receiver(rx),
134
+
});
73
135
tx
74
136
}
75
137
};
76
138
77
-
// TODO: `send` does not wait for the other end of the channel to receive the message,
78
-
// so if nobody waits for it, we might leak an entry in `stage_targets` that will never
79
-
// be cleaned up. We can either:
80
-
// 1. schedule a cleanup task that iterates the entries cleaning up old ones
81
-
// 2. find some other API that allows us to .await until the other end receives the message,
82
-
// and on a timeout, cleanup the entry anyway.
83
139
tx.send(next_stage_context)
84
140
.map_err(|_| exec_datafusion_err!("Could not send stage context info"))
85
141
}
@@ -95,16 +151,24 @@ impl StageDelegation {
95
151
actor_idx:usize,
96
152
) -> Result<StageContext,DataFusionError>{
97
153
let rx = matchself.stage_targets.entry((stage_id.clone(), actor_idx)){
98
-
Entry::Occupied(entry) => match entry.get(){
99
-
Oneof::Sender(_) => returnexec_err!("Programming error: while waiting for delegate info the entry in the StageDelegation target map cannot be a Sender"),
100
-
Oneof::Receiver(_) => match entry.remove(){
154
+
Entry::Occupied(entry) => match entry.get().value{
155
+
Oneof::Sender(_) => {
156
+
returnexec_err!(
157
+
"Programming error: while waiting for delegate info the entry in the \
158
+
StageDelegation target map cannot be a Sender"
159
+
)
160
+
}
161
+
Oneof::Receiver(_) => match entry.remove().value{
101
162
Oneof::Sender(_) => unreachable!(),
102
-
Oneof::Receiver(rx) => rx
163
+
Oneof::Receiver(rx) => rx,
103
164
},
104
165
},
105
166
Entry::Vacant(entry) => {
106
167
let(tx, rx) = oneshot::channel();
107
-
entry.insert(Oneof::Sender(tx));
168
+
entry.insert(Value{
169
+
expiry:Instant::now().add(self.gc_ttl()),
170
+
value:Oneof::Sender(tx),
171
+
});
108
172
rx
109
173
}
110
174
};
@@ -118,6 +182,17 @@ impl StageDelegation {
118
182
)
119
183
})
120
184
}
185
+
186
+
// gc_ttl is used to set the expiry of elements in the map. Use 2 * the waiter wait duration
187
+
// to avoid running gc too early.
188
+
fngc_ttl(&self) -> Duration{
189
+
self.wait_timeout*2
190
+
}
191
+
}
192
+
193
+
structValue{
194
+
expiry:Instant,
195
+
value:Oneof,
121
196
}
122
197
123
198
enumOneof{
@@ -129,6 +204,7 @@ enum Oneof {
129
204
mod tests {
130
205
usesuper::*;
131
206
usecrate::stage_delegation::StageContext;
207
+
use futures::TryFutureExt;
132
208
use std::sync::Arc;
133
209
use uuid::Uuid;
134
210
@@ -222,6 +298,7 @@ mod tests {
222
298
223
299
let received_context = wait_task1.await.unwrap().unwrap();
0 commit comments