Files
adler32
ahash
aho_corasick
ansi_term
antidote
anyhow
arc_swap
arrayvec
async_speed_limit
async_stream
async_stream_impl
async_trait
atty
aws
backtrace
backtrace_sys
backup
base64
batch_system
bitfield
bitflags
block_buffer
boolinator
bstr
byteorder
bytes
bzip2_sys
cargo_metadata
cdc
cfg_if
chrono
chrono_tz
clap
cloud
codec
collections
concurrency_manager
configuration
configuration_derive
const_fn
const_random
const_random_macro
coprocessor_plugin_api
cpuid_bool
crc32fast
crc64fast
crossbeam
crossbeam_channel
crossbeam_deque
crossbeam_epoch
crossbeam_queue
crossbeam_skiplist
crossbeam_utils
crypto_mac
darling
darling_core
darling_macro
dashmap
debugid
derive_more
digest
dirs
dirs_sys
doc_comment
dtoa
either
encoding_rs
encryption
encryption_export
engine_panic
engine_rocks
engine_test
engine_traits
engine_traits_tests
error_code
error_code_gen
example_plugin
external_storage
external_storage_export
fail
failure
failure_derive
farmhash
file_system
filetime
flate2
fnv
foreign_types
foreign_types_shared
fs2
futures
futures_channel
futures_core
futures_executor
futures_io
futures_macro
futures_sink
futures_task
futures_timer
futures_util
async_await
compat
future
io
lock
sink
stream
task
fuzz
fxhash
gcp
generic_array
getrandom
getset
grpcio
grpcio_health
grpcio_sys
h2
heck
hex
hmac
http
http_body
httparse
httpdate
hyper
hyper_openssl
hyper_tls
ident_case
idna
indexmap
inferno
inotify
inotify_sys
instant
into_other
iovec
ipnet
ipnetwork
itertools
itoa
keys
kvproto
lazy_static
lazycell
libc
libflate
libflate_lz77
libloading
librocksdb_sys
libtitan_sys
libz_sys
linked_hash_map
linked_hash_set
lock_api
log
log_wrappers
lz4_sys
match_template
matches
md5
memchr
memmap
memoffset
memory_trace_macros
mime
mime_guess
mio
mio_extras
mio_uds
more_asserts
murmur3
native_tls
net2
nix
nodrop
nom
notify
num
num_complex
num_cpus
num_derive
num_format
num_integer
num_iter
num_rational
num_traits
once_cell
opaque_debug
openssl
openssl_probe
openssl_sys
ordered_float
panic_hook
parking_lot
parking_lot_core
paste
paste_impl
pd_client
percent_encoding
pest
pin_project
pin_project_lite
pin_utils
pnet_base
pnet_datalink
pnet_sys
pprof
ppv_lite86
proc_macro2
proc_macro_error
proc_macro_error_attr
proc_macro_hack
proc_macro_nested
procfs
procinfo
profiler
prometheus
prometheus_static_metric
promptly
prost
prost_derive
protobuf
quick_xml
quote
raft
raft_engine
raft_log_engine
raft_proto
raftstore
rand
rand_chacha
rand_core
rand_isaac
rayon
rayon_core
regex
regex_automata
regex_syntax
remove_dir_all
reqwest
resolved_ts
rev_lines
rgb
ring
rle_decode_fast
rocksdb
rusoto_core
rusoto_credential
rusoto_kms
rusoto_s3
rusoto_signature
rusoto_sts
rustc_demangle
rustyline
ryu
safemem
same_file
scopeguard
security
semver
semver_parser
serde
serde_derive
serde_ignored
serde_json
serde_urlencoded
serde_with
serde_with_macros
server
sha2
shlex
signal
signal_hook_registry
slab
slog
slog_async
slog_derive
slog_global
slog_json
slog_term
smallvec
snappy_sys
socket2
spin
sst_importer
stable_deref_trait
standback
static_assertions
str_stack
strsim
structopt
structopt_derive
strum
strum_macros
subtle
symbolic_common
symbolic_demangle
syn
syn_mid
synstructure
sysinfo
take_mut
tame_gcs
tame_oauth
tempfile
term
test_backup
test_coprocessor
test_pd
test_raftstore
test_sst_importer
test_storage
test_util
textwrap
thiserror
thiserror_impl
thread_local
tidb_query_aggr
tidb_query_codegen
tidb_query_common
tidb_query_datatype
tidb_query_executors
tidb_query_expr
tikv
coprocessor
coprocessor_v2
import
server
storage
tikv_alloc
tikv_ctl
tikv_jemalloc_ctl
tikv_jemalloc_sys
tikv_jemallocator
tikv_kv
tikv_server
tikv_util
time
time_macros
time_macros_impl
tipb
tipb_helper
tokio
fs
future
io
loom
macros
net
park
process
runtime
signal
stream
sync
task
time
util
tokio_executor
tokio_macros
tokio_openssl
tokio_timer
tokio_tls
tokio_util
toml
tower_service
tracing
tracing_core
try_lock
twoway
twox_hash
txn_types
typenum
ucd_trie
unchecked_index
unicase
unicode_bidi
unicode_normalization
unicode_segmentation
unicode_width
unicode_xid
untrusted
url
utf8parse
uuid
vec_map
vlog
walkdir
want
xml
yatp
zeroize
zstd_sys
  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
// Copyright 2019 TiKV Project Authors. Licensed under Apache-2.0.

// Copyright 2015 The etcd Authors
//
// 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.

pub use super::read_only::{ReadOnlyOption, ReadState};
use super::util::NO_LIMIT;
use super::{
    errors::{Error, Result},
    INVALID_ID,
};

/// Config contains the parameters to start a raft.
#[derive(Clone)]
pub struct Config {
    /// The identity of the local raft. It cannot be 0, and must be unique in the group.
    pub id: u64,

    /// The number of node.tick invocations that must pass between
    /// elections. That is, if a follower does not receive any message from the
    /// leader of current term before ElectionTick has elapsed, it will become
    /// candidate and start an election. election_tick must be greater than
    /// HeartbeatTick. We suggest election_tick = 10 * HeartbeatTick to avoid
    /// unnecessary leader switching
    pub election_tick: usize,

    /// HeartbeatTick is the number of node.tick invocations that must pass between
    /// heartbeats. That is, a leader sends heartbeat messages to maintain its
    /// leadership every heartbeat ticks.
    pub heartbeat_tick: usize,

    /// Applied is the last applied index. It should only be set when restarting
    /// raft. raft will not return entries to the application smaller or equal to Applied.
    /// If Applied is unset when restarting, raft might return previous applied entries.
    /// This is a very application dependent configuration.
    pub applied: u64,

    /// Limit the max size of each append message. Smaller value lowers
    /// the raft recovery cost(initial probing and message lost during normal operation).
    /// On the other side, it might affect the throughput during normal replication.
    /// Note: math.MaxUusize64 for unlimited, 0 for at most one entry per message.
    pub max_size_per_msg: u64,

    /// Limit the max number of in-flight append messages during optimistic
    /// replication phase. The application transportation layer usually has its own sending
    /// buffer over TCP/UDP. Set to avoid overflowing that sending buffer.
    /// TODO: feedback to application to limit the proposal rate?
    pub max_inflight_msgs: usize,

    /// Specify if the leader should check quorum activity. Leader steps down when
    /// quorum is not active for an electionTimeout.
    pub check_quorum: bool,

    /// Enables the Pre-Vote algorithm described in raft thesis section
    /// 9.6. This prevents disruption when a node that has been partitioned away
    /// rejoins the cluster.
    pub pre_vote: bool,

    /// The range of election timeout. In some cases, we hope some nodes has less possibility
    /// to become leader. This configuration ensures that the randomized election_timeout
    /// will always be suit in [min_election_tick, max_election_tick).
    /// If it is 0, then election_tick will be chosen.
    pub min_election_tick: usize,

    /// If it is 0, then 2 * election_tick will be chosen.
    pub max_election_tick: usize,

    /// Choose the linearizability mode or the lease mode to read data. If you don’t care about the read consistency and want a higher read performance, you can use the lease mode.
    ///
    /// Setting this to `LeaseBased` requires `check_quorum = true`.
    pub read_only_option: ReadOnlyOption,

    /// Don't broadcast an empty raft entry to notify follower to commit an entry.
    /// This may make follower wait a longer time to apply an entry. This configuration
    /// May affect proposal forwarding and follower read.
    pub skip_bcast_commit: bool,

    /// Batches every append msg if any append msg already exists
    pub batch_append: bool,

    /// The election priority of this node.
    pub priority: u64,

    /// Specify maximum of uncommitted entry size.
    /// When this limit is reached, all proposals to append new log will be dropped
    pub max_uncommitted_size: u64,
}

impl Default for Config {
    fn default() -> Self {
        const HEARTBEAT_TICK: usize = 2;
        Self {
            id: 0,
            election_tick: HEARTBEAT_TICK * 10,
            heartbeat_tick: HEARTBEAT_TICK,
            applied: 0,
            max_size_per_msg: 0,
            max_inflight_msgs: 256,
            check_quorum: false,
            pre_vote: false,
            min_election_tick: 0,
            max_election_tick: 0,
            read_only_option: ReadOnlyOption::Safe,
            skip_bcast_commit: false,
            batch_append: false,
            priority: 0,
            max_uncommitted_size: NO_LIMIT,
        }
    }
}

impl Config {
    /// Creates a new config.
    pub fn new(id: u64) -> Self {
        Self {
            id,
            ..Self::default()
        }
    }

    /// The minimum number of ticks before an election.
    #[inline]
    pub fn min_election_tick(&self) -> usize {
        if self.min_election_tick == 0 {
            self.election_tick
        } else {
            self.min_election_tick
        }
    }

    /// The maximum number of ticks before an election.
    #[inline]
    pub fn max_election_tick(&self) -> usize {
        if self.max_election_tick == 0 {
            2 * self.election_tick
        } else {
            self.max_election_tick
        }
    }

    /// Runs validations against the config.
    pub fn validate(&self) -> Result<()> {
        if self.id == INVALID_ID {
            return Err(Error::ConfigInvalid("invalid node id".to_owned()));
        }

        if self.heartbeat_tick == 0 {
            return Err(Error::ConfigInvalid(
                "heartbeat tick must greater than 0".to_owned(),
            ));
        }

        if self.election_tick <= self.heartbeat_tick {
            return Err(Error::ConfigInvalid(
                "election tick must be greater than heartbeat tick".to_owned(),
            ));
        }

        let min_timeout = self.min_election_tick();
        let max_timeout = self.max_election_tick();
        if min_timeout < self.election_tick {
            return Err(Error::ConfigInvalid(format!(
                "min election tick {} must not be less than election_tick {}",
                min_timeout, self.election_tick
            )));
        }

        if min_timeout >= max_timeout {
            return Err(Error::ConfigInvalid(format!(
                "min election tick {} should be less than max election tick {}",
                min_timeout, max_timeout
            )));
        }

        if self.max_inflight_msgs == 0 {
            return Err(Error::ConfigInvalid(
                "max inflight messages must be greater than 0".to_owned(),
            ));
        }

        if self.read_only_option == ReadOnlyOption::LeaseBased && !self.check_quorum {
            return Err(Error::ConfigInvalid(
                "read_only_option == LeaseBased requires check_quorum == true".into(),
            ));
        }

        if self.max_uncommitted_size < self.max_size_per_msg {
            return Err(Error::ConfigInvalid(
                "max uncommitted size should greater than max_size_per_msg".to_owned(),
            ));
        }

        Ok(())
    }
}