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
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
// Copyright 2016 TiKV Project Authors. Licensed under Apache-2.0.

use std::error::Error as StdError;
use std::fmt::{self, Display, Formatter};
use std::sync::mpsc::Sender;

use thiserror::Error;

use engine_traits::{Engines, KvEngine, RaftEngine};
use file_system::{IOType, WithIOType};
use tikv_util::time::Duration;
use tikv_util::worker::{Runnable, RunnableWithTimer};
use tikv_util::{box_try, debug, error, warn};

use crate::store::{CasualMessage, CasualRouter};

const MAX_GC_REGION_BATCH: usize = 128;
const COMPACT_LOG_INTERVAL: Duration = Duration::from_secs(60);

pub enum Task {
    Gc {
        region_id: u64,
        start_idx: u64,
        end_idx: u64,
    },
    Purge,
}

impl Task {
    pub fn gc(region_id: u64, start: u64, end: u64) -> Self {
        Task::Gc {
            region_id,
            start_idx: start,
            end_idx: end,
        }
    }
}

impl Display for Task {
    fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result {
        match self {
            Task::Gc {
                region_id,
                start_idx,
                end_idx,
            } => write!(
                f,
                "GC Raft Logs [region: {}, from: {}, to: {}]",
                region_id, start_idx, end_idx
            ),
            Task::Purge => write!(f, "Purge Expired Files",),
        }
    }
}

#[derive(Debug, Error)]
enum Error {
    #[error("raftlog gc failed {0:?}")]
    Other(#[from] Box<dyn StdError + Sync + Send>),
}

pub struct Runner<EK: KvEngine, ER: RaftEngine, R: CasualRouter<EK>> {
    ch: R,
    tasks: Vec<Task>,
    engines: Engines<EK, ER>,
    gc_entries: Option<Sender<usize>>,
}

impl<EK: KvEngine, ER: RaftEngine, R: CasualRouter<EK>> Runner<EK, ER, R> {
    pub fn new(ch: R, engines: Engines<EK, ER>) -> Runner<EK, ER, R> {
        Runner {
            ch,
            engines,
            tasks: vec![],
            gc_entries: None,
        }
    }

    /// Does the GC job and returns the count of logs collected.
    fn gc_raft_log(
        &mut self,
        region_id: u64,
        start_idx: u64,
        end_idx: u64,
    ) -> Result<usize, Error> {
        let deleted = box_try!(self.engines.raft.gc(region_id, start_idx, end_idx));
        Ok(deleted)
    }

    fn report_collected(&self, collected: usize) {
        if let Some(ref ch) = self.gc_entries {
            ch.send(collected).unwrap();
        }
    }

    fn flush(&mut self) {
        // Sync wal of kv_db to make sure the data before apply_index has been persisted to disk.
        self.engines.kv.sync().unwrap_or_else(|e| {
            panic!("failed to sync kv_engine in raft_log_gc: {:?}", e);
        });
        let tasks = std::mem::take(&mut self.tasks);
        for t in tasks {
            match t {
                Task::Gc {
                    region_id,
                    start_idx,
                    end_idx,
                } => {
                    debug!("gc raft log"; "region_id" => region_id, "end_index" => end_idx);
                    match self.gc_raft_log(region_id, start_idx, end_idx) {
                        Err(e) => {
                            error!("failed to gc"; "region_id" => region_id, "err" => %e);
                            self.report_collected(0);
                        }
                        Ok(n) => {
                            debug!("gc log entries"; "region_id" => region_id, "entry_count" => n);
                            self.report_collected(n);
                        }
                    }
                }
                Task::Purge => {
                    let regions = match self.engines.raft.purge_expired_files() {
                        Ok(regions) => regions,
                        Err(e) => {
                            warn!("purge expired files"; "err" => %e);
                            return;
                        }
                    };
                    for region_id in regions {
                        let _ = self.ch.send(region_id, CasualMessage::ForceCompactRaftLogs);
                    }
                }
            }
        }
    }
}

impl<EK, ER, R> Runnable for Runner<EK, ER, R>
where
    EK: KvEngine,
    ER: RaftEngine,
    R: CasualRouter<EK>,
{
    type Task = Task;

    fn run(&mut self, task: Task) {
        let _io_type_guard = WithIOType::new(IOType::ForegroundWrite);
        self.tasks.push(task);
        if self.tasks.len() < MAX_GC_REGION_BATCH {
            return;
        }
        self.flush();
    }

    fn shutdown(&mut self) {
        self.flush();
    }
}

impl<EK, ER, R> RunnableWithTimer for Runner<EK, ER, R>
where
    EK: KvEngine,
    ER: RaftEngine,
    R: CasualRouter<EK>,
{
    fn on_timeout(&mut self) {
        self.flush();
    }

    fn get_interval(&self) -> Duration {
        COMPACT_LOG_INTERVAL
    }
}

#[cfg(test)]
mod tests {
    use super::*;
    use engine_traits::{KvEngine, Mutable, WriteBatch, WriteBatchExt, ALL_CFS, CF_DEFAULT};
    use std::sync::mpsc;
    use std::time::Duration;
    use tempfile::Builder;

    #[test]
    fn test_gc_raft_log() {
        let dir = Builder::new().prefix("gc-raft-log-test").tempdir().unwrap();
        let path_raft = dir.path().join("raft");
        let path_kv = dir.path().join("kv");
        let raft_db =
            engine_test::raft::new_engine(path_kv.to_str().unwrap(), None, CF_DEFAULT, None)
                .unwrap();
        let kv_db =
            engine_test::kv::new_engine(path_raft.to_str().unwrap(), None, ALL_CFS, None).unwrap();
        let engines = Engines::new(kv_db, raft_db.clone());

        let (tx, rx) = mpsc::channel();
        let (r, _) = mpsc::sync_channel(1);
        let mut runner = Runner {
            gc_entries: Some(tx),
            engines,
            ch: r,
            tasks: vec![],
        };

        // generate raft logs
        let region_id = 1;
        let mut raft_wb = raft_db.write_batch();
        for i in 0..100 {
            let k = keys::raft_log_key(region_id, i);
            raft_wb.put(&k, b"entry").unwrap();
        }
        raft_wb.write().unwrap();

        let tbls = vec![
            (Task::gc(region_id, 0, 10), 10, (0, 10), (10, 100)),
            (Task::gc(region_id, 0, 50), 40, (0, 50), (50, 100)),
            (Task::gc(region_id, 50, 50), 0, (0, 50), (50, 100)),
            (Task::gc(region_id, 50, 60), 10, (0, 60), (60, 100)),
        ];

        for (task, expected_collectd, not_exist_range, exist_range) in tbls {
            runner.run(task);
            runner.flush();
            let res = rx.recv_timeout(Duration::from_secs(3)).unwrap();
            assert_eq!(res, expected_collectd);
            raft_log_must_not_exist(&raft_db, 1, not_exist_range.0, not_exist_range.1);
            raft_log_must_exist(&raft_db, 1, exist_range.0, exist_range.1);
        }
    }

    fn raft_log_must_not_exist(
        raft_engine: &impl KvEngine,
        region_id: u64,
        start_idx: u64,
        end_idx: u64,
    ) {
        for i in start_idx..end_idx {
            let k = keys::raft_log_key(region_id, i);
            assert!(raft_engine.get_value(&k).unwrap().is_none());
        }
    }

    fn raft_log_must_exist(
        raft_engine: &impl KvEngine,
        region_id: u64,
        start_idx: u64,
        end_idx: u64,
    ) {
        for i in start_idx..end_idx {
            let k = keys::raft_log_key(region_id, i);
            assert!(raft_engine.get_value(&k).unwrap().is_some());
        }
    }
}