quickwit-storage 0.3.0

Quickwit's storage abstraction
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
// Copyright (C) 2021 Quickwit, Inc.
//
// Quickwit is offered under the AGPL v3.0 and as commercial software.
// For commercial licensing, contact us at hello@quickwit.io.
//
// AGPL:
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as
// published by the Free Software Foundation, either version 3 of the
// License, or (at your option) any later version.
//
// This program is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.

use std::collections::HashMap;
use std::fmt::Debug;
use std::io::{self, ErrorKind, SeekFrom};
use std::ops::Range;
use std::path::{Path, PathBuf};

use async_trait::async_trait;
use futures::{stream, StreamExt};
use rusoto_core::ByteStream;
use tokio::io::{AsyncReadExt, AsyncSeekExt};
use tokio_util::io::ReaderStream;

use crate::{BundleStorageFileOffsets, PutPayload};

/// Payload of a split which builds the split bundle and hotcache on the fly and streams it to the
/// storage.
#[derive(Clone)]
pub struct SplitPayload {
    payloads: Vec<Box<dyn PutPayload>>,
    /// bytes range of the footer (hotcache + bundle metadata)
    pub footer_range: Range<u64>,
}

async fn range_byte_stream_from_payloads(
    payloads: &[Box<dyn PutPayload>],
    range: Range<u64>,
) -> io::Result<ByteStream> {
    let mut bytestreams: Vec<_> = Vec::new();

    let payloads_and_ranges =
        chunk_payload_ranges(payloads, range.start as usize..range.end as usize);

    for (payload, range) in payloads_and_ranges {
        bytestreams.push(
            payload
                .range_byte_stream(range.start as u64..range.end as u64)
                .await?,
        );
    }

    let concat_stream = ByteStream::new(stream::iter(bytestreams).flatten());
    Ok(concat_stream)
}

#[async_trait]
impl PutPayload for SplitPayload {
    fn len(&self) -> u64 {
        self.payloads.iter().map(|payload| payload.len()).sum()
    }

    async fn range_byte_stream(&self, range: Range<u64>) -> io::Result<ByteStream> {
        range_byte_stream_from_payloads(&self.payloads, range).await
    }
}

#[derive(Clone)]
struct FilePayload {
    len: u64,
    path: PathBuf,
}

#[async_trait]
impl PutPayload for FilePayload {
    fn len(&self) -> u64 {
        self.len
    }

    async fn range_byte_stream(&self, range: Range<u64>) -> io::Result<ByteStream> {
        assert!(!range.is_empty());
        assert!(range.end <= self.len);
        let mut file = tokio::fs::File::open(&self.path).await?;
        if range.start > 0 {
            file.seek(SeekFrom::Start(range.start)).await?;
        }
        if range.end == self.len {
            return Ok(ByteStream::new(ReaderStream::new(file)));
        }
        Ok(ByteStream::new(ReaderStream::new(
            file.take(range.end - range.start),
        )))
    }
}

/// SplitPayloadBuilder is used to create a `SplitPayload`.
#[derive(Default, Debug)]
pub struct SplitPayloadBuilder {
    metadata: BundleStorageFileOffsets,
    current_offset: usize,
}

impl SplitPayloadBuilder {
    /// Creates a new SplitPayloadBuilder for given files and hotcache.
    pub fn get_split_payload(split_files: &[PathBuf], hotcache: &[u8]) -> io::Result<SplitPayload> {
        let mut split_payload_builder = SplitPayloadBuilder::default();
        for file in split_files {
            split_payload_builder.add_file(file)?;
        }
        let offsets = split_payload_builder.finalize(hotcache)?;
        Ok(offsets)
    }

    /// Adds the file to the bundle file.
    ///
    /// The hotcache needs to be the last file that is added, in order to be able to read
    /// the hotcache and the metadata in one continous read.
    pub fn add_file(&mut self, path: &Path) -> io::Result<()> {
        let file = std::fs::metadata(path)?;
        let file_range = self.current_offset as u64..self.current_offset as u64 + file.len() as u64;
        self.current_offset += file.len() as usize;
        self.metadata.files.insert(path.to_owned(), file_range);
        Ok(())
    }

    /// Writes the bundle file offsets metadata at the end of the bundle file,
    /// and returns the byte-range of this metadata information.
    pub fn finalize(self, hotcache: &[u8]) -> io::Result<SplitPayload> {
        // Build the footer.
        let mut footer_bytes = vec![];
        // Fix paths to be relative
        let metadata_with_fixed_paths = self
            .metadata
            .files
            .iter()
            .map(|(path, range)| {
                let file_name = PathBuf::from(path.file_name().ok_or_else(|| {
                    io::Error::new(
                        ErrorKind::InvalidInput,
                        format!("could not extract file_name from path {:?}", path),
                    )
                })?);
                Ok((file_name, range.start..range.end))
            })
            .collect::<Result<HashMap<_, _>, io::Error>>()?;

        let metadata_json = serde_json::to_string(&BundleStorageFileOffsets {
            files: metadata_with_fixed_paths,
        })?;

        footer_bytes.extend(metadata_json.as_bytes());
        let metadata_json_len = metadata_json.len() as u64;
        footer_bytes.extend(&metadata_json_len.to_le_bytes());
        footer_bytes.extend(hotcache);
        footer_bytes.extend(&hotcache.len().to_le_bytes());

        let mut payloads: Vec<Box<dyn PutPayload>> = Vec::new();

        let mut sorted_files = self.metadata.files.iter().collect::<Vec<_>>();
        sorted_files.sort_by_key(|(_file, range)| range.start);

        for (path, byte_range) in sorted_files {
            let file_payload = FilePayload {
                path: path.to_owned(),
                len: byte_range.end - byte_range.start,
            };
            payloads.push(Box::new(file_payload));
        }

        payloads.push(Box::new(footer_bytes.to_vec()));

        Ok(SplitPayload {
            payloads,
            footer_range: self.current_offset as u64
                ..self.current_offset as u64 + footer_bytes.len() as u64,
        })
    }
}

/// Returns the payloads with their absolute ranges.
fn get_payloads_with_absolute_range(
    payloads: &[Box<dyn PutPayload>],
) -> Vec<(Box<dyn PutPayload>, Range<usize>)> {
    let mut current = 0;
    payloads
        .iter()
        .map(|payload| {
            let start = current;
            current += payload.len();
            (payload.clone(), start as usize..current as usize)
        })
        .collect()
}

fn get_ranges_overlap(range1: &Range<usize>, range2: &Range<usize>) -> Range<usize> {
    range1.start.max(range2.start)..range1.end.min(range2.end)
}

// Returns payloads and their relative ranges for an absolute range.
fn chunk_payload_ranges(
    payloads: &[Box<dyn PutPayload>],
    range: Range<usize>,
) -> Vec<(Box<dyn PutPayload>, Range<usize>)> {
    let mut ranges = vec![];
    for (payload, payload_absolute_range) in get_payloads_with_absolute_range(payloads) {
        let absolute_range_overlap = get_ranges_overlap(&payload_absolute_range, &range);
        if !absolute_range_overlap.is_empty() {
            // Push the range relative to this payload as we will read from it.
            ranges.push((
                payload.clone(),
                (absolute_range_overlap.start - payload_absolute_range.start)
                    ..(absolute_range_overlap.end - payload_absolute_range.start),
            ));
        }
    }
    ranges
}

#[cfg(test)]
mod tests {
    use std::fs::File;
    use std::io::Write;

    use super::*;

    #[tokio::test]
    async fn test_split_offset_computer() -> anyhow::Result<()> {
        let temp_dir = tempfile::tempdir()?;
        let test_filepath1 = temp_dir.path().join("f1");
        let test_filepath2 = temp_dir.path().join("f2");

        let mut file1 = File::create(&test_filepath1)?;
        file1.write_all(&[123, 76])?;

        let mut file2 = File::create(&test_filepath2)?;
        file2.write_all(&[99, 55, 44])?;

        let _split_streamer =
            SplitPayloadBuilder::get_split_payload(&[test_filepath1, test_filepath2], &[1, 2, 3])?;

        Ok(())
    }

    #[cfg(test)]
    async fn fetch_data(
        split_streamer: &SplitPayload,
        range: Range<u64>,
    ) -> anyhow::Result<Vec<u8>> {
        let mut data = vec![];
        split_streamer
            .range_byte_stream(range)
            .await?
            .into_async_read()
            .read_to_end(&mut data)
            .await?;
        Ok(data)
    }

    #[test]
    fn test_chunk_payloads() -> anyhow::Result<()> {
        let payloads: Vec<Box<dyn PutPayload>> = vec![
            Box::new(vec![1, 2, 3]),
            Box::new(vec![4, 5, 6]),
            Box::new(vec![7, 8, 9, 10]),
        ];

        assert_eq!(
            chunk_payload_ranges(&payloads, 0..1)
                .iter()
                .map(|el| el.1.clone())
                .collect::<Vec<_>>(),
            vec![0..1]
        );
        assert_eq!(
            chunk_payload_ranges(&payloads, 0..2)
                .iter()
                .map(|el| el.1.clone())
                .collect::<Vec<_>>(),
            vec![0..2]
        );
        assert_eq!(
            chunk_payload_ranges(&payloads, 1..2)
                .iter()
                .map(|el| el.1.clone())
                .collect::<Vec<_>>(),
            vec![1..2]
        );
        assert_eq!(
            chunk_payload_ranges(&payloads, 2..3)
                .iter()
                .map(|el| el.1.clone())
                .collect::<Vec<_>>(),
            vec![2..3]
        );
        assert_eq!(
            chunk_payload_ranges(&payloads, 0..6)
                .iter()
                .map(|el| el.1.clone())
                .collect::<Vec<_>>(),
            vec![0..3, 0..3]
        );
        assert_eq!(
            chunk_payload_ranges(&payloads, 0..5)
                .iter()
                .map(|el| el.1.clone())
                .collect::<Vec<_>>(),
            vec![0..3, 0..2]
        );
        assert_eq!(
            chunk_payload_ranges(&payloads, 3..6)
                .iter()
                .map(|el| el.1.clone())
                .collect::<Vec<_>>(),
            vec![0..3]
        );
        assert_eq!(
            chunk_payload_ranges(&payloads, 4..6)
                .iter()
                .map(|el| el.1.clone())
                .collect::<Vec<_>>(),
            vec![1..3]
        );
        assert_eq!(
            chunk_payload_ranges(&payloads, 5..6)
                .iter()
                .map(|el| el.1.clone())
                .collect::<Vec<_>>(),
            vec![2..3]
        );
        assert_eq!(
            chunk_payload_ranges(&payloads, 2..6)
                .iter()
                .map(|el| el.1.clone())
                .collect::<Vec<_>>(),
            vec![2..3, 0..3]
        );
        assert_eq!(
            chunk_payload_ranges(&payloads, 2..5)
                .iter()
                .map(|el| el.1.clone())
                .collect::<Vec<_>>(),
            vec![2..3, 0..2]
        );

        assert_eq!(
            chunk_payload_ranges(&payloads, 7..8)
                .iter()
                .map(|el| el.1.clone())
                .collect::<Vec<_>>(),
            vec![1..2]
        );

        Ok(())
    }

    #[tokio::test]
    async fn test_split_streamer() -> anyhow::Result<()> {
        let temp_dir = tempfile::tempdir()?;
        let test_filepath1 = temp_dir.path().join("a");
        let test_filepath2 = temp_dir.path().join("b");

        let mut file1 = File::create(&test_filepath1)?;
        file1.write_all(&[123, 76])?;

        let mut file2 = File::create(&test_filepath2)?;
        file2.write_all(&[99, 55, 44])?;

        let split_streamer = SplitPayloadBuilder::get_split_payload(
            &[test_filepath1.clone(), test_filepath2.clone()],
            &[1, 2, 3],
        )?;

        // border case 1 exact start of first block
        assert_eq!(fetch_data(&split_streamer, 0..1).await?, vec![123]);
        assert_eq!(fetch_data(&split_streamer, 0..2).await?, vec![123, 76]);
        assert_eq!(fetch_data(&split_streamer, 0..3).await?, vec![123, 76, 99]);

        // border 2 case skip and take cross adjacent blocks
        assert_eq!(fetch_data(&split_streamer, 1..3).await?, vec![76, 99]);

        // border 3 case skip and take in seperate blocks with full block between
        assert_eq!(
            fetch_data(&split_streamer, 1..6).await?,
            vec![76, 99, 55, 44, 123]
        );

        // border case 4 exact middle block
        assert_eq!(fetch_data(&split_streamer, 2..5).await?, vec![99, 55, 44]);

        // border case 5, no skip but take in middle block
        assert_eq!(fetch_data(&split_streamer, 2..4).await?, vec![99, 55]);

        // border case 6 skip and take in middle block
        assert_eq!(fetch_data(&split_streamer, 3..4).await?, vec![55]);

        // border case 7 start exact last block - footer
        assert_eq!(
            fetch_data(&split_streamer, 5..10).await?,
            vec![123, 34, 102, 105, 108]
        );
        // border case 8 skip and take in last block  - footer
        assert_eq!(
            fetch_data(&split_streamer, 6..10).await?,
            vec![34, 102, 105, 108]
        );

        let total_len = split_streamer.len();
        let all_data = fetch_data(&split_streamer, 0..total_len).await?;

        // last 8 bytes are the length of the hotcache bytes
        assert_eq!(all_data[all_data.len() - 8..], 3_u64.to_le_bytes());
        Ok(())
    }
}