risingwave_storage/hummock/event_handler/
mod.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
// 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 std::collections::{HashMap, HashSet};
use std::sync::Arc;

use parking_lot::{RwLock, RwLockReadGuard};
use risingwave_common::bitmap::Bitmap;
use risingwave_common::catalog::TableId;
use risingwave_hummock_sdk::{HummockEpoch, HummockSstableObjectId};
use thiserror_ext::AsReport;
use tokio::sync::oneshot;

use crate::hummock::shared_buffer::shared_buffer_batch::{SharedBufferBatch, SharedBufferBatchId};
use crate::hummock::HummockResult;
use crate::mem_table::ImmutableMemtable;
use crate::store::SealCurrentEpochOptions;

pub mod hummock_event_handler;
pub mod refiller;
pub mod uploader;

pub use hummock_event_handler::HummockEventHandler;
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 {
    pub batch: SharedBufferBatch,
    pub epoch: HummockEpoch,
    pub grant_sender: oneshot::Sender<()>,
}

#[derive(Debug)]
pub enum HummockVersionUpdate {
    VersionDeltas(Vec<HummockVersionDelta>),
    PinnedVersion(Box<HummockVersion>),
}

pub enum HummockEvent {
    /// Notify that we may flush the shared buffer.
    BufferMayFlush,

    /// An epoch is going to be synced. Once the event is processed, there will be no more flush
    /// task on this epoch. Previous concurrent flush task join handle will be returned by the join
    /// handle sender.
    SyncEpoch {
        sync_result_sender: oneshot::Sender<HummockResult<SyncedData>>,
        sync_table_epochs: Vec<(HummockEpoch, HashSet<TableId>)>,
    },

    /// Clear shared buffer and reset all states
    Clear(oneshot::Sender<()>, Option<HashSet<TableId>>),

    Shutdown,

    ImmToUploader {
        instance_id: SharedBufferBatchId,
        imm: ImmutableMemtable,
    },

    StartEpoch {
        epoch: HummockEpoch,
        table_ids: HashSet<TableId>,
    },

    InitEpoch {
        instance_id: LocalInstanceId,
        init_epoch: HummockEpoch,
    },

    LocalSealEpoch {
        instance_id: LocalInstanceId,
        next_epoch: HummockEpoch,
        opts: SealCurrentEpochOptions,
    },

    #[cfg(any(test, feature = "test"))]
    /// Flush all previous event. When all previous events has been consumed, the event handler
    /// will notify
    FlushEvent(oneshot::Sender<()>),

    RegisterReadVersion {
        table_id: TableId,
        new_read_version_sender: oneshot::Sender<(HummockReadVersionRef, LocalInstanceGuard)>,
        is_replicated: bool,
        vnodes: Arc<Bitmap>,
    },

    DestroyReadVersion {
        instance_id: LocalInstanceId,
    },

    GetMinUncommittedSstId {
        result_tx: oneshot::Sender<Option<HummockSstableObjectId>>,
    },
}

impl HummockEvent {
    fn to_debug_string(&self) -> String {
        match self {
            HummockEvent::BufferMayFlush => "BufferMayFlush".to_string(),

            HummockEvent::SyncEpoch {
                sync_result_sender: _,
                sync_table_epochs,
            } => format!("AwaitSyncEpoch epoch {:?}", sync_table_epochs),

            HummockEvent::Clear(_, table_ids) => {
                format!("Clear {:?}", table_ids)
            }

            HummockEvent::Shutdown => "Shutdown".to_string(),

            HummockEvent::StartEpoch { epoch, table_ids } => {
                format!("StartEpoch {} {:?}", epoch, table_ids)
            }

            HummockEvent::InitEpoch {
                instance_id,
                init_epoch,
            } => {
                format!("InitEpoch {} {}", instance_id, init_epoch)
            }

            HummockEvent::ImmToUploader { instance_id, imm } => {
                format!("ImmToUploader {} {}", instance_id, imm.batch_id())
            }

            HummockEvent::LocalSealEpoch {
                instance_id,
                next_epoch,
                opts,
            } => {
                format!(
                    "LocalSealEpoch next_epoch: {}, instance_id: {}, opts: {:?}",
                    next_epoch, instance_id, opts
                )
            }

            HummockEvent::RegisterReadVersion {
                table_id,
                new_read_version_sender: _,
                is_replicated,
                vnodes: _,
            } => format!(
                "RegisterReadVersion table_id {:?}, is_replicated: {:?}",
                table_id, is_replicated
            ),

            HummockEvent::DestroyReadVersion { instance_id } => {
                format!("DestroyReadVersion instance_id {:?}", instance_id)
            }

            #[cfg(any(test, feature = "test"))]
            HummockEvent::FlushEvent(_) => "FlushEvent".to_string(),
            HummockEvent::GetMinUncommittedSstId { .. } => "GetMinSpilledSstId".to_string(),
        }
    }
}

impl std::fmt::Debug for HummockEvent {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("HummockEvent")
            .field("debug_string", &self.to_debug_string())
            .finish()
    }
}

pub type LocalInstanceId = u64;
pub const TEST_LOCAL_INSTANCE_ID: LocalInstanceId = 233;
pub type HummockReadVersionRef = Arc<RwLock<HummockReadVersion>>;
pub type ReadVersionMappingType = HashMap<TableId, HashMap<LocalInstanceId, HummockReadVersionRef>>;
pub type ReadOnlyReadVersionMapping = ReadOnlyRwLockRef<ReadVersionMappingType>;

pub struct ReadOnlyRwLockRef<T>(Arc<RwLock<T>>);

impl<T> Clone for ReadOnlyRwLockRef<T> {
    fn clone(&self) -> Self {
        Self(self.0.clone())
    }
}

impl<T> ReadOnlyRwLockRef<T> {
    pub fn new(inner: Arc<RwLock<T>>) -> Self {
        Self(inner)
    }

    pub fn read(&self) -> RwLockReadGuard<'_, T> {
        self.0.read()
    }
}

pub struct LocalInstanceGuard {
    pub table_id: TableId,
    pub instance_id: LocalInstanceId,
    // Only send destroy event when event_sender when is_some
    event_sender: Option<HummockEventSender>,
}

impl Drop for LocalInstanceGuard {
    fn drop(&mut self) {
        if let Some(sender) = self.event_sender.take() {
            // If sending fails, it means that event_handler and event_channel have been destroyed, no
            // need to handle failure
            sender
                .send(HummockEvent::DestroyReadVersion {
                    instance_id: self.instance_id,
                })
                .unwrap_or_else(|err| {
                    tracing::debug!(
                        error = %err.as_report(),
                        table_id = %self.table_id,
                        instance_id = self.instance_id,
                        "LocalInstanceGuard Drop SendError",
                    )
                })
        }
    }
}