From 4fa032a1370b526b02620e07cbef42a3f35ecabc Mon Sep 17 00:00:00 2001 From: William Wen Date: Tue, 18 Jun 2024 18:08:22 +0800 Subject: [PATCH 01/17] refactor(storage): remove sync finish event --- .../event_handler/hummock_event_handler.rs | 83 +++--- src/storage/src/hummock/event_handler/mod.rs | 5 +- .../src/hummock/event_handler/uploader.rs | 254 +++++++++++------- .../src/hummock/store/hummock_storage.rs | 2 +- 4 files changed, 203 insertions(+), 141 deletions(-) diff --git a/src/storage/src/hummock/event_handler/hummock_event_handler.rs b/src/storage/src/hummock/event_handler/hummock_event_handler.rs index 277984d3545dd..b760ea51c86ad 100644 --- a/src/storage/src/hummock/event_handler/hummock_event_handler.rs +++ b/src/storage/src/hummock/event_handler/hummock_event_handler.rs @@ -396,18 +396,9 @@ impl HummockEventHandler { // Handler for different events impl HummockEventHandler { - fn handle_epoch_synced(&mut self, epoch: HummockEpoch, data: SyncedData) { - debug!("epoch has been synced: {}.", epoch); - let SyncedData { - newly_upload_ssts: newly_uploaded_sstables, - .. - } = data; + fn handle_epoch_synced(&mut self, newly_uploaded_sstables: Arc<[Arc]>) { { - let newly_uploaded_sstables = newly_uploaded_sstables - .into_iter() - .map(Arc::new) - .collect_vec(); - if !newly_uploaded_sstables.is_empty() { + { let related_instance_ids: HashSet<_> = newly_uploaded_sstables .iter() .flat_map(|sst| sst.imm_ids().keys().cloned()) @@ -427,7 +418,7 @@ impl HummockEventHandler { }); }); } - }; + } } /// This function will be performed under the protection of the `read_version_mapping` read @@ -488,8 +479,7 @@ impl HummockEventHandler { } } - fn handle_data_spilled(&mut self, staging_sstable_info: StagingSstableInfo) { - let staging_sstable_info = Arc::new(staging_sstable_info); + fn handle_data_spilled(&mut self, staging_sstable_info: Arc) { trace!("data_spilled. SST size {}", staging_sstable_info.imm_size()); self.for_each_read_version( staging_sstable_info.imm_ids().keys().cloned(), @@ -504,7 +494,7 @@ impl HummockEventHandler { fn handle_sync_epoch( &mut self, new_sync_epoch: HummockEpoch, - sync_result_sender: oneshot::Sender>, + sync_result_sender: oneshot::Sender>, ) { debug!( "awaiting for epoch to be synced: {}, max_synced_epoch: {}", @@ -750,12 +740,12 @@ impl HummockEventHandler { fn handle_uploader_event(&mut self, event: UploaderEvent) { match event { - UploaderEvent::SyncFinish(epoch, data) => { + UploaderEvent::SyncFinish(data) => { let _timer = self .metrics .event_handler_on_sync_finish_latency .start_timer(); - self.handle_epoch_synced(epoch, data); + self.handle_epoch_synced(data); } UploaderEvent::DataSpilled(staging_sstable_info) => { @@ -925,39 +915,44 @@ impl HummockEventHandler { } pub(super) fn send_sync_result( - sender: oneshot::Sender>, - result: HummockResult<&SyncedData>, + sender: oneshot::Sender>, + result: HummockResult, ) { - let result = result.map( - |SyncedData { - newly_upload_ssts, - uploaded_ssts, - table_watermarks, - }| { - let mut sync_size = 0; - let mut uncommitted_ssts = Vec::new(); - let mut old_value_ssts = Vec::new(); - // The newly uploaded `sstable_infos` contains newer data. Therefore, - // `newly_upload_ssts` at the front - for sst in newly_upload_ssts.iter().chain(uploaded_ssts.iter()) { - sync_size += sst.imm_size(); - uncommitted_ssts.extend(sst.sstable_infos().iter().cloned()); - old_value_ssts.extend(sst.old_value_sstable_infos().iter().cloned()); - } - SyncResult { - sync_size, - uncommitted_ssts, - table_watermarks: table_watermarks.clone(), - old_value_ssts, - } - }, - ); - let _ = sender.send(result).inspect_err(|e| { error!("unable to send sync result. Err: {:?}", e); }); } +impl SyncedData { + pub fn into_sync_result(self) -> SyncResult { + let SyncedData { + newly_upload_ssts, + uploaded_ssts, + table_watermarks, + } = self; + let mut sync_size = 0; + let mut uncommitted_ssts = Vec::new(); + let mut old_value_ssts = Vec::new(); + // The newly uploaded `sstable_infos` contains newer data. Therefore, + // `newly_upload_ssts` at the front + for sst in newly_upload_ssts + .iter() + .flat_map(|ssts| ssts.iter()) + .chain(uploaded_ssts.iter()) + { + sync_size += sst.imm_size(); + uncommitted_ssts.extend(sst.sstable_infos().iter().cloned()); + old_value_ssts.extend(sst.old_value_sstable_infos().iter().cloned()); + } + SyncResult { + sync_size, + uncommitted_ssts, + table_watermarks: table_watermarks.clone(), + old_value_ssts, + } + } +} + #[cfg(test)] mod tests { use std::future::{poll_fn, Future}; diff --git a/src/storage/src/hummock/event_handler/mod.rs b/src/storage/src/hummock/event_handler/mod.rs index bbf69ae194f72..c88611cd0888a 100644 --- a/src/storage/src/hummock/event_handler/mod.rs +++ b/src/storage/src/hummock/event_handler/mod.rs @@ -18,7 +18,7 @@ use std::sync::Arc; use parking_lot::{RwLock, RwLockReadGuard}; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::TableId; -use risingwave_hummock_sdk::{HummockEpoch, SyncResult}; +use risingwave_hummock_sdk::HummockEpoch; use thiserror_ext::AsReport; use tokio::sync::oneshot; @@ -36,6 +36,7 @@ use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use super::store::version::HummockReadVersion; use crate::hummock::event_handler::hummock_event_handler::HummockEventSender; +use crate::hummock::event_handler::uploader::SyncedData; #[derive(Debug)] pub struct BufferWriteRequest { @@ -59,7 +60,7 @@ pub enum HummockEvent { /// handle sender. SyncEpoch { new_sync_epoch: HummockEpoch, - sync_result_sender: oneshot::Sender>, + sync_result_sender: oneshot::Sender>, }, /// Clear shared buffer and reset all states diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader.rs index f768aa23dcd89..cb1f293453859 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader.rs @@ -35,7 +35,7 @@ use risingwave_common::must_match; use risingwave_hummock_sdk::table_watermark::{ TableWatermarks, VnodeWatermark, WatermarkDirection, }; -use risingwave_hummock_sdk::{CompactionGroupId, HummockEpoch, LocalSstableInfo, SyncResult}; +use risingwave_hummock_sdk::{CompactionGroupId, HummockEpoch, LocalSstableInfo}; use thiserror_ext::AsReport; use tokio::sync::oneshot; use tokio::task::JoinHandle; @@ -265,7 +265,10 @@ impl UploadingTask { } /// Poll the result of the uploading task - fn poll_result(&mut self, cx: &mut Context<'_>) -> Poll> { + fn poll_result( + &mut self, + cx: &mut Context<'_>, + ) -> Poll>> { Poll::Ready(match ready!(self.join_handle.poll_unpin(cx)) { Ok(task_result) => task_result .inspect(|_| { @@ -277,13 +280,13 @@ impl UploadingTask { }) .inspect_err(|e| error!(task_info = ?self.task_info, err = ?e.as_report(), "upload task failed")) .map(|output| { - StagingSstableInfo::new( + Arc::new(StagingSstableInfo::new( output.new_value_ssts, output.old_value_ssts, self.task_info.epochs.clone(), self.task_info.imm_ids.clone(), self.task_info.task_size, - ) + )) }), Err(err) => Err(HummockError::other(format!( @@ -294,7 +297,7 @@ impl UploadingTask { } /// Poll the uploading task until it succeeds. If it fails, we will retry it. - fn poll_ok_with_retry(&mut self, cx: &mut Context<'_>) -> Poll { + fn poll_ok_with_retry(&mut self, cx: &mut Context<'_>) -> Poll> { loop { let result = ready!(self.poll_result(cx)); match result { @@ -323,7 +326,7 @@ impl UploadingTask { } impl Future for UploadingTask { - type Output = HummockResult; + type Output = HummockResult>; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { self.poll_result(cx) @@ -337,7 +340,7 @@ struct SpilledData { // ordered spilling tasks. Task at the back is spilling older data. uploading_tasks: VecDeque, // ordered spilled data. Data at the back is older. - uploaded_data: VecDeque, + uploaded_data: VecDeque>, } impl SpilledData { @@ -347,7 +350,10 @@ impl SpilledData { /// Poll the successful spill of the oldest uploading task. Return `Poll::Ready(None)` is there /// is no uploading task - fn poll_success_spill(&mut self, cx: &mut Context<'_>) -> Poll> { + fn poll_success_spill( + &mut self, + cx: &mut Context<'_>, + ) -> Poll>> { // only poll the oldest uploading task if there is any if let Some(task) = self.uploading_tasks.back_mut() { let staging_sstable_info = ready!(task.poll_ok_with_retry(cx)); @@ -910,14 +916,15 @@ struct SyncingData { // None means there is no pending uploading tasks uploading_tasks: Option>, // newer data at the front - uploaded: VecDeque, + uploaded: VecDeque>, table_watermarks: HashMap, - sync_result_sender: oneshot::Sender>, + sync_result_sender: oneshot::Sender>, } +#[derive(Debug)] pub struct SyncedData { - pub newly_upload_ssts: Vec, - pub uploaded_ssts: VecDeque, + pub newly_upload_ssts: Option]>>, + pub uploaded_ssts: VecDeque>, pub table_watermarks: HashMap, } @@ -1081,7 +1088,7 @@ impl HummockUploader { pub(super) fn start_sync_epoch( &mut self, epoch: HummockEpoch, - sync_result_sender: oneshot::Sender>, + sync_result_sender: oneshot::Sender>, ) { let data = match &mut self.state { UploaderState::Working(data) => data, @@ -1139,22 +1146,6 @@ impl HummockUploader { .set(data.syncing_data.len() as _); } - fn add_synced_data(&mut self, epoch: HummockEpoch) { - assert!( - epoch <= self.max_syncing_epoch, - "epoch {} that has been synced has not started syncing yet. previous max syncing epoch {}", - epoch, - self.max_syncing_epoch - ); - assert!( - epoch > self.max_synced_epoch, - "epoch {} has been synced. previous max synced epoch: {}", - epoch, - self.max_synced_epoch - ); - self.max_synced_epoch = epoch; - } - pub(crate) fn update_pinned_version(&mut self, pinned_version: PinnedVersion) { assert_ge!( pinned_version.max_committed_epoch(), @@ -1255,7 +1246,7 @@ impl UploaderData { Option<( HummockEpoch, HummockResult, - oneshot::Sender>, + oneshot::Sender>, )>, > { // Only poll the oldest epoch if there is any so that the syncing epoch are finished in @@ -1263,9 +1254,9 @@ impl UploaderData { if let Some(syncing_data) = self.syncing_data.back_mut() { // The syncing task has finished let result = if let Some(all_tasks) = &mut syncing_data.uploading_tasks { - ready!(all_tasks.poll_unpin(cx)) + ready!(all_tasks.poll_unpin(cx)).map(|sstable_infos| Some(Arc::from(sstable_infos))) } else { - Ok(Vec::new()) + Ok(None) }; let syncing_data = self.syncing_data.pop_back().expect("must exist"); context @@ -1274,9 +1265,12 @@ impl UploaderData { .set(self.syncing_data.len() as _); let epoch = syncing_data.sync_epoch; - let result = result.map(|newly_uploaded_sstable_infos| { + let result = result.map(|newly_uploaded_sstable_infos: Option>| { // take `rev` so that old data is acked first - for sstable_info in newly_uploaded_sstable_infos.iter().rev() { + for sstable_info in newly_uploaded_sstable_infos + .iter() + .flat_map(|sstable_infos| sstable_infos.iter().rev()) + { self.unsync_data.ack_flushed(sstable_info); } SyncedData { @@ -1294,7 +1288,7 @@ impl UploaderData { /// Poll the success of the oldest spilled task of unsync spill data. Return `Poll::Ready(None)` if /// there is no spilling task. - fn poll_spill_task(&mut self, cx: &mut Context<'_>) -> Poll> { + fn poll_spill_task(&mut self, cx: &mut Context<'_>) -> Poll>> { // iterator from older epoch to new epoch so that the spill task are finished in epoch order for epoch_data in self.unsync_data.epoch_data.values_mut() { // if None, there is no spilling task. Search for the unsync data of the next epoch in @@ -1310,8 +1304,8 @@ impl UploaderData { pub(super) enum UploaderEvent { // staging sstable info of newer data comes first - SyncFinish(HummockEpoch, SyncedData), - DataSpilled(StagingSstableInfo), + SyncFinish(Arc<[Arc]>), + DataSpilled(Arc), } impl HummockUploader { @@ -1320,14 +1314,31 @@ impl HummockUploader { let UploaderState::Working(data) = &mut self.state else { return Poll::Pending; }; - if let Some((epoch, result, result_sender)) = + while let Some((epoch, result, result_sender)) = ready!(data.poll_syncing_task(cx, &self.context)) { match result { Ok(data) => { - self.add_synced_data(epoch); - send_sync_result(result_sender, Ok(&data)); - return Poll::Ready(UploaderEvent::SyncFinish(epoch, data)); + assert!( + epoch <= self.max_syncing_epoch, + "epoch {} that has been synced has not started syncing yet. previous max syncing epoch {}", + epoch, + self.max_syncing_epoch + ); + assert!( + epoch > self.max_synced_epoch, + "epoch {} has been synced. previous max synced epoch: {}", + epoch, + self.max_synced_epoch + ); + self.max_synced_epoch = epoch; + let newly_upload_ssts = data.newly_upload_ssts.clone(); + send_sync_result(result_sender, Ok(data)); + if let Some(newly_upload_ssts) = newly_upload_ssts { + return Poll::Ready(UploaderEvent::SyncFinish(newly_upload_ssts)); + } else { + continue; + } } Err(e) => { send_sync_result( @@ -1420,10 +1431,6 @@ pub(crate) mod tests { fn data(&self) -> &UploaderData { must_match!(&self.state, UploaderState::Working(data) => data) } - - fn start_sync_epoch_for_test(&mut self, epoch: HummockEpoch) { - self.start_sync_epoch(epoch, oneshot::channel().0) - } } fn test_hummock_version(epoch: HummockEpoch) -> HummockVersion { @@ -1654,7 +1661,8 @@ pub(crate) mod tests { uploader.add_imm(TEST_LOCAL_INSTANCE_ID, imm.clone()); uploader.local_seal_epoch_for_test(TEST_LOCAL_INSTANCE_ID, epoch1); - uploader.start_sync_epoch_for_test(epoch1); + let (sync_tx, sync_rx) = oneshot::channel(); + uploader.start_sync_epoch(epoch1, sync_tx); assert_eq!(epoch1 as HummockEpoch, uploader.max_syncing_epoch); assert_eq!(1, uploader.data().syncing_data.len()); let syncing_data = uploader.data().syncing_data.front().unwrap(); @@ -1663,13 +1671,30 @@ pub(crate) mod tests { assert!(syncing_data.uploading_tasks.is_some()); match uploader.next_event().await { - UploaderEvent::SyncFinish(finished_epoch, data) => { - assert_eq!(epoch1, finished_epoch); + UploaderEvent::SyncFinish(newly_upload_ssts) => { + assert_eq!(1, newly_upload_ssts.len()); + let staging_sst = newly_upload_ssts.first().unwrap(); + assert_eq!(&vec![epoch1], staging_sst.epochs()); + assert_eq!( + &HashMap::from_iter([(TEST_LOCAL_INSTANCE_ID, vec![imm.batch_id()])]), + staging_sst.imm_ids() + ); + assert_eq!( + &dummy_success_upload_output().new_value_ssts, + staging_sst.sstable_infos() + ); + } + _ => unreachable!(), + }; + + match sync_rx.await { + Ok(Ok(data)) => { let SyncedData { newly_upload_ssts, uploaded_ssts, table_watermarks, } = data; + let newly_upload_ssts = newly_upload_ssts.unwrap(); assert_eq!(1, newly_upload_ssts.len()); let staging_sst = newly_upload_ssts.first().unwrap(); assert_eq!(&vec![epoch1], staging_sst.epochs()); @@ -1701,14 +1726,16 @@ pub(crate) mod tests { let mut uploader = test_uploader(dummy_success_upload_future); let epoch1 = INITIAL_EPOCH.next_epoch(); - uploader.start_sync_epoch_for_test(epoch1); + let (sync_tx, sync_rx) = oneshot::channel(); + uploader.start_sync_epoch(epoch1, sync_tx); assert_eq!(epoch1, uploader.max_syncing_epoch); - match uploader.next_event().await { - UploaderEvent::SyncFinish(finished_epoch, data) => { - assert_eq!(epoch1, finished_epoch); + assert_uploader_pending(&mut uploader).await; + + match sync_rx.await { + Ok(Ok(data)) => { assert!(data.uploaded_ssts.is_empty()); - assert!(data.newly_upload_ssts.is_empty()); + assert!(data.newly_upload_ssts.is_none()); } _ => unreachable!(), }; @@ -1733,14 +1760,16 @@ pub(crate) mod tests { uploader.local_seal_epoch_for_test(TEST_LOCAL_INSTANCE_ID, epoch1); uploader.add_imm(TEST_LOCAL_INSTANCE_ID, imm); - uploader.start_sync_epoch_for_test(epoch1); + let (sync_tx, sync_rx) = oneshot::channel(); + uploader.start_sync_epoch(epoch1, sync_tx); assert_eq!(epoch1, uploader.max_syncing_epoch); - match uploader.next_event().await { - UploaderEvent::SyncFinish(finished_epoch, data) => { - assert_eq!(epoch1, finished_epoch); + assert_uploader_pending(&mut uploader).await; + + match sync_rx.await { + Ok(Ok(data)) => { assert!(data.uploaded_ssts.is_empty()); - assert!(data.newly_upload_ssts.is_empty()); + assert!(data.newly_upload_ssts.is_none()); } _ => unreachable!(), }; @@ -1784,7 +1813,8 @@ pub(crate) mod tests { assert_eq!(epoch1, uploader.max_syncing_epoch); uploader.init_instance(TEST_LOCAL_INSTANCE_ID, TEST_TABLE_ID, epoch6); - uploader.add_imm(TEST_LOCAL_INSTANCE_ID, gen_imm(epoch6).await); + let imm = gen_imm(epoch6).await; + uploader.add_imm(TEST_LOCAL_INSTANCE_ID, imm.clone()); uploader.update_pinned_version(version2); assert_eq!(epoch2, uploader.max_synced_epoch); assert_eq!(epoch2, uploader.max_syncing_epoch); @@ -1794,18 +1824,32 @@ pub(crate) mod tests { assert_eq!(epoch3, uploader.max_synced_epoch); assert_eq!(epoch3, uploader.max_syncing_epoch); - uploader.start_sync_epoch_for_test(epoch6); + let (sync_tx, sync_rx) = oneshot::channel(); + uploader.start_sync_epoch(epoch6, sync_tx); assert_eq!(epoch6, uploader.max_syncing_epoch); uploader.update_pinned_version(version4); assert_eq!(epoch4, uploader.max_synced_epoch); assert_eq!(epoch6, uploader.max_syncing_epoch); match uploader.next_event().await { - UploaderEvent::SyncFinish(epoch, _) => { - assert_eq!(epoch6, epoch); + UploaderEvent::SyncFinish(newly_upload_ssts) => { + assert_eq!(1, newly_upload_ssts.len()); + assert_eq!(&get_imm_ids([&imm]), newly_upload_ssts[0].imm_ids()); + } + _ => unreachable!(), + } + + match sync_rx.await { + Ok(Ok(data)) => { + assert!(data.uploaded_ssts.is_empty()); + assert!(data.table_watermarks.is_empty()); + let newly_upload_ssts = data.newly_upload_ssts.unwrap(); + assert_eq!(1, newly_upload_ssts.len()); + assert_eq!(&get_imm_ids([&imm]), newly_upload_ssts[0].imm_ids()); } - UploaderEvent::DataSpilled(_) => unreachable!(), + _ => unreachable!(), } + uploader.update_pinned_version(version5); assert_eq!(epoch6, uploader.max_synced_epoch); assert_eq!(epoch6, uploader.max_syncing_epoch); @@ -1886,6 +1930,7 @@ pub(crate) mod tests { .is_pending() ) } + #[tokio::test] async fn test_uploader_finish_in_order() { let config = StorageOpts { @@ -1960,7 +2005,8 @@ pub(crate) mod tests { let (await_start1_4, finish_tx1_4) = new_task_notifier(get_payload_imm_ids(&epoch1_sync_payload)); uploader.local_seal_epoch_for_test(instance_id1, epoch1); - uploader.start_sync_epoch_for_test(epoch1); + let (sync_tx1, sync_rx1) = oneshot::channel(); + uploader.start_sync_epoch(epoch1, sync_tx1); await_start1_4.await; let epoch3 = epoch2.next_epoch(); @@ -2012,17 +2058,31 @@ pub(crate) mod tests { assert_uploader_pending(&mut uploader).await; finish_tx1_3.send(()).unwrap(); - if let UploaderEvent::SyncFinish(epoch, data) = uploader.next_event().await { - assert_eq!(epoch1, epoch); - assert_eq!(2, data.newly_upload_ssts.len()); + if let UploaderEvent::SyncFinish(newly_upload_ssts) = uploader.next_event().await { + assert_eq!(2, newly_upload_ssts.len()); + assert_eq!( + &get_payload_imm_ids(&epoch1_sync_payload), + newly_upload_ssts[0].imm_ids() + ); + assert_eq!( + &get_payload_imm_ids(&epoch1_spill_payload3), + newly_upload_ssts[1].imm_ids() + ); + } else { + unreachable!("should be sync finish"); + } + + if let Ok(Ok(data)) = sync_rx1.await { + let newly_upload_ssts = data.newly_upload_ssts.unwrap(); + assert_eq!(2, newly_upload_ssts.len()); assert_eq!(1, data.uploaded_ssts.len()); assert_eq!( &get_payload_imm_ids(&epoch1_sync_payload), - data.newly_upload_ssts[0].imm_ids() + newly_upload_ssts[0].imm_ids() ); assert_eq!( &get_payload_imm_ids(&epoch1_spill_payload3), - data.newly_upload_ssts[1].imm_ids() + newly_upload_ssts[1].imm_ids() ); assert_eq!( &get_payload_imm_ids(&epoch1_spill_payload12), @@ -2039,10 +2099,16 @@ pub(crate) mod tests { // syncing: empty // synced: epoch1: sst([imm1_4]), sst([imm1_3]), sst([imm1_2, imm1_1]) - uploader.start_sync_epoch_for_test(epoch2); - if let UploaderEvent::SyncFinish(epoch, data) = uploader.next_event().await { - assert_eq!(epoch2, epoch); - assert!(data.newly_upload_ssts.is_empty()); + let (sync_tx2, sync_rx2) = oneshot::channel(); + uploader.start_sync_epoch(epoch2, sync_tx2); + uploader.local_seal_epoch_for_test(instance_id2, epoch3); + if let UploaderEvent::DataSpilled(sst) = uploader.next_event().await { + assert_eq!(&get_payload_imm_ids(&epoch3_spill_payload1), sst.imm_ids()); + } else { + unreachable!("should be data spilled"); + } + if let Ok(Ok(data)) = sync_rx2.await { + assert!(data.newly_upload_ssts.is_none()); assert_eq!(data.uploaded_ssts.len(), 1); assert_eq!( &get_payload_imm_ids(&epoch2_spill_payload), @@ -2053,21 +2119,6 @@ pub(crate) mod tests { } assert_eq!(epoch2, uploader.max_synced_epoch); - // current uploader state: - // unsealed: epoch3: imm: imm3_3, uploading: [imm3_2], [imm3_1] - // epoch4: imm: imm4 - // sealed: empty - // syncing: empty - // synced: epoch1: sst([imm1_4]), sst([imm1_3]), sst([imm1_2, imm1_1]) - // epoch2: sst([imm2]) - - uploader.local_seal_epoch_for_test(instance_id2, epoch3); - if let UploaderEvent::DataSpilled(sst) = uploader.next_event().await { - assert_eq!(&get_payload_imm_ids(&epoch3_spill_payload1), sst.imm_ids()); - } else { - unreachable!("should be data spilled"); - } - // current uploader state: // unsealed: epoch4: imm: imm4 // sealed: imm: imm3_3, uploading: [imm3_2], uploaded: sst([imm3_1]) @@ -2080,7 +2131,8 @@ pub(crate) mod tests { let epoch4_sync_payload = HashMap::from_iter([(instance_id1, vec![imm4, imm3_3])]); let (await_start4_with_3_3, finish_tx4_with_3_3) = new_task_notifier(get_payload_imm_ids(&epoch4_sync_payload)); - uploader.start_sync_epoch_for_test(epoch4); + let (sync_tx4, sync_rx4) = oneshot::channel(); + uploader.start_sync_epoch(epoch4, sync_tx4); await_start4_with_3_3.await; // current uploader state: @@ -2095,16 +2147,30 @@ pub(crate) mod tests { assert_uploader_pending(&mut uploader).await; finish_tx4_with_3_3.send(()).unwrap(); - if let UploaderEvent::SyncFinish(epoch, data) = uploader.next_event().await { - assert_eq!(epoch4, epoch); - assert_eq!(2, data.newly_upload_ssts.len()); + if let UploaderEvent::SyncFinish(newly_upload_ssts) = uploader.next_event().await { + assert_eq!(2, newly_upload_ssts.len()); + assert_eq!( + &get_payload_imm_ids(&epoch4_sync_payload), + newly_upload_ssts[0].imm_ids() + ); + assert_eq!( + &get_payload_imm_ids(&epoch3_spill_payload2), + newly_upload_ssts[1].imm_ids() + ); + } else { + unreachable!("should be sync finish"); + } + + if let Ok(Ok(data)) = sync_rx4.await { + let newly_upload_ssts = data.newly_upload_ssts.unwrap(); + assert_eq!(2, newly_upload_ssts.len()); assert_eq!( &get_payload_imm_ids(&epoch4_sync_payload), - data.newly_upload_ssts[0].imm_ids() + newly_upload_ssts[0].imm_ids() ); assert_eq!( &get_payload_imm_ids(&epoch3_spill_payload2), - data.newly_upload_ssts[1].imm_ids() + newly_upload_ssts[1].imm_ids() ); assert_eq!(1, data.uploaded_ssts.len()); assert_eq!( diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index 7f9f956b62ddf..ff27c3cc24fef 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -570,7 +570,7 @@ impl StateStore for HummockStorage { sync_result_sender: tx, }) .expect("should send success"); - rx.map(|recv_result| Ok(recv_result.expect("should wait success")?)) + rx.map(|recv_result| Ok(recv_result.expect("should wait success")?.into_sync_result())) } fn seal_epoch(&self, epoch: u64, is_checkpoint: bool) { From 09950fc07e1b8420054469fecd565ef0e37f87d8 Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 19 Jun 2024 00:53:55 +0800 Subject: [PATCH 02/17] syncing data upload tasks partial succeed --- .../event_handler/hummock_event_handler.rs | 87 ++--- .../src/hummock/event_handler/uploader.rs | 321 ++++++++---------- 2 files changed, 158 insertions(+), 250 deletions(-) diff --git a/src/storage/src/hummock/event_handler/hummock_event_handler.rs b/src/storage/src/hummock/event_handler/hummock_event_handler.rs index b760ea51c86ad..c0d58150c754c 100644 --- a/src/storage/src/hummock/event_handler/hummock_event_handler.rs +++ b/src/storage/src/hummock/event_handler/hummock_event_handler.rs @@ -184,8 +184,7 @@ impl HummockEventReceiver { } struct HummockEventHandlerMetrics { - event_handler_on_sync_finish_latency: Histogram, - event_handler_on_spilled_latency: Histogram, + event_handler_on_upload_finish_latency: Histogram, event_handler_on_apply_version_update: Histogram, event_handler_on_recv_version_update: Histogram, } @@ -329,12 +328,9 @@ impl HummockEventHandler { let write_conflict_detector = ConflictDetector::new_from_config(storage_opts); let metrics = HummockEventHandlerMetrics { - event_handler_on_sync_finish_latency: state_store_metrics + event_handler_on_upload_finish_latency: state_store_metrics .event_handler_latency - .with_label_values(&["on_sync_finish"]), - event_handler_on_spilled_latency: state_store_metrics - .event_handler_latency - .with_label_values(&["on_spilled"]), + .with_label_values(&["on_upload_finish"]), event_handler_on_apply_version_update: state_store_metrics .event_handler_latency .with_label_values(&["apply_version"]), @@ -396,31 +392,6 @@ impl HummockEventHandler { // Handler for different events impl HummockEventHandler { - fn handle_epoch_synced(&mut self, newly_uploaded_sstables: Arc<[Arc]>) { - { - { - let related_instance_ids: HashSet<_> = newly_uploaded_sstables - .iter() - .flat_map(|sst| sst.imm_ids().keys().cloned()) - .collect(); - self.for_each_read_version(related_instance_ids, |instance_id, read_version| { - newly_uploaded_sstables - .iter() - // Take rev because newer data come first in `newly_uploaded_sstables` but we apply - // older data first - .rev() - .for_each(|staging_sstable_info| { - if staging_sstable_info.imm_ids().contains_key(&instance_id) { - read_version.update(VersionUpdate::Staging(StagingData::Sst( - staging_sstable_info.clone(), - ))); - } - }); - }); - } - } - } - /// This function will be performed under the protection of the `read_version_mapping` read /// lock, and add write lock on each `read_version` operation fn for_each_read_version( @@ -740,16 +711,11 @@ impl HummockEventHandler { fn handle_uploader_event(&mut self, event: UploaderEvent) { match event { - UploaderEvent::SyncFinish(data) => { + UploaderEvent::DataUploaded(staging_sstable_info) => { let _timer = self .metrics - .event_handler_on_sync_finish_latency + .event_handler_on_upload_finish_latency .start_timer(); - self.handle_epoch_synced(data); - } - - UploaderEvent::DataSpilled(staging_sstable_info) => { - let _timer = self.metrics.event_handler_on_spilled_latency.start_timer(); self.handle_data_spilled(staging_sstable_info); } } @@ -925,30 +891,27 @@ pub(super) fn send_sync_result( impl SyncedData { pub fn into_sync_result(self) -> SyncResult { - let SyncedData { - newly_upload_ssts, - uploaded_ssts, - table_watermarks, - } = self; - let mut sync_size = 0; - let mut uncommitted_ssts = Vec::new(); - let mut old_value_ssts = Vec::new(); - // The newly uploaded `sstable_infos` contains newer data. Therefore, - // `newly_upload_ssts` at the front - for sst in newly_upload_ssts - .iter() - .flat_map(|ssts| ssts.iter()) - .chain(uploaded_ssts.iter()) { - sync_size += sst.imm_size(); - uncommitted_ssts.extend(sst.sstable_infos().iter().cloned()); - old_value_ssts.extend(sst.old_value_sstable_infos().iter().cloned()); - } - SyncResult { - sync_size, - uncommitted_ssts, - table_watermarks: table_watermarks.clone(), - old_value_ssts, + let SyncedData { + uploaded_ssts, + table_watermarks, + } = self; + let mut sync_size = 0; + let mut uncommitted_ssts = Vec::new(); + let mut old_value_ssts = Vec::new(); + // The newly uploaded `sstable_infos` contains newer data. Therefore, + // `newly_upload_ssts` at the front + for sst in uploaded_ssts { + sync_size += sst.imm_size(); + uncommitted_ssts.extend(sst.sstable_infos().iter().cloned()); + old_value_ssts.extend(sst.old_value_sstable_infos().iter().cloned()); + } + SyncResult { + sync_size, + uncommitted_ssts, + table_watermarks: table_watermarks.clone(), + old_value_ssts, + } } } } diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader.rs index cb1f293453859..a4b19ecc0dd27 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader.rs @@ -22,7 +22,6 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{ready, Context, Poll}; -use futures::future::{try_join_all, TryJoinAll}; use futures::FutureExt; use itertools::Itertools; use more_asserts::{assert_ge, assert_gt}; @@ -911,10 +910,8 @@ impl UnsyncData { struct SyncingData { sync_epoch: HummockEpoch, - // TODO: may replace `TryJoinAll` with a future that will abort other join handles once - // one join handle failed. - // None means there is no pending uploading tasks - uploading_tasks: Option>, + // task of newer data at the front + uploading_tasks: VecDeque, // newer data at the front uploaded: VecDeque>, table_watermarks: HashMap, @@ -923,7 +920,6 @@ struct SyncingData { #[derive(Debug)] pub struct SyncedData { - pub newly_upload_ssts: Option]>>, pub uploaded_ssts: VecDeque>, pub table_watermarks: HashMap, } @@ -969,8 +965,10 @@ impl UploaderData { for (_, epoch_data) in self.unsync_data.epoch_data { epoch_data.spilled_data.abort(); } - // TODO: call `abort` on the uploading task join handle of syncing_data for syncing_data in self.syncing_data { + for task in syncing_data.uploading_tasks { + task.join_handle.abort(); + } send_sync_result(syncing_data.sync_result_sender, Err(err())); } } @@ -1126,15 +1124,9 @@ impl HummockUploader { .. } = sync_data; - let try_join_all_upload_task = if uploading_tasks.is_empty() { - None - } else { - Some(try_join_all(uploading_tasks)) - }; - data.syncing_data.push_front(SyncingData { sync_epoch: epoch, - uploading_tasks: try_join_all_upload_task, + uploading_tasks, uploaded: uploaded_data, table_watermarks, sync_result_sender, @@ -1235,57 +1227,6 @@ impl HummockUploader { } impl UploaderData { - /// Poll the syncing task of the syncing data of the oldest epoch. Return `Poll::Ready(None)` if - /// there is no syncing data. - #[expect(clippy::type_complexity)] - fn poll_syncing_task( - &mut self, - cx: &mut Context<'_>, - context: &UploaderContext, - ) -> Poll< - Option<( - HummockEpoch, - HummockResult, - oneshot::Sender>, - )>, - > { - // Only poll the oldest epoch if there is any so that the syncing epoch are finished in - // order - if let Some(syncing_data) = self.syncing_data.back_mut() { - // The syncing task has finished - let result = if let Some(all_tasks) = &mut syncing_data.uploading_tasks { - ready!(all_tasks.poll_unpin(cx)).map(|sstable_infos| Some(Arc::from(sstable_infos))) - } else { - Ok(None) - }; - let syncing_data = self.syncing_data.pop_back().expect("must exist"); - context - .stats - .uploader_syncing_epoch_count - .set(self.syncing_data.len() as _); - let epoch = syncing_data.sync_epoch; - - let result = result.map(|newly_uploaded_sstable_infos: Option>| { - // take `rev` so that old data is acked first - for sstable_info in newly_uploaded_sstable_infos - .iter() - .flat_map(|sstable_infos| sstable_infos.iter().rev()) - { - self.unsync_data.ack_flushed(sstable_info); - } - SyncedData { - newly_upload_ssts: newly_uploaded_sstable_infos, - uploaded_ssts: syncing_data.uploaded, - table_watermarks: syncing_data.table_watermarks, - } - }); - - Poll::Ready(Some((epoch, result, syncing_data.sync_result_sender))) - } else { - Poll::Ready(None) - } - } - /// Poll the success of the oldest spilled task of unsync spill data. Return `Poll::Ready(None)` if /// there is no spilling task. fn poll_spill_task(&mut self, cx: &mut Context<'_>) -> Poll>> { @@ -1303,9 +1244,7 @@ impl UploaderData { } pub(super) enum UploaderEvent { - // staging sstable info of newer data comes first - SyncFinish(Arc<[Arc]>), - DataSpilled(Arc), + DataUploaded(Arc), } impl HummockUploader { @@ -1314,58 +1253,92 @@ impl HummockUploader { let UploaderState::Working(data) = &mut self.state else { return Poll::Pending; }; - while let Some((epoch, result, result_sender)) = - ready!(data.poll_syncing_task(cx, &self.context)) - { - match result { - Ok(data) => { - assert!( - epoch <= self.max_syncing_epoch, - "epoch {} that has been synced has not started syncing yet. previous max syncing epoch {}", - epoch, - self.max_syncing_epoch - ); - assert!( - epoch > self.max_synced_epoch, - "epoch {} has been synced. previous max synced epoch: {}", - epoch, - self.max_synced_epoch - ); - self.max_synced_epoch = epoch; - let newly_upload_ssts = data.newly_upload_ssts.clone(); - send_sync_result(result_sender, Ok(data)); - if let Some(newly_upload_ssts) = newly_upload_ssts { - return Poll::Ready(UploaderEvent::SyncFinish(newly_upload_ssts)); - } else { - continue; + while let Some(syncing_data) = data.syncing_data.back_mut() { + let sstable_info = if let Some(task) = syncing_data.uploading_tasks.back_mut() { + let result = ready!(task.poll_result(cx)); + let _task = syncing_data.uploading_tasks.pop_back().expect("non-empty"); + let sstable_info = match result { + Ok(sstable_info) => sstable_info, + Err(e) => { + let SyncingData { + sync_epoch, + uploading_tasks, + sync_result_sender, + .. + } = data.syncing_data.pop_back().expect("non-empty"); + for task in uploading_tasks { + task.join_handle.abort(); + } + send_sync_result( + sync_result_sender, + Err(HummockError::other(format!( + "failed sync task: {:?}", + e.as_report() + ))), + ); + let data = must_match!(replace( + &mut self.state, + UploaderState::Err { + failed_epoch: sync_epoch, + reason: format!("{:?}", e.as_report()), + }, + ), UploaderState::Working(data) => data); + + data.abort(|| { + HummockError::other(format!( + "previous epoch {} failed to sync", + sync_epoch + )) + }); + return Poll::Pending; } - } - Err(e) => { - send_sync_result( - result_sender, - Err(HummockError::other(format!( - "failed sync task: {:?}", - e.as_report() - ))), - ); - let data = must_match!(replace( - &mut self.state, - UploaderState::Err { - failed_epoch: epoch, - reason: format!("{:?}", e.as_report()), - }, - ), UploaderState::Working(data) => data); - - data.abort(|| { - HummockError::other(format!("previous epoch {} failed to sync", epoch)) - }); - return Poll::Pending; - } + }; + syncing_data.uploaded.push_front(sstable_info.clone()); + data.unsync_data.ack_flushed(&sstable_info); + Some(sstable_info) + } else { + None }; + + if syncing_data.uploading_tasks.is_empty() { + let syncing_data = data.syncing_data.pop_back().expect("non-empty"); + let SyncingData { + sync_epoch: epoch, + uploading_tasks, + uploaded, + table_watermarks, + sync_result_sender, + } = syncing_data; + assert!(uploading_tasks.is_empty()); + assert!( + epoch <= self.max_syncing_epoch, + "epoch {} that has been synced has not started syncing yet. previous max syncing epoch {}", + epoch, + self.max_syncing_epoch + ); + assert!( + epoch > self.max_synced_epoch, + "epoch {} has been synced. previous max synced epoch: {}", + epoch, + self.max_synced_epoch + ); + self.max_synced_epoch = epoch; + send_sync_result( + sync_result_sender, + Ok(SyncedData { + uploaded_ssts: uploaded, + table_watermarks, + }), + ) + } + + if let Some(sstable_info) = sstable_info { + return Poll::Ready(UploaderEvent::DataUploaded(sstable_info)); + } } if let Some(sstable_info) = ready!(data.poll_spill_task(cx)) { - return Poll::Ready(UploaderEvent::DataSpilled(sstable_info)); + return Poll::Ready(UploaderEvent::DataUploaded(sstable_info)); } Poll::Pending @@ -1668,12 +1641,10 @@ pub(crate) mod tests { let syncing_data = uploader.data().syncing_data.front().unwrap(); assert_eq!(epoch1 as HummockEpoch, syncing_data.sync_epoch); assert!(syncing_data.uploaded.is_empty()); - assert!(syncing_data.uploading_tasks.is_some()); + assert!(!syncing_data.uploading_tasks.is_empty()); match uploader.next_event().await { - UploaderEvent::SyncFinish(newly_upload_ssts) => { - assert_eq!(1, newly_upload_ssts.len()); - let staging_sst = newly_upload_ssts.first().unwrap(); + UploaderEvent::DataUploaded(staging_sst) => { assert_eq!(&vec![epoch1], staging_sst.epochs()); assert_eq!( &HashMap::from_iter([(TEST_LOCAL_INSTANCE_ID, vec![imm.batch_id()])]), @@ -1684,19 +1655,16 @@ pub(crate) mod tests { staging_sst.sstable_infos() ); } - _ => unreachable!(), }; match sync_rx.await { Ok(Ok(data)) => { let SyncedData { - newly_upload_ssts, uploaded_ssts, table_watermarks, } = data; - let newly_upload_ssts = newly_upload_ssts.unwrap(); - assert_eq!(1, newly_upload_ssts.len()); - let staging_sst = newly_upload_ssts.first().unwrap(); + assert_eq!(1, uploaded_ssts.len()); + let staging_sst = &uploaded_ssts[0]; assert_eq!(&vec![epoch1], staging_sst.epochs()); assert_eq!( &HashMap::from_iter([(TEST_LOCAL_INSTANCE_ID, vec![imm.batch_id()])]), @@ -1706,7 +1674,6 @@ pub(crate) mod tests { &dummy_success_upload_output().new_value_ssts, staging_sst.sstable_infos() ); - assert!(uploaded_ssts.is_empty()); assert!(table_watermarks.is_empty()); } _ => unreachable!(), @@ -1735,7 +1702,6 @@ pub(crate) mod tests { match sync_rx.await { Ok(Ok(data)) => { assert!(data.uploaded_ssts.is_empty()); - assert!(data.newly_upload_ssts.is_none()); } _ => unreachable!(), }; @@ -1769,7 +1735,6 @@ pub(crate) mod tests { match sync_rx.await { Ok(Ok(data)) => { assert!(data.uploaded_ssts.is_empty()); - assert!(data.newly_upload_ssts.is_none()); } _ => unreachable!(), }; @@ -1787,9 +1752,6 @@ pub(crate) mod tests { async fn test_uploader_poll_empty() { let mut uploader = test_uploader(dummy_success_upload_future); let data = must_match!(&mut uploader.state, UploaderState::Working(data) => data); - assert!(poll_fn(|cx| data.poll_syncing_task(cx, &uploader.context)) - .await - .is_none()); assert!(poll_fn(|cx| data.poll_spill_task(cx)).await.is_none()); } @@ -1832,20 +1794,16 @@ pub(crate) mod tests { assert_eq!(epoch6, uploader.max_syncing_epoch); match uploader.next_event().await { - UploaderEvent::SyncFinish(newly_upload_ssts) => { - assert_eq!(1, newly_upload_ssts.len()); - assert_eq!(&get_imm_ids([&imm]), newly_upload_ssts[0].imm_ids()); + UploaderEvent::DataUploaded(sst) => { + assert_eq!(&get_imm_ids([&imm]), sst.imm_ids()); } - _ => unreachable!(), } match sync_rx.await { Ok(Ok(data)) => { - assert!(data.uploaded_ssts.is_empty()); assert!(data.table_watermarks.is_empty()); - let newly_upload_ssts = data.newly_upload_ssts.unwrap(); - assert_eq!(1, newly_upload_ssts.len()); - assert_eq!(&get_imm_ids([&imm]), newly_upload_ssts[0].imm_ids()); + assert_eq!(1, data.uploaded_ssts.len()); + assert_eq!(&get_imm_ids([&imm]), data.uploaded_ssts[0].imm_ids()); } _ => unreachable!(), } @@ -1978,18 +1936,16 @@ pub(crate) mod tests { assert_uploader_pending(&mut uploader).await; finish_tx1.send(()).unwrap(); - if let UploaderEvent::DataSpilled(sst) = uploader.next_event().await { + let UploaderEvent::DataUploaded(sst) = uploader.next_event().await; + { assert_eq!(&get_payload_imm_ids(&epoch1_spill_payload12), sst.imm_ids()); assert_eq!(&vec![epoch1], sst.epochs()); - } else { - unreachable!("") } - if let UploaderEvent::DataSpilled(sst) = uploader.next_event().await { + let UploaderEvent::DataUploaded(sst) = uploader.next_event().await; + { assert_eq!(&get_payload_imm_ids(&epoch2_spill_payload), sst.imm_ids()); assert_eq!(&vec![epoch2], sst.epochs()); - } else { - unreachable!("") } let imm1_3 = gen_imm_with_limiter(epoch1, memory_limiter).await; @@ -2005,7 +1961,7 @@ pub(crate) mod tests { let (await_start1_4, finish_tx1_4) = new_task_notifier(get_payload_imm_ids(&epoch1_sync_payload)); uploader.local_seal_epoch_for_test(instance_id1, epoch1); - let (sync_tx1, sync_rx1) = oneshot::channel(); + let (sync_tx1, mut sync_rx1) = oneshot::channel(); uploader.start_sync_epoch(epoch1, sync_tx1); await_start1_4.await; let epoch3 = epoch2.next_epoch(); @@ -2058,38 +2014,34 @@ pub(crate) mod tests { assert_uploader_pending(&mut uploader).await; finish_tx1_3.send(()).unwrap(); - if let UploaderEvent::SyncFinish(newly_upload_ssts) = uploader.next_event().await { - assert_eq!(2, newly_upload_ssts.len()); - assert_eq!( - &get_payload_imm_ids(&epoch1_sync_payload), - newly_upload_ssts[0].imm_ids() - ); - assert_eq!( - &get_payload_imm_ids(&epoch1_spill_payload3), - newly_upload_ssts[1].imm_ids() - ); - } else { - unreachable!("should be sync finish"); + let UploaderEvent::DataUploaded(sst) = uploader.next_event().await; + { + assert_eq!(&get_payload_imm_ids(&epoch1_spill_payload3), sst.imm_ids()); + } + + assert!(poll_fn(|cx| Poll::Ready(sync_rx1.poll_unpin(cx).is_pending())).await); + + let UploaderEvent::DataUploaded(sst) = uploader.next_event().await; + { + assert_eq!(&get_payload_imm_ids(&epoch1_sync_payload), sst.imm_ids()); } if let Ok(Ok(data)) = sync_rx1.await { - let newly_upload_ssts = data.newly_upload_ssts.unwrap(); - assert_eq!(2, newly_upload_ssts.len()); - assert_eq!(1, data.uploaded_ssts.len()); + assert_eq!(3, data.uploaded_ssts.len()); assert_eq!( &get_payload_imm_ids(&epoch1_sync_payload), - newly_upload_ssts[0].imm_ids() + data.uploaded_ssts[0].imm_ids() ); assert_eq!( &get_payload_imm_ids(&epoch1_spill_payload3), - newly_upload_ssts[1].imm_ids() + data.uploaded_ssts[1].imm_ids() ); assert_eq!( &get_payload_imm_ids(&epoch1_spill_payload12), - data.uploaded_ssts[0].imm_ids() + data.uploaded_ssts[2].imm_ids() ); } else { - unreachable!("should be sync finish"); + unreachable!() } // current uploader state: @@ -2102,13 +2054,11 @@ pub(crate) mod tests { let (sync_tx2, sync_rx2) = oneshot::channel(); uploader.start_sync_epoch(epoch2, sync_tx2); uploader.local_seal_epoch_for_test(instance_id2, epoch3); - if let UploaderEvent::DataSpilled(sst) = uploader.next_event().await { + let UploaderEvent::DataUploaded(sst) = uploader.next_event().await; + { assert_eq!(&get_payload_imm_ids(&epoch3_spill_payload1), sst.imm_ids()); - } else { - unreachable!("should be data spilled"); } if let Ok(Ok(data)) = sync_rx2.await { - assert!(data.newly_upload_ssts.is_none()); assert_eq!(data.uploaded_ssts.len(), 1); assert_eq!( &get_payload_imm_ids(&epoch2_spill_payload), @@ -2131,7 +2081,7 @@ pub(crate) mod tests { let epoch4_sync_payload = HashMap::from_iter([(instance_id1, vec![imm4, imm3_3])]); let (await_start4_with_3_3, finish_tx4_with_3_3) = new_task_notifier(get_payload_imm_ids(&epoch4_sync_payload)); - let (sync_tx4, sync_rx4) = oneshot::channel(); + let (sync_tx4, mut sync_rx4) = oneshot::channel(); uploader.start_sync_epoch(epoch4, sync_tx4); await_start4_with_3_3.await; @@ -2143,39 +2093,34 @@ pub(crate) mod tests { // epoch2: sst([imm2]) assert_uploader_pending(&mut uploader).await; + finish_tx3_2.send(()).unwrap(); - assert_uploader_pending(&mut uploader).await; + let UploaderEvent::DataUploaded(sst) = uploader.next_event().await; + { + assert_eq!(&get_payload_imm_ids(&epoch3_spill_payload2), sst.imm_ids()); + } + finish_tx4_with_3_3.send(()).unwrap(); + assert!(poll_fn(|cx| Poll::Ready(sync_rx4.poll_unpin(cx).is_pending())).await); - if let UploaderEvent::SyncFinish(newly_upload_ssts) = uploader.next_event().await { - assert_eq!(2, newly_upload_ssts.len()); - assert_eq!( - &get_payload_imm_ids(&epoch4_sync_payload), - newly_upload_ssts[0].imm_ids() - ); - assert_eq!( - &get_payload_imm_ids(&epoch3_spill_payload2), - newly_upload_ssts[1].imm_ids() - ); - } else { - unreachable!("should be sync finish"); + let UploaderEvent::DataUploaded(sst) = uploader.next_event().await; + { + assert_eq!(&get_payload_imm_ids(&epoch4_sync_payload), sst.imm_ids()); } if let Ok(Ok(data)) = sync_rx4.await { - let newly_upload_ssts = data.newly_upload_ssts.unwrap(); - assert_eq!(2, newly_upload_ssts.len()); + assert_eq!(3, data.uploaded_ssts.len()); assert_eq!( &get_payload_imm_ids(&epoch4_sync_payload), - newly_upload_ssts[0].imm_ids() + data.uploaded_ssts[0].imm_ids() ); assert_eq!( &get_payload_imm_ids(&epoch3_spill_payload2), - newly_upload_ssts[1].imm_ids() + data.uploaded_ssts[1].imm_ids() ); - assert_eq!(1, data.uploaded_ssts.len()); assert_eq!( &get_payload_imm_ids(&epoch3_spill_payload1), - data.uploaded_ssts[0].imm_ids(), + data.uploaded_ssts[2].imm_ids(), ) } else { unreachable!("should be sync finish"); From 85b2ded9bfb333698fb7dabca0db132651a3352e Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 19 Jun 2024 01:25:25 +0800 Subject: [PATCH 03/17] extract poll_syncing_task --- .../src/hummock/event_handler/uploader.rs | 225 +++++++++++------- .../src/hummock/store/hummock_storage.rs | 6 +- 2 files changed, 144 insertions(+), 87 deletions(-) diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader.rs index a4b19ecc0dd27..cb8fa3553459f 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader.rs @@ -974,12 +974,14 @@ impl UploaderData { } } +struct ErrState { + failed_epoch: HummockEpoch, + reason: String, +} + enum UploaderState { Working(UploaderData), - Err { - failed_epoch: HummockEpoch, - reason: String, - }, + Err(ErrState), } /// An uploader for hummock data. @@ -1090,10 +1092,10 @@ impl HummockUploader { ) { let data = match &mut self.state { UploaderState::Working(data) => data, - UploaderState::Err { + UploaderState::Err(ErrState { failed_epoch, reason, - } => { + }) => { let result = Err(HummockError::other(format!( "previous epoch {} failed due to [{}]", failed_epoch, reason @@ -1138,6 +1140,26 @@ impl HummockUploader { .set(data.syncing_data.len() as _); } + fn set_max_synced_epoch( + max_synced_epoch: &mut HummockEpoch, + max_syncing_epoch: HummockEpoch, + epoch: HummockEpoch, + ) { + assert!( + epoch <= max_syncing_epoch, + "epoch {} that has been synced has not started syncing yet. previous max syncing epoch {}", + epoch, + max_syncing_epoch + ); + assert!( + epoch > *max_synced_epoch, + "epoch {} has been synced. previous max synced epoch: {}", + epoch, + max_synced_epoch + ); + *max_synced_epoch = epoch; + } + pub(crate) fn update_pinned_version(&mut self, pinned_version: PinnedVersion) { assert_ge!( pinned_version.max_committed_epoch(), @@ -1227,6 +1249,82 @@ impl HummockUploader { } impl UploaderData { + /// Poll the syncing task of the syncing data of the oldest epoch. Return `Poll::Ready(None)` if + /// there is no syncing data. + fn poll_syncing_task( + &mut self, + cx: &mut Context<'_>, + context: &UploaderContext, + mut set_max_synced_epoch: impl FnMut(u64), + ) -> Poll, ErrState>>> { + while let Some(syncing_data) = self.syncing_data.back_mut() { + let sstable_info = if let Some(task) = syncing_data.uploading_tasks.back_mut() { + let result = ready!(task.poll_result(cx)); + let _task = syncing_data.uploading_tasks.pop_back().expect("non-empty"); + let sstable_info = match result { + Ok(sstable_info) => sstable_info, + Err(e) => { + let SyncingData { + sync_epoch, + uploading_tasks, + sync_result_sender, + .. + } = self.syncing_data.pop_back().expect("non-empty"); + for task in uploading_tasks { + task.join_handle.abort(); + } + send_sync_result( + sync_result_sender, + Err(HummockError::other(format!( + "failed sync task: {:?}", + e.as_report() + ))), + ); + + return Poll::Ready(Some(Err(ErrState { + failed_epoch: sync_epoch, + reason: format!("{:?}", e.as_report()), + }))); + } + }; + syncing_data.uploaded.push_front(sstable_info.clone()); + self.unsync_data.ack_flushed(&sstable_info); + Some(sstable_info) + } else { + None + }; + + if syncing_data.uploading_tasks.is_empty() { + let syncing_data = self.syncing_data.pop_back().expect("non-empty"); + let SyncingData { + sync_epoch, + uploading_tasks, + uploaded, + table_watermarks, + sync_result_sender, + } = syncing_data; + assert!(uploading_tasks.is_empty()); + context + .stats + .uploader_syncing_epoch_count + .set(self.syncing_data.len() as _); + set_max_synced_epoch(sync_epoch); + send_sync_result( + sync_result_sender, + Ok(SyncedData { + uploaded_ssts: uploaded, + table_watermarks, + }), + ) + } + + if let Some(sstable_info) = sstable_info { + return Poll::Ready(Some(Ok(sstable_info))); + } + } + Poll::Ready(None) + } + /// Poll the success of the oldest spilled task of unsync spill data. Return `Poll::Ready(None)` if /// there is no spilling task. fn poll_spill_task(&mut self, cx: &mut Context<'_>) -> Poll>> { @@ -1253,87 +1351,37 @@ impl HummockUploader { let UploaderState::Working(data) = &mut self.state else { return Poll::Pending; }; - while let Some(syncing_data) = data.syncing_data.back_mut() { - let sstable_info = if let Some(task) = syncing_data.uploading_tasks.back_mut() { - let result = ready!(task.poll_result(cx)); - let _task = syncing_data.uploading_tasks.pop_back().expect("non-empty"); - let sstable_info = match result { - Ok(sstable_info) => sstable_info, - Err(e) => { - let SyncingData { - sync_epoch, - uploading_tasks, - sync_result_sender, - .. - } = data.syncing_data.pop_back().expect("non-empty"); - for task in uploading_tasks { - task.join_handle.abort(); - } - send_sync_result( - sync_result_sender, - Err(HummockError::other(format!( - "failed sync task: {:?}", - e.as_report() - ))), - ); - let data = must_match!(replace( - &mut self.state, - UploaderState::Err { - failed_epoch: sync_epoch, - reason: format!("{:?}", e.as_report()), - }, - ), UploaderState::Working(data) => data); - - data.abort(|| { - HummockError::other(format!( - "previous epoch {} failed to sync", - sync_epoch - )) - }); - return Poll::Pending; - } - }; - syncing_data.uploaded.push_front(sstable_info.clone()); - data.unsync_data.ack_flushed(&sstable_info); - Some(sstable_info) - } else { - None - }; - - if syncing_data.uploading_tasks.is_empty() { - let syncing_data = data.syncing_data.pop_back().expect("non-empty"); - let SyncingData { - sync_epoch: epoch, - uploading_tasks, - uploaded, - table_watermarks, - sync_result_sender, - } = syncing_data; - assert!(uploading_tasks.is_empty()); - assert!( - epoch <= self.max_syncing_epoch, - "epoch {} that has been synced has not started syncing yet. previous max syncing epoch {}", - epoch, - self.max_syncing_epoch - ); - assert!( - epoch > self.max_synced_epoch, - "epoch {} has been synced. previous max synced epoch: {}", - epoch, - self.max_synced_epoch - ); - self.max_synced_epoch = epoch; - send_sync_result( - sync_result_sender, - Ok(SyncedData { - uploaded_ssts: uploaded, - table_watermarks, - }), - ) - } - if let Some(sstable_info) = sstable_info { - return Poll::Ready(UploaderEvent::DataUploaded(sstable_info)); + if let Some(result) = + ready!( + data.poll_syncing_task(cx, &self.context, |new_synced_epoch| { + Self::set_max_synced_epoch( + &mut self.max_synced_epoch, + self.max_syncing_epoch, + new_synced_epoch, + ) + }) + ) + { + match result { + Ok(data) => { + return Poll::Ready(UploaderEvent::DataUploaded(data)); + } + Err(e) => { + let failed_epoch = e.failed_epoch; + let data = must_match!(replace( + &mut self.state, + UploaderState::Err(e), + ), UploaderState::Working(data) => data); + + data.abort(|| { + HummockError::other(format!( + "previous epoch {} failed to sync", + failed_epoch + )) + }); + return Poll::Pending; + } } } @@ -1752,6 +1800,11 @@ pub(crate) mod tests { async fn test_uploader_poll_empty() { let mut uploader = test_uploader(dummy_success_upload_future); let data = must_match!(&mut uploader.state, UploaderState::Working(data) => data); + assert!( + poll_fn(|cx| data.poll_syncing_task(cx, &uploader.context, |_| unreachable!())) + .await + .is_none() + ); assert!(poll_fn(|cx| data.poll_spill_task(cx)).await.is_none()); } diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index ff27c3cc24fef..b4a3c55bbf829 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -570,7 +570,11 @@ impl StateStore for HummockStorage { sync_result_sender: tx, }) .expect("should send success"); - rx.map(|recv_result| Ok(recv_result.expect("should wait success")?.into_sync_result())) + rx.map(|recv_result| { + Ok(recv_result + .expect("should wait success")? + .into_sync_result()) + }) } fn seal_epoch(&self, epoch: u64, is_checkpoint: bool) { From 892dd29827842f2ccffd205408aed47b70053415 Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 19 Jun 2024 01:39:55 +0800 Subject: [PATCH 04/17] remove uploader event --- .../event_handler/hummock_event_handler.rs | 26 +++-- .../src/hummock/event_handler/uploader.rs | 94 +++++++------------ 2 files changed, 47 insertions(+), 73 deletions(-) diff --git a/src/storage/src/hummock/event_handler/hummock_event_handler.rs b/src/storage/src/hummock/event_handler/hummock_event_handler.rs index c0d58150c754c..464abdd722367 100644 --- a/src/storage/src/hummock/event_handler/hummock_event_handler.rs +++ b/src/storage/src/hummock/event_handler/hummock_event_handler.rs @@ -43,7 +43,7 @@ use crate::hummock::compactor::{await_tree_key, compact, CompactorContext}; use crate::hummock::conflict_detector::ConflictDetector; use crate::hummock::event_handler::refiller::{CacheRefillerEvent, SpawnRefillTask}; use crate::hummock::event_handler::uploader::{ - HummockUploader, SpawnUploadTask, SyncedData, UploadTaskInfo, UploadTaskOutput, UploaderEvent, + HummockUploader, SpawnUploadTask, SyncedData, UploadTaskInfo, UploadTaskOutput, }; use crate::hummock::event_handler::{ HummockEvent, HummockReadVersionRef, HummockVersionUpdate, ReadOnlyReadVersionMapping, @@ -450,8 +450,8 @@ impl HummockEventHandler { } } - fn handle_data_spilled(&mut self, staging_sstable_info: Arc) { - trace!("data_spilled. SST size {}", staging_sstable_info.imm_size()); + fn handle_uploaded_sst_inner(&mut self, staging_sstable_info: Arc) { + trace!("data_flushed. SST size {}", staging_sstable_info.imm_size()); self.for_each_read_version( staging_sstable_info.imm_ids().keys().cloned(), |_, read_version| { @@ -676,8 +676,8 @@ impl HummockEventHandler { pub async fn start_hummock_event_handler_worker(mut self) { loop { tokio::select! { - event = self.uploader.next_event() => { - self.handle_uploader_event(event); + sst = self.uploader.next_uploaded_sst() => { + self.handle_uploaded_sst(sst); } event = self.refiller.next_event() => { let CacheRefillerEvent {pinned_version, new_pinned_version } = event; @@ -709,16 +709,12 @@ impl HummockEventHandler { } } - fn handle_uploader_event(&mut self, event: UploaderEvent) { - match event { - UploaderEvent::DataUploaded(staging_sstable_info) => { - let _timer = self - .metrics - .event_handler_on_upload_finish_latency - .start_timer(); - self.handle_data_spilled(staging_sstable_info); - } - } + fn handle_uploaded_sst(&mut self, sst: Arc) { + let _timer = self + .metrics + .event_handler_on_upload_finish_latency + .start_timer(); + self.handle_uploaded_sst_inner(sst); } /// Gracefully shutdown if returns `true`. diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader.rs index cb8fa3553459f..ee0e975e5a3ea 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader.rs @@ -1341,12 +1341,10 @@ impl UploaderData { } } -pub(super) enum UploaderEvent { - DataUploaded(Arc), -} - impl HummockUploader { - pub(super) fn next_event(&mut self) -> impl Future + '_ { + pub(super) fn next_uploaded_sst( + &mut self, + ) -> impl Future> + '_ { poll_fn(|cx| { let UploaderState::Working(data) = &mut self.state else { return Poll::Pending; @@ -1365,7 +1363,7 @@ impl HummockUploader { { match result { Ok(data) => { - return Poll::Ready(UploaderEvent::DataUploaded(data)); + return Poll::Ready(data); } Err(e) => { let failed_epoch = e.failed_epoch; @@ -1386,7 +1384,7 @@ impl HummockUploader { } if let Some(sstable_info) = ready!(data.poll_spill_task(cx)) { - return Poll::Ready(UploaderEvent::DataUploaded(sstable_info)); + return Poll::Ready(sstable_info); } Poll::Pending @@ -1426,8 +1424,7 @@ pub(crate) mod tests { use crate::hummock::event_handler::uploader::uploader_imm::UploaderImm; use crate::hummock::event_handler::uploader::{ get_payload_imm_ids, HummockUploader, SyncedData, UploadTaskInfo, UploadTaskOutput, - UploadTaskPayload, UploaderContext, UploaderData, UploaderEvent, UploaderState, - UploadingTask, + UploadTaskPayload, UploaderContext, UploaderData, UploaderState, UploadingTask, }; use crate::hummock::event_handler::{LocalInstanceId, TEST_LOCAL_INSTANCE_ID}; use crate::hummock::local_version::pinned_version::PinnedVersion; @@ -1691,19 +1688,16 @@ pub(crate) mod tests { assert!(syncing_data.uploaded.is_empty()); assert!(!syncing_data.uploading_tasks.is_empty()); - match uploader.next_event().await { - UploaderEvent::DataUploaded(staging_sst) => { - assert_eq!(&vec![epoch1], staging_sst.epochs()); - assert_eq!( - &HashMap::from_iter([(TEST_LOCAL_INSTANCE_ID, vec![imm.batch_id()])]), - staging_sst.imm_ids() - ); - assert_eq!( - &dummy_success_upload_output().new_value_ssts, - staging_sst.sstable_infos() - ); - } - }; + let staging_sst = uploader.next_uploaded_sst().await; + assert_eq!(&vec![epoch1], staging_sst.epochs()); + assert_eq!( + &HashMap::from_iter([(TEST_LOCAL_INSTANCE_ID, vec![imm.batch_id()])]), + staging_sst.imm_ids() + ); + assert_eq!( + &dummy_success_upload_output().new_value_ssts, + staging_sst.sstable_infos() + ); match sync_rx.await { Ok(Ok(data)) => { @@ -1846,11 +1840,8 @@ pub(crate) mod tests { assert_eq!(epoch4, uploader.max_synced_epoch); assert_eq!(epoch6, uploader.max_syncing_epoch); - match uploader.next_event().await { - UploaderEvent::DataUploaded(sst) => { - assert_eq!(&get_imm_ids([&imm]), sst.imm_ids()); - } - } + let sst = uploader.next_uploaded_sst().await; + assert_eq!(&get_imm_ids([&imm]), sst.imm_ids()); match sync_rx.await { Ok(Ok(data)) => { @@ -1936,7 +1927,7 @@ pub(crate) mod tests { yield_now().await; } assert!( - poll_fn(|cx| Poll::Ready(uploader.next_event().poll_unpin(cx))) + poll_fn(|cx| Poll::Ready(uploader.next_uploaded_sst().poll_unpin(cx))) .await .is_pending() ) @@ -1989,17 +1980,13 @@ pub(crate) mod tests { assert_uploader_pending(&mut uploader).await; finish_tx1.send(()).unwrap(); - let UploaderEvent::DataUploaded(sst) = uploader.next_event().await; - { - assert_eq!(&get_payload_imm_ids(&epoch1_spill_payload12), sst.imm_ids()); - assert_eq!(&vec![epoch1], sst.epochs()); - } + let sst = uploader.next_uploaded_sst().await; + assert_eq!(&get_payload_imm_ids(&epoch1_spill_payload12), sst.imm_ids()); + assert_eq!(&vec![epoch1], sst.epochs()); - let UploaderEvent::DataUploaded(sst) = uploader.next_event().await; - { - assert_eq!(&get_payload_imm_ids(&epoch2_spill_payload), sst.imm_ids()); - assert_eq!(&vec![epoch2], sst.epochs()); - } + let sst = uploader.next_uploaded_sst().await; + assert_eq!(&get_payload_imm_ids(&epoch2_spill_payload), sst.imm_ids()); + assert_eq!(&vec![epoch2], sst.epochs()); let imm1_3 = gen_imm_with_limiter(epoch1, memory_limiter).await; uploader.add_imm(instance_id1, imm1_3.clone()); @@ -2067,17 +2054,13 @@ pub(crate) mod tests { assert_uploader_pending(&mut uploader).await; finish_tx1_3.send(()).unwrap(); - let UploaderEvent::DataUploaded(sst) = uploader.next_event().await; - { - assert_eq!(&get_payload_imm_ids(&epoch1_spill_payload3), sst.imm_ids()); - } + let sst = uploader.next_uploaded_sst().await; + assert_eq!(&get_payload_imm_ids(&epoch1_spill_payload3), sst.imm_ids()); assert!(poll_fn(|cx| Poll::Ready(sync_rx1.poll_unpin(cx).is_pending())).await); - let UploaderEvent::DataUploaded(sst) = uploader.next_event().await; - { - assert_eq!(&get_payload_imm_ids(&epoch1_sync_payload), sst.imm_ids()); - } + let sst = uploader.next_uploaded_sst().await; + assert_eq!(&get_payload_imm_ids(&epoch1_sync_payload), sst.imm_ids()); if let Ok(Ok(data)) = sync_rx1.await { assert_eq!(3, data.uploaded_ssts.len()); @@ -2107,10 +2090,9 @@ pub(crate) mod tests { let (sync_tx2, sync_rx2) = oneshot::channel(); uploader.start_sync_epoch(epoch2, sync_tx2); uploader.local_seal_epoch_for_test(instance_id2, epoch3); - let UploaderEvent::DataUploaded(sst) = uploader.next_event().await; - { - assert_eq!(&get_payload_imm_ids(&epoch3_spill_payload1), sst.imm_ids()); - } + let sst = uploader.next_uploaded_sst().await; + assert_eq!(&get_payload_imm_ids(&epoch3_spill_payload1), sst.imm_ids()); + if let Ok(Ok(data)) = sync_rx2.await { assert_eq!(data.uploaded_ssts.len(), 1); assert_eq!( @@ -2148,18 +2130,14 @@ pub(crate) mod tests { assert_uploader_pending(&mut uploader).await; finish_tx3_2.send(()).unwrap(); - let UploaderEvent::DataUploaded(sst) = uploader.next_event().await; - { - assert_eq!(&get_payload_imm_ids(&epoch3_spill_payload2), sst.imm_ids()); - } + let sst = uploader.next_uploaded_sst().await; + assert_eq!(&get_payload_imm_ids(&epoch3_spill_payload2), sst.imm_ids()); finish_tx4_with_3_3.send(()).unwrap(); assert!(poll_fn(|cx| Poll::Ready(sync_rx4.poll_unpin(cx).is_pending())).await); - let UploaderEvent::DataUploaded(sst) = uploader.next_event().await; - { - assert_eq!(&get_payload_imm_ids(&epoch4_sync_payload), sst.imm_ids()); - } + let sst = uploader.next_uploaded_sst().await; + assert_eq!(&get_payload_imm_ids(&epoch4_sync_payload), sst.imm_ids()); if let Ok(Ok(data)) = sync_rx4.await { assert_eq!(3, data.uploaded_ssts.len()); From 2874313d705fdbd2975f2cb62ea9da5498d46245 Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 19 Jun 2024 15:27:21 +0800 Subject: [PATCH 05/17] fix compile --- .../src/hummock/event_handler/hummock_event_handler.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/storage/src/hummock/event_handler/hummock_event_handler.rs b/src/storage/src/hummock/event_handler/hummock_event_handler.rs index 464abdd722367..53d7167e0167b 100644 --- a/src/storage/src/hummock/event_handler/hummock_event_handler.rs +++ b/src/storage/src/hummock/event_handler/hummock_event_handler.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{HashMap, HashSet, VecDeque}; +use std::collections::{HashMap, VecDeque}; use std::pin::pin; use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering::Relaxed; @@ -403,7 +403,7 @@ impl HummockEventHandler { #[cfg(debug_assertions)] { // check duplication on debug_mode - let mut id_set = HashSet::new(); + let mut id_set = std::collections::HashSet::new(); for instance in instances { assert!(id_set.insert(instance)); } From 82a41918645b183f12775e83e3d16b5bf5954c0a Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 26 Jun 2024 02:44:27 +0800 Subject: [PATCH 06/17] feat(storage): decouple upload task finish order from epoch --- .../src/hummock/event_handler/uploader.rs | 576 +++++++++--------- 1 file changed, 291 insertions(+), 285 deletions(-) diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader.rs index ee0e975e5a3ea..738454ad02efc 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader.rs @@ -14,12 +14,13 @@ use std::cmp::Ordering; use std::collections::btree_map::Entry; -use std::collections::{BTreeMap, HashMap, VecDeque}; +use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet, VecDeque}; use std::fmt::{Debug, Display, Formatter}; use std::future::{poll_fn, Future}; use std::mem::{replace, swap, take}; -use std::pin::Pin; -use std::sync::Arc; +use std::sync::atomic::AtomicUsize; +use std::sync::atomic::Ordering::Relaxed; +use std::sync::{Arc, LazyLock}; use std::task::{ready, Context, Poll}; use futures::FutureExt; @@ -161,9 +162,13 @@ mod uploader_imm { } } +#[derive(PartialEq, Eq, Hash, PartialOrd, Ord, Copy, Clone, Debug)] +struct UploadingTaskId(usize); + /// A wrapper for a uploading task that compacts and uploads the imm payload. Task context are /// stored so that when the task fails, it can be re-tried. struct UploadingTask { + task_id: UploadingTaskId, // newer data at the front input: UploadTaskInput, join_handle: JoinHandle>, @@ -253,7 +258,9 @@ impl UploadingTask { } let join_handle = (context.spawn_upload_task)(payload, task_info.clone()); context.stats.uploader_uploading_task_count.inc(); + static NEXT_TASK_ID: LazyLock = LazyLock::new(|| AtomicUsize::new(0)); Self { + task_id: UploadingTaskId(NEXT_TASK_ID.fetch_add(1, Relaxed)), input, join_handle, task_info, @@ -324,81 +331,157 @@ impl UploadingTask { } } -impl Future for UploadingTask { - type Output = HummockResult>; +mod task_manager { + use super::*; - fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - self.poll_result(cx) + #[derive(Debug)] + pub(super) enum UploadingTaskStatus { + Spilling, + Sync(HummockEpoch), } -} -#[derive(Default, Debug)] -/// Manage the spilled data. Task and uploaded data at the front is newer data. Task data are -/// always newer than uploaded data. Task holding oldest data is always collected first. -struct SpilledData { - // ordered spilling tasks. Task at the back is spilling older data. - uploading_tasks: VecDeque, - // ordered spilled data. Data at the back is older. - uploaded_data: VecDeque>, -} + #[derive(Debug)] + enum TaskStatus { + Uploading(UploadingTaskStatus), + Spilled(Arc), + } -impl SpilledData { - fn add_task(&mut self, task: UploadingTask) { - self.uploading_tasks.push_front(task); + #[derive(Default, Debug)] + pub(super) struct TaskManager { + // newer task at the front + uploading_tasks: VecDeque, + task_status: HashMap, } - /// Poll the successful spill of the oldest uploading task. Return `Poll::Ready(None)` is there - /// is no uploading task - fn poll_success_spill( - &mut self, - cx: &mut Context<'_>, - ) -> Poll>> { - // only poll the oldest uploading task if there is any - if let Some(task) = self.uploading_tasks.back_mut() { - let staging_sstable_info = ready!(task.poll_ok_with_retry(cx)); - self.uploaded_data.push_front(staging_sstable_info.clone()); - self.uploading_tasks.pop_back(); - Poll::Ready(Some(staging_sstable_info)) - } else { - Poll::Ready(None) + impl TaskManager { + fn add_task(&mut self, task: UploadingTask, status: UploadingTaskStatus) { + self.task_status + .insert(task.task_id, TaskStatus::Uploading(status)); + self.uploading_tasks.push_front(task); } - } - fn abort(self) { - for task in self.uploading_tasks { - task.join_handle.abort(); + #[expect(clippy::type_complexity)] + pub(super) fn poll_task_result( + &mut self, + cx: &mut Context<'_>, + _context: &UploaderContext, + ) -> Poll< + Option< + Result< + ( + UploadingTaskId, + UploadingTaskStatus, + Arc, + ), + ErrState, + >, + >, + > { + if let Some(task) = self.uploading_tasks.back_mut() { + let result = match self.task_status.get(&task.task_id).expect("should exist") { + TaskStatus::Uploading(UploadingTaskStatus::Spilling) => { + let sst = ready!(task.poll_ok_with_retry(cx)); + self.task_status + .insert(task.task_id, TaskStatus::Spilled(sst.clone())); + Ok((task.task_id, UploadingTaskStatus::Spilling, sst)) + } + TaskStatus::Uploading(UploadingTaskStatus::Sync(epoch)) => { + let epoch = *epoch; + let result = ready!(task.poll_result(cx)); + let _status = self.task_status.remove(&task.task_id); + result + .map(|sst| (task.task_id, UploadingTaskStatus::Sync(epoch), sst)) + .map_err(|e| ErrState { + failed_epoch: epoch, + reason: e.as_report().to_string(), + }) + } + TaskStatus::Spilled(_) => { + unreachable!("should be uploading task") + } + }; + + let _task = self.uploading_tasks.pop_back().expect("non-empty"); + Poll::Ready(Some(result)) + } else { + Poll::Ready(None) + } } - } -} -#[derive(Default, Debug)] -struct EpochData { - spilled_data: SpilledData, -} + pub(super) fn abort(self) { + for task in self.uploading_tasks { + task.join_handle.abort(); + } + } -impl EpochData { - fn flush( - &mut self, - context: &UploaderContext, - imms: HashMap>, - ) -> usize { - if !imms.is_empty() { + pub(super) fn spill( + &mut self, + context: &UploaderContext, + imms: HashMap>, + ) -> (UploadingTaskId, usize) { + assert!(!imms.is_empty()); let task = UploadingTask::new(imms, context); context.stats.spill_task_counts_from_unsealed.inc(); context .stats .spill_task_size_from_unsealed .inc_by(task.task_info.task_size as u64); - info!("Spill unsealed data. Task: {}", task.get_task_info()); + info!("Spill data. Task: {}", task.get_task_info()); let size = task.task_info.task_size; - self.spilled_data.add_task(task); - size - } else { - 0 + let id = task.task_id; + self.add_task(task, UploadingTaskStatus::Spilling); + (id, size) + } + + pub(super) fn sync( + &mut self, + context: &UploaderContext, + epoch: HummockEpoch, + spilled_task: BTreeSet, + unflushed_payload: UploadTaskInput, + ) -> (HashSet, VecDeque>) { + let mut remaining_tasks = HashSet::new(); + let total_task_count = if unflushed_payload.is_empty() { + spilled_task.len() + } else { + let task = UploadingTask::new(unflushed_payload, context); + remaining_tasks.insert(task.task_id); + self.task_status.insert( + task.task_id, + TaskStatus::Uploading(UploadingTaskStatus::Sync(epoch)), + ); + self.add_task(task, UploadingTaskStatus::Sync(epoch)); + spilled_task.len() + 1 + }; + let mut uploaded = VecDeque::with_capacity(total_task_count); + + // iterate from small task id to large, which means from old data to new data + for task_id in spilled_task { + let status = self.task_status.remove(&task_id).expect("should exist"); + match status { + TaskStatus::Uploading(UploadingTaskStatus::Spilling) => { + self.task_status.insert( + task_id, + TaskStatus::Uploading(UploadingTaskStatus::Sync(epoch)), + ); + remaining_tasks.insert(task_id); + } + TaskStatus::Uploading(UploadingTaskStatus::Sync(_)) => { + unreachable!("cannot be synced again") + } + TaskStatus::Spilled(sst) => { + self.task_status.remove(&task_id); + uploaded.push_front(sst); + } + } + } + (remaining_tasks, uploaded) } } } +use task_manager::{TaskManager, UploadingTaskStatus}; + impl TableUnsyncData { fn add_table_watermarks( &mut self, @@ -455,68 +538,14 @@ impl TableUnsyncData { #[derive(Default)] struct SyncDataBuilder { - // newer epochs come first - epochs: VecDeque, - - spilled_data: SpilledData, - table_watermarks: HashMap, + remaining_uploading_tasks: HashSet, + uploaded: VecDeque>, } impl SyncDataBuilder { - /// Add the data of a new epoch. - /// - /// Note: it may happen that, for example, currently we hold `imms` and `spilled_data` of epoch - /// 3, and after we add the spilled data of epoch 4, both `imms` and `spilled_data` hold data - /// of both epoch 3 and 4, which seems breaking the rules that data in `imms` are - /// always newer than data in `spilled_data`, because epoch 3 data of `imms` - /// seems older than epoch 4 data of `spilled_data`. However, if this happens, the epoch 3 - /// data of `imms` must not overlap with the epoch 4 data of `spilled_data`. The explanation is - /// as followed: - /// - /// First, unsync data has 3 stages, from earlier to later, imms, uploading task, and - /// uploaded. When we try to spill unsync data, we first pick the imms of older epoch until - /// the imms of older epoch are all picked. When we try to poll the uploading tasks of unsync - /// data, we first poll the task of older epoch, until there is no uploading task in older - /// epoch. Therefore, we can reach that, if two data are in the same stage, but - /// different epochs, data in the older epoch will always enter the next stage earlier than data - /// in the newer epoch. - /// - /// Second, we have an assumption that, if a key has been written in a newer epoch, e.g. epoch4, - /// it will no longer be written in an older epoch, e.g. epoch3, and then, if two data of the - /// same key are at the imm stage, the data of older epoch must appear earlier than the data - /// of newer epoch. - /// - /// Based on the two points above, we can reach that, if two data of a same key appear in - /// different epochs, the data of older epoch will not appear at a later stage than the data - /// of newer epoch. Therefore, we can safely merge the data of each stage when we seal an epoch. - fn add_new_epoch(&mut self, epoch: HummockEpoch, mut unseal_epoch_data: EpochData) { - if let Some(prev_max_epoch) = self.epochs.front() { - assert!( - epoch > *prev_max_epoch, - "epoch {} to seal not greater than prev max epoch {}", - epoch, - prev_max_epoch - ); - } - - self.epochs.push_front(epoch); - // for each local instance, earlier data must be spilled at earlier epoch. Therefore, since we add spill data from old epoch - // to new epoch, - unseal_epoch_data - .spilled_data - .uploading_tasks - .append(&mut self.spilled_data.uploading_tasks); - unseal_epoch_data - .spilled_data - .uploaded_data - .append(&mut self.spilled_data.uploaded_data); - self.spilled_data.uploading_tasks = unseal_epoch_data.spilled_data.uploading_tasks; - self.spilled_data.uploaded_data = unseal_epoch_data.spilled_data.uploaded_data; - } - fn add_table_watermarks( - &mut self, + all_table_watermarks: &mut HashMap, table_id: TableId, direction: WatermarkDirection, watermarks: impl Iterator)>, @@ -538,19 +567,11 @@ impl SyncDataBuilder { } } if let Some(table_watermarks) = table_watermarks { - assert!(self - .table_watermarks + assert!(all_table_watermarks .insert(table_id, table_watermarks) .is_none()); } } - - fn flush(&mut self, context: &UploaderContext, payload: UploadTaskInput) { - if !payload.is_empty() { - let task = UploadingTask::new(payload, context); - self.spilled_data.add_task(task); - } - } } struct LocalInstanceEpochData { @@ -740,6 +761,7 @@ struct TableUnsyncData { WatermarkDirection, BTreeMap, BitmapBuilder)>, )>, + spill_tasks: BTreeMap>, } impl TableUnsyncData { @@ -748,6 +770,7 @@ impl TableUnsyncData { table_id, instance_data: Default::default(), table_watermarks: None, + spill_tasks: Default::default(), } } @@ -760,6 +783,7 @@ impl TableUnsyncData { WatermarkDirection, impl Iterator)>, )>, + impl Iterator, ) { ( self.instance_data @@ -776,6 +800,9 @@ impl TableUnsyncData { .map(|(epoch, (watermarks, _))| (epoch, watermarks)), ) }), + take_before_epoch(&mut self.spill_tasks, epoch) + .into_values() + .flat_map(|tasks| tasks.into_iter()), ) } } @@ -789,7 +816,7 @@ struct UnsyncData { table_data: HashMap, // An index as a mapping from instance id to its table id instance_table_id: HashMap, - epoch_data: BTreeMap, + epochs: BTreeMap, } impl UnsyncData { @@ -818,7 +845,7 @@ impl UnsyncData { .instance_table_id .insert(instance_id, table_id) .is_none()); - self.epoch_data.entry(init_epoch).or_default(); + self.epochs.insert(init_epoch, ()); } fn instance_data( @@ -857,7 +884,7 @@ impl UnsyncData { .get_mut(&instance_id) .expect("should exist"); let epoch = instance_data.local_seal_epoch(next_epoch); - self.epoch_data.entry(next_epoch).or_default(); + self.epochs.insert(next_epoch, ()); if let Some((direction, table_watermarks)) = opts.table_watermarks { table_data.add_table_watermarks(epoch, table_watermarks, direction); } @@ -874,28 +901,47 @@ impl UnsyncData { } } - fn sync(&mut self, epoch: HummockEpoch, context: &UploaderContext) -> SyncDataBuilder { - let sync_epoch_data = take_before_epoch(&mut self.epoch_data, epoch); + fn sync( + &mut self, + epoch: HummockEpoch, + context: &UploaderContext, + task_manager: &mut TaskManager, + ) -> SyncDataBuilder { + // clean old epochs + let _epochs = take_before_epoch(&mut self.epochs, epoch); - let mut sync_data = SyncDataBuilder::default(); - for (epoch, epoch_data) in sync_epoch_data { - sync_data.add_new_epoch(epoch, epoch_data); - } + let mut all_table_watermarks = HashMap::new(); + let mut spill_tasks = BTreeSet::new(); let mut flush_payload = HashMap::new(); for (table_id, table_data) in &mut self.table_data { - let (unflushed_payload, table_watermarks) = table_data.sync(epoch); + let (unflushed_payload, table_watermarks, task_ids) = table_data.sync(epoch); for (instance_id, payload) in unflushed_payload { if !payload.is_empty() { flush_payload.insert(instance_id, payload); } } if let Some((direction, watermarks)) = table_watermarks { - sync_data.add_table_watermarks(*table_id, direction, watermarks); + SyncDataBuilder::add_table_watermarks( + &mut all_table_watermarks, + *table_id, + direction, + watermarks, + ); } + for task_id in task_ids { + spill_tasks.insert(task_id); + } + } + + let (remaining_uploading_tasks, uploaded) = + task_manager.sync(context, epoch, spill_tasks, flush_payload); + + SyncDataBuilder { + table_watermarks: all_table_watermarks, + remaining_uploading_tasks, + uploaded, } - sync_data.flush(context, flush_payload); - sync_data } fn ack_flushed(&mut self, sstable_info: &StagingSstableInfo) { @@ -910,8 +956,7 @@ impl UnsyncData { struct SyncingData { sync_epoch: HummockEpoch, - // task of newer data at the front - uploading_tasks: VecDeque, + remaining_uploading_tasks: HashSet, // newer data at the front uploaded: VecDeque>, table_watermarks: HashMap, @@ -957,18 +1002,15 @@ struct UploaderData { /// Data that has started syncing but not synced yet. `epoch` satisfies /// `max_synced_epoch < epoch <= max_syncing_epoch`. /// Newer epoch at the front - syncing_data: VecDeque, + syncing_data: BTreeMap, + + task_manager: TaskManager, } impl UploaderData { fn abort(self, err: impl Fn() -> HummockError) { - for (_, epoch_data) in self.unsync_data.epoch_data { - epoch_data.spilled_data.abort(); - } - for syncing_data in self.syncing_data { - for task in syncing_data.uploading_tasks { - task.join_handle.abort(); - } + self.task_manager.abort(); + for syncing_data in self.syncing_data.into_values() { send_sync_result(syncing_data.sync_result_sender, Err(err())); } } @@ -1019,10 +1061,7 @@ impl HummockUploader { Self { max_syncing_epoch: initial_epoch, max_synced_epoch: initial_epoch, - state: UploaderState::Working(UploaderData { - unsync_data: Default::default(), - syncing_data: Default::default(), - }), + state: UploaderState::Working(UploaderData::default()), context: UploaderContext::new( pinned_version, spawn_upload_task, @@ -1114,24 +1153,37 @@ impl HummockUploader { self.max_syncing_epoch = epoch; - let sync_data = data.unsync_data.sync(epoch, &self.context); + let sync_data = data + .unsync_data + .sync(epoch, &self.context, &mut data.task_manager); let SyncDataBuilder { - spilled_data: - SpilledData { - uploading_tasks, - uploaded_data, - }, table_watermarks, - .. + remaining_uploading_tasks, + uploaded, } = sync_data; - data.syncing_data.push_front(SyncingData { - sync_epoch: epoch, - uploading_tasks, - uploaded: uploaded_data, - table_watermarks, - sync_result_sender, + if let Some((prev_max_epoch, _)) = data.syncing_data.last_key_value() { + assert_gt!(epoch, *prev_max_epoch); + } + + data.syncing_data.insert( + epoch, + SyncingData { + sync_epoch: epoch, + remaining_uploading_tasks, + uploaded, + table_watermarks, + sync_result_sender, + }, + ); + + data.may_notify_sync_task(&self.context, |new_synced_epoch| { + Self::set_max_synced_epoch( + &mut self.max_synced_epoch, + self.max_syncing_epoch, + new_synced_epoch, + ) }); self.context @@ -1196,7 +1248,7 @@ impl HummockUploader { if self.context.buffer_tracker.need_flush() { let mut curr_batch_flush_size = 0; // iterate from older epoch to newer epoch - for (epoch, epoch_data) in &mut data.unsync_data.epoch_data { + for epoch in &mut data.unsync_data.epochs.keys() { if !self .context .buffer_tracker @@ -1204,19 +1256,31 @@ impl HummockUploader { { break; } + let mut spilled_table_ids = HashSet::new(); let mut payload = HashMap::new(); - for (instance_id, instance_data) in data - .unsync_data - .table_data - .values_mut() - .flat_map(|data| data.instance_data.iter_mut()) - { - let instance_payload = instance_data.spill(*epoch); - if !instance_payload.is_empty() { - payload.insert(*instance_id, instance_payload); + for (table_id, table_data) in &mut data.unsync_data.table_data { + for (instance_id, instance_data) in &mut table_data.instance_data { + let instance_payload = instance_data.spill(*epoch); + if !instance_payload.is_empty() { + payload.insert(*instance_id, instance_payload); + spilled_table_ids.insert(*table_id); + } + } + } + if !payload.is_empty() { + let (task_id, task_size) = data.task_manager.spill(&self.context, payload); + for table_id in spilled_table_ids { + data.unsync_data + .table_data + .get_mut(&table_id) + .expect("should exist") + .spill_tasks + .entry(*epoch) + .or_default() + .push_front(task_id); } + curr_batch_flush_size += task_size; } - curr_batch_flush_size += epoch_data.flush(&self.context, payload); } curr_batch_flush_size > 0 } else { @@ -1251,93 +1315,35 @@ impl HummockUploader { impl UploaderData { /// Poll the syncing task of the syncing data of the oldest epoch. Return `Poll::Ready(None)` if /// there is no syncing data. - fn poll_syncing_task( + fn may_notify_sync_task( &mut self, - cx: &mut Context<'_>, context: &UploaderContext, mut set_max_synced_epoch: impl FnMut(u64), - ) -> Poll, ErrState>>> { - while let Some(syncing_data) = self.syncing_data.back_mut() { - let sstable_info = if let Some(task) = syncing_data.uploading_tasks.back_mut() { - let result = ready!(task.poll_result(cx)); - let _task = syncing_data.uploading_tasks.pop_back().expect("non-empty"); - let sstable_info = match result { - Ok(sstable_info) => sstable_info, - Err(e) => { - let SyncingData { - sync_epoch, - uploading_tasks, - sync_result_sender, - .. - } = self.syncing_data.pop_back().expect("non-empty"); - for task in uploading_tasks { - task.join_handle.abort(); - } - send_sync_result( - sync_result_sender, - Err(HummockError::other(format!( - "failed sync task: {:?}", - e.as_report() - ))), - ); - - return Poll::Ready(Some(Err(ErrState { - failed_epoch: sync_epoch, - reason: format!("{:?}", e.as_report()), - }))); - } - }; - syncing_data.uploaded.push_front(sstable_info.clone()); - self.unsync_data.ack_flushed(&sstable_info); - Some(sstable_info) - } else { - None - }; - - if syncing_data.uploading_tasks.is_empty() { - let syncing_data = self.syncing_data.pop_back().expect("non-empty"); - let SyncingData { - sync_epoch, - uploading_tasks, - uploaded, + ) { + while let Some((_, syncing_data)) = self.syncing_data.first_key_value() + && syncing_data.remaining_uploading_tasks.is_empty() + { + let (_, syncing_data) = self.syncing_data.pop_first().expect("non-empty"); + let SyncingData { + sync_epoch, + remaining_uploading_tasks: _, + uploaded, + table_watermarks, + sync_result_sender, + } = syncing_data; + context + .stats + .uploader_syncing_epoch_count + .set(self.syncing_data.len() as _); + set_max_synced_epoch(sync_epoch); + send_sync_result( + sync_result_sender, + Ok(SyncedData { + uploaded_ssts: uploaded, table_watermarks, - sync_result_sender, - } = syncing_data; - assert!(uploading_tasks.is_empty()); - context - .stats - .uploader_syncing_epoch_count - .set(self.syncing_data.len() as _); - set_max_synced_epoch(sync_epoch); - send_sync_result( - sync_result_sender, - Ok(SyncedData { - uploaded_ssts: uploaded, - table_watermarks, - }), - ) - } - - if let Some(sstable_info) = sstable_info { - return Poll::Ready(Some(Ok(sstable_info))); - } - } - Poll::Ready(None) - } - - /// Poll the success of the oldest spilled task of unsync spill data. Return `Poll::Ready(None)` if - /// there is no spilling task. - fn poll_spill_task(&mut self, cx: &mut Context<'_>) -> Poll>> { - // iterator from older epoch to new epoch so that the spill task are finished in epoch order - for epoch_data in self.unsync_data.epoch_data.values_mut() { - // if None, there is no spilling task. Search for the unsync data of the next epoch in - // the next iteration. - if let Some(sstable_info) = ready!(epoch_data.spilled_data.poll_success_spill(cx)) { - self.unsync_data.ack_flushed(&sstable_info); - return Poll::Ready(Some(sstable_info)); - } + }), + ) } - Poll::Ready(None) } } @@ -1350,20 +1356,27 @@ impl HummockUploader { return Poll::Pending; }; - if let Some(result) = - ready!( - data.poll_syncing_task(cx, &self.context, |new_synced_epoch| { - Self::set_max_synced_epoch( - &mut self.max_synced_epoch, - self.max_syncing_epoch, - new_synced_epoch, - ) - }) - ) - { + if let Some(result) = ready!(data.task_manager.poll_task_result(cx, &self.context)) { match result { - Ok(data) => { - return Poll::Ready(data); + Ok((task_id, status, sst)) => { + data.unsync_data.ack_flushed(&sst); + + if let UploadingTaskStatus::Sync(sync_epoch) = status { + let syncing_data = data + .syncing_data + .get_mut(&sync_epoch) + .expect("should exist"); + syncing_data.uploaded.push_front(sst.clone()); + assert!(syncing_data.remaining_uploading_tasks.remove(&task_id)); + data.may_notify_sync_task(&self.context, |new_synced_epoch| { + Self::set_max_synced_epoch( + &mut self.max_synced_epoch, + self.max_syncing_epoch, + new_synced_epoch, + ) + }); + } + return Poll::Ready(sst); } Err(e) => { let failed_epoch = e.failed_epoch; @@ -1383,10 +1396,6 @@ impl HummockUploader { } } - if let Some(sstable_info) = ready!(data.poll_spill_task(cx)) { - return Poll::Ready(sstable_info); - } - Poll::Pending }) } @@ -1397,6 +1406,7 @@ pub(crate) mod tests { use std::collections::{HashMap, VecDeque}; use std::future::{poll_fn, Future}; use std::ops::Deref; + use std::pin::pin; use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering::SeqCst; use std::sync::Arc; @@ -1606,11 +1616,11 @@ pub(crate) mod tests { let imm = gen_imm(INITIAL_EPOCH).await; let imm_size = imm.size(); let imm_ids = get_imm_ids(vec![&imm]); - let task = UploadingTask::from_vec(vec![imm], &uploader_context); + let mut task = UploadingTask::from_vec(vec![imm], &uploader_context); assert_eq!(imm_size, task.task_info.task_size); assert_eq!(imm_ids, task.task_info.imm_ids); assert_eq!(vec![INITIAL_EPOCH], task.task_info.epochs); - let output = task.await.unwrap(); + let output = poll_fn(|cx| task.poll_result(cx)).await.unwrap(); assert_eq!( output.sstable_infos(), &dummy_success_upload_output().new_value_ssts @@ -1621,8 +1631,8 @@ pub(crate) mod tests { let uploader_context = test_uploader_context(dummy_fail_upload_future); let imm = gen_imm(INITIAL_EPOCH).await; - let task = UploadingTask::from_vec(vec![imm], &uploader_context); - let _ = task.await.unwrap_err(); + let mut task = UploadingTask::from_vec(vec![imm], &uploader_context); + let _ = poll_fn(|cx| task.poll_result(cx)).await.unwrap_err(); } #[tokio::test] @@ -1683,10 +1693,10 @@ pub(crate) mod tests { uploader.start_sync_epoch(epoch1, sync_tx); assert_eq!(epoch1 as HummockEpoch, uploader.max_syncing_epoch); assert_eq!(1, uploader.data().syncing_data.len()); - let syncing_data = uploader.data().syncing_data.front().unwrap(); + let (_, syncing_data) = uploader.data().syncing_data.first_key_value().unwrap(); assert_eq!(epoch1 as HummockEpoch, syncing_data.sync_epoch); assert!(syncing_data.uploaded.is_empty()); - assert!(!syncing_data.uploading_tasks.is_empty()); + assert!(!syncing_data.remaining_uploading_tasks.is_empty()); let staging_sst = uploader.next_uploaded_sst().await; assert_eq!(&vec![epoch1], staging_sst.epochs()); @@ -1793,13 +1803,9 @@ pub(crate) mod tests { #[tokio::test] async fn test_uploader_poll_empty() { let mut uploader = test_uploader(dummy_success_upload_future); - let data = must_match!(&mut uploader.state, UploaderState::Working(data) => data); - assert!( - poll_fn(|cx| data.poll_syncing_task(cx, &uploader.context, |_| unreachable!())) - .await - .is_none() - ); - assert!(poll_fn(|cx| data.poll_spill_task(cx)).await.is_none()); + let fut = uploader.next_uploaded_sst(); + let mut fut = pin!(fut); + assert!(poll_fn(|cx| Poll::Ready(fut.as_mut().poll(cx).is_pending())).await); } #[tokio::test] From eaac3f901de813938f04415b01ee84dec47a804b Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 26 Jun 2024 14:36:21 +0800 Subject: [PATCH 07/17] move to new file --- .../{uploader.rs => uploader/mod.rs} | 154 +---------------- .../event_handler/uploader/task_manager.rs | 160 ++++++++++++++++++ 2 files changed, 163 insertions(+), 151 deletions(-) rename src/storage/src/hummock/event_handler/{uploader.rs => uploader/mod.rs} (93%) create mode 100644 src/storage/src/hummock/event_handler/uploader/task_manager.rs diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader/mod.rs similarity index 93% rename from src/storage/src/hummock/event_handler/uploader.rs rename to src/storage/src/hummock/event_handler/uploader/mod.rs index 738454ad02efc..e4672db162f7e 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader/mod.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +mod task_manager; + use std::cmp::Ordering; use std::collections::btree_map::Entry; use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet, VecDeque}; @@ -36,6 +38,7 @@ use risingwave_hummock_sdk::table_watermark::{ TableWatermarks, VnodeWatermark, WatermarkDirection, }; use risingwave_hummock_sdk::{CompactionGroupId, HummockEpoch, LocalSstableInfo}; +use task_manager::{TaskManager, UploadingTaskStatus}; use thiserror_ext::AsReport; use tokio::sync::oneshot; use tokio::task::JoinHandle; @@ -331,157 +334,6 @@ impl UploadingTask { } } -mod task_manager { - use super::*; - - #[derive(Debug)] - pub(super) enum UploadingTaskStatus { - Spilling, - Sync(HummockEpoch), - } - - #[derive(Debug)] - enum TaskStatus { - Uploading(UploadingTaskStatus), - Spilled(Arc), - } - - #[derive(Default, Debug)] - pub(super) struct TaskManager { - // newer task at the front - uploading_tasks: VecDeque, - task_status: HashMap, - } - - impl TaskManager { - fn add_task(&mut self, task: UploadingTask, status: UploadingTaskStatus) { - self.task_status - .insert(task.task_id, TaskStatus::Uploading(status)); - self.uploading_tasks.push_front(task); - } - - #[expect(clippy::type_complexity)] - pub(super) fn poll_task_result( - &mut self, - cx: &mut Context<'_>, - _context: &UploaderContext, - ) -> Poll< - Option< - Result< - ( - UploadingTaskId, - UploadingTaskStatus, - Arc, - ), - ErrState, - >, - >, - > { - if let Some(task) = self.uploading_tasks.back_mut() { - let result = match self.task_status.get(&task.task_id).expect("should exist") { - TaskStatus::Uploading(UploadingTaskStatus::Spilling) => { - let sst = ready!(task.poll_ok_with_retry(cx)); - self.task_status - .insert(task.task_id, TaskStatus::Spilled(sst.clone())); - Ok((task.task_id, UploadingTaskStatus::Spilling, sst)) - } - TaskStatus::Uploading(UploadingTaskStatus::Sync(epoch)) => { - let epoch = *epoch; - let result = ready!(task.poll_result(cx)); - let _status = self.task_status.remove(&task.task_id); - result - .map(|sst| (task.task_id, UploadingTaskStatus::Sync(epoch), sst)) - .map_err(|e| ErrState { - failed_epoch: epoch, - reason: e.as_report().to_string(), - }) - } - TaskStatus::Spilled(_) => { - unreachable!("should be uploading task") - } - }; - - let _task = self.uploading_tasks.pop_back().expect("non-empty"); - Poll::Ready(Some(result)) - } else { - Poll::Ready(None) - } - } - - pub(super) fn abort(self) { - for task in self.uploading_tasks { - task.join_handle.abort(); - } - } - - pub(super) fn spill( - &mut self, - context: &UploaderContext, - imms: HashMap>, - ) -> (UploadingTaskId, usize) { - assert!(!imms.is_empty()); - let task = UploadingTask::new(imms, context); - context.stats.spill_task_counts_from_unsealed.inc(); - context - .stats - .spill_task_size_from_unsealed - .inc_by(task.task_info.task_size as u64); - info!("Spill data. Task: {}", task.get_task_info()); - let size = task.task_info.task_size; - let id = task.task_id; - self.add_task(task, UploadingTaskStatus::Spilling); - (id, size) - } - - pub(super) fn sync( - &mut self, - context: &UploaderContext, - epoch: HummockEpoch, - spilled_task: BTreeSet, - unflushed_payload: UploadTaskInput, - ) -> (HashSet, VecDeque>) { - let mut remaining_tasks = HashSet::new(); - let total_task_count = if unflushed_payload.is_empty() { - spilled_task.len() - } else { - let task = UploadingTask::new(unflushed_payload, context); - remaining_tasks.insert(task.task_id); - self.task_status.insert( - task.task_id, - TaskStatus::Uploading(UploadingTaskStatus::Sync(epoch)), - ); - self.add_task(task, UploadingTaskStatus::Sync(epoch)); - spilled_task.len() + 1 - }; - let mut uploaded = VecDeque::with_capacity(total_task_count); - - // iterate from small task id to large, which means from old data to new data - for task_id in spilled_task { - let status = self.task_status.remove(&task_id).expect("should exist"); - match status { - TaskStatus::Uploading(UploadingTaskStatus::Spilling) => { - self.task_status.insert( - task_id, - TaskStatus::Uploading(UploadingTaskStatus::Sync(epoch)), - ); - remaining_tasks.insert(task_id); - } - TaskStatus::Uploading(UploadingTaskStatus::Sync(_)) => { - unreachable!("cannot be synced again") - } - TaskStatus::Spilled(sst) => { - self.task_status.remove(&task_id); - uploaded.push_front(sst); - } - } - } - (remaining_tasks, uploaded) - } - } -} - -use task_manager::{TaskManager, UploadingTaskStatus}; - impl TableUnsyncData { fn add_table_watermarks( &mut self, diff --git a/src/storage/src/hummock/event_handler/uploader/task_manager.rs b/src/storage/src/hummock/event_handler/uploader/task_manager.rs new file mode 100644 index 0000000000000..2f19ffed53175 --- /dev/null +++ b/src/storage/src/hummock/event_handler/uploader/task_manager.rs @@ -0,0 +1,160 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use super::*; + +#[derive(Debug)] +pub(super) enum UploadingTaskStatus { + Spilling, + Sync(HummockEpoch), +} + +#[derive(Debug)] +enum TaskStatus { + Uploading(UploadingTaskStatus), + Spilled(Arc), +} + +#[derive(Default, Debug)] +pub(super) struct TaskManager { + // newer task at the front + uploading_tasks: VecDeque, + task_status: HashMap, +} + +impl TaskManager { + fn add_task(&mut self, task: UploadingTask, status: UploadingTaskStatus) { + self.task_status + .insert(task.task_id, TaskStatus::Uploading(status)); + self.uploading_tasks.push_front(task); + } + + #[expect(clippy::type_complexity)] + pub(super) fn poll_task_result( + &mut self, + cx: &mut Context<'_>, + _context: &UploaderContext, + ) -> Poll< + Option< + Result< + ( + UploadingTaskId, + UploadingTaskStatus, + Arc, + ), + ErrState, + >, + >, + > { + if let Some(task) = self.uploading_tasks.back_mut() { + let result = match self.task_status.get(&task.task_id).expect("should exist") { + TaskStatus::Uploading(UploadingTaskStatus::Spilling) => { + let sst = ready!(task.poll_ok_with_retry(cx)); + self.task_status + .insert(task.task_id, TaskStatus::Spilled(sst.clone())); + Ok((task.task_id, UploadingTaskStatus::Spilling, sst)) + } + TaskStatus::Uploading(UploadingTaskStatus::Sync(epoch)) => { + let epoch = *epoch; + let result = ready!(task.poll_result(cx)); + let _status = self.task_status.remove(&task.task_id); + result + .map(|sst| (task.task_id, UploadingTaskStatus::Sync(epoch), sst)) + .map_err(|e| ErrState { + failed_epoch: epoch, + reason: e.as_report().to_string(), + }) + } + TaskStatus::Spilled(_) => { + unreachable!("should be uploading task") + } + }; + + let _task = self.uploading_tasks.pop_back().expect("non-empty"); + Poll::Ready(Some(result)) + } else { + Poll::Ready(None) + } + } + + pub(super) fn abort(self) { + for task in self.uploading_tasks { + task.join_handle.abort(); + } + } + + pub(super) fn spill( + &mut self, + context: &UploaderContext, + imms: HashMap>, + ) -> (UploadingTaskId, usize) { + assert!(!imms.is_empty()); + let task = UploadingTask::new(imms, context); + context.stats.spill_task_counts_from_unsealed.inc(); + context + .stats + .spill_task_size_from_unsealed + .inc_by(task.task_info.task_size as u64); + info!("Spill data. Task: {}", task.get_task_info()); + let size = task.task_info.task_size; + let id = task.task_id; + self.add_task(task, UploadingTaskStatus::Spilling); + (id, size) + } + + pub(super) fn sync( + &mut self, + context: &UploaderContext, + epoch: HummockEpoch, + spilled_task: BTreeSet, + unflushed_payload: UploadTaskInput, + ) -> (HashSet, VecDeque>) { + let mut remaining_tasks = HashSet::new(); + let total_task_count = if unflushed_payload.is_empty() { + spilled_task.len() + } else { + let task = UploadingTask::new(unflushed_payload, context); + remaining_tasks.insert(task.task_id); + self.task_status.insert( + task.task_id, + TaskStatus::Uploading(UploadingTaskStatus::Sync(epoch)), + ); + self.add_task(task, UploadingTaskStatus::Sync(epoch)); + spilled_task.len() + 1 + }; + let mut uploaded = VecDeque::with_capacity(total_task_count); + + // iterate from small task id to large, which means from old data to new data + for task_id in spilled_task { + let status = self.task_status.remove(&task_id).expect("should exist"); + match status { + TaskStatus::Uploading(UploadingTaskStatus::Spilling) => { + self.task_status.insert( + task_id, + TaskStatus::Uploading(UploadingTaskStatus::Sync(epoch)), + ); + remaining_tasks.insert(task_id); + } + TaskStatus::Uploading(UploadingTaskStatus::Sync(_)) => { + unreachable!("cannot be synced again") + } + TaskStatus::Spilled(sst) => { + self.task_status.remove(&task_id); + uploaded.push_front(sst); + } + } + } + (remaining_tasks, uploaded) + } +} From 4136e88734a1c38e46e1c8bc7f71a678c1ff6895 Mon Sep 17 00:00:00 2001 From: William Wen Date: Thu, 27 Jun 2024 16:56:26 +0800 Subject: [PATCH 08/17] track spill table id --- .../src/hummock/event_handler/uploader/mod.rs | 172 ++++++++++++---- .../event_handler/uploader/task_manager.rs | 192 ++++++++++-------- 2 files changed, 242 insertions(+), 122 deletions(-) diff --git a/src/storage/src/hummock/event_handler/uploader/mod.rs b/src/storage/src/hummock/event_handler/uploader/mod.rs index 3c5a792075385..a35441dc5b3e0 100644 --- a/src/storage/src/hummock/event_handler/uploader/mod.rs +++ b/src/storage/src/hummock/event_handler/uploader/mod.rs @@ -690,6 +690,8 @@ struct UnsyncData { table_data: HashMap, // An index as a mapping from instance id to its table id instance_table_id: HashMap, + // TODO: this is only used in spill to get existing epochs and can be removed + // when we support spill not based on epoch epochs: BTreeMap, } @@ -764,32 +766,38 @@ impl UnsyncData { } } - fn may_destroy_instance(&mut self, instance_id: LocalInstanceId) { + fn may_destroy_instance(&mut self, instance_id: LocalInstanceId) -> Option { if let Some(table_id) = self.instance_table_id.remove(&instance_id) { debug!(instance_id, "destroy instance"); let table_data = self.table_data.get_mut(&table_id).expect("should exist"); assert!(table_data.instance_data.remove(&instance_id).is_some()); if table_data.instance_data.is_empty() { - self.table_data.remove(&table_id); + Some(self.table_data.remove(&table_id).expect("should exist")) + } else { + None } + } else { + None } } +} +impl UploaderData { fn sync( &mut self, epoch: HummockEpoch, context: &UploaderContext, table_ids: HashSet, - task_manager: &mut TaskManager, ) -> SyncDataBuilder { // clean old epochs - let _epochs = take_before_epoch(&mut self.epochs, epoch); + let _epochs = take_before_epoch(&mut self.unsync_data.epochs, epoch); let mut all_table_watermarks = HashMap::new(); - let mut spill_tasks = BTreeSet::new(); + let mut spilling_tasks = HashSet::new(); + let mut spilled_tasks = BTreeSet::new(); let mut flush_payload = HashMap::new(); - for (table_id, table_data) in &mut self.table_data { + for (table_id, table_data) in &mut self.unsync_data.table_data { if !table_ids.contains(table_id) { table_data.assert_after_epoch(epoch); continue; @@ -809,20 +817,50 @@ impl UnsyncData { ); } for task_id in task_ids { - spill_tasks.insert(task_id); + if self.spilled_data.contains_key(&task_id) { + spilled_tasks.insert(task_id); + } else { + spilling_tasks.insert(task_id); + } } } - let (remaining_uploading_tasks, uploaded) = - task_manager.sync(context, epoch, spill_tasks, flush_payload); + if let Some(extra_flush_task_id) = self.task_manager.sync( + context, + epoch, + flush_payload, + spilling_tasks.iter().cloned(), + &table_ids, + ) { + spilling_tasks.insert(extra_flush_task_id); + } + + // iter from large task_id to small one so that newer data at the front + let uploaded = spilled_tasks + .iter() + .rev() + .map(|task_id| { + let (sst, spill_table_ids) = + self.spilled_data.remove(task_id).expect("should exist"); + assert!( + spill_table_ids.is_subset(&table_ids), + "spill_table_ids: {spill_table_ids:?}, table_ids: {table_ids:?}" + ); + sst + }) + .collect(); + + self.check_spill_task_consistency(); SyncDataBuilder { table_watermarks: all_table_watermarks, - remaining_uploading_tasks, + remaining_uploading_tasks: spilling_tasks, uploaded, } } +} +impl UnsyncData { fn ack_flushed(&mut self, sstable_info: &StagingSstableInfo) { for (instance_id, imm_ids) in sstable_info.imm_ids() { if let Some(instance_data) = self.instance_data(*instance_id) { @@ -884,6 +922,7 @@ struct UploaderData { syncing_data: BTreeMap, task_manager: TaskManager, + spilled_data: HashMap, HashSet)>, } impl UploaderData { @@ -1033,9 +1072,7 @@ impl HummockUploader { self.max_syncing_epoch = epoch; - let sync_data = - data.unsync_data - .sync(epoch, &self.context, table_ids, &mut data.task_manager); + let sync_data = data.sync(epoch, &self.context, table_ids); let SyncDataBuilder { table_watermarks, @@ -1148,11 +1185,13 @@ impl HummockUploader { } } if !payload.is_empty() { - let (task_id, task_size) = data.task_manager.spill(&self.context, payload); + let (task_id, task_size, spilled_table_ids) = + data.task_manager + .spill(&self.context, spilled_table_ids, payload); for table_id in spilled_table_ids { data.unsync_data .table_data - .get_mut(&table_id) + .get_mut(table_id) .expect("should exist") .spill_tasks .entry(*epoch) @@ -1162,6 +1201,7 @@ impl HummockUploader { curr_batch_flush_size += task_size; } } + data.check_spill_task_consistency(); curr_batch_flush_size > 0 } else { false @@ -1188,7 +1228,27 @@ impl HummockUploader { let UploaderState::Working(data) = &mut self.state else { return; }; - data.unsync_data.may_destroy_instance(instance_id); + if let Some(removed_table_data) = data.unsync_data.may_destroy_instance(instance_id) { + data.task_manager.remove_table_spill_tasks( + removed_table_data.table_id, + removed_table_data + .spill_tasks + .into_values() + .flat_map(|task_ids| task_ids.into_iter()) + .filter(|task_id| { + if let Some((_, table_ids)) = data.spilled_data.get_mut(task_id) { + assert!(table_ids.remove(&removed_table_data.table_id)); + if table_ids.is_empty() { + data.spilled_data.remove(task_id); + } + false + } else { + true + } + }), + ) + } + data.check_spill_task_consistency(); } } @@ -1225,6 +1285,38 @@ impl UploaderData { ) } } + + fn check_spill_task_consistency(&self) { + #[cfg(debug_assertions)] + { + let mut spill_task_table_id_from_data: HashMap<_, HashSet<_>> = HashMap::new(); + for table_data in self.unsync_data.table_data.values() { + for task_id in table_data + .spill_tasks + .iter() + .flat_map(|(_, tasks)| tasks.iter()) + { + assert!(spill_task_table_id_from_data + .entry(*task_id) + .or_default() + .insert(table_data.table_id)); + } + } + let mut spill_task_table_id_from_manager: HashMap<_, HashSet<_>> = HashMap::new(); + for (task_id, (_, table_ids)) in &self.spilled_data { + spill_task_table_id_from_manager.insert(*task_id, table_ids.clone()); + } + for (task_id, table_ids) in self.task_manager.spilling_task() { + assert!(spill_task_table_id_from_manager + .insert(task_id, table_ids.clone()) + .is_none()); + } + assert_eq!( + spill_task_table_id_from_data, + spill_task_table_id_from_manager + ); + } + } } impl HummockUploader { @@ -1236,27 +1328,33 @@ impl HummockUploader { return Poll::Pending; }; - if let Some(result) = ready!(data.task_manager.poll_task_result(cx, &self.context)) { + if let Some((task_id, status, result)) = ready!(data.task_manager.poll_task_result(cx)) + { match result { - Ok((task_id, status, sst)) => { + Ok(sst) => { data.unsync_data.ack_flushed(&sst); - - if let UploadingTaskStatus::Sync(sync_epoch) = status { - let syncing_data = data - .syncing_data - .get_mut(&sync_epoch) - .expect("should exist"); - syncing_data.uploaded.push_front(sst.clone()); - assert!(syncing_data.remaining_uploading_tasks.remove(&task_id)); - data.may_notify_sync_task(&self.context, |new_synced_epoch| { - Self::set_max_synced_epoch( - &mut self.max_synced_epoch, - self.max_syncing_epoch, - new_synced_epoch, - ) - }); + match status { + UploadingTaskStatus::Sync(sync_epoch) => { + let syncing_data = data + .syncing_data + .get_mut(&sync_epoch) + .expect("should exist"); + syncing_data.uploaded.push_front(sst.clone()); + assert!(syncing_data.remaining_uploading_tasks.remove(&task_id)); + data.may_notify_sync_task(&self.context, |new_synced_epoch| { + Self::set_max_synced_epoch( + &mut self.max_synced_epoch, + self.max_syncing_epoch, + new_synced_epoch, + ) + }); + } + UploadingTaskStatus::Spilling(table_ids) => { + data.spilled_data.insert(task_id, (sst.clone(), table_ids)); + } } - return Poll::Ready(sst); + data.check_spill_task_consistency(); + Poll::Ready(sst) } Err(e) => { let failed_epoch = e.failed_epoch; @@ -1271,12 +1369,12 @@ impl HummockUploader { failed_epoch )) }); - return Poll::Pending; + Poll::Pending } } + } else { + Poll::Pending } - - Poll::Pending }) } } diff --git a/src/storage/src/hummock/event_handler/uploader/task_manager.rs b/src/storage/src/hummock/event_handler/uploader/task_manager.rs index 2f19ffed53175..0c75f549407d0 100644 --- a/src/storage/src/hummock/event_handler/uploader/task_manager.rs +++ b/src/storage/src/hummock/event_handler/uploader/task_manager.rs @@ -16,89 +16,93 @@ use super::*; #[derive(Debug)] pub(super) enum UploadingTaskStatus { - Spilling, + Spilling(HashSet), Sync(HummockEpoch), } #[derive(Debug)] -enum TaskStatus { - Uploading(UploadingTaskStatus), - Spilled(Arc), +struct TaskEntry { + task: UploadingTask, + status: UploadingTaskStatus, } #[derive(Default, Debug)] pub(super) struct TaskManager { + tasks: HashMap, // newer task at the front - uploading_tasks: VecDeque, - task_status: HashMap, + task_order: VecDeque, } impl TaskManager { - fn add_task(&mut self, task: UploadingTask, status: UploadingTaskStatus) { - self.task_status - .insert(task.task_id, TaskStatus::Uploading(status)); - self.uploading_tasks.push_front(task); + fn add_task( + &mut self, + task: UploadingTask, + status: UploadingTaskStatus, + ) -> &UploadingTaskStatus { + let task_id = task.task_id; + self.task_order.push_front(task.task_id); + self.tasks.insert(task.task_id, TaskEntry { task, status }); + &self.tasks.get(&task_id).expect("should exist").status + } + + fn poll_task( + &mut self, + cx: &mut Context<'_>, + task_id: UploadingTaskId, + ) -> Poll, ErrState>> { + let entry = self.tasks.get_mut(&task_id).expect("should exist"); + let result = match &entry.status { + UploadingTaskStatus::Spilling(_) => { + let sst = ready!(entry.task.poll_ok_with_retry(cx)); + Ok(sst) + } + UploadingTaskStatus::Sync(epoch) => { + let epoch = *epoch; + let result = ready!(entry.task.poll_result(cx)); + result.map_err(|e| ErrState { + failed_epoch: epoch, + reason: e.as_report().to_string(), + }) + } + }; + Poll::Ready(result) } #[expect(clippy::type_complexity)] pub(super) fn poll_task_result( &mut self, cx: &mut Context<'_>, - _context: &UploaderContext, ) -> Poll< - Option< - Result< - ( - UploadingTaskId, - UploadingTaskStatus, - Arc, - ), - ErrState, - >, - >, + Option<( + UploadingTaskId, + UploadingTaskStatus, + Result, ErrState>, + )>, > { - if let Some(task) = self.uploading_tasks.back_mut() { - let result = match self.task_status.get(&task.task_id).expect("should exist") { - TaskStatus::Uploading(UploadingTaskStatus::Spilling) => { - let sst = ready!(task.poll_ok_with_retry(cx)); - self.task_status - .insert(task.task_id, TaskStatus::Spilled(sst.clone())); - Ok((task.task_id, UploadingTaskStatus::Spilling, sst)) - } - TaskStatus::Uploading(UploadingTaskStatus::Sync(epoch)) => { - let epoch = *epoch; - let result = ready!(task.poll_result(cx)); - let _status = self.task_status.remove(&task.task_id); - result - .map(|sst| (task.task_id, UploadingTaskStatus::Sync(epoch), sst)) - .map_err(|e| ErrState { - failed_epoch: epoch, - reason: e.as_report().to_string(), - }) - } - TaskStatus::Spilled(_) => { - unreachable!("should be uploading task") - } - }; + if let Some(task_id) = self.task_order.back() { + let task_id = *task_id; + let result = ready!(self.poll_task(cx, task_id)); + self.task_order.pop_back(); + let entry = self.tasks.remove(&task_id).expect("should exist"); - let _task = self.uploading_tasks.pop_back().expect("non-empty"); - Poll::Ready(Some(result)) + Poll::Ready(Some((task_id, entry.status, result))) } else { Poll::Ready(None) } } pub(super) fn abort(self) { - for task in self.uploading_tasks { - task.join_handle.abort(); + for task in self.tasks.into_values() { + task.task.join_handle.abort(); } } pub(super) fn spill( &mut self, context: &UploaderContext, + table_ids: HashSet, imms: HashMap>, - ) -> (UploadingTaskId, usize) { + ) -> (UploadingTaskId, usize, &HashSet) { assert!(!imms.is_empty()); let task = UploadingTask::new(imms, context); context.stats.spill_task_counts_from_unsealed.inc(); @@ -109,52 +113,70 @@ impl TaskManager { info!("Spill data. Task: {}", task.get_task_info()); let size = task.task_info.task_size; let id = task.task_id; - self.add_task(task, UploadingTaskStatus::Spilling); - (id, size) + let status = self.add_task(task, UploadingTaskStatus::Spilling(table_ids)); + ( + id, + size, + must_match!(status, UploadingTaskStatus::Spilling(table_ids) => table_ids), + ) + } + + pub(super) fn remove_table_spill_tasks( + &mut self, + table_id: TableId, + task_ids: impl Iterator, + ) { + for task_id in task_ids { + let entry = self.tasks.get_mut(&task_id).expect("should exist"); + let empty = must_match!(&mut entry.status, UploadingTaskStatus::Spilling(table_ids) => { + assert!(table_ids.remove(&table_id)); + table_ids.is_empty() + }); + if empty { + let task = self.tasks.remove(&task_id).expect("should exist").task; + task.join_handle.abort(); + } + } } pub(super) fn sync( &mut self, context: &UploaderContext, epoch: HummockEpoch, - spilled_task: BTreeSet, unflushed_payload: UploadTaskInput, - ) -> (HashSet, VecDeque>) { - let mut remaining_tasks = HashSet::new(); - let total_task_count = if unflushed_payload.is_empty() { - spilled_task.len() + spill_task_ids: impl Iterator, + sync_table_ids: &HashSet, + ) -> Option { + let task = if unflushed_payload.is_empty() { + None } else { - let task = UploadingTask::new(unflushed_payload, context); - remaining_tasks.insert(task.task_id); - self.task_status.insert( - task.task_id, - TaskStatus::Uploading(UploadingTaskStatus::Sync(epoch)), - ); - self.add_task(task, UploadingTaskStatus::Sync(epoch)); - spilled_task.len() + 1 + Some(UploadingTask::new(unflushed_payload, context)) }; - let mut uploaded = VecDeque::with_capacity(total_task_count); - // iterate from small task id to large, which means from old data to new data - for task_id in spilled_task { - let status = self.task_status.remove(&task_id).expect("should exist"); - match status { - TaskStatus::Uploading(UploadingTaskStatus::Spilling) => { - self.task_status.insert( - task_id, - TaskStatus::Uploading(UploadingTaskStatus::Sync(epoch)), - ); - remaining_tasks.insert(task_id); - } - TaskStatus::Uploading(UploadingTaskStatus::Sync(_)) => { - unreachable!("cannot be synced again") - } - TaskStatus::Spilled(sst) => { - self.task_status.remove(&task_id); - uploaded.push_front(sst); - } - } + for task_id in spill_task_ids { + let entry = self.tasks.get_mut(&task_id).expect("should exist"); + must_match!(&entry.status, UploadingTaskStatus::Spilling(table_ids) => { + assert!(table_ids.is_subset(sync_table_ids), "spill table_ids: {table_ids:?}, sync_table_ids: {sync_table_ids:?}"); + }); + entry.status = UploadingTaskStatus::Sync(epoch); } - (remaining_tasks, uploaded) + + task.map(|task| { + let id = task.task_id; + self.add_task(task, UploadingTaskStatus::Sync(epoch)); + id + }) + } + + #[cfg(debug_assertions)] + pub(super) fn spilling_task( + &self, + ) -> impl Iterator)> { + self.tasks + .iter() + .filter_map(|(task_id, entry)| match &entry.status { + UploadingTaskStatus::Spilling(table_ids) => Some((*task_id, table_ids)), + UploadingTaskStatus::Sync(_) => None, + }) } } From 5aef22672da575f063043c13e05a07ea8a9bf819 Mon Sep 17 00:00:00 2001 From: William Wen Date: Fri, 28 Jun 2024 17:08:13 +0800 Subject: [PATCH 09/17] add check and refactor --- .../src/hummock/event_handler/uploader/mod.rs | 123 ++++++++++-------- .../event_handler/uploader/task_manager.rs | 29 ++--- 2 files changed, 77 insertions(+), 75 deletions(-) diff --git a/src/storage/src/hummock/event_handler/uploader/mod.rs b/src/storage/src/hummock/event_handler/uploader/mod.rs index a35441dc5b3e0..2cf5c0959917d 100644 --- a/src/storage/src/hummock/event_handler/uploader/mod.rs +++ b/src/storage/src/hummock/event_handler/uploader/mod.rs @@ -388,14 +388,7 @@ impl TableUnsyncData { } } -#[derive(Default)] -struct SyncDataBuilder { - table_watermarks: HashMap, - remaining_uploading_tasks: HashSet, - uploaded: VecDeque>, -} - -impl SyncDataBuilder { +impl UploaderData { fn add_table_watermarks( all_table_watermarks: &mut HashMap, table_id: TableId, @@ -788,7 +781,8 @@ impl UploaderData { epoch: HummockEpoch, context: &UploaderContext, table_ids: HashSet, - ) -> SyncDataBuilder { + sync_result_sender: oneshot::Sender>, + ) { // clean old epochs let _epochs = take_before_epoch(&mut self.unsync_data.epochs, epoch); @@ -809,7 +803,7 @@ impl UploaderData { } } if let Some((direction, watermarks)) = table_watermarks { - SyncDataBuilder::add_table_watermarks( + Self::add_table_watermarks( &mut all_table_watermarks, *table_id, direction, @@ -825,9 +819,12 @@ impl UploaderData { } } + static NEXT_SYNC_ID: LazyLock = LazyLock::new(|| AtomicUsize::new(0)); + let sync_id = SyncId(NEXT_SYNC_ID.fetch_add(1, Relaxed)); + if let Some(extra_flush_task_id) = self.task_manager.sync( context, - epoch, + sync_id, flush_payload, spilling_tasks.iter().cloned(), &table_ids, @@ -850,13 +847,18 @@ impl UploaderData { }) .collect(); - self.check_spill_task_consistency(); + self.syncing_data.insert( + sync_id, + SyncingData { + sync_epoch: epoch, + remaining_uploading_tasks: spilling_tasks, + uploaded, + table_watermarks: all_table_watermarks, + sync_result_sender, + }, + ); - SyncDataBuilder { - table_watermarks: all_table_watermarks, - remaining_uploading_tasks: spilling_tasks, - uploaded, - } + self.check_upload_task_consistency(); } } @@ -912,6 +914,9 @@ impl UploaderContext { } } +#[derive(PartialEq, Eq, Hash, PartialOrd, Ord, Copy, Clone, Debug)] +struct SyncId(usize); + #[derive(Default)] struct UploaderData { unsync_data: UnsyncData, @@ -919,7 +924,7 @@ struct UploaderData { /// Data that has started syncing but not synced yet. `epoch` satisfies /// `max_synced_epoch < epoch <= max_syncing_epoch`. /// Newer epoch at the front - syncing_data: BTreeMap, + syncing_data: BTreeMap, task_manager: TaskManager, spilled_data: HashMap, HashSet)>, @@ -1072,28 +1077,7 @@ impl HummockUploader { self.max_syncing_epoch = epoch; - let sync_data = data.sync(epoch, &self.context, table_ids); - - let SyncDataBuilder { - table_watermarks, - remaining_uploading_tasks, - uploaded, - } = sync_data; - - if let Some((prev_max_epoch, _)) = data.syncing_data.last_key_value() { - assert_gt!(epoch, *prev_max_epoch); - } - - data.syncing_data.insert( - epoch, - SyncingData { - sync_epoch: epoch, - remaining_uploading_tasks, - uploaded, - table_watermarks, - sync_result_sender, - }, - ); + data.sync(epoch, &self.context, table_ids, sync_result_sender); data.may_notify_sync_task(&self.context, |new_synced_epoch| { Self::set_max_synced_epoch( @@ -1201,7 +1185,7 @@ impl HummockUploader { curr_batch_flush_size += task_size; } } - data.check_spill_task_consistency(); + data.check_upload_task_consistency(); curr_batch_flush_size > 0 } else { false @@ -1248,7 +1232,7 @@ impl HummockUploader { }), ) } - data.check_spill_task_consistency(); + data.check_upload_task_consistency(); } } @@ -1286,7 +1270,7 @@ impl UploaderData { } } - fn check_spill_task_consistency(&self) { + fn check_upload_task_consistency(&self) { #[cfg(debug_assertions)] { let mut spill_task_table_id_from_data: HashMap<_, HashSet<_>> = HashMap::new(); @@ -1302,19 +1286,43 @@ impl UploaderData { .insert(table_data.table_id)); } } + let syncing_task_id_from_data: HashMap<_, HashSet<_>> = self + .syncing_data + .iter() + .filter_map(|(sync_id, data)| { + if data.remaining_uploading_tasks.is_empty() { + None + } else { + Some((*sync_id, data.remaining_uploading_tasks.clone())) + } + }) + .collect(); + let mut spill_task_table_id_from_manager: HashMap<_, HashSet<_>> = HashMap::new(); for (task_id, (_, table_ids)) in &self.spilled_data { spill_task_table_id_from_manager.insert(*task_id, table_ids.clone()); } - for (task_id, table_ids) in self.task_manager.spilling_task() { - assert!(spill_task_table_id_from_manager - .insert(task_id, table_ids.clone()) - .is_none()); + let mut syncing_task_from_manager: HashMap<_, HashSet<_>> = HashMap::new(); + for (task_id, status) in self.task_manager.tasks() { + match status { + UploadingTaskStatus::Spilling(table_ids) => { + assert!(spill_task_table_id_from_manager + .insert(task_id, table_ids.clone()) + .is_none()); + } + UploadingTaskStatus::Sync(sync_id) => { + assert!(syncing_task_from_manager + .entry(*sync_id) + .or_default() + .insert(task_id)); + } + } } assert_eq!( spill_task_table_id_from_data, spill_task_table_id_from_manager ); + assert_eq!(syncing_task_id_from_data, syncing_task_from_manager); } } } @@ -1334,11 +1342,9 @@ impl HummockUploader { Ok(sst) => { data.unsync_data.ack_flushed(&sst); match status { - UploadingTaskStatus::Sync(sync_epoch) => { - let syncing_data = data - .syncing_data - .get_mut(&sync_epoch) - .expect("should exist"); + UploadingTaskStatus::Sync(sync_id) => { + let syncing_data = + data.syncing_data.get_mut(&sync_id).expect("should exist"); syncing_data.uploaded.push_front(sst.clone()); assert!(syncing_data.remaining_uploading_tasks.remove(&task_id)); data.may_notify_sync_task(&self.context, |new_synced_epoch| { @@ -1353,14 +1359,19 @@ impl HummockUploader { data.spilled_data.insert(task_id, (sst.clone(), table_ids)); } } - data.check_spill_task_consistency(); + data.check_upload_task_consistency(); Poll::Ready(sst) } - Err(e) => { - let failed_epoch = e.failed_epoch; + Err((sync_id, e)) => { + let syncing_data = + data.syncing_data.remove(&sync_id).expect("should exist"); + let failed_epoch = syncing_data.sync_epoch; let data = must_match!(replace( &mut self.state, - UploaderState::Err(e), + UploaderState::Err(ErrState { + failed_epoch, + reason: e.as_report().to_string(), + }), ), UploaderState::Working(data) => data); data.abort(|| { diff --git a/src/storage/src/hummock/event_handler/uploader/task_manager.rs b/src/storage/src/hummock/event_handler/uploader/task_manager.rs index 0c75f549407d0..fd3ec142fbddf 100644 --- a/src/storage/src/hummock/event_handler/uploader/task_manager.rs +++ b/src/storage/src/hummock/event_handler/uploader/task_manager.rs @@ -17,7 +17,7 @@ use super::*; #[derive(Debug)] pub(super) enum UploadingTaskStatus { Spilling(HashSet), - Sync(HummockEpoch), + Sync(SyncId), } #[derive(Debug)] @@ -49,20 +49,16 @@ impl TaskManager { &mut self, cx: &mut Context<'_>, task_id: UploadingTaskId, - ) -> Poll, ErrState>> { + ) -> Poll, (SyncId, HummockError)>> { let entry = self.tasks.get_mut(&task_id).expect("should exist"); let result = match &entry.status { UploadingTaskStatus::Spilling(_) => { let sst = ready!(entry.task.poll_ok_with_retry(cx)); Ok(sst) } - UploadingTaskStatus::Sync(epoch) => { - let epoch = *epoch; + UploadingTaskStatus::Sync(sync_id) => { let result = ready!(entry.task.poll_result(cx)); - result.map_err(|e| ErrState { - failed_epoch: epoch, - reason: e.as_report().to_string(), - }) + result.map_err(|e| (*sync_id, e)) } }; Poll::Ready(result) @@ -76,7 +72,7 @@ impl TaskManager { Option<( UploadingTaskId, UploadingTaskStatus, - Result, ErrState>, + Result, (SyncId, HummockError)>, )>, > { if let Some(task_id) = self.task_order.back() { @@ -142,7 +138,7 @@ impl TaskManager { pub(super) fn sync( &mut self, context: &UploaderContext, - epoch: HummockEpoch, + sync_id: SyncId, unflushed_payload: UploadTaskInput, spill_task_ids: impl Iterator, sync_table_ids: &HashSet, @@ -158,25 +154,20 @@ impl TaskManager { must_match!(&entry.status, UploadingTaskStatus::Spilling(table_ids) => { assert!(table_ids.is_subset(sync_table_ids), "spill table_ids: {table_ids:?}, sync_table_ids: {sync_table_ids:?}"); }); - entry.status = UploadingTaskStatus::Sync(epoch); + entry.status = UploadingTaskStatus::Sync(sync_id); } task.map(|task| { let id = task.task_id; - self.add_task(task, UploadingTaskStatus::Sync(epoch)); + self.add_task(task, UploadingTaskStatus::Sync(sync_id)); id }) } #[cfg(debug_assertions)] - pub(super) fn spilling_task( - &self, - ) -> impl Iterator)> { + pub(super) fn tasks(&self) -> impl Iterator { self.tasks .iter() - .filter_map(|(task_id, entry)| match &entry.status { - UploadingTaskStatus::Spilling(table_ids) => Some((*task_id, table_ids)), - UploadingTaskStatus::Sync(_) => None, - }) + .map(|(task_id, entry)| (*task_id, &entry.status)) } } From e51e18647589503514ce002e10dc602b2e9a5c1f Mon Sep 17 00:00:00 2001 From: William Wen Date: Fri, 28 Jun 2024 17:58:35 +0800 Subject: [PATCH 10/17] temp save --- .../event_handler/hummock_event_handler.rs | 5 +- .../src/hummock/event_handler/uploader/mod.rs | 176 ++++++++---------- 2 files changed, 83 insertions(+), 98 deletions(-) diff --git a/src/storage/src/hummock/event_handler/hummock_event_handler.rs b/src/storage/src/hummock/event_handler/hummock_event_handler.rs index cbb0044e2f798..68ff9a4a0a901 100644 --- a/src/storage/src/hummock/event_handler/hummock_event_handler.rs +++ b/src/storage/src/hummock/event_handler/hummock_event_handler.rs @@ -469,9 +469,9 @@ impl HummockEventHandler { table_ids: HashSet, ) { debug!( - "awaiting for epoch to be synced: {}, max_synced_epoch: {}", new_sync_epoch, - self.uploader.max_synced_epoch() + ?table_ids, + "awaiting for epoch to be synced", ); self.uploader .start_sync_epoch(new_sync_epoch, sync_result_sender, table_ids); @@ -481,7 +481,6 @@ impl HummockEventHandler { info!( prev_epoch, max_committed_epoch = self.uploader.max_committed_epoch(), - max_synced_epoch = self.uploader.max_synced_epoch(), "handle clear event" ); diff --git a/src/storage/src/hummock/event_handler/uploader/mod.rs b/src/storage/src/hummock/event_handler/uploader/mod.rs index 2cf5c0959917d..97f81bcf413ce 100644 --- a/src/storage/src/hummock/event_handler/uploader/mod.rs +++ b/src/storage/src/hummock/event_handler/uploader/mod.rs @@ -618,15 +618,20 @@ struct TableUnsyncData { BTreeMap, BitmapBuilder)>, )>, spill_tasks: BTreeMap>, + // newer epoch at the front + syncing_epochs: VecDeque, + max_synced_epoch: Option, } impl TableUnsyncData { - fn new(table_id: TableId) -> Self { + fn new(table_id: TableId, committed_epoch: Option) -> Self { Self { table_id, instance_data: Default::default(), table_watermarks: None, spill_tasks: Default::default(), + syncing_epochs: Default::default(), + max_synced_epoch: committed_epoch, } } @@ -641,6 +646,10 @@ impl TableUnsyncData { )>, impl Iterator, ) { + if let Some(prev_epoch) = self.max_sync_epoch() { + assert_gt!(epoch, prev_epoch) + } + self.syncing_epochs.push_front(epoch); ( self.instance_data .iter_mut() @@ -662,7 +671,32 @@ impl TableUnsyncData { ) } + fn ack_synced(&mut self, sync_epoch: HummockEpoch) { + let min_sync_epoch = self.syncing_epochs.pop_back().expect("should exist"); + assert_eq!(sync_epoch, min_sync_epoch); + self.max_synced_epoch = Some(sync_epoch); + } + + fn ack_committed(&mut self, committed_epoch: HummockEpoch) { + let synced_epoch_advanced = { + if let Some(max_synced_epoch) = self.max_synced_epoch + && max_synced_epoch >= committed_epoch + { + false + } else { + true + } + }; + if synced_epoch_advanced { + self.max_synced_epoch = Some(committed_epoch); + self.assert_after_epoch(committed_epoch); + } + } + fn assert_after_epoch(&self, epoch: HummockEpoch) { + if let Some(min_syncing_epoch) = self.syncing_epochs.back() { + assert_gt!(*min_syncing_epoch, epoch); + } self.instance_data .values() .for_each(|instance_data| instance_data.assert_after_epoch(epoch)); @@ -672,6 +706,13 @@ impl TableUnsyncData { assert_gt!(*oldest_epoch, epoch); } } + + fn max_sync_epoch(&self) -> Option { + self.syncing_epochs + .front() + .cloned() + .or_else(|| self.max_synced_epoch) + } } #[derive(Default)] @@ -694,15 +735,27 @@ impl UnsyncData { table_id: TableId, instance_id: LocalInstanceId, init_epoch: HummockEpoch, + context: &UploaderContext, ) { debug!( table_id = table_id.table_id, instance_id, init_epoch, "init epoch" ); - let table_data = self - .table_data - .entry(table_id) - .or_insert_with(|| TableUnsyncData::new(table_id)); + let table_data = self.table_data.entry(table_id).or_insert_with(|| { + TableUnsyncData::new( + table_id, + context + .pinned_version + .version() + .state_table_info + .info() + .get(&table_id) + .map(|info| info.committed_epoch), + ) + }); + if let Some(max_prev_epoch) = table_data.max_sync_epoch() { + assert_gt!(init_epoch, max_prev_epoch); + } assert!(table_data .instance_data .insert( @@ -851,6 +904,7 @@ impl UploaderData { sync_id, SyncingData { sync_epoch: epoch, + table_ids, remaining_uploading_tasks: spilling_tasks, uploaded, table_watermarks: all_table_watermarks, @@ -875,6 +929,7 @@ impl UnsyncData { struct SyncingData { sync_epoch: HummockEpoch, + table_ids: HashSet, remaining_uploading_tasks: HashSet, // newer data at the front uploaded: VecDeque>, @@ -954,7 +1009,7 @@ enum UploaderState { /// Data have 3 sequential stages: unsync (inside each local instance, data can be unsealed, sealed), syncing, synced. /// /// The 3 stages are divided by 2 marginal epochs: `max_syncing_epoch`, -/// `max_synced_epoch`. Epochs satisfy the following inequality. +/// `max_synced_epoch` in each `TableUnSyncData`. Epochs satisfy the following inequality. /// /// (epochs of `synced_data`) <= `max_synced_epoch` < (epochs of `syncing_data`) <= /// `max_syncing_epoch` < (epochs of `unsync_data`) @@ -962,11 +1017,6 @@ enum UploaderState { /// Data are mostly stored in `VecDeque`, and the order stored in the `VecDeque` indicates the data /// order. Data at the front represents ***newer*** data. pub struct HummockUploader { - /// The maximum epoch that has started syncing - max_syncing_epoch: HummockEpoch, - /// The maximum epoch that has been synced - max_synced_epoch: HummockEpoch, - state: UploaderState, context: UploaderContext, @@ -980,10 +1030,7 @@ impl HummockUploader { buffer_tracker: BufferTracker, config: &StorageOpts, ) -> Self { - let initial_epoch = pinned_version.version().max_committed_epoch; Self { - max_syncing_epoch: initial_epoch, - max_synced_epoch: initial_epoch, state: UploaderState::Working(UploaderData::default()), context: UploaderContext::new( pinned_version, @@ -999,10 +1046,6 @@ impl HummockUploader { &self.context.buffer_tracker } - pub(super) fn max_synced_epoch(&self) -> HummockEpoch { - self.max_synced_epoch - } - pub(super) fn max_committed_epoch(&self) -> HummockEpoch { self.context.pinned_version.max_committed_epoch() } @@ -1028,9 +1071,8 @@ impl HummockUploader { let UploaderState::Working(data) = &mut self.state else { return; }; - assert_gt!(init_epoch, self.max_syncing_epoch); data.unsync_data - .init_instance(table_id, instance_id, init_epoch); + .init_instance(table_id, instance_id, init_epoch, &self.context); } pub(super) fn local_seal_epoch( @@ -1042,7 +1084,6 @@ impl HummockUploader { let UploaderState::Working(data) = &mut self.state else { return; }; - assert_gt!(next_epoch, self.max_syncing_epoch); data.unsync_data .local_seal_epoch(instance_id, next_epoch, opts); } @@ -1067,25 +1108,11 @@ impl HummockUploader { return; } }; - debug!("start sync epoch: {}", epoch); - assert!( - epoch > self.max_syncing_epoch, - "the epoch {} has started syncing already: {}", - epoch, - self.max_syncing_epoch - ); - - self.max_syncing_epoch = epoch; + debug!(epoch, ?table_ids, "start sync epoch"); data.sync(epoch, &self.context, table_ids, sync_result_sender); - data.may_notify_sync_task(&self.context, |new_synced_epoch| { - Self::set_max_synced_epoch( - &mut self.max_synced_epoch, - self.max_syncing_epoch, - new_synced_epoch, - ) - }); + data.may_notify_sync_task(&self.context); self.context .stats @@ -1093,53 +1120,20 @@ impl HummockUploader { .set(data.syncing_data.len() as _); } - fn set_max_synced_epoch( - max_synced_epoch: &mut HummockEpoch, - max_syncing_epoch: HummockEpoch, - epoch: HummockEpoch, - ) { - assert!( - epoch <= max_syncing_epoch, - "epoch {} that has been synced has not started syncing yet. previous max syncing epoch {}", - epoch, - max_syncing_epoch - ); - assert!( - epoch > *max_synced_epoch, - "epoch {} has been synced. previous max synced epoch: {}", - epoch, - max_synced_epoch - ); - *max_synced_epoch = epoch; - } - pub(crate) fn update_pinned_version(&mut self, pinned_version: PinnedVersion) { assert_ge!( pinned_version.max_committed_epoch(), self.context.pinned_version.max_committed_epoch() ); - let max_committed_epoch = pinned_version.max_committed_epoch(); - self.context.pinned_version = pinned_version; - if self.max_synced_epoch < max_committed_epoch { - self.max_synced_epoch = max_committed_epoch; - } - if self.max_syncing_epoch < max_committed_epoch { - self.max_syncing_epoch = max_committed_epoch; - if let UploaderState::Working(data) = &self.state { - for instance_data in data - .unsync_data - .table_data - .values() - .flat_map(|data| data.instance_data.values()) - { - if let Some(oldest_epoch) = instance_data.sealed_data.back() { - assert_gt!(oldest_epoch.epoch, max_committed_epoch); - } else if let Some(current_epoch) = &instance_data.current_epoch_data { - assert_gt!(current_epoch.epoch, max_committed_epoch); - } + if let UploaderState::Working(data) = &mut self.state { + // TODO: may only `ack_committed` on table whose `committed_epoch` is changed. + for (table_id, info) in pinned_version.version().state_table_info.info() { + if let Some(table_data) = data.unsync_data.table_data.get_mut(table_id) { + table_data.ack_committed(info.committed_epoch); } } } + self.context.pinned_version = pinned_version; } pub(crate) fn may_flush(&mut self) -> bool { @@ -1193,16 +1187,11 @@ impl HummockUploader { } pub(crate) fn clear(&mut self) { - let max_committed_epoch = self.context.pinned_version.max_committed_epoch(); - self.max_synced_epoch = max_committed_epoch; - self.max_syncing_epoch = max_committed_epoch; if let UploaderState::Working(data) = replace( &mut self.state, UploaderState::Working(UploaderData::default()), ) { - data.abort(|| { - HummockError::other(format!("uploader is reset to {}", max_committed_epoch)) - }); + data.abort(|| HummockError::other("uploader is reset")); } self.context.stats.uploader_syncing_epoch_count.set(0); @@ -1239,17 +1228,14 @@ impl HummockUploader { impl UploaderData { /// Poll the syncing task of the syncing data of the oldest epoch. Return `Poll::Ready(None)` if /// there is no syncing data. - fn may_notify_sync_task( - &mut self, - context: &UploaderContext, - mut set_max_synced_epoch: impl FnMut(u64), - ) { + fn may_notify_sync_task(&mut self, context: &UploaderContext) { while let Some((_, syncing_data)) = self.syncing_data.first_key_value() && syncing_data.remaining_uploading_tasks.is_empty() { let (_, syncing_data) = self.syncing_data.pop_first().expect("non-empty"); let SyncingData { sync_epoch, + table_ids, remaining_uploading_tasks: _, uploaded, table_watermarks, @@ -1259,7 +1245,13 @@ impl UploaderData { .stats .uploader_syncing_epoch_count .set(self.syncing_data.len() as _); - set_max_synced_epoch(sync_epoch); + + for table_id in table_ids { + if let Some(table_data) = self.unsync_data.table_data.get_mut(&table_id) { + table_data.ack_synced(sync_epoch); + } + } + send_sync_result( sync_result_sender, Ok(SyncedData { @@ -1347,13 +1339,7 @@ impl HummockUploader { data.syncing_data.get_mut(&sync_id).expect("should exist"); syncing_data.uploaded.push_front(sst.clone()); assert!(syncing_data.remaining_uploading_tasks.remove(&task_id)); - data.may_notify_sync_task(&self.context, |new_synced_epoch| { - Self::set_max_synced_epoch( - &mut self.max_synced_epoch, - self.max_syncing_epoch, - new_synced_epoch, - ) - }); + data.may_notify_sync_task(&self.context); } UploadingTaskStatus::Spilling(table_ids) => { data.spilled_data.insert(task_id, (sst.clone(), table_ids)); From d234bbee53c82ff40424c61a5752c43ddad78955 Mon Sep 17 00:00:00 2001 From: William Wen Date: Fri, 28 Jun 2024 17:58:35 +0800 Subject: [PATCH 11/17] refactor(storage): remove global epoch in uploader --- .../event_handler/hummock_event_handler.rs | 5 +- .../src/hummock/event_handler/uploader.rs | 263 ++++++++++-------- 2 files changed, 142 insertions(+), 126 deletions(-) diff --git a/src/storage/src/hummock/event_handler/hummock_event_handler.rs b/src/storage/src/hummock/event_handler/hummock_event_handler.rs index cbb0044e2f798..68ff9a4a0a901 100644 --- a/src/storage/src/hummock/event_handler/hummock_event_handler.rs +++ b/src/storage/src/hummock/event_handler/hummock_event_handler.rs @@ -469,9 +469,9 @@ impl HummockEventHandler { table_ids: HashSet, ) { debug!( - "awaiting for epoch to be synced: {}, max_synced_epoch: {}", new_sync_epoch, - self.uploader.max_synced_epoch() + ?table_ids, + "awaiting for epoch to be synced", ); self.uploader .start_sync_epoch(new_sync_epoch, sync_result_sender, table_ids); @@ -481,7 +481,6 @@ impl HummockEventHandler { info!( prev_epoch, max_committed_epoch = self.uploader.max_committed_epoch(), - max_synced_epoch = self.uploader.max_synced_epoch(), "handle clear event" ); diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader.rs index f341937d8795f..fed8a6e060e6d 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader.rs @@ -718,7 +718,7 @@ impl LocalInstanceUnsyncData { assert!(latest_epoch_data.is_empty()); assert!(!latest_epoch_data.has_spilled); if cfg!(debug_assertions) { - panic!("sync epoch exceeds latest epoch, and the current instance should have be archived"); + panic!("sync epoch exceeds latest epoch, and the current instance should have been archived"); } warn!( instance_id = self.instance_id, @@ -751,14 +751,19 @@ struct TableUnsyncData { WatermarkDirection, BTreeMap, BitmapBuilder)>, )>, + // newer epoch at the front + syncing_epochs: VecDeque, + max_synced_epoch: Option, } impl TableUnsyncData { - fn new(table_id: TableId) -> Self { + fn new(table_id: TableId, committed_epoch: Option) -> Self { Self { table_id, instance_data: Default::default(), table_watermarks: None, + syncing_epochs: Default::default(), + max_synced_epoch: committed_epoch, } } @@ -772,6 +777,10 @@ impl TableUnsyncData { impl Iterator)>, )>, ) { + if let Some(prev_epoch) = self.max_sync_epoch() { + assert_gt!(epoch, prev_epoch) + } + self.syncing_epochs.push_front(epoch); ( self.instance_data .iter_mut() @@ -790,7 +799,32 @@ impl TableUnsyncData { ) } + fn ack_synced(&mut self, sync_epoch: HummockEpoch) { + let min_sync_epoch = self.syncing_epochs.pop_back().expect("should exist"); + assert_eq!(sync_epoch, min_sync_epoch); + self.max_synced_epoch = Some(sync_epoch); + } + + fn ack_committed(&mut self, committed_epoch: HummockEpoch) { + let synced_epoch_advanced = { + if let Some(max_synced_epoch) = self.max_synced_epoch + && max_synced_epoch >= committed_epoch + { + false + } else { + true + } + }; + if synced_epoch_advanced { + self.max_synced_epoch = Some(committed_epoch); + self.assert_after_epoch(committed_epoch); + } + } + fn assert_after_epoch(&self, epoch: HummockEpoch) { + if let Some(min_syncing_epoch) = self.syncing_epochs.back() { + assert_gt!(*min_syncing_epoch, epoch); + } self.instance_data .values() .for_each(|instance_data| instance_data.assert_after_epoch(epoch)); @@ -800,6 +834,13 @@ impl TableUnsyncData { assert_gt!(*oldest_epoch, epoch); } } + + fn max_sync_epoch(&self) -> Option { + self.syncing_epochs + .front() + .cloned() + .or(self.max_synced_epoch) + } } #[derive(Default)] @@ -820,15 +861,27 @@ impl UnsyncData { table_id: TableId, instance_id: LocalInstanceId, init_epoch: HummockEpoch, + context: &UploaderContext, ) { debug!( table_id = table_id.table_id, instance_id, init_epoch, "init epoch" ); - let table_data = self - .table_data - .entry(table_id) - .or_insert_with(|| TableUnsyncData::new(table_id)); + let table_data = self.table_data.entry(table_id).or_insert_with(|| { + TableUnsyncData::new( + table_id, + context + .pinned_version + .version() + .state_table_info + .info() + .get(&table_id) + .map(|info| info.committed_epoch), + ) + }); + if let Some(max_prev_epoch) = table_data.max_sync_epoch() { + assert_gt!(init_epoch, max_prev_epoch); + } assert!(table_data .instance_data .insert( @@ -900,7 +953,7 @@ impl UnsyncData { &mut self, epoch: HummockEpoch, context: &UploaderContext, - table_ids: HashSet, + table_ids: &HashSet, ) -> SyncDataBuilder { let sync_epoch_data = take_before_epoch(&mut self.epoch_data, epoch); @@ -941,6 +994,7 @@ impl UnsyncData { struct SyncingData { sync_epoch: HummockEpoch, + table_ids: HashSet, // task of newer data at the front uploading_tasks: VecDeque, // newer data at the front @@ -1020,7 +1074,7 @@ enum UploaderState { /// Data have 3 sequential stages: unsync (inside each local instance, data can be unsealed, sealed), syncing, synced. /// /// The 3 stages are divided by 2 marginal epochs: `max_syncing_epoch`, -/// `max_synced_epoch`. Epochs satisfy the following inequality. +/// `max_synced_epoch` in each `TableUnSyncData`. Epochs satisfy the following inequality. /// /// (epochs of `synced_data`) <= `max_synced_epoch` < (epochs of `syncing_data`) <= /// `max_syncing_epoch` < (epochs of `unsync_data`) @@ -1028,11 +1082,6 @@ enum UploaderState { /// Data are mostly stored in `VecDeque`, and the order stored in the `VecDeque` indicates the data /// order. Data at the front represents ***newer*** data. pub struct HummockUploader { - /// The maximum epoch that has started syncing - max_syncing_epoch: HummockEpoch, - /// The maximum epoch that has been synced - max_synced_epoch: HummockEpoch, - state: UploaderState, context: UploaderContext, @@ -1046,14 +1095,8 @@ impl HummockUploader { buffer_tracker: BufferTracker, config: &StorageOpts, ) -> Self { - let initial_epoch = pinned_version.version().max_committed_epoch; Self { - max_syncing_epoch: initial_epoch, - max_synced_epoch: initial_epoch, - state: UploaderState::Working(UploaderData { - unsync_data: Default::default(), - syncing_data: Default::default(), - }), + state: UploaderState::Working(UploaderData::default()), context: UploaderContext::new( pinned_version, spawn_upload_task, @@ -1068,10 +1111,6 @@ impl HummockUploader { &self.context.buffer_tracker } - pub(super) fn max_synced_epoch(&self) -> HummockEpoch { - self.max_synced_epoch - } - pub(super) fn max_committed_epoch(&self) -> HummockEpoch { self.context.pinned_version.max_committed_epoch() } @@ -1097,9 +1136,8 @@ impl HummockUploader { let UploaderState::Working(data) = &mut self.state else { return; }; - assert_gt!(init_epoch, self.max_syncing_epoch); data.unsync_data - .init_instance(table_id, instance_id, init_epoch); + .init_instance(table_id, instance_id, init_epoch, &self.context); } pub(super) fn local_seal_epoch( @@ -1111,7 +1149,6 @@ impl HummockUploader { let UploaderState::Working(data) = &mut self.state else { return; }; - assert_gt!(next_epoch, self.max_syncing_epoch); data.unsync_data .local_seal_epoch(instance_id, next_epoch, opts); } @@ -1136,17 +1173,9 @@ impl HummockUploader { return; } }; - debug!("start sync epoch: {}", epoch); - assert!( - epoch > self.max_syncing_epoch, - "the epoch {} has started syncing already: {}", - epoch, - self.max_syncing_epoch - ); - - self.max_syncing_epoch = epoch; + debug!(epoch, ?table_ids, "start sync epoch"); - let sync_data = data.unsync_data.sync(epoch, &self.context, table_ids); + let sync_data = data.unsync_data.sync(epoch, &self.context, &table_ids); let SyncDataBuilder { spilled_data: @@ -1160,6 +1189,7 @@ impl HummockUploader { data.syncing_data.push_front(SyncingData { sync_epoch: epoch, + table_ids, uploading_tasks, uploaded: uploaded_data, table_watermarks, @@ -1172,53 +1202,20 @@ impl HummockUploader { .set(data.syncing_data.len() as _); } - fn set_max_synced_epoch( - max_synced_epoch: &mut HummockEpoch, - max_syncing_epoch: HummockEpoch, - epoch: HummockEpoch, - ) { - assert!( - epoch <= max_syncing_epoch, - "epoch {} that has been synced has not started syncing yet. previous max syncing epoch {}", - epoch, - max_syncing_epoch - ); - assert!( - epoch > *max_synced_epoch, - "epoch {} has been synced. previous max synced epoch: {}", - epoch, - max_synced_epoch - ); - *max_synced_epoch = epoch; - } - pub(crate) fn update_pinned_version(&mut self, pinned_version: PinnedVersion) { assert_ge!( pinned_version.max_committed_epoch(), self.context.pinned_version.max_committed_epoch() ); - let max_committed_epoch = pinned_version.max_committed_epoch(); - self.context.pinned_version = pinned_version; - if self.max_synced_epoch < max_committed_epoch { - self.max_synced_epoch = max_committed_epoch; - } - if self.max_syncing_epoch < max_committed_epoch { - self.max_syncing_epoch = max_committed_epoch; - if let UploaderState::Working(data) = &self.state { - for instance_data in data - .unsync_data - .table_data - .values() - .flat_map(|data| data.instance_data.values()) - { - if let Some(oldest_epoch) = instance_data.sealed_data.back() { - assert_gt!(oldest_epoch.epoch, max_committed_epoch); - } else if let Some(current_epoch) = &instance_data.current_epoch_data { - assert_gt!(current_epoch.epoch, max_committed_epoch); - } + if let UploaderState::Working(data) = &mut self.state { + // TODO: may only `ack_committed` on table whose `committed_epoch` is changed. + for (table_id, info) in pinned_version.version().state_table_info.info() { + if let Some(table_data) = data.unsync_data.table_data.get_mut(table_id) { + table_data.ack_committed(info.committed_epoch); } } } + self.context.pinned_version = pinned_version; } pub(crate) fn may_flush(&mut self) -> bool { @@ -1257,16 +1254,11 @@ impl HummockUploader { } pub(crate) fn clear(&mut self) { - let max_committed_epoch = self.context.pinned_version.max_committed_epoch(); - self.max_synced_epoch = max_committed_epoch; - self.max_syncing_epoch = max_committed_epoch; if let UploaderState::Working(data) = replace( &mut self.state, UploaderState::Working(UploaderData::default()), ) { - data.abort(|| { - HummockError::other(format!("uploader is reset to {}", max_committed_epoch)) - }); + data.abort(|| HummockError::other("uploader is reset")); } self.context.stats.uploader_syncing_epoch_count.set(0); @@ -1287,7 +1279,6 @@ impl UploaderData { &mut self, cx: &mut Context<'_>, context: &UploaderContext, - mut set_max_synced_epoch: impl FnMut(u64), ) -> Poll, ErrState>>> { while let Some(syncing_data) = self.syncing_data.back_mut() { let sstable_info = if let Some(task) = syncing_data.uploading_tasks.back_mut() { @@ -1330,6 +1321,7 @@ impl UploaderData { let syncing_data = self.syncing_data.pop_back().expect("non-empty"); let SyncingData { sync_epoch, + table_ids, uploading_tasks, uploaded, table_watermarks, @@ -1340,7 +1332,11 @@ impl UploaderData { .stats .uploader_syncing_epoch_count .set(self.syncing_data.len() as _); - set_max_synced_epoch(sync_epoch); + for table_id in table_ids { + if let Some(table_data) = self.unsync_data.table_data.get_mut(&table_id) { + table_data.ack_synced(sync_epoch); + } + } send_sync_result( sync_result_sender, Ok(SyncedData { @@ -1382,17 +1378,7 @@ impl HummockUploader { return Poll::Pending; }; - if let Some(result) = - ready!( - data.poll_syncing_task(cx, &self.context, |new_synced_epoch| { - Self::set_max_synced_epoch( - &mut self.max_synced_epoch, - self.max_syncing_epoch, - new_synced_epoch, - ) - }) - ) - { + if let Some(result) = ready!(data.poll_syncing_task(cx, &self.context)) { match result { Ok(data) => { return Poll::Ready(data); @@ -1442,10 +1428,11 @@ pub(crate) mod tests { use risingwave_common::catalog::TableId; use risingwave_common::must_match; use risingwave_common::util::epoch::{test_epoch, EpochExt}; + use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::{FullKey, TableKey}; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{HummockEpoch, LocalSstableInfo}; - use risingwave_pb::hummock::{KeyRange, SstableInfo}; + use risingwave_pb::hummock::{KeyRange, SstableInfo, StateTableInfoDelta}; use spin::Mutex; use tokio::spawn; use tokio::sync::mpsc::unbounded_channel; @@ -1455,8 +1442,9 @@ pub(crate) mod tests { use crate::hummock::event_handler::hummock_event_handler::BufferTracker; use crate::hummock::event_handler::uploader::uploader_imm::UploaderImm; use crate::hummock::event_handler::uploader::{ - get_payload_imm_ids, HummockUploader, SyncedData, UploadTaskInfo, UploadTaskOutput, - UploadTaskPayload, UploaderContext, UploaderData, UploaderState, UploadingTask, + get_payload_imm_ids, HummockUploader, SyncedData, TableUnsyncData, UploadTaskInfo, + UploadTaskOutput, UploadTaskPayload, UploaderContext, UploaderData, UploaderState, + UploadingTask, }; use crate::hummock::event_handler::{LocalInstanceId, TEST_LOCAL_INSTANCE_ID}; use crate::hummock::local_version::pinned_version::PinnedVersion; @@ -1481,12 +1469,39 @@ pub(crate) mod tests { fn data(&self) -> &UploaderData { must_match!(&self.state, UploaderState::Working(data) => data) } + + fn table_data(&self) -> &TableUnsyncData { + self.data() + .unsync_data + .table_data + .get(&TEST_TABLE_ID) + .expect("should exist") + } + + fn test_max_syncing_epoch(&self) -> HummockEpoch { + self.table_data().max_sync_epoch().unwrap() + } + + fn test_max_synced_epoch(&self) -> HummockEpoch { + self.table_data().max_synced_epoch.unwrap() + } } fn test_hummock_version(epoch: HummockEpoch) -> HummockVersion { let mut version = HummockVersion::default(); version.id = epoch; version.max_committed_epoch = epoch; + version.state_table_info.apply_delta( + &HashMap::from_iter([( + TEST_TABLE_ID, + StateTableInfoDelta { + committed_epoch: epoch, + safe_epoch: epoch, + compaction_group_id: StaticCompactionGroupId::StateDefault as _, + }, + )]), + &HashSet::new(), + ); version } @@ -1713,7 +1728,7 @@ pub(crate) mod tests { let (sync_tx, sync_rx) = oneshot::channel(); uploader.start_sync_epoch(epoch1, sync_tx, HashSet::from_iter([TEST_TABLE_ID])); - assert_eq!(epoch1 as HummockEpoch, uploader.max_syncing_epoch); + assert_eq!(epoch1 as HummockEpoch, uploader.test_max_syncing_epoch()); assert_eq!(1, uploader.data().syncing_data.len()); let syncing_data = uploader.data().syncing_data.front().unwrap(); assert_eq!(epoch1 as HummockEpoch, syncing_data.sync_epoch); @@ -1752,7 +1767,7 @@ pub(crate) mod tests { } _ => unreachable!(), }; - assert_eq!(epoch1, uploader.max_synced_epoch()); + assert_eq!(epoch1, uploader.test_max_synced_epoch()); let new_pinned_version = uploader .context @@ -1768,8 +1783,10 @@ pub(crate) mod tests { let epoch1 = INITIAL_EPOCH.next_epoch(); let (sync_tx, sync_rx) = oneshot::channel(); + uploader.init_instance(TEST_LOCAL_INSTANCE_ID, TEST_TABLE_ID, epoch1); + uploader.local_seal_epoch_for_test(TEST_LOCAL_INSTANCE_ID, epoch1); uploader.start_sync_epoch(epoch1, sync_tx, HashSet::from_iter([TEST_TABLE_ID])); - assert_eq!(epoch1, uploader.max_syncing_epoch); + assert_eq!(epoch1, uploader.test_max_syncing_epoch()); assert_uploader_pending(&mut uploader).await; @@ -1779,7 +1796,7 @@ pub(crate) mod tests { } _ => unreachable!(), }; - assert_eq!(epoch1, uploader.max_synced_epoch()); + assert_eq!(epoch1, uploader.test_max_synced_epoch()); let new_pinned_version = uploader .context .pinned_version @@ -1802,7 +1819,7 @@ pub(crate) mod tests { let (sync_tx, sync_rx) = oneshot::channel(); uploader.start_sync_epoch(epoch1, sync_tx, HashSet::from_iter([TEST_TABLE_ID])); - assert_eq!(epoch1, uploader.max_syncing_epoch); + assert_eq!(epoch1, uploader.test_max_syncing_epoch()); assert_uploader_pending(&mut uploader).await; @@ -1812,7 +1829,7 @@ pub(crate) mod tests { } _ => unreachable!(), }; - assert_eq!(epoch1, uploader.max_synced_epoch()); + assert_eq!(epoch1, uploader.test_max_synced_epoch()); let new_pinned_version = uploader .context .pinned_version @@ -1826,11 +1843,9 @@ pub(crate) mod tests { async fn test_uploader_poll_empty() { let mut uploader = test_uploader(dummy_success_upload_future); let data = must_match!(&mut uploader.state, UploaderState::Working(data) => data); - assert!( - poll_fn(|cx| data.poll_syncing_task(cx, &uploader.context, |_| unreachable!())) - .await - .is_none() - ); + assert!(poll_fn(|cx| data.poll_syncing_task(cx, &uploader.context)) + .await + .is_none()); assert!(poll_fn(|cx| data.poll_spill_task(cx)).await.is_none()); } @@ -1849,28 +1864,30 @@ pub(crate) mod tests { let version3 = initial_pinned_version.new_pin_version(test_hummock_version(epoch3)); let version4 = initial_pinned_version.new_pin_version(test_hummock_version(epoch4)); let version5 = initial_pinned_version.new_pin_version(test_hummock_version(epoch5)); - uploader.update_pinned_version(version1); - assert_eq!(epoch1, uploader.max_synced_epoch); - assert_eq!(epoch1, uploader.max_syncing_epoch); uploader.init_instance(TEST_LOCAL_INSTANCE_ID, TEST_TABLE_ID, epoch6); + + uploader.update_pinned_version(version1); + assert_eq!(epoch1, uploader.test_max_synced_epoch()); + assert_eq!(epoch1, uploader.test_max_syncing_epoch()); + let imm = gen_imm(epoch6).await; uploader.add_imm(TEST_LOCAL_INSTANCE_ID, imm.clone()); uploader.update_pinned_version(version2); - assert_eq!(epoch2, uploader.max_synced_epoch); - assert_eq!(epoch2, uploader.max_syncing_epoch); + assert_eq!(epoch2, uploader.test_max_synced_epoch()); + assert_eq!(epoch2, uploader.test_max_syncing_epoch()); uploader.local_seal_epoch_for_test(TEST_LOCAL_INSTANCE_ID, epoch6); uploader.update_pinned_version(version3); - assert_eq!(epoch3, uploader.max_synced_epoch); - assert_eq!(epoch3, uploader.max_syncing_epoch); + assert_eq!(epoch3, uploader.test_max_synced_epoch()); + assert_eq!(epoch3, uploader.test_max_syncing_epoch()); let (sync_tx, sync_rx) = oneshot::channel(); uploader.start_sync_epoch(epoch6, sync_tx, HashSet::from_iter([TEST_TABLE_ID])); - assert_eq!(epoch6, uploader.max_syncing_epoch); + assert_eq!(epoch6, uploader.test_max_syncing_epoch()); uploader.update_pinned_version(version4); - assert_eq!(epoch4, uploader.max_synced_epoch); - assert_eq!(epoch6, uploader.max_syncing_epoch); + assert_eq!(epoch4, uploader.test_max_synced_epoch()); + assert_eq!(epoch6, uploader.test_max_syncing_epoch()); let sst = uploader.next_uploaded_sst().await; assert_eq!(&get_imm_ids([&imm]), sst.imm_ids()); @@ -1885,8 +1902,8 @@ pub(crate) mod tests { } uploader.update_pinned_version(version5); - assert_eq!(epoch6, uploader.max_synced_epoch); - assert_eq!(epoch6, uploader.max_syncing_epoch); + assert_eq!(epoch6, uploader.test_max_synced_epoch()); + assert_eq!(epoch6, uploader.test_max_syncing_epoch()); } fn prepare_uploader_order_test( @@ -2134,7 +2151,7 @@ pub(crate) mod tests { } else { unreachable!("should be sync finish"); } - assert_eq!(epoch2, uploader.max_synced_epoch); + assert_eq!(epoch2, uploader.test_max_synced_epoch()); // current uploader state: // unsealed: epoch4: imm: imm4 @@ -2188,7 +2205,7 @@ pub(crate) mod tests { } else { unreachable!("should be sync finish"); } - assert_eq!(epoch4, uploader.max_synced_epoch); + assert_eq!(epoch4, uploader.test_max_synced_epoch()); // current uploader state: // unsealed: empty From 30a808edc06b61aeb199a178cbc2ae279f97d887 Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 3 Jul 2024 01:37:58 +0800 Subject: [PATCH 12/17] fix test and address comment --- .../src/hummock/event_handler/uploader/mod.rs | 52 +++++++++++-------- .../event_handler/uploader/task_manager.rs | 20 +++++-- .../src/hummock/store/hummock_storage.rs | 14 +++-- 3 files changed, 53 insertions(+), 33 deletions(-) diff --git a/src/storage/src/hummock/event_handler/uploader/mod.rs b/src/storage/src/hummock/event_handler/uploader/mod.rs index 21704d83072ff..eaaeb41a344bc 100644 --- a/src/storage/src/hummock/event_handler/uploader/mod.rs +++ b/src/storage/src/hummock/event_handler/uploader/mod.rs @@ -20,9 +20,7 @@ use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet, VecDeque}; use std::fmt::{Debug, Display, Formatter}; use std::future::{poll_fn, Future}; use std::mem::{replace, swap, take}; -use std::sync::atomic::AtomicUsize; -use std::sync::atomic::Ordering::Relaxed; -use std::sync::{Arc, LazyLock}; +use std::sync::Arc; use std::task::{ready, Context, Poll}; use futures::FutureExt; @@ -227,7 +225,7 @@ impl UploadingTask { .collect() } - fn new(input: UploadTaskInput, context: &UploaderContext) -> Self { + fn new(task_id: UploadingTaskId, input: UploadTaskInput, context: &UploaderContext) -> Self { assert!(!input.is_empty()); let mut epochs = input .iter() @@ -261,9 +259,8 @@ impl UploadingTask { } let join_handle = (context.spawn_upload_task)(payload, task_info.clone()); context.stats.uploader_uploading_task_count.inc(); - static NEXT_TASK_ID: LazyLock = LazyLock::new(|| AtomicUsize::new(0)); Self { - task_id: UploadingTaskId(NEXT_TASK_ID.fetch_add(1, Relaxed)), + task_id, input, join_handle, task_info, @@ -840,7 +837,7 @@ impl UploaderData { let _epochs = take_before_epoch(&mut self.unsync_data.epochs, epoch); let mut all_table_watermarks = HashMap::new(); - let mut spilling_tasks = HashSet::new(); + let mut uploading_tasks = HashSet::new(); let mut spilled_tasks = BTreeSet::new(); let mut flush_payload = HashMap::new(); @@ -867,22 +864,25 @@ impl UploaderData { if self.spilled_data.contains_key(&task_id) { spilled_tasks.insert(task_id); } else { - spilling_tasks.insert(task_id); + uploading_tasks.insert(task_id); } } } - static NEXT_SYNC_ID: LazyLock = LazyLock::new(|| AtomicUsize::new(0)); - let sync_id = SyncId(NEXT_SYNC_ID.fetch_add(1, Relaxed)); + let sync_id = { + let sync_id = self.next_sync_id; + self.next_sync_id += 1; + SyncId(sync_id) + }; if let Some(extra_flush_task_id) = self.task_manager.sync( context, sync_id, flush_payload, - spilling_tasks.iter().cloned(), + uploading_tasks.iter().cloned(), &table_ids, ) { - spilling_tasks.insert(extra_flush_task_id); + uploading_tasks.insert(extra_flush_task_id); } // iter from large task_id to small one so that newer data at the front @@ -905,7 +905,7 @@ impl UploaderData { SyncingData { sync_epoch: epoch, table_ids, - remaining_uploading_tasks: spilling_tasks, + remaining_uploading_tasks: uploading_tasks, uploaded, table_watermarks: all_table_watermarks, sync_result_sender, @@ -976,13 +976,11 @@ struct SyncId(usize); struct UploaderData { unsync_data: UnsyncData, - /// Data that has started syncing but not synced yet. `epoch` satisfies - /// `max_synced_epoch < epoch <= max_syncing_epoch`. - /// Newer epoch at the front syncing_data: BTreeMap, task_manager: TaskManager, spilled_data: HashMap, HashSet)>, + next_sync_id: usize, } impl UploaderData { @@ -1226,8 +1224,6 @@ impl HummockUploader { } impl UploaderData { - /// Poll the syncing task of the syncing data of the oldest epoch. Return `Poll::Ready(None)` if - /// there is no syncing data. fn may_notify_sync_task(&mut self, context: &UploaderContext) { while let Some((_, syncing_data)) = self.syncing_data.first_key_value() && syncing_data.remaining_uploading_tasks.is_empty() @@ -1360,6 +1356,13 @@ impl HummockUploader { }), ), UploaderState::Working(data) => data); + let _ = syncing_data + .sync_result_sender + .send(Err(HummockError::other(format!( + "failed to sync: {:?}", + e.as_report() + )))); + data.abort(|| { HummockError::other(format!( "previous epoch {} failed to sync", @@ -1383,8 +1386,8 @@ pub(crate) mod tests { use std::ops::Deref; use std::pin::pin; use std::sync::atomic::AtomicUsize; - use std::sync::atomic::Ordering::SeqCst; - use std::sync::Arc; + use std::sync::atomic::Ordering::{Relaxed, SeqCst}; + use std::sync::{Arc, LazyLock}; use std::task::Poll; use bytes::Bytes; @@ -1411,7 +1414,7 @@ pub(crate) mod tests { use crate::hummock::event_handler::uploader::{ get_payload_imm_ids, HummockUploader, SyncedData, TableUnsyncData, UploadTaskInfo, UploadTaskOutput, UploadTaskPayload, UploaderContext, UploaderData, UploaderState, - UploadingTask, + UploadingTask, UploadingTaskId, }; use crate::hummock::event_handler::{LocalInstanceId, TEST_LOCAL_INSTANCE_ID}; use crate::hummock::local_version::pinned_version::PinnedVersion; @@ -1590,7 +1593,12 @@ pub(crate) mod tests { TEST_LOCAL_INSTANCE_ID, imms.into_iter().map(UploaderImm::for_test).collect_vec(), )]); - Self::new(input, context) + static NEXT_TASK_ID: LazyLock = LazyLock::new(|| AtomicUsize::new(0)); + Self::new( + UploadingTaskId(NEXT_TASK_ID.fetch_add(1, Relaxed)), + input, + context, + ) } } diff --git a/src/storage/src/hummock/event_handler/uploader/task_manager.rs b/src/storage/src/hummock/event_handler/uploader/task_manager.rs index fd3ec142fbddf..2347be1ed57eb 100644 --- a/src/storage/src/hummock/event_handler/uploader/task_manager.rs +++ b/src/storage/src/hummock/event_handler/uploader/task_manager.rs @@ -31,6 +31,7 @@ pub(super) struct TaskManager { tasks: HashMap, // newer task at the front task_order: VecDeque, + next_task_id: usize, } impl TaskManager { @@ -41,7 +42,10 @@ impl TaskManager { ) -> &UploadingTaskStatus { let task_id = task.task_id; self.task_order.push_front(task.task_id); - self.tasks.insert(task.task_id, TaskEntry { task, status }); + assert!(self + .tasks + .insert(task.task_id, TaskEntry { task, status }) + .is_none()); &self.tasks.get(&task_id).expect("should exist").status } @@ -64,6 +68,12 @@ impl TaskManager { Poll::Ready(result) } + fn get_next_task_id(&mut self) -> UploadingTaskId { + let task_id = self.next_task_id; + self.next_task_id += 1; + UploadingTaskId(task_id) + } + #[expect(clippy::type_complexity)] pub(super) fn poll_task_result( &mut self, @@ -100,7 +110,7 @@ impl TaskManager { imms: HashMap>, ) -> (UploadingTaskId, usize, &HashSet) { assert!(!imms.is_empty()); - let task = UploadingTask::new(imms, context); + let task = UploadingTask::new(self.get_next_task_id(), imms, context); context.stats.spill_task_counts_from_unsealed.inc(); context .stats @@ -146,7 +156,11 @@ impl TaskManager { let task = if unflushed_payload.is_empty() { None } else { - Some(UploadingTask::new(unflushed_payload, context)) + Some(UploadingTask::new( + self.get_next_task_id(), + unflushed_payload, + context, + )) }; for task_id in spill_task_ids { diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index 11aa643c3659a..7b28588c5054a 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -565,16 +565,14 @@ impl StateStore for HummockStorage { fn sync(&self, epoch: u64, table_ids: HashSet) -> impl SyncFuture { let (tx, rx) = oneshot::channel(); - self.hummock_event_sender - .send(HummockEvent::SyncEpoch { - new_sync_epoch: epoch, - sync_result_sender: tx, - table_ids, - }) - .expect("should send success"); + let _ = self.hummock_event_sender.send(HummockEvent::SyncEpoch { + new_sync_epoch: epoch, + sync_result_sender: tx, + table_ids, + }); rx.map(|recv_result| { Ok(recv_result - .expect("should wait success")? + .map_err(|_| HummockError::other("failed to receive sync result"))?? .into_sync_result()) }) } From ffb99066513d1dd4f1dfcd30f4b453a6a1290a45 Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 8 Jul 2024 18:42:09 +0800 Subject: [PATCH 13/17] fix panic --- src/storage/src/hummock/event_handler/uploader.rs | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader.rs index fed8a6e060e6d..1ff6f654f5a5f 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader.rs @@ -841,6 +841,10 @@ impl TableUnsyncData { .cloned() .or(self.max_synced_epoch) } + + fn is_empty(&self) -> bool { + self.instance_data.is_empty() && self.syncing_epochs.is_empty() + } } #[derive(Default)] @@ -943,7 +947,7 @@ impl UnsyncData { debug!(instance_id, "destroy instance"); let table_data = self.table_data.get_mut(&table_id).expect("should exist"); assert!(table_data.instance_data.remove(&instance_id).is_some()); - if table_data.instance_data.is_empty() { + if table_data.is_empty() { self.table_data.remove(&table_id); } } @@ -1335,6 +1339,9 @@ impl UploaderData { for table_id in table_ids { if let Some(table_data) = self.unsync_data.table_data.get_mut(&table_id) { table_data.ack_synced(sync_epoch); + if table_data.is_empty() { + self.unsync_data.table_data.remove(&table_id); + } } } send_sync_result( From 3808d55450fbc36e73ab486dc2f1bcd8a5d8ef58 Mon Sep 17 00:00:00 2001 From: William Wen Date: Tue, 9 Jul 2024 14:07:38 +0800 Subject: [PATCH 14/17] fix --- src/storage/src/hummock/event_handler/uploader.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader.rs index 1ff6f654f5a5f..f705459844c9d 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader.rs @@ -817,14 +817,14 @@ impl TableUnsyncData { }; if synced_epoch_advanced { self.max_synced_epoch = Some(committed_epoch); + if let Some(min_syncing_epoch) = self.syncing_epochs.back() { + assert_gt!(*min_syncing_epoch, committed_epoch); + } self.assert_after_epoch(committed_epoch); } } fn assert_after_epoch(&self, epoch: HummockEpoch) { - if let Some(min_syncing_epoch) = self.syncing_epochs.back() { - assert_gt!(*min_syncing_epoch, epoch); - } self.instance_data .values() .for_each(|instance_data| instance_data.assert_after_epoch(epoch)); From d02775a87c3af9d146dafb710d2aa7efb63ccc3b Mon Sep 17 00:00:00 2001 From: William Wen Date: Tue, 9 Jul 2024 14:46:40 +0800 Subject: [PATCH 15/17] print error table id --- ci/scripts/deterministic-e2e-test.sh | 28 +++++++++---------- .../src/hummock/event_handler/uploader.rs | 5 +++- 2 files changed, 18 insertions(+), 15 deletions(-) diff --git a/ci/scripts/deterministic-e2e-test.sh b/ci/scripts/deterministic-e2e-test.sh index c561978e428aa..4e025d2a7268c 100755 --- a/ci/scripts/deterministic-e2e-test.sh +++ b/ci/scripts/deterministic-e2e-test.sh @@ -30,20 +30,20 @@ export LOGDIR=.risingwave/log mkdir -p $LOGDIR -echo "--- deterministic simulation e2e, ci-3cn-2fe, ddl" -seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation ./e2e_test/ddl/\*\*/\*.slt 2> $LOGDIR/ddl-{}.log && rm $LOGDIR/ddl-{}.log' - -echo "--- deterministic simulation e2e, ci-3cn-2fe, streaming" -seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation ./e2e_test/streaming/\*\*/\*.slt 2> $LOGDIR/streaming-{}.log && rm $LOGDIR/streaming-{}.log' - -echo "--- deterministic simulation e2e, ci-3cn-2fe, batch" -seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation ./e2e_test/batch/\*\*/\*.slt 2> $LOGDIR/batch-{}.log && rm $LOGDIR/batch-{}.log' - -echo "--- deterministic simulation e2e, ci-3cn-2fe, kafka source" -seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation --kafka-datadir=./scripts/source/test_data ./e2e_test/source/basic/kafka\*.slt 2> $LOGDIR/source-{}.log && rm $LOGDIR/source-{}.log' - -echo "--- deterministic simulation e2e, ci-3cn-2fe, parallel, streaming" -seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation -j 16 ./e2e_test/streaming/\*\*/\*.slt 2> $LOGDIR/parallel-streaming-{}.log && rm $LOGDIR/parallel-streaming-{}.log' +#echo "--- deterministic simulation e2e, ci-3cn-2fe, ddl" +#seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation ./e2e_test/ddl/\*\*/\*.slt 2> $LOGDIR/ddl-{}.log && rm $LOGDIR/ddl-{}.log' +# +#echo "--- deterministic simulation e2e, ci-3cn-2fe, streaming" +#seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation ./e2e_test/streaming/\*\*/\*.slt 2> $LOGDIR/streaming-{}.log && rm $LOGDIR/streaming-{}.log' +# +#echo "--- deterministic simulation e2e, ci-3cn-2fe, batch" +#seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation ./e2e_test/batch/\*\*/\*.slt 2> $LOGDIR/batch-{}.log && rm $LOGDIR/batch-{}.log' +# +#echo "--- deterministic simulation e2e, ci-3cn-2fe, kafka source" +#seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation --kafka-datadir=./scripts/source/test_data ./e2e_test/source/basic/kafka\*.slt 2> $LOGDIR/source-{}.log && rm $LOGDIR/source-{}.log' +# +#echo "--- deterministic simulation e2e, ci-3cn-2fe, parallel, streaming" +#seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation -j 16 ./e2e_test/streaming/\*\*/\*.slt 2> $LOGDIR/parallel-streaming-{}.log && rm $LOGDIR/parallel-streaming-{}.log' echo "--- deterministic simulation e2e, ci-3cn-2fe, parallel, batch" seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation -j 16 ./e2e_test/batch/\*\*/\*.slt 2> $LOGDIR/parallel-batch-{}.log && rm $LOGDIR/parallel-batch-{}.log' diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader.rs index f705459844c9d..5d65d7d8709c6 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader.rs @@ -800,7 +800,10 @@ impl TableUnsyncData { } fn ack_synced(&mut self, sync_epoch: HummockEpoch) { - let min_sync_epoch = self.syncing_epochs.pop_back().expect("should exist"); + let min_sync_epoch = self + .syncing_epochs + .pop_back() + .unwrap_or_else(|| panic!("should exist: {}", self.table_id.table_id)); assert_eq!(sync_epoch, min_sync_epoch); self.max_synced_epoch = Some(sync_epoch); } From 62b5cd4d3dd2fa15f60a177ac0c4c1684620e077 Mon Sep 17 00:00:00 2001 From: William Wen Date: Tue, 9 Jul 2024 14:57:42 +0800 Subject: [PATCH 16/17] ack sync with subset --- ci/scripts/deterministic-e2e-test.sh | 28 +++++++++---------- .../src/hummock/event_handler/uploader.rs | 21 +++++++++----- 2 files changed, 28 insertions(+), 21 deletions(-) diff --git a/ci/scripts/deterministic-e2e-test.sh b/ci/scripts/deterministic-e2e-test.sh index 4e025d2a7268c..c561978e428aa 100755 --- a/ci/scripts/deterministic-e2e-test.sh +++ b/ci/scripts/deterministic-e2e-test.sh @@ -30,20 +30,20 @@ export LOGDIR=.risingwave/log mkdir -p $LOGDIR -#echo "--- deterministic simulation e2e, ci-3cn-2fe, ddl" -#seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation ./e2e_test/ddl/\*\*/\*.slt 2> $LOGDIR/ddl-{}.log && rm $LOGDIR/ddl-{}.log' -# -#echo "--- deterministic simulation e2e, ci-3cn-2fe, streaming" -#seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation ./e2e_test/streaming/\*\*/\*.slt 2> $LOGDIR/streaming-{}.log && rm $LOGDIR/streaming-{}.log' -# -#echo "--- deterministic simulation e2e, ci-3cn-2fe, batch" -#seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation ./e2e_test/batch/\*\*/\*.slt 2> $LOGDIR/batch-{}.log && rm $LOGDIR/batch-{}.log' -# -#echo "--- deterministic simulation e2e, ci-3cn-2fe, kafka source" -#seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation --kafka-datadir=./scripts/source/test_data ./e2e_test/source/basic/kafka\*.slt 2> $LOGDIR/source-{}.log && rm $LOGDIR/source-{}.log' -# -#echo "--- deterministic simulation e2e, ci-3cn-2fe, parallel, streaming" -#seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation -j 16 ./e2e_test/streaming/\*\*/\*.slt 2> $LOGDIR/parallel-streaming-{}.log && rm $LOGDIR/parallel-streaming-{}.log' +echo "--- deterministic simulation e2e, ci-3cn-2fe, ddl" +seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation ./e2e_test/ddl/\*\*/\*.slt 2> $LOGDIR/ddl-{}.log && rm $LOGDIR/ddl-{}.log' + +echo "--- deterministic simulation e2e, ci-3cn-2fe, streaming" +seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation ./e2e_test/streaming/\*\*/\*.slt 2> $LOGDIR/streaming-{}.log && rm $LOGDIR/streaming-{}.log' + +echo "--- deterministic simulation e2e, ci-3cn-2fe, batch" +seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation ./e2e_test/batch/\*\*/\*.slt 2> $LOGDIR/batch-{}.log && rm $LOGDIR/batch-{}.log' + +echo "--- deterministic simulation e2e, ci-3cn-2fe, kafka source" +seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation --kafka-datadir=./scripts/source/test_data ./e2e_test/source/basic/kafka\*.slt 2> $LOGDIR/source-{}.log && rm $LOGDIR/source-{}.log' + +echo "--- deterministic simulation e2e, ci-3cn-2fe, parallel, streaming" +seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation -j 16 ./e2e_test/streaming/\*\*/\*.slt 2> $LOGDIR/parallel-streaming-{}.log && rm $LOGDIR/parallel-streaming-{}.log' echo "--- deterministic simulation e2e, ci-3cn-2fe, parallel, batch" seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation -j 16 ./e2e_test/batch/\*\*/\*.slt 2> $LOGDIR/parallel-batch-{}.log && rm $LOGDIR/parallel-batch-{}.log' diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader.rs index 5d65d7d8709c6..4a3a4e781507c 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader.rs @@ -461,6 +461,8 @@ struct SyncDataBuilder { spilled_data: SpilledData, table_watermarks: HashMap, + + table_ids_to_ack: HashSet, } impl SyncDataBuilder { @@ -800,10 +802,7 @@ impl TableUnsyncData { } fn ack_synced(&mut self, sync_epoch: HummockEpoch) { - let min_sync_epoch = self - .syncing_epochs - .pop_back() - .unwrap_or_else(|| panic!("should exist: {}", self.table_id.table_id)); + let min_sync_epoch = self.syncing_epochs.pop_back().expect("should exist"); assert_eq!(sync_epoch, min_sync_epoch); self.max_synced_epoch = Some(sync_epoch); } @@ -970,11 +969,13 @@ impl UnsyncData { } let mut flush_payload = HashMap::new(); + let mut table_ids_to_ack = HashSet::new(); for (table_id, table_data) in &mut self.table_data { if !table_ids.contains(table_id) { table_data.assert_after_epoch(epoch); continue; } + table_ids_to_ack.insert(*table_id); let (unflushed_payload, table_watermarks) = table_data.sync(epoch); for (instance_id, payload) in unflushed_payload { if !payload.is_empty() { @@ -986,6 +987,7 @@ impl UnsyncData { } } sync_data.flush(context, flush_payload); + sync_data.table_ids_to_ack = table_ids_to_ack; sync_data } @@ -1002,6 +1004,8 @@ impl UnsyncData { struct SyncingData { sync_epoch: HummockEpoch, table_ids: HashSet, + /// Subset of `table_ids` that has existing instance + table_ids_to_ack: HashSet, // task of newer data at the front uploading_tasks: VecDeque, // newer data at the front @@ -1191,12 +1195,14 @@ impl HummockUploader { uploaded_data, }, table_watermarks, - .. + table_ids_to_ack, + epochs: _, } = sync_data; data.syncing_data.push_front(SyncingData { sync_epoch: epoch, table_ids, + table_ids_to_ack, uploading_tasks, uploaded: uploaded_data, table_watermarks, @@ -1328,7 +1334,8 @@ impl UploaderData { let syncing_data = self.syncing_data.pop_back().expect("non-empty"); let SyncingData { sync_epoch, - table_ids, + table_ids: _, + table_ids_to_ack, uploading_tasks, uploaded, table_watermarks, @@ -1339,7 +1346,7 @@ impl UploaderData { .stats .uploader_syncing_epoch_count .set(self.syncing_data.len() as _); - for table_id in table_ids { + for table_id in table_ids_to_ack { if let Some(table_data) = self.unsync_data.table_data.get_mut(&table_id) { table_data.ack_synced(sync_epoch); if table_data.is_empty() { From 1010dbf3594a3b5483cc81ab0ec8906bbfb8954d Mon Sep 17 00:00:00 2001 From: William Wen Date: Tue, 9 Jul 2024 15:11:31 +0800 Subject: [PATCH 17/17] fix clippy --- src/storage/src/hummock/event_handler/uploader.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader.rs index 4a3a4e781507c..d25b9abd79c5e 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader.rs @@ -1334,7 +1334,7 @@ impl UploaderData { let syncing_data = self.syncing_data.pop_back().expect("non-empty"); let SyncingData { sync_epoch, - table_ids: _, + table_ids: _table_ids, table_ids_to_ack, uploading_tasks, uploaded,