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
use crate::error::{DbxError, DbxResult};
use crate::grid::protocol::{GridMessage, QueryMessage, StorageMessage};
use crate::grid::quic::{GridMessageWrapper, QuicChannel};
use crate::sql::executor::local_executor::LocalExecutor;
use crate::sql::planner::types::PhysicalPlan;
use crate::storage::erasure_coding::distributed_store::DistributedErasureCodingStore;
use dashmap::DashMap;
use std::sync::Arc;
use tokio::sync::mpsc;
use tracing::{error, info, warn};
type QuerySender = mpsc::Sender<DbxResult<Option<Vec<u8>>>>;
type QueryStreamMap = Arc<DashMap<(String, usize), QuerySender>>;
/// 그리드 중앙 제어기 (Thin Dispatcher)
///
/// 네트워크 채널로부터 오는 메시지를 분류하여 도메인별 핸들러로 배달합니다.
pub struct GridManager {
quic_channel: Arc<QuicChannel>,
ec_store: Arc<DistributedErasureCodingStore>,
receiver: mpsc::Receiver<GridMessageWrapper>,
query_streams: QueryStreamMap,
stage_barriers: Arc<DashMap<(String, usize, std::net::SocketAddr), mpsc::Sender<()>>>,
/// 워커 측 로컬 실행 엔진 (워커 노드일 때 사용)
local_executor: Option<Arc<LocalExecutor>>,
/// 이 노드의 식별자
node_id: u32,
}
impl GridManager {
pub fn new(
quic_channel: Arc<QuicChannel>,
ec_store: Arc<DistributedErasureCodingStore>,
receiver: mpsc::Receiver<GridMessageWrapper>,
) -> Self {
Self::with_node_id(quic_channel, ec_store, receiver, 0)
}
pub fn with_node_id(
quic_channel: Arc<QuicChannel>,
ec_store: Arc<DistributedErasureCodingStore>,
receiver: mpsc::Receiver<GridMessageWrapper>,
node_id: u32,
) -> Self {
Self {
quic_channel,
ec_store,
receiver,
query_streams: Arc::new(DashMap::new()),
stage_barriers: Arc::new(DashMap::new()),
local_executor: None,
node_id,
}
}
/// 워커 노드 로컬 실행기 설정
pub fn with_local_executor(mut self, executor: Arc<LocalExecutor>) -> Self {
self.local_executor = Some(executor);
self
}
pub fn get_query_streams(&self) -> QueryStreamMap {
Arc::clone(&self.query_streams)
}
pub fn get_stage_barriers(
&self,
) -> Arc<DashMap<(String, usize, std::net::SocketAddr), mpsc::Sender<()>>> {
Arc::clone(&self.stage_barriers)
}
/// 수신 루프 시작
pub async fn run(mut self) {
info!(
"GridManager receiver loop started on {}",
self.quic_channel.local_addr
);
while let Some(wrapper) = self.receiver.recv().await {
let ec_store = Arc::clone(&self.ec_store);
let query_streams = Arc::clone(&self.query_streams);
let stage_barriers = Arc::clone(&self.stage_barriers);
let quic_channel = Arc::clone(&self.quic_channel);
let local_executor = self.local_executor.clone();
let node_id = self.node_id;
// 각 메시지를 비동기적으로 처리하여 병목 방지
tokio::spawn(async move {
if let Err(e) = Self::handle_message(
ec_store,
query_streams,
stage_barriers,
quic_channel,
local_executor,
node_id,
wrapper,
)
.await
{
error!("Error handling GridMessage: {:?}", e);
}
});
}
info!("GridManager receiver loop terminated");
}
/// 메시지 종류별 분기 처리
async fn handle_message(
ec_store: Arc<DistributedErasureCodingStore>,
query_streams: QueryStreamMap,
stage_barriers: Arc<DashMap<(String, usize, std::net::SocketAddr), mpsc::Sender<()>>>,
quic_channel: Arc<QuicChannel>,
local_executor: Option<Arc<LocalExecutor>>,
node_id: u32,
wrapper: GridMessageWrapper,
) -> DbxResult<()> {
let GridMessageWrapper { msg, mut stream } = wrapper;
match msg {
GridMessage::Storage(storage_msg) => {
Self::handle_storage_message(ec_store, storage_msg, &mut stream).await
}
GridMessage::Query(query_msg) => {
Self::handle_query_message(
query_streams,
stage_barriers,
quic_channel,
local_executor,
node_id,
query_msg,
)
.await
}
GridMessage::Lock(_) => {
warn!("LockMessage received but not implemented yet");
Ok(())
}
GridMessage::Replication(_) => {
warn!("ReplicationMessage received but not implemented yet");
Ok(())
}
}
}
/// 쿼리(스트리밍) 메시지 처리
async fn handle_query_message(
query_streams: QueryStreamMap,
stage_barriers: Arc<DashMap<(String, usize, std::net::SocketAddr), mpsc::Sender<()>>>,
quic_channel: Arc<QuicChannel>,
local_executor: Option<Arc<LocalExecutor>>,
node_id: u32,
msg: QueryMessage,
) -> DbxResult<()> {
match msg {
QueryMessage::ExecuteFragment {
execution_id,
stage_id,
plans_bytes,
coordinator_addr,
} => {
info!(
"ExecuteFragment received for ID: {}, Stage: {} from coordinator: {}",
execution_id, stage_id, coordinator_addr
);
let executor = match local_executor {
Some(e) => e,
None => {
warn!(
"ExecuteFragment received but no LocalExecutor configured — ignoring"
);
return Ok(());
}
};
// 코디네이터 주소 파싱
let coord_addr: std::net::SocketAddr = match coordinator_addr.parse() {
Ok(a) => a,
Err(e) => {
return Err(DbxError::Network(format!(
"Invalid coordinator addr: {}",
e
)));
}
};
let exec_id = execution_id.clone();
let query_streams = Arc::clone(&query_streams);
// 역직렬화
let mut plans: Vec<PhysicalPlan> = Vec::new();
for bytes in plans_bytes {
let plan = bincode::deserialize(&bytes)
.map_err(|e| DbxError::Serialization(e.to_string()))?;
plans.push(plan);
}
let quic_master = Arc::clone(&quic_channel);
tokio::spawn(async move {
info!(
"Worker spawning execution for exec_id: {}, stage_id: {}",
exec_id, stage_id
);
let mut join_set = tokio::task::JoinSet::new();
for worker_plan in plans {
let executor_ref = Arc::clone(&executor);
let quic_ref = Arc::clone(&quic_master);
let exec_id_ref = exec_id.clone();
let q_streams = Arc::clone(&query_streams);
join_set.spawn(async move {
// 1. CPU-bound 쿼리 실행을 spawn_blocking 기반으로 넘김
let (batches, channels) = match tokio::task::spawn_blocking(move || {
let mut chs = crate::sql::executor::local_executor::DistributedChannels::default();
let b = executor_ref.execute_collect_distributed(&worker_plan, &mut chs)?;
Ok::<(Vec<arrow::array::RecordBatch>, _), DbxError>((b, chs))
}).await {
Ok(Ok(res)) => res,
Ok(Err(e)) => {
error!("Worker execution error for {}: {:?}", exec_id_ref, e);
let eof_msg = GridMessage::Query(QueryMessage::ExchangeData {
execution_id: exec_id_ref.clone(),
exchange_id: 0,
node_id,
is_eof: true,
batch_data: vec![],
});
let _ = quic_ref.send_message(coord_addr, eof_msg).await;
return;
}
Err(e) => {
error!("Worker spawn_blocking panic: {:?}", e);
return;
}
};
// 2. 수동생성된 수신 채널(tx)들을 DashMap에 등록 (GridExchange 용도)
for (e_id, tx) in channels.exchanges {
q_streams.insert((exec_id_ref.clone(), e_id), tx);
}
// 3. ShuffleWriter 발신 채널(rx)들을 타겟별로 묶어 송신 태스크 스폰
let mut shuffle_join_set = tokio::task::JoinSet::new();
for (e_id, receivers) in channels.shuffles {
for (target_addr, mut rx) in receivers {
let quic_sub = Arc::clone(&quic_ref);
let exec_sub = exec_id_ref.clone();
shuffle_join_set.spawn(async move {
while let Some(Ok(Some(batch_bytes))) = rx.recv().await {
let msg = GridMessage::Query(QueryMessage::ExchangeData {
execution_id: exec_sub.clone(),
exchange_id: e_id,
node_id,
is_eof: false,
batch_data: batch_bytes,
});
let _ = quic_sub.send_message(target_addr, msg).await;
}
let eof_msg = GridMessage::Query(QueryMessage::ExchangeData {
execution_id: exec_sub,
exchange_id: e_id,
node_id,
is_eof: true,
batch_data: vec![],
});
let _ = quic_sub.send_message(target_addr, eof_msg).await;
});
}
}
// 4. (분산 Agg 등) 최상위 Return RecordBatch들을 스트리밍 송신
for batch in batches {
let ipc_bytes = match crate::grid::protocol::serialize_batch_to_ipc(&batch) {
Ok(b) => b,
Err(_) => continue,
};
let msg = GridMessage::Query(QueryMessage::ExchangeData {
execution_id: exec_id_ref.clone(),
exchange_id: 0,
node_id,
is_eof: false,
batch_data: ipc_bytes,
});
let _ = quic_ref.send_message(coord_addr, msg).await;
}
// EOF
let _ = quic_ref.send_message(coord_addr, GridMessage::Query(QueryMessage::ExchangeData {
execution_id: exec_id_ref.clone(),
exchange_id: 0,
node_id,
is_eof: true,
batch_data: vec![],
})).await;
// 셔플 백그라운드 송출 태스크들이 다 이빨이 맞을 때까지 대기
while shuffle_join_set.join_next().await.is_some() {}
});
}
// 모든 Plan 태스크들 처리가 끝날 때까지 대기
while join_set.join_next().await.is_some() {}
// 코디네이터에게 해당 Stage의 모든 수행이 종료되었음을 알림
let complete_msg = GridMessage::Query(QueryMessage::FragmentCompleted {
execution_id: exec_id.clone(),
stage_id,
});
let _ = quic_master.send_message(coord_addr, complete_msg).await;
info!(
"Worker completed all plans for exec_id: {}, stage_id: {}",
exec_id, stage_id
);
});
Ok(())
}
QueryMessage::FragmentCompleted {
execution_id,
stage_id,
} => {
let barriers = stage_barriers;
let matched_keys: Vec<_> = barriers
.iter()
.filter(|entry| entry.key().0 == execution_id && entry.key().1 == stage_id)
.map(|entry| entry.key().clone())
.collect();
for key in matched_keys {
if let Some(sender) = barriers.get_mut(&key) {
let _ = sender.try_send(());
}
}
Ok(())
}
QueryMessage::ExchangeData {
execution_id,
exchange_id,
node_id: _,
is_eof,
batch_data,
} => {
// 코디네이터가 큐를 통해 Operator로 데이터 밀어넣기
// DashMap lock을 await 전에 해제하기 위해 Sender를 복제합니다.
let sender_opt = query_streams
.get(&(execution_id.clone(), exchange_id))
.map(|kv| kv.value().clone());
if let Some(sender) = sender_opt {
if is_eof {
let _ = sender.send(Ok(None)).await;
} else {
let _ = sender.send(Ok(Some(batch_data))).await;
}
} else {
warn!("ExchangeData for unknown execution_id: {}", execution_id);
}
Ok(())
}
}
}
/// 스토리지(EC 샤드) 메시지 처리
async fn handle_storage_message(
ec_store: Arc<DistributedErasureCodingStore>,
msg: StorageMessage,
stream: &mut Option<s2n_quic::stream::BidirectionalStream>,
) -> DbxResult<()> {
match msg {
StorageMessage::StoreShard {
key,
shard_id,
data,
} => {
info!("Storing shard {}:{} locally", key, shard_id);
ec_store.local_store_shard(&key, shard_id, &data)?;
Ok(())
}
StorageMessage::FetchShard { key, shard_id } => {
info!("Fetching shard {}:{} for remote request", key, shard_id);
let shard_data = ec_store.local_fetch_shard(&key, shard_id)?;
// 응답 전송
if let Some(s) = stream {
let reply = GridMessage::Storage(StorageMessage::ShardResponse {
key: key.clone(),
shard_id,
data: shard_data,
});
::tracing::debug!(
"Sending ShardResponse for {}:{} on stream...",
key,
shard_id
);
if let Err(e) = QuicChannel::send_response(s, reply).await {
::tracing::error!(
"Failed to send ShardResponse for {}:{}: {:?}",
key,
shard_id,
e
);
} else {
::tracing::debug!(
"Successfully sent ShardResponse for {}:{}",
key,
shard_id
);
}
}
Ok(())
}
StorageMessage::ShardResponse { .. } => {
// ShardResponse는 보통 send_request_and_wait에서 직접 받으므로
// 메인 루프에 도달했다면 무시하거나 에러 처리
warn!("Received unexpected ShardResponse in main handler loop");
Ok(())
}
}
}
}