├── .github ├── dependabot.yml └── workflows │ └── test.yml ├── .gitignore ├── Cargo.toml ├── Dockerfile ├── LICENSE ├── LICENSE-3rdparty.csv ├── Makefile ├── README.md ├── benches └── bench.rs ├── mrecordlog_cli ├── Cargo.toml └── src │ └── main.rs ├── rustfmt.toml └── src ├── block_read_write.rs ├── error.rs ├── frame ├── header.rs ├── mod.rs ├── reader.rs ├── tests.rs └── writer.rs ├── lib.rs ├── mem ├── mod.rs ├── queue.rs ├── queues.rs ├── rolling_buffer.rs ├── summary.rs └── tests.rs ├── multi_record_log.rs ├── persist_policy.rs ├── proptests.rs ├── record.rs ├── recordlog ├── mod.rs ├── reader.rs ├── tests.rs └── writer.rs ├── rolling ├── directory.rs ├── file_number.rs ├── mod.rs └── tests.rs └── tests.rs /.github/dependabot.yml: -------------------------------------------------------------------------------- 1 | version: 2 2 | updates: 3 | # Enable version updates for Cargo 4 | - package-ecosystem: "cargo" 5 | # Look for `Cargo.toml` and `Cargo.lock` files in the `root` directory 6 | directory: "/" 7 | # Check for updates once a week 8 | schedule: 9 | interval: "weekly" 10 | # Allow up to 10 open pull requests 11 | open-pull-requests-limit: 10 12 | # Create pull requests for patch and minor updates 13 | versioning-strategy: auto 14 | # Create a group of dependencies to be updated together 15 | groups: 16 | dependencies: 17 | patterns: 18 | - "*" 19 | 20 | # Enable version updates for GitHub Actions 21 | - package-ecosystem: "github-actions" 22 | directory: "/" 23 | schedule: 24 | interval: "weekly" 25 | labels: 26 | - "dependencies" 27 | - "github-actions" 28 | # Allow up to 5 open pull requests for GitHub Actions 29 | open-pull-requests-limit: 5 -------------------------------------------------------------------------------- /.github/workflows/test.yml: -------------------------------------------------------------------------------- 1 | name: Unit tests 2 | 3 | on: 4 | push: 5 | branches: [ main ] 6 | pull_request: 7 | branches: [ main ] 8 | 9 | env: 10 | CARGO_TERM_COLOR: always 11 | 12 | jobs: 13 | check: 14 | runs-on: ubuntu-latest 15 | 16 | steps: 17 | - uses: actions/checkout@v4 18 | 19 | - name: Install nightly 20 | uses: actions-rs/toolchain@v1 21 | with: 22 | toolchain: nightly 23 | profile: minimal 24 | components: rustfmt 25 | - name: Install stable 26 | uses: actions-rs/toolchain@v1 27 | with: 28 | toolchain: stable 29 | profile: minimal 30 | components: clippy 31 | 32 | - uses: Swatinem/rust-cache@v2 33 | 34 | - name: Check Formatting 35 | run: cargo +nightly fmt --all -- --check 36 | 37 | - uses: actions-rs/clippy-check@v1 38 | with: 39 | toolchain: stable 40 | token: ${{ secrets.GITHUB_TOKEN }} 41 | args: --tests 42 | 43 | test: 44 | runs-on: ubuntu-latest 45 | name: test 46 | 47 | steps: 48 | - uses: actions/checkout@v4 49 | 50 | - name: Install stable 51 | uses: actions-rs/toolchain@v1 52 | with: 53 | toolchain: stable 54 | profile: minimal 55 | override: true 56 | 57 | - uses: taiki-e/install-action@nextest 58 | - uses: Swatinem/rust-cache@v2 59 | 60 | - name: Run tests 61 | run: cargo +stable nextest run --verbose 62 | 63 | - name: Run doctests 64 | run: cargo +stable test --doc --verbose 65 | 66 | thirdparty-license: 67 | name: Check Datadog third-party license file 68 | runs-on: ubuntu-latest 69 | steps: 70 | - uses: actions/checkout@v4 71 | - name: Install Rust toolchain 72 | uses: dtolnay/rust-toolchain@stable 73 | 74 | - name: Cache cargo tools 75 | uses: actions/cache@v4 76 | with: 77 | path: ~/.cargo/bin 78 | key: ${{ runner.os }}-cargo-tools-${{ hashFiles('**/Cargo.lock') }} 79 | 80 | - name: Install dd-rust-license-tool 81 | run: dd-rust-license-tool --help || cargo install dd-rust-license-tool 82 | 83 | - name: Check Datadog third-party license file 84 | run: dd-rust-license-tool check -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | Cargo.lock 2 | proptest-regressions 3 | target 4 | 5 | -------------------------------------------------------------------------------- /Cargo.toml: -------------------------------------------------------------------------------- 1 | workspace = { members = ["mrecordlog_cli"] } 2 | [package] 3 | name = "mrecordlog" 4 | version = "0.5.0" 5 | edition = "2021" 6 | license = "MIT" 7 | description = "Quickwit's shared record log." 8 | rust-version = "1.68" # 1.67 contains an UB we would trigger 9 | 10 | # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html 11 | 12 | [dependencies] 13 | bytes = "1" 14 | crc32fast = "1.2" 15 | serde = { version = "1", features = ["derive"] } 16 | thiserror = "2" 17 | tracing = "0.1.37" 18 | 19 | [dev-dependencies] 20 | criterion = "0.5" 21 | futures = "0.3" 22 | proptest = "1" 23 | rand = "0.9" 24 | tempfile = "3" 25 | 26 | [[bench]] 27 | name = "bench" 28 | harness = false 29 | -------------------------------------------------------------------------------- /Dockerfile: -------------------------------------------------------------------------------- 1 | # --- Builder 2 | 3 | FROM rust:bullseye AS bin-builder 4 | 5 | COPY . /mrecordlog 6 | 7 | WORKDIR /mrecordlog/mrecordlog_cli 8 | 9 | RUN --mount=type=cache,target=/usr/local/cargo/registry \ 10 | --mount=type=cache,target=/mrecordlog/target \ 11 | cargo build --release --bin mrecordlog && \ 12 | ls /mrecordlog/target/release && \ 13 | mkdir -p /quickwit/bin && \ 14 | mv /mrecordlog/target/release/mrecordlog /quickwit/bin/mrecordlog 15 | 16 | # --- ACTUAL image. 17 | 18 | FROM debian:bullseye-slim AS quickwit 19 | 20 | LABEL org.opencontainers.image.title="Quickwit MRecordlog utils CLI" 21 | LABEL maintainer="Quickwit, Inc. " 22 | LABEL org.opencontainers.image.vendor="Quickwit, Inc." 23 | LABEL org.opencontainers.image.licenses="AGPL-3.0" 24 | 25 | WORKDIR /quickwit 26 | 27 | COPY --from=bin-builder /quickwit/bin/mrecordlog /usr/local/bin/mrecordlog 28 | 29 | ENTRYPOINT ["mrecordlog"] 30 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Copyright (c) 2022 by Quickwit, Inc. 2 | 3 | Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: 4 | 5 | The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. 6 | 7 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. 8 | -------------------------------------------------------------------------------- /LICENSE-3rdparty.csv: -------------------------------------------------------------------------------- 1 | Component,Origin,License,Copyright 2 | bytes,https://github.com/tokio-rs/bytes,MIT,"Carl Lerche , Sean McArthur " 3 | cfg-if,https://github.com/alexcrichton/cfg-if,MIT OR Apache-2.0,Alex Crichton 4 | crc32fast,https://github.com/srijs/rust-crc32fast,MIT OR Apache-2.0,"Sam Rijs , Alex Crichton " 5 | once_cell,https://github.com/matklad/once_cell,MIT OR Apache-2.0,Aleksey Kladov 6 | pin-project-lite,https://github.com/taiki-e/pin-project-lite,Apache-2.0 OR MIT,The pin-project-lite Authors 7 | proc-macro2,https://github.com/dtolnay/proc-macro2,MIT OR Apache-2.0,"David Tolnay , Alex Crichton " 8 | quote,https://github.com/dtolnay/quote,MIT OR Apache-2.0,David Tolnay 9 | serde,https://github.com/serde-rs/serde,MIT OR Apache-2.0,"Erick Tryzelaar , David Tolnay " 10 | syn,https://github.com/dtolnay/syn,MIT OR Apache-2.0,David Tolnay 11 | thiserror,https://github.com/dtolnay/thiserror,MIT OR Apache-2.0,David Tolnay 12 | tracing,https://github.com/tokio-rs/tracing,MIT,"Eliza Weisman , Tokio Contributors " 13 | tracing-attributes,https://github.com/tokio-rs/tracing,MIT,"Tokio Contributors , Eliza Weisman , David Barsky " 14 | tracing-core,https://github.com/tokio-rs/tracing,MIT,Tokio Contributors 15 | unicode-ident,https://github.com/dtolnay/unicode-ident,(MIT OR Apache-2.0) AND Unicode-3.0,David Tolnay 16 | -------------------------------------------------------------------------------- /Makefile: -------------------------------------------------------------------------------- 1 | help: 2 | @grep '^[^\.#[:space:]].*:' Makefile 3 | 4 | fmt: 5 | @echo "Formatting Rust files" 6 | @(rustup toolchain list | ( ! grep -q nightly && echo "Toolchain 'nightly' is not installed. Please install using 'rustup toolchain install nightly'.") ) || cargo +nightly fmt 7 | 8 | fix: fmt 9 | @echo "Running cargo clippy --fix" 10 | cargo clippy --fix --all-features --allow-dirty --allow-staged 11 | 12 | push: 13 | docker buildx build --platform linux/amd64 -t quickwit/mrecordlog:0.1 . --push 14 | 15 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # What is it? 2 | 3 | This crate implements a solution to efficiently handle several record logs. 4 | Each recordlog has its own "local" notion of position. 5 | It is possible to truncate each of the queues individually. 6 | 7 | # Goals 8 | 9 | - be durable, offer some flexibility on `fsync` strategies. 10 | - offer a way to truncate a queue after a specific position 11 | - handle an arbitrary number of queues 12 | - have limited IO 13 | - be fast 14 | - offer the possibility to implement push back 15 | 16 | ```rust 17 | pub struct MultiRecordLog { 18 | pub fn create_queue(&mut self, queue: &str) -> Result<(), CreateQueueError>; 19 | pub fn delete_queue(&mut self, queue: &str) -> Result<(), DeleteQueueError>; 20 | pub fn queue_exists(&self, queue: &str) -> bool; 21 | pub fn list_queues(&self) -> impl Iterator { 22 | pub fn append_record( 23 | &mut self, 24 | queue: &str, 25 | position_opt: Option, 26 | payload: &[u8], 27 | ); 28 | pub fn truncate(&mut self, queue: &str, position: u64) -> Result<(), TruncateError>; 29 | pub fn range( 30 | &self, 31 | queue: &str, 32 | range: R, 33 | ) -> Option + '_>; 34 | } 35 | ``` 36 | 37 | # Non-goals 38 | 39 | This is not Kafka. This recordlog is designed for a "small amount of data". 40 | All retained data can fit in RAM. 41 | 42 | In the context of Quickwit, this queue is used in the ingest API and is meant to contain 43 | 1 minute worth of data. (At 60MB/s, means 3.6 GB of RAM) 44 | 45 | Reading the recordlog files only happens on startup. 46 | High-performance when reading the recordlog files is not a goal. 47 | Writing fast on the other hand is important. 48 | 49 | # Implementation details. 50 | 51 | `mrecordlog` is multiplexing several independent queues into the same record log. 52 | This approach has the merit of limiting the number of file descriptors necessary, 53 | and more importantly, to limit the number of `fsync`. 54 | 55 | It also offers the possibility to truncate the queue for a given record log. 56 | The actual deletion of the data happens when a file only contains deleted records. 57 | Then, and only then, the entire file is deleted. 58 | 59 | That recordlog emits a new file every 1GB. 60 | A recordlog file is deleted once all queues have been truncated after the 61 | last record of a of a file. 62 | 63 | There is no compaction logic. 64 | 65 | # TODO 66 | 67 | - add backpressure. 68 | - add fsync policy 69 | - better testing. 70 | - non auto-inc position 71 | - less Arc 72 | -------------------------------------------------------------------------------- /benches/bench.rs: -------------------------------------------------------------------------------- 1 | use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; 2 | use mrecordlog::MultiRecordLog; 3 | 4 | fn bench_single_size(size: usize, count: usize, loop_count: usize) { 5 | let tempdir = tempfile::tempdir().unwrap(); 6 | let mut record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 7 | record_log.create_queue("q1").unwrap(); 8 | 9 | let record = vec![0; size]; 10 | 11 | for _ in 0..loop_count { 12 | record_log 13 | .append_records("q1", None, std::iter::repeat(&record[..]).take(count)) 14 | .unwrap(); 15 | } 16 | } 17 | 18 | fn insert_throughput(c: &mut Criterion) { 19 | let record_sizes: [usize; 2] = [1 << 8, 1 << 14]; 20 | let record_counts: [usize; 3] = [1, 16, 256]; 21 | let bytes_written: usize = 1 << 22; 22 | 23 | let mut group = c.benchmark_group("insert speed"); 24 | group.throughput(criterion::Throughput::Bytes(bytes_written as _)); 25 | 26 | for record_size in record_sizes { 27 | for record_count in record_counts { 28 | if record_size * record_count > bytes_written { 29 | continue; 30 | } 31 | let loop_count = bytes_written / record_count / record_size; 32 | 33 | group.bench_with_input( 34 | BenchmarkId::new( 35 | "bench_append_throughput", 36 | format!("size={},count={}", record_size, record_count), 37 | ), 38 | &(record_size, record_count, loop_count), 39 | |b, (record_size, record_count, loop_count)| { 40 | b.iter(|| bench_single_size(*record_size, *record_count, *loop_count)); 41 | }, 42 | ); 43 | } 44 | } 45 | } 46 | 47 | criterion_group!(benches, insert_throughput); 48 | criterion_main!(benches); 49 | -------------------------------------------------------------------------------- /mrecordlog_cli/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "mrecordlog_cli" 3 | version = "0.1.0" 4 | edition = "2021" 5 | 6 | # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html 7 | 8 | [dependencies] 9 | mrecordlog = { path = "../" } 10 | anyhow = "*" 11 | structopt = "0.3" 12 | 13 | [[bin]] 14 | name = "mrecordlog" 15 | path = "src/main.rs" 16 | -------------------------------------------------------------------------------- /mrecordlog_cli/src/main.rs: -------------------------------------------------------------------------------- 1 | use std::path::{Path, PathBuf}; 2 | 3 | use mrecordlog::MultiRecordLog; 4 | use structopt::StructOpt; 5 | 6 | #[derive(Debug, StructOpt)] 7 | enum Command { 8 | Summary { 9 | #[structopt(short = "f", default_value = "./wal")] 10 | wal_path: PathBuf, 11 | }, 12 | Read { 13 | #[structopt(short = "f", default_value = "./wal")] 14 | wal_path: PathBuf, 15 | queue_name: String, 16 | }, 17 | } 18 | 19 | fn run_summary(path: &Path) -> anyhow::Result<()> { 20 | let multi_record_log = MultiRecordLog::open(path)?; 21 | let summary = multi_record_log.summary(); 22 | for (queue, summary) in summary.queues { 23 | println!("{}", queue); 24 | println!("{summary:?}"); 25 | } 26 | Ok(()) 27 | } 28 | 29 | fn run_read_queue(path: &Path, queue_name: &str) -> anyhow::Result<()> { 30 | let multi_record_log = MultiRecordLog::open(path)?; 31 | for record in multi_record_log.range(queue_name, ..)? { 32 | let Ok(payload_str) = std::str::from_utf8(&record.payload) else { 33 | eprintln!("Payload is not utf8: {:?}", record.payload); 34 | continue; 35 | }; 36 | println!("{payload_str}"); 37 | } 38 | Ok(()) 39 | } 40 | 41 | fn main() -> anyhow::Result<()> { 42 | let command = Command::from_args(); 43 | match command { 44 | Command::Summary { wal_path } => { 45 | run_summary(&wal_path)?; 46 | } 47 | Command::Read { 48 | queue_name, 49 | wal_path, 50 | } => { 51 | run_read_queue(&wal_path, &queue_name)?; 52 | } 53 | } 54 | Ok(()) 55 | } 56 | -------------------------------------------------------------------------------- /rustfmt.toml: -------------------------------------------------------------------------------- 1 | ignore = [ 2 | "quickwit-proto/src/cluster.rs", 3 | "quickwit-proto/src/quickwit.rs", 4 | "quickwit-proto/src/quickwit_ingest_api.rs", 5 | "quickwit-proto/src/quickwit_metastore_api.rs", 6 | ] 7 | 8 | comment_width = 120 9 | format_strings = true 10 | group_imports = "StdExternalCrate" 11 | imports_granularity = "Module" 12 | normalize_comments = true 13 | where_single_line = true 14 | wrap_comments = true 15 | -------------------------------------------------------------------------------- /src/block_read_write.rs: -------------------------------------------------------------------------------- 1 | use std::io; 2 | 3 | use crate::PersistAction; 4 | 5 | pub const BLOCK_NUM_BYTES: usize = 32_768; 6 | 7 | pub trait BlockRead { 8 | /// Loads the next block. 9 | /// If `Ok(true)` is returned, the new block is available through 10 | /// `.block()`. 11 | /// 12 | /// If `Ok(false)` is returned, the end of the `BlockReader` 13 | /// has been reached and the content of `block()` could be anything. 14 | fn next_block(&mut self) -> io::Result; 15 | 16 | /// A `BlockReader` is always position on a specific block. 17 | /// 18 | /// There are no "unitialized state". 19 | /// 20 | /// # Panics 21 | /// 22 | /// May panic if the last call to next_block returned `false` 23 | /// or returned an io::Error. 24 | fn block(&self) -> &[u8; BLOCK_NUM_BYTES]; 25 | } 26 | 27 | pub trait BlockWrite { 28 | /// Must panic if buf is larger than `num_bytes_remaining_in_block`. 29 | fn write(&mut self, buf: &[u8]) -> io::Result<()>; 30 | /// Persist the data following the `persist_action`. 31 | fn persist(&mut self, persist_action: PersistAction) -> io::Result<()>; 32 | /// Number of bytes that can be added in the block. 33 | fn num_bytes_remaining_in_block(&self) -> usize; 34 | } 35 | 36 | pub struct ArrayReader<'a> { 37 | block: [u8; BLOCK_NUM_BYTES], 38 | data: &'a [u8], 39 | } 40 | 41 | impl<'a> From<&'a [u8]> for ArrayReader<'a> { 42 | fn from(data: &'a [u8]) -> Self { 43 | assert!(data.len() >= BLOCK_NUM_BYTES); 44 | let mut block = [0u8; BLOCK_NUM_BYTES]; 45 | let (head, tail) = data.split_at(BLOCK_NUM_BYTES); 46 | block.copy_from_slice(head); 47 | Self { block, data: tail } 48 | } 49 | } 50 | 51 | impl BlockRead for ArrayReader<'_> { 52 | fn next_block(&mut self) -> io::Result { 53 | if self.data.len() < BLOCK_NUM_BYTES { 54 | return Ok(false); 55 | } 56 | let (first_block, remaining) = self.data.split_at(BLOCK_NUM_BYTES); 57 | self.block.copy_from_slice(first_block); 58 | self.data = remaining; 59 | Ok(true) 60 | } 61 | 62 | fn block(&self) -> &[u8; BLOCK_NUM_BYTES] { 63 | &self.block 64 | } 65 | } 66 | 67 | #[derive(Default)] 68 | pub struct VecBlockWriter { 69 | cursor: usize, 70 | buffer: Vec, 71 | } 72 | 73 | fn ceil_to_block(len: usize) -> usize { 74 | BLOCK_NUM_BYTES * ((len + BLOCK_NUM_BYTES - 1) / BLOCK_NUM_BYTES) 75 | } 76 | 77 | impl From for Vec { 78 | fn from(vec_block_writer: VecBlockWriter) -> Vec { 79 | vec_block_writer.buffer 80 | } 81 | } 82 | 83 | impl BlockWrite for VecBlockWriter { 84 | fn write(&mut self, buf: &[u8]) -> io::Result<()> { 85 | assert!(buf.len() <= self.num_bytes_remaining_in_block()); 86 | if self.cursor + buf.len() > self.buffer.len() { 87 | let new_len = ceil_to_block((self.cursor + buf.len()) * 2 + 1); 88 | self.buffer.resize(new_len, 0u8); 89 | } 90 | self.buffer[self.cursor..][..buf.len()].copy_from_slice(buf); 91 | self.cursor += buf.len(); 92 | Ok(()) 93 | } 94 | 95 | fn persist(&mut self, _persist_action: PersistAction) -> io::Result<()> { 96 | Ok(()) 97 | } 98 | 99 | fn num_bytes_remaining_in_block(&self) -> usize { 100 | BLOCK_NUM_BYTES - (self.cursor % BLOCK_NUM_BYTES) 101 | } 102 | } 103 | -------------------------------------------------------------------------------- /src/error.rs: -------------------------------------------------------------------------------- 1 | use std::io; 2 | 3 | use thiserror::Error; 4 | 5 | #[derive(Debug, Copy, Clone)] 6 | pub struct AlreadyExists; 7 | 8 | #[derive(Error, Debug)] 9 | pub enum CreateQueueError { 10 | #[error("Already exists")] 11 | AlreadyExists, 12 | #[error("Io error: {0}")] 13 | IoError(#[from] io::Error), 14 | } 15 | 16 | impl From for CreateQueueError { 17 | fn from(_: AlreadyExists) -> Self { 18 | CreateQueueError::AlreadyExists 19 | } 20 | } 21 | 22 | #[derive(Error, Debug)] 23 | pub enum DeleteQueueError { 24 | #[error("Missing queue")] 25 | MissingQueue(String), 26 | #[error("Io error: {0}")] 27 | IoError(#[from] io::Error), 28 | } 29 | 30 | impl From for DeleteQueueError { 31 | fn from(missing_queue: MissingQueue) -> Self { 32 | DeleteQueueError::MissingQueue(missing_queue.0) 33 | } 34 | } 35 | 36 | #[derive(Debug, Error)] 37 | #[error("MultiRecordCorruption")] 38 | pub struct MultiRecordCorruption; 39 | 40 | impl From for ReadRecordError { 41 | fn from(_: MultiRecordCorruption) -> ReadRecordError { 42 | ReadRecordError::Corruption 43 | } 44 | } 45 | 46 | #[derive(Error, Debug)] 47 | pub enum TruncateError { 48 | #[error("Missing queue: {0}")] 49 | MissingQueue(String), 50 | #[error("Io error: {0}")] 51 | IoError(#[from] io::Error), 52 | } 53 | 54 | impl From for TruncateError { 55 | fn from(missing_queue: MissingQueue) -> Self { 56 | TruncateError::MissingQueue(missing_queue.0) 57 | } 58 | } 59 | 60 | #[derive(Error, Debug)] 61 | pub enum AppendError { 62 | #[error("Io error: {0}")] 63 | IoError(#[from] io::Error), 64 | #[error("Missing queue: {0}")] 65 | MissingQueue(String), 66 | #[error("Past")] 67 | Past, 68 | } 69 | 70 | impl From for AppendError { 71 | fn from(missing_queue: MissingQueue) -> Self { 72 | AppendError::MissingQueue(missing_queue.0) 73 | } 74 | } 75 | 76 | #[derive(Debug)] 77 | pub struct MissingQueue(pub String); 78 | 79 | impl std::fmt::Display for MissingQueue { 80 | fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { 81 | write!(f, "Missing queue: {}", self.0) 82 | } 83 | } 84 | 85 | impl std::error::Error for MissingQueue {} 86 | 87 | #[derive(Error, Debug)] 88 | pub enum ReadRecordError { 89 | #[error("Io error: {0}")] 90 | IoError(#[from] io::Error), 91 | #[error("Corruption")] 92 | Corruption, 93 | } 94 | -------------------------------------------------------------------------------- /src/frame/header.rs: -------------------------------------------------------------------------------- 1 | pub const HEADER_LEN: usize = 4 + 2 + 1; 2 | 3 | fn crc32(data: &[u8], frame_type: u8) -> u32 { 4 | let mut hash = crc32fast::Hasher::default(); 5 | hash.update(&[frame_type]); 6 | hash.update(data); 7 | hash.finalize() 8 | } 9 | 10 | #[derive(Debug, Eq, PartialEq)] 11 | pub(crate) struct Header { 12 | checksum: u32, 13 | len: u16, 14 | frame_type: FrameType, 15 | } 16 | 17 | impl Header { 18 | pub fn for_payload(frame_type: FrameType, payload: &[u8]) -> Header { 19 | assert!(payload.len() < crate::BLOCK_NUM_BYTES); 20 | Header { 21 | checksum: crc32(payload, frame_type as u8), 22 | len: payload.len() as u16, 23 | frame_type, 24 | } 25 | } 26 | 27 | pub fn len(&self) -> usize { 28 | self.len as usize 29 | } 30 | 31 | pub fn frame_type(&self) -> FrameType { 32 | self.frame_type 33 | } 34 | 35 | pub fn check(&self, payload: &[u8]) -> bool { 36 | crc32(payload, self.frame_type as u8) == self.checksum 37 | } 38 | 39 | /// Serialize the header 40 | /// 41 | /// # Panics 42 | /// panic if `dest` isn't exactly `HEADER_LEN` bytes long 43 | pub fn serialize(&self, dest: &mut [u8]) { 44 | assert_eq!(dest.len(), HEADER_LEN); 45 | dest[..4].copy_from_slice(&self.checksum.to_le_bytes()[..]); 46 | dest[4..6].copy_from_slice(&self.len.to_le_bytes()[..]); 47 | dest[6] = self.frame_type.to_u8(); 48 | } 49 | 50 | /// Deserialize a header 51 | /// 52 | /// # Panics 53 | /// panic if `data` isn't exactly `HEADER_LEN` bytes long 54 | pub fn deserialize(data: &[u8]) -> Option
{ 55 | assert_eq!(data.len(), HEADER_LEN); 56 | let checksum = u32::from_le_bytes([data[0], data[1], data[2], data[3]]); 57 | let len = u16::from_le_bytes([data[4], data[5]]); 58 | let frame_type = FrameType::from_u8(data[6])?; 59 | Some(Header { 60 | checksum, 61 | len, 62 | frame_type, 63 | }) 64 | } 65 | } 66 | 67 | #[repr(u8)] 68 | #[derive(Clone, Copy, PartialEq, Eq, Debug)] 69 | pub enum FrameType { 70 | Full = 1u8, 71 | First = 2u8, 72 | Middle = 3u8, 73 | Last = 4u8, 74 | } 75 | 76 | impl FrameType { 77 | fn from_u8(b: u8) -> Option { 78 | match b { 79 | 1u8 => Some(FrameType::Full), 80 | 2u8 => Some(FrameType::First), 81 | 3u8 => Some(FrameType::Middle), 82 | 4u8 => Some(FrameType::Last), 83 | _ => None, 84 | } 85 | } 86 | 87 | fn to_u8(self) -> u8 { 88 | self as u8 89 | } 90 | 91 | pub fn is_first_frame_of_record(&self) -> bool { 92 | match self { 93 | FrameType::Full | FrameType::First => true, 94 | FrameType::Last | FrameType::Middle => false, 95 | } 96 | } 97 | 98 | pub fn is_last_frame_of_record(&self) -> bool { 99 | match self { 100 | FrameType::Full | FrameType::Last => true, 101 | FrameType::First | FrameType::Middle => false, 102 | } 103 | } 104 | } 105 | 106 | #[cfg(test)] 107 | mod tests { 108 | use crate::frame::header::{Header, HEADER_LEN}; 109 | use crate::frame::FrameType; 110 | 111 | #[test] 112 | fn test_frame_type_serialize_deserialize() { 113 | const ALL_FRAME_TYPES: [FrameType; 4] = [ 114 | FrameType::Full, 115 | FrameType::First, 116 | FrameType::Middle, 117 | FrameType::Last, 118 | ]; 119 | for frame_type in ALL_FRAME_TYPES { 120 | assert_eq!(FrameType::from_u8(frame_type.to_u8()), Some(frame_type)); 121 | } 122 | } 123 | 124 | #[test] 125 | fn test_frame_deserialize_invalid() { 126 | assert_eq!(FrameType::from_u8(14u8), None); 127 | } 128 | 129 | #[test] 130 | fn test_header_serialize_deserialize() { 131 | let header = Header { 132 | checksum: 17u32, 133 | len: 42, 134 | frame_type: FrameType::Full, 135 | }; 136 | let mut buffer = [0u8; HEADER_LEN]; 137 | header.serialize(&mut buffer); 138 | let serdeser_header = Header::deserialize(&buffer).unwrap(); 139 | assert_eq!(header, serdeser_header); 140 | } 141 | 142 | #[test] 143 | fn test_header_deserialize_invalid() { 144 | let invalid_header_buffer = [14u8; HEADER_LEN]; 145 | assert_eq!(Header::deserialize(&invalid_header_buffer), None); 146 | } 147 | } 148 | -------------------------------------------------------------------------------- /src/frame/mod.rs: -------------------------------------------------------------------------------- 1 | mod header; 2 | mod reader; 3 | mod writer; 4 | 5 | use self::header::Header; 6 | pub(crate) use self::header::{FrameType, HEADER_LEN}; 7 | pub use self::reader::{FrameReader, ReadFrameError}; 8 | pub use self::writer::FrameWriter; 9 | 10 | #[cfg(test)] 11 | mod tests; 12 | -------------------------------------------------------------------------------- /src/frame/reader.rs: -------------------------------------------------------------------------------- 1 | use std::io; 2 | 3 | use thiserror::Error; 4 | 5 | use crate::frame::{FrameType, FrameWriter, Header, HEADER_LEN}; 6 | use crate::rolling::{RollingReader, RollingWriter}; 7 | use crate::{BlockRead, BLOCK_NUM_BYTES}; 8 | 9 | pub struct FrameReader { 10 | reader: R, 11 | 12 | /// In block cursor 13 | cursor: usize, 14 | 15 | // The current block is corrupted. 16 | block_corrupted: bool, 17 | } 18 | 19 | #[derive(Error, Debug)] 20 | pub enum ReadFrameError { 21 | #[error("Io error: {0}")] 22 | IoError(#[from] io::Error), 23 | #[error("Corruption in frame")] 24 | Corruption, 25 | #[error("Next frame not available")] 26 | NotAvailable, 27 | } 28 | 29 | impl FrameReader { 30 | pub fn open(reader: R) -> Self { 31 | FrameReader { 32 | reader, 33 | cursor: 0, 34 | block_corrupted: false, 35 | } 36 | } 37 | 38 | pub fn read(&self) -> &R { 39 | &self.reader 40 | } 41 | 42 | // Returns the number of bytes remaining into 43 | // the current block. 44 | // 45 | // These bytes may or may not be available. 46 | fn num_bytes_to_end_of_block(&self) -> usize { 47 | crate::BLOCK_NUM_BYTES - self.cursor 48 | } 49 | 50 | fn go_to_next_block_if_necessary(&mut self) -> Result<(), ReadFrameError> { 51 | let num_bytes_to_end_of_block = self.num_bytes_to_end_of_block(); 52 | let need_to_skip_block = self.block_corrupted || num_bytes_to_end_of_block < HEADER_LEN; 53 | if !need_to_skip_block { 54 | return Ok(()); 55 | } 56 | if !self.reader.next_block()? { 57 | return Err(ReadFrameError::NotAvailable); 58 | } 59 | 60 | self.cursor = 0; 61 | self.block_corrupted = false; 62 | Ok(()) 63 | } 64 | 65 | // Attempt to read the header of the next frame 66 | // This method does not consume any bytes (which is why it is called get and not read). 67 | fn get_frame_header(&mut self) -> Result { 68 | let header_bytes: &[u8] = &self.reader.block()[self.cursor..][..HEADER_LEN]; 69 | if header_bytes == [0u8; HEADER_LEN] { 70 | return Err(ReadFrameError::NotAvailable); 71 | } 72 | match Header::deserialize(header_bytes) { 73 | Some(header) => Ok(header), 74 | None => { 75 | self.block_corrupted = true; 76 | Err(ReadFrameError::Corruption) 77 | } 78 | } 79 | } 80 | 81 | // Reads the next frame. 82 | pub fn read_frame(&mut self) -> Result<(FrameType, &[u8]), ReadFrameError> { 83 | self.go_to_next_block_if_necessary()?; 84 | let header = self.get_frame_header()?; 85 | self.cursor += HEADER_LEN; 86 | if self.cursor + header.len() > BLOCK_NUM_BYTES { 87 | // The number of bytes for this frame would span over 88 | // the next block. 89 | // This is a corruption for which we need to drop the entire block. 90 | self.block_corrupted = true; 91 | return Err(ReadFrameError::Corruption); 92 | } 93 | let frame_payload = &self.reader.block()[self.cursor..][..header.len()]; 94 | self.cursor += header.len(); 95 | if !header.check(frame_payload) { 96 | // The CRC check is wrong. 97 | // We do not necessarily need to corrupt the block. 98 | // 99 | // With a little luck, a single frame payload byte was corrupted 100 | // but the frame length was correct. 101 | return Err(ReadFrameError::Corruption); 102 | } 103 | Ok((header.frame_type(), frame_payload)) 104 | } 105 | } 106 | 107 | impl FrameReader { 108 | pub fn into_writer(self) -> io::Result> { 109 | let mut rolling_writer: RollingWriter = self.reader.into_writer()?; 110 | rolling_writer.forward(self.cursor)?; 111 | Ok(FrameWriter::create(rolling_writer)) 112 | } 113 | } 114 | -------------------------------------------------------------------------------- /src/frame/tests.rs: -------------------------------------------------------------------------------- 1 | use std::io; 2 | 3 | use crate::block_read_write::{ArrayReader, VecBlockWriter}; 4 | use crate::frame::header::{FrameType, HEADER_LEN}; 5 | use crate::frame::{FrameReader, FrameWriter, ReadFrameError}; 6 | use crate::{PersistAction, BLOCK_NUM_BYTES}; 7 | 8 | #[test] 9 | fn test_frame_simple() { 10 | let block_writer = { 11 | let wrt: VecBlockWriter = VecBlockWriter::default(); 12 | let mut frame_writer = FrameWriter::create(wrt); 13 | frame_writer 14 | .write_frame(FrameType::First, &b"abc"[..]) 15 | .unwrap(); 16 | frame_writer 17 | .write_frame(FrameType::Middle, &b"de"[..]) 18 | .unwrap(); 19 | frame_writer 20 | .write_frame(FrameType::Last, &b"fgh"[..]) 21 | .unwrap(); 22 | frame_writer.persist(PersistAction::Flush).unwrap(); 23 | frame_writer.into_writer() 24 | }; 25 | let buffer: Vec = block_writer.into(); 26 | let mut frame_reader = FrameReader::open(ArrayReader::from(&buffer[..])); 27 | let read_frame_res = frame_reader.read_frame(); 28 | assert_eq!(read_frame_res.unwrap(), (FrameType::First, &b"abc"[..])); 29 | assert_eq!( 30 | frame_reader.read_frame().unwrap(), 31 | (FrameType::Middle, &b"de"[..]) 32 | ); 33 | assert_eq!( 34 | frame_reader.read_frame().unwrap(), 35 | (FrameType::Last, &b"fgh"[..]) 36 | ); 37 | assert!(matches!( 38 | frame_reader.read_frame().unwrap_err(), 39 | ReadFrameError::NotAvailable 40 | )); 41 | } 42 | 43 | #[test] 44 | fn test_frame_corruption_in_payload() -> io::Result<()> { 45 | let mut buf: Vec = { 46 | let mut frame_writer = FrameWriter::create(VecBlockWriter::default()); 47 | frame_writer.write_frame(FrameType::First, &b"abc"[..])?; 48 | frame_writer.persist(PersistAction::Flush)?; 49 | frame_writer.write_frame(FrameType::Middle, &b"de"[..])?; 50 | frame_writer.persist(PersistAction::Flush)?; 51 | frame_writer.into_writer().into() 52 | }; 53 | buf[8] = 0u8; 54 | let mut frame_reader = FrameReader::open(ArrayReader::from(&buf[..])); 55 | assert!(matches!( 56 | frame_reader.read_frame(), 57 | Err(ReadFrameError::Corruption) 58 | )); 59 | assert!(matches!( 60 | frame_reader.read_frame(), 61 | Ok((FrameType::Middle, b"de")) 62 | )); 63 | Ok(()) 64 | } 65 | 66 | fn repeat_empty_frame_util(repeat: usize) -> Vec { 67 | let mut frame_writer = FrameWriter::create(VecBlockWriter::default()); 68 | for _ in 0..repeat { 69 | frame_writer.write_frame(FrameType::Full, &b""[..]).unwrap(); 70 | } 71 | frame_writer.persist(PersistAction::Flush).unwrap(); 72 | frame_writer.into_writer().into() 73 | } 74 | 75 | #[test] 76 | fn test_simple_multiple_blocks() -> io::Result<()> { 77 | let num_frames = 1 + BLOCK_NUM_BYTES / HEADER_LEN; 78 | let buffer = repeat_empty_frame_util(num_frames); 79 | let mut frame_reader = FrameReader::open(ArrayReader::from(&buffer[..])); 80 | for _ in 0..num_frames { 81 | let read_frame_res = frame_reader.read_frame(); 82 | assert!(matches!(read_frame_res, Ok((FrameType::Full, &[])))); 83 | } 84 | assert!(matches!( 85 | frame_reader.read_frame(), 86 | Err(ReadFrameError::NotAvailable) 87 | )); 88 | Ok(()) 89 | } 90 | 91 | #[test] 92 | fn test_multiple_blocks_corruption_on_length() -> io::Result<()> { 93 | // We end up with 4681 frames on the first block. 94 | // 1 frame on the second block 95 | let num_frames = 1 + crate::BLOCK_NUM_BYTES / HEADER_LEN; 96 | let mut buffer = repeat_empty_frame_util(num_frames); 97 | buffer[2000 * HEADER_LEN + 5] = 255u8; 98 | let mut frame_reader = FrameReader::open(ArrayReader::from(&buffer[..])); 99 | for _ in 0..2000 { 100 | let read_frame_res = frame_reader.read_frame(); 101 | assert!(matches!(read_frame_res, Ok((FrameType::Full, &[])))); 102 | } 103 | assert!(matches!( 104 | frame_reader.read_frame(), 105 | Err(ReadFrameError::Corruption) 106 | )); 107 | assert!(matches!( 108 | frame_reader.read_frame(), 109 | Ok((FrameType::Full, &[])) 110 | )); 111 | assert!(matches!( 112 | frame_reader.read_frame(), 113 | Err(ReadFrameError::NotAvailable) 114 | )); 115 | Ok(()) 116 | } 117 | -------------------------------------------------------------------------------- /src/frame/writer.rs: -------------------------------------------------------------------------------- 1 | use std::io; 2 | 3 | use crate::frame::{FrameType, Header, HEADER_LEN}; 4 | use crate::rolling::{Directory, RollingWriter}; 5 | use crate::{BlockWrite, PersistAction, BLOCK_NUM_BYTES}; 6 | 7 | pub struct FrameWriter { 8 | wrt: W, 9 | // temporary buffer, not storing anything in particular after any function returns 10 | buffer: Box<[u8; BLOCK_NUM_BYTES]>, 11 | } 12 | 13 | impl FrameWriter { 14 | pub fn create(wrt: W) -> Self { 15 | FrameWriter { 16 | wrt, 17 | buffer: Box::new([0u8; BLOCK_NUM_BYTES]), 18 | } 19 | } 20 | 21 | /// Writes a frame. The payload has to be lower than the 22 | /// remaining space in the frame as defined 23 | /// by `max_writable_frame_length`. 24 | pub fn write_frame(&mut self, frame_type: FrameType, payload: &[u8]) -> io::Result<()> { 25 | let num_bytes_remaining_in_block = self.wrt.num_bytes_remaining_in_block(); 26 | if num_bytes_remaining_in_block < HEADER_LEN { 27 | let zero_bytes = [0u8; HEADER_LEN]; 28 | self.wrt 29 | .write(&zero_bytes[..num_bytes_remaining_in_block])?; 30 | } 31 | let record_len = HEADER_LEN + payload.len(); 32 | let (buffer_header, buffer_record) = self.buffer[..record_len].split_at_mut(HEADER_LEN); 33 | buffer_record.copy_from_slice(payload); 34 | Header::for_payload(frame_type, payload).serialize(buffer_header); 35 | self.wrt.write(&self.buffer[..record_len])?; 36 | Ok(()) 37 | } 38 | 39 | /// Flush the buffered writer used in the FrameWriter. 40 | /// 41 | /// When writing to a file, this performs a syscall and 42 | /// the OS will be in charge of eventually writing the data 43 | /// to disk, but this is not sufficient to ensure durability. 44 | pub fn persist(&mut self, persist_action: PersistAction) -> io::Result<()> { 45 | self.wrt.persist(persist_action) 46 | } 47 | 48 | /// Returns the maximum amount of bytes that can be written. 49 | pub fn max_writable_frame_length(&self) -> usize { 50 | let available_num_bytes_in_block = self.wrt.num_bytes_remaining_in_block(); 51 | if available_num_bytes_in_block >= HEADER_LEN { 52 | available_num_bytes_in_block - HEADER_LEN 53 | } else { 54 | // That block is finished. We will have to pad it. 55 | BLOCK_NUM_BYTES - HEADER_LEN 56 | } 57 | } 58 | 59 | pub fn get_underlying_wrt(&self) -> &W { 60 | &self.wrt 61 | } 62 | 63 | #[cfg(test)] 64 | pub fn into_writer(self) -> W { 65 | self.wrt 66 | } 67 | } 68 | 69 | impl FrameWriter { 70 | pub fn directory(&mut self) -> &mut Directory { 71 | &mut self.wrt.directory 72 | } 73 | } 74 | -------------------------------------------------------------------------------- /src/lib.rs: -------------------------------------------------------------------------------- 1 | use std::borrow::Cow; 2 | 3 | mod block_read_write; 4 | 5 | pub use block_read_write::{BlockRead, BlockWrite, BLOCK_NUM_BYTES}; 6 | pub mod error; 7 | mod frame; 8 | mod mem; 9 | mod multi_record_log; 10 | mod persist_policy; 11 | mod record; 12 | mod recordlog; 13 | mod rolling; 14 | 15 | pub use mem::{QueueSummary, QueuesSummary}; 16 | pub use multi_record_log::MultiRecordLog; 17 | pub(crate) use persist_policy::PersistState; 18 | pub use persist_policy::{PersistAction, PersistPolicy}; 19 | 20 | #[derive(Debug, PartialEq, Eq)] 21 | pub struct Record<'a> { 22 | pub position: u64, 23 | pub payload: Cow<'a, [u8]>, 24 | } 25 | 26 | impl<'a> Record<'a> { 27 | pub fn new(position: u64, payload: &'a [u8]) -> Self { 28 | Record { 29 | position, 30 | payload: Cow::Borrowed(payload), 31 | } 32 | } 33 | } 34 | 35 | /// Resources used by mrecordlog 36 | #[derive(Debug, Clone, Copy, PartialEq, Eq)] 37 | pub struct ResourceUsage { 38 | /// Actual size of the memory used 39 | pub memory_used_bytes: usize, 40 | /// Capacity allocated, a part of which may be unused right now 41 | pub memory_allocated_bytes: usize, 42 | /// Disk size used 43 | pub disk_used_bytes: usize, 44 | } 45 | 46 | #[cfg(test)] 47 | mod tests; 48 | 49 | #[cfg(test)] 50 | mod proptests; 51 | 52 | pub trait Serializable<'a>: Sized { 53 | /// Clears the buffer first. 54 | fn serialize(&self, buffer: &mut Vec); 55 | fn deserialize(buffer: &'a [u8]) -> Option; 56 | } 57 | 58 | impl<'a> Serializable<'a> for &'a str { 59 | fn serialize(&self, buffer: &mut Vec) { 60 | buffer.clear(); 61 | buffer.extend_from_slice(self.as_bytes()) 62 | } 63 | 64 | fn deserialize(buffer: &'a [u8]) -> Option { 65 | std::str::from_utf8(buffer).ok() 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /src/mem/mod.rs: -------------------------------------------------------------------------------- 1 | mod queue; 2 | mod queues; 3 | mod rolling_buffer; 4 | mod summary; 5 | 6 | pub(crate) use self::queue::MemQueue; 7 | pub(crate) use self::queues::MemQueues; 8 | pub use self::summary::{QueueSummary, QueuesSummary}; 9 | 10 | #[cfg(test)] 11 | mod tests; 12 | -------------------------------------------------------------------------------- /src/mem/queue.rs: -------------------------------------------------------------------------------- 1 | use std::ops::{Bound, RangeBounds, RangeToInclusive}; 2 | 3 | use super::rolling_buffer::RollingBuffer; 4 | use crate::error::AppendError; 5 | use crate::mem::QueueSummary; 6 | use crate::rolling::FileNumber; 7 | use crate::Record; 8 | 9 | #[derive(Clone)] 10 | struct RecordMeta { 11 | start_offset: usize, 12 | // in a vec of RecordMeta, this field should be set only on the last record 13 | // which relate to that File. 14 | file_number: Option, 15 | position: u64, 16 | } 17 | 18 | #[derive(Default)] 19 | pub(crate) struct MemQueue { 20 | // Concatenated records 21 | concatenated_records: RollingBuffer, 22 | start_position: u64, 23 | record_metas: Vec, 24 | } 25 | 26 | impl MemQueue { 27 | pub fn with_next_position(next_position: u64) -> Self { 28 | MemQueue { 29 | concatenated_records: RollingBuffer::new(), 30 | start_position: next_position, 31 | record_metas: Vec::new(), 32 | } 33 | } 34 | 35 | pub fn summary(&self) -> QueueSummary { 36 | QueueSummary { 37 | start: self.start_position(), 38 | end: self.last_position(), 39 | file_number: self.first_file_number(), 40 | } 41 | } 42 | 43 | pub fn is_empty(&self) -> bool { 44 | self.record_metas.is_empty() 45 | } 46 | 47 | pub(crate) fn first_file_number(&self) -> Option { 48 | let file_number: &FileNumber = self 49 | .record_metas 50 | .iter() 51 | .filter_map(|record_meta| record_meta.file_number.as_ref()) 52 | .next()?; 53 | Some(file_number.file_number()) 54 | } 55 | 56 | pub(crate) fn start_position(&self) -> u64 { 57 | self.start_position 58 | } 59 | 60 | /// Returns the position of the last record appended to the queue. 61 | pub fn last_position(&self) -> Option { 62 | self.next_position().checked_sub(1) 63 | } 64 | 65 | /// Returns the last record stored in the queue. 66 | pub fn last_record(&self) -> Option { 67 | self.record_metas.last().map(|record| Record { 68 | position: record.position, 69 | payload: self.concatenated_records.get_range(record.start_offset..), 70 | }) 71 | } 72 | 73 | /// Returns what the next position should be. 74 | pub fn next_position(&self) -> u64 { 75 | self.record_metas 76 | .last() 77 | .map(|record| record.position + 1) 78 | .unwrap_or(self.start_position) 79 | } 80 | 81 | /// Appends a new record at a given position. 82 | /// Returns an error if the record was not added. 83 | /// 84 | /// AppendError if the record is strangely in the past or is too much in the future. 85 | pub fn append_record( 86 | &mut self, 87 | file_number: &FileNumber, 88 | target_position: u64, 89 | payload: &[u8], 90 | ) -> Result<(), AppendError> { 91 | let next_position = self.next_position(); 92 | if target_position < next_position { 93 | return Err(AppendError::Past); 94 | } 95 | 96 | if self.start_position == 0u64 && self.record_metas.is_empty() { 97 | self.start_position = target_position; 98 | } 99 | 100 | let file_number = if let Some(record_meta) = self.record_metas.last_mut() { 101 | if record_meta.file_number.as_ref() == Some(file_number) { 102 | record_meta.file_number.take().unwrap() 103 | } else { 104 | file_number.clone() 105 | } 106 | } else { 107 | file_number.clone() 108 | }; 109 | 110 | let record_meta = RecordMeta { 111 | start_offset: self.concatenated_records.len(), 112 | file_number: Some(file_number), 113 | position: target_position, 114 | }; 115 | self.record_metas.push(record_meta); 116 | self.concatenated_records.extend(payload); 117 | Ok(()) 118 | } 119 | 120 | /// Get the position of the record. 121 | /// 122 | /// Returns Ok(_) if the record was found, or Err(idx) with idx being the index just after 123 | /// where that element would have been if it existed. 124 | fn position_to_idx(&self, position: u64) -> Result { 125 | self.record_metas 126 | .binary_search_by_key(&position, |record| record.position) 127 | } 128 | 129 | pub fn range(&self, range: R) -> impl Iterator + '_ 130 | where R: RangeBounds + 'static { 131 | let start_idx: usize = match range.start_bound() { 132 | Bound::Included(&start_from) => { 133 | // if pos is included, we can use position_to_idx result directly 134 | self.position_to_idx(start_from) 135 | .unwrap_or_else(std::convert::identity) 136 | } 137 | Bound::Excluded(&start_from) => { 138 | // if pos is excluded, an Err can be used directly, but an Ok must be incremented 139 | // by one to skip the element matching exactly. 140 | self.position_to_idx(start_from) 141 | .map(|idx| idx + 1) 142 | .unwrap_or_else(std::convert::identity) 143 | } 144 | Bound::Unbounded => 0, 145 | }; 146 | (start_idx..self.record_metas.len()) 147 | .take_while(move |idx| range.contains(&self.record_metas[*idx].position)) 148 | .map(move |idx| { 149 | let record = &self.record_metas[idx]; 150 | let position = record.position; 151 | let start_offset = record.start_offset; 152 | let payload = if let Some(next_record_meta) = self.record_metas.get(idx + 1) { 153 | let end_offset = next_record_meta.start_offset; 154 | self.concatenated_records 155 | .get_range(start_offset..end_offset) 156 | } else { 157 | self.concatenated_records.get_range(start_offset..) 158 | }; 159 | Record { position, payload } 160 | }) 161 | } 162 | 163 | /// Removes all records coming before position, and including the record at "position". 164 | /// 165 | /// If truncating to a future position, make the queue go forward to that position. 166 | /// Return the number of record removed. 167 | pub fn truncate_head(&mut self, truncate_range: RangeToInclusive) -> usize { 168 | let truncate_up_to_pos = truncate_range.end; 169 | if self.start_position > truncate_up_to_pos { 170 | return 0; 171 | } 172 | if truncate_up_to_pos + 1 >= self.next_position() { 173 | self.start_position = truncate_up_to_pos + 1; 174 | self.concatenated_records.clear(); 175 | let record_count = self.record_metas.len(); 176 | self.record_metas.clear(); 177 | return record_count; 178 | } 179 | let first_record_to_keep = self 180 | .position_to_idx(truncate_up_to_pos + 1) 181 | .unwrap_or_else(std::convert::identity); 182 | 183 | let start_offset_to_keep: usize = self.record_metas[first_record_to_keep].start_offset; 184 | self.record_metas.drain(..first_record_to_keep); 185 | for record_meta in &mut self.record_metas { 186 | record_meta.start_offset -= start_offset_to_keep; 187 | } 188 | self.concatenated_records 189 | .truncate_head(..start_offset_to_keep); 190 | self.start_position = truncate_up_to_pos + 1; 191 | first_record_to_keep 192 | } 193 | 194 | pub fn size(&self) -> usize { 195 | self.concatenated_records.len() 196 | + self.record_metas.len() * std::mem::size_of::() 197 | } 198 | 199 | pub fn capacity(&self) -> usize { 200 | self.concatenated_records.capacity() 201 | + self.record_metas.capacity() * std::mem::size_of::() 202 | } 203 | } 204 | -------------------------------------------------------------------------------- /src/mem/queues.rs: -------------------------------------------------------------------------------- 1 | use std::collections::HashMap; 2 | use std::ops::{RangeBounds, RangeToInclusive}; 3 | 4 | use tracing::{info, warn}; 5 | 6 | use crate::error::{AlreadyExists, AppendError, MissingQueue}; 7 | use crate::mem::{MemQueue, QueuesSummary}; 8 | use crate::rolling::FileNumber; 9 | use crate::Record; 10 | 11 | #[derive(Default)] 12 | pub(crate) struct MemQueues { 13 | queues: HashMap, 14 | } 15 | impl MemQueues { 16 | /// The file number argument is here unused. Its point is just to make sure we 17 | /// flushed the file before updating the in memory queue. 18 | pub fn create_queue(&mut self, queue: &str) -> Result<(), AlreadyExists> { 19 | if self.queues.contains_key(queue) { 20 | return Err(AlreadyExists); 21 | } 22 | self.queues.insert(queue.to_string(), MemQueue::default()); 23 | Ok(()) 24 | } 25 | 26 | pub fn summary(&self) -> QueuesSummary { 27 | let mut summary = QueuesSummary::default(); 28 | for (queue_name, queue) in &self.queues { 29 | summary.queues.insert(queue_name.clone(), queue.summary()); 30 | } 31 | summary 32 | } 33 | 34 | pub fn delete_queue(&mut self, queue: &str) -> Result<(), MissingQueue> { 35 | info!(queue = queue, "deleting queue"); 36 | if self.queues.remove(queue).is_none() { 37 | warn!(queue = queue, "attempted to remove a non-existing queue"); 38 | return Err(MissingQueue(queue.to_string())); 39 | } 40 | Ok(()) 41 | } 42 | 43 | /// Returns all sub-queues which are currently empty. 44 | pub fn empty_queues(&mut self) -> impl Iterator + '_ { 45 | self.queues.iter_mut().filter_map(|(queue, mem_queue)| { 46 | if mem_queue.is_empty() { 47 | Some((queue.as_str(), mem_queue)) 48 | } else { 49 | None 50 | } 51 | }) 52 | } 53 | 54 | pub fn range( 55 | &self, 56 | queue: &str, 57 | range: R, 58 | ) -> Result + '_, MissingQueue> 59 | where 60 | R: RangeBounds + 'static, 61 | { 62 | if let Some(queue) = self.queues.get(queue) { 63 | Ok(queue.range(range)) 64 | } else { 65 | Err(MissingQueue(queue.to_string())) 66 | } 67 | } 68 | 69 | pub(crate) fn get_queue(&self, queue: &str) -> Result<&MemQueue, MissingQueue> { 70 | // We do not rely on `entry` in order to avoid 71 | // the allocation. 72 | self.queues 73 | .get(queue) 74 | .ok_or_else(|| MissingQueue(queue.to_string())) 75 | } 76 | 77 | pub(crate) fn get_queue_mut(&mut self, queue: &str) -> Result<&mut MemQueue, MissingQueue> { 78 | // We do not rely on `entry` in order to avoid 79 | // the allocation. 80 | self.queues 81 | .get_mut(queue) 82 | .ok_or_else(|| MissingQueue(queue.to_string())) 83 | } 84 | 85 | pub fn append_record( 86 | &mut self, 87 | queue: &str, 88 | file_number: &FileNumber, 89 | target_position: u64, 90 | payload: &[u8], 91 | ) -> Result<(), AppendError> { 92 | self.get_queue_mut(queue)? 93 | .append_record(file_number, target_position, payload) 94 | } 95 | 96 | pub fn contains_queue(&self, queue: &str) -> bool { 97 | self.queues.contains_key(queue) 98 | } 99 | 100 | pub fn list_queues(&self) -> impl Iterator { 101 | self.queues.keys().map(|queue| queue.as_str()) 102 | } 103 | 104 | /// Ensure that the queue is empty and start_position = next_position. 105 | /// 106 | /// If the queue doesn't exist, create it. If it does, but isn't empty or the position doesn't 107 | /// match, truncate it and make it go forward to the requested position. 108 | /// 109 | /// This operation is meant only to rebuild the in memory queue from its on-disk state. 110 | pub fn ack_position(&mut self, queue_name: &str, next_position: u64) { 111 | if let Some(queue) = self.queues.get(queue_name) { 112 | // It is possible for `ack_position` to be called when a queue already exists. 113 | // 114 | // For instance, we may have recorded the position of an empty stale queue 115 | // twice in the same file. Nothing prevents that from happening today. 116 | // 117 | // Another possibility is if an IO error occured right after recording position 118 | // and before deleting files. 119 | if !queue.is_empty() || queue.next_position() != next_position { 120 | // if we are here, some updates to the queue were lost/corrupted, but it's no 121 | // big deal as they were no longer considered part of the active state. We can 122 | // delete and recreate the queue to put it in the expected state. 123 | self.queues.remove(queue_name); 124 | self.queues.insert( 125 | queue_name.to_string(), 126 | MemQueue::with_next_position(next_position), 127 | ); 128 | } 129 | } else { 130 | // The queue does not exist! Let's create it and set the right `next_position`. 131 | self.queues.insert( 132 | queue_name.to_string(), 133 | MemQueue::with_next_position(next_position), 134 | ); 135 | } 136 | } 137 | 138 | /// Returns the position of the last record appended to the queue. 139 | pub fn last_position(&self, queue: &str) -> Result, MissingQueue> { 140 | Ok(self.get_queue(queue)?.last_position()) 141 | } 142 | 143 | /// Returns the last record stored in the queue. 144 | pub fn last_record(&self, queue: &str) -> Result, MissingQueue> { 145 | Ok(self.get_queue(queue)?.last_record()) 146 | } 147 | 148 | pub fn next_position(&self, queue: &str) -> Result { 149 | Ok(self.get_queue(queue)?.next_position()) 150 | } 151 | 152 | /// Removes records up to the supplied `position`, 153 | /// including the position itself. 154 | /// 155 | /// If there are no records `<= position`, the method will 156 | /// not do anything. 157 | pub fn truncate(&mut self, queue: &str, position: RangeToInclusive) -> Option { 158 | if let Ok(queue) = self.get_queue_mut(queue) { 159 | Some(queue.truncate_head(position)) 160 | } else { 161 | None 162 | } 163 | } 164 | 165 | /// Return a tuple of (size, capacity) of memory used by the memqueues 166 | pub fn size(&self) -> (usize, usize) { 167 | let size = self 168 | .queues 169 | .iter() 170 | .map(|(name, queue)| name.len() + queue.size()) 171 | .sum(); 172 | 173 | let capacity = self 174 | .queues 175 | .iter() 176 | .map(|(name, queue)| name.capacity() + queue.capacity()) 177 | .sum(); 178 | 179 | (size, capacity) 180 | } 181 | } 182 | -------------------------------------------------------------------------------- /src/mem/rolling_buffer.rs: -------------------------------------------------------------------------------- 1 | use std::borrow::Cow; 2 | use std::collections::VecDeque; 3 | use std::ops::{Bound, RangeBounds, RangeTo}; 4 | 5 | #[derive(Default)] 6 | pub struct RollingBuffer { 7 | buffer: VecDeque, 8 | } 9 | 10 | impl RollingBuffer { 11 | pub fn new() -> Self { 12 | RollingBuffer { 13 | buffer: VecDeque::new(), 14 | } 15 | } 16 | 17 | pub fn len(&self) -> usize { 18 | self.buffer.len() 19 | } 20 | 21 | pub fn capacity(&self) -> usize { 22 | self.buffer.capacity() 23 | } 24 | 25 | pub fn clear(&mut self) { 26 | self.buffer.clear(); 27 | self.buffer.shrink_to_fit(); 28 | } 29 | 30 | // Removes all of the data up to pos byte excluded (meaning pos is kept). 31 | // 32 | // If we notice that the rolling buffer was very large, this function may shrink 33 | // it. 34 | pub fn truncate_head(&mut self, first_pos_to_keep: RangeTo) { 35 | let target_capacity = self.len() * 9 / 8; 36 | self.buffer.drain(first_pos_to_keep); 37 | // In order to avoid leaking memory we shrink the buffer. 38 | // The last maximum length (= the length before drain) 39 | // is a good estimate of what we will need in the future. 40 | // 41 | // We add 1/8 to that in order to make sure that we don't end up 42 | // shrinking / allocating for small variations. 43 | 44 | if self.buffer.capacity() > target_capacity { 45 | self.buffer.shrink_to(target_capacity); 46 | } 47 | } 48 | 49 | pub fn extend(&mut self, slice: &[u8]) { 50 | self.buffer.extend(slice.iter().copied()); 51 | } 52 | 53 | pub fn get_range(&self, bounds: impl RangeBounds) -> Cow<[u8]> { 54 | let start = match bounds.start_bound() { 55 | Bound::Included(pos) => *pos, 56 | Bound::Excluded(pos) => pos + 1, 57 | Bound::Unbounded => 0, 58 | }; 59 | 60 | let end = match bounds.end_bound() { 61 | Bound::Included(pos) => pos + 1, 62 | Bound::Excluded(pos) => *pos, 63 | Bound::Unbounded => self.len(), 64 | }; 65 | 66 | let (left_part_of_queue, right_part_of_queue) = self.buffer.as_slices(); 67 | 68 | if end < left_part_of_queue.len() { 69 | Cow::Borrowed(&left_part_of_queue[start..end]) 70 | } else if start >= left_part_of_queue.len() { 71 | let start = start - left_part_of_queue.len(); 72 | let end = end - left_part_of_queue.len(); 73 | 74 | Cow::Borrowed(&right_part_of_queue[start..end]) 75 | } else { 76 | // VecDeque is a rolling buffer. As a result, we do not have 77 | // access to a continuous buffer. 78 | // 79 | // Here the requested slice cross the boundary and we need to allocate and copy the data 80 | // in a new buffer. 81 | let mut res = Vec::with_capacity(end - start); 82 | res.extend_from_slice(&left_part_of_queue[start..]); 83 | let end = end - left_part_of_queue.len(); 84 | res.extend_from_slice(&right_part_of_queue[..end]); 85 | 86 | Cow::Owned(res) 87 | } 88 | } 89 | } 90 | -------------------------------------------------------------------------------- /src/mem/summary.rs: -------------------------------------------------------------------------------- 1 | use std::collections::BTreeMap; 2 | 3 | use serde::Serialize; 4 | 5 | #[derive(Default, Serialize, Debug)] 6 | pub struct QueueSummary { 7 | pub start: u64, 8 | pub end: Option, 9 | pub file_number: Option, 10 | } 11 | 12 | #[derive(Default, Serialize)] 13 | pub struct QueuesSummary { 14 | pub queues: BTreeMap, 15 | } 16 | -------------------------------------------------------------------------------- /src/mem/tests.rs: -------------------------------------------------------------------------------- 1 | use super::*; 2 | use crate::error::{AlreadyExists, AppendError}; 3 | use crate::rolling::FileNumber; 4 | use crate::Record; 5 | 6 | #[test] 7 | fn test_mem_queues_already_exists() { 8 | let mut mem_queues = MemQueues::default(); 9 | mem_queues.create_queue("droopy").unwrap(); 10 | assert!(matches!( 11 | mem_queues.create_queue("droopy"), 12 | Err(AlreadyExists) 13 | )); 14 | } 15 | 16 | #[test] 17 | fn test_mem_queues() { 18 | let mut mem_queues = MemQueues::default(); 19 | mem_queues.create_queue("droopy").unwrap(); 20 | mem_queues.create_queue("fable").unwrap(); 21 | { 22 | assert!(mem_queues 23 | .append_record("droopy", &FileNumber::for_test(1), 0, b"hello") 24 | .is_ok()); 25 | assert!(mem_queues 26 | .append_record("droopy", &FileNumber::for_test(1), 1, b"happy") 27 | .is_ok()); 28 | } 29 | 30 | { 31 | assert!(mem_queues 32 | .append_record("fable", &FileNumber::for_test(1), 0, b"maitre") 33 | .is_ok()); 34 | assert!(mem_queues 35 | .append_record("fable", &FileNumber::for_test(1), 1, b"corbeau") 36 | .is_ok()); 37 | } 38 | 39 | { 40 | assert!(mem_queues 41 | .append_record("droopy", &FileNumber::for_test(1), 2, b"tax") 42 | .is_ok()); 43 | assert!(mem_queues 44 | .append_record("droopy", &FileNumber::for_test(1), 3, b"payer") 45 | .is_ok()); 46 | assert_eq!( 47 | mem_queues.range("droopy", 0..).unwrap().next(), 48 | Some(Record::new(0, b"hello")) 49 | ); 50 | let droopy: Vec = mem_queues.range("droopy", 1..).unwrap().collect(); 51 | assert_eq!( 52 | &droopy, 53 | &[ 54 | Record::new(1, b"happy"), 55 | Record::new(2, b"tax"), 56 | Record::new(3, b"payer"), 57 | ], 58 | ); 59 | } 60 | let fable: Vec = mem_queues.range("fable", 1..).unwrap().collect(); 61 | assert_eq!(&fable, &[Record::new(1, b"corbeau")]); 62 | } 63 | 64 | #[test] 65 | fn test_mem_queues_truncate() { 66 | let mut mem_queues = MemQueues::default(); 67 | mem_queues.create_queue("droopy").unwrap(); 68 | { 69 | assert!(mem_queues 70 | .append_record("droopy", &1.into(), 0, b"hello") 71 | .is_ok()); 72 | assert!(mem_queues 73 | .append_record("droopy", &1.into(), 1, b"happy") 74 | .is_ok()); 75 | assert!(mem_queues 76 | .append_record("droopy", &1.into(), 2, b"tax") 77 | .is_ok()); 78 | assert!(mem_queues 79 | .append_record("droopy", &1.into(), 3, b"payer") 80 | .is_ok()); 81 | assert!(mem_queues 82 | .append_record("droopy", &1.into(), 4, b"!") 83 | .is_ok()); 84 | mem_queues 85 | .append_record("droopy", &1.into(), 5, b"payer") 86 | .unwrap(); 87 | } 88 | mem_queues.truncate("droopy", ..=3); 89 | let droopy: Vec = mem_queues.range("droopy", 0..).unwrap().collect(); 90 | assert_eq!( 91 | &droopy[..], 92 | &[Record::new(4, b"!"), Record::new(5, b"payer"),] 93 | ); 94 | } 95 | 96 | #[test] 97 | fn test_mem_queues_skip_advance() { 98 | let mut mem_queues = MemQueues::default(); 99 | mem_queues.create_queue("droopy").unwrap(); 100 | assert!(mem_queues 101 | .append_record("droopy", &1.into(), 0, b"hello") 102 | .is_ok()); 103 | assert!(mem_queues 104 | .append_record("droopy", &1.into(), 2, b"happy") 105 | .is_ok()); 106 | assert!(mem_queues 107 | .append_record("droopy", &1.into(), 3, b"happy") 108 | .is_ok()); 109 | assert!(mem_queues 110 | .append_record("droopy", &1.into(), 1, b"happy") 111 | .is_err()); 112 | let droopy: Vec = mem_queues.range("droopy", 0..).unwrap().collect(); 113 | assert_eq!( 114 | &droopy[..], 115 | &[ 116 | Record::new(0, b"hello"), 117 | Record::new(2, b"happy"), 118 | Record::new(3, b"happy"), 119 | ] 120 | ); 121 | let droopy: Vec = mem_queues.range("droopy", 1..).unwrap().collect(); 122 | assert_eq!( 123 | &droopy[..], 124 | &[Record::new(2, b"happy"), Record::new(3, b"happy"),] 125 | ); 126 | let droopy: Vec = mem_queues.range("droopy", 2..).unwrap().collect(); 127 | assert_eq!( 128 | &droopy[..], 129 | &[Record::new(2, b"happy"), Record::new(3, b"happy"),] 130 | ); 131 | let droopy: Vec = mem_queues.range("droopy", 3..).unwrap().collect(); 132 | assert_eq!(&droopy[..], &[Record::new(3, b"happy")]); 133 | } 134 | 135 | #[test] 136 | fn test_mem_queues_append_in_the_past_yield_error() { 137 | let mut mem_queues = MemQueues::default(); 138 | mem_queues.create_queue("droopy").unwrap(); 139 | assert!(mem_queues 140 | .append_record("droopy", &1.into(), 0, b"hello") 141 | .is_ok()); 142 | assert!(mem_queues 143 | .append_record("droopy", &1.into(), 1, b"happy") 144 | .is_ok()); 145 | assert!(matches!( 146 | mem_queues.append_record("droopy", &1.into(), 0, b"happy"), 147 | Err(AppendError::Past) 148 | )); 149 | } 150 | 151 | #[test] 152 | fn test_mem_queues_append_idempotence() { 153 | let mut mem_queues = MemQueues::default(); 154 | mem_queues.create_queue("droopy").unwrap(); 155 | assert!(mem_queues 156 | .append_record("droopy", &1.into(), 0, b"hello") 157 | .is_ok()); 158 | assert!(matches!( 159 | mem_queues 160 | .append_record("droopy", &1.into(), 0, b"different") 161 | .unwrap_err(), 162 | AppendError::Past 163 | )); 164 | let droopy: Vec = mem_queues.range("droopy", 0..).unwrap().collect(); 165 | assert_eq!(&droopy, &[Record::new(0, b"hello")]); 166 | } 167 | 168 | #[test] 169 | fn test_mem_queues_non_zero_first_el() { 170 | let mut mem_queues = MemQueues::default(); 171 | mem_queues.create_queue("droopy").unwrap(); 172 | assert!(mem_queues 173 | .append_record("droopy", &1.into(), 5, b"hello") 174 | .is_ok()); 175 | let droopy: Vec = mem_queues.range("droopy", 0..).unwrap().collect(); 176 | assert_eq!(droopy, &[Record::new(5, b"hello")]); 177 | } 178 | 179 | #[test] 180 | fn test_mem_queues_keep_filenum() { 181 | let mut mem_queues = MemQueues::default(); 182 | 183 | let files = (0..4).map(FileNumber::for_test).collect::>(); 184 | 185 | assert!(files.iter().all(FileNumber::can_be_deleted)); 186 | 187 | mem_queues.create_queue("droopy").unwrap(); 188 | mem_queues 189 | .append_record("droopy", &files[0], 0, b"hello") 190 | .unwrap(); 191 | 192 | assert!(!files[0].can_be_deleted()); 193 | 194 | mem_queues 195 | .append_record("droopy", &files[0], 1, b"hello") 196 | .unwrap(); 197 | 198 | assert!(!files[0].can_be_deleted()); 199 | 200 | mem_queues 201 | .append_record("droopy", &files[0], 2, b"hello") 202 | .unwrap(); 203 | 204 | assert!(!files[0].can_be_deleted()); 205 | 206 | mem_queues 207 | .append_record("droopy", &files[1], 3, b"hello") 208 | .unwrap(); 209 | 210 | assert!(!files[0].can_be_deleted()); 211 | assert!(!files[1].can_be_deleted()); 212 | 213 | mem_queues.truncate("droopy", ..=1); 214 | 215 | assert!(!files[0].can_be_deleted()); 216 | assert!(!files[1].can_be_deleted()); 217 | 218 | mem_queues 219 | .append_record("droopy", &files[2], 4, b"hello") 220 | .unwrap(); 221 | 222 | assert!(!files[0].can_be_deleted()); 223 | assert!(!files[1].can_be_deleted()); 224 | assert!(!files[2].can_be_deleted()); 225 | 226 | mem_queues.truncate("droopy", ..=3); 227 | 228 | assert!(files[0].can_be_deleted()); 229 | assert!(files[1].can_be_deleted()); 230 | assert!(!files[2].can_be_deleted()); 231 | 232 | mem_queues.truncate("droopy", ..=4); 233 | 234 | let empty_queues = mem_queues.empty_queues().collect::>(); 235 | assert_eq!(empty_queues.len(), 1); 236 | assert_eq!(empty_queues[0].0, "droopy"); 237 | 238 | mem_queues.ack_position("droopy", 5); 239 | 240 | assert!(files[2].can_be_deleted()); 241 | } 242 | -------------------------------------------------------------------------------- /src/multi_record_log.rs: -------------------------------------------------------------------------------- 1 | use std::io; 2 | use std::ops::{RangeBounds, RangeToInclusive}; 3 | use std::path::Path; 4 | 5 | use bytes::Buf; 6 | use tracing::{debug, event_enabled, info, warn, Level}; 7 | 8 | use crate::error::{ 9 | AppendError, CreateQueueError, DeleteQueueError, MissingQueue, ReadRecordError, TruncateError, 10 | }; 11 | use crate::mem::{MemQueue, QueuesSummary}; 12 | use crate::record::{MultiPlexedRecord, MultiRecord}; 13 | use crate::recordlog::RecordWriter; 14 | use crate::rolling::RollingWriter; 15 | use crate::{mem, PersistAction, PersistPolicy, PersistState, Record, ResourceUsage}; 16 | 17 | pub struct MultiRecordLog { 18 | record_log_writer: crate::recordlog::RecordWriter, 19 | in_mem_queues: mem::MemQueues, 20 | next_persist: PersistState, 21 | // A simple buffer we reuse to avoid allocation. 22 | multi_record_spare_buffer: Vec, 23 | } 24 | 25 | impl MultiRecordLog { 26 | /// Open the multi record log, flushing after each operation, but not fsyncing. 27 | pub fn open(directory_path: &Path) -> Result { 28 | Self::open_with_prefs(directory_path, PersistPolicy::Always(PersistAction::Flush)) 29 | } 30 | 31 | pub fn summary(&self) -> QueuesSummary { 32 | self.in_mem_queues.summary() 33 | } 34 | 35 | /// Open the multi record log, syncing following the provided policy. 36 | pub fn open_with_prefs( 37 | directory_path: &Path, 38 | persist_policy: PersistPolicy, 39 | ) -> Result { 40 | // io errors are non-recoverable 41 | let rolling_reader = crate::rolling::RollingReader::open(directory_path)?; 42 | let mut record_reader = crate::recordlog::RecordReader::open(rolling_reader); 43 | let mut in_mem_queues = crate::mem::MemQueues::default(); 44 | debug!("loading wal"); 45 | loop { 46 | let file_number = record_reader.read().current_file().clone(); 47 | let Ok(record) = record_reader.read_record::() else { 48 | warn!("Detected corrupted record: some data may have been lost"); 49 | continue; 50 | }; 51 | if let Some(record) = record { 52 | match record { 53 | MultiPlexedRecord::AppendRecords { 54 | queue, 55 | records, 56 | position, 57 | } => { 58 | if !in_mem_queues.contains_queue(queue) { 59 | in_mem_queues.ack_position(queue, position); 60 | } 61 | for record in records { 62 | // if this fails, it means some corruption wasn't detected at a lower 63 | // level, or we wrote invalid data. 64 | let (position, payload) = record?; 65 | // this can fail if queue doesn't exist (it was created just above, so 66 | // it does), or if the position is in the past. This can happen if the 67 | // queue is deleted and recreated in a block which get skipped for 68 | // corruption. In that case, maybe we should ack_position() and try 69 | // to insert again? 70 | in_mem_queues 71 | .append_record(queue, &file_number, position, payload) 72 | .map_err(|_| ReadRecordError::Corruption)?; 73 | } 74 | } 75 | MultiPlexedRecord::Truncate { 76 | truncate_range, 77 | queue, 78 | } => { 79 | in_mem_queues.truncate(queue, truncate_range); 80 | } 81 | MultiPlexedRecord::RecordPosition { queue, position } => { 82 | in_mem_queues.ack_position(queue, position); 83 | } 84 | MultiPlexedRecord::DeleteQueue { queue, position: _ } => { 85 | // can fail if we don't know about the queue getting deleted. It's fine to 86 | // just ignore the error, the queue no longer exists either way. 87 | let _ = in_mem_queues.delete_queue(queue); 88 | } 89 | } 90 | } else { 91 | break; 92 | } 93 | } 94 | // io errors are non-recoverable 95 | let record_log_writer: RecordWriter = record_reader.into_writer()?; 96 | let mut multi_record_log = MultiRecordLog { 97 | record_log_writer, 98 | in_mem_queues, 99 | next_persist: persist_policy.into(), 100 | multi_record_spare_buffer: Vec::new(), 101 | }; 102 | multi_record_log.run_gc_if_necessary()?; 103 | Ok(multi_record_log) 104 | } 105 | 106 | #[cfg(test)] 107 | pub fn list_file_numbers(&self) -> Vec { 108 | let rolling_writer = self.record_log_writer.get_underlying_wrt(); 109 | rolling_writer.list_file_numbers() 110 | } 111 | 112 | /// Creates a new queue. 113 | /// 114 | /// Returns an error if the queue already exists. 115 | pub fn create_queue(&mut self, queue: &str) -> Result<(), CreateQueueError> { 116 | info!(queue = queue, "create queue"); 117 | if self.queue_exists(queue) { 118 | return Err(CreateQueueError::AlreadyExists); 119 | } 120 | let record = MultiPlexedRecord::RecordPosition { queue, position: 0 }; 121 | self.record_log_writer.write_record(record)?; 122 | self.persist(PersistAction::FlushAndFsync)?; 123 | self.in_mem_queues.create_queue(queue)?; 124 | Ok(()) 125 | } 126 | 127 | pub fn delete_queue(&mut self, queue: &str) -> Result<(), DeleteQueueError> { 128 | info!(queue = queue, "delete queue"); 129 | let position = self.in_mem_queues.next_position(queue)?; 130 | let record = MultiPlexedRecord::DeleteQueue { queue, position }; 131 | self.record_log_writer.write_record(record)?; 132 | self.in_mem_queues.delete_queue(queue)?; 133 | self.run_gc_if_necessary()?; 134 | self.persist(PersistAction::FlushAndFsync)?; 135 | Ok(()) 136 | } 137 | 138 | pub fn queue_exists(&self, queue: &str) -> bool { 139 | self.in_mem_queues.contains_queue(queue) 140 | } 141 | 142 | pub fn list_queues(&self) -> impl Iterator { 143 | self.in_mem_queues.list_queues() 144 | } 145 | 146 | /// Appends a record to the log. 147 | /// 148 | /// The local_position argument can optionally be passed to enforce idempotence. 149 | /// TODO if an io Error is encounterred, the in mem queue and the record log will 150 | /// be in an inconsistent state. 151 | pub fn append_record( 152 | &mut self, 153 | queue: &str, 154 | position_opt: Option, 155 | payload: impl Buf, 156 | ) -> Result, AppendError> { 157 | self.append_records(queue, position_opt, std::iter::once(payload)) 158 | } 159 | 160 | /// Appends multiple records to the log. 161 | /// 162 | /// This operation is atomic: either all records get stored, or none do. 163 | /// However this function succeeding does not necessarily means records where stored, be sure 164 | /// to call [`Self::persist`] to make sure changes are persisted if you don't use 165 | /// [`PersistPolicy::Always`] (which is the default). 166 | pub fn append_records>( 167 | &mut self, 168 | queue: &str, 169 | position_opt: Option, 170 | payloads: T, 171 | ) -> Result, AppendError> { 172 | let next_position = self.in_mem_queues.next_position(queue)?; 173 | if let Some(position) = position_opt { 174 | // we accept position in the future, and move forward as required. 175 | if position + 1 == next_position { 176 | return Ok(None); 177 | } else if position < next_position { 178 | return Err(AppendError::Past); 179 | } 180 | } 181 | let position = position_opt.unwrap_or(next_position); 182 | let file_number = self.record_log_writer.current_file().clone(); 183 | 184 | let mut multi_record_spare_buffer = std::mem::take(&mut self.multi_record_spare_buffer); 185 | MultiRecord::serialize(payloads, position, &mut multi_record_spare_buffer); 186 | if multi_record_spare_buffer.is_empty() { 187 | self.multi_record_spare_buffer = multi_record_spare_buffer; 188 | // empty transaction: don't persist it 189 | return Ok(None); 190 | } 191 | 192 | let records = MultiRecord::new_unchecked(&multi_record_spare_buffer); 193 | let record = MultiPlexedRecord::AppendRecords { 194 | position, 195 | queue, 196 | records, 197 | }; 198 | self.record_log_writer.write_record(record)?; 199 | self.persist_on_policy()?; 200 | 201 | let mem_queue = self.in_mem_queues.get_queue_mut(queue)?; 202 | let mut max_position = position; 203 | for record in records { 204 | // we just serialized it, we know it's valid 205 | let (position, payload) = record.unwrap(); 206 | mem_queue.append_record(&file_number, position, payload)?; 207 | max_position = position; 208 | } 209 | 210 | self.multi_record_spare_buffer = multi_record_spare_buffer; 211 | Ok(Some(max_position)) 212 | } 213 | 214 | fn record_empty_queues_position(&mut self) -> io::Result<()> { 215 | let mut has_empty_queues = false; 216 | for (queue_id, queue) in self.in_mem_queues.empty_queues() { 217 | let next_position = queue.next_position(); 218 | let record = MultiPlexedRecord::RecordPosition { 219 | queue: queue_id, 220 | position: next_position, 221 | }; 222 | self.record_log_writer.write_record(record)?; 223 | has_empty_queues = true 224 | } 225 | if has_empty_queues { 226 | // We need to fsync here! We are remove files from the FS 227 | // so we need to make sure our empty queue positions are properly persisted. 228 | self.persist(PersistAction::FlushAndFsync)?; 229 | } 230 | Ok(()) 231 | } 232 | 233 | /// Truncates the queue up to a given `position`, included. This method immediately 234 | /// truncates the underlying in-memory queue whereas the backing log files are deleted 235 | /// asynchronously when they become exclusively composed of deleted records. 236 | /// 237 | /// This method will always truncate the record log and release the associated memory. 238 | /// It returns the number of records deleted. 239 | pub fn truncate( 240 | &mut self, 241 | queue: &str, 242 | truncate_range: RangeToInclusive, 243 | ) -> Result { 244 | info!(range=?truncate_range, queue = queue, "truncate queue"); 245 | if !self.queue_exists(queue) { 246 | return Err(TruncateError::MissingQueue(queue.to_string())); 247 | } 248 | self.record_log_writer 249 | .write_record(MultiPlexedRecord::Truncate { 250 | truncate_range, 251 | queue, 252 | })?; 253 | let removed_count = self 254 | .in_mem_queues 255 | .truncate(queue, truncate_range) 256 | .unwrap_or(0); 257 | self.run_gc_if_necessary()?; 258 | self.persist_on_policy()?; 259 | Ok(removed_count) 260 | } 261 | 262 | fn run_gc_if_necessary(&mut self) -> io::Result<()> { 263 | debug!("run_gc_if_necessary"); 264 | if self 265 | .record_log_writer 266 | .directory() 267 | .has_files_that_can_be_deleted() 268 | { 269 | // We are about to delete files. 270 | // Let's make sure we record the offsets of the empty queues 271 | // so that we don't lose that information after dropping the files. 272 | // 273 | // But first we clone the current file number to make sure that the file that will 274 | // contain the truncate positions it self won't be GC'ed. 275 | let _file_number = self.record_log_writer.current_file().clone(); 276 | self.record_empty_queues_position()?; 277 | self.record_log_writer.directory().gc()?; 278 | } 279 | // only execute the following if we are above the debug level in tokio tracing 280 | if event_enabled!(Level::DEBUG) { 281 | for queue in self.list_queues() { 282 | let queue: &MemQueue = self.in_mem_queues.get_queue(queue).unwrap(); 283 | let first_pos = queue.range(..).next().map(|record| record.position); 284 | let last_pos = queue.last_position(); 285 | debug!(first_pos=?first_pos, last_pos=?last_pos, "queue positions after gc"); 286 | } 287 | } 288 | Ok(()) 289 | } 290 | 291 | pub fn range( 292 | &self, 293 | queue: &str, 294 | range: R, 295 | ) -> Result, MissingQueue> 296 | where 297 | R: RangeBounds + 'static, 298 | { 299 | self.in_mem_queues.range(queue, range) 300 | } 301 | 302 | /// Flush if the policy says it should be done 303 | fn persist_on_policy(&mut self) -> io::Result<()> { 304 | if let Some(persist_action) = self.next_persist.should_persist() { 305 | self.persist(persist_action)?; 306 | self.next_persist.update_persisted(); 307 | } 308 | Ok(()) 309 | } 310 | 311 | /// Flush and optionnally fsync data 312 | pub fn persist(&mut self, persist_action: PersistAction) -> io::Result<()> { 313 | self.record_log_writer.persist(persist_action) 314 | } 315 | 316 | /// Returns the position of the last record appended to the queue. 317 | pub fn last_position(&self, queue: &str) -> Result, MissingQueue> { 318 | self.in_mem_queues.last_position(queue) 319 | } 320 | 321 | /// Returns the last record stored in the queue. 322 | pub fn last_record(&self, queue: &str) -> Result>, MissingQueue> { 323 | self.in_mem_queues.last_record(queue) 324 | } 325 | 326 | /// Return the amount of memory and disk space used by mrecordlog. 327 | pub fn resource_usage(&self) -> ResourceUsage { 328 | let disk_used_bytes = self.record_log_writer.size(); 329 | let (memory_used_bytes, memory_allocated_bytes) = self.in_mem_queues.size(); 330 | ResourceUsage { 331 | memory_used_bytes, 332 | memory_allocated_bytes, 333 | disk_used_bytes, 334 | } 335 | } 336 | } 337 | -------------------------------------------------------------------------------- /src/persist_policy.rs: -------------------------------------------------------------------------------- 1 | use std::time::{Duration, Instant}; 2 | 3 | #[derive(Copy, Clone, Debug, PartialEq, Eq)] 4 | pub enum PersistAction { 5 | /// The buffer will be flushed to the OS, but not necessarily to the disk. 6 | Flush, 7 | /// The buffer will be flushed to the OS, and the OS will be asked to flush 8 | /// it to the disk. 9 | FlushAndFsync, 10 | } 11 | 12 | impl PersistAction { 13 | pub fn is_fsync(self) -> bool { 14 | self == PersistAction::FlushAndFsync 15 | } 16 | } 17 | 18 | /// We have two type of operations on the mrecordlog. 19 | /// 20 | /// Critical records are relatively rare and really need to be persisted: 21 | /// - RecordPosition { queue: &'a str, position: u64 }, 22 | /// - DeleteQueue. 23 | /// 24 | /// For these operations, we want to always flush and fsync. 25 | /// 26 | /// On the other hand, 27 | /// - Truncate 28 | /// - AppendRecords 29 | /// 30 | /// are both considered are more frequent and one might want to sacrifice 31 | /// persistence guarantees for performance. 32 | /// 33 | /// The `PersistPolicy` defines the trade-off applied for the second kind of 34 | /// operations. 35 | #[derive(Clone, Debug)] 36 | pub enum PersistPolicy { 37 | /// Only ensure data is persisted when critical records are written. 38 | /// 39 | /// With this policy, the timing after which the data reaches the disk 40 | /// is up to the OS. 41 | DoNothing, 42 | /// Pesiste data once every interval, and when critical records are written 43 | OnDelay { 44 | interval: Duration, 45 | action: PersistAction, 46 | }, 47 | /// Persist data after each action 48 | Always(PersistAction), 49 | } 50 | 51 | #[derive(Debug)] 52 | pub(crate) enum PersistState { 53 | OnAppend(PersistAction), 54 | OnDelay { 55 | next_persist: Instant, 56 | interval: Duration, 57 | action: PersistAction, 58 | }, 59 | NoOp, 60 | } 61 | 62 | impl PersistState { 63 | pub fn should_persist(&self) -> Option { 64 | match self { 65 | PersistState::OnAppend(action) => Some(*action), 66 | PersistState::OnDelay { 67 | action, 68 | next_persist, 69 | .. 70 | } => { 71 | if *next_persist < Instant::now() { 72 | Some(*action) 73 | } else { 74 | None 75 | } 76 | } 77 | PersistState::NoOp => None, 78 | } 79 | } 80 | 81 | pub fn update_persisted(&mut self) { 82 | match self { 83 | PersistState::OnAppend(_) | PersistState::NoOp => (), 84 | PersistState::OnDelay { 85 | ref mut next_persist, 86 | interval, 87 | .. 88 | } => *next_persist = Instant::now() + *interval, 89 | } 90 | } 91 | } 92 | 93 | impl From for PersistState { 94 | fn from(val: PersistPolicy) -> PersistState { 95 | match val { 96 | PersistPolicy::Always(action) => PersistState::OnAppend(action), 97 | PersistPolicy::OnDelay { interval, action } => PersistState::OnDelay { 98 | next_persist: Instant::now() + interval, 99 | interval, 100 | action, 101 | }, 102 | PersistPolicy::DoNothing => PersistState::NoOp, 103 | } 104 | } 105 | } 106 | -------------------------------------------------------------------------------- /src/proptests.rs: -------------------------------------------------------------------------------- 1 | use std::borrow::Cow; 2 | use std::collections::HashMap; 3 | use std::ops::Range; 4 | 5 | use proptest::prelude::prop; 6 | use proptest::prop_oneof; 7 | use proptest::strategy::{Just, Strategy}; 8 | use tempfile::TempDir; 9 | 10 | use crate::record::{MultiPlexedRecord, MultiRecord}; 11 | use crate::{MultiRecordLog, Record, Serializable}; 12 | 13 | struct PropTestEnv { 14 | tempdir: TempDir, 15 | record_log: MultiRecordLog, 16 | state: HashMap<&'static str, (Range, u64)>, 17 | block_to_write: Vec, 18 | } 19 | 20 | impl PropTestEnv { 21 | pub fn new(block_size: usize) -> Self { 22 | let tempdir = tempfile::tempdir().unwrap(); 23 | let mut record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 24 | record_log.create_queue("q1").unwrap(); 25 | record_log.create_queue("q2").unwrap(); 26 | let mut state = HashMap::default(); 27 | state.insert("q1", (0..0, 0)); 28 | state.insert("q2", (0..0, 0)); 29 | PropTestEnv { 30 | tempdir, 31 | record_log, 32 | state, 33 | block_to_write: vec![b'A'; block_size], 34 | } 35 | } 36 | 37 | pub fn apply(&mut self, op: Operation) { 38 | match op { 39 | Operation::Reopen => { 40 | self.reload(); 41 | } 42 | Operation::MultiAppend { 43 | queue, 44 | count, 45 | skip_one_pos, 46 | } => { 47 | self.multi_append(queue, count, skip_one_pos); 48 | } 49 | Operation::RedundantAppend { 50 | queue, 51 | skip_one_pos, 52 | } => { 53 | self.double_append(queue, skip_one_pos); 54 | } 55 | Operation::Truncate { queue, pos } => { 56 | self.truncate(queue, pos); 57 | } 58 | } 59 | } 60 | 61 | pub fn reload(&mut self) { 62 | self.record_log = MultiRecordLog::open(self.tempdir.path()).unwrap(); 63 | for (queue, (_range, count)) in &self.state { 64 | assert_eq!( 65 | self.record_log.range(queue, ..).unwrap().count() as u64, 66 | *count, 67 | ); 68 | } 69 | } 70 | 71 | pub fn double_append(&mut self, queue: &str, skip_one_pos: bool) { 72 | let state = self.state.get_mut(queue).unwrap(); 73 | 74 | let new_pos = state.0.end + skip_one_pos as u64; 75 | let res = self 76 | .record_log 77 | .append_records(queue, Some(new_pos), std::iter::once(&b"BB"[..])) 78 | .unwrap() 79 | .unwrap(); 80 | 81 | assert!(self 82 | .record_log 83 | .append_records(queue, Some(new_pos), std::iter::once(&b"BB"[..])) 84 | .unwrap() 85 | .is_none()); 86 | 87 | assert_eq!(new_pos, res); 88 | state.0.end = new_pos + 1; 89 | state.1 += 1; 90 | } 91 | 92 | pub fn multi_append(&mut self, queue: &str, count: u64, skip_one_pos: bool) { 93 | let state = self.state.get_mut(queue).unwrap(); 94 | 95 | let new_pos = state.0.end + skip_one_pos as u64; 96 | let res = self 97 | .record_log 98 | .append_records( 99 | queue, 100 | Some(new_pos), 101 | std::iter::repeat(&self.block_to_write[..]).take(count as usize), 102 | ) 103 | .unwrap(); 104 | 105 | if count != 0 { 106 | let res = res.unwrap(); 107 | assert_eq!(new_pos + count - 1, res); 108 | state.0.end = new_pos + count; 109 | state.1 += count; 110 | } 111 | } 112 | 113 | pub fn truncate(&mut self, queue: &str, pos: u64) { 114 | let state = self.state.get_mut(queue).unwrap(); 115 | if state.0.contains(&pos) { 116 | state.0.start = pos + 1; 117 | state.1 -= self.record_log.truncate(queue, ..=pos).unwrap() as u64; 118 | } else if pos >= state.0.end { 119 | // advance the queue to the position. 120 | state.0 = (pos + 1)..(pos + 1); 121 | state.1 = 0; 122 | self.record_log.truncate(queue, ..=pos).unwrap(); 123 | } else { 124 | // should be a no-op 125 | self.record_log.truncate(queue, ..=pos).unwrap(); 126 | } 127 | } 128 | } 129 | 130 | fn queue_strategy() -> impl Strategy { 131 | prop_oneof![Just("q1"), Just("q2"),] 132 | } 133 | 134 | fn operation_strategy() -> impl Strategy { 135 | prop_oneof![ 136 | Just(Operation::Reopen), 137 | (queue_strategy(), proptest::bool::ANY).prop_map(|(queue, skip_one_pos)| { 138 | Operation::RedundantAppend { 139 | queue, 140 | skip_one_pos, 141 | } 142 | }), 143 | (queue_strategy(), 0u64..10u64).prop_map(|(queue, pos)| Operation::Truncate { queue, pos }), 144 | (queue_strategy(), 0u64..10u64, proptest::bool::ANY).prop_map( 145 | |(queue, count, skip_one_pos)| Operation::MultiAppend { 146 | queue, 147 | count, 148 | skip_one_pos 149 | } 150 | ), 151 | ] 152 | } 153 | 154 | fn operations_strategy() -> impl Strategy> { 155 | prop::collection::vec(operation_strategy(), 1..100) 156 | } 157 | 158 | fn random_bytevec_strategy(max_len: usize) -> impl Strategy> { 159 | prop::collection::vec(proptest::num::u8::ANY, 0..max_len) 160 | } 161 | 162 | fn random_multi_record_strategy( 163 | max_record_count: usize, 164 | max_len: usize, 165 | ) -> impl Strategy>> { 166 | prop::collection::vec(random_bytevec_strategy(max_len), 1..max_record_count) 167 | } 168 | 169 | #[test] 170 | fn test_scenario_end_on_full_file() { 171 | use Operation::*; 172 | let ops = [ 173 | MultiAppend { 174 | queue: "q1", 175 | count: 1, 176 | skip_one_pos: false, 177 | }, 178 | MultiAppend { 179 | queue: "q1", 180 | count: 1, 181 | skip_one_pos: false, 182 | }, 183 | MultiAppend { 184 | queue: "q2", 185 | count: 1, 186 | skip_one_pos: false, 187 | }, 188 | MultiAppend { 189 | queue: "q2", 190 | count: 1, 191 | skip_one_pos: false, 192 | }, 193 | MultiAppend { 194 | queue: "q2", 195 | count: 1, 196 | skip_one_pos: false, 197 | }, 198 | Reopen, 199 | RedundantAppend { 200 | queue: "q2", 201 | skip_one_pos: false, 202 | }, 203 | Reopen, 204 | ]; 205 | // this value is crafted to make so exactly two full files are stored, 206 | // but no 3rd is created: if anything about the format change, this test 207 | // will become useless (but won't fail spuriously). 208 | let mut env = PropTestEnv::new(52381); 209 | for op in ops { 210 | env.apply(op); 211 | } 212 | } 213 | 214 | #[test] 215 | fn test_scenario_big_records() { 216 | use Operation::*; 217 | let ops = [ 218 | MultiAppend { 219 | queue: "q1", 220 | count: 2, 221 | skip_one_pos: false, 222 | }, 223 | MultiAppend { 224 | queue: "q2", 225 | count: 4, 226 | skip_one_pos: false, 227 | }, 228 | Reopen, 229 | MultiAppend { 230 | queue: "q2", 231 | count: 1, 232 | skip_one_pos: false, 233 | }, 234 | Reopen, 235 | RedundantAppend { 236 | queue: "q2", 237 | skip_one_pos: false, 238 | }, 239 | Reopen, 240 | ]; 241 | let mut env = PropTestEnv::new(1 << 26); 242 | for op in ops { 243 | env.apply(op); 244 | } 245 | } 246 | 247 | fn queue_name_len() -> impl Strategy { 248 | prop_oneof![ 249 | 100 => 1..10usize, 250 | 1 => 65_534..=65_536usize 251 | ] 252 | } 253 | 254 | fn queue_name_strategy() -> impl Strategy { 255 | queue_name_len().prop_flat_map(|num_bytes| { 256 | proptest::collection::vec(proptest::prelude::any::(), num_bytes).prop_map( 257 | move |chars| { 258 | let mut s: String = String::new(); 259 | s.extend(chars); 260 | let boundaries = (0..num_bytes).rev(); 261 | for boundary in boundaries { 262 | if s.is_char_boundary(boundary) { 263 | s.truncate(boundary); 264 | break; 265 | } 266 | } 267 | s 268 | }, 269 | ) 270 | }) 271 | } 272 | 273 | proptest::proptest! { 274 | #[test] 275 | fn test_proptest_multirecord((ops, block_size) in (operations_strategy(), 0usize..65535)) { 276 | let mut env = PropTestEnv::new(block_size); 277 | for op in ops { 278 | env.apply(op); 279 | } 280 | } 281 | 282 | #[test] 283 | fn test_proptest_multiplexed_record_roundtrip((kind, queue, position, payload) in 284 | (0u8..4u8, queue_name_strategy(), proptest::num::u64::ANY, random_multi_record_strategy(64, 65536))) { 285 | let mut buffer = Vec::new(); 286 | MultiRecord::serialize(payload.iter().map(|p| p.as_ref()), position, &mut buffer); 287 | let record = match kind { 288 | 0 => MultiPlexedRecord::AppendRecords { 289 | queue: &queue, 290 | position, 291 | records: MultiRecord::new(&buffer).unwrap(), 292 | }, 293 | 1 => MultiPlexedRecord::Truncate { 294 | queue: &queue, 295 | truncate_range: ..=position 296 | }, 297 | 2 => MultiPlexedRecord::RecordPosition {queue: &queue, position}, 298 | 3 => MultiPlexedRecord::DeleteQueue {queue: &queue, position}, 299 | 4.. => unreachable!(), 300 | }; 301 | 302 | let mut buffer = Vec::new(); 303 | 304 | record.serialize(&mut buffer); 305 | 306 | let deser = MultiPlexedRecord::deserialize(&buffer).unwrap(); 307 | assert_eq!(record, deser); 308 | if let MultiPlexedRecord::AppendRecords { records, .. } = deser { 309 | assert!(records 310 | .map(|record| record.unwrap().1) 311 | .zip(payload) 312 | .all(|(record, payload)| record == payload)); 313 | } 314 | } 315 | } 316 | 317 | #[derive(Debug, Clone)] 318 | enum Operation { 319 | Reopen, 320 | MultiAppend { 321 | queue: &'static str, 322 | count: u64, 323 | skip_one_pos: bool, 324 | }, 325 | RedundantAppend { 326 | queue: &'static str, 327 | skip_one_pos: bool, 328 | }, 329 | Truncate { 330 | queue: &'static str, 331 | pos: u64, 332 | }, 333 | } 334 | 335 | #[test] 336 | fn test_multi_record() { 337 | let tempdir = tempfile::tempdir().unwrap(); 338 | eprintln!("dir={tempdir:?}"); 339 | { 340 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 341 | multi_record_log.create_queue("queue").unwrap(); 342 | assert_eq!( 343 | multi_record_log 344 | .append_record("queue", None, &b"1"[..]) 345 | .unwrap(), 346 | Some(0) 347 | ); 348 | } 349 | { 350 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 351 | assert_eq!( 352 | multi_record_log 353 | .append_record("queue", None, &b"22"[..]) 354 | .unwrap(), 355 | Some(1) 356 | ); 357 | } 358 | { 359 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 360 | multi_record_log.truncate("queue", ..=0).unwrap(); 361 | assert_eq!( 362 | multi_record_log 363 | .range("queue", ..) 364 | .unwrap() 365 | .collect::>(), 366 | [Record { 367 | position: 1, 368 | payload: Cow::Borrowed(&b"22"[..]) 369 | }], 370 | ); 371 | } 372 | { 373 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 374 | assert_eq!( 375 | multi_record_log 376 | .append_record("queue", None, &b"hello"[..]) 377 | .unwrap(), 378 | Some(2) 379 | ); 380 | } 381 | { 382 | let multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 383 | assert_eq!( 384 | multi_record_log 385 | .range("queue", ..) 386 | .unwrap() 387 | .collect::>(), 388 | [Record::new(1, b"22"), Record::new(2, b"hello"),] 389 | ); 390 | } 391 | } 392 | 393 | /// Unit tests reproducing bugs found with proptest in the past. 394 | #[test] 395 | fn test_proptest_multirecord_reproduce_1() { 396 | let block_size = 32_731; 397 | let mut env = PropTestEnv::new(block_size); 398 | env.apply(Operation::MultiAppend { 399 | queue: "q1", 400 | count: 4, 401 | skip_one_pos: false, 402 | }); 403 | env.apply(Operation::Truncate { 404 | queue: "q1", 405 | pos: 3, 406 | }); 407 | env.apply(Operation::Reopen {}); 408 | } 409 | -------------------------------------------------------------------------------- /src/record.rs: -------------------------------------------------------------------------------- 1 | use std::convert::{TryFrom, TryInto}; 2 | use std::ops::RangeToInclusive; 3 | 4 | use bytes::Buf; 5 | use tracing::error; 6 | 7 | use crate::error::MultiRecordCorruption; 8 | use crate::Serializable; 9 | 10 | #[derive(Copy, Clone, Eq, PartialEq)] 11 | pub(crate) enum MultiPlexedRecord<'a> { 12 | /// Adds multiple records to a specific queue. 13 | AppendRecords { 14 | queue: &'a str, 15 | position: u64, //< not used, the payload contain the position for each record 16 | records: MultiRecord<'a>, 17 | }, 18 | /// Records the truncation of a specific queue, up to and including the position. 19 | Truncate { 20 | queue: &'a str, 21 | truncate_range: RangeToInclusive, 22 | }, 23 | /// Records the next position of a given queue. 24 | /// If the queue does not exists, creates it. 25 | /// 26 | /// `position` is the position of the NEXT message to be appended. 27 | RecordPosition { queue: &'a str, position: u64 }, 28 | DeleteQueue { 29 | queue: &'a str, 30 | position: u64, //< not useful tbh 31 | }, 32 | } 33 | 34 | impl std::fmt::Debug for MultiPlexedRecord<'_> { 35 | fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { 36 | match self { 37 | Self::AppendRecords { 38 | queue, 39 | position, 40 | records, 41 | } => f 42 | .debug_struct("AppendRecords") 43 | .field("queue", queue) 44 | .field("position", position) 45 | .field("records_len", &records.count()) 46 | .finish(), 47 | Self::Truncate { 48 | queue, 49 | truncate_range, 50 | } => f 51 | .debug_struct("Truncate") 52 | .field("queue", queue) 53 | .field("range", truncate_range) 54 | .finish(), 55 | Self::RecordPosition { queue, position } => f 56 | .debug_struct("RecordPosition") 57 | .field("queue", queue) 58 | .field("position", position) 59 | .finish(), 60 | Self::DeleteQueue { queue, position } => f 61 | .debug_struct("DeleteQueue") 62 | .field("queue", queue) 63 | .field("position", position) 64 | .finish(), 65 | } 66 | } 67 | } 68 | 69 | impl<'a> MultiPlexedRecord<'a> { 70 | #[allow(dead_code)] 71 | pub fn queue_id(&self) -> &'a str { 72 | match self { 73 | Self::AppendRecords { queue, .. } => queue, 74 | Self::Truncate { queue, .. } => queue, 75 | Self::RecordPosition { queue, .. } => queue, 76 | Self::DeleteQueue { queue, .. } => queue, 77 | } 78 | } 79 | } 80 | 81 | #[repr(u8)] 82 | #[derive(Clone, Copy, Debug)] 83 | enum RecordType { 84 | Truncate = 1, 85 | Touch = 2, 86 | DeleteQueue = 3, 87 | AppendRecords = 4, 88 | } 89 | 90 | impl TryFrom for RecordType { 91 | type Error = (); 92 | 93 | fn try_from(code: u8) -> Result { 94 | match code { 95 | 1 => Ok(RecordType::Truncate), 96 | 2 => Ok(RecordType::Touch), 97 | 3 => Ok(RecordType::DeleteQueue), 98 | 4 => Ok(RecordType::AppendRecords), 99 | _ => Err(()), 100 | } 101 | } 102 | } 103 | 104 | fn serialize( 105 | record_type: RecordType, 106 | position: u64, 107 | queue: &str, 108 | payload: &[u8], 109 | buffer: &mut Vec, 110 | ) { 111 | assert!(queue.len() <= u16::MAX as usize); 112 | buffer.push(record_type as u8); 113 | buffer.extend_from_slice(&position.to_le_bytes()); 114 | buffer.extend_from_slice(&(queue.len() as u16).to_le_bytes()); 115 | buffer.extend_from_slice(queue.as_bytes()); 116 | buffer.extend(payload); 117 | } 118 | 119 | impl<'a> Serializable<'a> for MultiPlexedRecord<'a> { 120 | fn serialize(&self, buffer: &mut Vec) { 121 | buffer.clear(); 122 | match *self { 123 | MultiPlexedRecord::AppendRecords { 124 | position, 125 | queue, 126 | records, 127 | } => { 128 | serialize( 129 | RecordType::AppendRecords, 130 | position, 131 | queue, 132 | records.buffer, 133 | buffer, 134 | ); 135 | } 136 | 137 | MultiPlexedRecord::Truncate { 138 | queue, 139 | truncate_range, 140 | } => { 141 | serialize(RecordType::Truncate, truncate_range.end, queue, &[], buffer); 142 | } 143 | MultiPlexedRecord::RecordPosition { queue, position } => { 144 | serialize(RecordType::Touch, position, queue, &[], buffer); 145 | } 146 | MultiPlexedRecord::DeleteQueue { position, queue } => { 147 | serialize(RecordType::DeleteQueue, position, queue, &[], buffer); 148 | } 149 | } 150 | } 151 | 152 | fn deserialize(buffer: &'a [u8]) -> Option> { 153 | const HEADER_LEN: usize = 11; 154 | if buffer.len() < HEADER_LEN { 155 | error!(buffer=?buffer, "multiplexed record buffer too short"); 156 | return None; 157 | } 158 | let (header, body) = buffer.split_at(HEADER_LEN); 159 | let enum_tag = RecordType::try_from(header[0]).ok()?; 160 | let position = u64::from_le_bytes(header[1..9].try_into().unwrap()); 161 | let queue_len = u16::from_le_bytes(header[9..HEADER_LEN].try_into().unwrap()) as usize; 162 | if body.len() < queue_len { 163 | error!( 164 | queue_len = queue_len, 165 | body_len = body.len(), 166 | "record body too short" 167 | ); 168 | return None; 169 | } 170 | let (queue_bytes, payload) = body.split_at(queue_len); 171 | let Ok(queue) = std::str::from_utf8(queue_bytes) else { 172 | let truncated_len = queue_bytes.len().min(10); 173 | error!(queue_name_bytes=?queue_bytes[..truncated_len], "non-utf8 queue name"); 174 | return None; 175 | }; 176 | match enum_tag { 177 | RecordType::AppendRecords => Some(MultiPlexedRecord::AppendRecords { 178 | queue, 179 | position, 180 | records: MultiRecord::new(payload).ok()?, 181 | }), 182 | RecordType::Truncate => Some(MultiPlexedRecord::Truncate { 183 | queue, 184 | truncate_range: ..=position, 185 | }), 186 | RecordType::Touch => Some(MultiPlexedRecord::RecordPosition { queue, position }), 187 | RecordType::DeleteQueue => Some(MultiPlexedRecord::DeleteQueue { queue, position }), 188 | } 189 | } 190 | } 191 | 192 | #[derive(Debug, Copy, Clone, Eq, PartialEq)] 193 | pub(crate) struct MultiRecord<'a> { 194 | /// The buffer contains concatenated items following this pattern: 195 | /// 196 | /// The two integers are encoded as little endian. 197 | buffer: &'a [u8], 198 | /// Offset into the buffer above used while iterating over the serialized items. 199 | byte_offset: usize, 200 | } 201 | 202 | impl MultiRecord<'_> { 203 | pub fn new(buffer: &[u8]) -> Result { 204 | let mut mrecord = MultiRecord::new_unchecked(buffer); 205 | 206 | // verify the content is not corrupted 207 | for record in mrecord { 208 | record?; 209 | } 210 | 211 | mrecord.reset_position(); 212 | 213 | Ok(mrecord) 214 | } 215 | 216 | pub fn new_unchecked(buffer: &[u8]) -> MultiRecord { 217 | MultiRecord { 218 | buffer, 219 | byte_offset: 0, 220 | } 221 | } 222 | 223 | pub fn serialize>( 224 | record_payloads: T, 225 | position: u64, 226 | output: &mut Vec, 227 | ) { 228 | Self::serialize_with_pos((position..).zip(record_payloads), output); 229 | } 230 | 231 | fn serialize_with_pos( 232 | record_payloads: impl Iterator, 233 | output: &mut Vec, 234 | ) { 235 | output.clear(); 236 | for (position, mut record_payload) in record_payloads { 237 | assert!(record_payload.remaining() <= u32::MAX as usize); 238 | // TODO add assert for position monotonicity? 239 | let record_payload = &mut record_payload; 240 | output.extend_from_slice(&position.to_le_bytes()); 241 | output.extend_from_slice(&(record_payload.remaining() as u32).to_le_bytes()); 242 | while record_payload.has_remaining() { 243 | let chunk = record_payload.chunk(); 244 | output.extend_from_slice(record_payload.chunk()); 245 | record_payload.advance(chunk.len()); 246 | } 247 | } 248 | } 249 | 250 | pub fn reset_position(&mut self) { 251 | self.byte_offset = 0; 252 | } 253 | } 254 | 255 | impl<'a> Iterator for MultiRecord<'a> { 256 | type Item = Result<(u64, &'a [u8]), MultiRecordCorruption>; 257 | 258 | fn next(&mut self) -> Option { 259 | if self.byte_offset == self.buffer.len() { 260 | // no more record 261 | return None; 262 | } 263 | const HEADER_LEN: usize = 12; 264 | let buffer = &self.buffer[self.byte_offset..]; 265 | if buffer.len() < HEADER_LEN { 266 | // too short: corrupted 267 | self.byte_offset = buffer.len(); 268 | return Some(Err(MultiRecordCorruption)); 269 | } 270 | 271 | let position = u64::from_le_bytes(buffer[0..8].try_into().unwrap()); 272 | let len = u32::from_le_bytes(buffer[8..HEADER_LEN].try_into().unwrap()) as usize; 273 | 274 | let buffer = &buffer[HEADER_LEN..]; 275 | 276 | if buffer.len() < len { 277 | self.byte_offset = buffer.len(); 278 | return Some(Err(MultiRecordCorruption)); 279 | } 280 | 281 | self.byte_offset += HEADER_LEN + len; 282 | 283 | Some(Ok((position, &buffer[..len]))) 284 | } 285 | } 286 | 287 | #[cfg(test)] 288 | mod tests { 289 | use std::convert::TryFrom; 290 | 291 | use super::{MultiPlexedRecord, MultiRecord, RecordType}; 292 | use crate::Serializable; 293 | 294 | #[test] 295 | fn test_record_type_serialize() { 296 | let mut num_record_types = 0; 297 | for code in 0u8..=255u8 { 298 | if let Ok(record_type) = RecordType::try_from(code) { 299 | assert_eq!(record_type as u8, code); 300 | num_record_types += 1; 301 | } 302 | } 303 | assert_eq!(num_record_types, 4); 304 | } 305 | 306 | #[test] 307 | fn test_multirecord_deserialization_ok() { 308 | let mut buffer: Vec = vec![]; 309 | MultiRecord::serialize( 310 | [b"123".as_slice(), b"4567".as_slice()].into_iter(), 311 | 5, 312 | &mut buffer, 313 | ); 314 | match MultiRecord::new(&buffer) { 315 | Err(_) => panic!("Parsing serialized buffers should work"), 316 | Ok(record) => { 317 | let items: Vec<_> = record 318 | .into_iter() 319 | .map(|item| item.expect("Deserializing item should work")) 320 | .collect(); 321 | assert_eq!( 322 | items, 323 | vec![(5u64, b"123".as_slice()), (6u64, b"4567".as_slice())] 324 | ); 325 | } 326 | } 327 | } 328 | 329 | #[test] 330 | fn test_multirecord_deserialization_corruption() { 331 | let mut buffer: Vec = vec![]; 332 | MultiRecord::serialize( 333 | [b"123".as_slice(), b"4567".as_slice()].into_iter(), 334 | 5, 335 | &mut buffer, 336 | ); 337 | let mut num_errors = 0; 338 | for num_truncated_bytes in 1..buffer.len() { 339 | // This should not panic. Typically, this will be an error, but 340 | // deserializing can also succeed (but will have wrong data). 341 | num_errors += 342 | MultiRecord::new(&buffer[..buffer.len() - num_truncated_bytes]).is_err() as i32; 343 | } 344 | assert!(num_errors >= 1); 345 | } 346 | 347 | #[test] 348 | fn test_multiplexedrecord_deserialization_ok() { 349 | let mut buffer_multirecord: Vec = vec![]; 350 | MultiRecord::serialize([b"123".as_slice()].into_iter(), 2, &mut buffer_multirecord); 351 | let record = MultiPlexedRecord::AppendRecords { 352 | queue: "queue_name", 353 | position: 10, 354 | records: MultiRecord::new_unchecked(&buffer_multirecord), 355 | }; 356 | let mut buffer_multiplexed: Vec = vec![]; 357 | record.serialize(&mut buffer_multiplexed); 358 | match MultiPlexedRecord::deserialize(&buffer_multiplexed) { 359 | None => panic!("Deserialization should work"), 360 | Some(parsed_record) => assert_eq!(parsed_record, record), 361 | } 362 | } 363 | 364 | #[test] 365 | fn test_multiplexedrecord_deserialization_corruption() { 366 | let mut buffer_multirecord: Vec = vec![]; 367 | MultiRecord::serialize([b"123".as_slice()].into_iter(), 2, &mut buffer_multirecord); 368 | let record = MultiPlexedRecord::AppendRecords { 369 | queue: "queue_name", 370 | position: 10, 371 | records: MultiRecord::new_unchecked(&buffer_multirecord), 372 | }; 373 | let mut buffer_multiplexed: Vec = vec![]; 374 | record.serialize(&mut buffer_multiplexed); 375 | 376 | let mut num_errors = 0; 377 | for num_truncated_bytes in 1..buffer_multiplexed.len() { 378 | // This should not panic. Typically, this will be an error, but 379 | // deserializing can also succeed (but will have wrong data). 380 | num_errors += MultiPlexedRecord::deserialize( 381 | &buffer_multiplexed[..buffer_multiplexed.len() - num_truncated_bytes], 382 | ) 383 | .is_none() as i32; 384 | } 385 | assert!(num_errors >= 1); 386 | } 387 | } 388 | -------------------------------------------------------------------------------- /src/recordlog/mod.rs: -------------------------------------------------------------------------------- 1 | mod reader; 2 | mod writer; 3 | pub use self::reader::RecordReader; 4 | pub use self::writer::RecordWriter; 5 | 6 | #[cfg(test)] 7 | mod tests; 8 | -------------------------------------------------------------------------------- /src/recordlog/reader.rs: -------------------------------------------------------------------------------- 1 | use std::io; 2 | 3 | use crate::error::ReadRecordError; 4 | use crate::frame::{FrameReader, FrameWriter, ReadFrameError}; 5 | use crate::recordlog::RecordWriter; 6 | use crate::rolling::{RollingReader, RollingWriter}; 7 | use crate::{BlockRead, Serializable}; 8 | 9 | pub struct RecordReader { 10 | frame_reader: FrameReader, 11 | record_buffer: Vec, 12 | // true if we are in the middle of reading a multifragment record. 13 | // This is useful, as it makes it possible to drop a record 14 | // if one of its fragment was corrupted. 15 | within_record: bool, 16 | } 17 | 18 | impl RecordReader { 19 | pub fn open(reader: R) -> Self { 20 | let frame_reader = FrameReader::open(reader); 21 | RecordReader { 22 | frame_reader, 23 | record_buffer: Vec::with_capacity(10_000), 24 | within_record: false, 25 | } 26 | } 27 | 28 | pub fn read(&self) -> &R { 29 | self.frame_reader.read() 30 | } 31 | 32 | /// Deserialize a record without actually consuming data. 33 | pub fn record<'a, S: Serializable<'a>>(&'a self) -> Option { 34 | S::deserialize(&self.record_buffer) 35 | } 36 | 37 | /// Advance cursor and deserialize the next record. 38 | pub fn read_record<'a, S: Serializable<'a>>( 39 | &'a mut self, 40 | ) -> Result, ReadRecordError> { 41 | let has_record = self.go_next()?; 42 | if has_record { 43 | let record = self.record().ok_or(ReadRecordError::Corruption)?; 44 | Ok(Some(record)) 45 | } else { 46 | Ok(None) 47 | } 48 | } 49 | 50 | // Attempts to position the reader to the next record and return 51 | // true or false whether such a record is available or not. 52 | pub fn go_next(&mut self) -> Result { 53 | loop { 54 | let frame = self.frame_reader.read_frame(); 55 | match frame { 56 | Ok((frame_type, frame_payload)) => { 57 | if frame_type.is_first_frame_of_record() { 58 | self.within_record = true; 59 | self.record_buffer.clear(); 60 | } 61 | if self.within_record { 62 | self.record_buffer.extend_from_slice(frame_payload); 63 | if frame_type.is_last_frame_of_record() { 64 | self.within_record = false; 65 | return Ok(true); 66 | } 67 | } 68 | } 69 | Err(ReadFrameError::Corruption) => { 70 | self.within_record = false; 71 | return Err(ReadRecordError::Corruption); 72 | } 73 | Err(ReadFrameError::IoError(io_err)) => { 74 | self.within_record = false; 75 | return Err(ReadRecordError::IoError(io_err)); 76 | } 77 | Err(ReadFrameError::NotAvailable) => { 78 | return Ok(false); 79 | } 80 | } 81 | } 82 | } 83 | } 84 | 85 | impl RecordReader { 86 | pub fn into_writer(self) -> io::Result> { 87 | let frame_writer: FrameWriter = self.frame_reader.into_writer()?; 88 | Ok(RecordWriter::from(frame_writer)) 89 | } 90 | } 91 | -------------------------------------------------------------------------------- /src/recordlog/tests.rs: -------------------------------------------------------------------------------- 1 | use super::{RecordReader, RecordWriter}; 2 | use crate::block_read_write::ArrayReader; 3 | use crate::error::ReadRecordError; 4 | use crate::frame::HEADER_LEN; 5 | use crate::{PersistAction, BLOCK_NUM_BYTES}; 6 | 7 | #[test] 8 | fn test_no_data() { 9 | let data = vec![0u8; BLOCK_NUM_BYTES * 4]; 10 | let mut reader = RecordReader::open(ArrayReader::from(&data[..])); 11 | assert_eq!(reader.read_record::<&str>().unwrap(), None); 12 | } 13 | 14 | #[test] 15 | fn test_empty_record() { 16 | let mut writer = RecordWriter::in_memory(); 17 | writer.write_record("").unwrap(); 18 | writer.persist(PersistAction::Flush).unwrap(); 19 | let buf: Vec = writer.into_writer().into(); 20 | let mut reader = RecordReader::open(ArrayReader::from(&buf[..])); 21 | assert_eq!(reader.read_record::<&str>().unwrap(), Some("")); 22 | assert_eq!(reader.read_record::<&str>().unwrap(), None); 23 | } 24 | 25 | #[test] 26 | fn test_simple_record() { 27 | let mut writer = RecordWriter::in_memory(); 28 | let record = "hello"; 29 | writer.write_record(record).unwrap(); 30 | writer.persist(PersistAction::Flush).unwrap(); 31 | let buf: Vec = writer.into_writer().into(); 32 | let mut reader = RecordReader::open(ArrayReader::from(&buf[..])); 33 | assert!(matches!(reader.read_record::<&str>(), Ok(Some("hello")))); 34 | assert!(matches!(reader.read_record::<&str>(), Ok(None))); 35 | } 36 | 37 | fn make_long_entry(len: usize) -> String { 38 | "A".repeat(len) 39 | } 40 | 41 | #[test] 42 | fn test_spans_over_more_than_one_block() { 43 | let long_entry: String = make_long_entry(80_000); 44 | let mut writer = RecordWriter::in_memory(); 45 | writer.write_record(long_entry.as_str()).unwrap(); 46 | writer.persist(PersistAction::Flush).unwrap(); 47 | let buf: Vec = writer.into_writer().into(); 48 | let mut reader = RecordReader::open(ArrayReader::from(&buf[..])); 49 | let record_payload: &str = reader.read_record().unwrap().unwrap(); 50 | assert_eq!(record_payload, &long_entry); 51 | assert_eq!(reader.read_record::<&str>().unwrap(), None); 52 | } 53 | 54 | #[test] 55 | fn test_block_requires_padding() { 56 | // We'll miss 1 byte to be able to fit our next chunk header in the 57 | // first block. 58 | let long_record = make_long_entry(BLOCK_NUM_BYTES - HEADER_LEN - HEADER_LEN - 1 - 8); 59 | let short_record = "hello"; 60 | let mut writer = RecordWriter::in_memory(); 61 | writer.write_record(long_record.as_str()).unwrap(); 62 | writer.write_record(short_record).unwrap(); 63 | writer.persist(PersistAction::Flush).unwrap(); 64 | let buffer: Vec = writer.into_writer().into(); 65 | let mut reader = RecordReader::open(ArrayReader::from(&buffer[..])); 66 | assert_eq!( 67 | reader.read_record::<&str>().unwrap(), 68 | Some(long_record.as_str()) 69 | ); 70 | assert_eq!(reader.read_record::<&str>().unwrap(), Some(short_record)); 71 | assert_eq!(reader.read_record::<&str>().unwrap(), None); 72 | } 73 | 74 | #[test] 75 | fn test_first_chunk_empty() { 76 | // We'll miss 1 byte to be able to fit our next chunk header in the 77 | // first block. 78 | let long_record = make_long_entry(BLOCK_NUM_BYTES - HEADER_LEN - HEADER_LEN); 79 | let short_record = "hello"; 80 | let mut writer = RecordWriter::in_memory(); 81 | writer.write_record(&long_record[..]).unwrap(); 82 | writer.write_record(short_record).unwrap(); 83 | writer.persist(PersistAction::Flush).unwrap(); 84 | let buf: Vec = writer.into_writer().into(); 85 | let mut reader = RecordReader::open(ArrayReader::from(&buf[..])); 86 | assert_eq!( 87 | reader.read_record::<&str>().unwrap(), 88 | Some(long_record.as_str()) 89 | ); 90 | assert_eq!(reader.read_record::<&str>().unwrap(), Some(short_record)); 91 | assert_eq!(reader.read_record::<&str>().unwrap(), None); 92 | } 93 | 94 | #[test] 95 | fn test_behavior_upon_corruption() { 96 | let records: Vec = (0..1_000).map(|i| format!("hello{i}")).collect(); 97 | let mut writer = RecordWriter::in_memory(); 98 | for record in &records { 99 | writer.write_record(record.as_str()).unwrap(); 100 | } 101 | writer.persist(PersistAction::Flush).unwrap(); 102 | let mut buffer: Vec = writer.into_writer().into(); 103 | { 104 | let mut reader = RecordReader::open(ArrayReader::from(&buffer[..])); 105 | for record in &records { 106 | assert_eq!(reader.read_record::<&str>().unwrap(), Some(record.as_str())); 107 | } 108 | assert_eq!(reader.read_record::<&str>().unwrap(), None); 109 | } 110 | // Introducing a corruption. 111 | buffer[1_000] = 3; 112 | { 113 | let mut reader = RecordReader::open(ArrayReader::from(&buffer[..])); 114 | for record in &records[0..72] { 115 | // bug at i=72 116 | assert_eq!(reader.read_record::<&str>().unwrap(), Some(record.as_str())); 117 | } 118 | assert!(matches!( 119 | reader.read_record::<&str>(), 120 | Err(ReadRecordError::Corruption) 121 | )); 122 | } 123 | } 124 | -------------------------------------------------------------------------------- /src/recordlog/writer.rs: -------------------------------------------------------------------------------- 1 | use std::io; 2 | 3 | use crate::block_read_write::VecBlockWriter; 4 | use crate::frame::{FrameType, FrameWriter}; 5 | use crate::rolling::{Directory, FileNumber, RollingWriter}; 6 | use crate::{BlockWrite, PersistAction, Serializable}; 7 | 8 | pub struct RecordWriter { 9 | frame_writer: FrameWriter, 10 | buffer: Vec, 11 | } 12 | 13 | fn frame_type(is_first_frame: bool, is_last_frame: bool) -> FrameType { 14 | match (is_first_frame, is_last_frame) { 15 | (true, true) => FrameType::Full, 16 | (true, false) => FrameType::First, 17 | (false, true) => FrameType::Last, 18 | (false, false) => FrameType::Middle, 19 | } 20 | } 21 | 22 | impl From> for RecordWriter { 23 | fn from(frame_writer: FrameWriter) -> Self { 24 | RecordWriter { 25 | frame_writer, 26 | buffer: Vec::with_capacity(10_000), 27 | } 28 | } 29 | } 30 | 31 | impl RecordWriter { 32 | #[cfg(test)] 33 | pub fn into_writer(self) -> W { 34 | self.frame_writer.into_writer() 35 | } 36 | } 37 | 38 | impl RecordWriter { 39 | /// Writes a record. 40 | /// 41 | /// Even if this call returns `Ok(())`, at this point the data 42 | /// is likely to be not durably stored on disk. 43 | /// 44 | /// For instance, the data could be stale in a library level buffer, 45 | /// by a writer level buffer, or an application buffer, 46 | /// or could not be flushed to disk yet by the OS. 47 | pub fn write_record<'a>(&mut self, record: impl Serializable<'a>) -> io::Result<()> { 48 | let mut is_first_frame = true; 49 | self.buffer.clear(); 50 | record.serialize(&mut self.buffer); 51 | let mut payload = &self.buffer[..]; 52 | loop { 53 | let frame_payload_len = self 54 | .frame_writer 55 | .max_writable_frame_length() 56 | .min(payload.len()); 57 | let frame_payload = &payload[..frame_payload_len]; 58 | payload = &payload[frame_payload_len..]; 59 | let is_last_frame = payload.is_empty(); 60 | let frame_type = frame_type(is_first_frame, is_last_frame); 61 | self.frame_writer.write_frame(frame_type, frame_payload)?; 62 | is_first_frame = false; 63 | if is_last_frame { 64 | break; 65 | } 66 | } 67 | Ok(()) 68 | } 69 | 70 | /// Persist the data to disk, according to the persist_action. 71 | pub fn persist(&mut self, persist_action: PersistAction) -> io::Result<()> { 72 | self.frame_writer.persist(persist_action) 73 | } 74 | 75 | pub fn get_underlying_wrt(&self) -> &W { 76 | self.frame_writer.get_underlying_wrt() 77 | } 78 | } 79 | 80 | impl RecordWriter { 81 | pub fn directory(&mut self) -> &mut Directory { 82 | self.frame_writer.directory() 83 | } 84 | 85 | pub fn current_file(&mut self) -> &FileNumber { 86 | self.get_underlying_wrt().current_file() 87 | } 88 | 89 | pub fn size(&self) -> usize { 90 | self.get_underlying_wrt().size() 91 | } 92 | } 93 | 94 | impl RecordWriter { 95 | #[cfg(test)] 96 | pub fn in_memory() -> Self { 97 | FrameWriter::create(VecBlockWriter::default()).into() 98 | } 99 | } 100 | -------------------------------------------------------------------------------- /src/rolling/directory.rs: -------------------------------------------------------------------------------- 1 | use std::fs::{File, OpenOptions}; 2 | use std::io::{self, BufWriter, Read, Seek, SeekFrom, Write}; 3 | use std::path::{Path, PathBuf}; 4 | 5 | use tracing::info; 6 | 7 | use super::{FileNumber, FileTracker}; 8 | use crate::rolling::{FILE_NUM_BYTES, FRAME_NUM_BYTES}; 9 | use crate::{BlockRead, BlockWrite, PersistAction, BLOCK_NUM_BYTES}; 10 | 11 | pub struct Directory { 12 | dir: PathBuf, 13 | pub(crate) files: FileTracker, 14 | } 15 | 16 | fn filename_to_position(file_name: &str) -> Option { 17 | if file_name.len() != 24 { 18 | return None; 19 | } 20 | if !file_name.starts_with("wal-") { 21 | return None; 22 | } 23 | let seq_number_str = &file_name[4..]; 24 | if !seq_number_str.as_bytes().iter().all(u8::is_ascii_digit) { 25 | return None; 26 | } 27 | file_name[4..].parse::().ok() 28 | } 29 | 30 | pub(crate) fn filepath(dir: &Path, file_number: &FileNumber) -> PathBuf { 31 | dir.join(file_number.filename()) 32 | } 33 | 34 | fn create_file(dir_path: &Path, file_number: &FileNumber) -> io::Result { 35 | let new_filepath = filepath(dir_path, file_number); 36 | let mut file = OpenOptions::new() 37 | .create_new(true) 38 | .write(true) 39 | .open(new_filepath)?; 40 | file.set_len(FILE_NUM_BYTES as u64)?; 41 | file.seek(SeekFrom::Start(0))?; 42 | Ok(file) 43 | } 44 | 45 | impl Directory { 46 | /// Open a `Directory`, or create a new, empty, one. `dir_path` must exist and be a directory. 47 | pub fn open(dir_path: &Path) -> io::Result { 48 | let mut file_numbers: Vec = Default::default(); 49 | for dir_entry_res in std::fs::read_dir(dir_path)? { 50 | let dir_entry = dir_entry_res?; 51 | if !dir_entry.file_type()?.is_file() { 52 | continue; 53 | } 54 | let file_name = if let Some(file_name) = dir_entry.file_name().to_str() { 55 | file_name.to_string() 56 | } else { 57 | continue; 58 | }; 59 | if let Some(seq_number) = filename_to_position(&file_name) { 60 | file_numbers.push(seq_number); 61 | } 62 | } 63 | let files = if let Some(files) = FileTracker::from_file_numbers(file_numbers) { 64 | files 65 | } else { 66 | let files = FileTracker::new(); 67 | let file_number = files.first(); 68 | create_file(dir_path, file_number)?; 69 | files 70 | }; 71 | Ok(Directory { 72 | dir: dir_path.to_path_buf(), 73 | files, 74 | }) 75 | } 76 | 77 | /// Get the first still used FileNumber. 78 | pub fn first_file_number(&self) -> &FileNumber { 79 | self.files.first() 80 | } 81 | 82 | /// Returns true if some file could be GCed. 83 | pub fn has_files_that_can_be_deleted(&self) -> bool { 84 | self.files.count() >= 2 && self.files.first().can_be_deleted() 85 | } 86 | 87 | /// Delete FileNumbers and the associated wal files no longer used. 88 | /// 89 | /// We never delete the last file. 90 | pub(crate) fn gc(&mut self) -> io::Result<()> { 91 | while let Some(file) = self.files.take_first_unused() { 92 | let filepath = filepath(&self.dir, &file); 93 | info!(file=%filepath.display(), "gc remove file"); 94 | std::fs::remove_file(&filepath)?; 95 | } 96 | Ok(()) 97 | } 98 | 99 | /// Open the wal file with the provided FileNumber. 100 | pub fn open_file(&self, file_number: &FileNumber) -> io::Result { 101 | let filepath = filepath(&self.dir, file_number); 102 | let mut file = OpenOptions::new().read(true).write(true).open(filepath)?; 103 | file.seek(SeekFrom::Start(0u64))?; 104 | Ok(file) 105 | } 106 | 107 | fn sync_directory(&self) -> io::Result<()> { 108 | let mut open_opts = OpenOptions::new(); 109 | // Linux needs read to be set, otherwise returns EINVAL 110 | // write must not be set, or it fails with EISDIR 111 | open_opts.read(true); 112 | let fd = open_opts.open(&self.dir)?; 113 | fd.sync_data()?; 114 | Ok(()) 115 | } 116 | } 117 | 118 | pub struct RollingReader { 119 | file: File, 120 | directory: Directory, 121 | file_number: FileNumber, 122 | block_id: usize, 123 | block: Box<[u8; BLOCK_NUM_BYTES]>, 124 | } 125 | 126 | impl RollingReader { 127 | /// Open a directory for reading. 128 | pub fn open(dir_path: &Path) -> io::Result { 129 | let directory = Directory::open(dir_path)?; 130 | let first_file = directory.first_file_number().clone(); 131 | let mut file = directory.open_file(&first_file)?; 132 | let mut block = Box::new([0u8; BLOCK_NUM_BYTES]); 133 | file.read_exact(&mut *block)?; 134 | Ok(RollingReader { 135 | file, 136 | directory, 137 | file_number: first_file.clone(), 138 | block_id: 0, 139 | block, 140 | }) 141 | } 142 | 143 | pub fn current_file(&self) -> &FileNumber { 144 | &self.file_number 145 | } 146 | 147 | /// Creates a write positioned at the beginning of the last read block. 148 | /// 149 | /// If no block was read, positions itself at the beginning. 150 | pub fn into_writer(mut self) -> io::Result { 151 | let offset = self.block_id * crate::BLOCK_NUM_BYTES; 152 | self.file.seek(SeekFrom::Start(offset as u64))?; 153 | Ok(RollingWriter { 154 | file: BufWriter::with_capacity(FRAME_NUM_BYTES, self.file), 155 | offset, 156 | file_number: self.file_number.clone(), 157 | directory: self.directory, 158 | }) 159 | } 160 | } 161 | 162 | fn read_block(file: &mut File, block: &mut [u8; BLOCK_NUM_BYTES]) -> io::Result { 163 | match file.read_exact(block) { 164 | Ok(()) => Ok(true), 165 | Err(io_err) if io_err.kind() == io::ErrorKind::UnexpectedEof => Ok(false), 166 | Err(io_err) => Err(io_err), 167 | } 168 | } 169 | 170 | impl BlockRead for RollingReader { 171 | fn next_block(&mut self) -> io::Result { 172 | let success = read_block(&mut self.file, &mut self.block)?; 173 | if success { 174 | self.block_id += 1; 175 | return Ok(true); 176 | } 177 | 178 | let mut next_file_number = 179 | if let Some(next_file_number) = self.directory.files.next(&self.file_number) { 180 | next_file_number 181 | } else { 182 | return Ok(false); 183 | }; 184 | 185 | loop { 186 | let mut next_file: File = self.directory.open_file(&next_file_number)?; 187 | let success = read_block(&mut next_file, &mut self.block)?; 188 | if success { 189 | self.block_id = 0; 190 | self.file = next_file; 191 | self.file_number = next_file_number; 192 | return Ok(true); 193 | } 194 | 195 | next_file_number = 196 | if let Some(next_file_number) = self.directory.files.next(&next_file_number) { 197 | next_file_number 198 | } else { 199 | return Ok(false); 200 | }; 201 | } 202 | } 203 | 204 | fn block(&self) -> &[u8; BLOCK_NUM_BYTES] { 205 | &self.block 206 | } 207 | } 208 | 209 | pub struct RollingWriter { 210 | file: BufWriter, 211 | offset: usize, 212 | file_number: FileNumber, 213 | pub(crate) directory: Directory, 214 | } 215 | 216 | impl RollingWriter { 217 | /// Move forward of `num_bytes` without actually writing anything. 218 | pub fn forward(&mut self, num_bytes: usize) -> io::Result<()> { 219 | self.file.seek(SeekFrom::Current(num_bytes as i64))?; 220 | self.offset += num_bytes; 221 | Ok(()) 222 | } 223 | 224 | pub fn current_file(&self) -> &FileNumber { 225 | &self.file_number 226 | } 227 | 228 | pub fn size(&self) -> usize { 229 | self.directory.files.count() * FILE_NUM_BYTES 230 | } 231 | 232 | #[cfg(test)] 233 | pub fn list_file_numbers(&self) -> Vec { 234 | self.directory 235 | .first_file_number() 236 | .unroll(&self.directory.files) 237 | } 238 | } 239 | 240 | impl BlockWrite for RollingWriter { 241 | fn write(&mut self, buf: &[u8]) -> io::Result<()> { 242 | if buf.is_empty() { 243 | return Ok(()); 244 | } 245 | assert!(buf.len() <= self.num_bytes_remaining_in_block()); 246 | if self.offset + buf.len() > FILE_NUM_BYTES { 247 | self.file.flush()?; 248 | self.file.get_ref().sync_data()?; 249 | self.directory.sync_directory()?; 250 | 251 | let (file_number, file) = 252 | if let Some(next_file_number) = self.directory.files.next(&self.file_number) { 253 | let file = self.directory.open_file(&next_file_number)?; 254 | (next_file_number, file) 255 | } else { 256 | let next_file_number = self.directory.files.inc(&self.file_number); 257 | let file = create_file(&self.directory.dir, &next_file_number)?; 258 | (next_file_number, file) 259 | }; 260 | 261 | self.file = BufWriter::with_capacity(FRAME_NUM_BYTES, file); 262 | self.file_number = file_number; 263 | self.offset = 0; 264 | } 265 | self.offset += buf.len(); 266 | self.file.write_all(buf)?; 267 | Ok(()) 268 | } 269 | 270 | fn persist(&mut self, persist_action: PersistAction) -> io::Result<()> { 271 | match persist_action { 272 | PersistAction::FlushAndFsync => { 273 | self.file.flush()?; 274 | self.file.get_ref().sync_data()?; 275 | self.directory.sync_directory() 276 | } 277 | PersistAction::Flush => { 278 | // This will flush the buffer of the BufWriter to the underlying OS. 279 | self.file.flush() 280 | } 281 | } 282 | } 283 | 284 | fn num_bytes_remaining_in_block(&self) -> usize { 285 | BLOCK_NUM_BYTES - (self.offset % BLOCK_NUM_BYTES) 286 | } 287 | } 288 | 289 | #[cfg(test)] 290 | mod tests { 291 | use super::filename_to_position; 292 | 293 | #[test] 294 | fn test_filename_to_seq_number_invalid_prefix_rejected() { 295 | assert_eq!(filename_to_position("fil-00000000000000000001"), None); 296 | } 297 | 298 | #[test] 299 | fn test_filename_to_seq_number_invalid_padding_rejected() { 300 | assert_eq!(filename_to_position("wal-0000000000000000001"), None); 301 | } 302 | 303 | #[test] 304 | fn test_filename_to_seq_number_invalid_len_rejected() { 305 | assert_eq!(filename_to_position("wal-000000000000000000011"), None); 306 | } 307 | 308 | #[test] 309 | fn test_filename_to_seq_number_simple() { 310 | assert_eq!(filename_to_position("wal-00000000000000000001"), Some(1)); 311 | } 312 | 313 | #[test] 314 | fn test_filename_to_seq_number() { 315 | assert_eq!(filename_to_position("wal-00000000000000000001"), Some(1)); 316 | } 317 | 318 | #[test] 319 | fn test_filename_to_seq_number_33b() { 320 | // 2**32, overflow a u32 321 | assert_eq!( 322 | filename_to_position("wal-00000000004294967296"), 323 | Some(4294967296) 324 | ); 325 | } 326 | 327 | #[test] 328 | fn test_filename_to_seq_number_64b() { 329 | // 2**64-1, max supported value 330 | assert_eq!( 331 | filename_to_position(&format!("wal-{}", u64::MAX)), 332 | Some(u64::MAX) 333 | ); 334 | } 335 | } 336 | -------------------------------------------------------------------------------- /src/rolling/file_number.rs: -------------------------------------------------------------------------------- 1 | use std::collections::BTreeSet; 2 | use std::sync::Arc; 3 | 4 | /// RefCount a set of ordered files. Always track at least one file. 5 | pub struct FileTracker { 6 | files: BTreeSet, 7 | } 8 | 9 | impl FileTracker { 10 | /// Create a new FileTracker tracking a single file. 11 | pub fn new() -> FileTracker { 12 | FileTracker::from_file_numbers(vec![0]).unwrap() 13 | } 14 | 15 | /// Get the first FileNumber still tracked 16 | pub fn first(&self) -> &FileNumber { 17 | self.files.first().unwrap() 18 | } 19 | 20 | /// Remove the oldest tracked file if it is no longer used 21 | /// 22 | /// By design the last file is always considered used. 23 | pub fn take_first_unused(&mut self) -> Option { 24 | // correctness note: this takes a &mut self, so we know there can't be a &FileNumber 25 | // referencing inside self while this is called. 26 | 27 | // if len is 1, we need to keep that element to keep self.files not empty 28 | if self.files.len() < 2 { 29 | return None; 30 | } 31 | 32 | let first = self.files.first().unwrap(); 33 | if first.can_be_deleted() { 34 | self.files.pop_first() 35 | } else { 36 | None 37 | } 38 | } 39 | 40 | /// Get the FileNumber directly after `curr` if it already exists. 41 | pub fn next(&self, curr: &FileNumber) -> Option { 42 | use std::ops::Bound::{Excluded, Unbounded}; 43 | self.files 44 | .range((Excluded(*curr.file_number), Unbounded)) 45 | .next() 46 | .cloned() 47 | } 48 | 49 | /// Get the FileNumber directly after `curr`, creating it if it doesn't exist yet. 50 | pub fn inc(&mut self, curr: &FileNumber) -> FileNumber { 51 | use std::ops::Bound::{Excluded, Unbounded}; 52 | if let Some(file) = self 53 | .files 54 | .range((Excluded(*curr.file_number), Unbounded)) 55 | .next() 56 | { 57 | return file.clone(); 58 | } 59 | let new_number = *curr.file_number + 1u64; 60 | let new_file_number = FileNumber::new(new_number); 61 | self.files.insert(new_file_number.clone()); 62 | new_file_number 63 | } 64 | 65 | /// Create a FileTracker from a list of file id to track. 66 | pub fn from_file_numbers(file_numbers: Vec) -> Option { 67 | if file_numbers.is_empty() { 68 | return None; 69 | } 70 | 71 | let files = file_numbers.into_iter().map(FileNumber::new).collect(); 72 | 73 | Some(FileTracker { files }) 74 | } 75 | 76 | /// Return the number of file tracked. 77 | pub fn count(&self) -> usize { 78 | self.files.len() 79 | } 80 | } 81 | 82 | #[derive(Clone, Default, Debug, Ord, PartialOrd, Eq, PartialEq)] 83 | pub struct FileNumber { 84 | file_number: Arc, 85 | } 86 | 87 | impl FileNumber { 88 | fn new(file_number: u64) -> Self { 89 | FileNumber { 90 | file_number: Arc::new(file_number), 91 | } 92 | } 93 | 94 | /// Returns whether there is no clone of this FileNumber in existance. 95 | /// 96 | /// /!\ care should be taken to not have some other code store a &FileNumber which could alias 97 | /// with self as it might then be sementically incorrect to delete content based only on this 98 | /// returning `true`. 99 | pub fn can_be_deleted(&self) -> bool { 100 | Arc::strong_count(&self.file_number) == 1 101 | } 102 | 103 | #[cfg(test)] 104 | pub fn unroll(&self, tracker: &FileTracker) -> Vec { 105 | let mut file = self.clone(); 106 | let mut file_numbers = Vec::new(); 107 | loop { 108 | file_numbers.push(file.file_number()); 109 | if let Some(next_file) = tracker.next(&file) { 110 | file = next_file; 111 | } else { 112 | return file_numbers; 113 | } 114 | } 115 | } 116 | 117 | pub fn filename(&self) -> String { 118 | format!("wal-{:020}", self.file_number) 119 | } 120 | 121 | pub fn file_number(&self) -> u64 { 122 | *self.file_number 123 | } 124 | 125 | #[cfg(test)] 126 | pub fn for_test(file_number: u64) -> Self { 127 | FileNumber::new(file_number) 128 | } 129 | } 130 | 131 | impl std::borrow::Borrow for FileNumber { 132 | fn borrow(&self) -> &u64 { 133 | &self.file_number 134 | } 135 | } 136 | 137 | #[cfg(test)] 138 | impl From for FileNumber { 139 | fn from(file_number: u64) -> Self { 140 | FileNumber::for_test(file_number) 141 | } 142 | } 143 | 144 | #[cfg(test)] 145 | mod tests { 146 | use super::*; 147 | 148 | #[test] 149 | fn test_file_number_starts_at_0() { 150 | let file = FileNumber::default(); 151 | assert_eq!(file.file_number(), 0u64); 152 | } 153 | 154 | #[test] 155 | fn test_file_number_can_be_deleted() { 156 | let file = FileNumber::default(); 157 | assert!(file.can_be_deleted()); 158 | } 159 | 160 | #[test] 161 | fn test_file_number_with_clone_cannot_be_deleted() { 162 | let file = FileNumber::default(); 163 | #[allow(clippy::redundant_clone)] 164 | let _file_clone = file.clone(); 165 | assert!(!file.can_be_deleted()); 166 | } 167 | 168 | #[test] 169 | fn test_file_number_cannot_be_deleted_after_cloned_dropped() { 170 | let file = FileNumber::default(); 171 | let file_clone = file.clone(); 172 | assert!(!file_clone.can_be_deleted()); 173 | drop(file); 174 | assert!(file_clone.can_be_deleted()); 175 | } 176 | } 177 | -------------------------------------------------------------------------------- /src/rolling/mod.rs: -------------------------------------------------------------------------------- 1 | mod directory; 2 | mod file_number; 3 | 4 | pub use self::directory::{Directory, RollingReader, RollingWriter}; 5 | pub use self::file_number::{FileNumber, FileTracker}; 6 | 7 | const FRAME_NUM_BYTES: usize = 1 << 15; 8 | 9 | #[cfg(not(test))] 10 | const NUM_BLOCKS_PER_FILE: usize = 1 << 12; 11 | 12 | #[cfg(test)] 13 | const NUM_BLOCKS_PER_FILE: usize = 4; 14 | 15 | const FILE_NUM_BYTES: usize = FRAME_NUM_BYTES * NUM_BLOCKS_PER_FILE; 16 | #[cfg(test)] 17 | mod tests; 18 | -------------------------------------------------------------------------------- /src/rolling/tests.rs: -------------------------------------------------------------------------------- 1 | use super::*; 2 | use crate::{BlockRead, BlockWrite, PersistAction, BLOCK_NUM_BYTES}; 3 | 4 | #[test] 5 | fn test_read_write() { 6 | let tmp_dir = tempfile::tempdir().unwrap(); 7 | let mut buffer = [0u8; BLOCK_NUM_BYTES]; 8 | { 9 | let rolling_reader: RollingReader = RollingReader::open(tmp_dir.path()).unwrap(); 10 | assert!(&rolling_reader.block().iter().all(|&b| b == 0)); 11 | let mut writer: RollingWriter = rolling_reader.into_writer().unwrap(); 12 | buffer.fill(0u8); 13 | writer.write(&buffer[..]).unwrap(); 14 | buffer.fill(1u8); 15 | writer.write(&buffer[..]).unwrap(); 16 | buffer.fill(2u8); 17 | writer.write(&buffer[..]).unwrap(); 18 | writer.persist(PersistAction::Flush).unwrap(); 19 | } 20 | let mut rolling_reader: RollingReader = RollingReader::open(tmp_dir.path()).unwrap(); 21 | assert!(rolling_reader.block().iter().all(|&b| b == 0)); 22 | assert!(rolling_reader.next_block().unwrap()); 23 | assert!(rolling_reader.block().iter().all(|&b| b == 1)); 24 | assert!(rolling_reader.next_block().unwrap()); 25 | assert!(rolling_reader.block().iter().all(|&b| b == 2)); 26 | } 27 | 28 | #[test] 29 | fn test_read_write_2nd_block() { 30 | let tmp_dir = tempfile::tempdir().unwrap(); 31 | let mut buffer = [0u8; BLOCK_NUM_BYTES]; 32 | { 33 | let rolling_reader: RollingReader = RollingReader::open(tmp_dir.path()).unwrap(); 34 | let mut writer: RollingWriter = rolling_reader.into_writer().unwrap(); 35 | for i in 1..=10 { 36 | buffer.fill(i); 37 | writer.write(&buffer[..]).unwrap(); 38 | } 39 | writer.persist(PersistAction::Flush).unwrap(); 40 | } 41 | { 42 | let mut rolling_reader: RollingReader = RollingReader::open(tmp_dir.path()).unwrap(); 43 | assert!(rolling_reader.block().iter().all(|&b| b == 1)); 44 | assert!(rolling_reader.next_block().unwrap()); 45 | assert!(rolling_reader.block().iter().all(|&b| b == 2)); 46 | assert!(rolling_reader.next_block().unwrap()); 47 | assert!(rolling_reader.block().iter().all(|&b| b == 3)); 48 | let mut writer: RollingWriter = rolling_reader.into_writer().unwrap(); 49 | for i in 13..=23 { 50 | buffer.fill(i); 51 | writer.write(&buffer[..]).unwrap(); 52 | } 53 | writer.persist(PersistAction::Flush).unwrap(); 54 | } 55 | { 56 | let mut rolling_reader: RollingReader = RollingReader::open(tmp_dir.path()).unwrap(); 57 | assert!(rolling_reader.block().iter().all(|&b| b == 1)); 58 | assert!(rolling_reader.next_block().unwrap()); 59 | assert!(rolling_reader.block().iter().all(|&b| b == 2)); 60 | for i in 13..=23 { 61 | assert!(rolling_reader.next_block().unwrap()); 62 | assert!(rolling_reader.block().iter().all(|&b| b == i)); 63 | } 64 | } 65 | } 66 | 67 | #[test] 68 | fn test_read_truncated() { 69 | let tmp_dir = tempfile::tempdir().unwrap(); 70 | let mut buffer = [0u8; BLOCK_NUM_BYTES]; 71 | let to_write = NUM_BLOCKS_PER_FILE * 3; 72 | { 73 | let rolling_reader: RollingReader = RollingReader::open(tmp_dir.path()).unwrap(); 74 | let mut writer: RollingWriter = rolling_reader.into_writer().unwrap(); 75 | for i in 0..to_write { 76 | buffer.fill(i as u8); 77 | writer.write(&buffer[..]).unwrap(); 78 | } 79 | writer.persist(PersistAction::Flush).unwrap(); 80 | let file_ids = writer.list_file_numbers(); 81 | let middle_file = file_ids[1]; 82 | let filepath = 83 | crate::rolling::directory::filepath(tmp_dir.path(), &FileNumber::for_test(middle_file)); 84 | 85 | // voluntarily corrupt data by truncating a wal file. 86 | std::fs::OpenOptions::new() 87 | .truncate(true) 88 | .write(true) 89 | .open(filepath) 90 | .unwrap(); 91 | } 92 | { 93 | let mut rolling_reader: RollingReader = RollingReader::open(tmp_dir.path()).unwrap(); 94 | 95 | for i in 0..to_write { 96 | // ignore file 1 as it was corrupted 97 | if i / NUM_BLOCKS_PER_FILE == 1 { 98 | continue; 99 | } 100 | assert!(rolling_reader.block().iter().all(|&b| b == i as u8)); 101 | // check we manage to get the next block, except for the last block: there is nothing 102 | // after 103 | assert_eq!(rolling_reader.next_block().unwrap(), i != to_write - 1); 104 | } 105 | } 106 | } 107 | 108 | #[test] 109 | fn test_directory_single_file() { 110 | let tmp_dir = tempfile::tempdir().unwrap(); 111 | { 112 | let directory = Directory::open(tmp_dir.path()).unwrap(); 113 | let first_file = directory.first_file_number(); 114 | assert_eq!(first_file.unroll(&directory.files), &[0]); 115 | } 116 | let mut rolling_reader: RollingReader = RollingReader::open(tmp_dir.path()).unwrap(); 117 | for _ in 0..NUM_BLOCKS_PER_FILE - 1 { 118 | assert!(rolling_reader.next_block().unwrap()); 119 | } 120 | assert!(!rolling_reader.next_block().unwrap()); 121 | } 122 | 123 | #[test] 124 | fn test_directory_simple() { 125 | let tmp_dir = tempfile::tempdir().unwrap(); 126 | { 127 | let mut writer: RollingWriter = RollingReader::open(tmp_dir.path()) 128 | .unwrap() 129 | .into_writer() 130 | .unwrap(); 131 | let buf = vec![1u8; FRAME_NUM_BYTES]; 132 | for _ in 0..(NUM_BLOCKS_PER_FILE + 1) { 133 | writer.write(&buf).unwrap(); 134 | } 135 | } 136 | { 137 | let directory = Directory::open(tmp_dir.path()).unwrap(); 138 | let first_file: &FileNumber = directory.first_file_number(); 139 | assert_eq!(first_file.unroll(&directory.files), &[0, 1]); 140 | } 141 | } 142 | 143 | #[test] 144 | fn test_directory_truncate() { 145 | let tmp_dir = tempfile::tempdir().unwrap(); 146 | let file_0: FileNumber; 147 | let file_1: FileNumber; 148 | let file_2: FileNumber; 149 | let file_3: FileNumber; 150 | { 151 | let reader = RollingReader::open(tmp_dir.path()).unwrap(); 152 | file_0 = reader.current_file().clone(); 153 | assert!(!file_0.can_be_deleted()); 154 | let mut writer: RollingWriter = reader.into_writer().unwrap(); 155 | let buf = vec![1u8; FRAME_NUM_BYTES]; 156 | assert_eq!(&writer.current_file().unroll(&writer.directory.files), &[0]); 157 | for _ in 0..NUM_BLOCKS_PER_FILE + 1 { 158 | writer.write(&buf).unwrap(); 159 | } 160 | assert_eq!(&writer.list_file_numbers(), &[0, 1]); 161 | file_1 = writer.current_file().clone(); 162 | assert_eq!(file_1.file_number(), 1); 163 | for _ in 0..NUM_BLOCKS_PER_FILE { 164 | writer.write(&buf).unwrap(); 165 | } 166 | assert_eq!(&writer.list_file_numbers(), &[0, 1, 2]); 167 | file_2 = writer.current_file().clone(); 168 | assert_eq!(file_2.file_number(), 2); 169 | for _ in 0..NUM_BLOCKS_PER_FILE { 170 | writer.write(&buf).unwrap(); 171 | } 172 | file_3 = writer.current_file().clone(); 173 | assert_eq!(&writer.list_file_numbers(), &[0, 1, 2, 3]); 174 | assert!(!file_0.can_be_deleted()); 175 | drop(file_1); 176 | writer.directory.gc().unwrap(); 177 | assert_eq!(&writer.list_file_numbers(), &[0, 1, 2, 3]); 178 | drop(file_0); 179 | writer.directory.gc().unwrap(); 180 | assert_eq!(&writer.list_file_numbers(), &[2, 3]); 181 | drop(file_2); 182 | writer.directory.gc().unwrap(); 183 | assert_eq!(&writer.list_file_numbers(), &[3]); 184 | drop(file_3); 185 | writer.directory.gc().unwrap(); 186 | assert_eq!(&writer.list_file_numbers(), &[3]); 187 | } 188 | } 189 | -------------------------------------------------------------------------------- /src/tests.rs: -------------------------------------------------------------------------------- 1 | use std::borrow::Cow; 2 | 3 | use bytes::Buf; 4 | 5 | use crate::{MultiRecordLog, Record}; 6 | 7 | fn read_all_records<'a>(multi_record_log: &'a MultiRecordLog, queue: &str) -> Vec> { 8 | let mut records = Vec::new(); 9 | let mut next_pos = u64::default(); 10 | for Record { position, payload } in multi_record_log.range(queue, next_pos..).unwrap() { 11 | assert_eq!(position, next_pos); 12 | records.push(payload); 13 | next_pos += 1; 14 | } 15 | records 16 | } 17 | 18 | #[test] 19 | fn test_multi_record_log_new() { 20 | let tempdir = tempfile::tempdir().unwrap(); 21 | MultiRecordLog::open(tempdir.path()).unwrap(); 22 | } 23 | 24 | #[test] 25 | fn test_multi_record_log_create_queue() { 26 | let tempdir = tempfile::tempdir().unwrap(); 27 | { 28 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 29 | multi_record_log.create_queue("queue").unwrap(); 30 | assert!(&multi_record_log.queue_exists("queue")); 31 | } 32 | } 33 | 34 | #[test] 35 | fn test_multi_record_log_create_queue_after_reopen() { 36 | let tempdir = tempfile::tempdir().unwrap(); 37 | { 38 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 39 | multi_record_log.create_queue("queue").unwrap(); 40 | } 41 | { 42 | let multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 43 | assert!(&multi_record_log.queue_exists("queue")); 44 | } 45 | } 46 | 47 | #[test] 48 | fn test_multi_record_log_simple() { 49 | let tempdir = tempfile::tempdir().unwrap(); 50 | { 51 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 52 | multi_record_log.create_queue("queue").unwrap(); 53 | multi_record_log 54 | .append_record("queue", None, &b"hello"[..]) 55 | .unwrap(); 56 | multi_record_log 57 | .append_record("queue", None, &b"happy"[..]) 58 | .unwrap(); 59 | assert_eq!( 60 | &read_all_records(&multi_record_log, "queue"), 61 | &[b"hello".as_slice(), b"happy".as_slice()] 62 | ); 63 | assert_eq!(&multi_record_log.list_file_numbers(), &[0]); 64 | } 65 | } 66 | 67 | #[test] 68 | fn test_multi_record_log_chained() { 69 | let tempdir = tempfile::tempdir().unwrap(); 70 | { 71 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 72 | multi_record_log.create_queue("queue").unwrap(); 73 | multi_record_log 74 | .append_record( 75 | "queue", 76 | None, 77 | b"world".chain(&b" "[..]).chain(&b"order"[..]), 78 | ) 79 | .unwrap(); 80 | multi_record_log 81 | .append_record( 82 | "queue", 83 | None, 84 | b"nice"[..].chain(&b" "[..]).chain(&b"day"[..]), 85 | ) 86 | .unwrap(); 87 | assert_eq!( 88 | &read_all_records(&multi_record_log, "queue"), 89 | &[b"world order".as_slice(), b"nice day".as_slice()] 90 | ); 91 | assert_eq!(&multi_record_log.list_file_numbers(), &[0]); 92 | } 93 | } 94 | 95 | #[test] 96 | fn test_multi_record_log_reopen() { 97 | let tempdir = tempfile::tempdir().unwrap(); 98 | { 99 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 100 | multi_record_log.create_queue("queue").unwrap(); 101 | multi_record_log 102 | .append_record("queue", None, &b"hello"[..]) 103 | .unwrap(); 104 | multi_record_log 105 | .append_record("queue", None, &b"happy"[..]) 106 | .unwrap(); 107 | } 108 | { 109 | let multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 110 | assert_eq!( 111 | &read_all_records(&multi_record_log, "queue"), 112 | &[b"hello".as_slice(), b"happy".as_slice()] 113 | ); 114 | assert_eq!(&multi_record_log.list_file_numbers(), &[0]); 115 | } 116 | } 117 | 118 | #[test] 119 | fn test_multi_record_log() { 120 | let tempdir = tempfile::tempdir().unwrap(); 121 | { 122 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 123 | multi_record_log.create_queue("queue1").unwrap(); 124 | multi_record_log.create_queue("queue2").unwrap(); 125 | multi_record_log 126 | .append_record("queue1", None, &b"hello"[..]) 127 | .unwrap(); 128 | multi_record_log 129 | .append_record("queue2", None, &b"maitre"[..]) 130 | .unwrap(); 131 | multi_record_log 132 | .append_record("queue1", None, &b"happy"[..]) 133 | .unwrap(); 134 | multi_record_log 135 | .append_record("queue1", None, &b"tax"[..]) 136 | .unwrap(); 137 | multi_record_log 138 | .append_record("queue2", None, &b"corbeau"[..]) 139 | .unwrap(); 140 | assert_eq!( 141 | &read_all_records(&multi_record_log, "queue1"), 142 | &[b"hello".as_slice(), b"happy".as_slice(), b"tax".as_slice()] 143 | ); 144 | assert_eq!( 145 | &read_all_records(&multi_record_log, "queue2"), 146 | &[b"maitre".as_slice(), b"corbeau".as_slice()] 147 | ); 148 | assert_eq!(&multi_record_log.list_file_numbers(), &[0]); 149 | } 150 | { 151 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 152 | multi_record_log 153 | .append_record("queue1", None, &b"bubu"[..]) 154 | .unwrap(); 155 | assert_eq!( 156 | &read_all_records(&multi_record_log, "queue1"), 157 | &[ 158 | b"hello".as_slice(), 159 | b"happy".as_slice(), 160 | b"tax".as_slice(), 161 | b"bubu".as_slice() 162 | ] 163 | ); 164 | assert_eq!(&multi_record_log.list_file_numbers(), &[0]); 165 | } 166 | } 167 | 168 | #[test] 169 | fn test_multi_record_position_known_after_truncate() { 170 | let tempdir = tempfile::tempdir().unwrap(); 171 | { 172 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 173 | multi_record_log.create_queue("queue").unwrap(); 174 | assert_eq!( 175 | multi_record_log 176 | .append_record("queue", None, &b"1"[..]) 177 | .unwrap(), 178 | Some(0) 179 | ); 180 | } 181 | { 182 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 183 | assert_eq!( 184 | multi_record_log 185 | .append_record("queue", None, &b"2"[..]) 186 | .unwrap(), 187 | Some(1) 188 | ); 189 | assert_eq!(&multi_record_log.list_file_numbers(), &[0]); 190 | } 191 | { 192 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 193 | multi_record_log.truncate("queue", ..=1).unwrap(); 194 | assert_eq!(&multi_record_log.list_file_numbers(), &[0]); 195 | } 196 | { 197 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 198 | assert_eq!( 199 | multi_record_log 200 | .append_record("queue", None, &b"hello"[..]) 201 | .unwrap(), 202 | Some(2) 203 | ); 204 | } 205 | } 206 | 207 | #[test] 208 | fn test_multi_insert_truncate() { 209 | let tempdir = tempfile::tempdir().unwrap(); 210 | { 211 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 212 | multi_record_log.create_queue("queue").unwrap(); 213 | assert_eq!( 214 | multi_record_log 215 | .append_records( 216 | "queue", 217 | None, 218 | [b"1", b"2", b"3", b"4"].into_iter().map(|r| r.as_slice()) 219 | ) 220 | .unwrap(), 221 | Some(3) 222 | ); 223 | assert_eq!( 224 | &read_all_records(&multi_record_log, "queue"), 225 | &[ 226 | b"1".as_slice(), 227 | b"2".as_slice(), 228 | b"3".as_slice(), 229 | b"4".as_slice() 230 | ] 231 | ); 232 | 233 | multi_record_log.truncate("queue", ..=0).unwrap(); 234 | assert_eq!( 235 | &multi_record_log 236 | .range("queue", ..) 237 | .unwrap() 238 | .map(|record| record.payload) 239 | .collect::>(), 240 | &[b"2".as_slice(), b"3".as_slice(), b"4".as_slice()] 241 | ) 242 | } 243 | { 244 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 245 | multi_record_log.truncate("queue", ..=1).unwrap(); 246 | 247 | assert_eq!( 248 | &multi_record_log 249 | .range("queue", ..) 250 | .unwrap() 251 | .map(|record| record.payload) 252 | .collect::>(), 253 | &[b"3".as_slice(), b"4".as_slice()] 254 | ) 255 | } 256 | { 257 | let multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 258 | assert_eq!( 259 | &multi_record_log 260 | .range("queue", ..) 261 | .unwrap() 262 | .map(|record| record.payload) 263 | .collect::>(), 264 | &[b"3".as_slice(), b"4".as_slice()] 265 | ) 266 | } 267 | } 268 | 269 | #[test] 270 | fn test_truncate_range_correct_pos() { 271 | let tempdir = tempfile::tempdir().unwrap(); 272 | { 273 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 274 | multi_record_log.create_queue("queue").unwrap(); 275 | assert_eq!( 276 | multi_record_log 277 | .append_record("queue", None, &b"1"[..]) 278 | .unwrap(), 279 | Some(0) 280 | ); 281 | assert_eq!( 282 | multi_record_log 283 | .append_record("queue", None, &b"2"[..]) 284 | .unwrap(), 285 | Some(1) 286 | ); 287 | multi_record_log.truncate("queue", ..=1).unwrap(); 288 | assert_eq!( 289 | multi_record_log 290 | .append_record("queue", None, &b"3"[..]) 291 | .unwrap(), 292 | Some(2) 293 | ); 294 | assert_eq!( 295 | multi_record_log 296 | .range("queue", ..) 297 | .unwrap() 298 | .collect::>(), 299 | &[Record::new(2u64, b"3")] 300 | ); 301 | 302 | assert_eq!( 303 | multi_record_log 304 | .range("queue", 2..) 305 | .unwrap() 306 | .collect::>(), 307 | &[Record::new(2, b"3")] 308 | ); 309 | 310 | use std::ops::Bound; 311 | assert_eq!( 312 | multi_record_log 313 | .range("queue", (Bound::Excluded(1), Bound::Unbounded)) 314 | .unwrap() 315 | .collect::>(), 316 | &[Record::new(2, b"3")] 317 | ); 318 | } 319 | } 320 | 321 | #[test] 322 | fn test_multi_record_size() { 323 | let tempdir = tempfile::tempdir().unwrap(); 324 | { 325 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 326 | assert_eq!(multi_record_log.resource_usage().memory_used_bytes, 0); 327 | assert_eq!(multi_record_log.resource_usage().memory_allocated_bytes, 0); 328 | 329 | multi_record_log.create_queue("queue").unwrap(); 330 | let size_mem_create = multi_record_log.resource_usage(); 331 | assert!(size_mem_create.memory_used_bytes > 0); 332 | assert!(size_mem_create.memory_allocated_bytes >= size_mem_create.memory_used_bytes); 333 | 334 | multi_record_log 335 | .append_record("queue", None, &b"hello"[..]) 336 | .unwrap(); 337 | let size_mem_append = multi_record_log.resource_usage(); 338 | assert!(size_mem_append.memory_used_bytes > size_mem_create.memory_used_bytes); 339 | assert!(size_mem_append.memory_allocated_bytes >= size_mem_append.memory_used_bytes); 340 | assert!(size_mem_append.memory_allocated_bytes >= size_mem_create.memory_allocated_bytes); 341 | 342 | multi_record_log.truncate("queue", ..=0).unwrap(); 343 | let size_mem_truncate = multi_record_log.resource_usage(); 344 | assert!(size_mem_truncate.memory_used_bytes < size_mem_append.memory_used_bytes); 345 | } 346 | } 347 | 348 | #[test] 349 | fn test_open_corrupted() { 350 | // a single frame is 32k. We write more than 2 frames worth of data, corrupt one, 351 | // and verify we still read more than half the records successfully. 352 | let tempdir = tempfile::tempdir().unwrap(); 353 | { 354 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 355 | multi_record_log.create_queue("queue").unwrap(); 356 | 357 | // 8192 * 8bytes = 64k without overhead. 358 | for i in 0..8192 { 359 | multi_record_log 360 | .append_record("queue", Some(i), format!("{i:08}").as_bytes()) 361 | .unwrap(); 362 | } 363 | } 364 | { 365 | use std::fs::OpenOptions; 366 | use std::io::*; 367 | // corrupt the file 368 | let file = std::fs::read_dir(tempdir.path()) 369 | .unwrap() 370 | .filter_map(Result::ok) 371 | .find(|file| !file.file_name().to_str().unwrap().starts_with('.')) 372 | .unwrap(); 373 | 374 | let mut file = OpenOptions::new().write(true).open(file.path()).unwrap(); 375 | // jump somewhere in the middle 376 | file.seek(SeekFrom::Start(10240)).unwrap(); 377 | file.write_all(b"this will corrupt the file. Good :-)") 378 | .unwrap(); 379 | } 380 | { 381 | let multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 382 | 383 | let mut count = 0; 384 | for Record { position, payload } in multi_record_log.range("queue", ..).unwrap() { 385 | assert_eq!(payload, format!("{position:08}").as_bytes()); 386 | count += 1; 387 | } 388 | assert!(count > 4096); 389 | } 390 | } 391 | 392 | #[test] 393 | fn test_create_twice() { 394 | let tempdir = tempfile::tempdir().unwrap(); 395 | { 396 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 397 | multi_record_log.create_queue("queue1").unwrap(); 398 | multi_record_log 399 | .append_record("queue1", None, &b"hello"[..]) 400 | .unwrap(); 401 | multi_record_log.create_queue("queue1").unwrap_err(); 402 | assert_eq!(multi_record_log.range("queue1", ..).unwrap().count(), 1); 403 | } 404 | { 405 | let multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 406 | assert_eq!(multi_record_log.range("queue1", ..).unwrap().count(), 1); 407 | } 408 | } 409 | 410 | #[test] 411 | fn test_last_position() { 412 | let tempdir = tempfile::tempdir().unwrap(); 413 | 414 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 415 | multi_record_log.last_position("queue1").unwrap_err(); 416 | 417 | multi_record_log.create_queue("queue1").unwrap(); 418 | let last_pos = multi_record_log.last_position("queue1").unwrap(); 419 | assert!(last_pos.is_none()); 420 | 421 | multi_record_log 422 | .append_record("queue1", None, &b"hello"[..]) 423 | .unwrap(); 424 | 425 | let last_pos = multi_record_log.last_position("queue1").unwrap().unwrap(); 426 | assert_eq!(last_pos, 0); 427 | 428 | multi_record_log.truncate("queue1", ..=0).unwrap(); 429 | 430 | let last_pos = multi_record_log.last_position("queue1").unwrap().unwrap(); 431 | assert_eq!(last_pos, 0); 432 | } 433 | 434 | #[test] 435 | fn test_last_record() { 436 | let tempdir = tempfile::tempdir().unwrap(); 437 | 438 | let mut multi_record_log = MultiRecordLog::open(tempdir.path()).unwrap(); 439 | multi_record_log.last_position("queue1").unwrap_err(); 440 | 441 | multi_record_log.create_queue("queue1").unwrap(); 442 | let last_record = multi_record_log.last_position("queue1").unwrap(); 443 | assert!(last_record.is_none()); 444 | 445 | multi_record_log 446 | .append_record("queue1", None, &b"hello"[..]) 447 | .unwrap(); 448 | 449 | let Record { position, payload } = multi_record_log.last_record("queue1").unwrap().unwrap(); 450 | assert_eq!(position, 0); 451 | assert_eq!(payload, &b"hello"[..]); 452 | 453 | multi_record_log.truncate("queue1", ..=0).unwrap(); 454 | 455 | let last_record = multi_record_log.last_record("queue1").unwrap(); 456 | assert!(last_record.is_none()); 457 | } 458 | --------------------------------------------------------------------------------