2013-05-10 20:54:50 +00:00
|
|
|
%% -------------------------------------------------------------------
|
|
|
|
%%
|
|
|
|
%% Copyright (c) 2013 Basho Technologies, Inc.
|
|
|
|
%%
|
|
|
|
%% This file is provided to you under the Apache License,
|
|
|
|
%% Version 2.0 (the "License"); you may not use this file
|
|
|
|
%% except in compliance with the License. You may obtain
|
|
|
|
%% a copy of the License at
|
|
|
|
%%
|
|
|
|
%% http://www.apache.org/licenses/LICENSE-2.0
|
|
|
|
%%
|
|
|
|
%% Unless required by applicable law or agreed to in writing,
|
|
|
|
%% software distributed under the License is distributed on an
|
|
|
|
%% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
|
|
|
%% KIND, either express or implied. See the License for the
|
|
|
|
%% specific language governing permissions and limitations
|
|
|
|
%% under the License.
|
|
|
|
%%
|
|
|
|
%% -------------------------------------------------------------------
|
|
|
|
-module(overload).
|
|
|
|
-compile(export_all).
|
|
|
|
-include_lib("eunit/include/eunit.hrl").
|
|
|
|
|
2014-07-01 16:46:42 +00:00
|
|
|
-cover_modules([riak_kv_vnode,
|
|
|
|
riak_kv_ensemble_backend,
|
|
|
|
riak_core_vnode_proxy]).
|
|
|
|
|
2014-01-24 20:13:24 +00:00
|
|
|
-define(NUM_REQUESTS, 200).
|
|
|
|
-define(THRESHOLD, 100).
|
2014-07-24 18:47:46 +00:00
|
|
|
-define(LIST_KEYS_RETRIES, 1000).
|
|
|
|
-define(GET_RETRIES, 1000).
|
2013-05-10 20:54:50 +00:00
|
|
|
-define(BUCKET, <<"test">>).
|
|
|
|
-define(KEY, <<"hotkey">>).
|
|
|
|
|
|
|
|
confirm() ->
|
2014-07-01 16:46:42 +00:00
|
|
|
Nodes = setup(),
|
|
|
|
|
|
|
|
NormalType = <<"normal_type">>,
|
|
|
|
ConsistentType = <<"consistent_type">>,
|
2015-04-01 23:00:53 +00:00
|
|
|
WriteOnceType = <<"write_once_type">>,
|
2015-03-27 20:00:34 +00:00
|
|
|
|
2014-07-01 16:46:42 +00:00
|
|
|
ok = create_bucket_type(Nodes, NormalType, [{n_val, 3}]),
|
|
|
|
ok = create_bucket_type(Nodes, ConsistentType, [{consistent, true}, {n_val, 5}]),
|
2015-04-01 23:00:53 +00:00
|
|
|
ok = create_bucket_type(Nodes, WriteOnceType, [{write_once, true}, {n_val, 1}]),
|
2014-07-01 16:46:42 +00:00
|
|
|
rt:wait_until(ring_manager_check_fun(hd(Nodes))),
|
|
|
|
|
|
|
|
BKV1 = {{NormalType, ?BUCKET}, ?KEY, <<"test">>},
|
|
|
|
BKV2 = {{ConsistentType, ?BUCKET}, ?KEY, <<"test">>},
|
2015-04-01 23:00:53 +00:00
|
|
|
BKV3 = {{WriteOnceType, ?BUCKET}, ?KEY, <<"test">>},
|
2014-07-01 16:46:42 +00:00
|
|
|
|
|
|
|
Tests = [test_no_overload_protection,
|
|
|
|
test_vnode_protection,
|
|
|
|
test_fsm_protection,
|
|
|
|
test_cover_queries_overload],
|
|
|
|
|
2015-03-27 20:00:34 +00:00
|
|
|
[begin
|
|
|
|
lager:info("Starting Test ~p for ~p~n", [Test, BKV]),
|
|
|
|
ok = erlang:apply(?MODULE, Test, [Nodes, BKV, IsConsistent])
|
|
|
|
end || Test <- Tests,
|
|
|
|
{BKV, IsConsistent} <- [{BKV1, false},
|
|
|
|
{BKV2, true},
|
|
|
|
{BKV3, false}]],
|
2014-07-01 16:46:42 +00:00
|
|
|
pass.
|
|
|
|
|
|
|
|
|
|
|
|
setup() ->
|
2013-05-10 20:54:50 +00:00
|
|
|
Config = [{riak_core, [{ring_creation_size, 8},
|
2014-07-01 16:46:42 +00:00
|
|
|
{default_bucket_props, [{n_val, 5}]},
|
|
|
|
{vnode_management_timer, 1000},
|
2013-05-10 20:54:50 +00:00
|
|
|
{enable_health_checks, false},
|
2014-07-01 16:46:42 +00:00
|
|
|
{enable_consensus, true},
|
2013-05-10 20:54:50 +00:00
|
|
|
{vnode_overload_threshold, undefined}]},
|
|
|
|
{riak_kv, [{fsm_limit, undefined},
|
|
|
|
{storage_backend, riak_kv_memory_backend},
|
2014-07-01 16:46:42 +00:00
|
|
|
{anti_entropy_build_limit, {100, 1000}},
|
|
|
|
{anti_entropy_concurrency, 100},
|
|
|
|
{anti_entropy_tick, 100},
|
|
|
|
{anti_entropy, {on, []}},
|
2014-07-22 20:48:25 +00:00
|
|
|
{anti_entropy_timeout, 5000}]},
|
|
|
|
{riak_api, [{pb_backlog, 1024}]}],
|
2014-07-01 16:46:42 +00:00
|
|
|
ensemble_util:build_cluster(5, Config, 5).
|
|
|
|
|
|
|
|
test_no_overload_protection(_Nodes, _BKV, true) ->
|
|
|
|
ok;
|
|
|
|
test_no_overload_protection(Nodes, BKV, ConsistentType) ->
|
2013-05-10 20:54:50 +00:00
|
|
|
lager:info("Testing with no overload protection"),
|
2015-02-11 01:44:09 +00:00
|
|
|
ProcFun = build_predicate_gte(test_no_overload_protection, ?NUM_REQUESTS,
|
|
|
|
"ProcFun", "Procs"),
|
|
|
|
QueueFun = build_predicate_gte(test_no_overload_protection, ?NUM_REQUESTS,
|
|
|
|
"QueueFun", "Queue Size"),
|
2014-07-01 16:46:42 +00:00
|
|
|
verify_test_results(run_test(Nodes, BKV), ConsistentType, ProcFun, QueueFun).
|
|
|
|
|
|
|
|
verify_test_results({_NumProcs, QueueLen}, true, _, QueueFun) ->
|
|
|
|
?assert(QueueFun(QueueLen));
|
|
|
|
verify_test_results({NumProcs, QueueLen}, false, ProcFun, QueueFun) ->
|
|
|
|
?assert(ProcFun(NumProcs)),
|
|
|
|
?assert(QueueFun(QueueLen)).
|
|
|
|
|
|
|
|
test_vnode_protection(Nodes, BKV, ConsistentType) ->
|
2014-01-28 23:49:57 +00:00
|
|
|
%% Setting check_interval to one ensures that process_info is called
|
|
|
|
%% to check the queue length on each vnode send.
|
|
|
|
%% This allows us to artificially raise vnode queue lengths with dummy
|
|
|
|
%% messages instead of having to go through the vnode path for coverage
|
|
|
|
%% query overload testing.
|
2013-05-10 20:54:50 +00:00
|
|
|
lager:info("Testing with vnode queue protection enabled"),
|
|
|
|
lager:info("Setting vnode overload threshold to ~b", [?THRESHOLD]),
|
2014-01-28 23:49:57 +00:00
|
|
|
lager:info("Setting vnode check interval to 1"),
|
2014-07-01 16:46:42 +00:00
|
|
|
Config = [{riak_core, [{vnode_overload_threshold, ?THRESHOLD},
|
|
|
|
{vnode_check_interval, 1}]}],
|
2013-05-10 20:54:50 +00:00
|
|
|
rt:pmap(fun(Node) ->
|
2014-07-01 16:46:42 +00:00
|
|
|
rt:update_app_config(Node, Config)
|
2013-05-10 20:54:50 +00:00
|
|
|
end, Nodes),
|
2015-02-11 22:54:12 +00:00
|
|
|
ProcFun = build_predicate_lt(test_vnode_protection, (?NUM_REQUESTS+1), "ProcFun", "Procs"),
|
2015-02-11 01:49:16 +00:00
|
|
|
QueueFun = build_predicate_lt(test_vnode_protection, (?NUM_REQUESTS), "QueueFun", "QueueSize"),
|
2014-07-01 16:46:42 +00:00
|
|
|
verify_test_results(run_test(Nodes, BKV), ConsistentType, ProcFun, QueueFun),
|
2013-05-10 20:54:50 +00:00
|
|
|
|
2014-07-01 16:46:42 +00:00
|
|
|
[Node1 | _] = Nodes,
|
2013-05-10 20:54:50 +00:00
|
|
|
CheckInterval = ?THRESHOLD div 2,
|
|
|
|
Dropped = read_until_success(Node1),
|
|
|
|
lager:info("Unnecessary dropped requests: ~b", [Dropped]),
|
|
|
|
?assert(Dropped =< CheckInterval),
|
|
|
|
|
2014-07-01 16:46:42 +00:00
|
|
|
Victim = get_victim(Node1, BKV),
|
|
|
|
|
|
|
|
lager:info("Suspending vnode proxy for ~p", [Victim]),
|
|
|
|
Pid = suspend_vnode_proxy(Victim),
|
2015-02-11 01:44:09 +00:00
|
|
|
ProcFun2 = build_predicate_gte("test_vnode_protection after suspend",
|
|
|
|
(?NUM_REQUESTS), "ProcFun", "Procs"),
|
|
|
|
QueueFun2 = build_predicate_lt("test_vnode_protection after suspend",
|
2015-02-11 01:49:16 +00:00
|
|
|
(?NUM_REQUESTS), "QueueFun", "QueueSize"),
|
2014-07-01 16:46:42 +00:00
|
|
|
verify_test_results(run_test(Nodes, BKV), ConsistentType, ProcFun2, QueueFun2),
|
2014-01-24 20:13:24 +00:00
|
|
|
Pid ! resume,
|
2013-05-10 20:54:50 +00:00
|
|
|
ok.
|
|
|
|
|
2015-03-27 20:00:34 +00:00
|
|
|
%% Don't check on fast path
|
2015-04-01 23:00:53 +00:00
|
|
|
test_fsm_protection(_, {{<<"write_once_type">>, _}, _, _}, _) ->
|
2015-03-27 20:00:34 +00:00
|
|
|
ok;
|
2014-07-01 16:46:42 +00:00
|
|
|
test_fsm_protection(Nodes, BKV, ConsistentType) ->
|
2013-05-10 20:54:50 +00:00
|
|
|
lager:info("Testing with coordinator protection enabled"),
|
|
|
|
lager:info("Setting FSM limit to ~b", [?THRESHOLD]),
|
2014-07-01 16:46:42 +00:00
|
|
|
Config = [{riak_kv, [{fsm_limit, ?THRESHOLD}]}],
|
2013-05-10 20:54:50 +00:00
|
|
|
rt:pmap(fun(Node) ->
|
2014-07-01 16:46:42 +00:00
|
|
|
rt:update_app_config(Node, Config)
|
2013-05-10 20:54:50 +00:00
|
|
|
end, Nodes),
|
2015-02-11 01:49:16 +00:00
|
|
|
ProcFun = build_predicate_lt(test_fsm_protection, (?NUM_REQUESTS),
|
2015-02-11 01:44:09 +00:00
|
|
|
"ProcFun", "Procs"),
|
2015-02-11 01:49:16 +00:00
|
|
|
QueueFun = build_predicate_lt(test_fsm_protection, (?NUM_REQUESTS),
|
2015-02-11 01:44:09 +00:00
|
|
|
"QueueFun", "QueueSize"),
|
2014-07-01 16:46:42 +00:00
|
|
|
verify_test_results(run_test(Nodes, BKV), ConsistentType, ProcFun, QueueFun),
|
2013-05-10 20:54:50 +00:00
|
|
|
ok.
|
|
|
|
|
2014-07-01 16:46:42 +00:00
|
|
|
test_cover_queries_overload(_Nodes, _, true) ->
|
|
|
|
ok;
|
|
|
|
test_cover_queries_overload(Nodes, _, false) ->
|
|
|
|
lager:info("Testing cover queries with vnode queue protection enabled"),
|
|
|
|
lager:info("Setting vnode overload threshold to ~b", [?THRESHOLD]),
|
|
|
|
lager:info("Setting vnode check interval to 1"),
|
|
|
|
|
|
|
|
Config = [{riak_core, [{vnode_overload_threshold, ?THRESHOLD},
|
2014-08-22 21:55:59 +00:00
|
|
|
{vnode_request_check_interval, 2},
|
|
|
|
{vnode_check_interval, 1}]}],
|
2014-07-01 16:46:42 +00:00
|
|
|
rt:pmap(fun(Node) ->
|
|
|
|
rt:update_app_config(Node, Config)
|
|
|
|
end, Nodes),
|
|
|
|
|
2014-07-15 22:34:52 +00:00
|
|
|
[rt:wait_for_service(Node, riak_kv) || Node <- Nodes],
|
2014-07-01 16:46:42 +00:00
|
|
|
rt:load_modules_on_nodes([?MODULE], Nodes),
|
|
|
|
|
|
|
|
[Node1, Node2, Node3, Node4, Node5] = Nodes,
|
|
|
|
Pids = [begin
|
|
|
|
lager:info("Suspending all kv vnodes on ~p", [N]),
|
|
|
|
suspend_and_overload_all_kv_vnodes(N)
|
|
|
|
end || N <- [Node2, Node3, Node4, Node5]],
|
|
|
|
|
2014-07-15 22:34:52 +00:00
|
|
|
[?assertEqual({error, <<"mailbox_overload">>}, KeysRes) ||
|
|
|
|
KeysRes <- [list_keys(Node1) || _ <- lists:seq(1, 3)]],
|
|
|
|
|
2014-07-01 16:46:42 +00:00
|
|
|
lager:info("list_keys correctly handled overload"),
|
|
|
|
|
2014-07-15 22:34:52 +00:00
|
|
|
[?assertEqual({error, mailbox_overload}, BucketsRes) ||
|
|
|
|
BucketsRes <- [list_buckets(Node1) || _ <- lists:seq(1, 3)]],
|
2014-07-01 16:46:42 +00:00
|
|
|
lager:info("list_buckets correctly handled overload"),
|
|
|
|
|
|
|
|
lager:info("Resuming all kv vnodes"),
|
|
|
|
[resume_all_vnodes(Pid) || Pid <- Pids],
|
|
|
|
|
|
|
|
lager:info("Waiting for vnode queues to empty"),
|
|
|
|
wait_for_all_vnode_queues_empty(Node2).
|
|
|
|
|
|
|
|
run_test(Nodes, BKV) ->
|
|
|
|
[Node1 | _RestNodes] = Nodes,
|
2013-05-10 20:54:50 +00:00
|
|
|
rt:wait_for_cluster_service(Nodes, riak_kv),
|
2014-07-22 20:48:25 +00:00
|
|
|
lager:info("Sleeping for 5s to let process count stablize"),
|
|
|
|
timer:sleep(5000),
|
2013-05-10 20:54:50 +00:00
|
|
|
rt:load_modules_on_nodes([?MODULE], Nodes),
|
2014-07-22 20:48:25 +00:00
|
|
|
overload_proxy:start_link(),
|
|
|
|
rt_intercept:add(Node1, {riak_kv_get_fsm, [{{start_link, 4}, count_start_link_4}]}),
|
|
|
|
|
2014-07-01 16:46:42 +00:00
|
|
|
Victim = get_victim(Node1, BKV),
|
|
|
|
lager:info("Suspending vnode ~p/~p",
|
|
|
|
[element(1, Victim), element(2, Victim)]),
|
|
|
|
Suspended = suspend_vnode(Victim),
|
|
|
|
|
2014-07-22 20:48:25 +00:00
|
|
|
NumProcs1 = overload_proxy:get_count(),
|
|
|
|
|
2013-05-10 20:54:50 +00:00
|
|
|
lager:info("Initial process count on ~p: ~b", [Node1, NumProcs1]),
|
2014-01-17 02:06:26 +00:00
|
|
|
lager:info("Sending ~b read requests", [?NUM_REQUESTS]),
|
2014-07-01 16:46:42 +00:00
|
|
|
write_once(Node1, BKV),
|
|
|
|
Reads = spawn_reads(Node1, BKV, ?NUM_REQUESTS),
|
2013-05-10 20:54:50 +00:00
|
|
|
timer:sleep(5000),
|
|
|
|
|
2014-08-22 21:55:59 +00:00
|
|
|
rt:wait_until(fun() ->
|
2014-07-22 20:48:25 +00:00
|
|
|
overload_proxy:is_settled(10)
|
|
|
|
end, 5, 500),
|
|
|
|
NumProcs2 = overload_proxy:get_count(),
|
2013-05-10 20:54:50 +00:00
|
|
|
lager:info("Final process count on ~p: ~b", [Node1, NumProcs2]),
|
|
|
|
|
2014-07-01 16:46:42 +00:00
|
|
|
QueueLen = vnode_queue_len(Victim),
|
|
|
|
lager:info("Final vnode queue length for ~p: ~b",
|
|
|
|
[Victim, QueueLen]),
|
2014-07-22 20:48:25 +00:00
|
|
|
|
2013-05-10 20:54:50 +00:00
|
|
|
resume_vnode(Suspended),
|
2014-07-01 16:46:42 +00:00
|
|
|
rt:wait_until(fun() ->
|
2014-07-22 20:48:25 +00:00
|
|
|
vnode_queue_len(Victim) =:= 0
|
2014-07-01 16:46:42 +00:00
|
|
|
end),
|
2014-01-17 02:06:26 +00:00
|
|
|
kill_pids(Reads),
|
2014-07-22 20:48:25 +00:00
|
|
|
overload_proxy:stop(),
|
2013-05-10 20:54:50 +00:00
|
|
|
{NumProcs2 - NumProcs1, QueueLen}.
|
|
|
|
|
2014-07-15 22:34:52 +00:00
|
|
|
get_victim(ExcludeNode, {Bucket, Key, _}) ->
|
|
|
|
Hash = riak_core_util:chash_std_keyfun({Bucket, Key}),
|
|
|
|
PL = lists:sublist(riak_core_ring:preflist(Hash, rt:get_ring(ExcludeNode)), 5),
|
|
|
|
hd([IdxNode || {_, Node}=IdxNode <- PL, Node /= ExcludeNode]).
|
|
|
|
|
2014-07-01 16:46:42 +00:00
|
|
|
ring_manager_check_fun(Node) ->
|
|
|
|
fun() ->
|
|
|
|
case rpc:call(Node, riak_core_ring_manager, get_chash_bin, []) of
|
|
|
|
{ok, _R} ->
|
|
|
|
true;
|
|
|
|
_ ->
|
|
|
|
false
|
|
|
|
end
|
|
|
|
end.
|
2014-01-25 02:44:22 +00:00
|
|
|
|
2014-07-01 16:46:42 +00:00
|
|
|
create_bucket_type(Nodes, Type, Props) ->
|
|
|
|
lager:info("Create bucket type ~p, wait for propagation", [Type]),
|
|
|
|
rt:create_and_activate_bucket_type(hd(Nodes), Type, Props),
|
|
|
|
rt:wait_until_bucket_type_status(Type, active, Nodes),
|
|
|
|
rt:wait_until_bucket_props(Nodes, {Type, <<"bucket">>}, Props),
|
|
|
|
ok.
|
2014-01-17 02:06:26 +00:00
|
|
|
|
2014-08-22 21:55:59 +00:00
|
|
|
node_overload_check(Pid) ->
|
2014-07-15 22:34:52 +00:00
|
|
|
fun() ->
|
2014-08-22 21:55:59 +00:00
|
|
|
Pid ! {verify_overload, self()},
|
|
|
|
receive
|
|
|
|
true ->
|
|
|
|
true;
|
|
|
|
_ ->
|
|
|
|
false
|
|
|
|
end
|
2014-07-15 22:34:52 +00:00
|
|
|
end.
|
|
|
|
|
2014-01-25 02:44:22 +00:00
|
|
|
list_keys(Node) ->
|
2014-07-15 22:34:52 +00:00
|
|
|
Pid = rt:pbc(Node, [{auto_reconnect, true}, {queue_if_disconnected, true}]),
|
2014-07-24 18:47:46 +00:00
|
|
|
Res = riakc_pb_socket:list_keys(Pid, {<<"normal_type">>, ?BUCKET}, infinity),
|
2014-07-15 22:34:52 +00:00
|
|
|
riakc_pb_socket:stop(Pid),
|
|
|
|
Res.
|
2014-01-25 02:44:22 +00:00
|
|
|
|
|
|
|
list_buckets(Node) ->
|
|
|
|
{ok, C} = riak:client_connect(Node),
|
|
|
|
riak_client:list_buckets(30000, C).
|
2014-01-24 20:13:24 +00:00
|
|
|
|
2014-01-17 02:06:26 +00:00
|
|
|
wait_for_all_vnode_queues_empty(Node) ->
|
|
|
|
rt:wait_until(Node, fun(N) ->
|
2014-07-01 16:46:42 +00:00
|
|
|
vnode_queues_empty(N)
|
2014-01-17 02:06:26 +00:00
|
|
|
end).
|
|
|
|
|
|
|
|
vnode_queues_empty(Node) ->
|
|
|
|
rpc:call(Node, ?MODULE, remote_vnode_queues_empty, []).
|
|
|
|
|
|
|
|
remote_vnode_queues_empty() ->
|
|
|
|
lists:all(fun({_, _, Pid}) ->
|
2014-07-01 16:46:42 +00:00
|
|
|
{message_queue_len, Len} =
|
|
|
|
process_info(Pid, message_queue_len),
|
|
|
|
Len =:= 0
|
2014-01-17 02:06:26 +00:00
|
|
|
end, riak_core_vnode_manager:all_vnodes()).
|
|
|
|
|
2014-07-01 16:46:42 +00:00
|
|
|
write_once(Node, {Bucket, Key, Value}) ->
|
|
|
|
lager:info("Writing to node ~p", [Node]),
|
2014-07-15 22:34:52 +00:00
|
|
|
PBC = rt:pbc(Node, [{auto_reconnect, true}, {queue_if_disconnected, true}]),
|
2014-07-01 16:46:42 +00:00
|
|
|
rt:pbc_write(PBC, Bucket, Key, Value),
|
|
|
|
riakc_pb_socket:stop(PBC).
|
2013-05-10 20:54:50 +00:00
|
|
|
|
|
|
|
read_until_success(Node) ->
|
|
|
|
{ok, C} = riak:client_connect(Node),
|
|
|
|
read_until_success(C, 0).
|
|
|
|
|
|
|
|
read_until_success(C, Count) ->
|
|
|
|
case C:get(?BUCKET, ?KEY) of
|
2014-02-13 20:51:49 +00:00
|
|
|
{error, mailbox_overload} ->
|
2013-05-10 20:54:50 +00:00
|
|
|
read_until_success(C, Count+1);
|
|
|
|
_ ->
|
|
|
|
Count
|
|
|
|
end.
|
|
|
|
|
2014-07-01 16:46:42 +00:00
|
|
|
spawn_reads(Node, {Bucket, Key, _}, Num) ->
|
2013-05-10 20:54:50 +00:00
|
|
|
[spawn(fun() ->
|
2014-07-24 18:47:46 +00:00
|
|
|
PBC = rt:pbc(Node,
|
|
|
|
[{auto_reconnect, true},
|
2014-07-15 22:34:52 +00:00
|
|
|
{queue_if_disconnected, true}]),
|
2014-08-22 21:55:59 +00:00
|
|
|
rt:wait_until(pb_get_fun(PBC, Bucket, Key), ?GET_RETRIES, ?GET_RETRIES),
|
2014-07-24 18:47:46 +00:00
|
|
|
%pb_get(PBC, Bucket, Key),
|
|
|
|
riakc_pb_socket:stop(PBC)
|
2014-07-01 16:46:42 +00:00
|
|
|
end) || _ <- lists:seq(1, Num)].
|
2013-05-10 20:54:50 +00:00
|
|
|
|
2014-07-24 18:47:46 +00:00
|
|
|
pb_get_fun(PBC, Bucket, Key) ->
|
|
|
|
fun() ->
|
|
|
|
case riakc_pb_socket:get(PBC, Bucket, Key) of
|
|
|
|
{error, <<"overload">>} ->
|
|
|
|
% lager:info("overload detected in pb_get, continuing..."),
|
|
|
|
true;
|
|
|
|
{error, Type} ->
|
|
|
|
lager:error("riakc_pb_socket failed with ~p, retrying...", [Type]),
|
|
|
|
false;
|
|
|
|
{ok, _Res} ->
|
|
|
|
% lager:info("riakc_pb_socket:get(~p, ~p, ~p) succeeded, Res:~p", [PBC, Bucket, Key, Res]),
|
|
|
|
true
|
2014-08-22 21:55:59 +00:00
|
|
|
end
|
2014-07-24 18:47:46 +00:00
|
|
|
end.
|
|
|
|
|
|
|
|
pb_get(PBC, Bucket, Key) ->
|
|
|
|
case riakc_pb_socket:get(PBC, Bucket, Key) of
|
|
|
|
{error, <<"overload">>} ->
|
|
|
|
lager:info("overload detected in pb_get, continuing...");
|
|
|
|
{error, Type} ->
|
|
|
|
lager:error("riakc_pb_socket failed with ~p, retrying...", [Type]),
|
|
|
|
pb_get(PBC, Bucket, Key);
|
|
|
|
{ok, Res} ->
|
|
|
|
lager:info("riakc_pb_socket:get(~p, ~p, ~p) succeeded, Res:~p", [PBC, Bucket, Key, Res])
|
|
|
|
end.
|
|
|
|
|
2014-01-17 02:06:26 +00:00
|
|
|
kill_pids(Pids) ->
|
2013-05-10 20:54:50 +00:00
|
|
|
[exit(Pid, kill) || Pid <- Pids].
|
|
|
|
|
2014-01-28 23:49:57 +00:00
|
|
|
suspend_and_overload_all_kv_vnodes(Node) ->
|
2014-07-15 22:34:52 +00:00
|
|
|
lager:info("Suspending vnodes on ~p", [Node]),
|
2014-01-28 23:49:57 +00:00
|
|
|
Pid = rpc:call(Node, ?MODULE, remote_suspend_and_overload, []),
|
|
|
|
Pid ! {overload, self()},
|
|
|
|
receive overloaded ->
|
2014-07-01 16:46:42 +00:00
|
|
|
Pid
|
2014-08-22 21:55:59 +00:00
|
|
|
end,
|
|
|
|
rt:wait_until(node_overload_check(Pid)),
|
|
|
|
Pid.
|
2014-01-17 02:06:26 +00:00
|
|
|
|
2014-01-28 23:49:57 +00:00
|
|
|
remote_suspend_and_overload() ->
|
2014-01-17 02:06:26 +00:00
|
|
|
spawn(fun() ->
|
|
|
|
Vnodes = riak_core_vnode_manager:all_vnodes(),
|
2014-07-01 16:46:42 +00:00
|
|
|
[begin
|
2014-08-22 21:55:59 +00:00
|
|
|
lager:info("Suspending vnode pid: ~p~n", [Pid]),
|
|
|
|
erlang:suspend_process(Pid, [])
|
|
|
|
end || {riak_kv_vnode, _, Pid} <- Vnodes],
|
|
|
|
?MODULE:wait_for_input(Vnodes)
|
2014-01-17 02:06:26 +00:00
|
|
|
end).
|
|
|
|
|
2014-08-22 21:55:59 +00:00
|
|
|
wait_for_input(Vnodes) ->
|
|
|
|
receive
|
|
|
|
{overload, From} ->
|
|
|
|
[?MODULE:overload(Pid) ||
|
|
|
|
{riak_kv_vnode, _, Pid} <- Vnodes],
|
|
|
|
From ! overloaded,
|
|
|
|
wait_for_input(Vnodes);
|
|
|
|
{verify_overload, From} ->
|
|
|
|
OverloadCheck = ?MODULE:verify_overload(Vnodes),
|
|
|
|
From ! OverloadCheck,
|
|
|
|
wait_for_input(Vnodes);
|
|
|
|
resume ->
|
|
|
|
lager:info("Resuming vnodes~n"),
|
|
|
|
[erlang:resume_process(Pid) || {riak_kv_vnode, _, Pid}
|
|
|
|
<- Vnodes]
|
|
|
|
end.
|
|
|
|
|
|
|
|
verify_overload(Vnodes) ->
|
|
|
|
MessageLists = [element(2, process_info(Pid, messages)) ||
|
|
|
|
{riak_kv_vnode, _, Pid} <- Vnodes],
|
|
|
|
OverloadMsgCounts = lists:foldl(fun overload_msg_counter/2, [], MessageLists),
|
|
|
|
lists:all(fun(X) -> X >= ?NUM_REQUESTS end, OverloadMsgCounts).
|
|
|
|
|
|
|
|
overload_msg_counter(Messages, Acc) ->
|
|
|
|
Count = lists:foldl(fun count_overload_messages/2, 0, Messages),
|
|
|
|
[Count | Acc].
|
|
|
|
|
|
|
|
count_overload_messages(Message, Count) ->
|
|
|
|
case Message of
|
|
|
|
{set_concurrency_limit, some_lock, 1} ->
|
|
|
|
Count + 1;
|
|
|
|
_ ->
|
|
|
|
Count
|
|
|
|
end.
|
|
|
|
|
2014-01-28 23:49:57 +00:00
|
|
|
overload(Pid) ->
|
2014-07-01 16:46:42 +00:00
|
|
|
%% The actual message doesn't matter. This one just has the least
|
|
|
|
%% side effects.
|
|
|
|
[Pid ! {set_concurrency_limit, some_lock, 1} ||
|
|
|
|
_ <- lists:seq(1, ?NUM_REQUESTS)].
|
|
|
|
|
|
|
|
suspend_vnode({Idx, Node}) ->
|
|
|
|
suspend_vnode(Node, Idx).
|
2014-01-28 23:49:57 +00:00
|
|
|
|
2013-05-10 20:54:50 +00:00
|
|
|
suspend_vnode(Node, Idx) ->
|
2014-07-01 16:46:42 +00:00
|
|
|
rpc:call(Node, ?MODULE, remote_suspend_vnode, [Idx], infinity).
|
2013-05-10 20:54:50 +00:00
|
|
|
|
|
|
|
remote_suspend_vnode(Idx) ->
|
|
|
|
spawn(fun() ->
|
|
|
|
{ok, Pid} = riak_core_vnode_manager:get_vnode_pid(Idx, riak_kv_vnode),
|
2014-07-01 16:46:42 +00:00
|
|
|
lager:info("Suspending vnode pid: ~p", [Pid]),
|
2013-05-10 20:54:50 +00:00
|
|
|
erlang:suspend_process(Pid, []),
|
|
|
|
receive resume ->
|
|
|
|
erlang:resume_process(Pid)
|
|
|
|
end
|
|
|
|
end).
|
|
|
|
|
2014-07-01 16:46:42 +00:00
|
|
|
suspend_vnode_proxy({Idx, Node}) ->
|
|
|
|
suspend_vnode_proxy(Node, Idx).
|
|
|
|
|
2013-05-10 20:54:50 +00:00
|
|
|
suspend_vnode_proxy(Node, Idx) ->
|
2014-07-01 16:46:42 +00:00
|
|
|
rpc:call(Node, ?MODULE, remote_suspend_vnode_proxy, [Idx], infinity).
|
2013-05-10 20:54:50 +00:00
|
|
|
|
|
|
|
remote_suspend_vnode_proxy(Idx) ->
|
|
|
|
spawn(fun() ->
|
|
|
|
Name = riak_core_vnode_proxy:reg_name(riak_kv_vnode, Idx),
|
|
|
|
Pid = whereis(Name),
|
|
|
|
erlang:suspend_process(Pid, []),
|
|
|
|
receive resume ->
|
|
|
|
erlang:resume_process(Pid)
|
|
|
|
end
|
|
|
|
end).
|
|
|
|
|
2014-01-17 02:06:26 +00:00
|
|
|
resume_all_vnodes(Pid) ->
|
|
|
|
Pid ! resume.
|
|
|
|
|
2013-05-10 20:54:50 +00:00
|
|
|
resume_vnode(Pid) ->
|
|
|
|
Pid ! resume.
|
|
|
|
|
|
|
|
process_count(Node) ->
|
|
|
|
rpc:call(Node, erlang, system_info, [process_count]).
|
|
|
|
|
2014-07-01 16:46:42 +00:00
|
|
|
vnode_queue_len({Idx, Node}) ->
|
|
|
|
vnode_queue_len(Node, Idx).
|
|
|
|
|
2013-05-10 20:54:50 +00:00
|
|
|
vnode_queue_len(Node, Idx) ->
|
|
|
|
rpc:call(Node, ?MODULE, remote_vnode_queue, [Idx]).
|
|
|
|
|
|
|
|
dropped_stat(Node) ->
|
|
|
|
Stats = rpc:call(Node, riak_core_stat, get_stats, []),
|
|
|
|
proplists:get_value(dropped_vnode_requests_total, Stats).
|
|
|
|
|
2014-07-22 20:48:25 +00:00
|
|
|
get_fsm_active_stat(Node) ->
|
|
|
|
Stats = rpc:call(Node, riak_kv_stat, get_stats, []),
|
|
|
|
proplists:get_value(node_get_fsm_active, Stats).
|
|
|
|
|
|
|
|
run_count(Node) ->
|
|
|
|
timer:sleep(500),
|
|
|
|
lager:info("fsm count:~p", [get_num_running_gen_fsm(Node)]),
|
|
|
|
run_count(Node).
|
|
|
|
|
|
|
|
run_queue_len({Idx, Node}) ->
|
2014-08-22 21:55:59 +00:00
|
|
|
timer:sleep(500),
|
2014-07-22 20:48:25 +00:00
|
|
|
Len = vnode_queue_len(Node, Idx),
|
|
|
|
lager:info("queue len on ~p is:~p", [Node, Len]),
|
|
|
|
run_queue_len({Idx, Node}).
|
|
|
|
|
|
|
|
get_num_running_gen_fsm(Node) ->
|
|
|
|
Procs = rpc:call(Node, erlang, processes, []),
|
|
|
|
ProcInfo = [ rpc:call(Node, erlang, process_info, [P]) || P <- Procs, P /= undefined ],
|
|
|
|
|
|
|
|
InitCalls = [ [ proplists:get_value(initial_call, Proc) ] || Proc <- ProcInfo, Proc /= undefined ],
|
|
|
|
FsmList = [ proplists:lookup(riak_kv_get_fsm, Call) || Call <- InitCalls ],
|
|
|
|
length(proplists:lookup_all(riak_kv_get_fsm, FsmList)).
|
2014-08-22 21:55:59 +00:00
|
|
|
|
2013-05-10 20:54:50 +00:00
|
|
|
remote_vnode_queue(Idx) ->
|
|
|
|
{ok, Pid} = riak_core_vnode_manager:get_vnode_pid(Idx, riak_kv_vnode),
|
|
|
|
{message_queue_len, Len} = process_info(Pid, message_queue_len),
|
|
|
|
Len.
|
2015-02-11 01:44:09 +00:00
|
|
|
|
|
|
|
%% In tests that do not expect work to be shed, we want to confirm that
|
|
|
|
%% at least ?NUM_REQUESTS (processes|queue entries) are handled.
|
|
|
|
build_predicate_gte(Test, Metric, Label, ValueLabel) ->
|
|
|
|
fun (X) ->
|
2015-02-11 02:11:32 +00:00
|
|
|
lager:info("in test ~p ~p, ~p:~p, expected no overload, Metric:>=~p",
|
2015-02-11 01:44:09 +00:00
|
|
|
[Test, Label, ValueLabel, X, Metric]),
|
|
|
|
X >= Metric
|
|
|
|
end.
|
|
|
|
%% In tests that expect work to be shed due to overload, the success
|
|
|
|
%% condition is simply that the number of (fsms|queue entries) is
|
|
|
|
%% less than ?NUM_REQUESTS.
|
|
|
|
build_predicate_lt(Test, Metric, Label, ValueLabel) ->
|
|
|
|
fun (X) ->
|
2015-02-11 02:11:32 +00:00
|
|
|
lager:info("in test ~p ~p, ~p:~p, expected overload, Metric:<~p",
|
2015-02-11 01:44:09 +00:00
|
|
|
[Test, Label, ValueLabel, X, Metric]),
|
|
|
|
X < Metric
|
|
|
|
end.
|