├── include ├── plumtree.hrl └── plumtree_metadata.hrl ├── rebar3 ├── relx.config ├── tools.mk ├── .travis.yml ├── .gitignore ├── Makefile ├── src ├── plumtree.app.src ├── plumtree.erl ├── plumtree_app.erl ├── plumtree_peer_service_console.erl ├── plumtree_sup.erl ├── plumtree_util.erl ├── plumtree_broadcast_handler.erl ├── plumtree_peer_service_events.erl ├── plumtree_peer_service_gossip.erl ├── plumtree_peer_service.erl ├── plumtree_metadata_object.erl ├── plumtree_peer_service_manager.erl ├── plumtree_metadata_exchange_fsm.erl ├── plumtree_metadata_hashtree.erl ├── plumtree_metadata.erl ├── plumtree_broadcast.erl └── plumtree_metadata_manager.erl ├── rebar.lock ├── rebar.config ├── README.md ├── test ├── plumtree_test_utils.erl ├── cluster_membership_SUITE.erl └── metadata_SUITE.erl └── LICENSE /include/plumtree.hrl: -------------------------------------------------------------------------------- 1 | -define(SET, riak_dt_orswot). 2 | -------------------------------------------------------------------------------- /rebar3: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/helium/plumtree/HEAD/rebar3 -------------------------------------------------------------------------------- /relx.config: -------------------------------------------------------------------------------- 1 | {release, {plumtree, "0.0.1"}, [plumtree]}. 2 | {extended_start_script, true}. 3 | {overlay, [{mkdir, "data"}]}. 4 | -------------------------------------------------------------------------------- /tools.mk: -------------------------------------------------------------------------------- 1 | REBAR ?= ./rebar3 2 | 3 | docs: 4 | ${REBAR} doc skip_deps=true 5 | 6 | xref: compile 7 | ${REBAR} xref skip_deps=true 8 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: erlang 2 | otp_release: 3 | - 18.1 4 | - 17.4 5 | script: 6 | - make dialyzer 7 | - make xref 8 | - make 9 | - make test 10 | notifications: 11 | email: christopher.meiklejohn@gmail.com 12 | email: tom@helium.com 13 | sudo: false 14 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | .rebar3 2 | _* 3 | .eunit 4 | *.o 5 | *.beam 6 | *.plt 7 | *.swp 8 | *.swo 9 | .erlang.cookie 10 | ebin 11 | log 12 | erl_crash.dump 13 | .rebar 14 | _rel 15 | _deps 16 | _plugins 17 | _build 18 | _tdeps 19 | logs 20 | erln8.config 21 | .local_dialyzer_plt 22 | deps/* 23 | -------------------------------------------------------------------------------- /Makefile: -------------------------------------------------------------------------------- 1 | REBAR = $(shell pwd)/rebar3 2 | .PHONY: deps compile rel test 3 | 4 | DIALYZER_APPS = kernel stdlib erts sasl eunit syntax_tools compiler crypto 5 | DEP_DIR="_build/lib" 6 | 7 | all: compile 8 | 9 | include tools.mk 10 | 11 | test: common_test 12 | 13 | common_test: 14 | $(REBAR) ct 15 | 16 | compile: 17 | $(REBAR) compile 18 | 19 | rel: 20 | $(REBAR) release 21 | 22 | stage: 23 | $(REBAR) release -d 24 | 25 | dialyzer: 26 | $(REBAR) dialyzer 27 | -------------------------------------------------------------------------------- /src/plumtree.app.src: -------------------------------------------------------------------------------- 1 | {application, plumtree, 2 | [ 3 | {description, "Epidemic Broadcast Trees"}, 4 | {vsn, "0.0.1"}, 5 | {registered, []}, 6 | {applications, [ 7 | kernel, 8 | stdlib, 9 | crypto, 10 | lager, 11 | eleveldb, 12 | riak_dt, 13 | time_compat, 14 | lasp_support 15 | ]}, 16 | {mod, { plumtree_app, []}}, 17 | {modules, []}, 18 | {env, [ 19 | {plumtree_data_dir, "data"}]} 20 | ]}. 21 | -------------------------------------------------------------------------------- /rebar.lock: -------------------------------------------------------------------------------- 1 | [{<<"eleveldb">>, 2 | {git,"git://github.com/helium/eleveldb.git", 3 | {ref,"822a1043284ac697bc5f2870e6e5c5900e8a0908"}}, 4 | 0}, 5 | {<<"goldrush">>, 6 | {git,"git://github.com/DeadZen/goldrush.git", 7 | {ref,"71e63212f12c25827e0c1b4198d37d5d018a7fec"}}, 8 | 1}, 9 | {<<"lager">>, 10 | {git,"git://github.com/basho/lager.git", 11 | {ref,"d33ccf3b69de09a628fe38b4d7981bb8671b8a4f"}}, 12 | 0}, 13 | {<<"lasp_support">>, 14 | {git,"git://github.com/lasp-lang/lasp_support.git", 15 | {ref,"f682f79801d3573db86bc55d849529a2a98edcfc"}}, 16 | 0}, 17 | {<<"riak_dt">>, 18 | {git,"git://github.com/basho/riak_dt.git", 19 | {ref,"f7981d4ad7407ddc085f133f204dd71bf9d50c56"}}, 20 | 0}, 21 | {<<"time_compat">>, 22 | {git,"git://github.com/lasp-lang/time_compat.git", 23 | {ref,"adfae4409187cc1a9f79028986e92e8730b5eda5"}}, 24 | 0}]. 25 | -------------------------------------------------------------------------------- /include/plumtree_metadata.hrl: -------------------------------------------------------------------------------- 1 | -type metadata_prefix() :: {binary() | atom(), binary() | atom()}. 2 | -type metadata_key() :: any(). 3 | -type metadata_pkey() :: {metadata_prefix(), metadata_key()}. 4 | -type metadata_value() :: any(). 5 | -type metadata_tombstone() :: '$deleted'. 6 | -type metadata_resolver() :: fun((metadata_value() | metadata_tombstone(), 7 | metadata_value() | metadata_tombstone()) -> metadata_value()). 8 | -type metadata_modifier() :: fun(([metadata_value() | metadata_tombstone()] | undefined) -> 9 | metadata_value()). 10 | -type metadata_object() :: {metadata, dvvset:clock()}. 11 | -type metadata_context() :: dvvset:vector(). 12 | 13 | -record(metadata_broadcast, { 14 | pkey :: metadata_pkey(), 15 | obj :: metadata_object() 16 | }). 17 | -type metadata_broadcast() :: #metadata_broadcast{}. 18 | -------------------------------------------------------------------------------- /rebar.config: -------------------------------------------------------------------------------- 1 | {deps, [ 2 | {lasp_support, ".*", {git, "git://github.com/lasp-lang/lasp_support.git", {branch, "master"}}}, 3 | {time_compat, ".*", {git, "git://github.com/lasp-lang/time_compat.git", {branch, "master"}}}, 4 | {lager, ".*", {git, "git://github.com/basho/lager.git", {tag, "2.1.1"}}}, 5 | {riak_dt, ".*", {git, "git://github.com/basho/riak_dt.git", {tag, "2.1.0"}}}, 6 | {eleveldb, ".*", {git, "git://github.com/helium/eleveldb.git", {branch, "adt-helium"}}} 7 | ]}. 8 | 9 | {dialyzer_base_plt_apps, [kernel, stdlib, erts, sasl, eunit, syntax_tools, compiler, crypto]}. 10 | {xref_checks, [undefined_function_calls]}. 11 | {erl_opts, [debug_info, 12 | warnings_as_errors, 13 | {platform_define, "^[0-9]+", namespaced_types}, 14 | {parse_transform, lager_transform}]}. 15 | {cover_enabled, true}. 16 | {eunit_opts, [verbose, {report,{eunit_surefire,[{dir,"."}]}}]}. 17 | {edoc_opts, [{preprocess, true}]}. 18 | -------------------------------------------------------------------------------- /src/plumtree.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% Copyright (c) 2015 Helium Systems, Inc. All Rights Reserved. 4 | %% 5 | %% This file is provided to you under the Apache License, 6 | %% Version 2.0 (the "License"); you may not use this file 7 | %% except in compliance with the License. You may obtain 8 | %% a copy of the License at 9 | %% 10 | %% http://www.apache.org/licenses/LICENSE-2.0 11 | %% 12 | %% Unless required by applicable law or agreed to in writing, 13 | %% software distributed under the License is distributed on an 14 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | %% KIND, either express or implied. See the License for the 16 | %% specific language governing permissions and limitations 17 | %% under the License. 18 | %% 19 | %% ------------------------------------------------------------------- 20 | 21 | -module(plumtree). 22 | 23 | -export([start/0, stop/0]). 24 | 25 | start() -> 26 | application:ensure_all_started(plumtree). 27 | 28 | stop() -> 29 | application:stop(plumtree). 30 | -------------------------------------------------------------------------------- /src/plumtree_app.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% Copyright (c) 2015 Helium Systems, Inc. All Rights Reserved. 4 | %% 5 | %% This file is provided to you under the Apache License, 6 | %% Version 2.0 (the "License"); you may not use this file 7 | %% except in compliance with the License. You may obtain 8 | %% a copy of the License at 9 | %% 10 | %% http://www.apache.org/licenses/LICENSE-2.0 11 | %% 12 | %% Unless required by applicable law or agreed to in writing, 13 | %% software distributed under the License is distributed on an 14 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | %% KIND, either express or implied. See the License for the 16 | %% specific language governing permissions and limitations 17 | %% under the License. 18 | %% 19 | %% ------------------------------------------------------------------- 20 | 21 | -module(plumtree_app). 22 | 23 | -behaviour(application). 24 | 25 | -export([start/2, stop/1]). 26 | 27 | start(_StartType, _StartArgs) -> 28 | case plumtree_sup:start_link() of 29 | {ok, Pid} -> 30 | %% do nothing for now 31 | %% TODO plumtree_broadcast hooks 32 | {ok, Pid}; 33 | Other -> 34 | {error, Other} 35 | end. 36 | 37 | stop(_State) -> 38 | ok. 39 | -------------------------------------------------------------------------------- /src/plumtree_peer_service_console.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% Copyright (c) 2015 Helium Systems, Inc. All Rights Reserved. 4 | %% 5 | %% This file is provided to you under the Apache License, 6 | %% Version 2.0 (the "License"); you may not use this file 7 | %% except in compliance with the License. You may obtain 8 | %% a copy of the License at 9 | %% 10 | %% http://www.apache.org/licenses/LICENSE-2.0 11 | %% 12 | %% Unless required by applicable law or agreed to in writing, 13 | %% software distributed under the License is distributed on an 14 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | %% KIND, either express or implied. See the License for the 16 | %% specific language governing permissions and limitations 17 | %% under the License. 18 | %% 19 | %% ------------------------------------------------------------------- 20 | 21 | -module(plumtree_peer_service_console). 22 | 23 | -export([members/1]). 24 | 25 | -include("plumtree.hrl"). 26 | 27 | members([]) -> 28 | {ok, Members} = plumtree_peer_service_manager:members(), 29 | print_members(Members). 30 | 31 | print_members(Members) -> 32 | _ = io:format("~29..=s Cluster Membership ~30..=s~n", ["",""]), 33 | _ = io:format("Connected Nodes:~n~n", []), 34 | _ = [io:format("~p~n", [Node]) || Node <- Members], 35 | _ = io:format("~79..=s~n", [""]), 36 | ok. 37 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | Plumtree 2 | ======================================================= 3 | 4 | [![Build Status](https://travis-ci.org/helium/plumtree.svg?branch=master)](https://travis-ci.org/helium/plumtree) 5 | 6 | Plumtree is an implementation of [Plumtree](http://asc.di.fct.unl.pt/~jleitao/pdf/srds07-leitao.pdf), the epidemic broadcast protocol. It is extracted from the implementation in [Riak Core](https://github.com/basho/riak_core). Instead of the riak_core ring and riak's ring gossip protocol, it includes a standalone membership gossip, built around the [Riak DT](https://github.com/basho/riak_dt) [ORSWOT](http://haslab.uminho.pt/cbm/files/1210.3368v1.pdf). 7 | 8 | More information on the plumtree protocol and it's history we encourage you to watch Jordan West's [RICON West 2013 talk](https://www.youtube.com/watch?v=s4cCUTPU8GI) and Joao Leitao & Jordan West's [RICON 2014 talk](https://www.youtube.com/watch?v=bo367a6ZAwM). 9 | 10 | A special thanks to Jordan, Joao and the team at Basho for providing much of the code contained in this library. 11 | 12 | Build 13 | ----- 14 | 15 | $ make 16 | 17 | Testing 18 | ------- 19 | 20 | $ make test 21 | $ make xref 22 | $ make dialyzer 23 | 24 | Contributing 25 | ---- 26 | 27 | Contributions from the community are encouraged. This project follows the git-flow workflow. If you want to contribute: 28 | 29 | * Fork this repository 30 | * Make your changes and run the full test suite 31 | * Please include any additional tests for any additional code added 32 | * Commit your changes and push them to your fork 33 | * Open a pull request 34 | 35 | We will review your changes, make appropriate suggestions and/or provide feedback, and merge your changes when ready. 36 | -------------------------------------------------------------------------------- /src/plumtree_sup.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% Copyright (c) 2015 Helium Systems, Inc. All Rights Reserved. 4 | %% 5 | %% This file is provided to you under the Apache License, 6 | %% Version 2.0 (the "License"); you may not use this file 7 | %% except in compliance with the License. You may obtain 8 | %% a copy of the License at 9 | %% 10 | %% http://www.apache.org/licenses/LICENSE-2.0 11 | %% 12 | %% Unless required by applicable law or agreed to in writing, 13 | %% software distributed under the License is distributed on an 14 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | %% KIND, either express or implied. See the License for the 16 | %% specific language governing permissions and limitations 17 | %% under the License. 18 | %% 19 | %% ------------------------------------------------------------------- 20 | 21 | -module(plumtree_sup). 22 | 23 | -behaviour(supervisor). 24 | 25 | -export([start_link/0]). 26 | 27 | -export([init/1]). 28 | 29 | -define(CHILD(I, Type, Timeout), {I, {I, start_link, []}, permanent, Timeout, Type, [I]}). 30 | -define(CHILD(I, Type), ?CHILD(I, Type, 5000)). 31 | 32 | start_link() -> 33 | supervisor:start_link({local, ?MODULE}, ?MODULE, []). 34 | 35 | init([]) -> 36 | Children = lists:flatten( 37 | [ 38 | ?CHILD(plumtree_peer_service_manager, worker), 39 | ?CHILD(plumtree_peer_service_gossip, worker), 40 | ?CHILD(plumtree_peer_service_events, worker), 41 | ?CHILD(plumtree_broadcast, worker), 42 | ?CHILD(plumtree_metadata_manager, worker), 43 | ?CHILD(plumtree_metadata_hashtree, worker) 44 | ]), 45 | RestartStrategy = {one_for_one, 10, 10}, 46 | {ok, {RestartStrategy, Children}}. 47 | 48 | -------------------------------------------------------------------------------- /src/plumtree_util.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% Copyright (c) 2014 Basho Technologies, Inc. All Rights Reserved. 4 | %% 5 | %% This file is provided to you under the Apache License, 6 | %% Version 2.0 (the "License"); you may not use this file 7 | %% except in compliance with the License. You may obtain 8 | %% a copy of the License at 9 | %% 10 | %% http://www.apache.org/licenses/LICENSE-2.0 11 | %% 12 | %% Unless required by applicable law or agreed to in writing, 13 | %% software distributed under the License is distributed on an 14 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | %% KIND, either express or implied. See the License for the 16 | %% specific language governing permissions and limitations 17 | %% under the License. 18 | %% 19 | %% ------------------------------------------------------------------- 20 | 21 | -module(plumtree_util). 22 | 23 | -export([build_tree/3]). 24 | 25 | %% @doc Convert a list of elements into an N-ary tree. This conversion 26 | %% works by treating the list as an array-based tree where, for 27 | %% example in a binary 2-ary tree, a node at index i has children 28 | %% 2i and 2i+1. The conversion also supports a "cycles" mode where 29 | %% the array is logically wrapped around to ensure leaf nodes also 30 | %% have children by giving them backedges to other elements. 31 | 32 | -spec build_tree(N :: integer(), Nodes :: [term()], Opts :: [term()]) 33 | -> orddict:orddict(). 34 | build_tree(N, Nodes, Opts) -> 35 | case lists:member(cycles, Opts) of 36 | true -> 37 | Expand = lists:flatten(lists:duplicate(N+1, Nodes)); 38 | false -> 39 | Expand = Nodes 40 | end, 41 | {Tree, _} = 42 | lists:foldl(fun(Elm, {Result, Worklist}) -> 43 | Len = erlang:min(N, length(Worklist)), 44 | {Children, Rest} = lists:split(Len, Worklist), 45 | NewResult = [{Elm, Children} | Result], 46 | {NewResult, Rest} 47 | end, {[], tl(Expand)}, Nodes), 48 | orddict:from_list(Tree). 49 | -------------------------------------------------------------------------------- /src/plumtree_broadcast_handler.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% Copyright (c) 2013 Basho Technologies, Inc. All Rights Reserved. 4 | %% 5 | %% This file is provided to you under the Apache License, 6 | %% Version 2.0 (the "License"); you may not use this file 7 | %% except in compliance with the License. You may obtain 8 | %% a copy of the License at 9 | %% 10 | %% http://www.apache.org/licenses/LICENSE-2.0 11 | %% 12 | %% Unless required by applicable law or agreed to in writing, 13 | %% software distributed under the License is distributed on an 14 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | %% KIND, either express or implied. See the License for the 16 | %% specific language governing permissions and limitations 17 | %% under the License. 18 | %% 19 | %% ------------------------------------------------------------------- 20 | -module(plumtree_broadcast_handler). 21 | 22 | %% Return a two-tuple of message id and payload from a given broadcast 23 | -callback broadcast_data(any()) -> {any(), any()}. 24 | 25 | %% Given the message id and payload, merge the message in the local state. 26 | %% If the message has already been received return `false', otherwise return `true' 27 | -callback merge(any(), any()) -> boolean(). 28 | 29 | %% Return true if the message (given the message id) has already been received. 30 | %% `false' otherwise 31 | -callback is_stale(any()) -> boolean(). 32 | 33 | %% Return the message associated with the given message id. In some cases a message 34 | %% has already been sent with information that subsumes the message associated with the given 35 | %% message id. In this case, `stale' is returned. 36 | -callback graft(any()) -> stale | {ok, any()} | {error, any()}. 37 | 38 | %% Trigger an exchange between the local handler and the handler on the given node. 39 | %% How the exchange is performed is not defined but it should be performed as a background 40 | %% process and ensure that it delivers any messages missing on either the local or remote node. 41 | %% The exchange does not need to account for messages in-flight when it is started or broadcast 42 | %% during its operation. These can be taken care of in future exchanges. 43 | -callback exchange(node()) -> {ok, pid()} | {error, term()}. 44 | -------------------------------------------------------------------------------- /src/plumtree_peer_service_events.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% Copyright (c) 2015 Helium Systems, Inc. All Rights Reserved. 4 | %% 5 | %% This file is provided to you under the Apache License, 6 | %% Version 2.0 (the "License"); you may not use this file 7 | %% except in compliance with the License. You may obtain 8 | %% a copy of the License at 9 | %% 10 | %% http://www.apache.org/licenses/LICENSE-2.0 11 | %% 12 | %% Unless required by applicable law or agreed to in writing, 13 | %% software distributed under the License is distributed on an 14 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | %% KIND, either express or implied. See the License for the 16 | %% specific language governing permissions and limitations 17 | %% under the License. 18 | %% 19 | %% ------------------------------------------------------------------- 20 | 21 | -module(plumtree_peer_service_events). 22 | 23 | -behaviour(gen_event). 24 | 25 | %% API 26 | -export([start_link/0, 27 | add_handler/2, 28 | add_sup_handler/2, 29 | add_callback/1, 30 | add_sup_callback/1, 31 | update/1 32 | ]). 33 | 34 | %% gen_event callbacks 35 | -export([init/1, handle_event/2, handle_call/2, 36 | handle_info/2, terminate/2, code_change/3]). 37 | 38 | -record(state, { callback }). 39 | 40 | %% =================================================================== 41 | %% API functions 42 | %% =================================================================== 43 | 44 | start_link() -> 45 | gen_event:start_link({local, ?MODULE}). 46 | 47 | add_handler(Handler, Args) -> 48 | gen_event:add_handler(?MODULE, Handler, Args). 49 | 50 | add_sup_handler(Handler, Args) -> 51 | gen_event:add_sup_handler(?MODULE, Handler, Args). 52 | 53 | add_callback(Fn) when is_function(Fn) -> 54 | gen_event:add_handler(?MODULE, {?MODULE, make_ref()}, [Fn]). 55 | 56 | add_sup_callback(Fn) when is_function(Fn) -> 57 | gen_event:add_sup_handler(?MODULE, {?MODULE, make_ref()}, [Fn]). 58 | 59 | update(LocalState) -> 60 | gen_event:notify(?MODULE, {update, LocalState}). 61 | 62 | %% =================================================================== 63 | %% gen_event callbacks 64 | %% =================================================================== 65 | 66 | init([Fn]) -> 67 | {ok, LocalState} = plumtree_peer_service_manager:get_local_state(), 68 | Fn(LocalState), 69 | {ok, #state { callback = Fn }}. 70 | 71 | handle_event({update, LocalState}, State) -> 72 | (State#state.callback)(LocalState), 73 | {ok, State}. 74 | 75 | handle_call(_Request, State) -> 76 | {ok, ok, State}. 77 | 78 | handle_info(_Info, State) -> 79 | {ok, State}. 80 | 81 | terminate(_Reason, _State) -> 82 | ok. 83 | 84 | code_change(_OldVsn, State, _Extra) -> 85 | {ok, State}. 86 | -------------------------------------------------------------------------------- /src/plumtree_peer_service_gossip.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% Copyright (c) 2014 Helium Systems, Inc. All Rights Reserved. 4 | %% 5 | %% This file is provided to you under the Apache License, 6 | %% Version 2.0 (the "License"); you may not use this file 7 | %% except in compliance with the License. You may obtain 8 | %% a copy of the License at 9 | %% 10 | %% http://www.apache.org/licenses/LICENSE-2.0 11 | %% 12 | %% Unless required by applicable law or agreed to in writing, 13 | %% software distributed under the License is distributed on an 14 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | %% KIND, either express or implied. See the License for the 16 | %% specific language governing permissions and limitations 17 | %% under the License. 18 | %% 19 | %% ------------------------------------------------------------------- 20 | 21 | -module(plumtree_peer_service_gossip). 22 | 23 | -behavior(gen_server). 24 | 25 | -define(GOSSIP_INTERVAL, 15000). 26 | 27 | -export([start_link/0, 28 | stop/0]). 29 | 30 | -export([receive_state/1]). 31 | 32 | -export([init/1, 33 | handle_call/3, 34 | handle_cast/2, 35 | handle_info/2, 36 | terminate/2, 37 | code_change/3]). 38 | 39 | -include("plumtree.hrl"). 40 | 41 | %%%================================================================== 42 | %%% gen_server api 43 | %%%================================================================== 44 | 45 | start_link() -> 46 | gen_server:start_link({local, ?MODULE}, ?MODULE, [], []). 47 | 48 | stop() -> 49 | gen_server:call(?MODULE, stop). 50 | 51 | receive_state(PeerState) -> 52 | gen_server:cast(?MODULE, {receive_state, PeerState}). 53 | 54 | 55 | %%%=============================================================== 56 | %%% gen_server callbacks 57 | %%%=============================================================== 58 | 59 | init([]) -> 60 | erlang:send_after(?GOSSIP_INTERVAL, ?MODULE, gossip), 61 | {ok, []}. 62 | 63 | handle_call(stop, _From, State) -> 64 | {stop, normal, State}; 65 | handle_call(send_state, _From, State) -> 66 | {ok, LocalState} = plumtree_peer_service_manager:get_local_state(), 67 | {reply, {ok, LocalState}, State}. 68 | 69 | handle_cast({receive_state, PeerState}, State) -> 70 | {ok, LocalState} = plumtree_peer_service_manager:get_local_state(), 71 | case ?SET:equal(PeerState, LocalState) of 72 | true -> 73 | %% do nothing 74 | {noreply, State}; 75 | false -> 76 | Merged = ?SET:merge(PeerState, LocalState), 77 | plumtree_peer_service_manager:update_state(Merged), 78 | plumtree_peer_service_events:update(Merged), 79 | {noreply, State} 80 | end. 81 | 82 | handle_info(gossip, State) -> 83 | _ = do_gossip(), 84 | erlang:send_after(?GOSSIP_INTERVAL, self(), gossip), 85 | {noreply, State}; 86 | 87 | handle_info(_Info, State) -> 88 | lager:info("Unexpected: ~p,~p.~n", [_Info, State]), 89 | {noreply, State}. 90 | 91 | terminate(_Reason, _State) -> 92 | lager:info("terminate ~p, ~p.~n", [_Reason, _State]), 93 | {ok, _State}. 94 | 95 | code_change(_OldVsn, State, _Extra) -> 96 | {ok, State}. 97 | 98 | %%%=============================================================== 99 | %%% private functions 100 | %%%=============================================================== 101 | 102 | %% @doc initiate gossip on local node 103 | do_gossip() -> 104 | {ok, Local} = plumtree_peer_service_manager:get_local_state(), 105 | case get_peers(Local) of 106 | [] -> 107 | {error, singleton}; 108 | Peers -> 109 | {ok, Peer} = random_peer(Peers), 110 | gen_server:cast({?MODULE, Peer}, {receive_state, Local}) 111 | end. 112 | 113 | %% @doc returns a list of peer nodes 114 | get_peers(Local) -> 115 | Members = ?SET:value(Local), 116 | Peers = [X || X <- Members, X /= node()], 117 | Peers. 118 | 119 | %% @doc return random peer from nodelist 120 | random_peer(Peers) -> 121 | Idx = random:uniform(length(Peers)), 122 | Peer = lists:nth(Idx, Peers), 123 | {ok, Peer}. 124 | -------------------------------------------------------------------------------- /src/plumtree_peer_service.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% Copyright (c) 2014 Helium Systems, Inc. All Rights Reserved. 4 | %% 5 | %% This file is provided to you under the Apache License, 6 | %% Version 2.0 (the "License"); you may not use this file 7 | %% except in compliance with the License. You may obtain 8 | %% a copy of the License at 9 | %% 10 | %% http://www.apache.org/licenses/LICENSE-2.0 11 | %% 12 | %% Unless required by applicable law or agreed to in writing, 13 | %% software distributed under the License is distributed on an 14 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | %% KIND, either express or implied. See the License for the 16 | %% specific language governing permissions and limitations 17 | %% under the License. 18 | %% 19 | %% ------------------------------------------------------------------- 20 | 21 | -module(plumtree_peer_service). 22 | 23 | -export([join/1, 24 | join/2, 25 | join/3, 26 | attempt_join/1, 27 | attempt_join/2, 28 | leave/1, 29 | stop/0, 30 | stop/1]). 31 | 32 | -include("plumtree.hrl"). 33 | 34 | %% @doc prepare node to join a cluster 35 | join(Node) -> 36 | join(Node, true). 37 | 38 | %% @doc Convert nodename to atom 39 | join(NodeStr, Auto) when is_list(NodeStr) -> 40 | join(erlang:list_to_atom(lists:flatten(NodeStr)), Auto); 41 | join(Node, Auto) when is_atom(Node) -> 42 | join(node(), Node, Auto). 43 | 44 | %% @doc Initiate join. Nodes cannot join themselves. 45 | join(Node, Node, _) -> 46 | {error, self_join}; 47 | join(_, Node, _Auto) -> 48 | attempt_join(Node). 49 | 50 | attempt_join(Node) -> 51 | lager:info("Sent join request to: ~p~n", [Node]), 52 | case net_kernel:connect(Node) of 53 | false -> 54 | lager:info("Unable to connect to ~p~n", [Node]), 55 | {error, not_reachable}; 56 | true -> 57 | {ok, Local} = plumtree_peer_service_manager:get_local_state(), 58 | attempt_join(Node, Local) 59 | end. 60 | 61 | attempt_join(Node, Local) -> 62 | {ok, Remote} = gen_server:call({plumtree_peer_service_gossip, Node}, send_state), 63 | Merged = ?SET:merge(Remote, Local), 64 | _ = plumtree_peer_service_manager:update_state(Merged), 65 | plumtree_peer_service_events:update(Merged), 66 | %% broadcast to all nodes 67 | %% get peer list 68 | Members = ?SET:value(Merged), 69 | _ = [gen_server:cast({plumtree_peer_service_gossip, P}, {receive_state, Merged}) || P <- Members, P /= node()], 70 | ok. 71 | 72 | leave(Args) when is_list(Args) -> 73 | {ok, Local} = plumtree_peer_service_manager:get_local_state(), 74 | {ok, Actor} = plumtree_peer_service_manager:get_actor(), 75 | {ok, Leave} = ?SET:update({remove, node()}, Actor, Local), 76 | case random_peer(Leave, Args) of 77 | {ok, Peer} -> 78 | try gen_server:call({plumtree_peer_service_gossip, Peer}, send_state) of 79 | {ok, Remote} -> 80 | Merged = ?SET:merge(Leave, Remote), 81 | _ = gen_server:cast({plumtree_peer_service_gossip, Peer}, {receive_state, Merged}), 82 | {ok, Remote2} = gen_server:call({plumtree_peer_service_gossip, Peer}, send_state), 83 | Remote2List = ?SET:value(Remote2), 84 | case [P || P <- Remote2List, P =:= node()] of 85 | [] -> 86 | %% leaving the cluster shuts down the node 87 | plumtree_peer_service_manager:delete_state(), 88 | stop("Leaving cluster"); 89 | _ -> 90 | leave([]) 91 | end; 92 | {error, singleton} -> 93 | lager:warning("Cannot leave, not a member of a cluster.") 94 | catch 95 | What:Why -> 96 | lager:debug("Error leaving cluster. What: ~p, Why: ~p", [What, Why]), 97 | leave([Peer|Args]) 98 | end 99 | end; 100 | leave(_Args) -> 101 | leave([]). 102 | 103 | stop() -> 104 | stop("received stop request"). 105 | 106 | stop(Reason) -> 107 | lager:notice("~p", [Reason]), 108 | init:stop(). 109 | 110 | random_peer(Leave, Banned) -> 111 | Members = ?SET:value(Leave), 112 | Peers = [P || P <- Members], 113 | case Peers -- Banned of 114 | [] -> 115 | {error, singleton}; 116 | _ -> 117 | Idx = random:uniform(length(Peers)), 118 | Peer = lists:nth(Idx, Peers), 119 | {ok, Peer} 120 | end. 121 | -------------------------------------------------------------------------------- /test/plumtree_test_utils.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% Copyright (c) 2015 Helium Systems, Inc. All Rights Reserved. 4 | %% 5 | %% This file is provided to you under the Apache License, 6 | %% Version 2.0 (the "License"); you may not use this file 7 | %% except in compliance with the License. You may obtain 8 | %% a copy of the License at 9 | %% 10 | %% http://www.apache.org/licenses/LICENSE-2.0 11 | %% 12 | %% Unless required by applicable law or agreed to in writing, 13 | %% software distributed under the License is distributed on an 14 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | %% KIND, either express or implied. See the License for the 16 | %% specific language governing permissions and limitations 17 | %% under the License. 18 | %% 19 | %% ------------------------------------------------------------------- 20 | 21 | -module(plumtree_test_utils). 22 | 23 | -export([get_cluster_members/1, 24 | pmap/2, 25 | wait_until/3, 26 | wait_until_left/2, 27 | wait_until_joined/2, 28 | wait_until_offline/1, 29 | wait_until_disconnected/2, 30 | wait_until_connected/2, 31 | start_node/3, 32 | partition_cluster/2, 33 | heal_cluster/2]). 34 | 35 | -include("plumtree.hrl"). 36 | 37 | get_cluster_members(Node) -> 38 | {Node, {ok, Res}} = {Node, rpc:call(Node, plumtree_peer_service_manager, get_local_state, [])}, 39 | ?SET:value(Res). 40 | 41 | pmap(F, L) -> 42 | Parent = self(), 43 | lists:foldl( 44 | fun(X, N) -> 45 | spawn_link(fun() -> 46 | Parent ! {pmap, N, F(X)} 47 | end), 48 | N+1 49 | end, 0, L), 50 | L2 = [receive {pmap, N, R} -> {N,R} end || _ <- L], 51 | {_, L3} = lists:unzip(lists:keysort(1, L2)), 52 | L3. 53 | 54 | wait_until(Fun, Retry, Delay) when Retry > 0 -> 55 | Res = Fun(), 56 | case Res of 57 | true -> 58 | ok; 59 | _ when Retry == 1 -> 60 | {fail, Res}; 61 | _ -> 62 | timer:sleep(Delay), 63 | wait_until(Fun, Retry-1, Delay) 64 | end. 65 | 66 | wait_until_left(Nodes, LeavingNode) -> 67 | wait_until(fun() -> 68 | lists:all(fun(X) -> X == true end, 69 | pmap(fun(Node) -> 70 | not 71 | lists:member(LeavingNode, 72 | get_cluster_members(Node)) 73 | end, Nodes)) 74 | end, 60*2, 500). 75 | 76 | wait_until_joined(Nodes, ExpectedCluster) -> 77 | wait_until(fun() -> 78 | lists:all(fun(X) -> X == true end, 79 | pmap(fun(Node) -> 80 | lists:sort(ExpectedCluster) == 81 | lists:sort(get_cluster_members(Node)) 82 | end, Nodes)) 83 | end, 60*2, 500). 84 | 85 | wait_until_offline(Node) -> 86 | wait_until(fun() -> 87 | pang == net_adm:ping(Node) 88 | end, 60*2, 500). 89 | 90 | wait_until_disconnected(Node1, Node2) -> 91 | wait_until(fun() -> 92 | pang == rpc:call(Node1, net_adm, ping, [Node2]) 93 | end, 60*2, 500). 94 | 95 | wait_until_connected(Node1, Node2) -> 96 | wait_until(fun() -> 97 | pong == rpc:call(Node1, net_adm, ping, [Node2]) 98 | end, 60*2, 500). 99 | 100 | start_node(Name, Config, Case) -> 101 | CodePath = lists:filter(fun filelib:is_dir/1, code:get_path()), 102 | %% have the slave nodes monitor the runner node, so they can't outlive it 103 | NodeConfig = [ 104 | {monitor_master, true}, 105 | {erl_flags, "-smp"}, %% smp for the eleveldb god 106 | {startup_functions, [ 107 | {code, set_path, [CodePath]} 108 | ]}], 109 | case ct_slave:start(Name, NodeConfig) of 110 | {ok, Node} -> 111 | 112 | PrivDir = proplists:get_value(priv_dir, Config), 113 | NodeDir = filename:join([PrivDir, Node, Case]), 114 | ok = rpc:call(Node, application, load, [plumtree]), 115 | ok = rpc:call(Node, application, load, [lager]), 116 | ok = rpc:call(Node, application, set_env, [lager, 117 | log_root, 118 | NodeDir]), 119 | ok = rpc:call(Node, application, set_env, [plumtree, 120 | plumtree_data_dir, 121 | NodeDir]), 122 | {ok, _} = rpc:call(Node, application, ensure_all_started, [plumtree]), 123 | ok = wait_until(fun() -> 124 | case rpc:call(Node, plumtree_peer_service_manager, get_local_state, []) of 125 | {ok, _Res} -> true; 126 | _ -> false 127 | end 128 | end, 60, 500), 129 | Node; 130 | {error, already_started, Node} -> 131 | ct_slave:stop(Name), 132 | wait_until_offline(Node), 133 | start_node(Name, Config, Case) 134 | end. 135 | 136 | partition_cluster(ANodes, BNodes) -> 137 | pmap(fun({Node1, Node2}) -> 138 | true = rpc:call(Node1, erlang, set_cookie, [Node2, canttouchthis]), 139 | true = rpc:call(Node1, erlang, disconnect_node, [Node2]), 140 | ok = wait_until_disconnected(Node1, Node2) 141 | end, 142 | [{Node1, Node2} || Node1 <- ANodes, Node2 <- BNodes]), 143 | ok. 144 | 145 | heal_cluster(ANodes, BNodes) -> 146 | GoodCookie = erlang:get_cookie(), 147 | pmap(fun({Node1, Node2}) -> 148 | true = rpc:call(Node1, erlang, set_cookie, [Node2, GoodCookie]), 149 | ok = wait_until_connected(Node1, Node2) 150 | end, 151 | [{Node1, Node2} || Node1 <- ANodes, Node2 <- BNodes]), 152 | ok. 153 | 154 | -------------------------------------------------------------------------------- /src/plumtree_metadata_object.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% Copyright (c) 2013 Basho Technologies, Inc. All Rights Reserved. 4 | %% 5 | %% This file is provided to you under the Apache License, 6 | %% Version 2.0 (the "License"); you may not use this file 7 | %% except in compliance with the License. You may obtain 8 | %% a copy of the License at 9 | %% 10 | %% http://www.apache.org/licenses/LICENSE-2.0 11 | %% 12 | %% Unless required by applicable law or agreed to in writing, 13 | %% software distributed under the License is distributed on an 14 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | %% KIND, either express or implied. See the License for the 16 | %% specific language governing permissions and limitations 17 | %% under the License. 18 | %% 19 | %% ------------------------------------------------------------------- 20 | -module(plumtree_metadata_object). 21 | 22 | -export([value/1, 23 | values/1, 24 | value_count/1, 25 | context/1, 26 | empty_context/0, 27 | hash/1, 28 | modify/4, 29 | reconcile/2, 30 | resolve/2, 31 | is_stale/2, 32 | equal_context/2]). 33 | 34 | -include("plumtree_metadata.hrl"). 35 | 36 | %% @doc returns a single value. if the object holds more than one value an error is generated 37 | %% @see values/2 38 | -spec value(metadata_object()) -> metadata_value(). 39 | value(Metadata) -> 40 | [Value] = values(Metadata), 41 | Value. 42 | 43 | %% @doc returns a list of values held in the object 44 | -spec values(metadata_object()) -> [metadata_value()]. 45 | values({metadata, Object}) -> 46 | [Value || {Value, _Ts} <- dvvset:values(Object)]. 47 | 48 | %% @doc returns the number of siblings in the given object 49 | -spec value_count(metadata_object()) -> non_neg_integer(). 50 | value_count({metadata, Object}) -> 51 | dvvset:size(Object). 52 | 53 | %% @doc returns the context (opaque causal history) for the given object 54 | -spec context(metadata_object()) -> metadata_context(). 55 | context({metadata, Object}) -> 56 | dvvset:join(Object). 57 | 58 | %% @doc returns the representation for an empty context (opaque causal history) 59 | -spec empty_context() -> metadata_context(). 60 | empty_context() -> []. 61 | 62 | %% @doc returns a hash representing the metadata objects contents 63 | -spec hash(metadata_object()) -> binary(). 64 | hash({metadata, Object}) -> 65 | crypto:hash(sha, term_to_binary(Object)). 66 | 67 | %% @doc modifies a potentially existing object, setting its value and updating 68 | %% the causual history. If a function is provided as the third argument 69 | %% then this function also is used for conflict resolution. The difference 70 | %% between this function and resolve/2 is that the logical clock is advanced in the 71 | %% case of this function. Additionally, the resolution functions are slightly different. 72 | -spec modify(metadata_object() | undefined, 73 | metadata_context(), 74 | metadata_value() | metadata_modifier(), 75 | term()) -> metadata_object(). 76 | modify(undefined, Context, Fun, ServerId) when is_function(Fun) -> 77 | modify(undefined, Context, Fun(undefined), ServerId); 78 | modify(Obj, Context, Fun, ServerId) when is_function(Fun) -> 79 | modify(Obj, Context, Fun(values(Obj)), ServerId); 80 | modify(undefined, _Context, Value, ServerId) -> 81 | %% Ignore the context since we dont have a value, its invalid if not 82 | %% empty anyways, so give it a valid one 83 | NewRecord = dvvset:new(timestamped_value(Value)), 84 | {metadata, dvvset:update(NewRecord, ServerId)}; 85 | modify({metadata, Existing}, Context, Value, ServerId) -> 86 | InsertRec = dvvset:new(Context, timestamped_value(Value)), 87 | {metadata, dvvset:update(InsertRec, Existing, ServerId)}. 88 | 89 | %% @doc Reconciles a remote object received during replication or anti-entropy 90 | %% with a local object. If the remote object is an anscestor of or is equal to the local one 91 | %% `false' is returned, otherwise the reconciled object is returned as the second 92 | %% element of the two-tuple 93 | -spec reconcile(metadata_object(), metadata_object() | undefined) -> 94 | false | {true, metadata_object()}. 95 | reconcile(undefined, _LocalObj) -> 96 | false; 97 | reconcile(RemoteObj, undefined) -> 98 | {true, RemoteObj}; 99 | reconcile({metadata, RemoteObj}, {metadata, LocalObj}) -> 100 | Less = dvvset:less(RemoteObj, LocalObj), 101 | Equal = dvvset:equal(RemoteObj, LocalObj), 102 | case not (Equal or Less) of 103 | false -> false; 104 | true -> 105 | {true, {metadata, dvvset:sync([LocalObj, RemoteObj])}} 106 | end. 107 | 108 | %% @doc Resolves siblings using either last-write-wins or the provided function and returns 109 | %% an object containing a single value. The causal history is not updated 110 | -spec resolve(metadata_object(), lww | fun(([metadata_value()]) -> metadata_value())) -> 111 | metadata_object(). 112 | resolve({metadata, Object}, lww) -> 113 | LWW = fun ({_,TS1}, {_,TS2}) -> TS1 =< TS2 end, 114 | {metadata, dvvset:lww(LWW, Object)}; 115 | resolve({metadata, Existing}, Reconcile) when is_function(Reconcile) -> 116 | ResolveFun = fun({A, _}, {B, _}) -> timestamped_value(Reconcile(A, B)) end, 117 | F = fun([Value | Rest]) -> lists:foldl(ResolveFun, Value, Rest) end, 118 | {metadata, dvvset:reconcile(F, Existing)}. 119 | 120 | %% @doc Determines if the given context (version vector) is causually newer than 121 | %% an existing object. If the object missing or if the context does not represent 122 | %% an anscestor of the current key, false is returned. Otherwise, when the context 123 | %% does represent an ancestor of the existing object or the existing object itself, 124 | %% true is returned 125 | -spec is_stale(metadata_context(), metadata_object()) -> boolean(). 126 | is_stale(_, undefined) -> 127 | false; 128 | is_stale(RemoteContext, {metadata, Obj}) -> 129 | LocalContext = dvvset:join(Obj), 130 | %% returns true (stale) when local context is causally newer or equal to remote context 131 | descends(LocalContext, RemoteContext). 132 | 133 | descends(_, []) -> 134 | true; 135 | descends(Ca, Cb) -> 136 | [{NodeB, CtrB} | RestB] = Cb, 137 | case lists:keyfind(NodeB, 1, Ca) of 138 | false -> false; 139 | {_, CtrA} -> 140 | (CtrA >= CtrB) andalso descends(Ca, RestB) 141 | end. 142 | 143 | %% @doc Returns true if the given context and the context of the existing object are equal 144 | -spec equal_context(metadata_context(), metadata_object()) -> boolean(). 145 | equal_context(Context, {metadata, Obj}) -> 146 | Context =:= dvvset:join(Obj). 147 | 148 | timestamped_value(Value) -> 149 | {Value, os:timestamp()}. 150 | -------------------------------------------------------------------------------- /src/plumtree_peer_service_manager.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% Copyright (c) 2014 Helium Systems, Inc. All Rights Reserved. 4 | %% 5 | %% This file is provided to you under the Apache License, 6 | %% Version 2.0 (the "License"); you may not use this file 7 | %% except in compliance with the License. You may obtain 8 | %% a copy of the License at 9 | %% 10 | %% http://www.apache.org/licenses/LICENSE-2.0 11 | %% 12 | %% Unless required by applicable law or agreed to in writing, 13 | %% software distributed under the License is distributed on an 14 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | %% KIND, either express or implied. See the License for the 16 | %% specific language governing permissions and limitations 17 | %% under the License. 18 | %% 19 | %% ------------------------------------------------------------------- 20 | 21 | -module(plumtree_peer_service_manager). 22 | 23 | -behaviour(gen_server). 24 | 25 | %% API 26 | -export([start_link/0, 27 | members/0, 28 | get_local_state/0, 29 | get_actor/0, 30 | update_state/1, 31 | delete_state/0]). 32 | 33 | %% gen_server callbacks 34 | -export([init/1, 35 | handle_call/3, 36 | handle_cast/2, 37 | handle_info/2, 38 | terminate/2, 39 | code_change/3]). 40 | 41 | -include("plumtree.hrl"). 42 | 43 | -type actor() :: binary(). 44 | -type membership() :: ?SET:orswot(). 45 | 46 | -record(state, {actor :: actor(), 47 | membership :: membership() }). 48 | 49 | %%%=================================================================== 50 | %%% API 51 | %%%=================================================================== 52 | 53 | %% @doc Same as start_link([]). 54 | -spec start_link() -> {ok, pid()} | ignore | {error, term()}. 55 | start_link() -> 56 | gen_server:start_link({local, ?MODULE}, ?MODULE, [], []). 57 | 58 | %% @doc Return membership list. 59 | members() -> 60 | gen_server:call(?MODULE, members, infinity). 61 | 62 | %% @doc Return local node's view of cluster membership. 63 | get_local_state() -> 64 | gen_server:call(?MODULE, get_local_state, infinity). 65 | 66 | %% @doc Return local node's current actor. 67 | get_actor() -> 68 | gen_server:call(?MODULE, get_actor, infinity). 69 | 70 | %% @doc Update cluster state. 71 | update_state(State) -> 72 | gen_server:call(?MODULE, {update_state, State}, infinity). 73 | 74 | %% @doc Delete state. 75 | delete_state() -> 76 | gen_server:call(?MODULE, delete_state, infinity). 77 | 78 | %%%=================================================================== 79 | %%% gen_server callbacks 80 | %%%=================================================================== 81 | 82 | %% @private 83 | -spec init([]) -> {ok, #state{}}. 84 | init([]) -> 85 | Actor = gen_actor(), 86 | Membership = maybe_load_state_from_disk(Actor), 87 | {ok, #state{actor=Actor, membership=Membership}}. 88 | 89 | %% @private 90 | -spec handle_call(term(), {pid(), term()}, #state{}) -> {reply, term(), #state{}}. 91 | handle_call(members, _From, #state{membership=Membership}=State) -> 92 | {reply, {ok, ?SET:value(Membership)}, State}; 93 | handle_call(get_local_state, _From, #state{membership=Membership}=State) -> 94 | {reply, {ok, Membership}, State}; 95 | handle_call(get_actor, _From, #state{actor=Actor}=State) -> 96 | {reply, {ok, Actor}, State}; 97 | handle_call({update_state, NewState}, _From, #state{membership=Membership}=State) -> 98 | Merged = ?SET:merge(Membership, NewState), 99 | persist_state(Merged), 100 | {reply, ok, State#state{membership=Merged}}; 101 | handle_call(delete_state, _From, State) -> 102 | delete_state_from_disk(), 103 | {reply, ok, State}; 104 | handle_call(Msg, _From, State) -> 105 | lager:warning("Unhandled messages: ~p", [Msg]), 106 | {reply, ok, State}. 107 | 108 | %% @private 109 | -spec handle_cast(term(), #state{}) -> {noreply, #state{}}. 110 | handle_cast(Msg, State) -> 111 | lager:warning("Unhandled messages: ~p", [Msg]), 112 | {noreply, State}. 113 | 114 | %% @private 115 | -spec handle_info(term(), #state{}) -> {noreply, #state{}}. 116 | handle_info(Msg, State) -> 117 | lager:warning("Unhandled messages: ~p", [Msg]), 118 | {noreply, State}. 119 | 120 | %% @private 121 | -spec terminate(term(), #state{}) -> term(). 122 | terminate(_Reason, _State) -> 123 | ok. 124 | 125 | %% @private 126 | -spec code_change(term() | {down, term()}, #state{}, term()) -> {ok, #state{}}. 127 | code_change(_OldVsn, State, _Extra) -> 128 | {ok, State}. 129 | 130 | %%%=================================================================== 131 | %%% Internal functions 132 | %%%=================================================================== 133 | 134 | %% @private 135 | empty_membership(Actor) -> 136 | Initial = ?SET:new(), 137 | {ok, LocalState} = ?SET:update({add, node()}, Actor, Initial), 138 | persist_state(LocalState), 139 | LocalState. 140 | 141 | %% @private 142 | gen_actor() -> 143 | Node = atom_to_list(node()), 144 | Unique = time_compat:unique_integer([positive]), 145 | TS = integer_to_list(Unique), 146 | Term = Node ++ TS, 147 | crypto:hash(sha, Term). 148 | 149 | %% @private 150 | data_root() -> 151 | case application:get_env(plumtree, plumtree_data_dir) of 152 | {ok, PRoot} -> 153 | filename:join(PRoot, "peer_service"); 154 | undefined -> 155 | undefined 156 | end. 157 | 158 | %% @private 159 | write_state_to_disk(State) -> 160 | case data_root() of 161 | undefined -> 162 | ok; 163 | Dir -> 164 | File = filename:join(Dir, "cluster_state"), 165 | ok = filelib:ensure_dir(File), 166 | ok = file:write_file(File, ?SET:to_binary(State)) 167 | end. 168 | 169 | %% @private 170 | delete_state_from_disk() -> 171 | case data_root() of 172 | undefined -> 173 | ok; 174 | Dir -> 175 | File = filename:join(Dir, "cluster_state"), 176 | ok = filelib:ensure_dir(File), 177 | case file:delete(File) of 178 | ok -> 179 | lager:info("Leaving cluster, removed cluster_state"); 180 | {error, Reason} -> 181 | lager:info("Unable to remove cluster_state for reason ~p", [Reason]) 182 | end 183 | end. 184 | 185 | %% @private 186 | maybe_load_state_from_disk(Actor) -> 187 | case data_root() of 188 | undefined -> 189 | empty_membership(Actor); 190 | Dir -> 191 | case filelib:is_regular(filename:join(Dir, "cluster_state")) of 192 | true -> 193 | {ok, Bin} = file:read_file(filename:join(Dir, "cluster_state")), 194 | {ok, State} = ?SET:from_binary(Bin), 195 | State; 196 | false -> 197 | empty_membership(Actor) 198 | end 199 | end. 200 | 201 | %% @private 202 | persist_state(State) -> 203 | write_state_to_disk(State). 204 | -------------------------------------------------------------------------------- /test/cluster_membership_SUITE.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% Copyright (c) 2015 Helium Systems, Inc. All Rights Reserved. 4 | %% 5 | %% This file is provided to you under the Apache License, 6 | %% Version 2.0 (the "License"); you may not use this file 7 | %% except in compliance with the License. You may obtain 8 | %% a copy of the License at 9 | %% 10 | %% http://www.apache.org/licenses/LICENSE-2.0 11 | %% 12 | %% Unless required by applicable law or agreed to in writing, 13 | %% software distributed under the License is distributed on an 14 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | %% KIND, either express or implied. See the License for the 16 | %% specific language governing permissions and limitations 17 | %% under the License. 18 | %% 19 | %% ------------------------------------------------------------------- 20 | 21 | -module(cluster_membership_SUITE). 22 | -compile({parse_transform, lager_transform}). 23 | 24 | -export([ 25 | %% suite/0, 26 | init_per_suite/1, 27 | end_per_suite/1, 28 | init_per_testcase/2, 29 | end_per_testcase/2, 30 | all/0 31 | ]). 32 | 33 | -export([ 34 | singleton_test/1, 35 | join_test/1, 36 | join_nonexistant_node_test/1, 37 | join_self_test/1, 38 | leave_test/1, 39 | leave_rejoin_test/1, 40 | sticky_membership_test/1 41 | ]). 42 | 43 | -include_lib("common_test/include/ct.hrl"). 44 | -include_lib("eunit/include/eunit.hrl"). 45 | -include_lib("kernel/include/inet.hrl"). 46 | 47 | 48 | %% =================================================================== 49 | %% common_test callbacks 50 | %% =================================================================== 51 | 52 | init_per_suite(_Config) -> 53 | lager:start(), 54 | %% this might help, might not... 55 | os:cmd(os:find_executable("epmd")++" -daemon"), 56 | {ok, Hostname} = inet:gethostname(), 57 | case net_kernel:start([list_to_atom("runner@"++Hostname), shortnames]) of 58 | {ok, _} -> ok; 59 | {error, {already_started, _}} -> ok; 60 | {error, {{already_started, _},_}} -> ok 61 | end, 62 | _Config. 63 | 64 | end_per_suite(_Config) -> 65 | application:stop(lager), 66 | _Config. 67 | 68 | init_per_testcase(Case, Config) -> 69 | Nodes = plumtree_test_utils:pmap(fun(N) -> 70 | plumtree_test_utils:start_node(N, Config, Case) 71 | end, [jaguar, shadow, thorn, pyros]), 72 | {ok, _} = ct_cover:add_nodes(Nodes), 73 | [{nodes, Nodes}|Config]. 74 | 75 | end_per_testcase(_, _Config) -> 76 | plumtree_test_utils:pmap(fun(Node) ->ct_slave:stop(Node) end, [jaguar, shadow, thorn, pyros]), 77 | ok. 78 | 79 | all() -> 80 | [singleton_test, join_test, join_nonexistant_node_test, join_self_test, 81 | leave_test, leave_rejoin_test, sticky_membership_test]. 82 | 83 | singleton_test(Config) -> 84 | Nodes = proplists:get_value(nodes, Config), 85 | ok = ct_cover:remove_nodes(Nodes), 86 | [[Node] = plumtree_test_utils:get_cluster_members(Node) || Node <- Nodes], 87 | ok. 88 | 89 | join_test(Config) -> 90 | [Node1, Node2 |Nodes] = proplists:get_value(nodes, Config), 91 | ?assertEqual(ok, rpc:call(Node1, plumtree_peer_service, join, [Node2])), 92 | Expected = lists:sort([Node1, Node2]), 93 | ok = plumtree_test_utils:wait_until_joined([Node1, Node2], Expected), 94 | ?assertEqual(Expected, lists:sort(plumtree_test_utils:get_cluster_members(Node1))), 95 | ?assertEqual(Expected, lists:sort(plumtree_test_utils:get_cluster_members(Node2))), 96 | %% make sure the last 2 are still singletons 97 | [?assertEqual([Node], plumtree_test_utils:get_cluster_members(Node)) || Node <- Nodes], 98 | ok. 99 | 100 | join_nonexistant_node_test(Config) -> 101 | [Node1|_] = proplists:get_value(nodes, Config), 102 | ?assertEqual({error, not_reachable}, rpc:call(Node1, plumtree_peer_service, join, 103 | [fake@fakehost])), 104 | ?assertEqual([Node1], plumtree_test_utils:get_cluster_members(Node1)), 105 | ok. 106 | 107 | join_self_test(Config) -> 108 | [Node1|_] = proplists:get_value(nodes, Config), 109 | ?assertEqual({error, self_join}, rpc:call(Node1, plumtree_peer_service, join, 110 | [Node1])), 111 | ?assertEqual([Node1], plumtree_test_utils:get_cluster_members(Node1)), 112 | ok. 113 | 114 | leave_test(Config) -> 115 | [Node1|OtherNodes] = Nodes = proplists:get_value(nodes, Config), 116 | [?assertEqual(ok, rpc:call(Node, plumtree_peer_service, join, [Node1])) 117 | || Node <- OtherNodes], 118 | Expected = lists:sort(Nodes), 119 | ok = plumtree_test_utils:wait_until_joined(Nodes, Expected), 120 | [?assertEqual({Node, Expected}, {Node, 121 | lists:sort(plumtree_test_utils:get_cluster_members(Node))}) 122 | || Node <- Nodes], 123 | ?assertEqual(ok, rpc:call(Node1, plumtree_peer_service, leave, [[]])), 124 | Expected2 = lists:sort(OtherNodes), 125 | ok = plumtree_test_utils:wait_until_left(OtherNodes, Node1), 126 | %% should be a 3 node cluster now 127 | [?assertEqual({Node, Expected2}, {Node, 128 | lists:sort(plumtree_test_utils:get_cluster_members(Node))}) 129 | || Node <- OtherNodes], 130 | %% node1 should be offline 131 | ?assertEqual(pang, net_adm:ping(Node1)), 132 | ok. 133 | 134 | leave_rejoin_test(Config) -> 135 | [Node1|OtherNodes] = Nodes = proplists:get_value(nodes, Config), 136 | [Node2|_Rest] = OtherNodes, 137 | [?assertEqual(ok, rpc:call(Node, plumtree_peer_service, join, [Node1])) 138 | || Node <- OtherNodes], 139 | Expected = lists:sort(Nodes), 140 | ok = plumtree_test_utils:wait_until_joined(Nodes, Expected), 141 | [?assertEqual({Node, Expected}, {Node, 142 | lists:sort(plumtree_test_utils:get_cluster_members(Node))}) 143 | || Node <- Nodes], 144 | ?assertEqual(ok, rpc:call(Node1, plumtree_peer_service, leave, [[]])), 145 | Expected2 = lists:sort(OtherNodes), 146 | ok = plumtree_test_utils:wait_until_left(OtherNodes, Node1), 147 | %% should be a 3 node cluster now 148 | [?assertEqual({Node, Expected2}, {Node, 149 | lists:sort(plumtree_test_utils:get_cluster_members(Node))}) 150 | || Node <- OtherNodes], 151 | %% node1 should be offline 152 | ?assertEqual(pang, net_adm:ping(Node1)), 153 | plumtree_test_utils:start_node(jaguar, Config, leave_rejoin_test), 154 | %% rejoin cluster 155 | ?assertEqual(ok, rpc:call(Node1, plumtree_peer_service, join, [Node2])), 156 | ok = plumtree_test_utils:wait_until_joined(Nodes, Expected), 157 | [?assertEqual({Node, Expected}, {Node, 158 | lists:sort(plumtree_test_utils:get_cluster_members(Node))}) 159 | || Node <- Nodes], 160 | ok. 161 | 162 | sticky_membership_test(Config) -> 163 | [Node1|OtherNodes] = Nodes = proplists:get_value(nodes, Config), 164 | [?assertEqual(ok, rpc:call(Node, plumtree_peer_service, join, [Node1])) 165 | || Node <- OtherNodes], 166 | Expected = lists:sort(Nodes), 167 | ok = plumtree_test_utils:wait_until_joined(Nodes, Expected), 168 | [?assertEqual({Node, Expected}, {Node, 169 | lists:sort(plumtree_test_utils:get_cluster_members(Node))}) 170 | || Node <- Nodes], 171 | ct_slave:stop(jaguar), 172 | ok = plumtree_test_utils:wait_until_offline(Node1), 173 | %% check the membership is the same 174 | [?assertEqual({Node, Expected}, {Node, 175 | lists:sort(plumtree_test_utils:get_cluster_members(Node))}) 176 | || Node <- OtherNodes], 177 | plumtree_test_utils:start_node(jaguar, Config, sticky_membership_test), 178 | ?assertEqual({Node1, Expected}, {Node1, 179 | lists:sort(plumtree_test_utils:get_cluster_members(Node1))}), 180 | ct_slave:stop(jaguar), 181 | ok = plumtree_test_utils:wait_until_offline(Node1), 182 | [Node2|LastTwo] = OtherNodes, 183 | ?assertEqual(ok, rpc:call(Node2, plumtree_peer_service, leave, [[]])), 184 | ok = plumtree_test_utils:wait_until_left(LastTwo, Node2), 185 | ok = plumtree_test_utils:wait_until_offline(Node2), 186 | Expected2 = lists:sort(Nodes -- [Node2]), 187 | [?assertEqual({Node, Expected2}, {Node, 188 | lists:sort(plumtree_test_utils:get_cluster_members(Node))}) 189 | || Node <- LastTwo], 190 | plumtree_test_utils:start_node(jaguar, Config, sticky_membership_test), 191 | ok = plumtree_test_utils:wait_until_left([Node1], Node2), 192 | ?assertEqual({Node1, Expected2}, {Node1, 193 | lists:sort(plumtree_test_utils:get_cluster_members(Node1))}), 194 | plumtree_test_utils:start_node(shadow, Config, sticky_membership_test), 195 | %% node 2 should be a singleton now 196 | ?assertEqual([Node2], plumtree_test_utils:get_cluster_members(Node2)), 197 | ok. 198 | 199 | 200 | %% =================================================================== 201 | %% utility functions 202 | %% =================================================================== 203 | 204 | -------------------------------------------------------------------------------- /test/metadata_SUITE.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% Copyright (c) 2015 Helium Systems, Inc. All Rights Reserved. 4 | %% 5 | %% This file is provided to you under the Apache License, 6 | %% Version 2.0 (the "License"); you may not use this file 7 | %% except in compliance with the License. You may obtain 8 | %% a copy of the License at 9 | %% 10 | %% http://www.apache.org/licenses/LICENSE-2.0 11 | %% 12 | %% Unless required by applicable law or agreed to in writing, 13 | %% software distributed under the License is distributed on an 14 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | %% KIND, either express or implied. See the License for the 16 | %% specific language governing permissions and limitations 17 | %% under the License. 18 | %% 19 | %% ------------------------------------------------------------------- 20 | 21 | -module(metadata_SUITE). 22 | -compile({parse_transform, lager_transform}). 23 | 24 | -export([ 25 | %% suite/0, 26 | init_per_suite/1, 27 | end_per_suite/1, 28 | init_per_testcase/2, 29 | end_per_testcase/2, 30 | all/0 31 | ]). 32 | 33 | -export([ 34 | read_write_delete_test/1, 35 | partitioned_cluster_test/1, 36 | siblings_test/1 37 | ]). 38 | 39 | -include_lib("common_test/include/ct.hrl"). 40 | -include_lib("eunit/include/eunit.hrl"). 41 | -include_lib("kernel/include/inet.hrl"). 42 | 43 | 44 | %% =================================================================== 45 | %% common_test callbacks 46 | %% =================================================================== 47 | 48 | init_per_suite(_Config) -> 49 | lager:start(), 50 | %% this might help, might not... 51 | os:cmd(os:find_executable("epmd")++" -daemon"), 52 | {ok, Hostname} = inet:gethostname(), 53 | case net_kernel:start([list_to_atom("runner@"++Hostname), shortnames]) of 54 | {ok, _} -> ok; 55 | {error, {already_started, _}} -> ok; 56 | {error, {{already_started, _},_}} -> ok 57 | end, 58 | _Config. 59 | 60 | end_per_suite(_Config) -> 61 | application:stop(lager), 62 | _Config. 63 | 64 | init_per_testcase(Case, Config) -> 65 | Nodes = plumtree_test_utils:pmap(fun(N) -> 66 | plumtree_test_utils:start_node(N, Config, Case) 67 | end, [electra, katana, flail, gargoyle]), 68 | [{nodes, Nodes}|Config]. 69 | 70 | end_per_testcase(_, _Config) -> 71 | plumtree_test_utils:pmap(fun(Node) ->ct_slave:stop(Node) end, [electra, katana, flail, gargoyle]), 72 | ok. 73 | 74 | all() -> 75 | [read_write_delete_test, partitioned_cluster_test, siblings_test]. 76 | 77 | read_write_delete_test(Config) -> 78 | [Node1|OtherNodes] = Nodes = proplists:get_value(nodes, Config), 79 | [?assertEqual(ok, rpc:call(Node, plumtree_peer_service, join, [Node1])) 80 | || Node <- OtherNodes], 81 | Expected = lists:sort(Nodes), 82 | ok = plumtree_test_utils:wait_until_joined(Nodes, Expected), 83 | [?assertEqual({Node, Expected}, {Node, 84 | lists:sort(plumtree_test_utils:get_cluster_members(Node))}) 85 | || Node <- Nodes], 86 | ?assertEqual(undefined, get_metadata(Node1, {foo, bar}, baz, [])), 87 | ok = put_metadata(Node1, {foo, bar}, baz, quux, []), 88 | ?assertEqual(quux, get_metadata(Node1, {foo, bar}, baz, [])), 89 | ok = wait_until_converged(Nodes, {foo, bar}, baz, quux), 90 | ok = put_metadata(Node1, {foo, bar}, baz, norf, []), 91 | ok = wait_until_converged(Nodes, {foo, bar}, baz, norf), 92 | ok = delete_metadata(Node1, {foo, bar}, baz), 93 | ok = wait_until_converged(Nodes, {foo, bar}, baz, undefined), 94 | ok. 95 | 96 | partitioned_cluster_test(Config) -> 97 | [Node1|OtherNodes] = Nodes = proplists:get_value(nodes, Config), 98 | [?assertEqual(ok, rpc:call(Node, plumtree_peer_service, join, [Node1])) 99 | || Node <- OtherNodes], 100 | Expected = lists:sort(Nodes), 101 | ok = plumtree_test_utils:wait_until_joined(Nodes, Expected), 102 | [?assertEqual({Node, Expected}, {Node, 103 | lists:sort(plumtree_test_utils:get_cluster_members(Node))}) 104 | || Node <- Nodes], 105 | ok = wait_until_converged(Nodes, {foo, bar}, baz, undefined), 106 | ok = put_metadata(Node1, {foo, bar}, baz, quux, []), 107 | ok = wait_until_converged(Nodes, {foo, bar}, baz, quux), 108 | {ANodes, BNodes} = lists:split(2, Nodes), 109 | plumtree_test_utils:partition_cluster(ANodes, BNodes), 110 | %% write to one side 111 | ok = put_metadata(Node1, {foo, bar}, baz, norf, []), 112 | %% check that whole side has the new value 113 | ok = wait_until_converged(ANodes, {foo, bar}, baz, norf), 114 | %% the far side should have the old value 115 | ok = wait_until_converged(BNodes, {foo, bar}, baz, quux), 116 | plumtree_test_utils:heal_cluster(ANodes, BNodes), 117 | %% all the nodes should see the new value 118 | ok = wait_until_converged(Nodes, {foo, bar}, baz, norf), 119 | ok. 120 | 121 | siblings_test(Config) -> 122 | [Node1|OtherNodes] = Nodes = proplists:get_value(nodes, Config), 123 | [?assertEqual(ok, rpc:call(Node, plumtree_peer_service, join, [Node1])) 124 | || Node <- OtherNodes], 125 | Expected = lists:sort(Nodes), 126 | ok = plumtree_test_utils:wait_until_joined(Nodes, Expected), 127 | [?assertEqual({Node, Expected}, {Node, 128 | lists:sort(plumtree_test_utils:get_cluster_members(Node))}) 129 | || Node <- Nodes], 130 | ok = wait_until_converged(Nodes, {foo, bar}, baz, undefined), 131 | ok = put_metadata(Node1, {foo, bar}, baz, quux, []), 132 | ok = put_metadata(Node1, {foo, bar}, canary, 1, []), 133 | ok = wait_until_converged(Nodes, {foo, bar}, baz, quux), 134 | ok = wait_until_converged(Nodes, {foo, bar}, canary, 1), 135 | {ANodes, BNodes} = lists:split(2, Nodes), 136 | plumtree_test_utils:partition_cluster(ANodes, BNodes), 137 | %% write to one side 138 | ok = put_metadata(Node1, {foo, bar}, baz, norf, []), 139 | ok = put_metadata(Node1, {foo, bar}, canary, 2, []), 140 | %% check that whole side has the new value 141 | ok = wait_until_converged(ANodes, {foo, bar}, baz, norf), 142 | ok = wait_until_converged(ANodes, {foo, bar}, canary, 2), 143 | %% the far side should have the old value 144 | ok = wait_until_converged(BNodes, {foo, bar}, baz, quux), 145 | ok = wait_until_converged(BNodes, {foo, bar}, canary, 1), 146 | %% write a competing value to the other side 147 | [Node3|_] = BNodes, 148 | ok = put_metadata(Node3, {foo, bar}, baz, mork, []), 149 | ok = wait_until_converged(BNodes, {foo, bar}, baz, mork), 150 | plumtree_test_utils:heal_cluster(ANodes, BNodes), 151 | %% block until the canary key converges 152 | ok = wait_until_converged(Nodes, {foo, bar}, canary, 2), 153 | %% make sure we have siblings, but don't resolve them yet 154 | ok = wait_until_sibling(Nodes, {foo, bar}, baz), 155 | %% resolve the sibling 156 | spork = get_metadata(Node1, {foo, bar}, baz, [{resolver, fun(_A, _B) -> 157 | spork end}, {allow_put, false}]), 158 | %% without allow_put set, all the siblings are still there... 159 | ok = wait_until_sibling(Nodes, {foo, bar}, baz), 160 | %% resolve the sibling and write it back 161 | spork = get_metadata(Node1, {foo, bar}, baz, [{resolver, fun(_A, _B) -> 162 | spork end}, {allow_put, true}]), 163 | %% check all the nodes see the resolution 164 | ok = wait_until_converged(Nodes, {foo, bar}, baz, spork), 165 | ok. 166 | 167 | 168 | %% =================================================================== 169 | %% utility functions 170 | %% =================================================================== 171 | 172 | get_metadata(Node, Prefix, Key, Opts) -> 173 | rpc:call(Node, plumtree_metadata, get, [Prefix, Key, Opts]). 174 | 175 | put_metadata(Node, Prefix, Key, ValueOrFun, Opts) -> 176 | rpc:call(Node, plumtree_metadata, put, [Prefix, Key, ValueOrFun, Opts]). 177 | 178 | delete_metadata(Node, Prefix, Key) -> 179 | rpc:call(Node, plumtree_metadata, delete, [Prefix, Key]). 180 | 181 | wait_until_converged(Nodes, Prefix, Key, ExpectedValue) -> 182 | plumtree_test_utils:wait_until(fun() -> 183 | lists:all(fun(X) -> X == true end, 184 | plumtree_test_utils:pmap(fun(Node) -> 185 | ExpectedValue == get_metadata(Node, Prefix, 186 | Key, 187 | [{allow_put, 188 | false}]) 189 | end, Nodes)) 190 | end, 60*2, 500). 191 | 192 | 193 | wait_until_sibling(Nodes, Prefix, Key) -> 194 | plumtree_test_utils:wait_until(fun() -> 195 | lists:all(fun(X) -> X == true end, 196 | plumtree_test_utils:pmap(fun(Node) -> 197 | case rpc:call(Node, plumtree_metadata_manager, 198 | get, [{Prefix, Key}]) of 199 | undefined -> false; 200 | Value -> 201 | rpc:call(Node, 202 | plumtree_metadata_object, 203 | value_count, [Value]) > 1 204 | end 205 | end, Nodes)) 206 | end, 60*2, 500). 207 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Copyright (c) 2015, Christopher Meiklejohn 2 | Copyright (c) 2015, Basho Technologies, Inc 3 | Copyright (c) 2015, Helium Systems, Inc 4 | All rights reserved. 5 | 6 | 7 | Apache License 8 | Version 2.0, January 2004 9 | http://www.apache.org/licenses/ 10 | 11 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 12 | 13 | 1. Definitions. 14 | 15 | "License" shall mean the terms and conditions for use, reproduction, 16 | and distribution as defined by Sections 1 through 9 of this document. 17 | 18 | "Licensor" shall mean the copyright owner or entity authorized by 19 | the copyright owner that is granting the License. 20 | 21 | "Legal Entity" shall mean the union of the acting entity and all 22 | other entities that control, are controlled by, or are under common 23 | control with that entity. For the purposes of this definition, 24 | "control" means (i) the power, direct or indirect, to cause the 25 | direction or management of such entity, whether by contract or 26 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 27 | outstanding shares, or (iii) beneficial ownership of such entity. 28 | 29 | "You" (or "Your") shall mean an individual or Legal Entity 30 | exercising permissions granted by this License. 31 | 32 | "Source" form shall mean the preferred form for making modifications, 33 | including but not limited to software source code, documentation 34 | source, and configuration files. 35 | 36 | "Object" form shall mean any form resulting from mechanical 37 | transformation or translation of a Source form, including but 38 | not limited to compiled object code, generated documentation, 39 | and conversions to other media types. 40 | 41 | "Work" shall mean the work of authorship, whether in Source or 42 | Object form, made available under the License, as indicated by a 43 | copyright notice that is included in or attached to the work 44 | (an example is provided in the Appendix below). 45 | 46 | "Derivative Works" shall mean any work, whether in Source or Object 47 | form, that is based on (or derived from) the Work and for which the 48 | editorial revisions, annotations, elaborations, or other modifications 49 | represent, as a whole, an original work of authorship. For the purposes 50 | of this License, Derivative Works shall not include works that remain 51 | separable from, or merely link (or bind by name) to the interfaces of, 52 | the Work and Derivative Works thereof. 53 | 54 | "Contribution" shall mean any work of authorship, including 55 | the original version of the Work and any modifications or additions 56 | to that Work or Derivative Works thereof, that is intentionally 57 | submitted to Licensor for inclusion in the Work by the copyright owner 58 | or by an individual or Legal Entity authorized to submit on behalf of 59 | the copyright owner. For the purposes of this definition, "submitted" 60 | means any form of electronic, verbal, or written communication sent 61 | to the Licensor or its representatives, including but not limited to 62 | communication on electronic mailing lists, source code control systems, 63 | and issue tracking systems that are managed by, or on behalf of, the 64 | Licensor for the purpose of discussing and improving the Work, but 65 | excluding communication that is conspicuously marked or otherwise 66 | designated in writing by the copyright owner as "Not a Contribution." 67 | 68 | "Contributor" shall mean Licensor and any individual or Legal Entity 69 | on behalf of whom a Contribution has been received by Licensor and 70 | subsequently incorporated within the Work. 71 | 72 | 2. Grant of Copyright License. Subject to the terms and conditions of 73 | this License, each Contributor hereby grants to You a perpetual, 74 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 75 | copyright license to reproduce, prepare Derivative Works of, 76 | publicly display, publicly perform, sublicense, and distribute the 77 | Work and such Derivative Works in Source or Object form. 78 | 79 | 3. Grant of Patent License. Subject to the terms and conditions of 80 | this License, each Contributor hereby grants to You a perpetual, 81 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 82 | (except as stated in this section) patent license to make, have made, 83 | use, offer to sell, sell, import, and otherwise transfer the Work, 84 | where such license applies only to those patent claims licensable 85 | by such Contributor that are necessarily infringed by their 86 | Contribution(s) alone or by combination of their Contribution(s) 87 | with the Work to which such Contribution(s) was submitted. If You 88 | institute patent litigation against any entity (including a 89 | cross-claim or counterclaim in a lawsuit) alleging that the Work 90 | or a Contribution incorporated within the Work constitutes direct 91 | or contributory patent infringement, then any patent licenses 92 | granted to You under this License for that Work shall terminate 93 | as of the date such litigation is filed. 94 | 95 | 4. Redistribution. You may reproduce and distribute copies of the 96 | Work or Derivative Works thereof in any medium, with or without 97 | modifications, and in Source or Object form, provided that You 98 | meet the following conditions: 99 | 100 | (a) You must give any other recipients of the Work or 101 | Derivative Works a copy of this License; and 102 | 103 | (b) You must cause any modified files to carry prominent notices 104 | stating that You changed the files; and 105 | 106 | (c) You must retain, in the Source form of any Derivative Works 107 | that You distribute, all copyright, patent, trademark, and 108 | attribution notices from the Source form of the Work, 109 | excluding those notices that do not pertain to any part of 110 | the Derivative Works; and 111 | 112 | (d) If the Work includes a "NOTICE" text file as part of its 113 | distribution, then any Derivative Works that You distribute must 114 | include a readable copy of the attribution notices contained 115 | within such NOTICE file, excluding those notices that do not 116 | pertain to any part of the Derivative Works, in at least one 117 | of the following places: within a NOTICE text file distributed 118 | as part of the Derivative Works; within the Source form or 119 | documentation, if provided along with the Derivative Works; or, 120 | within a display generated by the Derivative Works, if and 121 | wherever such third-party notices normally appear. The contents 122 | of the NOTICE file are for informational purposes only and 123 | do not modify the License. You may add Your own attribution 124 | notices within Derivative Works that You distribute, alongside 125 | or as an addendum to the NOTICE text from the Work, provided 126 | that such additional attribution notices cannot be construed 127 | as modifying the License. 128 | 129 | You may add Your own copyright statement to Your modifications and 130 | may provide additional or different license terms and conditions 131 | for use, reproduction, or distribution of Your modifications, or 132 | for any such Derivative Works as a whole, provided Your use, 133 | reproduction, and distribution of the Work otherwise complies with 134 | the conditions stated in this License. 135 | 136 | 5. Submission of Contributions. Unless You explicitly state otherwise, 137 | any Contribution intentionally submitted for inclusion in the Work 138 | by You to the Licensor shall be under the terms and conditions of 139 | this License, without any additional terms or conditions. 140 | Notwithstanding the above, nothing herein shall supersede or modify 141 | the terms of any separate license agreement you may have executed 142 | with Licensor regarding such Contributions. 143 | 144 | 6. Trademarks. This License does not grant permission to use the trade 145 | names, trademarks, service marks, or product names of the Licensor, 146 | except as required for reasonable and customary use in describing the 147 | origin of the Work and reproducing the content of the NOTICE file. 148 | 149 | 7. Disclaimer of Warranty. Unless required by applicable law or 150 | agreed to in writing, Licensor provides the Work (and each 151 | Contributor provides its Contributions) on an "AS IS" BASIS, 152 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 153 | implied, including, without limitation, any warranties or conditions 154 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 155 | PARTICULAR PURPOSE. You are solely responsible for determining the 156 | appropriateness of using or redistributing the Work and assume any 157 | risks associated with Your exercise of permissions under this License. 158 | 159 | 8. Limitation of Liability. In no event and under no legal theory, 160 | whether in tort (including negligence), contract, or otherwise, 161 | unless required by applicable law (such as deliberate and grossly 162 | negligent acts) or agreed to in writing, shall any Contributor be 163 | liable to You for damages, including any direct, indirect, special, 164 | incidental, or consequential damages of any character arising as a 165 | result of this License or out of the use or inability to use the 166 | Work (including but not limited to damages for loss of goodwill, 167 | work stoppage, computer failure or malfunction, or any and all 168 | other commercial damages or losses), even if such Contributor 169 | has been advised of the possibility of such damages. 170 | 171 | 9. Accepting Warranty or Additional Liability. While redistributing 172 | the Work or Derivative Works thereof, You may choose to offer, 173 | and charge a fee for, acceptance of support, warranty, indemnity, 174 | or other liability obligations and/or rights consistent with this 175 | License. However, in accepting such obligations, You may act only 176 | on Your own behalf and on Your sole responsibility, not on behalf 177 | of any other Contributor, and only if You agree to indemnify, 178 | defend, and hold each Contributor harmless for any liability 179 | incurred by, or claims asserted against, such Contributor by reason 180 | of your accepting any such warranty or additional liability. 181 | 182 | END OF TERMS AND CONDITIONS 183 | 184 | -------------------------------------------------------------------------------- /src/plumtree_metadata_exchange_fsm.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% Copyright (c) 2013 Basho Technologies, Inc. All Rights Reserved. 4 | %% 5 | %% This file is provided to you under the Apache License, 6 | %% Version 2.0 (the "License"); you may not use this file 7 | %% except in compliance with the License. You may obtain 8 | %% a copy of the License at 9 | %% 10 | %% http://www.apache.org/licenses/LICENSE-2.0 11 | %% 12 | %% Unless required by applicable law or agreed to in writing, 13 | %% software distributed under the License is distributed on an 14 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | %% KIND, either express or implied. See the License for the 16 | %% specific language governing permissions and limitations 17 | %% under the License. 18 | %% 19 | %% ------------------------------------------------------------------- 20 | -module(plumtree_metadata_exchange_fsm). 21 | 22 | -behaviour(gen_fsm). 23 | 24 | %% API 25 | -export([start/2]). 26 | 27 | %% gen_fsm callbacks 28 | -export([init/1, handle_event/3, handle_sync_event/4, 29 | handle_info/3, terminate/3, code_change/4]). 30 | 31 | %% gen_fsm states 32 | -export([prepare/2, 33 | prepare/3, 34 | update/2, 35 | update/3, 36 | exchange/2, 37 | exchange/3]). 38 | 39 | -define(SERVER, ?MODULE). 40 | 41 | -record(state, { 42 | %% node the exchange is taking place with 43 | peer :: node(), 44 | 45 | %% count of trees that have been buit 46 | built :: non_neg_integer(), 47 | 48 | %% length of time waited to aqcuire remote lock or 49 | %% update trees 50 | timeout :: pos_integer() 51 | }). 52 | 53 | -record(exchange, { 54 | %% number of local prefixes repaired 55 | local :: non_neg_integer(), 56 | 57 | %% number of remote prefixes repaired 58 | remote :: non_neg_integer(), 59 | 60 | %% number of keys (missing, local, different) repaired, 61 | %% excluding those in prefixes counted by local/remote 62 | keys :: non_neg_integer() 63 | }). 64 | 65 | %%%=================================================================== 66 | %%% API 67 | %%%=================================================================== 68 | 69 | %% @doc Start an exchange of Cluster Metadata hashtrees between this node 70 | %% and `Peer'. `Timeout' is the number of milliseconds the process will wait 71 | %% to aqcuire the remote lock or to upate both trees. 72 | -spec start(node(), pos_integer()) -> {ok, pid()} | ignore | {error, term()}. 73 | start(Peer, Timeout) -> 74 | gen_fsm:start(?MODULE, [Peer, Timeout], []). 75 | 76 | %%%=================================================================== 77 | %%% gen_fsm callbacks 78 | %%%=================================================================== 79 | 80 | init([Peer, Timeout]) -> 81 | gen_fsm:send_event(self(), start), 82 | {ok, prepare, #state{peer=Peer,built=0,timeout=Timeout}}. 83 | 84 | handle_event(_Event, StateName, State) -> 85 | {next_state, StateName, State}. 86 | 87 | handle_sync_event(_Event, _From, StateName, State) -> 88 | Reply = ok, 89 | {reply, Reply, StateName, State}. 90 | 91 | handle_info(_Info, StateName, State) -> 92 | {next_state, StateName, State}. 93 | 94 | terminate(_Reason, _StateName, _State) -> 95 | ok. 96 | 97 | code_change(_OldVsn, StateName, State, _Extra) -> 98 | {ok, StateName, State}. 99 | 100 | %%%=================================================================== 101 | %%% gen_fsm states 102 | %%%=================================================================== 103 | prepare(start, State) -> 104 | %% get local lock 105 | case plumtree_metadata_hashtree:lock() of 106 | ok -> 107 | %% get remote lock 108 | remote_lock_request(State#state.peer), 109 | {next_state, prepare, State, State#state.timeout}; 110 | _Error -> 111 | {stop, normal, State} 112 | end; 113 | prepare(timeout, State=#state{peer=Peer}) -> 114 | %% getting remote lock timed out 115 | lager:error("metadata exchange with ~p timed out aquiring locks", [Peer]), 116 | {stop, normal, State}; 117 | prepare({remote_lock, ok}, State) -> 118 | %% getting remote lock succeeded 119 | update(start, State); 120 | prepare({remote_lock, _Error}, State) -> 121 | %% failed to get remote lock 122 | {stop, normal, State}. 123 | 124 | update(start, State) -> 125 | update_request(node()), 126 | update_request(State#state.peer), 127 | {next_state, update, State, State#state.timeout}; 128 | update(timeout, State=#state{peer=Peer}) -> 129 | lager:error("metadata exchange with ~p timed out updating trees", [Peer]), 130 | {stop, normal, State}; 131 | update(tree_updated, State) -> 132 | Built = State#state.built + 1, 133 | case Built of 134 | 2 -> 135 | {next_state, exchange, State, 0}; 136 | _ -> 137 | {next_state, update, State#state{built=Built}} 138 | end; 139 | update({update_error, _Error}, State) -> 140 | {stop, normal, State}. 141 | 142 | exchange(timeout, State=#state{peer=Peer}) -> 143 | RemoteFun = fun(Prefixes, {get_bucket, {Level, Bucket}}) -> 144 | plumtree_metadata_hashtree:get_bucket(Peer, Prefixes, Level, Bucket); 145 | (Prefixes, {key_hashes, Segment}) -> 146 | plumtree_metadata_hashtree:key_hashes(Peer, Prefixes, Segment) 147 | end, 148 | HandlerFun = fun(Diff, Acc) -> 149 | repair(Peer, Diff), 150 | track_repair(Diff, Acc) 151 | end, 152 | Res = plumtree_metadata_hashtree:compare(RemoteFun, HandlerFun, 153 | #exchange{local=0,remote=0,keys=0}), 154 | #exchange{local=LocalPrefixes, 155 | remote=RemotePrefixes, 156 | keys=Keys} = Res, 157 | Total = LocalPrefixes + RemotePrefixes + Keys, 158 | case Total > 0 of 159 | true -> 160 | lager:info("completed metadata exchange with ~p. repaired ~p missing local prefixes, " 161 | "~p missing remote prefixes, and ~p keys", [Peer, LocalPrefixes, RemotePrefixes, Keys]); 162 | false -> 163 | lager:debug("completed metadata exchange with ~p. nothing repaired", [Peer]) 164 | end, 165 | {stop, normal, State}. 166 | 167 | prepare(_Event, _From, State) -> 168 | {reply, ok, prepare, State}. 169 | 170 | update(_Event, _From, State) -> 171 | {reply, ok, update, State}. 172 | 173 | exchange(_Event, _From, State) -> 174 | {reply, ok, exchange, State}. 175 | 176 | %%%=================================================================== 177 | %%% Internal functions 178 | %%%=================================================================== 179 | 180 | %% @private 181 | repair(Peer, {missing_prefix, Type, Prefix}) -> 182 | repair_prefix(Peer, Type, Prefix); 183 | repair(Peer, {key_diffs, Prefix, Diffs}) -> 184 | _ = [repair_keys(Peer, Prefix, Diff) || Diff <- Diffs], 185 | ok. 186 | 187 | %% @private 188 | repair_prefix(Peer, Type, [Prefix]) -> 189 | ItType = repair_iterator_type(Type), 190 | repair_sub_prefixes(Type, Peer, Prefix, repair_iterator(ItType, Peer, Prefix)); 191 | repair_prefix(Peer, Type, [Prefix, SubPrefix]) -> 192 | FullPrefix = {Prefix, SubPrefix}, 193 | ItType = repair_iterator_type(Type), 194 | repair_full_prefix(Type, Peer, FullPrefix, repair_iterator(ItType, Peer, FullPrefix)). 195 | 196 | %% @private 197 | repair_sub_prefixes(Type, Peer, Prefix, It) -> 198 | case plumtree_metadata_manager:iterator_done(It) of 199 | true -> 200 | plumtree_metadata_manager:iterator_close(It); 201 | false -> 202 | SubPrefix = plumtree_metadata_manager:iterator_value(It), 203 | FullPrefix = {Prefix, SubPrefix}, 204 | 205 | ItType = repair_iterator_type(Type), 206 | ObjIt = repair_iterator(ItType, Peer, FullPrefix), 207 | repair_full_prefix(Type, Peer, FullPrefix, ObjIt), 208 | repair_sub_prefixes(Type, Peer, Prefix, 209 | plumtree_metadata_manager:iterate(It)) 210 | end. 211 | 212 | %% @private 213 | repair_full_prefix(Type, Peer, FullPrefix, ObjIt) -> 214 | case plumtree_metadata_manager:iterator_done(ObjIt) of 215 | true -> 216 | plumtree_metadata_manager:iterator_close(ObjIt); 217 | false -> 218 | {Key, Obj} = plumtree_metadata_manager:iterator_value(ObjIt), 219 | repair_other(Type, Peer, {FullPrefix, Key}, Obj), 220 | repair_full_prefix(Type, Peer, FullPrefix, 221 | plumtree_metadata_manager:iterate(ObjIt)) 222 | end. 223 | 224 | %% @private 225 | repair_other(local, _Peer, PKey, Obj) -> 226 | %% local missing data, merge remote data locally 227 | merge(undefined, PKey, Obj); 228 | repair_other(remote, Peer, PKey, Obj) -> 229 | %% remote missing data, merge local data into remote node 230 | merge(Peer, PKey, Obj). 231 | 232 | %% @private 233 | repair_keys(Peer, PrefixList, {_Type, KeyBin}) -> 234 | Key = binary_to_term(KeyBin), 235 | Prefix = list_to_tuple(PrefixList), 236 | PKey = {Prefix, Key}, 237 | LocalObj = plumtree_metadata_manager:get(PKey), 238 | RemoteObj = plumtree_metadata_manager:get(Peer, PKey), 239 | merge(undefined, PKey, RemoteObj), 240 | merge(Peer, PKey, LocalObj), 241 | ok. 242 | 243 | %% @private 244 | %% context is ignored since its in object, so pass undefined 245 | merge(undefined, PKey, RemoteObj) -> 246 | plumtree_metadata_manager:merge({PKey, undefined}, RemoteObj); 247 | merge(Peer, PKey, LocalObj) -> 248 | plumtree_metadata_manager:merge(Peer, {PKey, undefined}, LocalObj). 249 | 250 | 251 | %% @private 252 | repair_iterator(local, _, Prefix) when is_atom(Prefix) orelse is_binary(Prefix) -> 253 | plumtree_metadata_manager:iterator(Prefix); 254 | repair_iterator(local, _, Prefix) when is_tuple(Prefix) -> 255 | plumtree_metadata_manager:iterator(Prefix, undefined); 256 | repair_iterator(remote, Peer, PrefixOrFull) -> 257 | plumtree_metadata_manager:remote_iterator(Peer, PrefixOrFull). 258 | 259 | %% @private 260 | repair_iterator_type(local) -> 261 | %% local node missing prefix, need to iterate remote 262 | remote; 263 | repair_iterator_type(remote) -> 264 | %% remote node missing prefix, need to iterate local 265 | local. 266 | 267 | %% @private 268 | track_repair({missing_prefix, local, _}, Acc=#exchange{local=Local}) -> 269 | Acc#exchange{local=Local+1}; 270 | track_repair({missing_prefix, remote, _}, Acc=#exchange{remote=Remote}) -> 271 | Acc#exchange{remote=Remote+1}; 272 | track_repair({key_diffs, _, Diffs}, Acc=#exchange{keys=Keys}) -> 273 | Acc#exchange{keys=Keys+length(Diffs)}. 274 | %% @private 275 | remote_lock_request(Peer) -> 276 | Self = self(), 277 | as_event(fun() -> 278 | Res = plumtree_metadata_hashtree:lock(Peer, Self), 279 | {remote_lock, Res} 280 | end). 281 | 282 | %% @private 283 | update_request(Node) -> 284 | as_event(fun() -> 285 | %% acquired lock so we know there is no other update 286 | %% and tree is built 287 | case plumtree_metadata_hashtree:update(Node) of 288 | ok -> tree_updated; 289 | Error -> {update_error, Error} 290 | end 291 | end). 292 | 293 | %% @private 294 | %% "borrowed" from riak_kv_exchange_fsm 295 | as_event(F) -> 296 | Self = self(), 297 | spawn_link(fun() -> 298 | Result = F(), 299 | gen_fsm:send_event(Self, Result) 300 | end), 301 | ok. 302 | -------------------------------------------------------------------------------- /src/plumtree_metadata_hashtree.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% Copyright (c) 2013 Basho Technologies, Inc. All Rights Reserved. 4 | %% 5 | %% This file is provided to you under the Apache License, 6 | %% Version 2.0 (the "License"); you may not use this file 7 | %% except in compliance with the License. You may obtain 8 | %% a copy of the License at 9 | %% 10 | %% http://www.apache.org/licenses/LICENSE-2.0 11 | %% 12 | %% Unless required by applicable law or agreed to in writing, 13 | %% software distributed under the License is distributed on an 14 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | %% KIND, either express or implied. See the License for the 16 | %% specific language governing permissions and limitations 17 | %% under the License. 18 | %% 19 | %% ------------------------------------------------------------------- 20 | -module(plumtree_metadata_hashtree). 21 | 22 | -behaviour(gen_server). 23 | 24 | %% API 25 | -export([start_link/0, 26 | start_link/1, 27 | insert/2, 28 | insert/3, 29 | prefix_hash/1, 30 | get_bucket/4, 31 | key_hashes/3, 32 | lock/0, 33 | lock/1, 34 | lock/2, 35 | update/0, 36 | update/1, 37 | compare/3]). 38 | %% gen_server callbacks 39 | -export([init/1, handle_call/3, handle_cast/2, handle_info/2, 40 | terminate/2, code_change/3]). 41 | 42 | -include("plumtree_metadata.hrl"). 43 | 44 | -define(SERVER, ?MODULE). 45 | 46 | -record(state, { 47 | %% the tree managed by this process 48 | tree :: hashtree_tree:tree(), 49 | 50 | %% whether or not the tree has been built or a monitor ref 51 | %% if the tree is being built 52 | built :: boolean() | reference(), 53 | 54 | %% a monitor reference for a process that currently holds a 55 | %% lock on the tree. undefined otherwise 56 | lock :: {internal | external, reference()} | undefined 57 | }). 58 | 59 | %%%=================================================================== 60 | %%% API 61 | %%%=================================================================== 62 | 63 | %% @doc Starts the process using {@link start_link/1}, passing in the 64 | %% directory where other cluster metadata is stored in `plumtree_data_dir' 65 | %% as the data root. 66 | -spec start_link() -> {ok, pid()} | ignore | {error, term()}. 67 | start_link() -> 68 | PRoot = app_helper:get_env(plumtree, plumtree_data_dir), 69 | DataRoot = filename:join(PRoot, "trees"), 70 | start_link(DataRoot). 71 | 72 | %% @doc Starts a registered process that manages a {@link 73 | %% hashtree_tree} for Cluster Metadata. Data for the tree is stored, 74 | %% for the lifetime of the process (assuming it shutdowns gracefully), 75 | %% in the directory `DataRoot'. 76 | -spec start_link(file:filename()) -> {ok, pid()} | ignore | {error, term()}. 77 | start_link(DataRoot) -> 78 | gen_server:start_link({local, ?SERVER}, ?MODULE, [DataRoot], []). 79 | 80 | %% @doc Same as insert(PKey, Hash, false). 81 | -spec insert(metadata_pkey(), binary()) -> ok. 82 | insert(PKey, Hash) -> 83 | insert(PKey, Hash, false). 84 | 85 | %% @doc Insert a hash for a full-prefix and key into the tree 86 | %% managed by the process. If `IfMissing' is `true' the hash is only 87 | %% inserted into the tree if the key is not already present. 88 | -spec insert(metadata_pkey(), binary(), boolean()) -> ok. 89 | insert(PKey, Hash, IfMissing) -> 90 | gen_server:call(?SERVER, {insert, PKey, Hash, IfMissing}, infinity). 91 | 92 | %% @doc Return the hash for the given prefix or full-prefix 93 | -spec prefix_hash(metadata_prefix() | binary() | atom()) -> undefined | binary(). 94 | prefix_hash(Prefix) -> 95 | gen_server:call(?SERVER, {prefix_hash, Prefix}, infinity). 96 | 97 | %% @doc Return the bucket for a node in the tree managed by this 98 | %% process running on `Node'. 99 | -spec get_bucket(node(), hashtree_tree:tree_node(), 100 | non_neg_integer(), non_neg_integer()) -> orddict:orddict(). 101 | get_bucket(Node, Prefixes, Level, Bucket) -> 102 | gen_server:call({?SERVER, Node}, {get_bucket, Prefixes, Level, Bucket}, infinity). 103 | 104 | %% @doc Return the key hashes for a node in the tree managed by this 105 | %% process running on `Node'. 106 | -spec key_hashes(node(), hashtree_tree:tree_node(), non_neg_integer()) -> orddict:orddict(). 107 | key_hashes(Node, Prefixes, Segment) -> 108 | gen_server:call({?SERVER, Node}, {key_hashes, Prefixes, Segment}, infinity). 109 | 110 | %% @doc Locks the tree on this node for updating on behalf of the 111 | %% calling process. 112 | %% @see lock/2 113 | -spec lock() -> ok | not_built | locked. 114 | lock() -> 115 | lock(node()). 116 | 117 | %% @doc Locks the tree on `Node' for updating on behalf of the calling 118 | %% process. 119 | %% @see lock/2 120 | -spec lock(node()) -> ok | not_built | locked. 121 | lock(Node) -> 122 | lock(Node, self()). 123 | 124 | %% @doc Lock the tree for updating. This function must be called 125 | %% before updating the tree with {@link update/0} or {@link 126 | %% update/1}. If the tree is not built or already locked then the call 127 | %% will fail and the appropriate atom is returned. Otherwise, 128 | %% aqcuiring the lock succeeds and `ok' is returned. 129 | -spec lock(node(), pid()) -> ok | not_built | locked. 130 | lock(Node, Pid) -> 131 | gen_server:call({?SERVER, Node}, {lock, Pid}, infinity). 132 | 133 | %% @doc Updates the tree on this node. 134 | %% @see update/1 135 | -spec update() -> ok | not_locked | not_built | ongoing_update. 136 | update() -> 137 | update(node()). 138 | 139 | %% @doc Updates the tree on `Node'. The tree must be locked using one 140 | %% of the lock functions. If the tree is not locked or built the 141 | %% update will not be started and the appropriate atom is 142 | %% returned. Although this function should not be called without a 143 | %% lock, if it is and the tree is being updated by the background tick 144 | %% then `ongoing_update' is returned. If the tree is built and a lock 145 | %% has been acquired then the update is started and `ok' is 146 | %% returned. The update is performed asynchronously and does not block 147 | %% the process that manages the tree (e.g. future inserts). 148 | -spec update(node()) -> ok | not_locked | not_built | ongoing_update. 149 | update(Node) -> 150 | gen_server:call({?SERVER, Node}, update, infinity). 151 | 152 | %% @doc Compare the local tree managed by this process with the remote 153 | %% tree also managed by a metadata hashtree process. `RemoteFun' is 154 | %% used to access the buckets and segments of nodes in the remote tree 155 | %% and should usually call {@link get_bucket/4} and {@link 156 | %% key_hashes/3}. `HandlerFun' is used to process the differences 157 | %% found between the two trees. `HandlerAcc' is passed to the first 158 | %% invocation of `HandlerFun'. Subsequent calls are passed the return 159 | %% value from the previous call. This function returns the return 160 | %% value from the last call to `HandlerFun'. {@link hashtree_tree} for 161 | %% more details on `RemoteFun', `HandlerFun' and `HandlerAcc'. 162 | -spec compare(hashtree_tree:remote_fun(), hashtree_tree:handler_fun(X), X) -> X. 163 | compare(RemoteFun, HandlerFun, HandlerAcc) -> 164 | gen_server:call(?SERVER, {compare, RemoteFun, HandlerFun, HandlerAcc}, infinity). 165 | 166 | %%%=================================================================== 167 | %%% gen_server callbacks 168 | %%%=================================================================== 169 | 170 | init([DataRoot]) -> 171 | schedule_tick(), 172 | Tree = hashtree_tree:new(plumtree, [{data_dir, DataRoot}, {num_levels, 2}]), 173 | State = #state{tree=Tree, 174 | built=false, 175 | lock=undefined}, 176 | State1 = build_async(State), 177 | {ok, State1}. 178 | 179 | handle_call({compare, RemoteFun, HandlerFun, HandlerAcc}, From, State) -> 180 | maybe_compare_async(From, RemoteFun, HandlerFun, HandlerAcc, State), 181 | {noreply, State}; 182 | handle_call(update, From, State) -> 183 | State1 = maybe_external_update(From, State), 184 | {noreply, State1}; 185 | handle_call({lock, Pid}, _From, State) -> 186 | {Reply, State1} = maybe_external_lock(Pid, State), 187 | {reply, Reply, State1}; 188 | handle_call({get_bucket, Prefixes, Level, Bucket}, _From, State) -> 189 | Res = hashtree_tree:get_bucket(Prefixes, Level, Bucket, State#state.tree), 190 | {reply, Res, State}; 191 | handle_call({key_hashes, Prefixes, Segment}, _From, State) -> 192 | [{_, Res}] = hashtree_tree:key_hashes(Prefixes, Segment, State#state.tree), 193 | {reply, Res, State}; 194 | handle_call({prefix_hash, Prefix}, _From, State=#state{tree=Tree}) -> 195 | PrefixList = prefix_to_prefix_list(Prefix), 196 | PrefixHash = hashtree_tree:prefix_hash(PrefixList, Tree), 197 | {reply, PrefixHash, State}; 198 | handle_call({insert, PKey, Hash, IfMissing}, _From, State=#state{tree=Tree}) -> 199 | {Prefixes, Key} = prepare_pkey(PKey), 200 | Tree1 = hashtree_tree:insert(Prefixes, Key, Hash, [{if_missing, IfMissing}], Tree), 201 | {reply, ok, State#state{tree=Tree1}}. 202 | 203 | handle_cast(_Msg, State) -> 204 | {noreply, State}. 205 | 206 | handle_info({'DOWN', BuildRef, process, _Pid, normal}, State=#state{built=BuildRef}) -> 207 | State1 = build_done(State), 208 | {noreply, State1}; 209 | handle_info({'DOWN', BuildRef, process, _Pid, Reason}, State=#state{built=BuildRef}) -> 210 | lager:error("building tree failed: ~p", [Reason]), 211 | State1 = build_error(State), 212 | {noreply, State1}; 213 | handle_info({'DOWN', LockRef, process, _Pid, _Reason}, State=#state{lock={_, LockRef}}) -> 214 | State1 = release_lock(State), 215 | {noreply, State1}; 216 | handle_info(tick, State) -> 217 | schedule_tick(), 218 | State1 = maybe_build_async(State), 219 | State2 = maybe_update_async(State1), 220 | {noreply, State2}. 221 | 222 | terminate(_Reason, State) -> 223 | hashtree_tree:destroy(State#state.tree), 224 | ok. 225 | 226 | code_change(_OldVsn, State, _Extra) -> 227 | {ok, State}. 228 | 229 | %%%=================================================================== 230 | %%% Internal functions 231 | %%%=================================================================== 232 | 233 | %% @private 234 | maybe_compare_async(From, RemoteFun, HandlerFun, HandlerAcc, 235 | State=#state{built=true,lock={external,_}}) -> 236 | compare_async(From, RemoteFun, HandlerFun, HandlerAcc, State); 237 | maybe_compare_async(From, _, _, HandlerAcc, _State) -> 238 | gen_server:reply(From, HandlerAcc). 239 | 240 | %% @private 241 | compare_async(From, RemoteFun, HandlerFun, HandlerAcc, #state{tree=Tree}) -> 242 | spawn_link(fun() -> 243 | Res = hashtree_tree:compare(Tree, RemoteFun, 244 | HandlerFun, HandlerAcc), 245 | gen_server:reply(From, Res) 246 | end). 247 | 248 | %% @private 249 | maybe_external_update(From, State=#state{built=true,lock=undefined}) -> 250 | gen_server:reply(From, not_locked), 251 | State; 252 | maybe_external_update(From, State=#state{built=true,lock={internal,_}}) -> 253 | gen_server:reply(From, ongoing_update), 254 | State; 255 | maybe_external_update(From, State=#state{built=true,lock={external,_}}) -> 256 | update_async(From, false, State); 257 | maybe_external_update(From, State) -> 258 | gen_server:reply(From, not_built), 259 | State. 260 | 261 | %% @private 262 | maybe_update_async(State=#state{built=true,lock=undefined}) -> 263 | update_async(State); 264 | maybe_update_async(State) -> 265 | State. 266 | 267 | %% @private 268 | update_async(State) -> 269 | update_async(undefined, true, State). 270 | 271 | %% @private 272 | update_async(From, Lock, State=#state{tree=Tree}) -> 273 | Tree2 = hashtree_tree:update_snapshot(Tree), 274 | Pid = spawn_link(fun() -> 275 | hashtree_tree:update_perform(Tree2), 276 | case From of 277 | undefined -> ok; 278 | _ -> gen_server:reply(From, ok) 279 | end 280 | end), 281 | State1 = case Lock of 282 | true -> lock(Pid, internal, State); 283 | false -> State 284 | end, 285 | State1#state{tree=Tree2}. 286 | 287 | %% @private 288 | maybe_build_async(State=#state{built=false}) -> 289 | build_async(State); 290 | maybe_build_async(State) -> 291 | State. 292 | 293 | %% @private 294 | build_async(State) -> 295 | {_Pid, Ref} = spawn_monitor(fun build/0), 296 | State#state{built=Ref}. 297 | 298 | %% @private 299 | build() -> 300 | PrefixIt = plumtree_metadata_manager:iterator(), 301 | build(PrefixIt). 302 | 303 | %% @private 304 | build(PrefixIt) -> 305 | case plumtree_metadata_manager:iterator_done(PrefixIt) of 306 | true -> 307 | plumtree_metadata_manager:iterator_close(PrefixIt); 308 | false -> 309 | Prefix = plumtree_metadata_manager:iterator_value(PrefixIt), 310 | ObjIt = plumtree_metadata_manager:iterator(Prefix, undefined), 311 | build(PrefixIt, ObjIt) 312 | end. 313 | 314 | %% @private 315 | build(PrefixIt, ObjIt) -> 316 | case plumtree_metadata_manager:iterator_done(ObjIt) of 317 | true -> 318 | plumtree_metadata_manager:iterator_close(ObjIt), 319 | build(plumtree_metadata_manager:iterate(PrefixIt)); 320 | false -> 321 | FullPrefix = plumtree_metadata_manager:iterator_prefix(ObjIt), 322 | {Key, Obj} = plumtree_metadata_manager:iterator_value(ObjIt), 323 | Hash = plumtree_metadata_object:hash(Obj), 324 | %% insert only if missing to not clash w/ newer writes during build 325 | ?MODULE:insert({FullPrefix, Key}, Hash, true), 326 | build(PrefixIt, plumtree_metadata_manager:iterate(ObjIt)) 327 | end. 328 | 329 | %% @private 330 | build_done(State) -> 331 | State#state{built=true}. 332 | 333 | %% @private 334 | build_error(State) -> 335 | State#state{built=false}. 336 | 337 | %% @private 338 | maybe_external_lock(Pid, State=#state{lock=undefined,built=true}) -> 339 | {ok, lock(Pid, external, State)}; 340 | maybe_external_lock(_Pid, State=#state{built=true}) -> 341 | {locked, State}; 342 | maybe_external_lock(_Pid, State) -> 343 | {not_built, State}. 344 | 345 | %% @private 346 | lock(Pid, Type, State) -> 347 | LockRef = monitor(process, Pid), 348 | State#state{lock={Type, LockRef}}. 349 | 350 | %% @private 351 | release_lock(State) -> 352 | State#state{lock=undefined}. 353 | 354 | %% @private 355 | prefix_to_prefix_list(Prefix) when is_binary(Prefix) or is_atom(Prefix) -> 356 | [Prefix]; 357 | prefix_to_prefix_list({Prefix, SubPrefix}) -> 358 | [Prefix,SubPrefix]. 359 | 360 | %% @private 361 | prepare_pkey({FullPrefix, Key}) -> 362 | {prefix_to_prefix_list(FullPrefix), term_to_binary(Key)}. 363 | 364 | %% @private 365 | schedule_tick() -> 366 | TickMs = app_helper:get_env(plumtree, metadata_hashtree_timer, 10000), 367 | erlang:send_after(TickMs, ?MODULE, tick). 368 | -------------------------------------------------------------------------------- /src/plumtree_metadata.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% Copyright (c) 2013 Basho Technologies, Inc. All Rights Reserved. 4 | %% 5 | %% This file is provided to you under the Apache License, 6 | %% Version 2.0 (the "License"); you may not use this file 7 | %% except in compliance with the License. You may obtain 8 | %% a copy of the License at 9 | %% 10 | %% http://www.apache.org/licenses/LICENSE-2.0 11 | %% 12 | %% Unless required by applicable law or agreed to in writing, 13 | %% software distributed under the License is distributed on an 14 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | %% KIND, either express or implied. See the License for the 16 | %% specific language governing permissions and limitations 17 | %% under the License. 18 | %% 19 | %% ------------------------------------------------------------------- 20 | -module(plumtree_metadata). 21 | 22 | -export([get/2, 23 | get/3, 24 | fold/3, 25 | fold/4, 26 | to_list/1, 27 | to_list/2, 28 | iterator/1, 29 | iterator/2, 30 | itr_next/1, 31 | itr_close/1, 32 | itr_done/1, 33 | itr_key_values/1, 34 | itr_key/1, 35 | itr_values/1, 36 | itr_value/1, 37 | itr_default/1, 38 | prefix_hash/1, 39 | put/3, 40 | put/4, 41 | delete/2, 42 | delete/3]). 43 | 44 | -include("plumtree_metadata.hrl"). 45 | 46 | -export_type([iterator/0]). 47 | 48 | %% Get Option Types 49 | -type get_opt_default_val() :: {default, metadata_value()}. 50 | -type get_opt_resolver() :: {resolver, metadata_resolver()}. 51 | -type get_opt_allow_put() :: {allow_put, boolean()}. 52 | -type get_opt() :: get_opt_default_val() | get_opt_resolver() | get_opt_allow_put(). 53 | -type get_opts() :: [get_opt()]. 54 | 55 | %% Iterator Types 56 | -type it_opt_resolver() :: {resolver, metadata_resolver() | lww}. 57 | -type it_opt_default_fun() :: fun((metadata_key()) -> metadata_value()). 58 | -type it_opt_default() :: {default, metadata_value() | it_opt_default_fun()}. 59 | -type it_opt_keymatch() :: {match, term()}. 60 | -type it_opt() :: it_opt_resolver() | it_opt_default() | it_opt_keymatch(). 61 | -type it_opts() :: [it_opt()]. 62 | -type fold_opts() :: it_opts(). 63 | -type iterator() :: {plumtree_metadata_manager:metadata_iterator(), it_opts()}. 64 | 65 | %% Put Option Types 66 | -type put_opts() :: []. 67 | 68 | %% Delete Option types 69 | -type delete_opts() :: []. 70 | 71 | -define(TOMBSTONE, '$deleted'). 72 | 73 | %% @doc same as get(FullPrefix, Key, []) 74 | -spec get(metadata_prefix(), metadata_key()) -> metadata_value() | undefined. 75 | get(FullPrefix, Key) -> 76 | get(FullPrefix, Key, []). 77 | 78 | %% @doc Retrieves the local value stored at the given prefix and key. 79 | %% 80 | %% get/3 can take the following options: 81 | %% * default: value to return if no value is found, `undefined' if not given. 82 | %% * resolver: A function that resolves conflicts if they are encountered. If not given 83 | %% last-write-wins is used to resolve the conflicts 84 | %% * allow_put: whether or not to write and broadcast a resolved value. defaults to `true'. 85 | %% 86 | %% NOTE: an update will be broadcast if conflicts are resolved and 87 | %% `allow_put' is `true'. any further conflicts generated by 88 | %% concurrenct writes during resolution are not resolved 89 | -spec get(metadata_prefix(), metadata_key(), get_opts()) -> metadata_value(). 90 | get({Prefix, SubPrefix}=FullPrefix, Key, Opts) 91 | when (is_binary(Prefix) orelse is_atom(Prefix)) andalso 92 | (is_binary(SubPrefix) orelse is_atom(SubPrefix)) -> 93 | PKey = prefixed_key(FullPrefix, Key), 94 | Default = get_option(default, Opts, undefined), 95 | ResolveMethod = get_option(resolver, Opts, lww), 96 | AllowPut = get_option(allow_put, Opts, true), 97 | case plumtree_metadata_manager:get(PKey) of 98 | undefined -> Default; 99 | Existing -> 100 | maybe_tombstone(maybe_resolve(PKey, Existing, ResolveMethod, AllowPut), Default) 101 | end. 102 | 103 | %% @doc same as fold(Fun, Acc0, FullPrefix, []). 104 | -spec fold(fun(({metadata_key(), 105 | [metadata_value() | metadata_tombstone()] | 106 | metadata_value() | metadata_tombstone()}, any()) -> any()), 107 | any(), 108 | metadata_prefix()) -> any(). 109 | fold(Fun, Acc0, FullPrefix) -> 110 | fold(Fun, Acc0, FullPrefix, []). 111 | 112 | %% @doc Fold over all keys and values stored under a given prefix/subprefix. Available 113 | %% options are the same as those provided to iterator/2. 114 | -spec fold(fun(({metadata_key(), 115 | [metadata_value() | metadata_tombstone()] | 116 | metadata_value() | metadata_tombstone()}, any()) -> any()), 117 | any(), 118 | metadata_prefix(), 119 | fold_opts()) -> any(). 120 | fold(Fun, Acc0, FullPrefix, Opts) -> 121 | It = iterator(FullPrefix, Opts), 122 | fold_it(Fun, Acc0, It). 123 | 124 | fold_it(Fun, Acc, It) -> 125 | case itr_done(It) of 126 | true -> 127 | ok = itr_close(It), 128 | Acc; 129 | false -> 130 | Next = Fun(itr_key_values(It), Acc), 131 | fold_it(Fun, Next, itr_next(It)) 132 | end. 133 | 134 | %% @doc same as to_list(FullPrefix, []) 135 | -spec to_list(metadata_prefix()) -> [{metadata_key(), 136 | [metadata_value() | metadata_tombstone()] | 137 | metadata_value() | metadata_tombstone()}]. 138 | to_list(FullPrefix) -> 139 | to_list(FullPrefix, []). 140 | 141 | 142 | %% @doc Return a list of all keys and values stored under a given prefix/subprefix. Available 143 | %% options are the same as those provided to iterator/2. 144 | -spec to_list(metadata_prefix(), fold_opts()) -> [{metadata_key(), 145 | [metadata_value() | metadata_tombstone()] | 146 | metadata_value() | metadata_tombstone()}]. 147 | to_list(FullPrefix, Opts) -> 148 | fold(fun({Key, ValOrVals}, Acc) -> 149 | [{Key, ValOrVals} | Acc] 150 | end, [], FullPrefix, Opts). 151 | 152 | %% @doc same as iterator(FullPrefix, []). 153 | -spec iterator(metadata_prefix()) -> iterator(). 154 | iterator(FullPrefix) -> 155 | iterator(FullPrefix, []). 156 | 157 | %% @doc Return an iterator pointing to the first key stored under a prefix 158 | %% 159 | %% iterator/2 can take the following options: 160 | %% * resolver: either the atom `lww' or a function that resolves conflicts if they 161 | %% are encounted (see get/3 for more details). Conflict resolution 162 | %% is performed when values are retrieved (see itr_value/1 and itr_key_values/1). 163 | %% If no resolver is provided no resolution is performed. The default is to 164 | %% not provide a resolver. 165 | %% * allow_put: whether or not to write and broadcast a resolved value. defaults to `true'. 166 | %% * default: Used when the value an iterator points to is a tombstone. default is 167 | %% either an arity-1 function or a value. If a function, the key the iterator 168 | %% points to is passed as the argument and the result is returned in place 169 | %% of the tombstone. If default is a value, the value is returned in place of 170 | %% the tombstone. This applies when using functions such as itr_values/1 and 171 | %% itr_key_values/1. 172 | %% * match: A tuple containing erlang terms and '_'s. Match can be used to iterate 173 | %% over a subset of keys -- assuming the keys stored are tuples 174 | -spec iterator(metadata_prefix(), it_opts()) -> iterator(). 175 | iterator({Prefix, SubPrefix}=FullPrefix, Opts) 176 | when (is_binary(Prefix) orelse is_atom(Prefix)) andalso 177 | (is_binary(SubPrefix) orelse is_atom(SubPrefix)) -> 178 | KeyMatch = proplists:get_value(match, Opts), 179 | It = plumtree_metadata_manager:iterator(FullPrefix, KeyMatch), 180 | {It, Opts}. 181 | 182 | %% @doc Advances the iterator 183 | -spec itr_next(iterator()) -> iterator(). 184 | itr_next({It, Opts}) -> 185 | It1 = plumtree_metadata_manager:iterate(It), 186 | {It1, Opts}. 187 | 188 | %% @doc Closes the iterator 189 | -spec itr_close(iterator()) -> ok. 190 | itr_close({It, _Ots}) -> 191 | plumtree_metadata_manager:iterator_close(It). 192 | 193 | %% @doc Returns true if there is nothing more to iterate over 194 | -spec itr_done(iterator()) -> boolean(). 195 | itr_done({It, _Opts}) -> 196 | plumtree_metadata_manager:iterator_done(It). 197 | 198 | %% @doc Return the key and value(s) pointed at by the iterator. Before 199 | %% calling this function, check the iterator is not complete w/ itr_done/1. If a resolver 200 | %% was passed to iterator/0 when creating the given iterator, siblings will be resolved 201 | %% using the given function or last-write-wins (if `lww' is passed as the resolver). If 202 | %% no resolver was used then no conflict resolution will take place. If conflicts are 203 | %% resolved, the resolved value is written to local metadata and a broadcast is submitted 204 | %% to update other nodes in the cluster if `allow_put' is `true'. If `allow_put' is `false' 205 | %% the values are resolved but are not written or broadcast. A single value is returned as the second 206 | %% element of the tuple in the case values are resolved. If no resolution takes place then a list of 207 | %% values will be returned as the second element (even if there is only a single sibling). 208 | %% 209 | %% NOTE: if resolution may be performed this function must be called at most once 210 | %% before calling itr_next/1 on the iterator (at which point the function can be called 211 | %% once more). 212 | -spec itr_key_values(iterator()) -> {metadata_key(), 213 | [metadata_value() | metadata_tombstone()] | 214 | metadata_value() | 215 | metadata_tombstone()}. 216 | itr_key_values({It, Opts}) -> 217 | Default = itr_default({It, Opts}), 218 | {Key, Obj} = plumtree_metadata_manager:iterator_value(It), 219 | AllowPut = get_option(allow_put, Opts, true), 220 | case get_option(resolver, Opts, undefined) of 221 | undefined -> 222 | {Key, maybe_tombstones(plumtree_metadata_object:values(Obj), Default)}; 223 | Resolver -> 224 | Prefix = plumtree_metadata_manager:iterator_prefix(It), 225 | PKey = prefixed_key(Prefix, Key), 226 | Value = maybe_tombstone(maybe_resolve(PKey, Obj, Resolver, AllowPut), Default), 227 | {Key, Value} 228 | end. 229 | 230 | %% @doc Return the key pointed at by the iterator. Before 231 | %% calling this function, check the iterator is not complete w/ itr_done/1. 232 | %% No conflict resolution will be performed as a result of calling this function. 233 | -spec itr_key(iterator()) -> metadata_key(). 234 | itr_key({It, _Opts}) -> 235 | {Key, _} = plumtree_metadata_manager:iterator_value(It), 236 | Key. 237 | 238 | %% @doc Return all sibling values pointed at by the iterator. Before 239 | %% calling this function, check the iterator is not complete w/ itr_done/1. 240 | %% No conflict resolution will be performed as a result of calling this function. 241 | -spec itr_values(iterator()) -> [metadata_value() | metadata_tombstone()]. 242 | itr_values({It, Opts}) -> 243 | Default = itr_default({It, Opts}), 244 | {_, Obj} = plumtree_metadata_manager:iterator_value(It), 245 | maybe_tombstones(plumtree_metadata_object:values(Obj), Default). 246 | 247 | %% @doc Return a single value pointed at by the iterator. If there are conflicts and 248 | %% a resolver was specified in the options when creating this iterator, they will be 249 | %% resolved. Otherwise, and error is returned. If conflicts are resolved, the resolved 250 | %% value is written locally and a broadcast is performed to update other nodes 251 | %% in the cluster if `allow_put' is `true' (the default value). If `allow_put' is `false', 252 | %% values are resolved but not written or broadcast. 253 | %% 254 | %% NOTE: if resolution may be performed this function must be called at most once 255 | %% before calling itr_next/1 on the iterator (at which point the function can be called 256 | %% once more). 257 | -spec itr_value(iterator()) -> metadata_value() | metadata_tombstone() | {error, conflict}. 258 | itr_value({It, Opts}) -> 259 | Default = itr_default({It, Opts}), 260 | {Key, Obj} = plumtree_metadata_manager:iterator_value(It), 261 | AllowPut = get_option(allow_put, Opts, true), 262 | case get_option(resolver, Opts, undefined) of 263 | undefined -> 264 | case plumtree_metadata_object:value_count(Obj) of 265 | 1 -> 266 | maybe_tombstone(plumtree_metadata_object:value(Obj), Default); 267 | _ -> 268 | {error, conflict} 269 | end; 270 | Resolver -> 271 | Prefix = plumtree_metadata_manager:iterator_prefix(It), 272 | PKey = prefixed_key(Prefix, Key), 273 | maybe_tombstone(maybe_resolve(PKey, Obj, Resolver, AllowPut), Default) 274 | end. 275 | 276 | %% @doc Returns the value returned when an iterator points to a tombstone. If the default 277 | %% used when creating the given iterator is a function it will be applied to the current 278 | %% key the iterator points at. If no default was provided the tombstone value was returned. 279 | %% This function should only be called after checking itr_done/1. 280 | -spec itr_default(iterator()) -> metadata_tombstone() | metadata_value() | it_opt_default_fun(). 281 | itr_default({_, Opts}=It) -> 282 | case proplists:get_value(default, Opts, ?TOMBSTONE) of 283 | Fun when is_function(Fun) -> 284 | Fun(itr_key(It)); 285 | Val -> Val 286 | end. 287 | 288 | %% @doc Return the local hash associated with a full-prefix or prefix. The hash value is 289 | %% updated periodically and does not always reflect the most recent value. This function 290 | %% can be used to determine when keys stored under a full-prefix or prefix have changed. 291 | %% If the tree has not yet been updated or there are no keys stored the given 292 | %% (full-)prefix. `undefined' is returned. 293 | -spec prefix_hash(metadata_prefix() | binary() | atom()) -> binary() | undefined. 294 | prefix_hash(Prefix) when is_tuple(Prefix) or is_atom(Prefix) or is_binary(Prefix) -> 295 | plumtree_metadata_hashtree:prefix_hash(Prefix). 296 | 297 | %% @doc same as put(FullPrefix, Key, Value, []) 298 | -spec put(metadata_prefix(), metadata_key(), metadata_value() | metadata_modifier()) -> ok. 299 | put(FullPrefix, Key, ValueOrFun) -> 300 | put(FullPrefix, Key, ValueOrFun, []). 301 | 302 | %% @doc Stores or updates the value at the given prefix and key locally and then 303 | %% triggers a broadcast to notify other nodes in the cluster. Currently, there 304 | %% are no put options 305 | %% 306 | %% NOTE: because the third argument to this function can be a metadata_modifier(), 307 | %% used to resolve conflicts on write, metadata values cannot be functions. 308 | %% To store functions in metadata wrap them in another type like a tuple. 309 | -spec put(metadata_prefix(), 310 | metadata_key(), 311 | metadata_value() | metadata_modifier(), 312 | put_opts()) -> ok. 313 | put({Prefix, SubPrefix}=FullPrefix, Key, ValueOrFun, _Opts) 314 | when (is_binary(Prefix) orelse is_atom(Prefix)) andalso 315 | (is_binary(SubPrefix) orelse is_atom(SubPrefix)) -> 316 | PKey = prefixed_key(FullPrefix, Key), 317 | CurrentContext = current_context(PKey), 318 | Updated = plumtree_metadata_manager:put(PKey, CurrentContext, ValueOrFun), 319 | broadcast(PKey, Updated). 320 | 321 | %% @doc same as delete(FullPrefix, Key, []) 322 | -spec delete(metadata_prefix(), metadata_key()) -> ok. 323 | delete(FullPrefix, Key) -> 324 | delete(FullPrefix, Key, []). 325 | 326 | %% @doc Removes the value associated with the given prefix and key locally and then 327 | %% triggers a broradcast to notify other nodes in the cluster. Currently there are 328 | %% no delete options 329 | %% 330 | %% NOTE: currently deletion is logical and no GC is performed. 331 | -spec delete(metadata_prefix(), metadata_key(), delete_opts()) -> ok. 332 | delete(FullPrefix, Key, _Opts) -> 333 | put(FullPrefix, Key, ?TOMBSTONE, []). 334 | 335 | %%%=================================================================== 336 | %%% Internal functions 337 | %%%=================================================================== 338 | 339 | %% @private 340 | current_context(PKey) -> 341 | case plumtree_metadata_manager:get(PKey) of 342 | undefined -> plumtree_metadata_object:empty_context(); 343 | CurrentMeta -> plumtree_metadata_object:context(CurrentMeta) 344 | end. 345 | 346 | %% @private 347 | maybe_resolve(PKey, Existing, Method, AllowPut) -> 348 | SibCount = plumtree_metadata_object:value_count(Existing), 349 | maybe_resolve(PKey, Existing, SibCount, Method, AllowPut). 350 | 351 | %% @private 352 | maybe_resolve(_PKey, Existing, 1, _Method, _AllowPut) -> 353 | plumtree_metadata_object:value(Existing); 354 | maybe_resolve(PKey, Existing, _, Method, AllowPut) -> 355 | Reconciled = plumtree_metadata_object:resolve(Existing, Method), 356 | RContext = plumtree_metadata_object:context(Reconciled), 357 | RValue = plumtree_metadata_object:value(Reconciled), 358 | case AllowPut of 359 | false -> 360 | ok; 361 | true -> 362 | Stored = plumtree_metadata_manager:put(PKey, RContext, RValue), 363 | broadcast(PKey, Stored) 364 | end, 365 | RValue. 366 | 367 | %% @private 368 | maybe_tombstones(Values, Default) -> 369 | [maybe_tombstone(Value, Default) || Value <- Values]. 370 | 371 | %% @private 372 | maybe_tombstone(?TOMBSTONE, Default) -> 373 | Default; 374 | maybe_tombstone(Value, _Default) -> 375 | Value. 376 | 377 | %% @private 378 | broadcast(PKey, Obj) -> 379 | Broadcast = #metadata_broadcast{pkey = PKey, 380 | obj = Obj}, 381 | plumtree_broadcast:broadcast(Broadcast, plumtree_metadata_manager). 382 | 383 | %% @private 384 | -spec prefixed_key(metadata_prefix(), metadata_key()) -> metadata_pkey(). 385 | prefixed_key(FullPrefix, Key) -> 386 | {FullPrefix, Key}. 387 | 388 | get_option(Key, Opts, Default) -> 389 | proplists:get_value(Key, Opts, Default). 390 | -------------------------------------------------------------------------------- /src/plumtree_broadcast.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% Copyright (c) 2013 Basho Technologies, Inc. All Rights Reserved. 4 | %% 5 | %% This file is provided to you under the Apache License, 6 | %% Version 2.0 (the "License"); you may not use this file 7 | %% except in compliance with the License. You may obtain 8 | %% a copy of the License at 9 | %% 10 | %% http://www.apache.org/licenses/LICENSE-2.0 11 | %% 12 | %% Unless required by applicable law or agreed to in writing, 13 | %% software distributed under the License is distributed on an 14 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | %% KIND, either express or implied. See the License for the 16 | %% specific language governing permissions and limitations 17 | %% under the License. 18 | %% 19 | %% ------------------------------------------------------------------- 20 | -module(plumtree_broadcast). 21 | 22 | -behaviour(gen_server). 23 | 24 | %% API 25 | -export([start_link/0, 26 | start_link/4, 27 | broadcast/2, 28 | update/1, 29 | broadcast_members/0, 30 | broadcast_members/1, 31 | exchanges/0, 32 | exchanges/1, 33 | cancel_exchanges/1]). 34 | 35 | %% Debug API 36 | -export([debug_get_peers/2, 37 | debug_get_peers/3, 38 | debug_get_tree/2]). 39 | 40 | %% gen_server callbacks 41 | -export([init/1, 42 | handle_call/3, 43 | handle_cast/2, 44 | handle_info/2, 45 | terminate/2, 46 | code_change/3]). 47 | 48 | -include("plumtree.hrl"). 49 | 50 | -define(SERVER, ?MODULE). 51 | 52 | -type nodename() :: any(). 53 | -type message_id() :: any(). 54 | -type message_round() :: non_neg_integer(). 55 | -type outstanding() :: {message_id(), module(), message_round(), nodename()}. 56 | -type exchange() :: {module(), node(), reference(), pid()}. 57 | -type exchanges() :: [exchange()]. 58 | 59 | -record(state, { 60 | %% Initially trees rooted at each node are the same. 61 | %% Portions of that tree belonging to this node are 62 | %% shared in this set. 63 | common_eagers :: ordsets:ordset(nodename()), 64 | 65 | %% Initially trees rooted at each node share the same lazy links. 66 | %% Typically this set will contain a single element. However, it may 67 | %% contain more in large clusters and may be empty for clusters with 68 | %% less than three nodes. 69 | common_lazys :: ordsets:ordset(nodename()), 70 | 71 | %% A mapping of sender node (root of each broadcast tree) 72 | %% to this node's portion of the tree. Elements are 73 | %% added to this structure as messages rooted at a node 74 | %% propogate to this node. Nodes that are never the 75 | %% root of a message will never have a key added to 76 | %% `eager_sets' 77 | eager_sets :: [{nodename(), ordsets:ordset(nodename())}], 78 | 79 | %% A Mapping of sender node (root of each spanning tree) 80 | %% to this node's set of lazy peers. Elements are added 81 | %% to this structure as messages rooted at a node 82 | %% propogate to this node. Nodes that are never the root 83 | %% of a message will never have a key added to `lazy_sets' 84 | lazy_sets :: [{nodename(), ordsets:ordset(nodename())}], 85 | 86 | %% Lazy messages that have not been acked. Messages are added to 87 | %% this set when a node is sent a lazy message (or when it should be 88 | %% sent one sometime in the future). Messages are removed when the lazy 89 | %% pushes are acknowleged via graft or ignores. Entries are keyed by their 90 | %% destination 91 | outstanding :: [{nodename(), outstanding()}], 92 | 93 | %% Set of registered modules that may handle messages that 94 | %% have been broadcast 95 | mods :: [module()], 96 | 97 | %% List of outstanding exchanges 98 | exchanges :: exchanges(), 99 | 100 | %% Set of all known members. Used to determine 101 | %% which members have joined and left during a membership update 102 | all_members :: ordsets:ordset(nodename()) 103 | }). 104 | 105 | %%%=================================================================== 106 | %%% API 107 | %%%=================================================================== 108 | 109 | %% @doc Starts the broadcast server on this node. The initial membership list is 110 | %% fetched from the ring. If the node is a singleton then the initial eager and lazy 111 | %% sets are empty. If there are two nodes, each will be in the others eager set and the 112 | %% lazy sets will be empty. When number of members is less than 5, each node will initially 113 | %% have one other node in its eager set and lazy set. If there are more than five nodes 114 | %% each node will have at most two other nodes in its eager set and one in its lazy set, initally. 115 | %% In addition, after the broadcast server is started, a callback is registered with ring_events 116 | %% to generate membership updates as the ring changes. 117 | -spec start_link() -> {ok, pid()} | ignore | {error, term()}. 118 | start_link() -> 119 | {ok, Members} = plumtree_peer_service_manager:members(), 120 | {InitEagers, InitLazys} = init_peers(Members), 121 | Mods = app_helper:get_env(plumtree, broadcast_mods, [plumtree_metadata_manager]), 122 | Res = start_link(Members, InitEagers, InitLazys, Mods), 123 | plumtree_peer_service_events:add_sup_callback(fun ?MODULE:update/1), 124 | Res. 125 | 126 | %% @doc Starts the broadcast server on this node. `InitMembers' must be a list 127 | %% of all members known to this node when starting the broadcast server. 128 | %% `InitEagers' are the initial peers of this node for all broadcast trees. 129 | %% `InitLazys' is a list of random peers not in `InitEagers' that will be used 130 | %% as the initial lazy peer shared by all trees for this node. If the number 131 | %% of nodes in the cluster is less than 3, `InitLazys' should be an empty list. 132 | %% `InitEagers' and `InitLazys' must also be subsets of `InitMembers'. `Mods' is 133 | %% a list of modules that may be handlers for broadcasted messages. All modules in 134 | %% `Mods' should implement the `plumtree_broadcast_handler' behaviour. 135 | %% 136 | %% NOTE: When starting the server using start_link/2 no automatic membership update from 137 | %% ring_events is registered. Use start_link/0. 138 | -spec start_link([nodename()], [nodename()], [nodename()], [module()]) -> 139 | {ok, pid()} | ignore | {error, term()}. 140 | start_link(InitMembers, InitEagers, InitLazys, Mods) -> 141 | gen_server:start_link({local, ?SERVER}, ?MODULE, 142 | [InitMembers, InitEagers, InitLazys, Mods], []). 143 | 144 | %% @doc Broadcasts a message originating from this node. The message will be delivered to 145 | %% each node at least once. The `Mod' passed is responsible for handling the message on remote 146 | %% nodes as well as providing some other information both locally and and on other nodes. 147 | %% `Mod' must be loaded on all members of the clusters and implement the 148 | %% `riak_core_broadcast_handler' behaviour. 149 | -spec broadcast(any(), module()) -> ok. 150 | broadcast(Broadcast, Mod) -> 151 | {MessageId, Payload} = Mod:broadcast_data(Broadcast), 152 | gen_server:cast(?SERVER, {broadcast, MessageId, Payload, Mod}). 153 | 154 | %% @doc Notifies broadcast server of membership update 155 | update(LocalState) -> 156 | gen_server:cast(?SERVER, {update, LocalState}). 157 | 158 | %% @doc Returns the broadcast servers view of full cluster membership. 159 | %% Wait indefinitely for a response is returned from the process 160 | -spec broadcast_members() -> ordsets:ordset(nodename()). 161 | broadcast_members() -> 162 | broadcast_members(infinity). 163 | 164 | %% @doc Returns the broadcast servers view of full cluster membership. 165 | %% Waits `Timeout' ms for a response from the server 166 | -spec broadcast_members(infinity | pos_integer()) -> ordsets:ordset(nodename()). 167 | broadcast_members(Timeout) -> 168 | gen_server:call(?SERVER, broadcast_members, Timeout). 169 | 170 | %% @doc return a list of exchanges, started by broadcast on thisnode, that are running 171 | -spec exchanges() -> exchanges(). 172 | exchanges() -> 173 | exchanges(node()). 174 | 175 | %% @doc returns a list of exchanges, started by broadcast on `Node', that are running 176 | -spec exchanges(node()) -> exchanges(). 177 | exchanges(Node) -> 178 | gen_server:call({?SERVER, Node}, exchanges, infinity). 179 | 180 | %% @doc cancel exchanges started by this node. 181 | -spec cancel_exchanges(all | 182 | {peer, node()} | 183 | {mod, module()} | 184 | reference() | 185 | pid()) -> exchanges(). 186 | cancel_exchanges(WhichExchanges) -> 187 | gen_server:call(?SERVER, {cancel_exchanges, WhichExchanges}, infinity). 188 | 189 | %%%=================================================================== 190 | %%% Debug API 191 | %%%=================================================================== 192 | 193 | %% @doc return the peers for `Node' for the tree rooted at `Root'. 194 | %% Wait indefinitely for a response is returned from the process 195 | -spec debug_get_peers(node(), node()) -> {ordsets:ordset(node()), ordsets:ordset(node())}. 196 | debug_get_peers(Node, Root) -> 197 | debug_get_peers(Node, Root, infinity). 198 | 199 | %% @doc return the peers for `Node' for the tree rooted at `Root'. 200 | %% Waits `Timeout' ms for a response from the server 201 | -spec debug_get_peers(node(), node(), infinity | pos_integer()) -> 202 | {ordsets:ordset(node()), ordsets:ordset(node())}. 203 | debug_get_peers(Node, Root, Timeout) -> 204 | gen_server:call({?SERVER, Node}, {get_peers, Root}, Timeout). 205 | 206 | %% @doc return peers for all `Nodes' for tree rooted at `Root' 207 | %% Wait indefinitely for a response is returned from the process 208 | -spec debug_get_tree(node(), [node()]) -> 209 | [{node(), {ordsets:ordset(node()), ordsets:ordset(node())}}]. 210 | debug_get_tree(Root, Nodes) -> 211 | [begin 212 | Peers = try debug_get_peers(Node, Root) 213 | catch _:_ -> down 214 | end, 215 | {Node, Peers} 216 | end || Node <- Nodes]. 217 | 218 | %%%=================================================================== 219 | %%% gen_server callbacks 220 | %%%=================================================================== 221 | 222 | %% @private 223 | -spec init([[any()],...]) -> {ok, #state{}}. 224 | init([AllMembers, InitEagers, InitLazys, Mods]) -> 225 | schedule_lazy_tick(), 226 | schedule_exchange_tick(), 227 | State1 = #state{ 228 | outstanding = orddict:new(), 229 | mods = lists:usort(Mods), 230 | exchanges=[] 231 | }, 232 | State2 = reset_peers(AllMembers, InitEagers, InitLazys, State1), 233 | {ok, State2}. 234 | 235 | %% @private 236 | -spec handle_call(term(), {pid(), term()}, #state{}) -> {reply, term(), #state{}}. 237 | handle_call({get_peers, Root}, _From, State) -> 238 | EagerPeers = all_peers(Root, State#state.eager_sets, State#state.common_eagers), 239 | LazyPeers = all_peers(Root, State#state.lazy_sets, State#state.common_lazys), 240 | {reply, {EagerPeers, LazyPeers}, State}; 241 | handle_call(broadcast_members, _From, State=#state{all_members=AllMembers}) -> 242 | {reply, AllMembers, State}; 243 | handle_call(exchanges, _From, State=#state{exchanges=Exchanges}) -> 244 | {reply, Exchanges, State}; 245 | handle_call({cancel_exchanges, WhichExchanges}, _From, State) -> 246 | Cancelled = cancel_exchanges(WhichExchanges, State#state.exchanges), 247 | {reply, Cancelled, State}. 248 | 249 | %% @private 250 | -spec handle_cast(term(), #state{}) -> {noreply, #state{}}. 251 | handle_cast({broadcast, MessageId, Message, Mod}, State) -> 252 | State1 = eager_push(MessageId, Message, Mod, State), 253 | State2 = schedule_lazy_push(MessageId, Mod, State1), 254 | {noreply, State2}; 255 | handle_cast({broadcast, MessageId, Message, Mod, Round, Root, From}, State) -> 256 | Valid = Mod:merge(MessageId, Message), 257 | State1 = handle_broadcast(Valid, MessageId, Message, Mod, Round, Root, From, State), 258 | {noreply, State1}; 259 | handle_cast({prune, Root, From}, State) -> 260 | State1 = add_lazy(From, Root, State), 261 | {noreply, State1}; 262 | handle_cast({i_have, MessageId, Mod, Round, Root, From}, State) -> 263 | Stale = Mod:is_stale(MessageId), 264 | State1 = handle_ihave(Stale, MessageId, Mod, Round, Root, From, State), 265 | {noreply, State1}; 266 | handle_cast({ignored_i_have, MessageId, Mod, Round, Root, From}, State) -> 267 | State1 = ack_outstanding(MessageId, Mod, Round, Root, From, State), 268 | {noreply, State1}; 269 | handle_cast({graft, MessageId, Mod, Round, Root, From}, State) -> 270 | Result = Mod:graft(MessageId), 271 | State1 = handle_graft(Result, MessageId, Mod, Round, Root, From, State), 272 | {noreply, State1}; 273 | handle_cast({update, LocalState}, State=#state{all_members=BroadcastMembers}) -> 274 | Members = ?SET:value(LocalState), 275 | CurrentMembers = ordsets:from_list(Members), 276 | New = ordsets:subtract(CurrentMembers, BroadcastMembers), 277 | Removed = ordsets:subtract(BroadcastMembers, CurrentMembers), 278 | State1 = case ordsets:size(New) > 0 of 279 | false -> State; 280 | true -> 281 | {EagerPeers, LazyPeers} = init_peers(CurrentMembers), 282 | reset_peers(CurrentMembers, EagerPeers, LazyPeers, State) 283 | end, 284 | State2 = neighbors_down(Removed, State1), 285 | {noreply, State2}. 286 | 287 | %% @private 288 | -spec handle_info('exchange_tick' | 'lazy_tick' | {'DOWN', _, 'process', _, _}, #state{}) -> 289 | {noreply, #state{}}. 290 | handle_info(lazy_tick, State) -> 291 | schedule_lazy_tick(), 292 | _ = send_lazy(State), 293 | {noreply, State}; 294 | handle_info(exchange_tick, State) -> 295 | schedule_exchange_tick(), 296 | State1 = maybe_exchange(State), 297 | {noreply, State1}; 298 | handle_info({'DOWN', Ref, process, _Pid, _Reason}, State=#state{exchanges=Exchanges}) -> 299 | Exchanges1 = lists:keydelete(Ref, 3, Exchanges), 300 | {noreply, State#state{exchanges=Exchanges1}}. 301 | 302 | %% @private 303 | -spec terminate(term(), #state{}) -> term(). 304 | terminate(_Reason, _State) -> 305 | ok. 306 | 307 | %% @private 308 | -spec code_change(term() | {down, term()}, #state{}, term()) -> {ok, #state{}}. 309 | code_change(_OldVsn, State, _Extra) -> 310 | {ok, State}. 311 | 312 | %%%=================================================================== 313 | %%% Internal functions 314 | %%%=================================================================== 315 | handle_broadcast(false, _MessageId, _Message, _Mod, _Round, Root, From, State) -> %% stale msg 316 | State1 = add_lazy(From, Root, State), 317 | _ = send({prune, Root, node()}, From), 318 | State1; 319 | handle_broadcast(true, MessageId, Message, Mod, Round, Root, From, State) -> %% valid msg 320 | State1 = add_eager(From, Root, State), 321 | State2 = eager_push(MessageId, Message, Mod, Round+1, Root, From, State1), 322 | schedule_lazy_push(MessageId, Mod, Round+1, Root, From, State2). 323 | 324 | handle_ihave(true, MessageId, Mod, Round, Root, From, State) -> %% stale i_have 325 | _ = send({ignored_i_have, MessageId, Mod, Round, Root, node()}, From), 326 | State; 327 | handle_ihave(false, MessageId, Mod, Round, Root, From, State) -> %% valid i_have 328 | %% TODO: don't graft immediately 329 | _ = send({graft, MessageId, Mod, Round, Root, node()}, From), 330 | add_eager(From, Root, State). 331 | 332 | handle_graft(stale, MessageId, Mod, Round, Root, From, State) -> 333 | %% There has been a subsequent broadcast that is causally newer than this message 334 | %% according to Mod. We ack the outstanding message since the outstanding entry 335 | %% for the newer message exists 336 | ack_outstanding(MessageId, Mod, Round, Root, From, State); 337 | handle_graft({ok, Message}, MessageId, Mod, Round, Root, From, State) -> 338 | %% we don't ack outstanding here because the broadcast may fail to be delivered 339 | %% instead we will allow the i_have to be sent once more and let the subsequent 340 | %% ignore serve as the ack. 341 | State1 = add_eager(From, Root, State), 342 | _ = send({broadcast, MessageId, Message, Mod, Round, Root, node()}, From), 343 | State1; 344 | handle_graft({error, Reason}, _MessageId, Mod, _Round, _Root, _From, State) -> 345 | lager:error("unable to graft message from ~p. reason: ~p", [Mod, Reason]), 346 | State. 347 | 348 | neighbors_down(Removed, State=#state{common_eagers=CommonEagers,eager_sets=EagerSets, 349 | common_lazys=CommonLazys,lazy_sets=LazySets, 350 | outstanding=Outstanding}) -> 351 | NewCommonEagers = ordsets:subtract(CommonEagers, Removed), 352 | NewCommonLazys = ordsets:subtract(CommonLazys, Removed), 353 | %% TODO: once we have delayed grafting need to remove timers 354 | NewEagerSets = ordsets:from_list([{Root, ordsets:subtract(Existing, Removed)} || 355 | {Root, Existing} <- ordsets:to_list(EagerSets)]), 356 | NewLazySets = ordsets:from_list([{Root, ordsets:subtract(Existing, Removed)} || 357 | {Root, Existing} <- ordsets:to_list(LazySets)]), 358 | %% delete outstanding messages to removed peers 359 | NewOutstanding = ordsets:fold(fun(RPeer, OutstandingAcc) -> 360 | orddict:erase(RPeer, OutstandingAcc) 361 | end, 362 | Outstanding, Removed), 363 | State#state{common_eagers=NewCommonEagers, 364 | common_lazys=NewCommonLazys, 365 | eager_sets=NewEagerSets, 366 | lazy_sets=NewLazySets, 367 | outstanding=NewOutstanding}. 368 | 369 | eager_push(MessageId, Message, Mod, State) -> 370 | eager_push(MessageId, Message, Mod, 0, node(), node(), State). 371 | 372 | eager_push(MessageId, Message, Mod, Round, Root, From, State) -> 373 | Peers = eager_peers(Root, From, State), 374 | _ = send({broadcast, MessageId, Message, Mod, Round, Root, node()}, Peers), 375 | State. 376 | 377 | schedule_lazy_push(MessageId, Mod, State) -> 378 | schedule_lazy_push(MessageId, Mod, 0, node(), node(), State). 379 | 380 | schedule_lazy_push(MessageId, Mod, Round, Root, From, State) -> 381 | Peers = lazy_peers(Root, From, State), 382 | add_all_outstanding(MessageId, Mod, Round, Root, Peers, State). 383 | 384 | send_lazy(#state{outstanding=Outstanding}) -> 385 | [send_lazy(Peer, Messages) || {Peer, Messages} <- orddict:to_list(Outstanding)]. 386 | 387 | send_lazy(Peer, Messages) -> 388 | [send_lazy(MessageId, Mod, Round, Root, Peer) || 389 | {MessageId, Mod, Round, Root} <- ordsets:to_list(Messages)]. 390 | 391 | send_lazy(MessageId, Mod, Round, Root, Peer) -> 392 | send({i_have, MessageId, Mod, Round, Root, node()}, Peer). 393 | 394 | maybe_exchange(State) -> 395 | Root = random_root(State), 396 | Peer = random_peer(Root, State), 397 | maybe_exchange(Peer, State). 398 | 399 | maybe_exchange(undefined, State) -> 400 | State; 401 | maybe_exchange(Peer, State=#state{mods=[Mod | _],exchanges=Exchanges}) -> 402 | %% limit the number of exchanges this node can start concurrently. 403 | %% the exchange must (currently?) implement any "inbound" concurrency limits 404 | ExchangeLimit = app_helper:get_env(plumtree, broadcast_start_exchange_limit, 1), 405 | BelowLimit = not (length(Exchanges) >= ExchangeLimit), 406 | FreeMod = lists:keyfind(Mod, 1, Exchanges) =:= false, 407 | case BelowLimit and FreeMod of 408 | true -> exchange(Peer, State); 409 | false -> State 410 | end; 411 | maybe_exchange(_Peer, State=#state{mods=[]}) -> 412 | %% No registered handler. 413 | State. 414 | 415 | exchange(Peer, State=#state{mods=[Mod | Mods],exchanges=Exchanges}) -> 416 | State1 = case Mod:exchange(Peer) of 417 | {ok, Pid} -> 418 | lager:debug("started ~p exchange with ~p (~p)", [Mod, Peer, Pid]), 419 | Ref = monitor(process, Pid), 420 | State#state{exchanges=[{Mod, Peer, Ref, Pid} | Exchanges]}; 421 | {error, _Reason} -> 422 | State 423 | end, 424 | State1#state{mods=Mods ++ [Mod]}. 425 | 426 | cancel_exchanges(all, Exchanges) -> 427 | kill_exchanges(Exchanges); 428 | cancel_exchanges(WhichProc, Exchanges) when is_reference(WhichProc) orelse 429 | is_pid(WhichProc) -> 430 | KeyPos = case is_reference(WhichProc) of 431 | true -> 3; 432 | false -> 4 433 | end, 434 | case lists:keyfind(WhichProc, KeyPos, Exchanges) of 435 | false -> []; 436 | Exchange -> 437 | kill_exchange(Exchange), 438 | [Exchange] 439 | end; 440 | cancel_exchanges(Which, Exchanges) -> 441 | Filter = exchange_filter(Which), 442 | ToCancel = [Ex || Ex <- Exchanges, Filter(Ex)], 443 | kill_exchanges(ToCancel). 444 | 445 | kill_exchanges(Exchanges) -> 446 | _ = [kill_exchange(Exchange) || Exchange <- Exchanges], 447 | Exchanges. 448 | 449 | kill_exchange({_, _, _, ExchangePid}) -> 450 | exit(ExchangePid, cancel_exchange). 451 | 452 | 453 | exchange_filter({peer, Peer}) -> 454 | fun({_, ExchangePeer, _, _}) -> 455 | Peer =:= ExchangePeer 456 | end; 457 | exchange_filter({mod, Mod}) -> 458 | fun({ExchangeMod, _, _, _}) -> 459 | Mod =:= ExchangeMod 460 | end. 461 | 462 | %% picks random root uniformly 463 | random_root(#state{all_members=Members}) -> 464 | random_other_node(Members). 465 | 466 | %% picks random peer favoring peers not in eager or lazy set and ensuring 467 | %% peer is not this node 468 | random_peer(Root, State=#state{all_members=All}) -> 469 | Eagers = all_eager_peers(Root, State), 470 | Lazys = all_lazy_peers(Root, State), 471 | Union = ordsets:union([Eagers, Lazys]), 472 | Other = ordsets:del_element(node(), ordsets:subtract(All, Union)), 473 | case ordsets:size(Other) of 474 | 0 -> 475 | random_other_node(ordsets:del_element(node(), All)); 476 | _ -> 477 | random_other_node(Other) 478 | end. 479 | 480 | %% picks random node from ordset 481 | random_other_node(OrdSet) -> 482 | Size = ordsets:size(OrdSet), 483 | case Size of 484 | 0 -> undefined; 485 | _ -> 486 | lists:nth(random:uniform(Size), 487 | ordsets:to_list(OrdSet)) 488 | end. 489 | 490 | ack_outstanding(MessageId, Mod, Round, Root, From, State=#state{outstanding=All}) -> 491 | Existing = existing_outstanding(From, All), 492 | Updated = set_outstanding(From, 493 | ordsets:del_element({MessageId, Mod, Round, Root}, Existing), 494 | All), 495 | State#state{outstanding=Updated}. 496 | 497 | add_all_outstanding(MessageId, Mod, Round, Root, Peers, State) -> 498 | lists:foldl(fun(Peer, SAcc) -> add_outstanding(MessageId, Mod, Round, Root, Peer, SAcc) end, 499 | State, 500 | ordsets:to_list(Peers)). 501 | 502 | add_outstanding(MessageId, Mod, Round, Root, Peer, State=#state{outstanding=All}) -> 503 | Existing = existing_outstanding(Peer, All), 504 | Updated = set_outstanding(Peer, 505 | ordsets:add_element({MessageId, Mod, Round, Root}, Existing), 506 | All), 507 | State#state{outstanding=Updated}. 508 | 509 | set_outstanding(Peer, Outstanding, All) -> 510 | case ordsets:size(Outstanding) of 511 | 0 -> orddict:erase(Peer, All); 512 | _ -> orddict:store(Peer, Outstanding, All) 513 | end. 514 | 515 | existing_outstanding(Peer, All) -> 516 | case orddict:find(Peer, All) of 517 | error -> ordsets:new(); 518 | {ok, Outstanding} -> Outstanding 519 | end. 520 | 521 | add_eager(From, Root, State) -> 522 | update_peers(From, Root, fun ordsets:add_element/2, fun ordsets:del_element/2, State). 523 | 524 | add_lazy(From, Root, State) -> 525 | update_peers(From, Root, fun ordsets:del_element/2, fun ordsets:add_element/2, State). 526 | 527 | update_peers(From, Root, EagerUpdate, LazyUpdate, State) -> 528 | CurrentEagers = all_eager_peers(Root, State), 529 | CurrentLazys = all_lazy_peers(Root, State), 530 | NewEagers = EagerUpdate(From, CurrentEagers), 531 | NewLazys = LazyUpdate(From, CurrentLazys), 532 | set_peers(Root, NewEagers, NewLazys, State). 533 | 534 | set_peers(Root, Eagers, Lazys, State=#state{eager_sets=EagerSets,lazy_sets=LazySets}) -> 535 | NewEagers = orddict:store(Root, Eagers, EagerSets), 536 | NewLazys = orddict:store(Root, Lazys, LazySets), 537 | State#state{eager_sets=NewEagers, lazy_sets=NewLazys}. 538 | 539 | all_eager_peers(Root, State) -> 540 | all_peers(Root, State#state.eager_sets, State#state.common_eagers). 541 | 542 | all_lazy_peers(Root, State) -> 543 | all_peers(Root, State#state.lazy_sets, State#state.common_lazys). 544 | 545 | eager_peers(Root, From, #state{eager_sets=EagerSets, common_eagers=CommonEagers}) -> 546 | all_filtered_peers(Root, From, EagerSets, CommonEagers). 547 | 548 | lazy_peers(Root, From, #state{lazy_sets=LazySets, common_lazys=CommonLazys}) -> 549 | all_filtered_peers(Root, From, LazySets, CommonLazys). 550 | 551 | all_filtered_peers(Root, From, Sets, Common) -> 552 | All = all_peers(Root, Sets, Common), 553 | ordsets:del_element(From, All). 554 | 555 | all_peers(Root, Sets, Default) -> 556 | case orddict:find(Root, Sets) of 557 | error -> Default; 558 | {ok, Peers} -> Peers 559 | end. 560 | 561 | send(Msg, Peers) when is_list(Peers) -> 562 | [send(Msg, P) || P <- Peers]; 563 | send(Msg, P) -> 564 | %% TODO: add debug logging 565 | gen_server:cast({?SERVER, P}, Msg). 566 | 567 | schedule_lazy_tick() -> 568 | schedule_tick(lazy_tick, broadcast_lazy_timer, 1000). 569 | 570 | schedule_exchange_tick() -> 571 | schedule_tick(exchange_tick, broadcast_exchange_timer, 10000). 572 | 573 | schedule_tick(Message, Timer, Default) -> 574 | TickMs = app_helper:get_env(plumtree, Timer, Default), 575 | erlang:send_after(TickMs, ?MODULE, Message). 576 | 577 | reset_peers(AllMembers, EagerPeers, LazyPeers, State) -> 578 | State#state{ 579 | common_eagers = ordsets:del_element(node(), ordsets:from_list(EagerPeers)), 580 | common_lazys = ordsets:del_element(node(), ordsets:from_list(LazyPeers)), 581 | eager_sets = orddict:new(), 582 | lazy_sets = orddict:new(), 583 | all_members = ordsets:from_list(AllMembers) 584 | }. 585 | 586 | init_peers(Members) -> 587 | case length(Members) of 588 | 1 -> 589 | %% Single member, must be ourselves 590 | InitEagers = [], 591 | InitLazys = []; 592 | 2 -> 593 | %% Two members, just eager push to the other 594 | InitEagers = Members -- [node()], 595 | InitLazys = []; 596 | N when N < 5 -> 597 | %% 2 to 4 members, start with a fully connected tree 598 | %% with cycles. it will be adjusted as needed 599 | Tree = plumtree_util:build_tree(1, Members, [cycles]), 600 | InitEagers = orddict:fetch(node(), Tree), 601 | InitLazys = [lists:nth(random:uniform(N - 2), Members -- [node() | InitEagers])]; 602 | N when N < 10 -> 603 | %% 5 to 9 members, start with gossip tree used by 604 | %% riak_core_gossip. it will be adjusted as needed 605 | Tree = plumtree_util:build_tree(2, Members, [cycles]), 606 | InitEagers = orddict:fetch(node(), Tree), 607 | InitLazys = [lists:nth(random:uniform(N - 3), Members -- [node() | InitEagers])]; 608 | N -> 609 | %% 10 or more members, use a tree similar to riak_core_gossip 610 | %% but with higher fanout (larger initial eager set size) 611 | NEagers = round(math:log(N) + 1), 612 | Tree = plumtree_util:build_tree(NEagers, Members, [cycles]), 613 | InitEagers = orddict:fetch(node(), Tree), 614 | InitLazys = [lists:nth(random:uniform(N - (NEagers + 1)), Members -- [node() | InitEagers])] 615 | end, 616 | {InitEagers, InitLazys}. 617 | -------------------------------------------------------------------------------- /src/plumtree_metadata_manager.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% Copyright (c) 2013 Basho Technologies, Inc. All Rights Reserved. 4 | %% 5 | %% This file is provided to you under the Apache License, 6 | %% Version 2.0 (the "License"); you may not use this file 7 | %% except in compliance with the License. You may obtain 8 | %% a copy of the License at 9 | %% 10 | %% http://www.apache.org/licenses/LICENSE-2.0 11 | %% 12 | %% Unless required by applicable law or agreed to in writing, 13 | %% software distributed under the License is distributed on an 14 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | %% KIND, either express or implied. See the License for the 16 | %% specific language governing permissions and limitations 17 | %% under the License. 18 | %% 19 | %% ------------------------------------------------------------------- 20 | -module(plumtree_metadata_manager). 21 | 22 | -behaviour(gen_server). 23 | -behaviour(plumtree_broadcast_handler). 24 | 25 | %% API 26 | -export([start_link/0, 27 | start_link/1, 28 | get/1, 29 | get/2, 30 | iterator/0, 31 | iterator/1, 32 | iterator/2, 33 | remote_iterator/1, 34 | remote_iterator/2, 35 | iterate/1, 36 | iterator_prefix/1, 37 | iterator_value/1, 38 | iterator_done/1, 39 | iterator_close/1, 40 | put/3, 41 | merge/3]). 42 | 43 | %% plumtree_broadcast_handler callbacks 44 | -export([broadcast_data/1, 45 | merge/2, 46 | is_stale/1, 47 | graft/1, 48 | exchange/1]). 49 | 50 | %% gen_server callbacks 51 | -export([init/1, handle_call/3, handle_cast/2, handle_info/2, 52 | terminate/2, code_change/3]). 53 | 54 | -export_type([metadata_iterator/0]). 55 | 56 | -include("plumtree_metadata.hrl"). 57 | 58 | -define(SERVER, ?MODULE). 59 | -define(MANIFEST, plumtree_cluster_meta_manifest). 60 | -define(MANIFEST_FILENAME, "plumtree_meta_manifest.dets"). 61 | -define(ETS, plumtree_metadata_manager_prefixes_ets). 62 | 63 | -record(state, { 64 | %% identifier used in logical clocks 65 | serverid :: term(), 66 | 67 | %% where data files are stored 68 | data_root :: file:filename(), 69 | 70 | %% an ets table to hold iterators opened 71 | %% by other nodes 72 | iterators :: ets:tab() 73 | }). 74 | 75 | -record(metadata_iterator, { 76 | prefix :: metadata_prefix(), 77 | match :: term(), 78 | pos :: term(), 79 | obj :: {metadata_key(), metadata_object()} | undefined, 80 | done :: boolean(), 81 | tab :: ets:tab() 82 | }). 83 | 84 | -record(remote_iterator, { 85 | node :: node(), 86 | ref :: reference(), 87 | prefix :: metadata_prefix() | atom() | binary() 88 | }). 89 | 90 | -opaque metadata_iterator() :: #metadata_iterator{}. 91 | -type remote_iterator() :: #remote_iterator{}. 92 | 93 | -type mm_path_opt() :: {data_dir, file:name_all()}. 94 | -type mm_nodename_opt() :: {nodename, term()}. 95 | -type mm_opt() :: mm_path_opt() | mm_nodename_opt(). 96 | -type mm_opts() :: [mm_opt()]. 97 | 98 | %%%=================================================================== 99 | %%% API 100 | %%%=================================================================== 101 | 102 | %% @doc Same as start_link([]). 103 | -spec start_link() -> {ok, pid()} | ignore | {error, term()}. 104 | start_link() -> 105 | start_link([]). 106 | 107 | %% @doc Start plumtree_metadadata_manager and link to calling process. 108 | %% 109 | %% The following options can be provided: 110 | %% * data_dir: the root directory to place cluster metadata files. 111 | %% if not provided this defaults to the `cluster_meta' directory in 112 | %% plumtree's `platform_data_dir'. 113 | %% * nodename: the node identifier (for use in logical clocks). defaults to node() 114 | -spec start_link(mm_opts()) -> {ok, pid()} | ignore | {error, term()}. 115 | start_link(Opts) -> 116 | gen_server:start_link({local, ?SERVER}, ?MODULE, [Opts], []). 117 | 118 | %% @doc Reads the value for a prefixed key. If the value does not exist `undefined' is 119 | %% returned. otherwise a Dotted Version Vector Set is returned. When reading the value 120 | %% for a subsequent call to put/3 the context can be obtained using 121 | %% plumtree_metadata_object:context/1. Values can obtained w/ plumtree_metadata_object:values/1. 122 | -spec get(metadata_pkey()) -> metadata_object() | undefined. 123 | get({{Prefix, SubPrefix}, _Key}=PKey) when (is_binary(Prefix) orelse is_atom(Prefix)) andalso 124 | (is_binary(SubPrefix) orelse is_atom(SubPrefix)) -> 125 | read(PKey). 126 | 127 | %% @doc Same as get/1 but reads the value from `Node' 128 | -spec get(node(), metadata_pkey()) -> metadata_object() | undefined. 129 | get(Node, PKey) when node() =:= Node -> 130 | ?MODULE:get(PKey); 131 | get(Node, {{Prefix, SubPrefix}, _Key}=PKey) 132 | when (is_binary(Prefix) orelse is_atom(Prefix)) andalso 133 | (is_binary(SubPrefix) orelse is_atom(SubPrefix)) -> 134 | gen_server:call({?SERVER, Node}, {get, PKey}, infinity). 135 | 136 | 137 | %% @doc Returns a full-prefix iterator: an iterator for all full-prefixes that have keys stored under them 138 | %% When done with the iterator, iterator_close/1 must be called 139 | -spec iterator() -> metadata_iterator(). 140 | iterator() -> 141 | iterator(undefined). 142 | 143 | %% @doc Returns a sub-prefix iterator for a given prefix. 144 | %% When done with the iterator, iterator_close/1 must be called 145 | -spec iterator(binary() | atom()) -> metadata_iterator(). 146 | iterator(Prefix) when is_binary(Prefix) or is_atom(Prefix) -> 147 | open_iterator(undefined, Prefix). 148 | 149 | %% @doc Return an iterator for keys stored under a prefix. If KeyMatch is undefined then 150 | %% all keys will may be visted by the iterator. Otherwise only keys matching KeyMatch will be 151 | %% visited. 152 | %% 153 | %% KeyMatch can be either: 154 | %% * an erlang term - which will be matched exactly against a key 155 | %% * '_' - which is equivalent to undefined 156 | %% * an erlang tuple containing terms and '_' - if tuples are used as keys 157 | %% * this can be used to iterate over some subset of keys 158 | %% 159 | %% When done with the iterator, iterator_close/1 must be called 160 | -spec iterator(metadata_prefix() , term()) -> metadata_iterator(). 161 | iterator({Prefix, SubPrefix}=FullPrefix, KeyMatch) 162 | when (is_binary(Prefix) orelse is_atom(Prefix)) andalso 163 | (is_binary(SubPrefix) orelse is_atom(SubPrefix)) -> 164 | open_iterator(FullPrefix, KeyMatch). 165 | 166 | %% @doc Create an iterator on `Node'. This allows for remote iteration by having 167 | %% the metadata manager keep track of the actual iterator (since ets continuations cannot 168 | %% cross node boundaries). The iterator created iterates all full-prefixes. Once created 169 | %% the rest of the iterator API may be used as usual. When done with the iterator, 170 | %% iterator_close/1 must be called 171 | -spec remote_iterator(node()) -> remote_iterator(). 172 | remote_iterator(Node) -> 173 | remote_iterator(Node, undefined). 174 | 175 | %% @doc Create an iterator on `Node'. This allows for remote iteration 176 | %% by having the metadata manager keep track of the actual iterator 177 | %% (since ets continuations cannot cross node boundaries). When 178 | %% `Perfix' is not a full prefix, the iterator created iterates all 179 | %% sub-prefixes under `Prefix'. Otherse, the iterator iterates all keys 180 | %% under a prefix. Once created the rest of the iterator API may be used as usual. 181 | %% When done with the iterator, iterator_close/1 must be called 182 | -spec remote_iterator(node(), metadata_prefix() | binary() | atom() | undefined) -> remote_iterator(). 183 | remote_iterator(Node, Prefix) when is_atom(Prefix) or is_binary(Prefix) -> 184 | Ref = gen_server:call({?SERVER, Node}, {open_remote_iterator, self(), undefined, Prefix}, infinity), 185 | #remote_iterator{ref=Ref,prefix=Prefix,node=Node}; 186 | remote_iterator(Node, FullPrefix) when is_tuple(FullPrefix) -> 187 | Ref = gen_server:call({?SERVER, Node}, {open_remote_iterator, self(), FullPrefix, undefined}, infinity), 188 | #remote_iterator{ref=Ref,prefix=FullPrefix,node=Node}. 189 | 190 | %% @doc advance the iterator by one key, full-prefix or sub-prefix 191 | -spec iterate(metadata_iterator() | remote_iterator()) -> metadata_iterator() | remote_iterator(). 192 | iterate(It=#remote_iterator{ref=Ref,node=Node}) -> 193 | gen_server:call({?SERVER, Node}, {iterate, Ref}, infinity), 194 | It; 195 | iterate(Iterator) -> 196 | next_iterator(Iterator). 197 | 198 | %% @doc return the full-prefix or prefix being iterated by this iterator. If the iterator is a 199 | %% full-prefix iterator undefined is returned. 200 | -spec iterator_prefix(metadata_iterator() | remote_iterator()) -> 201 | metadata_prefix() | undefined | binary() | atom(). 202 | iterator_prefix(#remote_iterator{prefix=Prefix}) -> Prefix; 203 | iterator_prefix(#metadata_iterator{prefix=undefined,match=undefined}) -> undefined; 204 | iterator_prefix(#metadata_iterator{prefix=undefined,match=Prefix}) -> Prefix; 205 | iterator_prefix(#metadata_iterator{prefix=Prefix}) -> Prefix. 206 | 207 | %% @doc return the key and object or the prefix pointed to by the iterator 208 | -spec iterator_value(metadata_iterator() | remote_iterator()) -> 209 | {metadata_key(), metadata_object()} | 210 | metadata_prefix() | binary() | atom(). 211 | iterator_value(#remote_iterator{ref=Ref,node=Node}) -> 212 | gen_server:call({?SERVER, Node}, {iterator_value, Ref}, infinity); 213 | iterator_value(#metadata_iterator{prefix=undefined,match=undefined,pos=Pos}) -> Pos; 214 | iterator_value(#metadata_iterator{obj=Obj}) -> Obj. 215 | 216 | %% @doc returns true if there are no more keys or prefixes to iterate over 217 | -spec iterator_done(metadata_iterator() | remote_iterator()) -> boolean(). 218 | iterator_done(#remote_iterator{ref=Ref,node=Node}) -> 219 | gen_server:call({?SERVER, Node}, {iterator_done, Ref}, infinity); 220 | iterator_done(#metadata_iterator{done=Done}) -> Done. 221 | 222 | %% @doc Closes the iterator. This function must be called on all open iterators 223 | -spec iterator_close(metadata_iterator() | remote_iterator()) -> ok. 224 | iterator_close(#remote_iterator{ref=Ref,node=Node}) -> 225 | gen_server:call({?SERVER, Node}, {iterator_close, Ref}, infinity); 226 | iterator_close(#metadata_iterator{prefix=undefined,match=undefined,tab=Tab}) -> 227 | ets:safe_fixtable(Tab, false), 228 | ok; 229 | iterator_close(It) -> finish_iterator(It). 230 | 231 | %% @doc Sets the value of a prefixed key. The most recently read context (see get/1) 232 | %% should be passed as the second argument to prevent unneccessary siblings. 233 | -spec put(metadata_pkey(), 234 | metadata_context() | undefined, 235 | metadata_value() | metadata_modifier()) -> metadata_object(). 236 | put(PKey, undefined, ValueOrFun) -> 237 | %% nil is an empty version vector for dvvset 238 | put(PKey, [], ValueOrFun); 239 | put({{Prefix, SubPrefix}, _Key}=PKey, Context, ValueOrFun) 240 | when (is_binary(Prefix) orelse is_atom(Prefix)) andalso 241 | (is_binary(SubPrefix) orelse is_atom(SubPrefix)) -> 242 | gen_server:call(?SERVER, {put, PKey, Context, ValueOrFun}, infinity). 243 | 244 | %% @doc same as merge/2 but merges the object on `Node' 245 | -spec merge(node(), {metadata_pkey(), undefined | metadata_context()}, metadata_object()) -> boolean(). 246 | merge(Node, {PKey, _Context}, Obj) -> 247 | gen_server:call({?SERVER, Node}, {merge, PKey, Obj}, infinity). 248 | 249 | %%%=================================================================== 250 | %%% plumtree_broadcast_handler callbacks 251 | %%%=================================================================== 252 | 253 | %% @doc Deconstructs are broadcast that is sent using `plumtree_metadata_manager' as the 254 | %% handling module returning the message id and payload. 255 | -spec broadcast_data(metadata_broadcast()) -> {{metadata_pkey(), metadata_context()}, 256 | metadata_object()}. 257 | broadcast_data(#metadata_broadcast{pkey=Key, obj=Obj}) -> 258 | Context = plumtree_metadata_object:context(Obj), 259 | {{Key, Context}, Obj}. 260 | 261 | %% @doc Merges a remote copy of a metadata record sent via broadcast w/ the local view 262 | %% for the key contained in the message id. If the remote copy is causally older than 263 | %% the current data stored then `false' is returned and no updates are merged. Otherwise, 264 | %% the remote copy is merged (possibly generating siblings) and `true' is returned. 265 | -spec merge({metadata_pkey(), undefined | metadata_context()}, undefined | metadata_object()) -> boolean(). 266 | merge({PKey, _Context}, Obj) -> 267 | gen_server:call(?SERVER, {merge, PKey, Obj}, infinity). 268 | 269 | %% @doc Returns false if the update (or a causally newer update) has already been 270 | %% received (stored locally). 271 | -spec is_stale({metadata_pkey(), metadata_context()}) -> boolean(). 272 | is_stale({PKey, Context}) -> 273 | gen_server:call(?SERVER, {is_stale, PKey, Context}, infinity). 274 | 275 | %% @doc returns the object associated with the given key and context (message id) if 276 | %% the currently stored version has an equal context. otherwise stale is returned. 277 | %% because it assumed that a grafted context can only be causally older than the local view 278 | %% a stale response means there is another message that subsumes the grafted one 279 | -spec graft({metadata_pkey(), metadata_context()}) -> 280 | stale | {ok, metadata_object()} | {error, term()}. 281 | graft({PKey, Context}) -> 282 | case ?MODULE:get(PKey) of 283 | undefined -> 284 | %% There would have to be a serious error in implementation to hit this case. 285 | %% Catch if here b/c it would be much harder to detect 286 | lager:error("object not found during graft for key: ~p", [PKey]), 287 | {error, {not_found, PKey}}; 288 | Obj -> 289 | graft(Context, Obj) 290 | end. 291 | 292 | graft(Context, Obj) -> 293 | case plumtree_metadata_object:equal_context(Context, Obj) of 294 | false -> 295 | %% when grafting the context will never be causally newer 296 | %% than what we have locally. Since its not equal, it must be 297 | %% an ancestor. Thus we've sent another, newer update that contains 298 | %% this context's information in addition to its own. This graft 299 | %% is deemed stale 300 | stale; 301 | true -> 302 | {ok, Obj} 303 | end. 304 | 305 | %% @doc Trigger an exchange 306 | -spec exchange(node()) -> {ok, pid()} | {error, term()}. 307 | exchange(Peer) -> 308 | Timeout = app_helper:get_env(plumtree, metadata_exchange_timeout, 60000), 309 | case plumtree_metadata_exchange_fsm:start(Peer, Timeout) of 310 | {ok, Pid} -> 311 | {ok, Pid}; 312 | {error, Reason} -> 313 | {error, Reason}; 314 | ignore -> 315 | {error, ignore} 316 | end. 317 | 318 | %%%=================================================================== 319 | %%% gen_server callbacks 320 | %%%=================================================================== 321 | 322 | %% @private 323 | -spec init([mm_opts()]) -> {ok, #state{}} | 324 | {stop, no_data_dir}. 325 | init([Opts]) -> 326 | case data_root(Opts) of 327 | undefined -> 328 | {stop, no_data_dir}; 329 | DataRoot -> 330 | ?ETS = ets:new(?ETS, [named_table, 331 | {read_concurrency, true}, {write_concurrency, true}]), 332 | Nodename = proplists:get_value(nodename, Opts, node()), 333 | State = #state{serverid=Nodename, 334 | data_root=DataRoot, 335 | iterators=new_ets_tab()}, 336 | {ok, _} = init_manifest(State), 337 | %% TODO: should do this out-of-band from startup so we don't block 338 | init_from_files(State), 339 | {ok, State} 340 | end. 341 | 342 | %% @private 343 | -spec handle_call(term(), {pid(), term()}, #state{}) -> {reply, term(), #state{}}. 344 | handle_call({put, PKey, Context, ValueOrFun}, _From, State) -> 345 | {Result, NewState} = read_modify_write(PKey, Context, ValueOrFun, State), 346 | {reply, Result, NewState}; 347 | handle_call({merge, PKey, Obj}, _From, State) -> 348 | {Result, NewState} = read_merge_write(PKey, Obj, State), 349 | {reply, Result, NewState}; 350 | handle_call({get, PKey}, _From, State) -> 351 | Result = read(PKey), 352 | {reply, Result, State}; 353 | handle_call({open_remote_iterator, Pid, FullPrefix, KeyMatch}, _From, State) -> 354 | Iterator = new_remote_iterator(Pid, FullPrefix, KeyMatch, State), 355 | {reply, Iterator, State}; 356 | handle_call({iterate, RemoteRef}, _From, State) -> 357 | Next = next_iterator(RemoteRef, State), 358 | {reply, Next, State}; 359 | handle_call({iterator_value, RemoteRef}, _From, State) -> 360 | Res = from_remote_iterator(fun iterator_value/1, RemoteRef, State), 361 | {reply, Res, State}; 362 | handle_call({iterator_done, RemoteRef}, _From, State) -> 363 | Res = case from_remote_iterator(fun iterator_done/1, RemoteRef, State) of 364 | undefined -> true; %% if we don't know about iterator, treat it as done 365 | Other -> Other 366 | end, 367 | {reply, Res, State}; 368 | handle_call({iterator_close, RemoteRef}, _From, State) -> 369 | close_remote_iterator(RemoteRef, State), 370 | {reply, ok, State}; 371 | handle_call({is_stale, PKey, Context}, _From, State) -> 372 | Existing = read(PKey), 373 | IsStale = plumtree_metadata_object:is_stale(Context, Existing), 374 | {reply, IsStale, State}. 375 | 376 | %% @private 377 | -spec handle_cast(term(), #state{}) -> {noreply, #state{}}. 378 | handle_cast(_Msg, State) -> 379 | {noreply, State}. 380 | 381 | %% @private 382 | -spec handle_info({'DOWN', _, 'process', _, _}, #state{}) -> {noreply, #state{}}. 383 | handle_info({'DOWN', ItRef, process, _Pid, _Reason}, State) -> 384 | close_remote_iterator(ItRef, State), 385 | {noreply, State}. 386 | 387 | %% @private 388 | -spec terminate(term(), #state{}) -> term(). 389 | terminate(_Reason, _State) -> 390 | close_dets_tabs(), 391 | ok = close_manifest(). 392 | 393 | %% @private 394 | -spec code_change(term() | {down, term()}, #state{}, term()) -> {ok, #state{}}. 395 | code_change(_OldVsn, State, _Extra) -> 396 | {ok, State}. 397 | 398 | %%%=================================================================== 399 | %%% Internal functions 400 | %%%=================================================================== 401 | 402 | new_remote_iterator(Pid, FullPrefix, KeyMatch, #state{iterators=Iterators}) -> 403 | Ref = monitor(process, Pid), 404 | Iterator = open_iterator(FullPrefix, KeyMatch), 405 | ets:insert(Iterators, [{Ref, Iterator}]), 406 | Ref. 407 | 408 | 409 | from_remote_iterator(Fun, RemoteRef, State) -> 410 | case ets:lookup(State#state.iterators, RemoteRef) of 411 | [] -> undefined; 412 | [{RemoteRef, It}] -> Fun(It) 413 | end. 414 | 415 | close_remote_iterator(Ref, State=#state{iterators=Iterators}) -> 416 | from_remote_iterator(fun iterator_close/1, Ref, State), 417 | ets:delete(Iterators, Ref). 418 | 419 | open_iterator(undefined, KeyMatch) -> 420 | %% full or sub-prefix iterator 421 | new_iterator(undefined, KeyMatch, ?ETS); 422 | open_iterator(FullPrefix, KeyMatch) -> 423 | %% key/value iterator 424 | case ets_tab(FullPrefix) of 425 | undefined -> empty_iterator(FullPrefix, KeyMatch, undefined); 426 | Tab -> new_iterator(FullPrefix, KeyMatch, Tab) 427 | end. 428 | 429 | next_iterator(It=#metadata_iterator{done=true}) -> 430 | %% general catch-all for all iterators 431 | It; 432 | next_iterator(It=#metadata_iterator{prefix=undefined,match=undefined,tab=Tab,pos=Pos}) -> 433 | %% full-prefix iterator 434 | next_iterator(It, ets:next(Tab, Pos)); 435 | next_iterator(It=#metadata_iterator{prefix=undefined,pos=Pos}) -> 436 | %% sub-prefix iterator 437 | next_iterator(It, ets:select(Pos)); 438 | next_iterator(It=#metadata_iterator{pos=Pos}) -> 439 | %% key/value iterator 440 | next_iterator(It, ets:match_object(Pos)). 441 | 442 | next_iterator(Ref, #state{iterators=Iterators}) when is_reference(Ref) -> 443 | %% remote iterator 444 | case ets:lookup(Iterators, Ref) of 445 | [] -> ok; 446 | [{Ref, It}] -> 447 | Next = next_iterator(It), 448 | ets:insert(Iterators, [{Ref, Next}]) 449 | end, 450 | Ref; 451 | next_iterator(It, '$end_of_table') -> 452 | %% general catch-all for all iterators 453 | It#metadata_iterator{done=true, 454 | pos=undefined, 455 | obj=undefined}; 456 | next_iterator(It=#metadata_iterator{prefix=undefined,match=undefined},Next) -> 457 | %% full-prefix iterator 458 | It#metadata_iterator{pos=Next}; 459 | next_iterator(It, {[Next], Cont}) -> 460 | %% sub-prefix or key/value iterator 461 | It#metadata_iterator{pos=Cont, 462 | obj=Next}. 463 | 464 | %% universal empty iterator 465 | empty_iterator(FullPrefix, KeyMatch, Tab) -> 466 | #metadata_iterator{ 467 | prefix=FullPrefix, 468 | match=KeyMatch, 469 | pos=undefined, 470 | obj=undefined, 471 | done=true, 472 | tab=Tab 473 | }. 474 | 475 | new_iterator(undefined, undefined, Tab) -> 476 | %% full-prefix iterator 477 | ets:safe_fixtable(Tab, true), 478 | new_iterator(undefined, undefined, Tab, ets:first(Tab)); 479 | new_iterator(undefined, Prefix, Tab) -> 480 | %% sub-prefix iterator 481 | new_iterator(undefined, Prefix, Tab, 482 | ets:select(Tab, [{{{Prefix,'$1'},'_'},[],['$1']}], 1)); 483 | new_iterator(FullPrefix, KeyMatch, Tab) -> 484 | %% key/value iterator 485 | ObjectMatch = iterator_match(KeyMatch), 486 | new_iterator(FullPrefix, KeyMatch, Tab, ets:match_object(Tab, ObjectMatch, 1)). 487 | 488 | new_iterator(FullPrefix, KeyMatch, Tab, '$end_of_table') -> 489 | %% catch-all for empty iterator of all types 490 | empty_iterator(FullPrefix, KeyMatch, Tab); 491 | new_iterator(undefined, undefined, Tab, First) -> 492 | %% full-prefix iterator 493 | #metadata_iterator{ 494 | prefix=undefined, 495 | match=undefined, 496 | pos=First, 497 | obj=undefined, 498 | done=false, 499 | tab=Tab 500 | }; 501 | new_iterator(undefined, Prefix, Tab, {[First], Cont}) -> 502 | %% sub-prefix iterator 503 | #metadata_iterator{ 504 | prefix=undefined, 505 | match=Prefix, 506 | pos=Cont, 507 | obj=First, 508 | done=false, 509 | tab=Tab 510 | }; 511 | new_iterator(FullPrefix, KeyMatch, Tab, {[First], Cont}) -> 512 | %% key/value iterator 513 | #metadata_iterator{ 514 | prefix=FullPrefix, 515 | match=KeyMatch, 516 | pos=Cont, 517 | obj=First, 518 | done=false, 519 | tab=Tab 520 | }. 521 | 522 | finish_iterator(#metadata_iterator{done=true}) -> 523 | ok; 524 | finish_iterator(It) -> 525 | Next = next_iterator(It), 526 | finish_iterator(Next). 527 | 528 | iterator_match(undefined) -> 529 | '_'; 530 | iterator_match(KeyMatch) -> 531 | {KeyMatch, '_'}. 532 | 533 | read_modify_write(PKey, Context, ValueOrFun, State=#state{serverid=ServerId}) -> 534 | Existing = read(PKey), 535 | Modified = plumtree_metadata_object:modify(Existing, Context, ValueOrFun, ServerId), 536 | store(PKey, Modified, State). 537 | 538 | read_merge_write(PKey, Obj, State) -> 539 | Existing = read(PKey), 540 | case plumtree_metadata_object:reconcile(Obj, Existing) of 541 | false -> {false, State}; 542 | {true, Reconciled} -> 543 | {_, NewState} = store(PKey, Reconciled, State), 544 | {true, NewState} 545 | end. 546 | 547 | store({FullPrefix, Key}=PKey, Metadata, State) -> 548 | _ = maybe_init_ets(FullPrefix), 549 | maybe_init_dets(FullPrefix, State#state.data_root), 550 | 551 | Objs = [{Key, Metadata}], 552 | Hash = plumtree_metadata_object:hash(Metadata), 553 | ets:insert(ets_tab(FullPrefix), Objs), 554 | plumtree_metadata_hashtree:insert(PKey, Hash), 555 | ok = dets_insert(dets_tabname(FullPrefix), Objs), 556 | {Metadata, State}. 557 | 558 | read({FullPrefix, Key}) -> 559 | case ets_tab(FullPrefix) of 560 | undefined -> undefined; 561 | TabId -> read(Key, TabId) 562 | end. 563 | 564 | read(Key, TabId) -> 565 | case ets:lookup(TabId, Key) of 566 | [] -> undefined; 567 | [{Key, MetaRec}] -> MetaRec 568 | end. 569 | 570 | init_manifest(State) -> 571 | ManifestFile = filename:join(State#state.data_root, ?MANIFEST_FILENAME), 572 | ok = filelib:ensure_dir(ManifestFile), 573 | {ok, ?MANIFEST} = dets:open_file(?MANIFEST, [{file, ManifestFile}]). 574 | 575 | close_manifest() -> 576 | dets:close(?MANIFEST). 577 | 578 | init_from_files(State) -> 579 | %% TODO: do this in parallel 580 | dets_fold_tabnames(fun init_from_file/2, State). 581 | 582 | init_from_file(TabName, State) -> 583 | FullPrefix = dets_tabname_to_prefix(TabName), 584 | FileName = dets_file(State#state.data_root, FullPrefix), 585 | {ok, TabName} = dets:open_file(TabName, [{file, FileName}]), 586 | TabId = init_ets(FullPrefix), 587 | TabId = dets:to_ets(TabName, TabId), 588 | State. 589 | 590 | ets_tab(FullPrefix) -> 591 | case ets:lookup(?ETS, FullPrefix) of 592 | [] -> undefined; 593 | [{FullPrefix, TabId}] -> TabId 594 | end. 595 | 596 | maybe_init_ets(FullPrefix) -> 597 | case ets_tab(FullPrefix) of 598 | undefined -> init_ets(FullPrefix); 599 | _TabId -> ok 600 | end. 601 | 602 | init_ets(FullPrefix) -> 603 | TabId = new_ets_tab(), 604 | ets:insert(?ETS, [{FullPrefix, TabId}]), 605 | TabId. 606 | 607 | new_ets_tab() -> 608 | ets:new(undefined, [{read_concurrency, true}, {write_concurrency, true}]). 609 | 610 | maybe_init_dets(FullPrefix, DataRoot) -> 611 | case dets:info(dets_tabname(FullPrefix)) of 612 | undefined -> init_dets(FullPrefix, DataRoot); 613 | _ -> ok 614 | end. 615 | 616 | init_dets(FullPrefix, DataRoot) -> 617 | TabName = dets_tabname(FullPrefix), 618 | FileName = dets_file(DataRoot, FullPrefix), 619 | {ok, TabName} = dets:open_file(TabName, [{file, FileName}]), 620 | dets_insert(?MANIFEST, [{FullPrefix, TabName, FileName}]). 621 | 622 | close_dets_tabs() -> 623 | dets_fold_tabnames(fun close_dets_tab/2, undefined). 624 | 625 | close_dets_tab(TabName, _Acc) -> 626 | dets:close(TabName). 627 | 628 | dets_insert(TabName, Objs) -> 629 | ok = dets:insert(TabName, Objs), 630 | ok = dets:sync(TabName). 631 | 632 | dets_tabname(FullPrefix) -> {?MODULE, FullPrefix}. 633 | dets_tabname_to_prefix({?MODULE, FullPrefix}) -> FullPrefix. 634 | 635 | dets_file(DataRoot, FullPrefix) -> 636 | filename:join(DataRoot, dets_filename(FullPrefix)). 637 | 638 | dets_filename({Prefix, SubPrefix}=FullPrefix) -> 639 | MD5Prefix = dets_filename_part(Prefix), 640 | MD5SubPrefix = dets_filename_part(SubPrefix), 641 | Trailer = dets_filename_trailer(FullPrefix), 642 | io_lib:format("~s-~s-~s.dets", [MD5Prefix, MD5SubPrefix, Trailer]). 643 | 644 | dets_filename_part(Part) when is_atom(Part) -> 645 | dets_filename_part(list_to_binary(atom_to_list(Part))); 646 | dets_filename_part(Part) when is_binary(Part) -> 647 | <> = crypto:hash(md5, (Part)), 648 | integer_to_list(MD5Int, 16). 649 | 650 | dets_filename_trailer(FullPrefix) -> 651 | [dets_filename_trailer_part(Part) || Part <- tuple_to_list(FullPrefix)]. 652 | 653 | dets_filename_trailer_part(Part) when is_atom(Part) -> 654 | "1"; 655 | dets_filename_trailer_part(Part) when is_binary(Part)-> 656 | "0". 657 | 658 | dets_fold_tabnames(Fun, Acc0) -> 659 | dets:foldl(fun({_FullPrefix, TabName, _FileName}, Acc) -> 660 | Fun(TabName, Acc) 661 | end, Acc0, ?MANIFEST). 662 | 663 | data_root(Opts) -> 664 | case proplists:get_value(data_dir, Opts) of 665 | undefined -> default_data_root(); 666 | Root -> Root 667 | end. 668 | 669 | default_data_root() -> 670 | case application:get_env(plumtree, plumtree_data_dir) of 671 | {ok, PRoot} -> filename:join(PRoot, "cluster_meta"); 672 | undefined -> undefined 673 | end. 674 | --------------------------------------------------------------------------------