etcd-io / raft Goto Github PK
View Code? Open in Web Editor NEWRaft library for maintaining a replicated state machine
License: Apache License 2.0
Raft library for maintaining a replicated state machine
License: Apache License 2.0
This issue describes examples of limited control of the data flows in raft
package, and what this can lead to. This motivates improvements in this area. The umbrella issue for this is #64.
Consider a general class of systems1 for which a single server can host multiple/many instances of raft
Node / RawNode. Say, in a cluster of O(N) servers each server is hosting O(K) instances of raft.RawNode
. K can be in [tens of] thousands. RawNode
s of a single raft
group (typically, 3x or 5x replicas per group) are distributed across the cluster arbitrarily, e.g. one group can be on servers {0, 1, 2}
, another group on servers {0, 10, 57}
, etc.
In the picture below, we first consider the MsgApp
data flows from raft
leaders on server 0 to followers on other N-1 servers.
In the current design of raft
, each RawNode
acts independently of others, and makes local decisions on when to send a MsgApp
to a follower, based on merely the fact that the follower is behind on the log. The RawNode
does this as soon as possible (at any Step/Ready
iteration), and the application/server layer has little control of this behaviour. The RawNode
drives the process, and the application has to adapt.
Such design is prone to a bunch of overload scenarios.
In a loaded system, followers are always behind by a little, and sometimes by much (e.g. after a server crash/restart). As a result, many RawNode
s always independently decide to send a MsgApp
to their followers. Each RawNode
fetches a few entries, constructs a MsgApp
, and sends to the application layer via the Ready
struct.
Server 0 has limited capacity. If there are too many messages, server 0 can crash because of reaching an out-of-memory situation.
To prevent this, the application layer has a few options, which can be categorised as “workarounds”:
RawNode
s. This will indirectly cause RawNode
to send fewer MsgApp
traffic to the followers. [but only if most followers are up-to-date; otherwise, RawNode
will still push new messages to catch up slow followers]Ready
calls on the RawNode
s. However, RawNode
controls other critical components of raft protocol, such as heartbeats. Calling Ready
regularly is a necessary part of the API.Ready
tries to push if the overall volume approaches limits. This will cause RawNode
to retry sending these message later. However, we are still paying the cost of fetching the unnecessary entries from storage, and constructing the messages. The most unfortunate scenarios still can OOM, when many of these unnecessary message constructions happen simultaneously.raft
's built-in pacing mechanisms, such as max-in-flight bytes. These only work well when there is a single RawNode
per server. The overall volume of messages can still be excessive when there are tens of thousands. Setting these static limits to low values, like O(memory/K), artificially reduces the cluster throughput. Ideally, the flow needs to be controlled dynamically.Consider the symmetrical picture from the followers point of view. A single server hosts up to K RawNode
s. The leaders for these nodes are distributed across the cluster, and can send MsgApp
flows to followers hosted on server 0 independently.
Server 0 has limited capacity. Similarly, it is prone to overload if many leader RawNode
s independently decide to send some messages to server 0 (an example of such a high fan-in is when the server has been down for some non-trivial amount of time).
To protect from this, server 0 has fewer options:
MsgApp
messages, so that they don't consume resources on the receiver server. This is similar to option (3) for the leader nodes, except the cost of it is higher: the message has been fetched from the leader's storage and travelled across the network. This will result in unnecessary retries and more fetches from storage on the leader.To improve on the scenarios above, there needs to be a more general/flexible mechanism for flow control / back-pressure in raft
. It should be possible to slow down MsgApp
flows from RawNode
to the hosting server/application, and from an individual leader RawNode
to an individual follower RawNode
.
There are multiple ways to achieve this, on a spectrum between:
raft
. During the Ready
exchange, raft
would get some signals from the application, and pace the MsgApp
flows accordingly.raft
provides assistance for doing this correctly.There might be some quick wins in approach (1), but in the long term approach (2) seems more appropriate. Approach (2) makes raft
package simpler and more maintainable. Currently, raft
is a mixture of the core protocol and control mechanisms, and the latter can be factored out.
To implement approach (2), raft
needs to export/delegate the notion of "leader->follower flow" in such a way that the application layer can drive it. It would be similar to the Progress tracker that raft already uses internally, and exposes for informational purposes. The application would use the "progress" information to decide when and how much to send to a follower, and notify raft
to update the progress accordingly. This would be a shared data structure.
Currently, a recent portion of the raft log is stored in the unstable in-memory structure, and the rest of the log resides in the application-specific Storage implementation. Raft abstracts the whole log (which fetches from memory or storage) behind its internal raftLog wrapper. Some of the capabilities of this wrapper would need to be accessible to the application layer, in order to construct messages.
The use of such new capabilities would not be limited to leader->follower MsgApp
flows. The asynchronous log writes API [introduced here] was partly necessitated by the limitations of the current raft
design too. The design improvements considered here would make raft
asynchronous "from the box" and alleviate the need to make its API complex to support multiple modes of operation. I.e. instead of implementing workarounds for each new use-case, it is "cheaper" to expose a flexible API.
such as CockroachDB; or etcd if there is any "multi-tenancy" in the way it can be hosted - e.g. many etcd processes per VM. ↩
When entries exit the unstable
log structure
Line 156 in 3e6cb62
it shrinks itself based on the entries slice length and cap:
Lines 162 to 179 in 3e6cb62
Firstly, the len(u.entries)*lenMultiple < cap(u.entries)
does not necessarily do a right thing. After shifting the slice start in u.entries = u.entries[num:]
, the first num
entries "disappear" from len
and cap
of this slice: https://go.dev/play/p/ao1xaL0Edla.
So it's possible that len >= cap/2
all the time, even though only a small portion of the backing slice is used. We should take the cap
of the entire backing slice into account instead.
Secondly, this heuristic only takes len/cap
of the slice into consideration, but not the size of the entries referenced by it. It could be beneficial to, in addition, shrink the slice if the sum size of the "garbage" entries is more than a half. This would keep the overall memory consumption more controlled. Doing so would require maintaining a running sum of entry sizes in the unstable
struct (which we do anyway in other places for rate limiting purposes).
The same heuristic could be applied in MemoryStorage.Compact method to smooth out the allocation/copying cost of log truncations. Frequent truncations may incur a quadratic cost here, while the heuristic allows capping it at O(N).
This is an umbrella issue for all kinds of unit-test improvements. Unit-tests in this repository are significantly outdated, and need a revamp. For example:
Examples of things to fix/improve:
RawNode / raft
structs arbitrarily, which breaks internal invariants and makes tests prone to false failures on any change. This necessitates PR authors to spend time debugging and fixing incorrect tests.require/assert
packages. #146 (comment)r.Term >= r.raftLog.lastTerm()
. Tests that manipulate RawNode
structs directly may accidentally break invariants, and end up testing arbitrary behaviour rather than real one.Currently, a raft
node has a Term
field (and its storage counterpart in HardState
) which is the latest term seen by this node.
raftLog
, correspondingly, may or may not be consistent with the leader's log at this term, and the leader's log might not exist in the first place (if the election at this term does not win).This uncertainty is a source of workarounds and potential safety bugs, and is generally confusing. This also leads to unnecessary message rejects/drops in periods of leader change. To combat this, let's try to bring some structure, and then improve the codebase.
TODO: add action items.
TODO: add code links.
TODO: keep writing, this is a WIP text.
TODO: convert this to a tech note / design doc.
Every leader in raft
has a unique term. During election, a prospective leader (candidate) negotiates a starting point (prevIndex, prevTerm)
, and then appends a contiguous run of entries at index prevIndex+1
and higher. The set of all entries in the system thus forms a tree. Raft implements a bunch of safety rules that govern how the (prevIndex, prevTerm)
point is chosen (it has to be in the subtree of the "committed" point in this tree), and when entries are considered committed (a quorum of replicas are in the same subtree).
Consider an example of such a "tree" representing the log evolution:
term
▲
│
│ ┌→5 5 5
│ │
│ ┌→4 4
│ │
│ ┌→3 3 3 3 3 3 3 3 3 3 3 3
│ │
│ │ ┌→2 2 2 2 2 2
│ │ │
│ 1 1 1 1 1 1
└──────────────────────────────────────────► index
Every node in the raft
group is at some "point" in this tree. A node's log is the unique path in this tree from (0, 0)
to entry (term, index)
at the "tip" of the log.
The "leader log" is the path from (0, 0)
to the latest entry that this leader appended. For the picture above, here are the "leader logs" corresponding to the leader term:
t1: 1 1 1 1 1 1
t2: 1 1 1 2 2 2 2 2 2
t3: 1 1 3 3 3 3 3 3 3 3 3 3 3 3
t4: 1 1 3 3 3 4 4
t5: 1 1 3 3 3 4 4 5 5 5
Note that:
t1
has term 1, while the leader t4
observes an entry with term 3.t3
, t4
, and t5
logs.We have now established that there are two ways to identify entries in the system:
(term, index)
, where term
identifies the leader who created this entry.(term, index)
where term
identifies the leader who observes this entry in its log.Different parts of raft
codebase use one of these two ways, and there is no clear distinction between these 2 semantics. Some code uses just index
and implicitly assumes the term
part to be either the original entry term, or the leader log term. Such implicit assumptions complicate understanding, and may lead to correctness bugs. We may need some type safety to communicate these semantics in code.
Notably, the raft.Term
field by default has no straightforward relation to term
in (1) and (2). In the Leader
state, it can be used with either definition; in the Candidate
state, raft.Term
is a number unrelated to any entries in the log; in the Follower
state, raft.Term
first has no relation to the log, but eventually becomes in sync with the log (when the first append message from the leader succeeds) and can be used with (2).
This suggests that raftLog
data structure (which is agnostic to the node state and election) should track a more straightforward Term
that can be used in coordinate system (2) unconditionally. A sensible approach is to say that raftLog.Term
is the term of the leader whose append we accepted last. This is equivalent to behaviour of Paxos acceptors in phase one - the acceptor keeps track of the term for the latest accepted proposal. In raft
world, the raftLog
is the equivalent of a Paxos acceptor.
Currently, the raft.Term
field plays two roles:
raft.Term >= raftLog.Term
at all times.(2) makes it safe to use raft.Term
as a substitute for the raftLog.Term
for some safety checks. For example: the raftLog
must reject appends coming from leaders < raftLog.Term
. However, we currently reject terms < raft.Term
, which is a superset of < raftLog.Term
. We may reject appends unnecessarily (when raft.Term > raftLog.Term
), but we will never accept an append incorrectly, so this is safe.
TODO: keep writing
Plan to formally release raft 3.6.0 sometime in the following 1 ~ 2 months, so that users can depend on a tagged version instead of a commit.
Please advise what features should be included in raft 3.6.0.
We also need to cut a branch something like release-3.6 (raft 3.6.0 should be released based on this branch), and it only accepts bug fixes afterwards. Any new features should only go into the main branch after we release 3.6.0.
raft
keeps entries in memory as a single slice:
Line 37 in 3e6cb62
In the happy path (when the leader is stable), it just keeps appending to this slice:
Line 201 in 3e6cb62
In the unhappy path (when leader changes), it rewrites a suffix of the log by copying a prefix and appending the suffix:
Lines 212 to 213 in 3e6cb62
As a result, we again have a contiguous slice of entries. We do this copying because we need entries[i]
to be immutable (because it may have been sent up to Storage
through the Ready
struct, and is still in use / in flight).
Same reallocation/copying happens in MemoryStorage
:
Lines 300 to 302 in 3e6cb62
Another place where we have to reallocate/copy the slice of entries is the slice
method where we combine the entries returned from Storage
with the entries returned from the unstable
structure:
Lines 494 to 497 in 3e6cb62
In all these cases, it is not strictly necessary to reallocate and copy []pb.Entry
slices. Instead, it would be acceptable to operate on [][]pb.Entry
, or a struct that has [][]pb.Entry
inside, but provides an interface of a contiguous slice.
Most usages of []pb.Entry
need to be able to iterate it, so it's fine if it's multiple slices instead of one. Some usages may need random access by entry.Index
, so it's fine if there are a few slices, and it's good if access by Index
is safely encapsulated.
Prototype:
// Entries represents a span of the log. Functionally it is
// equivalent to []pb.Entry.
//
// Internally, it consists of a few []pb.Entry slices, ordered
// by its 0-th entry Index. By properties of the Raft log, it is
// also ordered by the 0-th entry Term. Example:
//
// Entries: (-----------------------]
// . . . . .
// t9: │ . . . (-------]
// t7: │ . . (---]-------]
// t4: │ . (---]
// t3: │ (-----------------]
// └───────────────────────────────────► index
// 10 20 30 40 50 60 70
//
// Most of the time, there will be one (when leader is stable) or
// two (when leader changes) slices in it. The struct is optimized
// for this case, and incurs zero allocations. Occasionally it may
// accumulate more slices (e.g. if the leader is unstable).
type Entries struct {
buf [3][]pb.Entry // used to back `ent` if len(ent) <= 3
ent [][]pb.Entry // Invariant: len(ent[i]) != 0
}
// Bounds returns the [begin, end) range of log indices that this
// slice represents.
func (e Entries) Bounds() (uint64, uint64) {
if len(e.ent) == 0 {
return 0, 0
}
last := e.ent[len(e.ent)-1]
return e.ent[0][0].Index, last[len(last)-1].Index+1
}
// Len returns the length of the log span.
func (e Entries) Len() int {
begin, end := e.Bounds()
return end - begin
}
// Iterate visits all entries in this log span.
func (e Entries) Iterate(visit func([]pb.Entry) bool) {
l := len(e.ent)
if l == 0 {
return
}
for i := 0; i + 1 < l; i++ {
// Compute the visible part of this slice, which is not
// overlapped by the next slice.
visible := e.ent[i+1].Index - e.ent[i].Index
if !visit(e.ent[i][:visible:visible]) { // protect from appends
return
}
}
visit(e.ent[l-1])
}
// To converts Entries to the equivalent []pb.Entry.
// Just in case it's needed. For instance:
//
// entries := e.To(make([]pb.Entry, 0, e.Len()))
func (e Entries) To(to []pb.Entry) []pb.Entry {
e.Iterate(func(part []pb.Entry) bool {
to = append(to, part...)
return true
})
return to
}
// Need to also have mutators to append to and truncate this slice.
// Could also add a method to access a log entry by Index. This would
// be a binary search by e.ent[i][0].Index (or a linear search if
// len(e.ent) is low, which is normally true), followed by a direct
// access by index within the found sub-slice.
// Could also add a method to access a [begin, end) sub-slice of this
// slice. It would boil down to finding the begin and end-1 index, and
// then either returning a sub-slice with zero copy if it's in a single
// sub-slice, or returning a copy if it is compound.
// Squash can be used to "compact" the log range to a single slice.
// For example, it could be used in MemoryStorage occasionally, once
// len(e.ent) grows above a threshold and increases the cost of random
// access.
func (e Entries) Squash() Entries {
if len(e.ent) <= 1 {
return e
}
var res Entries
res.buf[0] = e.To(make([]pb.Entry, 0, e.Len()))
res.ent = res.buf[:1]
return res
}
Why something like this struct is useful:
unstable
struct, the overlapped entries on leader changes are short-lived, and will be released to the heap soon anyway. The overlapped entries are also likely still referenced because they were returned through Ready
for being stored. So copying the slice (as of now) actually increases memory consumption.Index
is contiguous, and Term
is non-decreasing.https://github.com/etcd-io/raft/blob/7357439323abcd0654b7146022f7a58793d667eb/go.mod#LL7C43-L7C57
This is using an old version of cockroachdb/datadriven@c9e8355
Would you upgrade to v1.0.0 or later?
follow up of kubernetes/kubernetes#114912.
I wonder how etcd raft prevote make sure that at least one preCandidate will become Candidate? When I try to implement raft. I find that one case will deadlock. 5 nodes{1, 2, 3, 4, 5}. 3 become leader. node 2, node 1 can conneted the leader, but others can't. node 3 proposes one confchange(remove 3). At last, I find the node states{ node 1 (term 6, vote 3, latest log), node 2 (term 6, vote 3 latest log) node 4 (term 7 vote 5) node 5(term 7 vote 5)}.They are deadlock. Wonder how etcd raft solve this.
The gap is yet to be confirmed, it's unclear if it can happen. Discovered while cleaning up the raftLog/unstable
structures.
The unstable
structure tracks log entries and a snapshot in flight to storage.
The undocumented invariant seems to be: if there is a snapshot
, then the offset == snapshot.Metadata.Index + 1
. The underlying sense of this invariant is that the snapshot is a "wide entry" at index offset - 1
that immediately precedes all the entries
. Correspondingly, there is a requirement to write this snapshot
to storage earlier than the entries.
We maintain this invariant during the lifecycle of unstable
, and rely on it when extracting the log size, and terms of entries at specific indices.
However, there is one place where the invariant can be violated:
Lines 202 to 208 in 073f90d
When fromIndex < u.offset
(strictly less), we don't erase u.snapshot
, and as a result it's possible that u.offset < u.snapshot.Metadata.Index + 1
. It's also possible that the snapshot index exceeds lastIndex()
.
It's unclear if the above situation can happen, and what is the consequence. Best case: it can't happen because the layers above won't submit an append at index below the unstable.snapshot
. Worst case: it can happen, and things can be written or committed out of order.
In any case, we need to straighten invariants here, and fix this code to be safer.
To be investigated.
Sorry to bother.
While reviewing the code, I noticed that there is no check for whether a follower has already voted for another candidate during the transition to becoming a candidate. However, according to the second rule for Followers in the Rules for Servers section of the Raft paper, such a check should be performed. If possible, I would like to understand the reason for omitting this check in this particular implementation.
The original text of the paper is as follows:
Followers (§5.2):
• Respond to RPCs from candidates and leaders
• If election timeout elapses without receiving AppendEntries
RPC from current leader or granting vote to candidate:
convert to candidate
The Raft library provides the parameter DisableProposalForwarding. When the flag is true, it doesn’t forward MsgProp type messages from follower to leader. I guess providing a similar option for each Message
might be valuable. It’s great if I can get comments on this idea.
Context: a program which uses the Raft library can have asynchronous processes which can issue Raft request from a server process. In the case of etcd, lease and compaction are typical examples. I guess other users of the library might have similar processes.
Such an asynchronous process can be implemented a goroutine whose behavior depends a condition that its node is leader or not. If the node is a leader, the goroutine issues Raft messages asynchronously (and periodically).
This behavior might be problematic in some cases. If the goroutine can be paused by various reasons (high load on CPU, disk I/O, etc), the node can be a follower by a new leader election. The problem is that the goroutine can behave based on a stale information that the node is still a leader. In such a case, the goroutine can issue Raft requests because it thinks that it’s still a leader. If the messages shouldn’t be duplicated, it might be harmful. Otherwise it will be problematic. In the case of etcd, it can cause lease revoking from a stale leader: etcd-io/etcd#15247
Note that this problem should happen only if the Raft library logic recognizes itself as a follower:
MsgProp
messages will be handled by the stale node itself and result MsgApp
. In this case other nodes can reject the message because these messages have an old term.MsgProp
will be forwarded to a new leader and the new leader will send MsgApp
. Although the original source of the messages is the stale leader, the new cluster can accept the messages.(the above behavior is quite subtle and I'm still checking it, I'm glad if I can get feedback on it too)
I guess setting DisableProposalForwarding
true will be a simple solution for avoiding this situation. However, if a program which uses the Raft library doesn’t provide a client side mechanism of selecting a leader and sending messages to it (e.g. etcd clientv3), the parameter will make the program not functional because it affects all messages. So I think it’s nice if the Raft library can provide a mechanism to disable proposal forwarding only for specific message types.
There might be some possible approaches:
Message
: it’s simple but be too large change.I’d like to know other people’s opinions and how other programs which use the Raft library deal with this kind of issue.
cc @ahrtr @serathius @tbg @pavelkalinnikov
There are a few allocations in the hot path, for example
Line 548 in 228ee70
which would be nice to pool. One way we could to this is by introducing something like this:
var msgsAfterAppendPool = sync.Pool{
New: func() interface{} {
sl := make([]pb.Message, 0, 8)
return &sl
},
}
func PutMsgsAfterAppend(sl *[]pb.Message) {
for i := range *sl {
(*sl)[i] = pb.Message{}
}
msgsAfterAppendPool.Put(sl)
}
// send schedules persisting state to a stable storage and AFTER that
// sending the message (as part of next Ready message processing).
func (r *raft) send(m pb.Message) {
@@ -545,6 +559,9 @@ func (r *raft) send(m pb.Message) {
// because the safety of such behavior has not been formally verified,
// we err on the side of safety and omit a `&& !m.Reject` condition
// above.
+ if cap(r.msgsAfterAppend) == 0 {
+ r.msgsAfterAppend = *msgsAfterAppendPool.Get().(*[]pb.Message)
+ }
r.msgsAfterAppend = append(r.msgsAfterAppend, m)
} else {
if m.To == r.id {
This is discussed a little bit in #14 (comment).
This is in the hot path (called from newReady
). We shouldn't allocate here. Instead, return a SoftState
and make sure it doesn't escape to the heap at the callers. The improvement is ideally visible in a benchmark.
This is responsible for around 5% of the allocation count on a CockroachDB single-voter replica under a write-only workload.
The test added in #137 demonstrates that a short network blip / message drop can lead to delaying the follower to learn that entries in its log are committed up to HeartbeatInterval + 3/2 * RTT
. This is suboptimal, and can be reduced to HeartbeatInterval + 1/2 * RTT
.
The reason why it takes an extra roundtrip is that the leader cuts the commit index sent to the follower at Progress.Match
, i.e. it never sends a commit index exceeding the known match size of the follower's log. This is to ensure that the follower does not crash at receiving a commit index greater than its log size.
However, the follower can safely process out-of-bounds commit indices by simply (with a caveat: see below) cutting them at the log size on its end. In fact, the follower already does so for the commit indices it receives via MsgApp
messages. When sending MsgApp
, the leader does not cut the commit index, correspondingly.
Fixing this is a matter of:
(2) is true after the follower received at least one successful MsgApp
from this leader. From that point, the follower's log is guaranteed to be a prefix of the leader's log, and it is safe to assume that any index <= Commit
is committed.
However, simply doing (1) is unsafe in mixed-version clusters (during a rolling upgrade). If there are followers running old code, they will crash upon seeing a high commit index. To fix the problem completely, there must be a safe migration. Only after all the nodes are running the new code, it is safe to do (1).
The same delay reduction can be achieved if the leader sends an empty MsgApp
after the HeartbeatInterval
. We already have periodic empty MsgApp
sends in a throttled state, although they are currently hooked in to MsgHeartbeatResp
messages (to ensure MsgApp
flow is conditional to some connectivity with the follower), which in this case will result in the same +1 RTT
delay.
I'd like to nominate @pavelkalinnikov as a reviewer (in the sense of the etcd community guidelines1) for etcd-io/raft starting in mid-July 20232
As of mid-April, Pavel is recognized as an etcd/etcd-io member owing to his contributions to etcd-io/raft. As you can see from the references within and his activity since, Pavel has been very active on the raft codebase, fixing bugs, adding new functionality and documentation, and is starting to reflect on the future direction of the library as well (e.g. 3). As Pavel's co-worker on the Replication team at Cockroach Labs, I can attest to his intimacy with both consensus algorithms and their real-world deployment at scale and further foster his involvement with our community.
cc @ahrtr @mitake @ptabor @serathius @spzala
2023-05-31: edited to reflect outcome of discussion among maintainers about how to apply the community guidelines1
The gogoproto project is deprecated.
I noticed that there is an issue in the etcd repo but wanted to raise this here as well now that raft is in its own repo.
It seems like the consensus was to move to google.golang.org/protobuf
Please refer to tests.yaml#L36-L41 to enhance the test workflow for raft
The Raft algorithm requires an odd number of servers to maintain a quorum, meaning a minimum of three for a single point of failure. This isn't an issue for large systems but can be challenging for budget-limited customers needing fewer servers 1.
Efforts have been made in both scholarly circles 23 and commercial sectors 456 to resolve this issue. However, all existing research and implementations for small scale clusters (with two servers for example) either depend on another HA solution or necessitate a standalone server as a witness, adding to deployment complexity and potential performance bottlenecks.
I hereby propose extended Raft algorithm, a variant of Raft algorithm, which is designed for clusters with regular servers and a single witness, minimizing data traffic and access to witness while maintaining all key Raft properties. The witness in this algorithm is very suitable for implementation as a storage object with various options such as NFS, SMB, or cloud storage.
The extended Raft algorithm is backward compatible with Raft, meaning any cluster running with Raft can be seamlessly upgraded to support witness.
The correctness of the algorithm has been conclusively proven through a formal proof in https://github.com/joshuazh-x/extended-raft-paper. Besides that, we also validate the formal specification of the algorithm using TLC model checker.
Look forward to your suggestions and feedback.
@pav-kv @serathius @ahrtr @tbg @Elbehery @erikgrinaker @lemmy
https://github.com/etcd-io/etcd/issues/8934#issuecomment-398175955 ↩
Pâris, Jehan-François, and Darrell DE Long. "Pirogue, a lighter dynamic version of the Raft distributed consensus algorithm." 2015 IEEE 34th International Performance Computing and Communications Conference (IPCCC). IEEE, 2015. ↩
Yadav, Ritwik, and Anirban Rahut. "FlexiRaft: Flexible Quorums with Raft." The Conference on Innovative Data Systems Research (CIDR). 2023. ↩
https://platform9.com/blog/transforming-the-edge-platform9-introduces-highly-available-2-node-kubernetes-cluster/ ↩
https://www.spectrocloud.com/blog/two-node-edge-kubernetes-clusters-for-ha-and-cost-savings ↩
stepWait
was implemented to fail-fast in Propose method.
But it was never used in ProposeConfChange method.
Question was raised about it on original PR.
Also, it's was causing timeout in ExampleCluster_memberAddAsLearner test. comment
This isn't a critical issue, because in production ProposeConfChange isn't usually called consecutively. But it seams to me that stepWait
was missed for ProposeConfChange
method.
Append messages [MsgApp] that leader sends to a follower mainly serve 3 purposes:
Upon a successful MsgApp
processing, the follower sends a MsgAppResp
back to the leader, confirming the new size of its log. Note that it does not confirm the update of the commit index.
The follower can be completely up-to-date (both the log and the commit index), but the leader can still send empty MsgApp
for purpose (2), to update the commit index (e.g. here). The conditions for sending these empty messages are brittle and probabilistic.
Symmetrically, sometimes the leader does not choose to update the follower's commit index even though the latter is behind. The leader does not have precise understanding of where the follower's commit index is.
It would be good to have a simpler condition for sending commit index update messages, such that it is hard to get wrong and doesn't require guessing the follower's state. This will lead to: a) fewer unnecessary MsgApp
messages, b) faster convergence of the follower's commit index in some cases.
Observe that it is only necessary to send a commit index update to a follower if the follower's commit index is behind. This is because the commit index never regresses (it is stored in HardState
).
TODO: need to double-check whether storing commit index is synced before we send a reply. If not, the solution here is not completely working, and still relies on heartbeats to fill the gaps. To make it work, we should be sending the HardState.Commit
in the message, rather than raftLog.commit
.
The conditions for sending an empty MsgApp
with a commit index update will become simpler if the leader tracks the follower's commit index (similarly to how it tracks the follower's log size for deciding when to send new log entries).
To support this, the follower should notify the leader of the Commit
index. It should be as simple as setting the Commit
field of the MsgAppResp
message that it sends to the leader.
In confchange/confchange.go
, if !joint(cfg)
and if len(outgoing(cfg.Voters)) == 0
is the same
func (c Changer) LeaveJoint() (tracker.Config, tracker.ProgressMap, error) {
cfg, prs, err := c.checkAndCopy()
if err != nil {
return c.err(err)
}
if !joint(cfg) {
err := errors.New("can't leave a non-joint config")
return c.err(err)
}
if len(outgoing(cfg.Voters)) == 0 {
err := fmt.Errorf("configuration is not joint: %v", cfg)
return c.err(err)
}
The etcd example is too dependent and complex. This raft library stands alone and is difficult to adopt widely if it still relies on etcd.
See etcd-io/etcd#14627 (comment).
In becomeLeader
, we call r.reset()
which resets the uncommitted log size. This means that in theory the uncommitted log size isn't bounded:
We could either
Look at the following piece of code, the test at line will make raft send at most maxMsgSize
bytes of entries when raft is in probe state, ie. pr.State != tracker.StateProbe
:
Lines 565 to 585 in 3e6cb62
I have two questions:
maxMsgSize
bytes of entries when raft is in probe state)?pr.State != tracker.StateProbe
? Since in a probe state it's very likely for a append message to be rejected, sending just one or zero entry might accelerate the probe process.@ahrtr @pavelkalinnikov
This issue states a problem, and introduces a high-level idea on solving this and similar problems. This is not a ready design yet, more like a conversation starter.
Currently, the raft
package takes an active role in managing the flow of log entries in and out of Storage
, and pushing them from leader to followers. There is an argument for shifting the flow control responsibility to the application layer, and reducing raft
's responsibility to mostly managing correctness (i.e. make it more a passive "library" than an active "framework").
For example, currently, once an entry is committed, raft
:
Storage
(+unstable
)Ready
struct to the applicationReady
iteration (or a few iterations, in case of async storage writes)Since raft library is not fully aware of the application layer's resource allocation strategy or the semantics of the commands in the log, it sometimes may push too much work through the Ready
struct. The static "max bytes" policy is somewhat helpful in this regard, however in more complex setups it still does not suffice. For example, in CockroachDB one node may host and schedule tens/hundreds of thousands of raft
instances, and if many instances push many entries simultaneously, an out-of-memory situation may occur.
This necessitates introducing more ad-hoc back pressure / flow control mechanisms into raft
to constrain this flow dynamically. The mechanisms should be flexible enough to be used by many applications (such as etcd
and CockroachDB
). Generally, these mechanisms are two-fold: a) introspection into raft
state before it pushes more work to the application, so that the application can preallocate and/or signal raft
to slow down before it's too late; b) the feedback mechanism that signals raft
to slow down (e.g. see #60).
As another example, the MsgApp
flow from leader to a follower is driven by raft
too. There are (mostly) two states in which a flow can be: StateProbe and StateReplicate. In overflow situations, the application layer has no option other than dropping the messages, which eventually causes raft
to retry sending the same entry appends. It is currently impossible to ask raft
to gracefully slow down instead.
The above are examples of somewhat ad-hoc flow control mechanisms that we currently have or could introduce to workaround the resource overuse issues. For a holistic control, every link in the pipeline requires such a mechanism integrated into the raft
library. This enlarges the API surface and implementation complexity, is error-prone, and not necessarily solves the problems optimally for all raft
users.
For best control, the responsibility could be shifted to the users. For example, instead of fetching entries and pushing them to the application (+providing introspection and back pressure knobs), raft
could simply:
raft
algorithm correctness)A backwards compatibility consideration should be taken into account. There are applications already relying on flow control mechanisms currently built-in to raft
.
I believe @erikgrinaker meets all the requirement of a member (The only exception might be subscribing to etcd-dev, please ping me back when you finish subscribing the group @erikgrinaker).
We currently validate configuration changes at proposal time
Lines 1224 to 1260 in 4abd9e9
This is not very helpful because it has false positives (refusing a config change that is actually allowed) though at least it currently doesn't have false negatives (because the set of false positives is sufficiently large 😄)
It could be more effective to either compare against the actual most recent config change in the (including the unstable) log, or to move the verification to apply time (i.e. erroring out conf changes that are illegal).
See #81.
Related to #11.
In becomeLeader
, we have the following logic:
Lines 792 to 796 in 91eb751
This is a no-op. The payload size of emptyEnt
is 0, so there's no need to subtract it from the uncommitted size accounting.
If we delete the line, all tests pass. Let's delete it and then add a unit test verifying that payloadsSize(emptyEntry) == 0
with a suitable comment explaining why this is an important property.
In #79, we added StepDownOnRemoval
, which makes a leader step down to follower after removing itself from the group or demoting itself to learner. Without this, the old leader may keep heartbeating even though it's no longer part of the group, and it can't propose anything.
However, for backwards compatibility reasons, this behavior is off by default. In 4.0, we should always do this and remove the setting.
There have been multiple formal specifications of raft consensus algorithm in TLA+, following Diego Ongaro's Ph.D. dissertation. However, I have not seen a version that aligns to the raft library implemented in etcd-io/raft, which we know are different from the original raft algorithm in some behaviors, e.g., reconfiguration.
etcd and many other applications based on this raft library have been running good for long time, but I feel it would still be worthy to write a TLA+ spec for this specific implementation. It is not just to verify the correctness of the model, but also a foundation of a following up work in model-based trace validation.
The spec is based on George Pîrlea's raft TLA+ spec and inspired by the innovative work in Microsoft CCF's TLA+ spec.
Hi, I read the leader lease part of etcd-raft and it seems that the Leader was not expiring its leader lease when it was required to transfer its leadership. A transferer might just step back to leader(it has now a new leader lease of an election timeout long) and serve read-only reuqests after election timeout if now message from new leader was recieved. Suppose the transferee was elected leader just right after the transferer step back leader again, at this time both leader might be able to server read-only request and I think the old leader might return stale read-only results to clients.
Reincarnation of etcd-io/etcd#10166 (comment).
While it's good that raft tries to detect invariant violations (for example follower regressing no an acked log position), more flexibility in how to react these events would be desirable.
This isn't something we should do lightly, but the current failure modes are unnecessarily catastrophic. The point of raft is to provide high availability, so a faulty follower, even if it technically broke the rules, shouldn't crash the leader.
Please refer to etcd-io/etcd#15784
I'm learning probe mechanism and notice there is chance for replication starting from an index less than match index in probe state, unlike that in replicate state. For example, a rejected probe response got delayed and is received when the leader falls into another probe state. This is a rare case of cause.
I'm not sure if this is by design. But to my understanding, allowing such a smaller index implies we want to tolerate loss of already replicated log entries.
func (pr *Progress) MaybeDecrTo(rejected, matchHint uint64) bool {
if pr.State == StateReplicate {
// The rejection must be stale if the progress has matched and "rejected"
// is smaller than "match".
if rejected <= pr.Match {
return false
}
// Directly decrease next to match + 1.
//
// TODO(tbg): why not use matchHint if it's larger?
pr.Next = pr.Match + 1
return true
}
// The rejection must be stale if "rejected" does not match next - 1. This
// is because non-replicating followers are probed one entry at a time.
if pr.Next-1 != rejected {
return false
}
pr.Next = max(min(rejected, matchHint+1), 1)
pr.MsgAppFlowPaused = false
return true
}
This was deprecated in #62, and should be removed in the next major release.
Allow to run GitHub workflows for Pull Requests with the ok-to-test
label using the prow bot command /ok-to-test. This was already done in the etcd repo (refer to the issue etcd-io/etcd#16956 and its PR etcd-io/etcd#16965).
In the Raft paper, there is a mentioned optimization method for Lease Read using clock + heartbeat. When the leader sends a heartbeat, it first records a timestamp called "start". If the majority of the nodes in the system respond with heartbeat responses, it is assumed that the leader's lease is valid until the time point "start + election timeout / clock drift bound".
But when I read etcd LeaseRead code, it seems that LeaseRead depend on the checkquorum. When the leader receive the major of heartbeat responses in the half of election_timeout, the MsgCheckQuorum will not step the leader down on this election_timeout round. The next round, the leader step down into follower because the major of nodes is not recentActive.This case seems that the leasetime last 1.5 election_timeout.This confused me.
In #110, we made some minor changes to snapshot handling, which removed the main use of Progress.PendingSnapshot
. As described in #110 (comment), we should now be able to remove PendingSnapshot
completely, and take the opportunity to clean up some of the related state transitions.
Also, as described in cockroachdb/cockroach#87583 and #110 (comment), we should modify ReportSnapshot()
and MsgSnapStatus
to take the index of the applied snapshot -- in the common case, this will be the same as that in MsgSnap
, but complex systems (e.g. CockroachDB) may generate and apply snapshots at a different index than that requested by Raft (e.g. to send snapshots from a follower in the same region), and reporting back the actual index would simplify the logic and e.g. allow updating Match
and transitioning directly to StateReplicate
in this case.
These are all breaking changes to public APIs, so would have to be done for 4.0. There are possibly other related cleanups that could be made as well.
@pav-kv @ahrtr Please add any further ideas and context as needed.
The raftLog structure currently plays multiple roles. Notably:
unstable
in-memory part, and the Storage
part), and keeps the basic metadata about the log.raft
algorithm.The flow control role is quite distinct from the other roles, and is less fundamental. It should be decoupled. Once decoupling is done, raftLog
can be isolated in its own package and rigorously tested without assumptions about the flow.
Related to #64, though this clean-up has value on its own.
Looks like #8 gave us a regression in alloc count.
$ git checkout 65a0bf3
$ benchdiff --old HEAD~1 -r BenchmarkRawNode -c 5 .
checking out '91eb751'
building benchmark binaries for '91eb751' 1/1 |
test binaries already exist for '65a0bf3'; skipping build
pkg=1/1 iter=5/5 go.etcd.io/raft/v3 -
name old time/op new time/op delta
RawNode/two-voters-10 1.14µs ± 1% 1.39µs ± 3% +22.47% (p=0.008 n=5+5)
RawNode/single-voter-10 685ns ± 2% 918ns ± 2% +33.99% (p=0.008 n=5+5)
name old firstIndex/op new firstIndex/op delta
RawNode/single-voter-10 2.00 ± 0% 1.00 ± 0% -50.00% (p=0.008 n=5+5)
RawNode/two-voters-10 6.00 ± 0% 3.00 ± 0% -50.00% (p=0.008 n=5+5)
name old lastIndex/op new lastIndex/op delta
RawNode/single-voter-10 2.00 ± 0% 2.00 ± 0% ~ (all equal)
RawNode/two-voters-10 2.00 ± 0% 2.00 ± 0% ~ (all equal)
name old ready/op new ready/op delta
RawNode/single-voter-10 2.00 ± 0% 2.00 ± 0% ~ (all equal)
RawNode/two-voters-10 2.00 ± 0% 2.00 ± 0% ~ (all equal)
name old term/op new term/op delta
RawNode/two-voters-10 1.00 ± 0% 1.00 ± 0% ~ (all equal)
RawNode/single-voter-10 0.00 ± 0% 0.00 ± 0% +33.33% (p=0.008 n=5+5)
name old alloc/op new alloc/op delta
RawNode/single-voter-10 415B ± 1% 534B ± 0% +28.62% (p=0.016 n=5+4)
RawNode/two-voters-10 637B ± 0% 888B ± 1% +39.43% (p=0.008 n=5+5)
name old allocs/op new allocs/op delta
RawNode/two-voters-10 7.00 ± 0% 8.00 ± 0% +14.29% (p=0.008 n=5+5)
RawNode/single-voter-10 5.00 ± 0% 6.00 ± 0% +20.00% (p=0.008 n=5+5)
Consider an assertion such as
Line 324 in d9907d6
of which there are various across the codebase.
Hitting this panic usually means that a follower has not upheld its durability guarantees.
Violating invariants like these is not great, but crashing might not be the user's first choice here. An app might prefer to keep going despite some amount of risk that a write was lost (which often it won't have been).
The way I would structure this is by introducing event-based logging:
Instead of a line like this
Line 324 in d9907d6
We'd have something like this (total strawman just to get the idea across)
l.logger.Event(&CommitOutOfRangeIndex{Commit: tocommit, LastIndex: l.lastIndex()})
// Code here to actually handle the problem gracefully
...
where the default logger would panic but users could define a logger that would just log the event and keep going. We wouldn't have to make all events that are now panics recoverable at first but could allow this only for certain events like the one discussed here.
Extracted from #25 (comment)_
Note that while "help is wanted" here I don't have bandwidth to shepherd a pull request from humble beginnings to the end. Unless another maintainer steps up to "sponsor" this work I'll only be able to accept contributions that are "close enough" to a solution that passes the bar: good design, testing, sensibly documented, backwards compatible. This will be difficult for casual or even first-time contributors.
Is it possible to use this raft library to introduce members which will only participate in voting but will never become leader? The use case for such arbiter
nodes is to facilitate quorum but also potentially optimizes other resources (such as disk space etc.) for a large db.
Thanks
I'm using this implementation of RAFT in an application which is largely based on the example. It's running across three nodes and has no issues maintaining quorum, etc.
After an extended period, the nodes begin to panic with a nil pointer deference when trying to RLock the Mutex protecting the getSnapshot() in kvstore.
Strangely, this has been challenging to reproduce with the only curious indicator being the commit being 10,001 leading me to a storage issue or otherwise. Are there any known issues which could cause this? Apologies for the issue, just wanted to get it in front of the right eyes.
When the raft instance handles a MsgProp
message in Step
method, it populates Index
/Term
of the passed in slice of entries:
Lines 761 to 764 in 659ecbe
Document this side effect or make it more explicit. There are usages relying on it, e.g. in CRDB: cockroachdb/cockroach#98308.
Basically, it would be good to have an API in which we propose entries, and get back the index at which they ended up appended.
We are a group of researchers testing distributed protocol implementations. While testing the raft implementation, we encountered the following crash (with a 3 node cluster).
panic: need non-empty snapshot
goroutine 1 [running]:
github.com/ds-testing-user/raft-fuzzing/raft.(*raft).maybeSendAppend(0xc00033ac00, 0x1, 0x1)
/go/src/github.com/ds-testing-user/raft-fuzzing/raft/raft.go:599 +0x8ae
github.com/ds-testing-user/raft-fuzzing/raft.(*raft).sendAppend(...)
/go/src/github.com/ds-testing-user/raft-fuzzing/raft/raft.go:551
github.com/ds-testing-user/raft-fuzzing/raft.(*raft).bcastAppend.func1(@xa7df00?, 0xc0005b6c60?)
/go/src/github.com/ds-testing-user/raft-fuzzing/raft/raft.go:653 +0x38
github.com/ds-testing-user/raft-fuzzing/raft/tracker.(*ProgressTracker).Visit(0xc00033ac48, 0xc0000b07b8)
/go/src/github.com/ds-testing-user/raft-fuzzing/raft/tracker/tracker.go:211 +0x17a
github.com/ds-testing-user/raft-fuzzing/raft.(*raft).bcastAppend(0xc00033ac00?)
/go/src/github.com/ds-testing-user/raft-fuzzing/raft/raft.go:649 +0x3b
github.com/ds-testing-user/raft-fuzzing/raft.stepLeader(0xc00033ac00, {0x4, 0x2, 0x1, 0x2, 0x0, 0x2, {0x0, 0x0, 0x0}, ...})
/go/src/github.com/ds-testing-user/raft-fuzzing/raft/raft.go:1423 +0x57c
github.com/ds-testing-user/raft-fuzzing/raft.(*raft).Step(0xc00033ac00, {0x4, 0x2, 0x1, 0x2, 0x0, 0x2, {0x0, 0x0, 0x0}, ...})
/go/src/github.com/ds-testing-user/raft-fuzzing/raft/raft.go:1147 +0xe9c
github.com/ds-testing-user/raft-fuzzing/raft.(*RawNode).Step(0xc0000a28c0?, {0x4, 0x2, 0x1, 0x2, 0x0, 0x2, {0x0, 0x0, 0x0}, ...})
/go/src/github.com/ds-testing-user/raft-fuzzing/raft/rawnode.go:124 +0x138
main.(*RaftEnvironment).Step(0xc001f92c48?, 0xc0000b15f8?, {0x4, 0x2, 0x1, 0x2, 0x0, 0x2, {0x0, 0x0, .}, ...})
/go/src/github.com/ds-testing-user/raft-fuzzing/raft.go:129 +0xd8
main.(*Fuzzer).RunIteration(0xc0000a22d0, {0xc000028a40, 0x9}, 0x0)
/go/src/github.com/ds-testing-user/raft-fuzzing/fuzzer.go:363 +0x1305
main.(*Fuzzer).Run(0xc0000a22d0)
/go/src/github.com/ds-testing-user/raft-fuzzing/fuzzer.go:258 +0x398
main.(*Comparision).doRun(0xc00017d6c0, 0xf80000c000149c48?)
/go/src/github.com/ds-testing-user/raft-fuzzing/benchmarking.go:75 +0x2d9
main.(*Comparision).Run(0xc00017d6c0)
/go/src/github.com/ds-testing-user/raft-fuzzing/benchmarking.go:87 +0x3c
main.OneCommand.func1(0xc00023e600?, {0xb0ea46?, 0x8?, 0x8?})
/go/src/github.com/ds-testing-user/raft-fuzzing/main.go:99 +0x654
github.com/spf13/cobra.(*Command).execute(0xc00023e600, {0xc00019c100, 0x8, 0x8})
/go/pkg/mod/github.com/spf13/[email protected]/command.go:920 +0x847
github.com/spf13/cobra.(*Command).ExecuteC(0xc00023e000)
/go/pkg/mod/github.com/spf13/[email protected]/command.go:1044 +0x3bd
github.com/spf13/cobra.(*Command).Execute(...)
/go/pkg/mod/github.com/spf13/[email protected]/command.go:968
main.main()
/go/src/github.com/ds-testing-user/raft-fuzzing/main.go:31 +0x234
We have been able to reproduce the issue. The root cause seems to be that MemoryStorage.Snapshot
does not return ErrSnapshotTemporarilyUnavailable
when snapshot
is nil
. The error occurred when a leader was elected without the snapshot being recorded to storage.
Refer to raftexample . But this example should NOT depend on etcd at all.
cc @Elbehery
Hi there,
Is it even possible to remove node completely with this package?
I use a 3rd party wrapper around this raft package which sends this -
c := raftpb.ConfChange{
Type: raftpb.ConfChangeRemoveNode,
NodeID: uint64(*node.NodeId),
}
When I delete a node and then it's data directory, it's still present on the raft members list, raft leader doesn't trying to replicate data to it so I'm assuming the node was removed.
When I'm trying to join the same node to the cluster again it returns - node already joined under another ID
Thanks
This is nice to have, as discussed in the fortnightly etcd community meeting. But given that etcd has official support for ARM, it's important to run GitHub workflows using ARM64 runners.
This would be a good candidate for the good first issue label.
Refer to etcd-io/etcd#16801, etcd-io/bbolt#583 and the Pull Requests that close those issues.
In both the node.Propose and the raft example, an unreliable way to submit requests is used. In etcd raft, Propose will return when msg append into msgsAfterAppend or msgs.(May be I'm wrong?)
And in raft example, v send to channel then return:
func (h *httpKVAPI) ServeHTTP(w http.ResponseWriter, r *http.Request) {
key := r.RequestURI
defer r.Body.Close()
switch r.Method {
case http.MethodPut:
v, err := io.ReadAll(r.Body)
...
h.store.Propose(key, string(v))
w.WriteHeader(http.StatusNoContent)
return
...
func (s *kvstore) Propose(k string, v string) {
var buf strings.Builder
if err := gob.NewEncoder(&buf).Encode(kv{k, v}); err != nil {
log.Fatal(err)
}
s.proposeC <- buf.String()
}
I know that this is leaving Wait to the user to implement on their own, where they can choose their own timing to end Wait or even not Wait. But I guess most of the person that use raft want more reliability, so I have two proposals
put the wait interface and its default implementation from etcd into draft, and add new methods such as ProposeSyncApplied (wait for entry applied), ProposeSync (wait for entry commited, but not yet applied), and so on (or use option to distinguish between these behaviors). This requires adding a UUID to each Entry, and also allows the user to specify this UUID to be compatible with etcd's current RequestV2.ID.
Improve the raft example, the raft example should have a high reliability, after the user is familiar enough with the raft, adjust the behavior of the program to sacrifice reliability for performance. This can be done with #2 (PS: I think the raft example can be simpler, there is no need to implement a persistent store and a real network layer, but there is a demonstration of the use of each method of the node)
I would like to know what you guys suggest and what is planned for etcd-raft in the future ~ I would also like to update some documentation and other contributions for etcd-raft, but I saw the milestone for etcd-raft 4.0 and I would like to know what is planned for 4.0 and when it will be available!
Background: #144
At the moment, a raft
node only accepts MsgApp
log appends from the latest leader it knows about, i.e. when MsgApp.Term == raft.Term
. This restriction could be relaxed, which can reduce the message turnaround during the times when the leader changes.
The safety requirement is that we don't accept entries that are not in the raft.Term
leader log. If we can deduce that an entry is in the leader's log (before / other than by getting a MsgApp
directly from this leader), we can always safely accept it.
One way to achieve this:
(term, index)
of the last entry of the new leader's log. If the election wins, the new leader will not overwrite entries up to this index
, and will append new entries strictly after it.MsgApp
(from any leader) that contains this entry, we have the guarantee that all entries <= index
in this append are contained in the leader's log. It is safe to accept them.A more general way to achieve this is:
(index, term)
, but also a sample of K
other (index, term)
in its log. Specifically, it would be wise to attach the "fork" points of the last K
terms.MsgApp
, it can deduce from this sample the overlap between this append message and the leader's log. The overlapping part can be safely accepted regardless of who sent it.The practical K
would be 2 or 3, because leader changes are typically not frequent. 2 or 3 last term changes cover a significant section of the log.
This sampling technique is equivalent to the fork point search that the leader does in the StateProbe
state to establish the longest common prefix with the follower's log before transitioning it to the optimistic StateReplicate
state.
This gives significant benefits:
K
fork points rather than just the latest one, we increase chances of finding an overlap immediately, and reduce message turnaround.MsgApp
in the StateProbe
, and will typically be able to transition straight to StateReplicate
.MsgApp.Entries
(that arrived just slightly late) from a recent leader who is stepping down.This technique will minimize cluster disruption / slowdown during election, and reduce tail replication/commit latency in some cases.
A declarative, efficient, and flexible JavaScript library for building user interfaces.
🖖 Vue.js is a progressive, incrementally-adoptable JavaScript framework for building UI on the web.
TypeScript is a superset of JavaScript that compiles to clean JavaScript output.
An Open Source Machine Learning Framework for Everyone
The Web framework for perfectionists with deadlines.
A PHP framework for web artisans
Bring data to life with SVG, Canvas and HTML. 📊📈🎉
JavaScript (JS) is a lightweight interpreted programming language with first-class functions.
Some thing interesting about web. New door for the world.
A server is a program made to process requests and deliver data to clients.
Machine learning is a way of modeling and interpreting data that allows a piece of software to respond intelligently.
Some thing interesting about visualization, use data art
Some thing interesting about game, make everyone happy.
We are working to build community through open source technology. NB: members must have two-factor auth.
Open source projects and samples from Microsoft.
Google ❤️ Open Source for everyone.
Alibaba Open Source for everyone
Data-Driven Documents codes.
China tencent open source team.