mirror of https://github.com/tikv/raft-engine.git
Compare commits
83 Commits
Author | SHA1 | Date |
---|---|---|
|
d53b2b9505 | |
|
03f77d90c0 | |
|
392f5e66f8 | |
|
7e12385c31 | |
|
6bb58c379b | |
|
de1ec93752 | |
|
e1c5dd80a6 | |
|
209e00d8c0 | |
|
0e40c53566 | |
|
897e0ebf67 | |
|
c7828cb3e7 | |
|
3ad316a3c4 | |
|
8ef3b42057 | |
|
c1f81d4d06 | |
|
cf5d1b9144 | |
|
cd1533d79c | |
|
084c5ca0e4 | |
|
e505d631c8 | |
|
d043b4ad4e | |
|
e8de5d75ae | |
|
385182bfc2 | |
|
70a8873fbb | |
|
fa56f891fd | |
|
9aa124558c | |
|
22dfb426cd | |
|
3724bfc510 | |
|
ec6aa90163 | |
|
0a33383758 | |
|
d4943a9f72 | |
|
2dcaf5beee | |
|
58041ad3e6 | |
|
1c7a820b25 | |
|
e55641f532 | |
|
de3ad04a2d | |
|
2c766903a8 | |
|
c9a95c8244 | |
|
7016173984 | |
|
85627d1a93 | |
|
205b5d7a4b | |
|
773b89fb24 | |
|
c1d3dac28b | |
|
471efffa8e | |
|
3802c9c677 | |
|
39f4db4512 | |
|
3c442affad | |
|
09fe23053c | |
|
6e2381e5b4 | |
|
402f52e175 | |
|
c4b1864d3e | |
|
6f58a0f94f | |
|
404e3fefae | |
|
75d6d6ecf4 | |
|
33530112c3 | |
|
8dd2a39f35 | |
|
81aa5bb9a0 | |
|
82f6da7b8d | |
|
a0d29980f1 | |
|
5f718cfe50 | |
|
67b6348689 | |
|
b95dd014fa | |
|
087d36b0f8 | |
|
618eea7af8 | |
|
9751c6dd5c | |
|
62a69a8774 | |
|
cf225c200b | |
|
3f41881bf1 | |
|
6a6fe3bd2e | |
|
c3a6156b93 | |
|
70d27ee04b | |
|
ee024c75b9 | |
|
764ac71461 | |
|
f3c268bb95 | |
|
be261417f0 | |
|
ee0f6cf0d9 | |
|
7a436eae40 | |
|
07dcadbf51 | |
|
1a9dd43a74 | |
|
a950ec5c22 | |
|
b0ad1f7a22 | |
|
5e8e4bb9c6 | |
|
4e89901a3e | |
|
0459a82156 | |
|
812a8c0d00 |
|
@ -1,6 +1,15 @@
|
|||
name: Rust
|
||||
|
||||
on: [push, pull_request]
|
||||
on:
|
||||
push:
|
||||
branches-ignore:
|
||||
- dependabot/**
|
||||
|
||||
pull_request:
|
||||
paths-ignore:
|
||||
- "OWNERS"
|
||||
- "OWNERS_ALIASES"
|
||||
- "*.md"
|
||||
|
||||
env:
|
||||
CARGO_TERM_COLOR: always
|
||||
|
@ -10,7 +19,9 @@ jobs:
|
|||
runs-on: ${{ matrix.os }}
|
||||
strategy:
|
||||
matrix:
|
||||
os: [ ubuntu-latest, macos-latest ]
|
||||
# tests with sanitizer on `ubuntu-latest` is supported by
|
||||
# self-defined `nihtly test` job, so we don't need to test it here.
|
||||
os: [ macos-latest ]
|
||||
steps:
|
||||
- uses: actions/checkout@v2
|
||||
with:
|
||||
|
@ -19,7 +30,7 @@ jobs:
|
|||
uses: actions-rs/toolchain@v1
|
||||
with:
|
||||
profile: minimal
|
||||
toolchain: nightly-2022-01-07
|
||||
toolchain: nightly-2025-04-03
|
||||
override: true
|
||||
components: rustfmt, clippy, rust-src
|
||||
- uses: Swatinem/rust-cache@v1
|
||||
|
@ -29,24 +40,18 @@ jobs:
|
|||
if: ${{ matrix.os == 'ubuntu-latest' }}
|
||||
run: if [[ ! -e ~/.cargo/bin/grcov ]]; then cargo install grcov; fi
|
||||
- name: Format
|
||||
run: cargo fmt --all -- --check
|
||||
run: |
|
||||
make format
|
||||
git diff --exit-code
|
||||
- name: Clippy
|
||||
run: cargo clippy --all --all-features --all-targets -- -D clippy::all
|
||||
run: make clippy
|
||||
env:
|
||||
EXTRA_CARGO_ARGS: '--fix'
|
||||
- name: Run tests
|
||||
run: |
|
||||
cargo test --all --features all_except_failpoints --verbose -- --nocapture
|
||||
cargo test --test failpoints --all-features --verbose -- --test-threads 1 --nocapture
|
||||
run: make test
|
||||
env:
|
||||
RUST_BACKTRACE: 1
|
||||
- name: Run asan tests
|
||||
if: ${{ matrix.os == 'ubuntu-latest' }}
|
||||
run: |
|
||||
cargo test -Zbuild-std --target x86_64-unknown-linux-gnu --all --features all_except_failpoints --verbose -- --nocapture
|
||||
cargo test -Zbuild-std --target x86_64-unknown-linux-gnu --test failpoints --all-features --verbose -- --test-threads 1 --nocapture
|
||||
env:
|
||||
RUST_BACKTRACE: 1
|
||||
RUSTFLAGS: '-Zsanitizer=address'
|
||||
RUSTDOCFLAGS: '-Zsanitizer=address'
|
||||
EXTRA_CARGO_ARGS: '--verbose'
|
||||
stable:
|
||||
runs-on: ${{ matrix.os }}
|
||||
strategy:
|
||||
|
@ -60,22 +65,22 @@ jobs:
|
|||
uses: actions-rs/toolchain@v1
|
||||
with:
|
||||
profile: minimal
|
||||
toolchain: 1.57.0
|
||||
toolchain: 1.85.0
|
||||
override: true
|
||||
components: rustfmt, clippy, rust-src
|
||||
- uses: Swatinem/rust-cache@v1
|
||||
with:
|
||||
sharedKey: ${{ matrix.os }}-stable
|
||||
- name: Format
|
||||
run: cargo fmt --all -- --check
|
||||
- name: Clippy
|
||||
run: cargo clippy --all --features all_stable --all-targets -- -D clippy::all
|
||||
run: make clippy
|
||||
env:
|
||||
WITH_STABLE_TOOLCHAIN: 'force'
|
||||
- name: Run tests
|
||||
run: |
|
||||
cargo test --all --features all_stable_except_failpoints --verbose -- --nocapture
|
||||
cargo test --test failpoints --features all_stable --verbose -- --test-threads 1 --nocapture
|
||||
run: make test
|
||||
env:
|
||||
RUST_BACKTRACE: 1
|
||||
EXTRA_CARGO_ARGS: '--verbose'
|
||||
WITH_STABLE_TOOLCHAIN: 'force'
|
||||
coverage:
|
||||
runs-on: ubuntu-latest
|
||||
needs: nightly
|
||||
|
@ -87,7 +92,7 @@ jobs:
|
|||
uses: actions-rs/toolchain@v1
|
||||
with:
|
||||
profile: minimal
|
||||
toolchain: nightly-2022-01-07
|
||||
toolchain: nightly-2025-04-03
|
||||
override: true
|
||||
components: llvm-tools-preview
|
||||
- uses: Swatinem/rust-cache@v1
|
||||
|
@ -96,17 +101,16 @@ jobs:
|
|||
- name: Install grcov
|
||||
run: if [[ ! -e ~/.cargo/bin/grcov ]]; then cargo install --locked grcov; fi
|
||||
- name: Run tests
|
||||
run: |
|
||||
cargo test --all --features all_except_failpoints
|
||||
cargo test --test failpoints --all-features -- --test-threads 1
|
||||
cargo test --all --features all_stable_except_failpoints
|
||||
cargo test --test failpoints --features all_stable -- --test-threads 1
|
||||
run: make test_matrix
|
||||
env:
|
||||
RUSTFLAGS: '-Zinstrument-coverage'
|
||||
RUSTFLAGS: '-Cinstrument-coverage'
|
||||
LLVM_PROFILE_FILE: '%p-%m.profraw'
|
||||
EXTRA_CARGO_ARGS: '--verbose'
|
||||
- name: Run grcov
|
||||
run: grcov `find . \( -name "*.profraw" \) -print` --binary-path target/debug/deps/ -s . -t lcov --branch --ignore-not-existing --ignore '../**' --ignore '/*' -o coverage.lcov
|
||||
- name: Upload
|
||||
uses: codecov/codecov-action@v2
|
||||
uses: codecov/codecov-action@v3
|
||||
env:
|
||||
CODECOV_TOKEN: ${{ secrets.CODECOV_TOKEN }}
|
||||
with:
|
||||
file: coverage.lcov
|
||||
|
|
66
CHANGELOG.md
66
CHANGELOG.md
|
@ -2,6 +2,72 @@
|
|||
|
||||
## [Unreleased]
|
||||
|
||||
### New Features
|
||||
|
||||
* Add a new Prometheus metric `raft_engine_write_compression_ratio` to track compression ratio of write #358
|
||||
|
||||
## [0.4.2] - 2024-04-16
|
||||
|
||||
### Behavior Changes
|
||||
|
||||
* Periodically flush unsynced bytes when rewriting to avoid I/O jitters if flushing too many bytes impede the foreground writes. (#347)
|
||||
* Errors will be returned if rewriting fails, instread of `panic` directly. (#343)
|
||||
|
||||
## [0.4.1] - 2023-09-14
|
||||
|
||||
### Behavior Changes
|
||||
|
||||
* When log recycling is enabled, Raft Engine will now retain 50% more log files to reduce the chance of running out.
|
||||
* Reduce the scope of keys reserved for internal use.
|
||||
|
||||
## [0.4.0] - 2023-09-01
|
||||
|
||||
### Behavior Changes
|
||||
|
||||
* `LogBatch::put` returns a `Result<()>` instead of `()`. It errs when the key is reserved for internal use.
|
||||
* Possible to specify a permission in `FileSystem::open`.
|
||||
* Prometheus counter `raft_engine_log_file_count` no longer includes retired log files that are stashed for recycling. Those files are now tracked by a new counter `raft_engine_recycled_file_count`.
|
||||
|
||||
### Bug Fixes
|
||||
|
||||
* Fix data loss caused by aborted rewrite operation. Downgrading to an earlier version without the fix may produce phantom Raft Groups or keys, i.e. never written but appear in queries.
|
||||
* Fix a potential bug that an un-persisted log batch is mistakenly recovered and causes checksum mismatch error when being read later.
|
||||
|
||||
### New Features
|
||||
|
||||
* Support preparing prefilled logs to enable log recycling when start-up. The amount of logs to prepare is controlled by `Config::prefill_limit`.
|
||||
* Add a new configuration `spill-dir` to allow automatic placement of logs into an auxiliary directory when `dir` is full.
|
||||
* Add a new method `Engine::fork` to duplicate an `Engine` to a new place, with a few disk file copies.
|
||||
* Support configuring lz4 acceleration factor with `compression-level`.
|
||||
|
||||
## [0.3.0] - 2022-09-14
|
||||
|
||||
### Bug Fixes
|
||||
|
||||
* Unconditionally tolerate `fallocate` failures as a fix to its portability issue. Errors other than `EOPNOTSUPP` will still emit a warning.
|
||||
* Avoid leaving fractured write after failure by reseeking the file writer. Panic if the reseek fails as well.
|
||||
* Fix a parallel recovery panic bug.
|
||||
* Fix panic when an empty batch is written to engine and then reused.
|
||||
|
||||
### New Features
|
||||
|
||||
* Add `PerfContext` which records detailed time breakdown of the write process to thread-local storage.
|
||||
* Support recycling obsolete log files to reduce the cost of `fallocate`-ing new ones.
|
||||
|
||||
### Public API Changes
|
||||
|
||||
* Add `is_empty` to `Engine` API.
|
||||
* Add metadata deletion capability to `FileSystem` trait. Users can implement `exists_metadata` and `delete_metadata` to clean up obsolete metadata from older versions of Raft Engine.
|
||||
* Add `Engine::scan_messages` and `Engine::scan_raw_messages` for iterating over written key-values.
|
||||
* Add `Engine::get` for getting raw value.
|
||||
* Move `sync` from `env::WriteExt` to `env::Handle`.
|
||||
* Deprecate `bytes_per_sync`.
|
||||
|
||||
### Behavior Changes
|
||||
|
||||
* Change format version to 2 from 1 by default.
|
||||
* Enable log recycling by default.
|
||||
|
||||
## [0.2.0] - 2022-05-25
|
||||
|
||||
### Bug Fixes
|
||||
|
|
74
Cargo.toml
74
Cargo.toml
|
@ -1,9 +1,9 @@
|
|||
[package]
|
||||
name = "raft-engine"
|
||||
version = "0.2.0"
|
||||
version = "0.4.2"
|
||||
authors = ["The TiKV Project Developers"]
|
||||
edition = "2018"
|
||||
rust-version = "1.57"
|
||||
rust-version = "1.85.0"
|
||||
description = "A persistent storage engine for Multi-Raft logs"
|
||||
readme = "README.md"
|
||||
repository = "https://github.com/tikv/raft-engine"
|
||||
|
@ -16,6 +16,10 @@ features = ["internals"]
|
|||
name = "append-compact-purge"
|
||||
path = "examples/append_compact_purge.rs"
|
||||
|
||||
[[example]]
|
||||
name = "raft-engine-fork"
|
||||
path = "examples/fork.rs"
|
||||
|
||||
[[test]]
|
||||
name = "failpoints"
|
||||
path = "tests/failpoints/mod.rs"
|
||||
|
@ -33,62 +37,64 @@ crc32fast = "1.2"
|
|||
crossbeam = "0.8"
|
||||
fail = "0.5"
|
||||
fs2 = "0.4"
|
||||
hashbrown = "0.12"
|
||||
hashbrown = "0.14"
|
||||
hex = "0.4"
|
||||
if_chain = "1.0"
|
||||
lazy_static = "1.3"
|
||||
libc = "0.2"
|
||||
log = { version = "0.4", features = ["max_level_trace", "release_max_level_debug"] }
|
||||
lz4-sys = "1.9"
|
||||
memmap2 = { version = "0.5", optional = true }
|
||||
nix = "0.24"
|
||||
num-derive = "0.3"
|
||||
log = { version = "0.4", features = [
|
||||
"max_level_trace",
|
||||
"release_max_level_debug",
|
||||
] }
|
||||
lz4-sys = { version = "=1.9.5" }
|
||||
memmap2 = { version = "0.9", optional = true }
|
||||
nix = "0.26"
|
||||
num-derive = "0.4"
|
||||
num-traits = "0.2"
|
||||
parking_lot = "0.12"
|
||||
prometheus = { version = "0.13" }
|
||||
prometheus-static-metric = "0.5"
|
||||
protobuf = "2"
|
||||
rayon = "1.5"
|
||||
rhai = { version = "~1.4", features = ["sync"], optional = true }
|
||||
rhai = { version = "1.7", features = ["sync"], optional = true }
|
||||
scopeguard = "1.1"
|
||||
serde = { version = "1.0", features = ["derive"] }
|
||||
serde = { version = "=1.0.194", features = ["derive"] }
|
||||
serde_repr = "0.1"
|
||||
strum = { version = "0.26.2", features = ["derive"] }
|
||||
thiserror = "1.0"
|
||||
|
||||
[dev-dependencies]
|
||||
criterion = "0.3"
|
||||
ctor = "0.1"
|
||||
env_logger = "0.9"
|
||||
kvproto = { git = "https://github.com/pingcap/kvproto.git", default-features = false, features = ["protobuf-codec"] }
|
||||
raft = { git = "https://github.com/tikv/raft-rs", branch = "master", default-features = false, features = ["protobuf-codec"] }
|
||||
criterion = "0.4"
|
||||
ctor = "0.2"
|
||||
env_logger = "0.10"
|
||||
kvproto = { git = "https://github.com/pingcap/kvproto.git", default-features = false, features = [
|
||||
"protobuf-codec",
|
||||
] }
|
||||
raft = { git = "https://github.com/tikv/raft-rs", branch = "master", default-features = false, features = [
|
||||
"protobuf-codec",
|
||||
] }
|
||||
rand = "0.8"
|
||||
rand_distr = "0.4"
|
||||
tempfile = "3.1"
|
||||
toml = "0.5"
|
||||
tempfile = "3.6"
|
||||
toml = "0.8"
|
||||
|
||||
[features]
|
||||
default = ["internals", "scripting"]
|
||||
internals = []
|
||||
nightly = [
|
||||
"prometheus/nightly",
|
||||
]
|
||||
failpoints = [
|
||||
"fail/failpoints",
|
||||
]
|
||||
scripting = [
|
||||
"rhai",
|
||||
]
|
||||
swap = [
|
||||
"nightly",
|
||||
"memmap2",
|
||||
]
|
||||
nightly = ["prometheus/nightly"]
|
||||
failpoints = ["fail/failpoints"]
|
||||
scripting = ["rhai"]
|
||||
swap = ["nightly", "memmap2"]
|
||||
std_fs = []
|
||||
|
||||
# Shortcuts
|
||||
all_except_failpoints = ["internals", "scripting", "nightly", "swap"]
|
||||
all_stable = ["internals", "scripting", "failpoints"]
|
||||
all_stable_except_failpoints = ["internals", "scripting"]
|
||||
nightly_group = ["nightly", "swap"]
|
||||
|
||||
[patch.crates-io]
|
||||
raft-proto = { git = "https://github.com/tikv/raft-rs", branch = "master" }
|
||||
protobuf = { git = "https://github.com/pingcap/rust-protobuf", branch = "v2.8" }
|
||||
protobuf-codegen = { git = "https://github.com/pingcap/rust-protobuf", branch = "v2.8" }
|
||||
# Lock the version of cc-rs to avoid build failure on MacOS, ref https://github.com/rust-lang/cc-rs/issues/984.
|
||||
cc = { git = "https://github.com/rust-lang/cc-rs", tag = "1.0.98" }
|
||||
|
||||
[workspace]
|
||||
members = ["stress", "ctl"]
|
||||
|
|
|
@ -0,0 +1,79 @@
|
|||
# Makefile
|
||||
|
||||
## Additionaly arguments passed to cargo.
|
||||
EXTRA_CARGO_ARGS ?=
|
||||
## How to test stable toolchain.
|
||||
## - auto: use current default toolchain, disable nightly features.
|
||||
## - force: explicitly use stable toolchain, disable nightly features.
|
||||
WITH_STABLE_TOOLCHAIN ?=
|
||||
|
||||
WITH_NIGHTLY_FEATURES =
|
||||
ifeq (,$(filter $(WITH_STABLE_TOOLCHAIN),auto force))
|
||||
WITH_NIGHTLY_FEATURES = 1
|
||||
endif
|
||||
|
||||
TOOLCHAIN_ARGS =
|
||||
ifeq ($(shell (rustc --version | grep -q nightly); echo $$?), 1)
|
||||
ifdef WITH_NIGHTLY_FEATURES
|
||||
# Force use nightly toolchain if we are building with nightly features.
|
||||
TOOLCHAIN_ARGS = +nightly
|
||||
endif
|
||||
else
|
||||
ifeq ($(WITH_STABLE_TOOLCHAIN), force)
|
||||
TOOLCHAIN_ARGS = +stable
|
||||
endif
|
||||
endif
|
||||
|
||||
BIN_PATH = $(CURDIR)/bin
|
||||
CARGO_TARGET_DIR ?= $(CURDIR)/target/
|
||||
export RUST_LOG=info
|
||||
|
||||
.PHONY: clean format clippy test
|
||||
.PHONY: ctl
|
||||
|
||||
all: format clippy test
|
||||
|
||||
clean:
|
||||
cargo clean
|
||||
rm -rf ${BIN_PATH}
|
||||
|
||||
## Format code in-place using rustfmt.
|
||||
format:
|
||||
cargo ${TOOLCHAIN_ARGS} fmt --all
|
||||
|
||||
CLIPPY_WHITELIST += -A clippy::bool_assert_comparison
|
||||
## Run clippy.
|
||||
clippy:
|
||||
ifdef WITH_NIGHTLY_FEATURES
|
||||
cargo ${TOOLCHAIN_ARGS} clippy --all --features nightly_group,failpoints --all-targets -- -D clippy::all ${CLIPPY_WHITELIST}
|
||||
else
|
||||
cargo ${TOOLCHAIN_ARGS} clippy --all --features failpoints --all-targets -- -D clippy::all ${CLIPPY_WHITELIST}
|
||||
endif
|
||||
|
||||
## Run tests.
|
||||
test:
|
||||
ifdef WITH_NIGHTLY_FEATURES
|
||||
cargo ${TOOLCHAIN_ARGS} test --all --features nightly_group ${EXTRA_CARGO_ARGS} -- --nocapture
|
||||
cargo ${TOOLCHAIN_ARGS} test --test failpoints --features nightly_group,failpoints ${EXTRA_CARGO_ARGS} -- --test-threads 1 --nocapture
|
||||
else
|
||||
cargo ${TOOLCHAIN_ARGS} test --all ${EXTRA_CARGO_ARGS} -- --nocapture
|
||||
cargo ${TOOLCHAIN_ARGS} test --test failpoints --features failpoints ${EXTRA_CARGO_ARGS} -- --test-threads 1 --nocapture
|
||||
endif
|
||||
|
||||
## Run tests with various features for maximum code coverage.
|
||||
ifndef WITH_NIGHTLY_FEATURES
|
||||
test_matrix:
|
||||
$(error Must run test matrix with nightly features. Please reset WITH_STABLE_TOOLCHAIN.)
|
||||
else
|
||||
test_matrix: test
|
||||
cargo ${TOOLCHAIN_ARGS} test --all ${EXTRA_CARGO_ARGS} -- --nocapture
|
||||
cargo ${TOOLCHAIN_ARGS} test --test failpoints --features failpoints ${EXTRA_CARGO_ARGS} -- --test-threads 1 --nocapture
|
||||
cargo ${TOOLCHAIN_ARGS} test --all --features nightly_group,std_fs ${EXTRA_CARGO_ARGS} -- --nocapture
|
||||
cargo ${TOOLCHAIN_ARGS} test --test failpoints --features nightly_group,std_fs,failpoints ${EXTRA_CARGO_ARGS} -- --test-threads 1 --nocapture
|
||||
endif
|
||||
|
||||
## Build raft-engine-ctl.
|
||||
ctl:
|
||||
cargo build --release --package raft-engine-ctl
|
||||
@mkdir -p ${BIN_PATH}
|
||||
@cp -f ${CARGO_TARGET_DIR}/release/raft-engine-ctl ${BIN_PATH}/
|
|
@ -0,0 +1,80 @@
|
|||
# See the OWNERS docs at https://go.k8s.io/owners
|
||||
approvers:
|
||||
- 5kbpers
|
||||
- AndreMouche
|
||||
- andylokandy
|
||||
- breezewish
|
||||
- brson
|
||||
- bufferflies
|
||||
- BusyJay
|
||||
- cfzjywxk
|
||||
- Connor1996
|
||||
- coocood
|
||||
- crazycs520
|
||||
- disksing
|
||||
- ekexium
|
||||
- gengliqi
|
||||
- glorv
|
||||
- hicqu
|
||||
- hunterlxt
|
||||
- imtbkcat
|
||||
- innerr
|
||||
- iosmanthus
|
||||
- jackysp
|
||||
- kennytm
|
||||
- Little-Wallace
|
||||
- liuzix
|
||||
- lonng
|
||||
- LykxSassinator
|
||||
- lysu
|
||||
- marsishandsome
|
||||
- MyonKeminta
|
||||
- niedhui
|
||||
- NingLin-P
|
||||
- nrc
|
||||
- overvenus
|
||||
- pingyu
|
||||
- skyzh
|
||||
- SpadeA-Tang
|
||||
- sticnarf
|
||||
- sunxiaoguang
|
||||
- tabokie
|
||||
- TennyZhuang
|
||||
- tonyxuqqi
|
||||
- v01dstar
|
||||
- yiwu-arbug
|
||||
- you06
|
||||
- youjiali1995
|
||||
- YuJuncen
|
||||
- zhangjinpeng87
|
||||
- zhongzc
|
||||
- zhouqiang-cl
|
||||
- zyguan
|
||||
reviewers:
|
||||
- 3AceShowHand
|
||||
- 3pointer
|
||||
- CalvinNeo
|
||||
- ethercflow
|
||||
- fredchenbj
|
||||
- Fullstop000
|
||||
- gozssky
|
||||
- haojinming
|
||||
- hbisheng
|
||||
- hhwyt
|
||||
- HuSharp
|
||||
- jayzhan211
|
||||
- Jibbow
|
||||
- JmPotato
|
||||
- Leavrth
|
||||
- lhy1024
|
||||
- longfangsong
|
||||
- lzmhhh123
|
||||
- Mossaka
|
||||
- MrCroxx
|
||||
- nolouch
|
||||
- rleungx
|
||||
- Rustin170506
|
||||
- tier-cap
|
||||
- wjhuang2016
|
||||
- wshwsh12
|
||||
- Xuanwo
|
16
README.md
16
README.md
|
@ -54,7 +54,7 @@ Put this in your Cargo.toml:
|
|||
|
||||
```rust
|
||||
[dependencies]
|
||||
raft-engine = "0.2.0"
|
||||
raft-engine = "0.4"
|
||||
```
|
||||
|
||||
Available Cargo features:
|
||||
|
@ -74,18 +74,20 @@ Contributions are always welcome! Here are a few tips for making a PR:
|
|||
- All commits must be signed off (with `git commit -s`) to pass the [DCO check](https://probot.github.io/apps/dco/).
|
||||
- Tests are automatically run against the changes, some of them can be run locally:
|
||||
|
||||
```
|
||||
cargo fmt --all -- --check
|
||||
cargo +nightly clippy --all --all-features --all-targets -- -D clippy::all
|
||||
cargo +nightly test --all --features all_except_failpoints
|
||||
cargo +nightly test --test failpoints --all-features -- --test-threads 1
|
||||
```bash
|
||||
# run tests with nightly features
|
||||
make
|
||||
# run tests on stable toolchain
|
||||
make WITH_STABLE_TOOLCHAIN=force
|
||||
# filter a specific test case
|
||||
make test EXTRA_CARGO_ARGS=<testname>
|
||||
```
|
||||
|
||||
- For changes that might induce performance effects, please quote the targeted benchmark results in the PR description. In addition to micro-benchmarks, there is a standalone [stress test tool](https://github.com/tikv/raft-engine/tree/master/stress) which you can use to demonstrate the system performance.
|
||||
|
||||
```
|
||||
cargo +nightly bench --all-features <bench-case-name>
|
||||
cargo run --release --package stress --help
|
||||
cargo run --release --package stress -- --help
|
||||
```
|
||||
|
||||
## License
|
||||
|
|
|
@ -1,14 +1,17 @@
|
|||
[package]
|
||||
name = "raft-engine-ctl"
|
||||
version = "0.2.0"
|
||||
version = "0.4.2"
|
||||
authors = ["The TiKV Project Developers"]
|
||||
edition = "2018"
|
||||
rust-version = "1.57"
|
||||
rust-version = "1.75.0"
|
||||
description = "A control tool for Raft Engine"
|
||||
repository = "https://github.com/tikv/raft-engine"
|
||||
license = "Apache-2.0"
|
||||
|
||||
[dependencies]
|
||||
clap = { version = "3.0.0-rc.0", features = ["derive", "cargo"] }
|
||||
env_logger = "0.9"
|
||||
raft-engine = { path = "..", version = "0.2.0", features = ["scripting", "internals"] }
|
||||
clap = { version = "3.1", features = ["derive", "cargo"] }
|
||||
env_logger = "0.10"
|
||||
raft-engine = { path = "..", version = "0.4.1", features = [
|
||||
"scripting",
|
||||
"internals",
|
||||
] }
|
||||
|
|
|
@ -14,8 +14,10 @@
|
|||
//! # Raft Engine Control
|
||||
|
||||
use std::path::Path;
|
||||
use std::sync::Arc;
|
||||
|
||||
use clap::{crate_authors, crate_version, AppSettings, Parser};
|
||||
use clap::{crate_authors, crate_version, Parser};
|
||||
use raft_engine::env::{DefaultFileSystem, FileSystem};
|
||||
use raft_engine::internals::LogQueue;
|
||||
use raft_engine::{Engine, Error, Result as EngineResult};
|
||||
|
||||
|
@ -24,7 +26,7 @@ use raft_engine::{Engine, Error, Result as EngineResult};
|
|||
name = "ctl",
|
||||
author = crate_authors!(),
|
||||
version = crate_version!(),
|
||||
setting = AppSettings::DontCollapseArgsInUsage,
|
||||
dont_collapse_args_in_usage = true,
|
||||
)]
|
||||
pub struct ControlOpt {
|
||||
// sub command type
|
||||
|
@ -40,7 +42,7 @@ enum Cmd {
|
|||
#[clap(short, long)]
|
||||
path: String,
|
||||
|
||||
#[clap(short, long, use_delimiter = true)]
|
||||
#[clap(short, long, use_value_delimiter = true)]
|
||||
raft_groups: Vec<u64>,
|
||||
},
|
||||
|
||||
|
@ -87,22 +89,29 @@ fn convert_queue(queue: &str) -> Option<LogQueue> {
|
|||
}
|
||||
|
||||
impl ControlOpt {
|
||||
pub fn validate_and_execute(mut self) -> EngineResult<()> {
|
||||
pub fn validate_and_execute(self) -> EngineResult<()> {
|
||||
self.validate_and_execute_with_file_system(Arc::new(DefaultFileSystem))
|
||||
}
|
||||
|
||||
pub fn validate_and_execute_with_file_system<F: FileSystem>(
|
||||
mut self,
|
||||
fs: Arc<F>,
|
||||
) -> EngineResult<()> {
|
||||
if self.cmd.is_none() {
|
||||
return Err(Error::InvalidArgument("subcommand is needed".to_owned()));
|
||||
}
|
||||
|
||||
match self.cmd.take().unwrap() {
|
||||
Cmd::Dump { path, raft_groups } => {
|
||||
let it = Engine::dump(Path::new(&path))?;
|
||||
let it = Engine::dump_with_file_system(Path::new(&path), fs)?;
|
||||
for item in it {
|
||||
if let Ok(v) = item {
|
||||
if raft_groups.is_empty() || raft_groups.contains(&v.raft_group_id) {
|
||||
println!("{:?}", v)
|
||||
println!("{v:?}")
|
||||
}
|
||||
} else {
|
||||
// output error message
|
||||
println!("{:?}", item)
|
||||
println!("{item:?}")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -112,22 +121,30 @@ impl ControlOpt {
|
|||
script,
|
||||
} => {
|
||||
let script = std::fs::read_to_string(script)?;
|
||||
Engine::unsafe_repair(Path::new(&path), convert_queue(&queue), script)?;
|
||||
Engine::unsafe_repair_with_file_system(
|
||||
Path::new(&path),
|
||||
convert_queue(&queue),
|
||||
script,
|
||||
fs,
|
||||
)?;
|
||||
}
|
||||
Cmd::Check { path } => {
|
||||
let r = Engine::consistency_check(Path::new(&path))?;
|
||||
let r = Engine::consistency_check_with_file_system(Path::new(&path), fs)?;
|
||||
if r.is_empty() {
|
||||
println!("All data is Ok")
|
||||
} else {
|
||||
println!("Corrupted info are as follows:\nraft_group_id, last_intact_index\n");
|
||||
r.iter().for_each(|(x, y)| println!("{:?}, {:?}", x, y))
|
||||
r.iter().for_each(|(x, y)| println!("{x:?}, {y:?}"))
|
||||
}
|
||||
}
|
||||
Cmd::TryPurge { path } => {
|
||||
let e = Engine::open(raft_engine::Config {
|
||||
dir: path,
|
||||
..Default::default()
|
||||
})?;
|
||||
let e = Engine::open_with_file_system(
|
||||
raft_engine::Config {
|
||||
dir: path,
|
||||
..Default::default()
|
||||
},
|
||||
fs,
|
||||
)?;
|
||||
println!(
|
||||
"purge_expired_files() returns {:?}",
|
||||
e.purge_expired_files()?
|
||||
|
@ -138,10 +155,10 @@ impl ControlOpt {
|
|||
}
|
||||
}
|
||||
|
||||
pub fn run_command(mut args: Vec<String>) {
|
||||
pub fn run_command<F: FileSystem>(mut args: Vec<String>, fs: Arc<F>) {
|
||||
args.insert(0, "ctl".to_owned());
|
||||
let opts = ControlOpt::parse_from(args);
|
||||
if let Err(e) = opts.validate_and_execute() {
|
||||
println!("{:?}", e);
|
||||
if let Err(e) = opts.validate_and_execute_with_file_system(fs) {
|
||||
println!("{e:?}");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -8,6 +8,6 @@ fn main() {
|
|||
let opts: ControlOpt = ControlOpt::parse();
|
||||
|
||||
if let Err(e) = opts.validate_and_execute() {
|
||||
println!("{:?}", e);
|
||||
println!("{e:?}");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -51,13 +51,14 @@ fn main() {
|
|||
loop {
|
||||
for _ in 0..1024 {
|
||||
let region = rand_regions.next().unwrap();
|
||||
let state = engine
|
||||
let mut state = engine
|
||||
.get_message::<RaftLocalState>(region, b"last_index")
|
||||
.unwrap()
|
||||
.unwrap_or_else(|| init_state.clone());
|
||||
|
||||
state.last_index += 1; // manually update the state
|
||||
let mut e = entry.clone();
|
||||
e.index = state.last_index + 1;
|
||||
e.index = state.last_index;
|
||||
batch.add_entries::<MessageExtTyped>(region, &[e]).unwrap();
|
||||
batch
|
||||
.put_message(region, b"last_index".to_vec(), &state)
|
||||
|
@ -69,7 +70,7 @@ fn main() {
|
|||
if state.last_index > rand_compact_offset {
|
||||
let compact_to = state.last_index - rand_compact_offset;
|
||||
engine.compact_to(region, compact_to);
|
||||
println!("[EXAMPLE] compact {} to {}", region, compact_to);
|
||||
println!("[EXAMPLE] compact {region} to {compact_to}");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -80,8 +81,7 @@ fn main() {
|
|||
.unwrap();
|
||||
engine.compact_to(region, state.last_index - 7);
|
||||
println!(
|
||||
"[EXAMPLE] force compact {} to {}",
|
||||
region,
|
||||
"[EXAMPLE] force compact {region} to {}",
|
||||
state.last_index - 7
|
||||
);
|
||||
}
|
||||
|
|
|
@ -0,0 +1,27 @@
|
|||
use std::path::Path;
|
||||
use std::sync::Arc;
|
||||
|
||||
use raft_engine::env::DefaultFileSystem;
|
||||
use raft_engine::Config;
|
||||
use raft_engine::Engine;
|
||||
|
||||
fn main() {
|
||||
let mut args = std::env::args();
|
||||
let arg0 = args.next().unwrap();
|
||||
let prog = Path::new(&arg0)
|
||||
.file_name()
|
||||
.and_then(|x| x.to_str())
|
||||
.unwrap();
|
||||
println!("usage: {prog} {{source}} {{target}}");
|
||||
|
||||
let source = args.next().unwrap();
|
||||
let target = args.next().unwrap();
|
||||
|
||||
let cfg = Config {
|
||||
dir: source,
|
||||
..Default::default()
|
||||
};
|
||||
let fs = Arc::new(DefaultFileSystem);
|
||||
Engine::<_, _>::fork(&cfg, fs, target).unwrap();
|
||||
println!("success!");
|
||||
}
|
|
@ -360,7 +360,6 @@ mod tests {
|
|||
use super::*;
|
||||
use protobuf::CodedOutputStream;
|
||||
use std::io::ErrorKind;
|
||||
use std::{f32, f64, i16, i32, i64, u16, u32, u64};
|
||||
|
||||
const U16_TESTS: &[u16] = &[
|
||||
i16::MIN as u16,
|
||||
|
@ -670,6 +669,7 @@ mod tests {
|
|||
decode_var_u64(&mut buf.as_slice()),
|
||||
ErrorKind::UnexpectedEof
|
||||
);
|
||||
check_error!(decode_var_u64(&mut [].as_slice()), ErrorKind::UnexpectedEof);
|
||||
|
||||
buf.push(0);
|
||||
assert_eq!(0, decode_var_u64(&mut buf.as_slice()).unwrap());
|
||||
|
|
194
src/config.rs
194
src/config.rs
|
@ -1,14 +1,15 @@
|
|||
// Copyright (c) 2017-present, PingCAP, Inc. Licensed under Apache-2.0.
|
||||
|
||||
use log::warn;
|
||||
use log::{info, warn};
|
||||
use serde::{Deserialize, Serialize};
|
||||
|
||||
use crate::pipe_log::Version;
|
||||
use crate::{util::ReadableSize, Result};
|
||||
|
||||
const MIN_RECOVERY_READ_BLOCK_SIZE: usize = 512;
|
||||
const MIN_RECOVERY_THREADS: usize = 1;
|
||||
|
||||
#[derive(Clone, Copy, Debug, Eq, PartialEq, Serialize, Deserialize)]
|
||||
#[derive(Clone, Copy, Debug, PartialEq, Eq, Serialize, Deserialize)]
|
||||
#[serde(rename_all = "kebab-case")]
|
||||
pub enum RecoveryMode {
|
||||
AbsoluteConsistency,
|
||||
|
@ -25,18 +26,27 @@ pub enum RecoveryMode {
|
|||
#[serde(default)]
|
||||
#[serde(rename_all = "kebab-case")]
|
||||
pub struct Config {
|
||||
/// Directory to store log files. Will create on startup if not exists.
|
||||
/// Main directory to store log files. Will create on startup if not exists.
|
||||
///
|
||||
/// Default: ""
|
||||
pub dir: String,
|
||||
|
||||
/// Auxiliary directory to store log files. Will create on startup if
|
||||
/// set but not exists.
|
||||
///
|
||||
/// Newly logs will be put into this dir when the main `dir` is full
|
||||
/// and no spare space for new logs.
|
||||
///
|
||||
/// Default: None
|
||||
pub spill_dir: Option<String>,
|
||||
|
||||
/// How to deal with file corruption during recovery.
|
||||
///
|
||||
/// Default: "tolerate-tail-corruption".
|
||||
pub recovery_mode: RecoveryMode,
|
||||
/// Minimum I/O size for reading log files during recovery.
|
||||
///
|
||||
/// Default: "4KB". Minimum: "512B".
|
||||
/// Default: "16KB". Minimum: "512B".
|
||||
pub recovery_read_block_size: ReadableSize,
|
||||
/// The number of threads used to scan and recovery log files.
|
||||
///
|
||||
|
@ -48,11 +58,24 @@ pub struct Config {
|
|||
///
|
||||
/// Default: "8KB"
|
||||
pub batch_compression_threshold: ReadableSize,
|
||||
/// Acceleration factor for LZ4 compression. It can be fine tuned, with each
|
||||
/// successive value providing roughly +~3% to speed. The value will be
|
||||
/// capped within [1, 65537] by LZ4.
|
||||
///
|
||||
/// Default: 1.
|
||||
pub compression_level: Option<usize>,
|
||||
/// Deprecated.
|
||||
/// Incrementally sync log files after specified bytes have been written.
|
||||
/// Setting it to zero disables incremental sync.
|
||||
///
|
||||
/// Default: "4MB"
|
||||
pub bytes_per_sync: ReadableSize,
|
||||
pub bytes_per_sync: Option<ReadableSize>,
|
||||
|
||||
/// Version of the log file.
|
||||
///
|
||||
/// Default: 2
|
||||
pub format_version: Version,
|
||||
|
||||
/// Target file size for rotating log files.
|
||||
///
|
||||
/// Default: "128MB"
|
||||
|
@ -76,6 +99,28 @@ pub struct Config {
|
|||
///
|
||||
/// Default: None
|
||||
pub memory_limit: Option<ReadableSize>,
|
||||
|
||||
/// Whether to recycle stale log files.
|
||||
/// If `true`, logically purged log files will be reserved for recycling.
|
||||
/// Only available for `format_version` 2 and above.
|
||||
///
|
||||
/// Default: true
|
||||
pub enable_log_recycle: bool,
|
||||
|
||||
/// Whether to prepare log files for recycling when start.
|
||||
/// If `true`, batch empty log files will be prepared for recycling when
|
||||
/// starting engine.
|
||||
/// Only available for `enable-log-reycle` is true.
|
||||
///
|
||||
/// Default: false
|
||||
pub prefill_for_recycle: bool,
|
||||
|
||||
/// Maximum capacity for preparing log files for recycling when start.
|
||||
/// If `None`, its size is equal to `purge-threshold`*1.5.
|
||||
/// Only available for `prefill-for-recycle` is true.
|
||||
///
|
||||
/// Default: None
|
||||
pub prefill_limit: Option<ReadableSize>,
|
||||
}
|
||||
|
||||
impl Default for Config {
|
||||
|
@ -83,16 +128,22 @@ impl Default for Config {
|
|||
#[allow(unused_mut)]
|
||||
let mut cfg = Config {
|
||||
dir: "".to_owned(),
|
||||
spill_dir: None,
|
||||
recovery_mode: RecoveryMode::TolerateTailCorruption,
|
||||
recovery_read_block_size: ReadableSize::kb(16),
|
||||
recovery_threads: 4,
|
||||
batch_compression_threshold: ReadableSize::kb(8),
|
||||
bytes_per_sync: ReadableSize::mb(4),
|
||||
compression_level: None,
|
||||
bytes_per_sync: None,
|
||||
format_version: Version::V2,
|
||||
target_file_size: ReadableSize::mb(128),
|
||||
purge_threshold: ReadableSize::gb(10),
|
||||
purge_rewrite_threshold: None,
|
||||
purge_rewrite_garbage_ratio: 0.6,
|
||||
memory_limit: None,
|
||||
enable_log_recycle: true,
|
||||
prefill_for_recycle: false,
|
||||
prefill_limit: None,
|
||||
};
|
||||
// Test-specific configurations.
|
||||
#[cfg(test)]
|
||||
|
@ -114,30 +165,89 @@ impl Config {
|
|||
self.target_file_size.0,
|
||||
)));
|
||||
}
|
||||
if self.bytes_per_sync.0 == 0 {
|
||||
self.bytes_per_sync = ReadableSize(u64::MAX);
|
||||
if self.bytes_per_sync.is_some() {
|
||||
warn!("bytes-per-sync has been deprecated.");
|
||||
}
|
||||
let min_recovery_read_block_size = ReadableSize(MIN_RECOVERY_READ_BLOCK_SIZE as u64);
|
||||
if self.recovery_read_block_size < min_recovery_read_block_size {
|
||||
warn!(
|
||||
"recovery-read-block-size ({}) is too small, setting it to {}",
|
||||
self.recovery_read_block_size, min_recovery_read_block_size
|
||||
"recovery-read-block-size ({}) is too small, setting it to {min_recovery_read_block_size}",
|
||||
self.recovery_read_block_size
|
||||
);
|
||||
self.recovery_read_block_size = min_recovery_read_block_size;
|
||||
}
|
||||
if self.recovery_threads < MIN_RECOVERY_THREADS {
|
||||
warn!(
|
||||
"recovery-threads ({}) is too small, setting it to {}",
|
||||
self.recovery_threads, MIN_RECOVERY_THREADS
|
||||
"recovery-threads ({}) is too small, setting it to {MIN_RECOVERY_THREADS}",
|
||||
self.recovery_threads
|
||||
);
|
||||
self.recovery_threads = MIN_RECOVERY_THREADS;
|
||||
}
|
||||
if self.enable_log_recycle && !self.format_version.has_log_signing() {
|
||||
return Err(box_err!(
|
||||
"format version {} doesn't support log recycle, use 2 or above",
|
||||
self.format_version
|
||||
));
|
||||
}
|
||||
if !self.enable_log_recycle && self.prefill_for_recycle {
|
||||
return Err(box_err!(
|
||||
"prefill is not allowed when log recycle is disabled"
|
||||
));
|
||||
}
|
||||
if !self.prefill_for_recycle && self.prefill_limit.is_some() {
|
||||
warn!("prefill-limit will be ignored when prefill is disabled");
|
||||
self.prefill_limit = None;
|
||||
}
|
||||
if self.prefill_for_recycle && self.prefill_limit.is_none() {
|
||||
info!("prefill-limit will be calibrated to purge-threshold");
|
||||
self.prefill_limit = Some(self.purge_threshold);
|
||||
}
|
||||
#[cfg(not(feature = "swap"))]
|
||||
if self.memory_limit.is_some() {
|
||||
warn!("memory-limit will be ignored because swap feature is not enabled");
|
||||
warn!("memory-limit will be ignored because swap feature is disabled");
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Returns the capacity for recycling log files.
|
||||
pub(crate) fn recycle_capacity(&self) -> usize {
|
||||
// Attention please, log files with Version::V1 could not be recycled, it might
|
||||
// cause LogBatchs in a mess in the recycled file, where the reader might get
|
||||
// an obsolete entries (unexpected) from the recycled file.
|
||||
if !self.format_version.has_log_signing() {
|
||||
return 0;
|
||||
}
|
||||
if self.enable_log_recycle && self.purge_threshold.0 >= self.target_file_size.0 {
|
||||
// (1) At most u32::MAX so that the file number can be capped into an u32
|
||||
// without colliding. (2) Increase the threshold by 50% to add some more file
|
||||
// as an additional buffer to avoid jitters.
|
||||
std::cmp::min(
|
||||
(self.purge_threshold.0 / self.target_file_size.0) as usize * 3 / 2,
|
||||
u32::MAX as usize,
|
||||
)
|
||||
} else {
|
||||
0
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns the capacity for preparing log files for recycling when start.
|
||||
pub(crate) fn prefill_capacity(&self) -> usize {
|
||||
// Attention please, log files with Version::V1 could not be recycled, so it's
|
||||
// useless for prefill.
|
||||
if !self.enable_log_recycle || !self.format_version.has_log_signing() {
|
||||
return 0;
|
||||
}
|
||||
let prefill_limit = self.prefill_limit.unwrap_or(ReadableSize(0)).0;
|
||||
if self.prefill_for_recycle && prefill_limit >= self.target_file_size.0 {
|
||||
// Keep same with the maximum setting of `recycle_capacity`.
|
||||
std::cmp::min(
|
||||
(prefill_limit / self.target_file_size.0) as usize * 3 / 2,
|
||||
u32::MAX as usize,
|
||||
)
|
||||
} else {
|
||||
0
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
|
@ -150,23 +260,33 @@ mod tests {
|
|||
let dump = toml::to_string_pretty(&value).unwrap();
|
||||
let load = toml::from_str(&dump).unwrap();
|
||||
assert_eq!(value, load);
|
||||
assert!(load.spill_dir.is_none());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_custom() {
|
||||
let custom = r#"
|
||||
dir = "custom_dir"
|
||||
spill-dir = "custom_spill_dir"
|
||||
recovery-mode = "tolerate-tail-corruption"
|
||||
bytes-per-sync = "2KB"
|
||||
target-file-size = "1MB"
|
||||
purge-threshold = "3MB"
|
||||
format-version = 1
|
||||
enable-log-recycle = false
|
||||
prefill-for-recycle = false
|
||||
"#;
|
||||
let load: Config = toml::from_str(custom).unwrap();
|
||||
let mut load: Config = toml::from_str(custom).unwrap();
|
||||
assert_eq!(load.dir, "custom_dir");
|
||||
assert_eq!(load.spill_dir, Some("custom_spill_dir".to_owned()));
|
||||
assert_eq!(load.recovery_mode, RecoveryMode::TolerateTailCorruption);
|
||||
assert_eq!(load.bytes_per_sync, ReadableSize::kb(2));
|
||||
assert_eq!(load.bytes_per_sync, Some(ReadableSize::kb(2)));
|
||||
assert_eq!(load.target_file_size, ReadableSize::mb(1));
|
||||
assert_eq!(load.purge_threshold, ReadableSize::mb(3));
|
||||
assert_eq!(load.format_version, Version::V1);
|
||||
assert_eq!(load.enable_log_recycle, false);
|
||||
assert_eq!(load.prefill_for_recycle, false);
|
||||
load.sanitize().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
|
@ -179,21 +299,39 @@ mod tests {
|
|||
assert!(hard_load.sanitize().is_err());
|
||||
|
||||
let soft_error = r#"
|
||||
recovery-read-block-size = "1KB"
|
||||
recovery-read-block-size = 1
|
||||
recovery-threads = 0
|
||||
bytes-per-sync = "0KB"
|
||||
target-file-size = "5000MB"
|
||||
format-version = 2
|
||||
enable-log-recycle = true
|
||||
prefill-for-recycle = true
|
||||
"#;
|
||||
let soft_load: Config = toml::from_str(soft_error).unwrap();
|
||||
assert!(soft_load.recovery_read_block_size.0 < MIN_RECOVERY_READ_BLOCK_SIZE as u64);
|
||||
assert!(soft_load.recovery_threads < MIN_RECOVERY_THREADS);
|
||||
let mut soft_sanitized = soft_load;
|
||||
soft_sanitized.sanitize().unwrap();
|
||||
assert!(soft_sanitized.recovery_read_block_size.0 >= MIN_RECOVERY_READ_BLOCK_SIZE as u64);
|
||||
assert!(soft_sanitized.recovery_threads >= MIN_RECOVERY_THREADS);
|
||||
assert_eq!(soft_sanitized.bytes_per_sync.0, u64::MAX);
|
||||
assert_eq!(
|
||||
soft_sanitized.purge_rewrite_threshold.unwrap(),
|
||||
soft_sanitized.target_file_size
|
||||
);
|
||||
|
||||
let recycle_error = r#"
|
||||
enable-log-recycle = true
|
||||
format-version = 1
|
||||
"#;
|
||||
let mut cfg_load: Config = toml::from_str(recycle_error).unwrap();
|
||||
assert!(cfg_load.sanitize().is_err());
|
||||
|
||||
let prefill_error = r#"
|
||||
enable-log-recycle = false
|
||||
prefill-for-recycle = true
|
||||
format-version = 2
|
||||
"#;
|
||||
let mut cfg_load: Config = toml::from_str(prefill_error).unwrap();
|
||||
assert!(cfg_load.sanitize().is_err());
|
||||
}
|
||||
|
||||
#[test]
|
||||
|
@ -209,4 +347,24 @@ mod tests {
|
|||
.unwrap()
|
||||
.contains("tolerate-corrupted-tail-records"));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_prefill_for_recycle() {
|
||||
let default_prefill_v1 = r#"
|
||||
enable-log-recycle = true
|
||||
prefill-for-recycle = true
|
||||
"#;
|
||||
let mut cfg_load: Config = toml::from_str(default_prefill_v1).unwrap();
|
||||
assert!(cfg_load.sanitize().is_ok());
|
||||
assert_eq!(cfg_load.prefill_limit.unwrap(), cfg_load.purge_threshold);
|
||||
|
||||
let default_prefill_v2 = r#"
|
||||
enable-log-recycle = true
|
||||
prefill-for-recycle = false
|
||||
prefill-limit = "20GB"
|
||||
"#;
|
||||
let mut cfg_load: Config = toml::from_str(default_prefill_v2).unwrap();
|
||||
assert!(cfg_load.sanitize().is_ok());
|
||||
assert!(cfg_load.prefill_limit.is_none());
|
||||
}
|
||||
}
|
||||
|
|
1461
src/engine.rs
1461
src/engine.rs
File diff suppressed because it is too large
Load Diff
|
@ -1,197 +1,15 @@
|
|||
// Copyright (c) 2017-present, PingCAP, Inc. Licensed under Apache-2.0.
|
||||
|
||||
#[cfg(feature = "failpoints")]
|
||||
use std::io::{Error, ErrorKind};
|
||||
use std::io::{Read, Result as IoResult, Seek, SeekFrom, Write};
|
||||
use std::os::unix::io::RawFd;
|
||||
use std::path::Path;
|
||||
use std::sync::Arc;
|
||||
|
||||
use fail::fail_point;
|
||||
use log::error;
|
||||
use nix::errno::Errno;
|
||||
use nix::fcntl::{self, OFlag};
|
||||
use nix::sys::stat::Mode;
|
||||
use nix::sys::uio::{pread, pwrite};
|
||||
use nix::unistd::{close, ftruncate, lseek, Whence};
|
||||
use nix::NixPath;
|
||||
|
||||
use crate::env::{FileSystem, Handle, WriteExt};
|
||||
|
||||
fn from_nix_error(e: nix::Error, custom: &'static str) -> std::io::Error {
|
||||
let kind = std::io::Error::from(e).kind();
|
||||
std::io::Error::new(kind, custom)
|
||||
}
|
||||
|
||||
/// A RAII-style low-level file. Errors occurred during automatic resource
|
||||
/// release are logged and ignored.
|
||||
///
|
||||
/// A [`LogFd`] is essentially a thin wrapper around [`RawFd`]. It's only
|
||||
/// supported on *Unix*, and primarily optimized for *Linux*.
|
||||
///
|
||||
/// All [`LogFd`] instances are opened with read and write permission.
|
||||
pub struct LogFd(RawFd);
|
||||
|
||||
impl LogFd {
|
||||
/// Opens a file with the given `path`.
|
||||
pub fn open<P: ?Sized + NixPath>(path: &P) -> IoResult<Self> {
|
||||
fail_point!("log_fd::open::err", |_| {
|
||||
Err(from_nix_error(nix::Error::EINVAL, "fp"))
|
||||
});
|
||||
let flags = OFlag::O_RDWR;
|
||||
// Permission 644
|
||||
let mode = Mode::S_IRUSR | Mode::S_IWUSR | Mode::S_IRGRP | Mode::S_IROTH;
|
||||
fail_point!("log_fd::open::fadvise_dontneed", |_| {
|
||||
let fd = LogFd(fcntl::open(path, flags, mode).map_err(|e| from_nix_error(e, "open"))?);
|
||||
#[cfg(target_os = "linux")]
|
||||
unsafe {
|
||||
extern crate libc;
|
||||
libc::posix_fadvise64(fd.0, 0, fd.file_size()? as i64, libc::POSIX_FADV_DONTNEED);
|
||||
}
|
||||
Ok(fd)
|
||||
});
|
||||
Ok(LogFd(
|
||||
fcntl::open(path, flags, mode).map_err(|e| from_nix_error(e, "open"))?,
|
||||
))
|
||||
}
|
||||
|
||||
/// Opens a file with the given `path`. The specified file will be created
|
||||
/// first if not exists.
|
||||
pub fn create<P: ?Sized + NixPath>(path: &P) -> IoResult<Self> {
|
||||
fail_point!("log_fd::create::err", |_| {
|
||||
Err(from_nix_error(nix::Error::EINVAL, "fp"))
|
||||
});
|
||||
let flags = OFlag::O_RDWR | OFlag::O_CREAT;
|
||||
// Permission 644
|
||||
let mode = Mode::S_IRUSR | Mode::S_IWUSR | Mode::S_IRGRP | Mode::S_IROTH;
|
||||
let fd = fcntl::open(path, flags, mode).map_err(|e| from_nix_error(e, "open"))?;
|
||||
Ok(LogFd(fd))
|
||||
}
|
||||
|
||||
/// Closes the file.
|
||||
pub fn close(&self) -> IoResult<()> {
|
||||
fail_point!("log_fd::close::err", |_| {
|
||||
Err(from_nix_error(nix::Error::EINVAL, "fp"))
|
||||
});
|
||||
close(self.0).map_err(|e| from_nix_error(e, "close"))
|
||||
}
|
||||
|
||||
/// Synchronizes all in-memory data of the file except metadata to the
|
||||
/// filesystem.
|
||||
pub fn sync(&self) -> IoResult<()> {
|
||||
fail_point!("log_fd::sync::err", |_| {
|
||||
Err(from_nix_error(nix::Error::EINVAL, "fp"))
|
||||
});
|
||||
#[cfg(target_os = "linux")]
|
||||
{
|
||||
nix::unistd::fdatasync(self.0).map_err(|e| from_nix_error(e, "fdatasync"))
|
||||
}
|
||||
#[cfg(not(target_os = "linux"))]
|
||||
{
|
||||
nix::unistd::fsync(self.0).map_err(|e| from_nix_error(e, "fsync"))
|
||||
}
|
||||
}
|
||||
|
||||
/// Reads some bytes starting at `offset` from this file into the specified
|
||||
/// buffer. Returns how many bytes were read.
|
||||
pub fn read(&self, mut offset: usize, buf: &mut [u8]) -> IoResult<usize> {
|
||||
let mut readed = 0;
|
||||
while readed < buf.len() {
|
||||
fail_point!("log_fd::read::err", |_| {
|
||||
Err(from_nix_error(nix::Error::EINVAL, "fp"))
|
||||
});
|
||||
let bytes = match pread(self.0, &mut buf[readed..], offset as i64) {
|
||||
Ok(bytes) => bytes,
|
||||
Err(e) if e == Errno::EAGAIN => continue,
|
||||
Err(e) => return Err(from_nix_error(e, "pread")),
|
||||
};
|
||||
// EOF
|
||||
if bytes == 0 {
|
||||
break;
|
||||
}
|
||||
readed += bytes;
|
||||
offset += bytes;
|
||||
}
|
||||
Ok(readed)
|
||||
}
|
||||
|
||||
/// Writes some bytes to this file starting at `offset`. Returns how many
|
||||
/// bytes were written.
|
||||
pub fn write(&self, mut offset: usize, content: &[u8]) -> IoResult<usize> {
|
||||
fail_point!("log_fd::write::zero", |_| { Ok(0) });
|
||||
let mut written = 0;
|
||||
while written < content.len() {
|
||||
let bytes = match pwrite(self.0, &content[written..], offset as i64) {
|
||||
Ok(bytes) => bytes,
|
||||
Err(e) if e == Errno::EAGAIN => continue,
|
||||
Err(e) => return Err(from_nix_error(e, "pwrite")),
|
||||
};
|
||||
if bytes == 0 {
|
||||
break;
|
||||
}
|
||||
written += bytes;
|
||||
offset += bytes;
|
||||
}
|
||||
fail_point!("log_fd::write::err", |_| {
|
||||
Err(from_nix_error(nix::Error::EINVAL, "fp"))
|
||||
});
|
||||
Ok(written)
|
||||
}
|
||||
|
||||
/// Truncates all data after `offset`.
|
||||
pub fn truncate(&self, offset: usize) -> IoResult<()> {
|
||||
fail_point!("log_fd::truncate::err", |_| {
|
||||
Err(from_nix_error(nix::Error::EINVAL, "fp"))
|
||||
});
|
||||
ftruncate(self.0, offset as i64).map_err(|e| from_nix_error(e, "ftruncate"))
|
||||
}
|
||||
|
||||
/// Attempts to allocate space for `size` bytes starting at `offset`.
|
||||
#[allow(unused_variables)]
|
||||
pub fn allocate(&self, offset: usize, size: usize) -> IoResult<()> {
|
||||
fail_point!("log_fd::allocate::err", |_| {
|
||||
Err(from_nix_error(nix::Error::EINVAL, "fp"))
|
||||
});
|
||||
#[cfg(target_os = "linux")]
|
||||
{
|
||||
fcntl::fallocate(
|
||||
self.0,
|
||||
fcntl::FallocateFlags::empty(),
|
||||
offset as i64,
|
||||
size as i64,
|
||||
)
|
||||
.map_err(|e| from_nix_error(e, "fallocate"))
|
||||
}
|
||||
#[cfg(not(target_os = "linux"))]
|
||||
{
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Handle for LogFd {
|
||||
fn truncate(&self, offset: usize) -> IoResult<()> {
|
||||
fail_point!("log_fd::truncate::err", |_| {
|
||||
Err(from_nix_error(nix::Error::EINVAL, "fp"))
|
||||
});
|
||||
ftruncate(self.0, offset as i64).map_err(|e| from_nix_error(e, "ftruncate"))
|
||||
}
|
||||
|
||||
fn file_size(&self) -> IoResult<usize> {
|
||||
fail_point!("log_fd::file_size::err", |_| {
|
||||
Err(from_nix_error(nix::Error::EINVAL, "fp"))
|
||||
});
|
||||
lseek(self.0, 0, Whence::SeekEnd)
|
||||
.map(|n| n as usize)
|
||||
.map_err(|e| from_nix_error(e, "lseek"))
|
||||
}
|
||||
}
|
||||
|
||||
impl Drop for LogFd {
|
||||
fn drop(&mut self) {
|
||||
if let Err(e) = self.close() {
|
||||
error!("error while closing file: {}", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
use crate::env::log_fd::LogFd;
|
||||
use crate::env::{FileSystem, Handle, Permission, WriteExt};
|
||||
|
||||
/// A low-level file adapted for standard interfaces including [`Seek`],
|
||||
/// [`Write`] and [`Read`].
|
||||
|
@ -212,7 +30,14 @@ impl LogFile {
|
|||
|
||||
impl Write for LogFile {
|
||||
fn write(&mut self, buf: &[u8]) -> IoResult<usize> {
|
||||
fail_point!("log_file::write::zero", |_| { Ok(0) });
|
||||
|
||||
let len = self.inner.write(self.offset, buf)?;
|
||||
|
||||
fail_point!("log_file::write::err", |_| {
|
||||
Err(Error::new(ErrorKind::InvalidInput, "fp"))
|
||||
});
|
||||
|
||||
self.offset += len;
|
||||
Ok(len)
|
||||
}
|
||||
|
@ -224,6 +49,10 @@ impl Write for LogFile {
|
|||
|
||||
impl Read for LogFile {
|
||||
fn read(&mut self, buf: &mut [u8]) -> IoResult<usize> {
|
||||
fail_point!("log_file::read::err", |_| {
|
||||
Err(Error::new(ErrorKind::InvalidInput, "fp"))
|
||||
});
|
||||
|
||||
let len = self.inner.read(self.offset, buf)?;
|
||||
self.offset += len;
|
||||
Ok(len)
|
||||
|
@ -232,6 +61,9 @@ impl Read for LogFile {
|
|||
|
||||
impl Seek for LogFile {
|
||||
fn seek(&mut self, pos: SeekFrom) -> IoResult<u64> {
|
||||
fail_point!("log_file::seek::err", |_| {
|
||||
Err(std::io::Error::other("fp"))
|
||||
});
|
||||
match pos {
|
||||
SeekFrom::Start(offset) => self.offset = offset as usize,
|
||||
SeekFrom::Current(i) => self.offset = (self.offset as i64 + i) as usize,
|
||||
|
@ -243,16 +75,20 @@ impl Seek for LogFile {
|
|||
|
||||
impl WriteExt for LogFile {
|
||||
fn truncate(&mut self, offset: usize) -> IoResult<()> {
|
||||
fail_point!("log_file::truncate::err", |_| {
|
||||
Err(Error::new(ErrorKind::InvalidInput, "fp"))
|
||||
});
|
||||
|
||||
self.inner.truncate(offset)?;
|
||||
self.offset = offset;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn sync(&mut self) -> IoResult<()> {
|
||||
self.inner.sync()
|
||||
}
|
||||
|
||||
fn allocate(&mut self, offset: usize, size: usize) -> IoResult<()> {
|
||||
fail_point!("log_file::allocate::err", |_| {
|
||||
Err(Error::new(ErrorKind::InvalidInput, "fp"))
|
||||
});
|
||||
|
||||
self.inner.allocate(offset, size)
|
||||
}
|
||||
}
|
||||
|
@ -265,11 +101,28 @@ impl FileSystem for DefaultFileSystem {
|
|||
type Writer = LogFile;
|
||||
|
||||
fn create<P: AsRef<Path>>(&self, path: P) -> IoResult<Self::Handle> {
|
||||
fail_point!("default_fs::create::err", |_| {
|
||||
Err(Error::new(ErrorKind::InvalidInput, "fp"))
|
||||
});
|
||||
|
||||
LogFd::create(path.as_ref())
|
||||
}
|
||||
|
||||
fn open<P: AsRef<Path>>(&self, path: P) -> IoResult<Self::Handle> {
|
||||
LogFd::open(path.as_ref())
|
||||
fn open<P: AsRef<Path>>(&self, path: P, perm: Permission) -> IoResult<Self::Handle> {
|
||||
fail_point!("default_fs::open::err", |_| {
|
||||
Err(Error::new(ErrorKind::InvalidInput, "fp"))
|
||||
});
|
||||
|
||||
LogFd::open(path.as_ref(), perm)
|
||||
}
|
||||
|
||||
fn delete<P: AsRef<Path>>(&self, path: P) -> IoResult<()> {
|
||||
fail_point!("default_fs::delete_skipped", |_| { Ok(()) });
|
||||
std::fs::remove_file(path)
|
||||
}
|
||||
|
||||
fn rename<P: AsRef<Path>>(&self, src_path: P, dst_path: P) -> IoResult<()> {
|
||||
std::fs::rename(src_path, dst_path)
|
||||
}
|
||||
|
||||
fn new_reader(&self, handle: Arc<Self::Handle>) -> IoResult<Self::Reader> {
|
||||
|
|
|
@ -0,0 +1,11 @@
|
|||
// Copyright (c) 2017-present, PingCAP, Inc. Licensed under Apache-2.0.
|
||||
|
||||
#[cfg(not(any(windows, feature = "std_fs")))]
|
||||
mod unix;
|
||||
#[cfg(not(any(windows, feature = "std_fs")))]
|
||||
pub use unix::LogFd;
|
||||
|
||||
#[cfg(any(windows, feature = "std_fs"))]
|
||||
mod plain;
|
||||
#[cfg(any(windows, feature = "std_fs"))]
|
||||
pub use plain::LogFd;
|
|
@ -0,0 +1,84 @@
|
|||
// Copyright (c) 2017-present, PingCAP, Inc. Licensed under Apache-2.0.
|
||||
|
||||
//! A naive file handle implementation based on standard `File`. All I/O
|
||||
//! operations need to synchronize under a `RwLock`.
|
||||
|
||||
use crate::env::{Handle, Permission};
|
||||
|
||||
use fail::fail_point;
|
||||
use parking_lot::RwLock;
|
||||
|
||||
use std::fs::{File, OpenOptions};
|
||||
use std::io::{Error, ErrorKind, Read, Result, Seek, SeekFrom, Write};
|
||||
use std::path::Path;
|
||||
use std::sync::Arc;
|
||||
|
||||
pub struct LogFd(Arc<RwLock<File>>);
|
||||
|
||||
impl LogFd {
|
||||
pub fn open<P: AsRef<Path>>(path: P, _: Permission) -> Result<Self> {
|
||||
OpenOptions::new()
|
||||
.read(true)
|
||||
.write(true)
|
||||
.open(path)
|
||||
.map(|x| Self(Arc::new(RwLock::new(x))))
|
||||
}
|
||||
|
||||
pub fn create<P: AsRef<Path>>(path: P) -> Result<Self> {
|
||||
OpenOptions::new()
|
||||
.create(true)
|
||||
.read(true)
|
||||
.write(true)
|
||||
.open(path)
|
||||
.map(|x| Self(Arc::new(RwLock::new(x))))
|
||||
}
|
||||
|
||||
pub fn read(&self, offset: usize, buf: &mut [u8]) -> Result<usize> {
|
||||
let mut file = self.0.write();
|
||||
let _ = file.seek(SeekFrom::Start(offset as u64))?;
|
||||
file.read(buf)
|
||||
}
|
||||
|
||||
pub fn write(&self, offset: usize, content: &[u8]) -> Result<usize> {
|
||||
fail_point!("log_fd::write::no_space_err", |_| {
|
||||
Err(Error::new(ErrorKind::Other, "nospace"))
|
||||
});
|
||||
|
||||
let mut file = self.0.write();
|
||||
let _ = file.seek(SeekFrom::Start(offset as u64))?;
|
||||
file.write(content)
|
||||
}
|
||||
|
||||
pub fn truncate(&self, offset: usize) -> Result<()> {
|
||||
let file = self.0.write();
|
||||
file.set_len(offset as u64)
|
||||
}
|
||||
|
||||
pub fn allocate(&self, _offset: usize, _size: usize) -> Result<()> {
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
impl Handle for LogFd {
|
||||
fn truncate(&self, offset: usize) -> Result<()> {
|
||||
self.truncate(offset)
|
||||
}
|
||||
|
||||
fn file_size(&self) -> Result<usize> {
|
||||
fail_point!("log_fd::file_size::err", |_| {
|
||||
Err(Error::new(ErrorKind::InvalidInput, "fp"))
|
||||
});
|
||||
|
||||
let file = self.0.read();
|
||||
file.metadata().map(|x| x.len() as usize)
|
||||
}
|
||||
|
||||
fn sync(&self) -> Result<()> {
|
||||
fail_point!("log_fd::sync::err", |_| {
|
||||
Err(Error::new(ErrorKind::InvalidInput, "fp"))
|
||||
});
|
||||
|
||||
let file = self.0.write();
|
||||
file.sync_all()
|
||||
}
|
||||
}
|
|
@ -0,0 +1,185 @@
|
|||
// Copyright (c) 2017-present, PingCAP, Inc. Licensed under Apache-2.0.
|
||||
|
||||
use crate::env::{Handle, Permission};
|
||||
|
||||
use fail::fail_point;
|
||||
use log::error;
|
||||
|
||||
use std::io::Result as IoResult;
|
||||
use std::os::unix::io::RawFd;
|
||||
|
||||
use nix::errno::Errno;
|
||||
use nix::fcntl::{self, OFlag};
|
||||
use nix::sys::stat::Mode;
|
||||
use nix::sys::uio::{pread, pwrite};
|
||||
use nix::unistd::{close, ftruncate, lseek, Whence};
|
||||
use nix::NixPath;
|
||||
|
||||
fn from_nix_error(e: nix::Error, custom: &'static str) -> std::io::Error {
|
||||
let kind = std::io::Error::from(e).kind();
|
||||
std::io::Error::new(kind, custom)
|
||||
}
|
||||
|
||||
impl From<Permission> for OFlag {
|
||||
fn from(value: Permission) -> OFlag {
|
||||
match value {
|
||||
Permission::ReadOnly => OFlag::O_RDONLY,
|
||||
Permission::ReadWrite => OFlag::O_RDWR,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// A RAII-style low-level file. Errors occurred during automatic resource
|
||||
/// release are logged and ignored.
|
||||
///
|
||||
/// A [`LogFd`] is essentially a thin wrapper around [`RawFd`]. It's only
|
||||
/// supported on *Unix*, and primarily optimized for *Linux*.
|
||||
///
|
||||
/// All [`LogFd`] instances are opened with read and write permission.
|
||||
pub struct LogFd(RawFd);
|
||||
|
||||
impl LogFd {
|
||||
/// Opens a file with the given `path`.
|
||||
pub fn open<P: ?Sized + NixPath>(path: &P, perm: Permission) -> IoResult<Self> {
|
||||
// Permission 644
|
||||
let mode = Mode::S_IRUSR | Mode::S_IWUSR | Mode::S_IRGRP | Mode::S_IROTH;
|
||||
fail_point!("log_fd::open::fadvise_dontneed", |_| {
|
||||
let fd =
|
||||
LogFd(fcntl::open(path, perm.into(), mode).map_err(|e| from_nix_error(e, "open"))?);
|
||||
#[cfg(target_os = "linux")]
|
||||
unsafe {
|
||||
extern crate libc;
|
||||
libc::posix_fadvise64(fd.0, 0, fd.file_size()? as i64, libc::POSIX_FADV_DONTNEED);
|
||||
}
|
||||
Ok(fd)
|
||||
});
|
||||
Ok(LogFd(
|
||||
fcntl::open(path, perm.into(), mode).map_err(|e| from_nix_error(e, "open"))?,
|
||||
))
|
||||
}
|
||||
|
||||
/// Opens a file with the given `path`. The specified file will be created
|
||||
/// first if not exists.
|
||||
pub fn create<P: ?Sized + NixPath>(path: &P) -> IoResult<Self> {
|
||||
let flags = OFlag::O_RDWR | OFlag::O_CREAT;
|
||||
// Permission 644
|
||||
let mode = Mode::S_IRUSR | Mode::S_IWUSR | Mode::S_IRGRP | Mode::S_IROTH;
|
||||
let fd = fcntl::open(path, flags, mode).map_err(|e| from_nix_error(e, "open"))?;
|
||||
Ok(LogFd(fd))
|
||||
}
|
||||
|
||||
/// Closes the file.
|
||||
pub fn close(&self) -> IoResult<()> {
|
||||
fail_point!("log_fd::close::err", |_| {
|
||||
Err(from_nix_error(nix::Error::EINVAL, "fp"))
|
||||
});
|
||||
close(self.0).map_err(|e| from_nix_error(e, "close"))
|
||||
}
|
||||
|
||||
/// Reads some bytes starting at `offset` from this file into the specified
|
||||
/// buffer. Returns how many bytes were read.
|
||||
pub fn read(&self, mut offset: usize, buf: &mut [u8]) -> IoResult<usize> {
|
||||
let mut readed = 0;
|
||||
while readed < buf.len() {
|
||||
let bytes = match pread(self.0, &mut buf[readed..], offset as i64) {
|
||||
Ok(bytes) => bytes,
|
||||
Err(Errno::EINTR) => continue,
|
||||
Err(e) => return Err(from_nix_error(e, "pread")),
|
||||
};
|
||||
// EOF
|
||||
if bytes == 0 {
|
||||
break;
|
||||
}
|
||||
readed += bytes;
|
||||
offset += bytes;
|
||||
}
|
||||
Ok(readed)
|
||||
}
|
||||
|
||||
/// Writes some bytes to this file starting at `offset`. Returns how many
|
||||
/// bytes were written.
|
||||
pub fn write(&self, mut offset: usize, content: &[u8]) -> IoResult<usize> {
|
||||
fail_point!("log_fd::write::no_space_err", |_| {
|
||||
Err(from_nix_error(nix::Error::ENOSPC, "nospace"))
|
||||
});
|
||||
let mut written = 0;
|
||||
while written < content.len() {
|
||||
let bytes = match pwrite(self.0, &content[written..], offset as i64) {
|
||||
Ok(bytes) => bytes,
|
||||
Err(Errno::EINTR) => continue,
|
||||
Err(e) if e == Errno::ENOSPC => return Err(from_nix_error(e, "nospace")),
|
||||
Err(e) => return Err(from_nix_error(e, "pwrite")),
|
||||
};
|
||||
if bytes == 0 {
|
||||
break;
|
||||
}
|
||||
written += bytes;
|
||||
offset += bytes;
|
||||
}
|
||||
Ok(written)
|
||||
}
|
||||
|
||||
/// Truncates all data after `offset`.
|
||||
pub fn truncate(&self, offset: usize) -> IoResult<()> {
|
||||
ftruncate(self.0, offset as i64).map_err(|e| from_nix_error(e, "ftruncate"))
|
||||
}
|
||||
|
||||
/// Attempts to allocate space for `size` bytes starting at `offset`.
|
||||
#[allow(unused_variables)]
|
||||
pub fn allocate(&self, offset: usize, size: usize) -> IoResult<()> {
|
||||
#[cfg(target_os = "linux")]
|
||||
{
|
||||
if let Err(e) = fcntl::fallocate(
|
||||
self.0,
|
||||
fcntl::FallocateFlags::empty(),
|
||||
offset as i64,
|
||||
size as i64,
|
||||
) {
|
||||
if e != nix::Error::EOPNOTSUPP {
|
||||
return Err(from_nix_error(e, "fallocate"));
|
||||
}
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
impl Handle for LogFd {
|
||||
#[inline]
|
||||
fn truncate(&self, offset: usize) -> IoResult<()> {
|
||||
self.truncate(offset)
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn file_size(&self) -> IoResult<usize> {
|
||||
fail_point!("log_fd::file_size::err", |_| {
|
||||
Err(from_nix_error(nix::Error::EINVAL, "fp"))
|
||||
});
|
||||
lseek(self.0, 0, Whence::SeekEnd)
|
||||
.map(|n| n as usize)
|
||||
.map_err(|e| from_nix_error(e, "lseek"))
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn sync(&self) -> IoResult<()> {
|
||||
fail_point!("log_fd::sync::err", |_| {
|
||||
Err(from_nix_error(nix::Error::EINVAL, "fp"))
|
||||
});
|
||||
#[cfg(target_os = "linux")]
|
||||
{
|
||||
nix::unistd::fdatasync(self.0).map_err(|e| from_nix_error(e, "fdatasync"))
|
||||
}
|
||||
#[cfg(not(target_os = "linux"))]
|
||||
{
|
||||
nix::unistd::fsync(self.0).map_err(|e| from_nix_error(e, "fsync"))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Drop for LogFd {
|
||||
fn drop(&mut self) {
|
||||
if let Err(e) = self.close() {
|
||||
error!("error while closing file: {e}");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -5,11 +5,18 @@ use std::path::Path;
|
|||
use std::sync::Arc;
|
||||
|
||||
mod default;
|
||||
mod log_fd;
|
||||
mod obfuscated;
|
||||
|
||||
pub use default::DefaultFileSystem;
|
||||
pub use obfuscated::ObfuscatedFileSystem;
|
||||
|
||||
#[derive(Clone, Copy, PartialEq, Eq, Debug)]
|
||||
pub enum Permission {
|
||||
ReadOnly,
|
||||
ReadWrite,
|
||||
}
|
||||
|
||||
/// FileSystem
|
||||
pub trait FileSystem: Send + Sync {
|
||||
type Handle: Send + Sync + Handle;
|
||||
|
@ -17,8 +24,43 @@ pub trait FileSystem: Send + Sync {
|
|||
type Writer: Seek + Write + Send + WriteExt;
|
||||
|
||||
fn create<P: AsRef<Path>>(&self, path: P) -> Result<Self::Handle>;
|
||||
fn open<P: AsRef<Path>>(&self, path: P) -> Result<Self::Handle>;
|
||||
|
||||
fn open<P: AsRef<Path>>(&self, path: P, perm: Permission) -> Result<Self::Handle>;
|
||||
|
||||
fn delete<P: AsRef<Path>>(&self, path: P) -> Result<()>;
|
||||
|
||||
fn rename<P: AsRef<Path>>(&self, src_path: P, dst_path: P) -> Result<()>;
|
||||
|
||||
/// Reuses file at `src_path` as a new file at `dst_path`. The default
|
||||
/// implementation simply renames the file.
|
||||
fn reuse<P: AsRef<Path>>(&self, src_path: P, dst_path: P) -> Result<()> {
|
||||
self.rename(src_path, dst_path)
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn reuse_and_open<P: AsRef<Path>>(&self, src_path: P, dst_path: P) -> Result<Self::Handle> {
|
||||
self.reuse(src_path.as_ref(), dst_path.as_ref())?;
|
||||
self.open(dst_path, Permission::ReadWrite)
|
||||
}
|
||||
|
||||
/// Deletes user implemented metadata associated with `path`. Returns
|
||||
/// `true` if any metadata is deleted.
|
||||
///
|
||||
/// In older versions of Raft Engine, physical files are deleted without
|
||||
/// going through user implemented cleanup procedure. This method is used to
|
||||
/// detect and cleanup the user metadata that is no longer mapped to a
|
||||
/// physical file.
|
||||
fn delete_metadata<P: AsRef<Path>>(&self, _path: P) -> Result<()> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Returns whether there is any user metadata associated with given `path`.
|
||||
fn exists_metadata<P: AsRef<Path>>(&self, _path: P) -> bool {
|
||||
false
|
||||
}
|
||||
|
||||
fn new_reader(&self, handle: Arc<Self::Handle>) -> Result<Self::Reader>;
|
||||
|
||||
fn new_writer(&self, handle: Arc<Self::Handle>) -> Result<Self::Writer>;
|
||||
}
|
||||
|
||||
|
@ -27,11 +69,25 @@ pub trait Handle {
|
|||
|
||||
/// Returns the current size of this file.
|
||||
fn file_size(&self) -> Result<usize>;
|
||||
|
||||
fn sync(&self) -> Result<()>;
|
||||
}
|
||||
|
||||
/// WriteExt is writer extension api
|
||||
pub trait WriteExt {
|
||||
fn truncate(&mut self, offset: usize) -> Result<()>;
|
||||
fn sync(&mut self) -> Result<()>;
|
||||
fn allocate(&mut self, offset: usize, size: usize) -> Result<()>;
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::Permission;
|
||||
|
||||
#[test]
|
||||
fn test_copy_permission() {
|
||||
let perm = Permission::ReadWrite;
|
||||
let perm1 = perm;
|
||||
assert_eq!(perm, Permission::ReadWrite);
|
||||
assert_eq!(perm1, Permission::ReadWrite);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -2,9 +2,10 @@
|
|||
|
||||
use std::io::{Read, Result as IoResult, Seek, SeekFrom, Write};
|
||||
use std::path::Path;
|
||||
use std::sync::atomic::{AtomicUsize, Ordering};
|
||||
use std::sync::Arc;
|
||||
|
||||
use crate::env::{DefaultFileSystem, FileSystem, WriteExt};
|
||||
use crate::env::{DefaultFileSystem, FileSystem, Permission, WriteExt};
|
||||
|
||||
pub struct ObfuscatedReader(<DefaultFileSystem as FileSystem>::Reader);
|
||||
|
||||
|
@ -56,20 +57,31 @@ impl WriteExt for ObfuscatedWriter {
|
|||
self.0.truncate(offset)
|
||||
}
|
||||
|
||||
fn sync(&mut self) -> IoResult<()> {
|
||||
self.0.sync()
|
||||
}
|
||||
|
||||
fn allocate(&mut self, offset: usize, size: usize) -> IoResult<()> {
|
||||
self.0.allocate(offset, size)
|
||||
}
|
||||
}
|
||||
|
||||
pub struct ObfuscatedFileSystem(DefaultFileSystem);
|
||||
/// `[ObfuscatedFileSystem]` is a special implementation of `[FileSystem]`,
|
||||
/// which is used for constructing and simulating an abnormal file system for
|
||||
/// `[Read]` and `[Write]`.
|
||||
pub struct ObfuscatedFileSystem {
|
||||
inner: DefaultFileSystem,
|
||||
files: AtomicUsize,
|
||||
}
|
||||
|
||||
impl Default for ObfuscatedFileSystem {
|
||||
fn default() -> Self {
|
||||
ObfuscatedFileSystem(DefaultFileSystem)
|
||||
ObfuscatedFileSystem {
|
||||
inner: DefaultFileSystem,
|
||||
files: AtomicUsize::new(0),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl ObfuscatedFileSystem {
|
||||
pub fn file_count(&self) -> usize {
|
||||
self.files.load(Ordering::Relaxed)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -79,18 +91,40 @@ impl FileSystem for ObfuscatedFileSystem {
|
|||
type Writer = ObfuscatedWriter;
|
||||
|
||||
fn create<P: AsRef<Path>>(&self, path: P) -> IoResult<Self::Handle> {
|
||||
self.0.create(path)
|
||||
let r = self.inner.create(path);
|
||||
if r.is_ok() {
|
||||
self.files.fetch_add(1, Ordering::Relaxed);
|
||||
}
|
||||
r
|
||||
}
|
||||
|
||||
fn open<P: AsRef<Path>>(&self, path: P) -> IoResult<Self::Handle> {
|
||||
self.0.open(path)
|
||||
fn open<P: AsRef<Path>>(&self, path: P, perm: Permission) -> IoResult<Self::Handle> {
|
||||
self.inner.open(path, perm)
|
||||
}
|
||||
|
||||
fn new_reader(&self, inner: Arc<Self::Handle>) -> IoResult<Self::Reader> {
|
||||
Ok(ObfuscatedReader(self.0.new_reader(inner)?))
|
||||
fn delete<P: AsRef<Path>>(&self, path: P) -> IoResult<()> {
|
||||
let r = self.inner.delete(path);
|
||||
if r.is_ok() {
|
||||
self.files.fetch_sub(1, Ordering::Relaxed);
|
||||
}
|
||||
r
|
||||
}
|
||||
|
||||
fn new_writer(&self, inner: Arc<Self::Handle>) -> IoResult<Self::Writer> {
|
||||
Ok(ObfuscatedWriter(self.0.new_writer(inner)?))
|
||||
fn rename<P: AsRef<Path>>(&self, src_path: P, dst_path: P) -> IoResult<()> {
|
||||
self.inner.rename(src_path, dst_path)
|
||||
}
|
||||
|
||||
fn reuse<P: AsRef<Path>>(&self, src_path: P, dst_path: P) -> IoResult<()> {
|
||||
self.delete(src_path)?;
|
||||
self.create(dst_path)?;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn new_reader(&self, handle: Arc<Self::Handle>) -> IoResult<Self::Reader> {
|
||||
Ok(ObfuscatedReader(self.inner.new_reader(handle)?))
|
||||
}
|
||||
|
||||
fn new_writer(&self, handle: Arc<Self::Handle>) -> IoResult<Self::Writer> {
|
||||
Ok(ObfuscatedWriter(self.inner.new_writer(handle)?))
|
||||
}
|
||||
}
|
||||
|
|
|
@ -13,12 +13,14 @@ pub enum Error {
|
|||
InvalidArgument(String),
|
||||
#[error("Corruption: {0}")]
|
||||
Corruption(String),
|
||||
#[error("IO Error: {0}")]
|
||||
#[error("IO Error: {0:?}")]
|
||||
Io(#[from] IoError),
|
||||
#[error("Codec Error: {0}")]
|
||||
Codec(#[from] CodecError),
|
||||
#[error("Protobuf Error: {0}")]
|
||||
Protobuf(#[from] protobuf::ProtobufError),
|
||||
#[error("TryAgain Error: {0}")]
|
||||
TryAgain(String),
|
||||
#[error("Entry Compacted")]
|
||||
EntryCompacted,
|
||||
#[error("Entry Not Found")]
|
||||
|
@ -30,3 +32,10 @@ pub enum Error {
|
|||
}
|
||||
|
||||
pub type Result<T> = ::std::result::Result<T, Error>;
|
||||
|
||||
/// Check whether the given error is a nospace error.
|
||||
pub(crate) fn is_no_space_err(e: &IoError) -> bool {
|
||||
// TODO: make the following judgement more elegant when the error type
|
||||
// `ErrorKind::StorageFull` is stable.
|
||||
format!("{e}").contains("nospace")
|
||||
}
|
||||
|
|
|
@ -5,11 +5,10 @@
|
|||
use std::io::BufRead;
|
||||
use std::path::{Path, PathBuf};
|
||||
|
||||
use num_derive::{FromPrimitive, ToPrimitive};
|
||||
use num_traits::{FromPrimitive, ToPrimitive};
|
||||
|
||||
use crate::codec::{self, NumberEncoder};
|
||||
use crate::pipe_log::{FileId, LogQueue};
|
||||
use crate::pipe_log::{FileId, FileSeq, LogQueue, Version};
|
||||
use crate::{Error, Result};
|
||||
|
||||
/// Width to format log sequence number.
|
||||
|
@ -18,9 +17,25 @@ const LOG_SEQ_WIDTH: usize = 16;
|
|||
const LOG_APPEND_SUFFIX: &str = ".raftlog";
|
||||
/// Name suffix for Rewrite queue files.
|
||||
const LOG_REWRITE_SUFFIX: &str = ".rewrite";
|
||||
/// Name suffix for reserved log files that contain only zeros.
|
||||
const LOG_APPEND_RESERVED_SUFFIX: &str = ".raftlog.reserved";
|
||||
/// File header.
|
||||
const LOG_FILE_MAGIC_HEADER: &[u8] = b"RAFT-LOG-FILE-HEADER-9986AB3E47F320B394C8E84916EB0ED5";
|
||||
|
||||
/// Checks whether the given `buf` is padded with zeros.
|
||||
///
|
||||
/// To simplify the checking strategy, we just check the first
|
||||
/// and last byte in the `buf`.
|
||||
///
|
||||
/// In most common cases, the paddings will be filled with `0`,
|
||||
/// and several corner cases, where there exists corrupted blocks
|
||||
/// in the disk, might pass through this rule, but will failed in
|
||||
/// followed processing. So, we can just keep it simplistic.
|
||||
#[inline]
|
||||
pub(crate) fn is_zero_padded(buf: &[u8]) -> bool {
|
||||
buf.is_empty() || (buf[0] == 0 && buf[buf.len() - 1] == 0)
|
||||
}
|
||||
|
||||
/// `FileNameExt` offers file name formatting extensions to [`FileId`].
|
||||
pub trait FileNameExt: Sized {
|
||||
fn parse_file_name(file_name: &str) -> Option<Self>;
|
||||
|
@ -55,23 +70,32 @@ impl FileNameExt for FileId {
|
|||
}
|
||||
|
||||
fn build_file_name(&self) -> String {
|
||||
let width = LOG_SEQ_WIDTH;
|
||||
match self.queue {
|
||||
LogQueue::Append => format!(
|
||||
"{:0width$}{}",
|
||||
self.seq,
|
||||
LOG_APPEND_SUFFIX,
|
||||
width = LOG_SEQ_WIDTH
|
||||
),
|
||||
LogQueue::Rewrite => format!(
|
||||
"{:0width$}{}",
|
||||
self.seq,
|
||||
LOG_REWRITE_SUFFIX,
|
||||
width = LOG_SEQ_WIDTH
|
||||
),
|
||||
LogQueue::Append => format!("{:0width$}{LOG_APPEND_SUFFIX}", self.seq,),
|
||||
LogQueue::Rewrite => format!("{:0width$}{LOG_REWRITE_SUFFIX}", self.seq,),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub fn parse_reserved_file_name(file_name: &str) -> Option<FileSeq> {
|
||||
if file_name.len() > LOG_SEQ_WIDTH {
|
||||
if let Ok(seq) = file_name[..LOG_SEQ_WIDTH].parse::<u64>() {
|
||||
if file_name.ends_with(LOG_APPEND_RESERVED_SUFFIX) {
|
||||
// As reserved files are only used for LogQueue::Append,
|
||||
// we just return the related FileSeq of it.
|
||||
return Some(seq);
|
||||
}
|
||||
}
|
||||
}
|
||||
None
|
||||
}
|
||||
|
||||
pub fn build_reserved_file_name(seq: FileSeq) -> String {
|
||||
let width = LOG_SEQ_WIDTH;
|
||||
format!("{seq:0width$}{LOG_APPEND_RESERVED_SUFFIX}",)
|
||||
}
|
||||
|
||||
/// Path to the lock file under `dir`.
|
||||
pub(super) fn lock_file_path<P: AsRef<Path>>(dir: P) -> PathBuf {
|
||||
let mut path = PathBuf::from(dir.as_ref());
|
||||
|
@ -79,64 +103,105 @@ pub(super) fn lock_file_path<P: AsRef<Path>>(dir: P) -> PathBuf {
|
|||
path
|
||||
}
|
||||
|
||||
/// Version of log file format.
|
||||
#[derive(Clone, Copy, FromPrimitive, ToPrimitive)]
|
||||
#[repr(u64)]
|
||||
enum Version {
|
||||
V1 = 1,
|
||||
/// Log file format. It will be encoded to file header.
|
||||
#[derive(Copy, Clone, Debug, Eq, PartialEq, Default)]
|
||||
pub struct LogFileFormat {
|
||||
pub version: Version,
|
||||
/// 0 stands for no alignment.
|
||||
pub alignment: u64,
|
||||
}
|
||||
|
||||
/// In-memory representation of the log file header.
|
||||
pub(super) struct LogFileHeader {
|
||||
version: Version,
|
||||
}
|
||||
|
||||
impl Default for LogFileHeader {
|
||||
fn default() -> Self {
|
||||
Self {
|
||||
version: Version::V1,
|
||||
}
|
||||
impl LogFileFormat {
|
||||
pub fn new(version: Version, alignment: u64) -> Self {
|
||||
Self { version, alignment }
|
||||
}
|
||||
}
|
||||
|
||||
impl LogFileHeader {
|
||||
/// Length of header written on storage.
|
||||
pub const fn len() -> usize {
|
||||
const fn header_len() -> usize {
|
||||
LOG_FILE_MAGIC_HEADER.len() + std::mem::size_of::<Version>()
|
||||
}
|
||||
|
||||
/// Decodes a slice of bytes into a `LogFileHeader`.
|
||||
pub fn decode(buf: &mut &[u8]) -> Result<LogFileHeader> {
|
||||
if buf.len() < Self::len() {
|
||||
return Err(Error::Corruption("log file header too short".to_owned()));
|
||||
const fn payload_len(version: Version) -> usize {
|
||||
match version {
|
||||
Version::V1 => 0,
|
||||
Version::V2 => std::mem::size_of::<u64>(),
|
||||
}
|
||||
}
|
||||
|
||||
pub const fn max_encoded_len() -> usize {
|
||||
Self::header_len() + Self::payload_len(Version::V2)
|
||||
}
|
||||
|
||||
/// Length of whole `LogFileFormat` written on storage.
|
||||
pub fn encoded_len(version: Version) -> usize {
|
||||
Self::header_len() + Self::payload_len(version)
|
||||
}
|
||||
|
||||
/// Decodes a slice of bytes into a `LogFileFormat`.
|
||||
pub fn decode(buf: &mut &[u8]) -> Result<LogFileFormat> {
|
||||
let mut format = LogFileFormat::default();
|
||||
if !buf.starts_with(LOG_FILE_MAGIC_HEADER) {
|
||||
return Err(Error::Corruption(
|
||||
"log file magic header mismatch".to_owned(),
|
||||
));
|
||||
}
|
||||
buf.consume(LOG_FILE_MAGIC_HEADER.len());
|
||||
let v = codec::decode_u64(buf)?;
|
||||
if let Some(version) = Version::from_u64(v) {
|
||||
Ok(Self { version })
|
||||
|
||||
let version_u64 = codec::decode_u64(buf)?;
|
||||
if let Some(version) = Version::from_u64(version_u64) {
|
||||
format.version = version;
|
||||
} else {
|
||||
Err(Error::Corruption(format!(
|
||||
"unrecognized log file version: {}",
|
||||
v
|
||||
)))
|
||||
return Err(Error::Corruption(format!(
|
||||
"unrecognized log file version: {version_u64}",
|
||||
)));
|
||||
}
|
||||
|
||||
let payload_len = Self::payload_len(format.version);
|
||||
if buf.len() < payload_len {
|
||||
return Err(Error::Corruption("missing header payload".to_owned()));
|
||||
} else if payload_len > 0 {
|
||||
format.alignment = codec::decode_u64(buf)?;
|
||||
}
|
||||
|
||||
Ok(format)
|
||||
}
|
||||
|
||||
/// Encodes this header and appends the bytes to the provided buffer.
|
||||
pub fn encode(&self, buf: &mut Vec<u8>) -> Result<()> {
|
||||
buf.extend_from_slice(LOG_FILE_MAGIC_HEADER);
|
||||
buf.encode_u64(self.version.to_u64().unwrap())?;
|
||||
let corrupted = || {
|
||||
fail::fail_point!("log_file_header::corrupted", |_| true);
|
||||
false
|
||||
};
|
||||
if corrupted() {
|
||||
buf[0] += 1;
|
||||
if Self::payload_len(self.version) > 0 {
|
||||
buf.encode_u64(self.alignment)?;
|
||||
} else {
|
||||
assert_eq!(self.alignment, 0);
|
||||
}
|
||||
#[cfg(feature = "failpoints")]
|
||||
{
|
||||
// Set header corrupted.
|
||||
let corrupted = || {
|
||||
fail::fail_point!("log_file_header::corrupted", |_| true);
|
||||
false
|
||||
};
|
||||
// Set abnormal DataLayout.
|
||||
let too_large = || {
|
||||
fail::fail_point!("log_file_header::too_large", |_| true);
|
||||
false
|
||||
};
|
||||
// Set corrupted DataLayout for `payload`.
|
||||
let too_small = || {
|
||||
fail::fail_point!("log_file_header::too_small", |_| true);
|
||||
false
|
||||
};
|
||||
if corrupted() {
|
||||
buf[0] += 1;
|
||||
}
|
||||
assert!(!(too_large() && too_small()));
|
||||
if too_large() {
|
||||
buf.encode_u64(0_u64)?;
|
||||
}
|
||||
if too_small() {
|
||||
buf.pop();
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
@ -145,6 +210,28 @@ impl LogFileHeader {
|
|||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::pipe_log::LogFileContext;
|
||||
use crate::test_util::catch_unwind_silent;
|
||||
|
||||
#[test]
|
||||
fn test_check_paddings_is_valid() {
|
||||
// normal buffer
|
||||
let mut buf = vec![0; 128];
|
||||
// len < 8
|
||||
assert!(is_zero_padded(&buf[0..6]));
|
||||
// len == 8
|
||||
assert!(is_zero_padded(&buf[120..]));
|
||||
// len > 8
|
||||
assert!(is_zero_padded(&buf));
|
||||
|
||||
// abnormal buffer
|
||||
buf[127] = 3_u8;
|
||||
assert!(is_zero_padded(&buf[0..110]));
|
||||
assert!(is_zero_padded(&buf[120..125]));
|
||||
assert!(!is_zero_padded(&buf[124..128]));
|
||||
assert!(!is_zero_padded(&buf[120..]));
|
||||
assert!(!is_zero_padded(&buf));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_file_name() {
|
||||
|
@ -169,4 +256,68 @@ mod tests {
|
|||
assert!(FileId::parse_file_name(case).is_none());
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_version() {
|
||||
let version = Version::default();
|
||||
assert_eq!(Version::V1.to_u64().unwrap(), version.to_u64().unwrap());
|
||||
let version2 = Version::from_u64(1).unwrap();
|
||||
assert_eq!(version, version2);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_encoding_decoding_file_format() {
|
||||
fn enc_dec_file_format(file_format: LogFileFormat) -> Result<LogFileFormat> {
|
||||
let mut buf = Vec::with_capacity(
|
||||
LogFileFormat::header_len() + LogFileFormat::payload_len(file_format.version),
|
||||
);
|
||||
file_format.encode(&mut buf).unwrap();
|
||||
LogFileFormat::decode(&mut &buf[..])
|
||||
}
|
||||
// header with aligned-sized data_layout
|
||||
{
|
||||
let mut buf = Vec::with_capacity(LogFileFormat::header_len());
|
||||
let version = Version::V2;
|
||||
let alignment = 4096;
|
||||
buf.extend_from_slice(LOG_FILE_MAGIC_HEADER);
|
||||
buf.encode_u64(version.to_u64().unwrap()).unwrap();
|
||||
buf.encode_u64(alignment).unwrap();
|
||||
assert_eq!(
|
||||
LogFileFormat::decode(&mut &buf[..]).unwrap(),
|
||||
LogFileFormat::new(version, alignment)
|
||||
);
|
||||
}
|
||||
// header with abnormal version
|
||||
{
|
||||
let mut buf = Vec::with_capacity(LogFileFormat::header_len());
|
||||
let abnormal_version = 4_u64; /* abnormal version */
|
||||
buf.extend_from_slice(LOG_FILE_MAGIC_HEADER);
|
||||
buf.encode_u64(abnormal_version).unwrap();
|
||||
buf.encode_u64(16).unwrap();
|
||||
assert!(LogFileFormat::decode(&mut &buf[..]).is_err());
|
||||
}
|
||||
{
|
||||
let file_format = LogFileFormat::new(Version::default(), 0);
|
||||
assert_eq!(
|
||||
LogFileFormat::new(Version::default(), 0),
|
||||
enc_dec_file_format(file_format).unwrap()
|
||||
);
|
||||
let file_format = LogFileFormat::new(Version::default(), 4096);
|
||||
assert!(catch_unwind_silent(|| enc_dec_file_format(file_format)).is_err());
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_file_context() {
|
||||
let mut file_context =
|
||||
LogFileContext::new(FileId::dummy(LogQueue::Append), Version::default());
|
||||
assert_eq!(file_context.get_signature(), None);
|
||||
file_context.id.seq = 10;
|
||||
file_context.version = Version::V2;
|
||||
assert_eq!(file_context.get_signature().unwrap(), 10);
|
||||
let abnormal_seq = (file_context.id.seq << 32) + 100_u64;
|
||||
file_context.id.seq = abnormal_seq;
|
||||
assert_ne!(file_context.get_signature().unwrap() as u64, abnormal_seq);
|
||||
assert_eq!(file_context.get_signature().unwrap(), 100);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -2,120 +2,142 @@
|
|||
|
||||
//! Log file types.
|
||||
|
||||
use std::io::{Read, Seek, SeekFrom, Write};
|
||||
use std::io::{Read, Result as IoResult, Seek, SeekFrom, Write};
|
||||
use std::sync::Arc;
|
||||
|
||||
use fail::fail_point;
|
||||
use log::warn;
|
||||
|
||||
use crate::env::{FileSystem, Handle, WriteExt};
|
||||
use crate::metrics::*;
|
||||
use crate::pipe_log::FileBlockHandle;
|
||||
use crate::{Error, Result};
|
||||
|
||||
use super::format::LogFileHeader;
|
||||
use crate::env::{FileSystem, Handle, WriteExt};
|
||||
use super::format::LogFileFormat;
|
||||
|
||||
/// Maximum number of bytes to allocate ahead.
|
||||
const FILE_ALLOCATE_SIZE: usize = 2 * 1024 * 1024;
|
||||
|
||||
/// Builds a file writer.
|
||||
///
|
||||
/// # Arguments
|
||||
///
|
||||
/// * `handle`: standard handle of a log file.
|
||||
/// * `format`: format infos of the log file.
|
||||
/// * `force_reset`: if true => rewrite the header of this file.
|
||||
pub(super) fn build_file_writer<F: FileSystem>(
|
||||
system: &F,
|
||||
handle: Arc<F::Handle>,
|
||||
format: LogFileFormat,
|
||||
force_reset: bool,
|
||||
) -> Result<LogFileWriter<F>> {
|
||||
let writer = system.new_writer(handle.clone())?;
|
||||
LogFileWriter::open(handle, writer)
|
||||
LogFileWriter::open(handle, writer, format, force_reset)
|
||||
}
|
||||
|
||||
/// Append-only writer for log file.
|
||||
/// Append-only writer for log file. It also handles the file header write.
|
||||
pub struct LogFileWriter<F: FileSystem> {
|
||||
handle: Arc<F::Handle>,
|
||||
writer: F::Writer,
|
||||
written: usize,
|
||||
capacity: usize,
|
||||
last_sync: usize,
|
||||
}
|
||||
|
||||
// All APIs provided by `LogFileWriter` are fail-safe, i.e. caller can continue
|
||||
// using the same "writer" even if the previous operation failed.
|
||||
impl<F: FileSystem> LogFileWriter<F> {
|
||||
fn open(handle: Arc<F::Handle>, writer: F::Writer) -> Result<Self> {
|
||||
fn open(
|
||||
handle: Arc<F::Handle>,
|
||||
writer: F::Writer,
|
||||
format: LogFileFormat,
|
||||
force_reset: bool,
|
||||
) -> Result<Self> {
|
||||
let file_size = handle.file_size()?;
|
||||
let mut f = Self {
|
||||
handle,
|
||||
writer,
|
||||
written: file_size,
|
||||
capacity: file_size,
|
||||
last_sync: file_size,
|
||||
};
|
||||
if file_size < LogFileHeader::len() {
|
||||
f.write_header()?;
|
||||
// TODO: add tests for file_size in [header_len, max_encoded_len].
|
||||
if file_size < LogFileFormat::encoded_len(format.version) || force_reset {
|
||||
f.write_header(format)?;
|
||||
} else {
|
||||
f.writer.seek(SeekFrom::Start(file_size as u64))?;
|
||||
}
|
||||
Ok(f)
|
||||
}
|
||||
|
||||
fn write_header(&mut self) -> Result<()> {
|
||||
self.writer.seek(SeekFrom::Start(0))?;
|
||||
self.last_sync = 0;
|
||||
fn write_header(&mut self, format: LogFileFormat) -> IoResult<()> {
|
||||
self.writer.rewind()?;
|
||||
self.written = 0;
|
||||
let mut buf = Vec::with_capacity(LogFileHeader::len());
|
||||
LogFileHeader::default().encode(&mut buf)?;
|
||||
let mut buf = Vec::with_capacity(LogFileFormat::encoded_len(format.version));
|
||||
format.encode(&mut buf).unwrap();
|
||||
self.write(&buf, 0)
|
||||
}
|
||||
|
||||
pub fn close(&mut self) -> Result<()> {
|
||||
pub fn close(&mut self) -> IoResult<()> {
|
||||
// Necessary to truncate extra zeros from fallocate().
|
||||
self.truncate()?;
|
||||
self.sync()
|
||||
}
|
||||
|
||||
pub fn truncate(&mut self) -> Result<()> {
|
||||
pub fn truncate(&mut self) -> IoResult<()> {
|
||||
if self.written < self.capacity {
|
||||
fail_point!("file_pipe_log::log_file_writer::skip_truncate", |_| {
|
||||
Ok(())
|
||||
});
|
||||
self.writer.truncate(self.written)?;
|
||||
self.capacity = self.written;
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub fn write(&mut self, buf: &[u8], target_size_hint: usize) -> Result<()> {
|
||||
pub fn write(&mut self, buf: &[u8], target_size_hint: usize) -> IoResult<()> {
|
||||
let new_written = self.written + buf.len();
|
||||
if self.capacity < new_written {
|
||||
let _t = StopWatch::new(&LOG_ALLOCATE_DURATION_HISTOGRAM);
|
||||
let alloc = std::cmp::max(
|
||||
new_written - self.capacity,
|
||||
std::cmp::min(
|
||||
FILE_ALLOCATE_SIZE,
|
||||
target_size_hint.saturating_sub(self.capacity),
|
||||
),
|
||||
let _t = StopWatch::new(&*LOG_ALLOCATE_DURATION_HISTOGRAM);
|
||||
let alloc = std::cmp::min(
|
||||
FILE_ALLOCATE_SIZE,
|
||||
target_size_hint.saturating_sub(self.capacity),
|
||||
);
|
||||
self.writer.allocate(self.capacity, alloc)?;
|
||||
let alloc = std::cmp::max(new_written - self.capacity, alloc);
|
||||
if let Err(e) = self.writer.allocate(self.capacity, alloc) {
|
||||
warn!("log file allocation failed: {e}");
|
||||
}
|
||||
self.capacity += alloc;
|
||||
}
|
||||
self.writer.write_all(buf)?;
|
||||
self.writer.write_all(buf).inspect_err(|_| {
|
||||
self.writer
|
||||
.seek(SeekFrom::Start(self.written as u64))
|
||||
.unwrap_or_else(|e| {
|
||||
panic!("failed to reseek after write failure: {}", e);
|
||||
});
|
||||
})?;
|
||||
self.written = new_written;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub fn sync(&mut self) -> Result<()> {
|
||||
if self.last_sync < self.written {
|
||||
let _t = StopWatch::new(&LOG_SYNC_DURATION_HISTOGRAM);
|
||||
self.writer.sync()?;
|
||||
self.last_sync = self.written;
|
||||
}
|
||||
pub fn sync(&mut self) -> IoResult<()> {
|
||||
let _t = StopWatch::new(&*LOG_SYNC_DURATION_HISTOGRAM);
|
||||
// Panic if sync fails, in case of data loss.
|
||||
self.handle.sync().unwrap();
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn since_last_sync(&self) -> usize {
|
||||
self.written - self.last_sync
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn offset(&self) -> usize {
|
||||
self.written
|
||||
}
|
||||
}
|
||||
|
||||
/// Build a file reader.
|
||||
pub(super) fn build_file_reader<F: FileSystem>(
|
||||
system: &F,
|
||||
handle: Arc<F::Handle>,
|
||||
) -> Result<LogFileReader<F>> {
|
||||
let reader = system.new_reader(handle.clone())?;
|
||||
LogFileReader::open(handle, reader)
|
||||
Ok(LogFileReader::open(handle, reader))
|
||||
}
|
||||
|
||||
/// Random-access reader for log file.
|
||||
|
@ -127,17 +149,30 @@ pub struct LogFileReader<F: FileSystem> {
|
|||
}
|
||||
|
||||
impl<F: FileSystem> LogFileReader<F> {
|
||||
fn open(handle: Arc<F::Handle>, reader: F::Reader) -> Result<Self> {
|
||||
Ok(Self {
|
||||
fn open(handle: Arc<F::Handle>, reader: F::Reader) -> LogFileReader<F> {
|
||||
Self {
|
||||
handle,
|
||||
reader,
|
||||
// Set to an invalid offset to force a reseek at first read.
|
||||
offset: u64::MAX,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
/// Function for reading the header of the log file, and return a
|
||||
/// `[LogFileFormat]`.
|
||||
///
|
||||
/// Attention please, this function would move the `reader.offset`
|
||||
/// to `0`, that is, the beginning of the file, to parse the
|
||||
/// related `[LogFileFormat]`.
|
||||
pub fn parse_format(&mut self) -> Result<LogFileFormat> {
|
||||
let mut container = vec![0; LogFileFormat::max_encoded_len()];
|
||||
let size = self.read_to(0, &mut container)?;
|
||||
container.truncate(size);
|
||||
LogFileFormat::decode(&mut container.as_slice())
|
||||
}
|
||||
|
||||
pub fn read(&mut self, handle: FileBlockHandle) -> Result<Vec<u8>> {
|
||||
let mut buf = vec![0; handle.len as usize];
|
||||
let mut buf = vec![0; handle.len];
|
||||
let size = self.read_to(handle.offset, &mut buf)?;
|
||||
buf.truncate(size);
|
||||
Ok(buf)
|
||||
|
|
|
@ -10,10 +10,10 @@ mod pipe;
|
|||
mod pipe_builder;
|
||||
mod reader;
|
||||
|
||||
pub use format::FileNameExt;
|
||||
pub use format::{parse_reserved_file_name, FileNameExt};
|
||||
pub use pipe::DualPipes as FilePipeLog;
|
||||
pub use pipe_builder::{
|
||||
DefaultMachineFactory, DualPipesBuilder as FilePipeLogBuilder, ReplayMachine,
|
||||
DefaultMachineFactory, DualPipesBuilder as FilePipeLogBuilder, RecoveryConfig, ReplayMachine,
|
||||
};
|
||||
|
||||
pub mod debug {
|
||||
|
@ -23,12 +23,12 @@ pub mod debug {
|
|||
use std::path::{Path, PathBuf};
|
||||
use std::sync::Arc;
|
||||
|
||||
use crate::env::FileSystem;
|
||||
use crate::env::{FileSystem, Permission};
|
||||
use crate::log_batch::LogItem;
|
||||
use crate::pipe_log::FileId;
|
||||
use crate::{Error, Result};
|
||||
|
||||
use super::format::FileNameExt;
|
||||
use super::format::{FileNameExt, LogFileFormat};
|
||||
use super::log_file::{LogFileReader, LogFileWriter};
|
||||
use super::reader::LogItemBatchFileReader;
|
||||
|
||||
|
@ -38,15 +38,16 @@ pub mod debug {
|
|||
pub fn build_file_writer<F: FileSystem>(
|
||||
file_system: &F,
|
||||
path: &Path,
|
||||
format: LogFileFormat,
|
||||
create: bool,
|
||||
) -> Result<LogFileWriter<F>> {
|
||||
let fd = if create {
|
||||
file_system.create(path)?
|
||||
} else {
|
||||
file_system.open(path)?
|
||||
file_system.open(path, Permission::ReadWrite)?
|
||||
};
|
||||
let fd = Arc::new(fd);
|
||||
super::log_file::build_file_writer(file_system, fd)
|
||||
super::log_file::build_file_writer(file_system, fd, format, create /* force_reset */)
|
||||
}
|
||||
|
||||
/// Opens a log file for read.
|
||||
|
@ -54,7 +55,7 @@ pub mod debug {
|
|||
file_system: &F,
|
||||
path: &Path,
|
||||
) -> Result<LogFileReader<F>> {
|
||||
let fd = Arc::new(file_system.open(path)?);
|
||||
let fd = Arc::new(file_system.open(path, Permission::ReadOnly)?);
|
||||
super::log_file::build_file_reader(file_system, fd)
|
||||
}
|
||||
|
||||
|
@ -87,8 +88,7 @@ pub mod debug {
|
|||
let file_id = FileId::parse_file_name(file_name);
|
||||
if file_id.is_none() {
|
||||
return Err(Error::InvalidArgument(format!(
|
||||
"Invalid log file name: {}",
|
||||
file_name
|
||||
"Invalid log file name: {file_name}"
|
||||
)));
|
||||
}
|
||||
Ok(Self {
|
||||
|
@ -156,8 +156,8 @@ pub mod debug {
|
|||
|
||||
fn find_next_readable_file(&mut self) -> Result<()> {
|
||||
while let Some((file_id, path)) = self.files.pop_front() {
|
||||
self.batch_reader
|
||||
.open(file_id, build_file_reader(self.system.as_ref(), &path)?)?;
|
||||
let reader = build_file_reader(self.system.as_ref(), &path)?;
|
||||
self.batch_reader.open(file_id, reader)?;
|
||||
if let Some(b) = self.batch_reader.next()? {
|
||||
self.items.extend(b.into_items());
|
||||
break;
|
||||
|
@ -172,8 +172,8 @@ pub mod debug {
|
|||
use super::*;
|
||||
use crate::env::DefaultFileSystem;
|
||||
use crate::log_batch::{Command, LogBatch};
|
||||
use crate::pipe_log::{FileBlockHandle, LogQueue};
|
||||
use crate::test_util::generate_entries;
|
||||
use crate::pipe_log::{FileBlockHandle, LogFileContext, LogQueue, Version};
|
||||
use crate::test_util::{generate_entries, PanicGuard};
|
||||
use raft::eraftpb::Entry;
|
||||
|
||||
#[test]
|
||||
|
@ -195,11 +195,11 @@ pub mod debug {
|
|||
.add_entries::<Entry>(7, &generate_entries(1, 11, Some(&entry_data)))
|
||||
.unwrap();
|
||||
batch.add_command(7, Command::Clean);
|
||||
batch.put(7, b"key".to_vec(), b"value".to_vec());
|
||||
batch.put(7, b"key".to_vec(), b"value".to_vec()).unwrap();
|
||||
batch.delete(7, b"key2".to_vec());
|
||||
batches.push(vec![batch.clone()]);
|
||||
let mut batch2 = LogBatch::default();
|
||||
batch2.put(8, b"key3".to_vec(), b"value".to_vec());
|
||||
batch2.put(8, b"key3".to_vec(), b"value".to_vec()).unwrap();
|
||||
batch2
|
||||
.add_entries::<Entry>(8, &generate_entries(5, 15, Some(&entry_data)))
|
||||
.unwrap();
|
||||
|
@ -208,13 +208,20 @@ pub mod debug {
|
|||
for bs in batches.iter_mut() {
|
||||
let file_path = file_id.build_file_path(dir.path());
|
||||
// Write a file.
|
||||
let mut writer =
|
||||
build_file_writer(file_system.as_ref(), &file_path, true /* create */).unwrap();
|
||||
let mut writer = build_file_writer(
|
||||
file_system.as_ref(),
|
||||
&file_path,
|
||||
LogFileFormat::default(),
|
||||
true, /* create */
|
||||
)
|
||||
.unwrap();
|
||||
let log_file_format = LogFileContext::new(file_id, Version::default());
|
||||
for batch in bs.iter_mut() {
|
||||
let offset = writer.offset() as u64;
|
||||
let len = batch
|
||||
.finish_populate(1 /* compression_threshold */)
|
||||
let (len, _) = batch
|
||||
.finish_populate(1 /* compression_threshold */, None)
|
||||
.unwrap();
|
||||
batch.prepare_write(&log_file_format).unwrap();
|
||||
writer
|
||||
.write(batch.encoded_bytes(), 0 /* target_file_hint */)
|
||||
.unwrap();
|
||||
|
@ -257,18 +264,19 @@ pub mod debug {
|
|||
let file_system = Arc::new(DefaultFileSystem);
|
||||
// An unrelated sub-directory.
|
||||
let unrelated_dir = dir.path().join(Path::new("random_dir"));
|
||||
std::fs::create_dir(&unrelated_dir).unwrap();
|
||||
std::fs::create_dir(unrelated_dir).unwrap();
|
||||
// An unrelated file.
|
||||
let unrelated_file_path = dir.path().join(Path::new("random_file"));
|
||||
let _unrelated_file = std::fs::File::create(&unrelated_file_path).unwrap();
|
||||
// A corrupted log file.
|
||||
let corrupted_file_path = FileId::dummy(LogQueue::Append).build_file_path(dir.path());
|
||||
let _corrupted_file = std::fs::File::create(&corrupted_file_path).unwrap();
|
||||
let _corrupted_file = std::fs::File::create(corrupted_file_path).unwrap();
|
||||
// An empty log file.
|
||||
let empty_file_path = FileId::dummy(LogQueue::Rewrite).build_file_path(dir.path());
|
||||
let mut writer = build_file_writer(
|
||||
file_system.as_ref(),
|
||||
&empty_file_path,
|
||||
LogFileFormat::default(),
|
||||
true, /* create */
|
||||
)
|
||||
.unwrap();
|
||||
|
@ -281,10 +289,65 @@ pub mod debug {
|
|||
assert!(
|
||||
LogItemReader::new_directory_reader(file_system.clone(), &empty_file_path).is_err()
|
||||
);
|
||||
LogItemReader::new_file_reader(file_system.clone(), &empty_file_path).unwrap();
|
||||
|
||||
let mut reader = LogItemReader::new_directory_reader(file_system, dir.path()).unwrap();
|
||||
assert!(reader.next().unwrap().is_err());
|
||||
assert!(reader.next().is_none());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_recover_from_partial_write() {
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_debug_file_overwrite")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let file_system = Arc::new(DefaultFileSystem);
|
||||
|
||||
let path = FileId::dummy(LogQueue::Append).build_file_path(dir.path());
|
||||
|
||||
let formats = [
|
||||
LogFileFormat::new(Version::V1, 0),
|
||||
LogFileFormat::new(Version::V2, 1),
|
||||
];
|
||||
for from in formats {
|
||||
for to in formats {
|
||||
for shorter in [true, false] {
|
||||
if LogFileFormat::encoded_len(to.version)
|
||||
< LogFileFormat::encoded_len(from.version)
|
||||
{
|
||||
continue;
|
||||
}
|
||||
let _guard = PanicGuard::with_prompt(format!(
|
||||
"case: [{from:?}, {to:?}, {shorter:?}]",
|
||||
));
|
||||
let mut writer = build_file_writer(
|
||||
file_system.as_ref(),
|
||||
&path,
|
||||
from,
|
||||
true, /* create */
|
||||
)
|
||||
.unwrap();
|
||||
let f = std::fs::OpenOptions::new().write(true).open(&path).unwrap();
|
||||
let len = writer.offset();
|
||||
writer.close().unwrap();
|
||||
if shorter {
|
||||
f.set_len(len as u64 - 1).unwrap();
|
||||
}
|
||||
let mut writer = build_file_writer(
|
||||
file_system.as_ref(),
|
||||
&path,
|
||||
to,
|
||||
false, /* create */
|
||||
)
|
||||
.unwrap();
|
||||
writer.close().unwrap();
|
||||
let mut reader = build_file_reader(file_system.as_ref(), &path).unwrap();
|
||||
assert_eq!(reader.parse_format().unwrap(), to);
|
||||
std::fs::remove_file(&path).unwrap();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,56 +1,99 @@
|
|||
// Copyright (c) 2017-present, PingCAP, Inc. Licensed under Apache-2.0.
|
||||
|
||||
use std::collections::VecDeque;
|
||||
use std::fs::{self, File};
|
||||
use std::fs::File as StdFile;
|
||||
use std::path::PathBuf;
|
||||
use std::sync::Arc;
|
||||
|
||||
use crossbeam::utils::CachePadded;
|
||||
use fail::fail_point;
|
||||
use log::{error, warn};
|
||||
use fs2::FileExt;
|
||||
use log::error;
|
||||
use parking_lot::{Mutex, MutexGuard, RwLock};
|
||||
|
||||
use crate::config::Config;
|
||||
use crate::env::FileSystem;
|
||||
use crate::env::{FileSystem, Permission};
|
||||
use crate::errors::is_no_space_err;
|
||||
use crate::event_listener::EventListener;
|
||||
use crate::metrics::*;
|
||||
use crate::pipe_log::{FileBlockHandle, FileId, FileSeq, LogQueue, PipeLog};
|
||||
use crate::{Error, Result};
|
||||
use crate::pipe_log::{
|
||||
FileBlockHandle, FileId, FileSeq, LogFileContext, LogQueue, PipeLog, ReactiveBytes,
|
||||
};
|
||||
use crate::{perf_context, Error, Result};
|
||||
|
||||
use super::format::FileNameExt;
|
||||
use super::log_file::{build_file_reader, build_file_writer, LogFileWriter};
|
||||
use super::format::{build_reserved_file_name, FileNameExt, LogFileFormat};
|
||||
use super::log_file::build_file_reader;
|
||||
use super::log_file::{build_file_writer, LogFileWriter};
|
||||
|
||||
struct FileCollection<F: FileSystem> {
|
||||
first_seq: FileSeq,
|
||||
active_seq: FileSeq,
|
||||
fds: VecDeque<Arc<F::Handle>>,
|
||||
pub type PathId = usize;
|
||||
pub type Paths = Vec<PathBuf>;
|
||||
|
||||
/// Main directory path id.
|
||||
pub const DEFAULT_PATH_ID: PathId = 0;
|
||||
/// FileSeq of logs must start from `1` by default to keep backward
|
||||
/// compatibility.
|
||||
pub const DEFAULT_FIRST_FILE_SEQ: FileSeq = 1;
|
||||
|
||||
pub struct File<F: FileSystem> {
|
||||
pub seq: FileSeq,
|
||||
pub handle: Arc<F::Handle>,
|
||||
pub format: LogFileFormat,
|
||||
pub path_id: PathId,
|
||||
pub reserved: bool,
|
||||
}
|
||||
|
||||
struct ActiveFile<F: FileSystem> {
|
||||
seq: FileSeq,
|
||||
writer: LogFileWriter<F>,
|
||||
struct WritableFile<F: FileSystem> {
|
||||
pub seq: FileSeq,
|
||||
pub writer: LogFileWriter<F>,
|
||||
pub format: LogFileFormat,
|
||||
}
|
||||
|
||||
/// A file-based log storage that arranges files as one single queue.
|
||||
pub(super) struct SinglePipe<F: FileSystem> {
|
||||
queue: LogQueue,
|
||||
dir: String,
|
||||
target_file_size: usize,
|
||||
bytes_per_sync: usize,
|
||||
paths: Paths,
|
||||
file_system: Arc<F>,
|
||||
listeners: Vec<Arc<dyn EventListener>>,
|
||||
default_format: LogFileFormat,
|
||||
target_file_size: usize,
|
||||
|
||||
capacity: usize,
|
||||
active_files: CachePadded<RwLock<VecDeque<File<F>>>>,
|
||||
/// This contains both reserved files and files recycled from
|
||||
/// `active_files`.
|
||||
recycled_files: CachePadded<RwLock<VecDeque<File<F>>>>,
|
||||
|
||||
/// All log files.
|
||||
files: CachePadded<RwLock<FileCollection<F>>>,
|
||||
/// The log file opened for write.
|
||||
active_file: CachePadded<Mutex<ActiveFile<F>>>,
|
||||
///
|
||||
/// `writable_file` must be locked first to acquire both `files` and
|
||||
/// `writable_file`
|
||||
writable_file: CachePadded<Mutex<WritableFile<F>>>,
|
||||
}
|
||||
|
||||
impl<F: FileSystem> Drop for SinglePipe<F> {
|
||||
fn drop(&mut self) {
|
||||
let mut active_file = self.active_file.lock();
|
||||
if let Err(e) = active_file.writer.close() {
|
||||
error!("error while closing sigle pipe: {}", e);
|
||||
let mut writable_file = self.writable_file.lock();
|
||||
if let Err(e) = writable_file.writer.close() {
|
||||
error!("error while closing the active writer: {e}");
|
||||
}
|
||||
let mut recycled_files = self.recycled_files.write();
|
||||
let mut next_reserved_seq = recycled_files
|
||||
.iter()
|
||||
.rev()
|
||||
.find_map(|f| if f.reserved { Some(f.seq + 1) } else { None })
|
||||
.unwrap_or(DEFAULT_FIRST_FILE_SEQ);
|
||||
while let Some(f) = recycled_files.pop_back() {
|
||||
if f.reserved {
|
||||
break;
|
||||
}
|
||||
let file_id = FileId::new(self.queue, f.seq);
|
||||
let path = file_id.build_file_path(&self.paths[f.path_id]);
|
||||
let dst = self.paths[0].join(build_reserved_file_name(next_reserved_seq));
|
||||
if let Err(e) = self.file_system.reuse(path, dst) {
|
||||
error!("error while renaming recycled file during shutdown: {}", e);
|
||||
} else {
|
||||
next_reserved_seq += 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -59,116 +102,198 @@ impl<F: FileSystem> SinglePipe<F> {
|
|||
/// Opens a new [`SinglePipe`].
|
||||
pub fn open(
|
||||
cfg: &Config,
|
||||
paths: Paths,
|
||||
file_system: Arc<F>,
|
||||
listeners: Vec<Arc<dyn EventListener>>,
|
||||
queue: LogQueue,
|
||||
mut first_seq: FileSeq,
|
||||
mut fds: VecDeque<Arc<F::Handle>>,
|
||||
mut active_files: Vec<File<F>>,
|
||||
recycled_files: Vec<File<F>>,
|
||||
) -> Result<Self> {
|
||||
let create_file = first_seq == 0;
|
||||
let active_seq = if create_file {
|
||||
first_seq = 1;
|
||||
let file_id = FileId {
|
||||
queue,
|
||||
seq: first_seq,
|
||||
};
|
||||
let fd = Arc::new(file_system.create(&file_id.build_file_path(&cfg.dir))?);
|
||||
fds.push_back(fd);
|
||||
first_seq
|
||||
} else {
|
||||
first_seq + fds.len() as u64 - 1
|
||||
let alignment = || {
|
||||
fail_point!("file_pipe_log::open::force_set_alignment", |_| { 16 });
|
||||
0
|
||||
};
|
||||
let default_format = LogFileFormat::new(cfg.format_version, alignment());
|
||||
|
||||
// Open or create active file.
|
||||
let no_active_files = active_files.is_empty();
|
||||
if no_active_files {
|
||||
let path_id = find_available_dir(&paths, cfg.target_file_size.0 as usize);
|
||||
let file_id = FileId::new(queue, DEFAULT_FIRST_FILE_SEQ);
|
||||
let path = file_id.build_file_path(&paths[path_id]);
|
||||
active_files.push(File {
|
||||
seq: file_id.seq,
|
||||
handle: file_system.create(path)?.into(),
|
||||
format: default_format,
|
||||
path_id,
|
||||
reserved: false,
|
||||
});
|
||||
}
|
||||
let f = active_files.last().unwrap();
|
||||
// If starting from active_files.emtpy(), we should reset the first file with
|
||||
// given file format.
|
||||
let writable_file = WritableFile {
|
||||
seq: f.seq,
|
||||
writer: build_file_writer(
|
||||
file_system.as_ref(),
|
||||
f.handle.clone(),
|
||||
f.format,
|
||||
no_active_files, /* force_reset */
|
||||
)?,
|
||||
format: f.format,
|
||||
};
|
||||
|
||||
for seq in first_seq..=active_seq {
|
||||
let (len, recycled_len) = (active_files.len(), recycled_files.len());
|
||||
for f in active_files.iter() {
|
||||
for listener in &listeners {
|
||||
listener.post_new_log_file(FileId { queue, seq });
|
||||
listener.post_new_log_file(FileId { queue, seq: f.seq });
|
||||
}
|
||||
}
|
||||
|
||||
let active_fd = fds.back().unwrap().clone();
|
||||
let active_file = ActiveFile {
|
||||
seq: active_seq,
|
||||
writer: build_file_writer(file_system.as_ref(), active_fd)?,
|
||||
};
|
||||
|
||||
let total_files = fds.len();
|
||||
let pipe = Self {
|
||||
queue,
|
||||
dir: cfg.dir.clone(),
|
||||
target_file_size: cfg.target_file_size.0 as usize,
|
||||
bytes_per_sync: cfg.bytes_per_sync.0 as usize,
|
||||
paths,
|
||||
file_system,
|
||||
listeners,
|
||||
|
||||
files: CachePadded::new(RwLock::new(FileCollection {
|
||||
first_seq,
|
||||
active_seq,
|
||||
fds,
|
||||
})),
|
||||
active_file: CachePadded::new(Mutex::new(active_file)),
|
||||
default_format,
|
||||
target_file_size: cfg.target_file_size.0 as usize,
|
||||
capacity: if queue == LogQueue::Append {
|
||||
cfg.recycle_capacity()
|
||||
} else {
|
||||
0
|
||||
},
|
||||
active_files: RwLock::new(active_files.into()).into(),
|
||||
recycled_files: RwLock::new(recycled_files.into()).into(),
|
||||
writable_file: Mutex::new(writable_file).into(),
|
||||
};
|
||||
pipe.flush_metrics(total_files);
|
||||
pipe.flush_metrics(len);
|
||||
pipe.flush_recycle_metrics(recycled_len);
|
||||
Ok(pipe)
|
||||
}
|
||||
|
||||
/// Synchronizes all metadatas associated with the working directory to the
|
||||
/// filesystem.
|
||||
fn sync_dir(&self) -> Result<()> {
|
||||
let path = PathBuf::from(&self.dir);
|
||||
std::fs::File::open(path).and_then(|d| d.sync_all())?;
|
||||
fn sync_dir(&self, path_id: PathId) -> Result<()> {
|
||||
debug_assert!(!self.paths.is_empty());
|
||||
|
||||
// Skip syncing directory in Windows. Refer to badger's discussion for more
|
||||
// detail: https://github.com/dgraph-io/badger/issues/699
|
||||
//
|
||||
// Panic if sync calls fail, keep consistent with the behavior of
|
||||
// `LogFileWriter::sync()`.
|
||||
#[cfg(not(windows))]
|
||||
std::fs::File::open(PathBuf::from(&self.paths[path_id]))
|
||||
.and_then(|d| d.sync_all())
|
||||
.unwrap();
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Recycles one obsolete file from the recycled file list and return its
|
||||
/// [`PathId`] and [`F::Handle`] if success.
|
||||
fn recycle_file(&self, seq: FileSeq) -> Option<Result<(PathId, F::Handle)>> {
|
||||
let new_file_id = FileId {
|
||||
seq,
|
||||
queue: self.queue,
|
||||
};
|
||||
let (recycle_file, recycle_len) = {
|
||||
let mut recycled_files = self.recycled_files.write();
|
||||
(recycled_files.pop_front(), recycled_files.len())
|
||||
};
|
||||
if let Some(f) = recycle_file {
|
||||
let fname = if f.reserved {
|
||||
build_reserved_file_name(f.seq)
|
||||
} else {
|
||||
FileId::new(self.queue, f.seq).build_file_name()
|
||||
};
|
||||
let src_path = self.paths[f.path_id].join(fname);
|
||||
let dst_path = new_file_id.build_file_path(&self.paths[f.path_id]);
|
||||
if let Err(e) = self.file_system.reuse(&src_path, &dst_path) {
|
||||
error!("error while trying to reuse recycled file, err: {e}");
|
||||
if let Err(e) = self.file_system.delete(&src_path) {
|
||||
error!("error while trying to delete recycled file, err: {e}");
|
||||
}
|
||||
} else {
|
||||
self.flush_recycle_metrics(recycle_len);
|
||||
return match self.file_system.open(&dst_path, Permission::ReadWrite) {
|
||||
Ok(handle) => Some(Ok((f.path_id, handle))),
|
||||
Err(e) => Some(Err(e.into())),
|
||||
};
|
||||
}
|
||||
}
|
||||
None
|
||||
}
|
||||
|
||||
/// Creates a new log file according to the given [`FileSeq`].
|
||||
fn new_file(&self, seq: FileSeq) -> Result<(PathId, F::Handle)> {
|
||||
let new_file_id = FileId {
|
||||
seq,
|
||||
queue: self.queue,
|
||||
};
|
||||
let path_id = find_available_dir(&self.paths, self.target_file_size);
|
||||
let path = new_file_id.build_file_path(&self.paths[path_id]);
|
||||
Ok((path_id, self.file_system.create(path)?))
|
||||
}
|
||||
|
||||
/// Returns a shared [`LogFd`] for the specified file sequence number.
|
||||
fn get_fd(&self, file_seq: FileSeq) -> Result<Arc<F::Handle>> {
|
||||
let files = self.files.read();
|
||||
if file_seq < files.first_seq || file_seq > files.active_seq {
|
||||
let files = self.active_files.read();
|
||||
if !(files[0].seq..files[0].seq + files.len() as u64).contains(&file_seq) {
|
||||
return Err(Error::Corruption("file seqno out of range".to_owned()));
|
||||
}
|
||||
Ok(files.fds[(file_seq - files.first_seq) as usize].clone())
|
||||
Ok(files[(file_seq - files[0].seq) as usize].handle.clone())
|
||||
}
|
||||
|
||||
/// Creates a new file for write, and rotates the active log file.
|
||||
///
|
||||
/// This operation is atomic in face of errors.
|
||||
fn rotate_imp(&self, active_file: &mut MutexGuard<ActiveFile<F>>) -> Result<()> {
|
||||
let _t = StopWatch::new(&LOG_ROTATE_DURATION_HISTOGRAM);
|
||||
let seq = active_file.seq + 1;
|
||||
debug_assert!(seq > 1);
|
||||
fn rotate_imp(&self, writable_file: &mut MutexGuard<WritableFile<F>>) -> Result<()> {
|
||||
let _t = StopWatch::new((
|
||||
&*LOG_ROTATE_DURATION_HISTOGRAM,
|
||||
perf_context!(log_rotate_duration),
|
||||
));
|
||||
let new_seq = writable_file.seq + 1;
|
||||
debug_assert!(new_seq > DEFAULT_FIRST_FILE_SEQ);
|
||||
|
||||
active_file.writer.close()?;
|
||||
writable_file.writer.close()?;
|
||||
|
||||
let file_id = FileId {
|
||||
queue: self.queue,
|
||||
seq,
|
||||
let (path_id, handle) = self
|
||||
.recycle_file(new_seq)
|
||||
.unwrap_or_else(|| self.new_file(new_seq))?;
|
||||
let f = File::<F> {
|
||||
seq: new_seq,
|
||||
handle: handle.into(),
|
||||
format: self.default_format,
|
||||
path_id,
|
||||
reserved: false,
|
||||
};
|
||||
let path = file_id.build_file_path(&self.dir);
|
||||
let fd = Arc::new(self.file_system.create(&path)?);
|
||||
let mut new_file = ActiveFile {
|
||||
seq,
|
||||
writer: build_file_writer(self.file_system.as_ref(), fd.clone())?,
|
||||
let mut new_file = WritableFile {
|
||||
seq: new_seq,
|
||||
writer: build_file_writer(
|
||||
self.file_system.as_ref(),
|
||||
f.handle.clone(),
|
||||
f.format,
|
||||
true, /* force_reset */
|
||||
)?,
|
||||
format: f.format,
|
||||
};
|
||||
// File header must be persisted. This way we can recover gracefully if power
|
||||
// loss before a new entry is written.
|
||||
new_file.writer.sync()?;
|
||||
self.sync_dir()?;
|
||||
|
||||
**active_file = new_file;
|
||||
self.sync_dir(path_id)?;
|
||||
|
||||
**writable_file = new_file;
|
||||
let len = {
|
||||
let mut files = self.files.write();
|
||||
debug_assert!(files.active_seq + 1 == seq);
|
||||
files.active_seq = seq;
|
||||
files.fds.push_back(fd);
|
||||
for listener in &self.listeners {
|
||||
listener.post_new_log_file(FileId {
|
||||
queue: self.queue,
|
||||
seq,
|
||||
});
|
||||
}
|
||||
files.fds.len()
|
||||
let mut files = self.active_files.write();
|
||||
files.push_back(f);
|
||||
files.len()
|
||||
};
|
||||
self.flush_metrics(len);
|
||||
for listener in &self.listeners {
|
||||
listener.post_new_log_file(FileId {
|
||||
queue: self.queue,
|
||||
seq: new_seq,
|
||||
});
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
|
@ -179,30 +304,82 @@ impl<F: FileSystem> SinglePipe<F> {
|
|||
LogQueue::Rewrite => LOG_FILE_COUNT.rewrite.set(len as i64),
|
||||
}
|
||||
}
|
||||
|
||||
/// Synchronizes current recycled states to related metrics.
|
||||
fn flush_recycle_metrics(&self, len: usize) {
|
||||
match self.queue {
|
||||
LogQueue::Append => RECYCLED_FILE_COUNT.append.set(len as i64),
|
||||
LogQueue::Rewrite => RECYCLED_FILE_COUNT.rewrite.set(len as i64),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<F: FileSystem> SinglePipe<F> {
|
||||
fn read_bytes(&self, handle: FileBlockHandle) -> Result<Vec<u8>> {
|
||||
let fd = self.get_fd(handle.id.seq)?;
|
||||
// As the header of each log file already parsed in the processing of loading
|
||||
// log files, we just need to build the `LogFileReader`.
|
||||
let mut reader = build_file_reader(self.file_system.as_ref(), fd)?;
|
||||
reader.read(handle)
|
||||
}
|
||||
|
||||
fn append(&self, bytes: &[u8]) -> Result<FileBlockHandle> {
|
||||
fn append<T: ReactiveBytes + ?Sized>(&self, bytes: &mut T) -> Result<FileBlockHandle> {
|
||||
fail_point!("file_pipe_log::append");
|
||||
let mut active_file = self.active_file.lock();
|
||||
let seq = active_file.seq;
|
||||
let writer = &mut active_file.writer;
|
||||
let mut writable_file = self.writable_file.lock();
|
||||
if writable_file.writer.offset() >= self.target_file_size {
|
||||
self.rotate_imp(&mut writable_file)?;
|
||||
}
|
||||
|
||||
let start_offset = writer.offset();
|
||||
if let Err(e) = writer.write(bytes, self.target_file_size) {
|
||||
if let Err(te) = writer.truncate() {
|
||||
panic!(
|
||||
"error when truncate {} after error: {}, get: {}",
|
||||
seq, e, te
|
||||
);
|
||||
let seq = writable_file.seq;
|
||||
let format = writable_file.format;
|
||||
let ctx = LogFileContext {
|
||||
id: FileId::new(self.queue, seq),
|
||||
version: format.version,
|
||||
};
|
||||
let writer = &mut writable_file.writer;
|
||||
|
||||
#[cfg(feature = "failpoints")]
|
||||
{
|
||||
use crate::util::round_up;
|
||||
|
||||
let corrupted_padding = || {
|
||||
fail_point!("file_pipe_log::append::corrupted_padding", |_| true);
|
||||
false
|
||||
};
|
||||
if format.version.has_log_signing() && format.alignment > 0 {
|
||||
let s_off = round_up(writer.offset(), format.alignment as usize);
|
||||
if s_off > writer.offset() {
|
||||
let len = s_off - writer.offset();
|
||||
let mut zeros = vec![0; len];
|
||||
if corrupted_padding() {
|
||||
zeros[len - 1] = 8_u8;
|
||||
}
|
||||
writer.write(&zeros, self.target_file_size)?;
|
||||
}
|
||||
}
|
||||
return Err(e);
|
||||
}
|
||||
let start_offset = writer.offset();
|
||||
if let Err(e) = writer.write(bytes.as_bytes(&ctx), self.target_file_size) {
|
||||
writer.truncate()?;
|
||||
if is_no_space_err(&e) {
|
||||
// TODO: There exists several corner cases should be tackled if
|
||||
// `bytes.len()` > `target_file_size`. For example,
|
||||
// - [1] main-dir has no recycled logs, and spill-dir have several recycled
|
||||
// logs.
|
||||
// - [2] main-dir has several recycled logs, and sum(recycled_logs.size()) <
|
||||
// expected_file_size, but no recycled logs exist in spill-dir.
|
||||
// - [3] Both main-dir and spill-dir have several recycled logs.
|
||||
// But as `bytes.len()` is always smaller than `target_file_size` in common
|
||||
// cases, this issue will be ignored temprorarily.
|
||||
self.rotate_imp(&mut writable_file)?;
|
||||
// If there still exists free space for this record, rotate the file
|
||||
// and return a special TryAgain Err (for retry) to the caller.
|
||||
return Err(Error::TryAgain(format!(
|
||||
"error when append [{:?}:{seq}]: {e}",
|
||||
self.queue,
|
||||
)));
|
||||
}
|
||||
return Err(Error::Io(e));
|
||||
}
|
||||
let handle = FileBlockHandle {
|
||||
id: FileId {
|
||||
|
@ -218,70 +395,69 @@ impl<F: FileSystem> SinglePipe<F> {
|
|||
Ok(handle)
|
||||
}
|
||||
|
||||
fn maybe_sync(&self, force: bool) -> Result<()> {
|
||||
let mut active_file = self.active_file.lock();
|
||||
let seq = active_file.seq;
|
||||
let writer = &mut active_file.writer;
|
||||
if writer.offset() >= self.target_file_size {
|
||||
if let Err(e) = self.rotate_imp(&mut active_file) {
|
||||
panic!("error when rotate [{:?}:{}]: {}", self.queue, seq, e);
|
||||
}
|
||||
} else if writer.since_last_sync() >= self.bytes_per_sync || force {
|
||||
if let Err(e) = writer.sync() {
|
||||
panic!("error when sync [{:?}:{}]: {}", self.queue, seq, e,);
|
||||
}
|
||||
}
|
||||
|
||||
fn sync(&self) -> Result<()> {
|
||||
let mut writable_file = self.writable_file.lock();
|
||||
let writer = &mut writable_file.writer;
|
||||
let _t = StopWatch::new(perf_context!(log_sync_duration));
|
||||
writer.sync().map_err(Error::Io)?;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn file_span(&self) -> (FileSeq, FileSeq) {
|
||||
let files = self.files.read();
|
||||
(files.first_seq, files.active_seq)
|
||||
let files = self.active_files.read();
|
||||
(files[0].seq, files[files.len() - 1].seq)
|
||||
}
|
||||
|
||||
fn total_size(&self) -> usize {
|
||||
let files = self.files.read();
|
||||
(files.active_seq - files.first_seq + 1) as usize * self.target_file_size
|
||||
let (first_seq, last_seq) = self.file_span();
|
||||
(last_seq - first_seq + 1) as usize * self.target_file_size
|
||||
}
|
||||
|
||||
fn rotate(&self) -> Result<()> {
|
||||
self.rotate_imp(&mut self.active_file.lock())
|
||||
self.rotate_imp(&mut self.writable_file.lock())
|
||||
}
|
||||
|
||||
fn purge_to(&self, file_seq: FileSeq) -> Result<usize> {
|
||||
let (purged, remained) = {
|
||||
let mut files = self.files.write();
|
||||
if file_seq > files.active_seq {
|
||||
return Err(box_err!("Purge active or newer files"));
|
||||
let (len, purged_files) = {
|
||||
let mut files = self.active_files.write();
|
||||
if !(files[0].seq..files[0].seq + files.len() as u64).contains(&file_seq) {
|
||||
return Err(box_err!("FileSeq out of range, cannot be purged"));
|
||||
}
|
||||
let end_offset = file_seq.saturating_sub(files.first_seq) as usize;
|
||||
files.fds.drain(..end_offset);
|
||||
files.first_seq = file_seq;
|
||||
(end_offset, files.fds.len())
|
||||
let off = (file_seq - files[0].seq) as usize;
|
||||
let mut tail = files.split_off(off);
|
||||
std::mem::swap(&mut tail, &mut files);
|
||||
(files.len(), tail)
|
||||
};
|
||||
self.flush_metrics(remained);
|
||||
for seq in file_seq - purged as u64..file_seq {
|
||||
let file_id = FileId {
|
||||
queue: self.queue,
|
||||
seq,
|
||||
};
|
||||
let path = file_id.build_file_path(&self.dir);
|
||||
#[cfg(feature = "failpoints")]
|
||||
{
|
||||
let remove_failure = || {
|
||||
fail::fail_point!("file_pipe_log::remove_file_failure", |_| true);
|
||||
false
|
||||
let purged_len = purged_files.len();
|
||||
if purged_len > 0 {
|
||||
let remains_capacity = self.capacity.saturating_sub(len);
|
||||
let mut recycled_len = self.recycled_files.read().len();
|
||||
let mut new_recycled = VecDeque::new();
|
||||
// We don't rename the append file because on some platform it could cause I/O
|
||||
// jitters. Instead we best-effort rename them during shutdown to reduce
|
||||
// recovery time.
|
||||
for f in purged_files {
|
||||
let file_id = FileId {
|
||||
seq: f.seq,
|
||||
queue: self.queue,
|
||||
};
|
||||
if remove_failure() {
|
||||
let path = file_id.build_file_path(&self.paths[f.path_id]);
|
||||
// Recycle purged files whose version meets the requirement.
|
||||
if f.format.version.has_log_signing() && recycled_len < remains_capacity {
|
||||
new_recycled.push_back(f);
|
||||
recycled_len += 1;
|
||||
continue;
|
||||
}
|
||||
// Remove purged files which are out of capacity and files whose version is
|
||||
// marked not recycled.
|
||||
self.file_system.delete(path)?;
|
||||
}
|
||||
if let Err(e) = fs::remove_file(&path) {
|
||||
warn!("Remove purged log file {:?} failed: {}", path, e);
|
||||
}
|
||||
debug_assert!(recycled_len <= remains_capacity);
|
||||
self.recycled_files.write().append(&mut new_recycled);
|
||||
self.flush_recycle_metrics(recycled_len);
|
||||
}
|
||||
Ok(purged)
|
||||
self.flush_metrics(len);
|
||||
Ok(purged_len)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -289,14 +465,24 @@ impl<F: FileSystem> SinglePipe<F> {
|
|||
pub struct DualPipes<F: FileSystem> {
|
||||
pipes: [SinglePipe<F>; 2],
|
||||
|
||||
_dir_lock: File,
|
||||
dir_locks: Vec<StdFile>,
|
||||
}
|
||||
|
||||
impl<F: FileSystem> Drop for DualPipes<F> {
|
||||
fn drop(&mut self) {
|
||||
for lock in &self.dir_locks {
|
||||
if let Err(e) = FileExt::unlock(lock) {
|
||||
error!("error while unlocking directory: {e}");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<F: FileSystem> DualPipes<F> {
|
||||
/// Open a new [`DualPipes`]. Assumes the two [`SinglePipe`]s share the
|
||||
/// same directory, and that directory is locked by `dir_lock`.
|
||||
pub(super) fn open(
|
||||
dir_lock: File,
|
||||
dir_locks: Vec<StdFile>,
|
||||
appender: SinglePipe<F>,
|
||||
rewriter: SinglePipe<F>,
|
||||
) -> Result<Self> {
|
||||
|
@ -306,7 +492,7 @@ impl<F: FileSystem> DualPipes<F> {
|
|||
|
||||
Ok(Self {
|
||||
pipes: [appender, rewriter],
|
||||
_dir_lock: dir_lock,
|
||||
dir_locks,
|
||||
})
|
||||
}
|
||||
|
||||
|
@ -323,13 +509,17 @@ impl<F: FileSystem> PipeLog for DualPipes<F> {
|
|||
}
|
||||
|
||||
#[inline]
|
||||
fn append(&self, queue: LogQueue, bytes: &[u8]) -> Result<FileBlockHandle> {
|
||||
fn append<T: ReactiveBytes + ?Sized>(
|
||||
&self,
|
||||
queue: LogQueue,
|
||||
bytes: &mut T,
|
||||
) -> Result<FileBlockHandle> {
|
||||
self.pipes[queue as usize].append(bytes)
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn maybe_sync(&self, queue: LogQueue, force: bool) -> Result<()> {
|
||||
self.pipes[queue as usize].maybe_sync(force)
|
||||
fn sync(&self, queue: LogQueue) -> Result<()> {
|
||||
self.pipes[queue as usize].sync()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
|
@ -353,32 +543,60 @@ impl<F: FileSystem> PipeLog for DualPipes<F> {
|
|||
}
|
||||
}
|
||||
|
||||
/// Fetch and return a valid `PathId` of the specific directories.
|
||||
pub(crate) fn find_available_dir(paths: &Paths, target_size: usize) -> PathId {
|
||||
fail_point!("file_pipe_log::force_choose_dir", |s| s
|
||||
.map_or(DEFAULT_PATH_ID, |n| n.parse::<usize>().unwrap()));
|
||||
// Only if one single dir is set by `Config::dir`, can it skip the check of disk
|
||||
// space usage.
|
||||
if paths.len() > 1 {
|
||||
for (t, p) in paths.iter().enumerate() {
|
||||
if let Ok(disk_stats) = fs2::statvfs(p) {
|
||||
if target_size <= disk_stats.available_space() as usize {
|
||||
return t;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
DEFAULT_PATH_ID
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::path::Path;
|
||||
use tempfile::Builder;
|
||||
|
||||
use super::super::format::LogFileHeader;
|
||||
use super::super::format::LogFileFormat;
|
||||
use super::super::pipe_builder::lock_dir;
|
||||
use super::*;
|
||||
use crate::env::DefaultFileSystem;
|
||||
use crate::env::{DefaultFileSystem, ObfuscatedFileSystem};
|
||||
use crate::pipe_log::Version;
|
||||
use crate::util::ReadableSize;
|
||||
|
||||
fn new_test_pipe(cfg: &Config, queue: LogQueue) -> Result<SinglePipe<DefaultFileSystem>> {
|
||||
SinglePipe::open(
|
||||
cfg,
|
||||
Arc::new(DefaultFileSystem),
|
||||
Vec::new(),
|
||||
queue,
|
||||
0,
|
||||
VecDeque::new(),
|
||||
)
|
||||
fn new_test_pipe<F: FileSystem>(
|
||||
cfg: &Config,
|
||||
paths: Paths,
|
||||
queue: LogQueue,
|
||||
fs: Arc<F>,
|
||||
) -> Result<SinglePipe<F>> {
|
||||
SinglePipe::open(cfg, paths, fs, Vec::new(), queue, Vec::new(), Vec::new())
|
||||
}
|
||||
|
||||
fn new_test_pipes(cfg: &Config) -> Result<DualPipes<DefaultFileSystem>> {
|
||||
DualPipes::open(
|
||||
lock_dir(&cfg.dir)?,
|
||||
new_test_pipe(cfg, LogQueue::Append)?,
|
||||
new_test_pipe(cfg, LogQueue::Rewrite)?,
|
||||
vec![lock_dir(&cfg.dir)?],
|
||||
new_test_pipe(
|
||||
cfg,
|
||||
vec![Path::new(&cfg.dir).to_path_buf()],
|
||||
LogQueue::Append,
|
||||
Arc::new(DefaultFileSystem),
|
||||
)?,
|
||||
new_test_pipe(
|
||||
cfg,
|
||||
vec![Path::new(&cfg.dir).to_path_buf()],
|
||||
LogQueue::Rewrite,
|
||||
Arc::new(DefaultFileSystem),
|
||||
)?,
|
||||
)
|
||||
}
|
||||
|
||||
|
@ -407,7 +625,6 @@ mod tests {
|
|||
let cfg = Config {
|
||||
dir: path.to_owned(),
|
||||
target_file_size: ReadableSize::kb(1),
|
||||
bytes_per_sync: ReadableSize::kb(32),
|
||||
..Default::default()
|
||||
};
|
||||
let queue = LogQueue::Append;
|
||||
|
@ -415,21 +632,21 @@ mod tests {
|
|||
let pipe_log = new_test_pipes(&cfg).unwrap();
|
||||
assert_eq!(pipe_log.file_span(queue), (1, 1));
|
||||
|
||||
let header_size = LogFileHeader::len() as u64;
|
||||
let header_size = LogFileFormat::encoded_len(cfg.format_version) as u64;
|
||||
|
||||
// generate file 1, 2, 3
|
||||
let content: Vec<u8> = vec![b'a'; 1024];
|
||||
let file_handle = pipe_log.append(queue, &content).unwrap();
|
||||
pipe_log.maybe_sync(queue, false).unwrap();
|
||||
let file_handle = pipe_log.append(queue, &mut &content).unwrap();
|
||||
assert_eq!(file_handle.id.seq, 1);
|
||||
assert_eq!(file_handle.offset, header_size);
|
||||
assert_eq!(pipe_log.file_span(queue).1, 1);
|
||||
|
||||
let file_handle = pipe_log.append(queue, &mut &content).unwrap();
|
||||
assert_eq!(file_handle.id.seq, 2);
|
||||
assert_eq!(file_handle.offset, header_size);
|
||||
assert_eq!(pipe_log.file_span(queue).1, 2);
|
||||
|
||||
let file_handle = pipe_log.append(queue, &content).unwrap();
|
||||
pipe_log.maybe_sync(queue, false).unwrap();
|
||||
assert_eq!(file_handle.id.seq, 2);
|
||||
assert_eq!(file_handle.offset, header_size);
|
||||
assert_eq!(pipe_log.file_span(queue).1, 3);
|
||||
pipe_log.rotate(queue).unwrap();
|
||||
|
||||
// purge file 1
|
||||
assert_eq!(pipe_log.purge_to(FileId { queue, seq: 2 }).unwrap(), 1);
|
||||
|
@ -440,30 +657,102 @@ mod tests {
|
|||
|
||||
// append position
|
||||
let s_content = b"short content".to_vec();
|
||||
let file_handle = pipe_log.append(queue, &s_content).unwrap();
|
||||
pipe_log.maybe_sync(queue, false).unwrap();
|
||||
let file_handle = pipe_log.append(queue, &mut &s_content).unwrap();
|
||||
assert_eq!(file_handle.id.seq, 3);
|
||||
assert_eq!(file_handle.offset, header_size);
|
||||
|
||||
let file_handle = pipe_log.append(queue, &s_content).unwrap();
|
||||
pipe_log.maybe_sync(queue, false).unwrap();
|
||||
let file_handle = pipe_log.append(queue, &mut &s_content).unwrap();
|
||||
assert_eq!(file_handle.id.seq, 3);
|
||||
assert_eq!(
|
||||
file_handle.offset,
|
||||
header_size as u64 + s_content.len() as u64
|
||||
);
|
||||
assert_eq!(file_handle.offset, header_size + s_content.len() as u64);
|
||||
|
||||
let content_readed = pipe_log
|
||||
.read_bytes(FileBlockHandle {
|
||||
id: FileId { queue, seq: 3 },
|
||||
offset: header_size as u64,
|
||||
offset: header_size,
|
||||
len: s_content.len(),
|
||||
})
|
||||
.unwrap();
|
||||
assert_eq!(content_readed, s_content);
|
||||
// try to fetch abnormal entry
|
||||
let abnormal_content_readed = pipe_log.read_bytes(FileBlockHandle {
|
||||
id: FileId { queue, seq: 12 }, // abnormal seq
|
||||
offset: header_size,
|
||||
len: s_content.len(),
|
||||
});
|
||||
assert!(abnormal_content_readed.is_err());
|
||||
|
||||
// leave only 1 file to truncate
|
||||
assert!(pipe_log.purge_to(FileId { queue, seq: 3 }).is_ok());
|
||||
pipe_log.purge_to(FileId { queue, seq: 3 }).unwrap();
|
||||
assert_eq!(pipe_log.file_span(queue), (3, 3));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_pipe_log_with_recycle() {
|
||||
let dir = Builder::new()
|
||||
.prefix("test_pipe_log_with_recycle")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let path = dir.path().to_str().unwrap();
|
||||
let cfg = Config {
|
||||
dir: path.to_owned(),
|
||||
target_file_size: ReadableSize(1),
|
||||
// super large capacity for recycling
|
||||
purge_threshold: ReadableSize::mb(100),
|
||||
enable_log_recycle: true,
|
||||
format_version: Version::V2,
|
||||
..Default::default()
|
||||
};
|
||||
let queue = LogQueue::Append;
|
||||
let fs = Arc::new(ObfuscatedFileSystem::default());
|
||||
let pipe_log =
|
||||
new_test_pipe(&cfg, vec![Path::new(&cfg.dir).to_path_buf()], queue, fs).unwrap();
|
||||
assert_eq!(pipe_log.file_span(), (1, 1));
|
||||
|
||||
fn content(i: usize) -> Vec<u8> {
|
||||
vec![(i % (u8::MAX as usize)) as u8; 16]
|
||||
}
|
||||
let mut handles = Vec::new();
|
||||
for i in 0..10 {
|
||||
handles.push(pipe_log.append(&mut &content(i)).unwrap());
|
||||
pipe_log.sync().unwrap();
|
||||
}
|
||||
pipe_log.rotate().unwrap();
|
||||
let (first, last) = pipe_log.file_span();
|
||||
// Cannot purge already expired logs or not existsed logs.
|
||||
assert!(pipe_log.purge_to(first - 1).is_err());
|
||||
assert!(pipe_log.purge_to(last + 1).is_err());
|
||||
// Retire files.
|
||||
assert_eq!(pipe_log.purge_to(last).unwrap() as u64, last - first);
|
||||
// Try to read recycled file.
|
||||
for handle in handles.into_iter() {
|
||||
assert!(pipe_log.read_bytes(handle).is_err());
|
||||
}
|
||||
// Try to reuse.
|
||||
let mut handles = Vec::new();
|
||||
for i in 0..10 {
|
||||
handles.push(pipe_log.append(&mut &content(i + 1)).unwrap());
|
||||
pipe_log.sync().unwrap();
|
||||
}
|
||||
// Verify the data.
|
||||
for (i, handle) in handles.into_iter().enumerate() {
|
||||
assert_eq!(pipe_log.read_bytes(handle).unwrap(), content(i + 1));
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_release_on_drop() {
|
||||
let dir = Builder::new()
|
||||
.prefix("test_release_on_drop")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let path = dir.path().to_str().unwrap();
|
||||
let cfg = Config {
|
||||
dir: path.to_owned(),
|
||||
target_file_size: ReadableSize(1),
|
||||
..Default::default()
|
||||
};
|
||||
let pipe_log = new_test_pipes(&cfg).unwrap();
|
||||
drop(pipe_log);
|
||||
assert!(new_test_pipes(&cfg).is_ok());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -2,29 +2,37 @@
|
|||
|
||||
//! Helper types to recover in-memory states from log files.
|
||||
|
||||
use std::collections::VecDeque;
|
||||
use std::fs::{self, File};
|
||||
use std::fs::{self, File as StdFile};
|
||||
use std::io::Write;
|
||||
use std::marker::PhantomData;
|
||||
use std::path::Path;
|
||||
use std::path::{Path, PathBuf};
|
||||
use std::sync::Arc;
|
||||
use std::time::Instant;
|
||||
|
||||
use fs2::FileExt;
|
||||
use log::{error, info, warn};
|
||||
use rayon::prelude::*;
|
||||
|
||||
use crate::config::{Config, RecoveryMode};
|
||||
use crate::env::FileSystem;
|
||||
use crate::env::{FileSystem, Handle, Permission};
|
||||
use crate::errors::is_no_space_err;
|
||||
use crate::event_listener::EventListener;
|
||||
use crate::log_batch::LogItemBatch;
|
||||
use crate::log_batch::{LogItemBatch, LOG_BATCH_HEADER_LEN};
|
||||
use crate::pipe_log::{FileId, FileSeq, LogQueue};
|
||||
use crate::util::Factory;
|
||||
use crate::util::{Factory, ReadableSize};
|
||||
use crate::{Error, Result};
|
||||
|
||||
use super::format::{lock_file_path, FileNameExt, LogFileHeader};
|
||||
use super::format::{
|
||||
build_reserved_file_name, lock_file_path, parse_reserved_file_name, FileNameExt, LogFileFormat,
|
||||
};
|
||||
use super::log_file::build_file_reader;
|
||||
use super::pipe::{DualPipes, SinglePipe};
|
||||
use super::pipe::{
|
||||
find_available_dir, DualPipes, File, PathId, Paths, SinglePipe, DEFAULT_FIRST_FILE_SEQ,
|
||||
};
|
||||
use super::reader::LogItemBatchFileReader;
|
||||
use crate::env::Handle;
|
||||
|
||||
/// Maximum size for the buffer for prefilling.
|
||||
const PREFILL_BUFFER_SIZE: usize = ReadableSize::mb(16).0 as usize;
|
||||
|
||||
/// `ReplayMachine` is a type of deterministic state machine that obeys
|
||||
/// associative law.
|
||||
|
@ -55,9 +63,12 @@ impl<M: ReplayMachine + Default> Factory<M> for DefaultMachineFactory<M> {
|
|||
}
|
||||
}
|
||||
|
||||
struct FileToRecover<F: FileSystem> {
|
||||
seq: FileSeq,
|
||||
handle: Arc<F::Handle>,
|
||||
/// Container for basic settings on recovery.
|
||||
pub struct RecoveryConfig {
|
||||
pub queue: LogQueue,
|
||||
pub mode: RecoveryMode,
|
||||
pub concurrency: usize,
|
||||
pub read_block_size: u64,
|
||||
}
|
||||
|
||||
/// [`DualPipes`] factory that can also recover other customized memory states.
|
||||
|
@ -67,11 +78,16 @@ pub struct DualPipesBuilder<F: FileSystem> {
|
|||
listeners: Vec<Arc<dyn EventListener>>,
|
||||
|
||||
/// Only filled after a successful call of `DualPipesBuilder::scan`.
|
||||
dir_lock: Option<File>,
|
||||
/// Only filled after a successful call of `DualPipesBuilder::scan`.
|
||||
append_files: Vec<FileToRecover<F>>,
|
||||
/// Only filled after a successful call of `DualPipesBuilder::scan`.
|
||||
rewrite_files: Vec<FileToRecover<F>>,
|
||||
dirs: Paths,
|
||||
dir_locks: Vec<StdFile>,
|
||||
|
||||
pub(crate) append_file_names: Vec<FileName>,
|
||||
pub(crate) rewrite_file_names: Vec<FileName>,
|
||||
pub(crate) recycled_file_names: Vec<FileName>,
|
||||
|
||||
append_files: Vec<File<F>>,
|
||||
rewrite_files: Vec<File<F>>,
|
||||
recycled_files: Vec<File<F>>,
|
||||
}
|
||||
|
||||
impl<F: FileSystem> DualPipesBuilder<F> {
|
||||
|
@ -81,89 +97,213 @@ impl<F: FileSystem> DualPipesBuilder<F> {
|
|||
cfg,
|
||||
file_system,
|
||||
listeners,
|
||||
dir_lock: None,
|
||||
dirs: Vec::new(),
|
||||
dir_locks: Vec::new(),
|
||||
append_file_names: Vec::new(),
|
||||
rewrite_file_names: Vec::new(),
|
||||
recycled_file_names: Vec::new(),
|
||||
append_files: Vec::new(),
|
||||
rewrite_files: Vec::new(),
|
||||
recycled_files: Vec::new(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Scans for all log files under the working directory. The directory will
|
||||
/// be created if not exists.
|
||||
pub fn scan(&mut self) -> Result<()> {
|
||||
let dir = &self.cfg.dir;
|
||||
let path = Path::new(dir);
|
||||
if !path.exists() {
|
||||
info!("Create raft log directory: {}", dir);
|
||||
fs::create_dir(dir)?;
|
||||
self.dir_lock = Some(lock_dir(dir)?);
|
||||
return Ok(());
|
||||
}
|
||||
if !path.is_dir() {
|
||||
return Err(box_err!("Not directory: {}", dir));
|
||||
}
|
||||
self.dir_lock = Some(lock_dir(dir)?);
|
||||
self.scan_and_sort(true)?;
|
||||
|
||||
let (mut min_append_id, mut max_append_id) = (u64::MAX, 0);
|
||||
let (mut min_rewrite_id, mut max_rewrite_id) = (u64::MAX, 0);
|
||||
fs::read_dir(path)?.for_each(|e| {
|
||||
if let Ok(e) = e {
|
||||
let p = e.path();
|
||||
if p.is_file() {
|
||||
match FileId::parse_file_name(p.file_name().unwrap().to_str().unwrap()) {
|
||||
Some(FileId {
|
||||
queue: LogQueue::Append,
|
||||
seq,
|
||||
}) => {
|
||||
min_append_id = std::cmp::min(min_append_id, seq);
|
||||
max_append_id = std::cmp::max(max_append_id, seq);
|
||||
// Open all files with suitable permissions.
|
||||
self.append_files = Vec::with_capacity(self.append_file_names.len());
|
||||
for (i, file_name) in self.append_file_names.iter().enumerate() {
|
||||
let perm = if i == self.append_file_names.len() - 1
|
||||
|| self.cfg.recovery_mode == RecoveryMode::TolerateAnyCorruption
|
||||
{
|
||||
Permission::ReadWrite
|
||||
} else {
|
||||
Permission::ReadOnly
|
||||
};
|
||||
self.append_files.push(File {
|
||||
seq: file_name.seq,
|
||||
handle: Arc::new(self.file_system.open(&file_name.path, perm)?),
|
||||
format: LogFileFormat::default(),
|
||||
path_id: file_name.path_id,
|
||||
reserved: false,
|
||||
});
|
||||
}
|
||||
self.rewrite_files = Vec::with_capacity(self.rewrite_file_names.len());
|
||||
for (i, file_name) in self.rewrite_file_names.iter().enumerate() {
|
||||
let perm = if i == self.rewrite_file_names.len() - 1
|
||||
|| self.cfg.recovery_mode == RecoveryMode::TolerateAnyCorruption
|
||||
{
|
||||
Permission::ReadWrite
|
||||
} else {
|
||||
Permission::ReadOnly
|
||||
};
|
||||
self.rewrite_files.push(File {
|
||||
seq: file_name.seq,
|
||||
handle: Arc::new(self.file_system.open(&file_name.path, perm)?),
|
||||
format: LogFileFormat::default(),
|
||||
path_id: file_name.path_id,
|
||||
reserved: false,
|
||||
});
|
||||
}
|
||||
self.recycled_files = Vec::with_capacity(self.recycled_file_names.len());
|
||||
for file_name in &self.recycled_file_names {
|
||||
self.recycled_files.push(File {
|
||||
seq: file_name.seq,
|
||||
handle: Arc::new(
|
||||
self.file_system
|
||||
.open(&file_name.path, Permission::ReadOnly)?,
|
||||
),
|
||||
format: LogFileFormat::default(),
|
||||
path_id: file_name.path_id,
|
||||
reserved: true,
|
||||
});
|
||||
}
|
||||
|
||||
// Validate and clear obsolete metadata and log files.
|
||||
for (queue, files, is_recycled_file) in [
|
||||
(LogQueue::Append, &mut self.append_files, false),
|
||||
(LogQueue::Rewrite, &mut self.rewrite_files, false),
|
||||
(LogQueue::Append, &mut self.recycled_files, true),
|
||||
] {
|
||||
// Check the file_list and remove the hole of files.
|
||||
let mut invalid_idx = 0_usize;
|
||||
for (i, file_pair) in files.windows(2).enumerate() {
|
||||
// If there exists a black hole or duplicate scenario on FileSeq, these
|
||||
// files should be skipped and cleared.
|
||||
if file_pair[1].seq - file_pair[0].seq != 1 {
|
||||
invalid_idx = i + 1;
|
||||
}
|
||||
}
|
||||
files.drain(..invalid_idx);
|
||||
// Try to cleanup stale metadata left by the previous version.
|
||||
if files.is_empty() || is_recycled_file {
|
||||
continue;
|
||||
}
|
||||
let max_sample = 100;
|
||||
// Find the first obsolete metadata.
|
||||
let mut delete_start = None;
|
||||
for i in 0..max_sample {
|
||||
let seq = i * files[0].seq / max_sample;
|
||||
let file_id = FileId { queue, seq };
|
||||
for dir in self.dirs.iter() {
|
||||
if self
|
||||
.file_system
|
||||
.exists_metadata(file_id.build_file_path(dir))
|
||||
{
|
||||
delete_start = Some(i.saturating_sub(1) * files[0].seq / max_sample + 1);
|
||||
break;
|
||||
}
|
||||
}
|
||||
if delete_start.is_some() {
|
||||
break;
|
||||
}
|
||||
}
|
||||
// Delete metadata starting from the oldest. Abort on error.
|
||||
let mut cleared = 0_u64;
|
||||
if let Some(clear_start) = delete_start {
|
||||
for seq in (clear_start..files[0].seq).rev() {
|
||||
let file_id = FileId { queue, seq };
|
||||
for dir in self.dirs.iter() {
|
||||
let path = if is_recycled_file {
|
||||
dir.join(build_reserved_file_name(seq))
|
||||
} else {
|
||||
file_id.build_file_path(dir)
|
||||
};
|
||||
if self.file_system.exists_metadata(&path) {
|
||||
if let Err(e) = self.file_system.delete_metadata(&path) {
|
||||
error!("failed to delete metadata of {}: {e}.", path.display());
|
||||
break;
|
||||
}
|
||||
cleared += 1;
|
||||
}
|
||||
Some(FileId {
|
||||
queue: LogQueue::Rewrite,
|
||||
seq,
|
||||
}) => {
|
||||
min_rewrite_id = std::cmp::min(min_rewrite_id, seq);
|
||||
max_rewrite_id = std::cmp::max(max_rewrite_id, seq);
|
||||
}
|
||||
_ => {}
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
for (queue, min_id, max_id, files) in [
|
||||
(
|
||||
LogQueue::Append,
|
||||
min_append_id,
|
||||
max_append_id,
|
||||
&mut self.append_files,
|
||||
),
|
||||
(
|
||||
LogQueue::Rewrite,
|
||||
min_rewrite_id,
|
||||
max_rewrite_id,
|
||||
&mut self.rewrite_files,
|
||||
),
|
||||
] {
|
||||
if max_id > 0 {
|
||||
for seq in min_id..=max_id {
|
||||
let file_id = FileId { queue, seq };
|
||||
let path = file_id.build_file_path(dir);
|
||||
if !path.exists() {
|
||||
warn!(
|
||||
"Detected a hole when scanning directory, discarding files before {:?}.",
|
||||
file_id,
|
||||
);
|
||||
files.clear();
|
||||
} else {
|
||||
let handle = Arc::new(self.file_system.open(&path)?);
|
||||
files.push(FileToRecover { seq, handle });
|
||||
}
|
||||
}
|
||||
if cleared > 0 {
|
||||
warn!(
|
||||
"clear {cleared} stale metadata of {queue:?} in range [0, {}).",
|
||||
files[0].seq,
|
||||
);
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub(crate) fn scan_and_sort(&mut self, lock: bool) -> Result<()> {
|
||||
let dir = self.cfg.dir.clone();
|
||||
self.scan_dir(&dir, lock)?;
|
||||
|
||||
if let Some(dir) = self.cfg.spill_dir.clone() {
|
||||
self.scan_dir(&dir, lock)?;
|
||||
}
|
||||
|
||||
self.append_file_names.sort_by(|a, b| a.seq.cmp(&b.seq));
|
||||
self.rewrite_file_names.sort_by(|a, b| a.seq.cmp(&b.seq));
|
||||
self.recycled_file_names.sort_by(|a, b| a.seq.cmp(&b.seq));
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn scan_dir(&mut self, dir: &str, lock: bool) -> Result<()> {
|
||||
let dir = Path::new(dir);
|
||||
if !dir.exists() {
|
||||
if lock {
|
||||
info!("Create raft log directory: {}", dir.display());
|
||||
fs::create_dir(dir)?;
|
||||
self.dir_locks.push(lock_dir(dir)?);
|
||||
}
|
||||
self.dirs.push(dir.to_path_buf());
|
||||
return Ok(());
|
||||
}
|
||||
if !dir.is_dir() {
|
||||
return Err(box_err!("Not directory: {}", dir.display()));
|
||||
}
|
||||
if lock {
|
||||
self.dir_locks.push(lock_dir(dir)?);
|
||||
}
|
||||
self.dirs.push(dir.to_path_buf());
|
||||
let path_id = self.dirs.len() - 1;
|
||||
|
||||
fs::read_dir(dir)?.try_for_each(|e| -> Result<()> {
|
||||
let dir_entry = e?;
|
||||
let p = dir_entry.path();
|
||||
if !p.is_file() {
|
||||
return Ok(());
|
||||
}
|
||||
let file_name = p.file_name().unwrap().to_str().unwrap();
|
||||
match FileId::parse_file_name(file_name) {
|
||||
Some(FileId {
|
||||
queue: LogQueue::Append,
|
||||
seq,
|
||||
}) => self.append_file_names.push(FileName {
|
||||
seq,
|
||||
path: p,
|
||||
path_id,
|
||||
}),
|
||||
Some(FileId {
|
||||
queue: LogQueue::Rewrite,
|
||||
seq,
|
||||
}) => self.rewrite_file_names.push(FileName {
|
||||
seq,
|
||||
path: p,
|
||||
path_id,
|
||||
}),
|
||||
_ => {
|
||||
if let Some(seq) = parse_reserved_file_name(file_name) {
|
||||
self.recycled_file_names.push(FileName {
|
||||
seq,
|
||||
path: p,
|
||||
path_id,
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
})
|
||||
}
|
||||
|
||||
/// Reads through log items in all available log files, and replays them to
|
||||
/// specific [`ReplayMachine`]s that can be constructed via
|
||||
/// `machine_factory`.
|
||||
|
@ -171,7 +311,18 @@ impl<F: FileSystem> DualPipesBuilder<F> {
|
|||
&mut self,
|
||||
machine_factory: &FA,
|
||||
) -> Result<(M, M)> {
|
||||
let threads = self.cfg.recovery_threads;
|
||||
if self.append_files.is_empty() && self.rewrite_files.is_empty() {
|
||||
// Avoid creating a thread pool.
|
||||
return Ok((machine_factory.new_target(), machine_factory.new_target()));
|
||||
}
|
||||
let threads = std::cmp::min(
|
||||
self.cfg.recovery_threads,
|
||||
self.append_files.len() + self.rewrite_files.len(),
|
||||
);
|
||||
let pool = rayon::ThreadPoolBuilder::new()
|
||||
.num_threads(threads)
|
||||
.build()
|
||||
.unwrap();
|
||||
let (append_concurrency, rewrite_concurrency) =
|
||||
match (self.append_files.len(), self.rewrite_files.len()) {
|
||||
(a, b) if a > 0 && b > 0 => {
|
||||
|
@ -181,162 +332,292 @@ impl<F: FileSystem> DualPipesBuilder<F> {
|
|||
}
|
||||
_ => (threads, threads),
|
||||
};
|
||||
let pool = rayon::ThreadPoolBuilder::new()
|
||||
.num_threads(threads)
|
||||
.build()
|
||||
.unwrap();
|
||||
let (append, rewrite) = pool.join(
|
||||
|| self.recover_queue(LogQueue::Append, machine_factory, append_concurrency),
|
||||
|| self.recover_queue(LogQueue::Rewrite, machine_factory, rewrite_concurrency),
|
||||
);
|
||||
let append_recovery_cfg = RecoveryConfig {
|
||||
queue: LogQueue::Append,
|
||||
mode: self.cfg.recovery_mode,
|
||||
concurrency: append_concurrency,
|
||||
read_block_size: self.cfg.recovery_read_block_size.0,
|
||||
};
|
||||
let rewrite_recovery_cfg = RecoveryConfig {
|
||||
queue: LogQueue::Rewrite,
|
||||
concurrency: rewrite_concurrency,
|
||||
..append_recovery_cfg
|
||||
};
|
||||
|
||||
let append_files = &mut self.append_files;
|
||||
let rewrite_files = &mut self.rewrite_files;
|
||||
|
||||
let file_system = self.file_system.clone();
|
||||
// As the `recover_queue` would update the `LogFileFormat` of each log file
|
||||
// in `apend_files` and `rewrite_files`, we re-design the implementation on
|
||||
// `recover_queue` to make it compatiable to concurrent processing
|
||||
// with ThreadPool.
|
||||
let (append, rewrite) = pool.join(
|
||||
|| {
|
||||
DualPipesBuilder::recover_queue_imp(
|
||||
file_system.clone(),
|
||||
append_recovery_cfg,
|
||||
append_files,
|
||||
machine_factory,
|
||||
)
|
||||
},
|
||||
|| {
|
||||
DualPipesBuilder::recover_queue_imp(
|
||||
file_system.clone(),
|
||||
rewrite_recovery_cfg,
|
||||
rewrite_files,
|
||||
machine_factory,
|
||||
)
|
||||
},
|
||||
);
|
||||
Ok((append?, rewrite?))
|
||||
}
|
||||
|
||||
/// Reads through log items in all available log files of the specified
|
||||
/// queue, and replays them to specific [`ReplayMachine`]s that can be
|
||||
/// constructed via `machine_factory`.
|
||||
pub fn recover_queue<M: ReplayMachine, FA: Factory<M>>(
|
||||
&self,
|
||||
queue: LogQueue,
|
||||
replay_machine_factory: &FA,
|
||||
concurrency: usize,
|
||||
/// Manually reads through log items in all available log files of the
|
||||
/// specified queue, and replays them to specific [`ReplayMachine`]s
|
||||
/// that can be constructed via `machine_factory`.
|
||||
fn recover_queue_imp<M: ReplayMachine, FA: Factory<M>>(
|
||||
file_system: Arc<F>,
|
||||
recovery_cfg: RecoveryConfig,
|
||||
files: &mut [File<F>],
|
||||
machine_factory: &FA,
|
||||
) -> Result<M> {
|
||||
let files = if queue == LogQueue::Append {
|
||||
&self.append_files
|
||||
} else {
|
||||
&self.rewrite_files
|
||||
};
|
||||
if concurrency == 0 || files.is_empty() {
|
||||
return Ok(replay_machine_factory.new_target());
|
||||
if recovery_cfg.concurrency == 0 || files.is_empty() {
|
||||
return Ok(machine_factory.new_target());
|
||||
}
|
||||
let queue = recovery_cfg.queue;
|
||||
let concurrency = recovery_cfg.concurrency;
|
||||
let recovery_mode = recovery_cfg.mode;
|
||||
let recovery_read_block_size = recovery_cfg.read_block_size as usize;
|
||||
|
||||
let recovery_mode = self.cfg.recovery_mode;
|
||||
let max_chunk_size = std::cmp::max((files.len() + concurrency - 1) / concurrency, 1);
|
||||
let chunks = files.par_chunks(max_chunk_size);
|
||||
let max_chunk_size = files.len().div_ceil(concurrency);
|
||||
let chunks = files.par_chunks_mut(max_chunk_size);
|
||||
let chunk_count = chunks.len();
|
||||
debug_assert!(chunk_count <= concurrency);
|
||||
let sequential_replay_machine = chunks
|
||||
let machine = chunks
|
||||
.enumerate()
|
||||
.map(|(index, chunk)| {
|
||||
let mut reader =
|
||||
LogItemBatchFileReader::new(self.cfg.recovery_read_block_size.0 as usize);
|
||||
let mut sequential_replay_machine = replay_machine_factory.new_target();
|
||||
LogItemBatchFileReader::new(recovery_read_block_size);
|
||||
let mut machine = machine_factory.new_target();
|
||||
let file_count = chunk.len();
|
||||
for (i, f) in chunk.iter().enumerate() {
|
||||
for (i, f) in chunk.iter_mut().enumerate() {
|
||||
let is_last_file = index == chunk_count - 1 && i == file_count - 1;
|
||||
if let Err(e) = reader.open(
|
||||
FileId { queue, seq: f.seq },
|
||||
build_file_reader(self.file_system.as_ref(), f.handle.clone())?,
|
||||
) {
|
||||
if f.handle.file_size()? > LogFileHeader::len() {
|
||||
// This file contains some entries.
|
||||
error!(
|
||||
"Failed to open last log file due to broken header: {:?}:{}",
|
||||
queue, f.seq
|
||||
);
|
||||
return Err(e);
|
||||
}
|
||||
if recovery_mode == RecoveryMode::TolerateAnyCorruption {
|
||||
warn!(
|
||||
"File header is corrupted but ignored: {:?}:{}, {}",
|
||||
queue, f.seq, e
|
||||
);
|
||||
f.handle.truncate(0)?;
|
||||
} else if recovery_mode == RecoveryMode::TolerateTailCorruption
|
||||
&& is_last_file
|
||||
{
|
||||
warn!(
|
||||
"The last log file is corrupted but ignored: {:?}:{}, {}",
|
||||
queue, f.seq, e
|
||||
);
|
||||
f.handle.truncate(0)?;
|
||||
} else {
|
||||
error!(
|
||||
"Failed to open log file due to broken header: {:?}:{}",
|
||||
queue, f.seq
|
||||
);
|
||||
return Err(e);
|
||||
let file_reader = build_file_reader(file_system.as_ref(), f.handle.clone())?;
|
||||
match reader.open(FileId { queue, seq: f.seq }, file_reader) {
|
||||
Err(e) if matches!(e, Error::Io(_)) => return Err(e),
|
||||
Err(e) => {
|
||||
// TODO: More reliable tail detection.
|
||||
if recovery_mode == RecoveryMode::TolerateAnyCorruption
|
||||
|| recovery_mode == RecoveryMode::TolerateTailCorruption
|
||||
&& is_last_file {
|
||||
warn!(
|
||||
"Truncating log file due to broken header (queue={:?},seq={}): {}",
|
||||
queue, f.seq, e
|
||||
);
|
||||
f.handle.truncate(0)?;
|
||||
f.format = LogFileFormat::default();
|
||||
continue;
|
||||
} else {
|
||||
error!(
|
||||
"Failed to open log file due to broken header (queue={:?},seq={}): {}",
|
||||
queue, f.seq, e
|
||||
);
|
||||
return Err(e);
|
||||
}
|
||||
},
|
||||
Ok(format) => {
|
||||
f.format = format;
|
||||
}
|
||||
}
|
||||
let mut pending_item = None;
|
||||
loop {
|
||||
match reader.next() {
|
||||
match pending_item.unwrap_or_else(|| reader.next()) {
|
||||
Ok(Some(item_batch)) => {
|
||||
sequential_replay_machine
|
||||
.replay(item_batch, FileId { queue, seq: f.seq })?;
|
||||
let next_item = reader.next();
|
||||
// This is the last item. Check entries block.
|
||||
if_chain::if_chain! {
|
||||
if matches!(next_item, Err(_) | Ok(None));
|
||||
if let Some(ei) = item_batch.entry_index();
|
||||
let handle = ei.entries.unwrap();
|
||||
if let Err(e) = crate::LogBatch::decode_entries_block(
|
||||
&reader.reader.as_mut().unwrap().read(handle)?,
|
||||
handle,
|
||||
ei.compression_type,
|
||||
);
|
||||
then {
|
||||
let offset = handle.offset as usize - LOG_BATCH_HEADER_LEN;
|
||||
if recovery_mode == RecoveryMode::AbsoluteConsistency {
|
||||
error!(
|
||||
"Failed to open log file due to broken entry (queue={:?},seq={},offset={}): {}",
|
||||
queue, f.seq, offset, e
|
||||
);
|
||||
return Err(e);
|
||||
} else {
|
||||
warn!(
|
||||
"Truncating log file due to broken entries block (queue={:?},seq={},offset={}): {}",
|
||||
queue, f.seq, offset, e
|
||||
);
|
||||
f.handle.truncate(offset)?;
|
||||
f.handle.sync()?;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
pending_item = Some(next_item);
|
||||
machine.replay(item_batch, FileId { queue, seq: f.seq })?;
|
||||
}
|
||||
Ok(None) => break,
|
||||
Err(e)
|
||||
if recovery_mode == RecoveryMode::TolerateTailCorruption
|
||||
&& is_last_file =>
|
||||
&& is_last_file || recovery_mode == RecoveryMode::TolerateAnyCorruption =>
|
||||
{
|
||||
warn!(
|
||||
"The last log file is corrupted but ignored: {:?}:{}, {}",
|
||||
queue, f.seq, e
|
||||
);
|
||||
f.handle.truncate(reader.valid_offset())?;
|
||||
break;
|
||||
}
|
||||
Err(e) if recovery_mode == RecoveryMode::TolerateAnyCorruption => {
|
||||
warn!(
|
||||
"File is corrupted but ignored: {:?}:{}, {}",
|
||||
queue, f.seq, e
|
||||
"Truncating log file due to broken batch (queue={:?},seq={},offset={}): {}",
|
||||
queue, f.seq, reader.valid_offset(), e
|
||||
);
|
||||
f.handle.truncate(reader.valid_offset())?;
|
||||
f.handle.sync()?;
|
||||
break;
|
||||
}
|
||||
Err(e) => {
|
||||
error!(
|
||||
"Failed to open log file due to broken entry: {:?}:{} offset={}",
|
||||
queue, f.seq, reader.valid_offset()
|
||||
"Failed to open log file due to broken batch (queue={:?},seq={},offset={}): {}",
|
||||
queue, f.seq, reader.valid_offset(), e
|
||||
);
|
||||
return Err(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
Ok(sequential_replay_machine)
|
||||
Ok(machine)
|
||||
})
|
||||
.try_reduce(
|
||||
|| replay_machine_factory.new_target(),
|
||||
|mut sequential_replay_machine_left, sequential_replay_machine_right| {
|
||||
sequential_replay_machine_left.merge(sequential_replay_machine_right, queue)?;
|
||||
Ok(sequential_replay_machine_left)
|
||||
|| machine_factory.new_target(),
|
||||
|mut lhs, rhs| {
|
||||
lhs.merge(rhs, queue)?;
|
||||
Ok(lhs)
|
||||
},
|
||||
)?;
|
||||
|
||||
Ok(sequential_replay_machine)
|
||||
Ok(machine)
|
||||
}
|
||||
|
||||
/// Builds a new storage for the specified log queue.
|
||||
fn build_pipe(&self, queue: LogQueue) -> Result<SinglePipe<F>> {
|
||||
let files = match queue {
|
||||
LogQueue::Append => &self.append_files,
|
||||
LogQueue::Rewrite => &self.rewrite_files,
|
||||
/// Manually reads through log items in all available log files of the
|
||||
/// specified `[LogQueue]`, and replays them to specific
|
||||
/// [`ReplayMachine`]s that can be constructed via `machine_factory`.
|
||||
#[allow(dead_code)]
|
||||
pub fn recover_queue<M: ReplayMachine, FA: Factory<M>>(
|
||||
&mut self,
|
||||
file_system: Arc<F>,
|
||||
recovery_cfg: RecoveryConfig,
|
||||
replay_machine_factory: &FA,
|
||||
) -> Result<M> {
|
||||
let files = if recovery_cfg.queue == LogQueue::Append {
|
||||
&mut self.append_files
|
||||
} else {
|
||||
&mut self.rewrite_files
|
||||
};
|
||||
let first_seq = files.first().map(|f| f.seq).unwrap_or(0);
|
||||
let files: VecDeque<Arc<F::Handle>> = files.iter().map(|f| f.handle.clone()).collect();
|
||||
SinglePipe::open(
|
||||
&self.cfg,
|
||||
self.file_system.clone(),
|
||||
self.listeners.clone(),
|
||||
queue,
|
||||
first_seq,
|
||||
DualPipesBuilder::recover_queue_imp(
|
||||
file_system,
|
||||
recovery_cfg,
|
||||
files,
|
||||
replay_machine_factory,
|
||||
)
|
||||
}
|
||||
|
||||
fn initialize_files(&mut self) -> Result<()> {
|
||||
let target_file_size = self.cfg.target_file_size.0 as usize;
|
||||
let mut target = std::cmp::min(
|
||||
self.cfg.prefill_capacity(),
|
||||
self.cfg
|
||||
.recycle_capacity()
|
||||
.saturating_sub(self.append_files.len()),
|
||||
);
|
||||
let to_create = target.saturating_sub(self.recycled_files.len());
|
||||
if to_create > 0 {
|
||||
let now = Instant::now();
|
||||
for _ in 0..to_create {
|
||||
let seq = self
|
||||
.recycled_files
|
||||
.last()
|
||||
.map(|f| f.seq + 1)
|
||||
.unwrap_or_else(|| DEFAULT_FIRST_FILE_SEQ);
|
||||
let path_id = find_available_dir(&self.dirs, target_file_size);
|
||||
let root_path = &self.dirs[path_id];
|
||||
let path = root_path.join(build_reserved_file_name(seq));
|
||||
let handle = Arc::new(self.file_system.create(path)?);
|
||||
let mut writer = self.file_system.new_writer(handle.clone())?;
|
||||
let mut written = 0;
|
||||
let buf = vec![0; std::cmp::min(PREFILL_BUFFER_SIZE, target_file_size)];
|
||||
while written < target_file_size {
|
||||
if let Err(e) = writer.write_all(&buf) {
|
||||
warn!("failed to build reserved file, err: {e}");
|
||||
if is_no_space_err(&e) {
|
||||
warn!("no enough space for preparing reserved logs");
|
||||
// Clear partially prepared recycled log list if there has no enough
|
||||
// space for it.
|
||||
target = 0;
|
||||
}
|
||||
break;
|
||||
}
|
||||
written += buf.len();
|
||||
}
|
||||
self.recycled_files.push(File {
|
||||
seq,
|
||||
handle,
|
||||
format: LogFileFormat::default(),
|
||||
path_id,
|
||||
reserved: true,
|
||||
});
|
||||
}
|
||||
info!(
|
||||
"prefill logs takes {:?}, created {to_create} files",
|
||||
now.elapsed(),
|
||||
);
|
||||
}
|
||||
// If target recycled capacity has been changed when restarting by manually
|
||||
// modifications, such as setting `Config::enable-log-recycle` from TRUE to
|
||||
// FALSE, setting `Config::prefill-for-recycle` from TRUE to FALSE or
|
||||
// changing the recycle capacity, we should remove redundant
|
||||
// recycled files in advance.
|
||||
while self.recycled_files.len() > target {
|
||||
let f = self.recycled_files.pop().unwrap();
|
||||
let root_path = &self.dirs[f.path_id];
|
||||
let path = root_path.join(build_reserved_file_name(f.seq));
|
||||
let _ = self.file_system.delete(path);
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Builds a [`DualPipes`] that contains all available log files.
|
||||
pub fn finish(self) -> Result<DualPipes<F>> {
|
||||
let appender = self.build_pipe(LogQueue::Append)?;
|
||||
let rewriter = self.build_pipe(LogQueue::Rewrite)?;
|
||||
DualPipes::open(self.dir_lock.unwrap(), appender, rewriter)
|
||||
pub fn finish(mut self) -> Result<DualPipes<F>> {
|
||||
self.initialize_files()?;
|
||||
let appender = SinglePipe::open(
|
||||
&self.cfg,
|
||||
self.dirs.clone(),
|
||||
self.file_system.clone(),
|
||||
self.listeners.clone(),
|
||||
LogQueue::Append,
|
||||
self.append_files,
|
||||
self.recycled_files,
|
||||
)?;
|
||||
let rewriter = SinglePipe::open(
|
||||
&self.cfg,
|
||||
self.dirs,
|
||||
self.file_system.clone(),
|
||||
self.listeners.clone(),
|
||||
LogQueue::Rewrite,
|
||||
self.rewrite_files,
|
||||
Vec::new(),
|
||||
)?;
|
||||
DualPipes::open(self.dir_locks, appender, rewriter)
|
||||
}
|
||||
}
|
||||
|
||||
/// Creates and exclusively locks a lock file under the given directory.
|
||||
pub(super) fn lock_dir(dir: &str) -> Result<File> {
|
||||
let lock_file = File::create(lock_file_path(dir))?;
|
||||
pub(super) fn lock_dir<P: AsRef<Path>>(dir: P) -> Result<StdFile> {
|
||||
let lock_file = StdFile::create(lock_file_path(dir))?;
|
||||
lock_file.try_lock_exclusive().map_err(|e| {
|
||||
Error::Other(box_err!(
|
||||
"Failed to lock file: {}, maybe another instance is using this directory.",
|
||||
|
@ -345,3 +626,9 @@ pub(super) fn lock_dir(dir: &str) -> Result<File> {
|
|||
})?;
|
||||
Ok(lock_file)
|
||||
}
|
||||
|
||||
pub(crate) struct FileName {
|
||||
pub seq: FileSeq,
|
||||
pub path: PathBuf,
|
||||
path_id: PathId,
|
||||
}
|
||||
|
|
|
@ -2,16 +2,18 @@
|
|||
|
||||
use crate::env::FileSystem;
|
||||
use crate::log_batch::{LogBatch, LogItemBatch, LOG_BATCH_HEADER_LEN};
|
||||
use crate::pipe_log::{FileBlockHandle, FileId};
|
||||
use crate::pipe_log::{FileBlockHandle, FileId, LogFileContext};
|
||||
use crate::util::round_up;
|
||||
use crate::{Error, Result};
|
||||
|
||||
use super::format::LogFileHeader;
|
||||
use super::format::{is_zero_padded, LogFileFormat};
|
||||
use super::log_file::LogFileReader;
|
||||
|
||||
/// A reusable reader over [`LogItemBatch`]s in a log file.
|
||||
pub(super) struct LogItemBatchFileReader<F: FileSystem> {
|
||||
file_id: Option<FileId>,
|
||||
reader: Option<LogFileReader<F>>,
|
||||
format: Option<LogFileFormat>,
|
||||
pub(crate) reader: Option<LogFileReader<F>>,
|
||||
size: usize,
|
||||
|
||||
buffer: Vec<u8>,
|
||||
|
@ -29,6 +31,7 @@ impl<F: FileSystem> LogItemBatchFileReader<F> {
|
|||
pub fn new(read_block_size: usize) -> Self {
|
||||
Self {
|
||||
file_id: None,
|
||||
format: None,
|
||||
reader: None,
|
||||
size: 0,
|
||||
|
||||
|
@ -41,22 +44,22 @@ impl<F: FileSystem> LogItemBatchFileReader<F> {
|
|||
}
|
||||
|
||||
/// Opens a file that can be accessed through the given reader.
|
||||
pub fn open(&mut self, file_id: FileId, reader: LogFileReader<F>) -> Result<()> {
|
||||
pub fn open(&mut self, file_id: FileId, mut reader: LogFileReader<F>) -> Result<LogFileFormat> {
|
||||
let format = reader.parse_format()?;
|
||||
self.valid_offset = LogFileFormat::encoded_len(format.version);
|
||||
self.file_id = Some(file_id);
|
||||
self.format = Some(format);
|
||||
self.size = reader.file_size()?;
|
||||
self.reader = Some(reader);
|
||||
self.buffer.clear();
|
||||
self.buffer_offset = 0;
|
||||
self.valid_offset = 0;
|
||||
let mut header = self.peek(0, LogFileHeader::len(), LOG_BATCH_HEADER_LEN)?;
|
||||
LogFileHeader::decode(&mut header)?;
|
||||
self.valid_offset = LogFileHeader::len();
|
||||
Ok(())
|
||||
Ok(format)
|
||||
}
|
||||
|
||||
/// Closes any ongoing file access.
|
||||
pub fn reset(&mut self) {
|
||||
self.file_id = None;
|
||||
self.format = None;
|
||||
self.reader = None;
|
||||
self.size = 0;
|
||||
self.buffer.clear();
|
||||
|
@ -67,26 +70,53 @@ impl<F: FileSystem> LogItemBatchFileReader<F> {
|
|||
/// Returns the next [`LogItemBatch`] in current opened file. Returns
|
||||
/// `None` if there is no more data or no opened file.
|
||||
pub fn next(&mut self) -> Result<Option<LogItemBatch>> {
|
||||
if self.valid_offset < self.size {
|
||||
// TODO: [Fulfilled in writing progress when DIO is open.]
|
||||
// We should also consider that there might exists broken blocks when DIO
|
||||
// is open, and the following reading strategy should tolerate reading broken
|
||||
// blocks until it finds an accessible header of `LogBatch`.
|
||||
while self.valid_offset < self.size {
|
||||
let format = self.format.unwrap();
|
||||
if self.valid_offset < LOG_BATCH_HEADER_LEN {
|
||||
return Err(Error::Corruption(
|
||||
"attempt to read file with broken header".to_owned(),
|
||||
));
|
||||
}
|
||||
let (footer_offset, compression_type, len) = LogBatch::decode_header(&mut self.peek(
|
||||
let r = LogBatch::decode_header(&mut self.peek(
|
||||
self.valid_offset,
|
||||
LOG_BATCH_HEADER_LEN,
|
||||
0,
|
||||
)?)?;
|
||||
)?);
|
||||
if_chain::if_chain! {
|
||||
if r.is_err();
|
||||
if format.alignment > 0;
|
||||
let aligned_next_offset = round_up(self.valid_offset, format.alignment as usize);
|
||||
if self.valid_offset != aligned_next_offset;
|
||||
if is_zero_padded(self.peek(self.valid_offset, aligned_next_offset - self.valid_offset, 0)?);
|
||||
then {
|
||||
// In DataLayout::Alignment mode, tail data in the previous block
|
||||
// may be aligned with paddings, that is '0'. So, we need to
|
||||
// skip these redundant content and get the next valid header
|
||||
// of `LogBatch`.
|
||||
self.valid_offset = aligned_next_offset;
|
||||
continue;
|
||||
}
|
||||
// If we continued with aligned offset and get a parsed err,
|
||||
// it means that the header is broken or the padding is filled
|
||||
// with non-zero bytes, and the err will be returned.
|
||||
}
|
||||
let (footer_offset, compression_type, len) = r?;
|
||||
if self.valid_offset + len > self.size {
|
||||
return Err(Error::Corruption("log batch header broken".to_owned()));
|
||||
}
|
||||
|
||||
let handle = FileBlockHandle {
|
||||
id: self.file_id.unwrap(),
|
||||
offset: (self.valid_offset + LOG_BATCH_HEADER_LEN) as u64,
|
||||
len: footer_offset - LOG_BATCH_HEADER_LEN,
|
||||
};
|
||||
let context = LogFileContext {
|
||||
id: self.file_id.unwrap(),
|
||||
version: format.version,
|
||||
};
|
||||
let item_batch = LogItemBatch::decode(
|
||||
&mut self.peek(
|
||||
self.valid_offset + footer_offset,
|
||||
|
@ -95,6 +125,7 @@ impl<F: FileSystem> LogItemBatchFileReader<F> {
|
|||
)?,
|
||||
handle,
|
||||
compression_type,
|
||||
&context,
|
||||
)?;
|
||||
self.valid_offset += len;
|
||||
return Ok(Some(item_batch));
|
||||
|
|
|
@ -13,13 +13,13 @@ use crate::file_pipe_log::{FileNameExt, ReplayMachine};
|
|||
use crate::log_batch::{
|
||||
Command, EntryIndexes, KeyValue, LogBatch, LogItem, LogItemBatch, LogItemContent, OpType,
|
||||
};
|
||||
use crate::pipe_log::{FileId, LogQueue};
|
||||
use crate::pipe_log::{FileId, LogFileContext, LogQueue};
|
||||
use crate::util::Factory;
|
||||
use crate::{Error, Result};
|
||||
|
||||
/// `FilterResult` determines how to alter the existing log items in
|
||||
/// `RhaiFilterMachine`.
|
||||
#[derive(PartialEq)]
|
||||
#[derive(PartialEq, Eq)]
|
||||
enum FilterResult {
|
||||
/// Apply in the usual way.
|
||||
Default,
|
||||
|
@ -255,19 +255,18 @@ impl RhaiFilterMachine {
|
|||
// Backup file and set up a guard to recover on exit.
|
||||
let target_path = f.file_id.build_file_path(path);
|
||||
let bak_path = target_path.with_extension("bak");
|
||||
std::fs::rename(&target_path, &bak_path)?;
|
||||
system.rename(&target_path, &bak_path)?;
|
||||
guards.push((
|
||||
bak_path.clone(),
|
||||
guard(f.file_id, |f| {
|
||||
let original = f.build_file_path(path);
|
||||
let bak = original.with_extension("bak");
|
||||
if bak.exists() {
|
||||
std::fs::rename(bak, original).unwrap_or_else(|e| {
|
||||
system.rename(&bak, &original).unwrap_or_else(|e| {
|
||||
panic!(
|
||||
"Failed to recover original log file {} ({}),
|
||||
"Failed to recover original log file {} ({e}),
|
||||
you should manually replace it with {}.bak.",
|
||||
f.build_file_name(),
|
||||
e,
|
||||
f.build_file_name(),
|
||||
)
|
||||
});
|
||||
|
@ -275,7 +274,10 @@ impl RhaiFilterMachine {
|
|||
}),
|
||||
));
|
||||
let mut reader = build_file_reader(system, &bak_path)?;
|
||||
let mut writer = build_file_writer(system, &target_path, true /* create */)?;
|
||||
let format = reader.parse_format()?;
|
||||
let mut writer =
|
||||
build_file_writer(system, &target_path, format, true /* create */)?;
|
||||
let log_file_context = LogFileContext::new(f.file_id, format.version);
|
||||
// Write out new log file.
|
||||
for item in f.items.into_iter() {
|
||||
match item.content {
|
||||
|
@ -305,13 +307,16 @@ impl RhaiFilterMachine {
|
|||
value,
|
||||
..
|
||||
}) => match op_type {
|
||||
OpType::Put => log_batch.put(item.raft_group_id, key, value.unwrap()),
|
||||
OpType::Put => {
|
||||
log_batch.put(item.raft_group_id, key, value.unwrap())?
|
||||
}
|
||||
OpType::Del => log_batch.delete(item.raft_group_id, key),
|
||||
},
|
||||
}
|
||||
// Batch 64KB.
|
||||
if log_batch.approximate_size() >= 64 * 1024 {
|
||||
log_batch.finish_populate(0 /* compression_threshold */)?;
|
||||
log_batch.finish_populate(0 /* compression_threshold */, None)?;
|
||||
log_batch.prepare_write(&log_file_context)?;
|
||||
writer.write(
|
||||
log_batch.encoded_bytes(),
|
||||
usize::MAX, /* target_size_hint */
|
||||
|
@ -320,7 +325,8 @@ impl RhaiFilterMachine {
|
|||
}
|
||||
}
|
||||
if !log_batch.is_empty() {
|
||||
log_batch.finish_populate(0 /* compression_threshold */)?;
|
||||
log_batch.finish_populate(0 /* compression_threshold */, None)?;
|
||||
log_batch.prepare_write(&log_file_context)?;
|
||||
writer.write(
|
||||
log_batch.encoded_bytes(),
|
||||
usize::MAX, /* target_size_hint */
|
||||
|
@ -332,7 +338,7 @@ impl RhaiFilterMachine {
|
|||
}
|
||||
// Delete backup file and defuse the guard.
|
||||
for (bak, guard) in guards.into_iter() {
|
||||
let _ = std::fs::remove_file(&bak);
|
||||
let _ = std::fs::remove_file(bak);
|
||||
let _ = ScopeGuard::into_inner(guard);
|
||||
}
|
||||
Ok(())
|
||||
|
@ -409,7 +415,7 @@ pub struct RhaiFilterMachineFactory {
|
|||
impl RhaiFilterMachineFactory {
|
||||
pub fn from_script(script: String) -> Self {
|
||||
let engine = Engine::new();
|
||||
let ast = engine.compile(&script).unwrap();
|
||||
let ast = engine.compile(script).unwrap();
|
||||
engine.run_ast_with_scope(&mut Scope::new(), &ast).unwrap();
|
||||
Self {
|
||||
engine: Arc::new(engine),
|
||||
|
|
|
@ -0,0 +1,180 @@
|
|||
// Copyright (c) 2023-present, PingCAP, Inc. Licensed under Apache-2.0.
|
||||
|
||||
use std::fs::{copy, create_dir_all};
|
||||
use std::path::Path;
|
||||
use std::sync::Arc;
|
||||
|
||||
#[cfg(not(windows))]
|
||||
use std::os::unix::fs::symlink;
|
||||
#[cfg(windows)]
|
||||
use std::os::windows::fs::symlink_file as symlink;
|
||||
|
||||
use crate::config::{Config, RecoveryMode};
|
||||
use crate::env::FileSystem;
|
||||
use crate::file_pipe_log::{FileNameExt, FilePipeLog, FilePipeLogBuilder};
|
||||
use crate::pipe_log::{FileId, LogQueue};
|
||||
use crate::Engine;
|
||||
|
||||
/// Returned by `Engine::fork`.
|
||||
#[derive(Default)]
|
||||
pub struct CopyDetails {
|
||||
/// Paths of copied log files.
|
||||
pub copied: Vec<String>,
|
||||
/// Paths of symlinked log files.
|
||||
pub symlinked: Vec<String>,
|
||||
}
|
||||
|
||||
impl<F: FileSystem> Engine<F, FilePipeLog<F>> {
|
||||
/// Make a copy from `source` to `target`. `source` should exists but
|
||||
/// `target` shouldn't. And `source` shouldn't be opened, otherwise
|
||||
/// data corruption can happen.
|
||||
///
|
||||
/// *symlink* will be used if possbile, otherwise *copy* will be used
|
||||
/// instead. Generally all inactive log files will be symlinked, but the
|
||||
/// last active one will be copied.
|
||||
///
|
||||
/// After the copy is made both of 2 engines can be started and run at the
|
||||
/// same time.
|
||||
///
|
||||
/// It reports errors if the source instance
|
||||
/// * is specified with `enable_log_recycle = true`. `source` and `target`
|
||||
/// can share log files, so log file reusing can cause data corruption.
|
||||
/// * is specified with `recovery_mode = TolerateAnyCorruption`, in which
|
||||
/// case *symlink* can't be use. Users should consider to copy the
|
||||
/// instance directly.
|
||||
pub fn fork<T: AsRef<Path>>(
|
||||
source: &Config,
|
||||
fs: Arc<F>,
|
||||
target: T,
|
||||
) -> Result<CopyDetails, String> {
|
||||
minimum_copy(source, fs, target)
|
||||
}
|
||||
}
|
||||
|
||||
fn minimum_copy<F, P>(cfg: &Config, fs: Arc<F>, target: P) -> Result<CopyDetails, String>
|
||||
where
|
||||
F: FileSystem,
|
||||
P: AsRef<Path>,
|
||||
{
|
||||
if cfg.enable_log_recycle {
|
||||
return Err("enable_log_recycle should be false".to_owned());
|
||||
}
|
||||
if cfg.recovery_mode == RecoveryMode::TolerateAnyCorruption {
|
||||
return Err("recovery_mode shouldn't be TolerateAnyCorruption".to_owned());
|
||||
}
|
||||
|
||||
let mut cfg = cfg.clone();
|
||||
cfg.sanitize()
|
||||
.map_err(|e| format!("sanitize config: {e}"))?;
|
||||
|
||||
create_dir_all(&target)
|
||||
.map_err(|e| format!("create_dir_all({}): {e}", target.as_ref().display()))?;
|
||||
|
||||
let mut builder = FilePipeLogBuilder::new(cfg.clone(), fs, vec![]);
|
||||
builder
|
||||
.scan_and_sort(false)
|
||||
.map_err(|e| format!("scan files: {e}"))?;
|
||||
|
||||
// Iterate all log files and rewrite files.
|
||||
let mut details = CopyDetails::default();
|
||||
for (queue, files) in [
|
||||
(LogQueue::Append, &builder.append_file_names),
|
||||
(LogQueue::Rewrite, &builder.rewrite_file_names),
|
||||
] {
|
||||
let count = files.len();
|
||||
for (i, f) in files.iter().enumerate() {
|
||||
let src: &Path = f.path.as_ref();
|
||||
let dst = FileId::new(queue, f.seq).build_file_path(&target);
|
||||
if i < count - 1 {
|
||||
symlink(src, &dst)
|
||||
.map_err(|e| format!("symlink({}, {}): {e}", src.display(), dst.display()))?;
|
||||
let path = dst.canonicalize().unwrap().to_str().unwrap().to_owned();
|
||||
details.symlinked.push(path);
|
||||
} else {
|
||||
copy(src, &dst)
|
||||
.map(|_| ())
|
||||
.map_err(|e| format!("copy({}, {}): {e}", src.display(), dst.display()))?;
|
||||
let path = dst.canonicalize().unwrap().to_str().unwrap().to_owned();
|
||||
details.copied.push(path);
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
Ok(details)
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::engine::tests::RaftLogEngine;
|
||||
use crate::env::DefaultFileSystem;
|
||||
use crate::{LogBatch, ReadableSize};
|
||||
use std::path::PathBuf;
|
||||
|
||||
#[test]
|
||||
fn test_fork() {
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_engine_fork")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
|
||||
let mut source = PathBuf::from(dir.as_ref());
|
||||
source.push("source");
|
||||
let mut cfg = Config {
|
||||
dir: source.to_str().unwrap().to_owned(),
|
||||
target_file_size: ReadableSize::kb(1),
|
||||
enable_log_recycle: false,
|
||||
..Default::default()
|
||||
};
|
||||
let engine = RaftLogEngine::open(cfg.clone()).unwrap();
|
||||
|
||||
let mut log_batch = LogBatch::default();
|
||||
log_batch.put(1, vec![b'1'; 16], vec![b'v'; 1024]).unwrap();
|
||||
engine.write(&mut log_batch, false).unwrap();
|
||||
engine.purge_manager().must_rewrite_append_queue(None, None);
|
||||
|
||||
let mut log_batch = LogBatch::default();
|
||||
log_batch.put(1, vec![b'2'; 16], vec![b'v'; 1024]).unwrap();
|
||||
engine.write(&mut log_batch, false).unwrap();
|
||||
engine.purge_manager().must_rewrite_append_queue(None, None);
|
||||
|
||||
let mut log_batch = LogBatch::default();
|
||||
log_batch.put(1, vec![b'3'; 16], vec![b'v'; 1024]).unwrap();
|
||||
engine.write(&mut log_batch, false).unwrap();
|
||||
|
||||
let mut log_batch = LogBatch::default();
|
||||
log_batch.put(1, vec![b'4'; 16], vec![b'v'; 1024]).unwrap();
|
||||
engine.write(&mut log_batch, false).unwrap();
|
||||
|
||||
let mut target = PathBuf::from(dir.as_ref());
|
||||
target.push("target");
|
||||
Engine::<_, _>::fork(&cfg, Arc::new(DefaultFileSystem), &target).unwrap();
|
||||
cfg.dir = target.to_str().unwrap().to_owned();
|
||||
let engine1 = RaftLogEngine::open(cfg.clone()).unwrap();
|
||||
|
||||
assert!(engine1.get(1, vec![b'1'; 16].as_ref()).is_some());
|
||||
assert!(engine1.get(1, vec![b'2'; 16].as_ref()).is_some());
|
||||
assert!(engine1.get(1, vec![b'3'; 16].as_ref()).is_some());
|
||||
assert!(engine1.get(1, vec![b'4'; 16].as_ref()).is_some());
|
||||
|
||||
let mut log_batch = LogBatch::default();
|
||||
log_batch.put(1, vec![b'5'; 16], vec![b'v'; 1024]).unwrap();
|
||||
engine.write(&mut log_batch, false).unwrap();
|
||||
|
||||
let mut log_batch = LogBatch::default();
|
||||
log_batch.put(1, vec![b'6'; 16], vec![b'v'; 1024]).unwrap();
|
||||
engine1.write(&mut log_batch, false).unwrap();
|
||||
|
||||
assert!(engine.get(1, vec![b'5'; 16].as_ref()).is_some());
|
||||
assert!(engine1.get(1, vec![b'6'; 16].as_ref()).is_some());
|
||||
|
||||
let mut target = PathBuf::from(dir.as_ref());
|
||||
target.push("target-1");
|
||||
let mut cfg1 = cfg.clone();
|
||||
cfg1.enable_log_recycle = true;
|
||||
assert!(Engine::<_, _>::fork(&cfg1, Arc::new(DefaultFileSystem), &target).is_err());
|
||||
let mut cfg1 = cfg;
|
||||
cfg1.recovery_mode = RecoveryMode::TolerateAnyCorruption;
|
||||
assert!(Engine::<_, _>::fork(&cfg1, Arc::new(DefaultFileSystem), &target).is_err());
|
||||
}
|
||||
}
|
72
src/lib.rs
72
src/lib.rs
|
@ -16,12 +16,13 @@
|
|||
#![cfg_attr(feature = "nightly", feature(test))]
|
||||
#![cfg_attr(feature = "swap", feature(allocator_api))]
|
||||
#![cfg_attr(feature = "swap", feature(slice_ptr_get))]
|
||||
// Though the new nightly rust stablized this feature, keep it anyway
|
||||
// because some other project (like TiKV) is still using the old.
|
||||
#![cfg_attr(feature = "swap", feature(nonnull_slice_from_raw_parts))]
|
||||
#![cfg_attr(feature = "swap", feature(slice_ptr_len))]
|
||||
#![cfg_attr(feature = "swap", feature(alloc_layout_extra))]
|
||||
// For testing only.
|
||||
#![cfg_attr(feature = "swap", feature(alloc_error_hook))]
|
||||
#![cfg_attr(feature = "swap", feature(cfg_sanitize))]
|
||||
#![cfg_attr(all(test, feature = "swap"), feature(alloc_error_hook))]
|
||||
#![cfg_attr(all(test, feature = "swap"), feature(cfg_sanitize))]
|
||||
|
||||
#[macro_use]
|
||||
extern crate lazy_static;
|
||||
|
@ -49,6 +50,7 @@ mod event_listener;
|
|||
mod file_pipe_log;
|
||||
#[cfg(feature = "scripting")]
|
||||
mod filter;
|
||||
mod fork;
|
||||
mod log_batch;
|
||||
mod memtable;
|
||||
mod metrics;
|
||||
|
@ -67,6 +69,8 @@ pub use config::{Config, RecoveryMode};
|
|||
pub use engine::Engine;
|
||||
pub use errors::{Error, Result};
|
||||
pub use log_batch::{Command, LogBatch, MessageExt};
|
||||
pub use metrics::{get_perf_context, set_perf_context, take_perf_context, PerfContext};
|
||||
pub use pipe_log::Version;
|
||||
pub use util::ReadableSize;
|
||||
|
||||
#[cfg(feature = "internals")]
|
||||
|
@ -77,6 +81,7 @@ pub mod internals {
|
|||
pub use crate::file_pipe_log::*;
|
||||
pub use crate::memtable::*;
|
||||
pub use crate::pipe_log::*;
|
||||
pub use crate::purge::*;
|
||||
#[cfg(feature = "swap")]
|
||||
pub use crate::swappy_allocator::*;
|
||||
pub use crate::write_barrier::*;
|
||||
|
@ -159,6 +164,59 @@ impl GlobalStats {
|
|||
}
|
||||
}
|
||||
|
||||
pub(crate) const INTERNAL_KEY_PREFIX: &[u8] = b"__";
|
||||
|
||||
#[inline]
|
||||
#[cfg(test)]
|
||||
pub(crate) fn make_internal_key(k: &[u8]) -> Vec<u8> {
|
||||
assert!(!k.is_empty());
|
||||
let mut v = INTERNAL_KEY_PREFIX.to_vec();
|
||||
v.extend_from_slice(k);
|
||||
v
|
||||
}
|
||||
|
||||
#[cfg(not(test))]
|
||||
pub(crate) fn make_internal_key(k: &[u8]) -> Vec<u8> {
|
||||
use log_batch::ATOMIC_GROUP_KEY;
|
||||
|
||||
assert!(k == ATOMIC_GROUP_KEY);
|
||||
let mut v = INTERNAL_KEY_PREFIX.to_vec();
|
||||
v.extend_from_slice(k);
|
||||
v
|
||||
}
|
||||
|
||||
/// We ensure internal keys are not visible to the user by:
|
||||
/// (1) Writing internal keys will be rejected by `LogBatch::put`.
|
||||
/// (2) Internal keys are filtered out during apply and replay of both queues.
|
||||
/// This also makes sure future internal keys under the prefix won't become
|
||||
/// visible after downgrading.
|
||||
#[inline]
|
||||
#[cfg(test)]
|
||||
pub(crate) fn is_internal_key(s: &[u8], ext: Option<&[u8]>) -> bool {
|
||||
if let Some(ext) = ext {
|
||||
s.len() == INTERNAL_KEY_PREFIX.len() + ext.len()
|
||||
&& s[..INTERNAL_KEY_PREFIX.len()] == *INTERNAL_KEY_PREFIX
|
||||
&& s[INTERNAL_KEY_PREFIX.len()..] == *ext
|
||||
} else {
|
||||
s.len() > INTERNAL_KEY_PREFIX.len()
|
||||
&& s[..INTERNAL_KEY_PREFIX.len()] == *INTERNAL_KEY_PREFIX
|
||||
}
|
||||
}
|
||||
|
||||
#[inline]
|
||||
#[cfg(not(test))]
|
||||
pub(crate) fn is_internal_key(s: &[u8], ext: Option<&[u8]>) -> bool {
|
||||
use log_batch::ATOMIC_GROUP_KEY;
|
||||
|
||||
if let Some(ext) = ext {
|
||||
s.len() == INTERNAL_KEY_PREFIX.len() + ext.len()
|
||||
&& s[..INTERNAL_KEY_PREFIX.len()] == *INTERNAL_KEY_PREFIX
|
||||
&& s[INTERNAL_KEY_PREFIX.len()..] == *ext
|
||||
} else {
|
||||
is_internal_key(s, Some(ATOMIC_GROUP_KEY))
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use crate::log_batch::MessageExt;
|
||||
|
@ -176,4 +234,12 @@ mod tests {
|
|||
e.index
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_internal_key() {
|
||||
let key = crate::make_internal_key(&[0]);
|
||||
assert!(crate::is_internal_key(&key, None));
|
||||
assert!(crate::is_internal_key(&key, Some(&[0])));
|
||||
assert!(!crate::is_internal_key(&key, Some(&[1])));
|
||||
}
|
||||
}
|
||||
|
|
622
src/log_batch.rs
622
src/log_batch.rs
File diff suppressed because it is too large
Load Diff
482
src/memtable.rs
482
src/memtable.rs
|
@ -3,17 +3,19 @@
|
|||
use std::borrow::BorrowMut;
|
||||
use std::collections::{BTreeMap, HashSet, VecDeque};
|
||||
use std::marker::PhantomData;
|
||||
use std::ops::Bound;
|
||||
use std::sync::Arc;
|
||||
|
||||
use fail::fail_point;
|
||||
use hashbrown::HashMap;
|
||||
use log::{error, warn};
|
||||
use parking_lot::{Mutex, RwLock};
|
||||
|
||||
use crate::config::Config;
|
||||
use crate::file_pipe_log::ReplayMachine;
|
||||
use crate::log_batch::{
|
||||
Command, CompressionType, KeyValue, LogBatch, LogItemBatch, LogItemContent, LogItemDrain,
|
||||
OpType,
|
||||
AtomicGroupStatus, Command, CompressionType, KeyValue, LogBatch, LogItem, LogItemBatch,
|
||||
LogItemContent, OpType,
|
||||
};
|
||||
use crate::metrics::MEMORY_USAGE;
|
||||
use crate::pipe_log::{FileBlockHandle, FileId, FileSeq, LogQueue};
|
||||
|
@ -71,7 +73,7 @@ const CAPACITY_INIT: usize = 32 - 1;
|
|||
const MEMTABLE_SLOT_COUNT: usize = 128;
|
||||
|
||||
/// Location of a log entry.
|
||||
#[derive(Debug, Copy, Clone, PartialEq)]
|
||||
#[derive(Debug, Copy, Clone, PartialEq, Eq)]
|
||||
pub struct EntryIndex {
|
||||
/// Logical index.
|
||||
pub index: u64,
|
||||
|
@ -111,7 +113,7 @@ impl EntryIndex {
|
|||
}
|
||||
}
|
||||
|
||||
#[derive(Debug, Copy, Clone, PartialEq)]
|
||||
#[derive(Debug, Copy, Clone, PartialEq, Eq)]
|
||||
struct ThinEntryIndex {
|
||||
entries: Option<FileBlockHandle>,
|
||||
compression_type: CompressionType,
|
||||
|
@ -153,6 +155,16 @@ pub struct MemTable<A: AllocatorTrait> {
|
|||
/// A map of active key value pairs.
|
||||
kvs: BTreeMap<Vec<u8>, (Vec<u8>, FileId)>,
|
||||
|
||||
/// (start_seq, end_seq).
|
||||
/// If there's an active entry stored before end_seq, it possibly belongs to
|
||||
/// an atomic group. In order to not lose this entry, We cannot delete any
|
||||
/// other entries in that group.
|
||||
/// Only applies to Rewrite queue. Each Raft Group has at most one atomic
|
||||
/// group at a time, because we only use atomic group for rewrite-rewrite
|
||||
/// operation, a group always contains all the Rewrite entries in a Raft
|
||||
/// Group.
|
||||
atomic_group: Option<(FileSeq, FileSeq)>,
|
||||
|
||||
/// Shared statistics.
|
||||
global_stats: Arc<GlobalStats>,
|
||||
|
||||
|
@ -181,6 +193,7 @@ impl<A: AllocatorTrait> MemTable<A> {
|
|||
first_index: 0,
|
||||
rewrite_count: 0,
|
||||
kvs: BTreeMap::default(),
|
||||
atomic_group: None,
|
||||
global_stats,
|
||||
_phantom: PhantomData,
|
||||
}
|
||||
|
@ -214,6 +227,11 @@ impl<A: AllocatorTrait> MemTable<A> {
|
|||
self.put(key.clone(), value.clone(), *file_id);
|
||||
}
|
||||
|
||||
if let Some(g) = rhs.atomic_group.take() {
|
||||
assert!(self.atomic_group.is_none_or(|(_, end)| end <= g.0));
|
||||
self.atomic_group = Some(g);
|
||||
}
|
||||
|
||||
let deleted = rhs.global_stats.deleted_rewrite_entries();
|
||||
self.global_stats.add(LogQueue::Rewrite, deleted);
|
||||
self.global_stats.delete(LogQueue::Rewrite, deleted);
|
||||
|
@ -249,6 +267,8 @@ impl<A: AllocatorTrait> MemTable<A> {
|
|||
self.put(key.clone(), value.clone(), *file_id);
|
||||
}
|
||||
|
||||
assert!(rhs.atomic_group.is_none());
|
||||
|
||||
let deleted = rhs.global_stats.deleted_rewrite_entries();
|
||||
self.global_stats.add(LogQueue::Rewrite, deleted);
|
||||
self.global_stats.delete(LogQueue::Rewrite, deleted);
|
||||
|
@ -259,6 +279,37 @@ impl<A: AllocatorTrait> MemTable<A> {
|
|||
self.kvs.get(key).map(|v| v.0.clone())
|
||||
}
|
||||
|
||||
/// Iterates over [start_key, end_key) range and yields all key value pairs
|
||||
/// as bytes.
|
||||
pub fn scan<F>(
|
||||
&self,
|
||||
start_key: Option<&[u8]>,
|
||||
end_key: Option<&[u8]>,
|
||||
reverse: bool,
|
||||
mut f: F,
|
||||
) -> Result<()>
|
||||
where
|
||||
F: FnMut(&[u8], &[u8]) -> bool,
|
||||
{
|
||||
let lower = start_key.map(Bound::Included).unwrap_or(Bound::Unbounded);
|
||||
let upper = end_key.map(Bound::Excluded).unwrap_or(Bound::Unbounded);
|
||||
let iter = self.kvs.range::<[u8], _>((lower, upper));
|
||||
if reverse {
|
||||
for (key, (value, _)) in iter.rev() {
|
||||
if !f(key, value) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
for (key, (value, _)) in iter {
|
||||
if !f(key, value) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Deletes a key value pair.
|
||||
pub fn delete(&mut self, key: &[u8]) {
|
||||
if let Some(value) = self.kvs.remove(key) {
|
||||
|
@ -343,28 +394,25 @@ impl<A: AllocatorTrait> MemTable<A> {
|
|||
}
|
||||
}
|
||||
|
||||
/// Appends some entries from rewrite queue. Assumes this table has no
|
||||
/// append data.
|
||||
/// Appends some entries from append queue. Assumes this table has no
|
||||
/// rewrite data.
|
||||
///
|
||||
/// This method is only used for recovery.
|
||||
pub fn append_rewrite(&mut self, entry_indexes: Vec<EntryIndex>) {
|
||||
pub fn replay_append(&mut self, entry_indexes: Vec<EntryIndex>) {
|
||||
let len = entry_indexes.len();
|
||||
if len > 0 {
|
||||
debug_assert_eq!(self.rewrite_count, self.entry_indexes.len());
|
||||
debug_assert_eq!(self.rewrite_count, 0);
|
||||
self.prepare_append(
|
||||
entry_indexes[0].index,
|
||||
// Rewrite -> Compact Append -> Rewrite.
|
||||
true, /* allow_hole */
|
||||
// Refer to case in `merge_append_table`. They can be adapted
|
||||
// to attack this path via a global rewrite without deleting
|
||||
// obsolete rewrite files.
|
||||
false, /* allow_hole */
|
||||
// Refer to case in `merge_newer_neighbor`.
|
||||
true, /* allow_overwrite */
|
||||
);
|
||||
self.global_stats.add(LogQueue::Rewrite, len);
|
||||
self.global_stats.add(LogQueue::Append, len);
|
||||
for ei in &entry_indexes {
|
||||
debug_assert_eq!(ei.entries.unwrap().id.queue, LogQueue::Append);
|
||||
self.entry_indexes.push_back(ei.into());
|
||||
}
|
||||
self.rewrite_count = self.entry_indexes.len();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -431,15 +479,45 @@ impl<A: AllocatorTrait> MemTable<A> {
|
|||
}
|
||||
|
||||
if gate.is_none() {
|
||||
// We either replaced some old rewrite entries, or some incoming entries are
|
||||
// discarded.
|
||||
self.global_stats
|
||||
.delete(LogQueue::Rewrite, rewrite_indexes.len());
|
||||
// rewrite-rewrite could partially renew rewrite entries due to batch splitting.
|
||||
self.rewrite_count = std::cmp::max(self.rewrite_count, pos + rewrite_len);
|
||||
} else {
|
||||
self.global_stats.delete(LogQueue::Append, rewrite_len);
|
||||
self.global_stats
|
||||
.delete(LogQueue::Rewrite, rewrite_indexes.len() - rewrite_len);
|
||||
// rewrite-append always push forward.
|
||||
assert!(pos + rewrite_len >= self.rewrite_count);
|
||||
self.rewrite_count = pos + rewrite_len;
|
||||
}
|
||||
}
|
||||
|
||||
self.rewrite_count = pos + rewrite_len;
|
||||
/// Appends some entries from rewrite queue. Assumes this table has no
|
||||
/// append data.
|
||||
///
|
||||
/// This method is only used for recovery.
|
||||
pub fn replay_rewrite(&mut self, entry_indexes: Vec<EntryIndex>) {
|
||||
let len = entry_indexes.len();
|
||||
if len > 0 {
|
||||
debug_assert_eq!(self.rewrite_count, self.entry_indexes.len());
|
||||
self.prepare_append(
|
||||
entry_indexes[0].index,
|
||||
// Rewrite -> Compact Append -> Rewrite.
|
||||
true, /* allow_hole */
|
||||
// Refer to case in `merge_append_table`. They can be adapted
|
||||
// to attack this path via a global rewrite without deleting
|
||||
// obsolete rewrite files.
|
||||
true, /* allow_overwrite */
|
||||
);
|
||||
self.global_stats.add(LogQueue::Rewrite, len);
|
||||
for ei in &entry_indexes {
|
||||
self.entry_indexes.push_back(ei.into());
|
||||
}
|
||||
self.rewrite_count = self.entry_indexes.len();
|
||||
}
|
||||
}
|
||||
|
||||
/// Removes all entries with index smaller than `index`. Returns the number
|
||||
|
@ -466,6 +544,11 @@ impl<A: AllocatorTrait> MemTable<A> {
|
|||
count as u64
|
||||
}
|
||||
|
||||
pub fn apply_rewrite_atomic_group(&mut self, start: FileSeq, end: FileSeq) {
|
||||
assert!(self.atomic_group.is_none_or(|(_, b)| b <= start));
|
||||
self.atomic_group = Some((start, end));
|
||||
}
|
||||
|
||||
/// Removes all entry indexes with index greater than or equal to `index`.
|
||||
/// Assumes `index` <= `last`.
|
||||
///
|
||||
|
@ -659,24 +742,28 @@ impl<A: AllocatorTrait> MemTable<A> {
|
|||
Some(v.1.seq)
|
||||
}
|
||||
});
|
||||
match (ents_min, kvs_min) {
|
||||
(Some(ents_min), Some(kvs_min)) => Some(std::cmp::min(kvs_min, ents_min)),
|
||||
(Some(ents_min), None) => Some(ents_min),
|
||||
(None, Some(kvs_min)) => Some(kvs_min),
|
||||
(None, None) => None,
|
||||
let res = match (ents_min, kvs_min) {
|
||||
(Some(ents_min), Some(kvs_min)) => std::cmp::min(kvs_min, ents_min),
|
||||
(Some(ents_min), None) => ents_min,
|
||||
(None, Some(kvs_min)) => kvs_min,
|
||||
(None, None) => return None,
|
||||
};
|
||||
if queue == LogQueue::Rewrite {
|
||||
if let Some((start, end)) = self.atomic_group {
|
||||
if res <= end {
|
||||
return Some(std::cmp::min(start, res));
|
||||
}
|
||||
}
|
||||
}
|
||||
Some(res)
|
||||
}
|
||||
|
||||
/// Returns the number of entries smaller than or equal to `gate`.
|
||||
pub fn entries_count_before(&self, mut gate: FileId) -> usize {
|
||||
gate.seq += 1;
|
||||
let idx = self
|
||||
.entry_indexes
|
||||
.binary_search_by_key(&gate, |ei| ei.entries.unwrap().id);
|
||||
match idx {
|
||||
Ok(idx) => idx,
|
||||
Err(idx) => idx,
|
||||
}
|
||||
#[inline]
|
||||
pub fn has_at_least_some_entries_before(&self, gate: FileId, count: usize) -> bool {
|
||||
debug_assert!(count > 0);
|
||||
self.entry_indexes
|
||||
.get(count - 1)
|
||||
.is_some_and(|ei| ei.entries.unwrap().id.seq <= gate.seq)
|
||||
}
|
||||
|
||||
/// Returns the region ID.
|
||||
|
@ -902,7 +989,7 @@ impl<A: AllocatorTrait> MemTableAccessor<A> {
|
|||
/// [`MemTable`]s.
|
||||
///
|
||||
/// This method is only used for recovery.
|
||||
#[allow(dead_code)]
|
||||
#[cfg(test)]
|
||||
pub fn cleaned_region_ids(&self) -> HashSet<u64> {
|
||||
let mut ids = HashSet::default();
|
||||
let removed_memtables = self.removed_memtables.lock();
|
||||
|
@ -912,6 +999,16 @@ impl<A: AllocatorTrait> MemTableAccessor<A> {
|
|||
ids
|
||||
}
|
||||
|
||||
/// Returns `true` if it does not contains any memtable.
|
||||
pub fn is_empty(&self) -> bool {
|
||||
for i in 0..MEMTABLE_SLOT_COUNT {
|
||||
if !self.slots[i].read().is_empty() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
true
|
||||
}
|
||||
|
||||
/// Merges with a newer neighbor [`MemTableAccessor`].
|
||||
///
|
||||
/// This method is only used for recovery.
|
||||
|
@ -951,8 +1048,11 @@ impl<A: AllocatorTrait> MemTableAccessor<A> {
|
|||
}
|
||||
|
||||
/// Applies changes from log items that have been written to append queue.
|
||||
pub fn apply_append_writes(&self, log_items: LogItemDrain) {
|
||||
pub fn apply_append_writes(&self, log_items: impl Iterator<Item = LogItem>) {
|
||||
for item in log_items {
|
||||
if has_internal_key(&item) {
|
||||
continue;
|
||||
}
|
||||
let raft = item.raft_group_id;
|
||||
let memtable = self.get_or_insert(raft);
|
||||
fail_point!(
|
||||
|
@ -984,14 +1084,52 @@ impl<A: AllocatorTrait> MemTableAccessor<A> {
|
|||
}
|
||||
}
|
||||
|
||||
/// Applies changes from log items that are replayed from a append queue.
|
||||
/// Assumes it haven't applied any rewrite data.
|
||||
///
|
||||
/// This method is only used for recovery.
|
||||
pub fn replay_append_writes(&self, log_items: impl Iterator<Item = LogItem>) {
|
||||
for item in log_items {
|
||||
if has_internal_key(&item) {
|
||||
continue;
|
||||
}
|
||||
let raft = item.raft_group_id;
|
||||
let memtable = self.get_or_insert(raft);
|
||||
match item.content {
|
||||
LogItemContent::EntryIndexes(entries_to_add) => {
|
||||
memtable.write().replay_append(entries_to_add.0);
|
||||
}
|
||||
LogItemContent::Command(Command::Clean) => {
|
||||
self.remove(raft, true /* record_tombstone */);
|
||||
}
|
||||
LogItemContent::Command(Command::Compact { index }) => {
|
||||
memtable.write().compact_to(index);
|
||||
}
|
||||
LogItemContent::Kv(kv) => match kv.op_type {
|
||||
OpType::Put => {
|
||||
let value = kv.value.unwrap();
|
||||
memtable.write().put(kv.key, value, kv.file_id.unwrap());
|
||||
}
|
||||
OpType::Del => {
|
||||
let key = kv.key;
|
||||
memtable.write().delete(key.as_slice());
|
||||
}
|
||||
},
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Applies changes from log items that have been written to rewrite queue.
|
||||
pub fn apply_rewrite_writes(
|
||||
&self,
|
||||
log_items: LogItemDrain,
|
||||
log_items: impl Iterator<Item = LogItem>,
|
||||
watermark: Option<FileSeq>,
|
||||
new_file: FileSeq,
|
||||
) {
|
||||
for item in log_items {
|
||||
if has_internal_key(&item) {
|
||||
continue;
|
||||
}
|
||||
let raft = item.raft_group_id;
|
||||
let memtable = self.get_or_insert(raft);
|
||||
match item.content {
|
||||
|
@ -1015,15 +1153,19 @@ impl<A: AllocatorTrait> MemTableAccessor<A> {
|
|||
/// Assumes it haven't applied any append data.
|
||||
///
|
||||
/// This method is only used for recovery.
|
||||
pub fn apply_replayed_rewrite_writes(&self, log_items: LogItemDrain) {
|
||||
pub fn replay_rewrite_writes(&self, log_items: impl Iterator<Item = LogItem>) {
|
||||
for item in log_items {
|
||||
if has_internal_key(&item) {
|
||||
continue;
|
||||
}
|
||||
let raft = item.raft_group_id;
|
||||
let memtable = self.get_or_insert(raft);
|
||||
match item.content {
|
||||
LogItemContent::EntryIndexes(entries_to_add) => {
|
||||
memtable.write().append_rewrite(entries_to_add.0);
|
||||
memtable.write().replay_rewrite(entries_to_add.0);
|
||||
}
|
||||
LogItemContent::Command(Command::Clean) => {
|
||||
// Only append tombstone needs to be recorded.
|
||||
self.remove(raft, false /* record_tombstone */);
|
||||
}
|
||||
LogItemContent::Command(Command::Compact { index }) => {
|
||||
|
@ -1043,6 +1185,11 @@ impl<A: AllocatorTrait> MemTableAccessor<A> {
|
|||
}
|
||||
}
|
||||
|
||||
pub fn apply_rewrite_atomic_group(&self, raft: u64, start: FileSeq, end: FileSeq) {
|
||||
let memtable = self.get_or_insert(raft);
|
||||
memtable.write().apply_rewrite_atomic_group(start, end);
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn slot_index(id: u64) -> usize {
|
||||
debug_assert!(MEMTABLE_SLOT_COUNT.is_power_of_two());
|
||||
|
@ -1050,10 +1197,28 @@ impl<A: AllocatorTrait> MemTableAccessor<A> {
|
|||
}
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn has_internal_key(item: &LogItem) -> bool {
|
||||
matches!(&item.content, LogItemContent::Kv(KeyValue { key, .. }) if crate::is_internal_key(key, None))
|
||||
}
|
||||
|
||||
struct PendingAtomicGroup {
|
||||
status: AtomicGroupStatus,
|
||||
items: Vec<LogItem>,
|
||||
tombstone_items: Vec<LogItem>,
|
||||
start: FileSeq,
|
||||
end: FileSeq,
|
||||
}
|
||||
|
||||
pub struct MemTableRecoverContext<A: AllocatorTrait> {
|
||||
stats: Arc<GlobalStats>,
|
||||
log_batch: LogItemBatch,
|
||||
// Tombstones that needs to be transmitted to other context.
|
||||
tombstone_items: Vec<LogItem>,
|
||||
memtables: MemTableAccessor<A>,
|
||||
|
||||
// All atomic groups that are not yet completed.
|
||||
// Each id maps to a list of groups. Each list contains at least one, at most two groups.
|
||||
pending_atomic_groups: HashMap<u64, Vec<PendingAtomicGroup>>,
|
||||
}
|
||||
|
||||
impl MemTableRecoverContext<VacantAllocator> {
|
||||
|
@ -1061,8 +1226,9 @@ impl MemTableRecoverContext<VacantAllocator> {
|
|||
let stats = Arc::new(GlobalStats::default());
|
||||
Self {
|
||||
stats: stats.clone(),
|
||||
log_batch: LogItemBatch::default(),
|
||||
tombstone_items: Vec::new(),
|
||||
memtables: MemTableAccessor::new(stats),
|
||||
pending_atomic_groups: HashMap::new(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -1072,8 +1238,9 @@ impl<A: AllocatorTrait> MemTableRecoverContext<A> {
|
|||
let stats = Arc::new(GlobalStats::default());
|
||||
Self {
|
||||
stats: stats.clone(),
|
||||
log_batch: LogItemBatch::default(),
|
||||
tombstone_items: Vec::new(),
|
||||
memtables: MemTableAccessor::new_with_allocator(stats, allocator),
|
||||
pending_atomic_groups: HashMap::new(),
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1083,9 +1250,91 @@ impl<A: AllocatorTrait> MemTableRecoverContext<A> {
|
|||
|
||||
pub fn merge_append_context(&self, append: MemTableRecoverContext<A>) {
|
||||
self.memtables
|
||||
.apply_append_writes(append.log_batch.clone().drain());
|
||||
.apply_append_writes(append.tombstone_items.into_iter());
|
||||
self.memtables.merge_append_table(append.memtables);
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn is_tombstone(item: &LogItem) -> bool {
|
||||
match &item.content {
|
||||
LogItemContent::Command(Command::Clean)
|
||||
| LogItemContent::Command(Command::Compact { .. }) => true,
|
||||
LogItemContent::Kv(KeyValue { op_type, .. }) if *op_type == OpType::Del => true,
|
||||
_ => false,
|
||||
}
|
||||
}
|
||||
|
||||
fn accept_new_group(&mut self, queue: LogQueue, id: u64, mut new_group: PendingAtomicGroup) {
|
||||
assert_eq!(queue, LogQueue::Rewrite);
|
||||
if let Some(groups) = self.pending_atomic_groups.get_mut(&id) {
|
||||
let group = groups.last_mut().unwrap();
|
||||
match (group.status, new_group.status) {
|
||||
(AtomicGroupStatus::End, AtomicGroupStatus::Begin) => {
|
||||
groups.push(new_group);
|
||||
}
|
||||
// (begin, begin), (middle, begin)
|
||||
(_, AtomicGroupStatus::Begin) => {
|
||||
warn!(
|
||||
"discard old atomic group, status: {:?}, raft_group_id: {:?}",
|
||||
group.status,
|
||||
group.items.first().map(|item| item.raft_group_id)
|
||||
);
|
||||
*group = new_group;
|
||||
}
|
||||
// (end, middle), (end, end)
|
||||
(AtomicGroupStatus::End, _) => {
|
||||
warn!(
|
||||
"discard new atomic group, status: {:?}, raft_group_id: {:?}",
|
||||
new_group.status,
|
||||
new_group.items.first().map(|item| item.raft_group_id)
|
||||
);
|
||||
}
|
||||
(AtomicGroupStatus::Begin, AtomicGroupStatus::Middle)
|
||||
| (AtomicGroupStatus::Middle, AtomicGroupStatus::Middle) => {
|
||||
group.items.append(&mut new_group.items);
|
||||
group.tombstone_items.append(&mut new_group.tombstone_items);
|
||||
assert!(group.end <= new_group.start);
|
||||
group.end = new_group.end;
|
||||
}
|
||||
(AtomicGroupStatus::Middle, AtomicGroupStatus::End) => {
|
||||
group.items.append(&mut new_group.items);
|
||||
group.tombstone_items.append(&mut new_group.tombstone_items);
|
||||
group.status = new_group.status;
|
||||
assert!(group.end <= new_group.start);
|
||||
group.end = new_group.end;
|
||||
}
|
||||
(AtomicGroupStatus::Begin, AtomicGroupStatus::End) => {
|
||||
let mut group = groups.pop().unwrap();
|
||||
let mut rids = HashSet::with_capacity(1);
|
||||
for item in group
|
||||
.items
|
||||
.iter()
|
||||
.chain(group.tombstone_items.iter())
|
||||
.chain(new_group.items.iter())
|
||||
.chain(new_group.tombstone_items.iter())
|
||||
{
|
||||
rids.insert(item.raft_group_id);
|
||||
}
|
||||
self.tombstone_items.append(&mut group.tombstone_items);
|
||||
self.tombstone_items.append(&mut new_group.tombstone_items);
|
||||
self.memtables
|
||||
.replay_rewrite_writes(group.items.into_iter());
|
||||
self.memtables
|
||||
.replay_rewrite_writes(new_group.items.into_iter());
|
||||
assert!(group.end <= new_group.start);
|
||||
for rid in rids {
|
||||
self.memtables
|
||||
.apply_rewrite_atomic_group(rid, group.start, new_group.end);
|
||||
}
|
||||
}
|
||||
}
|
||||
if groups.is_empty() {
|
||||
self.pending_atomic_groups.remove(&id);
|
||||
}
|
||||
} else {
|
||||
self.pending_atomic_groups.insert(id, vec![new_group]);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Default for MemTableRecoverContext<VacantAllocator> {
|
||||
|
@ -1096,34 +1345,73 @@ impl Default for MemTableRecoverContext<VacantAllocator> {
|
|||
|
||||
impl<A: AllocatorTrait> ReplayMachine for MemTableRecoverContext<A> {
|
||||
fn replay(&mut self, mut item_batch: LogItemBatch, file_id: FileId) -> Result<()> {
|
||||
for item in item_batch.iter() {
|
||||
match &item.content {
|
||||
LogItemContent::Command(Command::Clean)
|
||||
| LogItemContent::Command(Command::Compact { .. }) => {
|
||||
self.log_batch.push((*item).clone());
|
||||
}
|
||||
LogItemContent::Kv(KeyValue { op_type, .. }) if *op_type == OpType::Del => {
|
||||
self.log_batch.push((*item).clone());
|
||||
}
|
||||
_ => {}
|
||||
if file_id.queue == LogQueue::Append {
|
||||
let mut new_tombstones = Vec::new();
|
||||
self.memtables
|
||||
.replay_append_writes(item_batch.drain().filter(|item| {
|
||||
if Self::is_tombstone(item) {
|
||||
new_tombstones.push(item.clone());
|
||||
}
|
||||
true
|
||||
}));
|
||||
self.tombstone_items.append(&mut new_tombstones);
|
||||
} else {
|
||||
let mut new_tombstones = Vec::new();
|
||||
let mut is_group = None;
|
||||
let items = item_batch
|
||||
.drain()
|
||||
.filter(|item| {
|
||||
if let Some(g) = AtomicGroupStatus::parse(item) {
|
||||
if is_group.is_none() {
|
||||
is_group = Some(g);
|
||||
} else {
|
||||
let msg = format!("skipped an atomic group: {g:?}");
|
||||
error!("{msg}");
|
||||
debug_assert!(false, "{}", msg);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
if Self::is_tombstone(item) {
|
||||
new_tombstones.push(item.clone());
|
||||
}
|
||||
true
|
||||
})
|
||||
.collect();
|
||||
if let Some((id, status)) = is_group {
|
||||
self.accept_new_group(
|
||||
file_id.queue,
|
||||
id,
|
||||
PendingAtomicGroup {
|
||||
status,
|
||||
items,
|
||||
tombstone_items: new_tombstones,
|
||||
start: file_id.seq,
|
||||
end: file_id.seq,
|
||||
},
|
||||
);
|
||||
} else {
|
||||
self.tombstone_items.append(&mut new_tombstones);
|
||||
self.memtables.replay_rewrite_writes(items.into_iter());
|
||||
}
|
||||
}
|
||||
match file_id.queue {
|
||||
LogQueue::Append => self.memtables.apply_append_writes(item_batch.drain()),
|
||||
LogQueue::Rewrite => self
|
||||
.memtables
|
||||
.apply_replayed_rewrite_writes(item_batch.drain()),
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn merge(&mut self, mut rhs: Self, queue: LogQueue) -> Result<()> {
|
||||
self.log_batch.merge(&mut rhs.log_batch.clone());
|
||||
self.tombstone_items
|
||||
.append(&mut rhs.tombstone_items.clone());
|
||||
for (id, groups) in rhs.pending_atomic_groups.drain() {
|
||||
for group in groups {
|
||||
self.accept_new_group(queue, id, group);
|
||||
}
|
||||
}
|
||||
match queue {
|
||||
LogQueue::Append => self.memtables.apply_append_writes(rhs.log_batch.drain()),
|
||||
LogQueue::Append => self
|
||||
.memtables
|
||||
.replay_append_writes(rhs.tombstone_items.into_iter()),
|
||||
LogQueue::Rewrite => self
|
||||
.memtables
|
||||
.apply_replayed_rewrite_writes(rhs.log_batch.drain()),
|
||||
.replay_rewrite_writes(rhs.tombstone_items.into_iter()),
|
||||
}
|
||||
self.memtables.merge_newer_neighbor(rhs.memtables);
|
||||
Ok(())
|
||||
|
@ -1589,37 +1877,75 @@ mod tests {
|
|||
|
||||
#[test]
|
||||
fn test_memtable_kv_operations() {
|
||||
fn key(i: u64) -> Vec<u8> {
|
||||
format!("k{i}").as_bytes().to_vec()
|
||||
}
|
||||
fn value(i: u64) -> Vec<u8> {
|
||||
format!("v{i}").as_bytes().to_vec()
|
||||
}
|
||||
|
||||
let region_id = 8;
|
||||
let mut memtable = MemTable::new(region_id, Arc::new(GlobalStats::default()));
|
||||
|
||||
let (k1, v1) = (b"key1", b"value1");
|
||||
let (k5, v5) = (b"key5", b"value5");
|
||||
memtable.put(k1.to_vec(), v1.to_vec(), FileId::new(LogQueue::Append, 1));
|
||||
memtable.put(k5.to_vec(), v5.to_vec(), FileId::new(LogQueue::Append, 5));
|
||||
memtable.put(key(1), value(1), FileId::new(LogQueue::Append, 1));
|
||||
memtable.put(key(5), value(5), FileId::new(LogQueue::Append, 5));
|
||||
assert_eq!(memtable.min_file_seq(LogQueue::Append).unwrap(), 1);
|
||||
assert_eq!(memtable.max_file_seq(LogQueue::Append).unwrap(), 5);
|
||||
assert_eq!(memtable.get(k1.as_ref()), Some(v1.to_vec()));
|
||||
assert_eq!(memtable.get(k5.as_ref()), Some(v5.to_vec()));
|
||||
assert_eq!(memtable.get(&key(1)), Some(value(1)));
|
||||
assert_eq!(memtable.get(&key(5)), Some(value(5)));
|
||||
|
||||
memtable.delete(k5.as_ref());
|
||||
assert_eq!(memtable.get(k5.as_ref()), None);
|
||||
let mut res = Vec::new();
|
||||
memtable
|
||||
.scan(None, None, false, |k, v| {
|
||||
res.push((k.to_vec(), v.to_vec()));
|
||||
false
|
||||
})
|
||||
.unwrap();
|
||||
assert_eq!(res, vec![(key(1), value(1))]);
|
||||
res.clear();
|
||||
memtable
|
||||
.scan(None, None, true, |k, v| {
|
||||
res.push((k.to_vec(), v.to_vec()));
|
||||
false
|
||||
})
|
||||
.unwrap();
|
||||
assert_eq!(res, vec![(key(5), value(5))]);
|
||||
res.clear();
|
||||
memtable
|
||||
.scan(Some(&key(5)), None, false, |key, value| {
|
||||
res.push((key.to_vec(), value.to_vec()));
|
||||
true
|
||||
})
|
||||
.unwrap();
|
||||
assert_eq!(res, vec![(key(5), value(5))]);
|
||||
res.clear();
|
||||
memtable
|
||||
.scan(Some(&key(1)), Some(&key(5)), false, |key, value| {
|
||||
res.push((key.to_vec(), value.to_vec()));
|
||||
true
|
||||
})
|
||||
.unwrap();
|
||||
assert_eq!(res, vec![(key(1), value(1))]);
|
||||
|
||||
memtable.delete(&key(5));
|
||||
assert_eq!(memtable.get(&key(5)), None);
|
||||
assert_eq!(memtable.min_file_seq(LogQueue::Append).unwrap(), 1);
|
||||
assert_eq!(memtable.max_file_seq(LogQueue::Append).unwrap(), 1);
|
||||
|
||||
memtable.put(k1.to_vec(), v1.to_vec(), FileId::new(LogQueue::Rewrite, 2));
|
||||
memtable.put(k5.to_vec(), v5.to_vec(), FileId::new(LogQueue::Rewrite, 3));
|
||||
memtable.put(key(1), value(1), FileId::new(LogQueue::Rewrite, 2));
|
||||
memtable.put(key(5), value(5), FileId::new(LogQueue::Rewrite, 3));
|
||||
assert_eq!(memtable.min_file_seq(LogQueue::Append), None);
|
||||
assert_eq!(memtable.max_file_seq(LogQueue::Append), None);
|
||||
assert_eq!(memtable.min_file_seq(LogQueue::Rewrite).unwrap(), 2);
|
||||
assert_eq!(memtable.max_file_seq(LogQueue::Rewrite).unwrap(), 3);
|
||||
assert_eq!(memtable.global_stats.rewrite_entries(), 2);
|
||||
|
||||
memtable.delete(k1.as_ref());
|
||||
memtable.delete(&key(1));
|
||||
assert_eq!(memtable.min_file_seq(LogQueue::Rewrite).unwrap(), 3);
|
||||
assert_eq!(memtable.max_file_seq(LogQueue::Rewrite).unwrap(), 3);
|
||||
assert_eq!(memtable.global_stats.deleted_rewrite_entries(), 1);
|
||||
|
||||
memtable.put(k5.to_vec(), v5.to_vec(), FileId::new(LogQueue::Append, 7));
|
||||
memtable.put(key(5), value(5), FileId::new(LogQueue::Append, 7));
|
||||
assert_eq!(memtable.min_file_seq(LogQueue::Rewrite), None);
|
||||
assert_eq!(memtable.max_file_seq(LogQueue::Rewrite), None);
|
||||
assert_eq!(memtable.min_file_seq(LogQueue::Append).unwrap(), 7);
|
||||
|
@ -1924,11 +2250,12 @@ mod tests {
|
|||
memtable.compact_to(7);
|
||||
}
|
||||
Some(LogQueue::Rewrite) => {
|
||||
memtable.append_rewrite(generate_entry_indexes(
|
||||
memtable.replay_rewrite(generate_entry_indexes(
|
||||
0,
|
||||
7,
|
||||
FileId::new(LogQueue::Rewrite, 1),
|
||||
));
|
||||
memtable.replay_rewrite(Vec::new());
|
||||
}
|
||||
}
|
||||
memtable
|
||||
|
@ -1966,7 +2293,7 @@ mod tests {
|
|||
memtable.compact_to(10);
|
||||
}
|
||||
Some(LogQueue::Rewrite) => {
|
||||
memtable.append_rewrite(generate_entry_indexes(
|
||||
memtable.replay_rewrite(generate_entry_indexes(
|
||||
0,
|
||||
7,
|
||||
FileId::new(LogQueue::Rewrite, 1),
|
||||
|
@ -2017,7 +2344,7 @@ mod tests {
|
|||
memtable.merge_newer_neighbor(&mut m1);
|
||||
}
|
||||
Some(LogQueue::Rewrite) => {
|
||||
memtable.append_rewrite(generate_entry_indexes(
|
||||
memtable.replay_rewrite(generate_entry_indexes(
|
||||
0,
|
||||
10,
|
||||
FileId::new(LogQueue::Rewrite, 1),
|
||||
|
@ -2099,6 +2426,13 @@ mod tests {
|
|||
batches[1].add_command(last_rid, Command::Compact { index: 5 });
|
||||
batches[2].add_entry_indexes(last_rid, generate_entry_indexes(11, 21, files[2]));
|
||||
|
||||
// entries [1, 10] => entries [11, 20][5, 10] => compact 8
|
||||
last_rid += 1;
|
||||
batches[0].add_entry_indexes(last_rid, generate_entry_indexes(1, 11, files[0]));
|
||||
batches[1].add_entry_indexes(last_rid, generate_entry_indexes(11, 21, files[1]));
|
||||
batches[1].add_entry_indexes(last_rid, generate_entry_indexes(5, 11, files[1]));
|
||||
batches[2].add_command(last_rid, Command::Compact { index: 8 });
|
||||
|
||||
for b in batches.iter_mut() {
|
||||
b.finish_write(FileBlockHandle::dummy(LogQueue::Append));
|
||||
}
|
||||
|
|
151
src/metrics.rs
151
src/metrics.rs
|
@ -1,36 +1,152 @@
|
|||
// Copyright 2021 TiKV Project Authors. Licensed under Apache-2.0.
|
||||
|
||||
use std::time::Instant;
|
||||
use std::{
|
||||
cell::{RefCell, RefMut},
|
||||
ops::AddAssign,
|
||||
time::{Duration, Instant},
|
||||
};
|
||||
|
||||
use prometheus::*;
|
||||
use prometheus_static_metric::*;
|
||||
|
||||
use crate::util::InstantExt;
|
||||
|
||||
pub struct StopWatch<'a> {
|
||||
histogram: &'a Histogram,
|
||||
pub struct StopWatch<M: TimeMetric> {
|
||||
metric: M,
|
||||
start: Instant,
|
||||
}
|
||||
|
||||
impl<'a> StopWatch<'a> {
|
||||
impl<M: TimeMetric> StopWatch<M> {
|
||||
#[inline]
|
||||
pub fn new(histogram: &'a Histogram) -> Self {
|
||||
pub fn new(metric: M) -> Self {
|
||||
Self {
|
||||
histogram,
|
||||
metric,
|
||||
start: Instant::now(),
|
||||
}
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn new_with(histogram: &'a Histogram, start: Instant) -> Self {
|
||||
Self { histogram, start }
|
||||
pub fn new_with(metric: M, start: Instant) -> Self {
|
||||
Self { metric, start }
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a> Drop for StopWatch<'a> {
|
||||
impl<M: TimeMetric> Drop for StopWatch<M> {
|
||||
fn drop(&mut self) {
|
||||
self.histogram
|
||||
.observe(self.start.saturating_elapsed().as_secs_f64());
|
||||
self.metric.observe(self.start.saturating_elapsed());
|
||||
}
|
||||
}
|
||||
|
||||
/// PerfContext records cumulative performance statistics of operations.
|
||||
///
|
||||
/// Raft Engine will update the data in the thread-local PerfContext whenever
|
||||
/// an opeartion is performed.
|
||||
#[derive(Debug, Clone, Default)]
|
||||
pub struct PerfContext {
|
||||
/// Time spent encoding and compressing log entries.
|
||||
pub log_populating_duration: Duration,
|
||||
|
||||
/// Time spent waiting for the write group to form and get processed.
|
||||
pub write_wait_duration: Duration,
|
||||
|
||||
/// Time spent writing the logs to files.
|
||||
pub log_write_duration: Duration,
|
||||
|
||||
/// Time spent rotating the active log file.
|
||||
pub log_rotate_duration: Duration,
|
||||
|
||||
// Time spent synchronizing logs to the disk.
|
||||
pub log_sync_duration: Duration,
|
||||
|
||||
// Time spent applying the appended logs.
|
||||
pub apply_duration: Duration,
|
||||
}
|
||||
|
||||
impl AddAssign<&'_ PerfContext> for PerfContext {
|
||||
fn add_assign(&mut self, rhs: &PerfContext) {
|
||||
self.log_populating_duration += rhs.log_populating_duration;
|
||||
self.write_wait_duration += rhs.write_wait_duration;
|
||||
self.log_write_duration += rhs.log_write_duration;
|
||||
self.log_rotate_duration += rhs.log_rotate_duration;
|
||||
self.log_sync_duration += rhs.log_sync_duration;
|
||||
self.apply_duration += rhs.apply_duration;
|
||||
}
|
||||
}
|
||||
|
||||
thread_local! {
|
||||
static TLS_PERF_CONTEXT: RefCell<PerfContext> = RefCell::new(PerfContext::default());
|
||||
}
|
||||
|
||||
/// Gets a copy of the thread-local PerfContext.
|
||||
pub fn get_perf_context() -> PerfContext {
|
||||
TLS_PERF_CONTEXT.with(|c| c.borrow().clone())
|
||||
}
|
||||
|
||||
/// Resets the thread-local PerfContext and takes its old value.
|
||||
pub fn take_perf_context() -> PerfContext {
|
||||
TLS_PERF_CONTEXT.with(|c| c.take())
|
||||
}
|
||||
|
||||
/// Sets the value of the thread-local PerfContext.
|
||||
pub fn set_perf_context(perf_context: PerfContext) {
|
||||
TLS_PERF_CONTEXT.with(|c| *c.borrow_mut() = perf_context);
|
||||
}
|
||||
|
||||
pub(crate) struct PerfContextField<P> {
|
||||
projector: P,
|
||||
}
|
||||
|
||||
impl<P> PerfContextField<P>
|
||||
where
|
||||
P: Fn(&mut PerfContext) -> &mut Duration,
|
||||
{
|
||||
pub fn new(projector: P) -> Self {
|
||||
PerfContextField { projector }
|
||||
}
|
||||
}
|
||||
|
||||
#[macro_export]
|
||||
macro_rules! perf_context {
|
||||
($field: ident) => {
|
||||
$crate::metrics::PerfContextField::new(|perf_context| &mut perf_context.$field)
|
||||
};
|
||||
}
|
||||
|
||||
pub trait TimeMetric {
|
||||
fn observe(&self, duration: Duration);
|
||||
|
||||
fn observe_since(&self, earlier: Instant) -> Duration {
|
||||
let dur = earlier.saturating_elapsed();
|
||||
self.observe(dur);
|
||||
dur
|
||||
}
|
||||
}
|
||||
|
||||
impl TimeMetric for &Histogram {
|
||||
fn observe(&self, duration: Duration) {
|
||||
Histogram::observe(self, duration.as_secs_f64());
|
||||
}
|
||||
}
|
||||
|
||||
impl<P> TimeMetric for PerfContextField<P>
|
||||
where
|
||||
P: Fn(&mut PerfContext) -> &mut Duration,
|
||||
{
|
||||
fn observe(&self, duration: Duration) {
|
||||
TLS_PERF_CONTEXT.with(|perf_context| {
|
||||
*RefMut::map(perf_context.borrow_mut(), &self.projector) += duration;
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
impl<M1, M2> TimeMetric for (M1, M2)
|
||||
where
|
||||
M1: TimeMetric,
|
||||
M2: TimeMetric,
|
||||
{
|
||||
fn observe(&self, duration: Duration) {
|
||||
self.0.observe(duration);
|
||||
self.1.observe(duration);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -85,6 +201,12 @@ lazy_static! {
|
|||
exponential_buckets(256.0, 1.8, 22).unwrap()
|
||||
)
|
||||
.unwrap();
|
||||
pub static ref ENGINE_WRITE_COMPRESSION_RATIO_HISTOGRAM: Histogram = register_histogram!(
|
||||
"raft_engine_write_compression_ratio",
|
||||
"Bucketed histogram of Raft Engine write compression ratio",
|
||||
exponential_buckets(0.0005, 1.8, 16).unwrap()
|
||||
)
|
||||
.unwrap();
|
||||
pub static ref LOG_ALLOCATE_DURATION_HISTOGRAM: Histogram = register_histogram!(
|
||||
"raft_engine_allocate_log_duration_seconds",
|
||||
"Bucketed histogram of Raft Engine allocate log duration",
|
||||
|
@ -156,6 +278,13 @@ lazy_static! {
|
|||
&["type"]
|
||||
)
|
||||
.unwrap();
|
||||
pub static ref RECYCLED_FILE_COUNT: LogQueueGaugeVec = register_static_int_gauge_vec!(
|
||||
LogQueueGaugeVec,
|
||||
"raft_engine_recycled_file_count",
|
||||
"Amount of recycled files in Raft engine",
|
||||
&["type"]
|
||||
)
|
||||
.unwrap();
|
||||
pub static ref SWAP_FILE_COUNT: IntGauge = register_int_gauge!(
|
||||
"raft_engine_swap_file_count",
|
||||
"Amount of swap files in Raft engine"
|
||||
|
|
117
src/pipe_log.rs
117
src/pipe_log.rs
|
@ -3,6 +3,13 @@
|
|||
//! A generic log storage.
|
||||
|
||||
use std::cmp::Ordering;
|
||||
use std::fmt::{self, Display};
|
||||
|
||||
use fail::fail_point;
|
||||
use num_derive::{FromPrimitive, ToPrimitive};
|
||||
use num_traits::ToPrimitive;
|
||||
use serde_repr::{Deserialize_repr, Serialize_repr};
|
||||
use strum::EnumIter;
|
||||
|
||||
use crate::Result;
|
||||
|
||||
|
@ -18,7 +25,7 @@ pub enum LogQueue {
|
|||
pub type FileSeq = u64;
|
||||
|
||||
/// A unique identifier for a log file.
|
||||
#[derive(Debug, Copy, Clone, Eq, PartialEq)]
|
||||
#[derive(Debug, Copy, Clone, PartialEq, Eq)]
|
||||
pub struct FileId {
|
||||
pub queue: LogQueue,
|
||||
pub seq: FileSeq,
|
||||
|
@ -55,7 +62,7 @@ impl std::cmp::PartialOrd for FileId {
|
|||
}
|
||||
|
||||
/// A logical pointer to a chunk of log file data.
|
||||
#[derive(Debug, Copy, Clone, PartialEq)]
|
||||
#[derive(Debug, Copy, Clone, PartialEq, Eq)]
|
||||
pub struct FileBlockHandle {
|
||||
pub id: FileId,
|
||||
pub offset: u64,
|
||||
|
@ -74,33 +81,113 @@ impl FileBlockHandle {
|
|||
}
|
||||
}
|
||||
|
||||
/// Version of log file format.
|
||||
#[repr(u64)]
|
||||
#[derive(
|
||||
Clone,
|
||||
Copy,
|
||||
Debug,
|
||||
Eq,
|
||||
PartialEq,
|
||||
FromPrimitive,
|
||||
ToPrimitive,
|
||||
Serialize_repr,
|
||||
Deserialize_repr,
|
||||
EnumIter,
|
||||
Default,
|
||||
)]
|
||||
pub enum Version {
|
||||
#[default]
|
||||
V1 = 1,
|
||||
V2 = 2,
|
||||
}
|
||||
|
||||
impl Version {
|
||||
pub fn has_log_signing(&self) -> bool {
|
||||
fail_point!("pipe_log::version::force_enable_log_signing", |_| { true });
|
||||
match self {
|
||||
Version::V1 => false,
|
||||
Version::V2 => true,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Display for Version {
|
||||
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
|
||||
write!(f, "{}", self.to_u64().unwrap())
|
||||
}
|
||||
}
|
||||
|
||||
pub struct LogFileContext {
|
||||
pub id: FileId,
|
||||
pub version: Version,
|
||||
}
|
||||
|
||||
impl LogFileContext {
|
||||
pub fn new(id: FileId, version: Version) -> Self {
|
||||
Self { id, version }
|
||||
}
|
||||
|
||||
/// Returns the `signature` in `u32` format.
|
||||
pub fn get_signature(&self) -> Option<u32> {
|
||||
if self.version.has_log_signing() {
|
||||
// Here, the count of files will be always limited to less than
|
||||
// `u32::MAX`. So, we just use the low 32 bit as the `signature`
|
||||
// by default.
|
||||
Some(self.id.seq as u32)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Some bytes whose value might be dependent on the file it is written to.
|
||||
pub trait ReactiveBytes {
|
||||
fn as_bytes(&mut self, ctx: &LogFileContext) -> &[u8];
|
||||
}
|
||||
|
||||
impl<T> ReactiveBytes for &T
|
||||
where
|
||||
T: AsRef<[u8]> + ?Sized,
|
||||
{
|
||||
fn as_bytes(&mut self, _ctx: &LogFileContext) -> &[u8] {
|
||||
(*self).as_ref()
|
||||
}
|
||||
}
|
||||
|
||||
/// A `PipeLog` serves reads and writes over multiple queues of log files.
|
||||
///
|
||||
/// # Safety
|
||||
///
|
||||
/// The pipe will panic if it encounters an unrecoverable failure. Otherwise the
|
||||
/// operations on it should be atomic, i.e. failed operation will not affect
|
||||
/// other ones, and user can still use it afterwards without breaking
|
||||
/// consistency.
|
||||
pub trait PipeLog: Sized {
|
||||
/// Reads some bytes from the specified position.
|
||||
fn read_bytes(&self, handle: FileBlockHandle) -> Result<Vec<u8>>;
|
||||
|
||||
/// Appends some bytes to the specified log queue. Returns file position of
|
||||
/// the written bytes.
|
||||
fn append(&self, queue: LogQueue, bytes: &[u8]) -> Result<FileBlockHandle>;
|
||||
fn append<T: ReactiveBytes + ?Sized>(
|
||||
&self,
|
||||
queue: LogQueue,
|
||||
bytes: &mut T,
|
||||
) -> Result<FileBlockHandle>;
|
||||
|
||||
/// Hints it to synchronize buffered writes. The synchronization is
|
||||
/// mandotory when `sync` is true.
|
||||
/// Synchronizes all buffered writes.
|
||||
///
|
||||
/// This operation might incurs a great latency overhead. It's advised to
|
||||
/// call it once every batch of writes.
|
||||
fn maybe_sync(&self, queue: LogQueue, sync: bool) -> Result<()>;
|
||||
fn sync(&self, queue: LogQueue) -> Result<()>;
|
||||
|
||||
/// Returns the smallest and largest file sequence number of the specified
|
||||
/// log queue.
|
||||
/// Returns the smallest and largest file sequence number, still in use,
|
||||
/// of the specified log queue.
|
||||
fn file_span(&self, queue: LogQueue) -> (FileSeq, FileSeq);
|
||||
|
||||
/// Returns the oldest file ID that is newer than `position`% of all files.
|
||||
fn file_at(&self, queue: LogQueue, mut position: f64) -> FileSeq {
|
||||
if position > 1.0 {
|
||||
position = 1.0;
|
||||
} else if position < 0.0 {
|
||||
position = 0.0;
|
||||
}
|
||||
position = position.clamp(0.0, 1.0);
|
||||
let (first, active) = self.file_span(queue);
|
||||
let count = active - first + 1;
|
||||
first + (count as f64 * position) as u64
|
||||
|
@ -115,8 +202,8 @@ pub trait PipeLog: Sized {
|
|||
/// necessarily panic-safe.
|
||||
fn rotate(&self, queue: LogQueue) -> Result<()>;
|
||||
|
||||
/// Deletes all log files up to the specified file ID. The scope is limited
|
||||
/// to the log queue of `file_id`.
|
||||
/// Deletes all log files smaller than the specified file ID. The scope is
|
||||
/// limited to the log queue of `file_id`.
|
||||
///
|
||||
/// Returns the number of deleted files.
|
||||
fn purge_to(&self, file_id: FileId) -> Result<usize>;
|
||||
|
|
245
src/purge.rs
245
src/purge.rs
|
@ -2,16 +2,18 @@
|
|||
|
||||
use std::collections::VecDeque;
|
||||
use std::collections::{HashMap, HashSet};
|
||||
use std::mem;
|
||||
use std::sync::atomic::{AtomicUsize, Ordering};
|
||||
use std::sync::Arc;
|
||||
|
||||
use fail::fail_point;
|
||||
use log::{info, warn};
|
||||
use parking_lot::{Mutex, RwLock};
|
||||
|
||||
use crate::config::Config;
|
||||
use crate::engine::read_entry_bytes_from_file;
|
||||
use crate::event_listener::EventListener;
|
||||
use crate::log_batch::LogBatch;
|
||||
use crate::log_batch::{AtomicGroupBuilder, LogBatch};
|
||||
use crate::memtable::{MemTableHandle, MemTables};
|
||||
use crate::metrics::*;
|
||||
use crate::pipe_log::{FileBlockHandle, FileId, FileSeq, LogQueue, PipeLog};
|
||||
|
@ -23,9 +25,20 @@ const FORCE_COMPACT_RATIO: f64 = 0.2;
|
|||
const REWRITE_RATIO: f64 = 0.7;
|
||||
// Only rewrite region with stale logs less than this threshold.
|
||||
const MAX_REWRITE_ENTRIES_PER_REGION: usize = 32;
|
||||
const MAX_REWRITE_BATCH_BYTES: usize = 128 * 1024;
|
||||
const MAX_COUNT_BEFORE_FORCE_REWRITE: u32 = 9;
|
||||
|
||||
fn max_batch_bytes() -> usize {
|
||||
fail_point!("max_rewrite_batch_bytes", |s| s
|
||||
.unwrap()
|
||||
.parse::<usize>()
|
||||
.unwrap());
|
||||
128 * 1024
|
||||
}
|
||||
|
||||
fn max_forcely_sync_bytes() -> usize {
|
||||
max_batch_bytes() * 4
|
||||
}
|
||||
|
||||
pub struct PurgeManager<P>
|
||||
where
|
||||
P: PipeLog,
|
||||
|
@ -65,7 +78,7 @@ where
|
|||
}
|
||||
|
||||
pub fn purge_expired_files(&self) -> Result<Vec<u64>> {
|
||||
let _t = StopWatch::new(&ENGINE_PURGE_DURATION_HISTOGRAM);
|
||||
let _t = StopWatch::new(&*ENGINE_PURGE_DURATION_HISTOGRAM);
|
||||
let guard = self.force_rewrite_candidates.try_lock();
|
||||
if guard.is_none() {
|
||||
warn!("Unable to purge expired files: locked");
|
||||
|
@ -76,7 +89,7 @@ where
|
|||
let mut should_compact = HashSet::new();
|
||||
if self.needs_rewrite_log_files(LogQueue::Rewrite) {
|
||||
should_compact.extend(self.rewrite_rewrite_queue()?);
|
||||
self.purge_to(
|
||||
self.rescan_memtables_and_purge_stale_files(
|
||||
LogQueue::Rewrite,
|
||||
self.pipe_log.file_span(LogQueue::Rewrite).1,
|
||||
)?;
|
||||
|
@ -94,23 +107,24 @@ where
|
|||
});
|
||||
|
||||
// Ordering
|
||||
// 1. Must rewrite tombstones AFTER acquiring
|
||||
// `append_queue_barrier`, or deletion marks might be lost
|
||||
// after restart.
|
||||
// 2. Must rewrite tombstones BEFORE rewrite entries, or
|
||||
// entries from recreated region might be lost after
|
||||
// restart.
|
||||
// 1. Must rewrite tombstones AFTER acquiring `append_queue_barrier`, or
|
||||
// deletion marks might be lost after restart.
|
||||
// 2. Must rewrite tombstones BEFORE rewrite entries, or entries from recreated
|
||||
// region might be lost after restart.
|
||||
self.rewrite_append_queue_tombstones()?;
|
||||
should_compact.extend(self.rewrite_or_compact_append_queue(
|
||||
rewrite_watermark,
|
||||
compact_watermark,
|
||||
&mut *rewrite_candidate_regions,
|
||||
&mut rewrite_candidate_regions,
|
||||
)?);
|
||||
|
||||
if append_queue_barrier == first_append && first_append < latest_append {
|
||||
warn!("Unable to purge expired files: blocked by barrier");
|
||||
}
|
||||
self.purge_to(LogQueue::Append, append_queue_barrier)?;
|
||||
self.rescan_memtables_and_purge_stale_files(
|
||||
LogQueue::Append,
|
||||
append_queue_barrier,
|
||||
)?;
|
||||
}
|
||||
}
|
||||
Ok(should_compact.into_iter().collect())
|
||||
|
@ -118,7 +132,6 @@ where
|
|||
|
||||
/// Rewrite append files with seqno no larger than `watermark`. When it's
|
||||
/// None, rewrite the entire queue. Returns the number of purged files.
|
||||
#[cfg(test)]
|
||||
pub fn must_rewrite_append_queue(
|
||||
&self,
|
||||
watermark: Option<FileSeq>,
|
||||
|
@ -139,24 +152,39 @@ where
|
|||
if exit_after_step == Some(2) {
|
||||
return;
|
||||
}
|
||||
self.purge_to(
|
||||
self.rescan_memtables_and_purge_stale_files(
|
||||
LogQueue::Append,
|
||||
self.pipe_log.file_span(LogQueue::Append).1,
|
||||
)
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
pub fn must_rewrite_rewrite_queue(&self) {
|
||||
let _lk = self.force_rewrite_candidates.try_lock().unwrap();
|
||||
self.rewrite_rewrite_queue().unwrap();
|
||||
self.purge_to(
|
||||
self.rescan_memtables_and_purge_stale_files(
|
||||
LogQueue::Rewrite,
|
||||
self.pipe_log.file_span(LogQueue::Rewrite).1,
|
||||
)
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
pub fn must_purge_all_stale(&self) {
|
||||
let _lk = self.force_rewrite_candidates.try_lock().unwrap();
|
||||
self.pipe_log.rotate(LogQueue::Rewrite).unwrap();
|
||||
self.rescan_memtables_and_purge_stale_files(
|
||||
LogQueue::Rewrite,
|
||||
self.pipe_log.file_span(LogQueue::Rewrite).1,
|
||||
)
|
||||
.unwrap();
|
||||
self.pipe_log.rotate(LogQueue::Append).unwrap();
|
||||
self.rescan_memtables_and_purge_stale_files(
|
||||
LogQueue::Append,
|
||||
self.pipe_log.file_span(LogQueue::Append).1,
|
||||
)
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
pub(crate) fn needs_rewrite_log_files(&self, queue: LogQueue) -> bool {
|
||||
let (first_file, active_file) = self.pipe_log.file_span(queue);
|
||||
if active_file == first_file {
|
||||
|
@ -202,29 +230,38 @@ where
|
|||
compact_watermark: FileSeq,
|
||||
rewrite_candidates: &mut HashMap<u64, u32>,
|
||||
) -> Result<Vec<u64>> {
|
||||
let _t = StopWatch::new(&ENGINE_REWRITE_APPEND_DURATION_HISTOGRAM);
|
||||
let _t = StopWatch::new(&*ENGINE_REWRITE_APPEND_DURATION_HISTOGRAM);
|
||||
debug_assert!(compact_watermark <= rewrite_watermark);
|
||||
let mut should_compact = Vec::with_capacity(16);
|
||||
|
||||
let mut new_candidates = HashMap::with_capacity(rewrite_candidates.len());
|
||||
let memtables = self.memtables.collect(|t| {
|
||||
if let Some(f) = t.min_file_seq(LogQueue::Append) {
|
||||
let sparse = t
|
||||
.entries_count_before(FileId::new(LogQueue::Append, rewrite_watermark))
|
||||
< MAX_REWRITE_ENTRIES_PER_REGION;
|
||||
// counter is the times that target region triggers force compact.
|
||||
let compact_counter = rewrite_candidates.get(&t.region_id()).unwrap_or(&0);
|
||||
if f < compact_watermark
|
||||
&& !sparse
|
||||
&& *compact_counter < MAX_COUNT_BEFORE_FORCE_REWRITE
|
||||
{
|
||||
let min_append_seq = t.min_file_seq(LogQueue::Append).unwrap_or(u64::MAX);
|
||||
let old = min_append_seq < compact_watermark || t.rewrite_count() > 0;
|
||||
let has_something_to_rewrite = min_append_seq <= rewrite_watermark;
|
||||
let append_heavy = t.has_at_least_some_entries_before(
|
||||
FileId::new(LogQueue::Append, rewrite_watermark),
|
||||
MAX_REWRITE_ENTRIES_PER_REGION + t.rewrite_count(),
|
||||
);
|
||||
let full_heavy = t.has_at_least_some_entries_before(
|
||||
FileId::new(LogQueue::Append, rewrite_watermark),
|
||||
MAX_REWRITE_ENTRIES_PER_REGION,
|
||||
);
|
||||
// counter is the times that target region triggers force compact.
|
||||
let compact_counter = rewrite_candidates.get(&t.region_id()).unwrap_or(&0);
|
||||
if old && full_heavy {
|
||||
if *compact_counter < MAX_COUNT_BEFORE_FORCE_REWRITE {
|
||||
// repeatedly ask user to compact these heavy regions.
|
||||
should_compact.push(t.region_id());
|
||||
new_candidates.insert(t.region_id(), *compact_counter + 1);
|
||||
} else if f < rewrite_watermark {
|
||||
return sparse || *compact_counter >= MAX_COUNT_BEFORE_FORCE_REWRITE;
|
||||
return false;
|
||||
} else {
|
||||
// user is not responsive, do the rewrite ourselves.
|
||||
should_compact.push(t.region_id());
|
||||
return has_something_to_rewrite;
|
||||
}
|
||||
}
|
||||
false
|
||||
!append_heavy && has_something_to_rewrite
|
||||
});
|
||||
|
||||
self.rewrite_memtables(
|
||||
|
@ -239,7 +276,7 @@ where
|
|||
|
||||
// Rewrites the entire rewrite queue into new log files.
|
||||
fn rewrite_rewrite_queue(&self) -> Result<Vec<u64>> {
|
||||
let _t = StopWatch::new(&ENGINE_REWRITE_REWRITE_DURATION_HISTOGRAM);
|
||||
let _t = StopWatch::new(&*ENGINE_REWRITE_REWRITE_DURATION_HISTOGRAM);
|
||||
self.pipe_log.rotate(LogQueue::Rewrite)?;
|
||||
|
||||
let mut force_compact_regions = vec![];
|
||||
|
@ -262,11 +299,12 @@ where
|
|||
&mut log_batch,
|
||||
None, /* rewrite_watermark */
|
||||
true, /* sync */
|
||||
)
|
||||
)?;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
// Exclusive.
|
||||
fn purge_to(&self, queue: LogQueue, seq: FileSeq) -> Result<()> {
|
||||
fn rescan_memtables_and_purge_stale_files(&self, queue: LogQueue, seq: FileSeq) -> Result<()> {
|
||||
let min_seq = self.memtables.fold(seq, |min, t| {
|
||||
t.min_file_seq(queue).map_or(min, |m| std::cmp::min(min, m))
|
||||
});
|
||||
|
@ -276,7 +314,7 @@ where
|
|||
seq: min_seq,
|
||||
})?;
|
||||
if purged > 0 {
|
||||
info!("purged {} expired log files for queue {:?}", purged, queue);
|
||||
info!("purged {purged} expired log files for queue {queue:?}");
|
||||
for listener in &self.listeners {
|
||||
listener.post_purge(FileId {
|
||||
queue,
|
||||
|
@ -293,11 +331,14 @@ where
|
|||
expect_rewrites_per_memtable: usize,
|
||||
rewrite: Option<FileSeq>,
|
||||
) -> Result<()> {
|
||||
// Only use atomic group for rewrite-rewrite operation.
|
||||
let needs_atomicity = (|| {
|
||||
fail_point!("force_use_atomic_group", |_| true);
|
||||
rewrite.is_none()
|
||||
})();
|
||||
let mut log_batch = LogBatch::default();
|
||||
let mut total_size = 0;
|
||||
for memtable in memtables {
|
||||
let mut entry_indexes = Vec::with_capacity(expect_rewrites_per_memtable);
|
||||
let mut entries = Vec::with_capacity(expect_rewrites_per_memtable);
|
||||
let mut kvs = Vec::new();
|
||||
let region_id = {
|
||||
let m = memtable.read();
|
||||
|
@ -311,34 +352,88 @@ where
|
|||
m.region_id()
|
||||
};
|
||||
|
||||
// FIXME: This code makes my brain hurt.
|
||||
let mut cursor = 0;
|
||||
while cursor < entry_indexes.len() {
|
||||
let entry =
|
||||
read_entry_bytes_from_file(self.pipe_log.as_ref(), &entry_indexes[cursor])?;
|
||||
total_size += entry.len();
|
||||
entries.push(entry);
|
||||
if total_size > MAX_REWRITE_BATCH_BYTES {
|
||||
let mut take_entries = Vec::with_capacity(expect_rewrites_per_memtable);
|
||||
std::mem::swap(&mut take_entries, &mut entries);
|
||||
let mut take_entry_indexes = entry_indexes.split_off(cursor + 1);
|
||||
std::mem::swap(&mut take_entry_indexes, &mut entry_indexes);
|
||||
log_batch.add_raw_entries(region_id, take_entry_indexes, take_entries)?;
|
||||
self.rewrite_impl(&mut log_batch, rewrite, false)?;
|
||||
total_size = 0;
|
||||
cursor = 0;
|
||||
} else {
|
||||
cursor += 1;
|
||||
let mut previous_size = log_batch.approximate_size();
|
||||
let mut atomic_group = None;
|
||||
let mut atomic_group_start = None;
|
||||
let mut current_entry_indexes = Vec::new();
|
||||
let mut current_entries = Vec::new();
|
||||
let mut current_size = 0;
|
||||
let mut unsynced_size = 0;
|
||||
// Split the entries into smaller chunks, so that we don't OOM, and the
|
||||
// compression overhead is not too high.
|
||||
let mut entry_indexes = entry_indexes.into_iter().peekable();
|
||||
while let Some(ei) = entry_indexes.next() {
|
||||
let entry = read_entry_bytes_from_file(self.pipe_log.as_ref(), &ei)?;
|
||||
current_size += entry.len();
|
||||
current_entries.push(entry);
|
||||
current_entry_indexes.push(ei);
|
||||
unsynced_size += current_size;
|
||||
// If this is the last entry, we handle them outside the loop.
|
||||
if entry_indexes.peek().is_some()
|
||||
&& current_size + previous_size > max_batch_bytes()
|
||||
{
|
||||
if needs_atomicity {
|
||||
if previous_size > 0 {
|
||||
// We are certain that prev raft group and current raft group cannot fit
|
||||
// inside one batch.
|
||||
// To avoid breaking atomicity, we need to flush.
|
||||
self.rewrite_impl(&mut log_batch, rewrite, false)?;
|
||||
previous_size = 0;
|
||||
if current_size <= max_batch_bytes() {
|
||||
continue;
|
||||
}
|
||||
}
|
||||
match atomic_group.as_mut() {
|
||||
None => {
|
||||
let mut g = AtomicGroupBuilder::default();
|
||||
g.begin(&mut log_batch);
|
||||
atomic_group = Some(g);
|
||||
}
|
||||
Some(g) => {
|
||||
g.add(&mut log_batch);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
log_batch.add_raw_entries(
|
||||
region_id,
|
||||
mem::take(&mut current_entry_indexes),
|
||||
mem::take(&mut current_entries),
|
||||
)?;
|
||||
current_size = 0;
|
||||
previous_size = 0;
|
||||
let sync = if unsynced_size >= max_forcely_sync_bytes() {
|
||||
// Avoiding too many unsynced size can make the later `fdatasync` in
|
||||
// the append progress blocked for too long.
|
||||
unsynced_size = 0;
|
||||
true
|
||||
} else {
|
||||
false
|
||||
};
|
||||
let handle = self.rewrite_impl(&mut log_batch, rewrite, sync)?.unwrap();
|
||||
if needs_atomicity && atomic_group_start.is_none() {
|
||||
atomic_group_start = Some(handle.id.seq);
|
||||
}
|
||||
}
|
||||
}
|
||||
if !entries.is_empty() {
|
||||
log_batch.add_raw_entries(region_id, entry_indexes, entries)?;
|
||||
}
|
||||
log_batch.add_raw_entries(region_id, current_entry_indexes, current_entries)?;
|
||||
for (k, v) in kvs {
|
||||
log_batch.put(region_id, k, v);
|
||||
log_batch.put(region_id, k, v)?;
|
||||
}
|
||||
if let Some(g) = atomic_group.as_mut() {
|
||||
g.end(&mut log_batch);
|
||||
let handle = self.rewrite_impl(&mut log_batch, rewrite, false)?.unwrap();
|
||||
self.memtables.apply_rewrite_atomic_group(
|
||||
region_id,
|
||||
atomic_group_start.unwrap(),
|
||||
handle.id.seq,
|
||||
);
|
||||
} else if log_batch.approximate_size() > max_batch_bytes() {
|
||||
self.rewrite_impl(&mut log_batch, rewrite, false)?;
|
||||
}
|
||||
}
|
||||
self.rewrite_impl(&mut log_batch, rewrite, true)
|
||||
self.rewrite_impl(&mut log_batch, rewrite, true)?;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn rewrite_impl(
|
||||
|
@ -346,15 +441,20 @@ where
|
|||
log_batch: &mut LogBatch,
|
||||
rewrite_watermark: Option<FileSeq>,
|
||||
sync: bool,
|
||||
) -> Result<()> {
|
||||
let len = log_batch.finish_populate(self.cfg.batch_compression_threshold.0 as usize)?;
|
||||
if len == 0 {
|
||||
return self.pipe_log.maybe_sync(LogQueue::Rewrite, sync);
|
||||
) -> Result<Option<FileBlockHandle>> {
|
||||
if log_batch.is_empty() {
|
||||
debug_assert!(sync);
|
||||
self.pipe_log.sync(LogQueue::Rewrite)?;
|
||||
return Ok(None);
|
||||
}
|
||||
log_batch.finish_populate(
|
||||
self.cfg.batch_compression_threshold.0 as usize,
|
||||
self.cfg.compression_level,
|
||||
)?;
|
||||
let file_handle = self.pipe_log.append(LogQueue::Rewrite, log_batch)?;
|
||||
if sync {
|
||||
self.pipe_log.sync(LogQueue::Rewrite)?;
|
||||
}
|
||||
let file_handle = self
|
||||
.pipe_log
|
||||
.append(LogQueue::Rewrite, log_batch.encoded_bytes())?;
|
||||
self.pipe_log.maybe_sync(LogQueue::Rewrite, sync)?;
|
||||
log_batch.finish_write(file_handle);
|
||||
self.memtables.apply_rewrite_writes(
|
||||
log_batch.drain(),
|
||||
|
@ -373,10 +473,11 @@ where
|
|||
.append
|
||||
.observe(file_handle.len as f64);
|
||||
}
|
||||
Ok(())
|
||||
Ok(Some(file_handle))
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Default)]
|
||||
pub struct PurgeHook {
|
||||
// Append queue log files that are not yet fully applied to MemTable must not be
|
||||
// purged even when not referenced by any MemTable.
|
||||
|
@ -386,14 +487,6 @@ pub struct PurgeHook {
|
|||
active_log_files: RwLock<VecDeque<(FileSeq, AtomicUsize)>>,
|
||||
}
|
||||
|
||||
impl PurgeHook {
|
||||
pub fn new() -> Self {
|
||||
PurgeHook {
|
||||
active_log_files: Default::default(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl EventListener for PurgeHook {
|
||||
fn post_new_log_file(&self, file_id: FileId) {
|
||||
if file_id.queue == LogQueue::Append {
|
||||
|
|
|
@ -20,7 +20,7 @@ const DEFAULT_PAGE_SIZE: usize = 64 * 1024 * 1024; // 64MB
|
|||
|
||||
struct SwappyAllocatorCore<A = Global>
|
||||
where
|
||||
A: Allocator,
|
||||
A: Allocator + Send + Sync,
|
||||
{
|
||||
budget: usize,
|
||||
path: PathBuf,
|
||||
|
@ -39,16 +39,15 @@ where
|
|||
/// The allocations of its internal metadata are not managed (i.e. allocated via
|
||||
/// `std::alloc::Global`). Do NOT use it as the global allocator.
|
||||
#[derive(Clone)]
|
||||
pub struct SwappyAllocator<A: Allocator>(Arc<SwappyAllocatorCore<A>>);
|
||||
pub struct SwappyAllocator<A: Allocator + Send + Sync>(Arc<SwappyAllocatorCore<A>>);
|
||||
|
||||
impl<A: Allocator> SwappyAllocator<A> {
|
||||
impl<A: Allocator + Send + Sync> SwappyAllocator<A> {
|
||||
pub fn new_over(path: &Path, budget: usize, alloc: A) -> SwappyAllocator<A> {
|
||||
if path.exists() {
|
||||
if let Err(e) = std::fs::remove_dir_all(&path) {
|
||||
if let Err(e) = std::fs::remove_dir_all(path) {
|
||||
error!(
|
||||
"Failed to clean up old swap directory: {}. \
|
||||
"Failed to clean up old swap directory: {e}. \
|
||||
There might be obsolete swap files left in {}.",
|
||||
e,
|
||||
path.display()
|
||||
);
|
||||
}
|
||||
|
@ -107,7 +106,7 @@ impl SwappyAllocator<Global> {
|
|||
}
|
||||
}
|
||||
|
||||
unsafe impl<A: Allocator> Allocator for SwappyAllocator<A> {
|
||||
unsafe impl<A: Allocator + Send + Sync> Allocator for SwappyAllocator<A> {
|
||||
#[inline]
|
||||
fn allocate(&self, layout: Layout) -> Result<NonNull<[u8]>, AllocError> {
|
||||
// Always use mem_allocator to allocate empty pointer.
|
||||
|
@ -121,9 +120,8 @@ unsafe impl<A: Allocator> Allocator for SwappyAllocator<A> {
|
|||
return swap_r;
|
||||
}
|
||||
}
|
||||
self.0.mem_allocator.allocate(layout).map_err(|e| {
|
||||
self.0.mem_allocator.allocate(layout).inspect_err(|_| {
|
||||
self.0.mem_usage.fetch_sub(layout.size(), Ordering::Relaxed);
|
||||
e
|
||||
})
|
||||
}
|
||||
|
||||
|
@ -192,9 +190,8 @@ unsafe impl<A: Allocator> Allocator for SwappyAllocator<A> {
|
|||
self.0
|
||||
.mem_allocator
|
||||
.grow(ptr, old_layout, new_layout)
|
||||
.map_err(|e| {
|
||||
.inspect_err(|_| {
|
||||
self.0.mem_usage.fetch_sub(diff, Ordering::Relaxed);
|
||||
e
|
||||
})
|
||||
}
|
||||
}
|
||||
|
@ -254,11 +251,10 @@ unsafe impl<A: Allocator> Allocator for SwappyAllocator<A> {
|
|||
self.0
|
||||
.mem_allocator
|
||||
.shrink(ptr, old_layout, new_layout)
|
||||
.map(|p| {
|
||||
.inspect(|_| {
|
||||
self.0
|
||||
.mem_usage
|
||||
.fetch_sub(old_layout.size() - new_layout.size(), Ordering::Relaxed);
|
||||
p
|
||||
})
|
||||
}
|
||||
}
|
||||
|
@ -280,8 +276,8 @@ impl Page {
|
|||
fail::fail_point!("swappy::page::new_failure", |_| None);
|
||||
if !root.exists() {
|
||||
// Create directory only when it's needed.
|
||||
std::fs::create_dir_all(&root)
|
||||
.map_err(|e| error!("Failed to create swap directory: {}.", e))
|
||||
std::fs::create_dir_all(root)
|
||||
.map_err(|e| error!("Failed to create swap directory: {e}."))
|
||||
.ok()?;
|
||||
}
|
||||
let path = root.join(Self::page_file_name(seq));
|
||||
|
@ -289,15 +285,16 @@ impl Page {
|
|||
.read(true)
|
||||
.write(true)
|
||||
.create(true)
|
||||
.open(&path)
|
||||
.map_err(|e| error!("Failed to open swap file: {}", e))
|
||||
.truncate(true)
|
||||
.open(path)
|
||||
.map_err(|e| error!("Failed to open swap file: {e}"))
|
||||
.ok()?;
|
||||
f.set_len(size as u64)
|
||||
.map_err(|e| error!("Failed to extend swap file: {}", e))
|
||||
.map_err(|e| error!("Failed to extend swap file: {e}"))
|
||||
.ok()?;
|
||||
let mmap = unsafe {
|
||||
MmapMut::map_mut(&f)
|
||||
.map_err(|e| error!("Failed to mmap swap file: {}", e))
|
||||
.map_err(|e| error!("Failed to mmap swap file: {e}"))
|
||||
.ok()?
|
||||
};
|
||||
SWAP_FILE_COUNT.inc();
|
||||
|
@ -343,9 +340,16 @@ impl Page {
|
|||
#[inline]
|
||||
fn release(self, root: &Path) {
|
||||
debug_assert_eq!(self.ref_counter, 0);
|
||||
|
||||
// Somehow in Windows, we have to drop the mmap file handle first, otherwise
|
||||
// the following file removal will return "Access Denied (OS Error 5)".
|
||||
// Not using `#[cfg(windows)]` here is because it might do no harm in other
|
||||
// operating systems - the mmap file handle is dropped anyhow.
|
||||
drop(self.mmap);
|
||||
|
||||
let path = root.join(Self::page_file_name(self.seq));
|
||||
if let Err(e) = std::fs::remove_file(&path) {
|
||||
warn!("Failed to delete swap file: {}", e);
|
||||
if let Err(e) = std::fs::remove_file(path) {
|
||||
warn!("Failed to delete swap file: {e}");
|
||||
}
|
||||
SWAP_FILE_COUNT.dec();
|
||||
}
|
||||
|
@ -617,14 +621,14 @@ mod tests {
|
|||
assert_eq!(allocator.memory_usage(), 16);
|
||||
assert_eq!(global.stats(), (2, 1, 0, 0));
|
||||
// Deallocate all pages, calls <allocate and deallocate> when memory use is low.
|
||||
disk_vec.clear();
|
||||
disk_vec.truncate(1);
|
||||
disk_vec.shrink_to_fit();
|
||||
assert_eq!(allocator.memory_usage(), 16);
|
||||
assert_eq!(allocator.memory_usage(), 16 + 1);
|
||||
assert_eq!(global.stats(), (3, 1, 0, 0));
|
||||
assert_eq!(file_count(dir.path()), 0);
|
||||
// Grow calls <grow> now.
|
||||
mem_vec.resize(32, 0);
|
||||
assert_eq!(allocator.memory_usage(), 32);
|
||||
assert_eq!(allocator.memory_usage(), 32 + 1);
|
||||
assert_eq!(global.stats(), (3, 1, 1, 0));
|
||||
}
|
||||
|
||||
|
@ -825,7 +829,7 @@ mod tests {
|
|||
// test_eq_after_rotation
|
||||
// test that two deques are equal even if elements are laid out differently
|
||||
let len = 28;
|
||||
let mut ring: VecDeque<i32> = collect(0..len as i32, allocator.clone());
|
||||
let mut ring: VecDeque<i32> = collect(0..len, allocator.clone());
|
||||
let mut shifted = ring.clone();
|
||||
for _ in 0..10 {
|
||||
// shift values 1 step to the right by pop, sub one, push
|
||||
|
@ -1067,7 +1071,7 @@ mod tests {
|
|||
// test_drain_leak
|
||||
static mut DROPS: i32 = 0;
|
||||
|
||||
#[derive(Debug, PartialEq)]
|
||||
#[derive(Debug, PartialEq, Eq)]
|
||||
struct D(u32, bool);
|
||||
|
||||
impl Drop for D {
|
||||
|
@ -1135,7 +1139,7 @@ mod tests {
|
|||
{
|
||||
// issue-58952
|
||||
let c = 2;
|
||||
let bv = vec![2];
|
||||
let bv = [2];
|
||||
let b = bv.iter().filter(|a| **a == c);
|
||||
|
||||
let _a = collect(
|
||||
|
@ -1160,8 +1164,8 @@ mod tests {
|
|||
}
|
||||
{
|
||||
// issue-54477
|
||||
let mut vecdeque_13 = collect(vec![].into_iter(), allocator.clone());
|
||||
let mut vecdeque_29 = collect(vec![0].into_iter(), allocator.clone());
|
||||
let mut vecdeque_13 = collect(vec![], allocator.clone());
|
||||
let mut vecdeque_29 = collect(vec![0], allocator.clone());
|
||||
vecdeque_29.insert(0, 30);
|
||||
vecdeque_29.insert(1, 31);
|
||||
vecdeque_29.insert(2, 32);
|
||||
|
@ -1173,7 +1177,7 @@ mod tests {
|
|||
|
||||
assert_eq!(
|
||||
vecdeque_13,
|
||||
collect(vec![30, 31, 32, 33, 34, 35, 0].into_iter(), allocator,)
|
||||
collect(vec![30, 31, 32, 33, 34, 35, 0], allocator,)
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -63,7 +63,7 @@ where
|
|||
}
|
||||
|
||||
pub struct PanicGuard {
|
||||
prev_hook: *mut (dyn Fn(&panic::PanicInfo<'_>) + Sync + Send + 'static),
|
||||
prev_hook: *mut (dyn Fn(&panic::PanicHookInfo<'_>) + Sync + Send + 'static),
|
||||
}
|
||||
|
||||
struct PointerHolder<T: ?Sized>(*mut T);
|
||||
|
@ -77,7 +77,7 @@ impl PanicGuard {
|
|||
let sendable_prev_hook = PointerHolder(prev_hook);
|
||||
// FIXME: Use thread local hook.
|
||||
panic::set_hook(Box::new(move |info| {
|
||||
eprintln!("{}", s);
|
||||
eprintln!("{s}");
|
||||
unsafe { (*sendable_prev_hook.0)(info) };
|
||||
}));
|
||||
PanicGuard { prev_hook }
|
||||
|
|
132
src/util.rs
132
src/util.rs
|
@ -19,7 +19,7 @@ pub const GIB: u64 = MIB * BINARY_DATA_MAGNITUDE;
|
|||
pub const TIB: u64 = GIB * BINARY_DATA_MAGNITUDE;
|
||||
pub const PIB: u64 = TIB * BINARY_DATA_MAGNITUDE;
|
||||
|
||||
#[derive(Clone, Debug, Copy, PartialEq, PartialOrd)]
|
||||
#[derive(Clone, Debug, Copy, PartialEq, Eq, PartialOrd)]
|
||||
pub struct ReadableSize(pub u64);
|
||||
|
||||
impl ReadableSize {
|
||||
|
@ -69,23 +69,8 @@ impl Serialize for ReadableSize {
|
|||
where
|
||||
S: Serializer,
|
||||
{
|
||||
let size = self.0;
|
||||
let mut buffer = String::new();
|
||||
if size == 0 {
|
||||
write!(buffer, "{}KiB", size).unwrap();
|
||||
} else if size % PIB == 0 {
|
||||
write!(buffer, "{}PiB", size / PIB).unwrap();
|
||||
} else if size % TIB == 0 {
|
||||
write!(buffer, "{}TiB", size / TIB).unwrap();
|
||||
} else if size % GIB as u64 == 0 {
|
||||
write!(buffer, "{}GiB", size / GIB).unwrap();
|
||||
} else if size % MIB as u64 == 0 {
|
||||
write!(buffer, "{}MiB", size / MIB).unwrap();
|
||||
} else if size % KIB as u64 == 0 {
|
||||
write!(buffer, "{}KiB", size / KIB).unwrap();
|
||||
} else {
|
||||
return serializer.serialize_u64(size);
|
||||
}
|
||||
write!(buffer, "{self}").unwrap();
|
||||
serializer.serialize_str(&buffer)
|
||||
}
|
||||
}
|
||||
|
@ -97,11 +82,11 @@ impl FromStr for ReadableSize {
|
|||
fn from_str(s: &str) -> Result<ReadableSize, String> {
|
||||
let size_str = s.trim();
|
||||
if size_str.is_empty() {
|
||||
return Err(format!("{:?} is not a valid size.", s));
|
||||
return Err(format!("{s:?} is not a valid size."));
|
||||
}
|
||||
|
||||
if !size_str.is_ascii() {
|
||||
return Err(format!("ASCII string is expected, but got {:?}", s));
|
||||
return Err(format!("ASCII string is expected, but got {s:?}"));
|
||||
}
|
||||
|
||||
// size: digits and '.' as decimal separator
|
||||
|
@ -123,33 +108,35 @@ impl FromStr for ReadableSize {
|
|||
"B" | "" => B,
|
||||
_ => {
|
||||
return Err(format!(
|
||||
"only B, KB, KiB, MB, MiB, GB, GiB, TB, TiB, PB, and PiB are supported: {:?}",
|
||||
s
|
||||
"only B, KB, KiB, MB, MiB, GB, GiB, TB, TiB, PB, and PiB are supported: {s:?}"
|
||||
));
|
||||
}
|
||||
};
|
||||
|
||||
match size.parse::<f64>() {
|
||||
Ok(n) => Ok(ReadableSize((n * unit as f64) as u64)),
|
||||
Err(_) => Err(format!("invalid size string: {:?}", s)),
|
||||
Err(_) => Err(format!("invalid size string: {s:?}")),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Display for ReadableSize {
|
||||
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
|
||||
if self.0 >= PIB {
|
||||
write!(f, "{:.1}PiB", self.0 as f64 / PIB as f64)
|
||||
} else if self.0 >= TIB {
|
||||
write!(f, "{:.1}TiB", self.0 as f64 / TIB as f64)
|
||||
} else if self.0 >= GIB {
|
||||
write!(f, "{:.1}GiB", self.0 as f64 / GIB as f64)
|
||||
} else if self.0 >= MIB {
|
||||
write!(f, "{:.1}MiB", self.0 as f64 / MIB as f64)
|
||||
} else if self.0 >= KIB {
|
||||
write!(f, "{:.1}KiB", self.0 as f64 / KIB as f64)
|
||||
let size = self.0;
|
||||
if size == 0 {
|
||||
write!(f, "{size}KiB")
|
||||
} else if size % PIB == 0 {
|
||||
write!(f, "{}PiB", size / PIB)
|
||||
} else if size % TIB == 0 {
|
||||
write!(f, "{}TiB", size / TIB)
|
||||
} else if size % GIB == 0 {
|
||||
write!(f, "{}GiB", size / GIB)
|
||||
} else if size % MIB == 0 {
|
||||
write!(f, "{}MiB", size / MIB)
|
||||
} else if size % KIB == 0 {
|
||||
write!(f, "{}KiB", size / KIB)
|
||||
} else {
|
||||
write!(f, "{}B", self.0)
|
||||
write!(f, "{size}B")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -161,7 +148,7 @@ impl<'de> Deserialize<'de> for ReadableSize {
|
|||
{
|
||||
struct SizeVisitor;
|
||||
|
||||
impl<'de> Visitor<'de> for SizeVisitor {
|
||||
impl Visitor<'_> for SizeVisitor {
|
||||
type Value = ReadableSize;
|
||||
|
||||
fn expecting(&self, formatter: &mut fmt::Formatter<'_>) -> fmt::Result {
|
||||
|
@ -234,17 +221,19 @@ pub fn unhash_u64(mut i: u64) -> u64 {
|
|||
|
||||
pub mod lz4 {
|
||||
use crate::{Error, Result};
|
||||
use std::{i32, ptr};
|
||||
use std::ptr;
|
||||
|
||||
pub const DEFAULT_LZ4_COMPRESSION_LEVEL: usize = 1;
|
||||
|
||||
/// Compress content in `buf[skip..]`, and append output to `buf`.
|
||||
pub fn append_compress_block(buf: &mut Vec<u8>, skip: usize) -> Result<()> {
|
||||
pub fn append_compress_block(buf: &mut Vec<u8>, skip: usize, level: usize) -> Result<f64> {
|
||||
let buf_len = buf.len();
|
||||
let content_len = buf_len - skip;
|
||||
let mut compression_ratio = 0.0;
|
||||
if content_len > 0 {
|
||||
if content_len > i32::MAX as usize {
|
||||
return Err(Error::InvalidArgument(format!(
|
||||
"Content too long {}",
|
||||
content_len
|
||||
"Content too long {content_len}"
|
||||
)));
|
||||
}
|
||||
unsafe {
|
||||
|
@ -258,19 +247,21 @@ pub mod lz4 {
|
|||
let le_len = content_len.to_le_bytes();
|
||||
ptr::copy_nonoverlapping(le_len.as_ptr(), buf_ptr.add(buf_len), 4);
|
||||
|
||||
let compressed = lz4_sys::LZ4_compress_default(
|
||||
let compressed = lz4_sys::LZ4_compress_fast(
|
||||
buf_ptr.add(skip) as _,
|
||||
buf_ptr.add(buf_len + 4) as _,
|
||||
content_len as i32,
|
||||
bound,
|
||||
level as i32,
|
||||
);
|
||||
if compressed == 0 {
|
||||
return Err(Error::Other(box_err!("Compression failed")));
|
||||
}
|
||||
compression_ratio = compressed as f64 / content_len as f64;
|
||||
buf.set_len(buf_len + 4 + compressed as usize);
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
Ok(compression_ratio)
|
||||
}
|
||||
|
||||
pub fn decompress_block(src: &[u8]) -> Result<Vec<u8>> {
|
||||
|
@ -288,17 +279,16 @@ pub mod lz4 {
|
|||
dst.set_len(l as usize);
|
||||
Ok(dst)
|
||||
} else if l < 0 {
|
||||
Err(Error::Other(box_err!("Decompression failed {}", l)))
|
||||
Err(Error::Other(box_err!("Decompression failed {l}")))
|
||||
} else {
|
||||
Err(Error::Corruption(format!(
|
||||
"Decompressed content length mismatch {} != {}",
|
||||
l, len
|
||||
"Decompressed content length mismatch {l} != {len}"
|
||||
)))
|
||||
}
|
||||
}
|
||||
} else if !src.is_empty() {
|
||||
Err(Error::Corruption(format!(
|
||||
"Content to compress to short {}",
|
||||
"Content to compress too short {}",
|
||||
src.len()
|
||||
)))
|
||||
} else {
|
||||
|
@ -313,7 +303,12 @@ pub mod lz4 {
|
|||
let vecs: Vec<Vec<u8>> = vec![b"".to_vec(), b"123".to_vec(), b"12345678910".to_vec()];
|
||||
for mut vec in vecs.into_iter() {
|
||||
let uncompressed_len = vec.len();
|
||||
super::append_compress_block(&mut vec, 0).unwrap();
|
||||
let compression_ratio =
|
||||
super::append_compress_block(&mut vec, 0, super::DEFAULT_LZ4_COMPRESSION_LEVEL)
|
||||
.unwrap();
|
||||
if uncompressed_len == 0 {
|
||||
assert_eq!(compression_ratio, 0.0);
|
||||
}
|
||||
let res = super::decompress_block(&vec[uncompressed_len..]).unwrap();
|
||||
assert_eq!(res, vec[..uncompressed_len].to_owned());
|
||||
}
|
||||
|
@ -325,6 +320,33 @@ pub trait Factory<Target>: Send + Sync {
|
|||
fn new_target(&self) -> Target;
|
||||
}
|
||||
|
||||
/// Returns an aligned `offset`.
|
||||
///
|
||||
/// # Example:
|
||||
///
|
||||
/// ```ignore
|
||||
/// assert_eq!(round_up(18, 4), 20);
|
||||
/// assert_eq!(round_up(64, 16), 64);
|
||||
/// ```
|
||||
#[inline]
|
||||
pub fn round_up(offset: usize, alignment: usize) -> usize {
|
||||
offset.div_ceil(alignment) * alignment
|
||||
}
|
||||
|
||||
/// Returns an aligned `offset`.
|
||||
///
|
||||
/// # Example:
|
||||
///
|
||||
/// ```ignore
|
||||
/// assert_eq!(round_down(18, 4), 16);
|
||||
/// assert_eq!(round_down(64, 16), 64);
|
||||
/// ```
|
||||
#[allow(dead_code)]
|
||||
#[inline]
|
||||
pub fn round_down(offset: usize, alignment: usize) -> usize {
|
||||
offset / alignment * alignment
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
|
@ -366,7 +388,7 @@ mod tests {
|
|||
s: ReadableSize(size),
|
||||
};
|
||||
let res_str = toml::to_string(&c).unwrap();
|
||||
let exp_str = format!("s = {:?}\n", exp);
|
||||
let exp_str = format!("s = {exp:?}\n");
|
||||
assert_eq!(res_str, exp_str);
|
||||
let res_size: SizeHolder = toml::from_str(&exp_str).unwrap();
|
||||
assert_eq!(res_size.s.0, size);
|
||||
|
@ -376,7 +398,7 @@ mod tests {
|
|||
s: ReadableSize(512),
|
||||
};
|
||||
let res_str = toml::to_string(&c).unwrap();
|
||||
assert_eq!(res_str, "s = 512\n");
|
||||
assert_eq!(res_str, "s = \"512B\"\n");
|
||||
let res_size: SizeHolder = toml::from_str(&res_str).unwrap();
|
||||
assert_eq!(res_size.s.0, c.s.0);
|
||||
|
||||
|
@ -416,7 +438,7 @@ mod tests {
|
|||
("0e+10MB", 0),
|
||||
];
|
||||
for (src, exp) in decode_cases {
|
||||
let src = format!("s = {:?}", src);
|
||||
let src = format!("s = {src:?}");
|
||||
let res: SizeHolder = toml::from_str(&src).unwrap();
|
||||
assert_eq!(res.s.0, exp);
|
||||
}
|
||||
|
@ -426,7 +448,7 @@ mod tests {
|
|||
"4B7", "5M_",
|
||||
];
|
||||
for src in illegal_cases {
|
||||
let src_str = format!("s = {:?}", src);
|
||||
let src_str = format!("s = {src:?}");
|
||||
assert!(toml::from_str::<SizeHolder>(&src_str).is_err(), "{}", src);
|
||||
}
|
||||
}
|
||||
|
@ -435,4 +457,16 @@ mod tests {
|
|||
fn test_unhash() {
|
||||
assert_eq!(unhash_u64(hash_u64(777)), 777);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_rounding() {
|
||||
// round_up
|
||||
assert_eq!(round_up(18, 4), 20);
|
||||
assert_eq!(round_up(64, 16), 64);
|
||||
assert_eq!(round_up(79, 4096), 4096);
|
||||
// round_down
|
||||
assert_eq!(round_down(18, 4), 16);
|
||||
assert_eq!(round_down(64, 16), 64);
|
||||
assert_eq!(round_down(79, 4096), 0);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -13,16 +13,18 @@ use std::time::Instant;
|
|||
use fail::fail_point;
|
||||
use parking_lot::{Condvar, Mutex};
|
||||
|
||||
use crate::PerfContext;
|
||||
|
||||
type Ptr<T> = Option<NonNull<T>>;
|
||||
|
||||
///
|
||||
pub struct Writer<P, O> {
|
||||
next: Cell<Ptr<Writer<P, O>>>,
|
||||
payload: *mut P,
|
||||
output: Option<O>,
|
||||
|
||||
pub(crate) sync: bool,
|
||||
pub(crate) start_time: Instant,
|
||||
pub(crate) entered_time: Option<Instant>,
|
||||
pub(crate) perf_context_diff: PerfContext,
|
||||
}
|
||||
|
||||
impl<P, O> Writer<P, O> {
|
||||
|
@ -32,19 +34,20 @@ impl<P, O> Writer<P, O> {
|
|||
///
|
||||
/// Data pointed by `payload` is mutably referenced by this writer. Do not
|
||||
/// access the payload by its original name during this writer's lifetime.
|
||||
pub fn new(payload: &mut P, sync: bool, start_time: Instant) -> Self {
|
||||
pub fn new(payload: &mut P, sync: bool) -> Self {
|
||||
Writer {
|
||||
next: Cell::new(None),
|
||||
payload: payload as *mut _,
|
||||
output: None,
|
||||
sync,
|
||||
start_time,
|
||||
entered_time: None,
|
||||
perf_context_diff: PerfContext::default(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns an immutable reference to the payload.
|
||||
pub fn get_payload(&self) -> &P {
|
||||
unsafe { &*self.payload }
|
||||
/// Returns a mutable reference to the payload.
|
||||
pub fn mut_payload(&mut self) -> &mut P {
|
||||
unsafe { &mut *self.payload }
|
||||
}
|
||||
|
||||
/// Sets the output. This method is re-entrant.
|
||||
|
@ -91,7 +94,7 @@ impl<'a, 'b, P, O> WriteGroup<'a, 'b, P, O> {
|
|||
}
|
||||
}
|
||||
|
||||
impl<'a, 'b, P, O> Drop for WriteGroup<'a, 'b, P, O> {
|
||||
impl<P, O> Drop for WriteGroup<'_, '_, P, O> {
|
||||
fn drop(&mut self) {
|
||||
self.ref_barrier.leader_exit();
|
||||
}
|
||||
|
@ -104,7 +107,7 @@ pub struct WriterIter<'a, 'b, 'c, P: 'c, O: 'c> {
|
|||
marker: PhantomData<&'a WriteGroup<'b, 'c, P, O>>,
|
||||
}
|
||||
|
||||
impl<'a, 'b, 'c, P, O> Iterator for WriterIter<'a, 'b, 'c, P, O> {
|
||||
impl<'a, P, O> Iterator for WriterIter<'a, '_, '_, P, O> {
|
||||
type Item = &'a mut Writer<P, O>;
|
||||
|
||||
fn next(&mut self) -> Option<Self::Item> {
|
||||
|
@ -233,12 +236,11 @@ mod tests {
|
|||
#[test]
|
||||
fn test_sequential_groups() {
|
||||
let barrier: WriteBarrier<(), u32> = Default::default();
|
||||
let mut payload = ();
|
||||
let mut leaders = 0;
|
||||
let mut processed_writers = 0;
|
||||
|
||||
for _ in 0..4 {
|
||||
let mut writer = Writer::new(&mut payload, false, Instant::now());
|
||||
let mut writer = Writer::new(&mut (), false);
|
||||
{
|
||||
let mut wg = barrier.enter(&mut writer).unwrap();
|
||||
leaders += 1;
|
||||
|
@ -290,13 +292,14 @@ mod tests {
|
|||
self.ths.push(
|
||||
ThreadBuilder::new()
|
||||
.spawn(move || {
|
||||
let mut writer = Writer::new(&mut seq, false, Instant::now());
|
||||
let mut writer = Writer::new(&mut seq, false);
|
||||
{
|
||||
let mut wg = barrier.enter(&mut writer).unwrap();
|
||||
leader_enter_tx.send(()).unwrap();
|
||||
let mut n = 0;
|
||||
for w in wg.iter_mut() {
|
||||
w.set_output(*w.get_payload());
|
||||
let p = *w.mut_payload();
|
||||
w.set_output(p);
|
||||
n += 1;
|
||||
}
|
||||
assert_eq!(n, 1);
|
||||
|
@ -324,13 +327,14 @@ mod tests {
|
|||
self.ths.push(
|
||||
ThreadBuilder::new()
|
||||
.spawn(move || {
|
||||
let mut writer = Writer::new(&mut seq, false, Instant::now());
|
||||
let mut writer = Writer::new(&mut seq, false);
|
||||
start_thread.wait();
|
||||
if let Some(mut wg) = barrier.enter(&mut writer) {
|
||||
leader_enter_tx_clone.send(()).unwrap();
|
||||
let mut idx = 0;
|
||||
for w in wg.iter_mut() {
|
||||
w.set_output(*w.get_payload());
|
||||
let p = *w.mut_payload();
|
||||
w.set_output(p);
|
||||
idx += 1;
|
||||
}
|
||||
assert_eq!(idx, n as u32);
|
||||
|
|
|
@ -1,13 +1,14 @@
|
|||
[package]
|
||||
name = "stress"
|
||||
version = "0.2.0"
|
||||
version = "0.4.2"
|
||||
authors = ["The TiKV Authors"]
|
||||
edition = "2018"
|
||||
|
||||
[dependencies]
|
||||
clap = { version = "3.0.0-rc.0", features = ["derive", "cargo"] }
|
||||
clap = { version = "3.1", features = ["derive", "cargo"] }
|
||||
const_format = "0.2.13"
|
||||
hdrhistogram = "7.4"
|
||||
num-traits = "0.2"
|
||||
parking_lot_core = "0.9"
|
||||
raft = { git = "https://github.com/tikv/raft-rs", branch = "master", default-features = false, features = ["protobuf-codec"] }
|
||||
raft-engine = { path = "..", features = ["internals"] }
|
||||
|
|
|
@ -8,14 +8,14 @@ use std::sync::Arc;
|
|||
use std::thread::{sleep, Builder as ThreadBuilder, JoinHandle};
|
||||
use std::time::{Duration, Instant};
|
||||
|
||||
use clap::{crate_authors, crate_version, AppSettings, Parser};
|
||||
|
||||
use clap::{crate_authors, crate_version, Parser};
|
||||
use const_format::formatcp;
|
||||
use hdrhistogram::Histogram;
|
||||
use num_traits::FromPrimitive;
|
||||
use parking_lot_core::SpinWait;
|
||||
use raft::eraftpb::Entry;
|
||||
use raft_engine::internals::{EventListener, FileBlockHandle};
|
||||
use raft_engine::{Command, Config, Engine, LogBatch, MessageExt, ReadableSize};
|
||||
use raft_engine::{Command, Config, Engine, LogBatch, MessageExt, ReadableSize, Version};
|
||||
use rand::{thread_rng, Rng, RngCore};
|
||||
|
||||
type WriteBatch = LogBatch;
|
||||
|
@ -52,7 +52,7 @@ const DEFAULT_WRITE_SYNC: bool = false;
|
|||
about = "A stress test tool for Raft Engine",
|
||||
author = crate_authors!(),
|
||||
version = crate_version!(),
|
||||
setting = AppSettings::DontCollapseArgsInUsage,
|
||||
dont_collapse_args_in_usage = true,
|
||||
)]
|
||||
struct ControlOpt {
|
||||
/// Path of raft-engine storage directory
|
||||
|
@ -229,6 +229,20 @@ struct ControlOpt {
|
|||
help = "Compress log batch bigger than this threshold"
|
||||
)]
|
||||
batch_compression_threshold: String,
|
||||
|
||||
#[clap(
|
||||
long = "format-version",
|
||||
takes_value = true,
|
||||
default_value = "1",
|
||||
help = "Format version of log files"
|
||||
)]
|
||||
format_version: u64,
|
||||
|
||||
#[clap(
|
||||
long = "enable-log-recycle",
|
||||
help = "Recycle purged and stale logs for incoming writing"
|
||||
)]
|
||||
enable_log_recycle: bool,
|
||||
}
|
||||
|
||||
#[derive(Debug, Clone)]
|
||||
|
@ -344,7 +358,7 @@ impl Summary {
|
|||
|
||||
fn print(&self, name: &str) {
|
||||
if !self.thread_qps.is_empty() {
|
||||
println!("[{}]", name);
|
||||
println!("[{name}]");
|
||||
println!(
|
||||
"Throughput(QPS) = {:.02}",
|
||||
self.thread_qps.iter().fold(0.0, |sum, qps| { sum + qps })
|
||||
|
@ -366,8 +380,8 @@ impl Summary {
|
|||
let stddev = statistical::standard_deviation(&self.thread_qps, None);
|
||||
stddev / median
|
||||
} else {
|
||||
let first = *self.thread_qps.first().unwrap() as f64;
|
||||
let last = *self.thread_qps.last().unwrap() as f64;
|
||||
let first = *self.thread_qps.first().unwrap();
|
||||
let last = *self.thread_qps.last().unwrap();
|
||||
f64::abs(first - last) / (first + last)
|
||||
};
|
||||
println!("Fairness = {:.01}%", 100.0 - fairness * 100.0);
|
||||
|
@ -382,7 +396,7 @@ fn spawn_write(
|
|||
shutdown: Arc<AtomicBool>,
|
||||
) -> JoinHandle<ThreadSummary> {
|
||||
ThreadBuilder::new()
|
||||
.name(format!("stress-write-thread-{}", index))
|
||||
.name(format!("stress-write-thread-{index}"))
|
||||
.spawn(move || {
|
||||
let mut summary = ThreadSummary::new();
|
||||
let mut log_batch = WriteBatch::with_capacity(4 * 1024);
|
||||
|
@ -430,7 +444,7 @@ fn spawn_write(
|
|||
wait_til(&mut start, last + i);
|
||||
}
|
||||
if let Err(e) = engine.write(&mut log_batch, !args.write_without_sync) {
|
||||
println!("write error {:?} in thread {}", e, index);
|
||||
println!("write error {e:?} in thread {index}");
|
||||
}
|
||||
let end = Instant::now();
|
||||
summary.record(start, end);
|
||||
|
@ -447,7 +461,7 @@ fn spawn_read(
|
|||
shutdown: Arc<AtomicBool>,
|
||||
) -> JoinHandle<ThreadSummary> {
|
||||
ThreadBuilder::new()
|
||||
.name(format!("stress-read-thread-{}", index))
|
||||
.name(format!("stress-read-thread-{index}"))
|
||||
.spawn(move || {
|
||||
let mut summary = ThreadSummary::new();
|
||||
let min_interval = if args.read_ops_per_thread > 0 {
|
||||
|
@ -468,7 +482,7 @@ fn spawn_read(
|
|||
// Read newest entry to avoid conflicting with compact
|
||||
if let Some(last) = engine.last_index(rid) {
|
||||
if let Err(e) = engine.get_entry::<MessageExtTyped>(rid, last) {
|
||||
println!("read error {:?} in thread {}", e, index);
|
||||
println!("read error {e:?} in thread {index}");
|
||||
}
|
||||
let end = Instant::now();
|
||||
summary.record(start, end);
|
||||
|
@ -486,7 +500,7 @@ fn spawn_purge(
|
|||
shutdown: Arc<AtomicBool>,
|
||||
) -> JoinHandle<()> {
|
||||
ThreadBuilder::new()
|
||||
.name(format!("stress-purge-thread-{}", index))
|
||||
.name(format!("stress-purge-thread-{index}"))
|
||||
.spawn(move || {
|
||||
while !shutdown.load(Ordering::Relaxed) {
|
||||
sleep(args.purge_interval);
|
||||
|
@ -501,7 +515,7 @@ fn spawn_purge(
|
|||
engine.compact_to(region, compact_to);
|
||||
}
|
||||
}
|
||||
Err(e) => println!("purge error {:?} in thread {}", e, index),
|
||||
Err(e) => println!("purge error {e:?} in thread {index}"),
|
||||
}
|
||||
}
|
||||
})
|
||||
|
@ -576,6 +590,8 @@ fn main() {
|
|||
config.purge_rewrite_garbage_ratio = opts.purge_rewrite_garbage_ratio;
|
||||
config.batch_compression_threshold =
|
||||
ReadableSize::from_str(&opts.batch_compression_threshold).unwrap();
|
||||
config.enable_log_recycle = opts.enable_log_recycle;
|
||||
config.format_version = Version::from_u64(opts.format_version).unwrap();
|
||||
args.time = Duration::from_secs(opts.time);
|
||||
args.regions = opts.regions;
|
||||
args.purge_interval = Duration::from_secs(opts.purge_interval);
|
||||
|
|
|
@ -50,7 +50,7 @@ impl fmt::Display for Config {
|
|||
self.batch_size,
|
||||
self.item_size,
|
||||
self.entry_size,
|
||||
self.batch_compression_threshold
|
||||
self.batch_compression_threshold,
|
||||
)
|
||||
}
|
||||
}
|
||||
|
@ -107,7 +107,7 @@ fn dir_size(path: &str) -> ReadableSize {
|
|||
ReadableSize(
|
||||
std::fs::read_dir(PathBuf::from(path))
|
||||
.unwrap()
|
||||
.map(|entry| std::fs::metadata(entry.unwrap().path()).unwrap().len() as u64)
|
||||
.map(|entry| std::fs::metadata(entry.unwrap().path()).unwrap().len())
|
||||
.sum(),
|
||||
)
|
||||
}
|
||||
|
@ -151,7 +151,7 @@ fn bench_recovery(c: &mut Criterion) {
|
|||
];
|
||||
|
||||
for (i, (name, cfg)) in cfgs.iter().enumerate() {
|
||||
println!("config-{}: [{}] {}", i, name, cfg);
|
||||
println!("config-{i}: [{name}] {cfg}");
|
||||
}
|
||||
|
||||
fail::cfg("log_fd::open::fadvise_dontneed", "return").unwrap();
|
||||
|
|
|
@ -7,9 +7,15 @@ mod util;
|
|||
mod test_engine;
|
||||
mod test_io_error;
|
||||
|
||||
use fail::FailGuard;
|
||||
use raft_engine::*;
|
||||
use util::*;
|
||||
|
||||
#[ctor::ctor]
|
||||
fn init() {
|
||||
env_logger::init();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_log_batch_full() {
|
||||
let _f = FailGuard::new("log_batch::1kb_entries_size_per_batch", "return");
|
||||
|
|
|
@ -1,7 +1,10 @@
|
|||
// Copyright (c) 2017-present, PingCAP, Inc. Licensed under Apache-2.0.
|
||||
|
||||
use std::sync::Arc;
|
||||
use std::sync::atomic::{AtomicU64, AtomicUsize, Ordering};
|
||||
use std::sync::{Arc, Barrier};
|
||||
use std::time::Duration;
|
||||
|
||||
use fail::FailGuard;
|
||||
use kvproto::raft_serverpb::RaftLocalState;
|
||||
use raft::eraftpb::Entry;
|
||||
use raft_engine::env::{FileSystem, ObfuscatedFileSystem};
|
||||
|
@ -38,8 +41,6 @@ fn append<FS: FileSystem>(
|
|||
#[test]
|
||||
fn test_pipe_log_listeners() {
|
||||
use std::collections::HashMap;
|
||||
use std::sync::atomic::{AtomicU64, AtomicUsize, Ordering};
|
||||
use std::time::Duration;
|
||||
|
||||
#[derive(Default)]
|
||||
struct QueueHook {
|
||||
|
@ -120,14 +121,14 @@ fn test_pipe_log_listeners() {
|
|||
append(
|
||||
&engine,
|
||||
region_id,
|
||||
(i as u64 + 1) / 2,
|
||||
(i as u64 + 1) / 2 + 1,
|
||||
(i as u64).div_ceil(2),
|
||||
(i as u64).div_ceil(2) + 1,
|
||||
Some(&data),
|
||||
);
|
||||
assert_eq!(hook.0[&LogQueue::Append].appends(), i);
|
||||
assert_eq!(hook.0[&LogQueue::Append].applys(), i);
|
||||
}
|
||||
assert_eq!(hook.0[&LogQueue::Append].files(), 11);
|
||||
assert_eq!(hook.0[&LogQueue::Append].files(), 10);
|
||||
|
||||
engine.purge_expired_files().unwrap();
|
||||
assert_eq!(hook.0[&LogQueue::Append].purged(), 8);
|
||||
|
@ -140,8 +141,8 @@ fn test_pipe_log_listeners() {
|
|||
append(
|
||||
&engine,
|
||||
region_id,
|
||||
(i as u64 + 1) / 2,
|
||||
(i as u64 + 1) / 2 + 1,
|
||||
(i as u64).div_ceil(2),
|
||||
(i as u64).div_ceil(2) + 1,
|
||||
Some(&data),
|
||||
);
|
||||
assert_eq!(hook.0[&LogQueue::Append].appends(), i);
|
||||
|
@ -154,7 +155,7 @@ fn test_pipe_log_listeners() {
|
|||
assert_eq!(hook.0[&LogQueue::Append].applys(), 32);
|
||||
|
||||
engine.purge_expired_files().unwrap();
|
||||
assert_eq!(hook.0[&LogQueue::Append].purged(), 13);
|
||||
assert_eq!(hook.0[&LogQueue::Append].purged(), 14);
|
||||
assert_eq!(hook.0[&LogQueue::Rewrite].purged(), rewrite_files as u64);
|
||||
|
||||
// Write region 3 without applying.
|
||||
|
@ -177,8 +178,8 @@ fn test_pipe_log_listeners() {
|
|||
append(
|
||||
&engine,
|
||||
region_id,
|
||||
(i as u64 + 1) / 2,
|
||||
(i as u64 + 1) / 2 + 1,
|
||||
(i as u64).div_ceil(2),
|
||||
(i as u64).div_ceil(2) + 1,
|
||||
Some(&data),
|
||||
);
|
||||
assert_eq!(hook.0[&LogQueue::Append].appends(), i + 3);
|
||||
|
@ -373,7 +374,7 @@ fn test_incomplete_purge() {
|
|||
let engine = Engine::open(cfg.clone()).unwrap();
|
||||
|
||||
{
|
||||
let _f = FailGuard::new("file_pipe_log::remove_file_failure", "return");
|
||||
let _f = FailGuard::new("default_fs::delete_skipped", "return");
|
||||
append(&engine, rid, 0, 20, Some(&data));
|
||||
let append_first = engine.file_span(LogQueue::Append).0;
|
||||
engine.compact_to(rid, 18);
|
||||
|
@ -411,6 +412,7 @@ fn test_tail_corruption() {
|
|||
.unwrap();
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
format_version: Version::V2,
|
||||
..Default::default()
|
||||
};
|
||||
let engine = Engine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
|
||||
|
@ -420,6 +422,42 @@ fn test_tail_corruption() {
|
|||
let engine = Engine::open_with_file_system(cfg, fs.clone()).unwrap();
|
||||
assert_eq!(engine.first_index(rid), None);
|
||||
}
|
||||
// Tail entries block is corrupted.
|
||||
{
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_tail_corruption_1_1")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
format_version: Version::V2,
|
||||
..Default::default()
|
||||
};
|
||||
let engine = Engine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
|
||||
let _f = FailGuard::new("log_batch::corrupted_entries", "return");
|
||||
append(&engine, rid, 1, 5, Some(&data));
|
||||
drop(engine);
|
||||
let engine = Engine::open_with_file_system(cfg, fs.clone()).unwrap();
|
||||
assert_eq!(engine.first_index(rid), None);
|
||||
}
|
||||
// Repeat with absolute consistency.
|
||||
{
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_tail_corruption_1_2")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
format_version: Version::V2,
|
||||
recovery_mode: RecoveryMode::AbsoluteConsistency,
|
||||
..Default::default()
|
||||
};
|
||||
let engine = Engine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
|
||||
let _f = FailGuard::new("log_batch::corrupted_entries", "return");
|
||||
append(&engine, rid, 1, 5, Some(&data));
|
||||
drop(engine);
|
||||
assert!(Engine::open_with_file_system(cfg, fs.clone()).is_err());
|
||||
}
|
||||
// Header is corrupted.
|
||||
{
|
||||
let _f = FailGuard::new("log_file_header::corrupted", "return");
|
||||
|
@ -449,6 +487,748 @@ fn test_tail_corruption() {
|
|||
let engine = Engine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
|
||||
append(&engine, rid, 1, 5, Some(&data));
|
||||
drop(engine);
|
||||
Engine::open_with_file_system(cfg, fs.clone()).unwrap();
|
||||
}
|
||||
// Version::V1 in header owns abnormal DataLayout.
|
||||
{
|
||||
let _f = FailGuard::new("log_file_header::too_large", "return");
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_tail_corruption_4")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
target_file_size: ReadableSize(1),
|
||||
purge_threshold: ReadableSize(1),
|
||||
..Default::default()
|
||||
};
|
||||
let engine = Engine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
|
||||
drop(engine);
|
||||
// Version::V1 will be parsed successfully as the data_layout when the related
|
||||
// `version == V1` will be ignored.
|
||||
let engine = Engine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
|
||||
append(&engine, rid, 1, 5, Some(&data));
|
||||
drop(engine);
|
||||
Engine::open_with_file_system(cfg, fs.clone()).unwrap();
|
||||
}
|
||||
// DataLayout in header is corrupted for Version::V2
|
||||
{
|
||||
let _f = FailGuard::new("log_file_header::too_small", "return");
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_tail_corruption_5")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
format_version: Version::V2,
|
||||
..Default::default()
|
||||
};
|
||||
let engine = Engine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
|
||||
drop(engine);
|
||||
Engine::open_with_file_system(cfg, fs.clone()).unwrap();
|
||||
}
|
||||
// DataLayout in header is abnormal for Version::V2
|
||||
{
|
||||
let _f = FailGuard::new("log_file_header::too_large", "return");
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_tail_corruption_6")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
format_version: Version::V2,
|
||||
..Default::default()
|
||||
};
|
||||
let engine = Engine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
|
||||
drop(engine);
|
||||
Engine::open_with_file_system(cfg, fs.clone()).unwrap();
|
||||
}
|
||||
// DataLayout in header is corrupted for Version::V2, followed with records
|
||||
{
|
||||
let _f = FailGuard::new("log_file_header::too_small", "return");
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_tail_corruption_7")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
target_file_size: ReadableSize(1),
|
||||
purge_threshold: ReadableSize(1),
|
||||
format_version: Version::V2,
|
||||
..Default::default()
|
||||
};
|
||||
let engine = Engine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
|
||||
drop(engine);
|
||||
let engine = Engine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
|
||||
append(&engine, rid, 1, 2, Some(&data));
|
||||
append(&engine, rid, 2, 3, Some(&data));
|
||||
drop(engine);
|
||||
assert!(Engine::open_with_file_system(cfg, fs).is_err());
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_concurrent_write_perf_context() {
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_concurrent_write_perf_context")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
let some_entries = vec![
|
||||
Entry::new(),
|
||||
Entry {
|
||||
index: 1,
|
||||
..Default::default()
|
||||
},
|
||||
];
|
||||
|
||||
let engine = Arc::new(Engine::open(cfg).unwrap());
|
||||
let barrier = Arc::new(Barrier::new(4));
|
||||
|
||||
let ths: Vec<_> = (1..=3)
|
||||
.map(|i| {
|
||||
let engine = engine.clone();
|
||||
let barrier = barrier.clone();
|
||||
let some_entries = some_entries.clone();
|
||||
std::thread::spawn(move || {
|
||||
barrier.wait();
|
||||
let mut log_batch = LogBatch::default();
|
||||
log_batch
|
||||
.add_entries::<MessageExtTyped>(i, &some_entries)
|
||||
.unwrap();
|
||||
let old_perf_context = get_perf_context();
|
||||
engine.write(&mut log_batch, true).unwrap();
|
||||
let new_perf_context = get_perf_context();
|
||||
(old_perf_context, new_perf_context)
|
||||
})
|
||||
})
|
||||
.collect();
|
||||
|
||||
fail::cfg_callback("write_barrier::leader_exit", move || {
|
||||
barrier.wait();
|
||||
// Sleep a while until new writers enter the next write group.
|
||||
std::thread::sleep(Duration::from_millis(100));
|
||||
fail::remove("write_barrier::leader_exit");
|
||||
})
|
||||
.unwrap();
|
||||
|
||||
let mut log_batch = LogBatch::default();
|
||||
log_batch
|
||||
.add_entries::<MessageExtTyped>(4, &some_entries)
|
||||
.unwrap();
|
||||
engine.write(&mut log_batch, true).unwrap();
|
||||
|
||||
for th in ths {
|
||||
let (old, new) = th.join().unwrap();
|
||||
assert_ne!(old.log_populating_duration, new.log_populating_duration);
|
||||
assert_ne!(old.write_wait_duration, new.write_wait_duration);
|
||||
assert_ne!(old.log_write_duration, new.log_write_duration);
|
||||
assert_ne!(old.apply_duration, new.apply_duration);
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_fetch_with_updated_entry_index() {
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_fetch_with_updated_entry_index")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let entry_data = vec![b'x'; 1024];
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
..Default::default()
|
||||
};
|
||||
let engine = Engine::open(cfg).unwrap();
|
||||
for i in 0..10 {
|
||||
append(
|
||||
&engine,
|
||||
i + 1,
|
||||
1 + i * 10,
|
||||
1 + i * 10 + 10,
|
||||
Some(&entry_data),
|
||||
);
|
||||
}
|
||||
assert_eq!(
|
||||
10,
|
||||
engine
|
||||
.fetch_entries_to::<MessageExtTyped>(1, 1, 11, None, &mut vec![])
|
||||
.unwrap()
|
||||
);
|
||||
// Mock: the file is broken or abnormal to read
|
||||
{
|
||||
let _f = FailGuard::new("log_file::read::err", "return");
|
||||
engine
|
||||
.fetch_entries_to::<MessageExtTyped>(10, 91, 101, None, &mut vec![])
|
||||
.unwrap_err();
|
||||
}
|
||||
// Mock: one entry have been updated by background rewrite thread.
|
||||
{
|
||||
let _f = FailGuard::new("log_file::read::err", "8*off->return->off");
|
||||
assert_eq!(
|
||||
10,
|
||||
engine
|
||||
.fetch_entries_to::<MessageExtTyped>(10, 91, 101, None, &mut vec![])
|
||||
.unwrap()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
// FIXME: this test no longer works because recovery cannot reliably detect
|
||||
// overwrite anomaly.
|
||||
// See https://github.com/tikv/raft-engine/issues/250
|
||||
#[test]
|
||||
#[should_panic]
|
||||
fn test_recycle_with_stale_logbatch_at_tail() {
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_recycle_with_stale_log_batch_at_tail")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let data = vec![b'x'; 1024];
|
||||
let rid = 1;
|
||||
let cfg_err = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
target_file_size: ReadableSize::kb(2),
|
||||
purge_threshold: ReadableSize::kb(4),
|
||||
enable_log_recycle: true,
|
||||
format_version: Version::V1,
|
||||
..Default::default()
|
||||
};
|
||||
// Force open Engine with `enable_log_recycle == true` and
|
||||
// `format_version == Version::V1`.
|
||||
let engine = {
|
||||
let _f = FailGuard::new("pipe_log::version::force_enable_log_signing", "return");
|
||||
Engine::open(cfg_err.clone()).unwrap()
|
||||
};
|
||||
// Do not truncate the active_file when exit
|
||||
let _f = FailGuard::new("file_pipe_log::log_file_writer::skip_truncate", "return");
|
||||
assert_eq!(cfg_err.format_version, Version::V1);
|
||||
append(&engine, rid, 1, 2, Some(&data)); // file_seq: 1
|
||||
append(&engine, rid, 2, 3, Some(&data));
|
||||
append(&engine, rid, 3, 4, Some(&data)); // file_seq: 2
|
||||
append(&engine, rid, 4, 5, Some(&data));
|
||||
append(&engine, rid, 5, 6, Some(&data)); // file_seq: 3
|
||||
let append_first = engine.file_span(LogQueue::Append).0;
|
||||
engine.compact_to(rid, 3);
|
||||
engine.purge_expired_files().unwrap();
|
||||
assert!(engine.file_span(LogQueue::Append).0 > append_first);
|
||||
// append, written into seq: 3
|
||||
append(&engine, rid, 4, 5, Some(&data));
|
||||
// recycle, written into seq: 1
|
||||
append(&engine, rid, 5, 6, Some(&data));
|
||||
drop(engine);
|
||||
// Recover the engine with invalid Version::default().
|
||||
// Causing the final log file is a recycled file, containing rewritten
|
||||
// LogBatchs and end with stale LogBatchs, `Engine::open(...)` should
|
||||
// `panic` when recovering the relate `Memtable`.
|
||||
assert!(catch_unwind_silent(|| {
|
||||
let cfg_v2 = Config {
|
||||
format_version: Version::V2,
|
||||
..cfg_err
|
||||
};
|
||||
Engine::open(cfg_v2)
|
||||
})
|
||||
.is_err());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_build_engine_with_multi_datalayout() {
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_build_engine_with_multi_datalayout")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let data = vec![b'x'; 12827];
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
target_file_size: ReadableSize::kb(2),
|
||||
purge_threshold: ReadableSize::kb(4),
|
||||
recovery_mode: RecoveryMode::AbsoluteConsistency,
|
||||
..Default::default()
|
||||
};
|
||||
// Defaultly, File with DataLayout::NoAlignment.
|
||||
let engine = Engine::open(cfg.clone()).unwrap();
|
||||
for rid in 1..=3 {
|
||||
append(&engine, rid, 1, 11, Some(&data));
|
||||
}
|
||||
drop(engine);
|
||||
// File with DataLayout::Alignment
|
||||
let _f = FailGuard::new("file_pipe_log::open::force_set_alignment", "return");
|
||||
let cfg_v2 = Config {
|
||||
format_version: Version::V2,
|
||||
..cfg
|
||||
};
|
||||
let engine = Engine::open(cfg_v2.clone()).unwrap();
|
||||
for rid in 1..=3 {
|
||||
append(&engine, rid, 11, 20, Some(&data));
|
||||
}
|
||||
drop(engine);
|
||||
Engine::open(cfg_v2).unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_build_engine_with_datalayout_abnormal() {
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_build_engine_with_datalayout_abnormal")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let data = vec![b'x'; 1024];
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
target_file_size: ReadableSize::kb(2),
|
||||
purge_threshold: ReadableSize::kb(4),
|
||||
recovery_mode: RecoveryMode::AbsoluteConsistency,
|
||||
format_version: Version::V2,
|
||||
..Default::default()
|
||||
};
|
||||
let _f = FailGuard::new("file_pipe_log::open::force_set_alignment", "return");
|
||||
let engine = Engine::open(cfg.clone()).unwrap();
|
||||
// Content durable with DataLayout::Alignment.
|
||||
append(&engine, 1, 1, 11, Some(&data));
|
||||
append(&engine, 2, 1, 11, Some(&data));
|
||||
{
|
||||
// Set failpoint to dump content with invalid paddings into log file.
|
||||
let _f1 = FailGuard::new("file_pipe_log::append::corrupted_padding", "return");
|
||||
append(&engine, 3, 1, 11, Some(&data));
|
||||
drop(engine);
|
||||
assert!(Engine::open(cfg.clone()).is_err());
|
||||
}
|
||||
{
|
||||
// Reopen the Engine with TolerateXXX mode.
|
||||
let mut cfg_v2 = cfg.clone();
|
||||
cfg_v2.recovery_mode = RecoveryMode::TolerateTailCorruption;
|
||||
let engine = Engine::open(cfg_v2).unwrap();
|
||||
for rid in 4..=8 {
|
||||
append(&engine, rid, 1, 11, Some(&data));
|
||||
}
|
||||
drop(engine);
|
||||
Engine::open(cfg).unwrap();
|
||||
}
|
||||
}
|
||||
|
||||
// issue-228
|
||||
#[test]
|
||||
fn test_partial_rewrite_rewrite() {
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_partial_rewrite_rewrite")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let _f = FailGuard::new("max_rewrite_batch_bytes", "return(1)");
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
recovery_threads: 1,
|
||||
..Default::default()
|
||||
};
|
||||
let engine = Engine::open(cfg.clone()).unwrap();
|
||||
let data = vec![b'x'; 128];
|
||||
|
||||
for rid in 1..=3 {
|
||||
append(&engine, rid, 1, 5, Some(&data));
|
||||
append(&engine, rid, 5, 11, Some(&data));
|
||||
}
|
||||
|
||||
let old_active_file = engine.file_span(LogQueue::Append).1;
|
||||
engine.purge_manager().must_rewrite_append_queue(None, None);
|
||||
assert_eq!(engine.file_span(LogQueue::Append).0, old_active_file + 1);
|
||||
|
||||
for rid in 1..=3 {
|
||||
append(&engine, rid, 11, 16, Some(&data));
|
||||
}
|
||||
|
||||
{
|
||||
let _f = FailGuard::new("log_file::write::err", "10*off->return->off");
|
||||
assert!(
|
||||
catch_unwind_silent(|| engine.purge_manager().must_rewrite_rewrite_queue()).is_err()
|
||||
);
|
||||
}
|
||||
|
||||
drop(engine);
|
||||
let engine = Engine::open(cfg).unwrap();
|
||||
for rid in 1..=3 {
|
||||
assert_eq!(engine.first_index(rid).unwrap(), 1);
|
||||
assert_eq!(engine.last_index(rid).unwrap(), 15);
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_partial_rewrite_rewrite_online() {
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_partial_rewrite_rewrite_online")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let _f = FailGuard::new("max_rewrite_batch_bytes", "return(1)");
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
..Default::default()
|
||||
};
|
||||
let engine = Engine::open(cfg.clone()).unwrap();
|
||||
let data = vec![b'x'; 128];
|
||||
|
||||
for rid in 1..=3 {
|
||||
append(&engine, rid, 1, 5, Some(&data));
|
||||
append(&engine, rid, 5, 11, Some(&data));
|
||||
}
|
||||
|
||||
let old_active_file = engine.file_span(LogQueue::Append).1;
|
||||
engine.purge_manager().must_rewrite_append_queue(None, None);
|
||||
assert_eq!(engine.file_span(LogQueue::Append).0, old_active_file + 1);
|
||||
|
||||
{
|
||||
let _f = FailGuard::new("log_file::write::err", "10*off->return->off");
|
||||
assert!(
|
||||
catch_unwind_silent(|| engine.purge_manager().must_rewrite_rewrite_queue()).is_err()
|
||||
);
|
||||
}
|
||||
|
||||
for rid in 1..=3 {
|
||||
append(&engine, rid, 11, 16, Some(&data));
|
||||
}
|
||||
let old_active_file = engine.file_span(LogQueue::Append).1;
|
||||
engine.purge_manager().must_rewrite_append_queue(None, None);
|
||||
assert_eq!(engine.file_span(LogQueue::Append).0, old_active_file + 1);
|
||||
|
||||
drop(engine);
|
||||
let engine = Engine::open(cfg).unwrap();
|
||||
for rid in 1..=3 {
|
||||
assert_eq!(engine.first_index(rid).unwrap(), 1);
|
||||
assert_eq!(engine.last_index(rid).unwrap(), 15);
|
||||
}
|
||||
}
|
||||
|
||||
fn test_split_rewrite_batch_imp(regions: u64, region_size: u64, split_size: u64, file_size: u64) {
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_split_rewrite_batch")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let _f1 = FailGuard::new("max_rewrite_batch_bytes", &format!("return({split_size})"));
|
||||
let _f2 = FailGuard::new("force_use_atomic_group", "return");
|
||||
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
target_file_size: ReadableSize(file_size),
|
||||
batch_compression_threshold: ReadableSize(0),
|
||||
..Default::default()
|
||||
};
|
||||
let engine = Engine::open(cfg.clone()).unwrap();
|
||||
let data = vec![b'x'; region_size as usize / 10];
|
||||
|
||||
for rid in 1..=regions {
|
||||
append(&engine, rid, 1, 5, Some(&data));
|
||||
append(&engine, rid, 5, 11, Some(&data));
|
||||
}
|
||||
|
||||
let old_active_file = engine.file_span(LogQueue::Append).1;
|
||||
engine.purge_manager().must_rewrite_append_queue(None, None);
|
||||
assert_eq!(engine.file_span(LogQueue::Append).0, old_active_file + 1);
|
||||
|
||||
drop(engine);
|
||||
let engine = Engine::open(cfg.clone()).unwrap();
|
||||
for rid in 1..=regions {
|
||||
assert_eq!(engine.first_index(rid).unwrap(), 1);
|
||||
assert_eq!(engine.last_index(rid).unwrap(), 10);
|
||||
}
|
||||
|
||||
for rid in 1..=regions {
|
||||
append(&engine, rid, 11, 16, Some(&data));
|
||||
}
|
||||
let old_active_file = engine.file_span(LogQueue::Append).1;
|
||||
engine.purge_manager().must_rewrite_append_queue(None, None);
|
||||
assert_eq!(engine.file_span(LogQueue::Append).0, old_active_file + 1);
|
||||
drop(engine);
|
||||
|
||||
for i in 1..=10 {
|
||||
let engine = Engine::open(cfg.clone()).unwrap();
|
||||
let count = AtomicU64::new(0);
|
||||
fail::cfg_callback("atomic_group::begin", move || {
|
||||
if count.fetch_add(1, Ordering::Relaxed) + 1 == i {
|
||||
fail::cfg("log_file::write::err", "return").unwrap();
|
||||
}
|
||||
})
|
||||
.unwrap();
|
||||
let r = catch_unwind_silent(|| engine.purge_manager().must_rewrite_rewrite_queue());
|
||||
fail::remove("atomic_group::begin");
|
||||
fail::remove("log_file::write::err");
|
||||
if r.is_ok() {
|
||||
break;
|
||||
}
|
||||
}
|
||||
for i in 1..=10 {
|
||||
let engine = Engine::open(cfg.clone()).unwrap();
|
||||
for rid in 1..=regions {
|
||||
assert_eq!(engine.first_index(rid).unwrap(), 1);
|
||||
assert_eq!(engine.last_index(rid).unwrap(), 15);
|
||||
}
|
||||
let count = AtomicU64::new(0);
|
||||
fail::cfg_callback("atomic_group::add", move || {
|
||||
if count.fetch_add(1, Ordering::Relaxed) + 1 == i {
|
||||
fail::cfg("log_file::write::err", "return").unwrap();
|
||||
}
|
||||
})
|
||||
.unwrap();
|
||||
let r = catch_unwind_silent(|| engine.purge_manager().must_rewrite_rewrite_queue());
|
||||
fail::remove("atomic_group::add");
|
||||
fail::remove("log_file::write::err");
|
||||
if r.is_ok() {
|
||||
break;
|
||||
}
|
||||
}
|
||||
let engine = Engine::open(cfg).unwrap();
|
||||
for rid in 1..=regions {
|
||||
assert_eq!(engine.first_index(rid).unwrap(), 1);
|
||||
assert_eq!(engine.last_index(rid).unwrap(), 15);
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_split_rewrite_batch() {
|
||||
test_split_rewrite_batch_imp(10, 40960, 1, 1);
|
||||
test_split_rewrite_batch_imp(10, 40960, 1, 40960 * 2);
|
||||
test_split_rewrite_batch_imp(25, 4096, 6000, 40960 * 2);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_split_rewrite_batch_with_only_kvs() {
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_split_rewrite_batch_with_only_kvs")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let _f = FailGuard::new("max_rewrite_batch_bytes", "return(1)");
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
..Default::default()
|
||||
};
|
||||
let engine = Engine::open(cfg.clone()).unwrap();
|
||||
let mut log_batch = LogBatch::default();
|
||||
let key = vec![b'x'; 2];
|
||||
let value = vec![b'y'; 8];
|
||||
|
||||
let mut rid = 1;
|
||||
{
|
||||
log_batch.put(rid, key.clone(), Vec::new()).unwrap();
|
||||
engine.write(&mut log_batch, false).unwrap();
|
||||
engine.purge_manager().must_rewrite_append_queue(None, None);
|
||||
|
||||
log_batch.put(rid, key.clone(), value.clone()).unwrap();
|
||||
engine.write(&mut log_batch, false).unwrap();
|
||||
engine.purge_manager().must_rewrite_append_queue(None, None);
|
||||
|
||||
engine.purge_manager().must_rewrite_rewrite_queue();
|
||||
|
||||
rid += 1;
|
||||
log_batch.put(rid, key.clone(), value.clone()).unwrap();
|
||||
rid += 1;
|
||||
log_batch.put(rid, key.clone(), value.clone()).unwrap();
|
||||
engine.write(&mut log_batch, false).unwrap();
|
||||
engine.purge_manager().must_rewrite_append_queue(None, None);
|
||||
|
||||
engine.purge_manager().must_rewrite_rewrite_queue();
|
||||
}
|
||||
{
|
||||
let _f = FailGuard::new("force_use_atomic_group", "return");
|
||||
log_batch.put(rid, key.clone(), Vec::new()).unwrap();
|
||||
engine.write(&mut log_batch, false).unwrap();
|
||||
engine.purge_manager().must_rewrite_append_queue(None, None);
|
||||
|
||||
log_batch.put(rid, key.clone(), value.clone()).unwrap();
|
||||
engine.write(&mut log_batch, false).unwrap();
|
||||
engine.purge_manager().must_rewrite_append_queue(None, None);
|
||||
|
||||
engine.purge_manager().must_rewrite_rewrite_queue();
|
||||
|
||||
rid += 1;
|
||||
log_batch.put(rid, key.clone(), value.clone()).unwrap();
|
||||
rid += 1;
|
||||
log_batch.put(rid, key.clone(), value.clone()).unwrap();
|
||||
engine.write(&mut log_batch, false).unwrap();
|
||||
engine.purge_manager().must_rewrite_append_queue(None, None);
|
||||
|
||||
engine.purge_manager().must_rewrite_rewrite_queue();
|
||||
}
|
||||
|
||||
drop(engine);
|
||||
let engine = Engine::open(cfg).unwrap();
|
||||
for i in 1..=rid {
|
||||
assert_eq!(engine.get(i, &key).unwrap(), value);
|
||||
}
|
||||
}
|
||||
|
||||
// issue-315
|
||||
#[test]
|
||||
fn test_split_rewrite_batch_then_delete_some() {
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_split_rewrite_batch_then_delete_some")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let _f = FailGuard::new("max_rewrite_batch_bytes", "return(1)");
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
target_file_size: ReadableSize(1),
|
||||
..Default::default()
|
||||
};
|
||||
let mut log_batch = LogBatch::default();
|
||||
let value = vec![b'y'; 8];
|
||||
|
||||
let rid = 1;
|
||||
let engine = Engine::open(cfg.clone()).unwrap();
|
||||
for i in 0..=5 {
|
||||
append(&engine, rid, i * 2, i * 2 + 2, Some(&value));
|
||||
engine.purge_manager().must_rewrite_append_queue(None, None);
|
||||
}
|
||||
engine.purge_manager().must_rewrite_rewrite_queue();
|
||||
log_batch.add_command(rid, Command::Compact { index: 7 });
|
||||
log_batch.delete(rid, b"last_index".to_vec());
|
||||
engine.write(&mut log_batch, true).unwrap();
|
||||
engine.purge_manager().must_purge_all_stale();
|
||||
|
||||
drop(engine);
|
||||
let engine = Engine::open(cfg.clone()).unwrap();
|
||||
// The Compact mark is dropped during `must_purge_all_stale`.
|
||||
assert_eq!(engine.first_index(rid).unwrap(), 0);
|
||||
assert_eq!(engine.last_index(rid).unwrap(), 11);
|
||||
|
||||
// Removes all rewrite entries.
|
||||
log_batch.add_command(rid, Command::Compact { index: 100 });
|
||||
engine.write(&mut log_batch, false).unwrap();
|
||||
append(&engine, rid, 5, 11, Some(&value));
|
||||
engine.purge_manager().must_rewrite_append_queue(None, None);
|
||||
engine.purge_manager().must_purge_all_stale();
|
||||
drop(engine);
|
||||
let engine = Engine::open(cfg).unwrap();
|
||||
assert_eq!(engine.first_index(rid).unwrap(), 5);
|
||||
assert_eq!(engine.last_index(rid).unwrap(), 10);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_build_engine_with_recycling_and_multi_dirs() {
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_build_engine_with_multi_dirs_main")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let spill_dir = tempfile::Builder::new()
|
||||
.prefix("test_build_engine_with_multi_dirs_spill")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
spill_dir: Some(spill_dir.path().to_str().unwrap().to_owned()),
|
||||
target_file_size: ReadableSize::kb(1),
|
||||
purge_threshold: ReadableSize::kb(20),
|
||||
enable_log_recycle: true,
|
||||
prefill_for_recycle: true,
|
||||
..Default::default()
|
||||
};
|
||||
let data = vec![b'x'; 1024];
|
||||
{
|
||||
// Prerequisite - case 1: all disks are full, Engine can be opened normally.
|
||||
{
|
||||
// Multi directories.
|
||||
let _f = FailGuard::new("file_pipe_log::force_choose_dir", "return");
|
||||
Engine::open(cfg.clone()).unwrap();
|
||||
// Single diretory - spill-dir is None.
|
||||
let cfg_single_dir = Config {
|
||||
spill_dir: None,
|
||||
..cfg.clone()
|
||||
};
|
||||
Engine::open(cfg_single_dir).unwrap();
|
||||
}
|
||||
// Prerequisite - case 2: all disks are full after writing, and the current
|
||||
// engine should be available for `read`.
|
||||
{
|
||||
let cfg_no_prefill = Config {
|
||||
prefill_for_recycle: false,
|
||||
..cfg.clone()
|
||||
};
|
||||
let engine = Engine::open(cfg_no_prefill.clone()).unwrap();
|
||||
engine
|
||||
.write(&mut generate_batch(101, 11, 21, Some(&data)), true)
|
||||
.unwrap();
|
||||
drop(engine);
|
||||
let _f1 = FailGuard::new("file_pipe_log::force_choose_dir", "return");
|
||||
let _f2 = FailGuard::new("log_fd::write::no_space_err", "return");
|
||||
let engine = Engine::open(cfg_no_prefill).unwrap();
|
||||
assert_eq!(
|
||||
10,
|
||||
engine
|
||||
.fetch_entries_to::<MessageExtTyped>(101, 11, 21, None, &mut vec![])
|
||||
.unwrap()
|
||||
);
|
||||
}
|
||||
// Prerequisite - case 3: prefill several recycled logs but no space for
|
||||
// remains, making prefilling progress exit in advance.
|
||||
{
|
||||
let _f1 = FailGuard::new(
|
||||
"file_pipe_log::force_choose_dir",
|
||||
"10*return(0)->5*return(1)",
|
||||
);
|
||||
let _f2 = FailGuard::new("log_fd::write::no_space_err", "return");
|
||||
let _ = Engine::open(cfg.clone()).unwrap();
|
||||
}
|
||||
// Clean-up the env for later testing.
|
||||
let cfg_err = Config {
|
||||
enable_log_recycle: false,
|
||||
prefill_for_recycle: false,
|
||||
..cfg.clone()
|
||||
};
|
||||
let _ = Engine::open(cfg_err).unwrap();
|
||||
}
|
||||
{
|
||||
// Case 1: prefill recycled logs into multi-dirs (when preparing recycled logs,
|
||||
// this circumstance also equals to `main dir is full, but spill-dir
|
||||
// is free`.)
|
||||
let engine = {
|
||||
let _f = FailGuard::new("file_pipe_log::force_choose_dir", "10*return(0)->return(1)");
|
||||
Engine::open(cfg.clone()).unwrap()
|
||||
};
|
||||
for rid in 1..10 {
|
||||
append(&engine, rid, 1, 5, Some(&data));
|
||||
}
|
||||
let append_first = engine.file_span(LogQueue::Append).0;
|
||||
for rid in 1..10 {
|
||||
engine.compact_to(rid, 3);
|
||||
}
|
||||
// Purge do not exceed purge_threshold, and first active file_seq won't change.
|
||||
engine.purge_expired_files().unwrap();
|
||||
assert_eq!(engine.file_span(LogQueue::Append).0, append_first);
|
||||
for rid in 1..20 {
|
||||
append(&engine, rid, 3, 5, Some(&data));
|
||||
engine.compact_to(rid, 4);
|
||||
}
|
||||
// Purge obsolete logs.
|
||||
engine.purge_expired_files().unwrap();
|
||||
assert!(engine.file_span(LogQueue::Append).0 > append_first);
|
||||
}
|
||||
{
|
||||
// Case 2: prefill is on but no spare space for new log files.
|
||||
let _f = FailGuard::new("file_pipe_log::force_choose_dir", "return");
|
||||
let engine = Engine::open(cfg.clone()).unwrap();
|
||||
let append_end = engine.file_span(LogQueue::Append).1;
|
||||
// As there still exists several recycled logs for incoming writes, so the
|
||||
// following writes will success.
|
||||
for rid in 1..10 {
|
||||
append(&engine, rid, 5, 7, Some(&data));
|
||||
}
|
||||
assert!(engine.file_span(LogQueue::Append).1 > append_end);
|
||||
}
|
||||
{
|
||||
// Case 3: no prefill and no spare space for new log files.
|
||||
let cfg_no_prefill = Config {
|
||||
enable_log_recycle: true,
|
||||
prefill_for_recycle: false,
|
||||
..cfg
|
||||
};
|
||||
let _f1 = FailGuard::new("file_pipe_log::force_choose_dir", "return");
|
||||
let engine = Engine::open(cfg_no_prefill).unwrap();
|
||||
let _f2 = FailGuard::new("log_fd::write::no_space_err", "return");
|
||||
let (append_first, append_end) = engine.file_span(LogQueue::Append);
|
||||
// Cannot append new data into engine as no spare space.
|
||||
for rid in 1..20 {
|
||||
assert!(catch_unwind_silent(|| append(&engine, rid, 8, 9, Some(&data))).is_err());
|
||||
}
|
||||
assert_eq!(
|
||||
engine.file_span(LogQueue::Append),
|
||||
(append_first, append_end)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -2,6 +2,7 @@
|
|||
|
||||
use std::sync::Arc;
|
||||
|
||||
use fail::FailGuard;
|
||||
use raft::eraftpb::Entry;
|
||||
use raft_engine::env::ObfuscatedFileSystem;
|
||||
use raft_engine::internals::*;
|
||||
|
@ -22,12 +23,11 @@ fn test_file_open_error() {
|
|||
let fs = Arc::new(ObfuscatedFileSystem::default());
|
||||
|
||||
{
|
||||
let _f = FailGuard::new("log_fd::create::err", "return");
|
||||
let _f = FailGuard::new("default_fs::create::err", "return");
|
||||
assert!(Engine::open_with_file_system(cfg.clone(), fs.clone()).is_err());
|
||||
}
|
||||
|
||||
{
|
||||
let _f = FailGuard::new("log_fd::open::err", "return");
|
||||
let _f = FailGuard::new("default_fs::open::err", "return");
|
||||
let _ = Engine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
|
||||
assert!(Engine::open_with_file_system(cfg, fs).is_err());
|
||||
}
|
||||
|
@ -66,7 +66,7 @@ fn test_file_read_error() {
|
|||
engine.write(&mut kv_batch, true).unwrap();
|
||||
|
||||
let mut entries = Vec::new();
|
||||
let _f = FailGuard::new("log_fd::read::err", "return");
|
||||
let _f = FailGuard::new("log_file::read::err", "return");
|
||||
engine
|
||||
.fetch_entries_to::<MessageExtTyped>(1, 0, 1, None, &mut entries)
|
||||
.unwrap();
|
||||
|
@ -84,7 +84,6 @@ fn test_file_write_error() {
|
|||
.unwrap();
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
bytes_per_sync: ReadableSize::kb(1024),
|
||||
target_file_size: ReadableSize::kb(1024),
|
||||
..Default::default()
|
||||
};
|
||||
|
@ -96,7 +95,7 @@ fn test_file_write_error() {
|
|||
.write(&mut generate_batch(1, 1, 2, Some(&entry)), false)
|
||||
.unwrap();
|
||||
{
|
||||
let _f = FailGuard::new("log_fd::write::err", "return");
|
||||
let _f = FailGuard::new("log_file::write::err", "return");
|
||||
engine
|
||||
.write(&mut generate_batch(1, 2, 3, Some(&entry)), false)
|
||||
.unwrap_err();
|
||||
|
@ -125,22 +124,20 @@ fn test_file_write_error() {
|
|||
assert_eq!(engine.last_index(2).unwrap(), 1);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_file_rotate_error() {
|
||||
fn test_file_rotate_error(restart_after_failure: bool) {
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_file_rotate_error")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
bytes_per_sync: ReadableSize::kb(1024),
|
||||
target_file_size: ReadableSize::kb(4),
|
||||
..Default::default()
|
||||
};
|
||||
let fs = Arc::new(ObfuscatedFileSystem::default());
|
||||
let entry = vec![b'x'; 1024];
|
||||
|
||||
let engine = Engine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
|
||||
let mut engine = Engine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
|
||||
engine
|
||||
.write(&mut generate_batch(1, 1, 2, Some(&entry)), false)
|
||||
.unwrap();
|
||||
|
@ -150,6 +147,9 @@ fn test_file_rotate_error() {
|
|||
engine
|
||||
.write(&mut generate_batch(1, 3, 4, Some(&entry)), false)
|
||||
.unwrap();
|
||||
engine
|
||||
.write(&mut generate_batch(1, 4, 5, Some(&entry)), false)
|
||||
.unwrap();
|
||||
assert_eq!(engine.file_span(LogQueue::Append).1, 1);
|
||||
// The next write will be followed by a rotate.
|
||||
{
|
||||
|
@ -159,27 +159,46 @@ fn test_file_rotate_error() {
|
|||
let _ = engine.write(&mut generate_batch(1, 4, 5, Some(&entry)), false);
|
||||
})
|
||||
.is_err());
|
||||
assert_eq!(engine.file_span(LogQueue::Append).1, 1);
|
||||
}
|
||||
if restart_after_failure {
|
||||
drop(engine);
|
||||
engine = Engine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
|
||||
}
|
||||
assert_eq!(engine.file_span(LogQueue::Append).1, 1);
|
||||
{
|
||||
// Fail to create new log file.
|
||||
let _f = FailGuard::new("log_fd::create::err", "return");
|
||||
assert!(catch_unwind_silent(|| {
|
||||
let _ = engine.write(&mut generate_batch(1, 4, 5, Some(&entry)), false);
|
||||
})
|
||||
.is_err());
|
||||
assert_eq!(engine.file_span(LogQueue::Append).1, 1);
|
||||
let _f = FailGuard::new("default_fs::create::err", "return");
|
||||
assert!(engine
|
||||
.write(&mut generate_batch(1, 4, 5, Some(&entry)), false)
|
||||
.is_err());
|
||||
}
|
||||
if restart_after_failure {
|
||||
drop(engine);
|
||||
engine = Engine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
|
||||
}
|
||||
let num_files_before = std::fs::read_dir(&dir).unwrap().count();
|
||||
{
|
||||
// Fail to write header of new log file.
|
||||
let _f = FailGuard::new("log_fd::write::err", "1*off->return");
|
||||
assert!(catch_unwind_silent(|| {
|
||||
let _ = engine.write(&mut generate_batch(1, 4, 5, Some(&entry)), false);
|
||||
})
|
||||
.is_err());
|
||||
let _f = FailGuard::new("log_file::write::err", "1*off->return");
|
||||
assert!(engine
|
||||
.write(&mut generate_batch(1, 4, 5, Some(&entry)), false)
|
||||
.is_err());
|
||||
}
|
||||
if restart_after_failure {
|
||||
drop(engine);
|
||||
engine = Engine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
|
||||
// The new log file is added during recovery phase of restart.
|
||||
assert_eq!(engine.file_span(LogQueue::Append).1, 2);
|
||||
} else {
|
||||
assert_eq!(engine.file_span(LogQueue::Append).1, 1);
|
||||
}
|
||||
{
|
||||
// Although the header is not written, the file is still created.
|
||||
assert_eq!(
|
||||
std::fs::read_dir(&dir).unwrap().count() - num_files_before,
|
||||
1
|
||||
);
|
||||
if !restart_after_failure {
|
||||
// If the engine restarted, the write does not require sync will succeed.
|
||||
// Fail to sync new log file. The old log file is already sync-ed at this point.
|
||||
let _f = FailGuard::new("log_fd::sync::err", "return");
|
||||
assert!(catch_unwind_silent(|| {
|
||||
|
@ -189,18 +208,39 @@ fn test_file_rotate_error() {
|
|||
assert_eq!(engine.file_span(LogQueue::Append).1, 1);
|
||||
}
|
||||
|
||||
// Only one log file should be created after all the incidents.
|
||||
assert_eq!(
|
||||
std::fs::read_dir(&dir).unwrap().count() - num_files_before,
|
||||
1
|
||||
);
|
||||
// We can continue writing after the incidents.
|
||||
engine
|
||||
.write(&mut generate_batch(2, 1, 2, Some(&entry)), true)
|
||||
.unwrap();
|
||||
drop(engine);
|
||||
let engine = Engine::open_with_file_system(cfg, fs).unwrap();
|
||||
if restart_after_failure {
|
||||
drop(engine);
|
||||
engine = Engine::open_with_file_system(cfg, fs).unwrap();
|
||||
}
|
||||
assert_eq!(
|
||||
std::fs::read_dir(&dir).unwrap().count() - num_files_before,
|
||||
1
|
||||
);
|
||||
assert_eq!(engine.first_index(1).unwrap(), 1);
|
||||
assert_eq!(engine.last_index(1).unwrap(), 4);
|
||||
assert_eq!(engine.first_index(2).unwrap(), 1);
|
||||
assert_eq!(engine.last_index(2).unwrap(), 1);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_file_rotate_error_without_restart() {
|
||||
test_file_rotate_error(false);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_file_rotate_error_with_restart() {
|
||||
test_file_rotate_error(true);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_concurrent_write_error() {
|
||||
let dir = tempfile::Builder::new()
|
||||
|
@ -209,7 +249,6 @@ fn test_concurrent_write_error() {
|
|||
.unwrap();
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
bytes_per_sync: ReadableSize::kb(1024),
|
||||
target_file_size: ReadableSize::kb(1024),
|
||||
..Default::default()
|
||||
};
|
||||
|
@ -220,7 +259,7 @@ fn test_concurrent_write_error() {
|
|||
let mut ctx = ConcurrentWriteContext::new(engine.clone());
|
||||
|
||||
// The second of three writes will fail.
|
||||
fail::cfg("log_fd::write::err", "1*off->1*return->off").unwrap();
|
||||
fail::cfg("log_file::write::err", "1*off->1*return->off").unwrap();
|
||||
let entry_clone = entry.clone();
|
||||
ctx.write_ext(move |e| {
|
||||
e.write(&mut generate_batch(1, 1, 11, Some(&entry_clone)), false)
|
||||
|
@ -258,14 +297,12 @@ fn test_concurrent_write_error() {
|
|||
);
|
||||
|
||||
{
|
||||
let _f1 = FailGuard::new("log_fd::write::err", "return");
|
||||
let _f2 = FailGuard::new("log_fd::truncate::err", "return");
|
||||
let _f1 = FailGuard::new("log_file::write::err", "return");
|
||||
let _f2 = FailGuard::new("log_file::truncate::err", "return");
|
||||
let entry_clone = entry.clone();
|
||||
ctx.write_ext(move |e| {
|
||||
catch_unwind_silent(|| {
|
||||
e.write(&mut generate_batch(1, 11, 21, Some(&entry_clone)), false)
|
||||
})
|
||||
.unwrap_err();
|
||||
e.write(&mut generate_batch(1, 11, 21, Some(&entry_clone)), false)
|
||||
.unwrap_err();
|
||||
});
|
||||
// We don't test followers, their panics are hard to catch.
|
||||
ctx.join();
|
||||
|
@ -287,6 +324,67 @@ fn test_concurrent_write_error() {
|
|||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_non_atomic_write_error() {
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_non_atomic_write_error")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
target_file_size: ReadableSize::kb(1024),
|
||||
..Default::default()
|
||||
};
|
||||
let fs = Arc::new(ObfuscatedFileSystem::default());
|
||||
let entry = vec![b'x'; 1024];
|
||||
let rid = 1;
|
||||
|
||||
{
|
||||
// Write partially succeeds. We can reopen.
|
||||
let engine = Engine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
|
||||
let _f1 = FailGuard::new("log_file::write::err", "return");
|
||||
engine
|
||||
.write(&mut generate_batch(rid, 0, 1, Some(&entry)), true)
|
||||
.unwrap_err();
|
||||
}
|
||||
{
|
||||
let engine = Engine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
|
||||
assert_eq!(engine.first_index(rid), None);
|
||||
}
|
||||
{
|
||||
// Write partially succeeds. We can overwrite.
|
||||
let engine = Engine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
|
||||
let _f1 = FailGuard::new("log_file::write::err", "1*off->1*return->off");
|
||||
engine
|
||||
.write(&mut generate_batch(rid, 0, 1, Some(&entry)), true)
|
||||
.unwrap_err();
|
||||
engine
|
||||
.write(&mut generate_batch(rid, 5, 6, Some(&entry)), true)
|
||||
.unwrap();
|
||||
assert_eq!(engine.first_index(rid).unwrap(), 5);
|
||||
}
|
||||
{
|
||||
let engine = Engine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
|
||||
assert_eq!(engine.first_index(rid).unwrap(), 5);
|
||||
}
|
||||
{
|
||||
// Write partially succeeds and can't be reverted. We panic.
|
||||
let engine = Engine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
|
||||
let _f1 = FailGuard::new("log_file::write::err", "return");
|
||||
let _f2 = FailGuard::new("log_file::seek::err", "return");
|
||||
assert!(catch_unwind_silent(|| {
|
||||
engine
|
||||
.write(&mut generate_batch(rid, 6, 7, Some(&entry)), true)
|
||||
.unwrap_err();
|
||||
})
|
||||
.is_err());
|
||||
}
|
||||
{
|
||||
let engine = Engine::open_with_file_system(cfg, fs).unwrap();
|
||||
assert_eq!(engine.last_index(rid), Some(5));
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(feature = "scripting")]
|
||||
#[test]
|
||||
fn test_error_during_repair() {
|
||||
|
@ -317,7 +415,7 @@ fn test_error_during_repair() {
|
|||
"
|
||||
.to_owned();
|
||||
{
|
||||
let _f = FailGuard::new("log_fd::write::err", "return");
|
||||
let _f = FailGuard::new("log_file::write::err", "return");
|
||||
assert!(
|
||||
Engine::unsafe_repair_with_file_system(dir.path(), None, script, fs.clone()).is_err()
|
||||
);
|
||||
|
@ -353,3 +451,207 @@ fn test_swappy_page_create_error() {
|
|||
vec.resize(1024, 0);
|
||||
assert_eq!(allocator.memory_usage(), 0);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_file_allocate_error() {
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_file_allocate_error")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
target_file_size: ReadableSize::mb(100),
|
||||
..Default::default()
|
||||
};
|
||||
let fs = Arc::new(ObfuscatedFileSystem::default());
|
||||
let entry = vec![b'x'; 1024];
|
||||
{
|
||||
let _f = FailGuard::new("log_file::allocate::err", "return");
|
||||
let engine = Engine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
|
||||
engine
|
||||
.write(&mut generate_batch(1, 1, 5, Some(&entry)), true)
|
||||
.unwrap();
|
||||
}
|
||||
let engine = Engine::open_with_file_system(cfg, fs).unwrap();
|
||||
assert_eq!(engine.first_index(1).unwrap(), 1);
|
||||
assert_eq!(engine.last_index(1).unwrap(), 4);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_start_with_recycled_file_allocate_error() {
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_start_with_recycled_file_allocate_error")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let cfg = Config {
|
||||
dir: dir.path().to_str().unwrap().to_owned(),
|
||||
target_file_size: ReadableSize::kb(1),
|
||||
purge_threshold: ReadableSize::kb(10), // capacity is 12
|
||||
enable_log_recycle: true,
|
||||
prefill_for_recycle: true,
|
||||
..Default::default()
|
||||
};
|
||||
let entry = vec![b'x'; 1024];
|
||||
// Mock that the engine starts with the circumstance where
|
||||
// the pref-reserved file with seqno[5] failed to be generated.
|
||||
{
|
||||
let _f = FailGuard::new("log_file::write::zero", "4*off->1*return->off");
|
||||
Engine::open(cfg.clone()).unwrap();
|
||||
}
|
||||
// Extra recycled files have been supplemented.
|
||||
let engine = Engine::open(cfg).unwrap();
|
||||
engine
|
||||
.write(&mut generate_batch(1, 1, 5, Some(&entry)), true)
|
||||
.unwrap();
|
||||
let (start, end) = engine.file_span(LogQueue::Append);
|
||||
assert_eq!(start, end);
|
||||
// Append several entries to make Engine reuse the recycled logs.
|
||||
for r in 2..6 {
|
||||
engine
|
||||
.write(&mut generate_batch(r, 1, 5, Some(&entry)), true)
|
||||
.unwrap();
|
||||
}
|
||||
let (reused_start, reused_end) = engine.file_span(LogQueue::Append);
|
||||
assert_eq!((reused_start, reused_end), (1, 5));
|
||||
assert!(reused_end > end);
|
||||
assert_eq!(engine.first_index(1).unwrap(), 1);
|
||||
assert_eq!(engine.last_index(1).unwrap(), 4);
|
||||
assert_eq!(engine.last_index(5).unwrap(), 4);
|
||||
let mut entries = Vec::new();
|
||||
engine
|
||||
.fetch_entries_to::<MessageExtTyped>(5, 1, 5, None, &mut entries)
|
||||
.unwrap();
|
||||
// Continously append entries to reach the purge_threshold.
|
||||
for r in 6..=15 {
|
||||
engine
|
||||
.write(&mut generate_batch(r, 1, 5, Some(&entry)), true)
|
||||
.unwrap();
|
||||
}
|
||||
assert_eq!(engine.file_span(LogQueue::Append).0, reused_start);
|
||||
assert!(engine.file_span(LogQueue::Append).1 > reused_end);
|
||||
let (start, _) = engine.file_span(LogQueue::Append);
|
||||
// Purge and check.
|
||||
engine.purge_expired_files().unwrap();
|
||||
assert!(engine.file_span(LogQueue::Append).0 > start);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_no_space_write_error() {
|
||||
let mut cfg_list = [
|
||||
Config {
|
||||
target_file_size: ReadableSize::kb(2),
|
||||
format_version: Version::V1,
|
||||
enable_log_recycle: false,
|
||||
..Default::default()
|
||||
},
|
||||
Config {
|
||||
target_file_size: ReadableSize::kb(2),
|
||||
format_version: Version::V2,
|
||||
enable_log_recycle: true,
|
||||
..Default::default()
|
||||
},
|
||||
];
|
||||
let entry = vec![b'x'; 1024];
|
||||
for cfg in cfg_list.iter_mut() {
|
||||
let dir = tempfile::Builder::new()
|
||||
.prefix("test_no_space_write_error_main")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
let spill_dir = tempfile::Builder::new()
|
||||
.prefix("test_no_space_write_error_spill")
|
||||
.tempdir()
|
||||
.unwrap();
|
||||
cfg.dir = dir.path().to_str().unwrap().to_owned();
|
||||
cfg.spill_dir = Some(spill_dir.path().to_str().unwrap().to_owned());
|
||||
{
|
||||
// Case 1: `Write` is abnormal for no space left, Engine should fail at
|
||||
// `rotate`.
|
||||
let cfg_err = Config {
|
||||
target_file_size: ReadableSize(1),
|
||||
..cfg.clone()
|
||||
};
|
||||
let engine = Engine::open(cfg_err).unwrap();
|
||||
let _f = FailGuard::new("log_fd::write::no_space_err", "return");
|
||||
assert!(engine
|
||||
.write(&mut generate_batch(2, 11, 21, Some(&entry)), true)
|
||||
.is_err());
|
||||
assert_eq!(
|
||||
0,
|
||||
engine
|
||||
.fetch_entries_to::<MessageExtTyped>(2, 11, 21, None, &mut vec![])
|
||||
.unwrap()
|
||||
);
|
||||
}
|
||||
{
|
||||
let engine = Engine::open(cfg.clone()).unwrap();
|
||||
// Case 2: disk goes from `full(nospace err)` -> `spare for writing`.
|
||||
let _f1 = FailGuard::new("log_fd::write::no_space_err", "2*return->off");
|
||||
let _f2 = FailGuard::new("file_pipe_log::force_choose_dir", "return");
|
||||
// The first write should fail, because all dirs run out of space for writing.
|
||||
assert!(engine
|
||||
.write(&mut generate_batch(2, 11, 21, Some(&entry)), true)
|
||||
.is_err());
|
||||
assert_eq!(
|
||||
0,
|
||||
engine
|
||||
.fetch_entries_to::<MessageExtTyped>(2, 11, 21, None, &mut vec![])
|
||||
.unwrap()
|
||||
);
|
||||
// The second write should success, as there exists free space for later writing
|
||||
// after cleaning up.
|
||||
engine
|
||||
.write(&mut generate_batch(3, 11, 21, Some(&entry)), true)
|
||||
.unwrap();
|
||||
assert_eq!(
|
||||
10,
|
||||
engine
|
||||
.fetch_entries_to::<MessageExtTyped>(3, 11, 21, None, &mut vec![])
|
||||
.unwrap()
|
||||
);
|
||||
}
|
||||
{
|
||||
// Case 3: disk status -- `main dir is full (has nospace err)` -> `spill-dir
|
||||
// is spare (has enough space)`.
|
||||
let engine = Engine::open(cfg.clone()).unwrap();
|
||||
let _f1 = FailGuard::new("log_fd::write::no_space_err", "1*return->off");
|
||||
let _f2 = FailGuard::new("file_pipe_log::force_choose_dir", "return(1)");
|
||||
engine
|
||||
.write(&mut generate_batch(5, 11, 21, Some(&entry)), true)
|
||||
.unwrap();
|
||||
engine
|
||||
.write(&mut generate_batch(6, 11, 21, Some(&entry)), true)
|
||||
.unwrap();
|
||||
assert_eq!(
|
||||
10,
|
||||
engine
|
||||
.fetch_entries_to::<MessageExtTyped>(5, 11, 21, None, &mut vec![])
|
||||
.unwrap()
|
||||
);
|
||||
assert_eq!(
|
||||
10,
|
||||
engine
|
||||
.fetch_entries_to::<MessageExtTyped>(6, 11, 21, None, &mut vec![])
|
||||
.unwrap()
|
||||
);
|
||||
}
|
||||
{
|
||||
// Case 4: disk status -- `main dir has free space for rotating new files
|
||||
// but no space for dumping LogBatch`, disk goes into endless `spare(nospace
|
||||
// err)`, engine do panic for multi-retrying.
|
||||
let engine = Engine::open(cfg.clone()).unwrap();
|
||||
let _f = FailGuard::new(
|
||||
"log_fd::write::no_space_err",
|
||||
"1*return->1*off->1*return->1*off",
|
||||
);
|
||||
assert!(engine
|
||||
.write(&mut generate_batch(7, 11, 21, Some(&entry)), true)
|
||||
.is_err());
|
||||
assert_eq!(
|
||||
0,
|
||||
engine
|
||||
.fetch_entries_to::<MessageExtTyped>(7, 11, 21, None, &mut vec![])
|
||||
.unwrap()
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -55,23 +55,6 @@ where
|
|||
result
|
||||
}
|
||||
|
||||
pub struct FailGuard {
|
||||
fp: String,
|
||||
}
|
||||
|
||||
impl FailGuard {
|
||||
pub fn new(fp: &str, action: &str) -> Self {
|
||||
fail::cfg(fp, action).unwrap();
|
||||
Self { fp: fp.to_owned() }
|
||||
}
|
||||
}
|
||||
|
||||
impl Drop for FailGuard {
|
||||
fn drop(&mut self) {
|
||||
fail::remove(&self.fp);
|
||||
}
|
||||
}
|
||||
|
||||
pub struct ConcurrentWriteContext<FS: 'static + FileSystem> {
|
||||
engine: Arc<Engine<FS>>,
|
||||
ths: Vec<std::thread::JoinHandle<()>>,
|
||||
|
|
Loading…
Reference in New Issue