-
Notifications
You must be signed in to change notification settings - Fork 17
/
Copy pathblock_reader.rs
578 lines (513 loc) · 20 KB
/
block_reader.rs
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
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
use std::{
collections::{HashMap, VecDeque},
sync::Arc,
};
use bytes::{Buf, BufMut, Bytes, BytesMut};
use futures::{
stream::{self, BoxStream},
Stream, StreamExt,
};
use log::{debug, warn};
use tokio::{
sync::mpsc::{self, Receiver, Sender},
task::JoinHandle,
};
use crate::{
ec::EcSchema,
hdfs::connection::{DatanodeConnection, Op, DATANODE_CACHE},
proto::{
common,
hdfs::{
self, BlockOpResponseProto, LocatedBlockProto, PacketHeaderProto,
ReadOpChecksumInfoProto,
},
},
HdfsError, Result,
};
use super::protocol::NamenodeProtocol;
// The number of packets to queue up on reads
const READ_PACKET_BUFFER_LEN: usize = 100;
pub(crate) fn get_block_stream(
protocol: Arc<NamenodeProtocol>,
block: hdfs::LocatedBlockProto,
offset: usize,
len: usize,
ec_schema: Option<EcSchema>,
) -> BoxStream<'static, Result<Bytes>> {
if let Some(ec_schema) = ec_schema {
StripedBlockStream::new(protocol, block, offset, len, ec_schema)
.into_stream()
.boxed()
} else {
ReplicatedBlockStream::new(protocol, block, offset, len)
.into_stream()
.boxed()
}
}
/// Connects to a DataNode to do a read, attempting to used cached connections.
async fn connect_and_send(
protocol: &Arc<NamenodeProtocol>,
datanode_id: &hdfs::DatanodeIdProto,
block: &hdfs::ExtendedBlockProto,
token: common::TokenProto,
offset: u64,
len: u64,
) -> Result<(DatanodeConnection, BlockOpResponseProto)> {
let mut remaining_attempts = 2;
while remaining_attempts > 0 {
if let Some(mut conn) = DATANODE_CACHE.get(datanode_id) {
let message = hdfs::OpReadBlockProto {
header: conn.build_header(block, Some(token.clone())),
offset,
len,
send_checksums: Some(true),
..Default::default()
};
debug!("Block read op request {:?}", &message);
match conn.send(Op::ReadBlock, &message).await {
Ok(response) => {
debug!("Block read op response {:?}", response);
return Ok((conn, response));
}
Err(e) => {
warn!("Failed to use cached connection: {:?}", e);
}
}
} else {
break;
}
remaining_attempts -= 1;
}
let mut conn = DatanodeConnection::connect(
datanode_id,
&token,
protocol.get_cached_data_encryption_key().await?,
)
.await?;
let message = hdfs::OpReadBlockProto {
header: conn.build_header(block, Some(token)),
offset,
len,
send_checksums: Some(true),
..Default::default()
};
debug!("Block read op request {:?}", &message);
let response = conn.send(Op::ReadBlock, &message).await?;
debug!("Block read op response {:?}", response);
Ok((conn, response))
}
struct ReplicatedBlockStream {
protocol: Arc<NamenodeProtocol>,
block: hdfs::LocatedBlockProto,
offset: usize,
len: usize,
listener: Option<JoinHandle<Result<DatanodeConnection>>>,
sender: Sender<Result<(PacketHeaderProto, Bytes)>>,
receiver: Receiver<Result<(PacketHeaderProto, Bytes)>>,
current_replica: usize,
}
impl ReplicatedBlockStream {
fn new(
protocol: Arc<NamenodeProtocol>,
block: hdfs::LocatedBlockProto,
offset: usize,
len: usize,
) -> Self {
let (sender, receiver) = mpsc::channel(READ_PACKET_BUFFER_LEN);
Self {
protocol,
block,
offset,
len,
listener: None,
sender,
receiver,
current_replica: 0,
}
}
async fn select_next_datanode(
&mut self,
) -> Result<(DatanodeConnection, Option<ReadOpChecksumInfoProto>)> {
if self.current_replica >= self.block.locs.len() {
return Err(HdfsError::DataTransferError(
"All DataNodes failed".to_string(),
));
}
let datanode = &self.block.locs[self.current_replica].id;
self.current_replica += 1;
let (connection, response) = connect_and_send(
&self.protocol,
datanode,
&self.block.b,
self.block.block_token.clone(),
self.offset as u64,
self.len as u64,
)
.await?;
if response.status() != hdfs::Status::Success {
return Err(HdfsError::DataTransferError(response.message().to_string()));
}
Ok((connection, response.read_op_checksum_info))
}
async fn next_packet(&mut self) -> Result<Option<Bytes>> {
// We've finished this read, just return None
if self.len == 0 {
return Ok(None);
}
let (header, data) = loop {
if self.listener.is_none() {
let (connection, checksum_info) = self.select_next_datanode().await?;
self.listener = Some(Self::start_packet_listener(
connection,
checksum_info,
self.sender.clone(),
));
}
match self.receiver.recv().await {
Some(Ok(data)) => break data,
Some(Err(e)) => {
// Some error communicating with datanode, log a warning and then retry on a different Datanode
warn!("Error occured while reading from DataNode: {:?}", e);
self.listener = None;
}
None => {
// This means there's a disconnect between the data we are getting back and what we asked for,
// so just raise an error
return Err(HdfsError::DataTransferError(
"Not enough data returned from DataNode".to_string(),
));
}
}
};
let packet_offset = if self.offset > header.offset_in_block as usize {
self.offset - header.offset_in_block as usize
} else {
0
};
let packet_len = usize::min(header.data_len as usize - packet_offset, self.len);
self.offset += packet_len;
self.len -= packet_len;
// We've consumed the whole read, there should be no more packets and the listener should complete
if self.len == 0 {
let conn = self.listener.take().unwrap().await.unwrap()?;
DATANODE_CACHE.release(conn);
}
Ok(Some(
data.slice(packet_offset..(packet_offset + packet_len)),
))
}
async fn get_next_packet(
connection: &mut DatanodeConnection,
checksum_info: Option<ReadOpChecksumInfoProto>,
) -> Result<(PacketHeaderProto, Bytes)> {
let packet = connection.read_packet().await?;
let header = packet.header;
Ok((header, packet.get_data(&checksum_info)?))
}
fn start_packet_listener(
mut connection: DatanodeConnection,
checksum_info: Option<ReadOpChecksumInfoProto>,
sender: Sender<Result<(PacketHeaderProto, Bytes)>>,
) -> JoinHandle<Result<DatanodeConnection>> {
tokio::spawn(async move {
loop {
let next_packet = Self::get_next_packet(&mut connection, checksum_info).await;
if next_packet.as_ref().is_ok_and(|(_, data)| data.is_empty()) {
// If the packet is empty it means it's the last packet
// so tell the DataNode the read was a success and finish this task
connection.send_read_success().await?;
break;
}
if sender.send(next_packet).await.is_err() {
// The block reader was dropped, so just kill the listener
return Err(HdfsError::DataTransferError(
"Reader was dropped without consuming all data".to_string(),
));
}
}
Ok(connection)
})
}
fn into_stream(self) -> impl Stream<Item = Result<Bytes>> {
stream::unfold(self, |mut state| async move {
let next = state.next_packet().await.transpose();
next.map(|n| (n, state))
})
}
}
// Reads cells of data from a DataNode connection
struct CellReader {
cell_size: usize,
cell_buffer: BytesMut,
current_packet: Bytes,
block_stream: Option<ReplicatedBlockStream>,
}
impl CellReader {
fn new(cell_size: usize, block_stream: Option<ReplicatedBlockStream>) -> Self {
Self {
cell_size,
cell_buffer: BytesMut::with_capacity(cell_size),
current_packet: Bytes::new(),
block_stream,
}
}
async fn next_cell(&mut self) -> Result<Bytes> {
// We always should be reading a full cell, no current optimizations for a partial cell
// Only exception is the final cell may be partial
while self.cell_buffer.len() < self.cell_size {
if !self.current_packet.has_remaining() {
if let Some(block_stream) = self.block_stream.as_mut() {
match block_stream.next_packet().await? {
Some(next_packet) => self.current_packet = next_packet,
None => {
break;
}
}
} else {
// At the end of a block, just return all 0s
break;
}
}
let bytes_to_copy = usize::min(
self.cell_size - self.cell_buffer.len(),
self.current_packet.remaining(),
);
self.cell_buffer
.put(self.current_packet.split_to(bytes_to_copy));
}
// Pad a partial final cell with zeros
self.cell_buffer.resize(self.cell_size, 0);
Ok(std::mem::replace(
&mut self.cell_buffer,
BytesMut::with_capacity(self.cell_size),
)
.freeze())
}
}
/// Erasure coded data is stored in "cells" that are striped across Data Nodes.
/// An example of what 3-2-1024k cells would look like:
/// ----------------------------------------------
/// | blk_0 | blk_1 | blk_2 | blk_3 | blk_4 |
/// |--------|--------|--------|--------|--------|
/// | cell_0 | cell_1 | cell_2 | parity | parity |
/// | cell_3 | cell_4 | cell_5 | parity | parity |
/// ----------------------------------------------
///
/// Where cell_0 contains the first 1024k bytes, cell_1 contains the next 1024k bytes, and so on.
///
/// For an initial, simple implementation, determine the cells containing the start and end
/// of the range being requested, and request all "rows" or horizontal stripes of data containing
/// and between the start and end cell. So if the read range starts in cell_1 and ends in cell_4,
/// simply read all data blocks for cell_0 through cell_5.
///
/// We then convert these logical horizontal stripes into vertical stripes to read from each block/DataNode.
/// In this case, we will have one read for cell_0 and cell_3 from blk_0, one for cell_1 and cell_4 from blk_1,
/// and one for cell_2 and cell_5 from blk_2. We read each block a cell at a time, and return the horizontal
/// stripe of cells when we have all cells in a row. So first the first call to read_slice returns cells
/// cell_0, cell_1, and cell_2, and the second returns cell_3, cell_4, and cell_5. If any cell fails to read
/// we construct a read of a parity block for the remaining horizontal slices we still need to process. If
/// more reads fail than we have parity rows, the entire read fails.
///
/// In the future we can look at making this more efficient by not reading as many extra cells that aren't
/// part of the range being requested at all. Currently the overhead of not doing this would be up to
/// `data_units * cell_size * 2` of extra data being read from disk (basically two extra "rows" of data).
struct StripedBlockStream {
protocol: Arc<NamenodeProtocol>,
block: LocatedBlockProto,
block_map: HashMap<usize, (hdfs::DatanodeInfoProto, common::TokenProto)>,
remaining: usize,
bytes_to_skip: usize,
ec_schema: EcSchema,
// Position of the start of the current cell in a single block
current_block_start: usize,
// End location in a single block we need to read
block_end: usize,
cell_readers: Vec<Option<CellReader>>,
}
impl StripedBlockStream {
fn new(
protocol: Arc<NamenodeProtocol>,
block: hdfs::LocatedBlockProto,
offset: usize,
len: usize,
ec_schema: EcSchema,
) -> Self {
assert_eq!(block.block_indices().len(), block.locs.len());
// Cell IDs for the range we are reading, inclusive
let starting_cell = ec_schema.cell_for_offset(offset);
let ending_cell = ec_schema.cell_for_offset(offset + len - 1);
// Logical rows or horizontal stripes we need to read, tail-exclusive
let starting_row = ec_schema.row_for_cell(starting_cell);
let ending_row = ec_schema.row_for_cell(ending_cell) + 1;
let block_end = ec_schema.offset_for_row(ending_row);
let current_block_start = ec_schema.offset_for_row(starting_row);
let bytes_to_skip = offset - starting_row * ec_schema.data_units * ec_schema.cell_size;
let datanode_infos: Vec<(hdfs::DatanodeInfoProto, common::TokenProto)> = block
.locs
.iter()
.cloned()
.zip(block.block_tokens.iter().cloned())
.collect();
let block_map: HashMap<usize, (hdfs::DatanodeInfoProto, common::TokenProto)> = block
.block_indices()
.iter()
.copied()
.map(|i| i as usize)
.zip(datanode_infos)
.collect();
Self {
protocol,
block,
block_map,
remaining: len,
bytes_to_skip,
ec_schema,
current_block_start,
block_end,
cell_readers: vec![],
}
}
// Reads the next slice of cells and decodes if necessary
async fn read_slice(&mut self) -> Result<Option<VecDeque<Bytes>>> {
if self.remaining == 0 {
return Ok(None);
}
// Check if we need to start any new reads
let mut good_blocks = self.cell_readers.iter().filter(|r| r.is_some()).count();
while good_blocks < self.ec_schema.data_units {
if self.start_next_reader().await? {
good_blocks += 1;
}
}
let mut slice = vec![None; self.ec_schema.data_units + self.ec_schema.parity_units];
let mut good_cells = 0;
let mut block_index = 0;
while good_cells < self.ec_schema.data_units {
if block_index >= self.cell_readers.len() {
// Need to start reading from the next parity
if !self.start_next_reader().await? {
block_index += 1;
continue;
}
}
if let Some(reader) = self.cell_readers[block_index].as_mut() {
match reader.next_cell().await {
Ok(bytes) => {
slice[block_index] = Some(bytes);
good_cells += 1;
}
Err(e) => {
warn!(
"Error reading erasure coded block, trying next replica: {:?}",
e
);
self.cell_readers[block_index] = None;
}
}
}
block_index += 1;
}
let mut decoded = VecDeque::from(self.ec_schema.ec_decode(slice)?);
// Skip any bytes at the beginning
while self.bytes_to_skip > 0 {
if decoded[0].len() < self.bytes_to_skip {
self.bytes_to_skip -= decoded.pop_front().unwrap().len();
} else {
let _ = decoded[0].split_to(self.bytes_to_skip);
self.bytes_to_skip = 0;
}
}
let total_size: usize = decoded.iter().map(|bytes| bytes.len()).sum();
if total_size > self.remaining {
let mut bytes_to_trim = total_size - self.remaining;
while bytes_to_trim > 0 {
if decoded.back().unwrap().len() <= bytes_to_trim {
bytes_to_trim -= decoded.pop_back().unwrap().len();
} else {
let last_cell = decoded.back_mut().unwrap();
let _ = last_cell.split_off(last_cell.len() - bytes_to_trim);
bytes_to_trim = 0;
}
}
self.remaining = 0;
} else {
self.remaining -= total_size;
}
self.current_block_start += self.ec_schema.cell_size;
Ok(Some(decoded))
}
async fn start_next_reader(&mut self) -> Result<bool> {
if self.cell_readers.len() >= self.ec_schema.data_units + self.ec_schema.parity_units {
return Err(HdfsError::ErasureCodingError(
"Not enough valid shards".to_string(),
));
}
let index = self.cell_readers.len();
#[cfg(feature = "integration-test")]
if let Some(fault_injection) = crate::test::EC_FAULT_INJECTOR.lock().unwrap().as_ref() {
if fault_injection.fail_blocks.contains(&index) {
debug!("Failing block read for {}", index);
self.cell_readers.push(None);
return Ok(false);
}
}
let max_block_offset = self
.ec_schema
.max_offset(index, self.block.b.num_bytes() as usize);
let end = usize::min(self.block_end, max_block_offset);
let len = end - self.current_block_start;
// Three cases we need to worry about
// 1. The length to read is 0, which means we are reading the last slice and this index
// is past the end of the data. Just create a reader that returns all 0s, regardless
// if the block exists or not.
// 2. We have the DataNode info, so start reading from it
// 3. We don't have the DataNode info, so this shard isn't valid
if len == 0 {
self.cell_readers
.push(Some(CellReader::new(self.ec_schema.cell_size, None)));
Ok(true)
} else if let Some((datanode_info, token)) = self.block_map.get(&index) {
let mut block = self.block.clone();
// Each vertical stripe has a block ID of the original located block ID + block index
// That was fun to figure out
block.b.block_id += index as u64;
block.locs = vec![datanode_info.clone()];
block.block_token = token.clone();
let block_stream = ReplicatedBlockStream::new(
Arc::clone(&self.protocol),
block,
self.current_block_start,
len,
);
self.cell_readers.push(Some(CellReader::new(
self.ec_schema.cell_size,
Some(block_stream),
)));
Ok(true)
} else {
self.cell_readers.push(None);
Ok(false)
}
}
fn into_stream(self) -> impl Stream<Item = Result<Bytes>> {
stream::unfold(
(self, VecDeque::new()),
|(mut stream, mut buffers)| async move {
if buffers.is_empty() {
match stream.read_slice().await {
Ok(Some(next_buffers)) => {
buffers = next_buffers;
}
Ok(None) => {
return None;
}
Err(e) => {
return Some((Err(e), (stream, buffers)));
}
}
}
buffers.pop_front().map(|b| (Ok(b), (stream, buffers)))
},
)
}
}