├── priv ├── .empty_for_hg └── mapred_builtins.js ├── include ├── .empty_for_hg ├── riak_kv_map_phase.hrl ├── riak_kv_js_pools.hrl └── riak_kv_vnode.hrl ├── rebar ├── .hgignore ├── .gitignore ├── Makefile ├── .hgtags ├── test ├── put_fsm_precommit.js ├── backend_eqc.erl └── fsm_eqc_util.erl ├── rebar.config ├── src ├── riak_kv_phase_proto.erl ├── riak_kv_wm_ping.erl ├── raw_link_walker.erl ├── riak_kv_js_sup.erl ├── riak_kv_pb_socket_sup.erl ├── lk.erl ├── riak_kv_keylister_sup.erl ├── riak_kv_mapper_sup.erl ├── riak_kv_delete_sup.erl ├── riak_kv_get_fsm_sup.erl ├── riak_kv_put_fsm_sup.erl ├── riak_kv_keys_fsm_sup.erl ├── riak_kv_cinfo.erl ├── riak_kv_wm_raw.hrl ├── riak_kv_pb_listener.erl ├── riak_kv_web.erl ├── riak_kv_keylister_master.erl ├── riak_kv_mapred_cache.erl ├── riak_kv_mapred_planner.erl ├── riak_kv_wm_stats.erl ├── riak_kv_mapred_term.erl ├── riak_kv_test_util.erl ├── riak_kv_backend.erl ├── riak_kv_reduce_phase.erl ├── riak_kv_keylister.erl ├── riak_kv_app.erl ├── riak_kv_status.erl ├── riak_kv_dets_backend.erl ├── riak_kv_ets_backend.erl ├── riak_kv_sup.erl ├── riak_kv_put_core.erl ├── riak_kv_gb_trees_backend.erl ├── riak_kv_util.erl ├── riak_kv_delete.erl ├── riak_kv_backup.erl ├── riak_kv_legacy_vnode.erl ├── riak.erl ├── riak_kv_mapred_query.erl ├── riak_kv_wm_mapred.erl ├── riak_kv_bitcask_backend.erl ├── riak_kv_js_manager.erl ├── riak_kv_map_master.erl └── riak_kv_get_core.erl ├── README.org └── ebin └── riak_kv.app /priv/.empty_for_hg: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /include/.empty_for_hg: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /rebar: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/b/riak_kv/master/rebar -------------------------------------------------------------------------------- /.hgignore: -------------------------------------------------------------------------------- 1 | syntax regex 2 | .*~ 3 | ^ebin/.*.beam 4 | include/riak_core_pb.hrl 5 | ^.eunit -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | .eunit/* 2 | deps/* 3 | priv/* 4 | *.o 5 | *.beam 6 | doc 7 | test.*-temp-data 8 | -------------------------------------------------------------------------------- /include/riak_kv_map_phase.hrl: -------------------------------------------------------------------------------- 1 | -record(riak_kv_map_input, {bkey, 2 | bprops, 3 | kd, 4 | preflist}). 5 | -------------------------------------------------------------------------------- /include/riak_kv_js_pools.hrl: -------------------------------------------------------------------------------- 1 | -define(JSPOOL_HOOK, riak_kv_js_hook). 2 | -define(JSPOOL_MAP, riak_kv_js_map). 3 | -define(JSPOOL_REDUCE, riak_kv_js_reduce). 4 | 5 | %% this list is used to inform the js_reload command of all JS VM 6 | %% managers that need to reload 7 | -define(JSPOOL_LIST, [?JSPOOL_HOOK, 8 | ?JSPOOL_MAP, 9 | ?JSPOOL_REDUCE]). 10 | 11 | -------------------------------------------------------------------------------- /Makefile: -------------------------------------------------------------------------------- 1 | 2 | 3 | .PHONY: deps 4 | 5 | all: deps compile 6 | 7 | compile: 8 | ./rebar compile 9 | 10 | deps: 11 | ./rebar get-deps 12 | 13 | clean: 14 | ./rebar clean 15 | rm -rf test.*-temp-data 16 | 17 | distclean: clean 18 | ./rebar delete-deps 19 | 20 | eunit: 21 | ./rebar skip_deps=true eunit 22 | 23 | docs: 24 | ./rebar skip_deps=true doc 25 | 26 | dialyzer: compile 27 | @dialyzer -Wno_return -c apps/riak_kv/ebin 28 | 29 | 30 | -------------------------------------------------------------------------------- /.hgtags: -------------------------------------------------------------------------------- 1 | e716ebd150ff8698a89a1ae28bc868385a164497 riak_kv-0.13.0rc1 2 | 04f5cfd0b8ca7c195e67658367afa5625c186218 riak_kv-0.13.0rc2 3 | a5e0a7b843b52fe846b8006543d1484b548b9a18 riak_kv-0.13.0rc3 4 | cda3363808baaae930bd21cc160f02d66a1b0e45 riak_kv-0.13.0rc5 5 | 3a8621c61c539b47e391ae413deec196b5825fd8 riak_kv-0.13.0rc6 6 | 3169321d36e79b4149fe8868ac079befb3fa9fa1 riak_kv-0.13.0rc7 7 | 00e6b6e1e9a286fced87b2f8772fc7e716f47334 riak_kv-0.13.0rc8 8 | c21191095c7f75fe7651930b9b0ee891852fdf7a riak_kv-0.13.0rc9 9 | 3fd46b5a5e9c2c8461e0de1837d38d9dbf04632c riak_kv-0.13.0 10 | -------------------------------------------------------------------------------- /test/put_fsm_precommit.js: -------------------------------------------------------------------------------- 1 | function precommit_noop(object) 2 | { 3 | return object; 4 | } 5 | 6 | function precommit_append_value(object) 7 | { 8 | upd_data = object.values[[0]].data + "_precommit_hook_was_here"; 9 | object.values[[0]].data = upd_data; 10 | return object; 11 | } 12 | 13 | function precommit_nonobj(object) 14 | { 15 | return "not_an_obj"; 16 | } 17 | 18 | function precommit_fail(object) 19 | { 20 | return "fail"; 21 | } 22 | 23 | function precommit_fail_reason(object) 24 | { 25 | return {"fail":"the hook says no"}; 26 | } 27 | 28 | function precommit_crash(object) 29 | { 30 | throw "wobbler"; 31 | } 32 | 33 | function postcommit_ok(object) 34 | { 35 | return ok; 36 | } 37 | 38 | function postcommit_crash(object) 39 | { 40 | throw "postcommit_crash"; 41 | } -------------------------------------------------------------------------------- /rebar.config: -------------------------------------------------------------------------------- 1 | {require_otp_vsn, "R13B04|R14"}. 2 | {cover_enabled, true}. 3 | {erl_opts, [debug_info, fail_on_warning]}. 4 | 5 | {erl_first_files, ["src/riak_kv_backend.erl"]}. 6 | 7 | {deps, [ 8 | {riak_core, "0.14.0", {git, "git://github.com/basho/riak_core", 9 | {branch, "master"}}}, 10 | {riakc, "1.1.0", {git, "git://github.com/basho/riak-erlang-client", 11 | {branch, "master"}}}, 12 | {luke, "0.2.4", {git, "git://github.com/basho/luke", 13 | {branch, "master"}}}, 14 | {erlang_js, "0.6.1", {git, "git://github.com/basho/erlang_js", 15 | {branch, "master"}}}, 16 | {bitcask, "1.1.6", {git, "git://github.com/basho/bitcask", 17 | {branch, "master"}}}, 18 | {ebloom, "1.0.2", {git, "git://github.com/basho/ebloom", 19 | {branch, "master"}}}, 20 | {eper, ".*", {git, "git://github.com/basho/eper.git", 21 | {branch, "master"}}} 22 | ]}. 23 | -------------------------------------------------------------------------------- /src/riak_kv_phase_proto.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_phase_proto: prototype for riak dataflow phases 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc prototype for riak dataflow phases 24 | 25 | -module(riak_kv_phase_proto). 26 | 27 | -export([mapexec_result/5, 28 | mapexec_error/3]). 29 | 30 | mapexec_result(PhasePid, VNode, BKey, Result, Id) -> 31 | gen_fsm:send_event(PhasePid, {mapexec_reply, VNode, BKey, Result, Id}). 32 | 33 | mapexec_error(PhasePid, Error, Id) -> 34 | gen_fsm:send_event(PhasePid, {mapexec_error, Id, Error}). 35 | 36 | -------------------------------------------------------------------------------- /src/riak_kv_wm_ping.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_kv_wm_ping: simple Webmachine resource for availability test 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc simple Webmachine resource for availability test 24 | 25 | -module(riak_kv_wm_ping). 26 | -author('Dave Smith '). 27 | 28 | %% webmachine resource exports 29 | -export([ 30 | init/1, 31 | to_html/2 32 | ]). 33 | 34 | -include_lib("webmachine/include/webmachine.hrl"). 35 | 36 | init([]) -> 37 | {ok, undefined}. 38 | 39 | to_html(ReqData, Ctx) -> 40 | {"OK", ReqData, Ctx}. 41 | -------------------------------------------------------------------------------- /src/raw_link_walker.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% raw_link_walker: Backwards compatibility module for link traversal 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | -module(raw_link_walker). 23 | 24 | -export([mapreduce_linkfun/3]). 25 | 26 | %% 27 | %% NOTICE 28 | %% 29 | %% This module is STRICTLY a pass-thru for backward compatibility. DO NOT 30 | %% add/modify this code! 31 | %% 32 | %% We use the explicit syntax for invoking riak_kv_wm_link_walker to ensure 33 | %% that code loading works appropriately. 34 | %% 35 | 36 | mapreduce_linkfun(Object, Term, Bucket) -> 37 | riak_kv_wm_link_walker:mapreduce_linkfun(Object, Term, Bucket). 38 | -------------------------------------------------------------------------------- /src/riak_kv_js_sup.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_js_sup: supervise JavaScript VMs 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc supervise JavaScript VMs 24 | 25 | -module(riak_kv_js_sup). 26 | -behaviour(supervisor). 27 | -export([start_link/0, init/1, stop/1]). 28 | -export([start_js/2]). 29 | 30 | start_js(Manager, Pool) when is_pid(Manager) -> 31 | supervisor:start_child(?MODULE, [Manager, Pool]). 32 | 33 | start_link() -> 34 | supervisor:start_link({local, ?MODULE}, ?MODULE, []). 35 | 36 | stop(_S) -> ok. 37 | 38 | %% @private 39 | init([]) -> 40 | {ok, 41 | {{simple_one_for_one, 10, 10}, 42 | [{undefined, 43 | {riak_kv_js_vm, start_link, []}, 44 | temporary, 2000, worker, [riak_kv_js_vm]}]}}. 45 | -------------------------------------------------------------------------------- /src/riak_kv_pb_socket_sup.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_kv_pb_socket_sup: supervise riak_kv_pb_socket processes 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc supervise riak_kv_pb_socket processes 24 | 25 | -module(riak_kv_pb_socket_sup). 26 | -behaviour(supervisor). 27 | -export([start_link/0, init/1, stop/1]). 28 | -export([start_socket/0]). 29 | 30 | start_socket() -> 31 | supervisor:start_child(?MODULE, []). 32 | 33 | start_link() -> 34 | supervisor:start_link({local, ?MODULE}, ?MODULE, []). 35 | 36 | stop(_S) -> ok. 37 | 38 | %% @private 39 | init([]) -> 40 | {ok, 41 | {{simple_one_for_one, 10, 10}, 42 | [{undefined, 43 | {riak_kv_pb_socket, start_link, []}, 44 | temporary, brutal_kill, worker, [riak_kv_pb_socket]}]}}. 45 | -------------------------------------------------------------------------------- /src/lk.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% lk: Helper functions for list keys 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | -module(lk). 23 | 24 | -export([fsm/1]). 25 | 26 | fsm(Bucket) -> 27 | ReqId = random:uniform(10000), 28 | Start = erlang:now(), 29 | riak_kv_keys_fsm:start_link(ReqId, Bucket, 60000, plain, 0.0001, self()), 30 | {ok, Count} = gather_fsm_results(ReqId, 0), 31 | End = erlang:now(), 32 | Ms = erlang:round(timer:now_diff(End, Start) / 1000), 33 | io:format("Found ~p keys in ~pms.~n", [Count, Ms]). 34 | 35 | gather_fsm_results(ReqId, Count) -> 36 | receive 37 | {ReqId, {keys, Keys}} -> 38 | gather_fsm_results(ReqId, Count + length(Keys)); 39 | {ReqId, done} -> 40 | {ok, Count} 41 | after 120000 -> 42 | {error, timeout} 43 | end. 44 | -------------------------------------------------------------------------------- /README.org: -------------------------------------------------------------------------------- 1 | * riak_kv 2 | ** Overview 3 | Riak KV is an open source Erlang application that is distributed using the [[https://github.com/basho/riak_core][riak_core]] Erlang 4 | library. Riak KV provides a key/value datastore and features MapReduce, lightweight data relations, and several different client APIs. 5 | 6 | ** Quick Start 7 | You must have [[http://erlang.org/download.html][Erlang/OTP R13B04]] or later and a GNU-style build 8 | system to compile and run =riak_kv=. The easiest way to utilize riak_kv is by installing the full 9 | Riak application available on [[https://github.com/basho/riak][Github]]. 10 | 11 | ** Contributing 12 | We encourage contributions to =riak_kv= from the community. 13 | 14 | 1) Fork the =riak_kv= repository on [[https://github.com/basho/riak_kv][Github]]. 15 | 2) Clone your fork or add the remote if you already have a clone of 16 | the repository. 17 | #+BEGIN_SRC shell 18 | git clone git@github.com:yourusername/riak_kv.git 19 | # or 20 | git remote add mine git@github.com:yourusername/riak_kv.git 21 | #+END_SRC 22 | 3) Create a topic branch for your change. 23 | #+BEGIN_SRC shell 24 | git checkout -b some-topic-branch 25 | #+END_SRC 26 | 4) Make your change and commit. Use a clear and descriptive commit 27 | message, spanning multiple lines if detailed explanation is 28 | needed. 29 | 5) Push to your fork of the repository and then send a pull-request 30 | through Github. 31 | #+BEGIN_SRC shell 32 | git push mine some-topic-branch 33 | #+END_SRC 34 | 6) A Basho engineer or community maintainer will review your patch 35 | and merge it into the main repository or send you feedback. 36 | -------------------------------------------------------------------------------- /include/riak_kv_vnode.hrl: -------------------------------------------------------------------------------- 1 | -include_lib("riak_core/include/riak_core_vnode.hrl"). 2 | 3 | -record(riak_kv_put_req_v1, { 4 | bkey :: {binary(),binary()}, 5 | object :: term(), 6 | req_id :: non_neg_integer(), 7 | start_time :: non_neg_integer(), 8 | options :: list()}). 9 | 10 | -record(riak_kv_get_req_v1, { 11 | bkey :: {binary(), binary()}, 12 | req_id :: non_neg_integer()}). 13 | 14 | -record(riak_kv_mget_req_v1, { 15 | bkeys :: list({binary(), binary()}), 16 | req_id :: non_neg_integer(), 17 | from :: term()}). 18 | 19 | -record(riak_kv_listkeys_req_v1, { 20 | bucket :: binary(), 21 | req_id :: non_neg_integer()}). 22 | 23 | -record(riak_kv_listkeys_req_v2, { 24 | bucket :: binary()|'_'|tuple(), 25 | req_id :: non_neg_integer(), 26 | caller :: pid()}). 27 | 28 | -record(riak_kv_delete_req_v1, { 29 | bkey :: {binary(), binary()}, 30 | req_id :: non_neg_integer()}). 31 | 32 | -record(riak_kv_map_req_v1, { 33 | bkey :: {binary(), binary()}, 34 | qterm :: term(), 35 | keydata :: term(), 36 | from :: term()}). 37 | 38 | -record(riak_kv_vclock_req_v1, { 39 | bkeys = [] :: [{binary(), binary()}] 40 | }). 41 | 42 | -define(KV_PUT_REQ, #riak_kv_put_req_v1). 43 | -define(KV_GET_REQ, #riak_kv_get_req_v1). 44 | -define(KV_MGET_REQ, #riak_kv_mget_req_v1). 45 | -define(KV_LISTKEYS_REQ, #riak_kv_listkeys_req_v2). 46 | -define(KV_DELETE_REQ, #riak_kv_delete_req_v1). 47 | -define(KV_MAP_REQ, #riak_kv_map_req_v1). 48 | -define(KV_VCLOCK_REQ, #riak_kv_vclock_req_v1). 49 | -------------------------------------------------------------------------------- /src/riak_kv_keylister_sup.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_kv_keylister_sup: Supervisor for starting keylister processes 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | -module(riak_kv_keylister_sup). 23 | 24 | -behaviour(supervisor). 25 | 26 | %% API 27 | -export([start_link/0, 28 | new_lister/3]). 29 | 30 | %% Supervisor callbacks 31 | -export([init/1]). 32 | 33 | new_lister(ReqId, Bucket, Caller) -> 34 | start_child([ReqId, Bucket, Caller]). 35 | 36 | start_link() -> 37 | supervisor:start_link({local, ?MODULE}, ?MODULE, []). 38 | 39 | init([]) -> 40 | SupFlags = {simple_one_for_one, 0, 1}, 41 | Process = {undefined, 42 | {riak_kv_keylister, start_link, []}, 43 | temporary, brutal_kill, worker, dynamic}, 44 | {ok, {SupFlags, [Process]}}. 45 | 46 | %% Internal functions 47 | start_child(Args) -> 48 | supervisor:start_child(?MODULE, Args). 49 | -------------------------------------------------------------------------------- /src/riak_kv_mapper_sup.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_kv_mapper_sup: Supervisor for starting mapper processes 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | -module(riak_kv_mapper_sup). 23 | 24 | -behaviour(supervisor). 25 | 26 | %% API 27 | -export([start_link/0, 28 | new_mapper/5]). 29 | 30 | %% Supervisor callbacks 31 | -export([init/1]). 32 | 33 | new_mapper({_, Node}=VNode, Id, QTerm, MapInputs, PhasePid) -> 34 | start_child(Node, [VNode, Id, QTerm, MapInputs, PhasePid]). 35 | 36 | start_link() -> 37 | supervisor:start_link({local, ?MODULE}, ?MODULE, []). 38 | 39 | init([]) -> 40 | SupFlags = {simple_one_for_one, 0, 1}, 41 | Process = {undefined, 42 | {riak_kv_mapper, start_link, []}, 43 | temporary, brutal_kill, worker, dynamic}, 44 | {ok, {SupFlags, [Process]}}. 45 | 46 | %% Internal functions 47 | start_child(Node, Args) -> 48 | supervisor:start_child({?MODULE, Node}, Args). 49 | -------------------------------------------------------------------------------- /src/riak_kv_delete_sup.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_kv_delete_sup: supervise the riak_kv delete state machines. 4 | %% 5 | %% Copyright (c) 2007-2011 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc supervise the riak_kv delete state machines 24 | 25 | -module(riak_kv_delete_sup). 26 | 27 | -behaviour(supervisor). 28 | 29 | -export([start_delete/2]). 30 | -export([start_link/0]). 31 | -export([init/1]). 32 | 33 | start_delete(Node, Args) -> 34 | supervisor:start_child({?MODULE, Node}, Args). 35 | 36 | %% @spec start_link() -> ServerRet 37 | %% @doc API for starting the supervisor. 38 | start_link() -> 39 | supervisor:start_link({local, ?MODULE}, ?MODULE, []). 40 | 41 | %% @spec init([]) -> SupervisorTree 42 | %% @doc supervisor callback. 43 | init([]) -> 44 | DeleteSpec = {undefined, 45 | {riak_kv_delete, start_link, []}, 46 | temporary, 5000, worker, [riak_kv_delete]}, 47 | 48 | {ok, {{simple_one_for_one, 10, 10}, [DeleteSpec]}}. 49 | -------------------------------------------------------------------------------- /src/riak_kv_get_fsm_sup.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_kv_get_fsm_sup: supervise the riak_kv get state machines. 4 | %% 5 | %% Copyright (c) 2007-2011 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc supervise the riak_kv get state machines 24 | 25 | -module(riak_kv_get_fsm_sup). 26 | 27 | -behaviour(supervisor). 28 | 29 | -export([start_get_fsm/2]). 30 | -export([start_link/0]). 31 | -export([init/1]). 32 | 33 | start_get_fsm(Node, Args) -> 34 | supervisor:start_child({?MODULE, Node}, Args). 35 | 36 | %% @spec start_link() -> ServerRet 37 | %% @doc API for starting the supervisor. 38 | start_link() -> 39 | supervisor:start_link({local, ?MODULE}, ?MODULE, []). 40 | 41 | %% @spec init([]) -> SupervisorTree 42 | %% @doc supervisor callback. 43 | init([]) -> 44 | GetFsmSpec = {undefined, 45 | {riak_kv_get_fsm, start_link, []}, 46 | temporary, 5000, worker, [riak_kv_get_fsm]}, 47 | 48 | {ok, {{simple_one_for_one, 10, 10}, [GetFsmSpec]}}. 49 | -------------------------------------------------------------------------------- /src/riak_kv_put_fsm_sup.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_kv_put_fsm_sup: supervise the riak_kv put state machines. 4 | %% 5 | %% Copyright (c) 2007-2011 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc supervise the riak_kv put state machines 24 | 25 | -module(riak_kv_put_fsm_sup). 26 | 27 | -behaviour(supervisor). 28 | 29 | -export([start_put_fsm/2]). 30 | -export([start_link/0]). 31 | -export([init/1]). 32 | 33 | start_put_fsm(Node, Args) -> 34 | supervisor:start_child({?MODULE, Node}, Args). 35 | 36 | %% @spec start_link() -> ServerRet 37 | %% @doc API for starting the supervisor. 38 | start_link() -> 39 | supervisor:start_link({local, ?MODULE}, ?MODULE, []). 40 | 41 | %% @spec init([]) -> SupervisorTree 42 | %% @doc supervisor callback. 43 | init([]) -> 44 | PutFsmSpec = {undefined, 45 | {riak_kv_put_fsm, start_link, []}, 46 | temporary, 5000, worker, [riak_kv_put_fsm]}, 47 | 48 | {ok, {{simple_one_for_one, 10, 10}, [PutFsmSpec]}}. 49 | -------------------------------------------------------------------------------- /src/riak_kv_keys_fsm_sup.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_kv_keys_fsm_sup: supervise the riak_kv keys state machines. 4 | %% 5 | %% Copyright (c) 2007-2011 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc supervise the riak_kv keys state machines 24 | 25 | -module(riak_kv_keys_fsm_sup). 26 | 27 | -behaviour(supervisor). 28 | 29 | -export([start_keys_fsm/2]). 30 | -export([start_link/0]). 31 | -export([init/1]). 32 | 33 | start_keys_fsm(Node, Args) -> 34 | supervisor:start_child({?MODULE, Node}, Args). 35 | 36 | %% @spec start_link() -> ServerRet 37 | %% @doc API for starting the supervisor. 38 | start_link() -> 39 | supervisor:start_link({local, ?MODULE}, ?MODULE, []). 40 | 41 | %% @spec init([]) -> SupervisorTree 42 | %% @doc supervisor callback. 43 | init([]) -> 44 | KeysFsmSpec = {undefined, 45 | {riak_kv_keys_fsm, start_link, []}, 46 | temporary, 5000, worker, [riak_kv_keys_fsm]}, 47 | 48 | {ok, {{simple_one_for_one, 10, 10}, [KeysFsmSpec]}}. 49 | -------------------------------------------------------------------------------- /src/riak_kv_cinfo.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% Riak: A lightweight, decentralized key-value store. 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | -module(riak_kv_cinfo). 24 | -export([cluster_info_init/0, cluster_info_generator_funs/0]). 25 | 26 | %% @spec () -> term() 27 | %% @doc Required callback function for cluster_info: initialization. 28 | %% 29 | %% This function doesn't have to do anything. 30 | 31 | cluster_info_init() -> 32 | ok. 33 | 34 | %% @spec () -> list({string(), fun()}) 35 | %% @doc Required callback function for cluster_info: return list of 36 | %% {NameForReport, FunOfArity_1} tuples to generate ASCII/UTF-8 37 | %% formatted reports. 38 | 39 | cluster_info_generator_funs() -> 40 | [ 41 | {"Riak KV status", fun status/1}, 42 | {"Riak KV ringready", fun ringready/1}, 43 | {"Riak KV transfers", fun transfers/1} 44 | ]. 45 | 46 | status(CPid) -> % CPid is the data collector's pid. 47 | cluster_info:format(CPid, "~p\n", [riak_kv_status:statistics()]). 48 | 49 | ringready(CPid) -> 50 | cluster_info:format(CPid, "~p\n", [riak_kv_status:ringready()]). 51 | 52 | transfers(CPid) -> 53 | cluster_info:format(CPid, "~p\n", [riak_kv_status:transfers()]). 54 | 55 | -------------------------------------------------------------------------------- /src/riak_kv_wm_raw.hrl: -------------------------------------------------------------------------------- 1 | %% This file is provided to you under the Apache License, 2 | %% Version 2.0 (the "License"); you may not use this file 3 | %% except in compliance with the License. You may obtain 4 | %% a copy of the License at 5 | 6 | %% http://www.apache.org/licenses/LICENSE-2.0 7 | 8 | %% Unless required by applicable law or agreed to in writing, 9 | %% software distributed under the License is distributed on an 10 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 11 | %% KIND, either express or implied. See the License for the 12 | %% specific language governing permissions and limitations 13 | %% under the License. 14 | 15 | %% Constants used by the raw_http resources 16 | 17 | %% Names of riak_object metadata fields 18 | -define(MD_CTYPE, <<"content-type">>). 19 | -define(MD_CHARSET, <<"charset">>). 20 | -define(MD_ENCODING, <<"content-encoding">>). 21 | -define(MD_VTAG, <<"X-Riak-VTag">>). 22 | -define(MD_LINKS, <<"Links">>). 23 | -define(MD_LASTMOD, <<"X-Riak-Last-Modified">>). 24 | -define(MD_USERMETA, <<"X-Riak-Meta">>). 25 | 26 | %% Names of HTTP header fields 27 | -define(HEAD_CTYPE, "Content-Type"). 28 | -define(HEAD_VCLOCK, "X-Riak-Vclock"). 29 | -define(HEAD_LINK, "Link"). 30 | -define(HEAD_ENCODING, "Content-Encoding"). 31 | -define(HEAD_CLIENT, "X-Riak-ClientId"). 32 | -define(HEAD_USERMETA_PREFIX, "x-riak-meta-"). 33 | 34 | %% Names of JSON fields in bucket properties 35 | -define(JSON_PROPS, <<"props">>). 36 | -define(JSON_BUCKETS, <<"buckets">>). 37 | -define(JSON_KEYS, <<"keys">>). 38 | -define(JSON_LINKFUN, <<"linkfun">>). 39 | -define(JSON_MOD, <<"mod">>). 40 | -define(JSON_FUN, <<"fun">>). 41 | -define(JSON_CHASH, <<"chash_keyfun">>). 42 | -define(JSON_JSFUN, <<"jsfun">>). 43 | -define(JSON_JSANON, <<"jsanon">>). 44 | -define(JSON_JSBUCKET, <<"bucket">>). 45 | -define(JSON_JSKEY, <<"key">>). 46 | -define(JSON_ALLOW_MULT, <<"allow_mult">>). 47 | 48 | 49 | %% Names of HTTP query parameters 50 | -define(Q_PROPS, "props"). 51 | -define(Q_BUCKETS, "buckets"). 52 | -define(Q_KEYS, "keys"). 53 | -define(Q_FALSE, "false"). 54 | -define(Q_TRUE, "true"). 55 | -define(Q_STREAM, "stream"). 56 | -define(Q_VTAG, "vtag"). 57 | -define(Q_RETURNBODY, "returnbody"). 58 | -------------------------------------------------------------------------------- /src/riak_kv_pb_listener.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_kv_pb_listener: Listen for protocol buffer clients 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc entry point for TCP-based protocol buffers service 24 | 25 | -module(riak_kv_pb_listener). 26 | -behavior(gen_nb_server). 27 | -export([start_link/0]). 28 | -export([init/1, handle_call/3, handle_cast/2, handle_info/2, 29 | terminate/2, code_change/3]). 30 | -export([sock_opts/0, new_connection/2]). 31 | -record(state, {portnum}). 32 | 33 | start_link() -> 34 | PortNum = app_helper:get_env(riak_kv, pb_port), 35 | IpAddr = app_helper:get_env(riak_kv, pb_ip), 36 | gen_nb_server:start_link(?MODULE, IpAddr, PortNum, [PortNum]). 37 | 38 | init([PortNum]) -> 39 | {ok, #state{portnum=PortNum}}. 40 | 41 | sock_opts() -> 42 | BackLog = app_helper:get_env(riak_kv, pb_backlog, 5), 43 | [binary, {packet, 4}, {reuseaddr, true}, {backlog, BackLog}]. 44 | 45 | handle_call(_Req, _From, State) -> 46 | {reply, not_implemented, State}. 47 | 48 | handle_cast(_Msg, State) -> {noreply, State}. 49 | 50 | handle_info(_Info, State) -> {noreply, State}. 51 | 52 | terminate(_Reason, _State) -> ok. 53 | 54 | code_change(_OldVsn, State, _Extra) -> {ok, State}. 55 | 56 | new_connection(Socket, State) -> 57 | {ok, Pid} = riak_kv_pb_socket_sup:start_socket(), 58 | ok = gen_tcp:controlling_process(Socket, Pid), 59 | ok = riak_kv_pb_socket:set_socket(Pid, Socket), 60 | {ok, State}. 61 | 62 | -------------------------------------------------------------------------------- /src/riak_kv_web.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_kv_web: setup Riak's KV HTTP interface 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc Convenience functions for setting up the HTTP interface 24 | %% of Riak. This module loads parameters from the application 25 | %% environment: 26 | %% 27 | %%
raw_name 28 | %%
the base path under which the riak_kv_wm_raw 29 | %% should be exposed; defaulted to "raw" 30 | %%
31 | -module(riak_kv_web). 32 | 33 | -export([dispatch_table/0]). 34 | 35 | dispatch_table() -> 36 | MapredProps = mapred_props(), 37 | StatsProps = stats_props(), 38 | 39 | lists:append( 40 | raw_dispatch(), 41 | [{[proplists:get_value(prefix, MapredProps)], 42 | riak_kv_wm_mapred, MapredProps}, 43 | {[proplists:get_value(prefix, StatsProps)], 44 | riak_kv_wm_stats, StatsProps}, 45 | {["ping"], riak_kv_wm_ping, []}]). 46 | 47 | raw_dispatch() -> 48 | case app_helper:get_env(riak_kv, raw_name) of 49 | undefined -> raw_dispatch("riak"); 50 | Name -> lists:append(raw_dispatch(Name), raw_dispatch("riak")) 51 | end. 52 | 53 | raw_dispatch(Name) -> 54 | Props = raw_props(Name), 55 | [{[Name], riak_kv_wm_raw, Props}, 56 | {[Name, bucket], riak_kv_wm_raw, Props}, 57 | {[Name, bucket, key], riak_kv_wm_raw, Props}, 58 | {[Name, bucket, key, '*'], riak_kv_wm_link_walker, Props}]. 59 | 60 | raw_props(Prefix) -> 61 | [{prefix, Prefix}, {riak, local}]. 62 | 63 | mapred_props() -> 64 | [{prefix, app_helper:get_env(riak_kv, mapred_name, "mapred")}]. 65 | 66 | stats_props() -> 67 | [{prefix, app_helper:get_env(riak_kv, stats_urlpath, "stats")}]. 68 | -------------------------------------------------------------------------------- /src/riak_kv_keylister_master.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_kv_keylister_master: Starts keylister processes on demand 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | -module(riak_kv_keylister_master). 23 | 24 | -behaviour(gen_server). 25 | 26 | %% API 27 | -export([start_link/0, 28 | start_keylist/3, 29 | start_keylist/4]). 30 | 31 | %% gen_server callbacks 32 | -export([init/1, handle_call/3, handle_cast/2, handle_info/2, 33 | terminate/2, code_change/3]). 34 | 35 | -define(SERVER, ?MODULE). 36 | -define(DEFAULT_TIMEOUT, 5000). 37 | 38 | -record(state, {}). 39 | 40 | start_keylist(Node, ReqId, Bucket) -> 41 | start_keylist(Node, ReqId, Bucket, ?DEFAULT_TIMEOUT). 42 | 43 | start_keylist(Node, ReqId, Bucket, Timeout) -> 44 | try 45 | case gen_server:call({?SERVER, Node}, {start_kl, ReqId, self(), Bucket}, Timeout) of 46 | {ok, Pid} -> 47 | {ok, Pid}; 48 | Error -> 49 | Error 50 | end 51 | catch 52 | exit:{timeout, _} -> 53 | {error, timeout} 54 | end. 55 | 56 | 57 | start_link() -> 58 | gen_server:start_link({local, ?SERVER}, ?MODULE, [], []). 59 | 60 | init([]) -> 61 | {ok, #state{}}. 62 | 63 | handle_call({start_kl, ReqId, Caller, Bucket}, _From, State) -> 64 | Reply = riak_kv_keylister_sup:new_lister(ReqId, Caller, Bucket), 65 | {reply, Reply, State}; 66 | 67 | handle_call(_Request, _From, State) -> 68 | {reply, ignore, State}. 69 | 70 | handle_cast(_Msg, State) -> 71 | {noreply, State}. 72 | 73 | handle_info(_Info, State) -> 74 | {noreply, State}. 75 | 76 | terminate(_Reason, _State) -> 77 | ok. 78 | 79 | code_change(_OldVsn, State, _Extra) -> 80 | {ok, State}. 81 | -------------------------------------------------------------------------------- /src/riak_kv_mapred_cache.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_kv_mapred_cache: gen_server to manage starting up and ejecting 4 | %% old data from the MapReduce cache 5 | %% 6 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 7 | %% 8 | %% This file is provided to you under the Apache License, 9 | %% Version 2.0 (the "License"); you may not use this file 10 | %% except in compliance with the License. You may obtain 11 | %% a copy of the License at 12 | %% 13 | %% http://www.apache.org/licenses/LICENSE-2.0 14 | %% 15 | %% Unless required by applicable law or agreed to in writing, 16 | %% software distributed under the License is distributed on an 17 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 18 | %% KIND, either express or implied. See the License for the 19 | %% specific language governing permissions and limitations 20 | %% under the License. 21 | %% 22 | %% ------------------------------------------------------------------- 23 | -module(riak_kv_mapred_cache). 24 | 25 | -behaviour(gen_server). 26 | 27 | %% API 28 | -export([start_link/0, 29 | clear/0, 30 | cache_ref/0, 31 | eject/1]). 32 | 33 | %% gen_server callbacks 34 | -export([init/1, handle_call/3, handle_cast/2, handle_info/2, 35 | terminate/2, code_change/3]). 36 | 37 | -define(SERVER, ?MODULE). 38 | 39 | -record(state, {lru}). 40 | 41 | clear() -> 42 | gen_server:call(?SERVER, clear, infinity). 43 | 44 | eject(BKey) -> 45 | gen_server:cast(?SERVER, {eject, BKey}). 46 | 47 | cache_ref() -> 48 | gen_server:call(?SERVER, cache_ref, infinity). 49 | 50 | start_link() -> 51 | gen_server:start_link({local, ?SERVER}, ?MODULE, [], []). 52 | 53 | init([]) -> 54 | CacheSize = app_helper:get_env(riak_kv, map_cache_size, 5000), 55 | {ok, #state{lru=riak_kv_lru:new(CacheSize)}}. 56 | 57 | handle_call(clear, _From, #state{lru=LRU}=State) -> 58 | riak_kv_lru:clear(LRU), 59 | {reply, ok, State}; 60 | handle_call(cache_ref, _From, #state{lru=LRU}=State) -> 61 | {reply, {ok, LRU}, State}; 62 | handle_call(_Request, _From, State) -> 63 | {reply, ignore, State}. 64 | 65 | handle_cast({eject, BKey}, #state{lru=LRU}=State) -> 66 | riak_kv_lru:clear_bkey(LRU, BKey), 67 | {noreply, State}; 68 | handle_cast(_Msg, State) -> 69 | {noreply, State}. 70 | 71 | handle_info(_Info, State) -> 72 | {noreply, State}. 73 | 74 | terminate(_Reason, _State) -> 75 | ok. 76 | 77 | code_change(_OldVsn, State, _Extra) -> 78 | {ok, State}. 79 | 80 | %% Internal functions 81 | -------------------------------------------------------------------------------- /src/riak_kv_mapred_planner.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_kv_mapred_planner: Plans batched mapreduce processing 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | -module(riak_kv_mapred_planner). 24 | -author('John Muellerleile '). 25 | -author('Kevin Smith '). 26 | 27 | -include("riak_kv_map_phase.hrl"). 28 | 29 | -export([plan_map/1]). 30 | 31 | plan_map(Inputs) -> 32 | build_claim_list(add_map_inputs(Inputs)). 33 | 34 | %% Internal functions 35 | build_claim_list(InputData) -> 36 | {keys, Keys} = lists:keyfind(keys, 1, InputData), 37 | InputData1 = lists:keydelete(keys, 1, InputData), 38 | F = fun({_, KeysA} , {_, KeysB}) -> length(KeysA) =< length(KeysB) end, 39 | PartList0 = lists:sort(F, InputData1), 40 | claim_keys(PartList0, [], Keys). 41 | 42 | claim_keys([], [], _) -> 43 | exit(exhausted_preflist); 44 | claim_keys(_, ClaimList, []) -> 45 | ClaimList; 46 | claim_keys([H|T], ClaimList, Keys) -> 47 | {P, PKeys} = H, 48 | PKeys1 = lists:filter(fun(PK) -> 49 | lists:member(PK, Keys) 50 | end, PKeys), 51 | case PKeys1 == [] of 52 | true -> 53 | claim_keys(T, ClaimList, Keys); 54 | false -> 55 | NewKeys = lists:subtract(Keys, PKeys1), 56 | claim_keys(T, ClaimList ++ [{P, PKeys1}], NewKeys) 57 | end. 58 | 59 | add_map_inputs(Inputs) -> 60 | add_map_inputs(Inputs, [{keys, []}]). 61 | add_map_inputs([], InputData) -> 62 | InputData; 63 | add_map_inputs([#riak_kv_map_input{preflist=PList}=H|T], InputData) -> 64 | {keys, Keys} = lists:keyfind(keys, 1, InputData), 65 | InputData1 = lists:foldl(fun(P, Acc) -> 66 | case lists:keyfind(P, 1, Acc) of 67 | false -> 68 | lists:keystore(P, 1, Acc, {P, [H]}); 69 | {P, PKeys} -> 70 | lists:keystore(P, 1, Acc, {P, PKeys ++ [H]}) 71 | end 72 | end, InputData, PList), 73 | add_map_inputs(T, lists:keystore(keys, 1, InputData1, {keys, Keys ++ [H]})). 74 | -------------------------------------------------------------------------------- /src/riak_kv_wm_stats.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% stats_http_resource: publishing Riak runtime stats via HTTP 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | -module(riak_kv_wm_stats). 24 | -author('Andy Gross '). 25 | 26 | %% webmachine resource exports 27 | -export([ 28 | init/1, 29 | encodings_provided/2, 30 | content_types_provided/2, 31 | service_available/2, 32 | produce_body/2, 33 | pretty_print/2 34 | ]). 35 | 36 | -include_lib("webmachine/include/webmachine.hrl"). 37 | 38 | -record(ctx, {}). 39 | 40 | init(_) -> 41 | {ok, #ctx{}}. 42 | 43 | %% @spec encodings_provided(webmachine:wrq(), context()) -> 44 | %% {[encoding()], webmachine:wrq(), context()} 45 | %% @doc Get the list of encodings this resource provides. 46 | %% "identity" is provided for all methods, and "gzip" is 47 | %% provided for GET as well 48 | encodings_provided(ReqData, Context) -> 49 | case wrq:method(ReqData) of 50 | 'GET' -> 51 | {[{"identity", fun(X) -> X end}, 52 | {"gzip", fun(X) -> zlib:gzip(X) end}], ReqData, Context}; 53 | _ -> 54 | {[{"identity", fun(X) -> X end}], ReqData, Context} 55 | end. 56 | 57 | %% @spec content_types_provided(webmachine:wrq(), context()) -> 58 | %% {[ctype()], webmachine:wrq(), context()} 59 | %% @doc Get the list of content types this resource provides. 60 | %% "application/json" and "text/plain" are both provided 61 | %% for all requests. "text/plain" is a "pretty-printed" 62 | %% version of the "application/json" content. 63 | content_types_provided(ReqData, Context) -> 64 | {[{"application/json", produce_body}, 65 | {"text/plain", pretty_print}], 66 | ReqData, Context}. 67 | 68 | 69 | service_available(ReqData, Ctx) -> 70 | case app_helper:get_env(riak_kv, riak_kv_stat, false) of 71 | false -> 72 | {false, wrq:append_to_response_body("riak_kv_stat is disabled on this node.\n", ReqData), 73 | Ctx}; 74 | true -> 75 | {true, ReqData, Ctx} 76 | end. 77 | 78 | produce_body(ReqData, Ctx) -> 79 | Body = mochijson2:encode({struct, get_stats()}), 80 | {Body, ReqData, Ctx}. 81 | 82 | %% @spec pretty_print(webmachine:wrq(), context()) -> 83 | %% {string(), webmachine:wrq(), context()} 84 | %% @doc Format the respons JSON object is a "pretty-printed" style. 85 | pretty_print(RD1, C1=#ctx{}) -> 86 | {Json, RD2, C2} = produce_body(RD1, C1), 87 | {json_pp:print(binary_to_list(list_to_binary(Json))), RD2, C2}. 88 | 89 | get_stats() -> 90 | proplists:delete(disk, riak_kv_stat:get_stats()). 91 | 92 | 93 | -------------------------------------------------------------------------------- /priv/mapred_builtins.js: -------------------------------------------------------------------------------- 1 | /** Helper functions start **/ 2 | 3 | var RiakHelper = function() { 4 | return { 5 | numericSorter: function(first, second) { 6 | return first - second; 7 | } 8 | }; 9 | }(); 10 | 11 | /** Helper functions end **/ 12 | 13 | var Riak = function() { 14 | 15 | return { 16 | getClassName: function(obj) { 17 | if (obj && obj.constructor && obj.constructor.toString) { 18 | var arr = obj.constructor.toString().match(/function\s*(\w+)/); 19 | if (arr && arr.length == 2) { 20 | return arr[1]; 21 | } 22 | } 23 | return undefined; 24 | }, 25 | filterNotFound: function(values) { 26 | return values.filter(function(value, index, data) { 27 | if (typeof value === 'object') { 28 | return value['not_found'] === undefined; 29 | } 30 | else { 31 | return true; 32 | } }); 33 | }, 34 | mapValues: function(value, keyData, arg) { 35 | if (value["not_found"]) { 36 | return [value]; 37 | } 38 | var data = value["values"][0]["data"]; 39 | if (Riak.getClassName(data) !== "Array") { 40 | return [data]; 41 | } 42 | else { 43 | return data; 44 | }}, 45 | mapValuesJson: function(value, keyData, arg) { 46 | if (value["not_found"]) { 47 | return [value]; 48 | } 49 | var newValues = Riak.mapValues(value, keyData, arg); 50 | return newValues.map(function(nv) { return JSON.parse(nv); }); 51 | }, 52 | mapByFields: function(value, keyData, fields) { 53 | if(!value.not_found) { 54 | var object = Riak.mapValuesJson(value)[0]; 55 | for(field in fields) { 56 | if(object[field] != fields[field]) { 57 | return []; 58 | } 59 | } 60 | return [object]; 61 | } 62 | else { 63 | return []; 64 | } 65 | }, 66 | reduceSum: function(values, arg) { 67 | values = Riak.filterNotFound(values); 68 | if (values.length > 0) { 69 | return [values.reduce(function(prev, curr, index, array) { return prev + curr; } )]; 70 | } 71 | else { 72 | return [0]; 73 | }}, 74 | reduceMin: function(values, arg) { 75 | if(values.length == 0) 76 | return []; 77 | else 78 | return [values.reduce(function(prev,next){ 79 | return (prev < next) ? prev : next; 80 | })]; 81 | }, 82 | reduceMax: function(values, arg) { 83 | if(values.length == 0) 84 | return []; 85 | else 86 | return [values.reduce(function(prev,next){ 87 | return (prev > next) ? prev : next; 88 | })]; 89 | }, 90 | reduceSort: function(value, arg) { 91 | try { 92 | var c = eval(arg); 93 | return value.sort(c); 94 | } 95 | catch (e) { 96 | return value.sort(); 97 | } 98 | }, 99 | reduceNumericSort: function(value, arg) { 100 | value.sort(RiakHelper.numericSorter); 101 | return value; 102 | }, 103 | reduceLimit: function(value, arg) { 104 | return value.slice(0, arg - 1); 105 | }, 106 | reduceSlice: function(value, arg) { 107 | var start = arg[0]; 108 | var end = arg[1]; 109 | if (end > value.length) { 110 | return value; 111 | } 112 | else { 113 | return value.slice(start, end); 114 | } 115 | } 116 | }; 117 | }(); 118 | -------------------------------------------------------------------------------- /ebin/riak_kv.app: -------------------------------------------------------------------------------- 1 | %% -*- tab-width: 4;erlang-indent-level: 4;indent-tabs-mode: nil -*- 2 | %% ex: ts=4 sw=4 et 3 | {application, riak_kv, 4 | [ 5 | {description, "Riak Key/Value Store"}, 6 | {vsn, "0.14.0"}, 7 | {modules, [ 8 | lk, 9 | raw_link_walker, 10 | riak, 11 | riak_client, 12 | riak_kv_app, 13 | riak_kv_backup, 14 | riak_kv_backend, 15 | riak_kv_bitcask_backend, 16 | riak_kv_cache_backend, 17 | riak_kv_cinfo, 18 | riak_kv_console, 19 | riak_kv_delete, 20 | riak_kv_delete_sup, 21 | riak_kv_dets_backend, 22 | riak_kv_ets_backend, 23 | riak_kv_fs_backend, 24 | riak_kv_gb_trees_backend, 25 | riak_kv_get_core, 26 | riak_kv_get_fsm, 27 | riak_kv_get_fsm_sup, 28 | riak_kv_js_manager, 29 | riak_kv_js_sup, 30 | riak_kv_js_vm, 31 | riak_kv_keylister, 32 | riak_kv_keylister_master, 33 | riak_kv_keylister_sup, 34 | riak_kv_keys_fsm, 35 | riak_kv_keys_fsm_sup, 36 | riak_kv_legacy_vnode, 37 | riak_kv_lru, 38 | riak_kv_map_master, 39 | riak_kv_mapper, 40 | riak_kv_mapper_sup, 41 | riak_kv_map_phase, 42 | riak_kv_mapred_cache, 43 | riak_kv_mapred_filters, 44 | riak_kv_mapred_json, 45 | riak_kv_mapred_planner, 46 | riak_kv_mapred_query, 47 | riak_kv_mapred_term, 48 | riak_kv_mapreduce, 49 | riak_kv_multi_backend, 50 | riak_kv_pb_listener, 51 | riak_kv_pb_socket, 52 | riak_kv_pb_socket_sup, 53 | riak_kv_phase_proto, 54 | riak_kv_put_core, 55 | riak_kv_put_fsm, 56 | riak_kv_put_fsm_sup, 57 | riak_kv_reduce_phase, 58 | riak_kv_stat, 59 | riak_kv_status, 60 | riak_kv_sup, 61 | riak_kv_test_util, 62 | riak_kv_util, 63 | riak_kv_vnode, 64 | riak_kv_web, 65 | riak_kv_wm_link_walker, 66 | riak_kv_wm_mapred, 67 | riak_kv_wm_ping, 68 | riak_kv_wm_raw, 69 | riak_kv_wm_stats, 70 | riak_object 71 | ]}, 72 | {applications, [ 73 | kernel, 74 | stdlib, 75 | sasl, 76 | crypto, 77 | riak_core, 78 | luke, 79 | erlang_js, 80 | mochiweb, 81 | webmachine, 82 | os_mon 83 | ]}, 84 | {registered, []}, 85 | {mod, {riak_kv_app, []}}, 86 | {env, [ 87 | %% Endpoint for system stats HTTP provider 88 | {stats_urlpath, "stats"}, 89 | 90 | %% Secondary code paths 91 | {add_paths, []}, 92 | 93 | %% Allow Erlang MapReduce functions to be specified as 94 | %% strings. 95 | %% 96 | %% !!!WARNING!!! 97 | %% This will allow arbitrary Erlang code to be submitted 98 | %% through the REST and Protocol Buffers interfaces. This 99 | %% should only be used for development purposes. 100 | {allow_strfun, false} 101 | ]} 102 | ]}. 103 | -------------------------------------------------------------------------------- /src/riak_kv_mapred_term.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_mapred_term: Term parsing for mapreduce 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc Term parsing for mapreduce 24 | 25 | -module(riak_kv_mapred_term). 26 | 27 | -export([parse_request/1]). 28 | 29 | -define(DEFAULT_TIMEOUT, 60000). 30 | 31 | %% 32 | %% Parse a map/reduce request encoded as a property list 33 | %% [{'inputs', Inputs}, 34 | %% {'query', Query}, 35 | %% {'timeout', Timeout}]. 36 | %% 37 | parse_request(BinReq) -> 38 | try 39 | Req = binary_to_term(BinReq), 40 | 41 | Timeout = proplists:get_value(timeout, Req, ?DEFAULT_TIMEOUT), 42 | Inputs = proplists:get_value(inputs, Req, undefined), 43 | Query = proplists:get_value('query', Req, undefined), 44 | 45 | case {valid_inputs(Inputs), valid_query(Query)} of 46 | {ok, ok} -> 47 | {ok, Inputs, Query, Timeout}; 48 | {{error, Reason}, _} -> 49 | {error, {'inputs', Reason}}; 50 | {_, {error, Reason}} -> 51 | {error, {'query', Reason}} 52 | end 53 | catch 54 | _:Error -> 55 | {error, Error} 56 | end. 57 | 58 | %% Return ok if inputs are valid, {error, Reason} if not 59 | %% @type mapred_inputs() = [bucket_key()] 60 | %% |bucket() 61 | %% |{bucket(), list()} 62 | %% |{modfun, atom(), atom(), list()} 63 | valid_inputs(Bucket) when is_binary(Bucket) -> 64 | ok; 65 | valid_inputs(Targets) when is_list(Targets) -> 66 | valid_input_targets(Targets); 67 | valid_inputs({modfun, Module, Function, _Options}) 68 | when is_atom(Module), is_atom(Function) -> 69 | ok; 70 | valid_inputs({Bucket, Filters}) when is_binary(Bucket), is_list(Filters) -> 71 | ok; 72 | valid_inputs(Invalid) -> 73 | {error, {"Inputs must be a binary bucket, a tuple of bucket and key-filters, a list of target tuples, or a modfun tuple:", Invalid}}. 74 | 75 | %% @type bucket_key() = {binary(), binary()} 76 | %% |{{binary(), binary()}, term()} 77 | valid_input_targets([]) -> 78 | ok; 79 | valid_input_targets([{B,K}|Rest]) when is_binary(B), is_binary(K) -> 80 | valid_input_targets(Rest); 81 | valid_input_targets([{{B,K},_KeyData}|Rest]) when is_binary(B), is_binary(K) -> 82 | valid_input_targets(Rest); 83 | valid_input_targets(Invalid) -> 84 | {error, {"Inputs target tuples must be {B,K} or {{B,K},KeyData}:", Invalid}}. 85 | 86 | %% Return ok if query are valid, {error, Reason} if not. Not very strong validation 87 | %% done here as riak_kv_mapred_query will check this. 88 | valid_query(Query) when is_list(Query) -> 89 | ok; 90 | valid_query(Invalid) -> 91 | {error, {"Query takes a list of step tuples", Invalid}}. 92 | -------------------------------------------------------------------------------- /src/riak_kv_test_util.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_test_util: utilities for test scripts 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc utilities for test scripts 24 | 25 | -module(riak_kv_test_util). 26 | 27 | -ifdef(TEST). 28 | 29 | -export([call_unused_fsm_funs/1, 30 | stop_process/1, 31 | wait_for_pid/1, 32 | wait_for_children/1]). 33 | -include_lib("eunit/include/eunit.hrl"). 34 | 35 | 36 | call_unused_fsm_funs(Mod) -> 37 | Mod:handle_event(event, statename, state), 38 | Mod:handle_sync_event(event, from, stateneame, state), 39 | Mod:handle_info(info, statename, statedata), 40 | Mod:terminate(reason, statename, state), 41 | Mod:code_change(oldvsn, statename, state, extra). 42 | 43 | 44 | 45 | %% Stop a running pid - unlink and exit(kill) the process 46 | %% 47 | stop_process(undefined) -> 48 | ok; 49 | stop_process(RegName) when is_atom(RegName) -> 50 | stop_process(whereis(RegName)); 51 | stop_process(Pid) when is_pid(Pid) -> 52 | unlink(Pid), 53 | exit(Pid, shutdown), 54 | ok = wait_for_pid(Pid). 55 | 56 | %% Wait for a pid to exit 57 | wait_for_pid(Pid) -> 58 | Mref = erlang:monitor(process, Pid), 59 | receive 60 | {'DOWN',Mref,process,_,_} -> 61 | ok 62 | after 63 | 5000 -> 64 | {error, didnotexit} 65 | end. 66 | 67 | %% Wait for children that were spawned with proc_lib. 68 | %% They have an '$ancestors' entry in their dictionary 69 | wait_for_children(PPid) -> 70 | F = fun(CPid) -> 71 | case process_info(CPid, initial_call) of 72 | {initial_call, {proc_lib, init_p, 3}} -> 73 | case process_info(CPid, dictionary) of 74 | {dictionary, Dict} -> 75 | case proplists:get_value('$ancestors', Dict) of 76 | undefined -> 77 | %% Process dictionary not updated yet 78 | true; 79 | Ancestors -> 80 | lists:member(PPid, Ancestors) 81 | end; 82 | undefined -> 83 | %% No dictionary - should be one if proclib spawned it 84 | true 85 | end; 86 | _ -> 87 | %% Not in proc_lib 88 | false 89 | end 90 | end, 91 | case lists:any(F, processes()) of 92 | true -> 93 | timer:sleep(1), 94 | wait_for_children(PPid); 95 | false -> 96 | ok 97 | end. 98 | 99 | -endif. % TEST 100 | -------------------------------------------------------------------------------- /src/riak_kv_backend.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_kv_backend: Riak backend behaviour 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | -module(riak_kv_backend). 24 | -export([behaviour_info/1]). 25 | -export([callback_after/3]). 26 | -ifdef(TEST). 27 | -include_lib("eunit/include/eunit.hrl"). 28 | -export([standard_test/2]). 29 | -endif. 30 | 31 | -spec behaviour_info(atom()) -> 'undefined' | [{atom(), arity()}]. 32 | behaviour_info(callbacks) -> 33 | [{start,2}, % (Partition, Config) 34 | {stop,1}, % (State) 35 | {get,2}, % (State, BKey) 36 | {put,3}, % (State, BKey, Val) 37 | {list,1}, % (State) 38 | {list_bucket,2}, % (State, Bucket) 39 | {delete,2}, % (State, BKey) 40 | {drop,1}, % (State) 41 | {fold,3}, % (State, Folder, Acc), Folder({B,K},V,Acc) 42 | {is_empty,1}, % (State) 43 | {callback,3}]; % (State, Ref, Msg) -> 44 | behaviour_info(_Other) -> 45 | undefined. 46 | 47 | %% Queue a callback for the backend after Time ms. 48 | -spec callback_after(integer(), reference(), term()) -> reference(). 49 | callback_after(Time, Ref, Msg) when is_integer(Time), is_reference(Ref) -> 50 | riak_core_vnode:send_command_after(Time, {backend_callback, Ref, Msg}). 51 | 52 | -ifdef(TEST). 53 | 54 | standard_test(BackendMod, Config) -> 55 | {ok, S} = BackendMod:start(42, Config), 56 | ?assertEqual(ok, BackendMod:put(S,{<<"b1">>,<<"k1">>},<<"v1">>)), 57 | ?assertEqual(ok, BackendMod:put(S,{<<"b2">>,<<"k2">>},<<"v2">>)), 58 | ?assertEqual({ok,<<"v2">>}, BackendMod:get(S,{<<"b2">>,<<"k2">>})), 59 | ?assertEqual({error, notfound}, BackendMod:get(S, {<<"b1">>,<<"k3">>})), 60 | ?assertEqual([{<<"b1">>,<<"k1">>},{<<"b2">>,<<"k2">>}], 61 | lists:sort(BackendMod:list(S))), 62 | ?assertEqual([<<"k2">>], BackendMod:list_bucket(S, <<"b2">>)), 63 | ?assertEqual([<<"k1">>], BackendMod:list_bucket(S, <<"b1">>)), 64 | ?assertEqual([<<"k1">>], BackendMod:list_bucket( 65 | S, {filter, <<"b1">>, fun(_K) -> true end})), 66 | ?assertEqual([], BackendMod:list_bucket( 67 | S, {filter, <<"b1">>, fun(_K) -> false end})), 68 | BucketList = BackendMod:list_bucket(S, '_'), 69 | ?assert(lists:member(<<"b1">>, BucketList)), 70 | ?assert(lists:member(<<"b2">>, BucketList)), 71 | ?assertEqual(ok, BackendMod:delete(S,{<<"b2">>,<<"k2">>})), 72 | ?assertEqual({error, notfound}, BackendMod:get(S, {<<"b2">>, <<"k2">>})), 73 | ?assertEqual([{<<"b1">>, <<"k1">>}], BackendMod:list(S)), 74 | Folder = fun(K, V, A) -> [{K,V}|A] end, 75 | ?assertEqual([{{<<"b1">>,<<"k1">>},<<"v1">>}], BackendMod:fold(S, Folder, [])), 76 | ?assertEqual(ok, BackendMod:put(S,{<<"b3">>,<<"k3">>},<<"v3">>)), 77 | ?assertEqual([{{<<"b1">>,<<"k1">>},<<"v1">>}, 78 | {{<<"b3">>,<<"k3">>},<<"v3">>}], lists:sort(BackendMod:fold(S, Folder, []))), 79 | ?assertEqual(false, BackendMod:is_empty(S)), 80 | ?assertEqual(ok, BackendMod:delete(S,{<<"b1">>,<<"k1">>})), 81 | ?assertEqual(ok, BackendMod:delete(S,{<<"b3">>,<<"k3">>})), 82 | ?assertEqual(true, BackendMod:is_empty(S)), 83 | ok = BackendMod:stop(S). 84 | 85 | -endif. % TEST 86 | -------------------------------------------------------------------------------- /src/riak_kv_reduce_phase.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_reduce_phase: manage the mechanics of a reduce phase of a MR job 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc manage the mechanics of a reduce phase of a MR job 24 | 25 | -module(riak_kv_reduce_phase). 26 | -include_lib("riak_kv_js_pools.hrl"). 27 | 28 | -behaviour(luke_phase). 29 | 30 | -export([init/1, handle_input/3, handle_input_done/1, handle_event/2, 31 | handle_sync_event/3, handle_timeout/1, handle_info/2, terminate/2]). 32 | 33 | -record(state, {qterm, reduced=[], new_inputs=[]}). 34 | 35 | %% @private 36 | init([QTerm]) -> 37 | {ok, #state{qterm=QTerm}}. 38 | 39 | handle_input(Inputs, #state{reduced=Reduced0, qterm=QTerm, new_inputs=New0}=State0, _Timeout) -> 40 | New1 = New0 ++ Inputs, 41 | if 42 | length(New1) > 20 -> 43 | case perform_reduce(QTerm, New1) of 44 | {ok, Reduced} -> 45 | {no_output, State0#state{reduced=Reduced0 ++ Reduced, new_inputs=[]}, 250}; 46 | Error -> 47 | {stop, Error, State0#state{reduced=[], new_inputs=[]}} 48 | end; 49 | true -> 50 | {no_output, State0#state{new_inputs=New1}, 250} 51 | end. 52 | 53 | handle_input_done(#state{qterm=QTerm, reduced=Reduced0, new_inputs=New0}=State) -> 54 | case perform_reduce(QTerm, Reduced0 ++ New0) of 55 | {ok, Reduced} -> 56 | luke_phase:complete(), 57 | {output, Reduced, State#state{reduced=Reduced}}; 58 | Error -> 59 | {stop, Error, State#state{reduced=[]}} 60 | end. 61 | 62 | handle_timeout(#state{qterm=QTerm, reduced=Reduced0, new_inputs=New0}=State) -> 63 | if 64 | length(New0) > 0 -> 65 | case perform_reduce(QTerm, New0) of 66 | {ok, Reduced} -> 67 | {no_output, State#state{reduced=Reduced0 ++ Reduced, new_inputs=[]}, 250}; 68 | Error -> 69 | {stop, Error, State#state{reduced=[], new_inputs=[]}} 70 | end; 71 | true -> 72 | {no_output, State, 250} 73 | end. 74 | 75 | handle_sync_event(_Event, _From, State) -> 76 | {reply, ignored, State}. 77 | 78 | handle_event(_Event, State) -> 79 | {no_output, State}. 80 | 81 | handle_info(_Info, State) -> 82 | {no_output, State}. 83 | 84 | terminate(_Reason, _State) -> 85 | ok. 86 | 87 | perform_reduce({Lang,{reduce,FunTerm,Arg,_Acc}}, 88 | Reduced) -> 89 | try 90 | case {Lang, FunTerm} of 91 | {erlang, {qfun,F}} -> 92 | Value = F(Reduced,Arg), 93 | {ok, Value}; 94 | {erlang, {modfun,M,F}} -> 95 | Value = M:F(Reduced,Arg), 96 | {ok, Value}; 97 | {javascript, _} -> 98 | case riak_kv_js_manager:blocking_dispatch(?JSPOOL_REDUCE, {FunTerm, 99 | [riak_kv_mapred_json:jsonify_not_found(R) || R <- Reduced], 100 | Arg}, 25) of 101 | {ok, Data} when is_list(Data) -> 102 | Data1 = [riak_kv_mapred_json:dejsonify_not_found(Datum) || Datum <- Data], 103 | {ok, Data1}; 104 | {error, timeout} -> 105 | throw({error, javascript_reduce_timeout}); 106 | Error -> 107 | throw(Error) 108 | end 109 | end 110 | catch _:R -> 111 | R 112 | end. 113 | -------------------------------------------------------------------------------- /src/riak_kv_keylister.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_kv_keylister: Manage streaming keys for a bucket from a 4 | %% cluster node 5 | %% 6 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 7 | %% 8 | %% This file is provided to you under the Apache License, 9 | %% Version 2.0 (the "License"); you may not use this file 10 | %% except in compliance with the License. You may obtain 11 | %% a copy of the License at 12 | %% 13 | %% http://www.apache.org/licenses/LICENSE-2.0 14 | %% 15 | %% Unless required by applicable law or agreed to in writing, 16 | %% software distributed under the License is distributed on an 17 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 18 | %% KIND, either express or implied. See the License for the 19 | %% specific language governing permissions and limitations 20 | %% under the License. 21 | %% 22 | %% ------------------------------------------------------------------- 23 | 24 | -module(riak_kv_keylister). 25 | 26 | -behaviour(gen_fsm). 27 | 28 | %% API 29 | -export([start_link/3, 30 | list_keys/2]). 31 | 32 | %% States 33 | -export([waiting/2]). 34 | 35 | %% gen_fsm callbacks 36 | -export([init/1, state_name/2, state_name/3, handle_event/3, 37 | handle_sync_event/4, handle_info/3, terminate/3, code_change/4]). 38 | 39 | -record(state, {reqid, 40 | caller, 41 | bucket, 42 | filter, 43 | bloom}). 44 | 45 | list_keys(ListerPid, VNode) -> 46 | gen_fsm:send_event(ListerPid, {lk, VNode}). 47 | 48 | start_link(ReqId, Caller, Bucket) -> 49 | gen_fsm:start_link(?MODULE, [ReqId, Caller, Bucket], []). 50 | 51 | init([ReqId, Caller, Inputs]) -> 52 | erlang:monitor(process, Caller), 53 | {ok, Bloom} = ebloom:new(10000000, 0.0001, crypto:rand_uniform(1, 5000)), 54 | {Bucket, Filter} = build_filter(Inputs), 55 | {ok, waiting, #state{reqid=ReqId, caller=Caller, bloom=Bloom, bucket=Bucket, 56 | filter=Filter}}. 57 | 58 | waiting({lk, VNode}, #state{reqid=ReqId, bucket=Bucket}=State) -> 59 | riak_kv_vnode:list_keys(VNode, ReqId, self(), Bucket), 60 | {next_state, waiting, State}. 61 | 62 | state_name(_Event, State) -> 63 | {next_state, waiting, State}. 64 | 65 | state_name(_Event, _From, State) -> 66 | {reply, ignored, state_name, State}. 67 | 68 | handle_event(_Event, StateName, State) -> 69 | {next_state, StateName, State}. 70 | 71 | handle_sync_event(_Event, _From, StateName, State) -> 72 | {reply, ignored, StateName, State}. 73 | 74 | handle_info({ReqId, {kl, Idx, Keys0}}, waiting, #state{reqid=ReqId, bloom=Bloom, 75 | filter=Filter, caller=Caller}=State) -> 76 | F = fun(Key, Acc) -> 77 | case ebloom:contains(Bloom, Key) of 78 | true -> 79 | Acc; 80 | false -> 81 | case is_function(Filter) of 82 | true -> 83 | case Filter(Key) of 84 | true -> 85 | ebloom:insert(Bloom, Key), 86 | [Key|Acc]; 87 | false -> 88 | Acc 89 | end; 90 | false -> 91 | [Key|Acc] 92 | end end end, 93 | case lists:foldl(F, [], Keys0) of 94 | [] -> 95 | ok; 96 | Keys -> 97 | gen_fsm:send_event(Caller, {ReqId, {kl, Idx, Keys}}) 98 | end, 99 | {next_state, waiting, State}; 100 | handle_info({ReqId, Idx, done}, waiting, #state{reqid=ReqId, caller=Caller}=State) -> 101 | gen_fsm:send_event(Caller, {ReqId, Idx, done}), 102 | {next_state, waiting, State}; 103 | handle_info({'DOWN', _MRef, _Type, Caller, _Info}, waiting, #state{caller=Caller}=State) -> 104 | {stop, normal, State}; 105 | handle_info(_Info, StateName, State) -> 106 | {next_state, StateName, State}. 107 | 108 | terminate(_Reason, _StateName, #state{bloom=Bloom}) -> 109 | ebloom:clear(Bloom), 110 | ok. 111 | 112 | code_change(_OldVsn, StateName, State, _Extra) -> 113 | {ok, StateName, State}. 114 | 115 | %% Internal functions 116 | build_filter('_') -> 117 | {'_', []}; 118 | build_filter(Bucket) when is_binary(Bucket) -> 119 | {Bucket, []}; 120 | build_filter({filter, Bucket, Fun}) when is_function(Fun) -> 121 | %% this is the representation used by riak_client:filter_keys 122 | {Bucket, Fun}; 123 | build_filter({Bucket, Filters}) -> 124 | FilterFun = riak_kv_mapred_filters:compose(Filters), 125 | {Bucket, FilterFun}. 126 | -------------------------------------------------------------------------------- /src/riak_kv_app.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_app: application startup for Riak 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | 21 | %% @doc Bootstrapping the Riak application. 22 | 23 | -module(riak_kv_app). 24 | 25 | -behaviour(application). 26 | -export([start/2,stop/1]). 27 | 28 | %% @spec start(Type :: term(), StartArgs :: term()) -> 29 | %% {ok,Pid} | ignore | {error,Error} 30 | %% @doc The application:start callback for riak. 31 | %% Arguments are ignored as all configuration is done via the erlenv file. 32 | start(_Type, _StartArgs) -> 33 | riak_core_util:start_app_deps(riak_kv), 34 | 35 | %% Look at the epoch and generating an error message if it doesn't match up 36 | %% to our expectations 37 | check_epoch(), 38 | 39 | %% Append user-provided code paths 40 | case app_helper:get_env(riak_kv, add_paths) of 41 | List when is_list(List) -> 42 | ok = code:add_paths(List); 43 | _ -> 44 | ok 45 | end, 46 | 47 | %% Append defaults for riak_kv buckets to the bucket defaults 48 | %% TODO: Need to revisit this. Buckets are typically created 49 | %% by a specific entity; seems lame to append a bunch of unused 50 | %% metadata to buckets that may not be appropriate for the bucket. 51 | riak_core_bucket:append_bucket_defaults( 52 | [{linkfun, {modfun, riak_kv_wm_link_walker, mapreduce_linkfun}}, 53 | {old_vclock, 86400}, 54 | {young_vclock, 20}, 55 | {big_vclock, 50}, 56 | {small_vclock, 10}, 57 | {pr, 0}, 58 | {r, quorum}, 59 | {w, quorum}, 60 | {pw, 0}, 61 | {dw, quorum}, 62 | {rw, quorum}, 63 | {pr, 0}, 64 | {basic_quorum, true}, 65 | {notfound_ok, false} 66 | ]), 67 | 68 | %% Check the storage backend 69 | StorageBackend = app_helper:get_env(riak_kv, storage_backend), 70 | case code:ensure_loaded(StorageBackend) of 71 | {error,nofile} -> 72 | error_logger:error_msg("storage_backend ~p is non-loadable.\n", 73 | [StorageBackend]), 74 | throw({error, invalid_storage_backend}); 75 | _ -> 76 | ok 77 | end, 78 | 79 | %% Register our cluster_info app callback modules, with catch if 80 | %% the app is missing or packaging is broken. 81 | catch cluster_info:register_app(riak_kv_cinfo), 82 | 83 | %% Spin up supervisor 84 | case riak_kv_sup:start_link() of 85 | {ok, Pid} -> 86 | %% Go ahead and mark the riak_kv service as up in the node watcher. 87 | %% The riak_core_ring_handler blocks until all vnodes have been started 88 | %% synchronously. 89 | riak_core:register_vnode_module(riak_kv_vnode), 90 | riak_core_node_watcher:service_up(riak_kv, self()), 91 | {ok, Pid}; 92 | {error, Reason} -> 93 | {error, Reason} 94 | end. 95 | 96 | %% @spec stop(State :: term()) -> ok 97 | %% @doc The application:stop callback for riak. 98 | stop(_State) -> 99 | ok. 100 | 101 | %% 719528 days from Jan 1, 0 to Jan 1, 1970 102 | %% *86400 seconds/day 103 | -define(SEC_TO_EPOCH, 62167219200). 104 | 105 | %% @spec check_epoch() -> ok 106 | %% @doc 107 | check_epoch() -> 108 | %% doc for erlang:now/0 says return value is platform-dependent 109 | %% -> let's emit an error if this platform doesn't think the epoch 110 | %% is Jan 1, 1970 111 | {MSec, Sec, _} = erlang:now(), 112 | GSec = calendar:datetime_to_gregorian_seconds( 113 | calendar:universal_time()), 114 | case GSec - ((MSec*1000000)+Sec) of 115 | N when (N < ?SEC_TO_EPOCH+5 andalso N > ?SEC_TO_EPOCH-5); 116 | (N < -?SEC_TO_EPOCH+5 andalso N > -?SEC_TO_EPOCH-5) -> 117 | %% if epoch is within 10 sec of expected, accept it 118 | ok; 119 | N -> 120 | Epoch = calendar:gregorian_seconds_to_datetime(N), 121 | error_logger:error_msg("Riak expects your system's epoch to be Jan 1, 1970,~n" 122 | "but your system says the epoch is ~p~n", [Epoch]), 123 | ok 124 | end. 125 | 126 | -------------------------------------------------------------------------------- /src/riak_kv_status.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% Riak: A lightweight, decentralized key-value store. 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | -module(riak_kv_status). 23 | 24 | -export([statistics/0, 25 | ringready/0, 26 | transfers/0]). 27 | 28 | %% =================================================================== 29 | %% Public API 30 | %% =================================================================== 31 | 32 | -spec(statistics() -> [any()]). 33 | statistics() -> 34 | case whereis(riak_kv_stat) of 35 | undefined -> 36 | []; 37 | _ -> 38 | riak_kv_stat:get_stats() 39 | end. 40 | 41 | -spec(ringready() -> {ok, [atom()]} | {error, any()}). 42 | ringready() -> 43 | case get_rings() of 44 | {[], Rings} -> 45 | {N1,R1}=hd(Rings), 46 | case rings_match(hash_ring(R1), tl(Rings)) of 47 | true -> 48 | Nodes = [N || {N,_} <- Rings], 49 | {ok, Nodes}; 50 | 51 | {false, N2} -> 52 | {error, {different_owners, N1, N2}} 53 | end; 54 | 55 | {Down, _Rings} -> 56 | {error, {nodes_down, Down}} 57 | end. 58 | 59 | 60 | -spec(transfers() -> {[atom()], [{waiting_to_handoff, atom(), integer()} | 61 | {stopped, atom(), integer()}]}). 62 | transfers() -> 63 | {Down, Rings} = get_rings(), 64 | 65 | %% Work out which vnodes are running and which partitions they claim 66 | F = fun({N,R}, Acc) -> 67 | {_Pri, Sec, Stopped} = partitions(N, R), 68 | Acc1 = case Sec of 69 | [] -> 70 | []; 71 | _ -> 72 | [{waiting_to_handoff, N, length(Sec)}] 73 | end, 74 | case Stopped of 75 | [] -> 76 | Acc1 ++ Acc; 77 | _ -> 78 | Acc1 ++ [{stopped, N, length(Stopped)} | Acc] 79 | end 80 | end, 81 | {Down, lists:foldl(F, [], Rings)}. 82 | 83 | 84 | %% =================================================================== 85 | %% Internal functions 86 | %% =================================================================== 87 | 88 | %% Retrieve the rings for all other nodes by RPC 89 | get_rings() -> 90 | {RawRings, Down} = riak_core_util:rpc_every_member( 91 | riak_core_ring_manager, get_my_ring, [], 30000), 92 | Rings = orddict:from_list([{riak_core_ring:owner_node(R), R} || {ok, R} <- RawRings]), 93 | {lists:sort(Down), Rings}. 94 | 95 | %% Produce a hash of the 'chash' portion of the ring 96 | hash_ring(R) -> 97 | erlang:phash2(riak_core_ring:all_owners(R)). 98 | 99 | %% Check if all rings match given a hash and a list of [{N,P}] to check 100 | rings_match(_, []) -> 101 | true; 102 | rings_match(R1hash, [{N2, R2} | Rest]) -> 103 | case hash_ring(R2) of 104 | R1hash -> 105 | rings_match(R1hash, Rest); 106 | _ -> 107 | {false, N2} 108 | end. 109 | 110 | 111 | %% Get a list of active partition numbers - regardless of vnode type 112 | active_partitions(Node) -> 113 | lists:foldl(fun({_,P}, Ps) -> 114 | ordsets:add_element(P, Ps) 115 | end, [], running_vnodes(Node)). 116 | 117 | %% Get a list of running vnodes for a node 118 | running_vnodes(Node) -> 119 | Pids = vnode_pids(Node), 120 | [rpc:call(Node, riak_core_vnode, get_mod_index, [Pid], 30000) || Pid <- Pids]. 121 | 122 | %% Get a list of vnode pids for a node 123 | vnode_pids(Node) -> 124 | [Pid || {_,Pid,_,_} <- supervisor:which_children({riak_core_vnode_sup, Node})]. 125 | 126 | %% Return a list of active primary partitions, active secondary partitions (to be handed off) 127 | %% and stopped partitions that should be started 128 | partitions(Node, Ring) -> 129 | Owners = riak_core_ring:all_owners(Ring), 130 | Owned = ordsets:from_list(owned_partitions(Owners, Node)), 131 | Active = ordsets:from_list(active_partitions(Node)), 132 | Stopped = ordsets:subtract(Owned, Active), 133 | Secondary = ordsets:subtract(Active, Owned), 134 | Primary = ordsets:subtract(Active, Secondary), 135 | {Primary, Secondary, Stopped}. 136 | 137 | %% Return the list of partitions owned by a node 138 | owned_partitions(Owners, Node) -> 139 | [P || {P, Owner} <- Owners, Owner =:= Node]. 140 | -------------------------------------------------------------------------------- /src/riak_kv_dets_backend.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_dets_backend: storage engine based on DETS tables 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc riak_kv_dets_backend is a Riak storage backend using dets. 24 | 25 | -module(riak_kv_dets_backend). 26 | -behavior(riak_kv_backend). 27 | -ifdef(TEST). 28 | -include_lib("eunit/include/eunit.hrl"). 29 | -endif. 30 | -export([start/2,stop/1,get/2,put/3,list/1,list_bucket/2, 31 | delete/2,fold/3, is_empty/1, drop/1, callback/3]). 32 | 33 | % @type state() = term(). 34 | -record(state, {table, path}). 35 | 36 | % @spec start(Partition :: integer(), Config :: proplist()) -> 37 | % {ok, state()} | {{error, Reason :: term()}, state()} 38 | start(Partition, Config) -> 39 | ConfigRoot = proplists:get_value(riak_kv_dets_backend_root, Config), 40 | if ConfigRoot =:= undefined -> 41 | riak:stop("riak_kv_dets_backend_root unset, failing.~n"); 42 | true -> ok 43 | end, 44 | 45 | TablePath = filename:join([ConfigRoot, integer_to_list(Partition)]), 46 | case filelib:ensure_dir(TablePath) of 47 | ok -> ok; 48 | _Error -> 49 | riak:stop("riak_kv_dets_backend could not ensure" 50 | " the existence of its root directory") 51 | end, 52 | 53 | DetsName = list_to_atom(integer_to_list(Partition)), 54 | case dets:open_file(DetsName, [{file, TablePath}, 55 | {min_no_slots, 8192}, 56 | {max_no_slots, 16777216}]) of 57 | {ok, DetsName} -> 58 | ok = dets:sync(DetsName), 59 | {ok, #state{table=DetsName, path=TablePath}}; 60 | {error, Reason} -> 61 | riak:stop("dets:open_file failed"), 62 | {error, Reason} 63 | end. 64 | 65 | % @spec stop(state()) -> ok | {error, Reason :: term()} 66 | stop(#state{table=T}) -> dets:close(T). 67 | 68 | % get(state(), riak_object:bkey()) -> 69 | % {ok, Val :: binary()} | {error, Reason :: term()} 70 | % key must be 160b 71 | get(#state{table=T}, BKey) -> 72 | case dets:lookup(T, BKey) of 73 | [] -> {error, notfound}; 74 | [{BKey,Val}] -> {ok, Val}; 75 | {error, Err} -> {error, Err} 76 | end. 77 | 78 | % put(state(), riak_object:bkey(), Val :: binary()) -> 79 | % ok | {error, Reason :: term()} 80 | % key must be 160b 81 | put(#state{table=T},BKey,Val) -> dets:insert(T, {BKey,Val}). 82 | 83 | % delete(state(), riak_object:bkey()) -> 84 | % ok | {error, Reason :: term()} 85 | % key must be 160b 86 | delete(#state{table=T}, BKey) -> dets:delete(T, BKey). 87 | 88 | % list(state()) -> [riak_object:bkey()] 89 | list(#state{table=T}) -> 90 | MList = dets:match(T,{'$1','_'}), 91 | list(MList,[]). 92 | list([],Acc) -> Acc; 93 | list([[K]|Rest],Acc) -> list(Rest,[K|Acc]). 94 | 95 | list_bucket(#state{table=T}, {filter, Bucket, Fun}) -> 96 | MList = lists:filter(Fun, dets:match(T,{{Bucket,'$1'},'_'})), 97 | list(MList,[]); 98 | list_bucket(#state{table=T}, Bucket) -> 99 | case Bucket of 100 | '_' -> MatchSpec = {{'$1','_'},'_'}; 101 | _ -> MatchSpec = {{Bucket,'$1'},'_'} 102 | end, 103 | MList = dets:match(T,MatchSpec), 104 | list(MList,[]). 105 | 106 | fold(#state{table=T}, Fun0, Acc) -> 107 | Fun = fun({{B,K}, V}, AccIn) -> Fun0({B,K}, V, AccIn) end, 108 | dets:foldl(Fun, Acc, T). 109 | 110 | is_empty(#state{table=T}) -> 111 | ok = dets:sync(T), 112 | dets:info(T, size) =:= 0. 113 | 114 | drop(#state{table=T, path=P}) -> 115 | ok = dets:close(T), 116 | ok = file:delete(P). 117 | 118 | %% Ignore callbacks for other backends so multi backend works 119 | callback(_State, _Ref, _Msg) -> 120 | ok. 121 | 122 | -ifdef(TEST). 123 | %% 124 | %% Test 125 | %% 126 | 127 | simple_test() -> 128 | ?assertCmd("rm -rf test/dets-backend"), 129 | Config = [{riak_kv_dets_backend_root, "test/dets-backend"}], 130 | riak_kv_backend:standard_test(?MODULE, Config). 131 | 132 | -ifdef(EQC). 133 | 134 | eqc_test_() -> 135 | {timeout, 60, 136 | [{"eqc test", ?_test(eqc_test_inner())}]}. 137 | 138 | eqc_test_inner() -> 139 | Cleanup = 140 | fun(State, OldS) -> 141 | case State of 142 | #state{} -> 143 | drop(State); 144 | _ -> 145 | ok 146 | end, 147 | [file:delete(S#state.path) || S <- OldS] 148 | end, 149 | Config = [{riak_kv_dets_backend_root, "test/dets-backend"}], 150 | ?assertCmd("rm -rf test/dets-backend"), 151 | ?assertEqual(true, backend_eqc:test(?MODULE, false, Config, Cleanup)). 152 | -endif. % EQC 153 | -endif. % TEST 154 | -------------------------------------------------------------------------------- /test/backend_eqc.erl: -------------------------------------------------------------------------------- 1 | -module(backend_eqc). 2 | 3 | -ifdef(EQC). 4 | 5 | -include_lib("eqc/include/eqc.hrl"). 6 | -include_lib("eqc/include/eqc_fsm.hrl"). 7 | -include_lib("eunit/include/eunit.hrl"). 8 | 9 | -export([initial_state/0, 10 | initial_state_data/0, 11 | next_state_data/5, 12 | precondition/4, 13 | postcondition/5]). 14 | 15 | -export([stopped/3, 16 | running/3, 17 | init_backend/2, 18 | fold/2]). 19 | -export([test/1, test/2, test/3, test/4, test/5]). 20 | -export([prop_backend/4]). 21 | 22 | -record(qcst, {c, % Backend config 23 | s, % Module state returned by Backend:start 24 | olds=sets:new(), % Old states after a stop 25 | d}).% Orddict of values stored 26 | 27 | test(Backend) -> 28 | test(Backend, false). 29 | 30 | test(Backend, Volatile) -> 31 | test(Backend, Volatile, []). 32 | 33 | test(Backend, Volatile, Config) -> 34 | test(Backend, Volatile, Config, fun(_BeState,_Olds) -> ok end). 35 | 36 | test(Backend, Volatile, Config, Cleanup) -> 37 | test(Backend, Volatile, Config, Cleanup, 30). 38 | 39 | test(Backend, Volatile, Config, Cleanup, NumTests) -> 40 | eqc:quickcheck(eqc:numtests(NumTests, 41 | prop_backend(Backend, Volatile, Config, Cleanup))). 42 | 43 | prop_backend(Backend, Volatile, Config, Cleanup) -> 44 | ?FORALL(Cmds, commands(?MODULE, {{stopped, Backend, Volatile}, initial_state_data(Config)}), 45 | aggregate(command_names(Cmds), 46 | begin 47 | {H,{_F,S},Res} = run_commands(?MODULE, Cmds), 48 | Cleanup(S#qcst.s, sets:to_list(S#qcst.olds)), 49 | ?WHENFAIL( 50 | begin 51 | io:format("History: ~p\n", [H]), 52 | io:format("BE Config: ~p\nBE State: ~p\nD: ~p\n", 53 | [S#qcst.c, S#qcst.s, orddict:to_list(S#qcst.d)]), 54 | io:format("Result: ~p\n", [Res]) 55 | end, 56 | Res == ok) 57 | end)). 58 | 59 | 60 | bucket() -> 61 | elements([<<"b1">>,<<"b2">>,<<"b3">>,<<"b4">>]). 62 | 63 | key() -> 64 | elements([<<"k1">>,<<"k2">>,<<"k3">>,<<"k4">>]). 65 | 66 | bkey() -> 67 | {bucket(),key()}. 68 | 69 | val() -> 70 | binary(). 71 | 72 | initial_state() -> 73 | {stopped, riak_kv_ets_backend, true}. 74 | 75 | initial_state_data() -> 76 | #qcst{d = orddict:new()}. 77 | 78 | initial_state_data(Config) -> 79 | #qcst{c = Config, d = orddict:new()}. 80 | 81 | next_state_data({running,Backend,Volatile},{stopped,Backend,Volatile},S,_R, 82 | {call,_M,stop,_}) -> 83 | S1 = S#qcst{s = undefined, olds = sets:add_element(S#qcst.s, S#qcst.olds)}, 84 | case Volatile of 85 | true -> 86 | S1#qcst{d = orddict:new()}; 87 | false -> 88 | S1 89 | end; 90 | next_state_data({running,Backend,Volatile},{stopped,Backend,Volatile},S,_R, 91 | {call,_M,drop,_}) -> 92 | S#qcst{s=undefined, olds=sets:add_element(S#qcst.s, S#qcst.olds), 93 | d=orddict:new()}; 94 | next_state_data(_From,_To,S,BeState,{call,_M,init_backend,_}) -> 95 | S#qcst{s = BeState}; 96 | next_state_data(_From,_To,S,_R,{call,_M,put,[_S, Key, Val]}) -> 97 | S#qcst{d = orddict:store(Key, Val, S#qcst.d)}; 98 | next_state_data(_From,_To,S,_R,{call,_M,delete,[_S, Key]}) -> 99 | S#qcst{d = orddict:erase(Key, S#qcst.d)}; 100 | next_state_data(_From,_To,S,_R,_C) -> 101 | S. 102 | 103 | stopped(Backend, Volatile, S) -> 104 | [{{running, Backend, Volatile}, {call,?MODULE,init_backend,[Backend,S#qcst.c]}}]. 105 | 106 | running(Backend, Volatile, S) -> 107 | [{history, {call,Backend,put,[S#qcst.s,bkey(),val()]}}, 108 | {history, {call,Backend,get,[S#qcst.s,bkey()]}}, 109 | {history, {call,Backend,delete,[S#qcst.s,bkey()]}}, 110 | {history, {call,Backend,list,[S#qcst.s]}}, 111 | {history, {call,?MODULE,fold,[Backend,S#qcst.s]}}, 112 | {history, {call,Backend,is_empty,[S#qcst.s]}}, 113 | {history, {call,Backend,list_bucket,[S#qcst.s,bucket()]}}, 114 | {{stopped, Backend, Volatile}, {call,Backend,drop,[S#qcst.s]}}, 115 | {{stopped, Backend, Volatile}, {call,Backend,stop,[S#qcst.s]}} 116 | ]. 117 | 118 | 119 | precondition(_From,_To,_S,_C) -> 120 | true. 121 | 122 | postcondition(_From,_To,S,_C={call,_M,get,[_BeState, Key]},R) -> 123 | case R of 124 | {error, notfound} -> 125 | not orddict:is_key(Key, S#qcst.d); 126 | {ok, Val} -> 127 | {ok, Val} =:= orddict:find(Key, S#qcst.d) 128 | end; 129 | postcondition(_From,_To,_S,_C={call,_M,put,[_BeState, _Key, _Val]},R) -> 130 | R =:= ok; 131 | postcondition(_From,_To,_S,_C={call,_M,delete,[_BeState, _Key]},R) -> 132 | R =:= ok; 133 | postcondition(_From,_To,S,_C={call,_M,list,[_BeState]},R) -> 134 | lists:sort(orddict:fetch_keys(S#qcst.d)) =:= lists:sort(R); 135 | postcondition(_From,_To,S,_C={call,_M,fold,[_Backend,_BeState]},R) -> 136 | lists:sort(orddict:to_list(S#qcst.d)) =:= lists:sort(R); 137 | postcondition(_From,_To,S,_C={call,_M,is_empty,[_BeState]},R) -> 138 | R =:= (orddict:size(S#qcst.d) =:= 0); 139 | postcondition(_From,_To,S,_C={call,_M,list_bucket,[_BeState,Bucket]},R) -> 140 | AllKeys = orddict:fetch_keys(S#qcst.d), 141 | lists:sort(R) =:= lists:sort([K || {B,K} <- AllKeys, B =:= Bucket]); 142 | postcondition(_From,_To,_S,_C,_R) -> 143 | true. 144 | 145 | 146 | init_backend(Backend, Config) -> 147 | {ok, S} = Backend:start(42, Config), 148 | S. 149 | 150 | fold_fun(K,V,Acc) -> 151 | [{K,V}|Acc]. 152 | 153 | fold(Backend, BeState) -> 154 | Backend:fold(BeState, fun fold_fun/3, []). 155 | 156 | -endif. 157 | 158 | -------------------------------------------------------------------------------- /src/riak_kv_ets_backend.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_ets_backend: storage engine based on ETS tables 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | % @doc riak_kv_ets_backend is a Riak storage backend using ets. 24 | 25 | -module(riak_kv_ets_backend). 26 | -behavior(riak_kv_backend). 27 | -behavior(gen_server). 28 | -ifdef(TEST). 29 | -include_lib("eunit/include/eunit.hrl"). 30 | -endif. 31 | -export([start/2,stop/1,get/2,put/3,list/1,list_bucket/2,delete/2, 32 | is_empty/1, drop/1, fold/3, fold_bucket_keys/4, callback/3]). 33 | 34 | -export([init/1, handle_call/3, handle_cast/2, handle_info/2, 35 | terminate/2, code_change/3]). 36 | 37 | % @type state() = term(). 38 | -record(state, {t}). 39 | 40 | % @spec start(Partition :: integer(), Config :: proplist()) -> 41 | % {ok, state()} | {{error, Reason :: term()}, state()} 42 | start(Partition, _Config) -> 43 | gen_server:start_link(?MODULE, [Partition], []). 44 | 45 | %% @private 46 | init([Partition]) -> 47 | {ok, #state{t=ets:new(list_to_atom(integer_to_list(Partition)),[])}}. 48 | 49 | %% @private 50 | handle_cast(_, State) -> {noreply, State}. 51 | 52 | %% @private 53 | handle_call(stop,_From,State) -> {reply, srv_stop(State), State}; 54 | handle_call({get,BKey},_From,State) -> {reply, srv_get(State,BKey), State}; 55 | handle_call({put,BKey,Val},_From,State) -> 56 | {reply, srv_put(State,BKey,Val),State}; 57 | handle_call({delete,BKey},_From,State) -> {reply, srv_delete(State,BKey),State}; 58 | handle_call(list,_From,State) -> {reply, srv_list(State), State}; 59 | handle_call({list_bucket,Bucket},_From,State) -> 60 | {reply, srv_list_bucket(State, Bucket), State}; 61 | handle_call(is_empty, _From, State) -> 62 | {reply, ets:info(State#state.t, size) =:= 0, State}; 63 | handle_call(drop, _From, State) -> 64 | ets:delete(State#state.t), 65 | {reply, ok, State}; 66 | handle_call({fold, Fun0, Acc}, _From, State) -> 67 | Fun = fun({{B,K}, V}, AccIn) -> Fun0({B,K}, V, AccIn) end, 68 | Reply = ets:foldl(Fun, Acc, State#state.t), 69 | {reply, Reply, State}; 70 | handle_call({fold_bucket_keys, _Bucket, Fun0, Acc}, From, State) -> 71 | %% We could do something with the Bucket arg, but for this backend 72 | %% there isn't much point, so we'll do the same thing as the older 73 | %% API fold. 74 | handle_call({fold, Fun0, Acc}, From, State). 75 | 76 | % @spec stop(state()) -> ok | {error, Reason :: term()} 77 | stop(SrvRef) -> gen_server:call(SrvRef,stop). 78 | srv_stop(State) -> 79 | catch ets:delete(State#state.t), 80 | ok. 81 | 82 | % get(state(), riak_object:bkey()) -> 83 | % {ok, Val :: binary()} | {error, Reason :: term()} 84 | % key must be 160b 85 | get(SrvRef, BKey) -> gen_server:call(SrvRef,{get,BKey}). 86 | srv_get(State, BKey) -> 87 | case ets:lookup(State#state.t,BKey) of 88 | [] -> {error, notfound}; 89 | [{BKey,Val}] -> {ok, Val}; 90 | Err -> {error, Err} 91 | end. 92 | 93 | % put(state(), riak_object:bkey(), Val :: binary()) -> 94 | % ok | {error, Reason :: term()} 95 | % key must be 160b 96 | put(SrvRef, BKey, Val) -> gen_server:call(SrvRef,{put,BKey,Val}). 97 | srv_put(State,BKey,Val) -> 98 | true = ets:insert(State#state.t, {BKey,Val}), 99 | ok. 100 | 101 | % delete(state(), riak_object:bkey()) -> 102 | % ok | {error, Reason :: term()} 103 | % key must be 160b 104 | delete(SrvRef, BKey) -> gen_server:call(SrvRef,{delete,BKey}). 105 | srv_delete(State, BKey) -> 106 | true = ets:delete(State#state.t, BKey), 107 | ok. 108 | 109 | % list(state()) -> [riak_object:bkey()] 110 | list(SrvRef) -> gen_server:call(SrvRef,list). 111 | srv_list(State) -> 112 | MList = ets:match(State#state.t,{'$1','_'}), 113 | list(MList,[]). 114 | list([],Acc) -> Acc; 115 | list([[K]|Rest],Acc) -> list(Rest,[K|Acc]). 116 | 117 | % list_bucket(term(), Bucket :: riak_object:bucket()) -> [Key :: binary()] 118 | list_bucket(SrvRef, Bucket) -> 119 | gen_server:call(SrvRef,{list_bucket, Bucket}). 120 | srv_list_bucket(State, {filter, Bucket, Fun}) -> 121 | MList = lists:filter(Fun, ets:match(State#state.t,{{Bucket,'$1'},'_'})), 122 | list(MList,[]); 123 | srv_list_bucket(State, Bucket) -> 124 | case Bucket of 125 | '_' -> MatchSpec = {{'$1','_'},'_'}; 126 | _ -> MatchSpec = {{Bucket,'$1'},'_'} 127 | end, 128 | MList = ets:match(State#state.t,MatchSpec), 129 | list(MList,[]). 130 | 131 | is_empty(SrvRef) -> gen_server:call(SrvRef, is_empty). 132 | 133 | drop(SrvRef) -> gen_server:call(SrvRef, drop). 134 | 135 | fold(SrvRef, Fun, Acc0) -> gen_server:call(SrvRef, {fold, Fun, Acc0}, infinity). 136 | 137 | fold_bucket_keys(SrvRef, Bucket, Fun, Acc0) -> 138 | gen_server:call(SrvRef, {fold_bucket_keys, Bucket, Fun, Acc0}, infinity). 139 | 140 | %% Ignore callbacks for other backends so multi backend works 141 | callback(_State, _Ref, _Msg) -> 142 | ok. 143 | 144 | %% @private 145 | handle_info(_Msg, State) -> {noreply, State}. 146 | 147 | %% @private 148 | terminate(_Reason, _State) -> ok. 149 | 150 | %% @private 151 | code_change(_OldVsn, State, _Extra) -> {ok, State}. 152 | 153 | %% 154 | %% Test 155 | %% 156 | -ifdef(TEST). 157 | 158 | simple_test() -> 159 | riak_kv_backend:standard_test(?MODULE, []). 160 | 161 | -ifdef(EQC). 162 | eqc_test() -> 163 | ?assertEqual(true, backend_eqc:test(?MODULE, true)). 164 | 165 | -endif. % EQC 166 | -endif. % TEST 167 | -------------------------------------------------------------------------------- /src/riak_kv_sup.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_sup: supervise the core Riak services 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc supervise the core Riak services 24 | 25 | -module(riak_kv_sup). 26 | 27 | -include_lib("riak_kv_js_pools.hrl"). 28 | 29 | -behaviour(supervisor). 30 | 31 | -export([start_link/0]). 32 | -export([init/1]). 33 | 34 | -define (IF (Bool, A, B), if Bool -> A; true -> B end). 35 | 36 | %% @spec start_link() -> ServerRet 37 | %% @doc API for starting the supervisor. 38 | start_link() -> 39 | supervisor:start_link({local, ?MODULE}, ?MODULE, []). 40 | 41 | %% @spec init([]) -> SupervisorTree 42 | %% @doc supervisor callback. 43 | init([]) -> 44 | [ webmachine_router:add_route(R) 45 | || R <- lists:reverse(riak_kv_web:dispatch_table()) ], 46 | VMaster = {riak_kv_vnode_master, 47 | {riak_core_vnode_master, start_link, 48 | [riak_kv_vnode, riak_kv_legacy_vnode]}, 49 | permanent, 5000, worker, [riak_core_vnode_master]}, 50 | RiakPb = [ {riak_kv_pb_socket_sup, {riak_kv_pb_socket_sup, start_link, []}, 51 | permanent, infinity, supervisor, [riak_kv_pb_socket_sup]}, 52 | {riak_kv_pb_listener, {riak_kv_pb_listener, start_link, []}, 53 | permanent, 5000, worker, [riak_kv_pb_listener]} 54 | ], 55 | RiakStat = {riak_kv_stat, 56 | {riak_kv_stat, start_link, []}, 57 | permanent, 5000, worker, [riak_kv_stat]}, 58 | MapJSPool = {?JSPOOL_MAP, 59 | {riak_kv_js_manager, start_link, 60 | [?JSPOOL_MAP, read_js_pool_size(map_js_vm_count, "map")]}, 61 | permanent, 30000, worker, [riak_kv_js_manager]}, 62 | ReduceJSPool = {?JSPOOL_REDUCE, 63 | {riak_kv_js_manager, start_link, 64 | [?JSPOOL_REDUCE, read_js_pool_size(reduce_js_vm_count, "reduce")]}, 65 | permanent, 30000, worker, [riak_kv_js_manager]}, 66 | HookJSPool = {?JSPOOL_HOOK, 67 | {riak_kv_js_manager, start_link, 68 | [?JSPOOL_HOOK, read_js_pool_size(hook_js_vm_count, "hook callback")]}, 69 | permanent, 30000, worker, [riak_kv_js_manager]}, 70 | JSSup = {riak_kv_js_sup, 71 | {riak_kv_js_sup, start_link, []}, 72 | permanent, infinity, supervisor, [riak_kv_js_sup]}, 73 | KLMaster = {riak_kv_keylister_master, 74 | {riak_kv_keylister_master, start_link, []}, 75 | permanent, 30000, worker, [riak_kv_keylister_master]}, 76 | KLSup = {riak_kv_keylister_sup, 77 | {riak_kv_keylister_sup, start_link, []}, 78 | permanent, infinity, supervisor, [riak_kv_keylister_sup]}, 79 | MapCache = {riak_kv_mapred_cache, 80 | {riak_kv_mapred_cache, start_link, []}, 81 | permanent, 30000, worker, [riak_kv_mapred_cache]}, 82 | MapMaster = {riak_kv_map_master, 83 | {riak_kv_map_master, start_link, []}, 84 | permanent, 30000, worker, [riak_kv_map_master]}, 85 | MapperSup = {riak_kv_mapper_sup, 86 | {riak_kv_mapper_sup, start_link, []}, 87 | permanent, infinity, supervisor, [riak_kv_mapper_sup]}, 88 | GetFsmSup = {riak_kv_get_fsm_sup, 89 | {riak_kv_get_fsm_sup, start_link, []}, 90 | permanent, infinity, supervisor, [riak_kv_get_fsm_sup]}, 91 | PutFsmSup = {riak_kv_put_fsm_sup, 92 | {riak_kv_put_fsm_sup, start_link, []}, 93 | permanent, infinity, supervisor, [riak_kv_put_fsm_sup]}, 94 | DeleteSup = {riak_kv_delete_sup, 95 | {riak_kv_delete_sup, start_link, []}, 96 | permanent, infinity, supervisor, [riak_kv_delete_sup]}, 97 | KeysFsmSup = {riak_kv_keys_fsm_sup, 98 | {riak_kv_keys_fsm_sup, start_link, []}, 99 | permanent, infinity, supervisor, [riak_kv_keys_fsm_sup]}, 100 | 101 | % Figure out which processes we should run... 102 | IsPbConfigured = (app_helper:get_env(riak_kv, pb_ip) /= undefined) 103 | andalso (app_helper:get_env(riak_kv, pb_port) /= undefined), 104 | HasStorageBackend = (app_helper:get_env(riak_kv, storage_backend) /= undefined), 105 | IsStatEnabled = (app_helper:get_env(riak_kv, riak_kv_stat) == true), 106 | 107 | % Build the process list... 108 | Processes = lists:flatten([ 109 | ?IF(HasStorageBackend, VMaster, []), 110 | ?IF(IsPbConfigured, RiakPb, []), 111 | ?IF(IsStatEnabled, RiakStat, []), 112 | GetFsmSup, 113 | PutFsmSup, 114 | DeleteSup, 115 | KeysFsmSup, 116 | KLSup, 117 | KLMaster, 118 | JSSup, 119 | MapJSPool, 120 | ReduceJSPool, 121 | HookJSPool, 122 | MapperSup, 123 | MapMaster, 124 | MapCache 125 | ]), 126 | 127 | % Run the proesses... 128 | {ok, {{one_for_one, 10, 10}, Processes}}. 129 | 130 | %% Internal functions 131 | read_js_pool_size(Entry, PoolType) -> 132 | case app_helper:get_env(riak_kv, Entry, undefined) of 133 | undefined -> 134 | OldSize = app_helper:get_env(riak_kv, js_vm_count, 0), 135 | error_logger:warning_msg("js_vm_count has been deprecated. " ++ 136 | "Please use ~p to configure the ~s pool.", [Entry, PoolType]), 137 | case OldSize > 8 of 138 | true -> 139 | OldSize div 3; 140 | false -> 141 | OldSize 142 | end; 143 | Size -> 144 | Size 145 | end. 146 | -------------------------------------------------------------------------------- /src/riak_kv_put_core.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_kv_put_core: Riak put logic 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | -module(riak_kv_put_core). 23 | -export([init/7, add_result/2, enough/1, response/1, 24 | final/1]). 25 | -export_type([putcore/0, result/0, reply/0]). 26 | 27 | -type vput_result() :: any(). 28 | 29 | -type result() :: w | 30 | {dw, undefined} | 31 | {dw, riak_object:riak_object()} | 32 | {error, any()}. 33 | 34 | -type reply() :: ok | 35 | {ok, riak_object:riak_object()} | 36 | {error, notfound} | 37 | {error, any()}. 38 | -type idxresult() :: {non_neg_integer(), result()}. 39 | -record(putcore, {n :: pos_integer(), 40 | w :: non_neg_integer(), 41 | dw :: non_neg_integer(), 42 | w_fail_threshold :: pos_integer(), 43 | dw_fail_threshold :: pos_integer(), 44 | returnbody :: boolean(), 45 | allowmult :: boolean(), 46 | results = [] :: [idxresult()], 47 | final_obj :: undefined | riak_object:riak_object(), 48 | num_w = 0 :: non_neg_integer(), 49 | num_dw = 0 :: non_neg_integer(), 50 | num_fail = 0 :: non_neg_integer()}). 51 | -opaque putcore() :: #putcore{}. 52 | 53 | %% ==================================================================== 54 | %% Public API 55 | %% ==================================================================== 56 | 57 | %% Initialize a put and return an opaque put core context 58 | -spec init(pos_integer(), non_neg_integer(), non_neg_integer(), 59 | pos_integer(), pos_integer(), boolean(), boolean()) -> putcore(). 60 | init(N, W, DW, WFailThreshold, DWFailThreshold, AllowMult, ReturnBody) -> 61 | #putcore{n = N, w = W, dw = DW, 62 | w_fail_threshold = WFailThreshold, 63 | dw_fail_threshold = DWFailThreshold, 64 | allowmult = AllowMult, 65 | returnbody = ReturnBody}. 66 | 67 | %% Add a result from the vnode 68 | -spec add_result(vput_result(), putcore()) -> putcore(). 69 | add_result({w, Idx, _ReqId}, PutCore = #putcore{results = Results, 70 | num_w = NumW}) -> 71 | PutCore#putcore{results = [{Idx, w} | Results], 72 | num_w = NumW + 1}; 73 | add_result({dw, Idx, _ReqId}, PutCore = #putcore{results = Results, 74 | num_dw = NumDW}) -> 75 | PutCore#putcore{results = [{Idx, {dw, undefined}} | Results], 76 | num_dw = NumDW + 1}; 77 | add_result({dw, Idx, ResObj, _ReqId}, PutCore = #putcore{results = Results, 78 | num_dw = NumDW}) -> 79 | PutCore#putcore{results = [{Idx, {dw, ResObj}} | Results], 80 | num_dw = NumDW + 1}; 81 | add_result({fail, Idx, _ReqId}, PutCore = #putcore{results = Results, 82 | num_fail = NumFail}) -> 83 | PutCore#putcore{results = [{Idx, {error, undefined}} | Results], 84 | num_fail = NumFail + 1}; 85 | add_result(_Other, PutCore = #putcore{num_fail = NumFail}) -> 86 | %% Treat unrecognized messages as failures - no index to store them against 87 | PutCore#putcore{num_fail = NumFail + 1}. 88 | 89 | %% Check if enough results have been added to respond 90 | -spec enough(putcore()) -> boolean(). 91 | enough(#putcore{w = W, num_w = NumW, dw = DW, num_dw = NumDW, 92 | num_fail = NumFail, w_fail_threshold = WFailThreshold, 93 | dw_fail_threshold = DWFailThreshold}) -> 94 | (NumW >= W andalso NumDW >= DW) orelse 95 | (NumW >= W andalso NumFail >= DWFailThreshold) orelse 96 | (NumW < W andalso NumFail >= WFailThreshold). 97 | 98 | %% Get success/fail response once enough results received 99 | -spec response(putcore()) -> {reply(), putcore()}. 100 | response(PutCore = #putcore{w = W, num_w = NumW, dw = DW, num_dw = NumDW, 101 | num_fail = NumFail, 102 | w_fail_threshold = WFailThreshold, 103 | dw_fail_threshold = DWFailThreshold}) -> 104 | if 105 | NumW >= W andalso NumDW >= DW -> 106 | maybe_return_body(PutCore); 107 | 108 | NumW >= W andalso NumFail >= DWFailThreshold -> 109 | {{error, too_many_fails}, PutCore}; 110 | 111 | NumW < W andalso NumFail >= WFailThreshold -> 112 | {{error, too_many_fails}, PutCore}; 113 | 114 | true -> 115 | {{error, {w_val_unsatisfied, NumW, NumDW, W, DW}}, PutCore} 116 | end. 117 | 118 | %% Get final value - if returnbody did not need the result it allows delaying 119 | %% running reconcile until after the client reply is sent. 120 | -spec final(putcore()) -> {riak_object:riak_object()|undefined, putcore()}. 121 | final(PutCore = #putcore{final_obj = FinalObj, 122 | results = Results, allowmult = AllowMult}) -> 123 | case FinalObj of 124 | undefined -> 125 | RObjs = [RObj || {_Idx, {dw, RObj}} <- Results, RObj /= undefined], 126 | ReplyObj = case RObjs of 127 | [] -> 128 | undefined; 129 | _ -> 130 | riak_object:reconcile(RObjs, AllowMult) 131 | end, 132 | {ReplyObj, PutCore#putcore{final_obj = ReplyObj}}; 133 | _ -> 134 | {FinalObj, PutCore} 135 | end. 136 | 137 | %% ==================================================================== 138 | %% Internal functions 139 | %% ==================================================================== 140 | maybe_return_body(PutCore = #putcore{returnbody = false}) -> 141 | {ok, PutCore}; 142 | maybe_return_body(PutCore = #putcore{returnbody = true}) -> 143 | {ReplyObj, UpdPutCore} = final(PutCore), 144 | {{ok, ReplyObj}, UpdPutCore}. 145 | 146 | -------------------------------------------------------------------------------- /src/riak_kv_gb_trees_backend.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_gb_trees_backend: storage engine based on gb_trees 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc riak_kv_gb_trees_backend is a Riak storage backend using Erlang gb_trees. 24 | 25 | -module(riak_kv_gb_trees_backend). 26 | -behavior(riak_kv_backend). 27 | -ifdef(TEST). 28 | -include_lib("eunit/include/eunit.hrl"). 29 | -endif. 30 | -export([start/2, stop/1,get/2,put/3,list/1,list_bucket/2, 31 | delete/2,is_empty/1,fold/3,drop/1,callback/3]). 32 | 33 | % @type state() = term(). 34 | -record(state, {pid}). 35 | 36 | 37 | % @spec start(Partition :: integer(), Config :: integer()) -> 38 | % {ok, state()} | {{error, Reason :: term()}, state()} 39 | start(_Partition, _Config) -> 40 | Pid = spawn_link(fun() -> 41 | {A1,A2,A3} = now(), 42 | random:seed(A1, A2, A3), 43 | tree_loop(gb_trees:empty()) 44 | end), 45 | {ok, #state { pid=Pid }}. 46 | 47 | % @spec stop(state()) -> ok | {error, Reason :: term()} 48 | stop(#state { pid=Pid }) -> 49 | Ref = make_ref(), 50 | %% Make sure the backend process is alive or 51 | %% the call to receive will wait indefinitely. 52 | case is_process_alive(Pid) of 53 | true -> 54 | Pid ! {stop, self(), Ref}, 55 | receive {stop_response, Result, Ref} -> Result end; 56 | false -> 57 | ok 58 | end. 59 | 60 | % get(state(), Key :: binary()) -> 61 | % {ok, Val :: binary()} | {error, Reason :: term()} 62 | get(#state { pid=Pid }, BKey) -> 63 | Ref = make_ref(), 64 | Pid ! {get, BKey, self(), Ref}, 65 | receive {get_response, Result, Ref} -> Result end. 66 | 67 | % put(state(), Key :: binary(), Val :: binary()) -> 68 | % ok | {error, Reason :: term()} 69 | put(#state { pid=Pid }, BKey, Value) -> 70 | Ref = make_ref(), 71 | Pid ! {put, BKey, Value, self(), Ref}, 72 | receive {put_response, Result, Ref} -> Result end. 73 | 74 | % delete(state(), Key :: binary()) -> 75 | % ok | {error, Reason :: term()} 76 | delete(#state { pid=Pid }, BKey) -> 77 | Ref = make_ref(), 78 | Pid ! {delete, BKey, self(), Ref}, 79 | receive {delete_response, Result, Ref} -> Result end. 80 | 81 | % list(state()) -> [Key :: binary()] 82 | list(#state { pid=Pid }) -> 83 | Ref = make_ref(), 84 | Pid ! {list, self(), Ref}, 85 | receive {list_response, Result, Ref} -> Result end. 86 | 87 | list_bucket(#state { pid=Pid }, Bucket) -> 88 | Ref = make_ref(), 89 | Pid ! {list_bucket, Bucket, self(), Ref}, 90 | receive {list_bucket_response, Result, Ref} -> Result end. 91 | 92 | is_empty(#state { pid=Pid }) -> 93 | Ref = make_ref(), 94 | Pid ! {is_empty, self(), Ref}, 95 | receive {is_empty_response, Result, Ref} -> Result end. 96 | 97 | fold(#state{ pid=Pid }, Fun0, Acc) -> 98 | Ref = make_ref(), 99 | Pid ! {fold, Fun0, Acc, self(), Ref}, 100 | receive {fold_response, Result, Ref} -> Result end. 101 | 102 | drop(#state{ pid=Pid }) -> 103 | Ref = make_ref(), 104 | Pid ! {drop, self(), Ref}, 105 | receive {drop_response, Result, Ref} -> Result end. 106 | 107 | %% Ignore callbacks for other backends so multi backend works 108 | callback(_State, _Ref, _Msg) -> 109 | ok. 110 | 111 | tree_loop(Tree) -> 112 | receive 113 | {get, BKey, Pid, Ref} -> 114 | case gb_trees:lookup(BKey, Tree) of 115 | {value, Value} -> Pid ! {get_response, {ok, Value}, Ref}; 116 | none -> Pid ! {get_response, {error, notfound}, Ref} 117 | end, 118 | tree_loop(Tree); 119 | 120 | {put, BKey, Value, Pid, Ref} -> 121 | Tree1 = case gb_trees:is_defined(BKey, Tree) of 122 | true -> gb_trees:update(BKey, Value, Tree); 123 | false -> gb_trees:insert(BKey, Value, Tree) 124 | end, 125 | Pid ! {put_response, ok, Ref}, 126 | tree_loop(Tree1); 127 | 128 | 129 | {delete, BKey, Pid, Ref} -> 130 | Tree1 = case gb_trees:is_defined(BKey, Tree) of 131 | true -> gb_trees:delete(BKey, Tree); 132 | false -> Tree 133 | end, 134 | Pid ! {delete_response, ok, Ref}, 135 | tree_loop(Tree1); 136 | 137 | {list, Pid, Ref} -> 138 | Pid ! {list_response, gb_trees:keys(Tree), Ref}, 139 | tree_loop(Tree); 140 | 141 | {list_bucket, Bucket, Pid, Ref} -> 142 | Pid ! {list_bucket_response, 143 | srv_list_bucket(Tree, Bucket), Ref}, 144 | tree_loop(Tree); 145 | 146 | {is_empty, Pid, Ref} -> 147 | Pid ! {is_empty_response, gb_trees:is_empty(Tree), Ref}, 148 | tree_loop(Tree); 149 | 150 | {drop, Pid, Ref} -> 151 | Pid ! {drop_response, ok, Ref}, 152 | tree_loop(gb_trees:empty()); 153 | 154 | {fold, Fun0, Acc, Pid, Ref} -> 155 | Pid ! {fold_response, 156 | srv_fold(Tree, Fun0, Acc), Ref}, 157 | tree_loop(Tree); 158 | 159 | {stop, Pid, Ref} -> 160 | Pid ! {stop_response, ok, Ref} 161 | end. 162 | 163 | srv_list_bucket(Tree, {filter, Bucket, Fun}) -> 164 | lists:append( 165 | lists:filter( 166 | Fun, 167 | [ [Key] || {B, Key} <- gb_trees:keys(Tree), B == Bucket ])); 168 | srv_list_bucket(Tree, '_') -> 169 | sets:to_list( 170 | sets:from_list( 171 | [ Bucket || {Bucket, _} <- gb_trees:keys(Tree) ])); 172 | srv_list_bucket(Tree, Bucket) -> 173 | [ Key || {B, Key} <- gb_trees:keys(Tree), B == Bucket ]. 174 | 175 | srv_fold(Tree, Fun0, Acc) -> 176 | Iter0 = gb_trees:iterator(Tree), 177 | srv_fold1(gb_trees:next(Iter0), Fun0, Acc). 178 | 179 | srv_fold1(none, _Fun0, Acc) -> 180 | Acc; 181 | srv_fold1({K,V,Iter}, Fun0, Acc) -> 182 | srv_fold1(gb_trees:next(Iter), Fun0, Fun0(K,V,Acc)). 183 | 184 | 185 | %% 186 | %% Test 187 | %% 188 | -ifdef(TEST). 189 | 190 | simple_test() -> 191 | riak_kv_backend:standard_test(?MODULE, []). 192 | 193 | -ifdef(EQC). 194 | eqc_test() -> 195 | ?assertEqual(true, backend_eqc:test(?MODULE, true)). 196 | 197 | -endif. % EQC 198 | -endif. % TEST 199 | -------------------------------------------------------------------------------- /src/riak_kv_util.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_util: functions that are useful throughout Riak 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | 24 | %% @doc Various functions that are useful throughout riak_kv. 25 | -module(riak_kv_util). 26 | 27 | 28 | -export([is_x_deleted/1, 29 | obj_not_deleted/1, 30 | try_cast/3, 31 | fallback/4, 32 | expand_value/3, 33 | expand_rw_value/4, 34 | normalize_rw_value/2, 35 | make_request/2]). 36 | 37 | -include_lib("riak_kv_vnode.hrl"). 38 | 39 | -ifdef(TEST). 40 | -include_lib("eunit/include/eunit.hrl"). 41 | -endif. 42 | 43 | %% =================================================================== 44 | %% Public API 45 | %% =================================================================== 46 | 47 | %% @spec is_x_deleted(riak_object:riak_object()) -> boolean() 48 | %% @doc 'true' if all contents of the input object are marked 49 | %% as deleted; 'false' otherwise 50 | %% @equiv obj_not_deleted(Obj) == undefined 51 | is_x_deleted(Obj) -> 52 | case obj_not_deleted(Obj) of 53 | undefined -> true; 54 | _ -> false 55 | end. 56 | 57 | %% @spec obj_not_deleted(riak_object:riak_object()) -> 58 | %% undefined|riak_object:riak_object() 59 | %% @doc Determine whether all contents of an object are marked as 60 | %% deleted. Return is the atom 'undefined' if all contents 61 | %% are marked deleted, or the input Obj if any of them are not. 62 | obj_not_deleted(Obj) -> 63 | case [{M, V} || {M, V} <- riak_object:get_contents(Obj), 64 | dict:is_key(<<"X-Riak-Deleted">>, M) =:= false] of 65 | [] -> undefined; 66 | _ -> Obj 67 | end. 68 | 69 | %% @spec try_cast(term(), [node()], [{Index :: term(), Node :: node()}]) -> 70 | %% {[{Index :: term(), Node :: node(), Node :: node()}], 71 | %% [{Index :: term(), Node :: node()}]} 72 | %% @doc Cast {Cmd, {Index,Node}, Msg} at riak_kv_vnode_master on Node 73 | %% if Node is in UpNodes. The list of successful casts is the 74 | %% first element of the return tuple, and the list of unavailable 75 | %% nodes is the second element. Used in riak_kv_put_fsm and riak_kv_get_fsm. 76 | try_cast(Msg, UpNodes, Targets) -> 77 | try_cast(Msg, UpNodes, Targets, [], []). 78 | try_cast(_Msg, _UpNodes, [], Sent, Pangs) -> {Sent, Pangs}; 79 | try_cast(Msg, UpNodes, [{Index,Node}|Targets], Sent, Pangs) -> 80 | case lists:member(Node, UpNodes) of 81 | false -> 82 | try_cast(Msg, UpNodes, Targets, Sent, [{Index,Node}|Pangs]); 83 | true -> 84 | gen_server:cast({riak_kv_vnode_master, Node}, make_request(Msg, Index)), 85 | try_cast(Msg, UpNodes, Targets, [{Index,Node,Node}|Sent],Pangs) 86 | end. 87 | 88 | %% @spec fallback(term(), term(), [{Index :: term(), Node :: node()}], 89 | %% [{any(), Fallback :: node()}]) -> 90 | %% [{Index :: term(), Node :: node(), Fallback :: node()}] 91 | %% @doc Cast {Cmd, {Index,Node}, Msg} at a node in the Fallbacks list 92 | %% for each node in the Pangs list. Pangs should have come 93 | %% from the second element of the response tuple of a call to 94 | %% try_cast/3. 95 | %% Used in riak_kv_put_fsm and riak_kv_get_fsm 96 | 97 | fallback(Cmd, UpNodes, Pangs, Fallbacks) -> 98 | fallback(Cmd, UpNodes, Pangs, Fallbacks, []). 99 | fallback(_Cmd, _UpNodes, [], _Fallbacks, Sent) -> Sent; 100 | fallback(_Cmd, _UpNodes, _Pangs, [], Sent) -> Sent; 101 | fallback(Cmd, UpNodes, [{Index,Node}|Pangs], [{_,FN}|Fallbacks], Sent) -> 102 | case lists:member(FN, UpNodes) of 103 | false -> fallback(Cmd, UpNodes, [{Index,Node}|Pangs], Fallbacks, Sent); 104 | true -> 105 | gen_server:cast({riak_kv_vnode_master, FN}, make_request(Cmd, Index)), 106 | fallback(Cmd, UpNodes, Pangs, Fallbacks, [{Index,Node,FN}|Sent]) 107 | end. 108 | 109 | 110 | -spec make_request(vnode_req(), partition()) -> #riak_vnode_req_v1{}. 111 | make_request(Request, Index) -> 112 | riak_core_vnode_master:make_request(Request, 113 | {fsm, undefined, self()}, 114 | Index). 115 | 116 | get_bucket_option(Type, BucketProps) -> 117 | case proplists:get_value(Type, BucketProps, default) of 118 | default -> 119 | {ok, DefaultProps} = application:get_env(riak_core, default_bucket_props), 120 | proplists:get_value(Type, DefaultProps, error); 121 | Val -> Val 122 | end. 123 | 124 | expand_value(Type, default, BucketProps) -> 125 | get_bucket_option(Type, BucketProps); 126 | expand_value(_Type, Value, _BucketProps) -> 127 | Value. 128 | 129 | expand_rw_value(Type, default, BucketProps, N) -> 130 | normalize_rw_value(get_bucket_option(Type, BucketProps), N); 131 | expand_rw_value(_Type, Val, _BucketProps, N) -> 132 | normalize_rw_value(Val, N). 133 | 134 | normalize_rw_value(RW, _N) when is_integer(RW) -> RW; 135 | normalize_rw_value(RW, N) when is_binary(RW) -> 136 | try 137 | ExistingAtom = binary_to_existing_atom(RW, utf8), 138 | normalize_rw_value(ExistingAtom, N) 139 | catch _:badarg -> 140 | error 141 | end; 142 | normalize_rw_value(one, _N) -> 1; 143 | normalize_rw_value(quorum, N) -> erlang:trunc((N/2)+1); 144 | normalize_rw_value(all, N) -> N; 145 | normalize_rw_value(_, _) -> error. 146 | 147 | 148 | %% =================================================================== 149 | %% EUnit tests 150 | %% =================================================================== 151 | -ifdef(TEST). 152 | 153 | normalize_test() -> 154 | 3 = normalize_rw_value(3, 3), 155 | 1 = normalize_rw_value(one, 3), 156 | 2 = normalize_rw_value(quorum, 3), 157 | 3 = normalize_rw_value(all, 3), 158 | 1 = normalize_rw_value(<<"one">>, 3), 159 | 2 = normalize_rw_value(<<"quorum">>, 3), 160 | 3 = normalize_rw_value(<<"all">>, 3), 161 | error = normalize_rw_value(garbage, 3), 162 | error = normalize_rw_value(<<"garbage">>, 3). 163 | 164 | 165 | deleted_test() -> 166 | O = riak_object:new(<<"test">>, <<"k">>, "v"), 167 | false = is_x_deleted(O), 168 | MD = dict:new(), 169 | O1 = riak_object:apply_updates( 170 | riak_object:update_metadata( 171 | O, dict:store(<<"X-Riak-Deleted">>, true, MD))), 172 | true = is_x_deleted(O1). 173 | 174 | -endif. 175 | -------------------------------------------------------------------------------- /src/riak_kv_delete.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_delete: two-step object deletion 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc two-step object deletion 24 | 25 | -module(riak_kv_delete). 26 | 27 | %-ifdef(TEST). 28 | -include_lib("eunit/include/eunit.hrl"). 29 | %-endif. 30 | 31 | -export([start_link/6, start_link/7, start_link/8, delete/8]). 32 | 33 | start_link(ReqId, Bucket, Key, RW, Timeout, Client) -> 34 | {ok, proc_lib:spawn_link(?MODULE, delete, [ReqId, Bucket, Key, 35 | RW, Timeout, Client, undefined, 36 | undefined])}. 37 | 38 | start_link(ReqId, Bucket, Key, RW, Timeout, Client, ClientId) -> 39 | {ok, proc_lib:spawn_link(?MODULE, delete, [ReqId, Bucket, Key, 40 | RW, Timeout, Client, ClientId, 41 | undefined])}. 42 | 43 | start_link(ReqId, Bucket, Key, RW, Timeout, Client, ClientId, VClock) -> 44 | {ok, proc_lib:spawn_link(?MODULE, delete, [ReqId, Bucket, Key, 45 | RW, Timeout, Client, ClientId, 46 | VClock])}. 47 | 48 | %% @spec delete(ReqId :: binary(), riak_object:bucket(), riak_object:key(), 49 | %% RW :: integer(), TimeoutMillisecs :: integer(), Client :: pid()) 50 | %% -> term() 51 | %% @doc Delete the object at Bucket/Key. Direct return value is uninteresting, 52 | %% see riak_client:delete/3 for expected gen_server replies to Client. 53 | delete(ReqId,Bucket,Key,RW0,Timeout,Client,ClientId,undefined) -> 54 | case get_rw_val(Bucket, RW0) of 55 | error -> 56 | Client ! {ReqId, {error, {rw_val_violation, RW0}}}; 57 | RW -> 58 | RealStartTime = riak_core_util:moment(), 59 | {ok, C} = riak:local_client(), 60 | case C:get(Bucket,Key,RW,Timeout) of 61 | {ok, OrigObj} -> 62 | RemainingTime = Timeout - (riak_core_util:moment() - RealStartTime), 63 | delete(ReqId,Bucket,Key,RW,RemainingTime,Client,ClientId,riak_object:vclock(OrigObj)); 64 | {error, notfound} -> 65 | Client ! {ReqId, {error, notfound}}; 66 | X -> 67 | Client ! {ReqId, X} 68 | end 69 | end; 70 | delete(ReqId,Bucket,Key,RW0,Timeout,Client,ClientId,VClock) -> 71 | case get_rw_val(Bucket, RW0) of 72 | error -> 73 | Client ! {ReqId, {error, {rw_val_violation, RW0}}}; 74 | RW -> 75 | 76 | Obj0 = riak_object:new(Bucket, Key, <<>>, dict:store(<<"X-Riak-Deleted">>, 77 | "true", dict:new())), 78 | Tombstone = riak_object:set_vclock(Obj0, VClock), 79 | {ok,C} = riak:local_client(ClientId), 80 | Reply = C:put(Tombstone, RW, RW, Timeout), 81 | Client ! {ReqId, Reply}, 82 | case Reply of 83 | ok -> 84 | {ok, C2} = riak:local_client(), 85 | C2:get(Bucket, Key, all, Timeout); 86 | _ -> nop 87 | end 88 | end. 89 | 90 | get_rw_val(Bucket, RW0) -> 91 | {ok, Ring} = riak_core_ring_manager:get_my_ring(), 92 | BucketProps = riak_core_bucket:get_bucket(Bucket, Ring), 93 | N = proplists:get_value(n_val,BucketProps), 94 | riak_kv_util:expand_rw_value(rw, RW0, BucketProps, N). 95 | 96 | 97 | %% =================================================================== 98 | %% EUnit tests 99 | %% =================================================================== 100 | -ifdef(TEST). 101 | 102 | delete_test_() -> 103 | %% Execute the test cases 104 | {spawn, [ 105 | { foreach, 106 | fun setup/0, 107 | fun cleanup/1, 108 | [ 109 | fun invalid_rw_delete/0 110 | ] 111 | }]}. 112 | 113 | invalid_rw_delete() -> 114 | RW = <<"abc">>, 115 | %% Start the gen_fsm process 116 | RequestId = erlang:phash2(erlang:now()), 117 | Bucket = <<"testbucket">>, 118 | Key = <<"testkey">>, 119 | Timeout = 60000, 120 | riak_kv_delete_sup:start_delete(node(), [RequestId, Bucket, Key, RW, Timeout, self()]), 121 | %% Wait for error response 122 | receive 123 | {_RequestId, Result} -> 124 | ?assertEqual({error, {rw_val_violation, <<"abc">>}}, Result) 125 | after 126 | 5000 -> 127 | ?assert(false) 128 | end. 129 | 130 | setup() -> 131 | %% Shut logging up - too noisy. 132 | application:load(sasl), 133 | application:set_env(sasl, sasl_error_logger, {file, "riak_kv_delete_test_sasl.log"}), 134 | error_logger:tty(false), 135 | error_logger:logfile({open, "riak_kv_delete_test.log"}), 136 | %% Start erlang node 137 | {ok, _} = net_kernel:start([testnode, shortnames]), 138 | cleanup(unused_arg), 139 | do_dep_apps(start, dep_apps()), 140 | %% There's some weird interaction with the quickcheck tests in put_fsm_eqc 141 | %% that somehow makes the riak_kv_delete sup not be running if those tests 142 | %% run before these. I'm sick of trying to figure out what is not being 143 | %% cleaned up right, thus the following workaround. 144 | case whereis(riak_kv_delete_sup) of 145 | undefined -> 146 | {ok, _} = riak_kv_delete_sup:start_link(); 147 | _ -> 148 | ok 149 | end, 150 | timer:sleep(500). 151 | 152 | cleanup(_Pid) -> 153 | do_dep_apps(stop, lists:reverse(dep_apps())), 154 | catch exit(whereis(riak_kv_vnode_master), kill), %% Leaks occasionally 155 | catch exit(whereis(riak_sysmon_filter), kill), %% Leaks occasionally 156 | net_kernel:stop(), 157 | %% Reset the riak_core vnode_modules 158 | application:set_env(riak_core, vnode_modules, []). 159 | 160 | dep_apps() -> 161 | SetupFun = 162 | fun(start) -> 163 | %% Set some missing env vars that are normally 164 | %% part of release packaging. 165 | application:set_env(riak_core, ring_creation_size, 64), 166 | application:set_env(riak_kv, storage_backend, riak_kv_ets_backend), 167 | %% Create a fresh ring for the test 168 | Ring = riak_core_ring:fresh(), 169 | riak_core_ring_manager:set_ring_global(Ring), 170 | 171 | %% Start riak_kv 172 | timer:sleep(500); 173 | (stop) -> 174 | ok 175 | end, 176 | XX = fun(_) -> error_logger:info_msg("Registered: ~w\n", [lists:sort(registered())]) end, 177 | [sasl, crypto, riak_sysmon, webmachine, XX, riak_core, XX, luke, erlang_js, 178 | mochiweb, os_mon, SetupFun, riak_kv]. 179 | 180 | do_dep_apps(StartStop, Apps) -> 181 | lists:map(fun(A) when is_atom(A) -> application:StartStop(A); 182 | (F) -> F(StartStop) 183 | end, Apps). 184 | 185 | -endif. 186 | -------------------------------------------------------------------------------- /src/riak_kv_backup.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_backup: utilities for backup and restore of Riak nodes and clusters 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc utilities for backup and restore of Riak nodes and clusters. 24 | 25 | %% Note that if you want to restore to exactly the contents of 26 | %% a dump, you should restore to an empty cluster. Otherwise, 27 | %% restore will reconcile values with the existing data. 28 | 29 | -module(riak_kv_backup). 30 | -export ([backup/3, restore/2]). 31 | -define (TABLE, riak_kv_backup_table). 32 | 33 | %%% BACKUP %%% 34 | 35 | %% @doc 36 | %% Connect to the cluster of which EntryNode is a member, 37 | %% read data from the cluster, and save the data in the specified file. 38 | backup(EntryNode, BaseFilename, Mode) -> 39 | % Make sure we can reach the node... 40 | ensure_connected(EntryNode), 41 | 42 | % Get a list of nodes... 43 | {ok, Ring} = rpc:call(EntryNode, riak_core_ring_manager, get_my_ring, []), 44 | Members = riak_core_ring:all_members(Ring), 45 | 46 | FileName = 47 | case Mode of 48 | "all" -> 49 | io:format("Backing up (all nodes) to '~s'.~n", [BaseFilename]), 50 | io:format("...from ~p~n", [Members]), 51 | BaseFilename; 52 | "node" -> 53 | io:format("Backing up (node ~p) to '~s'.~n", [EntryNode, 54 | BaseFilename++"-"++atom_to_list(EntryNode)]), 55 | BaseFilename ++ "-" ++ EntryNode 56 | end, 57 | 58 | % Make sure all nodes in the cluster agree on the ring... 59 | ensure_synchronized(Ring, Members), 60 | 61 | % Backup the data... 62 | {ok, ?TABLE} = disk_log:open([{name, ?TABLE}, 63 | {file, FileName}, 64 | {mode, read_write}, 65 | {type, halt}]), 66 | 67 | 68 | case Mode of 69 | "all" -> 70 | [backup_node(Node, Ring) || Node <- Members]; 71 | "node" -> 72 | backup_node(EntryNode, Ring) 73 | end, 74 | io:format("syncing and closing log~n"), 75 | ok = disk_log:sync(?TABLE), 76 | ok = disk_log:close(?TABLE), 77 | 78 | % Make sure the nodes are still synchronized... 79 | ensure_synchronized(Ring, Members), 80 | ok. 81 | 82 | backup_node(Node, Ring) -> 83 | Partitions = [I || {I,N} <- riak_core_ring:all_owners(Ring), N =:= Node], 84 | backup_vnodes(Partitions, Node). 85 | 86 | backup_vnodes([], Node) -> 87 | io:format("Backup of ~p complete~n", [Node]); 88 | backup_vnodes([Partition|T], Node) -> 89 | Self = self(), 90 | Pid = spawn_link(fun() -> result_collector(Self) end), 91 | riak_kv_vnode:fold({Partition, Node}, fun backup_folder/3, Pid), 92 | Pid ! stop, 93 | receive stop -> stop end, 94 | backup_vnodes(T, Node). 95 | 96 | backup_folder(_, V, Pid) -> 97 | Pid ! {backup, V}, 98 | Pid. 99 | 100 | result_collector(PPid) -> 101 | receive 102 | stop -> 103 | PPid ! stop; 104 | {backup, M} when is_binary(M) -> 105 | disk_log:log(?TABLE, M), 106 | result_collector(PPid) 107 | end. 108 | 109 | %%% RESTORE %%% 110 | 111 | %% @doc 112 | %% Read data from the specified file created by backup/2, 113 | %% and write it to the cluster of which EntryNode is a member. 114 | restore(EntryNode, Filename) -> 115 | io:format("Restoring from '~s' to cluster to which '~s' belongs.~n", [Filename, EntryNode]), 116 | 117 | % Connect to the node... 118 | {ok, Client} = riak:client_connect(EntryNode), 119 | 120 | % Open the table, write it out, close the table... 121 | {ok, ?TABLE} = disk_log:open([{name, ?TABLE}, 122 | {file, Filename}, 123 | {mode, read_only}, 124 | {type, halt}]), 125 | Count = traverse_backup( 126 | disk_log:chunk(?TABLE, start), 127 | fun(Entry) -> read_and_restore_function(Client, Entry) end, 0), 128 | ok = disk_log:close(?TABLE), 129 | io:format("Restored ~p records.~n", [Count]), 130 | ok. 131 | 132 | traverse_backup(eof, _VisitorFun, Count) -> 133 | Count; 134 | traverse_backup({Cont, Terms}, VisitorFun, Count) when is_list(Terms) -> 135 | [VisitorFun(T) || T <- Terms], 136 | traverse_backup(disk_log:chunk(?TABLE, Cont), 137 | VisitorFun, Count+length(Terms)). 138 | 139 | 140 | read_and_restore_function(Client, BinTerm) -> 141 | Obj = binary_to_term(BinTerm), 142 | Bucket = riak_object:bucket(Obj), 143 | Key = riak_object:key(Obj), 144 | % Data Cleaning... 145 | Obj1 = make_binary_bucket(Bucket, Key, Obj), 146 | 147 | %% Store the object; be sure to tell the FSM not to update last modified! 148 | Response = Client:put(Obj1,1,1,1200000, [{update_last_modified, false}]), 149 | {continue, Response}. 150 | 151 | %%% DATA CLEANING %%% 152 | 153 | %% If the bucket name is an atom, convert it to a binary... 154 | make_binary_bucket(Bucket, Key, OriginalObj) when is_atom(Bucket) -> 155 | Bucket1 = list_to_binary(atom_to_list(Bucket)), 156 | OriginalContents = riak_object:get_contents(OriginalObj), 157 | OriginalVClock = riak_object:vclock(OriginalObj), 158 | 159 | % We can't change the bucket name without creating a new object... 160 | NewObj = riak_object:new(Bucket1, Key, placeholder), 161 | NewObj1 = riak_object:set_contents(NewObj, OriginalContents), 162 | _NewObj2 = riak_object:set_vclock(NewObj1, OriginalVClock); 163 | 164 | %% If the bucket name is a binary, just pass it on through... 165 | make_binary_bucket(Bucket, _Key, Obj) when is_binary(Bucket) -> Obj. 166 | 167 | %% @private 168 | %% Try to reach the specified node, throw exception on failure. 169 | ensure_connected(Node) -> 170 | case net_adm:ping(Node) of 171 | pang -> throw({could_not_reach_node, Node}); 172 | pong -> ok 173 | end. 174 | 175 | %% @private 176 | %% Make sure that rings of all members are synchronized, 177 | %% throw exception on failure. 178 | ensure_synchronized(Ring, Members) -> 179 | F = fun(Node) -> 180 | {ok, Ring2} = rpc:call(Node, riak_core_ring_manager, get_my_ring, []), 181 | riak_core_ring:equal_rings(Ring, Ring2) 182 | end, 183 | case lists:all(F, Members) of 184 | true -> ok; 185 | false -> throw({nodes_not_synchronized, Members}) 186 | end. 187 | 188 | % pmap(Fun, List) -> 189 | % Workers = [spawn_worker(self(), Pred, Data) || X <- List], 190 | % [wait_result(Worker) || Worker <- Workers]. 191 | % 192 | % spawn_worker(Parent, Fun, Data) -> 193 | % erlang:spawn_monitor(fun() -> Parent ! {self(), Fun(Data)} end). 194 | % 195 | % wait_result({Pid,Ref}) -> 196 | % receive 197 | % {'DOWN', Ref, _, _, normal} -> receive {Pid,Result} -> Result end; 198 | % {'DOWN', Ref, _, _, Reason} -> exit(Reason) 199 | % end. 200 | -------------------------------------------------------------------------------- /src/riak_kv_legacy_vnode.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_kv_legacy_vnode: Message translation for legacy vnode requests 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc QuickCheck tests for riak_core_vnode code 24 | 25 | -module(riak_kv_legacy_vnode). 26 | -export([rewrite_cast/1, 27 | rewrite_call/2]). 28 | -include_lib("riak_kv_vnode.hrl"). 29 | -ifdef(TEST). 30 | -include_lib("eunit/include/eunit.hrl"). 31 | -endif. 32 | 33 | %% @private 34 | 35 | rewrite_cast({vnode_map, {Partition,_Node}, 36 | {ClientPid,QTerm,BKey,KeyData}}) -> 37 | Req = riak_core_vnode_master:make_request( 38 | ?KV_MAP_REQ{ 39 | qterm = QTerm, 40 | bkey = BKey, 41 | keydata = KeyData}, 42 | {fsm, undefined, ClientPid}, 43 | Partition), 44 | {ok, Req}; 45 | rewrite_cast({vnode_put, {Partition,_Node}, 46 | {FSM_pid,BKey,RObj,ReqID,FSMTime,Options}}) -> 47 | Req = riak_core_vnode_master:make_request( 48 | ?KV_PUT_REQ{ 49 | bkey = BKey, 50 | object = RObj, 51 | req_id = ReqID, 52 | start_time = FSMTime, 53 | options = Options}, 54 | {fsm, undefined, FSM_pid}, 55 | Partition), 56 | {ok, Req}; 57 | rewrite_cast({vnode_get, {Partition,_Node}, 58 | {FSM_pid,BKey,ReqId}}) -> 59 | Req = riak_core_vnode_master:make_request( 60 | ?KV_GET_REQ{ 61 | bkey = BKey, 62 | req_id = ReqId 63 | }, 64 | {fsm, undefined, FSM_pid}, 65 | Partition), 66 | {ok, Req}; 67 | %% rewrite_cast({vnode_merkle, {RemoteVN,Partition,Merkle,ObjList}}, State) -> 68 | %% Pid = get_vnode(Partition, State), 69 | %% gen_fsm:send_event(Pid, {vnode_merkle, {RemoteVN,Merkle,ObjList}}), 70 | %% {noreply, State}; 71 | rewrite_cast({vnode_list_bucket, {Partition,_Node}, 72 | {FSM_pid, Bucket, ReqID}}) -> 73 | Req = riak_core_vnode_master:make_request( 74 | #riak_kv_listkeys_req_v1{ 75 | bucket=Bucket, 76 | req_id=ReqID}, 77 | {fsm, undefined, FSM_pid}, 78 | Partition), 79 | {ok, Req}. 80 | %% rewrite_cast({add_exclusion, Partition}, State=#state{excl=Excl}) -> 81 | %% {ok, Ring} = riak_core_ring_manager:get_my_ring(), 82 | %% riak_core_ring_events:ring_update(Ring), 83 | %% {noreply, State#state{excl=ordsets:add_element(Partition, Excl)}}. 84 | 85 | %% @private 86 | %% rewrite_call(all_possible_vnodes, _From, State) -> 87 | %% {reply, make_all_active(State), State}; 88 | %% rewrite_call(all_vnodes, _From, State) -> 89 | %% {reply, all_vnodes(State), State}; 90 | rewrite_call({vnode_del, {Partition,_Node}, 91 | {BKey,ReqID}}, _From) -> 92 | Req = riak_core_vnode_master:make_request( 93 | ?KV_DELETE_REQ{bkey=BKey, 94 | req_id=ReqID}, 95 | ignore, 96 | Partition), 97 | {ok, Req}. 98 | %% rewrite_call({get_merkle, Partition}, From, State) -> 99 | %% Pid = get_vnode(Partition, State), 100 | %% spawn(fun() -> gen_fsm:send_all_state_event(Pid, {get_merkle, From}) end), 101 | %% {noreply, State}; 102 | %% rewrite_call({get_vclocks,Partition,KeyList},From,State) -> 103 | %% Pid = get_vnode(Partition, State), 104 | %% spawn(fun() -> gen_fsm:send_all_state_event( 105 | %% Pid,{get_vclocks,From,KeyList}) end), 106 | %% {noreply, State}; 107 | %% rewrite_call({fold, {Partition, Fun, Acc0}}, From, State) -> 108 | %% Pid = get_vnode(Partition, State), 109 | %% spawn( 110 | %% fun() -> gen_fsm:send_all_state_event(Pid, {fold, {Fun,Acc0,From}}) end), 111 | %% {noreply, State}; 112 | %% rewrite_call({get_vnode, Partition}, _From, State) -> 113 | %% {reply, {ok, get_vnode(Partition, State)}, State}; 114 | %% rewrite_call(get_exclusions, _From, State=#state{excl=Excl}) -> 115 | %% {reply, {ok, ordsets:to_list(Excl)}, State}. 116 | 117 | -ifdef(TEST). 118 | 119 | -define(RING_KEY, riak_ring). 120 | 121 | start_servers() -> 122 | %% dbgh:start(), 123 | %% dbgh:trace(riak_core_vnode_master), 124 | %% dbgh:trace(riak_core_vnode), 125 | %% dbgh:trace(riak_kv_vnode), 126 | %% dbgh:trace(?MODULE), 127 | application:set_env(riak_kv, storage_backend, riak_kv_gb_trees_backend), 128 | application:set_env(riak_core, default_bucket_props, []), 129 | Ring = riak_core_ring:fresh(16, node()), 130 | mochiglobal:put(?RING_KEY, Ring), 131 | riak_kv_test_util:stop_process(riak_kv_vnode_master), 132 | riak_kv_test_util:stop_process(riak_core_vnode_sup), 133 | {ok, _Sup} = riak_core_vnode_sup:start_link(), 134 | {ok, _Vmaster} = riak_core_vnode_master:start_link(riak_kv_vnode, ?MODULE), 135 | ok. 136 | 137 | stop_servers(_R) -> 138 | riak_kv_test_util:stop_process(riak_kv_vnode_master), 139 | riak_kv_test_util:stop_process(riak_core_vnode_sup). 140 | 141 | legacy_kv_test_() -> 142 | {spawn, 143 | {setup, fun start_servers/0, fun stop_servers/1, 144 | [{"get", ?_test( 145 | begin 146 | send_0_11_0_cmd(vnode_get, {self(), {<<"bucket">>,<<"key">>}, 123}), 147 | receive 148 | Msg -> 149 | ?assertEqual({'$gen_event',{r,{error,notfound},0,123}}, Msg) 150 | after 151 | 100 -> 152 | ?assert(false) 153 | end 154 | end)}, 155 | {"put", ?_test( 156 | begin 157 | Bucket = <<"bucket">>, 158 | Key = <<"key">>, 159 | RObj1 = riak_object:new(Bucket, Key, <<"val">>), 160 | ReqId = 456, 161 | RealStartTime = {0,0,0}, 162 | Options = [], 163 | send_0_11_0_cmd(vnode_put, 164 | {self(), {Bucket,Key}, RObj1, ReqId, RealStartTime, Options}), 165 | receive 166 | Msg -> 167 | ?assertEqual({'$gen_event',{w,0,456}}, Msg), 168 | receive 169 | Msg2 -> 170 | ?assertEqual({'$gen_event',{dw,0,456}}, Msg2) 171 | after 172 | 100 -> 173 | ?assert(false) 174 | end 175 | after 176 | 100 -> 177 | ?assert(false) 178 | end 179 | end)}, 180 | {"list bucket", ?_test( 181 | begin 182 | Bucket = <<"listbucket">>, 183 | ReqID = 789, 184 | send_0_11_0_cmd(vnode_list_bucket,{self(), Bucket, ReqID}), 185 | receive 186 | Msg -> 187 | ?assertEqual({'$gen_event',{kl,[],0,789}}, Msg) 188 | after 189 | 100 -> 190 | ?assert(false) 191 | end 192 | end)}] 193 | }}. 194 | 195 | send_0_11_0_cmd(Cmd, Msg) -> 196 | gen_server:cast({riak_kv_vnode_master, node()}, 197 | {Cmd, {0, node()}, Msg}). 198 | 199 | 200 | -endif. 201 | 202 | 203 | -------------------------------------------------------------------------------- /test/fsm_eqc_util.erl: -------------------------------------------------------------------------------- 1 | -module(fsm_eqc_util). 2 | -compile([export_all]). 3 | 4 | -ifdef(EQC). 5 | 6 | -include_lib("eqc/include/eqc.hrl"). 7 | -define(RING_KEY, riak_ring). 8 | 9 | not_empty(G) -> 10 | ?SUCHTHAT(X, G, X /= [] andalso X /= <<>>). 11 | 12 | longer_list(K, G) -> 13 | ?SIZED(Size, resize(trunc(K*Size), list(resize(Size, G)))). 14 | 15 | node_status() -> 16 | frequency([{1, ?SHRINK(down, [up])}, 17 | {9, up}]). 18 | 19 | %% Make sure at least one node is up - code in riak_kv_util makes 20 | %% some assumptions that the node the get FSM is running on is 21 | %% in the cluster causing problems if it isn't. 22 | at_least_one_up(G) -> 23 | ?SUCHTHAT(X, G, lists:member(up, X)). 24 | 25 | num_partitions() -> 26 | %% TODO: use some unfortunate partition counts (1, 50, etc.) 27 | % elements([4, 16, 64]). 28 | ?LET(N, choose(0, 6), pow(2, N)). 29 | 30 | largenat() -> 31 | ?LET(X, largeint(), abs(X)). 32 | 33 | bkey() -> 34 | %%TODO: "make this nastier" 35 | %%TODO: once json encoding of bkeys as binaries rather than utf8 strings 36 | %% start creating general binaries instead 37 | {non_blank_string(), %% bucket 38 | non_blank_string()}. %% key 39 | 40 | non_blank_string() -> 41 | ?LET(X,not_empty(list(lower_char())), list_to_binary(X)). 42 | 43 | %% Generate a lower 7-bit ACSII character that should not cause any problems 44 | %% with utf8 conversion. 45 | lower_char() -> 46 | choose(16#20, 16#7f). 47 | 48 | 49 | vclock() -> 50 | ?LET(VclockSym, vclock_sym(), eval(VclockSym)). 51 | 52 | vclock_sym() -> 53 | ?LAZY( 54 | oneof([ 55 | {call, vclock, fresh, []}, 56 | ?LETSHRINK([Clock], [vclock_sym()], 57 | {call, ?MODULE, increment, 58 | [noshrink(binary(4)), nat(), Clock]}) 59 | ])). 60 | 61 | increment(Actor, Count, Vclock) -> 62 | lists:foldl( 63 | fun vclock:increment/2, 64 | Vclock, 65 | lists:duplicate(Count, Actor)). 66 | 67 | riak_object() -> 68 | ?LET({{Bucket, Key}, Vclock, Value}, 69 | {bkey(), vclock(), binary()}, 70 | riak_object:set_vclock( 71 | riak_object:new(Bucket, Key, Value), 72 | Vclock)). 73 | 74 | maybe_tombstone() -> 75 | weighted_default({2, notombstone}, {1, tombstone}). 76 | 77 | %% 78 | %% ancestor 79 | %% / | \ 80 | %% brother sister otherbrother 81 | %% \ | / 82 | %% current 83 | %% 84 | lineage() -> 85 | elements([current, ancestor, brother, sister, otherbrother]). 86 | 87 | merge(ancestor, Lineage) -> Lineage; % order should match Clocks list in riak_objects 88 | merge(Lineage, ancestor) -> Lineage; % as last modified is used as tie breaker with 89 | merge(_, current) -> current; % allow_mult=false 90 | merge(current, _) -> current; 91 | merge(otherbrother, _) -> otherbrother; 92 | merge(_, otherbrother) -> otherbrother; 93 | merge(sister, _) -> sister; 94 | merge(_, sister) -> sister; 95 | merge(brother, _) -> brother; 96 | merge(_, brother) -> brother. 97 | 98 | merge([Lin]) -> 99 | Lin; 100 | merge([Lin|Lins]) -> 101 | merge(Lin, merge(Lins)). 102 | 103 | partval() -> 104 | Shrink = fun(G) -> ?SHRINK(G, [{ok, current}]) end, 105 | frequency([{2,{ok, lineage()}}, 106 | {1,Shrink(notfound)}, 107 | {1,Shrink(timeout)}, 108 | {1,Shrink(error)}]). 109 | 110 | partvals() -> 111 | not_empty(fsm_eqc_util:longer_list(2, partval())). 112 | 113 | %% Generate 5 riak objects with the same bkey 114 | %% 115 | riak_objects() -> 116 | ?LET({{Bucket,Key},AncestorVclock0,Tombstones}, 117 | {noshrink(bkey()),vclock(),vector(5, maybe_tombstone())}, 118 | begin 119 | AncestorVclock = vclock:increment(<<"dad">>, AncestorVclock0), 120 | BrotherVclock = vclock:increment(<<"bro!">>, AncestorVclock), 121 | OtherBroVclock = vclock:increment(<<"bro2">>, AncestorVclock), 122 | SisterVclock = vclock:increment(<<"sis!">>, AncestorVclock), 123 | CurrentVclock = vclock:merge([BrotherVclock,SisterVclock,OtherBroVclock]), 124 | Clocks = [{ancestor, AncestorVclock, <<"ancestor">>}, 125 | {brother, BrotherVclock, <<"brother">>}, 126 | {sister, SisterVclock, <<"sister">>}, 127 | {otherbrother, OtherBroVclock, <<"otherbrother">>}, 128 | {current, CurrentVclock, <<"current">>}], 129 | [ {Lineage, build_riak_obj(Bucket, Key, Vclock, Value, Tombstone)} 130 | || {{Lineage, Vclock, Value}, Tombstone} <- lists:zip(Clocks, Tombstones) ] 131 | end). 132 | 133 | build_riak_obj(B,K,Vc,Val,notombstone) -> 134 | riak_object:set_contents( 135 | riak_object:set_vclock( 136 | riak_object:new(B,K,Val), 137 | Vc), 138 | [{dict:from_list([{<<"X-Riak-Last-Modified">>,now()}]), Val}]); 139 | build_riak_obj(B,K,Vc,Val,tombstone) -> 140 | Obj = build_riak_obj(B,K,Vc,Val,notombstone), 141 | add_tombstone(Obj). 142 | 143 | add_tombstone(Obj) -> 144 | [{M,V}] = riak_object:get_contents(Obj), 145 | NewM = dict:store(<<"X-Riak-Deleted">>, true, M), 146 | riak_object:set_contents(Obj, [{NewM, V}]). 147 | 148 | 149 | some_up_node_status(NumNodes) -> 150 | at_least_one_up(nodes_status(NumNodes)). 151 | 152 | nodes_status(NumNodes) -> 153 | non_empty(longer_list(NumNodes, node_status())). 154 | 155 | pow(_, 0) -> 1; 156 | pow(A, N) -> A * pow(A, N - 1). 157 | 158 | make_power_of_two(Q) -> make_power_of_two(Q, 1). 159 | 160 | make_power_of_two(Q, P) when P >= Q -> P; 161 | make_power_of_two(Q, P) -> make_power_of_two(Q, P*2). 162 | 163 | cycle(N, Xs=[_|_]) when N >= 0 -> 164 | cycle(Xs, N, Xs). 165 | 166 | cycle(_Zs, 0, _Xs) -> 167 | []; 168 | cycle(Zs, N, [X|Xs]) -> 169 | [X|cycle(Zs, N - 1, Xs)]; 170 | cycle(Zs, N, []) -> 171 | cycle(Zs, N, Zs). 172 | 173 | start_mock_servers() -> 174 | %% Start new core_vnode based EQC FSM test mock 175 | case whereis(fsm_eqc_vnode) of 176 | undefined -> ok; 177 | Pid2 -> 178 | unlink(Pid2), 179 | exit(Pid2, shutdown), 180 | riak_kv_test_util:wait_for_pid(Pid2) 181 | end, 182 | {ok, _Pid3} = fsm_eqc_vnode:start_link(), 183 | application:unload(riak_core), 184 | ok = application:load(riak_core), 185 | application:start(crypto), 186 | riak_core_ring_events:start_link(), 187 | riak_core_node_watcher_events:start_link(), 188 | riak_core_node_watcher:start_link(), 189 | riak_core_node_watcher:service_up(riak_kv, self()), 190 | ok. 191 | 192 | cleanup_mock_servers() -> 193 | application:unload(riak_core). 194 | 195 | make_options([], Options) -> 196 | Options; 197 | make_options([{_Name, missing} | Rest], Options) -> 198 | make_options(Rest, Options); 199 | make_options([Option | Rest], Options) -> 200 | make_options(Rest, [Option | Options]). 201 | 202 | mock_ring(Q0, NodeStatus0) -> 203 | %% Round up to next power of two 204 | Q = fsm_eqc_util:make_power_of_two(Q0), 205 | 206 | %% Expand the node status to match the size of the ring 207 | NodeStatus = cycle(Q, NodeStatus0), 208 | 209 | %% Assign the node owners and store the ring. 210 | Ring = reassign_nodes(NodeStatus, riak_core_ring:fresh(Q, node())), 211 | mochiglobal:put(?RING_KEY, Ring), 212 | 213 | %% Return details - useful for ?WHENFAILs 214 | {Q, Ring, NodeStatus}. 215 | 216 | reassign_nodes(Status, Ring) -> 217 | Ids = [ I || {I, _} <- riak_core_ring:all_owners(Ring) ], 218 | lists:foldl( 219 | fun({down, Id}, R) -> 220 | riak_core_ring:transfer_node(Id, 'notanode@localhost', R); 221 | (_, R) -> R 222 | end, Ring, lists:zip(Status, Ids)). 223 | 224 | 225 | wait_for_req_id(ReqId, Pid) -> 226 | receive 227 | {'EXIT', Pid, _Reason} -> 228 | io:format(user, "FSM died:\n~p\n", [_Reason]), 229 | %{exit, _Reason}; 230 | %% Mark as timeout for now - no reply is coming, so why wait 231 | timeout; 232 | {'EXIT', _OtherPid, _Reason} -> 233 | %% Probably from previous test death 234 | wait_for_req_id(ReqId, Pid); 235 | {ReqId, Response} -> 236 | Response; 237 | Anything1 -> 238 | {anything, Anything1} 239 | after 400 -> 240 | timeout 241 | end. 242 | 243 | -endif. % EQC 244 | -------------------------------------------------------------------------------- /src/riak.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% Riak: A lightweight, decentralized key-value store. 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | -module(riak). 24 | -author('Andy Gross '). 25 | -author('Justin Sheehy '). 26 | -author('Bryan Fink '). 27 | -export([stop/0, stop/1]). 28 | -export([get_app_env/0, get_app_env/1,get_app_env/2]). 29 | -export([client_connect/1,client_connect/2, 30 | client_test/1, 31 | local_client/0,local_client/1, 32 | join/1]). 33 | -export([code_hash/0]). 34 | 35 | -ifdef(TEST). 36 | -include_lib("eunit/include/eunit.hrl"). 37 | -endif. 38 | 39 | 40 | %% @spec stop() -> ok 41 | %% @doc Stop the riak application and the calling process. 42 | stop() -> stop("riak stop requested"). 43 | stop(Reason) -> 44 | % we never do an application:stop because that makes it very hard 45 | % to really halt the runtime, which is what we need here. 46 | error_logger:info_msg(io_lib:format("~p~n",[Reason])), 47 | init:stop(). 48 | 49 | %% @spec get_app_env() -> [{Key :: atom(), Value :: term()}] 50 | %% @doc Retrieve all values set in riak's configuration file. 51 | %% Returns a list of Key/Value pairs. 52 | get_app_env() -> 53 | application:get_all_env(riak) ++ init:get_arguments(). 54 | 55 | %% @spec get_app_env(Opt :: atom()) -> term() 56 | %% @doc The official way to get the values set in riak's configuration file. 57 | %% Will return the undefined atom if that option is unset. 58 | get_app_env(Opt) -> get_app_env(Opt, undefined). 59 | 60 | %% @spec get_app_env(Opt :: atom(), Default :: term()) -> term() 61 | %% @doc The official way to get the values set in riak's configuration file. 62 | %% Will return Default if that option is unset. 63 | get_app_env(Opt, Default) -> 64 | case application:get_env(riak, Opt) of 65 | {ok, Val} -> Val; 66 | _ -> 67 | case init:get_argument(Opt) of 68 | {ok, [[Val | _]]} -> Val; 69 | error -> Default 70 | end 71 | end. 72 | 73 | %% @spec local_client() -> {ok, Client :: riak_client()} 74 | %% @equiv local_client(undefined) 75 | local_client() -> 76 | local_client(undefined). 77 | 78 | %% @spec local_client(binary()|undefined) -> {ok, Client :: riak_client()} 79 | %% @doc When you want a client for use on a running Riak node. 80 | %% ClientId should be a 32-bit binary. If it is not, a 81 | %% 32-bit binary will be created from ClientId by phash2/1. 82 | %% If ClientId is the atom 'undefined', a random ClientId will 83 | %% be chosen. 84 | local_client(ClientId) -> 85 | client_connect(node(), ClientId). 86 | 87 | %% @spec client_connect(Node :: node()) 88 | %% -> {ok, Client :: riak_client()} | {error, timeout} 89 | %% @equiv client_connect(Node, undefined) 90 | client_connect(Node) -> 91 | client_connect(Node, undefined). 92 | 93 | %% @spec client_connect(node(), binary()|undefined) 94 | %% -> {ok, Client :: riak_client} | {error, timeout} 95 | %% @doc The usual way to get a client. Timeout often means either a bad 96 | %% cookie or a poorly-connected distributed erlang network. 97 | %% ClientId should be a 32-bit binary. If it is not, a 98 | %% 32-bit binary will be created from ClientId by phash2/1. 99 | %% If ClientId is the atom 'undefined', a random ClientId will 100 | %% be chosen. 101 | client_connect(Node, ClientId= <<_:32>>) -> 102 | % Make sure we can reach this node... 103 | case net_adm:ping(Node) of 104 | pang -> {error, {could_not_reach_node, Node}}; 105 | pong -> {ok, riak_client:new(Node, ClientId)} 106 | end; 107 | client_connect(Node, undefined) -> 108 | client_connect(Node, riak_core_util:mkclientid(Node)); 109 | client_connect(Node, Other) -> 110 | client_connect(Node, <<(erlang:phash2(Other)):32>>). 111 | 112 | %% 113 | %% @doc Validate that a specified node is accessible and functional. 114 | %% 115 | client_test(NodeStr) when is_list(NodeStr) -> 116 | client_test(riak_core_util:str_to_node(NodeStr)); 117 | client_test(Node) -> 118 | case net_adm:ping(Node) of 119 | pong -> 120 | case client_connect(Node) of 121 | {ok, Client} -> 122 | case client_test_phase1(Client) of 123 | ok -> 124 | error_logger:info_msg("Successfully completed 1 read/write cycle to ~p\n", [Node]), 125 | ok; 126 | error -> 127 | error 128 | end; 129 | Error -> 130 | error_logger:error_msg("Error creating client connection to ~s: ~p\n", 131 | [Node, Error]), 132 | error 133 | end; 134 | pang -> 135 | error_logger:error_msg("Node ~p is not reachable from ~p.\n", [Node, node()]), 136 | error 137 | end. 138 | 139 | 140 | %% 141 | %% @doc Join the ring found on the specified remote node 142 | %% 143 | join(NodeStr) when is_list(NodeStr) -> 144 | join(riak_core_util:str_to_node(NodeStr)); 145 | join(Node) when is_atom(Node) -> 146 | {ok, OurRingSize} = application:get_env(riak_core, ring_creation_size), 147 | case net_adm:ping(Node) of 148 | pong -> 149 | case rpc:call(Node, 150 | application, 151 | get_env, 152 | [riak_core, ring_creation_size]) of 153 | {ok, OurRingSize} -> 154 | riak_core_gossip:send_ring(Node, node()); 155 | _ -> 156 | {error, different_ring_sizes} 157 | end; 158 | pang -> 159 | {error, not_reachable} 160 | end. 161 | 162 | 163 | code_hash() -> 164 | {ok, AllMods0} = application:get_key(riak, modules), 165 | AllMods = lists:sort(AllMods0), 166 | <> = erlang:md5_final( 167 | lists:foldl( 168 | fun(D, C) -> erlang:md5_update(C, D) end, 169 | erlang:md5_init(), 170 | [C || {_, C, _} <- [code:get_object_code(M) || M <- AllMods]] 171 | )), 172 | riak_core_util:integer_to_list(MD5Sum, 62). 173 | 174 | 175 | %% 176 | %% Internal functions for testing a Riak node through single read/write cycle 177 | %% 178 | -define(CLIENT_TEST_BUCKET, <<"__riak_client_test__">>). 179 | -define(CLIENT_TEST_KEY, <<"key1">>). 180 | 181 | client_test_phase1(Client) -> 182 | case Client:get(?CLIENT_TEST_BUCKET, ?CLIENT_TEST_KEY, 1) of 183 | {ok, Object} -> 184 | client_test_phase2(Client, Object); 185 | {error, notfound} -> 186 | client_test_phase2(Client, riak_object:new(?CLIENT_TEST_BUCKET, ?CLIENT_TEST_KEY, undefined)); 187 | Error -> 188 | error_logger:error_msg("Failed to read test value: ~p\n", [Error]), 189 | error 190 | end. 191 | 192 | client_test_phase2(Client, Object0) -> 193 | Now = calendar:universal_time(), 194 | Object = riak_object:update_value(Object0, Now), 195 | case Client:put(Object, 1) of 196 | ok -> 197 | client_test_phase3(Client, Now); 198 | Error -> 199 | error_logger:error_msg("Failed to write test value: ~p\n", [Error]), 200 | error 201 | end. 202 | 203 | client_test_phase3(Client, WrittenValue) -> 204 | case Client:get(?CLIENT_TEST_BUCKET, ?CLIENT_TEST_KEY, 1) of 205 | {ok, Object} -> 206 | case lists:member(WrittenValue, riak_object:get_values(Object)) of 207 | true -> 208 | ok; 209 | false -> 210 | error_logger:error_msg("Failed to find test value in list of objects. Expected: ~p\Actual: ~p\n", 211 | [WrittenValue, riak_object:get_values(Object)]), 212 | error 213 | end; 214 | Error -> 215 | error_logger:error_msg("Failed to read test value: ~p\n", [Error]), 216 | error 217 | end. 218 | -------------------------------------------------------------------------------- /src/riak_kv_mapred_query.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_mapred_query: driver for mapreduce query 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc riak_kv_mapred_query is the driver of a mapreduce query. 24 | %% 25 | %% Map phases are expected to have inputs of the form 26 | %% [{Bucket,Key}] or [{{Bucket,Key},KeyData}] (the first form is 27 | %% equivalent to [{{Bucket,Key},undefined}]) and will execute 28 | %% with locality to each key and must return a list that is valid 29 | %% input to the next phase 30 | %% 31 | %% Reduce phases take any list, but the function must be 32 | %% commutative and associative, and the next phase will block 33 | %% until the reduce phase is entirely done, and the reduce fun 34 | %% must return a list that is valid input to the next phase 35 | %% 36 | %% Valid terms for Query: 37 | %%
    38 | %%
  • {link, Bucket, Tag, Acc}
  • 39 | %%
  • {map, FunTerm, Arg, Acc}
  • 40 | %%
  • {reduce, FunTerm, Arg, Acc}
  • 41 | %%
42 | %% where FunTerm is one of: 43 | %%
    44 | %%
  • {modfun, Mod, Fun} : Mod and Fun both atoms -> 45 | %% Mod:Fun(Object,KeyData,Arg)
  • 46 | %%
  • {qfun, Fun} : Fun is an actual fun -> 47 | %% Fun(Object,KeyData,Arg)
  • 48 | %%
  • {strfun, Fun} : Fun is a string (list or binary) 49 | %% containing the definition of an anonymous 50 | %% Erlang function.
  • 51 | %%
52 | %% @type mapred_queryterm() = 53 | %% {map, mapred_funterm(), Arg :: term(), 54 | %% Accumulate :: boolean()} | 55 | %% {reduce, mapred_funterm(), Arg :: term(), 56 | %% Accumulate :: boolean()} | 57 | %% {link, Bucket :: riak_object:bucket(), Tag :: term(), 58 | %% Accumulate :: boolean()} 59 | %% @type mapred_funterm() = 60 | %% {modfun, Module :: atom(), Function :: atom()}| 61 | %% {qfun, function()}| 62 | %% {strfun, list() | binary()} 63 | %% @type mapred_result() = [term()] 64 | 65 | -module(riak_kv_mapred_query). 66 | 67 | -export([start/6]). 68 | 69 | start(Node, Client, ReqId, Query0, ResultTransformer, Timeout) -> 70 | EffectiveTimeout = erlang:trunc(Timeout * 1.1), 71 | case check_query_syntax(Query0) of 72 | {ok, Query} -> 73 | luke:new_flow(Node, Client, ReqId, Query, ResultTransformer, EffectiveTimeout); 74 | {bad_qterm, QTerm} -> 75 | {stop, {bad_qterm, QTerm}} 76 | end. 77 | 78 | check_query_syntax(Query) -> 79 | check_query_syntax(lists:reverse(Query), []). 80 | 81 | check_query_syntax([], Accum) -> 82 | {ok, Accum}; 83 | check_query_syntax([QTerm={QTermType, QueryFun, Misc, Acc}|Rest], Accum) when is_boolean(Acc) -> 84 | PhaseDef = case QTermType of 85 | link -> 86 | {phase_mod(link), phase_behavior(link, QueryFun, Acc), [{erlang, QTerm}]}; 87 | T when T =:= map orelse T=:= reduce -> 88 | case QueryFun of 89 | {modfun, Mod, Fun} when is_atom(Mod), 90 | is_atom(Fun) -> 91 | {phase_mod(T), phase_behavior(T, QueryFun, Acc), [{erlang, QTerm}]}; 92 | {qfun, Fun} when is_function(Fun) -> 93 | {phase_mod(T), phase_behavior(T, QueryFun, Acc), [{erlang, QTerm}]}; 94 | {strfun, FunStr} -> 95 | handle_strfun(FunStr, QTerm); 96 | {jsanon, JS} when is_binary(JS) -> 97 | {phase_mod(T), phase_behavior(T, QueryFun, Acc), [{javascript, QTerm}]}; 98 | {jsanon, {Bucket, Key}} when is_binary(Bucket), 99 | is_binary(Key) -> 100 | case fetch_src(Bucket, Key) of 101 | {ok, JS} -> 102 | {phase_mod(T), phase_behavior(T, QueryFun, Acc), [{javascript, 103 | {T, {jsanon, JS}, Misc, Acc}}]}; 104 | _ -> 105 | {bad_qterm, QTerm} 106 | end; 107 | {jsfun, JS} when is_binary(JS) -> 108 | {phase_mod(T), phase_behavior(T, QueryFun, Acc), [{javascript, QTerm}]}; 109 | _ -> 110 | {bad_qterm, QTerm} 111 | end 112 | end, 113 | case PhaseDef of 114 | {bad_qterm, _} -> 115 | PhaseDef; 116 | _ -> 117 | check_query_syntax(Rest, [PhaseDef|Accum]) 118 | end. 119 | 120 | phase_mod(link) -> 121 | riak_kv_map_phase; 122 | phase_mod(map) -> 123 | riak_kv_map_phase; 124 | phase_mod(reduce) -> 125 | riak_kv_reduce_phase. 126 | 127 | phase_behavior(link, _QueryFun, true) -> 128 | [accumulate]; 129 | phase_behavior(link, _QueryFun, false) -> 130 | []; 131 | phase_behavior(map, _QueryFun, true) -> 132 | [accumulate]; 133 | phase_behavior(map, _QueryFun, false) -> 134 | []; 135 | phase_behavior(reduce, _QueryFun, Accumulate) -> 136 | Behaviors0 = [{converge, 2}], 137 | case Accumulate of 138 | true -> 139 | [accumulate|Behaviors0]; 140 | false -> 141 | Behaviors0 142 | end. 143 | 144 | fetch_src(Bucket, Key) -> 145 | {ok, Client} = riak:local_client(), 146 | case Client:get(Bucket, Key, 1) of 147 | {ok, Obj} -> 148 | {ok, riak_object:get_value(Obj)}; 149 | _ -> 150 | {error, bad_fetch} 151 | end. 152 | 153 | define_anon_erl(FunStr) when is_binary(FunStr) -> 154 | define_anon_erl(binary_to_list(FunStr)); 155 | define_anon_erl(FunStr) when is_list(FunStr) -> 156 | {ok, Tokens, _} = erl_scan:string(FunStr), 157 | {ok, [Form]} = erl_parse:parse_exprs(Tokens), 158 | {value, Fun, _} = erl_eval:expr(Form, erl_eval:new_bindings()), 159 | Fun. 160 | 161 | handle_strfun(FunStr, QTerm) -> 162 | case application:get_env(riak_kv, allow_strfun) of 163 | {ok, true} -> 164 | handle_strfun1(FunStr, QTerm); 165 | _ -> 166 | {bad_qterm, QTerm} 167 | end. 168 | 169 | handle_strfun1({Bucket, Key}, QTerm) when is_binary(Bucket), is_binary(Key) -> 170 | case fetch_src(Bucket, Key) of 171 | {ok, FunStr} -> 172 | handle_strfun(FunStr, QTerm); 173 | _ -> 174 | {bad_qterm, QTerm} 175 | end; 176 | handle_strfun1(FunStr, QTerm={T, QueryFun, Misc, Acc}) 177 | when is_binary(FunStr); is_list(FunStr) -> 178 | case catch define_anon_erl(FunStr) of 179 | Fun when is_function(Fun, 3) -> 180 | {phase_mod(T), phase_behavior(T, QueryFun, Acc), 181 | [{erlang, {T, {qfun, Fun}, Misc, Acc}}]}; 182 | _ -> 183 | {bad_qterm, QTerm} 184 | end; 185 | handle_strfun1(_, QTerm) -> 186 | {bad_qterm, QTerm}. 187 | 188 | 189 | -ifdef(TEST). 190 | -include_lib("eunit/include/eunit.hrl"). 191 | 192 | strfun_test() -> 193 | application:set_env(riak_kv, allow_strfun, true), 194 | Query = [{map, {strfun, "fun(_,_,_) -> [] end."}, none, true}], 195 | {ok, [{riak_kv_map_phase, [accumulate], [{erlang, {map, {qfun, Fun}, none, true}}]}]} 196 | = check_query_syntax(Query), 197 | ?assertEqual(true, erlang:is_function(Fun, 3)). 198 | 199 | bad_strfun_test() -> 200 | application:set_env(riak_kv, allow_strfun, true), 201 | Query = [{map, {strfun, "fun(_,_,_) -> [] end"}, none, true}], 202 | {bad_qterm, _} = check_query_syntax(Query). 203 | 204 | unbound_var_strfun_test() -> 205 | application:set_env(riak_kv, allow_strfun, true), 206 | Query = [{map, {strfun, "fun(_,_,_) -> [UnboundVar] end."}, none, true}], 207 | {bad_qterm, _} = check_query_syntax(Query). 208 | 209 | disabled_strfun_test() -> 210 | application:set_env(riak_kv, allow_strfun, false), 211 | Query = [{map, {strfun, "fun(_,_,_) -> [] end."}, none, true}], 212 | {bad_qterm, _} = check_query_syntax(Query). 213 | 214 | -endif. 215 | -------------------------------------------------------------------------------- /src/riak_kv_wm_mapred.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_kv_wm_mapred: webmachine resource for mapreduce requests 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc webmachine resource for mapreduce requests 24 | 25 | -module(riak_kv_wm_mapred). 26 | 27 | -export([init/1, service_available/2, allowed_methods/2]). 28 | -export([malformed_request/2, process_post/2, content_types_provided/2]). 29 | -export([nop/2]). 30 | 31 | -include_lib("webmachine/include/webmachine.hrl"). 32 | 33 | -define(DEFAULT_TIMEOUT, 60000). 34 | 35 | 36 | -record(state, {client, inputs, timeout, mrquery, boundary}). 37 | 38 | init(_) -> 39 | {ok, undefined}. 40 | 41 | service_available(RD, State) -> 42 | case riak:local_client() of 43 | {ok, Client} -> 44 | {true, RD, #state{client=Client}}; 45 | Error -> 46 | error_logger:error_report(Error), 47 | {false, RD, State} 48 | end. 49 | 50 | allowed_methods(RD, State) -> 51 | {['GET','HEAD','POST'], RD, State}. 52 | 53 | malformed_request(RD, State) -> 54 | {Verified, Message, NewState} = 55 | case {wrq:method(RD), wrq:req_body(RD)} of 56 | {'POST', Body} when Body /= undefined -> 57 | verify_body(Body, State); 58 | _ -> 59 | {false, usage(), State} 60 | end, 61 | {not Verified, 62 | if Verified -> RD; 63 | true -> 64 | wrq:set_resp_header( 65 | "Content-Type", "text/plain", 66 | wrq:set_resp_body(Message, RD)) 67 | end, 68 | NewState}. 69 | 70 | content_types_provided(RD, State) -> 71 | {[{"application/json", nop}], RD, State}. 72 | 73 | nop(RD, State) -> 74 | {usage(), RD, State}. 75 | 76 | process_post(RD, #state{inputs=Inputs, mrquery=Query, timeout=Timeout}=State) -> 77 | Me = self(), 78 | {ok, Client} = riak:local_client(), 79 | ResultTransformer = fun riak_kv_mapred_json:jsonify_not_found/1, 80 | case wrq:get_qs_value("chunked", RD) of 81 | "true" -> 82 | {ok, ReqId} = 83 | case is_binary(Inputs) orelse is_key_filter(Inputs) of 84 | true -> 85 | Client:mapred_bucket_stream(Inputs, Query, Me, ResultTransformer, Timeout); 86 | false -> 87 | if is_list(Inputs) -> 88 | {ok, {RId, FSM}} = Client:mapred_stream(Query, Me, ResultTransformer, Timeout), 89 | luke_flow:add_inputs(FSM, Inputs), 90 | luke_flow:finish_inputs(FSM), 91 | {ok, RId}; 92 | is_tuple(Inputs) -> 93 | {ok, {RId, FSM}} = Client:mapred_stream(Query, Me, ResultTransformer, Timeout), 94 | Client:mapred_dynamic_inputs_stream(FSM, Inputs, Timeout), 95 | luke_flow:finish_inputs(FSM), 96 | {ok, RId} 97 | end 98 | end, 99 | Boundary = riak_core_util:unique_id_62(), 100 | RD1 = wrq:set_resp_header("Content-Type", "multipart/mixed;boundary=" ++ Boundary, RD), 101 | State1 = State#state{boundary=Boundary}, 102 | {true, wrq:set_resp_body({stream, stream_mapred_results(RD1, ReqId, State1)}, RD1), State1}; 103 | Param when Param =:= "false"; 104 | Param =:= undefined -> 105 | Results = case is_binary(Inputs) orelse is_key_filter(Inputs) of 106 | true -> 107 | Client:mapred_bucket(Inputs, Query, ResultTransformer, Timeout); 108 | false -> 109 | if is_list(Inputs) -> 110 | Client:mapred(Inputs, Query, ResultTransformer, Timeout); 111 | is_tuple(Inputs) -> 112 | case Client:mapred_stream(Query,Me,ResultTransformer,Timeout) of 113 | {ok, {ReqId, FlowPid}} -> 114 | Client:mapred_dynamic_inputs_stream(FlowPid, Inputs, Timeout), 115 | luke_flow:finish_inputs(FlowPid), 116 | luke_flow:collect_output(ReqId, Timeout); 117 | Error -> 118 | Error 119 | end 120 | end 121 | end, 122 | RD1 = wrq:set_resp_header("Content-Type", "application/json", RD), 123 | case Results of 124 | "all nodes failed" -> 125 | {{halt, 500}, wrq:set_resp_body("All nodes failed", RD), State}; 126 | {error, _} -> 127 | {{halt, 500}, send_error(Results, RD1), State}; 128 | {ok, Result} -> 129 | {true, wrq:set_resp_body(mochijson2:encode(Result), RD1), State} 130 | end 131 | end. 132 | 133 | %% Internal functions 134 | send_error(Error, RD) -> 135 | wrq:set_resp_body(format_error(Error), RD). 136 | 137 | format_error({error, Message}=Error) when is_atom(Message); 138 | is_binary(Message) -> 139 | mochijson2:encode({struct, [Error]}); 140 | format_error({error, Error}) when is_list(Error) -> 141 | mochijson2:encode({struct, Error}); 142 | format_error(_Error) -> 143 | mochijson2:encode({struct, [{error, map_reduce_error}]}). 144 | 145 | stream_mapred_results(RD, ReqId, #state{timeout=Timeout}=State) -> 146 | FinalTimeout = erlang:trunc(Timeout * 1.02), 147 | receive 148 | {flow_results, ReqId, done} -> {iolist_to_binary(["\r\n--", State#state.boundary, "--\r\n"]), done}; 149 | {flow_results, ReqId, {error, Error}} -> 150 | {format_error(Error), done}; 151 | {flow_error, ReqId, Error} -> 152 | {format_error({error, Error}), done}; 153 | {flow_results, PhaseId, ReqId, Res} -> 154 | Data = mochijson2:encode({struct, [{phase, PhaseId}, {data, Res}]}), 155 | Body = ["\r\n--", State#state.boundary, "\r\n", 156 | "Content-Type: application/json\r\n\r\n", 157 | Data], 158 | {iolist_to_binary(Body), fun() -> stream_mapred_results(RD, ReqId, State) end} 159 | after FinalTimeout -> 160 | {format_error({error, timeout}), done} 161 | end. 162 | 163 | verify_body(Body, State) -> 164 | case riak_kv_mapred_json:parse_request(Body) of 165 | {ok, ParsedInputs, ParsedQuery, Timeout} -> 166 | {true, [], State#state{inputs=ParsedInputs, 167 | mrquery=ParsedQuery, 168 | timeout=Timeout}}; 169 | {error, {'query', Message}} -> 170 | {false, ["An error occurred parsing the \"query\" field.\n", 171 | Message], State}; 172 | {error, {inputs, Message}} -> 173 | {false, ["An error occurred parsing the \"inputs\" field.\n", 174 | Message], State}; 175 | {error, missing_field} -> 176 | {false, "The post body was missing the " 177 | "\"inputs\" or \"query\" field.\n", State}; 178 | {error, {invalid_json, Message}} -> 179 | {false, 180 | io_lib:format("The POST body was not valid JSON.~n" 181 | "The error from the parser was:~n~p~n", 182 | [Message]), 183 | State}; 184 | {error, not_json} -> 185 | {false, "The POST body was not a JSON object.\n", State} 186 | end. 187 | 188 | usage() -> 189 | "This resource accepts POSTs with bodies containing JSON of the form:\n" 190 | "{\n" 191 | " \"inputs\":[...list of inputs...],\n" 192 | " \"query\":[...list of map/reduce phases...]\n" 193 | "}\n". 194 | 195 | is_key_filter({Bucket, Filters}) when is_binary(Bucket), 196 | is_list(Filters) -> 197 | true; 198 | is_key_filter(_) -> 199 | false. 200 | -------------------------------------------------------------------------------- /src/riak_kv_bitcask_backend.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_kv_bitcask_backend: Bitcask Driver for Riak 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | -module(riak_kv_bitcask_backend). 24 | -behavior(riak_kv_backend). 25 | -author('Andy Gross '). 26 | -author('Dave Smith '). 27 | 28 | %% KV Backend API 29 | -export([start/2, 30 | stop/1, 31 | get/2, 32 | put/3, 33 | delete/2, 34 | list/1, 35 | list_bucket/2, 36 | fold/3, 37 | fold_keys/3, 38 | fold_bucket_keys/4, 39 | drop/1, 40 | is_empty/1, 41 | callback/3]). 42 | 43 | %% Helper API 44 | -export([key_counts/0]). 45 | 46 | -ifdef(TEST). 47 | -include_lib("eunit/include/eunit.hrl"). 48 | -endif. 49 | 50 | -include_lib("bitcask/include/bitcask.hrl"). 51 | 52 | -define(MERGE_CHECK_INTERVAL, timer:minutes(3)). 53 | 54 | start(Partition, Config) -> 55 | 56 | %% Get the data root directory 57 | DataDir = 58 | case proplists:get_value(data_root, Config) of 59 | undefined -> 60 | case application:get_env(bitcask, data_root) of 61 | {ok, Dir} -> 62 | Dir; 63 | _ -> 64 | riak:stop("bitcask data_root unset, failing") 65 | end; 66 | Value -> 67 | Value 68 | end, 69 | 70 | %% Setup actual bitcask dir for this partition 71 | BitcaskRoot = filename:join([DataDir, 72 | integer_to_list(Partition)]), 73 | case filelib:ensure_dir(BitcaskRoot) of 74 | ok -> 75 | ok; 76 | {error, Reason} -> 77 | error_logger:error_msg("Failed to create bitcask dir ~s: ~p\n", 78 | [BitcaskRoot, Reason]), 79 | riak:stop("riak_kv_bitcask_backend failed to start.") 80 | end, 81 | 82 | BitcaskOpts = [{read_write, true}|Config], 83 | case bitcask:open(BitcaskRoot, BitcaskOpts) of 84 | Ref when is_reference(Ref) -> 85 | schedule_merge(Ref), 86 | maybe_schedule_sync(Ref), 87 | {ok, {Ref, BitcaskRoot}}; 88 | {error, Reason2} -> 89 | {error, Reason2} 90 | end. 91 | 92 | 93 | stop({Ref, _}) -> 94 | bitcask:close(Ref). 95 | 96 | 97 | get({Ref, _}, BKey) -> 98 | Key = term_to_binary(BKey), 99 | case bitcask:get(Ref, Key) of 100 | {ok, Value} -> 101 | {ok, Value}; 102 | not_found -> 103 | {error, notfound}; 104 | {error, Reason} -> 105 | {error, Reason} 106 | end. 107 | 108 | put({Ref, _}, BKey, Val) -> 109 | Key = term_to_binary(BKey), 110 | ok = bitcask:put(Ref, Key, Val). 111 | 112 | delete({Ref, _}, BKey) -> 113 | ok = bitcask:delete(Ref, term_to_binary(BKey)). 114 | 115 | list({Ref, _}) -> 116 | case bitcask:list_keys(Ref) of 117 | KeyList when is_list(KeyList) -> 118 | [binary_to_term(K) || K <- KeyList]; 119 | Other -> 120 | Other 121 | end. 122 | 123 | list_bucket({Ref, _}, {filter, Bucket, Fun}) -> 124 | bitcask:fold_keys(Ref, 125 | fun(#bitcask_entry{key=BK},Acc) -> 126 | {B,K} = binary_to_term(BK), 127 | case (B =:= Bucket) andalso Fun(K) of 128 | true -> 129 | [K|Acc]; 130 | false -> 131 | Acc 132 | end 133 | end, []); 134 | list_bucket({Ref, _}, '_') -> 135 | bitcask:fold_keys(Ref, 136 | fun(#bitcask_entry{key=BK},Acc) -> 137 | {B,_K} = binary_to_term(BK), 138 | case lists:member(B,Acc) of 139 | true -> Acc; 140 | false -> [B|Acc] 141 | end 142 | end, []); 143 | list_bucket({Ref, _}, Bucket) -> 144 | bitcask:fold_keys(Ref, 145 | fun(#bitcask_entry{key=BK},Acc) -> 146 | {B,K} = binary_to_term(BK), 147 | case B of 148 | Bucket -> [K|Acc]; 149 | _ -> Acc 150 | end 151 | end, []). 152 | 153 | fold({Ref, _}, Fun0, Acc0) -> 154 | %% When folding across the bitcask, the bucket/key tuple must 155 | %% be decoded. The intermediate binary_to_term call handles this 156 | %% and yields the expected fun({B, K}, Value, Acc) 157 | bitcask:fold(Ref, 158 | fun(K, V, Acc) -> 159 | Fun0(binary_to_term(K), V, Acc) 160 | end, 161 | Acc0). 162 | 163 | fold_keys({Ref, _}, Fun, Acc) -> 164 | F = fun(#bitcask_entry{key=K}, Acc1) -> 165 | Fun(binary_to_term(K), Acc1) end, 166 | bitcask:fold_keys(Ref, F, Acc). 167 | 168 | fold_bucket_keys(ModState, _Bucket, Fun, Acc) -> 169 | fold_keys(ModState, fun(Key2, Acc2) -> Fun(Key2, dummy_val, Acc2) end, Acc). 170 | 171 | drop({Ref, BitcaskRoot}) -> 172 | %% todo: once bitcask has a more friendly drop function 173 | %% of its own, use that instead. 174 | bitcask:close(Ref), 175 | {ok, FNs} = file:list_dir(BitcaskRoot), 176 | [file:delete(filename:join(BitcaskRoot, FN)) || FN <- FNs], 177 | file:del_dir(BitcaskRoot), 178 | ok. 179 | 180 | is_empty({Ref, _}) -> 181 | %% Determining if a bitcask is empty requires us to find at least 182 | %% one value that is NOT a tombstone. Accomplish this by doing a fold_keys 183 | %% that forcibly bails on the very first key encountered. 184 | F = fun(_K, _Acc0) -> 185 | throw(found_one_value) 186 | end, 187 | (catch bitcask:fold_keys(Ref, F, undefined)) /= found_one_value. 188 | 189 | callback({Ref, _}, Ref, {sync, SyncInterval}) when is_reference(Ref) -> 190 | bitcask:sync(Ref), 191 | schedule_sync(Ref, SyncInterval); 192 | callback({Ref, BitcaskRoot}, Ref, merge_check) when is_reference(Ref) -> 193 | case bitcask:needs_merge(Ref) of 194 | {true, Files} -> 195 | bitcask_merge_worker:merge(BitcaskRoot, [], Files); 196 | false -> 197 | ok 198 | end, 199 | schedule_merge(Ref); 200 | %% Ignore callbacks for other backends so multi backend works 201 | callback(_State, _Ref, _Msg) -> 202 | ok. 203 | 204 | key_counts() -> 205 | case application:get_env(bitcask, data_root) of 206 | {ok, RootDir} -> 207 | [begin 208 | {Keys, _} = status(filename:join(RootDir, Dir)), 209 | {Dir, Keys} 210 | end || Dir <- element(2, file:list_dir(RootDir))]; 211 | undefined -> 212 | {error, data_root_not_set} 213 | end. 214 | 215 | %% =================================================================== 216 | %% Internal functions 217 | %% =================================================================== 218 | 219 | %% @private 220 | %% Invoke bitcask:status/1 for a given directory 221 | status(Dir) -> 222 | Ref = bitcask:open(Dir), 223 | try bitcask:status(Ref) 224 | after bitcask:close(Ref) 225 | end. 226 | 227 | %% @private 228 | %% Schedule sync (if necessary) 229 | maybe_schedule_sync(Ref) when is_reference(Ref) -> 230 | case application:get_env(bitcask, sync_strategy) of 231 | {ok, {seconds, Seconds}} -> 232 | SyncIntervalMs = timer:seconds(Seconds), 233 | schedule_sync(Ref, SyncIntervalMs); 234 | %% erlang:send_after(SyncIntervalMs, self(), 235 | %% {?MODULE, {sync, SyncIntervalMs}}); 236 | {ok, none} -> 237 | ok; 238 | BadStrategy -> 239 | error_logger:info_msg("Ignoring invalid bitcask sync strategy: ~p\n", 240 | [BadStrategy]), 241 | ok 242 | end. 243 | 244 | schedule_sync(Ref, SyncIntervalMs) when is_reference(Ref) -> 245 | riak_kv_backend:callback_after(SyncIntervalMs, Ref, {sync, SyncIntervalMs}). 246 | 247 | schedule_merge(Ref) when is_reference(Ref) -> 248 | riak_kv_backend:callback_after(?MERGE_CHECK_INTERVAL, Ref, merge_check). 249 | 250 | %% =================================================================== 251 | %% EUnit tests 252 | %% =================================================================== 253 | -ifdef(TEST). 254 | 255 | simple_test() -> 256 | ?assertCmd("rm -rf test/bitcask-backend"), 257 | application:set_env(bitcask, data_root, "test/bitcask-backend"), 258 | riak_kv_backend:standard_test(?MODULE, []). 259 | 260 | custom_config_test() -> 261 | ?assertCmd("rm -rf test/bitcask-backend"), 262 | application:set_env(bitcask, data_root, ""), 263 | riak_kv_backend:standard_test(?MODULE, [{data_root, "test/bitcask-backend"}]). 264 | 265 | -endif. 266 | -------------------------------------------------------------------------------- /src/riak_kv_js_manager.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_js_manager: dispatch work to JavaScript VMs 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | %% @doc dispatch work to JavaScript VMs 24 | -module(riak_kv_js_manager). 25 | -author('Kevin Smith '). 26 | -author('John Muellerleile '). 27 | 28 | -behaviour(gen_server). 29 | 30 | %% API 31 | -export([start_link/2, 32 | add_vm/1, 33 | reload/1, 34 | mark_idle/1, 35 | reserve_vm/1, 36 | reserve_batch_vm/2, 37 | dispatch/3, 38 | blocking_dispatch/3, 39 | pool_size/1]). 40 | 41 | %% gen_server callbacks 42 | -export([init/1, 43 | handle_call/3, 44 | handle_cast/2, 45 | handle_info/2, 46 | terminate/2, 47 | code_change/3]). 48 | 49 | -include("riak_kv_js_pools.hrl"). 50 | 51 | -record('DOWN', {ref, type, pid, info}). 52 | -record(vm_state, {pid, needs_reload=false}). 53 | -record(state, {name, master, idle, reserve}). 54 | 55 | start_link(Name, ChildCount) -> 56 | gen_server:start_link({local, Name}, ?MODULE, [Name, ChildCount], []). 57 | 58 | %% @spec reload([string()|atom()]) -> ok 59 | %% @doc Reload the Javascript VMs in the named pools. If no pool 60 | %% names are given, all pools in the JSPOOL_LIST (defined in 61 | %% riak_kv_js_pools.hrl) are reloaded. Pool names may be given 62 | %% as either list-strings (as they will be when this function is 63 | %% invoked via 'riak-admin js_reload') or as atoms (as they are 64 | %% defined in riak_kv_js_pools.hrl). 65 | reload([]) -> 66 | %% no names == reload all vms 67 | reload(?JSPOOL_LIST); 68 | reload(Names) -> 69 | reload_internal(Names). 70 | 71 | %% @spec reload_internal([string()|atom()]) -> ok 72 | %% @doc Recursive implementation of reload/1. 73 | reload_internal([Name|Rest]) when is_atom(Name) -> 74 | gen_server:call(Name, reload_vms, infinity), 75 | reload_internal(Rest); 76 | reload_internal([Name|Rest]) when is_list(Name) -> 77 | %% convert riak-admin string argument to atom gen_server name 78 | reload_internal([list_to_existing_atom(Name)|Rest]); 79 | reload_internal([]) -> 80 | ok. 81 | 82 | add_vm(Name) -> 83 | gen_server:cast(Name, {add_vm, self()}). 84 | 85 | mark_idle(Name) -> 86 | gen_server:call(Name, {mark_idle, self()}, infinity). 87 | 88 | dispatch(Name, JSCall, Tries) -> 89 | dispatch(Name, JSCall, Tries, Tries). 90 | 91 | blocking_dispatch(Name, JSCall, Tries) -> 92 | blocking_dispatch(Name, JSCall, Tries, Tries). 93 | 94 | reserve_vm(Name) -> 95 | gen_server:call(Name, reserve_vm, infinity). 96 | 97 | reserve_batch_vm(Name, Tries) -> 98 | reserve_batch_vm(Name, Tries, Tries). 99 | 100 | pool_size(Name) -> 101 | gen_server:call(Name, pool_size, infinity). 102 | 103 | init([Name, ChildCount]) -> 104 | Master = ets:new(Name, [private, {keypos, 2}]), 105 | Idle = ets:new(Name, [private]), 106 | start_vms(Name, ChildCount), 107 | {ok, #state{name=Name, master=Master, idle=Idle}}. 108 | 109 | handle_call({mark_idle, VM}, _From, #state{master=Master, 110 | idle=Idle}=State) -> 111 | case needs_reload(Master, VM) of 112 | true -> 113 | riak_kv_js_vm:reload(VM), 114 | clear_reload(Master, VM); 115 | false -> 116 | ok 117 | end, 118 | ets:insert(Idle, {VM}), 119 | {reply, ok, State}; 120 | 121 | handle_call(reload_vms, _From, #state{master=Master, idle=Idle}=State) -> 122 | reload_idle_vms(Idle), 123 | mark_pending_reloads(Master, Idle), 124 | if State#state.name == ?JSPOOL_MAP -> 125 | riak_kv_mapred_cache:clear(); 126 | true -> 127 | ok 128 | end, 129 | {reply, ok, State}; 130 | 131 | handle_call(reserve_batch_vm, _From, State) -> 132 | {Reply, State1} = case handle_call(reserve_vm, _From, State) of 133 | {reply, {ok, VM}, NewState} -> 134 | riak_kv_js_vm:start_batch(VM), 135 | {{ok, VM}, NewState}; 136 | {reply, Error, NewState} -> 137 | {Error, NewState} 138 | end, 139 | {reply, Reply, State1}; 140 | 141 | handle_call(reserve_vm, _From, #state{idle=Idle}=State) -> 142 | Reply = case ets:first(Idle) of 143 | '$end_of_table' -> 144 | {error, no_vms}; 145 | VM -> 146 | ets:delete(Idle, VM), 147 | {ok, VM} 148 | end, 149 | {reply, Reply, State}; 150 | 151 | handle_call(pool_size, _From, #state{idle=Idle}=State) -> 152 | {reply, ets:info(Idle, size), State}; 153 | 154 | handle_call(_Request, _From, State) -> 155 | {reply, ignore, State}. 156 | 157 | handle_cast({add_vm, VMPid}, #state{master=Master, idle=Idle}=State) -> 158 | erlang:monitor(process, VMPid), 159 | VMState = #vm_state{pid=VMPid}, 160 | ets:insert(Master, VMState), 161 | ets:insert(Idle, {VMPid}), 162 | {noreply, State}; 163 | 164 | handle_cast(_Msg, State) -> 165 | {noreply, State}. 166 | 167 | handle_info(#'DOWN'{pid=Pid}, #state{master=Master, idle=Idle, name=Pool}=State) -> 168 | ets:delete(Master, Pid), 169 | ets:delete(Idle, Pid), 170 | riak_kv_js_sup:start_js(self(), Pool), 171 | {noreply, State}; 172 | 173 | handle_info(_Info, State) -> 174 | {noreply, State}. 175 | 176 | terminate(_Reason, _State) -> 177 | ok. 178 | 179 | code_change(_OldVsn, State, _Extra) -> 180 | {ok, State}. 181 | 182 | %% Internal functions 183 | needs_reload(Master, VMPid) -> 184 | [VMState] = ets:lookup(Master, VMPid), 185 | VMState#vm_state.needs_reload. 186 | 187 | clear_reload(Master, VMPid) -> 188 | [VMState] = ets:lookup(Master, VMPid), 189 | VMState1 = VMState#vm_state{needs_reload=false}, 190 | ets:insert(Master, VMState1). 191 | 192 | is_vm_idle(Idle, VMPid) -> 193 | case ets:lookup(Idle, {VMPid}) of 194 | [] -> 195 | false; 196 | _ -> 197 | true 198 | end. 199 | 200 | start_vms(_Pool, 0) -> 201 | ok; 202 | start_vms(Pool, Count) -> 203 | riak_kv_js_sup:start_js(self(), Pool), 204 | start_vms(Pool, Count - 1). 205 | 206 | reload_idle_vms(Tid) -> 207 | reload_idle_vms(ets:first(Tid), Tid). 208 | 209 | reload_idle_vms('$end_of_table', _Tid) -> 210 | ok; 211 | reload_idle_vms(Current, Tid) -> 212 | riak_kv_js_vm:reload(Current), 213 | reload_idle_vms(ets:next(Tid, Current), Tid). 214 | 215 | mark_pending_reloads(Master, Idle) -> 216 | mark_pending_reloads(ets:first(Master), Master, Idle). 217 | 218 | mark_pending_reloads('$end_of_table', _Master, _Idle) -> 219 | ok; 220 | mark_pending_reloads(VMPid, Master, Idle) -> 221 | case is_vm_idle(Idle, VMPid) of 222 | true -> 223 | ok; 224 | false -> 225 | [VMState] = ets:lookup(Master, VMPid), 226 | VMState1 = VMState#vm_state{needs_reload=true}, 227 | ets:insert(Master, VMState1) 228 | end, 229 | mark_pending_reloads(ets:next(Master, VMPid), Master, Idle). 230 | 231 | dispatch(_Name, _JSCall, _MaxCount, 0) -> 232 | error_logger:info_msg("JS call failed: All VMs are busy.~n"), 233 | {error, no_vms}; 234 | dispatch(Name, JSCall, MaxCount, Count) -> 235 | case reserve_vm(Name) of 236 | {ok, VM} -> 237 | JobId = {VM, make_ref()}, 238 | riak_kv_js_vm:dispatch(VM, self(), JobId, JSCall), 239 | {ok, JobId}; 240 | {error, no_vms} -> 241 | back_off(MaxCount, Count), 242 | dispatch(Name, JSCall, MaxCount, Count - 1) 243 | end. 244 | 245 | blocking_dispatch(_Name, _JSCall, _MaxCount, 0) -> 246 | error_logger:info_msg("JS call failed: All VMs are busy.~n"), 247 | {error, no_vms}; 248 | blocking_dispatch(Name, JSCall, MaxCount, Count) -> 249 | case reserve_vm(Name) of 250 | {ok, VM} -> 251 | JobId = {VM, make_ref()}, 252 | riak_kv_js_vm:blocking_dispatch(VM, JobId, JSCall); 253 | {error, no_vms} -> 254 | back_off(MaxCount, Count), 255 | blocking_dispatch(Name, JSCall, MaxCount, Count - 1) 256 | end. 257 | 258 | reserve_batch_vm(_Name, _MaxCount, 0) -> 259 | {error, no_vms}; 260 | reserve_batch_vm(Name, MaxCount, Count) -> 261 | case gen_server:call(Name, reserve_batch_vm) of 262 | {error, no_vms} -> 263 | back_off(MaxCount, Count), 264 | reserve_batch_vm(Name, MaxCount, Count - 1); 265 | {ok, VM} -> 266 | {ok, VM} 267 | end. 268 | 269 | back_off(MaxCount, Count) -> 270 | ScalingFactor = (1 + (MaxCount - Count)) * 271 | (0.1 + random:uniform(100) * 0.001), 272 | timer:sleep(erlang:round(500 * ScalingFactor)). 273 | -------------------------------------------------------------------------------- /src/riak_kv_map_master.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_map_master: spins up batched map tasks on behalf of map phases 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | 23 | -module(riak_kv_map_master). 24 | -include_lib("riak_kv_js_pools.hrl"). 25 | 26 | -behaviour(gen_server2). 27 | 28 | %% API 29 | -export([start_link/0, 30 | queue_depth/0, 31 | new_mapper/4]). 32 | 33 | %% gen_server callbacks 34 | -export([init/1, handle_call/3, handle_cast/2, handle_info/2, 35 | terminate/2, code_change/3]). 36 | 37 | -define(SERVER, ?MODULE). 38 | 39 | -record(mapper, {vnode, 40 | qterm, 41 | inputs, 42 | phase}). 43 | 44 | -record(state, {datadir, 45 | store, 46 | highest, 47 | next}). 48 | 49 | new_mapper({_, Node}=VNode, QTerm, MapInputs, PhasePid) -> 50 | gen_server2:pcall({?SERVER, Node}, 5, {new_mapper, VNode, 51 | QTerm, MapInputs, PhasePid}, infinity). 52 | 53 | queue_depth() -> 54 | Nodes = [node()|nodes()], 55 | [{Node, gen_server2:pcall({?SERVER, Node}, 0, queue_depth, 56 | infinity)} || Node <- Nodes]. 57 | 58 | 59 | start_link() -> 60 | gen_server2:start_link({local, ?SERVER}, ?MODULE, [], []). 61 | 62 | init([]) -> 63 | process_flag(trap_exit, true), 64 | DataDir = init_data_dir(), 65 | Store = bitcask:open(DataDir, [read_write]), 66 | {ok, NextCounter} = file:open(filename:join(DataDir, "_next_"), [read, write, raw, binary]), 67 | {ok, HighestCounter} = file:open(filename:join(DataDir, "_highest_"), [read, write, raw, binary]), 68 | State = #state{datadir=DataDir, store=Store, highest=HighestCounter, 69 | next=NextCounter}, 70 | reset_counters(State), 71 | timer:send_interval(60000, merge_storage), 72 | {ok, State}. 73 | 74 | handle_call({new_mapper, VNode, {erlang, _}=QTerm, MapInputs, PhasePid}, _From, State) -> 75 | Id = make_id(), 76 | case riak_kv_mapper_sup:new_mapper(VNode, Id, QTerm, MapInputs, PhasePid) of 77 | {ok, _Pid} -> 78 | {reply, {ok, Id}, State}; 79 | {error, Reason} -> 80 | {reply, {error, Reason}, State} 81 | end; 82 | 83 | handle_call({new_mapper, VNode, {javascript, _}=QTerm, MapInputs, PhasePid}, _From, State) -> 84 | case riak_kv_js_manager:pool_size(?JSPOOL_MAP) > 0 of 85 | true -> 86 | Id = make_id(), 87 | case riak_kv_mapper_sup:new_mapper(VNode, Id, QTerm, MapInputs, PhasePid) of 88 | {ok, Pid} -> 89 | erlang:monitor(process, Pid), 90 | {reply, {ok, Id}, State}; 91 | {error, Reason} -> 92 | {reply, {error, Reason}, State} 93 | end; 94 | false -> 95 | Id = defer_mapper(VNode, QTerm, MapInputs, PhasePid, State), 96 | {reply, {ok, {Id, node()}}, State} 97 | end; 98 | 99 | handle_call(queue_depth, _From, #state{highest=Highest, next=Next}=State) -> 100 | H = read_counter(Highest), 101 | N = read_counter(Next), 102 | Reply = H - N, 103 | {reply, Reply, State}; 104 | 105 | handle_call(_Request, _From, State) -> 106 | {reply, ignore, State}. 107 | 108 | handle_cast(_Msg, State) -> 109 | {noreply, State}. 110 | 111 | %% Dequeue a deferred mapper when a mapper exits 112 | handle_info({'DOWN', _A, _B, _Mapper, _C}, State) -> 113 | dequeue_mapper(State), 114 | {noreply, State}; 115 | 116 | handle_info(merge_storage, #state{store=Store, datadir=DataDir}=State) -> 117 | case bitcask:needs_merge(Store) of 118 | {true, Files} -> 119 | bitcask_merge_worker:merge(DataDir, [], Files); 120 | false -> 121 | ok 122 | end, 123 | {noreply, State}; 124 | 125 | handle_info(_Info, State) -> 126 | {noreply, State}. 127 | 128 | terminate(_Reason, #state{store=Store, highest=Highest, next=Next}) -> 129 | file:close(Highest), 130 | file:close(Next), 131 | bitcask:close(Store). 132 | 133 | code_change(_OldVsn, State, _Extra) -> 134 | {ok, State}. 135 | 136 | %% Internal functions 137 | make_id() -> 138 | {_, _, T3} = erlang:now(), 139 | {T3, node()}. 140 | 141 | dequeue_mapper(State) -> 142 | case are_mappers_waiting(State) of 143 | false -> 144 | ok; 145 | true -> 146 | Id = read(State#state.next), 147 | Mapper = read_entry(Id, State#state.store), 148 | case is_mapper_runnable(Mapper) of 149 | false -> 150 | incr(State#state.next), 151 | delete_entry(Id, State#state.store), 152 | dequeue_mapper(State); 153 | true -> 154 | #mapper{vnode=VNode, qterm=QTerm, 155 | inputs=MapInputs, phase=Phase} = Mapper, 156 | case riak_kv_js_manager:pool_size(?JSPOOL_MAP) > 0 of 157 | true -> 158 | {ok, Pid} = riak_kv_mapper_sup:new_mapper(VNode, {Id, node()}, QTerm, 159 | MapInputs, Phase), 160 | erlang:monitor(process, Pid), 161 | incr(State#state.next), 162 | delete_entry(Id, State#state.store), 163 | dequeue_mapper(State); 164 | false -> 165 | ok 166 | end 167 | end 168 | end. 169 | 170 | defer_mapper(VNode, QTerm, MapInputs, PhasePid, State) -> 171 | Mapper = #mapper{vnode=VNode, qterm=QTerm, inputs=MapInputs, phase=PhasePid}, 172 | Id = read_incr(State#state.highest), 173 | write_entry(Id, Mapper, State#state.store). 174 | 175 | reset_counters(State) -> 176 | case are_mappers_waiting(State) of 177 | false -> 178 | file:pwrite(State#state.highest, 0, <<0:64>>), 179 | file:sync(State#state.highest), 180 | file:pwrite(State#state.next, 0, <<0:64>>), 181 | file:sync(State#state.next); 182 | true -> 183 | dequeue_mapper(State) 184 | end. 185 | 186 | read(CounterFile) -> 187 | Counter = read_counter(CounterFile), 188 | list_to_binary(integer_to_list(Counter)). 189 | 190 | incr(CounterFile) -> 191 | Counter = read_counter(CounterFile), 192 | NewCounter = Counter + 1, 193 | ok = file:pwrite(CounterFile, 0, <>), 194 | file:sync(CounterFile). 195 | 196 | read_incr(CounterFile) -> 197 | Counter = read_counter(CounterFile), 198 | NewCounter = Counter + 1, 199 | ok = file:pwrite(CounterFile, 0, <>), 200 | file:sync(CounterFile), 201 | list_to_binary(integer_to_list(Counter)). 202 | 203 | read_counter(Counter) -> 204 | case file:pread(Counter, 0, 8) of 205 | eof -> 206 | 0; 207 | {ok, Data} -> 208 | <> = Data, 209 | V; 210 | Error -> 211 | throw(Error) 212 | end. 213 | 214 | are_mappers_waiting(State) -> 215 | Highest = read_counter(State#state.highest), 216 | Next = read_counter(State#state.next), 217 | Next < Highest. 218 | 219 | is_mapper_runnable({error,_}) -> false; 220 | is_mapper_runnable(not_found) -> false; 221 | is_mapper_runnable(#mapper{phase=Phase}) -> 222 | Node = node(Phase), 223 | ClusterNodes = riak_core_node_watcher:nodes(riak_kv), 224 | lists:member(Node, ClusterNodes) andalso rpc:call(Node, erlang, is_process_alive, 225 | [Phase]). 226 | 227 | write_entry(Id, Mapper, Store) -> 228 | ok = bitcask:put(Store, Id, term_to_binary(Mapper, [compressed])), 229 | Id. 230 | 231 | read_entry(Id, Store) -> 232 | case bitcask:get(Store, Id) of 233 | {ok, D} -> binary_to_term(D); 234 | Err -> Err 235 | end. 236 | 237 | delete_entry(Id, Store) -> 238 | bitcask:delete(Store, Id). 239 | 240 | ensure_dir(Dir) -> 241 | filelib:ensure_dir(filename:join(Dir, ".empty")). 242 | 243 | init_data_dir() -> 244 | %% There are some upgrade situations where the mapred_queue_dir, is not 245 | %% specified and as such we'll wind up using the default data/mr_queue, 246 | %% relative to current working dir. This causes problems in a packaged env 247 | %% (such as rpm/deb) where the current working dir is NOT writable. To 248 | %% accomodate these situations, we fallback to creating the mr_queue in 249 | %% /tmp. 250 | {ok, Cwd} = file:get_cwd(), 251 | DataDir0 = app_helper:get_env(riak_kv, mapred_queue_dir, 252 | filename:join(Cwd, "data/mr_queue")), 253 | case ensure_dir(DataDir0) of 254 | ok -> 255 | DataDir0; 256 | {error, Reason} -> 257 | error_logger:warning_msg("Failed to create ~p for mapred_queue_dir (~p); " 258 | "defaulting to /tmp/mr_queue\n", 259 | [DataDir0, Reason]), 260 | ok = ensure_dir("/tmp/mr_queue"), 261 | "/tmp/mr_queue" 262 | end. 263 | 264 | -------------------------------------------------------------------------------- /src/riak_kv_get_core.erl: -------------------------------------------------------------------------------- 1 | %% ------------------------------------------------------------------- 2 | %% 3 | %% riak_kv_get_core: Riak get logic 4 | %% 5 | %% Copyright (c) 2007-2010 Basho Technologies, Inc. All Rights Reserved. 6 | %% 7 | %% This file is provided to you under the Apache License, 8 | %% Version 2.0 (the "License"); you may not use this file 9 | %% except in compliance with the License. You may obtain 10 | %% a copy of the License at 11 | %% 12 | %% http://www.apache.org/licenses/LICENSE-2.0 13 | %% 14 | %% Unless required by applicable law or agreed to in writing, 15 | %% software distributed under the License is distributed on an 16 | %% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | %% KIND, either express or implied. See the License for the 18 | %% specific language governing permissions and limitations 19 | %% under the License. 20 | %% 21 | %% ------------------------------------------------------------------- 22 | -module(riak_kv_get_core). 23 | -export([init/6, add_result/3, enough/1, response/1, 24 | has_all_results/1, final_action/1, info/1]). 25 | -export_type([getcore/0, result/0, reply/0, final_action/0]). 26 | 27 | -type result() :: {ok, riak_object:riak_object()} | 28 | {error, notfound} | % for dialyzer 29 | {error, any()}. 30 | -type reply() :: {ok, riak_object:riak_object()} | 31 | {error, notfound} | 32 | {error, any()}. 33 | -type final_action() :: nop | 34 | {read_repair, [non_neg_integer()], riak_object:riak_object()} | 35 | delete. 36 | -type idxresult() :: {non_neg_integer(), result()}. 37 | 38 | -record(getcore, {n :: pos_integer(), 39 | r :: pos_integer(), 40 | fail_threshold :: pos_integer(), 41 | notfound_ok :: boolean(), 42 | allow_mult :: boolean(), 43 | deletedvclock :: boolean(), 44 | results = [] :: [idxresult()], 45 | merged :: {notfound | tombstone | ok, 46 | riak_object:riak_object() | undefined}, 47 | num_ok = 0 :: non_neg_integer(), 48 | num_notfound = 0 :: non_neg_integer(), 49 | num_fail = 0 :: non_neg_integer()}). 50 | -opaque getcore() :: #getcore{}. 51 | 52 | %% ==================================================================== 53 | %% Public API 54 | %% ==================================================================== 55 | 56 | %% Initialize a get and return an opaque get core context 57 | -spec init(pos_integer(), pos_integer(), pos_integer(), boolean(), boolean(), 58 | boolean()) -> getcore(). 59 | init(N, R, FailThreshold, NotFoundOk, AllowMult, DeletedVClock) -> 60 | #getcore{n = N, 61 | r = R, 62 | fail_threshold = FailThreshold, 63 | notfound_ok = NotFoundOk, 64 | allow_mult = AllowMult, 65 | deletedvclock = DeletedVClock}. 66 | 67 | %% Add a result for a vnode index 68 | -spec add_result(non_neg_integer(), result(), getcore()) -> getcore(). 69 | add_result(Idx, Result, GetCore = #getcore{results = Results}) -> 70 | UpdResults = [{Idx, Result} | Results], 71 | case Result of 72 | {ok, _RObj} -> 73 | GetCore#getcore{results = UpdResults, merged = undefined, 74 | num_ok = GetCore#getcore.num_ok + 1}; 75 | {error, notfound} -> 76 | case GetCore#getcore.notfound_ok of 77 | true -> 78 | GetCore#getcore{results = UpdResults, merged = undefined, 79 | num_ok = GetCore#getcore.num_ok + 1}; 80 | _ -> 81 | GetCore#getcore{results = UpdResults, merged = undefined, 82 | num_notfound = GetCore#getcore.num_notfound + 1} 83 | end; 84 | {error, _Reason} -> 85 | GetCore#getcore{results = UpdResults, merged = undefined, 86 | num_fail = GetCore#getcore.num_fail + 1} 87 | end. 88 | 89 | %% Check if enough results have been added to respond 90 | -spec enough(getcore()) -> boolean(). 91 | enough(#getcore{r = R, num_ok = NumOk, 92 | num_notfound = NumNotFound, 93 | num_fail = NumFail, 94 | fail_threshold = FailThreshold}) -> 95 | if 96 | NumOk >= R -> 97 | true; 98 | NumNotFound + NumFail >= FailThreshold -> 99 | true; 100 | true -> 101 | false 102 | end. 103 | 104 | %% Get success/fail response once enough results received 105 | -spec response(getcore()) -> {reply(), getcore()}. 106 | response(GetCore = #getcore{r = R, num_ok = NumOk, num_notfound = NumNotFound, 107 | results = Results, allow_mult = AllowMult, 108 | deletedvclock = DeletedVClock}) -> 109 | {ObjState, MObj} = Merged = merge(Results, AllowMult), 110 | Reply = case NumOk >= R of 111 | true -> 112 | case ObjState of 113 | ok -> 114 | Merged; % {ok, MObj} 115 | tombstone when DeletedVClock -> 116 | {error, {deleted, riak_object:vclock(MObj)}}; 117 | _ -> % tombstone or notfound 118 | {error, notfound} 119 | end; 120 | false -> 121 | DelObjs = length([xx || {_Idx, {ok, RObj}} <- Results, 122 | riak_kv_util:is_x_deleted(RObj)]), 123 | Fails = [F || F = {_Idx, {error, Reason}} <- Results, 124 | Reason /= notfound], 125 | fail_reply(R, NumOk, NumOk - DelObjs, 126 | NumNotFound + DelObjs, Fails) 127 | end, 128 | {Reply, GetCore#getcore{merged = Merged}}. 129 | 130 | %% Check if all expected results have been added 131 | -spec has_all_results(getcore()) -> boolean(). 132 | has_all_results(#getcore{n = N, num_ok = NOk, 133 | num_fail = NFail, num_notfound = NNF}) -> 134 | NOk + NFail + NNF >= N. 135 | 136 | %% Decide on any post-response actions 137 | %% nop - do nothing 138 | %% {readrepair, Indices, MObj} - send read repairs iff any vnode has ancestor data 139 | %% (including tombstones) 140 | %% delete - issue deletes if all vnodes returned tombstones. This needs to be 141 | %% supplemented with a check that the vnodes were all primaries. 142 | %% 143 | -spec final_action(getcore()) -> {final_action(), getcore()}. 144 | final_action(GetCore = #getcore{n = N, merged = Merged0, results = Results, 145 | allow_mult = AllowMult}) -> 146 | Merged = case Merged0 of 147 | undefined -> 148 | merge(Results, AllowMult); 149 | _ -> 150 | Merged0 151 | end, 152 | {ObjState, MObj} = Merged, 153 | ReadRepairs = case ObjState of 154 | notfound -> 155 | []; 156 | _ -> % ok or tombstone 157 | [Idx || {Idx, {ok, RObj}} <- Results, 158 | strict_descendant(MObj, RObj)] ++ 159 | [Idx || {Idx, {error, notfound}} <- Results] 160 | end, 161 | Action = case ReadRepairs of 162 | [] when ObjState == tombstone -> 163 | %% Allow delete if merge object is deleted, 164 | %% there are no read repairs pending and 165 | %% a value was received from all vnodes 166 | case riak_kv_util:is_x_deleted(MObj) andalso 167 | length([xx || {_Idx, {ok, _RObj}} <- Results]) == N of 168 | true -> 169 | delete; 170 | _ -> 171 | nop 172 | end; 173 | [] -> 174 | nop; 175 | _ -> 176 | {read_repair, ReadRepairs, MObj} 177 | end, 178 | {Action, GetCore#getcore{merged = Merged}}. 179 | 180 | %% Return request info 181 | -spec info(undefined | getcore()) -> [{vnode_oks, non_neg_integer()} | 182 | {vnode_errors, [any()]}]. 183 | 184 | info(undefined) -> 185 | []; % make uninitialized case easier 186 | info(#getcore{num_ok = NumOks, num_fail = NumFail, results = Results}) -> 187 | Oks = [{vnode_oks, NumOks}], 188 | case NumFail of 189 | 0 -> 190 | Oks; 191 | _ -> 192 | Errors = [Reason || {_Idx, {error, Reason}} <- Results, 193 | Reason /= undefined], 194 | [{vnode_errors, Errors} | Oks] 195 | end. 196 | 197 | %% ==================================================================== 198 | %% Internal functions 199 | %% ==================================================================== 200 | 201 | strict_descendant(O1, O2) -> 202 | vclock:descends(riak_object:vclock(O1),riak_object:vclock(O2)) andalso 203 | not vclock:descends(riak_object:vclock(O2),riak_object:vclock(O1)). 204 | 205 | merge(Replies, AllowMult) -> 206 | RObjs = [RObj || {_I, {ok, RObj}} <- Replies], 207 | case RObjs of 208 | [] -> 209 | {notfound, undefined}; 210 | _ -> 211 | Merged = riak_object:reconcile(RObjs, AllowMult), % include tombstones 212 | case riak_kv_util:is_x_deleted(Merged) of 213 | true -> 214 | {tombstone, Merged}; 215 | _ -> 216 | {ok, Merged} 217 | end 218 | end. 219 | 220 | fail_reply(_R, _NumR, 0, NumNotFound, []) when NumNotFound > 0 -> 221 | {error, notfound}; 222 | fail_reply(R, NumR, _NumNotDeleted, _NumNotFound, _Fails) -> 223 | {error, {r_val_unsatisfied, R, NumR}}. 224 | 225 | 226 | --------------------------------------------------------------------------------