Line data Source code
1 : //!
2 : //! Utilities for vectored reading of variable-sized "blobs".
3 : //!
4 : //! The "blob" api is an abstraction on top of the "block" api,
5 : //! with the main difference being that blobs do not have a fixed
6 : //! size (each blob is prefixed with 1 or 4 byte length field)
7 : //!
8 : //! The vectored apis provided in this module allow for planning
9 : //! and executing disk IO which covers multiple blobs.
10 : //!
11 : //! Reads are planned with [`VectoredReadPlanner`] which will coalesce
12 : //! adjacent blocks into a single disk IO request and exectuted by
13 : //! [`VectoredBlobReader`] which does all the required offset juggling
14 : //! and returns a buffer housing all the blobs and a list of offsets.
15 : //!
16 : //! Note that the vectored blob api does *not* go through the page cache.
17 :
18 : use std::collections::BTreeMap;
19 : use std::ops::Deref;
20 :
21 : use bytes::Bytes;
22 : use pageserver_api::key::Key;
23 : use tokio::io::AsyncWriteExt;
24 : use tokio_epoll_uring::BoundedBuf;
25 : use utils::lsn::Lsn;
26 : use utils::vec_map::VecMap;
27 :
28 : use crate::context::RequestContext;
29 : use crate::tenant::blob_io::{BYTE_UNCOMPRESSED, BYTE_ZSTD, Header};
30 : use crate::virtual_file::{self, IoBufferMut, VirtualFile};
31 :
32 : /// Metadata bundled with the start and end offset of a blob.
33 : #[derive(Copy, Clone, Debug)]
34 : pub struct BlobMeta {
35 : pub key: Key,
36 : pub lsn: Lsn,
37 : pub will_init: bool,
38 : }
39 :
40 : /// A view into the vectored blobs read buffer.
41 : #[derive(Clone, Debug)]
42 : pub(crate) enum BufView<'a> {
43 : Slice(&'a [u8]),
44 : Bytes(bytes::Bytes),
45 : }
46 :
47 : impl<'a> BufView<'a> {
48 : /// Creates a new slice-based view on the blob.
49 137288 : pub fn new_slice(slice: &'a [u8]) -> Self {
50 137288 : Self::Slice(slice)
51 137288 : }
52 :
53 : /// Creates a new [`bytes::Bytes`]-based view on the blob.
54 1 : pub fn new_bytes(bytes: bytes::Bytes) -> Self {
55 1 : Self::Bytes(bytes)
56 1 : }
57 :
58 : /// Convert the view into `Bytes`.
59 : ///
60 : /// If using slice as the underlying storage, the copy will be an O(n) operation.
61 863616 : pub fn into_bytes(self) -> Bytes {
62 863616 : match self {
63 863616 : BufView::Slice(slice) => Bytes::copy_from_slice(slice),
64 0 : BufView::Bytes(bytes) => bytes,
65 : }
66 863616 : }
67 :
68 : /// Creates a sub-view of the blob based on the range.
69 1945923 : fn view(&self, range: std::ops::Range<usize>) -> Self {
70 1945923 : match self {
71 1945923 : BufView::Slice(slice) => BufView::Slice(&slice[range]),
72 0 : BufView::Bytes(bytes) => BufView::Bytes(bytes.slice(range)),
73 : }
74 1945923 : }
75 : }
76 :
77 : impl Deref for BufView<'_> {
78 : type Target = [u8];
79 :
80 1084396 : fn deref(&self) -> &Self::Target {
81 1084396 : match self {
82 1084395 : BufView::Slice(slice) => slice,
83 1 : BufView::Bytes(bytes) => bytes,
84 : }
85 1084396 : }
86 : }
87 :
88 : impl AsRef<[u8]> for BufView<'_> {
89 0 : fn as_ref(&self) -> &[u8] {
90 0 : match self {
91 0 : BufView::Slice(slice) => slice,
92 0 : BufView::Bytes(bytes) => bytes.as_ref(),
93 : }
94 0 : }
95 : }
96 :
97 : impl<'a> From<&'a [u8]> for BufView<'a> {
98 0 : fn from(value: &'a [u8]) -> Self {
99 0 : Self::new_slice(value)
100 0 : }
101 : }
102 :
103 : impl From<Bytes> for BufView<'_> {
104 0 : fn from(value: Bytes) -> Self {
105 0 : Self::new_bytes(value)
106 0 : }
107 : }
108 :
109 : /// Blob offsets into [`VectoredBlobsBuf::buf`]. The byte ranges is potentially compressed,
110 : /// subject to [`VectoredBlob::compression_bits`].
111 : pub struct VectoredBlob {
112 : /// Blob metadata.
113 : pub meta: BlobMeta,
114 : /// Header start offset.
115 : header_start: usize,
116 : /// Data start offset.
117 : data_start: usize,
118 : /// End offset.
119 : end: usize,
120 : /// Compression used on the data, extracted from the header.
121 : compression_bits: u8,
122 : }
123 :
124 : impl VectoredBlob {
125 : /// Reads a decompressed view of the blob.
126 1935675 : pub(crate) async fn read<'a>(&self, buf: &BufView<'a>) -> Result<BufView<'a>, std::io::Error> {
127 1935675 : let view = buf.view(self.data_start..self.end);
128 :
129 1935675 : match self.compression_bits {
130 1935674 : BYTE_UNCOMPRESSED => Ok(view),
131 : BYTE_ZSTD => {
132 1 : let mut decompressed_vec = Vec::new();
133 1 : let mut decoder =
134 1 : async_compression::tokio::write::ZstdDecoder::new(&mut decompressed_vec);
135 1 : decoder.write_all(&view).await?;
136 1 : decoder.flush().await?;
137 : // Zero-copy conversion from `Vec` to `Bytes`
138 1 : Ok(BufView::new_bytes(Bytes::from(decompressed_vec)))
139 : }
140 0 : bits => {
141 0 : let error = std::io::Error::new(
142 0 : std::io::ErrorKind::InvalidData,
143 0 : format!(
144 0 : "Failed to decompress blob for {}@{}, {}..{}: invalid compression byte {bits:x}",
145 : self.meta.key, self.meta.lsn, self.data_start, self.end
146 : ),
147 : );
148 0 : Err(error)
149 : }
150 : }
151 1935675 : }
152 :
153 : /// Returns the raw blob including header.
154 10248 : pub(crate) fn raw_with_header<'a>(&self, buf: &BufView<'a>) -> BufView<'a> {
155 10248 : buf.view(self.header_start..self.end)
156 10248 : }
157 : }
158 :
159 : impl std::fmt::Display for VectoredBlob {
160 0 : fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
161 0 : write!(
162 0 : f,
163 0 : "{}@{}, {}..{}",
164 : self.meta.key, self.meta.lsn, self.data_start, self.end
165 : )
166 0 : }
167 : }
168 :
169 : /// Return type of [`VectoredBlobReader::read_blobs`]
170 : pub struct VectoredBlobsBuf {
171 : /// Buffer for all blobs in this read
172 : pub buf: IoBufferMut,
173 : /// Offsets into the buffer and metadata for all blobs in this read
174 : pub blobs: Vec<VectoredBlob>,
175 : }
176 :
177 : /// Description of one disk read for multiple blobs.
178 : /// Used as the argument form [`VectoredBlobReader::read_blobs`]
179 : #[derive(Debug)]
180 : pub struct VectoredRead {
181 : pub start: u64,
182 : pub end: u64,
183 : /// Start offset and metadata for each blob in this read
184 : pub blobs_at: VecMap<u64, BlobMeta>,
185 : }
186 :
187 : impl VectoredRead {
188 626039 : pub(crate) fn size(&self) -> usize {
189 626039 : (self.end - self.start) as usize
190 626039 : }
191 : }
192 :
193 : #[derive(Eq, PartialEq, Debug)]
194 : pub(crate) enum VectoredReadExtended {
195 : Yes,
196 : No,
197 : }
198 :
199 : /// A vectored read builder that tries to coalesce all reads that fits in a chunk.
200 : pub(crate) struct ChunkedVectoredReadBuilder {
201 : /// Start block number
202 : start_blk_no: usize,
203 : /// End block number (exclusive).
204 : end_blk_no: usize,
205 : /// Start offset and metadata for each blob in this read
206 : blobs_at: VecMap<u64, BlobMeta>,
207 : max_read_size: Option<usize>,
208 : }
209 :
210 : impl ChunkedVectoredReadBuilder {
211 : const CHUNK_SIZE: usize = virtual_file::get_io_buffer_alignment();
212 : /// Start building a new vectored read.
213 : ///
214 : /// Note that by design, this does not check against reading more than `max_read_size` to
215 : /// support reading larger blobs than the configuration value. The builder will be single use
216 : /// however after that.
217 137326 : fn new_impl(
218 137326 : start_offset: u64,
219 137326 : end_offset: u64,
220 137326 : meta: BlobMeta,
221 137326 : max_read_size: Option<usize>,
222 137326 : ) -> Self {
223 137326 : let mut blobs_at = VecMap::default();
224 137326 : blobs_at
225 137326 : .append(start_offset, meta)
226 137326 : .expect("First insertion always succeeds");
227 :
228 137326 : let start_blk_no = start_offset as usize / Self::CHUNK_SIZE;
229 137326 : let end_blk_no = (end_offset as usize).div_ceil(Self::CHUNK_SIZE);
230 137326 : Self {
231 137326 : start_blk_no,
232 137326 : end_blk_no,
233 137326 : blobs_at,
234 137326 : max_read_size,
235 137326 : }
236 137326 : }
237 :
238 117045 : pub(crate) fn new(
239 117045 : start_offset: u64,
240 117045 : end_offset: u64,
241 117045 : meta: BlobMeta,
242 117045 : max_read_size: usize,
243 117045 : ) -> Self {
244 117045 : Self::new_impl(start_offset, end_offset, meta, Some(max_read_size))
245 117045 : }
246 :
247 20281 : pub(crate) fn new_streaming(start_offset: u64, end_offset: u64, meta: BlobMeta) -> Self {
248 20281 : Self::new_impl(start_offset, end_offset, meta, None)
249 20281 : }
250 :
251 : /// Attempts to extend the current read with a new blob if the new blob resides in the same or the immediate next chunk.
252 : ///
253 : /// The resulting size also must be below the max read size.
254 1828523 : pub(crate) fn extend(&mut self, start: u64, end: u64, meta: BlobMeta) -> VectoredReadExtended {
255 1828523 : tracing::trace!(start, end, "trying to extend");
256 1828523 : let start_blk_no = start as usize / Self::CHUNK_SIZE;
257 1828523 : let end_blk_no = (end as usize).div_ceil(Self::CHUNK_SIZE);
258 :
259 1828523 : let not_limited_by_max_read_size = {
260 1828523 : if let Some(max_read_size) = self.max_read_size {
261 784782 : let coalesced_size = (end_blk_no - self.start_blk_no) * Self::CHUNK_SIZE;
262 784782 : coalesced_size <= max_read_size
263 : } else {
264 1043741 : true
265 : }
266 : };
267 :
268 : // True if the second block starts in the same block or the immediate next block where the first block ended.
269 : //
270 : // Note: This automatically handles the case where two blocks are adjacent to each other,
271 : // whether they starts on chunk size boundary or not.
272 1828523 : let is_adjacent_chunk_read = {
273 : // 1. first.end & second.start are in the same block
274 1828523 : self.end_blk_no == start_blk_no + 1 ||
275 : // 2. first.end ends one block before second.start
276 21630 : self.end_blk_no == start_blk_no
277 : };
278 :
279 1828523 : if is_adjacent_chunk_read && not_limited_by_max_read_size {
280 1806597 : self.end_blk_no = end_blk_no;
281 1806597 : self.blobs_at
282 1806597 : .append(start, meta)
283 1806597 : .expect("LSNs are ordered within vectored reads");
284 :
285 1806597 : return VectoredReadExtended::Yes;
286 21926 : }
287 :
288 21926 : VectoredReadExtended::No
289 1828523 : }
290 :
291 1063694 : pub(crate) fn size(&self) -> usize {
292 1063694 : (self.end_blk_no - self.start_blk_no) * Self::CHUNK_SIZE
293 1063694 : }
294 :
295 137326 : pub(crate) fn build(self) -> VectoredRead {
296 137326 : let start = (self.start_blk_no * Self::CHUNK_SIZE) as u64;
297 137326 : let end = (self.end_blk_no * Self::CHUNK_SIZE) as u64;
298 137326 : VectoredRead {
299 137326 : start,
300 137326 : end,
301 137326 : blobs_at: self.blobs_at,
302 137326 : }
303 137326 : }
304 : }
305 :
306 : #[derive(Copy, Clone, Debug)]
307 : pub enum BlobFlag {
308 : None,
309 : Ignore,
310 : ReplaceAll,
311 : }
312 :
313 : /// Planner for vectored blob reads.
314 : ///
315 : /// Blob offsets are received via [`VectoredReadPlanner::handle`]
316 : /// and coalesced into disk reads.
317 : ///
318 : /// The implementation is very simple:
319 : /// * Collect all blob offsets in an ordered structure
320 : /// * Iterate over the collected blobs and coalesce them into reads at the end
321 : pub struct VectoredReadPlanner {
322 : // Track all the blob offsets. Start offsets must be ordered.
323 : // Values in the value tuples are:
324 : // (
325 : // lsn of the blob,
326 : // start offset of the blob in the underlying file,
327 : // end offset of the blob in the underlying file,
328 : // whether the blob initializes the page image or not
329 : // see [`pageserver_api::record::NeonWalRecord::will_init`]
330 : // )
331 : blobs: BTreeMap<Key, Vec<(Lsn, u64, u64, bool)>>,
332 : // Arguments for previous blob passed into [`VectoredReadPlanner::handle`]
333 : prev: Option<(Key, Lsn, u64, BlobFlag)>,
334 :
335 : max_read_size: usize,
336 : }
337 :
338 : impl VectoredReadPlanner {
339 138959 : pub fn new(max_read_size: usize) -> Self {
340 138959 : Self {
341 138959 : blobs: BTreeMap::new(),
342 138959 : prev: None,
343 138959 : max_read_size,
344 138959 : }
345 138959 : }
346 :
347 : /// Include a new blob in the read plan.
348 : ///
349 : /// This function is called from a B-Tree index visitor (see `DeltaLayerInner::plan_reads`
350 : /// and `ImageLayerInner::plan_reads`). Said visitor wants to collect blob offsets for all
351 : /// keys in a given keyspace. This function must be called for each key in the desired
352 : /// keyspace (monotonically continuous). [`Self::handle_range_end`] must
353 : /// be called after every range in the offset.
354 : ///
355 : /// In the event that keys are skipped, the behaviour is undefined and can lead to an
356 : /// incorrect read plan. We can end up asserting, erroring in wal redo or returning
357 : /// incorrect data to the user.
358 : ///
359 : /// The `flag` argument has two interesting values:
360 : /// * [`BlobFlag::ReplaceAll`]: The blob for this key should replace all existing blobs.
361 : /// This is used for WAL records that `will_init`.
362 : /// * [`BlobFlag::Ignore`]: This blob should not be included in the read. This happens
363 : /// if the blob is cached.
364 1755334 : pub fn handle(&mut self, key: Key, lsn: Lsn, offset: u64, flag: BlobFlag) {
365 : // Implementation note: internally lag behind by one blob such that
366 : // we have a start and end offset when initialising [`VectoredRead`]
367 1755334 : let (prev_key, prev_lsn, prev_offset, prev_flag) = match self.prev {
368 : None => {
369 102901 : self.prev = Some((key, lsn, offset, flag));
370 102901 : return;
371 : }
372 1652433 : Some(prev) => prev,
373 : };
374 :
375 1652433 : self.add_blob(prev_key, prev_lsn, prev_offset, offset, prev_flag);
376 :
377 1652433 : self.prev = Some((key, lsn, offset, flag));
378 1755334 : }
379 :
380 148863 : pub fn handle_range_end(&mut self, offset: u64) {
381 148863 : if let Some((prev_key, prev_lsn, prev_offset, prev_flag)) = self.prev {
382 102901 : self.add_blob(prev_key, prev_lsn, prev_offset, offset, prev_flag);
383 102901 : }
384 :
385 148863 : self.prev = None;
386 148863 : }
387 :
388 1755334 : fn add_blob(&mut self, key: Key, lsn: Lsn, start_offset: u64, end_offset: u64, flag: BlobFlag) {
389 1755334 : match flag {
390 1127264 : BlobFlag::None => {
391 1127264 : let blobs_for_key = self.blobs.entry(key).or_default();
392 1127264 : blobs_for_key.push((lsn, start_offset, end_offset, false));
393 1127264 : }
394 184422 : BlobFlag::ReplaceAll => {
395 184422 : let blobs_for_key = self.blobs.entry(key).or_default();
396 184422 : blobs_for_key.clear();
397 184422 : blobs_for_key.push((lsn, start_offset, end_offset, true));
398 184422 : }
399 443648 : BlobFlag::Ignore => {}
400 : }
401 1755334 : }
402 :
403 138959 : pub fn finish(self) -> Vec<VectoredRead> {
404 138959 : let mut current_read_builder: Option<ChunkedVectoredReadBuilder> = None;
405 138959 : let mut reads = Vec::new();
406 :
407 317372 : for (key, blobs_for_key) in self.blobs {
408 1056242 : for (lsn, start_offset, end_offset, will_init) in blobs_for_key {
409 877829 : let extended = match &mut current_read_builder {
410 784771 : Some(read_builder) => read_builder.extend(
411 784771 : start_offset,
412 784771 : end_offset,
413 784771 : BlobMeta {
414 784771 : key,
415 784771 : lsn,
416 784771 : will_init,
417 784771 : },
418 : ),
419 93058 : None => VectoredReadExtended::No,
420 : };
421 :
422 877829 : if extended == VectoredReadExtended::No {
423 114981 : let next_read_builder = ChunkedVectoredReadBuilder::new(
424 114981 : start_offset,
425 114981 : end_offset,
426 114981 : BlobMeta {
427 114981 : key,
428 114981 : lsn,
429 114981 : will_init,
430 114981 : },
431 114981 : self.max_read_size,
432 : );
433 :
434 114981 : let prev_read_builder = current_read_builder.replace(next_read_builder);
435 :
436 : // `current_read_builder` is None in the first iteration of the outer loop
437 114981 : if let Some(read_builder) = prev_read_builder {
438 21923 : reads.push(read_builder.build());
439 93058 : }
440 762848 : }
441 : }
442 : }
443 :
444 138959 : if let Some(read_builder) = current_read_builder {
445 93058 : reads.push(read_builder.build());
446 93058 : }
447 :
448 138959 : reads
449 138959 : }
450 : }
451 :
452 : /// Disk reader for vectored blob spans (does not go through the page cache)
453 : pub struct VectoredBlobReader<'a> {
454 : file: &'a VirtualFile,
455 : }
456 :
457 : impl<'a> VectoredBlobReader<'a> {
458 125474 : pub fn new(file: &'a VirtualFile) -> Self {
459 125474 : Self { file }
460 125474 : }
461 :
462 : /// Read the requested blobs into the buffer.
463 : ///
464 : /// We have to deal with the fact that blobs are not fixed size.
465 : /// Each blob is prefixed by a size header.
466 : ///
467 : /// The success return value is a struct which contains the buffer
468 : /// filled from disk and a list of offsets at which each blob lies
469 : /// in the buffer.
470 137288 : pub async fn read_blobs(
471 137288 : &self,
472 137288 : read: &VectoredRead,
473 137288 : buf: IoBufferMut,
474 137288 : ctx: &RequestContext,
475 137288 : ) -> Result<VectoredBlobsBuf, std::io::Error> {
476 137288 : assert!(read.size() > 0);
477 137288 : assert!(
478 137288 : read.size() <= buf.capacity(),
479 0 : "{} > {}",
480 0 : read.size(),
481 0 : buf.capacity()
482 : );
483 :
484 137288 : if cfg!(debug_assertions) {
485 : const ALIGN: u64 = virtual_file::get_io_buffer_alignment() as u64;
486 137288 : debug_assert_eq!(
487 137288 : read.start % ALIGN,
488 : 0,
489 0 : "Read start at {} does not satisfy the required io buffer alignment ({} bytes)",
490 : read.start,
491 : ALIGN
492 : );
493 0 : }
494 :
495 137288 : let buf = self
496 137288 : .file
497 137288 : .read_exact_at(buf.slice(0..read.size()), read.start, ctx)
498 137288 : .await?
499 137288 : .into_inner();
500 :
501 137288 : let blobs_at = read.blobs_at.as_slice();
502 :
503 137288 : let mut blobs = Vec::with_capacity(blobs_at.len());
504 : // Blobs in `read` only provide their starting offset. The end offset
505 : // of a blob is implicit: the start of the next blob if one exists
506 : // or the end of the read.
507 :
508 1943867 : for (blob_start, meta) in blobs_at.iter().copied() {
509 1943867 : let header_start = (blob_start - read.start) as usize;
510 1943867 : let header = Header::decode(&buf[header_start..]).map_err(|anyhow_err| {
511 0 : std::io::Error::new(std::io::ErrorKind::InvalidData, anyhow_err)
512 0 : })?;
513 1943867 : let data_start = header_start + header.header_len;
514 1943867 : let end = data_start + header.data_len;
515 1943867 : let compression_bits = header.compression_bits;
516 :
517 1943867 : blobs.push(VectoredBlob {
518 1943867 : header_start,
519 1943867 : data_start,
520 1943867 : end,
521 1943867 : meta,
522 1943867 : compression_bits,
523 1943867 : });
524 : }
525 :
526 137288 : Ok(VectoredBlobsBuf { buf, blobs })
527 137288 : }
528 : }
529 :
530 : /// Read planner used in [`crate::tenant::storage_layer::image_layer::ImageLayerIterator`].
531 : ///
532 : /// It provides a streaming API for getting read blobs. It returns a batch when
533 : /// `handle` gets called and when the current key would just exceed the read_size and
534 : /// max_cnt constraints.
535 : pub struct StreamingVectoredReadPlanner {
536 : read_builder: Option<ChunkedVectoredReadBuilder>,
537 : // Arguments for previous blob passed into [`StreamingVectoredReadPlanner::handle`]
538 : prev: Option<(Key, Lsn, u64, bool)>,
539 : /// Max read size per batch. This is not a strict limit. If there are [0, 100) and [100, 200), while the `max_read_size` is 150,
540 : /// we will produce a single batch instead of split them.
541 : max_read_size: u64,
542 : /// Max item count per batch
543 : max_cnt: usize,
544 : /// Size of the current batch
545 : cnt: usize,
546 : }
547 :
548 : impl StreamingVectoredReadPlanner {
549 357 : pub fn new(max_read_size: u64, max_cnt: usize) -> Self {
550 357 : assert!(max_cnt > 0);
551 357 : assert!(max_read_size > 0);
552 357 : Self {
553 357 : read_builder: None,
554 357 : prev: None,
555 357 : max_cnt,
556 357 : max_read_size,
557 357 : cnt: 0,
558 357 : }
559 357 : }
560 :
561 1064050 : pub fn handle(
562 1064050 : &mut self,
563 1064050 : key: Key,
564 1064050 : lsn: Lsn,
565 1064050 : offset: u64,
566 1064050 : will_init: bool,
567 1064050 : ) -> Option<VectoredRead> {
568 : // Implementation note: internally lag behind by one blob such that
569 : // we have a start and end offset when initialising [`VectoredRead`]
570 1064050 : let (prev_key, prev_lsn, prev_offset, prev_will_init) = match self.prev {
571 : None => {
572 356 : self.prev = Some((key, lsn, offset, will_init));
573 356 : return None;
574 : }
575 1063694 : Some(prev) => prev,
576 : };
577 :
578 1063694 : let res = self.add_blob(
579 1063694 : prev_key,
580 1063694 : prev_lsn,
581 1063694 : prev_offset,
582 1063694 : offset,
583 : false,
584 1063694 : prev_will_init,
585 : );
586 :
587 1063694 : self.prev = Some((key, lsn, offset, will_init));
588 :
589 1063694 : res
590 1064050 : }
591 :
592 329 : pub fn handle_range_end(&mut self, offset: u64) -> Option<VectoredRead> {
593 329 : let res = if let Some((prev_key, prev_lsn, prev_offset, prev_will_init)) = self.prev {
594 328 : self.add_blob(
595 328 : prev_key,
596 328 : prev_lsn,
597 328 : prev_offset,
598 328 : offset,
599 : true,
600 328 : prev_will_init,
601 : )
602 : } else {
603 1 : None
604 : };
605 :
606 329 : self.prev = None;
607 :
608 329 : res
609 329 : }
610 :
611 1064022 : fn add_blob(
612 1064022 : &mut self,
613 1064022 : key: Key,
614 1064022 : lsn: Lsn,
615 1064022 : start_offset: u64,
616 1064022 : end_offset: u64,
617 1064022 : is_last_blob_in_read: bool,
618 1064022 : will_init: bool,
619 1064022 : ) -> Option<VectoredRead> {
620 1064022 : match &mut self.read_builder {
621 1043741 : Some(read_builder) => {
622 1043741 : let extended = read_builder.extend(
623 1043741 : start_offset,
624 1043741 : end_offset,
625 1043741 : BlobMeta {
626 1043741 : key,
627 1043741 : lsn,
628 1043741 : will_init,
629 1043741 : },
630 : );
631 1043741 : assert_eq!(extended, VectoredReadExtended::Yes);
632 : }
633 20281 : None => {
634 20281 : self.read_builder = {
635 20281 : Some(ChunkedVectoredReadBuilder::new_streaming(
636 20281 : start_offset,
637 20281 : end_offset,
638 20281 : BlobMeta {
639 20281 : key,
640 20281 : lsn,
641 20281 : will_init,
642 20281 : },
643 20281 : ))
644 20281 : };
645 20281 : }
646 : }
647 1064022 : let read_builder = self.read_builder.as_mut().unwrap();
648 1064022 : self.cnt += 1;
649 1064022 : if is_last_blob_in_read
650 1063694 : || read_builder.size() >= self.max_read_size as usize
651 1049272 : || self.cnt >= self.max_cnt
652 : {
653 20281 : let prev_read_builder = self.read_builder.take();
654 20281 : self.cnt = 0;
655 :
656 : // `current_read_builder` is None in the first iteration
657 20281 : if let Some(read_builder) = prev_read_builder {
658 20281 : return Some(read_builder.build());
659 0 : }
660 1043741 : }
661 1043741 : None
662 1064022 : }
663 : }
664 :
665 : #[cfg(test)]
666 : mod tests {
667 :
668 : use super::super::blob_io::tests::{random_array, write_maybe_compressed};
669 : use super::*;
670 : use crate::context::DownloadBehavior;
671 : use crate::page_cache::PAGE_SZ;
672 : use crate::task_mgr::TaskKind;
673 :
674 24 : fn validate_read(read: &VectoredRead, offset_range: &[(Key, Lsn, u64, BlobFlag)]) {
675 : const ALIGN: u64 = virtual_file::get_io_buffer_alignment() as u64;
676 24 : assert_eq!(read.start % ALIGN, 0);
677 24 : assert_eq!(read.start / ALIGN, offset_range.first().unwrap().2 / ALIGN);
678 :
679 24 : let expected_offsets_in_read: Vec<_> = offset_range.iter().map(|o| o.2).collect();
680 :
681 24 : let offsets_in_read: Vec<_> = read
682 24 : .blobs_at
683 24 : .as_slice()
684 24 : .iter()
685 24 : .map(|(offset, _)| *offset)
686 24 : .collect();
687 :
688 24 : assert_eq!(expected_offsets_in_read, offsets_in_read);
689 24 : }
690 :
691 : #[test]
692 1 : fn planner_chunked_coalesce_all_test() {
693 : use crate::virtual_file;
694 :
695 : const CHUNK_SIZE: u64 = virtual_file::get_io_buffer_alignment() as u64;
696 :
697 1 : let max_read_size = CHUNK_SIZE as usize * 8;
698 1 : let key = Key::MIN;
699 1 : let lsn = Lsn(0);
700 :
701 1 : let blob_descriptions = [
702 1 : (key, lsn, CHUNK_SIZE / 8, BlobFlag::None), // Read 1 BEGIN
703 1 : (key, lsn, CHUNK_SIZE / 4, BlobFlag::Ignore), // Gap
704 1 : (key, lsn, CHUNK_SIZE / 2, BlobFlag::None),
705 1 : (key, lsn, CHUNK_SIZE - 2, BlobFlag::Ignore), // Gap
706 1 : (key, lsn, CHUNK_SIZE, BlobFlag::None),
707 1 : (key, lsn, CHUNK_SIZE * 2 - 1, BlobFlag::None),
708 1 : (key, lsn, CHUNK_SIZE * 2 + 1, BlobFlag::Ignore), // Gap
709 1 : (key, lsn, CHUNK_SIZE * 3 + 1, BlobFlag::None),
710 1 : (key, lsn, CHUNK_SIZE * 5 + 1, BlobFlag::None),
711 1 : (key, lsn, CHUNK_SIZE * 6 + 1, BlobFlag::Ignore), // skipped chunk size, but not a chunk: should coalesce.
712 1 : (key, lsn, CHUNK_SIZE * 7 + 1, BlobFlag::None),
713 1 : (key, lsn, CHUNK_SIZE * 8, BlobFlag::None), // Read 2 BEGIN (b/c max_read_size)
714 1 : (key, lsn, CHUNK_SIZE * 9, BlobFlag::Ignore), // ==== skipped a chunk
715 1 : (key, lsn, CHUNK_SIZE * 10, BlobFlag::None), // Read 3 BEGIN (cannot coalesce)
716 1 : ];
717 :
718 1 : let ranges = [
719 1 : &[
720 1 : blob_descriptions[0],
721 1 : blob_descriptions[2],
722 1 : blob_descriptions[4],
723 1 : blob_descriptions[5],
724 1 : blob_descriptions[7],
725 1 : blob_descriptions[8],
726 1 : blob_descriptions[10],
727 1 : ],
728 1 : &blob_descriptions[11..12],
729 1 : &blob_descriptions[13..],
730 1 : ];
731 :
732 1 : let mut planner = VectoredReadPlanner::new(max_read_size);
733 15 : for (key, lsn, offset, flag) in blob_descriptions {
734 14 : planner.handle(key, lsn, offset, flag);
735 14 : }
736 :
737 1 : planner.handle_range_end(652 * 1024);
738 :
739 1 : let reads = planner.finish();
740 :
741 1 : assert_eq!(reads.len(), ranges.len());
742 :
743 3 : for (idx, read) in reads.iter().enumerate() {
744 3 : validate_read(read, ranges[idx]);
745 3 : }
746 1 : }
747 :
748 : #[test]
749 1 : fn planner_max_read_size_test() {
750 1 : let max_read_size = 128 * 1024;
751 1 : let key = Key::MIN;
752 1 : let lsn = Lsn(0);
753 :
754 1 : let blob_descriptions = vec![
755 1 : (key, lsn, 0, BlobFlag::None),
756 1 : (key, lsn, 32 * 1024, BlobFlag::None),
757 1 : (key, lsn, 96 * 1024, BlobFlag::None), // Last in read 1
758 1 : (key, lsn, 128 * 1024, BlobFlag::None), // Last in read 2
759 1 : (key, lsn, 198 * 1024, BlobFlag::None), // Last in read 3
760 1 : (key, lsn, 268 * 1024, BlobFlag::None), // Last in read 4
761 1 : (key, lsn, 396 * 1024, BlobFlag::None), // Last in read 5
762 1 : (key, lsn, 652 * 1024, BlobFlag::None), // Last in read 6
763 : ];
764 :
765 1 : let ranges = [
766 1 : &blob_descriptions[0..3],
767 1 : &blob_descriptions[3..4],
768 1 : &blob_descriptions[4..5],
769 1 : &blob_descriptions[5..6],
770 1 : &blob_descriptions[6..7],
771 1 : &blob_descriptions[7..],
772 1 : ];
773 :
774 1 : let mut planner = VectoredReadPlanner::new(max_read_size);
775 8 : for (key, lsn, offset, flag) in blob_descriptions.clone() {
776 8 : planner.handle(key, lsn, offset, flag);
777 8 : }
778 :
779 1 : planner.handle_range_end(652 * 1024);
780 :
781 1 : let reads = planner.finish();
782 :
783 1 : assert_eq!(reads.len(), 6);
784 :
785 : // TODO: could remove zero reads to produce 5 reads here
786 :
787 6 : for (idx, read) in reads.iter().enumerate() {
788 6 : validate_read(read, ranges[idx]);
789 6 : }
790 1 : }
791 :
792 : #[test]
793 1 : fn planner_replacement_test() {
794 : const CHUNK_SIZE: u64 = virtual_file::get_io_buffer_alignment() as u64;
795 1 : let max_read_size = 128 * CHUNK_SIZE as usize;
796 1 : let first_key = Key::MIN;
797 1 : let second_key = first_key.next();
798 1 : let lsn = Lsn(0);
799 :
800 1 : let blob_descriptions = vec![
801 1 : (first_key, lsn, 0, BlobFlag::None), // First in read 1
802 1 : (first_key, lsn, CHUNK_SIZE, BlobFlag::None), // Last in read 1
803 1 : (second_key, lsn, 2 * CHUNK_SIZE, BlobFlag::ReplaceAll),
804 1 : (second_key, lsn, 3 * CHUNK_SIZE, BlobFlag::None),
805 1 : (second_key, lsn, 4 * CHUNK_SIZE, BlobFlag::ReplaceAll), // First in read 2
806 1 : (second_key, lsn, 5 * CHUNK_SIZE, BlobFlag::None), // Last in read 2
807 : ];
808 :
809 1 : let ranges = [&blob_descriptions[0..2], &blob_descriptions[4..]];
810 :
811 1 : let mut planner = VectoredReadPlanner::new(max_read_size);
812 6 : for (key, lsn, offset, flag) in blob_descriptions.clone() {
813 6 : planner.handle(key, lsn, offset, flag);
814 6 : }
815 :
816 1 : planner.handle_range_end(6 * CHUNK_SIZE);
817 :
818 1 : let reads = planner.finish();
819 1 : assert_eq!(reads.len(), 2);
820 :
821 2 : for (idx, read) in reads.iter().enumerate() {
822 2 : validate_read(read, ranges[idx]);
823 2 : }
824 1 : }
825 :
826 : #[test]
827 1 : fn streaming_planner_max_read_size_test() {
828 1 : let max_read_size = 128 * 1024;
829 1 : let key = Key::MIN;
830 1 : let lsn = Lsn(0);
831 :
832 1 : let blob_descriptions = vec![
833 1 : (key, lsn, 0, BlobFlag::None),
834 1 : (key, lsn, 32 * 1024, BlobFlag::None),
835 1 : (key, lsn, 96 * 1024, BlobFlag::None),
836 1 : (key, lsn, 128 * 1024, BlobFlag::None),
837 1 : (key, lsn, 198 * 1024, BlobFlag::None),
838 1 : (key, lsn, 268 * 1024, BlobFlag::None),
839 1 : (key, lsn, 396 * 1024, BlobFlag::None),
840 1 : (key, lsn, 652 * 1024, BlobFlag::None),
841 : ];
842 :
843 1 : let ranges = [
844 1 : &blob_descriptions[0..3],
845 1 : &blob_descriptions[3..5],
846 1 : &blob_descriptions[5..6],
847 1 : &blob_descriptions[6..7],
848 1 : &blob_descriptions[7..],
849 1 : ];
850 :
851 1 : let mut planner = StreamingVectoredReadPlanner::new(max_read_size, 1000);
852 1 : let mut reads = Vec::new();
853 8 : for (key, lsn, offset, _) in blob_descriptions.clone() {
854 8 : reads.extend(planner.handle(key, lsn, offset, false));
855 8 : }
856 1 : reads.extend(planner.handle_range_end(652 * 1024));
857 :
858 1 : assert_eq!(reads.len(), ranges.len());
859 :
860 5 : for (idx, read) in reads.iter().enumerate() {
861 5 : validate_read(read, ranges[idx]);
862 5 : }
863 1 : }
864 :
865 : #[test]
866 1 : fn streaming_planner_max_cnt_test() {
867 1 : let max_read_size = 1024 * 1024;
868 1 : let key = Key::MIN;
869 1 : let lsn = Lsn(0);
870 :
871 1 : let blob_descriptions = vec![
872 1 : (key, lsn, 0, BlobFlag::None),
873 1 : (key, lsn, 32 * 1024, BlobFlag::None),
874 1 : (key, lsn, 96 * 1024, BlobFlag::None),
875 1 : (key, lsn, 128 * 1024, BlobFlag::None),
876 1 : (key, lsn, 198 * 1024, BlobFlag::None),
877 1 : (key, lsn, 268 * 1024, BlobFlag::None),
878 1 : (key, lsn, 396 * 1024, BlobFlag::None),
879 1 : (key, lsn, 652 * 1024, BlobFlag::None),
880 : ];
881 :
882 1 : let ranges = [
883 1 : &blob_descriptions[0..2],
884 1 : &blob_descriptions[2..4],
885 1 : &blob_descriptions[4..6],
886 1 : &blob_descriptions[6..],
887 1 : ];
888 :
889 1 : let mut planner = StreamingVectoredReadPlanner::new(max_read_size, 2);
890 1 : let mut reads = Vec::new();
891 8 : for (key, lsn, offset, _) in blob_descriptions.clone() {
892 8 : reads.extend(planner.handle(key, lsn, offset, false));
893 8 : }
894 1 : reads.extend(planner.handle_range_end(652 * 1024));
895 :
896 1 : assert_eq!(reads.len(), ranges.len());
897 :
898 4 : for (idx, read) in reads.iter().enumerate() {
899 4 : validate_read(read, ranges[idx]);
900 4 : }
901 1 : }
902 :
903 : #[test]
904 1 : fn streaming_planner_edge_test() {
905 1 : let max_read_size = 1024 * 1024;
906 1 : let key = Key::MIN;
907 1 : let lsn = Lsn(0);
908 : {
909 1 : let mut planner = StreamingVectoredReadPlanner::new(max_read_size, 1);
910 1 : let mut reads = Vec::new();
911 1 : reads.extend(planner.handle_range_end(652 * 1024));
912 1 : assert!(reads.is_empty());
913 : }
914 : {
915 1 : let mut planner = StreamingVectoredReadPlanner::new(max_read_size, 1);
916 1 : let mut reads = Vec::new();
917 1 : reads.extend(planner.handle(key, lsn, 0, false));
918 1 : reads.extend(planner.handle_range_end(652 * 1024));
919 1 : assert_eq!(reads.len(), 1);
920 1 : validate_read(&reads[0], &[(key, lsn, 0, BlobFlag::None)]);
921 : }
922 : {
923 1 : let mut planner = StreamingVectoredReadPlanner::new(max_read_size, 1);
924 1 : let mut reads = Vec::new();
925 1 : reads.extend(planner.handle(key, lsn, 0, false));
926 1 : reads.extend(planner.handle(key, lsn, 128 * 1024, false));
927 1 : reads.extend(planner.handle_range_end(652 * 1024));
928 1 : assert_eq!(reads.len(), 2);
929 1 : validate_read(&reads[0], &[(key, lsn, 0, BlobFlag::None)]);
930 1 : validate_read(&reads[1], &[(key, lsn, 128 * 1024, BlobFlag::None)]);
931 : }
932 : {
933 1 : let mut planner = StreamingVectoredReadPlanner::new(max_read_size, 2);
934 1 : let mut reads = Vec::new();
935 1 : reads.extend(planner.handle(key, lsn, 0, false));
936 1 : reads.extend(planner.handle(key, lsn, 128 * 1024, false));
937 1 : reads.extend(planner.handle_range_end(652 * 1024));
938 1 : assert_eq!(reads.len(), 1);
939 1 : validate_read(
940 1 : &reads[0],
941 1 : &[
942 1 : (key, lsn, 0, BlobFlag::None),
943 1 : (key, lsn, 128 * 1024, BlobFlag::None),
944 1 : ],
945 : );
946 : }
947 1 : }
948 :
949 4 : async fn round_trip_test_compressed(
950 4 : blobs: &[Vec<u8>],
951 4 : compression: bool,
952 4 : ) -> anyhow::Result<()> {
953 4 : let ctx =
954 4 : RequestContext::new(TaskKind::UnitTest, DownloadBehavior::Error).with_scope_unit_test();
955 4 : let (_temp_dir, pathbuf, offsets) =
956 4 : write_maybe_compressed(blobs, compression, &ctx).await?;
957 :
958 4 : let file = VirtualFile::open_v2(&pathbuf, &ctx).await?;
959 4 : let file_len = std::fs::metadata(&pathbuf)?.len();
960 :
961 : // Multiply by two (compressed data might need more space), and add a few bytes for the header
962 2060 : let reserved_bytes = blobs.iter().map(|bl| bl.len()).max().unwrap() * 2 + 16;
963 4 : let mut buf = IoBufferMut::with_capacity(reserved_bytes);
964 :
965 4 : let vectored_blob_reader = VectoredBlobReader::new(&file);
966 4 : let meta = BlobMeta {
967 4 : key: Key::MIN,
968 4 : lsn: Lsn(0),
969 4 : will_init: false,
970 4 : };
971 :
972 2060 : for (idx, (blob, offset)) in blobs.iter().zip(offsets.iter()).enumerate() {
973 2060 : let end = offsets.get(idx + 1).unwrap_or(&file_len);
974 2060 : if idx + 1 == offsets.len() {
975 4 : continue;
976 2056 : }
977 2056 : let read_builder = ChunkedVectoredReadBuilder::new(*offset, *end, meta, 16 * 4096);
978 2056 : let read = read_builder.build();
979 2056 : let result = vectored_blob_reader.read_blobs(&read, buf, &ctx).await?;
980 2056 : assert_eq!(result.blobs.len(), 1);
981 2056 : let read_blob = &result.blobs[0];
982 2056 : let view = BufView::new_slice(&result.buf);
983 2056 : let read_buf = read_blob.read(&view).await?;
984 2056 : assert_eq!(
985 2056 : &blob[..],
986 2056 : &read_buf[..],
987 0 : "mismatch for idx={idx} at offset={offset}"
988 : );
989 :
990 : // Check that raw_with_header returns a valid header.
991 2056 : let raw = read_blob.raw_with_header(&view);
992 2056 : let header = Header::decode(&raw)?;
993 2056 : if !compression || header.header_len == 1 {
994 1038 : assert_eq!(header.compression_bits, BYTE_UNCOMPRESSED);
995 1018 : }
996 2056 : assert_eq!(raw.len(), header.total_len());
997 :
998 2056 : buf = result.buf;
999 : }
1000 4 : Ok(())
1001 4 : }
1002 :
1003 : #[tokio::test]
1004 1 : async fn test_really_big_array() -> anyhow::Result<()> {
1005 1 : let blobs = &[
1006 1 : b"test".to_vec(),
1007 1 : random_array(10 * PAGE_SZ),
1008 1 : b"hello".to_vec(),
1009 1 : random_array(66 * PAGE_SZ),
1010 1 : vec![0xf3; 24 * PAGE_SZ],
1011 1 : b"foobar".to_vec(),
1012 1 : ];
1013 1 : round_trip_test_compressed(blobs, false).await?;
1014 1 : round_trip_test_compressed(blobs, true).await?;
1015 2 : Ok(())
1016 1 : }
1017 :
1018 : #[tokio::test]
1019 1 : async fn test_arrays_inc() -> anyhow::Result<()> {
1020 1 : let blobs = (0..PAGE_SZ / 8)
1021 1024 : .map(|v| random_array(v * 16))
1022 1 : .collect::<Vec<_>>();
1023 1 : round_trip_test_compressed(&blobs, false).await?;
1024 1 : round_trip_test_compressed(&blobs, true).await?;
1025 2 : Ok(())
1026 1 : }
1027 : }
|