diff --git a/checksum.zig b/checksum.zig new file mode 100644 index 0000000..4cae35a --- /dev/null +++ b/checksum.zig @@ -0,0 +1,205 @@ +//! This file implements vsr.checksum. TigerBeetle uses this checksum to: +//! +//! - detect bitrot in data on disk, +//! - validate network messages before casting raw bytes to an `extern struct` type, +//! - hash-chain prepares and client requests to have strong consistency and ordering guarantees. +//! +//! As this checksum is stored on disk, it is set in stone and impossible to change. +//! +//! We need this checksum to be fast (it's in all our hotpaths) and strong (it's our ultimate line +//! of defense against storage failures and some classes of software bugs). +//! +//! Our checksum of choice is based on Aegis: +//! +//! +//! +//! We use the implementation from the Zig standard library, but here's the overall overview of the +//! thing works: +//! +//! - AES-block is a widely supported in hardware symmetric encryption primitive (`vaesenc`, +//! `vaesdec` instructions). Hardware acceleration is what provides speed. +//! - Aegis is an modern Authenticated Encryption with Associated Data (AEAD) scheme based on +//! AES-block. +//! - In AEAD, the user provides, a key, a nonce, a secret message, and associated data, and gets +//! a ciphertext and an authentication tag back. Associated data is expected to be sent as plain +//! text (eg, it could be routing information). The tag authenticates _both_ the secret message +//! and associated data. +//! - AEAD can be specialized to be a MAC by using an empty secret message and zero nonce. NB: +//! in mac mode, message to sign is treated as AD, not as a secret message. +//! - A MAC can further be specialized to be a checksum by setting the secret key to zero. +//! And that's what we do here! + +const std = @import("std"); +const builtin = @import("builtin"); +const mem = std.mem; +const testing = std.testing; +const assert = std.debug.assert; + +const Aegis128LMac_128 = std.crypto.auth.aegis.Aegis128LMac_128; + +var seed_once = std.once(seed_init); +var seed_state: Aegis128LMac_128 = undefined; + +comptime { + // As described above, TigerBeetle uses Aegis (and thus AES Blocks), for its checksumming. + // While there is a software implementation, it's much slower and we don't expect to ever be + // using it considering we target platforms with AES hardware acceleration. + // + // If you're trying to compile TigerBeetle for an older CPU without AES hardware acceleration, + // you'll need to disable the following assert. + assert(std.crypto.core.aes.has_hardware_support); +} + +fn seed_init() void { + const key = mem.zeroes([16]u8); + seed_state = Aegis128LMac_128.init(&key); +} + +// Lazily initialize the Aegis State instead of recomputing it on each call to checksum(). +// Then, make a copy of the state and use that to hash the source input bytes. +pub fn checksum(source: []const u8) u128 { + if (@inComptime()) { + // Aegis128 uses hardware accelerated AES via inline asm which isn't available at comptime. + // Use a hard-coded value instead and verify via a test. + if (source.len == 0) return 0x49F174618255402DE6E7E3C40D60CC83; + } + var stream = ChecksumStream.init(); + stream.add(source); + return stream.checksum(); +} + +test "checksum empty" { + var stream = ChecksumStream.init(); + stream.add(&.{}); + try std.testing.expectEqual(stream.checksum(), comptime checksum(&.{})); +} + +pub const ChecksumStream = struct { + state: Aegis128LMac_128, + + pub fn init() ChecksumStream { + seed_once.call(); + return ChecksumStream{ .state = seed_state }; + } + + pub fn add(stream: *ChecksumStream, bytes: []const u8) void { + stream.state.update(bytes); + } + + pub fn checksum(stream: *ChecksumStream) u128 { + var result: u128 = undefined; + stream.state.final(mem.asBytes(&result)); + stream.* = undefined; + return result; + } +}; + +// Note: these test vectors are not independent --- there are test vectors in AEAD papers, but they +// don't zero all of (nonce, key, secret message). However, the as underlying AEAD implementation +// matches those test vectors, the entries here are correct. +// +// They can be used to smoke-test independent implementations of TigerBeetle checksum. +// +// "checksum stability" test further nails down the exact behavior. +test "checksum test vectors" { + const TestVector = struct { + source: []const u8, + hash: u128, + }; + + for (&[_]TestVector{ + .{ + .source = &[_]u8{0x00} ** 16, + .hash = @byteSwap(@as(u128, 0xf72ad48dd05dd1656133101cd4be3a26)), + }, + .{ + .source = &[_]u8{}, + .hash = @byteSwap(@as(u128, 0x83cc600dc4e3e7e62d4055826174f149)), + }, + }) |test_vector| { + try testing.expectEqual(test_vector.hash, checksum(test_vector.source)); + } +} + +test "checksum simple fuzzing" { + var prng = std.rand.DefaultPrng.init(42); + + const msg_min = 1; + const msg_max = 1 * 1024 * 1024; + + var msg_buf = try testing.allocator.alloc(u8, msg_max); + defer testing.allocator.free(msg_buf); + + const cipher_buf = try testing.allocator.alloc(u8, msg_max); + defer testing.allocator.free(cipher_buf); + + var i: usize = 0; + while (i < 1_000) : (i += 1) { + const msg_len = prng.random().intRangeAtMostBiased(usize, msg_min, msg_max); + const msg = msg_buf[0..msg_len]; + prng.fill(msg); + + const msg_checksum = checksum(msg); + + // Sanity check that it's a pure function. + const msg_checksum_again = checksum(msg); + try testing.expectEqual(msg_checksum, msg_checksum_again); + + // Change the message and make sure the checksum changes. + msg[prng.random().uintLessThan(usize, msg.len)] +%= 1; + const changed_checksum = checksum(msg); + try testing.expect(changed_checksum != msg_checksum); + } +} + +// Change detector test to ensure we don't inadvertency modify our checksum function. +test "checksum stability" { + var buf: [1024]u8 = undefined; + var cases: [896]u128 = undefined; + var case_index: usize = 0; + + // Zeros of various lengths. + var subcase: usize = 0; + while (subcase < 128) : (subcase += 1) { + const message = buf[0..subcase]; + @memset(message, 0); + + cases[case_index] = checksum(message); + case_index += 1; + } + + // 64 bytes with exactly one bit set. + subcase = 0; + while (subcase < 64 * 8) : (subcase += 1) { + const message = buf[0..64]; + @memset(message, 0); + message[@divFloor(subcase, 8)] = @shlExact(@as(u8, 1), @as(u3, @intCast(subcase % 8))); + + cases[case_index] = checksum(message); + case_index += 1; + } + + // Pseudo-random data from a specific PRNG of various lengths. + var prng = std.rand.Xoshiro256.init(92); + subcase = 0; + while (subcase < 256) : (subcase += 1) { + const message = buf[0 .. subcase + 13]; + prng.fill(message); + + cases[case_index] = checksum(message); + case_index += 1; + } + + // Sanity check that we are not getting trivial answers. + for (cases, 0..) |case_a, i| { + assert(case_a != 0); + assert(case_a != std.math.maxInt(u128)); + for (cases[0..i]) |case_b| assert(case_a != case_b); + } + + // Hash me, baby, one more time! If this final hash changes, we broke compatibility in a major + // way. + comptime assert(builtin.target.cpu.arch.endian() == .little); + const hash = checksum(mem.sliceAsBytes(&cases)); + try testing.expectEqual(hash, 0x82dcaacf4875b279446825b6830d1263); +} diff --git a/constants.zig b/constants.zig index e2e71e2..cd3682c 100644 --- a/constants.zig +++ b/constants.zig @@ -16,3 +16,28 @@ pub const sector_size = 4096; /// when they were never written to disk. pub const direct_io = true; pub const direct_io_required = true; + +/// The number of milliseconds between each replica tick, the basic unit of time in TigerBeetle. +/// Used to regulate heartbeats, retries and timeouts, all specified as multiples of a tick. +pub const tick_ms = 10; + +/// TigerBeetle uses asserts proactively, unless they severely degrade performance. For production, +/// 5% slow down might be deemed critical, tests tolerate slowdowns up to 5x. Tests should be +/// reasonably fast to make deterministic simulation effective. `constants.verify` disambiguate the +/// two cases. +/// +/// In the control plane (eg, vsr proper) assert unconditionally. Due to batching, control plane +/// overhead is negligible. It is acceptable to spend O(N) time to verify O(1) computation. +/// +/// In the data plane (eg, lsm tree), finer grained judgement is required. Do an unconditional O(1) +/// assert before an O(N) loop (e.g, a bounds check). Inside the loop, it might or might not be +/// feasible to add an extra assert per iteration. In the latter case, guard the assert with `if +/// (constants.verify)`, but prefer an unconditional assert unless benchmarks prove it to be costly. +/// +/// In the data plane, never use O(N) asserts for O(1) computations --- due to do randomized testing +/// the overall coverage is proportional to the number of tests run. Slow thorough assertions +/// decrease the overall test coverage. +/// +/// Specific data structures might use a comptime parameter, to enable extra costly verification +/// only during unit tests of the data structure. +pub const verify = false; diff --git a/fifo.zig b/fifo.zig index 153d63d..47a6b01 100644 --- a/fifo.zig +++ b/fifo.zig @@ -1,16 +1,28 @@ const std = @import("std"); const assert = std.debug.assert; +const constants = @import("./constants.zig"); + /// An intrusive first in/first out linked list. /// The element type T must have a field called "next" of type ?*T -pub fn FIFO(comptime T: type) type { +pub fn FIFOType(comptime T: type) type { return struct { - const Self = @This(); + const FIFO = @This(); in: ?*T = null, out: ?*T = null, + count: u64 = 0, + + // This should only be null if you're sure we'll never want to monitor `count`. + name: ?[]const u8, + + // If the number of elements is large, the constants.verify check in push() can be too + // expensive. Allow the user to gate it. Could also be a comptime param? + verify_push: bool = true, + + pub fn push(self: *FIFO, elem: *T) void { + if (constants.verify and self.verify_push) assert(!self.contains(elem)); - pub fn push(self: *Self, elem: *T) void { assert(elem.next == null); if (self.in) |in| { in.next = elem; @@ -20,28 +32,43 @@ pub fn FIFO(comptime T: type) type { self.in = elem; self.out = elem; } + self.count += 1; } - pub fn pop(self: *Self) ?*T { + pub fn pop(self: *FIFO) ?*T { const ret = self.out orelse return null; self.out = ret.next; ret.next = null; if (self.in == ret) self.in = null; + self.count -= 1; return ret; } - pub fn peek(self: Self) ?*T { + pub fn peek_last(self: FIFO) ?*T { + return self.in; + } + + pub fn peek(self: FIFO) ?*T { return self.out; } - pub fn empty(self: Self) bool { + pub fn empty(self: FIFO) bool { return self.peek() == null; } + /// Returns whether the linked list contains the given *exact element* (pointer comparison). + pub fn contains(self: *const FIFO, elem_needle: *const T) bool { + var iterator = self.peek(); + while (iterator) |elem| : (iterator = elem.next) { + if (elem == elem_needle) return true; + } + return false; + } + /// Remove an element from the FIFO. Asserts that the element is /// in the FIFO. This operation is O(N), if this is done often you /// probably want a different data structure. - pub fn remove(self: *Self, to_remove: *T) void { + pub fn remove(self: *FIFO, to_remove: *T) void { if (to_remove == self.out) { _ = self.pop(); return; @@ -52,14 +79,19 @@ pub fn FIFO(comptime T: type) type { if (to_remove == self.in) self.in = elem; elem.next = to_remove.next; to_remove.next = null; + self.count -= 1; break; } } else unreachable; } + + pub fn reset(self: *FIFO) void { + self.* = .{ .name = self.name }; + } }; } -test "push/pop/peek/remove/empty" { +test "FIFO: push/pop/peek/remove/empty" { const testing = @import("std").testing; const Foo = struct { next: ?*@This() = null }; @@ -68,17 +100,23 @@ test "push/pop/peek/remove/empty" { var two: Foo = .{}; var three: Foo = .{}; - var fifo: FIFO(Foo) = .{}; + var fifo: FIFOType(Foo) = .{ .name = null }; try testing.expect(fifo.empty()); fifo.push(&one); try testing.expect(!fifo.empty()); try testing.expectEqual(@as(?*Foo, &one), fifo.peek()); + try testing.expect(fifo.contains(&one)); + try testing.expect(!fifo.contains(&two)); + try testing.expect(!fifo.contains(&three)); fifo.push(&two); fifo.push(&three); try testing.expect(!fifo.empty()); try testing.expectEqual(@as(?*Foo, &one), fifo.peek()); + try testing.expect(fifo.contains(&one)); + try testing.expect(fifo.contains(&two)); + try testing.expect(fifo.contains(&three)); fifo.remove(&one); try testing.expect(!fifo.empty()); @@ -86,6 +124,9 @@ test "push/pop/peek/remove/empty" { try testing.expectEqual(@as(?*Foo, &three), fifo.pop()); try testing.expectEqual(@as(?*Foo, null), fifo.pop()); try testing.expect(fifo.empty()); + try testing.expect(!fifo.contains(&one)); + try testing.expect(!fifo.contains(&two)); + try testing.expect(!fifo.contains(&three)); fifo.push(&one); fifo.push(&two); diff --git a/io.zig b/io.zig index 475fab9..c29703c 100644 --- a/io.zig +++ b/io.zig @@ -1,6 +1,5 @@ const std = @import("std"); const builtin = @import("builtin"); -const assert = std.debug.assert; const os = std.os; const FIFO = @import("fifo.zig").FIFO; @@ -13,6 +12,12 @@ pub const IO = switch (builtin.target.os.tag) { else => @compileError("IO is not supported for platform"), }; +pub const DirectIO = enum { + direct_io_required, + direct_io_optional, + direct_io_disabled, +}; + pub fn buffer_limit(buffer_len: usize) usize { // Linux limits how much may be written in a `pwrite()/pread()` call, which is `0x7ffff000` on // both 64-bit and 32-bit systems, due to using a signed C int as the return value, as well as diff --git a/io/darwin.zig b/io/darwin.zig index 48c1461..da7aa29 100644 --- a/io/darwin.zig +++ b/io/darwin.zig @@ -4,18 +4,21 @@ const mem = std.mem; const assert = std.debug.assert; const log = std.log.scoped(.io); +const stdx = @import("../stdx.zig"); const constants = @import("../constants.zig"); -const FIFO = @import("../fifo.zig").FIFO; +const FIFOType = @import("../fifo.zig").FIFOType; const Time = @import("../time.zig").Time; const buffer_limit = @import("../io.zig").buffer_limit; +const DirectIO = @import("../io.zig").DirectIO; pub const IO = struct { - kq: posix.fd_t, + kq: fd_t, + event_id: Event = 0, time: Time = .{}, io_inflight: usize = 0, - timeouts: FIFO(Completion) = .{}, - completed: FIFO(Completion) = .{}, - io_pending: FIFO(Completion) = .{}, + timeouts: FIFOType(Completion) = .{ .name = "io_timeouts" }, + completed: FIFOType(Completion) = .{ .name = "io_completed" }, + io_pending: FIFOType(Completion) = .{ .name = "io_pending" }, pub fn init(entries: u12, flags: u32) !IO { _ = entries; @@ -118,14 +121,14 @@ pub const IO = struct { self.io_inflight -= new_events; for (events[0..new_events]) |event| { - const completion = @as(*Completion, @ptrFromInt(event.udata)); + const completion: *Completion = @ptrFromInt(event.udata); completion.next = null; self.completed.push(completion); } } var completed = self.completed; - self.completed = .{}; + self.completed.reset(); while (completed.pop()) |completion| { (completion.callback)(self, completion); } @@ -199,15 +202,18 @@ pub const IO = struct { socket: posix.socket_t, }, close: struct { - fd: posix.fd_t, + fd: fd_t, }, connect: struct { socket: posix.socket_t, address: std.net.Address, initiated: bool, }, + fsync: struct { + fd: fd_t, + }, read: struct { - fd: posix.fd_t, + fd: fd_t, buf: [*]u8, len: u32, offset: u64, @@ -226,7 +232,7 @@ pub const IO = struct { expires: u64, }, write: struct { - fd: posix.fd_t, + fd: fd_t, buf: [*]const u8, len: u32, offset: u64, @@ -242,10 +248,9 @@ pub const IO = struct { operation_data: anytype, comptime OperationImpl: type, ) void { - const Context = @TypeOf(context); const onCompleteFn = struct { fn onComplete(io: *IO, _completion: *Completion) void { - // Perform the actual operaton + // Perform the actual operation const op_data = &@field(_completion.operation, @tagName(operation_tag)); const result = OperationImpl.do_operation(op_data); @@ -265,8 +270,9 @@ pub const IO = struct { } // Complete the Completion + return callback( - @as(Context, @ptrFromInt(@intFromPtr(_completion.context))), + @ptrCast(@alignCast(_completion.context)), _completion, result, ); @@ -286,6 +292,16 @@ pub const IO = struct { } } + pub fn cancel(_: *IO, _: *Completion) void { + // TODO implement cancellation w/ kqueue. + log.debug("cancel implementation is missing on macOS", .{}); + } + + pub fn cancel_all(_: *IO) void { + // TODO Cancel in-flight async IO and wait for all completions. + log.debug("cancel all implementation is missing on macOS", .{}); + } + pub const AcceptError = posix.AcceptError || posix.SetSockOptError; pub fn accept( @@ -318,8 +334,9 @@ pub const IO = struct { ); errdefer posix.close(fd); - // Darwin doesn't support posix.MSG_NOSIGNAL to avoid getting SIGPIPE on socket send(). - // Instead, it uses the SO_NOSIGPIPE socket option which does the same for all send()s. + // Darwin doesn't support posix.MSG_NOSIGNAL to avoid getting SIGPIPE on + // socket send(). Instead, it uses the SO_NOSIGPIPE socket option which does + // the same for all send()s. posix.setsockopt( fd, posix.SOL.SOCKET, @@ -356,7 +373,7 @@ pub const IO = struct { result: CloseError!void, ) void, completion: *Completion, - fd: posix.fd_t, + fd: fd_t, ) void { self.submit( context, @@ -373,7 +390,7 @@ pub const IO = struct { .BADF => error.FileDescriptorInvalid, .INTR => {}, // A success, see https://github.com/ziglang/zig/issues/2425 .IO => error.InputOutput, - else => |errno| posix.unexpectedErrno(errno), + else => |errno| stdx.unexpected_errno("close", errno), }; } }, @@ -411,7 +428,11 @@ pub const IO = struct { // Instead, check the socket error to see if has been connected successfully. const result = switch (op.initiated) { true => posix.getsockoptError(op.socket), - else => posix.connect(op.socket, &op.address.any, op.address.getOsSockLen()), + else => posix.connect( + op.socket, + &op.address.any, + op.address.getOsSockLen(), + ), }; op.initiated = true; @@ -421,6 +442,38 @@ pub const IO = struct { ); } + pub const FsyncError = posix.SyncError || posix.UnexpectedError; + + pub fn fsync( + self: *IO, + comptime Context: type, + context: Context, + comptime callback: fn ( + context: Context, + completion: *Completion, + result: FsyncError!void, + ) void, + completion: *Completion, + fd: fd_t, + ) void { + self.submit( + context, + callback, + completion, + .fsync, + .{ + .fd = fd, + }, + struct { + fn do_operation(op: anytype) FsyncError!void { + return fs_sync(op.fd); + } + }, + ); + } + + pub const OpenatError = posix.OpenError || posix.UnexpectedError; + pub const ReadError = error{ WouldBlock, NotOpenForReading, @@ -443,7 +496,7 @@ pub const IO = struct { result: ReadError!usize, ) void, completion: *Completion, - fd: posix.fd_t, + fd: fd_t, buffer: []u8, offset: u64, ) void { @@ -465,10 +518,10 @@ pub const IO = struct { op.fd, op.buf, op.len, - @as(isize, @bitCast(op.offset)), + @bitCast(op.offset), ); return switch (posix.errno(rc)) { - .SUCCESS => @as(usize, @intCast(rc)), + .SUCCESS => @intCast(rc), .INTR => continue, .AGAIN => error.WouldBlock, .BADF => error.NotOpenForReading, @@ -483,7 +536,7 @@ pub const IO = struct { .OVERFLOW => error.Unseekable, .SPIPE => error.Unseekable, .TIMEDOUT => error.ConnectionTimedOut, - else => |err| posix.unexpectedErrno(err), + else => |err| stdx.unexpected_errno("read", err), }; } } @@ -557,29 +610,6 @@ pub const IO = struct { ); } - pub const CancelError = error{ NotFound, ExpirationInProgress } || posix.UnexpectedError; - - pub fn cancel( - self: *IO, - comptime Context: type, - context: Context, - comptime callback: fn ( - context: Context, - completion: *Completion, - result: CancelError!void, - ) void, - completion: *Completion, - cancel_completion: *Completion, - ) void { - _ = self; - _ = context; - _ = callback; - _ = completion; - _ = cancel_completion; - // TODO implement cancellation w/ kqueue. - log.debug("cancel implementation is missing on macOS", .{}); - } - pub const TimeoutError = error{Canceled} || posix.UnexpectedError; pub fn timeout( @@ -603,7 +633,7 @@ pub const IO = struct { .callback = struct { fn on_complete(_io: *IO, _completion: *Completion) void { _ = _io; - const _context = @as(Context, @ptrFromInt(@intFromPtr(_completion.context))); + const _context: Context = @ptrCast(@alignCast(_completion.context)); callback(_context, _completion, {}); } }.on_complete, @@ -641,7 +671,7 @@ pub const IO = struct { result: WriteError!usize, ) void, completion: *Completion, - fd: posix.fd_t, + fd: fd_t, buffer: []const u8, offset: u64, ) void { @@ -658,32 +688,120 @@ pub const IO = struct { }, struct { fn do_operation(op: anytype) WriteError!usize { - return posix.pwrite(op.fd, op.buf[0..op.len], op.offset); + // In the current implementation, Darwin file IO (namely, the posix.pwrite + // below) is _synchronous_, so it's safe to call fs_sync after it has + // completed. + const result = posix.pwrite(op.fd, op.buf[0..op.len], op.offset); + try fs_sync(op.fd); + + return result; } }, ); } + pub const Event = usize; + pub const INVALID_EVENT: Event = 0; + + pub fn open_event( + self: *IO, + ) !Event { + self.event_id += 1; + const event = self.event_id; + assert(event != INVALID_EVENT); + + var kev = mem.zeroes([1]posix.Kevent); + kev[0].ident = event; + kev[0].filter = posix.system.EVFILT_USER; + kev[0].flags = posix.system.EV_ADD | posix.system.EV_ENABLE | posix.system.EV_CLEAR; + + const polled = posix.kevent(self.kq, &kev, kev[0..0], null) catch |err| switch (err) { + error.AccessDenied => unreachable, // EV_FILTER is allowed for every user. + error.EventNotFound => unreachable, // We're not modifying or deleting an existing one. + error.ProcessNotFound => unreachable, // We're not monitoring a process. + error.Overflow, error.SystemResources => return error.SystemResources, + }; + assert(polled == 0); + + return event; + } + + pub fn event_listen( + self: *IO, + event: Event, + completion: *Completion, + comptime on_event: fn (*Completion) void, + ) void { + assert(event != INVALID_EVENT); + completion.* = .{ + .next = null, + .context = null, + .operation = undefined, + .callback = struct { + fn on_complete(_: *IO, completion_inner: *Completion) void { + on_event(completion_inner); + } + }.on_complete, + }; + + self.io_inflight += 1; + } + + pub fn event_trigger(self: *IO, event: Event, completion: *Completion) void { + assert(event != INVALID_EVENT); + + var kev = mem.zeroes([1]posix.Kevent); + kev[0].ident = event; + kev[0].filter = posix.system.EVFILT_USER; + kev[0].fflags = posix.system.NOTE_TRIGGER; + kev[0].udata = @intFromPtr(completion); + + const polled: usize = posix.kevent(self.kq, &kev, kev[0..0], null) catch unreachable; + assert(polled == 0); + } + + pub fn close_event(self: *IO, event: Event) void { + assert(event != INVALID_EVENT); + + var kev = mem.zeroes([1]posix.Kevent); + kev[0].ident = event; + kev[0].filter = posix.system.EVFILT_USER; + kev[0].flags = posix.system.EV_DELETE; + kev[0].udata = 0; // Not needed for EV_DELETE. + + const polled = posix.kevent(self.kq, &kev, kev[0..0], null) catch unreachable; + assert(polled == 0); + } + pub const INVALID_SOCKET = -1; /// Creates a socket that can be used for async operations with the IO instance. pub fn open_socket(self: *IO, family: u32, sock_type: u32, protocol: u32) !posix.socket_t { - _ = self; - const fd = try posix.socket(family, sock_type | posix.SOCK.NONBLOCK, protocol); - errdefer posix.closeSocket(fd); + errdefer self.close_socket(fd); + + // darwin doesn't support SOCK_CLOEXEC. + _ = try posix.fcntl(fd, posix.F.SETFD, posix.FD_CLOEXEC); // darwin doesn't support posix.MSG_NOSIGNAL, but instead a socket option to avoid SIGPIPE. try posix.setsockopt(fd, posix.SOL.SOCKET, posix.SO.NOSIGPIPE, &mem.toBytes(@as(c_int, 1))); + return fd; } + /// Closes a socket opened by the IO instance. + pub fn close_socket(self: *IO, socket: posix.socket_t) void { + _ = self; + posix.close(socket); + } + /// Opens a directory with read only access. - pub fn open_dir(dir_path: []const u8) !posix.fd_t { - return posix.open(dir_path, posix.O.CLOEXEC | posix.O.RDONLY, 0); + pub fn open_dir(dir_path: []const u8) !fd_t { + return posix.open(dir_path, .{ .CLOEXEC = true, .ACCMODE = .RDONLY }, 0); } - pub const INVALID_FILE: posix.fd_t = -1; + pub const fd_t = posix.fd_t; + pub const INVALID_FILE: fd_t = -1; /// Opens or creates a journal file: /// - For reading and writing. @@ -693,38 +811,55 @@ pub const IO = struct { /// - Ensures that the file data (and file inode in the parent directory) is durable on disk. /// The caller is responsible for ensuring that the parent directory inode is durable. /// - Verifies that the file size matches the expected file size before returning. - pub fn open_file( - dir_fd: posix.fd_t, + pub fn open_data_file( + self: *IO, + dir_fd: fd_t, relative_path: []const u8, size: u64, - must_create: bool, - ) !posix.fd_t { + method: enum { create, create_or_open, open, open_read_only }, + direct_io: DirectIO, + ) !fd_t { + _ = self; + assert(relative_path.len > 0); - assert(size >= constants.sector_size); assert(size % constants.sector_size == 0); // TODO Use O_EXCL when opening as a block device to obtain a mandatory exclusive lock. // This is much stronger than an advisory exclusive lock, and is required on some platforms. - // Opening with O_DSYNC is essential for both durability and correctness. - // O_DSYNC enables us to omit fsync() calls in the data plane, since we sync to the disk on every write. - var flags: u32 = posix.O.CLOEXEC | posix.O.RDWR | posix.O.DSYNC; + // Normally, O_DSYNC enables us to omit fsync() calls in the data plane, since we sync to + // the disk on every write, but that's not the case for Darwin: + // https://x.com/TigerBeetleDB/status/1536628729031581697 + // To work around this, fs_sync() is explicitly called after writing in do_operation. + var flags: posix.O = .{ + .CLOEXEC = true, + .ACCMODE = if (method == .open_read_only) .RDONLY else .RDWR, + .DSYNC = true, + }; var mode: posix.mode_t = 0; // TODO Document this and investigate whether this is in fact correct to set here. - if (@hasDecl(posix.O, "LARGEFILE")) flags |= posix.O.LARGEFILE; - - if (must_create) { - log.info("creating \"{s}\"...", .{relative_path}); - flags |= posix.O.CREAT; - flags |= posix.O.EXCL; - mode = 0o666; - } else { - log.info("opening \"{s}\"...", .{relative_path}); + if (@hasField(posix.O, "LARGEFILE")) flags.LARGEFILE = true; + + switch (method) { + .create => { + flags.CREAT = true; + flags.EXCL = true; + mode = 0o666; + log.info("creating \"{s}\"...", .{relative_path}); + }, + .create_or_open => { + flags.CREAT = true; + mode = 0o666; + log.info("opening or creating \"{s}\"...", .{relative_path}); + }, + .open, .open_read_only => { + log.info("opening \"{s}\"...", .{relative_path}); + }, } // This is critical as we rely on O_DSYNC for fsync() whenever we write to the file: - assert((flags & posix.O.DSYNC) > 0); + assert(flags.DSYNC); // Be careful with openat(2): "If pathname is absolute, then dirfd is ignored." (man page) assert(!std.fs.path.isAbsolute(relative_path)); @@ -736,7 +871,7 @@ pub const IO = struct { // On darwin assume that Direct I/O is always supported. // Use F_NOCACHE to disable the page cache as O_DIRECT doesn't exist. - if (constants.direct_io) { + if (direct_io != .direct_io_disabled) { _ = try posix.fcntl(fd, posix.F.NOCACHE, 1); } @@ -750,7 +885,7 @@ pub const IO = struct { // Ask the file system to allocate contiguous sectors for the file (if possible): // If the file system does not support `fallocate()`, then this could mean more seeks or a // panic if we run out of disk space (ENOSPC). - if (must_create) try fs_allocate(fd, size); + if (method == .create) try fs_allocate(fd, size); // The best fsync strategy is always to fsync before reading because this prevents us from // making decisions on data that was never durably written by a previously crashed process. @@ -770,15 +905,19 @@ pub const IO = struct { return fd; } - /// Darwin's fsync() syscall does not flush past the disk cache. We must use F_FULLFSYNC instead. + /// Darwin's fsync() syscall does not flush past the disk cache. We must use F_FULLFSYNC + /// instead. /// https://twitter.com/TigerBeetleDB/status/1422491736224436225 - fn fs_sync(fd: posix.fd_t) !void { + fn fs_sync(fd: fd_t) !void { + // TODO: This is of dubious safety - it's _not_ safe to fall back on posix.fsync unless it's + // known at startup that the disk (eg, an external disk on a Mac) doesn't support + // F_FULLFSYNC. _ = posix.fcntl(fd, posix.F.FULLFSYNC, 1) catch return posix.fsync(fd); } /// Allocates a file contiguously using fallocate() if supported. /// Alternatively, writes to the last sector so that at least the file size is correct. - fn fs_allocate(fd: posix.fd_t, size: u64) !void { + fn fs_allocate(fd: fd_t, size: u64) !void { log.info("allocating {}...", .{std.fmt.fmtIntSizeBin(size)}); // Darwin doesn't have fallocate() but we can simulate it using fcntl()s. @@ -802,7 +941,7 @@ pub const IO = struct { .fst_flags = F_ALLOCATECONTIG | F_ALLOCATEALL, .fst_posmode = F_PEOFPOSMODE, .fst_offset = 0, - .fst_length = @as(posix.off_t, @intCast(size)), + .fst_length = @intCast(size), .fst_bytesalloc = 0, }; @@ -824,8 +963,10 @@ pub const IO = struct { .NOLCK => unreachable, // F_SETLK or F_SETLKW .OVERFLOW => return error.FileTooBig, .SRCH => unreachable, // F_SETOWN - .OPNOTSUPP => return error.OperationNotSupported, // not reported but need same error union - else => |errno| return posix.unexpectedErrno(errno), + + // not reported but need same error union + .OPNOTSUPP => return error.OperationNotSupported, + else => |errno| return stdx.unexpected_errno("fs_allocate", errno), } // Now actually perform the allocation. diff --git a/io/linux.zig b/io/linux.zig index 1cbc994..5be5304 100644 --- a/io/linux.zig +++ b/io/linux.zig @@ -1,28 +1,61 @@ const std = @import("std"); const assert = std.debug.assert; +const os = std.os; const posix = std.posix; -const linux = std.os.linux; +const linux = os.linux; const IO_Uring = linux.IoUring; const io_uring_cqe = linux.io_uring_cqe; const io_uring_sqe = linux.io_uring_sqe; const log = std.log.scoped(.io); -const stdx = @import("../stdx.zig"); -const parse_dirty_semver = stdx.parse_dirty_semver; - const constants = @import("../constants.zig"); -const FIFO = @import("../fifo.zig").FIFO; +const stdx = @import("../stdx.zig"); +const FIFOType = @import("../fifo.zig").FIFOType; const buffer_limit = @import("../io.zig").buffer_limit; +const DirectIO = @import("../io.zig").DirectIO; +const DoublyLinkedListType = @import("../list.zig").DoublyLinkedListType; +const parse_dirty_semver = stdx.parse_dirty_semver; +const maybe = stdx.maybe; pub const IO = struct { + const CompletionList = DoublyLinkedListType(Completion, .awaiting_back, .awaiting_next); + ring: IO_Uring, /// Operations not yet submitted to the kernel and waiting on available space in the /// submission queue. - unqueued: FIFO(Completion) = .{}, + unqueued: FIFOType(Completion) = .{ .name = "io_unqueued" }, /// Completions that are ready to have their callbacks run. - completed: FIFO(Completion) = .{}, + completed: FIFOType(Completion) = .{ .name = "io_completed" }, + + // TODO Track these as metrics: + ios_queued: u32 = 0, + ios_in_kernel: u32 = 0, + + /// The head of a doubly-linked list of all operations that are: + /// - in the submission queue, or + /// - in the kernel, or + /// - in the completion queue, or + /// - in the `completed` list (excluding zero-duration timeouts) + awaiting: CompletionList = .{}, + + // This is the completion that performs the cancellation. + // This is *not* the completion that is being canceled. + cancel_completion: Completion = undefined, + + cancel_status: union(enum) { + // Not canceling. + inactive, + // Waiting to start canceling the next awaiting operation. + next, + // The target's cancellation SQE is queued; waiting for the cancellation's completion. + queued: struct { target: *Completion }, + // Currently canceling the target operation. + wait: struct { target: *Completion }, + // All operations have been canceled. + done, + } = .inactive, pub fn init(entries: u12, flags: u32) !IO { // Detect the linux version to ensure that we support all io_uring ops used. @@ -32,6 +65,19 @@ pub const IO = struct { @panic("Linux kernel 5.5 or greater is required for io_uring OP_ACCEPT"); } + errdefer |err| switch (err) { + error.SystemOutdated => { + log.err("io_uring is not available", .{}); + log.err("likely cause: the syscall is disabled by seccomp", .{}); + }, + error.PermissionDenied => { + log.err("io_uring is not available", .{}); + log.err("likely cause: the syscall is disabled by sysctl, " ++ + "try 'sysctl -w kernel.io_uring_disabled=0'", .{}); + }, + else => {}, + }; + return IO{ .ring = try IO_Uring.init(entries, flags) }; } @@ -41,6 +87,8 @@ pub const IO = struct { /// Pass all queued submissions to the kernel and peek for completions. pub fn tick(self: *IO) !void { + assert(self.cancel_status != .done); + // We assume that all timeouts submitted by `run_for_ns()` will be reaped by `run_for_ns()` // and that `tick()` and `run_for_ns()` cannot be run concurrently. // Therefore `timeouts` here will never be decremented and `etime` will always be false. @@ -66,13 +114,15 @@ pub const IO = struct { /// The `nanoseconds` argument is a u63 to allow coercion to the i64 used /// in the kernel_timespec struct. pub fn run_for_ns(self: *IO, nanoseconds: u63) !void { + assert(self.cancel_status != .done); + // We must use the same clock source used by io_uring (CLOCK_MONOTONIC) since we specify the // timeout below as an absolute value. Otherwise, we may deadlock if the clock sources are // dramatically different. Any kernel that supports io_uring will support CLOCK_MONOTONIC. var current_ts: posix.timespec = undefined; posix.clock_gettime(posix.CLOCK.MONOTONIC, ¤t_ts) catch unreachable; // The absolute CLOCK_MONOTONIC time after which we may return from this function: - const timeout_ts: linux.kernel_timespec = .{ + const timeout_ts: os.linux.kernel_timespec = .{ .tv_sec = current_ts.tv_sec, .tv_nsec = current_ts.tv_nsec + nanoseconds, }; @@ -85,9 +135,14 @@ pub const IO = struct { break :blk self.ring.get_sqe() catch unreachable; }; // Submit an absolute timeout that will be canceled if any other SQE completes first: - timeout_sqe.prep_timeout(&timeout_ts, 1, linux.IORING_TIMEOUT_ABS); + timeout_sqe.prep_timeout(&timeout_ts, 1, os.linux.IORING_TIMEOUT_ABS); timeout_sqe.user_data = 0; timeouts += 1; + + // We don't really want to count this timeout as an io, + // but it's tricky to track separately. + self.ios_queued += 1; + // The amount of time this call will block is bounded by the timeout we just submitted: try self.flush(1, &timeouts, &etime); } @@ -109,18 +164,39 @@ pub const IO = struct { // Loop over a copy to avoid an infinite loop of `enqueue()` re-adding to `self.unqueued`. { var copy = self.unqueued; - self.unqueued = .{}; + self.unqueued.reset(); while (copy.pop()) |completion| self.enqueue(completion); } // Run completions only after all completions have been flushed: - // Loop on a copy of the linked list, having reset the list first, so that any synchronous - // append on running a completion is executed only the next time round the event loop, - // without creating an infinite loop. - { - var copy = self.completed; - self.completed = .{}; - while (copy.pop()) |completion| completion.complete(); + // Loop until all completions are processed. Calls to complete() may queue more work + // and extend the duration of the loop, but this is fine as it 1) executes completions + // that become ready without going through another syscall from flush_submissions() and + // 2) potentially queues more SQEs to take advantage more of the next flush_submissions(). + while (self.completed.pop()) |completion| { + if (completion.operation == .timeout and + completion.operation.timeout.timespec.tv_sec == 0 and + completion.operation.timeout.timespec.tv_nsec == 0) + { + // Zero-duration timeouts are a special case, and aren't listed in `awaiting`. + maybe(self.awaiting.empty()); + assert(completion.result == -@as(i32, @intFromEnum(posix.E.TIME))); + assert(completion.awaiting_back == null); + assert(completion.awaiting_next == null); + } else { + assert(!self.awaiting.empty()); + self.awaiting.remove(completion); + } + + switch (self.cancel_status) { + .inactive => completion.complete(), + .next => {}, + .queued => if (completion.operation == .cancel) completion.complete(), + .wait => |wait| if (wait.target == completion) { + self.cancel_status = .next; + }, + .done => unreachable, + } } // At this point, unqueued could have completions either by 1) those who didn't get an SQE @@ -140,6 +216,8 @@ pub const IO = struct { }; if (completed > wait_remaining) wait_remaining = 0 else wait_remaining -= completed; for (cqes[0..completed]) |cqe| { + self.ios_in_kernel -= 1; + if (cqe.user_data == 0) { timeouts.* -= 1; // We are only done if the timeout submitted was completed due to time, not if @@ -149,7 +227,7 @@ pub const IO = struct { if (-cqe.res == @intFromEnum(posix.E.TIME)) etime.* = true; continue; } - const completion = @as(*Completion, @ptrFromInt(@as(usize, @intCast(cqe.user_data)))); + const completion: *Completion = @ptrFromInt(cqe.user_data); completion.result = cqe.res; // We do not run the completion here (instead appending to a linked list) to avoid: // * recursion through `flush_submissions()` and `flush_completions()`, @@ -157,13 +235,14 @@ pub const IO = struct { // * confusing stack traces. self.completed.push(completion); } + if (completed < cqes.len) break; } } fn flush_submissions(self: *IO, wait_nr: u32, timeouts: *usize, etime: *bool) !void { while (true) { - _ = self.ring.submit_and_wait(wait_nr) catch |err| switch (err) { + const submitted = self.ring.submit_and_wait(wait_nr) catch |err| switch (err) { error.SignalInterrupt => continue, // Wait for some completions and then try again: // See https://github.com/axboe/liburing/issues/281 re: error.SystemResources. @@ -175,11 +254,21 @@ pub const IO = struct { }, else => return err, }; + + self.ios_queued -= submitted; + self.ios_in_kernel += submitted; + break; } } fn enqueue(self: *IO, completion: *Completion) void { + switch (self.cancel_status) { + .inactive => {}, + .queued => assert(completion.operation == .cancel), + else => unreachable, + } + const sqe = self.ring.get_sqe() catch |err| switch (err) { error.SubmissionQueueFull => { self.unqueued.push(completion); @@ -187,6 +276,104 @@ pub const IO = struct { }, }; completion.prep(sqe); + + self.awaiting.push(completion); + self.ios_queued += 1; + } + + /// Cancel should be invoked at most once, before any of the memory owned by read/recv buffers + /// is freed (so that lingering async operations do not write to them). + /// + /// After this function is invoked: + /// - No more completion callbacks will be called. + /// - No more IO may be submitted. + /// + /// This function doesn't return until either: + /// - All events submitted to io_uring have completed. + /// (They may complete with `error.Canceled`). + /// - Or, an io_uring error occurs. + /// + /// TODO(Linux): + /// - Linux kernel ≥5.19 supports the IORING_ASYNC_CANCEL_ALL and IORING_ASYNC_CANCEL_ANY flags, + /// which would allow all events to be cancelled simultaneously with a single "cancel" + /// operation, without IO needing to maintain the `awaiting` doubly-linked list and the `next` + /// cancellation stage. + /// - Linux kernel ≥6.0 supports `io_uring_register_sync_cancel` which would remove the `queued` + /// cancellation stage. + pub fn cancel_all(self: *IO) void { + assert(self.cancel_status == .inactive); + + // Even if we return early due to an io_uring error, IO won't allow more operations. + defer self.cancel_status = .done; + + self.cancel_status = .next; + + // Discard any operations that haven't started yet. + while (self.unqueued.pop()) |_| {} + + while (self.awaiting.tail) |target| { + assert(!self.awaiting.empty()); + assert(self.cancel_status == .next); + assert(target.operation != .cancel); + + self.cancel(target); + assert(self.cancel_status == .queued); + + while (self.cancel_status == .queued or self.cancel_status == .wait) { + self.run_for_ns(constants.tick_ms * std.time.ns_per_ms) catch |err| { + std.debug.panic("IO.cancel_all: run_for_ns error: {}", .{err}); + }; + } + assert(self.cancel_status == .next); + } + assert(self.awaiting.empty()); + assert(self.ios_queued == 0); + assert(self.ios_in_kernel == 0); + } + + pub fn cancel(self: *IO, target: *Completion) void { + self.cancel_completion = .{ + .io = self, + .context = self, + .callback = struct { + fn wrapper( + ctx: ?*anyopaque, + comp: *Completion, + res: *const anyopaque, + ) void { + const io: *IO = @ptrCast(@alignCast(ctx.?)); + const result = + @as(*const CancelError!void, @ptrCast(@alignCast(res))).*; + io.cancel_callback(comp, result); + } + }.wrapper, + .operation = .{ .cancel = .{ .target = target } }, + }; + + self.cancel_status = .{ .queued = .{ .target = target } }; + self.enqueue(&self.cancel_completion); + } + + const CancelError = error{ + NotRunning, + NotInterruptable, + } || posix.UnexpectedError; + + fn cancel_callback(self: *IO, completion: *Completion, result: CancelError!void) void { + assert(self.cancel_status == .queued); + assert(completion == &self.cancel_completion); + assert(completion.operation == .cancel); + assert(completion.operation.cancel.target == self.cancel_status.queued.target); + + self.cancel_status = status: { + result catch |err| switch (err) { + error.NotRunning => break :status .next, + error.NotInterruptable => {}, + error.Unexpected => unreachable, + }; + // Wait for the target operation to complete or abort. + break :status .{ .wait = .{ .target = self.cancel_status.queued.target } }; + }; } /// This struct holds the data needed for a single io_uring operation @@ -196,10 +383,21 @@ pub const IO = struct { next: ?*Completion = null, operation: Operation, context: ?*anyopaque, - callback: *const fn (context: ?*anyopaque, completion: *Completion, result: *const anyopaque) void, + callback: *const fn ( + context: ?*anyopaque, + completion: *Completion, + result: *const anyopaque, + ) void, + + /// Used by the `IO.awaiting` doubly-linked list. + awaiting_back: ?*Completion = null, + awaiting_next: ?*Completion = null, fn prep(completion: *Completion, sqe: *io_uring_sqe) void { switch (completion.operation) { + .cancel => |op| { + sqe.prep_cancel(@intFromPtr(op.target), 0); + }, .accept => |*op| { sqe.prep_accept( op.socket, @@ -218,6 +416,17 @@ pub const IO = struct { op.address.getOsSockLen(), ); }, + .fsync => |op| { + sqe.prep_fsync(op.fd, op.flags); + }, + .openat => |op| { + sqe.prep_openat( + op.dir_fd, + op.file_path, + op.flags, + op.mode, + ); + }, .read => |op| { sqe.prep_read( op.fd, @@ -231,6 +440,15 @@ pub const IO = struct { .send => |op| { sqe.prep_send(op.socket, op.buffer, posix.MSG.NOSIGNAL); }, + .statx => |op| { + sqe.prep_statx( + op.dir_fd, + op.file_path, + op.flags, + op.mask, + op.statxbuf, + ); + }, .timeout => |*op| { sqe.prep_timeout(&op.timespec, 0, 0); }, @@ -241,15 +459,30 @@ pub const IO = struct { op.offset, ); }, - .cancel => |op| { - sqe.prep_cancel(op.c, 0); - }, } sqe.user_data = @intFromPtr(completion); } fn complete(completion: *Completion) void { switch (completion.operation) { + .cancel => { + const result: CancelError!void = result: { + if (completion.result < 0) { + break :result switch (@as(posix.E, @enumFromInt(-completion.result))) { + // No operation matching the completion is queued, so there is + // nothing to cancel. + .NOENT => error.NotRunning, + // The operation as far enough along that it cannot be canceled. + // It should complete soon. + .ALREADY => error.NotInterruptable, + // SQE is invalid. + .INVAL => unreachable, + else => |errno| stdx.unexpected_errno("cancel", errno), + }; + } + }; + completion.callback(completion.context, completion, &result); + }, .accept => { const result: AcceptError!posix.socket_t = blk: { if (completion.result < 0) { @@ -271,12 +504,11 @@ pub const IO = struct { .OPNOTSUPP => error.OperationNotSupported, .PERM => error.PermissionDenied, .PROTO => error.ProtocolFailure, - .CANCELED => error.Canceled, - else => |errno| posix.unexpectedErrno(errno), + else => |errno| stdx.unexpected_errno("accept", errno), }; break :blk err; } else { - break :blk @as(posix.socket_t, @intCast(completion.result)); + break :blk @intCast(completion.result); } }; completion.callback(completion.context, completion, &result); @@ -285,13 +517,13 @@ pub const IO = struct { const result: CloseError!void = blk: { if (completion.result < 0) { const err = switch (@as(posix.E, @enumFromInt(-completion.result))) { - .INTR => {}, // A success, see https://github.com/ziglang/zig/issues/2425 + // A success, see https://github.com/ziglang/zig/issues/2425 + .INTR => {}, .BADF => error.FileDescriptorInvalid, .DQUOT => error.DiskQuota, .IO => error.InputOutput, .NOSPC => error.NoSpaceLeft, - .CANCELED => error.Canceled, - else => |errno| posix.unexpectedErrno(errno), + else => |errno| stdx.unexpected_errno("close", errno), }; break :blk err; } else { @@ -320,13 +552,13 @@ pub const IO = struct { .FAULT => unreachable, .ISCONN => error.AlreadyConnected, .NETUNREACH => error.NetworkUnreachable, + .HOSTUNREACH => error.HostUnreachable, .NOENT => error.FileNotFound, .NOTSOCK => error.FileDescriptorNotASocket, .PERM => error.PermissionDenied, .PROTOTYPE => error.ProtocolNotSupported, .TIMEDOUT => error.ConnectionTimedOut, - .CANCELED => error.Canceled, - else => |errno| posix.unexpectedErrno(errno), + else => |errno| stdx.unexpected_errno("connect", errno), }; break :blk err; } else { @@ -335,15 +567,75 @@ pub const IO = struct { }; completion.callback(completion.context, completion, &result); }, - .read => { - const result: ReadError!usize = blk: { + .fsync => { + const result: anyerror!void = blk: { if (completion.result < 0) { const err = switch (@as(posix.E, @enumFromInt(-completion.result))) { .INTR => { completion.io.enqueue(completion); return; }, + .BADF => error.FileDescriptorInvalid, + .IO => error.InputOutput, + .INVAL => unreachable, + else => |errno| stdx.unexpected_errno("fsync", errno), + }; + break :blk err; + } else { + assert(completion.result == 0); + } + }; + completion.callback(completion.context, completion, &result); + }, + .openat => { + const result: OpenatError!fd_t = blk: { + if (completion.result < 0) { + const err = switch (@as(posix.E, @enumFromInt(-completion.result))) { + .INTR => { + completion.io.enqueue(completion); + return; + }, + .FAULT => unreachable, + .INVAL => unreachable, + .BADF => unreachable, + .ACCES => error.AccessDenied, + .FBIG => error.FileTooBig, + .OVERFLOW => error.FileTooBig, + .ISDIR => error.IsDir, + .LOOP => error.SymLinkLoop, + .MFILE => error.ProcessFdQuotaExceeded, + .NAMETOOLONG => error.NameTooLong, + .NFILE => error.SystemFdQuotaExceeded, + .NODEV => error.NoDevice, + .NOENT => error.FileNotFound, + .NOMEM => error.SystemResources, + .NOSPC => error.NoSpaceLeft, + .NOTDIR => error.NotDir, + .PERM => error.AccessDenied, + .EXIST => error.PathAlreadyExists, + .BUSY => error.DeviceBusy, + .OPNOTSUPP => error.FileLocksNotSupported, .AGAIN => error.WouldBlock, + .TXTBSY => error.FileBusy, + else => |errno| stdx.unexpected_errno("openat", errno), + }; + break :blk err; + } else { + break :blk @intCast(completion.result); + } + }; + completion.callback(completion.context, completion, &result); + }, + .read => { + const result: ReadError!usize = blk: { + if (completion.result < 0) { + const err = switch (@as(posix.E, @enumFromInt(-completion.result))) { + .INTR, .AGAIN => { + // Some file systems, like XFS, can return EAGAIN even when + // reading from a blocking file without flags like RWF_NOWAIT. + completion.io.enqueue(completion); + return; + }, .BADF => error.NotOpenForReading, .CONNRESET => error.ConnectionResetByPeer, .FAULT => unreachable, @@ -356,12 +648,11 @@ pub const IO = struct { .OVERFLOW => error.Unseekable, .SPIPE => error.Unseekable, .TIMEDOUT => error.ConnectionTimedOut, - .CANCELED => error.Canceled, - else => |errno| posix.unexpectedErrno(errno), + else => |errno| stdx.unexpected_errno("read", errno), }; break :blk err; } else { - break :blk @as(usize, @intCast(completion.result)); + break :blk @intCast(completion.result); } }; completion.callback(completion.context, completion, &result); @@ -385,12 +676,11 @@ pub const IO = struct { .CONNRESET => error.ConnectionResetByPeer, .TIMEDOUT => error.ConnectionTimedOut, .OPNOTSUPP => error.OperationNotSupported, - .CANCELED => error.Canceled, - else => |errno| posix.unexpectedErrno(errno), + else => |errno| stdx.unexpected_errno("recv", errno), }; break :blk err; } else { - break :blk @as(usize, @intCast(completion.result)); + break :blk @intCast(completion.result); } }; completion.callback(completion.context, completion, &result); @@ -421,27 +711,53 @@ pub const IO = struct { .OPNOTSUPP => error.OperationNotSupported, .PIPE => error.BrokenPipe, .TIMEDOUT => error.ConnectionTimedOut, - .CANCELED => error.Canceled, - else => |errno| posix.unexpectedErrno(errno), + else => |errno| stdx.unexpected_errno("send", errno), }; break :blk err; } else { - break :blk @as(usize, @intCast(completion.result)); + break :blk @intCast(completion.result); + } + }; + completion.callback(completion.context, completion, &result); + }, + .statx => { + const result: StatxError!void = blk: { + if (completion.result < 0) { + const err = switch (@as(posix.E, @enumFromInt(-completion.result))) { + .INTR => { + completion.io.enqueue(completion); + return; + }, + .FAULT => unreachable, + .INVAL => unreachable, + .BADF => unreachable, + .ACCES => error.AccessDenied, + .LOOP => error.SymLinkLoop, + .NAMETOOLONG => error.NameTooLong, + .NOENT => error.FileNotFound, + .NOMEM => error.SystemResources, + .NOTDIR => error.NotDir, + else => |errno| stdx.unexpected_errno("statx", errno), + }; + break :blk err; + } else { + assert(completion.result == 0); } }; completion.callback(completion.context, completion, &result); }, .timeout => { assert(completion.result < 0); - const result: TimeoutError!void = switch (@as(posix.E, @enumFromInt(-completion.result))) { + const err = switch (@as(posix.E, @enumFromInt(-completion.result))) { .INTR => { completion.io.enqueue(completion); return; }, .CANCELED => error.Canceled, .TIME => {}, // A success. - else => |errno| posix.unexpectedErrno(errno), + else => |errno| stdx.unexpected_errno("timeout", errno), }; + const result: TimeoutError!void = err; completion.callback(completion.context, completion, &result); }, .write => { @@ -466,28 +782,11 @@ pub const IO = struct { .PERM => error.AccessDenied, .PIPE => error.BrokenPipe, .SPIPE => error.Unseekable, - .CANCELED => error.Canceled, - else => |errno| posix.unexpectedErrno(errno), - }; - break :blk err; - } else { - break :blk @as(usize, @intCast(completion.result)); - } - }; - completion.callback(completion.context, completion, &result); - }, - .cancel => { - const result: CancelError!void = blk: { - if (completion.result < 0) { - const err = switch (@as(posix.E, @enumFromInt(-completion.result))) { - .SUCCESS => {}, - .NOENT => error.NotFound, - .ALREADY => error.ExpirationInProgress, - else => |errno| posix.unexpectedErrno(errno), + else => |errno| stdx.unexpected_errno("write", errno), }; break :blk err; } else { - break :blk; + break :blk @intCast(completion.result); } }; completion.callback(completion.context, completion, &result); @@ -498,20 +797,33 @@ pub const IO = struct { /// This union encodes the set of operations supported as well as their arguments. const Operation = union(enum) { + cancel: struct { + target: *Completion, + }, accept: struct { socket: posix.socket_t, address: posix.sockaddr = undefined, address_size: posix.socklen_t = @sizeOf(posix.sockaddr), }, close: struct { - fd: posix.fd_t, + fd: fd_t, }, connect: struct { socket: posix.socket_t, address: std.net.Address, }, + fsync: struct { + fd: fd_t, + flags: u32, + }, + openat: struct { + dir_fd: fd_t, + file_path: [*:0]const u8, + flags: posix.O, + mode: posix.mode_t, + }, read: struct { - fd: posix.fd_t, + fd: fd_t, buffer: []u8, offset: u64, }, @@ -523,15 +835,22 @@ pub const IO = struct { socket: posix.socket_t, buffer: []const u8, }, + statx: struct { + dir_fd: fd_t, + file_path: [*:0]const u8, + flags: u32, + mask: u32, + statxbuf: *std.os.linux.Statx, + }, timeout: struct { - timespec: linux.kernel_timespec, + timespec: os.linux.kernel_timespec, }, write: struct { - fd: posix.fd_t, + fd: fd_t, buffer: []const u8, offset: u64, }, - cancel: struct { + cancelOne: struct { c: u64, }, }; @@ -548,7 +867,6 @@ pub const IO = struct { OperationNotSupported, PermissionDenied, ProtocolFailure, - Canceled, } || posix.UnexpectedError; pub fn accept( @@ -569,9 +887,9 @@ pub const IO = struct { .callback = struct { fn wrapper(ctx: ?*anyopaque, comp: *Completion, res: *const anyopaque) void { callback( - @as(Context, @ptrFromInt(@intFromPtr(ctx))), + @ptrCast(@alignCast(ctx)), comp, - @as(*const AcceptError!posix.socket_t, @ptrFromInt(@intFromPtr(res))).*, + @as(*const AcceptError!posix.socket_t, @ptrCast(@alignCast(res))).*, ); } }.wrapper, @@ -591,7 +909,6 @@ pub const IO = struct { DiskQuota, InputOutput, NoSpaceLeft, - Canceled, } || posix.UnexpectedError; pub fn close( @@ -604,7 +921,7 @@ pub const IO = struct { result: CloseError!void, ) void, completion: *Completion, - fd: posix.fd_t, + fd: fd_t, ) void { completion.* = .{ .io = self, @@ -612,9 +929,9 @@ pub const IO = struct { .callback = struct { fn wrapper(ctx: ?*anyopaque, comp: *Completion, res: *const anyopaque) void { callback( - @as(Context, @ptrFromInt(@intFromPtr(ctx))), + @ptrCast(@alignCast(ctx)), comp, - @as(*const CloseError!void, @ptrFromInt(@intFromPtr(res))).*, + @as(*const CloseError!void, @ptrCast(@alignCast(res))).*, ); } }.wrapper, @@ -637,12 +954,13 @@ pub const IO = struct { ConnectionResetByPeer, AlreadyConnected, NetworkUnreachable, + HostUnreachable, FileNotFound, FileDescriptorNotASocket, PermissionDenied, ProtocolNotSupported, ConnectionTimedOut, - Canceled, + SystemResources, } || posix.UnexpectedError; pub fn connect( @@ -664,9 +982,9 @@ pub const IO = struct { .callback = struct { fn wrapper(ctx: ?*anyopaque, comp: *Completion, res: *const anyopaque) void { callback( - @as(Context, @ptrFromInt(@intFromPtr(ctx))), + @ptrCast(@alignCast(ctx)), comp, - @as(*const ConnectError!void, @ptrFromInt(@intFromPtr(res))).*, + @as(*const ConnectError!void, @ptrCast(@alignCast(res))).*, ); } }.wrapper, @@ -680,6 +998,89 @@ pub const IO = struct { self.enqueue(completion); } + pub const FsyncError = error{ + FileDescriptorInvalid, + InputOutput, + } || posix.UnexpectedError; + + pub fn fsync( + self: *IO, + comptime Context: type, + context: Context, + comptime callback: fn ( + context: Context, + completion: *Completion, + result: FsyncError!void, + ) void, + completion: *Completion, + fd: fd_t, + ) void { + completion.* = .{ + .io = self, + .context = context, + .callback = struct { + fn wrapper(ctx: ?*anyopaque, comp: *Completion, res: *const anyopaque) void { + callback( + @ptrCast(@alignCast(ctx)), + comp, + @as(*const FsyncError!void, @ptrCast(@alignCast(res))).*, + ); + } + }.wrapper, + .operation = .{ + .fsync = .{ + .fd = fd, + .flags = os.linux.IORING_FSYNC_DATASYNC, + }, + }, + }; + self.enqueue(completion); + } + + pub const OpenatError = posix.OpenError || posix.UnexpectedError; + + pub fn openat( + self: *IO, + comptime Context: type, + context: Context, + comptime callback: fn ( + context: Context, + completion: *Completion, + result: OpenatError!fd_t, + ) void, + completion: *Completion, + dir_fd: fd_t, + file_path: [*:0]const u8, + flags: posix.O, + mode: posix.mode_t, + ) void { + var new_flags = flags; + new_flags.CLOEXEC = true; + + completion.* = .{ + .io = self, + .context = context, + .callback = struct { + fn wrapper(ctx: ?*anyopaque, comp: *Completion, res: *const anyopaque) void { + callback( + @ptrCast(@alignCast(ctx)), + comp, + @as(*const OpenatError!fd_t, @ptrCast(@alignCast(res))).*, + ); + } + }.wrapper, + .operation = .{ + .openat = .{ + .dir_fd = dir_fd, + .file_path = file_path, + .flags = new_flags, + .mode = mode, + }, + }, + }; + self.enqueue(completion); + } + pub const ReadError = error{ WouldBlock, NotOpenForReading, @@ -690,7 +1091,6 @@ pub const IO = struct { SystemResources, Unseekable, ConnectionTimedOut, - Canceled, } || posix.UnexpectedError; pub fn read( @@ -703,7 +1103,7 @@ pub const IO = struct { result: ReadError!usize, ) void, completion: *Completion, - fd: posix.fd_t, + fd: fd_t, buffer: []u8, offset: u64, ) void { @@ -713,9 +1113,9 @@ pub const IO = struct { .callback = struct { fn wrapper(ctx: ?*anyopaque, comp: *Completion, res: *const anyopaque) void { callback( - @as(Context, @ptrFromInt(@intFromPtr(ctx))), + @ptrCast(@alignCast(ctx)), comp, - @as(*const ReadError!usize, @ptrFromInt(@intFromPtr(res))).*, + @as(*const ReadError!usize, @ptrCast(@alignCast(res))).*, ); } }.wrapper, @@ -740,7 +1140,6 @@ pub const IO = struct { ConnectionResetByPeer, ConnectionTimedOut, OperationNotSupported, - Canceled, } || posix.UnexpectedError; pub fn recv( @@ -762,9 +1161,9 @@ pub const IO = struct { .callback = struct { fn wrapper(ctx: ?*anyopaque, comp: *Completion, res: *const anyopaque) void { callback( - @as(Context, @ptrFromInt(@intFromPtr(ctx))), + @ptrCast(@alignCast(ctx)), comp, - @as(*const RecvError!usize, @ptrFromInt(@intFromPtr(res))).*, + @as(*const RecvError!usize, @ptrCast(@alignCast(res))).*, ); } }.wrapper, @@ -792,7 +1191,6 @@ pub const IO = struct { OperationNotSupported, BrokenPipe, ConnectionTimedOut, - Canceled, } || posix.UnexpectedError; pub fn send( @@ -814,9 +1212,9 @@ pub const IO = struct { .callback = struct { fn wrapper(ctx: ?*anyopaque, comp: *Completion, res: *const anyopaque) void { callback( - @as(Context, @ptrFromInt(@intFromPtr(ctx))), + @ptrCast(@alignCast(ctx)), comp, - @as(*const SendError!usize, @ptrFromInt(@intFromPtr(res))).*, + @as(*const SendError!usize, @ptrCast(@alignCast(res))).*, ); } }.wrapper, @@ -830,19 +1228,28 @@ pub const IO = struct { self.enqueue(completion); } - pub const CancelError = error{ NotFound, ExpirationInProgress } || posix.UnexpectedError; + pub const StatxError = error{ + SymLinkLoop, + FileNotFound, + NameTooLong, + NotDir, + } || std.fs.File.StatError || posix.UnexpectedError; - pub fn cancel( + pub fn statx( self: *IO, comptime Context: type, context: Context, comptime callback: fn ( context: Context, completion: *Completion, - result: CancelError!void, + result: StatxError!void, ) void, completion: *Completion, - cancel_completion: *Completion, + dir_fd: fd_t, + file_path: [*:0]const u8, + flags: u32, + mask: u32, + statxbuf: *std.os.linux.Statx, ) void { completion.* = .{ .io = self, @@ -850,19 +1257,22 @@ pub const IO = struct { .callback = struct { fn wrapper(ctx: ?*anyopaque, comp: *Completion, res: *const anyopaque) void { callback( - @as(Context, @ptrFromInt(@intFromPtr(ctx))), + @ptrCast(@alignCast(ctx)), comp, - @as(*const CancelError!void, @ptrFromInt(@intFromPtr(res))).*, + @as(*const StatxError!void, @ptrCast(@alignCast(res))).*, ); } }.wrapper, .operation = .{ - .cancel = .{ - .c = @intFromPtr(cancel_completion), + .statx = .{ + .dir_fd = dir_fd, + .file_path = file_path, + .flags = flags, + .mask = mask, + .statxbuf = statxbuf, }, }, }; - self.enqueue(completion); } @@ -886,9 +1296,9 @@ pub const IO = struct { .callback = struct { fn wrapper(ctx: ?*anyopaque, comp: *Completion, res: *const anyopaque) void { callback( - @as(Context, @ptrFromInt(@intFromPtr(ctx))), + @ptrCast(@alignCast(ctx)), comp, - @as(*const TimeoutError!void, @ptrFromInt(@intFromPtr(res))).*, + @as(*const TimeoutError!void, @ptrCast(@alignCast(res))).*, ); } }.wrapper, @@ -901,7 +1311,7 @@ pub const IO = struct { // Special case a zero timeout as a yield. if (nanoseconds == 0) { - completion.result = -@as(i32, @intCast(@intFromEnum(posix.E.TIME))); + completion.result = -@as(i32, @intFromEnum(posix.E.TIME)); self.completed.push(completion); return; } @@ -921,7 +1331,6 @@ pub const IO = struct { Unseekable, AccessDenied, BrokenPipe, - Canceled, } || posix.UnexpectedError; pub fn write( @@ -934,7 +1343,7 @@ pub const IO = struct { result: WriteError!usize, ) void, completion: *Completion, - fd: posix.fd_t, + fd: fd_t, buffer: []const u8, offset: u64, ) void { @@ -944,9 +1353,9 @@ pub const IO = struct { .callback = struct { fn wrapper(ctx: ?*anyopaque, comp: *Completion, res: *const anyopaque) void { callback( - @as(Context, @ptrFromInt(@intFromPtr(ctx))), + @ptrCast(@alignCast(ctx)), comp, - @as(*const WriteError!usize, @ptrFromInt(@intFromPtr(res))).*, + @as(*const WriteError!usize, @ptrCast(@alignCast(res))).*, ); } }.wrapper, @@ -961,20 +1370,97 @@ pub const IO = struct { self.enqueue(completion); } + pub const Event = posix.fd_t; + pub const INVALID_EVENT: Event = -1; + + pub fn open_event(self: *IO) !Event { + _ = self; + + // eventfd initialized with no (zero) previous write value. + const event_fd = posix.eventfd(0, linux.EFD.CLOEXEC) catch |err| switch (err) { + error.SystemResources, + error.SystemFdQuotaExceeded, + error.ProcessFdQuotaExceeded, + => return error.SystemResources, + error.Unexpected => return error.Unexpected, + }; + assert(event_fd != INVALID_EVENT); + errdefer os.close(event_fd); + + return event_fd; + } + + pub fn event_listen( + self: *IO, + event: Event, + completion: *Completion, + comptime on_event: fn (*Completion) void, + ) void { + assert(event != INVALID_EVENT); + const Context = struct { + const Context = @This(); + var buffer: u64 = undefined; + + fn on_read( + _: *Context, + completion_inner: *Completion, + result: ReadError!usize, + ) void { + const bytes = result catch unreachable; // eventfd reads should not fail. + assert(bytes == @sizeOf(u64)); + on_event(completion_inner); + } + }; + + self.read( + *Context, + undefined, + Context.on_read, + completion, + event, + std.mem.asBytes(&Context.buffer), + 0, // eventfd reads must always start from 0 offset. + ); + } + + pub fn event_trigger(self: *IO, event: Event, completion: *Completion) void { + assert(event != INVALID_EVENT); + _ = self; + _ = completion; + + const value: u64 = 1; + const bytes = posix.write(event, std.mem.asBytes(&value)) catch unreachable; + assert(bytes == @sizeOf(u64)); + } + + pub fn close_event(self: *IO, event: Event) void { + assert(event != INVALID_EVENT); + _ = self; + + posix.close(event); + } + pub const INVALID_SOCKET = -1; /// Creates a socket that can be used for async operations with the IO instance. pub fn open_socket(self: *IO, family: u32, sock_type: u32, protocol: u32) !posix.socket_t { _ = self; - return posix.socket(family, sock_type, protocol); + return posix.socket(family, sock_type | posix.SOCK.CLOEXEC, protocol); + } + + /// Closes a socket opened by the IO instance. + pub fn close_socket(self: *IO, socket: posix.socket_t) void { + _ = self; + posix.close(socket); } /// Opens a directory with read only access. - pub fn open_dir(dir_path: []const u8) !posix.fd_t { - return posix.open(dir_path, posix.O.CLOEXEC | posix.O.RDONLY, 0); + pub fn open_dir(dir_path: []const u8) !fd_t { + return posix.open(dir_path, .{ .CLOEXEC = true, .ACCMODE = .RDONLY }, 0); } - pub const INVALID_FILE: posix.fd_t = -1; + pub const fd_t = posix.fd_t; + pub const INVALID_FILE: fd_t = -1; /// Opens or creates a journal file: /// - For reading and writing. @@ -984,80 +1470,196 @@ pub const IO = struct { /// - Ensures that the file data (and file inode in the parent directory) is durable on disk. /// The caller is responsible for ensuring that the parent directory inode is durable. /// - Verifies that the file size matches the expected file size before returning. - pub fn open_file( - dir_fd: posix.fd_t, + pub fn open_data_file( + self: *IO, + dir_fd: fd_t, relative_path: []const u8, size: u64, - must_create: bool, - ) !posix.fd_t { + method: enum { create, create_or_open, open, open_read_only }, + direct_io: DirectIO, + ) !fd_t { + _ = self; + assert(relative_path.len > 0); - assert(size >= constants.sector_size); assert(size % constants.sector_size == 0); + // Be careful with openat(2): "If pathname is absolute, then dirfd is ignored." (man page) + assert(!std.fs.path.isAbsolute(relative_path)); - // TODO Use O_EXCL when opening as a block device to obtain a mandatory exclusive lock. - // This is much stronger than an advisory exclusive lock, and is required on some platforms. - - var flags: u32 = posix.O.CLOEXEC | posix.O.RDWR | posix.O.DSYNC; + var flags: posix.O = .{ + .CLOEXEC = true, + .ACCMODE = if (method == .open_read_only) .RDONLY else .RDWR, + .DSYNC = true, + }; var mode: posix.mode_t = 0; - // TODO Document this and investigate whether this is in fact correct to set here. - if (@hasDecl(posix.O, "LARGEFILE")) flags |= posix.O.LARGEFILE; - - var direct_io_supported = false; - if (constants.direct_io) { - direct_io_supported = try fs_supports_direct_io(dir_fd); - if (direct_io_supported) { - flags |= posix.O.DIRECT; - } else if (!constants.direct_io_required) { - log.warn("file system does not support Direct I/O", .{}); + const kind: enum { file, block_device } = blk: { + const stat = posix.fstatat( + dir_fd, + relative_path, + 0, + ) catch |err| switch (err) { + error.FileNotFound => { + if (method == .create or method == .create_or_open) { + // It's impossible to distinguish creating a new file and opening a new + // block device with the current API. So if it's possible that we should + // create a file we try that instead of failing here. + break :blk .file; + } else { + @panic("Path does not exist."); + } + }, + else => |err_| return err_, + }; + if (posix.S.ISBLK(stat.mode)) { + break :blk .block_device; } else { - // We require Direct I/O for safety to handle fsync failure correctly, and therefore - // panic in production if it is not supported. - @panic("file system does not support Direct I/O"); + if (!posix.S.ISREG(stat.mode)) { + @panic("file path does not point to block device or regular file."); + } + break :blk .file; } - } + }; - if (must_create) { - log.info("creating \"{s}\"...", .{relative_path}); - flags |= posix.O.CREAT; - flags |= posix.O.EXCL; - mode = 0o666; - } else { - log.info("opening \"{s}\"...", .{relative_path}); + // This is not strictly necessary on 64bit systems but it's harmless. + // This will avoid errors with handling large files on certain configurations + // of 32bit kernels. In all other cases, it's a noop. + // See: + if (@hasField(posix.O, "LARGEFILE")) flags.LARGEFILE = true; + + switch (kind) { + .block_device => { + if (direct_io != .direct_io_disabled) { + // Block devices should always support Direct IO. + flags.DIRECT = true; + // Use O_EXCL when opening as a block device to obtain an advisory exclusive + // lock. Normally, you can't do this for files you don't create, but for + // block devices this guarantees: + // - that there are no mounts using this block device + // - that no new mounts can use this block device while we have it open + // + // However it doesn't prevent other processes with root from opening without + // O_EXCL and writing (mount is just a special case that always checks O_EXCL). + // + // This should be stronger than flock(2) locks, which work on a separate system. + // The relevant kernel code (as of v6.7) is here: + // + flags.EXCL = true; + } + log.info("opening block device \"{s}\"...", .{relative_path}); + }, + .file => { + var direct_io_supported = false; + const dir_on_tmpfs = try fs_is_tmpfs(dir_fd); + + if (dir_on_tmpfs) { + log.warn( + "tmpfs is not durable, and your data will be lost on reboot", + .{}, + ); + } + + // Special case. tmpfs doesn't support Direct I/O. Normally we would panic + // here (see below) but being able to benchmark production workloads + // on tmpfs is very useful for removing + // disk speed from the equation. + if (direct_io != .direct_io_disabled and !dir_on_tmpfs) { + direct_io_supported = try fs_supports_direct_io(dir_fd); + if (direct_io_supported) { + flags.DIRECT = true; + } else if (direct_io == .direct_io_optional) { + log.warn("This file system does not support Direct I/O.", .{}); + } else { + assert(direct_io == .direct_io_required); + // We require Direct I/O for safety to handle fsync failure correctly, and + // therefore panic in production if it is not supported. + log.err("This file system does not support Direct I/O.", .{}); + log.err("TigerBeetle uses Direct I/O to bypass the kernel page cache, " ++ + "to ensure that data is durable when writes complete.", .{}); + log.err("If this is a production replica, Direct I/O is required.", .{}); + log.err("If this is a development/testing replica, " ++ + "re-run with --development set to bypass this error.", .{}); + @panic("file system does not support Direct I/O"); + } + } + + switch (method) { + .create => { + flags.CREAT = true; + flags.EXCL = true; + mode = 0o666; + log.info("creating \"{s}\"...", .{relative_path}); + }, + .create_or_open => { + flags.CREAT = true; + mode = 0o666; + log.info("opening or creating \"{s}\"...", .{relative_path}); + }, + .open, .open_read_only => { + log.info("opening \"{s}\"...", .{relative_path}); + }, + } + }, } // This is critical as we rely on O_DSYNC for fsync() whenever we write to the file: - assert((flags & posix.O.DSYNC) > 0); + assert(flags.DSYNC); - // Be careful with openat(2): "If pathname is absolute, then dirfd is ignored." (man page) - assert(!std.fs.path.isAbsolute(relative_path)); const fd = try posix.openat(dir_fd, relative_path, flags, mode); // TODO Return a proper error message when the path exists or does not exist (init/start). errdefer posix.close(fd); - // TODO Check that the file is actually a file. + { + // Make sure we're getting the type of file descriptor we expect. + const stat = try posix.fstat(fd); + switch (kind) { + .file => assert(posix.S.ISREG(stat.mode)), + .block_device => assert(posix.S.ISBLK(stat.mode)), + } + } // Obtain an advisory exclusive lock that works only if all processes actually use flock(). // LOCK_NB means that we want to fail the lock without waiting if another process has it. - posix.flock(fd, posix.LOCK.EX | posix.LOCK.NB) catch |err| switch (err) { - error.WouldBlock => @panic("another process holds the data file lock"), - else => return err, - }; + // + // This is wrapped inside a retry loop with a sleep because of the interaction between + // io_uring semantics and flock: flocks are held per fd, but io_uring will keep a reference + // to the fd alive even once a process has been terminated, until all async operations have + // been completed. + // + // This means that when killing and starting a tigerbeetle process in an automated way, you + // can see "another process holds the data file lock" errors, even though the process really + // has terminated. + for (0..2) |_| { + posix.flock(fd, posix.LOCK.EX | posix.LOCK.NB) catch |err| switch (err) { + error.WouldBlock => { + std.time.sleep(50 * std.time.ns_per_ms); + continue; + }, + else => return err, + }; + break; + } else { + posix.flock(fd, posix.LOCK.EX | posix.LOCK.NB) catch |err| switch (err) { + error.WouldBlock => @panic("another process holds the data file lock"), + else => return err, + }; + } // Ask the file system to allocate contiguous sectors for the file (if possible): // If the file system does not support `fallocate()`, then this could mean more seeks or a // panic if we run out of disk space (ENOSPC). - if (must_create) { + if (method == .create and kind == .file) { log.info("allocating {}...", .{std.fmt.fmtIntSizeBin(size)}); fs_allocate(fd, size) catch |err| switch (err) { error.OperationNotSupported => { log.warn("file system does not support fallocate(), an ENOSPC will panic", .{}); - log.info("allocating by writing to the last sector of the file instead...", .{}); + log.info("allocating by writing to the last sector " ++ + "of the file instead...", .{}); const sector_size = constants.sector_size; const sector: [sector_size]u8 align(sector_size) = [_]u8{0} ** sector_size; - // Handle partial writes where the physical sector is less than a logical sector: + // Handle partial writes where the physical sector is + // less than a logical sector: const write_offset = size - sector.len; var written: usize = 0; while (written < sector.len) { @@ -1079,47 +1681,132 @@ pub const IO = struct { // We always do this when opening because we don't know if this was done before crashing. try posix.fsync(dir_fd); - const stat = try posix.fstat(fd); - if (stat.size < size) @panic("data file inode size was truncated or corrupted"); + switch (kind) { + .file => { + if ((try posix.fstat(fd)).size < size) { + @panic("data file inode size was truncated or corrupted"); + } + }, + .block_device => { + const BLKGETSIZE64 = os.linux.IOCTL.IOR(0x12, 114, usize); + var block_device_size: usize = 0; + + switch (os.linux.E.init(os.linux.ioctl( + fd, + BLKGETSIZE64, + @intFromPtr(&block_device_size), + ))) { + .SUCCESS => {}, + + // These are the only errors that are supposed to be possible from ioctl(2). + .BADF => return error.InvalidFileDescriptor, + .NOTTY => return error.BadRequest, + .FAULT => return error.InvalidAddress, + else => |err| return stdx.unexpected_errno("open_file:ioctl", err), + } + + if (block_device_size < size) { + std.debug.panic( + "The block device used is too small ({} available/{} needed).", + .{ + std.fmt.fmtIntSizeBin(block_device_size), + std.fmt.fmtIntSizeBin(size), + }, + ); + } + + if (method == .create or method == .create_or_open) { + // Check that the first superblock_zone_size bytes are 0. + // - It'll ensure that the block device is not directly TigerBeetle. + // - It'll be very likely to catch any cases where there's an existing + // other filesystem. + // - In the case of there being a partition table (eg, two partitions, + // one starting at 0MiB, one at 1024MiB) and the operator tries to format + // the raw disk (/dev/sda) while a partition later is + // TigerBeetle (/dev/sda2) it'll be blocked by the MBR/GPT existing. + const superblock_zone_size = + @import("../superblock.zig").superblock_zone_size; + var read_buf: [superblock_zone_size]u8 align(constants.sector_size) = undefined; + + // We can do this without worrying about retrying partial reads because on + // linux, read(2) on block devices can not be interrupted by signals. + // See signal(7). + assert(superblock_zone_size == try posix.read(fd, &read_buf)); + if (!std.mem.allEqual(u8, &read_buf, 0)) { + std.debug.panic( + "Superblock on block device not empty. " ++ + "If this is the correct block device to use, " ++ + "please zero the first {} using a tool like dd.", + .{std.fmt.fmtIntSizeBin(superblock_zone_size)}, + ); + } + // Reset position in the block device to compensate for read(2). + try posix.lseek_CUR(fd, -superblock_zone_size); + assert(try posix.lseek_CUR_get(fd) == 0); + } + }, + } return fd; } + /// Detects whether the underlying file system for a given directory fd is tmpfs. This is used + /// to relax our Direct I/O check - running on tmpfs for benchmarking is useful. + fn fs_is_tmpfs(dir_fd: fd_t) !bool { + var statfs: stdx.StatFs = undefined; + + while (true) { + const res = stdx.fstatfs(dir_fd, &statfs); + switch (os.linux.E.init(res)) { + .SUCCESS => { + return statfs.f_type == stdx.TmpfsMagic; + }, + .INTR => continue, + else => |err| return stdx.unexpected_errno("fs_is_tmpfs", err), + } + } + } + /// Detects whether the underlying file system for a given directory fd supports Direct I/O. /// Not all Linux file systems support `O_DIRECT`, e.g. a shared macOS volume. - fn fs_supports_direct_io(dir_fd: posix.fd_t) !bool { - if (!@hasDecl(posix.O, "DIRECT")) return false; + fn fs_supports_direct_io(dir_fd: fd_t) !bool { + if (!@hasField(posix.O, "DIRECT")) return false; + + var cookie: [16]u8 = .{'0'} ** 16; + _ = stdx.array_print(16, &cookie, "{0x}", .{std.crypto.random.int(u64)}); - const path = "fs_supports_direct_io"; + const path: [:0]const u8 = "fs_supports_direct_io-" ++ cookie ++ ""; const dir = std.fs.Dir{ .fd = dir_fd }; - const fd = try posix.openatZ(dir_fd, path, posix.O.CLOEXEC | posix.O.CREAT | posix.O.TRUNC, 0o666); + const flags: posix.O = .{ .CLOEXEC = true, .CREAT = true, .TRUNC = true }; + const fd = try posix.openatZ(dir_fd, path, flags, 0o666); defer posix.close(fd); defer dir.deleteFile(path) catch {}; while (true) { - const res = linux.openat(dir_fd, path, posix.O.CLOEXEC | posix.O.RDONLY | posix.O.DIRECT, 0); - switch (linux.getErrno(res)) { + const dir_flags: posix.O = .{ .CLOEXEC = true, .ACCMODE = .RDONLY, .DIRECT = true }; + const res = os.linux.openat(dir_fd, path, dir_flags, 0); + switch (os.linux.E.init(res)) { .SUCCESS => { - posix.close(@as(posix.fd_t, @intCast(res))); + posix.close(@intCast(res)); return true; }, .INTR => continue, .INVAL => return false, - else => |err| return posix.unexpectedErrno(err), + else => |err| return stdx.unexpected_errno("fs_supports_direct_io", err), } } } /// Allocates a file contiguously using fallocate() if supported. /// Alternatively, writes to the last sector so that at least the file size is correct. - fn fs_allocate(fd: posix.fd_t, size: u64) !void { + fn fs_allocate(fd: fd_t, size: u64) !void { const mode: i32 = 0; const offset: i64 = 0; - const length = @as(i64, @intCast(size)); + const length: i64 = @intCast(size); while (true) { - const rc = linux.fallocate(fd, mode, offset, length); - switch (linux.getErrno(rc)) { + const rc = os.linux.fallocate(fd, mode, offset, length); + switch (os.linux.E.init(rc)) { .SUCCESS => return, .BADF => return error.FileDescriptorInvalid, .FBIG => return error.FileTooBig, @@ -1133,7 +1820,7 @@ pub const IO = struct { .PERM => return error.PermissionDenied, .SPIPE => return error.Unseekable, .TXTBSY => return error.FileBusy, - else => |errno| return posix.unexpectedErrno(errno), + else => |errno| return stdx.unexpected_errno("fs_allocate", errno), } } } diff --git a/io/test.zig b/io/test.zig index 803f950..f7d87fd 100644 --- a/io/test.zig +++ b/io/test.zig @@ -1,5 +1,6 @@ const std = @import("std"); const builtin = @import("builtin"); +const os = std.os; const posix = std.posix; const testing = std.testing; const assert = std.debug.assert; @@ -7,47 +8,80 @@ const assert = std.debug.assert; const Time = @import("../time.zig").Time; const IO = @import("../io.zig").IO; -test "write/read/close" { +test "open/write/read/close/statx" { try struct { const Context = @This(); + const StatxType = if (builtin.target.os.tag == .linux) std.os.linux.Statx else void; + path: [:0]const u8 = "test_io_write_read_close", io: IO, done: bool = false, - fd: posix.fd_t, + fd: ?posix.fd_t = null, write_buf: [20]u8 = [_]u8{97} ** 20, read_buf: [20]u8 = [_]u8{98} ** 20, written: usize = 0, read: usize = 0, - fn run_test() !void { - const path = "test_io_write_read_close"; - const file = try std.fs.cwd().createFile(path, .{ .read = true, .truncate = true }); - defer std.fs.cwd().deleteFile(path) catch {}; + statx: StatxType = undefined, + fn run_test() !void { var self: Context = .{ .io = try IO.init(32, 0), - .fd = file.handle, }; defer self.io.deinit(); + // The file gets created below, either by createFile or openat. + defer std.fs.cwd().deleteFile(self.path) catch {}; + var completion: IO.Completion = undefined; + if (builtin.target.os.tag == .linux) { + self.io.openat( + *Context, + &self, + openat_callback, + &completion, + posix.AT.FDCWD, + self.path, + .{ .ACCMODE = .RDWR, .TRUNC = true, .CREAT = true }, + std.fs.File.default_mode, + ); + } else { + const file = try std.fs.cwd().createFile(self.path, .{ + .read = true, + .truncate = true, + }); + self.openat_callback(&completion, file.handle); + } + while (!self.done) try self.io.tick(); + + try testing.expectEqual(self.write_buf.len, self.written); + try testing.expectEqual(self.read_buf.len, self.read); + try testing.expectEqualSlices(u8, &self.write_buf, &self.read_buf); + + if (builtin.target.os.tag == .linux) { + // Offset of 10 specified to read / write below. + try testing.expectEqual(self.statx.size - 10, self.written); + } + } + + fn openat_callback( + self: *Context, + completion: *IO.Completion, + result: anyerror!posix.fd_t, + ) void { + self.fd = result catch @panic("openat error"); self.io.write( *Context, - &self, + self, write_callback, - &completion, - self.fd, + completion, + self.fd.?, &self.write_buf, 10, ); - while (!self.done) try self.io.tick(); - - try testing.expectEqual(self.write_buf.len, self.written); - try testing.expectEqual(self.read_buf.len, self.read); - try testing.expectEqualSlices(u8, &self.write_buf, &self.read_buf); } fn write_callback( @@ -56,7 +90,7 @@ test "write/read/close" { result: IO.WriteError!usize, ) void { self.written = result catch @panic("write error"); - self.io.read(*Context, self, read_callback, completion, self.fd, &self.read_buf, 10); + self.io.read(*Context, self, read_callback, completion, self.fd.?, &self.read_buf, 10); } fn read_callback( @@ -65,7 +99,7 @@ test "write/read/close" { result: IO.ReadError!usize, ) void { self.read = result catch @panic("read error"); - self.io.close(*Context, self, close_callback, completion, self.fd); + self.io.close(*Context, self, close_callback, completion, self.fd.?); } fn close_callback( @@ -73,9 +107,34 @@ test "write/read/close" { completion: *IO.Completion, result: IO.CloseError!void, ) void { - _ = completion; _ = result catch @panic("close error"); + if (builtin.target.os.tag == .linux) { + self.io.statx( + *Context, + self, + statx_callback, + completion, + posix.AT.FDCWD, + self.path, + 0, + os.linux.STATX_BASIC_STATS, + &self.statx, + ); + } else { + self.done = true; + } + } + + fn statx_callback( + self: *Context, + completion: *IO.Completion, + result: IO.StatxError!void, + ) void { + _ = completion; + _ = result catch @panic("statx error"); + + assert(!self.done); self.done = true; } }.run_test(); @@ -102,13 +161,22 @@ test "accept/connect/send/receive" { var io = try IO.init(32, 0); defer io.deinit(); - const address = try std.net.Address.parseIp4("127.0.0.1", 3131); + const address = try std.net.Address.parseIp4("127.0.0.1", 0); const kernel_backlog = 1; - const server = try io.open_socket(address.any.family, posix.SOCK.STREAM, posix.IPPROTO.TCP); - defer posix.close(server); - const client = try io.open_socket(address.any.family, posix.SOCK.STREAM, posix.IPPROTO.TCP); - defer posix.close(client); + const server = try io.open_socket( + address.any.family, + posix.SOCK.STREAM, + posix.IPPROTO.TCP, + ); + defer io.close_socket(server); + + const client = try io.open_socket( + address.any.family, + posix.SOCK.STREAM, + posix.IPPROTO.TCP, + ); + defer io.close_socket(client); try posix.setsockopt( server, @@ -119,6 +187,10 @@ test "accept/connect/send/receive" { try posix.bind(server, &address.any, address.getOsSockLen()); try posix.listen(server, kernel_backlog); + var client_address = std.net.Address.initIp4(undefined, undefined); + var client_address_len = client_address.getOsSockLen(); + try posix.getsockname(server, &client_address.any, &client_address_len); + var self: Context = .{ .io = &io, .server = server, @@ -132,7 +204,7 @@ test "accept/connect/send/receive" { connect_callback, &client_completion, client, - address, + client_address, ); var server_completion: IO.Completion = undefined; @@ -204,7 +276,7 @@ test "accept/connect/send/receive" { test "timeout" { const ms = 20; - const margin = 5; + const margin = 100; const count = 10; try struct { @@ -260,6 +332,71 @@ test "timeout" { }.run_test(); } +test "event" { + try struct { + const Context = @This(); + + io: IO, + count: u32 = 0, + main_thread_id: std.Thread.Id, + event: IO.Event = IO.INVALID_EVENT, + event_completion: IO.Completion = undefined, + + const delay = 5 * std.time.ns_per_ms; + const events_count = 5; + + fn run_test() !void { + var self: Context = .{ + .io = try IO.init(32, 0), + .main_thread_id = std.Thread.getCurrentId(), + }; + defer self.io.deinit(); + + self.event = try self.io.open_event(); + defer self.io.close_event(self.event); + + var timer = Time{}; + const start = timer.monotonic(); + + // Listen to the event and spawn a thread that triggers the completion after some time. + self.io.event_listen(self.event, &self.event_completion, on_event); + const thread = try std.Thread.spawn(.{}, Context.trigger_event, .{&self}); + + // Wait for the number of events to complete. + while (self.count < events_count) try self.io.tick(); + thread.join(); + + // Make sure the event was triggered multiple times. + assert(self.count == events_count); + + // Make sure at least some time has passed. + const elapsed = timer.monotonic() - start; + assert(elapsed >= delay); + } + + fn trigger_event(self: *Context) void { + assert(std.Thread.getCurrentId() != self.main_thread_id); + while (self.count < events_count) { + std.time.sleep(delay + 1); + + // Triggering the event: + self.io.event_trigger(self.event, &self.event_completion); + } + } + + fn on_event(completion: *IO.Completion) void { + const self: *Context = @fieldParentPtr("event_completion", completion); + assert(std.Thread.getCurrentId() == self.main_thread_id); + + self.count += 1; + if (self.count == events_count) return; + + // Reattaching the event. + self.io.event_listen(self.event, &self.event_completion, on_event); + } + }.run_test(); +} + test "submission queue full" { const ms = 20; const count = 10; @@ -304,7 +441,7 @@ test "submission queue full" { } test "tick to wait" { - // Use only IO.tick() to see if pending IO is actually processsed + // Use only IO.tick() to see if pending IO is actually processed try struct { const Context = @This(); @@ -318,11 +455,12 @@ test "tick to wait" { var self: Context = .{ .io = try IO.init(1, 0) }; defer self.io.deinit(); - const address = try std.net.Address.parseIp4("127.0.0.1", 3131); + const address = try std.net.Address.parseIp4("127.0.0.1", 0); const kernel_backlog = 1; - const server = try self.io.open_socket(address.any.family, posix.SOCK.STREAM, posix.IPPROTO.TCP); - defer posix.close(server); + const server = + try self.io.open_socket(address.any.family, posix.SOCK.STREAM, posix.IPPROTO.TCP); + defer self.io.close_socket(server); try posix.setsockopt( server, @@ -333,8 +471,16 @@ test "tick to wait" { try posix.bind(server, &address.any, address.getOsSockLen()); try posix.listen(server, kernel_backlog); - const client = try self.io.open_socket(address.any.family, posix.SOCK.STREAM, posix.IPPROTO.TCP); - defer posix.close(client); + var client_address = std.net.Address.initIp4(undefined, undefined); + var client_address_len = client_address.getOsSockLen(); + try posix.getsockname(server, &client_address.any, &client_address_len); + + const client = try self.io.open_socket( + client_address.any.family, + posix.SOCK.STREAM, + posix.IPPROTO.TCP, + ); + defer self.io.close_socket(client); // Start the accept var server_completion: IO.Completion = undefined; @@ -348,7 +494,7 @@ test "tick to wait" { connect_callback, &client_completion, client, - address, + client_address, ); // Tick the IO to drain the accept & connect completions @@ -360,7 +506,7 @@ test "tick to wait" { assert(self.connected); assert(self.accepted != IO.INVALID_SOCKET); - defer posix.close(self.accepted); + defer self.io.close_socket(self.accepted); // Start receiving on the client var recv_completion: IO.Completion = undefined; @@ -440,9 +586,9 @@ test "tick to wait" { return posix.send(sock, buf, flags); } - const rc = posix.windows.sendto(sock, buf.ptr, buf.len, flags, null, 0); - if (rc == posix.windows.ws2_32.SOCKET_ERROR) { - switch (posix.windows.ws2_32.WSAGetLastError()) { + const rc = os.windows.sendto(sock, buf.ptr, buf.len, flags, null, 0); + if (rc == os.windows.ws2_32.SOCKET_ERROR) { + switch (os.windows.ws2_32.WSAGetLastError()) { .WSAEACCES => return error.AccessDenied, .WSAEADDRNOTAVAIL => return error.AddressNotAvailable, .WSAECONNRESET => return error.ConnectionResetByPeer, @@ -451,7 +597,10 @@ test "tick to wait" { .WSAENOTSOCK => return error.FileDescriptorNotASocket, .WSAEAFNOSUPPORT => return error.AddressFamilyNotSupported, .WSAEDESTADDRREQ => unreachable, // A destination address is required. - .WSAEFAULT => unreachable, // The lpBuffers, lpTo, lpOverlapped, lpNumberOfBytesSent, or lpCompletionRoutine parameters are not part of the user address space, or the lpTo parameter is too small. + // The lpBuffers, lpTo, lpOverlapped, lpNumberOfBytesSent, + // or lpCompletionRoutine parameters are not part of the user address space, + // or the lpTo parameter is too small. + .WSAEFAULT => unreachable, .WSAEHOSTUNREACH => return error.NetworkUnreachable, // TODO: WSAEINPROGRESS, WSAEINTR .WSAEINVAL => unreachable, @@ -459,13 +608,16 @@ test "tick to wait" { .WSAENETRESET => return error.ConnectionResetByPeer, .WSAENETUNREACH => return error.NetworkUnreachable, .WSAENOTCONN => return error.SocketNotConnected, - .WSAESHUTDOWN => unreachable, // The socket has been shut down; it is not possible to WSASendTo on a socket after shutdown has been invoked with how set to SD_SEND or SD_BOTH. + // The socket has been shut down; it is not possible to WSASendTo on a socket + // after shutdown has been invoked with how set to SD_SEND or SD_BOTH. + .WSAESHUTDOWN => unreachable, .WSAEWOULDBLOCK => return error.WouldBlock, - .WSANOTINITIALISED => unreachable, // A successful WSAStartup call must occur before using this function. - else => |err| return posix.windows.unexpectedWSAError(err), + // A successful WSAStartup call must occur before using this function. + .WSANOTINITIALISED => unreachable, + else => |err| return os.windows.unexpectedWSAError(err), } } else { - return @as(usize, @intCast(rc)); + return @intCast(rc); } } }.run_test(); @@ -506,10 +658,14 @@ test "pipe data over socket" { }; defer self.io.deinit(); - self.server.fd = try self.io.open_socket(posix.AF.INET, posix.SOCK.STREAM, posix.IPPROTO.TCP); - defer posix.close(self.server.fd); + self.server.fd = try self.io.open_socket( + posix.AF.INET, + posix.SOCK.STREAM, + posix.IPPROTO.TCP, + ); + defer self.io.close_socket(self.server.fd); - const address = try std.net.Address.parseIp4("127.0.0.1", 3131); + const address = try std.net.Address.parseIp4("127.0.0.1", 0); try posix.setsockopt( self.server.fd, posix.SOL.SOCKET, @@ -520,6 +676,10 @@ test "pipe data over socket" { try posix.bind(self.server.fd, &address.any, address.getOsSockLen()); try posix.listen(self.server.fd, 1); + var client_address = std.net.Address.initIp4(undefined, undefined); + var client_address_len = client_address.getOsSockLen(); + try posix.getsockname(self.server.fd, &client_address.any, &client_address_len); + self.io.accept( *Context, &self, @@ -528,8 +688,12 @@ test "pipe data over socket" { self.server.fd, ); - self.tx.socket.fd = try self.io.open_socket(posix.AF.INET, posix.SOCK.STREAM, posix.IPPROTO.TCP); - defer posix.close(self.tx.socket.fd); + self.tx.socket.fd = try self.io.open_socket( + posix.AF.INET, + posix.SOCK.STREAM, + posix.IPPROTO.TCP, + ); + defer self.io.close_socket(self.tx.socket.fd); self.io.connect( *Context, @@ -537,7 +701,7 @@ test "pipe data over socket" { on_connect, &self.tx.socket.completion, self.tx.socket.fd, - address, + client_address, ); var tick: usize = 0xdeadbeef; @@ -553,7 +717,7 @@ test "pipe data over socket" { try testing.expect(self.server.fd != IO.INVALID_SOCKET); try testing.expect(self.tx.socket.fd != IO.INVALID_SOCKET); try testing.expect(self.rx.socket.fd != IO.INVALID_SOCKET); - posix.close(self.rx.socket.fd); + self.io.close_socket(self.rx.socket.fd); try testing.expectEqual(self.tx.transferred, buffer_size); try testing.expectEqual(self.rx.transferred, buffer_size); @@ -641,61 +805,98 @@ test "pipe data over socket" { }.run(); } -test "cancel" { +test "cancel_all" { + const checksum = @import("../checksum.zig").checksum; + const allocator = std.testing.allocator; + const file_path = "test_cancel_all"; + const read_count = 8; + const read_size = 1024 * 16; + + // For this test to be useful, we rely on open(DIRECT). + // (See below). + if (builtin.target.os.tag != .linux) return; + try struct { const Context = @This(); io: IO, - timeout_res: IO.TimeoutError!void = undefined, - timeout_done: bool = false, - cancel_done: bool = false, + canceled: bool = false, fn run_test() !void { - var self: Context = .{ - .io = try IO.init(32, 0), - }; - defer self.io.deinit(); + defer std.fs.cwd().deleteFile(file_path) catch {}; - var completion: IO.Completion = undefined; - self.io.timeout( - *Context, - &self, - timeout_callback, - &completion, - 100 * std.time.ns_per_ms, - ); + var context: Context = .{ .io = try IO.init(32, 0) }; + defer context.io.deinit(); - var cancel_completion: IO.Completion = undefined; - self.io.cancel( - *Context, - &self, - cancel_callback, - &cancel_completion, - &completion, - ); - while (!self.cancel_done and !self.timeout_done) try self.io.tick(); + { + // Initialize a file filled with test data. + const file_buffer = try allocator.alloc(u8, read_size); + defer allocator.free(file_buffer); + for (file_buffer, 0..) |*b, i| b.* = @intCast(i % 256); - try testing.expectEqual(true, self.timeout_done); - try testing.expectEqual(true, self.cancel_done); - try testing.expectError(IO.TimeoutError.Canceled, self.timeout_res); - } + try std.fs.cwd().writeFile(.{ .sub_path = file_path, .data = file_buffer }); + } - fn timeout_callback( - self: *Context, - _: *IO.Completion, - result: IO.TimeoutError!void, - ) void { - self.timeout_res = result; - self.timeout_done = true; + var read_completions: [read_count]IO.Completion = undefined; + var read_buffers: [read_count][]u8 = undefined; + var read_buffer_checksums: [read_count]u128 = undefined; + var read_buffers_allocated: u32 = 0; + defer for (read_buffers[0..read_buffers_allocated]) |b| allocator.free(b); + + for (&read_buffers) |*read_buffer| { + read_buffer.* = try allocator.alloc(u8, read_size); + read_buffers_allocated += 1; + } + + // Test cancellation: + // 1. Re-open the file. + // 2. Kick off multiple (async) reads. + // 3. Abort the reads (ideally before they can complete, since that is more interesting + // to test). + // + // The reason to re-open the file with DIRECT is that it slows down the reads enough to + // actually test the interesting case -- cancelling an in-flight read and verifying that + // the buffer is not written to after `cancel_all()` completes. + // + // (Without DIRECT the reads all finish their callbacks even before io.tick() returns.) + const file = try std.posix.open(file_path, .{ .DIRECT = true }, 0); + defer std.posix.close(file); + + for (&read_completions, read_buffers) |*completion, buffer| { + context.io.read(*Context, &context, read_callback, completion, file, buffer, 0); + } + try context.io.tick(); + + // Set to true *before* calling cancel_all() to ensure that any farther callbacks from + // IO completion will panic. + context.canceled = true; + + context.io.cancel_all(); + + // All of the in-flight reads are canceled at this point. + // To verify, checksum all of the read buffer memory, then wait and make sure that there + // are no farther modifications to the buffers. + for (read_buffers, &read_buffer_checksums) |buffer, *buffer_checksum| { + buffer_checksum.* = checksum(buffer); + } + + const sleep_ms = 50; + std.time.sleep(sleep_ms * std.time.ns_per_ms); + + for (read_buffers, read_buffer_checksums) |buffer, buffer_checksum| { + try testing.expectEqual(checksum(buffer), buffer_checksum); + } } - fn cancel_callback( - self: *Context, - _: *IO.Completion, - result: IO.CancelError!void, + fn read_callback( + context: *Context, + completion: *IO.Completion, + result: IO.ReadError!usize, ) void { - result catch |err| std.debug.panic("cancel error: {}", .{err}); - self.cancel_done = true; + _ = completion; + _ = result catch @panic("read error"); + + assert(!context.canceled); } }.run_test(); } diff --git a/list.zig b/list.zig new file mode 100644 index 0000000..3f514be --- /dev/null +++ b/list.zig @@ -0,0 +1,159 @@ +const std = @import("std"); +const assert = std.debug.assert; + +const constants = @import("constants.zig"); +const stdx = @import("stdx.zig"); + +/// An intrusive doubly-linked list. +/// Currently it is LIFO for simplicity because its consumer (IO.awaiting) doesn't care about order. +pub fn DoublyLinkedListType( + comptime Node: type, + comptime field_back_enum: std.meta.FieldEnum(Node), + comptime field_next_enum: std.meta.FieldEnum(Node), +) type { + assert(@typeInfo(Node) == .Struct); + assert(field_back_enum != field_next_enum); + assert(std.meta.FieldType(Node, field_back_enum) == ?*Node); + assert(std.meta.FieldType(Node, field_next_enum) == ?*Node); + + const field_back = @tagName(field_back_enum); + const field_next = @tagName(field_next_enum); + + return struct { + const DoublyLinkedList = @This(); + + tail: ?*Node = null, + count: u32 = 0, + + pub fn verify(list: *const DoublyLinkedList) void { + assert((list.count == 0) == (list.tail == null)); + + var count: u32 = 0; + var iterator = list.tail; + + if (iterator) |node| { + assert(@field(node, field_next) == null); + } + + while (iterator) |node| { + const back = @field(node, field_back); + if (back) |back_node| { + assert(back_node != node); // There are no cycles. + assert(@field(back_node, field_next) == node); + } + count += 1; + iterator = back; + } + assert(count == list.count); + } + + fn contains(list: *const DoublyLinkedList, target: *const Node) bool { + var count: u32 = 0; + + var iterator = list.tail; + while (iterator) |node| { + if (node == target) return true; + iterator = @field(node, field_back); + count += 1; + } + + assert(count == list.count); + return false; + } + + pub fn empty(list: *const DoublyLinkedList) bool { + assert((list.count == 0) == (list.tail == null)); + return list.count == 0; + } + + pub fn push(list: *DoublyLinkedList, node: *Node) void { + if (constants.verify) assert(!list.contains(node)); + if (constants.verify) list.verify(); + assert(@field(node, field_back) == null); + assert(@field(node, field_next) == null); + + if (list.tail) |tail| { + assert(list.count > 0); + assert(@field(tail, field_next) == null); + + @field(node, field_back) = tail; + @field(tail, field_next) = node; + } else { + assert(list.count == 0); + } + + list.tail = node; + list.count += 1; + } + + pub fn pop(list: *DoublyLinkedList) ?*Node { + if (constants.verify) list.verify(); + + if (list.tail) |tail_old| { + assert(list.count > 0); + assert(@field(tail_old, field_next) == null); + + list.tail = @field(tail_old, field_back); + list.count -= 1; + + if (list.tail) |tail_new| { + assert(@field(tail_new, field_next) == tail_old); + @field(tail_new, field_next) = null; + } + + @field(tail_old, field_back) = null; + return tail_old; + } else { + assert(list.count == 0); + return null; + } + } + + pub fn remove(list: *DoublyLinkedList, node: *Node) void { + if (constants.verify) assert(list.contains(node)); + if (constants.verify) list.verify(); + assert(list.count > 0); + assert(list.tail != null); + + const tail = list.tail.?; + + if (node == tail) { + // Pop the last element of the list. + assert(@field(node, field_next) == null); + list.tail = @field(node, field_back); + } + if (@field(node, field_back)) |node_back| { + assert(@field(node_back, field_next).? == node); + @field(node_back, field_next) = @field(node, field_next); + } + if (@field(node, field_next)) |node_next| { + assert(@field(node_next, field_back).? == node); + @field(node_next, field_back) = @field(node, field_back); + } + @field(node, field_back) = null; + @field(node, field_next) = null; + list.count -= 1; + + if (constants.verify) list.verify(); + assert((list.count == 0) == (list.tail == null)); + } + }; +} + +test "DoublyLinkedList LIFO" { + const Node = struct { id: u32, back: ?*@This() = null, next: ?*@This() = null }; + const List = DoublyLinkedListType(Node, .back, .next); + + var nodes: [3]Node = undefined; + for (&nodes, 0..) |*node, i| node.* = .{ .id = @intCast(i) }; + + var list = List{}; + list.push(&nodes[0]); + list.push(&nodes[1]); + list.push(&nodes[2]); + + try std.testing.expectEqual(list.pop().?, &nodes[2]); + try std.testing.expectEqual(list.pop().?, &nodes[1]); + try std.testing.expectEqual(list.pop().?, &nodes[0]); + try std.testing.expectEqual(list.pop(), null); +} diff --git a/stdx.zig b/stdx.zig index d30259f..c9c3a54 100644 --- a/stdx.zig +++ b/stdx.zig @@ -1,6 +1,34 @@ //! Extensions to the standard library -- things which could have been in std, but aren't. const std = @import("std"); +const builtin = @import("builtin"); +const assert = std.debug.assert; + +/// `maybe` is the dual of `assert`: it signals that condition is sometimes true +/// and sometimes false. +/// +/// Currently we use it for documentation, but maybe one day we plug it into +/// coverage. +pub fn maybe(ok: bool) void { + assert(ok or !ok); +} + +pub const log = if (builtin.is_test) + // Downgrade `err` to `warn` for tests. + // Zig fails any test that does `log.err`, but we want to test those code paths here. + struct { + pub fn scoped(comptime scope: @Type(.EnumLiteral)) type { + const base = std.log.scoped(scope); + return struct { + pub const err = warn; + pub const warn = base.warn; + pub const info = base.info; + pub const debug = base.debug; + }; + } + } +else + std.log; // std.SemanticVersion requires there be no extra characters after the // major/minor/patch numbers. But when we try to parse `uname @@ -62,3 +90,18 @@ test "stdx.zig: parse_dirty_semver" { try std.testing.expectEqual(case.expected_version, version); } } + +/// Like std.posix's `unexpectedErrno()` but log unconditionally, not just when mode=Debug. +/// The added `label` argument works around the absence of stack traces in ReleaseSafe builds. +pub fn unexpected_errno(label: []const u8, err: std.posix.system.E) std.posix.UnexpectedError { + log.scoped(.stdx).err("unexpected errno: {s}: code={d} name={?s}", .{ + label, + @intFromEnum(err), + std.enums.tagName(std.posix.system.E, err), + }); + + if (builtin.mode == .Debug) { + std.debug.dumpCurrentStackTrace(null); + } + return error.Unexpected; +} diff --git a/superblock.zig b/superblock.zig new file mode 100644 index 0000000..7186cb2 --- /dev/null +++ b/superblock.zig @@ -0,0 +1,1723 @@ +//! SuperBlock invariants: +//! +//! * vsr_state +//! - vsr_state.replica and vsr_state.replica_count are immutable for now. +//! - vsr_state.checkpoint.header.op is initially 0 (for a newly-formatted replica). +//! - vsr_state.checkpoint.header.op ≤ vsr_state.commit_max +//! - vsr_state.checkpoint.header.op_before ≤ vsr_state.checkpoint.header.op +//! - vsr_state.log_view ≤ vsr_state.view +//! - vsr_state.sync_op_min ≤ vsr_state.sync_op_max +//! +//! - vsr_state.checkpoint.manifest_block_count = 0 implies: +//! vsr_state.checkpoint.manifest_oldest_address=0 +//! vsr_state.checkpoint.manifest_oldest_checksum=0 +//! vsr_state.checkpoint.manifest_newest_address=0 +//! vsr_state.checkpoint.manifest_newest_checksum=0 +//! vsr_state.checkpoint.manifest_oldest_address=0 +//! +//! - vsr_state.checkpoint.manifest_block_count > 0 implies: +//! vsr_state.checkpoint.manifest_oldest_address>0 +//! vsr_state.checkpoint.manifest_newest_address>0 +//! +//! - checkpoint() must advance the superblock's vsr_state.checkpoint.header.op. +//! - view_change() must not advance the superblock's vsr_state.checkpoint.header.op. +//! - The following are monotonically increasing: +//! - vsr_state.log_view +//! - vsr_state.view +//! - vsr_state.commit_max +//! - vsr_state.checkpoint.header.op may backtrack due to state sync. +//! +const std = @import("std"); +const assert = std.debug.assert; +const maybe = stdx.maybe; +const mem = std.mem; +const meta = std.meta; + +const constants = @import("../constants.zig"); +const stdx = @import("../stdx.zig"); +const vsr = @import("../vsr.zig"); +const log = std.log.scoped(.superblock); + +pub const Quorums = @import("superblock_quorums.zig").QuorumsType(.{ + .superblock_copies = constants.superblock_copies, +}); + +pub const SuperBlockVersion: u16 = + // Make sure that data files created by development builds are distinguished through version. + if (constants.config.process.release.value == vsr.Release.minimum.value) 0 else 2; + +const vsr_headers_reserved_size = constants.sector_size - + ((constants.view_change_headers_max * @sizeOf(vsr.Header)) % constants.sector_size); + +// Fields are aligned to work as an extern or packed struct. +pub const SuperBlockHeader = extern struct { + checksum: u128 = undefined, + checksum_padding: u128 = 0, + + /// Protects against misdirected reads at startup. + /// For example, if multiple reads are all misdirected to a single copy of the superblock. + /// Excluded from the checksum calculation to ensure that all copies have the same checksum. + /// This simplifies writing and comparing multiple copies. + /// TODO: u8 should be enough here, we use u16 only for alignment. + copy: u16 = 0, + + /// The version of the superblock format in use, reserved for major breaking changes. + version: u16, + + /// The release that the data file was originally formatted by. + /// (Upgrades do not update this field.) + release_format: vsr.Release, + + /// A monotonically increasing counter to locate the latest superblock at startup. + sequence: u64, + + /// Protects against writing to or reading from the wrong data file. + cluster: u128, + + /// The checksum of the previous superblock to hash chain across sequence numbers. + parent: u128, + parent_padding: u128 = 0, + + /// State stored on stable storage for the Viewstamped Replication consensus protocol. + vsr_state: VSRState, + + /// Reserved for future minor features (e.g. changing a compression algorithm). + flags: u64 = 0, + + /// The number of headers in vsr_headers_all. + vsr_headers_count: u32, + + reserved: [1940]u8 = [_]u8{0} ** 1940, + + /// SV/DVC header suffix. Headers are ordered from high-to-low op. + /// Unoccupied headers (after vsr_headers_count) are zeroed. + /// + /// When `vsr_state.log_view < vsr_state.view`, the headers are for a DVC. + /// When `vsr_state.log_view = vsr_state.view`, the headers are for a SV. + vsr_headers_all: [constants.view_change_headers_max]vsr.Header.Prepare, + vsr_headers_reserved: [vsr_headers_reserved_size]u8 = + [_]u8{0} ** vsr_headers_reserved_size, + + comptime { + assert(@sizeOf(SuperBlockHeader) % constants.sector_size == 0); + assert(@divExact(@sizeOf(SuperBlockHeader), constants.sector_size) >= 2); + assert(@offsetOf(SuperBlockHeader, "parent") % @sizeOf(u256) == 0); + assert(@offsetOf(SuperBlockHeader, "vsr_state") % @sizeOf(u256) == 0); + assert(@offsetOf(SuperBlockHeader, "vsr_headers_all") == constants.sector_size); + // Assert that there is no implicit padding in the struct. + assert(stdx.no_padding(SuperBlockHeader)); + } + + pub const VSRState = extern struct { + checkpoint: CheckpointState, + + /// Globally unique identifier of the replica, must be non-zero. + replica_id: u128, + + members: vsr.Members, + + /// The highest operation up to which we may commit. + commit_max: u64, + + /// See `sync_op_max`. + sync_op_min: u64, + + /// When zero, all of the grid blocks and replies are synced. + /// (When zero, `sync_op_min` is also zero.) + /// + /// When nonzero, we must repair grid-blocks/client-replies that would have been written + /// during the commits between `sync_op_min` and `sync_op_max` (inclusive). + /// (Those grid-blocks and client-replies were not written normally because we "skipped" + /// past them via state sync.) + sync_op_max: u64, + + /// This field was used by the old state sync protocol, but is now unused and is always set + /// to zero. + /// TODO: rename to reserved and assert that it is zero, once it is actually set to zero + /// in all superblocks (in the next release). + sync_view: u32 = 0, + + /// The last view in which the replica's status was normal. + log_view: u32, + + /// The view number of the replica. + view: u32, + + /// Number of replicas (determines sizes of the quorums), part of VSR configuration. + replica_count: u8, + + reserved: [779]u8 = [_]u8{0} ** 779, + + comptime { + assert(@sizeOf(VSRState) == 2048); + // Assert that there is no implicit padding in the struct. + assert(stdx.no_padding(VSRState)); + } + + pub fn root(options: struct { + cluster: u128, + replica_id: u128, + members: vsr.Members, + replica_count: u8, + release: vsr.Release, + }) VSRState { + return .{ + .checkpoint = .{ + .header = vsr.Header.Prepare.root(options.cluster), + .parent_checkpoint_id = 0, + .grandparent_checkpoint_id = 0, + .free_set_blocks_acquired_checksum = comptime vsr.checksum(&.{}), + .free_set_blocks_released_checksum = comptime vsr.checksum(&.{}), + .free_set_blocks_acquired_last_block_checksum = 0, + .free_set_blocks_released_last_block_checksum = 0, + .free_set_blocks_acquired_last_block_address = 0, + .free_set_blocks_released_last_block_address = 0, + .free_set_blocks_acquired_size = 0, + .free_set_blocks_released_size = 0, + .client_sessions_checksum = comptime vsr.checksum(&.{}), + .client_sessions_last_block_checksum = 0, + .client_sessions_last_block_address = 0, + .client_sessions_size = 0, + .manifest_oldest_checksum = 0, + .manifest_oldest_address = 0, + .manifest_newest_checksum = 0, + .manifest_newest_address = 0, + .manifest_block_count = 0, + .snapshots_block_checksum = 0, + .snapshots_block_address = 0, + .storage_size = data_file_size_min, + .release = options.release, + }, + .replica_id = options.replica_id, + .members = options.members, + .replica_count = options.replica_count, + .commit_max = 0, + .sync_op_min = 0, + .sync_op_max = 0, + .log_view = 0, + .view = 0, + }; + } + + pub fn assert_internally_consistent(state: VSRState) void { + assert(state.commit_max >= state.checkpoint.header.op); + assert(state.sync_op_max >= state.sync_op_min); + assert(state.view >= state.log_view); + assert(state.replica_count > 0); + assert(state.replica_count <= constants.replicas_max); + assert(vsr.member_index(&state.members, state.replica_id) != null); + + // These fields are unused at the moment: + assert(state.checkpoint.snapshots_block_checksum == 0); + assert(state.checkpoint.snapshots_block_address == 0); + + assert(state.checkpoint.manifest_oldest_checksum_padding == 0); + assert(state.checkpoint.manifest_newest_checksum_padding == 0); + assert(state.checkpoint.snapshots_block_checksum_padding == 0); + assert(state.checkpoint.free_set_blocks_acquired_last_block_checksum_padding == 0); + assert(state.checkpoint.free_set_blocks_released_last_block_checksum_padding == 0); + + assert(state.checkpoint.client_sessions_last_block_checksum_padding == 0); + assert(state.checkpoint.storage_size >= data_file_size_min); + + if (state.checkpoint.free_set_blocks_acquired_last_block_address == 0) { + assert(state.checkpoint.free_set_blocks_acquired_size == 0); + assert(state.checkpoint.free_set_blocks_acquired_checksum == + comptime vsr.checksum(&.{})); + assert(state.checkpoint.free_set_blocks_acquired_last_block_checksum == 0); + } else { + assert(state.checkpoint.free_set_blocks_acquired_size > 0); + } + + if (state.checkpoint.free_set_blocks_released_last_block_address == 0) { + assert(state.checkpoint.free_set_blocks_released_size == 0); + assert(state.checkpoint.free_set_blocks_released_checksum == + comptime vsr.checksum(&.{})); + assert(state.checkpoint.free_set_blocks_released_last_block_checksum == 0); + } else { + assert(state.checkpoint.free_set_blocks_released_size > 0); + } + + if (state.checkpoint.client_sessions_last_block_address == 0) { + assert(state.checkpoint.client_sessions_last_block_checksum == 0); + assert(state.checkpoint.client_sessions_size == 0); + assert(state.checkpoint.client_sessions_checksum == comptime vsr.checksum(&.{})); + } else { + assert(state.checkpoint.client_sessions_size == vsr.ClientSessions.encode_size); + } + + if (state.checkpoint.manifest_block_count == 0) { + assert(state.checkpoint.manifest_oldest_address == 0); + assert(state.checkpoint.manifest_newest_address == 0); + assert(state.checkpoint.manifest_oldest_checksum == 0); + assert(state.checkpoint.manifest_newest_checksum == 0); + } else { + assert(state.checkpoint.manifest_oldest_address != 0); + assert(state.checkpoint.manifest_newest_address != 0); + + assert((state.checkpoint.manifest_block_count == 1) == + (state.checkpoint.manifest_oldest_address == + state.checkpoint.manifest_newest_address)); + + assert((state.checkpoint.manifest_block_count == 1) == + (state.checkpoint.manifest_oldest_checksum == + state.checkpoint.manifest_newest_checksum)); + } + } + + pub fn monotonic(old: VSRState, new: VSRState) bool { + old.assert_internally_consistent(); + new.assert_internally_consistent(); + if (old.checkpoint.header.op == new.checkpoint.header.op) { + if (old.checkpoint.header.checksum == 0 and old.checkpoint.header.op == 0) { + // "old" is the root VSRState. + assert(old.commit_max == 0); + assert(old.sync_op_min == 0); + assert(old.sync_op_max == 0); + assert(old.log_view == 0); + assert(old.view == 0); + } else { + assert(stdx.equal_bytes(CheckpointState, &old.checkpoint, &new.checkpoint)); + } + } else { + assert(old.checkpoint.header.checksum != new.checkpoint.header.checksum); + assert(old.checkpoint.parent_checkpoint_id != + new.checkpoint.parent_checkpoint_id); + } + assert(old.replica_id == new.replica_id); + assert(old.replica_count == new.replica_count); + assert(stdx.equal_bytes([constants.members_max]u128, &old.members, &new.members)); + + if (old.checkpoint.header.op > new.checkpoint.header.op) return false; + if (old.view > new.view) return false; + if (old.log_view > new.log_view) return false; + if (old.commit_max > new.commit_max) return false; + + return true; + } + + pub fn would_be_updated_by(old: VSRState, new: VSRState) bool { + assert(monotonic(old, new)); + + return !stdx.equal_bytes(VSRState, &old, &new); + } + + /// Compaction is one bar ahead of superblock's commit_min. + /// The commits from the bar following commit_min were in the mutable table, and + /// thus not preserved in the checkpoint. + /// But the corresponding `compact()` updates were preserved, and must not be repeated + /// to ensure deterministic storage. + pub fn op_compacted(state: VSRState, op: u64) bool { + // If commit_min is 0, we have never checkpointed, so no compactions are checkpointed. + return state.checkpoint.header.op > 0 and + op <= vsr.Checkpoint.trigger_for_checkpoint(state.checkpoint.header.op).?; + } + }; + + /// CheckpointState for SuperBlockVersion=1 (and 0 for development builds). + /// + /// We maintain this so replicas with SuperBlockVersion=2 are able to parse older SuperBlocks + /// (see `read_header_callback` for translation from CheckpointStateOld → CheckpointState). + pub const CheckpointStateOld = extern struct { + /// The last prepare of the checkpoint committed to the state machine. + /// At startup, replay the log hereafter. + header: vsr.Header.Prepare, + + free_set_last_block_checksum: u128, + free_set_last_block_checksum_padding: u128 = 0, + client_sessions_last_block_checksum: u128, + client_sessions_last_block_checksum_padding: u128 = 0, + manifest_oldest_checksum: u128, + manifest_oldest_checksum_padding: u128 = 0, + manifest_newest_checksum: u128, + manifest_newest_checksum_padding: u128 = 0, + snapshots_block_checksum: u128, + snapshots_block_checksum_padding: u128 = 0, + + /// Checksum covering the entire encoded free set. Strictly speaking it is redundant: + /// free_set_last_block_checksum indirectly covers the same data. It is still useful + /// to protect from encoding-decoding bugs as a defense in depth. + free_set_checksum: u128, + + /// Checksum covering the entire client sessions, as defense-in-depth. + client_sessions_checksum: u128, + + /// The checkpoint_id() of the checkpoint which last updated our commit_min. + /// Following state sync, this is set to the last checkpoint that we skipped. + parent_checkpoint_id: u128, + /// The parent_checkpoint_id of the parent checkpoint. + /// TODO We might be able to remove this when + /// https://github.com/tigerbeetle/tigerbeetle/issues/1378 is fixed. + grandparent_checkpoint_id: u128, + + free_set_last_block_address: u64, + client_sessions_last_block_address: u64, + manifest_oldest_address: u64, + manifest_newest_address: u64, + snapshots_block_address: u64, + + // Logical storage size in bytes. + // + // If storage_size is less than the data file size, then the grid blocks beyond storage_size + // were used previously, but have since been freed. + // + // If storage_size is more than the data file size, then the data file might have been + // truncated/corrupted. + storage_size: u64, + + // Size of the encoded trailers in bytes. + // It is equal to the sum of sizes of individual trailer blocks and is used for assertions. + free_set_size: u64, + client_sessions_size: u64, + + /// The number of manifest blocks in the manifest log. + manifest_block_count: u32, + + /// All prepares between `CheckpointStateOld.commit_min` (i.e. `op_checkpoint`) and + /// `trigger_for_checkpoint(checkpoint_after(commit_min))` must be executed by this release. + /// (Prepares with `operation=upgrade` are the exception – upgrades in the last + /// `lsm_compaction_ops` before a checkpoint trigger may be replayed by a different release. + release: vsr.Release, + + reserved: [472]u8 = [_]u8{0} ** 472, + + comptime { + assert(@sizeOf(CheckpointStateOld) % @sizeOf(u128) == 0); + assert(@sizeOf(CheckpointStateOld) == 1024); + assert(stdx.no_padding(CheckpointStateOld)); + } + }; + + /// The content of CheckpointState is deterministic for the corresponding checkpoint. + /// + /// This struct is sent in a `start_view` message from the primary to a syncing replica. + pub const CheckpointState = extern struct { + /// The last prepare of the checkpoint committed to the state machine. + /// At startup, replay the log hereafter. + header: vsr.Header.Prepare, + + free_set_blocks_acquired_last_block_checksum: u128, + free_set_blocks_acquired_last_block_checksum_padding: u128 = 0, + + free_set_blocks_released_last_block_checksum: u128, + free_set_blocks_released_last_block_checksum_padding: u128 = 0, + + client_sessions_last_block_checksum: u128, + client_sessions_last_block_checksum_padding: u128 = 0, + manifest_oldest_checksum: u128, + manifest_oldest_checksum_padding: u128 = 0, + manifest_newest_checksum: u128, + manifest_newest_checksum_padding: u128 = 0, + snapshots_block_checksum: u128, + snapshots_block_checksum_padding: u128 = 0, + + /// Checksum covering the entire encoded free set. Strictly speaking it is redundant: + /// free_set_last_block_checksum indirectly covers the same data. It is still useful + /// to protect from encoding-decoding bugs as a defense in depth. + free_set_blocks_acquired_checksum: u128, + free_set_blocks_released_checksum: u128, + + /// Checksum covering the entire client sessions, as defense-in-depth. + client_sessions_checksum: u128, + + /// The checkpoint_id() of the checkpoint which last updated our commit_min. + /// Following state sync, this is set to the last checkpoint that we skipped. + parent_checkpoint_id: u128, + /// The parent_checkpoint_id of the parent checkpoint. + /// TODO We might be able to remove this when + /// https://github.com/tigerbeetle/tigerbeetle/issues/1378 is fixed. + grandparent_checkpoint_id: u128, + + free_set_blocks_acquired_last_block_address: u64, + free_set_blocks_released_last_block_address: u64, + + client_sessions_last_block_address: u64, + manifest_oldest_address: u64, + manifest_newest_address: u64, + snapshots_block_address: u64, + + // Logical storage size in bytes. + // + // If storage_size is less than the data file size, then the grid blocks beyond storage_size + // were used previously, but have since been freed. + // + // If storage_size is more than the data file size, then the data file might have been + // truncated/corrupted. + storage_size: u64, + + // Size of the encoded trailers in bytes. + // It is equal to the sum of sizes of individual trailer blocks and is used for assertions. + free_set_blocks_acquired_size: u64, + free_set_blocks_released_size: u64, + + client_sessions_size: u64, + + /// The number of manifest blocks in the manifest log. + manifest_block_count: u32, + + /// All prepares between `CheckpointState.commit_min` (i.e. `op_checkpoint`) and + /// `trigger_for_checkpoint(checkpoint_after(commit_min))` must be executed by this release. + /// (Prepares with `operation=upgrade` are the exception – upgrades in the last + /// `lsm_compaction_ops` before a checkpoint trigger may be replayed by a different release. + release: vsr.Release, + + reserved: [408]u8 = [_]u8{0} ** 408, + + comptime { + assert(@sizeOf(CheckpointState) % @sizeOf(u128) == 0); + assert(@sizeOf(CheckpointState) == 1024); + assert(stdx.no_padding(CheckpointState)); + } + }; + + pub fn calculate_checksum(superblock: *const SuperBlockHeader) u128 { + comptime assert(meta.fieldIndex(SuperBlockHeader, "checksum") == 0); + comptime assert(meta.fieldIndex(SuperBlockHeader, "checksum_padding") == 1); + comptime assert(meta.fieldIndex(SuperBlockHeader, "copy") == 2); + + const checksum_size = @sizeOf(@TypeOf(superblock.checksum)); + comptime assert(checksum_size == @sizeOf(u128)); + + const checksum_padding_size = @sizeOf(@TypeOf(superblock.checksum_padding)); + comptime assert(checksum_padding_size == @sizeOf(u128)); + + const copy_size = @sizeOf(@TypeOf(superblock.copy)); + comptime assert(copy_size == 2); + + const ignore_size = checksum_size + checksum_padding_size + copy_size; + + return vsr.checksum(std.mem.asBytes(superblock)[ignore_size..]); + } + + pub fn set_checksum(superblock: *SuperBlockHeader) void { + // `copy` is not covered by the checksum, but for our staging/working superblock headers it + // should always be zero. + assert(superblock.copy < constants.superblock_copies); + assert(superblock.copy == 0); + + assert(superblock.version == SuperBlockVersion); + assert(superblock.release_format.value > 0); + assert(superblock.flags == 0); + + assert(stdx.zeroed(&superblock.reserved)); + assert(stdx.zeroed(&superblock.vsr_state.reserved)); + assert(stdx.zeroed(&superblock.vsr_state.checkpoint.reserved)); + assert(stdx.zeroed(&superblock.vsr_headers_reserved)); + + assert(superblock.checksum_padding == 0); + assert(superblock.parent_padding == 0); + + superblock.checksum = superblock.calculate_checksum(); + } + + pub fn valid_checksum(superblock: *const SuperBlockHeader) bool { + return superblock.checksum == superblock.calculate_checksum() and + superblock.checksum_padding == 0; + } + + pub fn checkpoint_id(superblock: *const SuperBlockHeader) u128 { + return vsr.checksum(std.mem.asBytes(&superblock.vsr_state.checkpoint)); + } + + pub fn parent_checkpoint_id(superblock: *const SuperBlockHeader) u128 { + return superblock.vsr_state.checkpoint.parent_checkpoint_id; + } + + /// Does not consider { checksum, copy } when comparing equality. + pub fn equal(a: *const SuperBlockHeader, b: *const SuperBlockHeader) bool { + assert(a.release_format.value == b.release_format.value); + + assert(stdx.zeroed(&a.reserved)); + assert(stdx.zeroed(&b.reserved)); + + assert(stdx.zeroed(&a.vsr_state.reserved)); + assert(stdx.zeroed(&b.vsr_state.reserved)); + + assert(stdx.zeroed(&a.vsr_headers_reserved)); + assert(stdx.zeroed(&b.vsr_headers_reserved)); + + assert(a.checksum_padding == 0); + assert(b.checksum_padding == 0); + assert(a.parent_padding == 0); + assert(b.parent_padding == 0); + + if (a.version != b.version) return false; + if (a.cluster != b.cluster) return false; + if (a.sequence != b.sequence) return false; + if (a.parent != b.parent) return false; + if (!stdx.equal_bytes(VSRState, &a.vsr_state, &b.vsr_state)) return false; + if (a.vsr_headers_count != b.vsr_headers_count) return false; + if (!stdx.equal_bytes( + [constants.view_change_headers_max]vsr.Header.Prepare, + &a.vsr_headers_all, + &b.vsr_headers_all, + )) return false; + + return true; + } + + pub fn vsr_headers(superblock: *const SuperBlockHeader) vsr.Headers.ViewChangeSlice { + return vsr.Headers.ViewChangeSlice.init( + if (superblock.vsr_state.log_view < superblock.vsr_state.view) + .do_view_change + else + .start_view, + superblock.vsr_headers_all[0..superblock.vsr_headers_count], + ); + } + + pub fn manifest_references(superblock: *const SuperBlockHeader) ManifestReferences { + const checkpoint_state = &superblock.vsr_state.checkpoint; + return .{ + .oldest_address = checkpoint_state.manifest_oldest_address, + .oldest_checksum = checkpoint_state.manifest_oldest_checksum, + .newest_address = checkpoint_state.manifest_newest_address, + .newest_checksum = checkpoint_state.manifest_newest_checksum, + .block_count = checkpoint_state.manifest_block_count, + }; + } + + pub fn free_set_reference( + superblock: *const SuperBlockHeader, + bitset: vsr.FreeSet.BitsetKind, + ) TrailerReference { + switch (bitset) { + .blocks_acquired => { + return .{ + .checksum = superblock.vsr_state.checkpoint + .free_set_blocks_acquired_checksum, + .last_block_address = superblock.vsr_state.checkpoint + .free_set_blocks_acquired_last_block_address, + .last_block_checksum = superblock.vsr_state.checkpoint + .free_set_blocks_acquired_last_block_checksum, + .trailer_size = superblock.vsr_state.checkpoint + .free_set_blocks_acquired_size, + }; + }, + .blocks_released => { + return .{ + .checksum = superblock.vsr_state.checkpoint + .free_set_blocks_released_checksum, + .last_block_address = superblock.vsr_state.checkpoint + .free_set_blocks_released_last_block_address, + .last_block_checksum = superblock.vsr_state.checkpoint + .free_set_blocks_released_last_block_checksum, + .trailer_size = superblock.vsr_state.checkpoint + .free_set_blocks_released_size, + }; + }, + } + } + + pub fn client_sessions_reference(superblock: *const SuperBlockHeader) TrailerReference { + const checkpoint = &superblock.vsr_state.checkpoint; + return .{ + .checksum = checkpoint.client_sessions_checksum, + .last_block_address = checkpoint.client_sessions_last_block_address, + .last_block_checksum = checkpoint.client_sessions_last_block_checksum, + .trailer_size = checkpoint.client_sessions_size, + }; + } +}; + +pub const ManifestReferences = struct { + /// The chronologically first manifest block in the chain. + oldest_checksum: u128, + oldest_address: u64, + /// The chronologically last manifest block in the chain. + newest_checksum: u128, + newest_address: u64, + /// The number of manifest blocks in the chain. + block_count: u32, + + pub fn empty(references: *const ManifestReferences) bool { + if (references.block_count == 0) { + assert(references.oldest_address == 0); + assert(references.oldest_checksum == 0); + assert(references.newest_address == 0); + assert(references.newest_checksum == 0); + return true; + } else { + assert(references.oldest_address != 0); + assert(references.newest_address != 0); + return false; + } + } +}; + +pub const TrailerReference = struct { + /// Checksum over the entire encoded trailer. + checksum: u128, + last_block_address: u64, + last_block_checksum: u128, + trailer_size: u64, + + pub fn empty(reference: *const TrailerReference) bool { + if (reference.trailer_size == 0) { + assert(reference.checksum == vsr.checksum(&.{})); + assert(reference.last_block_address == 0); + assert(reference.last_block_checksum == 0); + return true; + } else { + assert(reference.last_block_address > 0); + return false; + } + } +}; + +comptime { + switch (constants.superblock_copies) { + 4, 6, 8 => {}, + else => @compileError("superblock_copies must be either { 4, 6, 8 } for flexible quorums."), + } +} + +/// The size of the entire superblock storage zone. +pub const superblock_zone_size = superblock_copy_size * constants.superblock_copies; + +/// Leave enough padding after every superblock copy so that it is feasible, in the future, to +/// modify the `pipeline_prepare_queue_max` of an existing cluster (up to a maximum of clients_max). +/// (That is, this space is reserved for potential `vsr_headers`). +const superblock_copy_padding: comptime_int = stdx.div_ceil( + (constants.clients_max - constants.pipeline_prepare_queue_max) * @sizeOf(vsr.Header), + constants.sector_size, +) * constants.sector_size; + +/// The size of an individual superblock header copy, including padding. +pub const superblock_copy_size = @sizeOf(SuperBlockHeader) + superblock_copy_padding; +comptime { + assert(superblock_copy_padding % constants.sector_size == 0); + assert(superblock_copy_size % constants.sector_size == 0); +} + +/// The size of a data file that has an empty grid. +pub const data_file_size_min = + superblock_zone_size + + constants.journal_size + + constants.client_replies_size + + vsr.Zone.size(.grid_padding).?; + +/// This table shows the sequence number progression of the SuperBlock's headers. +/// +/// action working staging disk +/// format seq seq seq +/// 0 - Initially the file has no headers. +/// 0 1 - +/// 0 1 1 Write a copyset for the first sequence. +/// 1 1 1 Read quorum; verify 3/4 are valid. +/// +/// open seq seq seq +/// a +/// a a Read quorum; verify 2/4 are valid. +/// a (a) a Repair any broken copies of `a`. +/// +/// checkpoint seq seq seq +/// a a a +/// a a+1 +/// a a+1 a+1 +/// a+1 a+1 a+1 Read quorum; verify 3/4 are valid. +/// +/// view_change seq seq seq +/// a a +/// a a+1 a The new sequence reuses the original parent. +/// a a+1 a+1 +/// a+1 a+1 a+1 Read quorum; verify 3/4 are valid. +/// working staging disk +/// +pub fn SuperBlockType(comptime Storage: type) type { + return struct { + const SuperBlock = @This(); + + pub const Context = struct { + superblock: *SuperBlock, + callback: *const fn (context: *Context) void, + caller: Caller, + + write: Storage.Write = undefined, + read: Storage.Read = undefined, + read_threshold: ?Quorums.Threshold = null, + copy: ?u8 = null, + /// Used by format(), checkpoint(), view_change(). + vsr_state: ?SuperBlockHeader.VSRState = null, + /// Used by format() and view_change(). + vsr_headers: ?vsr.Headers.ViewChangeArray = null, + repairs: ?Quorums.RepairIterator = null, // Used by open(). + }; + + storage: *Storage, + + /// The superblock that was recovered at startup after a crash or that was last written. + working: *align(constants.sector_size) SuperBlockHeader, + + /// The superblock that will replace the current working superblock once written. + /// We cannot mutate any working state directly until it is safely on stable storage. + /// Otherwise, we may accidentally externalize guarantees that are not yet durable. + staging: *align(constants.sector_size) SuperBlockHeader, + + /// The copies that we read into at startup or when verifying the written superblock. + reading: []align(constants.sector_size) SuperBlockHeader, + + /// It might seem that, at startup, we simply install the copy with the highest sequence. + /// + /// However, there's a scenario where: + /// 1. We are able to write sequence 7 to 3/4 copies, with the last write being lost. + /// 2. We startup and read all copies, with reads misdirected to the copy with sequence 6. + /// + /// Another scenario: + /// 1. We begin to write sequence 7 to 1 copy and then crash. + /// 2. At startup, the read to this copy fails, and we recover at sequence=6. + /// 3. We then checkpoint another sequence 7 to 3/4 copies and crash. + /// 4. At startup, we then see 4 copies with the same sequence with 1 checksum different. + /// + /// To mitigate these scenarios, we ensure that we are able to read a quorum of copies. + /// This also gives us confidence that our working superblock has sufficient redundancy. + quorums: Quorums = Quorums{}, + + /// Whether the superblock has been opened. An open superblock may not be formatted. + opened: bool = false, + /// Runtime limit on the size of the datafile. + storage_size_limit: u64, + + /// There may only be a single caller queued at a time, to ensure that the VSR protocol is + /// careful to submit at most one view change at a time. + queue_head: ?*Context = null, + queue_tail: ?*Context = null, + + /// Set to non-null after open(). + /// Used for logging. + replica_index: ?u8 = null, + + pub const Options = struct { + storage: *Storage, + storage_size_limit: u64, + }; + + pub fn init(allocator: mem.Allocator, options: Options) !SuperBlock { + assert(options.storage_size_limit >= data_file_size_min); + assert(options.storage_size_limit <= constants.storage_size_limit_max); + assert(options.storage_size_limit % constants.sector_size == 0); + + const a = try allocator.alignedAlloc(SuperBlockHeader, constants.sector_size, 1); + errdefer allocator.free(a); + + const b = try allocator.alignedAlloc(SuperBlockHeader, constants.sector_size, 1); + errdefer allocator.free(b); + + const reading = try allocator.alignedAlloc( + [constants.superblock_copies]SuperBlockHeader, + constants.sector_size, + 1, + ); + errdefer allocator.free(reading); + + return SuperBlock{ + .storage = options.storage, + .working = &a[0], + .staging = &b[0], + .reading = &reading[0], + .storage_size_limit = options.storage_size_limit, + }; + } + + pub fn deinit(superblock: *SuperBlock, allocator: mem.Allocator) void { + allocator.destroy(superblock.working); + allocator.destroy(superblock.staging); + allocator.free(superblock.reading); + } + + pub const FormatOptions = struct { + cluster: u128, + release: vsr.Release, + replica: u8, + replica_count: u8, + }; + + pub fn format( + superblock: *SuperBlock, + callback: *const fn (context: *Context) void, + context: *Context, + options: FormatOptions, + ) void { + assert(!superblock.opened); + assert(superblock.replica_index == null); + + assert(options.release.value > 0); + assert(options.replica_count > 0); + assert(options.replica_count <= constants.replicas_max); + assert(options.replica < options.replica_count + constants.standbys_max); + + const members = vsr.root_members(options.cluster); + const replica_id = members[options.replica]; + + superblock.replica_index = vsr.member_index(&members, replica_id); + + // This working copy provides the parent checksum, and will not be written to disk. + // We therefore use zero values to make this parent checksum as stable as possible. + superblock.working.* = .{ + .copy = 0, + .version = SuperBlockVersion, + .sequence = 0, + .release_format = options.release, + .cluster = options.cluster, + .parent = 0, + .vsr_state = .{ + .checkpoint = .{ + .header = mem.zeroes(vsr.Header.Prepare), + .parent_checkpoint_id = 0, + .grandparent_checkpoint_id = 0, + .manifest_oldest_checksum = 0, + .manifest_oldest_address = 0, + .manifest_newest_checksum = 0, + .manifest_newest_address = 0, + .manifest_block_count = 0, + .free_set_blocks_acquired_checksum = 0, + .free_set_blocks_released_checksum = 0, + .free_set_blocks_acquired_last_block_checksum = 0, + .free_set_blocks_released_last_block_checksum = 0, + .free_set_blocks_acquired_last_block_address = 0, + .free_set_blocks_released_last_block_address = 0, + .free_set_blocks_acquired_size = 0, + .free_set_blocks_released_size = 0, + .client_sessions_checksum = 0, + .client_sessions_last_block_checksum = 0, + .client_sessions_last_block_address = 0, + .client_sessions_size = 0, + .storage_size = 0, + .snapshots_block_checksum = 0, + .snapshots_block_address = 0, + .release = vsr.Release.zero, + }, + .replica_id = replica_id, + .members = members, + .commit_max = 0, + .sync_op_min = 0, + .sync_op_max = 0, + .sync_view = 0, + .log_view = 0, + .view = 0, + .replica_count = options.replica_count, + }, + .vsr_headers_count = 0, + .vsr_headers_all = mem.zeroes( + [constants.view_change_headers_max]vsr.Header.Prepare, + ), + }; + + superblock.working.set_checksum(); + + context.* = .{ + .superblock = superblock, + .callback = callback, + .caller = .format, + .vsr_state = SuperBlockHeader.VSRState.root(.{ + .cluster = options.cluster, + .release = options.release, + .replica_id = replica_id, + .members = members, + .replica_count = options.replica_count, + }), + .vsr_headers = vsr.Headers.ViewChangeArray.root(options.cluster), + }; + + // TODO At a higher layer, we must: + // 1. verify that there is no valid superblock, and + // 2. zero the superblock, WAL and client table to ensure storage determinism. + + superblock.acquire(context); + } + + pub fn open( + superblock: *SuperBlock, + callback: *const fn (context: *Context) void, + context: *Context, + ) void { + assert(!superblock.opened); + + context.* = .{ + .superblock = superblock, + .callback = callback, + .caller = .open, + }; + + superblock.acquire(context); + } + + const UpdateCheckpoint = struct { + header: vsr.Header.Prepare, + view_attributes: ?struct { + log_view: u32, + view: u32, + headers: *const vsr.Headers.ViewChangeArray, + }, + commit_max: u64, + sync_op_min: u64, + sync_op_max: u64, + manifest_references: ManifestReferences, + free_set_references: struct { + blocks_acquired: TrailerReference, + blocks_released: TrailerReference, + }, + client_sessions_reference: TrailerReference, + storage_size: u64, + release: vsr.Release, + }; + + /// Must update the commit_min and commit_min_checksum. + pub fn checkpoint( + superblock: *SuperBlock, + callback: *const fn (context: *Context) void, + context: *Context, + update: UpdateCheckpoint, + ) void { + assert(superblock.opened); + assert(update.header.op <= update.commit_max); + assert(update.header.op > superblock.staging.vsr_state.checkpoint.header.op); + assert(update.header.checksum != + superblock.staging.vsr_state.checkpoint.header.checksum); + assert(update.sync_op_min <= update.sync_op_max); + assert(update.release.value >= superblock.staging.vsr_state.checkpoint.release.value); + + assert(update.storage_size <= superblock.storage_size_limit); + assert(update.storage_size >= data_file_size_min); + assert((update.storage_size == data_file_size_min) == + (update.free_set_references.blocks_acquired.empty() and + update.free_set_references.blocks_released.empty())); + + // NOTE: Within the vsr_state.checkpoint assignment below, do not read from vsr_state + // directly. A miscompilation bug (as of Zig 0.11.0) causes fields to receive the + // incorrect values. + const vsr_state_staging = superblock.staging.vsr_state; + const update_client_sessions = &update.client_sessions_reference; + + var vsr_state = superblock.staging.vsr_state; + vsr_state.checkpoint = .{ + .header = update.header, + .parent_checkpoint_id = superblock.staging.checkpoint_id(), + .grandparent_checkpoint_id = vsr_state_staging.checkpoint.parent_checkpoint_id, + + .free_set_blocks_acquired_checksum = update.free_set_references + .blocks_acquired.checksum, + .free_set_blocks_released_checksum = update.free_set_references + .blocks_released.checksum, + + .free_set_blocks_acquired_size = update.free_set_references + .blocks_acquired.trailer_size, + .free_set_blocks_released_size = update.free_set_references + .blocks_released.trailer_size, + + .free_set_blocks_acquired_last_block_checksum = update.free_set_references + .blocks_acquired.last_block_checksum, + .free_set_blocks_released_last_block_checksum = update.free_set_references + .blocks_released.last_block_checksum, + + .free_set_blocks_acquired_last_block_address = update.free_set_references + .blocks_acquired.last_block_address, + .free_set_blocks_released_last_block_address = update.free_set_references + .blocks_released.last_block_address, + + .client_sessions_checksum = update_client_sessions.checksum, + .client_sessions_last_block_checksum = update_client_sessions.last_block_checksum, + .client_sessions_last_block_address = update_client_sessions.last_block_address, + .client_sessions_size = update.client_sessions_reference.trailer_size, + + .manifest_oldest_checksum = update.manifest_references.oldest_checksum, + .manifest_oldest_address = update.manifest_references.oldest_address, + .manifest_newest_checksum = update.manifest_references.newest_checksum, + .manifest_newest_address = update.manifest_references.newest_address, + .manifest_block_count = update.manifest_references.block_count, + + .storage_size = update.storage_size, + .snapshots_block_checksum = vsr_state_staging.checkpoint.snapshots_block_checksum, + .snapshots_block_address = vsr_state_staging.checkpoint.snapshots_block_address, + .release = update.release, + }; + vsr_state.commit_max = update.commit_max; + vsr_state.sync_op_min = update.sync_op_min; + vsr_state.sync_op_max = update.sync_op_max; + vsr_state.sync_view = 0; + if (update.view_attributes) |*view_attributes| { + assert(view_attributes.log_view <= view_attributes.view); + view_attributes.headers.verify(); + vsr_state.log_view = view_attributes.log_view; + vsr_state.view = view_attributes.view; + } + + assert(superblock.staging.vsr_state.would_be_updated_by(vsr_state)); + + context.* = .{ + .superblock = superblock, + .callback = callback, + .caller = .checkpoint, + .vsr_state = vsr_state, + .vsr_headers = if (update.view_attributes) |*view_attributes| + view_attributes.headers.* + else + vsr.Headers.ViewChangeArray.init( + superblock.staging.vsr_headers().command, + superblock.staging.vsr_headers().slice, + ), + }; + superblock.log_context(context); + superblock.acquire(context); + } + + const UpdateViewChange = struct { + commit_max: u64, + log_view: u32, + view: u32, + headers: *const vsr.Headers.ViewChangeArray, + sync_checkpoint: ?struct { + checkpoint: *const vsr.CheckpointState, + sync_op_min: u64, + sync_op_max: u64, + }, + }; + + /// The replica calls view_change(): + /// + /// - to persist its view/log_view — it cannot advertise either value until it is certain + /// they will never backtrack. + /// - to update checkpoint during sync + /// + /// The update must advance view/log_view (monotonically increasing) or checkpoint. + // TODO: the current naming confusing and needs changing: during sync, this function doesn't + // necessary advance the view. + pub fn view_change( + superblock: *SuperBlock, + callback: *const fn (context: *Context) void, + context: *Context, + update: UpdateViewChange, + ) void { + assert(superblock.opened); + assert(superblock.staging.vsr_state.commit_max <= update.commit_max); + assert(superblock.staging.vsr_state.view <= update.view); + assert(superblock.staging.vsr_state.log_view <= update.log_view); + assert(superblock.staging.vsr_state.log_view < update.log_view or + superblock.staging.vsr_state.view < update.view or + update.sync_checkpoint != null); + assert((update.headers.command == .start_view and update.log_view == update.view) or + (update.headers.command == .do_view_change and update.log_view < update.view)); + assert( + superblock.staging.vsr_state.checkpoint.header.op <= update.headers.array.get(0).op, + ); + + update.headers.verify(); + assert(update.view >= update.log_view); + + var vsr_state = superblock.staging.vsr_state; + vsr_state.commit_max = update.commit_max; + vsr_state.log_view = update.log_view; + vsr_state.view = update.view; + if (update.sync_checkpoint) |*sync_checkpoint| { + assert(superblock.staging.vsr_state.checkpoint.header.op < + sync_checkpoint.checkpoint.header.op); + + const checkpoint_next = vsr.Checkpoint.checkpoint_after( + superblock.staging.vsr_state.checkpoint.header.op, + ); + const checkpoint_next_next = vsr.Checkpoint.checkpoint_after(checkpoint_next); + + if (sync_checkpoint.checkpoint.header.op == checkpoint_next) { + assert(sync_checkpoint.checkpoint.parent_checkpoint_id == + superblock.staging.checkpoint_id()); + } else if (sync_checkpoint.checkpoint.header.op == checkpoint_next_next) { + assert(sync_checkpoint.checkpoint.grandparent_checkpoint_id == + superblock.staging.checkpoint_id()); + } + + vsr_state.checkpoint = sync_checkpoint.checkpoint.*; + vsr_state.sync_op_min = sync_checkpoint.sync_op_min; + vsr_state.sync_op_max = sync_checkpoint.sync_op_max; + } + assert(superblock.staging.vsr_state.would_be_updated_by(vsr_state)); + + context.* = .{ + .superblock = superblock, + .callback = callback, + .caller = .view_change, + .vsr_state = vsr_state, + .vsr_headers = update.headers.*, + }; + superblock.log_context(context); + superblock.acquire(context); + } + + pub fn updating(superblock: *const SuperBlock, caller: Caller) bool { + assert(superblock.opened); + + if (superblock.queue_head) |head| { + if (head.caller == caller) return true; + } + + if (superblock.queue_tail) |tail| { + if (tail.caller == caller) return true; + } + + return false; + } + + fn write_staging(superblock: *SuperBlock, context: *Context) void { + assert(context.caller != .open); + assert(context.caller == .format or superblock.opened); + assert(context.copy == null); + context.vsr_state.?.assert_internally_consistent(); + assert(superblock.queue_head == context); + assert(superblock.queue_tail == null); + + superblock.staging.* = superblock.working.*; + superblock.staging.sequence = superblock.staging.sequence + 1; + superblock.staging.parent = superblock.staging.checksum; + superblock.staging.vsr_state = context.vsr_state.?; + + if (context.vsr_headers) |*headers| { + assert(context.caller.updates_vsr_headers()); + + superblock.staging.vsr_headers_count = headers.array.count_as(u32); + stdx.copy_disjoint( + .exact, + vsr.Header.Prepare, + superblock.staging.vsr_headers_all[0..headers.array.count()], + headers.array.const_slice(), + ); + @memset( + superblock.staging.vsr_headers_all[headers.array.count()..], + std.mem.zeroes(vsr.Header.Prepare), + ); + } else { + assert(!context.caller.updates_vsr_headers()); + } + + context.copy = 0; + superblock.staging.set_checksum(); + superblock.write_header(context); + } + + fn write_header(superblock: *SuperBlock, context: *Context) void { + assert(superblock.queue_head == context); + + // We update the working superblock for a checkpoint/format/view_change: + // open() does not update the working superblock, since it only writes to repair. + if (context.caller == .open) { + assert(superblock.staging.sequence == superblock.working.sequence); + } else { + assert(superblock.staging.sequence == superblock.working.sequence + 1); + assert(superblock.staging.parent == superblock.working.checksum); + } + + // The superblock cluster and replica should never change once formatted: + assert(superblock.staging.cluster == superblock.working.cluster); + assert(superblock.staging.vsr_state.replica_id == + superblock.working.vsr_state.replica_id); + + const storage_size = superblock.staging.vsr_state.checkpoint.storage_size; + assert(storage_size >= data_file_size_min); + assert(storage_size <= constants.storage_size_limit_max); + + assert(context.copy.? < constants.superblock_copies); + superblock.staging.copy = context.copy.?; + // Updating the copy number should not affect the checksum, which was previously set: + assert(superblock.staging.valid_checksum()); + + const buffer = mem.asBytes(superblock.staging); + const offset = superblock_copy_size * @as(u32, context.copy.?); + + log.debug("{?}: {s}: write_header: " ++ + "checksum={x:0>32} sequence={} copy={} size={} offset={}", .{ + superblock.replica_index, + @tagName(context.caller), + superblock.staging.checksum, + superblock.staging.sequence, + context.copy.?, + buffer.len, + offset, + }); + + SuperBlock.assert_bounds(offset, buffer.len); + + superblock.storage.write_sectors( + write_header_callback, + &context.write, + buffer, + .superblock, + offset, + ); + } + + fn write_header_callback(write: *Storage.Write) void { + const context: *Context = @alignCast(@fieldParentPtr("write", write)); + const superblock = context.superblock; + const copy = context.copy.?; + + assert(superblock.queue_head == context); + + assert(copy < constants.superblock_copies); + assert(copy == superblock.staging.copy); + + if (context.caller == .open) { + context.copy = null; + superblock.repair(context); + return; + } + + if (copy + 1 == constants.superblock_copies) { + context.copy = null; + superblock.read_working(context, .verify); + } else { + context.copy = copy + 1; + superblock.write_header(context); + } + } + + fn read_working( + superblock: *SuperBlock, + context: *Context, + threshold: Quorums.Threshold, + ) void { + assert(superblock.queue_head == context); + assert(context.copy == null); + assert(context.read_threshold == null); + + // We do not submit reads in parallel, as while this would shave off 1ms, it would also + // increase the risk that a single fault applies to more reads due to temporal locality. + // This would make verification reads more flaky when we do experience a read fault. + // See "An Analysis of Data Corruption in the Storage Stack". + + context.copy = 0; + context.read_threshold = threshold; + for (superblock.reading) |*copy| copy.* = undefined; + superblock.read_header(context); + } + + fn read_header(superblock: *SuperBlock, context: *Context) void { + assert(superblock.queue_head == context); + assert(context.copy.? < constants.superblock_copies); + assert(context.read_threshold != null); + + const buffer = mem.asBytes(&superblock.reading[context.copy.?]); + const offset = superblock_copy_size * @as(u32, context.copy.?); + + log.debug("{?}: {s}: read_header: copy={} size={} offset={}", .{ + superblock.replica_index, + @tagName(context.caller), + context.copy.?, + buffer.len, + offset, + }); + + SuperBlock.assert_bounds(offset, buffer.len); + + superblock.storage.read_sectors( + read_header_callback, + &context.read, + buffer, + .superblock, + offset, + ); + } + + fn read_header_callback(read: *Storage.Read) void { + const context: *Context = @alignCast(@fieldParentPtr("read", read)); + const superblock = context.superblock; + const threshold = context.read_threshold.?; + + assert(superblock.queue_head == context); + + assert(context.copy.? < constants.superblock_copies); + if (context.copy.? + 1 != constants.superblock_copies) { + context.copy = context.copy.? + 1; + superblock.read_header(context); + return; + } + + context.read_threshold = null; + context.copy = null; + + if (superblock.quorums.working(superblock.reading, threshold)) |quorum| { + assert(quorum.valid); + assert(quorum.copies.count() >= threshold.count()); + maybe(quorum.header.copy >= constants.superblock_copies); // `copy` may be corrupt. + + const working = quorum.header; + + // TODO: Remove the second condition when logic to translate CheckpointStateOld to + // CheckpointState is removed. + if (working.version != SuperBlockVersion and working.version != 1) { + log.err("found incompatible superblock version {}", .{working.version}); + @panic("cannot read superblock with incompatible version"); + } + + if (threshold == .verify) { + if (working.checksum != superblock.staging.checksum) { + @panic("superblock failed verification after writing"); + } + assert(working.equal(superblock.staging)); + } + + if (context.caller == .format) { + assert(working.sequence == 1); + assert(working.vsr_state.checkpoint.header.checksum == + vsr.Header.Prepare.root(working.cluster).checksum); + assert(working.vsr_state.checkpoint.free_set_blocks_acquired_size == 0); + assert(working.vsr_state.checkpoint.free_set_blocks_released_size == 0); + assert(working.vsr_state.checkpoint.client_sessions_size == 0); + assert(working.vsr_state.checkpoint.storage_size == data_file_size_min); + assert(working.vsr_state.checkpoint.header.op == 0); + assert(working.vsr_state.commit_max == 0); + assert(working.vsr_state.log_view == 0); + assert(working.vsr_state.view == 0); + assert(working.vsr_headers_count == 1); + + assert(working.vsr_state.replica_count <= constants.replicas_max); + assert(vsr.member_index( + &working.vsr_state.members, + working.vsr_state.replica_id, + ) != null); + } + + superblock.working.* = working.*; + superblock.staging.* = working.*; + + // Reset the copies, which may be nonzero due to corruption. + superblock.working.copy = 0; + superblock.staging.copy = 0; + + const working_checkpoint = &superblock.working.vsr_state.checkpoint; + const staging_checkpoint = &superblock.staging.vsr_state.checkpoint; + + // The SuperBlock on disk is an older version wherein @TypeOf(VSRState.checkpoint) + // is CheckpointStateOld. Translate CheckpointStateOld → CheckpointState (zeroing + // out new fields) and update the working and staging superblocks. + if (working.version == 1) { + const checkpoint_old: *const vsr.CheckpointStateOld = + @ptrCast(&working.vsr_state.checkpoint); + const checkpoint_new = vsr.CheckpointState{ + .header = checkpoint_old.header, + .parent_checkpoint_id = checkpoint_old.parent_checkpoint_id, + .grandparent_checkpoint_id = checkpoint_old.grandparent_checkpoint_id, + + .free_set_blocks_acquired_checksum = checkpoint_old.free_set_checksum, + .free_set_blocks_released_checksum = comptime vsr.checksum(&.{}), + .free_set_blocks_acquired_last_block_checksum = checkpoint_old + .free_set_last_block_checksum, + .free_set_blocks_released_last_block_checksum = 0, + .free_set_blocks_acquired_last_block_address = checkpoint_old + .free_set_last_block_address, + .free_set_blocks_released_last_block_address = 0, + .free_set_blocks_acquired_size = checkpoint_old.free_set_size, + .free_set_blocks_released_size = 0, + + .client_sessions_checksum = checkpoint_old.client_sessions_checksum, + .client_sessions_last_block_checksum = checkpoint_old + .client_sessions_last_block_checksum, + .client_sessions_last_block_address = checkpoint_old + .client_sessions_last_block_address, + .client_sessions_size = checkpoint_old.client_sessions_size, + + .manifest_oldest_checksum = checkpoint_old.manifest_oldest_checksum, + .manifest_oldest_address = checkpoint_old.manifest_oldest_address, + .manifest_newest_checksum = checkpoint_old.manifest_newest_checksum, + .manifest_newest_address = checkpoint_old.manifest_newest_address, + .manifest_block_count = checkpoint_old.manifest_block_count, + + .snapshots_block_checksum = checkpoint_old.snapshots_block_checksum, + .snapshots_block_address = checkpoint_old.snapshots_block_address, + + .storage_size = checkpoint_old.storage_size, + .release = checkpoint_old.release, + }; + + working_checkpoint.* = checkpoint_new; + staging_checkpoint.* = checkpoint_new; + superblock.working.version = SuperBlockVersion; + superblock.staging.version = SuperBlockVersion; + } + + log.debug( + "{[replica]?}: " ++ + "{[caller]s}: installed working superblock: checksum={[checksum]x:0>32} " ++ + "sequence={[sequence]} " ++ + "release={[release]} " ++ + "cluster={[cluster]x:0>32} replica_id={[replica_id]} " ++ + "size={[size]} " ++ + "free_set_blocks_acquired_size={[free_set_blocks_acquired_size]} " ++ + "free_set_blocks_released_size={[free_set_blocks_released_size]} " ++ + "client_sessions_size={[client_sessions_size]} " ++ + "checkpoint_id={[checkpoint_id]x:0>32} " ++ + "commit_min_checksum={[commit_min_checksum]} commit_min={[commit_min]} " ++ + "commit_max={[commit_max]} log_view={[log_view]} view={[view]} " ++ + "sync_op_min={[sync_op_min]} sync_op_max={[sync_op_max]} " ++ + "manifest_oldest_checksum={[manifest_oldest_checksum]} " ++ + "manifest_oldest_address={[manifest_oldest_address]} " ++ + "manifest_newest_checksum={[manifest_newest_checksum]} " ++ + "manifest_newest_address={[manifest_newest_address]} " ++ + "manifest_block_count={[manifest_block_count]} " ++ + "snapshots_block_checksum={[snapshots_block_checksum]} " ++ + "snapshots_block_address={[snapshots_block_address]}", + .{ + .replica = superblock.replica_index, + .caller = @tagName(context.caller), + .checksum = superblock.working.checksum, + .sequence = superblock.working.sequence, + .release = working_checkpoint.release, + .cluster = superblock.working.cluster, + .replica_id = superblock.working.vsr_state.replica_id, + .size = working_checkpoint.storage_size, + .free_set_blocks_acquired_size = working_checkpoint + .free_set_blocks_acquired_size, + .free_set_blocks_released_size = working_checkpoint + .free_set_blocks_released_size, + .client_sessions_size = working_checkpoint.client_sessions_size, + .checkpoint_id = superblock.working.checkpoint_id(), + .commit_min_checksum = working_checkpoint.header.checksum, + .commit_min = working_checkpoint.header.op, + .commit_max = superblock.working.vsr_state.commit_max, + .sync_op_min = superblock.working.vsr_state.sync_op_min, + .sync_op_max = superblock.working.vsr_state.sync_op_max, + .log_view = superblock.working.vsr_state.log_view, + .view = superblock.working.vsr_state.view, + .manifest_oldest_checksum = working_checkpoint.manifest_oldest_checksum, + .manifest_oldest_address = working_checkpoint.manifest_oldest_address, + .manifest_newest_checksum = working_checkpoint.manifest_newest_checksum, + .manifest_newest_address = working_checkpoint.manifest_newest_address, + .manifest_block_count = working_checkpoint.manifest_block_count, + .snapshots_block_checksum = working_checkpoint.snapshots_block_checksum, + .snapshots_block_address = working_checkpoint.snapshots_block_address, + }, + ); + for (superblock.working.vsr_headers().slice) |*header| { + log.debug("{?}: {s}: vsr_header: op={} checksum={}", .{ + superblock.replica_index, + @tagName(context.caller), + header.op, + header.checksum, + }); + } + + if (superblock.working.vsr_state.checkpoint.storage_size > + superblock.storage_size_limit) + { + vsr.fatal( + .storage_size_exceeds_limit, + "data file too large size={} > limit={}, " ++ + "restart the replica increasing '--limit-storage'", + .{ + superblock.working.vsr_state.checkpoint.storage_size, + superblock.storage_size_limit, + }, + ); + } + + if (context.caller == .open) { + if (context.repairs) |_| { + // We just verified that the repair completed. + assert(threshold == .verify); + superblock.release(context); + } else { + assert(threshold == .open); + + context.repairs = quorum.repairs(); + context.copy = null; + superblock.repair(context); + } + } else { + // TODO Consider calling TRIM() on Grid's free suffix after checkpointing. + superblock.release(context); + } + } else |err| switch (err) { + error.Fork => @panic("superblock forked"), + error.NotFound => @panic("superblock not found"), + error.QuorumLost => @panic("superblock quorum lost"), + error.ParentNotConnected => @panic("superblock parent not connected"), + error.ParentSkipped => @panic("superblock parent superseded"), + error.VSRStateNotMonotonic => @panic("superblock vsr state not monotonic"), + } + } + + fn repair(superblock: *SuperBlock, context: *Context) void { + assert(context.caller == .open); + assert(context.copy == null); + assert(superblock.queue_head == context); + + if (context.repairs.?.next()) |repair_copy| { + context.copy = repair_copy; + log.warn("{?}: repair: copy={}", .{ superblock.replica_index, repair_copy }); + + superblock.staging.* = superblock.working.*; + superblock.write_header(context); + } else { + superblock.release(context); + } + } + + fn acquire(superblock: *SuperBlock, context: *Context) void { + if (superblock.queue_head) |head| { + // All operations are mutually exclusive with themselves. + assert(head.caller != context.caller); + assert(Caller.transitions.get(head.caller).?.contains(context.caller)); + assert(superblock.queue_tail == null); + + log.debug("{?}: {s}: enqueued after {s}", .{ + superblock.replica_index, + @tagName(context.caller), + @tagName(head.caller), + }); + + superblock.queue_tail = context; + } else { + assert(superblock.queue_tail == null); + + superblock.queue_head = context; + log.debug("{?}: {s}: started", .{ + superblock.replica_index, + @tagName(context.caller), + }); + + if (Storage == @import("../testing/storage.zig").Storage) { + // We should have finished all pending superblock io before starting any more. + superblock.storage.assert_no_pending_reads(.superblock); + superblock.storage.assert_no_pending_writes(.superblock); + } + + if (context.caller == .open) { + superblock.read_working(context, .open); + } else { + superblock.write_staging(context); + } + } + } + + fn release(superblock: *SuperBlock, context: *Context) void { + assert(superblock.queue_head == context); + + log.debug("{?}: {s}: complete", .{ + superblock.replica_index, + @tagName(context.caller), + }); + + if (Storage == @import("../testing/storage.zig").Storage) { + // We should have finished all pending io by now. + superblock.storage.assert_no_pending_reads(.superblock); + superblock.storage.assert_no_pending_writes(.superblock); + } + + switch (context.caller) { + .format => {}, + .open => { + assert(!superblock.opened); + superblock.opened = true; + superblock.replica_index = vsr.member_index( + &superblock.working.vsr_state.members, + superblock.working.vsr_state.replica_id, + ).?; + }, + .checkpoint, + .view_change, + => { + assert(stdx.equal_bytes( + SuperBlockHeader.VSRState, + &superblock.staging.vsr_state, + &context.vsr_state.?, + )); + assert(stdx.equal_bytes( + SuperBlockHeader.VSRState, + &superblock.working.vsr_state, + &context.vsr_state.?, + )); + }, + } + + const queue_tail = superblock.queue_tail; + superblock.queue_head = null; + superblock.queue_tail = null; + if (queue_tail) |tail| superblock.acquire(tail); + + context.callback(context); + } + + fn assert_bounds(offset: u64, size: u64) void { + assert(offset + size <= superblock_zone_size); + } + + fn log_context(superblock: *const SuperBlock, context: *const Context) void { + log.debug("{[replica]?}: {[caller]s}: " ++ + "commit_min={[commit_min_old]}..{[commit_min_new]} " ++ + "commit_max={[commit_max_old]}..{[commit_max_new]} " ++ + "commit_min_checksum={[commit_min_checksum_old]}..{[commit_min_checksum_new]} " ++ + "log_view={[log_view_old]}..{[log_view_new]} " ++ + "view={[view_old]}..{[view_new]} " ++ + "head={[head_old]}..{[head_new]?}", .{ + .replica = superblock.replica_index, + .caller = @tagName(context.caller), + + .commit_min_old = superblock.staging.vsr_state.checkpoint.header.op, + .commit_min_new = context.vsr_state.?.checkpoint.header.op, + + .commit_max_old = superblock.staging.vsr_state.commit_max, + .commit_max_new = context.vsr_state.?.commit_max, + + .commit_min_checksum_old = superblock.staging.vsr_state.checkpoint.header.checksum, + .commit_min_checksum_new = context.vsr_state.?.checkpoint.header.checksum, + + .log_view_old = superblock.staging.vsr_state.log_view, + .log_view_new = context.vsr_state.?.log_view, + + .view_old = superblock.staging.vsr_state.view, + .view_new = context.vsr_state.?.view, + + .head_old = superblock.staging.vsr_headers().slice[0].checksum, + .head_new = if (context.vsr_headers) |*headers| + @as(?u128, headers.array.get(0).checksum) + else + null, + }); + } + }; +} + +pub const Caller = enum { + format, + open, + checkpoint, + view_change, + + /// Beyond formatting and opening of the superblock, which are mutually exclusive of all + /// other operations, only the following queue combinations are allowed: + /// + /// from state → to states + const transitions = sets: { + const Set = std.enums.EnumSet(Caller); + break :sets std.enums.EnumMap(Caller, Set).init(.{ + .format = Set.init(.{}), + .open = Set.init(.{}), + .checkpoint = Set.init(.{ .view_change = true }), + .view_change = Set.init(.{ .checkpoint = true }), + }); + }; + + fn updates_vsr_headers(caller: Caller) bool { + return switch (caller) { + .format => true, + .open => unreachable, + .checkpoint => true, + .view_change => true, + }; + } +}; + +test "SuperBlockHeader" { + const expect = std.testing.expect; + + var a = std.mem.zeroes(SuperBlockHeader); + a.version = SuperBlockVersion; + a.release_format = vsr.Release.minimum; + a.set_checksum(); + + assert(a.copy == 0); + try expect(a.valid_checksum()); + + a.copy += 1; + try expect(a.valid_checksum()); + + a.version += 1; + try expect(!a.valid_checksum()); +} diff --git a/superblock_quorums.zig b/superblock_quorums.zig new file mode 100644 index 0000000..838c39d --- /dev/null +++ b/superblock_quorums.zig @@ -0,0 +1,373 @@ +const std = @import("std"); +const assert = std.debug.assert; +const log = std.log.scoped(.superblock_quorums); + +const superblock = @import("./superblock.zig"); +const SuperBlockHeader = superblock.SuperBlockHeader; + +pub const Options = struct { + superblock_copies: u8, +}; + +pub fn QuorumsType(comptime options: Options) type { + return struct { + const Quorums = @This(); + + const Quorum = struct { + header: *const SuperBlockHeader, + valid: bool = false, + /// Track which copies are a member of the quorum. + /// Used to ignore duplicate copies of a header when determining a quorum. + copies: QuorumCount = QuorumCount.initEmpty(), + /// An integer value indicates the copy index found in the corresponding slot. + /// A `null` value indicates that the copy is invalid or not a member of the working + /// quorum. All copies belong to the same (valid, working) quorum. + slots: [options.superblock_copies]?u8 = [_]?u8{null} ** options.superblock_copies, + + pub fn repairs(quorum: Quorum) RepairIterator { + assert(quorum.valid); + return .{ .slots = quorum.slots }; + } + }; + + pub const QuorumCount = std.StaticBitSet(options.superblock_copies); + + pub const Error = error{ + Fork, + NotFound, + QuorumLost, + ParentNotConnected, + ParentSkipped, + VSRStateNotMonotonic, + }; + + /// We use flexible quorums for even quorums with write quorum > read quorum, for example: + /// * When writing, we must verify that at least 3/4 copies were written. + /// * At startup, we must verify that at least 2/4 copies were read. + /// + /// This ensures that our read and write quorums will intersect. + /// Using flexible quorums in this way increases resiliency of the superblock. + pub const Threshold = enum { + verify, + open, + // Working these threshold out by formula is easy to get wrong, so enumerate them: + // The rule is that the write quorum plus the read quorum must be exactly copies + 1. + + pub fn count(threshold: Threshold) u8 { + return switch (threshold) { + .verify => switch (options.superblock_copies) { + 4 => 3, + 6 => 4, + 8 => 5, + else => unreachable, + }, + // The open quorum must allow for at least two copy faults, because we update + // copies in place, temporarily impairing one copy. + .open => switch (options.superblock_copies) { + 4 => 2, + 6 => 3, + 8 => 4, + else => unreachable, + }, + }; + } + }; + + array: [options.superblock_copies]Quorum = undefined, + count: u8 = 0, + + /// Returns the working superblock according to the quorum with the highest sequence number. + /// + /// * When a member of the parent quorum is still present, verify that the highest quorum is + /// connected. + /// * When there are 2 quorums: 1/4 new and 3/4 old, favor the 3/4 old since it is safer to + /// repair. + /// TODO Re-examine this now that there are no superblock trailers to worry about. + pub fn working( + quorums: *Quorums, + copies: []const SuperBlockHeader, + threshold: Threshold, + ) Error!Quorum { + assert(copies.len == options.superblock_copies); + assert(threshold.count() >= 2 and threshold.count() <= 5); + + quorums.array = undefined; + quorums.count = 0; + + for (copies, 0..) |*copy, index| quorums.count_copy(copy, index, threshold); + + std.mem.sort(Quorum, quorums.slice(), {}, sort_priority_descending); + + for (quorums.slice()) |quorum| { + if (quorum.copies.count() == options.superblock_copies) { + log.debug("quorum: checksum={x} parent={x} sequence={} count={} valid={}", .{ + quorum.header.checksum, + quorum.header.parent, + quorum.header.sequence, + quorum.copies.count(), + quorum.valid, + }); + } else { + log.warn("quorum: checksum={x} parent={x} sequence={} count={} valid={}", .{ + quorum.header.checksum, + quorum.header.parent, + quorum.header.sequence, + quorum.copies.count(), + quorum.valid, + }); + } + } + + // No working copies of any sequence number exist in the superblock storage zone at all. + if (quorums.slice().len == 0) return error.NotFound; + + // At least one copy or quorum exists. + const b = quorums.slice()[0]; + + // Verify that the remaining quorums are correctly sorted: + for (quorums.slice()[1..]) |a| { + assert(sort_priority_descending({}, b, a)); + assert(a.header.valid_checksum()); + } + + // Even the best copy with the most quorum still has inadequate quorum. + if (!b.valid) return error.QuorumLost; + + // If a parent quorum is present (either complete or incomplete) it must be connected to + // the new working quorum. The parent quorum can exist due to: + // - a crash during checkpoint()/view_change() before writing all copies + // - a lost or misdirected write + // - a latent sector error that prevented a write + for (quorums.slice()[1..]) |a| { + if (a.header.cluster != b.header.cluster) { + log.warn("superblock copy={} has cluster={} instead of {}", .{ + a.header.copy, + a.header.cluster, + b.header.cluster, + }); + continue; + } + + if (a.header.vsr_state.replica_id != b.header.vsr_state.replica_id) { + log.warn("superblock copy={} has replica_id={} instead of {}", .{ + a.header.copy, + a.header.vsr_state.replica_id, + b.header.vsr_state.replica_id, + }); + continue; + } + + if (a.header.sequence == b.header.sequence) { + // Two quorums, same cluster+replica+sequence, but different checksums. + // This shouldn't ever happen — but if it does, we can't safely repair. + assert(a.header.checksum != b.header.checksum); + return error.Fork; + } + + if (a.header.sequence > b.header.sequence + 1) { + // We read sequences such as (2,2,2,4) — 2 isn't safe to use, but there isn't a + // valid quorum for 4 either. + return error.ParentSkipped; + } + + if (a.header.sequence + 1 == b.header.sequence) { + assert(a.header.checksum != b.header.checksum); + assert(a.header.cluster == b.header.cluster); + assert(a.header.vsr_state.replica_id == b.header.vsr_state.replica_id); + + if (a.header.checksum != b.header.parent) { + return error.ParentNotConnected; + } else if (!a.header.vsr_state.monotonic(b.header.vsr_state)) { + return error.VSRStateNotMonotonic; + } else { + assert(b.header.valid_checksum()); + + return b; + } + } + } + + assert(b.header.valid_checksum()); + return b; + } + + fn count_copy( + quorums: *Quorums, + copy: *const SuperBlockHeader, + slot: usize, + threshold: Threshold, + ) void { + assert(slot < options.superblock_copies); + assert(threshold.count() >= 2 and threshold.count() <= 5); + + if (!copy.valid_checksum()) { + log.debug("copy: {}/{}: invalid checksum", .{ slot, options.superblock_copies }); + return; + } + + if (copy.copy == slot) { + log.debug("copy: {}/{}: checksum={x} parent={x} sequence={}", .{ + slot, + options.superblock_copies, + copy.checksum, + copy.parent, + copy.sequence, + }); + } else if (copy.copy >= options.superblock_copies) { + log.warn("copy: {}/{}: checksum={x} parent={x} sequence={} corrupt copy={}", .{ + slot, + options.superblock_copies, + copy.checksum, + copy.parent, + copy.sequence, + copy.copy, + }); + } else { + // If our read was misdirected, we definitely still want to count the copy. + // We must just be careful to count it idempotently. + log.warn( + "copy: {}/{}: checksum={x} parent={x} sequence={} misdirected from copy={}", + .{ + slot, + options.superblock_copies, + copy.checksum, + copy.parent, + copy.sequence, + copy.copy, + }, + ); + } + + var quorum = quorums.find_or_insert_quorum_for_copy(copy); + assert(quorum.header.checksum == copy.checksum); + assert(quorum.header.equal(copy)); + + if (copy.copy >= options.superblock_copies) { + // This header is a valid member of the quorum, but with an unexpected copy number. + // The "SuperBlockHeader.copy" field is not protected by the checksum, so if that + // byte (and only that byte) is corrupted, the superblock is still valid — but we + // don't know for certain which copy this was supposed to be. + // We make the assumption that this was not a double-fault (corrupt + misdirect) — + // that is, the copy is in the correct slot, and its copy index is simply corrupt. + quorum.slots[slot] = @intCast(slot); + quorum.copies.set(slot); + } else if (quorum.copies.isSet(copy.copy)) { + // Ignore the duplicate copy. + } else { + quorum.slots[slot] = @intCast(copy.copy); + quorum.copies.set(copy.copy); + } + + quorum.valid = quorum.copies.count() >= threshold.count(); + } + + fn find_or_insert_quorum_for_copy( + quorums: *Quorums, + copy: *const SuperBlockHeader, + ) *Quorum { + assert(copy.valid_checksum()); + + for (quorums.array[0..quorums.count]) |*quorum| { + if (copy.checksum == quorum.header.checksum) return quorum; + } else { + quorums.array[quorums.count] = Quorum{ .header = copy }; + quorums.count += 1; + + return &quorums.array[quorums.count - 1]; + } + } + + fn slice(quorums: *Quorums) []Quorum { + return quorums.array[0..quorums.count]; + } + + fn sort_priority_descending(_: void, a: Quorum, b: Quorum) bool { + assert(a.header.checksum != b.header.checksum); + + if (a.valid and !b.valid) return true; + if (b.valid and !a.valid) return false; + + if (a.header.sequence > b.header.sequence) return true; + if (b.header.sequence > a.header.sequence) return false; + + if (a.copies.count() > b.copies.count()) return true; + if (b.copies.count() > a.copies.count()) return false; + + // The sort order must be stable and deterministic: + return a.header.checksum > b.header.checksum; + } + + /// Repair a quorum's copies in the safest known order. + /// Repair is complete when every copy is on-disk (not necessarily in its home slot). + /// + /// We must be careful when repairing superblock headers to avoid endangering our quorum if + /// an additional fault occurs. We primarily guard against torn header writes — preventing a + /// misdirected write from derailing repair is far more expensive and complex — but they are + /// likewise far less likely to occur. + /// + /// For example, consider this case: + /// 0. Sequence is initially A. + /// 1. Checkpoint sequence B. + /// 2. Write B₀ — ok. + /// 3. Write B₁ — misdirected to B₂'s slot. + /// 4. Crash. + /// 5. Recover with quorum B[B₀,A₁,B₁,A₃]. + /// If we repair the superblock quorum while only considering the valid copies (and not + /// slots) the following scenario could occur: + /// 6. We already have a valid B₀ and B₁, so begin writing B₂. + /// 7. Crash, tearing the B₂ write. + /// 8. Recover with quorum A[B₀,A₁,_,A₂]. + /// The working quorum backtracked from B to A! + pub const RepairIterator = struct { + /// An integer value indicates the copy index found in the corresponding slot. + /// A `null` value indicates that the copy is invalid or not a member of the working + /// quorum. All copies belong to the same (valid, working) quorum. + slots: [options.superblock_copies]?u8, + + /// Returns the slot/copy to repair next. + /// We never (deliberately) write a copy to a slot other than its own. This is simpler + /// to implement, and also reduces risk when one of open()'s reads was misdirected. + pub fn next(iterator: *RepairIterator) ?u8 { + // Corrupt copy indices have already been normalized. + for (iterator.slots) |slot| { + assert(slot == null or slot.? < options.superblock_copies); + } + + // Set bits indicate that the corresponding copy was found at least once. + var copies_any = QuorumCount.initEmpty(); + // Set bits indicate that the corresponding copy was found more than once. + var copies_duplicate = QuorumCount.initEmpty(); + + for (iterator.slots) |slot| { + if (slot) |copy| { + if (copies_any.isSet(copy)) copies_duplicate.set(copy); + copies_any.set(copy); + } + } + + // In descending order, our priorities for repair are: + // 1. The slot holds no header, and the copy was not found anywhere. + // 2. The slot holds no header, but its copy was found elsewhere. + // 3. The slot holds a misdirected header, but that copy is in another slot as well. + var a: ?u8 = null; + var b: ?u8 = null; + var c: ?u8 = null; + for (iterator.slots, 0..) |slot, i| { + if (slot == null and !copies_any.isSet(i)) a = @intCast(i); + if (slot == null and copies_any.isSet(i)) b = @intCast(i); + if (slot) |slot_copy| { + if (slot_copy != i and copies_duplicate.isSet(slot_copy)) c = @intCast(i); + } + } + + const repair = a orelse b orelse c orelse { + for (iterator.slots) |slot| assert(slot != null); + return null; + }; + + iterator.slots[repair] = repair; + return repair; + } + }; + }; +}