feat: track persist time in wal file content (#25614)

praveen/ring-buffer-optimizations
Trevor Hilton 2024-12-03 12:37:43 -08:00 committed by GitHub
parent d2fbd65a44
commit 0daa3f2f1d
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
5 changed files with 86 additions and 25 deletions

View File

@ -6,11 +6,14 @@ use influxdb3_id::{ColumnId, DbId};
use crate::*; use crate::*;
/// Create a new [`WalContents`] with the provided arguments that will have a `persisted_timestamp_ns`
/// of `0`.
pub fn wal_contents( pub fn wal_contents(
(min_timestamp_ns, max_timestamp_ns, wal_file_number): (i64, i64, u64), (min_timestamp_ns, max_timestamp_ns, wal_file_number): (i64, i64, u64),
ops: impl IntoIterator<Item = WalOp>, ops: impl IntoIterator<Item = WalOp>,
) -> WalContents { ) -> WalContents {
WalContents { WalContents {
persist_timestamp_ms: 0,
min_timestamp_ns, min_timestamp_ns,
max_timestamp_ns, max_timestamp_ns,
wal_file_number: WalFileSequenceNumber::new(wal_file_number), wal_file_number: WalFileSequenceNumber::new(wal_file_number),
@ -19,6 +22,23 @@ pub fn wal_contents(
} }
} }
/// Create a new [`WalContents`] with the provided arguments that will have a `persisted_timestamp_ns`
/// of `0`.
pub fn wal_contents_with_snapshot(
(min_timestamp_ns, max_timestamp_ns, wal_file_number): (i64, i64, u64),
ops: impl IntoIterator<Item = WalOp>,
snapshot: SnapshotDetails,
) -> WalContents {
WalContents {
persist_timestamp_ms: 0,
min_timestamp_ns,
max_timestamp_ns,
wal_file_number: WalFileSequenceNumber::new(wal_file_number),
ops: ops.into_iter().collect(),
snapshot: Some(snapshot),
}
}
pub fn write_batch_op(write_batch: WriteBatch) -> WalOp { pub fn write_batch_op(write_batch: WriteBatch) -> WalOp {
WalOp::Write(write_batch) WalOp::Write(write_batch)
} }

View File

@ -701,11 +701,15 @@ impl<'a> From<&FieldValue<'a>> for FieldData {
#[derive(Debug, Clone, Eq, PartialEq, Serialize, Deserialize)] #[derive(Debug, Clone, Eq, PartialEq, Serialize, Deserialize)]
pub struct WalContents { pub struct WalContents {
/// The time at which this WAL file is being persisted
pub persist_timestamp_ms: i64,
/// The min timestamp from any writes in the WAL file /// The min timestamp from any writes in the WAL file
pub min_timestamp_ns: i64, pub min_timestamp_ns: i64,
/// The max timestamp from any writes in the WAL file /// The max timestamp from any writes in the WAL file
pub max_timestamp_ns: i64, pub max_timestamp_ns: i64,
/// A number that increments for every generated WAL file
pub wal_file_number: WalFileSequenceNumber, pub wal_file_number: WalFileSequenceNumber,
/// The operations contained in the WAL file
pub ops: Vec<WalOp>, pub ops: Vec<WalOp>,
/// If present, the buffer should be snapshot after the contents of this file are loaded. /// If present, the buffer should be snapshot after the contents of this file are loaded.
pub snapshot: Option<SnapshotDetails>, pub snapshot: Option<SnapshotDetails>,

View File

@ -8,6 +8,7 @@ use bytes::Bytes;
use data_types::Timestamp; use data_types::Timestamp;
use futures_util::stream::StreamExt; use futures_util::stream::StreamExt;
use hashbrown::HashMap; use hashbrown::HashMap;
use iox_time::TimeProvider;
use object_store::path::{Path, PathPart}; use object_store::path::{Path, PathPart};
use object_store::{ObjectStore, PutPayload}; use object_store::{ObjectStore, PutPayload};
use observability_deps::tracing::{debug, error, info}; use observability_deps::tracing::{debug, error, info};
@ -29,6 +30,7 @@ impl WalObjectStore {
/// Creates a new WAL. This will replay files into the notifier and trigger any snapshots that /// Creates a new WAL. This will replay files into the notifier and trigger any snapshots that
/// exist in the WAL files that haven't been cleaned up yet. /// exist in the WAL files that haven't been cleaned up yet.
pub async fn new( pub async fn new(
time_provider: Arc<dyn TimeProvider>,
object_store: Arc<dyn ObjectStore>, object_store: Arc<dyn ObjectStore>,
host_identifier_prefix: impl Into<String> + Send, host_identifier_prefix: impl Into<String> + Send,
file_notifier: Arc<dyn WalFileNotifier>, file_notifier: Arc<dyn WalFileNotifier>,
@ -38,6 +40,7 @@ impl WalObjectStore {
) -> Result<Arc<Self>, crate::Error> { ) -> Result<Arc<Self>, crate::Error> {
let flush_interval = config.flush_interval; let flush_interval = config.flush_interval;
let wal = Self::new_without_replay( let wal = Self::new_without_replay(
time_provider,
object_store, object_store,
host_identifier_prefix, host_identifier_prefix,
file_notifier, file_notifier,
@ -54,6 +57,7 @@ impl WalObjectStore {
} }
fn new_without_replay( fn new_without_replay(
time_provider: Arc<dyn TimeProvider>,
object_store: Arc<dyn ObjectStore>, object_store: Arc<dyn ObjectStore>,
host_identifier_prefix: impl Into<String>, host_identifier_prefix: impl Into<String>,
file_notifier: Arc<dyn WalFileNotifier>, file_notifier: Arc<dyn WalFileNotifier>,
@ -67,7 +71,9 @@ impl WalObjectStore {
host_identifier_prefix: host_identifier_prefix.into(), host_identifier_prefix: host_identifier_prefix.into(),
file_notifier, file_notifier,
flush_buffer: Mutex::new(FlushBuffer::new( flush_buffer: Mutex::new(FlushBuffer::new(
Arc::clone(&time_provider),
WalBuffer { WalBuffer {
time_provider,
is_shutdown: false, is_shutdown: false,
wal_file_sequence_number, wal_file_sequence_number,
op_limit: config.max_write_buffer_size, op_limit: config.max_write_buffer_size,
@ -401,14 +407,20 @@ impl Wal for WalObjectStore {
#[derive(Debug)] #[derive(Debug)]
struct FlushBuffer { struct FlushBuffer {
time_provider: Arc<dyn TimeProvider>,
wal_buffer: WalBuffer, wal_buffer: WalBuffer,
snapshot_tracker: SnapshotTracker, snapshot_tracker: SnapshotTracker,
snapshot_semaphore: Arc<Semaphore>, snapshot_semaphore: Arc<Semaphore>,
} }
impl FlushBuffer { impl FlushBuffer {
fn new(wal_buffer: WalBuffer, snapshot_tracker: SnapshotTracker) -> Self { fn new(
time_provider: Arc<dyn TimeProvider>,
wal_buffer: WalBuffer,
snapshot_tracker: SnapshotTracker,
) -> Self {
Self { Self {
time_provider,
wal_buffer, wal_buffer,
snapshot_tracker, snapshot_tracker,
snapshot_semaphore: Arc::new(Semaphore::new(1)), snapshot_semaphore: Arc::new(Semaphore::new(1)),
@ -460,6 +472,7 @@ impl FlushBuffer {
fn flush_buffer_with_responses(&mut self) -> (WalContents, Vec<oneshot::Sender<WriteResult>>) { fn flush_buffer_with_responses(&mut self) -> (WalContents, Vec<oneshot::Sender<WriteResult>>) {
// swap out the filled buffer with a new one // swap out the filled buffer with a new one
let mut new_buffer = WalBuffer { let mut new_buffer = WalBuffer {
time_provider: Arc::clone(&self.time_provider),
is_shutdown: false, is_shutdown: false,
wal_file_sequence_number: self.wal_buffer.wal_file_sequence_number.next(), wal_file_sequence_number: self.wal_buffer.wal_file_sequence_number.next(),
op_limit: self.wal_buffer.op_limit, op_limit: self.wal_buffer.op_limit,
@ -481,8 +494,9 @@ impl FlushBuffer {
} }
} }
#[derive(Debug, Default)] #[derive(Debug)]
struct WalBuffer { struct WalBuffer {
time_provider: Arc<dyn TimeProvider>,
is_shutdown: bool, is_shutdown: bool,
wal_file_sequence_number: WalFileSequenceNumber, wal_file_sequence_number: WalFileSequenceNumber,
op_limit: usize, op_limit: usize,
@ -584,6 +598,7 @@ impl WalBuffer {
( (
WalContents { WalContents {
persist_timestamp_ms: self.time_provider.now().timestamp_millis(),
min_timestamp_ns, min_timestamp_ns,
max_timestamp_ns, max_timestamp_ns,
wal_file_number: self.wal_file_sequence_number, wal_file_number: self.wal_file_sequence_number,
@ -622,17 +637,21 @@ impl<'a> TryFrom<&'a Path> for WalFileSequenceNumber {
mod tests { mod tests {
use super::*; use super::*;
use crate::{ use crate::{
Field, FieldData, Gen1Duration, Row, SnapshotSequenceNumber, TableChunk, TableChunks, create, Field, FieldData, Gen1Duration, Row, SnapshotSequenceNumber, TableChunk,
TableChunks,
}; };
use async_trait::async_trait; use async_trait::async_trait;
use indexmap::IndexMap; use indexmap::IndexMap;
use influxdb3_id::{ColumnId, DbId, TableId}; use influxdb3_id::{ColumnId, DbId, TableId};
use iox_time::{MockProvider, Time};
use object_store::memory::InMemory; use object_store::memory::InMemory;
use std::any::Any; use std::any::Any;
use tokio::sync::oneshot::Receiver; use tokio::sync::oneshot::Receiver;
#[tokio::test(flavor = "multi_thread", worker_threads = 2)] #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
async fn write_flush_delete_and_load() { async fn write_flush_delete_and_load() {
let time_provider: Arc<dyn TimeProvider> =
Arc::new(MockProvider::new(Time::from_timestamp_nanos(0)));
let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new()); let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
let notifier: Arc<dyn WalFileNotifier> = Arc::new(TestNotfiier::default()); let notifier: Arc<dyn WalFileNotifier> = Arc::new(TestNotfiier::default());
let wal_config = WalConfig { let wal_config = WalConfig {
@ -642,6 +661,7 @@ mod tests {
gen1_duration: Gen1Duration::new_1m(), gen1_duration: Gen1Duration::new_1m(),
}; };
let wal = WalObjectStore::new_without_replay( let wal = WalObjectStore::new_without_replay(
Arc::clone(&time_provider),
Arc::clone(&object_store), Arc::clone(&object_store),
"my_host", "my_host",
Arc::clone(&notifier), Arc::clone(&notifier),
@ -738,11 +758,9 @@ mod tests {
// create wal file 1 // create wal file 1
let ret = wal.flush_buffer().await; let ret = wal.flush_buffer().await;
assert!(ret.is_none()); assert!(ret.is_none());
let file_1_contents = WalContents { let file_1_contents = create::wal_contents(
min_timestamp_ns: 1, (1, 62_000_000_000, 1),
max_timestamp_ns: 62_000000000, [WalOp::Write(WriteBatch {
wal_file_number: WalFileSequenceNumber(1),
ops: vec![WalOp::Write(WriteBatch {
database_id: DbId::from(0), database_id: DbId::from(0),
database_name: "db1".into(), database_name: "db1".into(),
table_chunks: IndexMap::from([( table_chunks: IndexMap::from([(
@ -802,18 +820,15 @@ mod tests {
min_time_ns: 1, min_time_ns: 1,
max_time_ns: 62_000000000, max_time_ns: 62_000000000,
})], })],
snapshot: None, );
};
// create wal file 2 // create wal file 2
wal.buffer_op_unconfirmed(op2.clone()).await.unwrap(); wal.buffer_op_unconfirmed(op2.clone()).await.unwrap();
assert!(wal.flush_buffer().await.is_none()); assert!(wal.flush_buffer().await.is_none());
let file_2_contents = WalContents { let file_2_contents = create::wal_contents(
min_timestamp_ns: 62000000000, (62_000_000_000, 62_000_000_000, 2),
max_timestamp_ns: 62000000000, [WalOp::Write(WriteBatch {
wal_file_number: WalFileSequenceNumber(2),
ops: vec![WalOp::Write(WriteBatch {
database_id: DbId::from(0), database_id: DbId::from(0),
database_name: "db1".into(), database_name: "db1".into(),
table_chunks: IndexMap::from([( table_chunks: IndexMap::from([(
@ -845,12 +860,12 @@ mod tests {
min_time_ns: 62_000000000, min_time_ns: 62_000000000,
max_time_ns: 62_000000000, max_time_ns: 62_000000000,
})], })],
snapshot: None, );
};
// before we trigger a snapshot, test replay with a new wal and notifier // before we trigger a snapshot, test replay with a new wal and notifier
let replay_notifier: Arc<dyn WalFileNotifier> = Arc::new(TestNotfiier::default()); let replay_notifier: Arc<dyn WalFileNotifier> = Arc::new(TestNotfiier::default());
let replay_wal = WalObjectStore::new_without_replay( let replay_wal = WalObjectStore::new_without_replay(
Arc::clone(&time_provider),
Arc::clone(&object_store), Arc::clone(&object_store),
"my_host", "my_host",
Arc::clone(&replay_notifier), Arc::clone(&replay_notifier),
@ -938,11 +953,9 @@ mod tests {
assert_eq!(expected_info, snapshot_info); assert_eq!(expected_info, snapshot_info);
snapshot_done.await.unwrap(); snapshot_done.await.unwrap();
let file_3_contents = WalContents { let file_3_contents = create::wal_contents_with_snapshot(
min_timestamp_ns: 128_000000000, (128_000_000_000, 128_000_000_000, 3),
max_timestamp_ns: 128_000000000, [WalOp::Write(WriteBatch {
wal_file_number: WalFileSequenceNumber(3),
ops: vec![WalOp::Write(WriteBatch {
database_id: DbId::from(0), database_id: DbId::from(0),
database_name: "db1".into(), database_name: "db1".into(),
table_chunks: IndexMap::from([( table_chunks: IndexMap::from([(
@ -974,12 +987,12 @@ mod tests {
min_time_ns: 128_000000000, min_time_ns: 128_000000000,
max_time_ns: 128_000000000, max_time_ns: 128_000000000,
})], })],
snapshot: Some(SnapshotDetails { SnapshotDetails {
snapshot_sequence_number: SnapshotSequenceNumber::new(1), snapshot_sequence_number: SnapshotSequenceNumber::new(1),
end_time_marker: 120_000000000, end_time_marker: 120_000000000,
last_wal_sequence_number: WalFileSequenceNumber(2), last_wal_sequence_number: WalFileSequenceNumber(2),
}), },
}; );
let notifier = notifier.as_any().downcast_ref::<TestNotfiier>().unwrap(); let notifier = notifier.as_any().downcast_ref::<TestNotfiier>().unwrap();
@ -997,6 +1010,7 @@ mod tests {
// test that replay now only has file 3 // test that replay now only has file 3
let replay_notifier: Arc<dyn WalFileNotifier> = Arc::new(TestNotfiier::default()); let replay_notifier: Arc<dyn WalFileNotifier> = Arc::new(TestNotfiier::default());
let replay_wal = WalObjectStore::new_without_replay( let replay_wal = WalObjectStore::new_without_replay(
Arc::clone(&time_provider),
object_store, object_store,
"my_host", "my_host",
Arc::clone(&replay_notifier), Arc::clone(&replay_notifier),
@ -1021,6 +1035,8 @@ mod tests {
#[tokio::test(flavor = "multi_thread", worker_threads = 2)] #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
async fn flush_for_empty_buffer_skips_notify() { async fn flush_for_empty_buffer_skips_notify() {
let time_provider: Arc<dyn TimeProvider> =
Arc::new(MockProvider::new(Time::from_timestamp_nanos(0)));
let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new()); let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
let notifier: Arc<dyn WalFileNotifier> = Arc::new(TestNotfiier::default()); let notifier: Arc<dyn WalFileNotifier> = Arc::new(TestNotfiier::default());
let wal_config = WalConfig { let wal_config = WalConfig {
@ -1030,6 +1046,7 @@ mod tests {
gen1_duration: Gen1Duration::new_1m(), gen1_duration: Gen1Duration::new_1m(),
}; };
let wal = WalObjectStore::new_without_replay( let wal = WalObjectStore::new_without_replay(
time_provider,
Arc::clone(&object_store), Arc::clone(&object_store),
"my_host", "my_host",
Arc::clone(&notifier), Arc::clone(&notifier),
@ -1046,6 +1063,24 @@ mod tests {
assert!(object_store.list(None).next().await.is_none()); assert!(object_store.list(None).next().await.is_none());
} }
#[test]
fn persist_time_is_tracked() {
let time_provider = Arc::new(MockProvider::new(Time::from_timestamp_nanos(0)));
let wal_buffer = WalBuffer {
time_provider: Arc::clone(&time_provider) as _,
is_shutdown: false,
wal_file_sequence_number: WalFileSequenceNumber(0),
op_limit: 10,
op_count: 0,
database_to_write_batch: Default::default(),
catalog_batches: vec![],
write_op_responses: vec![],
};
time_provider.set(Time::from_timestamp_millis(1234).unwrap());
let (wal_contents, _) = wal_buffer.into_wal_contents_and_responses();
assert_eq!(1234, wal_contents.persist_timestamp_ms);
}
#[derive(Debug, Default)] #[derive(Debug, Default)]
struct TestNotfiier { struct TestNotfiier {
notified_writes: parking_lot::Mutex<Vec<WalContents>>, notified_writes: parking_lot::Mutex<Vec<WalContents>>,

View File

@ -120,6 +120,7 @@ mod tests {
table_chunks.insert(table_id, chunks); table_chunks.insert(table_id, chunks);
let contents = WalContents { let contents = WalContents {
persist_timestamp_ms: 10,
min_timestamp_ns: 0, min_timestamp_ns: 0,
max_timestamp_ns: 10, max_timestamp_ns: 10,
wal_file_number: WalFileSequenceNumber::new(1), wal_file_number: WalFileSequenceNumber::new(1),

View File

@ -197,6 +197,7 @@ impl WriteBufferImpl {
// create the wal instance, which will replay into the queryable buffer and start // create the wal instance, which will replay into the queryable buffer and start
// the background flush task. // the background flush task.
let wal = WalObjectStore::new( let wal = WalObjectStore::new(
Arc::clone(&time_provider),
persister.object_store(), persister.object_store(),
persister.host_identifier_prefix(), persister.host_identifier_prefix(),
Arc::clone(&queryable_buffer) as Arc<dyn WalFileNotifier>, Arc::clone(&queryable_buffer) as Arc<dyn WalFileNotifier>,