├── README.md ├── ebin └── riak_zab.app ├── include ├── riak_zab_peer.hrl └── riak_zab_vnode.hrl ├── rebar ├── rebar.config └── src ├── riak_zab_app.erl ├── riak_zab_backend.erl ├── riak_zab_bitcask_backend.erl ├── riak_zab_console.erl ├── riak_zab_ensemble_master.erl ├── riak_zab_ensemble_util.erl ├── riak_zab_fast_election_fsm.erl ├── riak_zab_leader_test.erl ├── riak_zab_log.erl ├── riak_zab_peer.erl ├── riak_zab_process.erl ├── riak_zab_process_sup.erl ├── riak_zab_ring_handler.erl ├── riak_zab_status.erl ├── riak_zab_sup.erl ├── riak_zab_util.erl ├── riak_zab_vnode.erl └── riak_zab_watcher.erl /README.md: -------------------------------------------------------------------------------- 1 | riak_zab 2 | ======== 3 | 4 | riak_zab is an extension for [riak_core](https://github.com/basho/riak_core) 5 | that provides totally ordered atomic broadcast capabilities. This is 6 | accomplished through a pure Erlang implementation of Zab, the Zookeeper Atomic 7 | Broadcast protocol invented by Yahoo! Research. Zab has nothing to do with the 8 | high-level Zookeeper API that you may be familar with, but is the underlying 9 | broadcast layer used by Zookeeper to implement that API. riak_zab does not 10 | implement the Zookeeper API, only Zab. Zab is conceptually similar to the 11 | [Paxos algorithm](http://en.wikipedia.org/wiki/Paxos_algorithm) with different 12 | design choices that lead to higher throughput and, in my opinion, a more 13 | straight-forward implementation. 14 | 15 | At the high-level, Zab is a leader-based quorum/ensemble protocol that utilizes 16 | two-phase commit. An ensemble (a group of nodes) elects a leader, and all 17 | messages across the ensemble are sent through the leader therefore providing a 18 | total order. The leader utilizes a two-phase propose/commit protocol that ensures 19 | certain guarantees even in the presence of node failures. All in all, Zab guarantees the 20 | following properties: 21 | 22 | * __Reliable delivery:__ If a message m is delivered by one node, than it will 23 | eventually be delivered by all valid nodes. 24 | * __Total order:__ If one node delivers message a before message b, than all 25 | nodes that deliver messages a and b will deliver a before b. 26 | 27 | More details on Zab can be found in the [original research 28 | paper](http://research.yahoo.com/pub/3274), the more recent [technical 29 | report](http://research.fy4.b.yahoo.com/files/YL-2010-007.pdf) (which includes 30 | a detailed formal proof), as well on the Zookeeper 31 | [wiki](https://cwiki.apache.org/confluence/display/ZOOKEEPER/Zab1.0). 32 | 33 | Design 34 | ------ 35 | The goal of riak_zab is to provide the atomic broadcast capabilities of Zab to 36 | riak_core while staying as true as possible to the underlying philosophy that 37 | riak_core is based upon. To this end, an application that utilizes riak_zab 38 | remains essentially a riak_core application. The ideas championed by 39 | riak_core/Dynamo remain: vnodes, consistent hashing, the ring, an (object, 40 | payload) command set, handoff, etc. Likewise, a cluster can scale transparently 41 | over time by adding additional nodes. 42 | 43 | To acheive these goals, riak_zab partitions a riak cluster into a set of 44 | ensembles where a given ensemble is responsible for a subset of preference 45 | lists within the riak ring. The members of the ensemble correspond to the 46 | nodes that own the indicies within the respective preflists. riak_zab then 47 | provides a command interface that is similar to that provided by riak_core, 48 | where commands are dispatched to preference lists and preference lists 49 | are obtained by hashing object keys to locations in the ring. Each ensemble 50 | is an independent Zab instance, that elects its own ensemble leader and 51 | engages in atomic broadcast amongst themselves. Commands that are dispatched 52 | to a particular preference list are sent to the leader of the ensemble 53 | associated with the preflist, and the command is totally ordered and 54 | reliably delivered to all vnodes within the preflist. Unlike traditional 55 | riak, this enforces a strict quorum at all times, which is necessary in 56 | a quorum based protocol. 57 | 58 | %% Verbose example 59 | Idx = chash:key_of(Key), 60 | ESize = riak_zab_ensemble_master:get_ensemble_size(), 61 | {ok, Ring} = riak_core_ring_manager:get_my_ring(), 62 | Preflist = preflist(Idx, ESize, Ring), 63 | Msg = {store, Key, Value}, 64 | riak_zab_ensemble_master:command(Preflist, Msg, Sender, MyApp). 65 | 66 | %% Or using riak_zab_util: 67 | riak_zab_util:command(Key, {store, Key, Value}, MyApp). 68 | riak_zab_util:sync_command(Key, {get, Key}, MyApp). 69 | %% ... 70 | 71 | Okay, I lied. 72 | ------------- 73 | Technically, riak_zab isn't focused on message delivery. Rather, riak_zab 74 | provides consistent replicated state. 75 | 76 | When all nodes are up, riak_zab does ensure that all commands are delivered 77 | in a fixed order to all ensemble nodes. However, if a node is down, riak_zab 78 | obviously cannot deliver a command. So what happens when that node comes back 79 | online? In that case, riak_zab is aware the node is behind, and forces the 80 | node to synchronize with the current leader. The leader performs a state 81 | transfer which brings the lagging node up to date. Therefore, riak_zab 82 | guarantees that all active nodes within an ensemble have a consistent state 83 | that is derived from a particular ordering of delivered messages. 84 | 85 | In reality, Zookeeper does something similar at the Zab layer, but this is 86 | masked by the higher-level Zookeper API that Zookeeper based apps interact 87 | with. 88 | 89 | So, what is this state? How is synchronization performed? 90 | 91 | State and synchronization are features of your application that builds on 92 | riak_zab. Just like riak_core, riak_zab applications are implemented using 93 | vnodes. State is whatever your application wants it to be. riak_zab simply 94 | provides the following features: 95 | 96 | * An atomic command concept, where commands are sent like riak_core but are 97 | guaranteed to be ordered and delievered to a quorum of nodes (or fail). 98 | 99 | * Synchronization. When a vnode rejoins a cluster (restart, net split, etc), 100 | riak_zab will ask the current leader vnode to sync with the out-of-date 101 | follower vnode. The leader vnode then does a state transfer similar to 102 | standard handoff in order to bring the node up to date. Most applications 103 | can utilize the standard synchronization logic provided with riak_zab, which 104 | folds over a (key, value) representation of the vnode state and ships it off 105 | to the follower vnode. More complex apps are free to implement custom 106 | synchronization behaviour. 107 | 108 | Scaling 109 | ------- 110 | 111 | How does a ensemble based protocol scale? Don't ensemble protocols get slower 112 | as you add more nodes? 113 | 114 | Yes. Yes, they do. That's why riak_zab fixes the number of nodes in an 115 | ensemble. It's an app.config value. The default is 3, and there really 116 | isn't a reason to use a value other than 3 or 5. With 3 you can tolerate 117 | 1 node failure and remain available. With 5, you can tolerate 2. Many 118 | people chose 5 (in the Zookeeper case), so that they can take 1 node offline 119 | for maintainence and still be able to tolerate 1 additional node failure. 120 | 121 | Since the ensemble size is fixed, as you add more nodes to a riak_zab cluster, 122 | you end up with more and more ensembles, not more nodes per ensemble. Each 123 | ensemble therefore owns fewer and fewer preflists as you scale out. 124 | 125 | Assuming your workload is distributed across preflists (a similar assumption 126 | underlying riak_core performance), your workload will be distributed across 127 | different ensembles and therefore different nodes and Zab instances. You will 128 | always pay a price (latency) for leader based two-phase commit, but you can 129 | have multiple independent two-phase commit protocols running completely 130 | independently across completely independent nodes. Thus, predictable scaling of 131 | throughput just like riak_core. 132 | 133 | What about ring rebalancing and handoff? 134 | ----------------------------------------- 135 | An astute reader (probably from Basho) will likely wonder how this all works 136 | with dynamic ring membership. Assuming the ring is static, it's easy to see 137 | how ensembles map to preflists and how preflist state is owned by static vnodes. 138 | Dynamic vnode ownership and handoff complicate matters. 139 | 140 | The current solution isn't elegant, but it works. riak_zab assumes the ring 141 | state is stable. When a riak_zab cluster starts up, the zab layer is disabled. 142 | It's up to the user (ie. operations) to setup the cluster and then manually 143 | start the zab layer. Specifically, the user must ensure that the ring is 144 | consistent across the cluster and that no handoffs are pending. The 'ringready' 145 | and 'transfers' commands for the riak-admin/riak-zab-admin tool are the 146 | solution here. 147 | 148 | > ./dev/dev1/bin/riak-zab-admin join dev2@127.0.0.1 149 | Sent join request to dev2@127.0.0.1 150 | > ./dev/dev2/bin/riak-zab-admin join dev3@127.0.0.1 151 | Sent join request to dev3@127.0.0.1 152 | > ./dev/dev3/bin/riak-zab-admin join dev4@127.0.0.1 153 | Sent join request to dev4@127.0.0.1 154 | > ./dev/dev4/bin/riak-zab-admin join dev1@127.0.0.1 155 | Sent join request to dev1@127.0.0.1 156 | > ./dev/dev1/bin/riak-zab-admin ringready 157 | TRUE All nodes agree on the ring ['dev1@127.0.0.1','dev2@127.0.0.1', 158 | 'dev3@127.0.0.1','dev4@127.0.0.1'] 159 | 160 | > ./dev/dev1/bin/riak-zab-admin transfers 161 | No transfers active. 162 | 163 | > ./dev/dev1/bin/riak-zab-admin zab-up 164 | > ./dev/dev2/bin/riak-zab-admin zab-up 165 | > ./dev/dev3/bin/riak-zab-admin zab-up 166 | > ./dev/dev4/bin/riak-zab-admin zab-up 167 | 168 | > ./dev/dev1/bin/riak-zab-admin info 169 | 170 | ================================ Riak Zab Info ================================ 171 | Ring size: 64 172 | Ensemble size: 3 173 | Nodes: ['dev1@127.0.0.1','dev2@127.0.0.1','dev3@127.0.0.1', 174 | 'dev4@127.0.0.1'] 175 | ================================== Ensembles ================================== 176 | Ensemble Ring Leader Nodes 177 | ------------------------------------------------------------------------------- 178 | 1 25.0% dev2@127.0.0.1 ['dev1@127.0.0.1', 179 | 'dev2@127.0.0.1', 180 | 'dev3@127.0.0.1'] 181 | ------------------------------------------------------------------------------- 182 | 2 25.0% dev2@127.0.0.1 ['dev1@127.0.0.1', 183 | 'dev2@127.0.0.1', 184 | 'dev4@127.0.0.1'] 185 | ------------------------------------------------------------------------------- 186 | 3 25.0% dev3@127.0.0.1 ['dev1@127.0.0.1', 187 | 'dev3@127.0.0.1', 188 | 'dev4@127.0.0.1'] 189 | ------------------------------------------------------------------------------- 190 | 4 25.0% dev3@127.0.0.1 ['dev2@127.0.0.1', 191 | 'dev3@127.0.0.1', 192 | 'dev4@127.0.0.1'] 193 | ------------------------------------------------------------------------------- 194 | 195 | Likewise, if the ring membership ever changes while zab is running, 196 | the zab layer will disable itself, thus making zab ensembles 197 | unavailable but consistent. It's once again up to the user to 198 | ensure that all handoffs are completed and so forth, and then 199 | once again issue a 'zab-up' command. 200 | 201 | For those unfamilar with Riak, after the ring is stable the 202 | membership only changes when a user explictly adds or removes 203 | nodes from the cluster. Nodes going offline or being partitioned 204 | don't rebalance the ring. Therefore, this is a rare, operationally 205 | initiated event in the first place. 206 | 207 | Quick Start 208 | ----------- 209 | This repository isn't a stand-alone Erlang release, much like riak_core. You 210 | can clone the [riak_zab_example](https://github.com/jtuple/riak_zab_example) 211 | repository to have a runnable release, that also demonstrates how to build a 212 | simple riak_zab vnode-based application. 213 | 214 | Testing 215 | ------- 216 | riak_zab is very much an alpha release right now, and woefully lacking in 217 | tests. I have many script based systems-level integration tests that test a 218 | cluster of riak_zab nodes in a variety of scenarios, including using virtual 219 | networking to simulate net-splits. I am slowly porting this over to a pure 220 | Erlang based multi-node expectation framework that I am designing concurrently 221 | with this porting task. An example is the riak_zab_leader_test.erl file included 222 | in this repository. Further porting is underway. 223 | 224 | However, the real plan is to test the entire application using QuickCheck. 225 | Testing an atomic broadcast protocol isn't easy, and relying on traditional 226 | unit tests and systems-level tests is a bad idea. QuickCheck is a much better 227 | approach, however it will take some effort to make everything testable. I am 228 | hoping to follow in the footsteps of [jonmeredith](https://github.com/jonmeredith) 229 | and [slfritchie](https://github.com/slfritchie) from Basho who has embarked 230 | on a heroric effort to test Riak using QuickCheck. 231 | 232 | Specifically, my plan is to test that all of the invariants and claims listed 233 | within the formal Zab [technical 234 | report](http://research.fy4.b.yahoo.com/files/YL-2010-007.pdf)from Yahoo! do in 235 | fact hold for this implementation. 236 | 237 | Can I use this in production? 238 | ----------------------------- 239 | As just mentioned, there is still a long way to go with testing riak_zab. 240 | I suggest folks who are interested play around with it, but not use it in 241 | production. I know of a few very rare corner cases that currently cause 242 | trouble (fixed within Zookeeper within the last year), that I still need 243 | to address. 244 | 245 | I used an older version of this within a closed beta for a social gaming 246 | project. While that may be a quasi-production deployment, the reality was 247 | that money was never on the line -- if things didn't work, you just buy 248 | the players off with free virtual goods. 249 | 250 | Zookeeper has been out for years, and yet new edge cases are found on 251 | occasion. This is a difficult task to get right. My hope is that future testing 252 | with QuickCheck (and possibly McErlang or McErlang/QuickCheck) will eventually 253 | lead to a truly robust and trustable atomic broadcast implementation. Building 254 | this in Erlang also helps greatly. 255 | 256 | Where's the killer app? 257 | ----------------------- 258 | By the way, riak_zab is actually an extracted reusable layer that arose as part 259 | of a project I called riakual, that provides an immediately consistent access 260 | layer on top of riak_kv that supports optimistic concurrency. I'll soon be 261 | releasing a version of riakual that is built as a riak_zab vnode application. 262 | Hopefully that gives you an idea of what you can do with riak_zab. 263 | 264 | And for those who ask, why would you add immediate consistent to riak? Well, 265 | Amazon's SimpleDB (Dynamo based) added such a few years back. It's certainly 266 | not that crazy. 267 | 268 | Also, if your application can be expressed in an eventually consistent manner 269 | 80-90% of the time, wouldn't it be nice if you could use the same predictably 270 | scable and simple to deploy datastore (ie. Riak) for the 10% that must be 271 | immediately consistent? 272 | 273 | What if your app was more eventually consistent than you previously thought? 274 | The awesome 275 | [CALM](http://databeta.wordpress.com/2010/10/28/the-calm-conjecture-reasoning-about-consistency) 276 | and [Bloom](http://www.bloom-lang.net) research from Professor Hellerstein 277 | group may expand your mind. 278 | -------------------------------------------------------------------------------- /ebin/riak_zab.app: -------------------------------------------------------------------------------- 1 | %% -*- tab-width: 4;erlang-indent-level: 4;indent-tabs-mode: nil -*- 2 | %% ex: ts=4 sw=4 etnnononono 3 | {application,riak_zab, 4 | [{description,[]}, 5 | {vsn,"0.1.0"}, 6 | {registered,[]}, 7 | {applications, [ 8 | kernel, 9 | stdlib, 10 | riak_core 11 | ]}, 12 | {mod,{riak_zab_app,[]}}, 13 | {env,[]}, 14 | {modules, [ 15 | riak_zab_app, 16 | riak_zab_sup, 17 | riak_zab_peer, 18 | riak_zab_vnode, 19 | riak_zab_ring_handler, 20 | riak_zab_fast_election_fsm, 21 | riak_zab_watcher, 22 | riak_zab_backend, 23 | riak_zab_bitcask_backend, 24 | riak_zab_status, 25 | riak_zab_console, 26 | riak_zab_process, 27 | riak_zab_process_sup, 28 | riak_zab_ensemble_master, 29 | riak_zab_ensemble_util, 30 | riak_zab_util, 31 | riak_zab_log, 32 | riak_zab_leader_test 33 | ]} 34 | ]}. 35 | -------------------------------------------------------------------------------- /include/riak_zab_peer.hrl: -------------------------------------------------------------------------------- 1 | -include_lib("riak_core/include/riak_core_vnode.hrl"). 2 | 3 | -type zsender() :: {fsm_sync, reference(), term()} | 4 | {fsm_sync, undefined, undefined} | 5 | sender(). 6 | 7 | -record(riak_zab_proposal, { 8 | preflist, 9 | sender, 10 | message}). 11 | 12 | -define(ZAB_PROPOSAL, #riak_zab_proposal). 13 | -------------------------------------------------------------------------------- /include/riak_zab_vnode.hrl: -------------------------------------------------------------------------------- 1 | %-include_lib("riak_core/include/riak_core_vnode.hrl"). 2 | 3 | -record(riak_zab_last_zxid, {}). 4 | -record(riak_zab_sync, {peer, idxs}). 5 | -record(riak_zab_req, {req, zxid, sender, leading}). 6 | -record(riak_zab_sync_data, {data}). 7 | 8 | -define(ZAB_LAST_ZXID, #riak_zab_last_zxid). 9 | -define(ZAB_SYNC, #riak_zab_sync). 10 | -define(ZAB_REQ, #riak_zab_req). 11 | -define(ZAB_SYNC_DATA, #riak_zab_sync_data). 12 | 13 | -------------------------------------------------------------------------------- /rebar: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jtuple/riak_zab/ebd54aa540cf65e49d6e0890005e714ee756914a/rebar -------------------------------------------------------------------------------- /rebar.config: -------------------------------------------------------------------------------- 1 | {require_otp_vsn, "R13B04|R14"}. 2 | {cover_enabled, true}. 3 | {erl_opts, [debug_info, fail_on_warning]}. 4 | 5 | {deps, [ 6 | {riak_core, "0.14..*", {git, "git://github.com/basho/riak_core", 7 | {tag, "riak_core-0.14.1"}}}, 8 | {bitcask, "1.1.5", {git, "git://github.com/basho/bitcask", 9 | {tag, "bitcask-1.1.5"}}}, 10 | {eprobe, "0.1.0", {git, "git://github.com/jtuple/eprobe", 11 | {branch, "master"}}} 12 | ]}. 13 | -------------------------------------------------------------------------------- /src/riak_zab_app.erl: -------------------------------------------------------------------------------- 1 | -module(riak_zab_app). 2 | 3 | -behaviour(application). 4 | 5 | %% Application callbacks 6 | -export([start/2, stop/1, full_stop/0, full_stop/1]). 7 | 8 | %% =================================================================== 9 | %% Application callbacks 10 | %% =================================================================== 11 | 12 | start(_StartType, _StartArgs) -> 13 | riak_core_util:start_app_deps(riak_zab), 14 | 15 | case riak_zab_sup:start_link() of 16 | {ok, Pid} -> 17 | ok = riak_core_ring_events:add_handler(riak_zab_ring_handler, []), 18 | {ok, Pid}; 19 | {error, Reason} -> 20 | {error, Reason} 21 | end. 22 | 23 | stop(_State) -> 24 | ok. 25 | 26 | 27 | %% @spec full_stop() -> ok 28 | %% @doc Stop the riak application and the calling process. 29 | full_stop() -> stop("riak stop requested"). 30 | full_stop(Reason) -> 31 | % we never do an application:stop because that makes it very hard 32 | % to really halt the runtime, which is what we need here. 33 | error_logger:info_msg(io_lib:format("~p~n",[Reason])), 34 | init:stop(). 35 | -------------------------------------------------------------------------------- /src/riak_zab_backend.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_zab_backend: Copy of riak_kv_backend to avoid requiring 4 | %% entire riak_kv app as a dependency. 5 | %% 6 | %% ------------------------------------------------------------------- 7 | %% 8 | %% riak_kv_backend: Riak backend behaviour 9 | %% 10 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 11 | %% 12 | %% This file is provided to you under the Apache License, 13 | %% Version 2.0 (the "License"); you may not use this file 14 | %% except in compliance with the License. You may obtain 15 | %% a copy of the License at 16 | %% 17 | %% http://www.apache.org/licenses/LICENSE-2.0 18 | %% 19 | %% Unless required by applicable law or agreed to in writing, 20 | %% software distributed under the License is distributed on an 21 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 22 | %% KIND, either express or implied. See the License for the 23 | %% specific language governing permissions and limitations 24 | %% under the License. 25 | %% 26 | %% ------------------------------------------------------------------- 27 | 28 | -module(riak_zab_backend). 29 | -export([behaviour_info/1]). 30 | -export([callback_after/3]). 31 | -ifdef(TEST). 32 | -include_lib("eunit/include/eunit.hrl"). 33 | -export([standard_test/2]). 34 | -endif. 35 | 36 | -spec behaviour_info(atom()) -> 'undefined' | [{atom(), arity()}]. 37 | behaviour_info(callbacks) -> 38 | [{start,2}, % (Partition, Config) 39 | {stop,1}, % (State) 40 | {get,2}, % (State, BKey) 41 | {put,3}, % (State, BKey, Val) 42 | {list,1}, % (State) 43 | {list_bucket,2}, % (State, Bucket) 44 | {delete,2}, % (State, BKey) 45 | {drop,1}, % (State) 46 | {fold,3}, % (State, Folder, Acc), Folder({B,K},V,Acc) 47 | {is_empty,1}, % (State) 48 | {callback,3}]; % (State, Ref, Msg) -> 49 | behaviour_info(_Other) -> 50 | undefined. 51 | 52 | %% Queue a callback for the backend after Time ms. 53 | -spec callback_after(integer(), reference(), term()) -> reference(). 54 | callback_after(Time, Ref, Msg) when is_integer(Time), is_reference(Ref) -> 55 | riak_core_vnode:send_command_after(Time, {backend_callback, Ref, Msg}). 56 | 57 | -ifdef(TEST). 58 | 59 | standard_test(BackendMod, Config) -> 60 | {ok, S} = BackendMod:start(42, Config), 61 | ?assertEqual(ok, BackendMod:put(S,{<<"b1">>,<<"k1">>},<<"v1">>)), 62 | ?assertEqual(ok, BackendMod:put(S,{<<"b2">>,<<"k2">>},<<"v2">>)), 63 | ?assertEqual({ok,<<"v2">>}, BackendMod:get(S,{<<"b2">>,<<"k2">>})), 64 | ?assertEqual({error, notfound}, BackendMod:get(S, {<<"b1">>,<<"k3">>})), 65 | ?assertEqual([{<<"b1">>,<<"k1">>},{<<"b2">>,<<"k2">>}], 66 | lists:sort(BackendMod:list(S))), 67 | ?assertEqual([<<"k2">>], BackendMod:list_bucket(S, <<"b2">>)), 68 | ?assertEqual([<<"k1">>], BackendMod:list_bucket(S, <<"b1">>)), 69 | ?assertEqual([<<"k1">>], BackendMod:list_bucket( 70 | S, {filter, <<"b1">>, fun(_K) -> true end})), 71 | ?assertEqual([], BackendMod:list_bucket( 72 | S, {filter, <<"b1">>, fun(_K) -> false end})), 73 | BucketList = BackendMod:list_bucket(S, '_'), 74 | ?assert(lists:member(<<"b1">>, BucketList)), 75 | ?assert(lists:member(<<"b2">>, BucketList)), 76 | ?assertEqual(ok, BackendMod:delete(S,{<<"b2">>,<<"k2">>})), 77 | ?assertEqual({error, notfound}, BackendMod:get(S, {<<"b2">>, <<"k2">>})), 78 | ?assertEqual([{<<"b1">>, <<"k1">>}], BackendMod:list(S)), 79 | Folder = fun(K, V, A) -> [{K,V}|A] end, 80 | ?assertEqual([{{<<"b1">>,<<"k1">>},<<"v1">>}], BackendMod:fold(S, Folder, [])), 81 | ?assertEqual(ok, BackendMod:put(S,{<<"b3">>,<<"k3">>},<<"v3">>)), 82 | ?assertEqual([{{<<"b1">>,<<"k1">>},<<"v1">>}, 83 | {{<<"b3">>,<<"k3">>},<<"v3">>}], lists:sort(BackendMod:fold(S, Folder, []))), 84 | ?assertEqual(false, BackendMod:is_empty(S)), 85 | ?assertEqual(ok, BackendMod:delete(S,{<<"b1">>,<<"k1">>})), 86 | ?assertEqual(ok, BackendMod:delete(S,{<<"b3">>,<<"k3">>})), 87 | ?assertEqual(true, BackendMod:is_empty(S)), 88 | ok = BackendMod:stop(S). 89 | 90 | -endif. % TEST 91 | -------------------------------------------------------------------------------- /src/riak_zab_bitcask_backend.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_zab_bitcask_backend: Copy of riak_kv_bitcask_backend to avoid 4 | %% requiring entire riak_kv app as a 5 | %% dependency. Changed application:get_env 6 | %% from bitcask to riak_zab. 7 | %% 8 | %% ------------------------------------------------------------------- 9 | %% 10 | %% riak_kv_bitcask_backend: Bitcask Driver for Riak 11 | %% 12 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 13 | %% 14 | %% This file is provided to you under the Apache License, 15 | %% Version 2.0 (the "License"); you may not use this file 16 | %% except in compliance with the License. You may obtain 17 | %% a copy of the License at 18 | %% 19 | %% http://www.apache.org/licenses/LICENSE-2.0 20 | %% 21 | %% Unless required by applicable law or agreed to in writing, 22 | %% software distributed under the License is distributed on an 23 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 24 | %% KIND, either express or implied. See the License for the 25 | %% specific language governing permissions and limitations 26 | %% under the License. 27 | %% 28 | %% ------------------------------------------------------------------- 29 | 30 | -module(riak_zab_bitcask_backend). 31 | -behavior(riak_zab_backend). 32 | -author('Andy Gross '). 33 | -author('Dave Smith '). 34 | 35 | %% KV Backend API 36 | -export([start/2, 37 | stop/1, 38 | get/2, 39 | put/3, 40 | delete/2, 41 | list/1, 42 | list_bucket/2, 43 | fold/3, 44 | fold_keys/3, 45 | drop/1, 46 | is_empty/1, 47 | callback/3]). 48 | 49 | %% Helper API 50 | -export([key_counts/0]). 51 | 52 | -ifdef(TEST). 53 | -include_lib("eunit/include/eunit.hrl"). 54 | -endif. 55 | 56 | -include_lib("bitcask/include/bitcask.hrl"). 57 | 58 | -define(MERGE_CHECK_INTERVAL, timer:minutes(3)). 59 | 60 | start(Partition, Config) -> 61 | 62 | %% Get the data root directory 63 | DataDir = 64 | case proplists:get_value(data_root, Config) of 65 | undefined -> 66 | case application:get_env(riak_zab, data_root) of 67 | {ok, Dir} -> 68 | Dir; 69 | _ -> 70 | riak_zab_app:full_stop("bitcask data_root unset, " 71 | "failing") 72 | end; 73 | Value -> 74 | Value 75 | end, 76 | 77 | %% Setup actual bitcask dir for this partition 78 | BitcaskRoot = filename:join([DataDir, 79 | integer_to_list(Partition)]), 80 | case filelib:ensure_dir(BitcaskRoot) of 81 | ok -> 82 | ok; 83 | {error, Reason} -> 84 | error_logger:error_msg("Failed to create bitcask dir ~s: ~p\n", 85 | [BitcaskRoot, Reason]), 86 | riak_zab_app:full_stop("riak_zab_bitcask_backend failed to start.") 87 | end, 88 | 89 | BitcaskOpts = [{read_write, true}|Config], 90 | case bitcask:open(BitcaskRoot, BitcaskOpts) of 91 | Ref when is_reference(Ref) -> 92 | schedule_merge(Ref), 93 | maybe_schedule_sync(Ref), 94 | {ok, {Ref, BitcaskRoot}}; 95 | {error, Reason2} -> 96 | {error, Reason2} 97 | end. 98 | 99 | 100 | stop({Ref, _}) -> 101 | bitcask:close(Ref). 102 | 103 | 104 | get({Ref, _}, BKey) -> 105 | Key = term_to_binary(BKey), 106 | case bitcask:get(Ref, Key) of 107 | {ok, Value} -> 108 | {ok, Value}; 109 | not_found -> 110 | {error, notfound}; 111 | {error, Reason} -> 112 | {error, Reason} 113 | end. 114 | 115 | put({Ref, _}, BKey, Val) -> 116 | Key = term_to_binary(BKey), 117 | case bitcask:put(Ref, Key, Val) of 118 | ok -> ok; 119 | {error, Reason} -> 120 | {error, Reason} 121 | end. 122 | 123 | delete({Ref, _}, BKey) -> 124 | case bitcask:delete(Ref, term_to_binary(BKey)) of 125 | ok -> ok; 126 | {error, Reason} -> 127 | {error, Reason} 128 | end. 129 | 130 | list({Ref, _}) -> 131 | case bitcask:list_keys(Ref) of 132 | KeyList when is_list(KeyList) -> 133 | [binary_to_term(K) || K <- KeyList]; 134 | Other -> 135 | Other 136 | end. 137 | 138 | list_bucket({Ref, _}, {filter, Bucket, Fun}) -> 139 | bitcask:fold_keys(Ref, 140 | fun(#bitcask_entry{key=BK},Acc) -> 141 | {B,K} = binary_to_term(BK), 142 | case B of 143 | Bucket -> 144 | case Fun(K) of 145 | true -> [K|Acc]; 146 | false -> Acc 147 | end; 148 | _ -> 149 | Acc 150 | end 151 | end, []); 152 | list_bucket({Ref, _}, '_') -> 153 | bitcask:fold_keys(Ref, 154 | fun(#bitcask_entry{key=BK},Acc) -> 155 | {B,_K} = binary_to_term(BK), 156 | case lists:member(B,Acc) of 157 | true -> Acc; 158 | false -> [B|Acc] 159 | end 160 | end, []); 161 | list_bucket({Ref, _}, Bucket) -> 162 | bitcask:fold_keys(Ref, 163 | fun(#bitcask_entry{key=BK},Acc) -> 164 | {B,K} = binary_to_term(BK), 165 | case B of 166 | Bucket -> [K|Acc]; 167 | _ -> Acc 168 | end 169 | end, []). 170 | 171 | fold({Ref, _}, Fun0, Acc0) -> 172 | %% When folding across the bitcask, the bucket/key tuple must 173 | %% be decoded. The intermediate binary_to_term call handles this 174 | %% and yields the expected fun({B, K}, Value, Acc) 175 | bitcask:fold(Ref, 176 | fun(K, V, Acc) -> 177 | Fun0(binary_to_term(K), V, Acc) 178 | end, 179 | Acc0). 180 | 181 | fold_keys({Ref, _}, Fun, Acc) -> 182 | F = fun(#bitcask_entry{key=K}, Acc1) -> 183 | Fun(binary_to_term(K), Acc1) end, 184 | bitcask:fold_keys(Ref, F, Acc). 185 | 186 | drop({Ref, BitcaskRoot}) -> 187 | %% todo: once bitcask has a more friendly drop function 188 | %% of its own, use that instead. 189 | bitcask:close(Ref), 190 | {ok, FNs} = file:list_dir(BitcaskRoot), 191 | [file:delete(filename:join(BitcaskRoot, FN)) || FN <- FNs], 192 | file:del_dir(BitcaskRoot), 193 | ok. 194 | 195 | is_empty({Ref, _}) -> 196 | %% Determining if a bitcask is empty requires us to find at least 197 | %% one value that is NOT a tombstone. Accomplish this by doing a fold_keys 198 | %% that forcibly bails on the very first key encountered. 199 | F = fun(_K, _Acc0) -> 200 | throw(found_one_value) 201 | end, 202 | case catch(bitcask:fold_keys(Ref, F, undefined)) of 203 | found_one_value -> 204 | false; 205 | _ -> 206 | true 207 | end. 208 | 209 | callback({Ref, _}, Ref, {sync, SyncInterval}) when is_reference(Ref) -> 210 | bitcask:sync(Ref), 211 | schedule_sync(Ref, SyncInterval); 212 | callback({Ref, BitcaskRoot}, Ref, merge_check) when is_reference(Ref) -> 213 | case bitcask:needs_merge(Ref) of 214 | {true, Files} -> 215 | bitcask_merge_worker:merge(BitcaskRoot, [], Files); 216 | false -> 217 | ok 218 | end, 219 | schedule_merge(Ref); 220 | %% Ignore callbacks for other backends so multi backend works 221 | callback(_State, _Ref, _Msg) -> 222 | ok. 223 | 224 | key_counts() -> 225 | case application:get_env(riak_zab, data_root) of 226 | {ok, RootDir} -> 227 | [begin 228 | {Keys, _} = status(filename:join(RootDir, Dir)), 229 | {Dir, Keys} 230 | end || Dir <- element(2, file:list_dir(RootDir))]; 231 | {error, Reason} -> 232 | {error, Reason} 233 | end. 234 | 235 | %% =================================================================== 236 | %% Internal functions 237 | %% =================================================================== 238 | 239 | %% @private 240 | %% Invoke bitcask:status/1 for a given directory 241 | status(Dir) -> 242 | Ref = bitcask:open(Dir), 243 | Status = bitcask:status(Ref), 244 | bitcask:close(Ref), 245 | Status. 246 | 247 | %% @private 248 | %% Schedule sync (if necessary) 249 | maybe_schedule_sync(Ref) when is_reference(Ref) -> 250 | case application:get_env(riak_zab, sync_strategy) of 251 | {ok, {seconds, Seconds}} -> 252 | SyncIntervalMs = timer:seconds(Seconds), 253 | schedule_sync(Ref, SyncIntervalMs); 254 | %% erlang:send_after(SyncIntervalMs, self(), 255 | %% {?MODULE, {sync, SyncIntervalMs}}); 256 | {ok, none} -> 257 | ok; 258 | BadStrategy -> 259 | error_logger:info_msg("Ignoring invalid bitcask sync strategy: ~p\n", 260 | [BadStrategy]), 261 | ok 262 | end. 263 | 264 | schedule_sync(Ref, SyncIntervalMs) when is_reference(Ref) -> 265 | riak_zab_backend:callback_after(SyncIntervalMs, Ref, {sync, SyncIntervalMs}). 266 | 267 | schedule_merge(Ref) when is_reference(Ref) -> 268 | riak_zab_backend:callback_after(?MERGE_CHECK_INTERVAL, Ref, merge_check). 269 | 270 | %% =================================================================== 271 | %% EUnit tests 272 | %% =================================================================== 273 | -ifdef(TEST). 274 | 275 | simple_test() -> 276 | ?assertCmd("rm -rf test/bitcask-backend"), 277 | application:set_env(riak_zab, data_root, "test/bitcask-backend"), 278 | riak_zab_backend:standard_test(?MODULE, []). 279 | 280 | custom_config_test() -> 281 | ?assertCmd("rm -rf test/bitcask-backend"), 282 | application:set_env(riak_zab, data_root, ""), 283 | riak_zab_backend:standard_test(?MODULE, [{data_root, "test/bitcask-backend"}]). 284 | 285 | -endif. 286 | -------------------------------------------------------------------------------- /src/riak_zab_console.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_zab_console: Copy of riak_kv_console to avoid requiring 4 | %% entire riak_kv app as a dependency. Should 5 | %% probably be part of riak_core anyway. Slightly 6 | %% modified to remove unneeded riak_kv dependencies. 7 | %% 8 | %% ------------------------------------------------------------------- 9 | %% 10 | %% riak_console: interface for Riak admin commands 11 | %% 12 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 13 | %% 14 | %% This file is provided to you under the Apache License, 15 | %% Version 2.0 (the "License"); you may not use this file 16 | %% except in compliance with the License. You may obtain 17 | %% a copy of the License at 18 | %% 19 | %% http://www.apache.org/licenses/LICENSE-2.0 20 | %% 21 | %% Unless required by applicable law or agreed to in writing, 22 | %% software distributed under the License is distributed on an 23 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 24 | %% KIND, either express or implied. See the License for the 25 | %% specific language governing permissions and limitations 26 | %% under the License. 27 | %% 28 | %% ------------------------------------------------------------------- 29 | 30 | %% @doc interface for Riak admin commands 31 | 32 | -module(riak_zab_console). 33 | 34 | -export([join/1, leave/1, remove/1, reip/1, ringready/1, transfers/1]). 35 | 36 | 37 | join([NodeStr]) -> 38 | case try_join(NodeStr) of 39 | ok -> 40 | io:format("Sent join request to ~s\n", [NodeStr]), 41 | ok; 42 | {error, not_reachable} -> 43 | io:format("Node ~s is not reachable!\n", [NodeStr]), 44 | error; 45 | {error, different_ring_sizes} -> 46 | io:format("Failed: ~s has a different ring_creation_size~n", 47 | [NodeStr]), 48 | error 49 | end; 50 | join(_) -> 51 | io:format("Join requires a node to join with.\n"), 52 | error. 53 | 54 | leave([]) -> 55 | remove_node(node()). 56 | 57 | remove([Node]) -> 58 | remove_node(list_to_atom(Node)). 59 | 60 | remove_node(Node) when is_atom(Node) -> 61 | Res = riak_core_gossip:remove_from_cluster(Node), 62 | io:format("~p\n", [Res]). 63 | 64 | %% -spec(status([]) -> ok). 65 | %% status([]) -> 66 | %% case riak_zab_status:statistics() of 67 | %% [] -> 68 | %% io:format("riak_kv_stat is not enabled.\n", []); 69 | %% Stats -> 70 | %% StatString = format_stats(Stats, 71 | %% ["-------------------------------------------\n", 72 | %% io_lib:format("1-minute stats for ~p~n",[node()])]), 73 | %% io:format("~s\n", [StatString]) 74 | %% end. 75 | 76 | reip([OldNode, NewNode]) -> 77 | application:load(riak_core), 78 | RingStateDir = app_helper:get_env(riak_core, ring_state_dir), 79 | {ok, RingFile} = riak_core_ring_manager:find_latest_ringfile(), 80 | BackupFN = filename:join([RingStateDir, filename:basename(RingFile)++".BAK"]), 81 | {ok, _} = file:copy(RingFile, BackupFN), 82 | io:format("Backed up existing ring file to ~p~n", [BackupFN]), 83 | Ring = riak_core_ring_manager:read_ringfile(RingFile), 84 | NewRing = riak_core_ring:rename_node(Ring, OldNode, NewNode), 85 | riak_core_ring_manager:do_write_ringfile(NewRing), 86 | io:format("New ring file written to ~p~n", 87 | [element(2, riak_core_ring_manager:find_latest_ringfile())]). 88 | 89 | %% Check if all nodes in the cluster agree on the partition assignment 90 | -spec(ringready([]) -> ok | error). 91 | ringready([]) -> 92 | case riak_zab_status:ringready() of 93 | {ok, Nodes} -> 94 | io:format("TRUE All nodes agree on the ring ~p\n", [Nodes]); 95 | {error, {different_owners, N1, N2}} -> 96 | io:format("FALSE Node ~p and ~p list different partition owners\n", [N1, N2]), 97 | error; 98 | {error, {nodes_down, Down}} -> 99 | io:format("FALSE ~p down. All nodes need to be up to check.\n", [Down]), 100 | error 101 | end. 102 | 103 | %% Provide a list of nodes with pending partition transfers (i.e. any secondary vnodes) 104 | %% and list any owned vnodes that are *not* running 105 | -spec(transfers([]) -> ok). 106 | transfers([]) -> 107 | {DownNodes, Pending} = riak_zab_status:transfers(), 108 | case DownNodes of 109 | [] -> ok; 110 | _ -> io:format("Nodes ~p are currently down.\n", [DownNodes]) 111 | end, 112 | F = fun({waiting_to_handoff, Node, Count}, Acc) -> 113 | io:format("~p waiting to handoff ~p partitions\n", [Node, Count]), 114 | Acc + 1; 115 | ({stopped, Node, Count}, Acc) -> 116 | io:format("~p does not have ~p primary partitions running\n", [Node, Count]), 117 | Acc + 1 118 | end, 119 | case lists:foldl(F, 0, Pending) of 120 | 0 -> 121 | io:format("No transfers active\n"), 122 | ok; 123 | _ -> 124 | error 125 | end. 126 | 127 | %% cluster_info([OutFile|Rest]) -> 128 | %% case lists:reverse(atomify_nodestrs(Rest)) of 129 | %% [] -> 130 | %% cluster_info:dump_all_connected(OutFile); 131 | %% Nodes -> 132 | %% cluster_info:dump_nodes(Nodes, OutFile) 133 | %% end; 134 | %% cluster_info(_) -> 135 | %% io:format("Usage: output-file ['local'|node_name ['local'|node_name] [...]]\n"), 136 | %% error. 137 | 138 | %% format_stats([], Acc) -> 139 | %% lists:reverse(Acc); 140 | %% format_stats([{vnode_gets, V}|T], Acc) -> 141 | %% format_stats(T, [io_lib:format("vnode gets : ~p~n", [V])|Acc]); 142 | %% format_stats([{Stat, V}|T], Acc) -> 143 | %% format_stats(T, [io_lib:format("~p : ~p~n", [Stat, V])|Acc]). 144 | 145 | %% atomify_nodestrs(Strs) -> 146 | %% lists:foldl(fun("local", Acc) -> [node()|Acc]; 147 | %% (NodeStr, Acc) -> try 148 | %% [list_to_existing_atom(NodeStr)|Acc] 149 | %% catch error:badarg -> 150 | %% io:format("Bad node: ~s\n", [NodeStr]), 151 | %% Acc 152 | %% end 153 | %% end, [], Strs). 154 | 155 | %% 156 | %% @doc Join the ring found on the specified remote node 157 | %% @private 158 | %% 159 | try_join(NodeStr) when is_list(NodeStr) -> 160 | try_join(riak_core_util:str_to_node(NodeStr)); 161 | try_join(Node) when is_atom(Node) -> 162 | {ok, OurRingSize} = application:get_env(riak_core, ring_creation_size), 163 | case net_adm:ping(Node) of 164 | pong -> 165 | case rpc:call(Node, 166 | application, 167 | get_env, 168 | [riak_core, ring_creation_size]) of 169 | {ok, OurRingSize} -> 170 | riak_core_gossip:send_ring(Node, node()); 171 | _ -> 172 | {error, different_ring_sizes} 173 | end; 174 | pang -> 175 | {error, not_reachable} 176 | end. 177 | -------------------------------------------------------------------------------- /src/riak_zab_ensemble_master.erl: -------------------------------------------------------------------------------- 1 | -module(riak_zab_ensemble_master). 2 | -behaviour(gen_server). 3 | -include_lib("riak_zab_peer.hrl"). 4 | 5 | %% API 6 | -export([start_link/0]). 7 | -export([command/3, command/4, sync_command/3, get_ensemble_size/0]). 8 | 9 | %% API for riak-zab-admin 10 | -export([zab_up/1, zab_down/1, zab_info/1]). 11 | 12 | %% API used by riak_zab_peer 13 | -export([get_peers/1]). 14 | 15 | -export([ring_changed/0]). 16 | -export([init/1, handle_call/3, handle_cast/2, handle_info/2, 17 | terminate/2, code_change/3]). 18 | 19 | -type index() :: non_neg_integer(). 20 | -type preflist() :: [{index(), node()}]. 21 | -type ensemble() :: {[node()], [preflist()]}. 22 | 23 | -record(state, {esize :: pos_integer(), 24 | quorum :: pos_integer(), 25 | ringhash :: binary(), 26 | ensembles :: tid(), 27 | metabc :: term()}). 28 | 29 | -define(BC, riak_zab_bitcask_backend). 30 | 31 | %-define(DOUT(Msg, Args), io:format(Msg, Args)). 32 | -define(DOUT(Msg, Args), true). 33 | 34 | %% API 35 | start_link() -> 36 | gen_server:start_link({local, ?MODULE}, ?MODULE, [], []). 37 | 38 | -spec command(preflist(), term(), atom()) -> ok | {error, term()}. 39 | command(Preflist, Msg, VMaster) -> 40 | command(Preflist, Msg, noreply, VMaster). 41 | 42 | -spec command(preflist(), term(), zsender(), atom()) -> ok | {error, term()}. 43 | command(Preflist, Msg, Sender, _VMaster) -> 44 | Ensemble = ensemble(Preflist), 45 | {ENodes, _Prefs} = Ensemble, 46 | Leader = ensemble_leader(ENodes), 47 | case Leader of 48 | none -> 49 | {error, ensemble_down}; 50 | {_LNode, LPid} -> 51 | Proposal = make_proposal(Preflist, Sender, Msg), 52 | ?DOUT("Broadcast to ensemble: ~p :: ~p~n", [ENodes, Leader]), 53 | gen_fsm:send_event(LPid, Proposal), 54 | ok 55 | end. 56 | 57 | -spec sync_command(preflist(), term(), atom()) -> {ok, term()} | {error, term()}. 58 | sync_command(Preflist, Msg, _VMaster) -> 59 | Ensemble = ensemble(Preflist), 60 | {ENodes, _Prefs} = Ensemble, 61 | Leader = ensemble_leader(ENodes), 62 | case Leader of 63 | none -> 64 | {error, ensemble_down}; 65 | {_LNode, LPid} -> 66 | Sender = {fsm_sync, undefined, undefined}, 67 | Proposal = make_proposal(Preflist, Sender, Msg), 68 | ?DOUT("Broadcast to ensemble: ~p :: ~p~n", [ENodes, Leader]), 69 | gen_fsm:sync_send_event(LPid, Proposal) 70 | end. 71 | 72 | -spec get_ensemble_size() -> pos_integer(). 73 | get_ensemble_size() -> 74 | case application:get_env(riak_zab, ensemble_size) of 75 | {ok, Num} -> 76 | Num; 77 | _ -> 78 | error_logger:warning_msg("Ensemble_size unset, defaulting to 3~n", 79 | []), 80 | 3 81 | end. 82 | 83 | %% API used by riak-zab-admin 84 | -spec zab_up([]) -> ok. 85 | zab_up([]) -> 86 | gen_server:cast(?MODULE, zab_up). 87 | 88 | -spec zab_down([]) -> ok. 89 | zab_down([]) -> 90 | gen_server:cast(?MODULE, zab_down). 91 | 92 | -spec zab_info([]) -> ok. 93 | zab_info([]) -> 94 | ESize = get_ensemble_size(), 95 | {ok, Ring} = riak_core_ring_manager:get_my_ring(), 96 | RingSize = riak_core_ring:num_partitions(Ring), 97 | Ens = riak_zab_ensemble_util:all_ensembles(ESize), 98 | EInfo = [{Nodes, Prefs, ensemble_leader(Nodes)} || {Nodes, Prefs} <- Ens], 99 | io:format("~32..=s Riak Zab Info ~32..=s~n", ["", ""]), 100 | io:format("Ring size: ~B~n", [RingSize]), 101 | io:format("Ensemble size: ~B~n", [ESize]), 102 | io:format("Nodes: ~p~n", [riak_core_ring:all_members(Ring)]), 103 | io:format("~34..=s Ensembles ~34..=s~n", ["", ""]), 104 | io:format("~8s ~6s ~-30s ~s~n", 105 | ["Ensemble", "Ring", "Leader", "Nodes"]), 106 | io:format("~79..-s~n", [""]), 107 | lists:foldl(fun zab_ensemble_info/2, {RingSize, 1}, EInfo), 108 | ok. 109 | 110 | %% @private 111 | zab_ensemble_info({Nodes, Prefs, Leader}, {RingSize, Num}) -> 112 | RingPercent = erlang:length(Prefs) * 100 / RingSize, 113 | LeaderOut = case Leader of 114 | none -> "(down)"; 115 | {Node,_} -> Node 116 | end, 117 | io:format("~8B ~5.1f% ~-30.30s ~p~n", 118 | [Num, RingPercent, LeaderOut, Nodes]), 119 | io:format("~79..-s~n", [""]), 120 | {RingSize, Num + 1}. 121 | 122 | %% API used by riak-zab-peer 123 | %% Returns the pid for the riak_zab_peer processes associated with an ensemble 124 | -spec get_peers([node()]) -> [{node(), pid()}]. 125 | get_peers(Ensemble) -> 126 | {Peers, _} = gen_server:multi_call(Ensemble, ?MODULE, 127 | {get_peer, Ensemble}, 100), 128 | Peers. 129 | 130 | %% Called by riak_zab_ring_handler on ring change. 131 | ring_changed() -> 132 | gen_server:cast(?MODULE, ring_changed). 133 | 134 | %% @private 135 | init(_Args) -> 136 | ESize = get_ensemble_size(), 137 | Quorum = erlang:trunc(ESize / 2 + 1), 138 | RingHash = get_ring_hash(), 139 | {ok, BC} = ?BC:start(0, [{data_root, "data/zab_meta_bitcask"}]), 140 | timer:apply_after(500, gen_server, cast, [?MODULE, restart_ensembles]), 141 | State=#state{esize=ESize, 142 | quorum=Quorum, 143 | ringhash=RingHash, 144 | ensembles=ets:new(ensemble_tbl, []), 145 | metabc=BC}, 146 | {ok, State}. 147 | 148 | handle_cast(ring_changed, State=#state{ringhash=RingHash}) -> 149 | case get_ring_hash() of 150 | RingHash -> 151 | ?DOUT("Ring update. Same.~n", []), 152 | {noreply, State}; 153 | NewRingHash -> 154 | ?DOUT("Ring update. Changed~n", []), 155 | stop_all_ensembles(), 156 | {noreply, State#state{ringhash=NewRingHash}} 157 | end; 158 | handle_cast(zab_up, State) -> 159 | put_zab_ring_hash(get_ring_hash(), State), 160 | ensure_ensembles_started(State), 161 | {noreply, State}; 162 | handle_cast(zab_down, State) -> 163 | %% Delete ring hash so zab won't restart on node failure. 164 | del_zab_ring_hash(State), 165 | stop_all_ensembles(), 166 | {noreply, State}; 167 | handle_cast(restart_ensembles, State) -> 168 | RingHash = get_ring_hash(), 169 | ZabRingHash = get_zab_ring_hash(State), 170 | case RingHash == ZabRingHash of 171 | true -> 172 | ensure_ensembles_started(State), 173 | {noreply, State}; 174 | false -> 175 | %% Delete zab hash so that future restart attempts do not 176 | %% succeed given a future qualifying ring update. We want 177 | %% the user to manually restart zab if restart ever fails. 178 | del_zab_ring_hash(State), 179 | {noreply, State} 180 | end; 181 | handle_cast({ensemble_all_state_event, Ensemble, Event}, State) -> 182 | ensemble_all_state_event(Ensemble, Event), 183 | {noreply, State}; 184 | handle_cast(_Req, State) -> 185 | {noreply, State}. 186 | 187 | handle_call({ensemble, Preflist}, _From, State=#state{ensembles=Ens}) -> 188 | Reply = case ets:lookup(Ens, Preflist) of 189 | [] -> 190 | {[], []}; 191 | [{Preflist,Val}] -> 192 | Val 193 | end, 194 | {reply, Reply, State}; 195 | handle_call({get_peer, Ensemble}, _From, State) -> 196 | Pid = riak_zab_process:get_pid({peer, Ensemble}), 197 | {reply, Pid, State}; 198 | handle_call({ensemble_sync_all_state_event, Ensemble, Event}, From, State) -> 199 | spawn(fun() -> 200 | Reply = ensemble_sync_all_state_event(Ensemble, Event), 201 | case Reply of 202 | error -> noreply; 203 | Value -> gen_server:reply(From, Value) 204 | end 205 | end), 206 | {noreply, State}. 207 | %%handle_call(_Req, _From, State) -> 208 | %% {noreply, State}. 209 | 210 | handle_info(_Info, State) -> 211 | {noreply, State}. 212 | 213 | terminate(_Reason, _State) -> 214 | ok. 215 | 216 | code_change(_OldVsn, State, _Extra) -> 217 | {ok, State}. 218 | 219 | %% @private 220 | make_proposal(Preflist, Sender, Msg) -> 221 | #riak_zab_proposal{preflist=Preflist, 222 | sender=Sender, 223 | message=Msg}. 224 | 225 | %% @private 226 | ensemble_leader([]) -> 227 | ?DOUT("M: ensemble_leader :: empty enodes~n", []), 228 | none; 229 | ensemble_leader(ENodes) -> 230 | Quorum = quorum_size(ENodes), 231 | Msg = {ensemble_sync_all_state_event, ENodes, get_leader}, 232 | R = gen_server:multi_call(ENodes, ?MODULE, Msg, 1000), 233 | ?DOUT("M: ensemble_leader results: ~p~n", [R]), 234 | case R of 235 | {[], _} -> 236 | none; 237 | {Replies, _} -> 238 | {_Nodes, Leaders} = lists:unzip(Replies), 239 | Counts = count_elements(Leaders), 240 | Valid = [Leader || {Leader, Count} <- Counts, Count >= Quorum], 241 | case Valid of 242 | [] -> 243 | none; 244 | [Leader|_] -> 245 | Leader 246 | end 247 | end. 248 | 249 | %% Routes a gen_fsm all_state_event to a local ensemble peer. 250 | %% @private 251 | ensemble_all_state_event(Ensemble, Event) -> 252 | Pid = riak_zab_process:get_pid({peer, Ensemble}), 253 | ensemble_all_state_event(Pid, Ensemble, Event). 254 | ensemble_all_state_event(no_match, _Ensemble, _Event) -> 255 | ok; 256 | ensemble_all_state_event(Pid, _Ensemble, Event) -> 257 | gen_fsm:send_all_state_event(Pid, Event). 258 | 259 | %% Routes a gen_fsm sync_all_state_event to a local ensemble peer. 260 | %% @private 261 | ensemble_sync_all_state_event(Ensemble, Event) -> 262 | Pid = riak_zab_process:get_pid({peer, Ensemble}), 263 | ensemble_sync_all_state_event(Pid, Ensemble, Event). 264 | ensemble_sync_all_state_event(no_match, _Ensemble, _Event) -> 265 | error; 266 | ensemble_sync_all_state_event(Pid, _Ensemble, Event) -> 267 | gen_fsm:sync_send_all_state_event(Pid, Event). 268 | 269 | %% @private 270 | quorum_size([_]) -> 271 | 1; 272 | quorum_size(Ensemble) -> 273 | ESize = erlang:max(ordsets:size(Ensemble), get_ensemble_size()), 274 | erlang:trunc(ESize / 2 + 1). 275 | 276 | %% @private 277 | count_elements(L) -> 278 | Counts = lists:foldl(fun(E, Acc) -> 279 | dict:update_counter(E, 1, Acc) 280 | end, dict:new(), L), 281 | dict:to_list(Counts). 282 | 283 | %% @private 284 | -spec ensemble(preflist()) -> ensemble(). 285 | ensemble(Preflist) -> 286 | %% {_, L} = lists:unzip(Preflist), 287 | %% lists:sort(L). 288 | gen_server:call(?MODULE, {ensemble, Preflist}). 289 | 290 | %% @private 291 | get_ring_hash() -> 292 | {ok, Ring} = riak_core_ring_manager:get_my_ring(), 293 | crypto:md5(term_to_binary(Ring)). 294 | 295 | %% @private 296 | really_start_ensemble(App, Quorum, {Nodes, Prefs}) -> 297 | ?DOUT("Starting ensemble ~p~n", [{Nodes,Prefs}]), 298 | riak_zab_process:start_process({peer, Nodes}, riak_zab_peer, 299 | [App, Quorum, Nodes, Prefs]). 300 | 301 | %% @private 302 | start_ensemble(App, Quorum, Ensemble={Nodes, _Prefs}) -> 303 | Size = ordsets:size(Nodes), 304 | case {Size == 1, Size < Quorum} of 305 | {true, _} -> 306 | error_logger:warning_msg("Starting ensemble of size 1. Consistent " 307 | "but not highly available.~n", []), 308 | really_start_ensemble(App, 1, Ensemble); 309 | {false, true} -> 310 | error_logger:warning_msg("Ensemble can never meet quorum. " 311 | "Skipping.~n", []); 312 | {false, false} -> 313 | really_start_ensemble(App, Quorum, Ensemble) 314 | end. 315 | 316 | %% @private 317 | ensure_ensembles_started(State) -> 318 | case application:get_env(riak_zab, application_vnode) of 319 | {ok, App} -> 320 | ensure_ensembles_started(App, State); 321 | undefined -> 322 | error_logger:error_msg("Unable to start zab, no application vnode " 323 | "defined.~n") 324 | end. 325 | 326 | %% @private 327 | add_ensemble_to_index(Ensemble={_Nodes, Prefs}, State) -> 328 | [ets:insert(State#state.ensembles, {P, Ensemble}) || P <- Prefs]. 329 | 330 | %% @private 331 | ensure_ensembles_started(App, State=#state{esize=ESize, quorum=Quorum}) -> 332 | Ensembles = riak_zab_ensemble_util:all_ensembles(ESize), 333 | [begin 334 | add_ensemble_to_index(Ensemble, State), 335 | ShouldStart = ordsets:is_element(node(), Nodes), 336 | ShouldStart andalso start_ensemble(App, Quorum, Ensemble) 337 | end || Ensemble={Nodes,_} <- Ensembles]. 338 | 339 | %% @private 340 | stop_all_ensembles() -> 341 | Keys = riak_zab_process:all_processes(peer), 342 | [riak_zab_process:stop_process(K) || K <- Keys]. 343 | 344 | %% @private 345 | put_zab_ring_hash(RingHash, #state{metabc=BC}) -> 346 | ok = ?BC:put(BC, {<<"meta">>, <<"ringhash">>}, RingHash). 347 | 348 | %% @private 349 | del_zab_ring_hash(#state{metabc=BC}) -> 350 | ok = ?BC:delete(BC, {<<"meta">>, <<"ringhash">>}). 351 | 352 | %% @private 353 | get_zab_ring_hash(#state{metabc=BC}) -> 354 | case ?BC:get(BC, {<<"meta">>, <<"ringhash">>}) of 355 | {ok, Value} -> 356 | Value; 357 | _ -> 358 | <<>> 359 | end. 360 | -------------------------------------------------------------------------------- /src/riak_zab_ensemble_util.erl: -------------------------------------------------------------------------------- 1 | -module(riak_zab_ensemble_util). 2 | -export([all_ensembles/1]). 3 | 4 | -type index() :: non_neg_integer(). 5 | -type preflist() :: [{index(), node()}]. 6 | -type ensemble() :: {[node()], [preflist()]}. 7 | 8 | %% Determine the set of ensembles. Currently, there is one ensemble of each 9 | %% unique set of preflist owning nodes. 10 | -spec all_ensembles(pos_integer()) -> [ensemble()]. 11 | all_ensembles(Size) -> 12 | GroupF = fun(Prefs, Groups) -> 13 | {_Indicies, Nodes} = lists:unzip(Prefs), 14 | Key = lists:usort(Nodes), 15 | case dict:is_key(Key, Groups) of 16 | false -> 17 | dict:store(Key, [Prefs], Groups); 18 | true -> 19 | dict:append_list(Key, [Prefs], Groups) 20 | end 21 | end, 22 | {ok, Ring} = riak_core_ring_manager:get_my_ring(), 23 | AllPrefs = riak_core_ring:all_preflists(Ring, Size), 24 | PrefGroups = lists:foldl(GroupF, dict:new(), AllPrefs), 25 | dict:to_list(PrefGroups). 26 | -------------------------------------------------------------------------------- /src/riak_zab_fast_election_fsm.erl: -------------------------------------------------------------------------------- 1 | -module(riak_zab_fast_election_fsm). 2 | -behaviour(gen_fsm). 3 | 4 | %% API 5 | -export([start_link/4]). 6 | -export([init/1, handle_info/3, terminate/3, code_change/4, handle_event/3, 7 | handle_sync_event/4]). 8 | 9 | %% States 10 | -export([initialize/2, not_looking/2, looking/2, finalize/2]). 11 | 12 | -type peer() :: {node(), pid()}. 13 | -type zxid() :: {non_neg_integer(), non_neg_integer()}. 14 | 15 | -record(vote, {from :: peer(), 16 | leader :: peer(), 17 | zxid :: zxid(), 18 | epoch :: non_neg_integer(), 19 | state :: term()}). 20 | 21 | -record(state, {curvote :: #vote{}, 22 | proposed :: #vote{}, 23 | parent :: pid(), 24 | epoch :: non_neg_integer(), 25 | ensemble :: [node()], 26 | quorum :: non_neg_integer(), 27 | last_zxid :: zxid(), 28 | peers :: [peer()], 29 | ntimeout :: pos_integer(), 30 | votes :: term(), 31 | outofelection :: term()}). 32 | 33 | -define(ELECTION_FINALIZE_WAIT, 200). 34 | -define(ELECTION_MAX_WAIT, 60000). 35 | 36 | %-define(DOUT(Msg, Args), io:format(Msg, Args)). 37 | -define(DOUT(Msg, Args), true). 38 | 39 | -spec start_link(pid(), zxid(), [node()], pos_integer()) -> term(). 40 | start_link(ParentPid, LastZxid, Ensemble, Quorum) -> 41 | gen_fsm:start_link(?MODULE, [ParentPid, LastZxid, Ensemble, Quorum], []). 42 | 43 | %% @private 44 | init([ParentPid, LastZxid, Ensemble, Quorum]) -> 45 | State = #state{curvote=#vote{}, 46 | proposed=#vote{}, 47 | parent = ParentPid, 48 | epoch=0, 49 | ensemble=Ensemble, 50 | quorum=Quorum, 51 | last_zxid=LastZxid, 52 | peers=[], 53 | votes=dict:new(), 54 | outofelection=dict:new(), 55 | ntimeout=?ELECTION_FINALIZE_WAIT}, 56 | {ok, initialize, State, 0}. 57 | 58 | initialize(timeout, State=#state{epoch=Epoch}) -> 59 | SelfVote = nominate_self(Epoch, State), 60 | {next_state, not_looking, State#state{curvote=SelfVote, 61 | proposed=SelfVote}}. 62 | 63 | %% @private 64 | start_election(Peers, LastZxid, State=#state{epoch=Epoch}) -> 65 | NEpoch = Epoch + 1, 66 | State2 = State#state{last_zxid=LastZxid}, 67 | SelfVote = nominate_self(NEpoch, State2), 68 | State3 = State2#state{curvote=SelfVote, 69 | proposed=SelfVote, 70 | epoch=NEpoch, 71 | votes=dict:new(), 72 | peers=Peers, 73 | outofelection=dict:new(), 74 | ntimeout=?ELECTION_FINALIZE_WAIT}, 75 | {next_state, looking, State3, 0}. 76 | 77 | %% Peer is not looking, but responds to others who are with current vote. 78 | not_looking(#vote{from=Peer, state=looking}, 79 | State=#state{curvote=CurVote, parent=ParentPid, ensemble=Ens}) -> 80 | ParentState = riak_zab_peer:node_state(ParentPid), 81 | Reply = CurVote#vote{state=ParentState}, 82 | riak_zab_peer:election_event(Ens, Peer, Reply), 83 | {next_state, not_looking, State}; 84 | not_looking({start_election, Peers, LastZxid}, State) -> 85 | ?DOUT("Starting election~n", []), 86 | start_election(Peers, LastZxid, State); 87 | not_looking(_Event, State) -> 88 | {next_state, not_looking, State}. 89 | 90 | %% @private 91 | continue_looking(State=#state{ntimeout=Wait}) -> 92 | {next_state, looking, State, Wait}. 93 | 94 | looking({start_election, Peers, LastZxid}, State) -> 95 | ?DOUT("Restarting election~n", []), 96 | start_election(Peers, LastZxid, State); 97 | looking(timeout, State=#state{ntimeout=Wait}) -> 98 | ?DOUT("Sending notifications...~n", []), 99 | send_notifications(State), 100 | Wait2 = erlang:min(Wait*2, ?ELECTION_MAX_WAIT), 101 | continue_looking(State#state{ntimeout=Wait2}); 102 | looking(Vote=#vote{state=looking}, State) -> 103 | handle_vote(Vote, State); 104 | looking(Vote=#vote{}, State) -> 105 | handle_outofelection(Vote, State); 106 | looking(_Event, State) -> 107 | continue_looking(State). 108 | 109 | finalize(timeout, State) -> 110 | %% We have waited long enough, accept the proposed leader 111 | ?DOUT("Election complete, finalized! ~p~n", [State#state.proposed]), 112 | finish_election(State); 113 | finalize(Event, State) -> 114 | %% We have received an event, return to election process 115 | looking(Event, State). 116 | 117 | %% @private 118 | finish_election(State=#state{proposed=Elected, parent=Parent, epoch=Epoch}) -> 119 | #vote{leader=Leader, zxid=Zxid} = Elected, 120 | Elected2 = Elected#vote{epoch=Epoch}, 121 | ?DOUT("FE: Elected: ~p~n", [Elected2]), 122 | riak_zab_peer:elected(Parent, Leader, Zxid), 123 | {next_state, not_looking, State#state{curvote=Elected2}}. 124 | 125 | %% @private 126 | handle_vote(Vote=#vote{from=Peer, epoch=PeerEpoch}, 127 | State=#state{proposed=Proposed, epoch=Epoch, ensemble=Ens}) -> 128 | if 129 | PeerEpoch < Epoch -> 130 | %% Ignore vote from previous election and respond with proposal. 131 | ?DOUT("Ignoring old vote~n", []), 132 | riak_zab_peer:election_event(Ens, Peer, Proposed), 133 | continue_looking(State); 134 | PeerEpoch > Epoch -> 135 | %% Fast forward to more recent election. 136 | ?DOUT("Fast forwarding to new epoch~n", []), 137 | {_Changed, Proposed2} = update_proposed(Proposed, Vote), 138 | Proposed3 = Proposed2#vote{epoch=PeerEpoch}, 139 | State2 = State#state{epoch=PeerEpoch, 140 | proposed=Proposed3, 141 | votes=dict:new()}, 142 | send_notifications(State2), 143 | add_vote(Vote, State2); 144 | PeerEpoch == Epoch -> 145 | ?DOUT("Considering vote...~n", []), 146 | {Changed, Proposed2} = update_proposed(Proposed, Vote), 147 | State2 = State#state{proposed=Proposed2}, 148 | %% Send notifications if vote has changed 149 | case Changed of 150 | true -> 151 | ?DOUT("Vote changed~p~n", [Proposed2]), 152 | send_notifications(State2); 153 | false -> 154 | ?DOUT("Vote didn't change~n", []), 155 | true 156 | end, 157 | add_vote(Vote, State2) 158 | end. 159 | 160 | %% @private 161 | add_vote(Vote, State=#state{votes=Votes, ensemble=Ens, proposed=Proposed}) -> 162 | NVotes = dict:store(Vote#vote.from, Vote, Votes), 163 | State2 = State#state{votes=NVotes}, 164 | ReceivedAll = (dict:size(NVotes) == ordsets:size(Ens)), 165 | HaveQuorum = count_votes(State2, Proposed, NVotes), 166 | case {HaveQuorum, ReceivedAll} of 167 | {false, _} -> 168 | %% No quorum yet, continue looking. 169 | continue_looking(State2); 170 | {true, true} -> 171 | %% Quorum met and we have heard from all nodes, election complete. 172 | ?DOUT("Election complete! ~p~n", [Proposed]), 173 | finish_election(State2); 174 | {true, false} -> 175 | %% Quorum met without heading from all nodes, move to finalize. 176 | {next_state, finalize, State2, ?ELECTION_FINALIZE_WAIT} 177 | end. 178 | 179 | %% @private 180 | handle_outofelection(Vote, State) when Vote#vote.epoch == State#state.epoch -> 181 | #vote{from=Peer, state=PeerState, leader=Leader} = Vote, 182 | case PeerState of 183 | leading -> 184 | %% There is at most one leader for each epoch, so if a peer 185 | %% claims to be the leader, then it must be the leader. 186 | ?DOUT("Found current leader: ~p~n", [Leader]), 187 | State2 = set_proposed(State, Vote), 188 | finish_election(State2); 189 | _ -> 190 | NVotes = dict:store(Peer, Vote, State#state.votes), 191 | State2 = State#state{votes=NVotes}, 192 | HaveQuorum = count_votes(State2, Vote, NVotes), 193 | IsLeader = check_leader(State2#state.outofelection, Leader, 194 | get_id(State)), 195 | if 196 | HaveQuorum and IsLeader -> 197 | ?DOUT("Found supported leader: ~p~n", [Leader]), 198 | State3 = set_proposed(State2, Vote), 199 | finish_election(State3); 200 | true -> 201 | check_outofelection(Vote, State) 202 | end 203 | end; 204 | handle_outofelection(Vote, State) -> 205 | check_outofelection(Vote, State). 206 | 207 | %% @private 208 | check_outofelection(Vote=#vote{from=Peer, leader=Leader, epoch=PeerEpoch}, 209 | State=#state{outofelection=OutVotes}) -> 210 | OutVotes2 = dict:store(Peer, Vote, OutVotes), 211 | State2 = State#state{outofelection=OutVotes2}, 212 | HaveQuorum = count_votes(State2, Vote, OutVotes2), 213 | IsLeader = check_leader(OutVotes2, Leader, get_id(State)), 214 | if 215 | HaveQuorum and IsLeader -> 216 | ?DOUT("Found leader from different epoch: ~p~n", [Leader]), 217 | State3 = set_proposed(State2, Vote), 218 | State4 = State3#state{epoch=PeerEpoch}, 219 | finish_election(State4); 220 | true -> 221 | continue_looking(State2) 222 | end. 223 | 224 | handle_info(_Msg, StateName, State) -> 225 | {next_state, StateName, State}. 226 | 227 | %% send_all_state_event is used by riak_zab_peer to route messages to the 228 | %% election FSM. Chain to the defined state handlers. 229 | handle_event(Event, StateName, State) -> 230 | ?MODULE:StateName(Event, State). 231 | 232 | handle_sync_event(_Event, _From, StateName, State) -> 233 | {next_state, StateName, State}. 234 | 235 | %% @private 236 | get_id(State) -> 237 | {node(), State#state.parent}. 238 | 239 | %% @private 240 | nominate_self(Epoch, State=#state{last_zxid=LastZxid}) -> 241 | Self = get_id(State), 242 | #vote{from=Self, leader=Self, zxid=LastZxid, epoch=Epoch}. 243 | 244 | %% @private 245 | total_order(NewId, NewZxid, CurId, CurZxid) -> 246 | (NewZxid > CurZxid) orelse ((NewZxid == CurZxid) andalso (NewId > CurId)). 247 | 248 | %% @private 249 | update_proposed(Proposed=#vote{leader=CurLeader, zxid=CurZxid}, 250 | _Vote=#vote{leader=NewLeader, zxid=NewZxid}) -> 251 | Proposed2 = case total_order(NewLeader, NewZxid, CurLeader, CurZxid) of 252 | true -> 253 | Proposed#vote{leader=NewLeader, zxid=NewZxid}; 254 | false -> 255 | Proposed 256 | end, 257 | {Proposed2 =/= Proposed, Proposed2}. 258 | 259 | %% @private 260 | send_notifications(#state{proposed=Proposed, ensemble=Ens, peers=Peers}) -> 261 | Vote = Proposed#vote{state=looking}, 262 | [riak_zab_peer:election_event(Ens, Peer, Vote) || Peer <- Peers]. 263 | 264 | %% @private 265 | count_votes(#state{quorum=Quorum}, Proposed, Votes) -> 266 | {_, NumVotes} = dict:fold(fun check_vote/3, {Proposed, 0}, Votes), 267 | ?DOUT("Checking votes: ~p / ~p ~n", [NumVotes, Quorum]), 268 | ?DOUT("Ensemble: ~p~n", [State#state.ensemble]), 269 | ?DOUT(" Votes: ~p~n", [dict:to_list(Votes)]), 270 | if 271 | NumVotes >= Quorum -> 272 | ?DOUT("Found quorum~n", []), 273 | true; 274 | NumVotes < Quorum -> 275 | ?DOUT("No quorum yet~n", []), 276 | false 277 | end. 278 | 279 | %% @private 280 | check_vote(_PeerId, Vote, {Proposed, Count}) -> 281 | V1 = {Vote#vote.leader, Vote#vote.zxid, Vote#vote.epoch}, 282 | V2 = {Proposed#vote.leader, Proposed#vote.zxid, Proposed#vote.epoch}, 283 | case V1 of 284 | V2 -> 285 | {Proposed, Count + 1}; 286 | _ -> 287 | {Proposed, Count} 288 | end. 289 | 290 | %% (Comment from Zookeeper Implementation) 291 | %% In the case there is a leader elected, and a quorum supporting 292 | %% this leader, we have to check if the leader has voted and acked 293 | %% that it is leading. We need this check to avoid that peers keep 294 | %% electing over and over a peer that has crashed and it is no 295 | %% longer leading. 296 | %% 297 | %% @private 298 | check_leader(_Votes, Leader, MyId) when (Leader == MyId) -> 299 | %% If everyone thinks I'm the leader, then I must be. 300 | true; 301 | check_leader(Votes, Leader, _MyId) -> 302 | case dict:find(Leader, Votes) of 303 | error -> 304 | false; 305 | {ok, Vote} -> 306 | Vote#vote.state == leading 307 | end. 308 | 309 | %% @private 310 | set_proposed(State=#state{proposed=Proposed}, 311 | #vote{leader=Leader, zxid=Zxid}) -> 312 | Proposed2 = Proposed#vote{leader=Leader, zxid=Zxid}, 313 | State#state{proposed=Proposed2}. 314 | 315 | %% @private 316 | terminate(_Reason, _StateName, _State) -> 317 | ok. 318 | 319 | %% @private 320 | code_change(_OldVsn, StateName, State, _Extra) -> 321 | {ok, StateName, State}. 322 | -------------------------------------------------------------------------------- /src/riak_zab_leader_test.erl: -------------------------------------------------------------------------------- 1 | -module(riak_zab_leader_test). 2 | -include_lib("eunit/include/eunit.hrl"). 3 | -compile(export_all). 4 | 5 | setup_zab() -> 6 | Nodes = ['dev1@127.0.0.1', 'dev2@127.0.0.1', 'dev3@127.0.0.1'], 7 | net_kernel:start(['test@127.0.0.1']), 8 | erlang:set_cookie(node(), riak_zab), 9 | mock_ensemble(Nodes), 10 | Nodes. 11 | 12 | cleanup_zab(Nodes) -> 13 | [rpc:cast(N, riak_zab_ensemble_master, zab_down, [[]]) || N <- Nodes], 14 | ep:stop(Nodes), 15 | unload_mocks(), 16 | ok. 17 | 18 | start_zab(Nodes) -> 19 | [begin 20 | rpc:call(N, riak_zab_ensemble_master, zab_up, [[]], 75), 21 | timer:sleep(250) 22 | end || N <- Nodes]. 23 | 24 | get_mock_ensembles(Size) -> 25 | GroupF = fun(Prefs, Groups) -> 26 | {_Indicies, Nodes} = lists:unzip(Prefs), 27 | Key = lists:usort(Nodes), 28 | case dict:is_key(Key, Groups) of 29 | false -> 30 | dict:store(Key, [Prefs], Groups); 31 | true -> 32 | dict:append_list(Key, [Prefs], Groups) 33 | end 34 | end, 35 | {ok, Ring} = riak_core_ring_manager:get_my_ring(), 36 | AllPrefs = riak_core_ring:all_preflists(Ring, Size), 37 | PrefGroups = lists:foldl(GroupF, dict:new(), AllPrefs), 38 | L = dict:to_list(PrefGroups), 39 | E1 = ['dev1@127.0.0.1', 'dev2@127.0.0.1', 'dev3@127.0.0.1'], 40 | [{Key, Prefs} || {Key, Prefs} <- L, Key == E1]. 41 | 42 | mock_ensemble(Nodes) -> 43 | [rpc:call(N, ?MODULE, mock_ensemble, [], 75) || N <- Nodes], 44 | ok. 45 | mock_ensemble() -> 46 | code:unstick_mod(riak_zab_ensemble_util), 47 | meck:new(riak_zab_ensemble_util, [passthrough, no_link]), 48 | meck:expect(riak_zab_ensemble_util, all_ensembles, 49 | fun(Size) -> get_mock_ensembles(Size) end). 50 | 51 | unload_mocks(Nodes) -> 52 | [rpc:call(N, ?MODULE, unload_mocks, [], 75) || N <- Nodes], 53 | ok. 54 | unload_mocks() -> 55 | meck:unload(riak_zab_ensemble_util). 56 | 57 | test_initialize(Nodes) -> 58 | mock_ensemble(), 59 | ep:load_expectations(?MODULE, initialize_expect, [], Nodes), 60 | start_zab(Nodes), 61 | timer:sleep(2000), 62 | ?assert(ep:check_expectations(Nodes)). 63 | initialize_expect() -> 64 | ep:new(), 65 | 66 | %% Nodes start correctly 67 | ep:once([], 'dev1@127.0.0.1', initialize, ep:wait()), 68 | ep:once([], 'dev2@127.0.0.1', initialize, ep:wait()), 69 | ep:once([], 'dev3@127.0.0.1', initialize, ep:wait()), 70 | 71 | ep:start(), 72 | ok. 73 | 74 | test_initial_election(Nodes) -> 75 | ep:load_expectations(?MODULE, initial_election_expect, [], Nodes), 76 | ep:release(Nodes), 77 | timer:sleep(7000), 78 | ?assert(ep:check_expectations(Nodes)). 79 | initial_election_expect() -> 80 | ep:clear(), 81 | 82 | %% Nodes all start election 83 | S1 = ep:once([], 'dev1@127.0.0.1', start_election), 84 | S2 = ep:once([], 'dev2@127.0.0.1', start_election), 85 | S3 = ep:once([], 'dev3@127.0.0.1', start_election), 86 | 87 | %% Nodes elect dev3 88 | ep:once([S1], 'dev1@127.0.0.1', {elected, 'dev3@127.0.0.1'}), 89 | ep:once([S2], 'dev2@127.0.0.1', {elected, 'dev3@127.0.0.1'}), 90 | 91 | %% Saving dev3's ensemble peer pid in order to kill it in the next test 92 | ep:once([S3], 'dev3@127.0.0.1', {elected, 'dev3@127.0.0.1'}, 93 | fun() -> 94 | ep:put(leader, self()) 95 | end), 96 | 97 | ep:start(), 98 | ok. 99 | 100 | test_kill_leader(Nodes) -> 101 | ep:load_expectations(?MODULE, kill_leader_expect, [], Nodes), 102 | 103 | %% Let's kill dev3 to kick off the expected behavior 104 | {ok, Pid} = ep:get('dev3@127.0.0.1', leader), 105 | exit(Pid, kill), 106 | 107 | timer:sleep(10000), 108 | ?assert(ep:check_expectations(Nodes)). 109 | kill_leader_expect() -> 110 | ep:clear(), 111 | 112 | %% dev3 should restart, which we delay to ensure dev2 is elected. 113 | _I3 = ep:once([], 'dev3@127.0.0.1', initialize, ep:wait()), 114 | 115 | %% dev1/2 should start a new election, and elect dev2 116 | S1 = ep:once([], 'dev1@127.0.0.1', start_election), 117 | S2 = ep:once([], 'dev2@127.0.0.1', start_election), 118 | 119 | ep:once([S1], 'dev1@127.0.0.1', {elected, 'dev2@127.0.0.1'}), 120 | ep:once([S2], 'dev2@127.0.0.1', {elected, 'dev2@127.0.0.1'}), 121 | 122 | ep:start(), 123 | ok. 124 | 125 | test_elect_existing(Nodes) -> 126 | ep:load_expectations(?MODULE, elect_existing_expect, [], Nodes), 127 | ep:release(Nodes), 128 | timer:sleep(7000), 129 | ?assert(ep:check_expectations(Nodes)). 130 | elect_existing_expect() -> 131 | %% dev3 will start election and elect existing leader dev2 132 | S3 = ep:once([], 'dev3@127.0.0.1', start_election), 133 | ep:once([S3], 'dev3@127.0.0.1', {elected, 'dev2@127.0.0.1'}), 134 | 135 | ep:start(), 136 | ok. 137 | 138 | run_timed(Nodes, TestFn, Time) -> 139 | {timeout, Time, ?_test(TestFn(Nodes))}. 140 | 141 | leader_test_() -> 142 | {setup, 143 | fun setup_zab/0, 144 | fun cleanup_zab/1, 145 | fun(Nodes) -> 146 | {inorder, 147 | [run_timed(Nodes, fun test_initialize/1, 20000), 148 | run_timed(Nodes, fun test_initial_election/1, 20000), 149 | run_timed(Nodes, fun test_kill_leader/1, 20000), 150 | run_timed(Nodes, fun test_elect_existing/1, 20000)]} 151 | end}. 152 | -------------------------------------------------------------------------------- /src/riak_zab_log.erl: -------------------------------------------------------------------------------- 1 | -module(riak_zab_log). 2 | -compile(export_all). 3 | 4 | -define(BC, riak_zab_bitcask_backend). 5 | 6 | -record(synclog, {sync_log :: term(), 7 | sync_idx :: non_neg_integer()}). 8 | 9 | %-define(DOUT(Msg, Args), io:format(Msg, Args)). 10 | -define(DOUT(Msg, Args), true). 11 | 12 | %% History API 13 | init_history_log(VG) -> 14 | <> = crypto:md5(term_to_binary(VG)), 15 | {ok, BC} = ?BC:start(P, [{data_root, "data/zab_bitcask"}]), 16 | F = fun({propose, Zxid}, _V, {MaxP, MaxC}) -> 17 | NewP = erlang:max(Zxid, MaxP), 18 | {NewP, MaxC}; 19 | ({last_commit, _Prefs}, Zxid, {MaxP, MaxC}) -> 20 | NewC = erlang:max(binary_to_term(Zxid), MaxC), 21 | {MaxP, NewC}; 22 | ({_B, _K}, _V, {MaxP, MaxC}) -> 23 | {MaxP, MaxC} 24 | end, 25 | {LastProposedZxid, LastCommittedZxid} = ?BC:fold(BC, F, {{0,0}, {0,0}}), 26 | LastZxid = erlang:max(LastProposedZxid, LastCommittedZxid), 27 | {LastZxid, LastCommittedZxid, BC}. 28 | 29 | get_or_default(BC, BKey, Default) -> 30 | case ?BC:get(BC, BKey) of 31 | {ok, Value} -> 32 | binary_to_term(Value); 33 | _ -> 34 | Default 35 | end. 36 | 37 | get_old_proposals(BC, LastCommittedZxid) -> 38 | ?BC:fold(BC, 39 | fun({propose, Zxid}, V, L) -> 40 | case Zxid > LastCommittedZxid of 41 | true -> [{Zxid, binary_to_term(V)} | L]; 42 | false -> L 43 | end; 44 | ({_B, _K}, _V, L) -> 45 | L 46 | end, 47 | []). 48 | 49 | sync(BC={Ref,_}) -> 50 | bitcask:sync(Ref), 51 | BC. 52 | 53 | log_proposal(Zxid, Msg, BC) -> 54 | ok = ?BC:put(BC, {propose, Zxid}, term_to_binary(Msg)), 55 | sync(BC). 56 | 57 | get_proposed_keys(BC) -> 58 | ?BC:list_bucket(BC, propose). 59 | 60 | get_proposal(Zxid, BC) -> 61 | {ok, Value} = ?BC:get(BC, {propose, Zxid}), 62 | binary_to_term(Value). 63 | 64 | del_proposal(Zxid, BC) -> 65 | ok = ?BC:delete(BC, {propose, Zxid}), 66 | sync(BC). 67 | 68 | log_accepted_epoch(BC, Epoch) -> 69 | ok = ?BC:put(BC, {meta, accepted_epoch}, term_to_binary(Epoch)), 70 | sync(BC). 71 | 72 | log_current_epoch(BC, Epoch) -> 73 | ok = ?BC:put(BC, {meta, current_epoch}, term_to_binary(Epoch)), 74 | sync(BC). 75 | 76 | log_last_commit(Preflist, Zxid, BC) -> 77 | ok = ?BC:put(BC, {last_commit, Preflist}, term_to_binary(Zxid)), 78 | sync(BC). 79 | 80 | get_accepted_epoch(BC) -> 81 | get_or_default(BC, {meta, accepted_epoch}, 0). 82 | get_current_epoch(BC) -> 83 | get_or_default(BC, {meta, current_epoch}, 0). 84 | get_last_commit(Preflist, BC) -> 85 | get_or_default(BC, {last_commit, Preflist}, {0,0}). 86 | 87 | clear_proposals(BC) -> 88 | Keys = get_proposed_keys(BC), 89 | [?BC:delete(BC, Key) || Key <- Keys], 90 | sync(BC). 91 | 92 | init_sync_log(VG) -> 93 | <> = crypto:md5(term_to_binary(VG)), 94 | {ok, BC} = ?BC:start(P, [{data_root, "data/zab_sync_bitcask"}]), 95 | SyncIdx = ?BC:fold(BC, 96 | fun({_B,K}, _V, Max) -> erlang:max(K, Max) end, 97 | 0), 98 | #synclog{sync_log=BC, sync_idx=SyncIdx}. 99 | 100 | log_sync_message(Msg, SyncLog=#synclog{sync_log=BC, sync_idx=Idx}) -> 101 | Idx2 = Idx + 1, 102 | ?BC:put(BC, {<<"sync">>, Idx2}, term_to_binary(Msg)), 103 | ?DOUT("Sync: Logging ~p~n", [{Idx2, Msg}]), 104 | SyncLog#synclog{sync_idx=Idx2}. 105 | 106 | clear_sync_log(SyncLog=#synclog{sync_log=BC, sync_idx=Idx}) -> 107 | [?BC:delete(BC, {<<"sync">>, N}) || N <- lists:seq(1,Idx)], 108 | SyncLog#synclog{sync_idx=0}. 109 | 110 | fold_sync_log(SyncLog=#synclog{}, Fun, Acc) -> 111 | fold_sync_log(1, SyncLog, Fun, Acc). 112 | 113 | fold_sync_log(Idx, #synclog{sync_idx=N}, _Fun, Acc) when Idx > N -> 114 | Acc; 115 | fold_sync_log(Idx, SyncLog=#synclog{sync_log=BC}, Fun, Acc) -> 116 | {ok, Value} = ?BC:get(BC, {<<"sync">>, Idx}), 117 | Msg = binary_to_term(Value), 118 | ?DOUT("Sync: Handling ~p~n", [{Idx, Msg}]), 119 | Acc2 = Fun(Msg, Acc), 120 | fold_sync_log(Idx + 1, SyncLog, Fun, Acc2). 121 | 122 | %% Old History API 123 | %% init_history_log() -> 124 | %% {{0,0}, {0,0}, dict:new()}. 125 | 126 | %% get_old_proposals(#state{history=History, 127 | %% last_committed_zxid=LastCommittedZxid}) -> 128 | %% lists:keysort(1, dict:to_list(dict:filter(fun(K,_V) -> (K > LastCommittedZxid) end, History))). 129 | 130 | %% log_proposal(Zxid, Msg, History) -> 131 | %% dict:store(Zxid, Msg, History). 132 | 133 | %% get_proposal(Zxid, History) -> 134 | %% dict:fetch(Zxid, History). 135 | 136 | -------------------------------------------------------------------------------- /src/riak_zab_peer.erl: -------------------------------------------------------------------------------- 1 | -module(riak_zab_peer). 2 | -behaviour(gen_fsm). 3 | -include_lib("riak_zab_peer.hrl"). 4 | 5 | %% API 6 | -export([start_link/1]). 7 | -export([init/1, handle_info/3, terminate/3, code_change/4, handle_event/3, 8 | handle_sync_event/4]). 9 | 10 | %% API used by election FSM 11 | -export([node_state/1, elected/3, election_event/3]). 12 | -export([send_sync_message/2]). 13 | 14 | %% Shared leader/follower states 15 | -export([initialize/2, looking/2]). 16 | %% Leader states 17 | -export([leading/2, leading/3, lead_new_quorum/2, lead_new_epoch/2, 18 | lead_synchronize/2]). 19 | %% Follower states 20 | -export([following/2, follow_new_quorum/2, follow_new_epoch/2, 21 | follow_synchronize/2]). 22 | 23 | %% Type definitions 24 | -type index() :: non_neg_integer(). 25 | -type preflist() :: [{index(), node()}]. 26 | %%-type ensemble() :: {[node()], [preflist()]}. 27 | -type zxid() :: {non_neg_integer(), non_neg_integer()}. 28 | -type allzxids() :: [{preflist(), [{index(), zxid()}]}]. 29 | -type peer() :: {node(), pid()}. 30 | 31 | -record(state, {node_state, 32 | electmod, 33 | electfsm, 34 | timer, 35 | ping_timer, 36 | ping_acks, 37 | id, 38 | app, 39 | quorum, 40 | ensemble, 41 | synchronize_pid, 42 | peerinfo, 43 | peers, 44 | preflists, 45 | history, 46 | sync_log, 47 | last_zxid, 48 | last_commit_zxid, 49 | followers0, 50 | followers, 51 | accepted_epoch, 52 | largest_epoch, 53 | current_epoch, 54 | proposals, 55 | leader}). 56 | 57 | -record(serverinfo, {from, 58 | current_epoch, 59 | accepted_epoch, 60 | zxid}). 61 | 62 | -include_lib("riak_zab_vnode.hrl"). 63 | -include_lib("eprobe/include/eprobe.hrl"). 64 | 65 | -define(TIMEOUT, 4000). 66 | -define(PINGTIME, 1000). 67 | 68 | -define(INFO, error_logger:info_msg). 69 | 70 | %-define(DOUT(Msg, Args), io:format(Msg, Args)). 71 | -define(DOUT(Msg, Args), true). 72 | 73 | %% API 74 | start_link(Args) -> 75 | gen_fsm:start_link(?MODULE, Args, []). 76 | 77 | %% API used by riak_zab_vnode synchronization 78 | -spec send_sync_message({index(), pid()}, term()) -> ok. 79 | send_sync_message({Idx, Pid}, Msg) -> 80 | gen_fsm:send_event(Pid, {sync, Idx, Msg}). 81 | 82 | %% API used by election FSM 83 | -spec node_state(pid()) -> looking | leading | following. 84 | node_state(Pid) -> 85 | gen_fsm:sync_send_all_state_event(Pid, node_state). 86 | 87 | -spec election_event(any(), peer(), term()) -> ok. 88 | election_event(_Ensemble, {_Node, Pid}, Event) -> 89 | gen_fsm:send_all_state_event(Pid, {election, Event}). 90 | 91 | -spec elected(pid(), peer(), zxid()) -> ok. 92 | elected(Pid, Leader, Zxid) -> 93 | gen_fsm:send_event(Pid, {elected, Leader, Zxid}). 94 | 95 | %% @private 96 | init([App, Quorum, Ensemble, Prefs]) -> 97 | ElectMod = riak_zab_fast_election_fsm, 98 | {LastZxid, LastCommittedZxid, History} = 99 | riak_zab_log:init_history_log(Ensemble), 100 | SyncLog = riak_zab_log:init_sync_log(Ensemble), 101 | AcceptedEpoch = riak_zab_log:get_accepted_epoch(History), 102 | CurrentEpoch = riak_zab_log:get_current_epoch(History), 103 | 104 | ?INFO("~p: riak_zab_peer started~n" 105 | " Quorum: ~p~n" 106 | " Ensemble: ~p~n" 107 | " Last Zxid: ~p~n" 108 | " Laxt Commit: ~p~n" 109 | " Current Epoch: ~p~n" 110 | " Accepted Epoch: ~p~n", 111 | [self(), Quorum, Ensemble, LastZxid, LastCommittedZxid, 112 | CurrentEpoch, AcceptedEpoch]), 113 | %timer:sleep(2000), 114 | 115 | %% Ensure history sanity 116 | {LastEpoch, _} = LastZxid, 117 | true = (LastCommittedZxid =< LastZxid), 118 | true = (CurrentEpoch =< AcceptedEpoch), 119 | true = (LastEpoch =< CurrentEpoch), 120 | 121 | State = #state{node_state=looking, 122 | electmod=ElectMod, 123 | electfsm=undefined, 124 | timer=none, 125 | ping_timer=none, 126 | ping_acks=ordsets:new(), 127 | id=get_id(), 128 | app=App, 129 | quorum=Quorum, 130 | ensemble=Ensemble, 131 | peerinfo = ets:new(peertable, []), 132 | peers = [], 133 | preflists=Prefs, 134 | history=History, 135 | sync_log = SyncLog, 136 | followers0=[], 137 | followers=[], 138 | last_zxid=LastZxid, 139 | last_commit_zxid=LastCommittedZxid, 140 | proposals=dict:new(), 141 | accepted_epoch=AcceptedEpoch, 142 | largest_epoch=0, 143 | current_epoch=CurrentEpoch}, 144 | {ok, initialize, State, 0}. 145 | 146 | initialize(timeout, State=#state{electmod=ElectMod, ensemble=Ens, quorum=Q, 147 | last_zxid=LastZxid}) -> 148 | ?PROBE(node(), initialize), 149 | {ok, Pid} = ElectMod:start_link(self(), LastZxid, Ens, Q), 150 | State2 = State#state{electfsm=Pid}, 151 | State3 = play_sync_log(State2), 152 | start_election(State3). 153 | 154 | looking(timeout, State) -> 155 | start_election(State); 156 | looking(cancel, State) -> 157 | start_election(State); 158 | looking({elected, Leader, Zxid}, State) -> 159 | monitor_peer(Leader, State), 160 | State2 = restart_timer(State), 161 | ?PROBE(node(), {elected, element(1,Leader)}), 162 | case Leader == get_id() of 163 | true -> 164 | ?INFO("riak_zab_peer: elected myself :: ~p~n", [[Leader, Zxid]]), 165 | case State#state.quorum of 166 | 1 -> 167 | %% Quorum of one. Move straight to leading/broadcast. 168 | Epoch = State#state.accepted_epoch + 1, 169 | State3 = restart_timer(State2), 170 | State4 = restart_ping_timer(State3), 171 | State5 = update_accepted_epoch(Epoch, State4), 172 | State6 = update_current_epoch(Epoch, State5), 173 | State7 = State6#state{leader=Leader, 174 | node_state=leading, 175 | followers=[get_id()], 176 | last_zxid={Epoch, 0}, 177 | proposals=dict:new()}, 178 | {next_state, leading, State7}; 179 | _ -> 180 | State3 = State2#state{node_state=leading, 181 | leader=Leader, 182 | followers=[get_id()]}, 183 | {next_state, lead_new_quorum, State3} 184 | end; 185 | false -> 186 | ?INFO("riak_zab_peer: elected ~p~n", [[Leader, Zxid]]), 187 | State3 = State2#state{node_state=following, leader=Leader}, 188 | {next_state, follow_new_quorum, State3, 0} 189 | end; 190 | looking(_Event, State) -> 191 | {next_state, looking, State}. 192 | 193 | %% @private 194 | wait_for_quorum(Peer, StateName, NextStateName, 195 | State=#state{quorum=Quorum, followers=Followers}) -> 196 | Followers2 = ordsets:add_element(Peer, Followers), 197 | case length(Followers2) >= Quorum of 198 | true -> 199 | State2 = State#state{followers0=Followers2, followers=[]}, 200 | State3 = restart_timer(State2), 201 | ?MODULE:NextStateName(init, State3); 202 | false -> 203 | State2 = State#state{followers=Followers2}, 204 | {next_state, StateName, State2} 205 | end. 206 | 207 | lead_new_quorum(cancel, State) -> 208 | ?INFO("L: ~p :: Timed out while establishing quorum~n", [self()]), 209 | start_election(State); 210 | lead_new_quorum(#serverinfo{from=From, 211 | current_epoch=CurrentEpoch, 212 | accepted_epoch=AcceptedEpoch, 213 | zxid=FollowerZxid}, 214 | State=#state{current_epoch=LeaderEpoch, 215 | largest_epoch=LargestEpoch, 216 | last_zxid=LastZxid}) -> 217 | monitor_peer(From, State), 218 | peer_send_event(State, From, msg_serverinfo(State)), 219 | case valid_follower_epoch(LeaderEpoch, LastZxid, 220 | CurrentEpoch, FollowerZxid) of 221 | true -> 222 | LargestEpoch2 = erlang:max(AcceptedEpoch, LargestEpoch), 223 | State2 = State#state{largest_epoch=LargestEpoch2}, 224 | wait_for_quorum(From, lead_new_quorum, lead_new_epoch, State2); 225 | false -> 226 | {next_state, lead_new_quorum, State} 227 | end; 228 | lead_new_quorum(_Event, State) -> 229 | {next_state, lead_new_quorum, State}. 230 | 231 | lead_new_epoch(cancel, State) -> 232 | ?INFO("L: ~p :: Timed out while establishing epoch~n", [self()]), 233 | start_election(State); 234 | lead_new_epoch(init, State=#state{followers0=Followers, 235 | largest_epoch=LargestEpoch}) -> 236 | ?INFO("L: ~p :: New quorum established :: ~p~n", [self(), Followers]), 237 | NewEpochMsg = {newepoch, get_id(), LargestEpoch+1}, 238 | [peer_send_event(State, N, NewEpochMsg) || N <- Followers, N /= get_id()], 239 | {next_state, lead_new_epoch, State#state{followers=[get_id()]}}; 240 | lead_new_epoch({ack, From, {Epoch,0}}, State) 241 | when (Epoch == (State#state.largest_epoch+1)) -> 242 | wait_for_quorum(From, lead_new_epoch, lead_synchronize, State); 243 | lead_new_epoch(_Event, State) -> 244 | {next_state, lead_new_epoch, State}. 245 | 246 | lead_synchronize(cancel, State) -> 247 | ?INFO("L: ~p :: Timed out while trying to synchronize~n", [self()]), 248 | start_election(State); 249 | lead_synchronize(init, State=#state{largest_epoch=LargestEpoch, 250 | followers0=Followers0}) -> 251 | Epoch = LargestEpoch + 1, 252 | ?INFO("L: ~p :: New epoch (~p) established :: ~p~n", 253 | [self(), Epoch, Followers0]), 254 | State2 = update_accepted_epoch(Epoch, State), 255 | 256 | ?INFO("L: ~p :: Starting synchronization~n", [self()]), 257 | Proposals = get_old_proposals(State2), 258 | LeaderAllZxids = get_all_zxids(State2), 259 | 260 | %% Synchronize in parallel 261 | SyncFn = 262 | fun(N) -> 263 | synchronize_peer(LeaderAllZxids, Proposals, N, Epoch, State2) 264 | end, 265 | SyncPid = 266 | spawn(fun() -> [spawn_link(fun() -> SyncFn(N) end) || N <- Followers0, 267 | N /= get_id()] 268 | end), 269 | 270 | State3 = State2#state{synchronize_pid=SyncPid, followers=[get_id()]}, 271 | {next_state, lead_synchronize, State3}; 272 | lead_synchronize({ack, From, Zxid}, State=#state{accepted_epoch=Epoch}) 273 | when (Zxid == {Epoch,0}) -> 274 | wait_for_quorum(From, lead_synchronize, leading, State); 275 | lead_synchronize(_Event, State) -> 276 | {next_state, lead_synchronize, State}. 277 | 278 | leading(init, State=#state{followers0=Followers0, accepted_epoch=Epoch}) -> 279 | ?INFO("L: ~p :: leading ; followers: ~p~n", [self(), Followers0]), 280 | [peer_send_event(State, N, {uptodate, get_id()}) 281 | || N <- Followers0, N /= get_id()], 282 | State2 = restart_timer(State), 283 | State3 = restart_ping_timer(State2), 284 | State4 = update_current_epoch(Epoch, State3), 285 | {next_state, leading, State4#state{followers=Followers0, 286 | last_zxid={Epoch, 0}, 287 | proposals=dict:new()}}; 288 | leading(cancel, State=#state{ping_acks=Acks}) -> 289 | Quorum = State#state.quorum, 290 | case length(Acks) >= Quorum of 291 | true -> 292 | State2 = restart_timer(State), 293 | {next_state, leading, State2#state{ping_acks=ordsets:new()}}; 294 | false -> 295 | ?INFO("L: ~p :: Timed out while leading~n", [self()]), 296 | start_election(State) 297 | end; 298 | leading(ping, State=#state{followers=Followers}) -> 299 | [peer_send_event(State, N, {ping, get_id()}) || N <- Followers], 300 | State2 = restart_ping_timer(State), 301 | {next_state, leading, State2}; 302 | leading({ackping, From}, State=#state{ping_acks=Acks}) -> 303 | Acks2 = ordsets:add_element(From, Acks), 304 | {next_state, leading, State#state{ping_acks=Acks2}}; 305 | leading(Event=?ZAB_PROPOSAL{}, State) -> 306 | {ok, State2} = handle_broadcast(Event, State), 307 | {next_state, leading, State2}; 308 | leading(Event={ping, _From}, State) -> 309 | {ok, State2} = handle_ping(Event, State), 310 | {next_state, leading, State2}; 311 | leading(Event={propose, _From, _Zxid, _Msg}, State) -> 312 | {ok, State2} = handle_propose(Event, State), 313 | {next_state, leading, State2}; 314 | leading(Event={commit, _From, _Zxid}, State) -> 315 | {ok, State2} = handle_commit(Event, State), 316 | {next_state, leading, State2}; 317 | leading(#serverinfo{from=From}, State=#state{current_epoch=Epoch, 318 | followers=Followers}) -> 319 | monitor_peer(From, State), 320 | peer_send_event(State, From, msg_serverinfo(State)), 321 | peer_send_event(State, From, {newepoch, get_id(), Epoch}), 322 | synchronize_peer(From, Epoch, State), 323 | peer_send_event(State, From, {uptodate, get_id()}), 324 | ?INFO("L: ~p :: Synchronized with new follower: ~p~n", [self(), From]), 325 | Followers2 = ordsets:add_element(From, Followers), 326 | {next_state, leading, State#state{followers=Followers2}}; 327 | leading({ack, From, Zxid}, State=#state{proposals=Proposals, 328 | ping_acks=Acks, 329 | followers=Followers}) -> 330 | Quorum = State#state.quorum, 331 | Acks2 = ordsets:add_element(From, Acks), 332 | State2 = State#state{ping_acks=Acks2}, 333 | case dict:find(Zxid, Proposals) of 334 | {ok, Count} -> 335 | Count2 = Count + 1, 336 | case Count2 of 337 | Quorum -> 338 | ?DOUT("L: Received enough ACK(~p), sending COMMIT~n", 339 | [Zxid]), 340 | [peer_send_event(State, N, {commit, get_id(), Zxid}) 341 | || N <- Followers], 342 | Proposals2 = dict:erase(Zxid, Proposals), 343 | {next_state, leading, State2#state{proposals=Proposals2}}; 344 | _ -> 345 | ?DOUT("L: Received ACK(~p), still need more~n", [Zxid]), 346 | Proposals2 = dict:store(Zxid, Count2, Proposals), 347 | {next_state, leading, State2#state{proposals=Proposals2}} 348 | end; 349 | error -> 350 | {next_state, leading, State} 351 | end; 352 | leading(_Event, State) -> 353 | {next_state, leading, State}. 354 | 355 | leading(Event=?ZAB_PROPOSAL{sender={fsm_sync, undefined, undefined}}, 356 | From, State) -> 357 | Event2 = Event?ZAB_PROPOSAL{sender={fsm_sync, undefined, From}}, 358 | {ok, State2} = handle_broadcast(Event2, State), 359 | {next_state, leading, State2}; 360 | leading(_Event, _From, State) -> 361 | {next_state, leading, State}. 362 | 363 | follow_new_quorum(cancel, State) -> 364 | ?INFO("F: ~p :: Timed out while establishing quorum~n", [self()]), 365 | start_election(State); 366 | follow_new_quorum(timeout, State=#state{leader=Leader}) -> 367 | peer_send_event(State, Leader, msg_serverinfo(State)), 368 | {next_state, follow_new_quorum, State, 1000}; 369 | follow_new_quorum(#serverinfo{}, State) -> 370 | State2 = restart_timer(State), 371 | {next_state, follow_new_epoch, State2}; 372 | follow_new_quorum(_Event, State) -> 373 | {next_state, follow_new_quorum, State}. 374 | 375 | follow_new_epoch(cancel, State) -> 376 | ?INFO("F: ~p :: Timed out while establishing epoch~n", [self()]), 377 | start_election(State); 378 | follow_new_epoch({newepoch, From, Epoch}, 379 | State=#state{accepted_epoch=AccEpoch}) -> 380 | if 381 | Epoch > AccEpoch -> 382 | %% Accept the new epoch proposal 383 | State2 = restart_timer(State), 384 | peer_send_event(State, From, {ack, get_id(), {Epoch,0}}), 385 | State3 = update_accepted_epoch(Epoch, State2), 386 | {next_state, follow_synchronize, State3}; 387 | Epoch < AccEpoch -> 388 | %% Abandon the leader 389 | ?INFO("F: ~p :: Abandoning leader (new epoch)~n", [self()]), 390 | start_election(State); 391 | Epoch == AccEpoch -> 392 | %% Move into synchronize state without ACKing proposal 393 | State2 = restart_timer(State), 394 | {next_state, follow_synchronize, State2} 395 | end; 396 | follow_new_epoch(_Event, State) -> 397 | {next_state, follow_new_epoch, State}. 398 | 399 | follow_synchronize(cancel, State) -> 400 | ?INFO("F: ~p :: Timed out while trying to synchronize~n", [self()]), 401 | start_election(State); 402 | follow_synchronize(Msg={sync, _Idx, _SyncMsg}, State) -> 403 | State2 = log_sync_message(Msg, State), 404 | {next_state, follow_synchronize, State2}; 405 | follow_synchronize(Msg={diff, _From, _Preflist, _Zxid}, State) -> 406 | State2 = log_sync_message(Msg, State), 407 | {next_state, follow_synchronize, State2}; 408 | follow_synchronize(Msg={oldpropose, _From, _Zxid, _Msg}, State) -> 409 | State2 = log_sync_message(Msg, State), 410 | {next_state, following, State2}; 411 | follow_synchronize(Msg={newleader, From, _Epoch}, State) -> 412 | State2 = log_sync_message(Msg, State), 413 | State3 = play_sync_log(State2), 414 | Epoch2 = State3#state.current_epoch, 415 | peer_send_event(State3, From, {ack, get_id(), {Epoch2, 0}}), 416 | {next_state, follow_synchronize, State3}; 417 | follow_synchronize({uptodate, _From}, State) -> 418 | State2 = restart_timer(State), 419 | ?INFO("F: ~p :: Moving to following/broadcast~n", [self()]), 420 | State3 = commit_all_proposals(State2), 421 | {next_state, following, State3}; 422 | follow_synchronize(_Event, State) -> 423 | {next_state, follow_synchronize, State}. 424 | 425 | handle_sync_message({sync, Idx, Msg}, State=#state{app=App}) -> 426 | ok = riak_core_vnode_master:sync_command({Idx, node()}, Msg, App), 427 | State; 428 | handle_sync_message({diff, _From, Preflist, Zxid}, 429 | State=#state{history=Hist, 430 | last_commit_zxid=LastCommit}) -> 431 | Hist2 = riak_zab_log:log_last_commit(Preflist, Zxid, Hist), 432 | LastCommit2 = erlang:max(LastCommit, Zxid), 433 | State#state{last_zxid=LastCommit2, last_commit_zxid=LastCommit2, 434 | history=Hist2}; 435 | handle_sync_message({oldpropose, _From, Zxid, Msg}, 436 | State=#state{history=History}) -> 437 | History2 = riak_zab_log:log_proposal(Zxid, Msg, History), 438 | {ok, State#state{history=History2, last_zxid=Zxid}}; 439 | handle_sync_message({newleader, _From, Epoch}, 440 | State=#state{last_zxid=LastZxid}) -> 441 | LastZxid2 = erlang:max(LastZxid, {Epoch, 0}), 442 | State2 = update_current_epoch(Epoch, State), 443 | State2#state{last_zxid=LastZxid2}. 444 | 445 | following(cancel, State) -> 446 | ?INFO("F: ~p :: Timed out while following~n", [self()]), 447 | start_election(State); 448 | following(Event={ping, _From}, State) -> 449 | {ok, State2} = handle_ping(Event, State), 450 | State3 = restart_timer(State2), 451 | {next_state, following, State3}; 452 | following(Event={propose, _From, _Zxid, _Msg}, State) -> 453 | {ok, State2} = handle_propose(Event, State), 454 | State3 = restart_timer(State2), 455 | {next_state, following, State3}; 456 | following(Event={commit, _From, _Zxid}, State) -> 457 | {ok, State2} = handle_commit(Event, State), 458 | {next_state, following, State2}; 459 | following(_Event, State) -> 460 | {next_state, following, State}. 461 | 462 | valid_follower_epoch(LeaderEpoch, LeaderZxid, FollowerEpoch, FollowerZxid) -> 463 | (FollowerEpoch =< LeaderEpoch) 464 | orelse ((FollowerEpoch == LeaderEpoch) 465 | andalso (FollowerZxid < LeaderZxid)). 466 | 467 | log_sync_message(Msg, State=#state{sync_log=SyncLog}) -> 468 | SyncLog2 = riak_zab_log:log_sync_message(Msg, SyncLog), 469 | State#state{sync_log=SyncLog2}. 470 | 471 | %% @private 472 | peer_send_event(#state{ensemble=Ensemble}, Node, Msg) -> 473 | peer_send_event(Ensemble, Node, Msg); 474 | peer_send_event(_Ensemble, {_Node, Pid}, Msg) -> 475 | gen_fsm:send_event(Pid, Msg). 476 | 477 | %% @private 478 | peer_all_sync_event(_Ensemble, {_Node, Pid}, Event) -> 479 | gen_fsm:sync_send_all_state_event(Pid, Event). 480 | 481 | %% @private 482 | get_id() -> 483 | {node(), self()}. 484 | 485 | %% @private 486 | monitor_peer({_Node, no_match}, _State) -> 487 | ok; 488 | monitor_peer({Node, Pid}, State) -> 489 | PInfo = State#state.peerinfo, 490 | case ets:match_object(PInfo, {Node, Pid, '_'}) of 491 | [] -> 492 | %% Start monitoring peer 493 | M = erlang:monitor(process, Pid), 494 | ets:insert(PInfo, {Node, Pid, M}); 495 | _ -> 496 | %% Already monitoring peer 497 | ok 498 | end. 499 | 500 | %% @private 501 | remove_peer(Pid, State=#state{peerinfo=PInfo, 502 | followers0=Followers0, 503 | followers=Followers}) -> 504 | case ets:match_object(PInfo, {'_', Pid, '_'}) of 505 | [{Node, Pid, _}] -> 506 | ?DOUT("Removing peer ~p~n", [{Node, Pid}]), 507 | ets:delete(PInfo, Node), 508 | NFollowers0 = ordsets:del_element(Node, Followers0), 509 | NFollowers = ordsets:del_element(Node, Followers), 510 | Peers = ets:foldl(fun({N,P,_},L) -> [{N,P}|L] end, [], PInfo), 511 | State#state{followers0=NFollowers0, 512 | followers=NFollowers, 513 | peers=Peers}; 514 | [] -> 515 | State 516 | end. 517 | 518 | %% @private 519 | start_election(State=#state{ensemble=Ensemble, 520 | peerinfo=PInfo, 521 | quorum=Quorum, 522 | last_zxid=LastZxid, 523 | synchronize_pid=SyncPid}) -> 524 | cancel_timer(State#state.ping_timer), 525 | catch exit(SyncPid, kill), 526 | ?PROBE(node(), start_election), 527 | ?DOUT("M: Looking for ensemble peers~n ~p~n", [Ensemble]), 528 | Peers = riak_zab_ensemble_master:get_peers(Ensemble), 529 | ?DOUT("Peers: ~p~n", [Peers]), 530 | [monitor_peer(Peer, State) || Peer <- Peers], 531 | AllPeers = ets:foldl(fun({N,P,_},L) -> [{N,P}|L] end, [], PInfo), 532 | ?DOUT("AllPeers: ~p~n", [AllPeers]), 533 | SyncLog2 = riak_zab_log:clear_sync_log(State#state.sync_log), 534 | State2 = State#state{sync_log=SyncLog2}, 535 | State3 = restart_timer(State2), 536 | case length(AllPeers) >= Quorum of 537 | true -> 538 | ?INFO("~p: starting election~n", [self()]), 539 | StartMsg = {start_election, AllPeers, LastZxid}, 540 | State4 = handle_election_event(StartMsg, State3), 541 | {next_state, looking, State4#state{node_state=looking, 542 | peers=AllPeers, 543 | followers0=[], 544 | followers=[]}}; 545 | false -> 546 | {next_state, looking, State3#state{node_state=looking, 547 | peers=AllPeers}} 548 | end. 549 | 550 | %% @private 551 | play_sync_log(State=#state{history=History, sync_log=SyncLog}) -> 552 | History2 = riak_zab_log:clear_proposals(History), 553 | State2 = State#state{history=History2}, 554 | State3 = riak_zab_log:fold_sync_log(SyncLog, 555 | fun handle_sync_message/2, 556 | State2), 557 | SyncLog2 = riak_zab_log:clear_sync_log(SyncLog), 558 | State3#state{sync_log=SyncLog2}. 559 | 560 | cancel_timer(none) -> 561 | ok; 562 | cancel_timer(T) -> 563 | gen_fsm:cancel_timer(T). 564 | restart_timer(State=#state{timer=T}) -> 565 | cancel_timer(T), 566 | T2 = gen_fsm:send_event_after(?TIMEOUT, cancel), 567 | State#state{timer=T2}. 568 | restart_ping_timer(State=#state{ping_timer=T}) -> 569 | cancel_timer(T), 570 | T2 = gen_fsm:send_event_after(?PINGTIME, ping), 571 | State#state{ping_timer=T2}. 572 | 573 | msg_serverinfo(#state{current_epoch=CurrentEpoch, 574 | accepted_epoch=AcceptedEpoch, 575 | last_zxid=LastZxid}) -> 576 | #serverinfo{from=get_id(), 577 | current_epoch=CurrentEpoch, 578 | accepted_epoch=AcceptedEpoch, 579 | zxid=LastZxid}. 580 | 581 | handle_broadcast(Msg=?ZAB_PROPOSAL{}, State=#state{last_zxid={ZEpoch, Zxid}, 582 | followers=Followers, 583 | proposals=Proposals}) -> 584 | Zxid2 = {ZEpoch, Zxid+1}, 585 | ?DOUT("L: Sending PROPOSE(~p)~n", [Zxid2]), 586 | ?DOUT(" Followers: ~p~n", [Followers]), 587 | [peer_send_event(State, N, {propose, get_id(), Zxid2, Msg}) 588 | || N <- Followers], 589 | Proposals2 = dict:store(Zxid2, 0, Proposals), 590 | State2 = State#state{last_zxid=Zxid2, proposals=Proposals2}, 591 | {ok, State2}. 592 | 593 | handle_ping({ping, From}, State) when From == State#state.leader -> 594 | peer_send_event(State, From, {ackping, get_id()}), 595 | {ok, State}; 596 | handle_ping(_Event, State) -> 597 | {ok, State}. 598 | 599 | handle_propose({propose, From, Zxid, Msg}, State=#state{history=History}) 600 | when From == State#state.leader -> 601 | ?DOUT("F: Received PROPOSE(~p)~n", [Zxid]), 602 | ?DOUT("F: Sending ACK(~p)~n", [Zxid]), 603 | History2 = riak_zab_log:log_proposal(Zxid, Msg, History), 604 | peer_send_event(State, From, {ack, get_id(), Zxid}), 605 | {ok, State#state{history=History2, last_zxid=Zxid}}; 606 | handle_propose(_Event, State) -> 607 | {ok, State}. 608 | 609 | commit_all_proposals(State=#state{history=History}) -> 610 | Proposals = lists:sort(riak_zab_log:get_proposed_keys(History)), 611 | lists:foldl(fun do_commit/2, State, Proposals). 612 | 613 | do_commit(Zxid, State=#state{history=History, app=App, leader=Leader}) -> 614 | Proposal = riak_zab_log:get_proposal(Zxid, History), 615 | ?ZAB_PROPOSAL{preflist=Preflist, 616 | sender=Sender, 617 | message=Msg} = Proposal, 618 | LeaderVN = ensemble_leader_vnode(Preflist, Leader), 619 | LocalVN = [{Idx, Node} || {Idx, Node} <- Preflist, Node == node()], 620 | Req = ?ZAB_REQ{zxid=Zxid, req=Msg, sender=Sender, leading=false}, 621 | [begin 622 | Leading = (VN == LeaderVN), 623 | Req2 = Req?ZAB_REQ{leading=Leading}, 624 | ok = riak_core_vnode_master:sync_command(VN, Req2, App, 1000) 625 | end || VN <- LocalVN], 626 | 627 | History2 = riak_zab_log:log_last_commit(Preflist, Zxid, History), 628 | History3 = riak_zab_log:del_proposal(Zxid, History2), 629 | State#state{history=History3, last_commit_zxid=Zxid}. 630 | 631 | handle_commit({commit, From, Zxid}, State) when From == State#state.leader -> 632 | ?DOUT("F: Received COMMIT(~p)~n", [Zxid]), 633 | State2 = do_commit(Zxid, State), 634 | {ok, State2}; 635 | handle_commit(_Event, State) -> 636 | {ok, State}. 637 | 638 | handle_election_event(Event, State) -> 639 | gen_fsm:send_event(State#state.electfsm, Event), 640 | State. 641 | 642 | handle_all_election_event(Event, State) -> 643 | gen_fsm:send_all_state_event(State#state.electfsm, Event), 644 | State. 645 | 646 | handle_info({'DOWN', _Ref, process, Pid, _Reason}, StateName, State) -> 647 | State2 = remove_peer(Pid, State), 648 | case State#state.leader of 649 | {_, Pid} -> 650 | ?INFO("~p: Leader is down, abandoning.~n", [self()]), 651 | start_election(State2); 652 | _ -> 653 | {next_state, StateName, State2} 654 | end; 655 | handle_info(_Msg, StateName, State) -> 656 | {next_state, StateName, State}. 657 | 658 | handle_event({election, ElectionEvent}, StateName, State) -> 659 | State2 = handle_all_election_event(ElectionEvent, State), 660 | {next_state, StateName, State2}; 661 | handle_event(_Event, StateName, State) -> 662 | {next_state, StateName, State}. 663 | 664 | handle_sync_event(get_leader, _From, StateName, 665 | State=#state{leader=Leader, node_state=NState}) -> 666 | Reply = case NState of 667 | leading -> Leader; 668 | following -> Leader; 669 | _Other -> none 670 | end, 671 | {reply, Reply, StateName, State}; 672 | handle_sync_event(all_zxids, _From, StateName, State) -> 673 | Zxids = get_all_zxids(State), 674 | {reply, Zxids, StateName, State}; 675 | handle_sync_event(node_state, _From, StateName, State) -> 676 | {reply, State#state.node_state, StateName, State}; 677 | handle_sync_event(last_zxid, _From, StateName, State) -> 678 | {reply, State#state.last_zxid, StateName, State}; 679 | handle_sync_event(_Event, _From, StateName, State) -> 680 | {next_state, StateName, State}. 681 | 682 | %% @private 683 | terminate(_Reason, _StateName, _State) -> 684 | ok. 685 | 686 | %% @private 687 | code_change(_OldVsn, StateName, State, _Extra) -> 688 | {ok, StateName, State}. 689 | 690 | %% Select a vnode as the leader vnode for a preflist. 691 | %% For now, select the first vnode owned by the leader. 692 | %% @private 693 | ensemble_leader_vnode(Preflist, {LNode, _LPid}) -> 694 | %% We sort the preflist to be safe about ordering. 695 | L = lists:keysort(1, Preflist), 696 | VNodes = [VN || VN={_,Node} <- L, Node == LNode], 697 | hd(VNodes). 698 | 699 | get_old_proposals(#state{history=Hist, last_commit_zxid=LastCommit}) -> 700 | riak_zab_log:get_old_proposals(Hist, LastCommit). 701 | 702 | update_accepted_epoch(Epoch, State=#state{history=Hist}) -> 703 | Hist2 = riak_zab_log:log_accepted_epoch(Hist, Epoch), 704 | State#state{accepted_epoch=Epoch, history=Hist2}. 705 | 706 | update_current_epoch(Epoch, State=#state{history=Hist}) -> 707 | Hist2 = riak_zab_log:log_current_epoch(Hist, Epoch), 708 | State#state{current_epoch=Epoch, history=Hist2}. 709 | 710 | synchronize_peer(Follower, Epoch, State) -> 711 | LeaderAllZxids = get_all_zxids(State), 712 | Proposals = get_old_proposals(State), 713 | synchronize_peer(LeaderAllZxids, Proposals, Follower, Epoch, State). 714 | 715 | synchronize_peer(LeaderAllZxids, Proposals, Follower, Epoch, State) -> 716 | Id = State#state.id, 717 | synchronize_vnodes(Follower, LeaderAllZxids, State), 718 | ?DOUT("L: Sending old proposals to ~p~n", [Follower]), 719 | [peer_send_event(State, Follower, {oldpropose, Id, Zxid, Msg}) 720 | || {Zxid, Msg} <- Proposals], 721 | peer_send_event(State, Follower, {newleader, Id, Epoch}). 722 | 723 | %% For all preflists owned by this ensemble peer, return the last committed 724 | %% zxid for each partition index owned by this node. 725 | -spec get_all_zxids(#state{}) -> allzxids(). 726 | get_all_zxids(State=#state{preflists=Preflists}) -> 727 | [{Preflist, get_zxids(Preflist, State)} || Preflist <- Preflists]. 728 | 729 | %% Given a preflist, return the last committed zxid for each partition index 730 | %% owned by this node. 731 | -spec get_zxids(preflist(), term()) -> [{index(), zxid()}]. 732 | get_zxids(Preflist, #state{history=Hist}) -> 733 | LoggedZxid = riak_zab_log:get_last_commit(Preflist, Hist), 734 | [{Idx, LoggedZxid} || {Idx, Node} <- Preflist, Node == node()]. 735 | 736 | %% Application state is owned by application vnodes, but riak_zab is in charge 737 | %% of coordinating necessary synchronization. This function determines which 738 | %% follower preflists are out of date with respect to the leader, and has the 739 | %% appropriate leader vnode initiate synchronization with the corresponding 740 | %% follower vnodes. Staleness is determined based on the last committed zxid 741 | %% metadata that riak_zab maintains for each preflist. 742 | synchronize_vnodes(Follower={_FNode, FPid}, 743 | LeaderAllZxids, 744 | State=#state{ensemble=Ensemble, app=App}) -> 745 | FollowerAllZxids = case Follower == State#state.id of 746 | true -> 747 | LeaderAllZxids; 748 | false -> 749 | peer_all_sync_event(Ensemble, Follower, 750 | all_zxids) 751 | end, 752 | StaleIndices = get_stale_indices(LeaderAllZxids, FollowerAllZxids, []), 753 | 754 | %% Merge multiple (leader-zxid, follower-zxid) cases in the stale indices 755 | %% set (possible due to preflist overlap) into a single case. Reduces the 756 | %% number of vnode synchronizations needed. 757 | VNodesToSync = merge_stale_indices(StaleIndices), 758 | 759 | %% Determine the last committed zxid for each stale preflist. 760 | LatestPreflistZxids = get_latest_preflist_zxids(StaleIndices), 761 | 762 | %% Have stale vnodes perform synchronization 763 | [begin 764 | Cmd = ?ZAB_SYNC{peer={FIdx, FPid}, idxs=Idxs}, 765 | riak_core_vnode_master:sync_command({LIdx, node()}, Cmd, App) 766 | end || {{LIdx, FIdx}, Idxs} <- VNodesToSync], 767 | 768 | %% Send DIFF messages to indiciate new preflist state after sync 769 | [begin 770 | Msg = {diff, State#state.id, Preflist, LRecent}, 771 | ?DOUT("Sending DIFF(~p, ~p)~n", [Preflist, LRecent]), 772 | peer_send_event(State, Follower, Msg) 773 | end || {Preflist, LRecent} <- LatestPreflistZxids]. 774 | 775 | get_stale_indices([], [], Acc) -> 776 | Acc; 777 | get_stale_indices([X|Xs], [Y|Ys], Acc) -> 778 | {LPreflist, LZxids} = X, 779 | {FPreflist, FZxids} = Y, 780 | %% Test invariant 781 | LPreflist = FPreflist, 782 | %% Compare preflists 783 | {LastIdx, LastZxid} = most_recent_index(LZxids), 784 | StaleIndices = [{LastIdx, FIdx, LastZxid, LPreflist} 785 | || {FIdx, FZxid} <- FZxids, FZxid < LastZxid], 786 | get_stale_indices(Xs, Ys, StaleIndices ++ Acc). 787 | 788 | merge_stale_indices(StaleIndices) -> 789 | D = lists:foldl(fun({LastIdx, FIdx, _LastZxid, Preflist}, Acc) -> 790 | Key = {LastIdx, FIdx}, 791 | Idxs = [Idx || {Idx,_N} <- Preflist], 792 | dict:update(Key, fun(L) -> 793 | lists:usort(Idxs ++ L) 794 | end, Idxs, Acc) 795 | end, dict:new(), StaleIndices), 796 | dict:to_list(D). 797 | 798 | get_latest_preflist_zxids(StaleIndices) -> 799 | L = [{Preflist, LastZxid} 800 | || {_LastIdx, _FIdx, LastZxid, Preflist} <- StaleIndices], 801 | lists:ukeysort(1, L). 802 | 803 | most_recent_index(Zxids) -> 804 | lists:foldl(fun({Idx, Zxid}, Acc={_, RecentZxid}) -> 805 | case Zxid > RecentZxid of 806 | true -> {Idx, Zxid}; 807 | false -> Acc 808 | end 809 | end, 810 | hd(Zxids), 811 | Zxids). 812 | -------------------------------------------------------------------------------- /src/riak_zab_process.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_vnode_master: dispatch to vnodes 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc dispatch to vnodes 24 | 25 | -module(riak_zab_process). 26 | -behaviour(gen_server). 27 | -export([start_link/0, start_process/3, stop_process/1, start_process_link/3, get_pid/1, all_pids/1, all_processes/1]). 28 | -export([init/1, handle_call/3, handle_cast/2, handle_info/2, 29 | terminate/2, code_change/3]). 30 | -record(idxrec, {idx, pid, monref}). 31 | -record(state, {idxtab}). 32 | 33 | start_link() -> 34 | gen_server:start_link({local, ?MODULE}, ?MODULE, [], []). 35 | 36 | start_process_link(Key, Mod, Args) -> 37 | {ok, Pid} = Mod:start_link(Args), 38 | gen_server:cast(?MODULE, {add_pid, Key, Pid}), 39 | {ok, Pid}. 40 | 41 | start_process(Key, Mod, Args) -> 42 | {ok, Pid} = gen_server:call(?MODULE, {start_process, Key, Mod, Args}, infinity), 43 | Pid. 44 | 45 | stop_process(Key) -> 46 | gen_server:call(?MODULE, {stop_process, Key}, infinity). 47 | 48 | get_pid(Key) -> 49 | {ok, Pid} = gen_server:call(?MODULE, {get_pid, Key}, infinity), 50 | Pid. 51 | 52 | all_pids(Type) -> 53 | gen_server:call(?MODULE, {all_pids, Type}, infinity). 54 | 55 | all_processes(Type) -> 56 | gen_server:call(?MODULE, {all_processes, Type}, infinity). 57 | 58 | %% @private 59 | init(_Args) -> 60 | %% Get the current list of pids running in the supervisor. We use this 61 | %% to rebuild our ETS table. 62 | Pids = [{Pid, Key} || {Key, Pid, worker, _} 63 | <- supervisor:which_children(riak_zab_process_sup)], 64 | IdxTable = ets:new(idxtable, [{keypos, 2}]), 65 | 66 | %% Populate the ETS table with processes running this VNodeMod (filtered 67 | %% in the list comprehension) 68 | F = fun(Pid, Idx) -> 69 | Mref = erlang:monitor(process, Pid), 70 | #idxrec { idx = Idx, pid = Pid, monref = Mref } 71 | end, 72 | IdxRecs = [F(Pid, Idx) || {Pid, Idx} <- Pids], 73 | true = ets:insert_new(IdxTable, IdxRecs), 74 | {ok, #state{idxtab=IdxTable}}. 75 | 76 | handle_cast({add_pid, Key, Pid}, State) -> 77 | MonRef = erlang:monitor(process, Pid), 78 | add_vnode_rec(#idxrec{idx=Key,pid=Pid,monref=MonRef}, State), 79 | {noreply, State}. 80 | 81 | handle_call({all_pids, Type}, _From, State) -> 82 | {reply, lists:flatten(ets:match(State#state.idxtab, 83 | {idxrec, {Type, '_'}, '$1', '_'})), State}; 84 | handle_call({all_processes, Type}, _From, State) -> 85 | M = ets:match(State#state.idxtab, {idxrec, {Type, '$1'}, '_', '_'}), 86 | {reply, [{Type, hd(L)} || L <- M], State}; 87 | handle_call({get_pid, Key}, _From, State) -> 88 | Pid = idx2vnode(Key, State), 89 | {reply, {ok, Pid}, State}; 90 | handle_call({start_process, Key, Mod, Args}, _From, State) -> 91 | Ref = {Key, {?MODULE, start_process_link, [Key, Mod, Args]}, 92 | permanent, 5000, worker, [Mod]}, 93 | Pid = case supervisor:start_child(riak_zab_process_sup, Ref) of 94 | {ok, Child} -> Child; 95 | {error, {already_started, Child}} -> Child 96 | end, 97 | {reply, {ok, Pid}, State}; 98 | handle_call({stop_process, Key}, _From, State) -> 99 | supervisor:terminate_child(riak_zab_process_sup, Key), 100 | supervisor:delete_child(riak_zab_process_sup, Key), 101 | {reply, ok, State}. 102 | 103 | handle_info({'DOWN', MonRef, process, _P, _I}, State) -> 104 | delmon(MonRef, State), 105 | {noreply, State}. 106 | 107 | %% @private 108 | terminate(_Reason, _State) -> 109 | ok. 110 | 111 | %% @private 112 | code_change(_OldVsn, State, _Extra) -> {ok, State}. 113 | 114 | %% @private 115 | idx2vnode(Idx, _State=#state{idxtab=T}) -> 116 | case ets:match(T, {idxrec, Idx, '$1', '_'}) of 117 | [[VNodePid]] -> VNodePid; 118 | [] -> no_match 119 | end. 120 | 121 | %% @private 122 | delmon(MonRef, _State=#state{idxtab=T}) -> 123 | ets:match_delete(T, {idxrec, '_', '_', MonRef}). 124 | 125 | %% @private 126 | add_vnode_rec(I, _State=#state{idxtab=T}) -> ets:insert(T,I). 127 | -------------------------------------------------------------------------------- /src/riak_zab_process_sup.erl: -------------------------------------------------------------------------------- 1 | -module(riak_zab_process_sup). 2 | -behaviour(supervisor). 3 | -export([start_link/0, init/1]). 4 | -export([start_vnode/2]). 5 | 6 | start_vnode(Mod, Index) when is_integer(Index) -> 7 | supervisor:start_child(?MODULE, [Mod, Index]). 8 | 9 | start_link() -> 10 | supervisor:start_link({local, ?MODULE}, ?MODULE, []). 11 | 12 | init([]) -> 13 | %% Restart ensembles if riak_zab_process_sup is restarting after error. 14 | %% case whereis(riak_zab_ensemble_master) of 15 | %% undefined -> 16 | %% ok; 17 | %% _Pid -> 18 | %% timer:apply_after(500, gen_server, cast, 19 | %% [riak_zab_ensemble_master, restart_ensembles]) 20 | %% end, 21 | {ok, {{one_for_one, 5, 10}, []}}. 22 | -------------------------------------------------------------------------------- /src/riak_zab_ring_handler.erl: -------------------------------------------------------------------------------- 1 | -module(riak_zab_ring_handler). 2 | -behaviour(gen_event). 3 | 4 | %% gen_event callbacks 5 | -export([init/1, handle_event/2, handle_call/2, 6 | handle_info/2, terminate/2, code_change/3]). 7 | -record(state, {}). 8 | 9 | 10 | %% =================================================================== 11 | %% gen_event callbacks 12 | %% =================================================================== 13 | 14 | init([]) -> 15 | %% Pull the initial ring and make sure all vnodes are started 16 | {ok, _Ring} = riak_core_ring_manager:get_my_ring(), 17 | {ok, #state{}}. 18 | 19 | handle_event({ring_update, _Ring}, State) -> 20 | riak_zab_ensemble_master:ring_changed(), 21 | {ok, State}. 22 | 23 | handle_call(_Event, State) -> 24 | {ok, ok, State}. 25 | 26 | handle_info(_Info, State) -> 27 | {ok, State}. 28 | 29 | terminate(_Reason, _State) -> 30 | ok. 31 | 32 | code_change(_OldVsn, State, _Extra) -> 33 | {ok, State}. 34 | -------------------------------------------------------------------------------- /src/riak_zab_status.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_zab_status: Copy of riak_kv_status to avoid requiring 4 | %% entire riak_kv app as a dependency. Should 5 | %% probably be part of riak_core anyway. 6 | %% 7 | %% ------------------------------------------------------------------- 8 | %% 9 | %% Riak: A lightweight, decentralized key-value store. 10 | %% 11 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 12 | %% 13 | %% This file is provided to you under the Apache License, 14 | %% Version 2.0 (the "License"); you may not use this file 15 | %% except in compliance with the License. You may obtain 16 | %% a copy of the License at 17 | %% 18 | %% http://www.apache.org/licenses/LICENSE-2.0 19 | %% 20 | %% Unless required by applicable law or agreed to in writing, 21 | %% software distributed under the License is distributed on an 22 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 23 | %% KIND, either express or implied. See the License for the 24 | %% specific language governing permissions and limitations 25 | %% under the License. 26 | %% 27 | %% ------------------------------------------------------------------- 28 | -module(riak_zab_status). 29 | 30 | -export([ringready/0, 31 | transfers/0]). 32 | 33 | %% =================================================================== 34 | %% Public API 35 | %% =================================================================== 36 | 37 | %% -spec(statistics() -> [any()]). 38 | %% statistics() -> 39 | %% case whereis(riak_kv_stat) of 40 | %% undefined -> 41 | %% []; 42 | %% _ -> 43 | %% riak_kv_stat:get_stats() 44 | %% end. 45 | 46 | -spec(ringready() -> {ok, [atom()]} | {error, any()}). 47 | ringready() -> 48 | case get_rings() of 49 | {[], Rings} -> 50 | {N1,R1}=hd(Rings), 51 | case rings_match(hash_ring(R1), tl(Rings)) of 52 | true -> 53 | Nodes = [N || {N,_} <- Rings], 54 | {ok, Nodes}; 55 | 56 | {false, N2} -> 57 | {error, {different_owners, N1, N2}} 58 | end; 59 | 60 | {Down, _Rings} -> 61 | {error, {nodes_down, Down}} 62 | end. 63 | 64 | 65 | -spec(transfers() -> {[atom()], [{waiting_to_handoff, atom(), integer()} | 66 | {stopped, atom(), integer()}]}). 67 | transfers() -> 68 | {Down, Rings} = get_rings(), 69 | 70 | %% Work out which vnodes are running and which partitions they claim 71 | F = fun({N,R}, Acc) -> 72 | {_Pri, Sec, Stopped} = partitions(N, R), 73 | Acc1 = case Sec of 74 | [] -> 75 | []; 76 | _ -> 77 | [{waiting_to_handoff, N, length(Sec)}] 78 | end, 79 | case Stopped of 80 | [] -> 81 | Acc1 ++ Acc; 82 | _ -> 83 | Acc1 ++ [{stopped, N, length(Stopped)} | Acc] 84 | end 85 | end, 86 | {Down, lists:foldl(F, [], Rings)}. 87 | 88 | 89 | %% =================================================================== 90 | %% Internal functions 91 | %% =================================================================== 92 | 93 | %% Retrieve the rings for all other nodes by RPC 94 | get_rings() -> 95 | {RawRings, Down} = riak_core_util:rpc_every_member( 96 | riak_core_ring_manager, get_my_ring, [], 30000), 97 | Rings = orddict:from_list([{riak_core_ring:owner_node(R), R} || {ok, R} <- RawRings]), 98 | {lists:sort(Down), Rings}. 99 | 100 | %% Produce a hash of the 'chash' portion of the ring 101 | hash_ring(R) -> 102 | erlang:phash2(riak_core_ring:all_owners(R)). 103 | 104 | %% Check if all rings match given a hash and a list of [{N,P}] to check 105 | rings_match(_, []) -> 106 | true; 107 | rings_match(R1hash, [{N2, R2} | Rest]) -> 108 | case hash_ring(R2) of 109 | R1hash -> 110 | rings_match(R1hash, Rest); 111 | _ -> 112 | {false, N2} 113 | end. 114 | 115 | 116 | %% Get a list of active partition numbers - regardless of vnode type 117 | active_partitions(Node) -> 118 | lists:foldl(fun({_,P}, Ps) -> 119 | ordsets:add_element(P, Ps) 120 | end, [], running_vnodes(Node)). 121 | 122 | %% Get a list of running vnodes for a node 123 | running_vnodes(Node) -> 124 | Pids = vnode_pids(Node), 125 | [rpc:call(Node, riak_core_vnode, get_mod_index, [Pid], 30000) || Pid <- Pids]. 126 | 127 | %% Get a list of vnode pids for a node 128 | vnode_pids(Node) -> 129 | [Pid || {_,Pid,_,_} <- supervisor:which_children({riak_core_vnode_sup, Node})]. 130 | 131 | %% Return a list of active primary partitions, active secondary partitions (to be handed off) 132 | %% and stopped partitions that should be started 133 | partitions(Node, Ring) -> 134 | Owners = riak_core_ring:all_owners(Ring), 135 | Owned = ordsets:from_list(owned_partitions(Owners, Node)), 136 | Active = ordsets:from_list(active_partitions(Node)), 137 | Stopped = ordsets:subtract(Owned, Active), 138 | Secondary = ordsets:subtract(Active, Owned), 139 | Primary = ordsets:subtract(Active, Secondary), 140 | {Primary, Secondary, Stopped}. 141 | 142 | %% Return the list of partitions owned by a node 143 | owned_partitions(Owners, Node) -> 144 | [P || {P, Owner} <- Owners, Owner =:= Node]. 145 | -------------------------------------------------------------------------------- /src/riak_zab_sup.erl: -------------------------------------------------------------------------------- 1 | 2 | -module(riak_zab_sup). 3 | 4 | -behaviour(supervisor). 5 | 6 | %% API 7 | -export([start_link/0]). 8 | 9 | %% Supervisor callbacks 10 | -export([init/1]). 11 | 12 | %% Helper macro for declaring children of supervisor 13 | -define(CHILD(I, Type), {I, {I, start_link, []}, permanent, 5000, Type, [I]}). 14 | 15 | %% =================================================================== 16 | %% API functions 17 | %% =================================================================== 18 | 19 | start_link() -> 20 | supervisor:start_link({local, ?MODULE}, ?MODULE, []). 21 | 22 | %% =================================================================== 23 | %% Supervisor callbacks 24 | %% =================================================================== 25 | 26 | init([]) -> 27 | ProcessSup = ?CHILD(riak_zab_process_sup, supervisor), 28 | Process = ?CHILD(riak_zab_process, worker), 29 | EnsembleMaster = ?CHILD(riak_zab_ensemble_master, worker), 30 | 31 | Processes = lists:flatten([ProcessSup, Process, EnsembleMaster]), 32 | 33 | {ok, { {one_for_one, 5, 10}, Processes} }. 34 | -------------------------------------------------------------------------------- /src/riak_zab_util.erl: -------------------------------------------------------------------------------- 1 | -module(riak_zab_util). 2 | -export([command/3, command/4, sync_command/3]). 3 | 4 | %% API 5 | command(Preflist, Msg, VMaster) -> 6 | command(Preflist, Msg, noreply, VMaster). 7 | command(Key, Msg, Sender, VMaster) -> 8 | Preflist = get_preflist(Key), 9 | riak_zab_ensemble_master:command(Preflist, Msg, Sender, VMaster). 10 | 11 | sync_command(Key, Msg, VMaster) -> 12 | Preflist = get_preflist(Key), 13 | riak_zab_ensemble_master:sync_command(Preflist, Msg, VMaster). 14 | 15 | %% @private 16 | get_preflist(Key) -> 17 | Idx = chash:key_of(Key), 18 | ESize = riak_zab_ensemble_master:get_ensemble_size(), 19 | {ok, Ring} = riak_core_ring_manager:get_my_ring(), 20 | Preflist = preflist(Idx, ESize, Ring), 21 | Preflist. 22 | 23 | %% @private 24 | preflist(Key, N, Ring) -> 25 | Preflist = riak_core_ring:preflist(Key, Ring), 26 | {Preflist2, _} = lists:split(N, Preflist), 27 | Preflist2. 28 | -------------------------------------------------------------------------------- /src/riak_zab_vnode.erl: -------------------------------------------------------------------------------- 1 | -module(riak_zab_vnode). 2 | -export([command/3, sync_command/3, standard_sync/4, reply/2]). 3 | 4 | -include_lib("riak_core/include/riak_core_vnode.hrl"). 5 | -include_lib("riak_zab_vnode.hrl"). 6 | 7 | %-define(DOUT(Msg, Args), io:format(Msg, Args)). 8 | -define(DOUT(Msg, Args), true). 9 | 10 | command(VNode, Req, VMaster) -> 11 | riak_core_vnode_master:command(VNode, Req, VMaster). 12 | 13 | sync_command(VNode, Req, VMaster) -> 14 | riak_core_vnode_master:sync_command(VNode, Req, VMaster). 15 | 16 | standard_sync(Mod, State, Peer, Idxs) -> 17 | {ok, Ring} = riak_core_ring_manager:get_my_ring(), 18 | Mod:handle_command(?FOLD_REQ{foldfun=fun sync_item/3, 19 | acc0={Mod, Peer, Idxs, Ring}}, 20 | none, State). 21 | 22 | sync_item(K, V, Acc={Mod, Peer, Idxs, Ring}) -> 23 | BKey = Mod:hash_key(K), 24 | {Idx, _INode} = hd(riak_core_ring:preflist(BKey, Ring)), 25 | 26 | case ordsets:is_element(Idx, Idxs) of 27 | true -> 28 | ?DOUT("Standard sync :: sending ~p~n", [{K,V}]), 29 | riak_zab_peer:send_sync_message(Peer, ?ZAB_SYNC_DATA{data={K,V}}), 30 | Acc; 31 | false -> 32 | ?DOUT("Standard sync :: skipping ~p~n", [{K,V}]), 33 | ?DOUT(" K: ~p~n", [K]), 34 | ?DOUT(" BKey: ~p~n", [BKey]), 35 | ?DOUT(" Idx: ~p~n", [Idx]), 36 | ?DOUT(" Idxs: ~p~n", [Idxs]), 37 | Acc 38 | end. 39 | 40 | reply({fsm_sync, undefined, From}, Reply) -> 41 | gen_fsm:reply(From, Reply); 42 | reply(Sender, Reply) -> 43 | riak_core_vnode:reply(Sender, Reply). 44 | 45 | -------------------------------------------------------------------------------- /src/riak_zab_watcher.erl: -------------------------------------------------------------------------------- 1 | -module(riak_zab_watcher). 2 | -behaviour(gen_event). 3 | 4 | %% gen_event callbacks 5 | -export([init/1, handle_event/2, handle_call/2, 6 | handle_info/2, terminate/2, code_change/3]). 7 | 8 | -record(state, {}). 9 | 10 | init(_Args) -> 11 | {ok, #state{}}. 12 | 13 | handle_event({service_update, _Services}, State) -> 14 | {ok, State}. 15 | 16 | handle_call(_Request, State) -> 17 | {ok, ok, State}. 18 | 19 | handle_info(_Info, State) -> 20 | {ok, State}. 21 | 22 | terminate(_Reason, _State) -> 23 | ok. 24 | 25 | code_change(_OldVsn, State, _Extra) -> 26 | {ok, State}. 27 | --------------------------------------------------------------------------------