From 6d9fbb356f4fb6b87bca822dbb1c118e0960f5c1 Mon Sep 17 00:00:00 2001 From: Kelly McLaughlin Date: Fri, 28 Feb 2014 11:41:08 -0700 Subject: [PATCH 001/157] Make some changes to riak_test to make it more flexible * Change riak_test API to add a properties and setup function in addition to confirm. The goal is to remove test environment specification from actual test case logic. * Change confirm/0 to confirm/2 to accept data about the test environment as input. * Add capability for any test to be run as a rolling upgrade test. * Add rt_cluster module with some supporting functions. * Change verify_build_cluster and secondary_index_tests modules to conform to the new API in order to demonstrate the changes. --- include/rt.hrl | 15 +++ src/riak_test_escript.erl | 8 +- src/riak_test_runner.erl | 80 +++++++++++-- src/rt.erl | 17 +++ src/rt_cluster.erl | 78 +++++++++++++ src/rt_cover.erl | 2 +- tests/secondary_index_tests.erl | 200 ++++++++++++++++---------------- tests/verify_build_cluster.erl | 34 +++--- 8 files changed, 305 insertions(+), 129 deletions(-) create mode 100644 src/rt_cluster.erl diff --git a/include/rt.hrl b/include/rt.hrl index 78de7e028..a40996d61 100644 --- a/include/rt.hrl +++ b/include/rt.hrl @@ -23,3 +23,18 @@ url :: string(), headers=[] :: [{atom(), string()}] }). + +-record(rt_properties, { + nodes :: [node()], + node_count=6 :: non_neg_integer(), + metadata=[] :: proplists:proplist(), + properties=[] :: proplists:proplist(), + rolling_upgrade=false :: boolean(), + start_version=current :: atom(), + upgrade_version=current :: atom(), + wait_for_transfers=false :: boolean(), + valid_backends=all :: all | [atom()], + make_cluster=true :: boolean(), + config :: term() + }). +-type rt_properties() :: #rt_properties{}. diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index a34da59f8..ffc9de750 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -282,14 +282,14 @@ filter_merge_meta(SMeta, CMeta, [Field|Rest]) -> %% Check for api compatibility is_runnable_test({TestModule, _}) -> - {Mod, Fun} = riak_test_runner:function_name(TestModule), + {Mod, Fun} = riak_test_runner:function_name(confirm, TestModule), code:ensure_loaded(Mod), - erlang:function_exported(Mod, Fun, 0). + erlang:function_exported(Mod, Fun, 0) orelse + erlang:function_exported(Mod, Fun, 2). run_test(Test, Outdir, TestMetaData, Report, HarnessArgs, NumTests) -> rt_cover:maybe_start(Test), - SingleTestResult = riak_test_runner:confirm(Test, Outdir, TestMetaData, - HarnessArgs), + SingleTestResult = riak_test_runner:run(Test, Outdir, TestMetaData, HarnessArgs), CoverDir = rt_config:get(cover_output, "coverage"), case NumTests of 1 -> keep_them_up; diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index eac78838b..62da93043 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -20,9 +20,11 @@ %% @doc riak_test_runner runs a riak_test module's run/0 function. -module(riak_test_runner). --export([confirm/4, metadata/0, metadata/1, function_name/1]). + %% Need to export to use with `spawn_link'. -export([return_to_exit/3]). +-export([run/4, metadata/0, metadata/1, function_name/2]). +-include("rt.hrl"). -include_lib("eunit/include/eunit.hrl"). -spec(metadata() -> [{atom(), term()}]). @@ -39,10 +41,10 @@ metadata(Pid) -> {metadata, TestMeta} -> TestMeta end. --spec(confirm(integer(), atom(), [{atom(), term()}], list()) -> [tuple()]). +-spec(run(integer(), atom(), [{atom(), term()}], list()) -> [tuple()]). %% @doc Runs a module's run/0 function after setting up a log capturing backend for lager. %% It then cleans up that backend and returns the logs as part of the return proplist. -confirm(TestModule, Outdir, TestMetaData, HarnessArgs) -> +run(TestModule, Outdir, TestMetaData, HarnessArgs) -> start_lager_backend(TestModule, Outdir), rt:setup_harness(TestModule, HarnessArgs), BackendExtras = case proplists:get_value(multi_config, TestMetaData) of @@ -50,11 +52,17 @@ confirm(TestModule, Outdir, TestMetaData, HarnessArgs) -> Value -> [{multi_config, Value}] end, Backend = rt:set_backend(proplists:get_value(backend, TestMetaData), BackendExtras), - {Mod, Fun} = function_name(TestModule), - {Status, Reason} = case check_prereqs(Mod) of + {PropsMod, PropsFun} = function_name(properties, TestModule, 0, rt_cluster), + {SetupMod, SetupFun} = function_name(setup, TestModule, 2, rt_cluster), + {ConfirmMod, ConfirmFun} = function_name(confirm, TestModule), + {Status, Reason} = case check_prereqs(ConfirmMod) of true -> lager:notice("Running Test ~s", [TestModule]), - execute(TestModule, {Mod, Fun}, TestMetaData); + execute(TestModule, + {PropsMod, PropsFun}, + {SetupMod, SetupFun}, + {ConfirmMod, ConfirmFun}, + TestMetaData); not_present -> {fail, test_does_not_exist}; _ -> @@ -88,7 +96,7 @@ stop_lager_backend() -> gen_event:delete_handler(lager_event, riak_test_lager_backend, []). %% does some group_leader swapping, in the style of EUnit. -execute(TestModule, {Mod, Fun}, TestMetaData) -> +execute(TestModule, PropsModFun, SetupModFun, ConfirmModFun, TestMetaData) -> process_flag(trap_exit, true), OldGroupLeader = group_leader(), NewGroupLeader = riak_test_group_leader:new_group_leader(self()), @@ -97,7 +105,8 @@ execute(TestModule, {Mod, Fun}, TestMetaData) -> {0, UName} = rt:cmd("uname -a"), lager:info("Test Runner `uname -a` : ~s", [UName]), - Pid = spawn_link(?MODULE, return_to_exit, [Mod, Fun, []]), + %% Pid = spawn_link(?MODULE, return_to_exit, [Mod, Fun, []]), + Pid = spawn_link(test_fun(PropsModFun, SetupModFun, ConfirmModFun, TestMetaData)), Ref = case rt_config:get(test_timeout, undefined) of Timeout when is_integer(Timeout) -> erlang:send_after(Timeout, self(), test_took_too_long); @@ -123,7 +132,52 @@ execute(TestModule, {Mod, Fun}, TestMetaData) -> end, {Status, Reason}. -function_name(TestModule) -> +-spec test_fun({atom(), atom()}, {atom(), atom()}, {atom(), atom()}, proplists:proplist()) -> function(). +test_fun({PropsMod, PropsFun}, {SetupMod, SetupFun}, ConfirmModFun, MetaData) -> + fun() -> + Properties = PropsMod:PropsFun(), + case SetupMod:SetupFun(Properties, MetaData) of + {ok, SetupData} -> + lager:info("Wait for transfers? ~p", [SetupData#rt_properties.wait_for_transfers]), + ConfirmFun = compose_confirm_fun(ConfirmModFun, + SetupData, + MetaData), + ConfirmFun(); + _ -> + fail + end + end. + +compose_confirm_fun({ConfirmMod, ConfirmFun}, + SetupData=#rt_properties{rolling_upgrade=true}, + MetaData) -> + Nodes = SetupData#rt_properties.nodes, + WaitForTransfers = SetupData#rt_properties.wait_for_transfers, + UpgradeVersion = SetupData#rt_properties.upgrade_version, + fun() -> + InitialResult = ConfirmMod:ConfirmFun(SetupData, MetaData), + OtherResults = [begin + ensure_all_nodes_running(Nodes), + _ = rt:upgrade(Node, UpgradeVersion), + _ = rt_cluster:maybe_wait_for_transfers(Nodes, WaitForTransfers), + ConfirmMod:ConfirmFun(SetupData, MetaData) + end || Node <- Nodes], + lists:all(fun(R) -> R =:= pass end, [InitialResult | OtherResults]) + end; +compose_confirm_fun({ConfirmMod, ConfirmFun}, + SetupData=#rt_properties{rolling_upgrade=false}, + MetaData) -> + fun() -> + ConfirmMod:ConfirmFun(SetupData, MetaData) + end. + +ensure_all_nodes_running(Nodes) -> + [begin + ok = rt:start_and_wait(Node), + ok = rt:wait_until_registered(Node, riak_core_ring_manager) + end || Node <- Nodes]. + +function_name(confirm, TestModule) -> TMString = atom_to_list(TestModule), Tokz = string:tokens(TMString, ":"), case length(Tokz) of @@ -133,6 +187,14 @@ function_name(TestModule) -> {list_to_atom(Module), list_to_atom(Function)} end. +function_name(FunName, TestModule, Arity, Default) when is_atom(TestModule) -> + case erlang:function_exported(TestModule, FunName, Arity) of + true -> + {TestModule, FunName}; + false -> + {Default, FunName} + end. + rec_loop(Pid, TestModule, TestMetaData) -> receive test_took_too_long -> diff --git a/src/rt.erl b/src/rt.erl index 8b4db1490..e9eebeef7 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -1306,6 +1306,23 @@ update_acc(false, {error, _}=Val, N, Acc) -> update_acc(false, Val, N, Acc) -> [{N, {wrong_val, Val}} | Acc]. +verify_systest_value(N, Acc, CommonValBin, Obj) -> + Values = riak_object:get_values(Obj), + Res = [begin + case V of + <> -> + ok; + _WrongVal -> + wrong_val + end + end || V <- Values], + case lists:any(fun(X) -> X =:= ok end, Res) of + true -> + Acc; + false -> + [{N, {wrong_val, hd(Values)}} | Acc] + end. + % @doc Reads a single replica of a value. This issues a get command directly % to the vnode handling the Nth primary partition of the object's preflist. get_replica(Node, Bucket, Key, I, N) -> diff --git a/src/rt_cluster.erl b/src/rt_cluster.erl new file mode 100644 index 000000000..372bd0869 --- /dev/null +++ b/src/rt_cluster.erl @@ -0,0 +1,78 @@ +%% ------------------------------------------------------------------- +%% +%% Copyright (c) 2014 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(rt_cluster). + +-export([properties/0, + setup/2, + config/0, + augment_config/3]). +-export([maybe_wait_for_transfers/2]). + +-include("rt.hrl"). + +%% @doc Default properties used if a riak_test module does not specify +%% a custom properties function. +-spec properties() -> rt_properties(). +properties() -> + #rt_properties{config=config()}. + +-spec setup(rt_properties(), proplists:proplist()) -> + {ok, rt_properties()} | {error, term()}. +setup(Properties, MetaData) -> + rt:set_conf(all, [{"buckets.default.allow_mult", "false"}]), + + RollingUpgrade = proplists:get_value(rolling_upgrade, + MetaData, + Properties#rt_properties.rolling_upgrade), + Version = Properties#rt_properties.start_version, + Versions = [{Version, Properties#rt_properties.config} || + _ <- lists:seq(1, Properties#rt_properties.node_count)], + Nodes = deploy_or_build_cluster(Versions, Properties#rt_properties.make_cluster), + + maybe_wait_for_transfers(Nodes, Properties#rt_properties.wait_for_transfers), + UpdProperties = Properties#rt_properties{nodes=Nodes, + rolling_upgrade=RollingUpgrade}, + {ok, UpdProperties}. + +deploy_or_build_cluster(Versions, true) -> + rt:build_cluster(Versions); +deploy_or_build_cluster(Versions, false) -> + rt:deploy_nodes(Versions). + +maybe_wait_for_transfers(Nodes, true) -> + lager:info("Waiting for transfers"), + rt:wait_until_transfers_complete(Nodes); +maybe_wait_for_transfers(_Nodes, false) -> + ok. + +config() -> + [{riak_core, [{handoff_concurrency, 11}]}, + {riak_search, [{enabled, true}]}, + {riak_pipe, [{worker_limit, 200}]}]. + +augment_config(Section, Property, Config) -> + UpdSectionConfig = update_section(Section, Property, lists:keyfind(Section, 1, Config)), + lists:keyreplace(Section, 1, Config, UpdSectionConfig). + +update_section(Section, Property, false) -> + {Section, [Property]}; +update_section(Section, Property, {Section, SectionConfig}) -> + {Section, [Property | SectionConfig]}. diff --git a/src/rt_cover.erl b/src/rt_cover.erl index 5a970039f..b158b5035 100644 --- a/src/rt_cover.erl +++ b/src/rt_cover.erl @@ -118,7 +118,7 @@ start2(CoverMods) -> %% These are read, per test, from the test module attributes %% `cover_modules' or `cover_apps'. find_cover_modules(Test) -> - {Mod, _Fun} = riak_test_runner:function_name(Test), + {Mod, _Fun} = riak_test_runner:function_name(confirm, Test), case proplists:get_value(cover_modules, Mod:module_info(attributes), []) of [] -> case proplists:get_value(cover_apps, Mod:module_info(attributes), []) of diff --git a/tests/secondary_index_tests.erl b/tests/secondary_index_tests.erl index aa4618623..26827c6d5 100644 --- a/tests/secondary_index_tests.erl +++ b/tests/secondary_index_tests.erl @@ -18,89 +18,96 @@ %% %% ------------------------------------------------------------------- -module(secondary_index_tests). --behavior(riak_test). --export([confirm/0]). --export([put_an_object/2, put_an_object/4, int_to_key/1, - stream_pb/2, stream_pb/3, pb_query/3, http_query/2, - http_query/3, http_stream/3, int_to_field1_bin/1, url/2, +-export([confirm/2, + properties/0]). +-export([put_an_object/3, put_an_object/5, int_to_key/1, + stream_pb/3, stream_pb/4, pb_query/4, http_query/3, + http_query/4, http_stream/4, int_to_field1_bin/1, url/2, assertExactQuery/5, assertRangeQuery/7]). +-include("rt.hrl"). -include_lib("eunit/include/eunit.hrl"). -include_lib("riakc/include/riakc.hrl"). --define(BUCKET, <<"2ibucket">>). -define(KEYS(A), [int_to_key(A)]). -define(KEYS(A,B), [int_to_key(N) || N <- lists:seq(A,B)]). -define(KEYS(A,B,C), [int_to_key(N) || N <- lists:seq(A,B), C]). -define(KEYS(A,B,G1,G2), [int_to_key(N) || N <- lists:seq(A,B), G1, G2]). -confirm() -> - Nodes = rt:build_cluster(3), - ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), - - %% First test with sorting non-paginated results off by default - SetResult = rpc:multicall(Nodes, application, set_env, - [riak_kv, secondary_index_sort_default, false]), - AOK = [ok || _ <- lists:seq(1, length(Nodes))], - ?assertMatch({AOK, []}, SetResult), - +properties() -> + DefaultProps = rt_cluster:properties(), + DefaultProps#rt_properties{node_count=3, + wait_for_transfers=true, + rolling_upgrade=false, + start_version=previous, + config=config()}. + +config() -> + [ + {riak_kv, [{secondary_index_sort_default, false}]} + ]. + +confirm(#rt_properties{nodes=Nodes}, _MD) -> + Bucket = druuid:v4_str(), + lager:info("Bucket: ~p", [Bucket]), PBC = rt:pbc(hd(Nodes)), HTTPC = rt:httpc(hd(Nodes)), Clients = [{pb, PBC}, {http, HTTPC}], - - [put_an_object(PBC, N) || N <- lists:seq(0, 20)], - + + [put_an_object(PBC, Bucket, N) || N <- lists:seq(0, 20)], + K = fun int_to_key/1, - assertExactQuery(Clients, ?KEYS(5), <<"field1_bin">>, <<"val5">>), - assertExactQuery(Clients, ?KEYS(5), <<"field2_int">>, 5), - assertExactQuery(Clients, ?KEYS(5, 9), <<"field3_int">>, 5), - assertRangeQuery(Clients, ?KEYS(10, 18), <<"field1_bin">>, <<"val10">>, <<"val18">>), - assertRangeQuery(Clients, ?KEYS(12), <<"field1_bin">>, <<"val10">>, <<"val18">>, <<"v...2">>), - assertRangeQuery(Clients, ?KEYS(10, 19), <<"field2_int">>, 10, 19), - assertRangeQuery(Clients, ?KEYS(10, 17), <<"$key">>, <<"obj10">>, <<"obj17">>), - assertRangeQuery(Clients, ?KEYS(12), <<"$key">>, <<"obj10">>, <<"obj17">>, <<"ob..2">>), + assertExactQuery(Clients, Bucket, ?KEYS(5), <<"field1_bin">>, <<"val5">>), + assertExactQuery(Clients, Bucket, ?KEYS(5), <<"field2_int">>, 5), + assertExactQuery(Clients, Bucket, ?KEYS(5, 9), <<"field3_int">>, 5), + assertRangeQuery(Clients, Bucket, ?KEYS(10, 18), <<"field1_bin">>, <<"val10">>, <<"val18">>), + assertRangeQuery(Clients, Bucket, ?KEYS(12), <<"field1_bin">>, <<"val10">>, <<"val18">>, <<"v...2">>), + assertRangeQuery(Clients, Bucket, ?KEYS(10, 19), <<"field2_int">>, 10, 19), + assertRangeQuery(Clients, Bucket, ?KEYS(10, 17), <<"$key">>, <<"obj10">>, <<"obj17">>), + assertRangeQuery(Clients, Bucket, ?KEYS(12), <<"$key">>, <<"obj10">>, <<"obj17">>, <<"ob..2">>), lager:info("Delete an object, verify deletion..."), ToDel = [<<"obj05">>, <<"obj11">>], - [?assertMatch(ok, riakc_pb_socket:delete(PBC, ?BUCKET, KD)) || KD <- ToDel], + [?assertMatch(ok, riakc_pb_socket:delete(PBC, Bucket, KD)) || KD <- ToDel], lager:info("Make sure the tombstone is reaped..."), - ?assertMatch(ok, rt:wait_until(fun() -> rt:pbc_really_deleted(PBC, ?BUCKET, ToDel) end)), - - assertExactQuery(Clients, [], <<"field1_bin">>, <<"val5">>), - assertExactQuery(Clients, [], <<"field2_int">>, 5), - assertExactQuery(Clients, ?KEYS(6, 9), <<"field3_int">>, 5), - assertRangeQuery(Clients, ?KEYS(10, 18, N /= 11), <<"field1_bin">>, <<"val10">>, <<"val18">>), - assertRangeQuery(Clients, ?KEYS(10), <<"field1_bin">>, <<"val10">>, <<"val18">>, <<"10$">>), - assertRangeQuery(Clients, ?KEYS(10, 19, N /= 11), <<"field2_int">>, 10, 19), - assertRangeQuery(Clients, ?KEYS(10, 17, N /= 11), <<"$key">>, <<"obj10">>, <<"obj17">>), - assertRangeQuery(Clients, ?KEYS(12), <<"$key">>, <<"obj10">>, <<"obj17">>, <<"2">>), + ?assertMatch(ok, rt:wait_until(fun() -> rt:pbc_really_deleted(PBC, Bucket, ToDel) end)), + + assertExactQuery(Clients, Bucket, [], <<"field1_bin">>, <<"val5">>), + assertExactQuery(Clients, Bucket, [], <<"field2_int">>, 5), + assertExactQuery(Clients, Bucket, ?KEYS(6, 9), <<"field3_int">>, 5), + assertRangeQuery(Clients, Bucket, ?KEYS(10, 18, N /= 11), <<"field1_bin">>, <<"val10">>, <<"val18">>), + assertRangeQuery(Clients, Bucket, ?KEYS(10), <<"field1_bin">>, <<"val10">>, <<"val18">>, <<"10$">>), + assertRangeQuery(Clients, Bucket, ?KEYS(10, 19, N /= 11), <<"field2_int">>, 10, 19), + assertRangeQuery(Clients, Bucket, ?KEYS(10, 17, N /= 11), <<"$key">>, <<"obj10">>, <<"obj17">>), + assertRangeQuery(Clients, Bucket, ?KEYS(12), <<"$key">>, <<"obj10">>, <<"obj17">>, <<"2">>), %% Verify the $key index, and riak_kv#367 regression - assertRangeQuery(Clients, ?KEYS(6), <<"$key">>, <<"obj06">>, <<"obj06">>), - assertRangeQuery(Clients, ?KEYS(6,7), <<"$key">>, <<"obj06">>, <<"obj07">>), + assertRangeQuery(Clients, Bucket, ?KEYS(6), <<"$key">>, <<"obj06">>, <<"obj06">>), + assertRangeQuery(Clients, Bucket, ?KEYS(6,7), <<"$key">>, <<"obj06">>, <<"obj07">>), %% Exercise sort set to true by default SetResult2 = rpc:multicall(Nodes, application, set_env, [riak_kv, secondary_index_sort_default, true]), - ?assertMatch({AOK, []}, SetResult2), - - assertExactQuery(Clients, ?KEYS(15, 19), + ?assertMatch({_, []}, SetResult2), + + assertExactQuery(Clients, Bucket, ?KEYS(15, 19), <<"field3_int">>, 15, {undefined, true}), %% Keys ordered by val index term, since 2i order is {term, key} KsVal = [A || {_, A} <- lists:sort([{int_to_field1_bin(N), K(N)} || N <- lists:seq(0, 20), N /= 11, N /= 5])], - assertRangeQuery(Clients, KsVal, + assertRangeQuery(Clients, Bucket, KsVal, <<"field1_bin">>, <<"val0">>, <<"val9">>, undefined, {undefined, true}), - assertRangeQuery(Clients, ?KEYS(0, 20, N /= 11, N /= 5), + assertRangeQuery(Clients, Bucket, ?KEYS(0, 20, N /= 11, N /= 5), <<"field2_int">>, 0, 20, undefined, {undefined, true}), - assertRangeQuery(Clients, ?KEYS(0, 20, N /= 11, N /= 5), + assertRangeQuery(Clients, Bucket, ?KEYS(0, 20, N /= 11, N /= 5), <<"$key">>, <<"obj00">>, <<"obj20">>, undefined, {undefined, true}), %% Verify bignum sort order in sext -- eleveldb only (riak_kv#499) TestIdxVal = 1362400142028, - put_an_object(PBC, TestIdxVal), + put_an_object(PBC, Bucket, TestIdxVal), assertRangeQuery(Clients, + Bucket, [<<"obj1362400142028">>], <<"field2_int">>, 1000000000000, @@ -108,24 +115,24 @@ confirm() -> pass. -assertExactQuery(Clients, Expected, Index, Value) -> - assertExactQuery(Clients, Expected, Index, Value, {false, false}), - assertExactQuery(Clients, Expected, Index, Value, {true, true}). +assertExactQuery(Clients, Bucket, Expected, Index, Value) -> + assertExactQuery(Clients, Bucket, Expected, Index, Value, {false, false}), + assertExactQuery(Clients, Bucket, Expected, Index, Value, {true, true}). -assertExactQuery(Clients, Expected, Index, Value, Sorted) when is_list(Clients) -> - [assertExactQuery(C, Expected, Index, Value, Sorted) || C <- Clients]; -assertExactQuery({ClientType, Client}, Expected, Index, Value, - {Sort, ExpectSorted}) -> +assertExactQuery(Clients, Bucket, Expected, Index, Value, Sorted) when is_list(Clients) -> + [assertExactQuery(C, Bucket, Expected, Index, Value, Sorted) || C <- Clients]; +assertExactQuery({ClientType, Client}, Bucket, Expected, Index, Value, + {Sort, ExpectSorted}) -> lager:info("Searching Index ~p for ~p, sort: ~p ~p with client ~p", [Index, Value, Sort, ExpectSorted, ClientType]), {ok, ?INDEX_RESULTS{keys=Results}} = case ClientType of pb -> - riakc_pb_socket:get_index_eq(Client, ?BUCKET, Index, Value, + riakc_pb_socket:get_index_eq(Client, Bucket, Index, Value, [{pagination_sort, Sort} || Sort /= undefined]); http -> - rhc:get_index(Client, ?BUCKET, Index, Value, [{pagination_sort, Sort}]) + rhc:get_index(Client, Bucket, Index, Value, [{pagination_sort, Sort}]) end, - + ActualKeys = case ExpectSorted of true -> Results; _ -> lists:sort(Results) @@ -133,28 +140,28 @@ assertExactQuery({ClientType, Client}, Expected, Index, Value, lager:info("Expected: ~p", [Expected]), lager:info("Actual : ~p", [Results]), lager:info("Sorted : ~p", [ActualKeys]), - ?assertEqual(Expected, ActualKeys). - -assertRangeQuery(Clients, Expected, Index, StartValue, EndValue) -> - assertRangeQuery(Clients, Expected, Index, StartValue, EndValue, undefined). - -assertRangeQuery(Clients, Expected, Index, StartValue, EndValue, Re) -> - assertRangeQuery(Clients, Expected, Index, StartValue, EndValue, Re, {false, false}), - assertRangeQuery(Clients, Expected, Index, StartValue, EndValue, Re, {true, true}). - -assertRangeQuery(Clients, Expected, Index, StartValue, EndValue, Re, Sort) when is_list(Clients) -> - [assertRangeQuery(C, Expected, Index, StartValue, EndValue, Re, Sort) || C <- Clients]; -assertRangeQuery({ClientType, Client}, Expected, Index, StartValue, EndValue, Re, + ?assertEqual(Expected, ActualKeys). + +assertRangeQuery(Clients, Bucket, Expected, Index, StartValue, EndValue) -> + assertRangeQuery(Clients, Bucket, Expected, Index, StartValue, EndValue, undefined). + +assertRangeQuery(Clients, Bucket, Expected, Index, StartValue, EndValue, Re) -> + assertRangeQuery(Clients, Bucket, Expected, Index, StartValue, EndValue, Re, {false, false}), + assertRangeQuery(Clients, Bucket, Expected, Index, StartValue, EndValue, Re, {true, true}). + +assertRangeQuery(Clients, Bucket, Expected, Index, StartValue, EndValue, Re, Sort) when is_list(Clients) -> + [assertRangeQuery(C, Bucket, Expected, Index, StartValue, EndValue, Re, Sort) || C <- Clients]; +assertRangeQuery({ClientType, Client}, Bucket, Expected, Index, StartValue, EndValue, Re, {Sort, ExpectSorted}) -> lager:info("Searching Index ~p for ~p-~p re:~p, sort: ~p, ~p with ~p client", [Index, StartValue, EndValue, Re, Sort, ExpectSorted, ClientType]), {ok, ?INDEX_RESULTS{keys=Results}} = case ClientType of pb -> - riakc_pb_socket:get_index_range(Client, ?BUCKET, Index, StartValue, EndValue, + riakc_pb_socket:get_index_range(Client, Bucket, Index, StartValue, EndValue, [{term_regex, Re} || Re /= undefined] ++ [{pagination_sort, Sort} || Sort /= undefined]); http -> - rhc:get_index(Client, ?BUCKET, Index, {StartValue, EndValue}, + rhc:get_index(Client, Bucket, Index, {StartValue, EndValue}, [{term_regex, Re} || Re /= undefined] ++ [{pagination_sort, Sort}]) end, @@ -168,7 +175,7 @@ assertRangeQuery({ClientType, Client}, Expected, Index, StartValue, EndValue, Re ?assertEqual(Expected, ActualKeys). %% general 2i utility -put_an_object(Pid, N) -> +put_an_object(Pid, Bucket, N) -> Key = int_to_key(N), Data = io_lib:format("data~p", [N]), BinIndex = int_to_field1_bin(N), @@ -177,17 +184,17 @@ put_an_object(Pid, N) -> % every 5 items indexed together {"field3_int", N - (N rem 5)} ], - put_an_object(Pid, Key, Data, Indexes). + put_an_object(Pid, Bucket, Key, Data, Indexes). -put_an_object(Pid, Key, Data, Indexes) when is_list(Indexes) -> +put_an_object(Pid, Bucket, Key, Data, Indexes) when is_list(Indexes) -> lager:info("Putting object ~p", [Key]), MetaData = dict:from_list([{<<"index">>, Indexes}]), - Robj0 = riakc_obj:new(?BUCKET, Key), + Robj0 = riakc_obj:new(Bucket, Key), Robj1 = riakc_obj:update_value(Robj0, Data), Robj2 = riakc_obj:update_metadata(Robj1, MetaData), riakc_pb_socket:put(Pid, Robj2); -put_an_object(Pid, Key, IntIndex, BinIndex) when is_integer(IntIndex), is_binary(BinIndex) -> - put_an_object(Pid, Key, Key, [{"field1_bin", BinIndex},{"field2_int", IntIndex}]). +put_an_object(Pid, Bucket, Key, IntIndex, BinIndex) when is_integer(IntIndex), is_binary(BinIndex) -> + put_an_object(Pid, Bucket, Key, Key, [{"field1_bin", BinIndex},{"field2_int", IntIndex}]). int_to_key(N) -> case N < 100 of @@ -200,12 +207,12 @@ int_to_key(N) -> int_to_field1_bin(N) -> list_to_binary(io_lib:format("val~p", [N])). -stream_pb(Pid, Q) -> - pb_query(Pid, Q, [stream]), +stream_pb(Pid, Bucket, Q) -> + pb_query(Pid, Bucket, Q, [stream]), stream_loop(). -stream_pb(Pid, Q, Opts) -> - pb_query(Pid, Q, [stream|Opts]), +stream_pb(Pid, Bucket, Q, Opts) -> + pb_query(Pid, Bucket, Q, [stream|Opts]), stream_loop(). stream_loop() -> @@ -230,29 +237,29 @@ stream_loop(Acc) -> stream_loop(Acc) end. -pb_query(Pid, {Field, Val}, Opts) -> - riakc_pb_socket:get_index_eq(Pid, ?BUCKET, Field, Val, Opts); -pb_query(Pid, {Field, Start, End}, Opts) -> - riakc_pb_socket:get_index_range(Pid, ?BUCKET, Field, Start, End, Opts). +pb_query(Pid, Bucket, {Field, Val}, Opts) -> + riakc_pb_socket:get_index_eq(Pid, Bucket, Field, Val, Opts); +pb_query(Pid, Bucket, {Field, Start, End}, Opts) -> + riakc_pb_socket:get_index_range(Pid, Bucket, Field, Start, End, Opts). -http_stream(NodePath, Query, Opts) -> - http_query(NodePath, Query, [{stream, true} | Opts], stream). +http_stream(NodePath, Bucket, Query, Opts) -> + http_query(NodePath, Bucket, Query, [{stream, true} | Opts], stream). -http_query(NodePath, Q) -> - http_query(NodePath, Q, []). +http_query(NodePath, Bucket, Q) -> + http_query(NodePath, Bucket, Q, []). -http_query(NodePath, Query, Opts) -> - http_query(NodePath, Query, Opts, undefined). +http_query(NodePath, Bucket, Query, Opts) -> + http_query(NodePath, Bucket, Query, Opts, undefined). -http_query(NodePath, {Field, Value}, Opts, Pid) -> +http_query(NodePath, Bucket, {Field, Value}, Opts, Pid) -> QString = opts_to_qstring(Opts, []), Flag = case is_integer(Value) of true -> "w"; false -> "s" end, - Url = url("~s/buckets/~s/index/~s/~"++Flag++"~s", [NodePath, ?BUCKET, Field, Value, QString]), + Url = url("~s/buckets/~s/index/~s/~"++Flag++"~s", [NodePath, Bucket, Field, Value, QString]), http_get(Url, Pid); -http_query(NodePath, {Field, Start, End}, Opts, Pid) -> +http_query(NodePath, Bucket, {Field, Start, End}, Opts, Pid) -> QString = opts_to_qstring(Opts, []), Flag = case is_integer(Start) of true -> "w"; false -> "s" end, - Url = url("~s/buckets/~s/index/~s/~"++Flag++"/~"++Flag++"~s", [NodePath, ?BUCKET, Field, Start, End, QString]), + Url = url("~s/buckets/~s/index/~s/~"++Flag++"/~"++Flag++"~s", [NodePath, Bucket, Field, Start, End, QString]), http_get(Url, Pid). url(Format, Elements) -> @@ -300,7 +307,7 @@ start_http_stream(Ref) -> Other -> lager:error("Unexpected message ~p", [Other]), {error, unknown_message} after 60000 -> - {error, timeout_local} + {error, timeout_local} end. http_stream_loop(Ref, Acc, {Boundary, BLen}=B) -> @@ -329,4 +336,3 @@ get_boundary("multipart/mixed;boundary=" ++ Boundary) -> {B, byte_size(B)}; get_boundary(_) -> undefined. - diff --git a/tests/verify_build_cluster.erl b/tests/verify_build_cluster.erl index e934eb434..a5f2e5abd 100644 --- a/tests/verify_build_cluster.erl +++ b/tests/verify_build_cluster.erl @@ -18,26 +18,24 @@ %% %% ------------------------------------------------------------------- -module(verify_build_cluster). --behavior(riak_test). --export([confirm/0]). +-export([properties/0, + confirm/2]). +-include("rt.hrl"). -include_lib("eunit/include/eunit.hrl"). --import(rt, [wait_until_nodes_ready/1, - wait_until_no_pending_changes/1]). - -confirm() -> - %% test requires allow_mult=false b/c of rt:systest_read - rt:set_conf(all, [{"buckets.default.allow_mult", "false"}]), - %% Deploy a set of new nodes - lager:info("Deploying 4 nodes"), - %% handoff_concurrency needs to be raised to make the leave operation faster. - %% most clusters go up to 10, but this one is one louder, isn't it? - [Node1, Node2, Node3, Node4] = Nodes = rt:deploy_nodes(4, [{riak_core, [{handoff_concurrency, 11}]}]), - - %% Ensure each node owns 100% of it's own ring - lager:info("Ensure each nodes 100% of it's own ring"), - - [rt:wait_until_owners_according_to(Node, [Node]) || Node <- Nodes], +properties() -> + DefaultProps = rt_cluster:properties(), + UpdConfig = rt_cluster:augment_config(riak_core, + {default_bucket_props, [{allow_mult, false}]}, + DefaultProps#rt_properties.config), + DefaultProps#rt_properties{config=UpdConfig, + node_count=4, + rolling_upgrade=true, + make_cluster=false, + start_version=previous}. + +confirm(#rt_properties{nodes=Nodes}, _MD) -> + [Node1, Node2, Node3, Node4] = Nodes, lager:info("Loading some data up in this cluster."), ?assertEqual([], rt:systest_write(Node1, 0, 1000, <<"verify_build_cluster">>, 2)), From 8348835389e13c153b7a91d738f6bcdd6fc71e47 Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Mon, 28 Jul 2014 13:38:07 -0400 Subject: [PATCH 002/157] convert bucket_types test to decoupled r_t framework --- tests/bucket_types.erl | 137 +++++++++-------------------------------- 1 file changed, 28 insertions(+), 109 deletions(-) diff --git a/tests/bucket_types.erl b/tests/bucket_types.erl index 1715cee82..c08538188 100644 --- a/tests/bucket_types.erl +++ b/tests/bucket_types.erl @@ -1,17 +1,23 @@ -module(bucket_types). --behavior(riak_test). --export([confirm/0, mapred_modfun/3, mapred_modfun_type/3]). +-export([properties/0,confirm/2, mapred_modfun/3, mapred_modfun_type/3]). -include_lib("eunit/include/eunit.hrl"). - -confirm() -> - application:start(inets), - lager:info("Deploy some nodes"), - Nodes = rt:build_cluster(4, [], [ - {riak_core, [{default_bucket_props, - [{n_val, 2}]}]}]), +-include("rt.hrl"). + +properties() -> + DefaultProps = rt_cluster:properties(), + CustomConfig = rt_cluster:augment_config(riak_core, + {default_bucket_props, [{n_val, 2}]}, + DefaultProps#rt_properties.config), + DefaultProps#rt_properties{node_count=4, + rolling_upgrade=false, + make_cluster=true, + config=CustomConfig}. + +confirm(#rt_properties{nodes=Nodes}, _MD) -> Node = hd(Nodes), + application:start(inets), RMD = riak_test_runner:metadata(), HaveIndexes = case proplists:get_value(backend, RMD) of @@ -20,7 +26,10 @@ confirm() -> _ -> true end, - PB = rt:pbc(Node), + {ok, [{"127.0.0.1", Port}]} = rpc:call(Node, application, get_env, + [riak_api, pb]), + + {ok, PB} = riakc_pb_socket:start_link("127.0.0.1", Port, []), lager:info("default type get/put test"), %% write explicitly to the default type @@ -42,7 +51,7 @@ confirm() -> %% write implicitly to the default bucket riakc_pb_socket:put(PB, riakc_obj:update_value(O1, <<"newvalue">>)), - + %% read from the default bucket explicitly {ok, O3} = riakc_pb_socket:get(PB, {<<"default">>, <<"bucket">>}, <<"key">>), @@ -81,13 +90,9 @@ confirm() -> %% read from the default bucket explicitly {error, notfound} = riakc_pb_socket:get(PB, {<<"default">>, <<"bucket">>}, <<"key">>), - ?assertEqual(ok, rt:wait_until(fun() -> - rt:pbc_really_deleted(PB, - <<"bucket">>, - [<<"key">>]) - end)), + timer:sleep(5000), %% wait for delete_mode 3s to expire - %% now there should be no buckets or keys to be listed... + %% now there shoyld be no buckets or keys to be listed... %% %% list keys ?assertEqual({ok, []}, riakc_pb_socket:list_keys(PB, <<"bucket">>)), @@ -100,20 +105,13 @@ confirm() -> lager:info("custom type get/put test"), Type = <<"mytype">>, - TypeProps = [{n_val, 3}], - lager:info("Create bucket type ~p, wait for propagation", [Type]), - rt:create_and_activate_bucket_type(Node, Type, TypeProps), + rt:create_and_activate_bucket_type(Node, Type, [{n_val, 3}]), rt:wait_until_bucket_type_status(Type, active, Nodes), - rt:wait_until_bucket_props(Nodes, {Type, <<"bucket">>}, TypeProps), lager:info("doing put"), riakc_pb_socket:put(PB, riakc_obj:new({Type, <<"bucket">>}, <<"key">>, <<"newestvalue">>)), - lager:info("custom type list_keys test"), - ?assertEqual({ok, []}, riakc_pb_socket:list_keys(PB, <<"bucket">>)), - ?assertEqual({ok, [<<"key">>]}, riakc_pb_socket:list_keys(PB, {Type, - <<"bucket">>})), lager:info("doing get"), {ok, O5} = riakc_pb_socket:get(PB, {Type, <<"bucket">>}, <<"key">>), @@ -123,42 +121,15 @@ confirm() -> %% this type is NOT aliased to the default buckey {error, notfound} = riakc_pb_socket:get(PB, <<"bucket">>, <<"key">>), + lager:info("custom type list_keys test"), + ?assertEqual({ok, []}, riakc_pb_socket:list_keys(PB, <<"bucket">>)), + ?assertEqual({ok, [<<"key">>]}, riakc_pb_socket:list_keys(PB, {Type, + <<"bucket">>})), lager:info("custom type list_buckets test"), %% list buckets ?assertEqual({ok, []}, riakc_pb_socket:list_buckets(PB)), ?assertEqual({ok, [<<"bucket">>]}, riakc_pb_socket:list_buckets(PB, Type)), - %%% Beginning of UTF-8 test - - lager:info("UTF-8 type get/put test"), - %% こんにちは - konnichiwa (Japanese) - UnicodeType = unicode:characters_to_binary([12371,12435,12395,12385,12399], utf8), - %% سلام - Salam (Arabic) - UnicodeBucket = unicode:characters_to_binary([1587,1604,1575,1605], utf8), - lager:info("Create bucket type, wait for propagation"), - rt:create_and_activate_bucket_type(Node, UnicodeType, TypeProps), - rt:wait_until_bucket_type_status(UnicodeType, active, Nodes), - rt:wait_until_bucket_props(Nodes, {UnicodeType, UnicodeBucket}, TypeProps), - - lager:info("doing put"), - riakc_pb_socket:put(PB, riakc_obj:new({UnicodeType, UnicodeBucket}, - <<"key">>, <<"yetanothervalue">>)), - - lager:info("custom type list_keys test"), - ?assertEqual({ok, [<<"key">>]}, riakc_pb_socket:list_keys(PB, - {UnicodeType, - UnicodeBucket})), - lager:info("doing get"), - {ok, O6} = riakc_pb_socket:get(PB, {UnicodeType, UnicodeBucket}, <<"key">>), - - ?assertEqual(<<"yetanothervalue">>, riakc_obj:get_value(O6)), - - lager:info("custom type list_buckets test"), - %% list buckets - ?assertEqual({ok, [UnicodeBucket]}, riakc_pb_socket:list_buckets(PB, UnicodeType)), - - %%% End of UTF-8 test - lager:info("bucket properties tests"), riakc_pb_socket:set_bucket(PB, {<<"default">>, <<"mybucket">>}, [{n_val, 5}]), @@ -180,38 +151,12 @@ confirm() -> <<"mybucket">>}), ?assertEqual(5, proplists:get_value(n_val, BProps3)), - %% Check our unicode brethren - riakc_pb_socket:set_bucket(PB, {UnicodeType, UnicodeBucket}, - [{n_val, 4}]), - {ok, UBProps1} = riakc_pb_socket:get_bucket(PB, {UnicodeType, - UnicodeBucket}), - ?assertEqual(4, proplists:get_value(n_val, UBProps1)), - riakc_pb_socket:reset_bucket(PB, {Type, <<"mybucket">>}), {ok, BProps4} = riakc_pb_socket:get_bucket(PB, {Type, <<"mybucket">>}), ?assertEqual(3, proplists:get_value(n_val, BProps4)), - riakc_pb_socket:reset_bucket(PB, {UnicodeType, UnicodeBucket}), - - {ok, UBProps2} = riakc_pb_socket:get_bucket(PB, {UnicodeType, - UnicodeBucket}), - - ?assertEqual(3, proplists:get_value(n_val, UBProps2)), - - {error, NTGR} = riakc_pb_socket:get_bucket(PB, {<<"nonexistent">>, <<"mybucket">>}), - - lager:info("GOT ERROR ~s", [NTGR]), - - ?assertMatch(<<"No bucket-type named 'nonexistent'", _/binary>>, NTGR), - - {error, NTSR} = riakc_pb_socket:set_bucket(PB, {<<"nonexistent">>, <<"mybucket">>}, [{n_val, 3}]), - - lager:info("GOT ERROR ~s", [NTSR]), - - ?assertMatch(<<"No bucket-type named 'nonexistent'", _/binary>>, NTSR), - lager:info("bucket type properties test"), riakc_pb_socket:set_bucket_type(PB, Type, @@ -232,27 +177,6 @@ confirm() -> ?assertEqual(3, proplists:get_value(n_val, BProps7)), - %% Repeat type checks for unicode type/bucket - - riakc_pb_socket:set_bucket_type(PB, UnicodeType, - [{n_val, 5}]), - - {ok, UBProps3} = riakc_pb_socket:get_bucket_type(PB, UnicodeType), - - ?assertEqual(5, proplists:get_value(n_val, UBProps3)), - - %% check that the bucket inherits from its type - {ok, UBProps4} = riakc_pb_socket:get_bucket(PB, {UnicodeType, - UnicodeBucket}), - - ?assertEqual(5, proplists:get_value(n_val, UBProps4)), - - riakc_pb_socket:set_bucket_type(PB, UnicodeType, [{n_val, 3}]), - - {ok, UBProps5} = riakc_pb_socket:get_bucket_type(PB, UnicodeType), - - ?assertEqual(3, proplists:get_value(n_val, UBProps5)), - %% make sure a regular bucket under the default type reflects app.config {ok, BProps8} = riakc_pb_socket:get_bucket(PB, {<<"default">>, <<"mybucket">>}), @@ -363,11 +287,6 @@ confirm() -> {<<"baz">>, <<"4">>, <<"a">>, 4}, {<<"bam">>, <<"5">>, <<"a">>, 3}]], - ?assertEqual({ok, [{0, [<<"2">>]}]}, - riakc_pb_socket:mapred(PB, {{Type, <<"MRbucket">>}, - [[<<"starts_with">>, <<"f">>]]}, - [{map, {modfun, riak_kv_mapreduce, map_object_value}, none, true}])), - ?assertEqual({ok, [{1, [14]}]}, riakc_pb_socket:mapred_bucket(PB, {Type, <<"MRbucket">>}, [{map, {jsfun, <<"Riak.mapValuesJson">>}, undefined, false}, @@ -487,7 +406,7 @@ confirm() -> undefined, true}])), riakc_pb_socket:stop(PB), - pass. + ok. accumulate(ReqID) -> receive From e69f962d9612ac90caa08bf876d995bb4a0af8f0 Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Mon, 28 Jul 2014 15:18:11 -0400 Subject: [PATCH 003/157] convert basic_command_line test to decoupled r_t framework --- tests/basic_command_line.erl | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/tests/basic_command_line.erl b/tests/basic_command_line.erl index 958a5455e..88afad49d 100644 --- a/tests/basic_command_line.erl +++ b/tests/basic_command_line.erl @@ -19,17 +19,20 @@ %% ------------------------------------------------------------------- -module(basic_command_line). -include_lib("eunit/include/eunit.hrl"). +-include("rt.hrl"). --behavior(riak_test). --compile(export_all). --export([confirm/0]). +-export([properties/0, confirm/2]). -confirm() -> +properties() -> + DefaultProps = rt_cluster:properties(), + DefaultProps#rt_properties{node_count=1, + rolling_upgrade=false, + make_cluster=true}. +confirm(#rt_properties{nodes=Nodes}, _MD) -> + Node = hd(Nodes), %% Deploy a node to test against lager:info("Deploy node to test command line"), - [Node] = rt:deploy_nodes(1), - ?assertEqual(ok, rt:wait_until_nodes_ready([Node])), %% Verify node-up behavior ping_up_test(Node), From 86b8c6fb3761364a0c16e4bc68a92832209f5204 Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Tue, 29 Jul 2014 08:12:57 -0400 Subject: [PATCH 004/157] convert bucket_types test to decoupled r_t framework --- tests/cluster_meta_basic.erl | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/tests/cluster_meta_basic.erl b/tests/cluster_meta_basic.erl index b659ec107..188763ad6 100644 --- a/tests/cluster_meta_basic.erl +++ b/tests/cluster_meta_basic.erl @@ -18,10 +18,11 @@ %% %% ------------------------------------------------------------------- -module(cluster_meta_basic). --behavior(riak_test). --export([confirm/0, object_count/2]). +-export([properties/0, confirm/2, object_count/2]). + -include_lib("eunit/include/eunit.hrl"). +-include("rt.hrl"). -define(PREFIX1, {a, b}). -define(PREFIX2, {fold, prefix}). -define(KEY1, key1). @@ -29,8 +30,13 @@ -define(VAL1, val1). -define(VAL2, val2). -confirm() -> - Nodes = rt:build_cluster(5), +properties() -> + DefaultProps = rt_cluster:properties(), + DefaultProps#rt_properties{node_count=5, + rolling_upgrade=false, + make_cluster=true}. + +confirm(#rt_properties{nodes=Nodes}, _MD) -> ok = test_fold_full_prefix(Nodes), ok = test_metadata_conflicts(Nodes), ok = test_writes_after_partial_cluster_failure(Nodes), From 5fe4c03fe3bb2df750c096ab54fa2fdfec1d93e0 Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Tue, 29 Jul 2014 08:20:37 -0400 Subject: [PATCH 005/157] convert bucket_props_roundtrip test to decoupled r_t framework --- tests/bucket_props_roundtrip.erl | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/tests/bucket_props_roundtrip.erl b/tests/bucket_props_roundtrip.erl index ef2fe5c84..5febb6c80 100644 --- a/tests/bucket_props_roundtrip.erl +++ b/tests/bucket_props_roundtrip.erl @@ -18,9 +18,10 @@ %% %% ------------------------------------------------------------------- -module(bucket_props_roundtrip). --behaviour(riak_test). --export([confirm/0]). +-export([properties/0, confirm/2]). + -include_lib("eunit/include/eunit.hrl"). +-include("rt.hrl"). -define(BUCKET, <<"pbc_props_verify">>). -define(COMMIT_HOOK, {struct, [{<<"mod">>, <<"foo">>}, {<<"fun">>, <<"bar">>}]}). @@ -52,9 +53,14 @@ {young_vclock, 0, 20} ]). -confirm() -> - [Node] = Nodes = rt:build_cluster(1), - ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), +properties() -> + DefaultProps = rt_cluster:properties(), + DefaultProps#rt_properties{node_count=1, + rolling_upgrade=false, + make_cluster=true}. + +confirm(#rt_properties{nodes=Nodes}, _MD) -> + [Node] = Nodes, [ check_prop_set_and_get(Node, Prop, FirstVal, SecondVal) || {Prop, FirstVal, SecondVal} <- ?PROPS ], From 381ff9f3525e3773d59106aeaaebfb2f4490894a Mon Sep 17 00:00:00 2001 From: Kelly McLaughlin Date: Tue, 29 Jul 2014 10:18:14 -0600 Subject: [PATCH 006/157] Move replication tests to separate subdirectory --- tests/{ => replication}/repl_aae_fullsync.erl | 0 tests/{ => replication}/repl_aae_fullsync_bench.erl | 0 tests/{ => replication}/repl_aae_fullsync_custom_n.erl | 0 tests/{ => replication}/repl_aae_fullsync_util.erl | 0 tests/{ => replication}/repl_bucket_types.erl | 0 tests/{ => replication}/repl_cancel_fullsync.erl | 0 tests/{ => replication}/repl_consistent_object_filter.erl | 0 tests/{ => replication}/repl_fs_bench.erl | 0 tests/{ => replication}/repl_fs_stat_caching.erl | 0 tests/{ => replication}/repl_location_failures.erl | 0 tests/{ => replication}/repl_rt_cascading_rtq.erl | 0 tests/{ => replication}/repl_rt_heartbeat.erl | 0 tests/{ => replication}/repl_rt_overload.erl | 0 tests/{ => replication}/repl_rt_pending.erl | 0 tests/{ => replication}/repl_util.erl | 0 tests/{ => replication}/replication.erl | 0 tests/{ => replication}/replication2.erl | 0 tests/{ => replication}/replication2_connections.erl | 0 tests/{ => replication}/replication2_console_tests.erl | 0 tests/{ => replication}/replication2_dirty.erl | 0 tests/{ => replication}/replication2_fsschedule.erl | 0 tests/{ => replication}/replication2_pg.erl | 0 tests/{ => replication}/replication2_rt_sink_connection.erl | 0 tests/{ => replication}/replication2_ssl.erl | 0 tests/{ => replication}/replication2_upgrade.erl | 0 tests/{ => replication}/replication_object_reformat.erl | 0 tests/{ => replication}/replication_ssl.erl | 0 tests/{ => replication}/replication_stats.erl | 0 tests/{ => replication}/replication_upgrade.erl | 0 tests/{ => replication}/rt_cascading.erl | 0 30 files changed, 0 insertions(+), 0 deletions(-) rename tests/{ => replication}/repl_aae_fullsync.erl (100%) rename tests/{ => replication}/repl_aae_fullsync_bench.erl (100%) rename tests/{ => replication}/repl_aae_fullsync_custom_n.erl (100%) rename tests/{ => replication}/repl_aae_fullsync_util.erl (100%) rename tests/{ => replication}/repl_bucket_types.erl (100%) rename tests/{ => replication}/repl_cancel_fullsync.erl (100%) rename tests/{ => replication}/repl_consistent_object_filter.erl (100%) rename tests/{ => replication}/repl_fs_bench.erl (100%) rename tests/{ => replication}/repl_fs_stat_caching.erl (100%) rename tests/{ => replication}/repl_location_failures.erl (100%) rename tests/{ => replication}/repl_rt_cascading_rtq.erl (100%) rename tests/{ => replication}/repl_rt_heartbeat.erl (100%) rename tests/{ => replication}/repl_rt_overload.erl (100%) rename tests/{ => replication}/repl_rt_pending.erl (100%) rename tests/{ => replication}/repl_util.erl (100%) rename tests/{ => replication}/replication.erl (100%) rename tests/{ => replication}/replication2.erl (100%) rename tests/{ => replication}/replication2_connections.erl (100%) rename tests/{ => replication}/replication2_console_tests.erl (100%) rename tests/{ => replication}/replication2_dirty.erl (100%) rename tests/{ => replication}/replication2_fsschedule.erl (100%) rename tests/{ => replication}/replication2_pg.erl (100%) rename tests/{ => replication}/replication2_rt_sink_connection.erl (100%) rename tests/{ => replication}/replication2_ssl.erl (100%) rename tests/{ => replication}/replication2_upgrade.erl (100%) rename tests/{ => replication}/replication_object_reformat.erl (100%) rename tests/{ => replication}/replication_ssl.erl (100%) rename tests/{ => replication}/replication_stats.erl (100%) rename tests/{ => replication}/replication_upgrade.erl (100%) rename tests/{ => replication}/rt_cascading.erl (100%) diff --git a/tests/repl_aae_fullsync.erl b/tests/replication/repl_aae_fullsync.erl similarity index 100% rename from tests/repl_aae_fullsync.erl rename to tests/replication/repl_aae_fullsync.erl diff --git a/tests/repl_aae_fullsync_bench.erl b/tests/replication/repl_aae_fullsync_bench.erl similarity index 100% rename from tests/repl_aae_fullsync_bench.erl rename to tests/replication/repl_aae_fullsync_bench.erl diff --git a/tests/repl_aae_fullsync_custom_n.erl b/tests/replication/repl_aae_fullsync_custom_n.erl similarity index 100% rename from tests/repl_aae_fullsync_custom_n.erl rename to tests/replication/repl_aae_fullsync_custom_n.erl diff --git a/tests/repl_aae_fullsync_util.erl b/tests/replication/repl_aae_fullsync_util.erl similarity index 100% rename from tests/repl_aae_fullsync_util.erl rename to tests/replication/repl_aae_fullsync_util.erl diff --git a/tests/repl_bucket_types.erl b/tests/replication/repl_bucket_types.erl similarity index 100% rename from tests/repl_bucket_types.erl rename to tests/replication/repl_bucket_types.erl diff --git a/tests/repl_cancel_fullsync.erl b/tests/replication/repl_cancel_fullsync.erl similarity index 100% rename from tests/repl_cancel_fullsync.erl rename to tests/replication/repl_cancel_fullsync.erl diff --git a/tests/repl_consistent_object_filter.erl b/tests/replication/repl_consistent_object_filter.erl similarity index 100% rename from tests/repl_consistent_object_filter.erl rename to tests/replication/repl_consistent_object_filter.erl diff --git a/tests/repl_fs_bench.erl b/tests/replication/repl_fs_bench.erl similarity index 100% rename from tests/repl_fs_bench.erl rename to tests/replication/repl_fs_bench.erl diff --git a/tests/repl_fs_stat_caching.erl b/tests/replication/repl_fs_stat_caching.erl similarity index 100% rename from tests/repl_fs_stat_caching.erl rename to tests/replication/repl_fs_stat_caching.erl diff --git a/tests/repl_location_failures.erl b/tests/replication/repl_location_failures.erl similarity index 100% rename from tests/repl_location_failures.erl rename to tests/replication/repl_location_failures.erl diff --git a/tests/repl_rt_cascading_rtq.erl b/tests/replication/repl_rt_cascading_rtq.erl similarity index 100% rename from tests/repl_rt_cascading_rtq.erl rename to tests/replication/repl_rt_cascading_rtq.erl diff --git a/tests/repl_rt_heartbeat.erl b/tests/replication/repl_rt_heartbeat.erl similarity index 100% rename from tests/repl_rt_heartbeat.erl rename to tests/replication/repl_rt_heartbeat.erl diff --git a/tests/repl_rt_overload.erl b/tests/replication/repl_rt_overload.erl similarity index 100% rename from tests/repl_rt_overload.erl rename to tests/replication/repl_rt_overload.erl diff --git a/tests/repl_rt_pending.erl b/tests/replication/repl_rt_pending.erl similarity index 100% rename from tests/repl_rt_pending.erl rename to tests/replication/repl_rt_pending.erl diff --git a/tests/repl_util.erl b/tests/replication/repl_util.erl similarity index 100% rename from tests/repl_util.erl rename to tests/replication/repl_util.erl diff --git a/tests/replication.erl b/tests/replication/replication.erl similarity index 100% rename from tests/replication.erl rename to tests/replication/replication.erl diff --git a/tests/replication2.erl b/tests/replication/replication2.erl similarity index 100% rename from tests/replication2.erl rename to tests/replication/replication2.erl diff --git a/tests/replication2_connections.erl b/tests/replication/replication2_connections.erl similarity index 100% rename from tests/replication2_connections.erl rename to tests/replication/replication2_connections.erl diff --git a/tests/replication2_console_tests.erl b/tests/replication/replication2_console_tests.erl similarity index 100% rename from tests/replication2_console_tests.erl rename to tests/replication/replication2_console_tests.erl diff --git a/tests/replication2_dirty.erl b/tests/replication/replication2_dirty.erl similarity index 100% rename from tests/replication2_dirty.erl rename to tests/replication/replication2_dirty.erl diff --git a/tests/replication2_fsschedule.erl b/tests/replication/replication2_fsschedule.erl similarity index 100% rename from tests/replication2_fsschedule.erl rename to tests/replication/replication2_fsschedule.erl diff --git a/tests/replication2_pg.erl b/tests/replication/replication2_pg.erl similarity index 100% rename from tests/replication2_pg.erl rename to tests/replication/replication2_pg.erl diff --git a/tests/replication2_rt_sink_connection.erl b/tests/replication/replication2_rt_sink_connection.erl similarity index 100% rename from tests/replication2_rt_sink_connection.erl rename to tests/replication/replication2_rt_sink_connection.erl diff --git a/tests/replication2_ssl.erl b/tests/replication/replication2_ssl.erl similarity index 100% rename from tests/replication2_ssl.erl rename to tests/replication/replication2_ssl.erl diff --git a/tests/replication2_upgrade.erl b/tests/replication/replication2_upgrade.erl similarity index 100% rename from tests/replication2_upgrade.erl rename to tests/replication/replication2_upgrade.erl diff --git a/tests/replication_object_reformat.erl b/tests/replication/replication_object_reformat.erl similarity index 100% rename from tests/replication_object_reformat.erl rename to tests/replication/replication_object_reformat.erl diff --git a/tests/replication_ssl.erl b/tests/replication/replication_ssl.erl similarity index 100% rename from tests/replication_ssl.erl rename to tests/replication/replication_ssl.erl diff --git a/tests/replication_stats.erl b/tests/replication/replication_stats.erl similarity index 100% rename from tests/replication_stats.erl rename to tests/replication/replication_stats.erl diff --git a/tests/replication_upgrade.erl b/tests/replication/replication_upgrade.erl similarity index 100% rename from tests/replication_upgrade.erl rename to tests/replication/replication_upgrade.erl diff --git a/tests/rt_cascading.erl b/tests/replication/rt_cascading.erl similarity index 100% rename from tests/rt_cascading.erl rename to tests/replication/rt_cascading.erl From d15efa049289fcf070dc47b33383a8d61ccd0ae0 Mon Sep 17 00:00:00 2001 From: Kelly McLaughlin Date: Tue, 29 Jul 2014 10:35:42 -0600 Subject: [PATCH 007/157] Move rebar plugin to separate directory --- {src => .plugins}/rebar_riak_test_plugin.erl | 0 rebar.config | 2 +- 2 files changed, 1 insertion(+), 1 deletion(-) rename {src => .plugins}/rebar_riak_test_plugin.erl (100%) diff --git a/src/rebar_riak_test_plugin.erl b/.plugins/rebar_riak_test_plugin.erl similarity index 100% rename from src/rebar_riak_test_plugin.erl rename to .plugins/rebar_riak_test_plugin.erl diff --git a/rebar.config b/rebar.config index 2ecbeec34..adbd1fb66 100644 --- a/rebar.config +++ b/rebar.config @@ -23,7 +23,7 @@ {escript_incl_apps, [goldrush, lager, getopt, riakhttpc, riakc, ibrowse, mochiweb, kvc]}. {escript_emu_args, "%%! -escript main riak_test_escript +K true +P 10000 -env ERL_MAX_PORTS 10000\n"}. -{plugin_dir, "src"}. +{plugin_dir, ".plugins"}. {plugins, [rebar_riak_test_plugin]}. {riak_test, [ {test_paths, ["tests", "perf"]}, From 333e67d806da806e1145fa0482e591aeab69e175 Mon Sep 17 00:00:00 2001 From: Kelly McLaughlin Date: Tue, 29 Jul 2014 10:41:20 -0600 Subject: [PATCH 008/157] Add missing node function to rt.erl --- src/rt.erl | 1822 ++++++++++++++++++++++++++-------------------------- 1 file changed, 911 insertions(+), 911 deletions(-) diff --git a/src/rt.erl b/src/rt.erl index e9eebeef7..31b5e59b3 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -171,26 +171,26 @@ -define(HARNESS, (rt_config:get(rt_harness))). priv_dir() -> - LocalPrivDir = "./priv", - %% XXX for some reason, codew:priv_dir returns riak_test/riak_test/priv, - %% which is wrong, so fix it. - DepPrivDir = re:replace(code:priv_dir(riak_test), "riak_test(/riak_test)*", - "riak_test", [{return, list}]), - PrivDir = case {filelib:is_dir(LocalPrivDir), filelib:is_dir(DepPrivDir)} of - {true, _} -> - lager:debug("Local ./priv detected, using that..."), - %% we want an absolute path! - filename:absname(LocalPrivDir); - {false, true} -> - lager:debug("riak_test dependency priv_dir detected, using that..."), - DepPrivDir; - _ -> - ?assertEqual({true, bad_priv_dir}, {false, bad_priv_dir}) - end, - - lager:info("priv dir: ~p -> ~p", [code:priv_dir(riak_test), PrivDir]), - ?assert(filelib:is_dir(PrivDir)), - PrivDir. +LocalPrivDir = "./priv", +%% XXX for some reason, codew:priv_dir returns riak_test/riak_test/priv, +%% which is wrong, so fix it. +DepPrivDir = re:replace(code:priv_dir(riak_test), "riak_test(/riak_test)*", +"riak_test", [{return, list}]), +PrivDir = case {filelib:is_dir(LocalPrivDir), filelib:is_dir(DepPrivDir)} of +{true, _} -> + lager:debug("Local ./priv detected, using that..."), + %% we want an absolute path! + filename:absname(LocalPrivDir); +{false, true} -> + lager:debug("riak_test dependency priv_dir detected, using that..."), + DepPrivDir; +_ -> + ?assertEqual({true, bad_priv_dir}, {false, bad_priv_dir}) +end, + +lager:info("priv dir: ~p -> ~p", [code:priv_dir(riak_test), PrivDir]), +?assert(filelib:is_dir(PrivDir)), +PrivDir. %% @doc gets riak deps from the appropriate harness -spec get_deps() -> list(). @@ -199,51 +199,51 @@ get_deps() -> ?HARNESS:get_deps(). %% @doc if String contains Substr, return true. -spec str(string(), string()) -> boolean(). str(String, Substr) -> - case string:str(String, Substr) of - 0 -> false; - _ -> true - end. +case string:str(String, Substr) of +0 -> false; +_ -> true +end. -spec set_conf(atom(), [{string(), string()}]) -> ok. set_conf(all, NameValuePairs) -> - ?HARNESS:set_conf(all, NameValuePairs); +?HARNESS:set_conf(all, NameValuePairs); set_conf(Node, NameValuePairs) -> - stop(Node), - ?assertEqual(ok, rt:wait_until_unpingable(Node)), - ?HARNESS:set_conf(Node, NameValuePairs), - start(Node). +stop(Node), +?assertEqual(ok, rt:wait_until_unpingable(Node)), +?HARNESS:set_conf(Node, NameValuePairs), +start(Node). -spec set_advanced_conf(atom(), [{string(), string()}]) -> ok. set_advanced_conf(all, NameValuePairs) -> - ?HARNESS:set_advanced_conf(all, NameValuePairs); +?HARNESS:set_advanced_conf(all, NameValuePairs); set_advanced_conf(Node, NameValuePairs) -> - stop(Node), - ?assertEqual(ok, rt:wait_until_unpingable(Node)), - ?HARNESS:set_advanced_conf(Node, NameValuePairs), - start(Node). +stop(Node), +?assertEqual(ok, rt:wait_until_unpingable(Node)), +?HARNESS:set_advanced_conf(Node, NameValuePairs), +start(Node). %% @doc Rewrite the given node's app.config file, overriding the varialbes %% in the existing app.config with those in `Config'. update_app_config(all, Config) -> - ?HARNESS:update_app_config(all, Config); +?HARNESS:update_app_config(all, Config); update_app_config(Node, Config) -> - stop(Node), - ?assertEqual(ok, rt:wait_until_unpingable(Node)), - ?HARNESS:update_app_config(Node, Config), - start(Node). +stop(Node), +?assertEqual(ok, rt:wait_until_unpingable(Node)), +?HARNESS:update_app_config(Node, Config), +start(Node). %% @doc Helper that returns first successful application get_env result, %% used when different versions of Riak use different app vars for %% the same setting. rpc_get_env(_, []) -> - undefined; +undefined; rpc_get_env(Node, [{App,Var}|Others]) -> - case rpc:call(Node, application, get_env, [App, Var]) of - {ok, Value} -> - {ok, Value}; - _ -> - rpc_get_env(Node, Others) - end. +case rpc:call(Node, application, get_env, [App, Var]) of +{ok, Value} -> + {ok, Value}; +_ -> + rpc_get_env(Node, Others) +end. -type interface() :: {http, tuple()} | {pb, tuple()}. -type interfaces() :: [interface()]. @@ -251,325 +251,325 @@ rpc_get_env(Node, [{App,Var}|Others]) -> -spec connection_info(node() | [node()]) -> interfaces() | conn_info(). connection_info(Node) when is_atom(Node) -> - {ok, [{PB_IP, PB_Port}]} = get_pb_conn_info(Node), - {ok, [{HTTP_IP, HTTP_Port}]} = get_http_conn_info(Node), - case get_https_conn_info(Node) of - undefined -> - [{http, {HTTP_IP, HTTP_Port}}, {pb, {PB_IP, PB_Port}}]; - {ok, [{HTTPS_IP, HTTPS_Port}]} -> - [{http, {HTTP_IP, HTTP_Port}}, {https, {HTTPS_IP, HTTPS_Port}}, {pb, {PB_IP, PB_Port}}] - end; +{ok, [{PB_IP, PB_Port}]} = get_pb_conn_info(Node), +{ok, [{HTTP_IP, HTTP_Port}]} = get_http_conn_info(Node), +case get_https_conn_info(Node) of +undefined -> + [{http, {HTTP_IP, HTTP_Port}}, {pb, {PB_IP, PB_Port}}]; +{ok, [{HTTPS_IP, HTTPS_Port}]} -> + [{http, {HTTP_IP, HTTP_Port}}, {https, {HTTPS_IP, HTTPS_Port}}, {pb, {PB_IP, PB_Port}}] +end; connection_info(Nodes) when is_list(Nodes) -> - [ {Node, connection_info(Node)} || Node <- Nodes]. +[ {Node, connection_info(Node)} || Node <- Nodes]. -spec get_pb_conn_info(node()) -> [{inet:ip_address(), pos_integer()}]. get_pb_conn_info(Node) -> - case rpc_get_env(Node, [{riak_api, pb}, - {riak_api, pb_ip}, - {riak_kv, pb_ip}]) of - {ok, [{NewIP, NewPort}|_]} -> - {ok, [{NewIP, NewPort}]}; - {ok, PB_IP} -> - {ok, PB_Port} = rpc_get_env(Node, [{riak_api, pb_port}, - {riak_kv, pb_port}]), - {ok, [{PB_IP, PB_Port}]}; - _ -> - undefined - end. +case rpc_get_env(Node, [{riak_api, pb}, + {riak_api, pb_ip}, + {riak_kv, pb_ip}]) of +{ok, [{NewIP, NewPort}|_]} -> + {ok, [{NewIP, NewPort}]}; +{ok, PB_IP} -> + {ok, PB_Port} = rpc_get_env(Node, [{riak_api, pb_port}, + {riak_kv, pb_port}]), + {ok, [{PB_IP, PB_Port}]}; +_ -> + undefined +end. -spec get_http_conn_info(node()) -> [{inet:ip_address(), pos_integer()}]. get_http_conn_info(Node) -> - case rpc_get_env(Node, [{riak_api, http}, - {riak_core, http}]) of - {ok, [{IP, Port}|_]} -> - {ok, [{IP, Port}]}; - _ -> - undefined - end. +case rpc_get_env(Node, [{riak_api, http}, + {riak_core, http}]) of +{ok, [{IP, Port}|_]} -> + {ok, [{IP, Port}]}; +_ -> + undefined +end. -spec get_https_conn_info(node()) -> [{inet:ip_address(), pos_integer()}]. get_https_conn_info(Node) -> - case rpc_get_env(Node, [{riak_api, https}, - {riak_core, https}]) of - {ok, [{IP, Port}|_]} -> - {ok, [{IP, Port}]}; - _ -> - undefined - end. +case rpc_get_env(Node, [{riak_api, https}, + {riak_core, https}]) of +{ok, [{IP, Port}|_]} -> + {ok, [{IP, Port}]}; +_ -> + undefined +end. %% @doc Deploy a set of freshly installed Riak nodes, returning a list of the %% nodes deployed. %% @todo Re-add -spec after adding multi-version support deploy_nodes(Versions) when is_list(Versions) -> - deploy_nodes(Versions, [riak_kv]); +deploy_nodes(Versions, [riak_kv]); deploy_nodes(NumNodes) when is_integer(NumNodes) -> - deploy_nodes([ current || _ <- lists:seq(1, NumNodes)]). +deploy_nodes([ current || _ <- lists:seq(1, NumNodes)]). %% @doc Deploy a set of freshly installed Riak nodes with the given %% `InitialConfig', returning a list of the nodes deployed. -spec deploy_nodes(NumNodes :: integer(), any()) -> [node()]. deploy_nodes(NumNodes, InitialConfig) when is_integer(NumNodes) -> - deploy_nodes(NumNodes, InitialConfig, [riak_kv]); +deploy_nodes(NumNodes, InitialConfig, [riak_kv]); deploy_nodes(Versions, Services) -> - NodeConfig = [ version_to_config(Version) || Version <- Versions ], - Nodes = ?HARNESS:deploy_nodes(NodeConfig), - lager:info("Waiting for services ~p to start on ~p.", [Services, Nodes]), - [ ok = wait_for_service(Node, Service) || Node <- Nodes, - Service <- Services ], - Nodes. +NodeConfig = [ version_to_config(Version) || Version <- Versions ], +Nodes = ?HARNESS:deploy_nodes(NodeConfig), +lager:info("Waiting for services ~p to start on ~p.", [Services, Nodes]), +[ ok = wait_for_service(Node, Service) || Node <- Nodes, + Service <- Services ], +Nodes. deploy_nodes(NumNodes, InitialConfig, Services) when is_integer(NumNodes) -> - NodeConfig = [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)], - deploy_nodes(NodeConfig, Services). +NodeConfig = [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)], +deploy_nodes(NodeConfig, Services). version_to_config(Config) when is_tuple(Config)-> Config; version_to_config(Version) -> {Version, default}. deploy_clusters(Settings) -> - ClusterConfigs = [case Setting of - Configs when is_list(Configs) -> - Configs; - NumNodes when is_integer(NumNodes) -> - [{current, default} || _ <- lists:seq(1, NumNodes)]; - {NumNodes, InitialConfig} when is_integer(NumNodes) -> - [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)]; - {NumNodes, Vsn, InitialConfig} when is_integer(NumNodes) -> - [{Vsn, InitialConfig} || _ <- lists:seq(1,NumNodes)] - end || Setting <- Settings], - ?HARNESS:deploy_clusters(ClusterConfigs). +ClusterConfigs = [case Setting of + Configs when is_list(Configs) -> + Configs; + NumNodes when is_integer(NumNodes) -> + [{current, default} || _ <- lists:seq(1, NumNodes)]; + {NumNodes, InitialConfig} when is_integer(NumNodes) -> + [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)]; + {NumNodes, Vsn, InitialConfig} when is_integer(NumNodes) -> + [{Vsn, InitialConfig} || _ <- lists:seq(1,NumNodes)] + end || Setting <- Settings], +?HARNESS:deploy_clusters(ClusterConfigs). build_clusters(Settings) -> - Clusters = deploy_clusters(Settings), - [begin - join_cluster(Nodes), - lager:info("Cluster built: ~p", [Nodes]) - end || Nodes <- Clusters], - Clusters. +Clusters = deploy_clusters(Settings), +[begin + join_cluster(Nodes), + lager:info("Cluster built: ~p", [Nodes]) +end || Nodes <- Clusters], +Clusters. %% @doc Start the specified Riak node start(Node) -> - ?HARNESS:start(Node). +?HARNESS:start(Node). %% @doc Start the specified Riak `Node' and wait for it to be pingable start_and_wait(Node) -> - start(Node), - ?assertEqual(ok, wait_until_pingable(Node)). +start(Node), +?assertEqual(ok, wait_until_pingable(Node)). async_start(Node) -> - spawn(fun() -> start(Node) end). +spawn(fun() -> start(Node) end). %% @doc Stop the specified Riak `Node'. stop(Node) -> - lager:info("Stopping riak on ~p", [Node]), - timer:sleep(10000), %% I know, I know! - ?HARNESS:stop(Node). - %%rpc:call(Node, init, stop, []). +lager:info("Stopping riak on ~p", [Node]), +timer:sleep(10000), %% I know, I know! +?HARNESS:stop(Node). +%%rpc:call(Node, init, stop, []). %% @doc Stop the specified Riak `Node' and wait until it is not pingable stop_and_wait(Node) -> - stop(Node), - ?assertEqual(ok, wait_until_unpingable(Node)). +stop(Node), +?assertEqual(ok, wait_until_unpingable(Node)). %% @doc Upgrade a Riak `Node' to the specified `NewVersion'. upgrade(Node, NewVersion) -> - ?HARNESS:upgrade(Node, NewVersion). +?HARNESS:upgrade(Node, NewVersion). %% @doc Upgrade a Riak `Node' to the specified `NewVersion' and update %% the config based on entries in `Config'. upgrade(Node, NewVersion, Config) -> - ?HARNESS:upgrade(Node, NewVersion, Config). +?HARNESS:upgrade(Node, NewVersion, Config). %% @doc Upgrade a Riak node to a specific version using the alternate %% leave/upgrade/rejoin approach slow_upgrade(Node, NewVersion, Nodes) -> - lager:info("Perform leave/upgrade/join upgrade on ~p", [Node]), - lager:info("Leaving ~p", [Node]), - leave(Node), - ?assertEqual(ok, rt:wait_until_unpingable(Node)), - upgrade(Node, NewVersion), - lager:info("Rejoin ~p", [Node]), - join(Node, hd(Nodes -- [Node])), - lager:info("Wait until all nodes are ready and there are no pending changes"), - ?assertEqual(ok, wait_until_nodes_ready(Nodes)), - ?assertEqual(ok, wait_until_no_pending_changes(Nodes)), - ok. +lager:info("Perform leave/upgrade/join upgrade on ~p", [Node]), +lager:info("Leaving ~p", [Node]), +leave(Node), +?assertEqual(ok, rt:wait_until_unpingable(Node)), +upgrade(Node, NewVersion), +lager:info("Rejoin ~p", [Node]), +join(Node, hd(Nodes -- [Node])), +lager:info("Wait until all nodes are ready and there are no pending changes"), +?assertEqual(ok, wait_until_nodes_ready(Nodes)), +?assertEqual(ok, wait_until_no_pending_changes(Nodes)), +ok. %% @doc Have `Node' send a join request to `PNode' join(Node, PNode) -> - R = rpc:call(Node, riak_core, join, [PNode]), - lager:info("[join] ~p to (~p): ~p", [Node, PNode, R]), - ?assertEqual(ok, R), - ok. +R = rpc:call(Node, riak_core, join, [PNode]), +lager:info("[join] ~p to (~p): ~p", [Node, PNode, R]), +?assertEqual(ok, R), +ok. %% @doc Have `Node' send a join request to `PNode' staged_join(Node, PNode) -> - R = rpc:call(Node, riak_core, staged_join, [PNode]), - lager:info("[join] ~p to (~p): ~p", [Node, PNode, R]), - ?assertEqual(ok, R), - ok. +R = rpc:call(Node, riak_core, staged_join, [PNode]), +lager:info("[join] ~p to (~p): ~p", [Node, PNode, R]), +?assertEqual(ok, R), +ok. plan_and_commit(Node) -> - timer:sleep(500), - lager:info("planning and commiting cluster join"), - case rpc:call(Node, riak_core_claimant, plan, []) of - {error, ring_not_ready} -> - lager:info("plan: ring not ready"), - timer:sleep(100), - plan_and_commit(Node); - {ok, _, _} -> - lager:info("plan: done"), - do_commit(Node) - end. +timer:sleep(500), +lager:info("planning and commiting cluster join"), +case rpc:call(Node, riak_core_claimant, plan, []) of +{error, ring_not_ready} -> + lager:info("plan: ring not ready"), + timer:sleep(100), + plan_and_commit(Node); +{ok, _, _} -> + lager:info("plan: done"), + do_commit(Node) +end. do_commit(Node) -> - case rpc:call(Node, riak_core_claimant, commit, []) of - {error, plan_changed} -> - lager:info("commit: plan changed"), - timer:sleep(100), - maybe_wait_for_changes(Node), - plan_and_commit(Node); - {error, ring_not_ready} -> - lager:info("commit: ring not ready"), - timer:sleep(100), - maybe_wait_for_changes(Node), - do_commit(Node); - {error,nothing_planned} -> - %% Assume plan actually committed somehow - ok; - ok -> - ok - end. +case rpc:call(Node, riak_core_claimant, commit, []) of +{error, plan_changed} -> + lager:info("commit: plan changed"), + timer:sleep(100), + maybe_wait_for_changes(Node), + plan_and_commit(Node); +{error, ring_not_ready} -> + lager:info("commit: ring not ready"), + timer:sleep(100), + maybe_wait_for_changes(Node), + do_commit(Node); +{error,nothing_planned} -> + %% Assume plan actually committed somehow + ok; +ok -> + ok +end. maybe_wait_for_changes(Node) -> - Ring = get_ring(Node), - Changes = riak_core_ring:pending_changes(Ring), - Joining = riak_core_ring:members(Ring, [joining]), - lager:info("maybe_wait_for_changes, changes: ~p joining: ~p", - [Changes, Joining]), - if Changes =:= [] -> - ok; - Joining =/= [] -> - ok; - true -> - ok = wait_until_no_pending_changes([Node]) - end. +Ring = get_ring(Node), +Changes = riak_core_ring:pending_changes(Ring), +Joining = riak_core_ring:members(Ring, [joining]), +lager:info("maybe_wait_for_changes, changes: ~p joining: ~p", + [Changes, Joining]), +if Changes =:= [] -> + ok; +Joining =/= [] -> + ok; +true -> + ok = wait_until_no_pending_changes([Node]) +end. %% @doc Have the `Node' leave the cluster leave(Node) -> - R = rpc:call(Node, riak_core, leave, []), - lager:info("[leave] ~p: ~p", [Node, R]), - ?assertEqual(ok, R), - ok. +R = rpc:call(Node, riak_core, leave, []), +lager:info("[leave] ~p: ~p", [Node, R]), +?assertEqual(ok, R), +ok. %% @doc Have `Node' remove `OtherNode' from the cluster remove(Node, OtherNode) -> - ?assertEqual(ok, - rpc:call(Node, riak_kv_console, remove, [[atom_to_list(OtherNode)]])). +?assertEqual(ok, + rpc:call(Node, riak_kv_console, remove, [[atom_to_list(OtherNode)]])). %% @doc Have `Node' mark `OtherNode' as down down(Node, OtherNode) -> - rpc:call(Node, riak_kv_console, down, [[atom_to_list(OtherNode)]]). +rpc:call(Node, riak_kv_console, down, [[atom_to_list(OtherNode)]]). %% @doc partition the `P1' from `P2' nodes %% note: the nodes remained connected to riak_test@local, %% which is how `heal/1' can still work. partition(P1, P2) -> - OldCookie = rpc:call(hd(P1), erlang, get_cookie, []), - NewCookie = list_to_atom(lists:reverse(atom_to_list(OldCookie))), - [true = rpc:call(N, erlang, set_cookie, [N, NewCookie]) || N <- P1], - [[true = rpc:call(N, erlang, disconnect_node, [P2N]) || N <- P1] || P2N <- P2], - wait_until_partitioned(P1, P2), - {NewCookie, OldCookie, P1, P2}. +OldCookie = rpc:call(hd(P1), erlang, get_cookie, []), +NewCookie = list_to_atom(lists:reverse(atom_to_list(OldCookie))), +[true = rpc:call(N, erlang, set_cookie, [N, NewCookie]) || N <- P1], +[[true = rpc:call(N, erlang, disconnect_node, [P2N]) || N <- P1] || P2N <- P2], +wait_until_partitioned(P1, P2), +{NewCookie, OldCookie, P1, P2}. %% @doc heal the partition created by call to `partition/2' %% `OldCookie' is the original shared cookie heal({_NewCookie, OldCookie, P1, P2}) -> - Cluster = P1 ++ P2, - % set OldCookie on P1 Nodes - [true = rpc:call(N, erlang, set_cookie, [N, OldCookie]) || N <- P1], - wait_until_connected(Cluster), - {_GN, []} = rpc:sbcast(Cluster, riak_core_node_watcher, broadcast), - ok. +Cluster = P1 ++ P2, +% set OldCookie on P1 Nodes +[true = rpc:call(N, erlang, set_cookie, [N, OldCookie]) || N <- P1], +wait_until_connected(Cluster), +{_GN, []} = rpc:sbcast(Cluster, riak_core_node_watcher, broadcast), +ok. %% @doc Spawn `Cmd' on the machine running the test harness spawn_cmd(Cmd) -> - ?HARNESS:spawn_cmd(Cmd). +?HARNESS:spawn_cmd(Cmd). %% @doc Spawn `Cmd' on the machine running the test harness spawn_cmd(Cmd, Opts) -> - ?HARNESS:spawn_cmd(Cmd, Opts). +?HARNESS:spawn_cmd(Cmd, Opts). %% @doc Wait for a command spawned by `spawn_cmd', returning %% the exit status and result wait_for_cmd(CmdHandle) -> - ?HARNESS:wait_for_cmd(CmdHandle). +?HARNESS:wait_for_cmd(CmdHandle). %% @doc Spawn `Cmd' on the machine running the test harness, returning %% the exit status and result cmd(Cmd) -> - ?HARNESS:cmd(Cmd). +?HARNESS:cmd(Cmd). %% @doc Spawn `Cmd' on the machine running the test harness, returning %% the exit status and result cmd(Cmd, Opts) -> - ?HARNESS:cmd(Cmd, Opts). +?HARNESS:cmd(Cmd, Opts). %% @doc pretty much the same as os:cmd/1 but it will stream the output to lager. %% If you're running a long running command, it will dump the output %% once per second, as to not create the impression that nothing is happening. -spec stream_cmd(string()) -> {integer(), string()}. stream_cmd(Cmd) -> - Port = open_port({spawn, binary_to_list(iolist_to_binary(Cmd))}, [stream, stderr_to_stdout, exit_status]), - stream_cmd_loop(Port, "", "", now()). +Port = open_port({spawn, binary_to_list(iolist_to_binary(Cmd))}, [stream, stderr_to_stdout, exit_status]), +stream_cmd_loop(Port, "", "", now()). %% @doc same as rt:stream_cmd/1, but with options, like open_port/2 -spec stream_cmd(string(), string()) -> {integer(), string()}. stream_cmd(Cmd, Opts) -> - Port = open_port({spawn, binary_to_list(iolist_to_binary(Cmd))}, [stream, stderr_to_stdout, exit_status] ++ Opts), - stream_cmd_loop(Port, "", "", now()). +Port = open_port({spawn, binary_to_list(iolist_to_binary(Cmd))}, [stream, stderr_to_stdout, exit_status] ++ Opts), +stream_cmd_loop(Port, "", "", now()). stream_cmd_loop(Port, Buffer, NewLineBuffer, Time={_MegaSecs, Secs, _MicroSecs}) -> - receive - {Port, {data, Data}} -> - {_, Now, _} = now(), - NewNewLineBuffer = case Now > Secs of - true -> - lager:info(NewLineBuffer), - ""; - _ -> - NewLineBuffer - end, - case rt:str(Data, "\n") of - true -> - lager:info(NewNewLineBuffer), - Tokens = string:tokens(Data, "\n"), - [ lager:info(Token) || Token <- Tokens ], - stream_cmd_loop(Port, Buffer ++ NewNewLineBuffer ++ Data, "", Time); - _ -> - stream_cmd_loop(Port, Buffer, NewNewLineBuffer ++ Data, now()) - end; - {Port, {exit_status, Status}} -> - catch port_close(Port), - {Status, Buffer} - after rt:config(rt_max_wait_time) -> - {-1, Buffer} - end. +receive +{Port, {data, Data}} -> + {_, Now, _} = now(), + NewNewLineBuffer = case Now > Secs of + true -> + lager:info(NewLineBuffer), + ""; + _ -> + NewLineBuffer + end, + case rt:str(Data, "\n") of + true -> + lager:info(NewNewLineBuffer), + Tokens = string:tokens(Data, "\n"), + [ lager:info(Token) || Token <- Tokens ], + stream_cmd_loop(Port, Buffer ++ NewNewLineBuffer ++ Data, "", Time); + _ -> + stream_cmd_loop(Port, Buffer, NewNewLineBuffer ++ Data, now()) + end; +{Port, {exit_status, Status}} -> + catch port_close(Port), + {Status, Buffer} +after rt:config(rt_max_wait_time) -> + {-1, Buffer} +end. %%%=================================================================== %%% Remote code management %%%=================================================================== load_modules_on_nodes([], Nodes) - when is_list(Nodes) -> - ok; +when is_list(Nodes) -> +ok; load_modules_on_nodes([Module | MoreModules], Nodes) - when is_list(Nodes) -> - case code:get_object_code(Module) of - {Module, Bin, File} -> - {_, []} = rpc:multicall(Nodes, code, load_binary, [Module, File, Bin]); - error -> - error(lists:flatten(io_lib:format("unable to get_object_code(~s)", [Module]))) - end, - load_modules_on_nodes(MoreModules, Nodes). +when is_list(Nodes) -> +case code:get_object_code(Module) of +{Module, Bin, File} -> + {_, []} = rpc:multicall(Nodes, code, load_binary, [Module, File, Bin]); +error -> + error(lists:flatten(io_lib:format("unable to get_object_code(~s)", [Module]))) +end, +load_modules_on_nodes(MoreModules, Nodes). %%%=================================================================== @@ -578,36 +578,36 @@ load_modules_on_nodes([Module | MoreModules], Nodes) %% @doc Is the `Node' up according to net_adm:ping is_pingable(Node) -> - net_adm:ping(Node) =:= pong. +net_adm:ping(Node) =:= pong. is_mixed_cluster(Nodes) when is_list(Nodes) -> - %% If the nodes are bad, we don't care what version they are - {Versions, _BadNodes} = rpc:multicall(Nodes, init, script_id, [], rt_config:get(rt_max_wait_time)), - length(lists:usort(Versions)) > 1; +%% If the nodes are bad, we don't care what version they are +{Versions, _BadNodes} = rpc:multicall(Nodes, init, script_id, [], rt_config:get(rt_max_wait_time)), +length(lists:usort(Versions)) > 1; is_mixed_cluster(Node) -> - Nodes = rpc:call(Node, erlang, nodes, []), - is_mixed_cluster(Nodes). +Nodes = rpc:call(Node, erlang, nodes, []), +is_mixed_cluster(Nodes). %% @private is_ready(Node) -> - case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of - {ok, Ring} -> - case lists:member(Node, riak_core_ring:ready_members(Ring)) of - true -> true; - false -> {not_ready, Node} - end; - Other -> - Other - end. +case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of +{ok, Ring} -> + case lists:member(Node, riak_core_ring:ready_members(Ring)) of + true -> true; + false -> {not_ready, Node} + end; +Other -> + Other +end. %% @private is_ring_ready(Node) -> - case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of - {ok, Ring} -> - riak_core_ring:ring_ready(Ring); - _ -> - false - end. +case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of +{ok, Ring} -> + riak_core_ring:ring_ready(Ring); +_ -> + false +end. %% @doc Utility function used to construct test predicates. Retries the %% function `Fun' until it returns `true', or until the maximum @@ -615,151 +615,151 @@ is_ring_ready(Node) -> %% provided `rt_max_wait_time' and `rt_retry_delay' parameters in %% specified `riak_test' config file. wait_until(Fun) when is_function(Fun) -> - MaxTime = rt_config:get(rt_max_wait_time), - Delay = rt_config:get(rt_retry_delay), - Retry = MaxTime div Delay, - wait_until(Fun, Retry, Delay). +MaxTime = rt_config:get(rt_max_wait_time), +Delay = rt_config:get(rt_retry_delay), +Retry = MaxTime div Delay, +wait_until(Fun, Retry, Delay). %% @doc Convenience wrapper for wait_until for the myriad functions that %% take a node as single argument. wait_until(Node, Fun) when is_atom(Node), is_function(Fun) -> - wait_until(fun() -> Fun(Node) end). +wait_until(fun() -> Fun(Node) end). %% @doc Retry `Fun' until it returns `Retry' times, waiting `Delay' %% milliseconds between retries. This is our eventual consistency bread %% and butter wait_until(Fun, Retry, Delay) when Retry > 0 -> - Res = Fun(), - case Res of - true -> - ok; - _ when Retry == 1 -> - {fail, Res}; - _ -> - timer:sleep(Delay), - wait_until(Fun, Retry-1, Delay) - end. +Res = Fun(), +case Res of +true -> + ok; +_ when Retry == 1 -> + {fail, Res}; +_ -> + timer:sleep(Delay), + wait_until(Fun, Retry-1, Delay) +end. %% @doc Wait until the specified node is considered ready by `riak_core'. %% As of Riak 1.0, a node is ready if it is in the `valid' or `leaving' %% states. A ready node is guaranteed to have current preflist/ownership %% information. wait_until_ready(Node) -> - lager:info("Wait until ~p ready", [Node]), - ?assertEqual(ok, wait_until(Node, fun is_ready/1)), - ok. +lager:info("Wait until ~p ready", [Node]), +?assertEqual(ok, wait_until(Node, fun is_ready/1)), +ok. %% @doc Wait until status can be read from riak_kv_console wait_until_status_ready(Node) -> - lager:info("Wait until status ready in ~p", [Node]), - ?assertEqual(ok, wait_until(Node, - fun(_) -> - case rpc:call(Node, riak_kv_console, status, [[]]) of - ok -> - true; - Res -> - Res - end - end)). +lager:info("Wait until status ready in ~p", [Node]), +?assertEqual(ok, wait_until(Node, + fun(_) -> + case rpc:call(Node, riak_kv_console, status, [[]]) of + ok -> + true; + Res -> + Res + end + end)). %% @doc Given a list of nodes, wait until all nodes believe there are no %% on-going or pending ownership transfers. -spec wait_until_no_pending_changes([node()]) -> ok | fail. wait_until_no_pending_changes(Nodes) -> - lager:info("Wait until no pending changes on ~p", [Nodes]), - F = fun() -> - rpc:multicall(Nodes, riak_core_vnode_manager, force_handoffs, []), - {Rings, BadNodes} = rpc:multicall(Nodes, riak_core_ring_manager, get_raw_ring, []), - Changes = [ riak_core_ring:pending_changes(Ring) =:= [] || {ok, Ring} <- Rings ], - BadNodes =:= [] andalso length(Changes) =:= length(Nodes) andalso lists:all(fun(T) -> T end, Changes) - end, - ?assertEqual(ok, wait_until(F)), - ok. +lager:info("Wait until no pending changes on ~p", [Nodes]), +F = fun() -> + rpc:multicall(Nodes, riak_core_vnode_manager, force_handoffs, []), + {Rings, BadNodes} = rpc:multicall(Nodes, riak_core_ring_manager, get_raw_ring, []), + Changes = [ riak_core_ring:pending_changes(Ring) =:= [] || {ok, Ring} <- Rings ], + BadNodes =:= [] andalso length(Changes) =:= length(Nodes) andalso lists:all(fun(T) -> T end, Changes) +end, +?assertEqual(ok, wait_until(F)), +ok. %% @doc Waits until no transfers are in-flight or pending, checked by %% riak_core_status:transfers(). -spec wait_until_transfers_complete([node()]) -> ok | fail. wait_until_transfers_complete([Node0|_]) -> - lager:info("Wait until transfers complete ~p", [Node0]), - F = fun(Node) -> - {DownNodes, Transfers} = rpc:call(Node, riak_core_status, transfers, []), - DownNodes =:= [] andalso Transfers =:= [] - end, - ?assertEqual(ok, wait_until(Node0, F)), - ok. +lager:info("Wait until transfers complete ~p", [Node0]), +F = fun(Node) -> + {DownNodes, Transfers} = rpc:call(Node, riak_core_status, transfers, []), + DownNodes =:= [] andalso Transfers =:= [] +end, +?assertEqual(ok, wait_until(Node0, F)), +ok. wait_for_service(Node, Services) when is_list(Services) -> - F = fun(N) -> - case rpc:call(N, riak_core_node_watcher, services, [N]) of - {badrpc, Error} -> - {badrpc, Error}; - CurrServices when is_list(CurrServices) -> - lists:all(fun(Service) -> lists:member(Service, CurrServices) end, Services); - Res -> - Res - end - end, - ?assertEqual(ok, wait_until(Node, F)), - ok; +F = fun(N) -> + case rpc:call(N, riak_core_node_watcher, services, [N]) of + {badrpc, Error} -> + {badrpc, Error}; + CurrServices when is_list(CurrServices) -> + lists:all(fun(Service) -> lists:member(Service, CurrServices) end, Services); + Res -> + Res + end +end, +?assertEqual(ok, wait_until(Node, F)), +ok; wait_for_service(Node, Service) -> - wait_for_service(Node, [Service]). +wait_for_service(Node, [Service]). wait_for_cluster_service(Nodes, Service) -> - lager:info("Wait for cluster service ~p in ~p", [Service, Nodes]), - F = fun(N) -> - UpNodes = rpc:call(N, riak_core_node_watcher, nodes, [Service]), - (Nodes -- UpNodes) == [] - end, - [?assertEqual(ok, wait_until(Node, F)) || Node <- Nodes], - ok. +lager:info("Wait for cluster service ~p in ~p", [Service, Nodes]), +F = fun(N) -> + UpNodes = rpc:call(N, riak_core_node_watcher, nodes, [Service]), + (Nodes -- UpNodes) == [] +end, +[?assertEqual(ok, wait_until(Node, F)) || Node <- Nodes], +ok. %% @doc Given a list of nodes, wait until all nodes are considered ready. %% See {@link wait_until_ready/1} for definition of ready. wait_until_nodes_ready(Nodes) -> - lager:info("Wait until nodes are ready : ~p", [Nodes]), - [?assertEqual(ok, wait_until(Node, fun is_ready/1)) || Node <- Nodes], - ok. +lager:info("Wait until nodes are ready : ~p", [Nodes]), +[?assertEqual(ok, wait_until(Node, fun is_ready/1)) || Node <- Nodes], +ok. %% @doc Wait until all nodes in the list `Nodes' believe each other to be %% members of the cluster. wait_until_all_members(Nodes) -> - wait_until_all_members(Nodes, Nodes). +wait_until_all_members(Nodes, Nodes). %% @doc Wait until all nodes in the list `Nodes' believes all nodes in the %% list `Members' are members of the cluster. wait_until_all_members(Nodes, ExpectedMembers) -> - lager:info("Wait until all members ~p ~p", [Nodes, ExpectedMembers]), - S1 = ordsets:from_list(ExpectedMembers), - F = fun(Node) -> - case members_according_to(Node) of - {badrpc, _} -> - false; - ReportedMembers -> - S2 = ordsets:from_list(ReportedMembers), - ordsets:is_subset(S1, S2) - end - end, - [?assertEqual(ok, wait_until(Node, F)) || Node <- Nodes], - ok. +lager:info("Wait until all members ~p ~p", [Nodes, ExpectedMembers]), +S1 = ordsets:from_list(ExpectedMembers), +F = fun(Node) -> + case members_according_to(Node) of + {badrpc, _} -> + false; + ReportedMembers -> + S2 = ordsets:from_list(ReportedMembers), + ordsets:is_subset(S1, S2) + end +end, +[?assertEqual(ok, wait_until(Node, F)) || Node <- Nodes], +ok. %% @doc Given a list of nodes, wait until all nodes believe the ring has %% converged (ie. `riak_core_ring:is_ready' returns `true'). wait_until_ring_converged(Nodes) -> - lager:info("Wait until ring converged on ~p", [Nodes]), - [?assertEqual(ok, wait_until(Node, fun is_ring_ready/1)) || Node <- Nodes], - ok. +lager:info("Wait until ring converged on ~p", [Nodes]), +[?assertEqual(ok, wait_until(Node, fun is_ring_ready/1)) || Node <- Nodes], +ok. wait_until_legacy_ringready(Node) -> - lager:info("Wait until legacy ring ready on ~p", [Node]), - rt:wait_until(Node, - fun(_) -> - case rpc:call(Node, riak_kv_status, ringready, []) of - {ok, _Nodes} -> - true; - Res -> - Res - end - end). +lager:info("Wait until legacy ring ready on ~p", [Node]), +rt:wait_until(Node, + fun(_) -> + case rpc:call(Node, riak_kv_status, ringready, []) of + {ok, _Nodes} -> + true; + Res -> + Res + end + end). %% @doc wait until each node in Nodes is disterl connected to each. wait_until_connected(Nodes) -> @@ -774,83 +774,83 @@ wait_until_connected(Nodes) -> %% @doc Wait until the specified node is pingable wait_until_pingable(Node) -> - lager:info("Wait until ~p is pingable", [Node]), - F = fun(N) -> - net_adm:ping(N) =:= pong - end, - ?assertEqual(ok, wait_until(Node, F)), - ok. +lager:info("Wait until ~p is pingable", [Node]), +F = fun(N) -> + net_adm:ping(N) =:= pong +end, +?assertEqual(ok, wait_until(Node, F)), +ok. %% @doc Wait until the specified node is no longer pingable wait_until_unpingable(Node) -> - lager:info("Wait until ~p is not pingable", [Node]), - _OSPidToKill = rpc:call(Node, os, getpid, []), - F = fun() -> net_adm:ping(Node) =:= pang end, - %% riak stop will kill -9 after 5 mins, so we try to wait at least that - %% amount of time. - Delay = rt_config:get(rt_retry_delay), - Retry = lists:max([360000, rt_config:get(rt_max_wait_time)]) div Delay, - case wait_until(F, Retry, Delay) of - ok -> ok; - _ -> - lager:error("Timed out waiting for node ~p to shutdown", [Node]), - ?assert(node_shutdown_timed_out) - end. +lager:info("Wait until ~p is not pingable", [Node]), +_OSPidToKill = rpc:call(Node, os, getpid, []), +F = fun() -> net_adm:ping(Node) =:= pang end, +%% riak stop will kill -9 after 5 mins, so we try to wait at least that +%% amount of time. +Delay = rt_config:get(rt_retry_delay), +Retry = lists:max([360000, rt_config:get(rt_max_wait_time)]) div Delay, +case wait_until(F, Retry, Delay) of +ok -> ok; +_ -> + lager:error("Timed out waiting for node ~p to shutdown", [Node]), + ?assert(node_shutdown_timed_out) +end. % Waits until a certain registered name pops up on the remote node. wait_until_registered(Node, Name) -> - lager:info("Wait until ~p is up on ~p", [Name, Node]), - - F = fun() -> - Registered = rpc:call(Node, erlang, registered, []), - lists:member(Name, Registered) - end, - case wait_until(F) of - ok -> - ok; - _ -> - lager:info("The server with the name ~p on ~p is not coming up.", - [Name, Node]), - ?assert(registered_name_timed_out) - end. +lager:info("Wait until ~p is up on ~p", [Name, Node]), + +F = fun() -> + Registered = rpc:call(Node, erlang, registered, []), + lists:member(Name, Registered) +end, +case wait_until(F) of +ok -> + ok; +_ -> + lager:info("The server with the name ~p on ~p is not coming up.", + [Name, Node]), + ?assert(registered_name_timed_out) +end. %% Waits until the cluster actually detects that it is partitioned. wait_until_partitioned(P1, P2) -> - lager:info("Waiting until partition acknowledged: ~p ~p", [P1, P2]), - [ begin - lager:info("Waiting for ~p to be partitioned from ~p", [Node, P2]), - wait_until(fun() -> is_partitioned(Node, P2) end) - end || Node <- P1 ], - [ begin - lager:info("Waiting for ~p to be partitioned from ~p", [Node, P1]), - wait_until(fun() -> is_partitioned(Node, P1) end) - end || Node <- P2 ]. +lager:info("Waiting until partition acknowledged: ~p ~p", [P1, P2]), +[ begin + lager:info("Waiting for ~p to be partitioned from ~p", [Node, P2]), + wait_until(fun() -> is_partitioned(Node, P2) end) +end || Node <- P1 ], +[ begin + lager:info("Waiting for ~p to be partitioned from ~p", [Node, P1]), + wait_until(fun() -> is_partitioned(Node, P1) end) +end || Node <- P2 ]. is_partitioned(Node, Peers) -> - AvailableNodes = rpc:call(Node, riak_core_node_watcher, nodes, [riak_kv]), - lists:all(fun(Peer) -> not lists:member(Peer, AvailableNodes) end, Peers). +AvailableNodes = rpc:call(Node, riak_core_node_watcher, nodes, [riak_kv]), +lists:all(fun(Peer) -> not lists:member(Peer, AvailableNodes) end, Peers). % when you just can't wait brutal_kill(Node) -> - rt_cover:maybe_stop_on_node(Node), - lager:info("Killing node ~p", [Node]), - OSPidToKill = rpc:call(Node, os, getpid, []), - %% try a normal kill first, but set a timer to - %% kill -9 after 5 seconds just in case - rpc:cast(Node, timer, apply_after, - [5000, os, cmd, [io_lib:format("kill -9 ~s", [OSPidToKill])]]), - rpc:cast(Node, os, cmd, [io_lib:format("kill -15 ~s", [OSPidToKill])]), - ok. +rt_cover:maybe_stop_on_node(Node), +lager:info("Killing node ~p", [Node]), +OSPidToKill = rpc:call(Node, os, getpid, []), +%% try a normal kill first, but set a timer to +%% kill -9 after 5 seconds just in case +rpc:cast(Node, timer, apply_after, + [5000, os, cmd, [io_lib:format("kill -9 ~s", [OSPidToKill])]]), +rpc:cast(Node, os, cmd, [io_lib:format("kill -15 ~s", [OSPidToKill])]), +ok. capability(Node, all) -> - rpc:call(Node, riak_core_capability, all, []); +rpc:call(Node, riak_core_capability, all, []); capability(Node, Capability) -> - rpc:call(Node, riak_core_capability, get, [Capability]). +rpc:call(Node, riak_core_capability, get, [Capability]). capability(Node, Capability, Default) -> - rpc:call(Node, riak_core_capability, get, [Capability, Default]). +rpc:call(Node, riak_core_capability, get, [Capability, Default]). wait_until_capability(Node, Capability, Value) -> rt:wait_until(Node, @@ -877,103 +877,103 @@ wait_until_capability_contains(Node, Capability, Value) -> end). cap_equal(Val, Cap) when is_list(Cap) -> - lists:sort(Cap) == lists:sort(Val); +lists:sort(Cap) == lists:sort(Val); cap_equal(Val, Cap) -> - Val == Cap. +Val == Cap. cap_subset(Val, Cap) when is_list(Cap) -> sets:is_subset(sets:from_list(Val), sets:from_list(Cap)). wait_until_owners_according_to(Node, Nodes) -> - SortedNodes = lists:usort(Nodes), - F = fun(N) -> - owners_according_to(N) =:= SortedNodes - end, - ?assertEqual(ok, wait_until(Node, F)), - ok. +SortedNodes = lists:usort(Nodes), +F = fun(N) -> +owners_according_to(N) =:= SortedNodes +end, +?assertEqual(ok, wait_until(Node, F)), +ok. wait_until_nodes_agree_about_ownership(Nodes) -> - lager:info("Wait until nodes agree about ownership ~p", [Nodes]), - Results = [ wait_until_owners_according_to(Node, Nodes) || Node <- Nodes ], - ?assert(lists:all(fun(X) -> ok =:= X end, Results)). +lager:info("Wait until nodes agree about ownership ~p", [Nodes]), +Results = [ wait_until_owners_according_to(Node, Nodes) || Node <- Nodes ], +?assert(lists:all(fun(X) -> ok =:= X end, Results)). %% AAE support wait_until_aae_trees_built(Nodes) -> - lager:info("Wait until AAE builds all partition trees across ~p", [Nodes]), - BuiltFun = fun() -> lists:foldl(aae_tree_built_fun(), true, Nodes) end, - ?assertEqual(ok, wait_until(BuiltFun)), - ok. +lager:info("Wait until AAE builds all partition trees across ~p", [Nodes]), +BuiltFun = fun() -> lists:foldl(aae_tree_built_fun(), true, Nodes) end, +?assertEqual(ok, wait_until(BuiltFun)), +ok. aae_tree_built_fun() -> - fun(Node, _AllBuilt = true) -> - case get_aae_tree_info(Node) of - {ok, TreeInfos} -> - case all_trees_have_build_times(TreeInfos) of - true -> - Partitions = [I || {I, _} <- TreeInfos], - all_aae_trees_built(Node, Partitions); - false -> - some_trees_not_built - end; - Err -> - Err - end; - (_Node, Err) -> - Err - end. +fun(Node, _AllBuilt = true) -> + case get_aae_tree_info(Node) of + {ok, TreeInfos} -> + case all_trees_have_build_times(TreeInfos) of + true -> + Partitions = [I || {I, _} <- TreeInfos], + all_aae_trees_built(Node, Partitions); + false -> + some_trees_not_built + end; + Err -> + Err + end; +(_Node, Err) -> + Err +end. % It is unlikely but possible to get a tree built time from compute_tree_info % but an attempt to use the tree returns not_built. This is because the build % process has finished, but the lock on the tree won't be released until it % dies and the manager detects it. Yes, this is super freaking paranoid. all_aae_trees_built(Node, Partitions) -> - %% Notice that the process locking is spawned by the - %% pmap. That's important! as it should die eventually - %% so the lock is released and the test can lock the tree. - IndexBuilts = rt:pmap(index_built_fun(Node), Partitions), - BadOnes = [R || R <- IndexBuilts, R /= true], - case BadOnes of - [] -> - true; - _ -> - BadOnes - end. +%% Notice that the process locking is spawned by the +%% pmap. That's important! as it should die eventually +%% so the lock is released and the test can lock the tree. +IndexBuilts = rt:pmap(index_built_fun(Node), Partitions), +BadOnes = [R || R <- IndexBuilts, R /= true], +case BadOnes of +[] -> + true; +_ -> + BadOnes +end. get_aae_tree_info(Node) -> - case rpc:call(Node, riak_kv_entropy_info, compute_tree_info, []) of - {badrpc, _} -> - {error, {badrpc, Node}}; - Info -> - lager:debug("Entropy table on node ~p : ~p", [Node, Info]), - {ok, Info} - end. +case rpc:call(Node, riak_kv_entropy_info, compute_tree_info, []) of +{badrpc, _} -> + {error, {badrpc, Node}}; +Info -> + lager:debug("Entropy table on node ~p : ~p", [Node, Info]), + {ok, Info} +end. all_trees_have_build_times(Info) -> - not lists:keymember(undefined, 2, Info). +not lists:keymember(undefined, 2, Info). index_built_fun(Node) -> - fun(Idx) -> - case rpc:call(Node, riak_kv_vnode, - hashtree_pid, [Idx]) of - {ok, TreePid} -> - case rpc:call(Node, riak_kv_index_hashtree, - get_lock, [TreePid, for_riak_test]) of - {badrpc, _} -> - {error, {badrpc, Node}}; - TreeLocked when TreeLocked == ok; - TreeLocked == already_locked -> - true; - Err -> - % Either not_built or some unhandled result, - % in which case update this case please! - {error, {index_not_built, Node, Idx, Err}} - end; - {error, _}=Err -> - Err; - {badrpc, _} -> - {error, {badrpc, Node}} - end - end. +fun(Idx) -> + case rpc:call(Node, riak_kv_vnode, + hashtree_pid, [Idx]) of + {ok, TreePid} -> + case rpc:call(Node, riak_kv_index_hashtree, + get_lock, [TreePid, for_riak_test]) of + {badrpc, _} -> + {error, {badrpc, Node}}; + TreeLocked when TreeLocked == ok; + TreeLocked == already_locked -> + true; + Err -> + % Either not_built or some unhandled result, + % in which case update this case please! + {error, {index_not_built, Node, Idx, Err}} + end; + {error, _}=Err -> + Err; + {badrpc, _} -> + {error, {badrpc, Node}} + end +end. %%%=================================================================== %%% Ring Functions @@ -982,84 +982,84 @@ index_built_fun(Node) -> %% @doc Ensure that the specified node is a singleton node/cluster -- a node %% that owns 100% of the ring. check_singleton_node(Node) -> - lager:info("Check ~p is a singleton", [Node]), - {ok, Ring} = rpc:call(Node, riak_core_ring_manager, get_raw_ring, []), - Owners = lists:usort([Owner || {_Idx, Owner} <- riak_core_ring:all_owners(Ring)]), - ?assertEqual([Node], Owners), - ok. +lager:info("Check ~p is a singleton", [Node]), +{ok, Ring} = rpc:call(Node, riak_core_ring_manager, get_raw_ring, []), +Owners = lists:usort([Owner || {_Idx, Owner} <- riak_core_ring:all_owners(Ring)]), +?assertEqual([Node], Owners), +ok. % @doc Get list of partitions owned by node (primary). partitions_for_node(Node) -> - Ring = get_ring(Node), - [Idx || {Idx, Owner} <- riak_core_ring:all_owners(Ring), Owner == Node]. +Ring = get_ring(Node), +[Idx || {Idx, Owner} <- riak_core_ring:all_owners(Ring), Owner == Node]. %% @doc Get the raw ring for `Node'. get_ring(Node) -> - {ok, Ring} = rpc:call(Node, riak_core_ring_manager, get_raw_ring, []), - Ring. +{ok, Ring} = rpc:call(Node, riak_core_ring_manager, get_raw_ring, []), +Ring. assert_nodes_agree_about_ownership(Nodes) -> - ?assertEqual(ok, wait_until_ring_converged(Nodes)), - ?assertEqual(ok, wait_until_all_members(Nodes)), - [ ?assertEqual({Node, Nodes}, {Node, owners_according_to(Node)}) || Node <- Nodes]. +?assertEqual(ok, wait_until_ring_converged(Nodes)), +?assertEqual(ok, wait_until_all_members(Nodes)), +[ ?assertEqual({Node, Nodes}, {Node, owners_according_to(Node)}) || Node <- Nodes]. %% @doc Return a list of nodes that own partitions according to the ring %% retrieved from the specified node. owners_according_to(Node) -> - case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of - {ok, Ring} -> - Owners = [Owner || {_Idx, Owner} <- riak_core_ring:all_owners(Ring)], - lists:usort(Owners); - {badrpc, _}=BadRpc -> - BadRpc - end. +case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of +{ok, Ring} -> + Owners = [Owner || {_Idx, Owner} <- riak_core_ring:all_owners(Ring)], + lists:usort(Owners); +{badrpc, _}=BadRpc -> + BadRpc +end. %% @doc Return a list of cluster members according to the ring retrieved from %% the specified node. members_according_to(Node) -> - case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of - {ok, Ring} -> - Members = riak_core_ring:all_members(Ring), - Members; - {badrpc, _}=BadRpc -> - BadRpc - end. +case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of +{ok, Ring} -> + Members = riak_core_ring:all_members(Ring), + Members; +{badrpc, _}=BadRpc -> + BadRpc +end. %% @doc Return an appropriate ringsize for the node count passed %% in. 24 is the number of cores on the bigger intel machines, but this %% may be too large for the single-chip machines. nearest_ringsize(Count) -> - nearest_ringsize(Count * 24, 2). +nearest_ringsize(Count * 24, 2). nearest_ringsize(Count, Power) -> - case Count < trunc(Power * 0.9) of - true -> - Power; - false -> - nearest_ringsize(Count, Power * 2) - end. +case Count < trunc(Power * 0.9) of +true -> + Power; +false -> + nearest_ringsize(Count, Power * 2) +end. %% @doc Return the cluster status of `Member' according to the ring %% retrieved from `Node'. status_of_according_to(Member, Node) -> - case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of - {ok, Ring} -> - Status = riak_core_ring:member_status(Ring, Member), - Status; - {badrpc, _}=BadRpc -> - BadRpc - end. +case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of +{ok, Ring} -> + Status = riak_core_ring:member_status(Ring, Member), + Status; +{badrpc, _}=BadRpc -> + BadRpc +end. %% @doc Return a list of nodes that own partitions according to the ring %% retrieved from the specified node. claimant_according_to(Node) -> - case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of - {ok, Ring} -> - Claimant = riak_core_ring:claimant(Ring), - Claimant; - {badrpc, _}=BadRpc -> - BadRpc - end. +case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of +{ok, Ring} -> + Claimant = riak_core_ring:claimant(Ring), + Claimant; +{badrpc, _}=BadRpc -> + BadRpc +end. %%%=================================================================== %%% Cluster Utility Functions @@ -1068,28 +1068,28 @@ claimant_according_to(Node) -> %% @doc Safely construct a new cluster and return a list of the deployed nodes %% @todo Add -spec and update doc to reflect mult-version changes build_cluster(Versions) when is_list(Versions) -> - build_cluster(length(Versions), Versions, default); +build_cluster(length(Versions), Versions, default); build_cluster(NumNodes) -> - build_cluster(NumNodes, default). +build_cluster(NumNodes, default). %% @doc Safely construct a `NumNode' size cluster using %% `InitialConfig'. Return a list of the deployed nodes. build_cluster(NumNodes, InitialConfig) -> - build_cluster(NumNodes, [], InitialConfig). +build_cluster(NumNodes, [], InitialConfig). build_cluster(NumNodes, Versions, InitialConfig) -> - %% Deploy a set of new nodes - Nodes = - case Versions of - [] -> - deploy_nodes(NumNodes, InitialConfig); - _ -> - deploy_nodes(Versions) - end, - - join_cluster(Nodes), - lager:info("Cluster built: ~p", [Nodes]), - Nodes. +%% Deploy a set of new nodes +Nodes = +case Versions of + [] -> + deploy_nodes(NumNodes, InitialConfig); + _ -> + deploy_nodes(Versions) +end, + +join_cluster(Nodes), +lager:info("Cluster built: ~p", [Nodes]), +Nodes. join_cluster(Nodes) -> %% Ensure each node owns 100% of it's own ring @@ -1113,12 +1113,12 @@ join_cluster(Nodes) -> try_nodes_ready(Nodes, 3, 500) end, - ?assertEqual(ok, wait_until_nodes_ready(Nodes)), +?assertEqual(ok, wait_until_nodes_ready(Nodes)), - %% Ensure each node owns a portion of the ring - wait_until_nodes_agree_about_ownership(Nodes), - ?assertEqual(ok, wait_until_no_pending_changes(Nodes)), - ok. +%% Ensure each node owns a portion of the ring +wait_until_nodes_agree_about_ownership(Nodes), +?assertEqual(ok, wait_until_no_pending_changes(Nodes)), +ok. -type products() :: riak | riak_ee | riak_cs | unknown. @@ -1136,54 +1136,54 @@ product(Node) -> end. try_nodes_ready([Node1 | _Nodes], 0, _SleepMs) -> - lager:info("Nodes not ready after initial plan/commit, retrying"), - plan_and_commit(Node1); +lager:info("Nodes not ready after initial plan/commit, retrying"), +plan_and_commit(Node1); try_nodes_ready(Nodes, N, SleepMs) -> - ReadyNodes = [Node || Node <- Nodes, is_ready(Node) =:= true], - case ReadyNodes of - Nodes -> - ok; - _ -> - timer:sleep(SleepMs), - try_nodes_ready(Nodes, N-1, SleepMs) - end. +ReadyNodes = [Node || Node <- Nodes, is_ready(Node) =:= true], +case ReadyNodes of +Nodes -> + ok; +_ -> + timer:sleep(SleepMs), + try_nodes_ready(Nodes, N-1, SleepMs) +end. %% @doc Stop nodes and wipe out their data directories clean_cluster(Nodes) when is_list(Nodes) -> - [stop_and_wait(Node) || Node <- Nodes], - clean_data_dir(Nodes). +[stop_and_wait(Node) || Node <- Nodes], +clean_data_dir(Nodes). clean_data_dir(Nodes) -> - clean_data_dir(Nodes, ""). +clean_data_dir(Nodes, ""). clean_data_dir(Nodes, SubDir) when not is_list(Nodes) -> - clean_data_dir([Nodes], SubDir); +clean_data_dir([Nodes], SubDir); clean_data_dir(Nodes, SubDir) when is_list(Nodes) -> - ?HARNESS:clean_data_dir(Nodes, SubDir). +?HARNESS:clean_data_dir(Nodes, SubDir). %% @doc Shutdown every node, this is for after a test run is complete. teardown() -> - %% stop all connected nodes, 'cause it'll be faster that - %%lager:info("RPC stopping these nodes ~p", [nodes()]), - %%[ rt:stop(Node) || Node <- nodes()], - %% Then do the more exhaustive harness thing, in case something was up - %% but not connected. - ?HARNESS:teardown(). +%% stop all connected nodes, 'cause it'll be faster that +%%lager:info("RPC stopping these nodes ~p", [nodes()]), +%%[ rt:stop(Node) || Node <- nodes()], +%% Then do the more exhaustive harness thing, in case something was up +%% but not connected. +?HARNESS:teardown(). versions() -> - ?HARNESS:versions(). +?HARNESS:versions(). %%%=================================================================== %%% Basic Read/Write Functions %%%=================================================================== systest_write(Node, Size) -> - systest_write(Node, Size, 2). +systest_write(Node, Size, 2). systest_write(Node, Size, W) -> - systest_write(Node, 1, Size, <<"systest">>, W). +systest_write(Node, 1, Size, <<"systest">>, W). systest_write(Node, Start, End, Bucket, W) -> - systest_write(Node, Start, End, Bucket, W, <<>>). +systest_write(Node, Start, End, Bucket, W, <<>>). %% @doc Write (End-Start)+1 objects to Node. Objects keys will be %% `Start', `Start+1' ... `End', each encoded as a 32-bit binary @@ -1194,36 +1194,36 @@ systest_write(Node, Start, End, Bucket, W) -> %% empty list. Each error has the form `{N :: integer(), Error :: term()}', %% where N is the unencoded key of the object that failed to store. systest_write(Node, Start, End, Bucket, W, CommonValBin) - when is_binary(CommonValBin) -> - rt:wait_for_service(Node, riak_kv), - {ok, C} = riak:client_connect(Node), - F = fun(N, Acc) -> - Obj = riak_object:new(Bucket, <>, - <>), - try C:put(Obj, W) of - ok -> - Acc; - Other -> - [{N, Other} | Acc] - catch - What:Why -> - [{N, {What, Why}} | Acc] - end - end, - lists:foldl(F, [], lists:seq(Start, End)). +when is_binary(CommonValBin) -> +rt:wait_for_service(Node, riak_kv), +{ok, C} = riak:client_connect(Node), +F = fun(N, Acc) -> + Obj = riak_object:new(Bucket, <>, + <>), + try C:put(Obj, W) of + ok -> + Acc; + Other -> + [{N, Other} | Acc] + catch + What:Why -> + [{N, {What, Why}} | Acc] + end +end, +lists:foldl(F, [], lists:seq(Start, End)). systest_read(Node, Size) -> - systest_read(Node, Size, 2). +systest_read(Node, Size, 2). systest_read(Node, Size, R) -> - systest_read(Node, 1, Size, <<"systest">>, R). +systest_read(Node, 1, Size, <<"systest">>, R). systest_read(Node, Start, End, Bucket, R) -> - systest_read(Node, Start, End, Bucket, R, <<>>). +systest_read(Node, Start, End, Bucket, R, <<>>). systest_read(Node, Start, End, Bucket, R, CommonValBin) - when is_binary(CommonValBin) -> - systest_read(Node, Start, End, Bucket, R, CommonValBin, false). +when is_binary(CommonValBin) -> +systest_read(Node, Start, End, Bucket, R, CommonValBin, false). %% Read and verify the values of objects written with %% `systest_write'. The `SquashSiblings' parameter exists to @@ -1236,165 +1236,165 @@ systest_read(Node, Start, End, Bucket, R, CommonValBin) %% fails to send an acknowledgment within the timeout window and %% another put request is issued. systest_read(Node, Start, End, Bucket, R, CommonValBin, SquashSiblings) - when is_binary(CommonValBin) -> - rt:wait_for_service(Node, riak_kv), - {ok, C} = riak:client_connect(Node), - lists:foldl(systest_read_fold_fun(C, Bucket, R, CommonValBin, SquashSiblings), - [], - lists:seq(Start, End)). +when is_binary(CommonValBin) -> +rt:wait_for_service(Node, riak_kv), +{ok, C} = riak:client_connect(Node), +lists:foldl(systest_read_fold_fun(C, Bucket, R, CommonValBin, SquashSiblings), + [], + lists:seq(Start, End)). systest_read_fold_fun(C, Bucket, R, CommonValBin, SquashSiblings) -> - fun(N, Acc) -> - GetRes = C:get(Bucket, <>, R), - Val = object_value(GetRes, SquashSiblings), - update_acc(value_matches(Val, N, CommonValBin), Val, N, Acc) - end. +fun(N, Acc) -> + GetRes = C:get(Bucket, <>, R), + Val = object_value(GetRes, SquashSiblings), + update_acc(value_matches(Val, N, CommonValBin), Val, N, Acc) +end. object_value({error, _}=Error, _) -> - Error; +Error; object_value({ok, Obj}, SquashSiblings) -> - object_value(riak_object:value_count(Obj), Obj, SquashSiblings). +object_value(riak_object:value_count(Obj), Obj, SquashSiblings). object_value(1, Obj, _SquashSiblings) -> - riak_object:get_value(Obj); +riak_object:get_value(Obj); object_value(_ValueCount, Obj, false) -> - riak_object:get_value(Obj); +riak_object:get_value(Obj); object_value(_ValueCount, Obj, true) -> - lager:debug("Siblings detected for ~p:~p", [riak_object:bucket(Obj), riak_object:key(Obj)]), - Contents = riak_object:get_contents(Obj), - case lists:foldl(fun sibling_compare/2, {true, undefined}, Contents) of - {true, {_, _, _, Value}} -> - lager:debug("Siblings determined to be a single value"), - Value; - {false, _} -> - {error, siblings} - end. +lager:debug("Siblings detected for ~p:~p", [riak_object:bucket(Obj), riak_object:key(Obj)]), +Contents = riak_object:get_contents(Obj), +case lists:foldl(fun sibling_compare/2, {true, undefined}, Contents) of +{true, {_, _, _, Value}} -> + lager:debug("Siblings determined to be a single value"), + Value; +{false, _} -> + {error, siblings} +end. sibling_compare({MetaData, Value}, {true, undefined}) -> - Dot = case dict:find(<<"dot">>, MetaData) of - {ok, DotVal} -> - DotVal; - error -> - {error, no_dot} - end, - VTag = dict:fetch(<<"X-Riak-VTag">>, MetaData), - LastMod = dict:fetch(<<"X-Riak-Last-Modified">>, MetaData), - {true, {element(2, Dot), VTag, LastMod, Value}}; +Dot = case dict:find(<<"dot">>, MetaData) of + {ok, DotVal} -> + DotVal; + error -> + {error, no_dot} + end, +VTag = dict:fetch(<<"X-Riak-VTag">>, MetaData), +LastMod = dict:fetch(<<"X-Riak-Last-Modified">>, MetaData), +{true, {element(2, Dot), VTag, LastMod, Value}}; sibling_compare(_, {false, _}=InvalidMatch) -> - InvalidMatch; +InvalidMatch; sibling_compare({MetaData, Value}, {true, PreviousElements}) -> - Dot = case dict:find(<<"dot">>, MetaData) of - {ok, DotVal} -> - DotVal; - error -> - {error, no_dot} - end, - VTag = dict:fetch(<<"X-Riak-VTag">>, MetaData), - LastMod = dict:fetch(<<"X-Riak-Last-Modified">>, MetaData), - ComparisonElements = {element(2, Dot), VTag, LastMod, Value}, - {ComparisonElements =:= PreviousElements, ComparisonElements}. +Dot = case dict:find(<<"dot">>, MetaData) of + {ok, DotVal} -> + DotVal; + error -> + {error, no_dot} + end, +VTag = dict:fetch(<<"X-Riak-VTag">>, MetaData), +LastMod = dict:fetch(<<"X-Riak-Last-Modified">>, MetaData), +ComparisonElements = {element(2, Dot), VTag, LastMod, Value}, +{ComparisonElements =:= PreviousElements, ComparisonElements}. value_matches(<>, N, CommonValBin) -> - true; +true; value_matches(_WrongVal, _N, _CommonValBin) -> - false. +false. update_acc(true, _, _, Acc) -> - Acc; +Acc; update_acc(false, {error, _}=Val, N, Acc) -> - [{N, Val} | Acc]; +[{N, Val} | Acc]; update_acc(false, Val, N, Acc) -> - [{N, {wrong_val, Val}} | Acc]. +[{N, {wrong_val, Val}} | Acc]. verify_systest_value(N, Acc, CommonValBin, Obj) -> - Values = riak_object:get_values(Obj), - Res = [begin - case V of - <> -> - ok; - _WrongVal -> - wrong_val - end - end || V <- Values], - case lists:any(fun(X) -> X =:= ok end, Res) of - true -> - Acc; - false -> - [{N, {wrong_val, hd(Values)}} | Acc] - end. +Values = riak_object:get_values(Obj), +Res = [begin + case V of + <> -> + ok; + _WrongVal -> + wrong_val + end + end || V <- Values], +case lists:any(fun(X) -> X =:= ok end, Res) of +true -> + Acc; +false -> + [{N, {wrong_val, hd(Values)}} | Acc] +end. % @doc Reads a single replica of a value. This issues a get command directly % to the vnode handling the Nth primary partition of the object's preflist. get_replica(Node, Bucket, Key, I, N) -> - BKey = {Bucket, Key}, - Chash = rpc:call(Node, riak_core_util, chash_key, [BKey]), - Pl = rpc:call(Node, riak_core_apl, get_primary_apl, [Chash, N, riak_kv]), - {{Partition, PNode}, primary} = lists:nth(I, Pl), - Ref = Reqid = make_ref(), - Sender = {raw, Ref, self()}, - rpc:call(PNode, riak_kv_vnode, get, - [{Partition, PNode}, BKey, Ref, Sender]), - receive - {Ref, {r, Result, _, Reqid}} -> - Result; - {Ref, Reply} -> - Reply - after - 60000 -> - lager:error("Replica ~p get for ~p/~p timed out", - [I, Bucket, Key]), - ?assert(false) - end. +BKey = {Bucket, Key}, +Chash = rpc:call(Node, riak_core_util, chash_key, [BKey]), +Pl = rpc:call(Node, riak_core_apl, get_primary_apl, [Chash, N, riak_kv]), +{{Partition, PNode}, primary} = lists:nth(I, Pl), +Ref = Reqid = make_ref(), +Sender = {raw, Ref, self()}, +rpc:call(PNode, riak_kv_vnode, get, + [{Partition, PNode}, BKey, Ref, Sender]), +receive +{Ref, {r, Result, _, Reqid}} -> + Result; +{Ref, Reply} -> + Reply +after +60000 -> + lager:error("Replica ~p get for ~p/~p timed out", + [I, Bucket, Key]), + ?assert(false) +end. %%%=================================================================== %% @doc PBC-based version of {@link systest_write/1} pbc_systest_write(Node, Size) -> - pbc_systest_write(Node, Size, 2). +pbc_systest_write(Node, Size, 2). pbc_systest_write(Node, Size, W) -> - pbc_systest_write(Node, 1, Size, <<"systest">>, W). +pbc_systest_write(Node, 1, Size, <<"systest">>, W). pbc_systest_write(Node, Start, End, Bucket, W) -> - rt:wait_for_service(Node, riak_kv), - Pid = pbc(Node), - F = fun(N, Acc) -> - Obj = riakc_obj:new(Bucket, <>, <>), - try riakc_pb_socket:put(Pid, Obj, W) of - ok -> - Acc; - Other -> - [{N, Other} | Acc] - catch - What:Why -> - [{N, {What, Why}} | Acc] - end - end, - lists:foldl(F, [], lists:seq(Start, End)). +rt:wait_for_service(Node, riak_kv), +Pid = pbc(Node), +F = fun(N, Acc) -> + Obj = riakc_obj:new(Bucket, <>, <>), + try riakc_pb_socket:put(Pid, Obj, W) of + ok -> + Acc; + Other -> + [{N, Other} | Acc] + catch + What:Why -> + [{N, {What, Why}} | Acc] + end +end, +lists:foldl(F, [], lists:seq(Start, End)). pbc_systest_read(Node, Size) -> - pbc_systest_read(Node, Size, 2). +pbc_systest_read(Node, Size, 2). pbc_systest_read(Node, Size, R) -> - pbc_systest_read(Node, 1, Size, <<"systest">>, R). +pbc_systest_read(Node, 1, Size, <<"systest">>, R). pbc_systest_read(Node, Start, End, Bucket, R) -> - rt:wait_for_service(Node, riak_kv), - Pid = pbc(Node), - F = fun(N, Acc) -> - case riakc_pb_socket:get(Pid, Bucket, <>, R) of - {ok, Obj} -> - case riakc_obj:get_value(Obj) of - <> -> - Acc; - WrongVal -> - [{N, {wrong_val, WrongVal}} | Acc] - end; - Other -> - [{N, Other} | Acc] - end - end, - lists:foldl(F, [], lists:seq(Start, End)). +rt:wait_for_service(Node, riak_kv), +Pid = pbc(Node), +F = fun(N, Acc) -> + case riakc_pb_socket:get(Pid, Bucket, <>, R) of + {ok, Obj} -> + case riakc_obj:get_value(Obj) of + <> -> + Acc; + WrongVal -> + [{N, {wrong_val, WrongVal}} | Acc] + end; + Other -> + [{N, Other} | Acc] + end +end, +lists:foldl(F, [], lists:seq(Start, End)). %%%=================================================================== %%% PBC & HTTPC Functions @@ -1403,128 +1403,128 @@ pbc_systest_read(Node, Start, End, Bucket, R) -> %% @doc get me a protobuf client process and hold the mayo! -spec pbc(node()) -> pid(). pbc(Node) -> - pbc(Node, [{auto_reconnect, true}]). +pbc(Node, [{auto_reconnect, true}]). -spec pbc(node(), proplists:proplist()) -> pid(). pbc(Node, Options) -> - rt:wait_for_service(Node, riak_kv), - ConnInfo = proplists:get_value(Node, connection_info([Node])), - {IP, PBPort} = proplists:get_value(pb, ConnInfo), - {ok, Pid} = riakc_pb_socket:start_link(IP, PBPort, Options), - Pid. +rt:wait_for_service(Node, riak_kv), +ConnInfo = proplists:get_value(Node, connection_info([Node])), +{IP, PBPort} = proplists:get_value(pb, ConnInfo), +{ok, Pid} = riakc_pb_socket:start_link(IP, PBPort, Options), +Pid. %% @doc does a read via the erlang protobuf client -spec pbc_read(pid(), binary(), binary()) -> binary(). pbc_read(Pid, Bucket, Key) -> - pbc_read(Pid, Bucket, Key, []). +pbc_read(Pid, Bucket, Key, []). -spec pbc_read(pid(), binary(), binary(), [any()]) -> binary(). pbc_read(Pid, Bucket, Key, Options) -> - {ok, Value} = riakc_pb_socket:get(Pid, Bucket, Key, Options), - Value. +{ok, Value} = riakc_pb_socket:get(Pid, Bucket, Key, Options), +Value. -spec pbc_read_check(pid(), binary(), binary(), [any()]) -> boolean(). pbc_read_check(Pid, Bucket, Key, Allowed) -> - pbc_read_check(Pid, Bucket, Key, Allowed, []). +pbc_read_check(Pid, Bucket, Key, Allowed, []). -spec pbc_read_check(pid(), binary(), binary(), [any()], [any()]) -> boolean(). pbc_read_check(Pid, Bucket, Key, Allowed, Options) -> - case riakc_pb_socket:get(Pid, Bucket, Key, Options) of - {ok, _} -> - true = lists:member(ok, Allowed); - Other -> - lists:member(Other, Allowed) orelse throw({failed, Other, Allowed}) - end. +case riakc_pb_socket:get(Pid, Bucket, Key, Options) of +{ok, _} -> + true = lists:member(ok, Allowed); +Other -> + lists:member(Other, Allowed) orelse throw({failed, Other, Allowed}) +end. %% @doc does a write via the erlang protobuf client -spec pbc_write(pid(), binary(), binary(), binary()) -> atom(). pbc_write(Pid, Bucket, Key, Value) -> - Object = riakc_obj:new(Bucket, Key, Value), - riakc_pb_socket:put(Pid, Object). +Object = riakc_obj:new(Bucket, Key, Value), +riakc_pb_socket:put(Pid, Object). %% @doc does a write via the erlang protobuf client plus content-type -spec pbc_write(pid(), binary(), binary(), binary(), list()) -> atom(). pbc_write(Pid, Bucket, Key, Value, CT) -> - Object = riakc_obj:new(Bucket, Key, Value, CT), - riakc_pb_socket:put(Pid, Object). +Object = riakc_obj:new(Bucket, Key, Value, CT), +riakc_pb_socket:put(Pid, Object). %% @doc sets a bucket property/properties via the erlang protobuf client -spec pbc_set_bucket_prop(pid(), binary(), [proplists:property()]) -> atom(). pbc_set_bucket_prop(Pid, Bucket, PropList) -> - riakc_pb_socket:set_bucket(Pid, Bucket, PropList). +riakc_pb_socket:set_bucket(Pid, Bucket, PropList). %% @doc Puts the contents of the given file into the given bucket using the %% filename as a key and assuming a plain text content type. pbc_put_file(Pid, Bucket, Key, Filename) -> - {ok, Contents} = file:read_file(Filename), - riakc_pb_socket:put(Pid, riakc_obj:new(Bucket, Key, Contents, "text/plain")). +{ok, Contents} = file:read_file(Filename), +riakc_pb_socket:put(Pid, riakc_obj:new(Bucket, Key, Contents, "text/plain")). %% @doc Puts all files in the given directory into the given bucket using the %% filename as a key and assuming a plain text content type. pbc_put_dir(Pid, Bucket, Dir) -> - lager:info("Putting files from dir ~p into bucket ~p", [Dir, Bucket]), - {ok, Files} = file:list_dir(Dir), - [pbc_put_file(Pid, Bucket, list_to_binary(F), filename:join([Dir, F])) - || F <- Files]. +lager:info("Putting files from dir ~p into bucket ~p", [Dir, Bucket]), +{ok, Files} = file:list_dir(Dir), +[pbc_put_file(Pid, Bucket, list_to_binary(F), filename:join([Dir, F])) +|| F <- Files]. %% @doc True if the given keys have been really, really deleted. %% Useful when you care about the keys not being there. Delete simply writes %% tombstones under the given keys, so those are still seen by key folding %% operations. pbc_really_deleted(Pid, Bucket, Keys) -> - StillThere = - fun(K) -> - Res = riakc_pb_socket:get(Pid, Bucket, K, - [{r, 1}, - {notfound_ok, false}, - {basic_quorum, false}, - deletedvclock]), - case Res of - {error, notfound} -> - false; - _ -> - %% Tombstone still around - true - end - end, - [] == lists:filter(StillThere, Keys). +StillThere = +fun(K) -> + Res = riakc_pb_socket:get(Pid, Bucket, K, + [{r, 1}, + {notfound_ok, false}, + {basic_quorum, false}, + deletedvclock]), + case Res of + {error, notfound} -> + false; + _ -> + %% Tombstone still around + true + end +end, +[] == lists:filter(StillThere, Keys). %% @doc Returns HTTPS URL information for a list of Nodes https_url(Nodes) when is_list(Nodes) -> - [begin - {Host, Port} = orddict:fetch(https, Connections), - lists:flatten(io_lib:format("https://~s:~b", [Host, Port])) - end || {_Node, Connections} <- connection_info(Nodes)]; +[begin + {Host, Port} = orddict:fetch(https, Connections), + lists:flatten(io_lib:format("https://~s:~b", [Host, Port])) +end || {_Node, Connections} <- connection_info(Nodes)]; https_url(Node) -> - hd(https_url([Node])). +hd(https_url([Node])). %% @doc Returns HTTP URL information for a list of Nodes http_url(Nodes) when is_list(Nodes) -> - [begin - {Host, Port} = orddict:fetch(http, Connections), - lists:flatten(io_lib:format("http://~s:~b", [Host, Port])) - end || {_Node, Connections} <- connection_info(Nodes)]; +[begin + {Host, Port} = orddict:fetch(http, Connections), + lists:flatten(io_lib:format("http://~s:~b", [Host, Port])) +end || {_Node, Connections} <- connection_info(Nodes)]; http_url(Node) -> - hd(http_url([Node])). +hd(http_url([Node])). %% @doc get me an http client. -spec httpc(node()) -> term(). httpc(Node) -> - rt:wait_for_service(Node, riak_kv), - {ok, [{IP, Port}]} = get_http_conn_info(Node), - rhc:create(IP, Port, "riak", []). +rt:wait_for_service(Node, riak_kv), +{ok, [{IP, Port}]} = get_http_conn_info(Node), +rhc:create(IP, Port, "riak", []). %% @doc does a read via the http erlang client. -spec httpc_read(term(), binary(), binary()) -> binary(). httpc_read(C, Bucket, Key) -> - {_, Value} = rhc:get(C, Bucket, Key), - Value. +{_, Value} = rhc:get(C, Bucket, Key), +Value. %% @doc does a write via the http erlang client. -spec httpc_write(term(), binary(), binary(), binary()) -> atom(). httpc_write(C, Bucket, Key, Value) -> - Object = riakc_obj:new(Bucket, Key, Value), - rhc:put(C, Object). +Object = riakc_obj:new(Bucket, Key, Value), +rhc:put(C, Object). %%%=================================================================== %%% Command Line Functions @@ -1532,26 +1532,26 @@ httpc_write(C, Bucket, Key, Value) -> %% @doc Call 'bin/riak-admin' command on `Node' with arguments `Args' admin(Node, Args) -> - admin(Node, Args, []). +admin(Node, Args, []). %% @doc Call 'bin/riak-admin' command on `Node' with arguments `Args'. %% The third parameter is a list of options. Valid options are: %% * `return_exit_code' - Return the exit code along with the command output admin(Node, Args, Options) -> - ?HARNESS:admin(Node, Args, Options). +?HARNESS:admin(Node, Args, Options). %% @doc Call 'bin/riak' command on `Node' with arguments `Args' riak(Node, Args) -> - ?HARNESS:riak(Node, Args). +?HARNESS:riak(Node, Args). %% @doc Call 'bin/riak-repl' command on `Node' with arguments `Args' riak_repl(Node, Args) -> - ?HARNESS:riak_repl(Node, Args). +?HARNESS:riak_repl(Node, Args). search_cmd(Node, Args) -> - {ok, Cwd} = file:get_cwd(), - rpc:call(Node, riak_search_cmd, command, [[Cwd | Args]]). +{ok, Cwd} = file:get_cwd(), +rpc:call(Node, riak_search_cmd, command, [[Cwd | Args]]). %% @doc Runs `riak attach' on a specific node, and tests for the expected behavoir. %% Here's an example: ``` @@ -1567,17 +1567,17 @@ search_cmd(Node, Args) -> %% expect will process based on the output following the sent data. %% attach(Node, Expected) -> - ?HARNESS:attach(Node, Expected). +?HARNESS:attach(Node, Expected). %% @doc Runs 'riak attach-direct' on a specific node %% @see rt:attach/2 attach_direct(Node, Expected) -> - ?HARNESS:attach_direct(Node, Expected). +?HARNESS:attach_direct(Node, Expected). %% @doc Runs `riak console' on a specific node %% @see rt:attach/2 console(Node, Expected) -> - ?HARNESS:console(Node, Expected). +?HARNESS:console(Node, Expected). %%%=================================================================== %%% Search @@ -1587,8 +1587,8 @@ console(Node, Expected) -> %% in the cluster may be used as the change is propagated via the %% Ring. enable_search_hook(Node, Bucket) when is_binary(Bucket) -> - lager:info("Installing search hook for bucket ~p", [Bucket]), - ?assertEqual(ok, rpc:call(Node, riak_search_kv_hook, install, [Bucket])). +lager:info("Installing search hook for bucket ~p", [Bucket]), +?assertEqual(ok, rpc:call(Node, riak_search_kv_hook, install, [Bucket])). %%%=================================================================== %%% Test harness setup, configuration, and internal utilities @@ -1603,161 +1603,161 @@ enable_search_hook(Node, Bucket) when is_binary(Bucket) -> %% list return type should be considered an error. -spec set_backend(atom()) -> atom()|[atom()]. set_backend(Backend) -> - set_backend(Backend, []). +set_backend(Backend, []). -spec set_backend(atom(), [{atom(), term()}]) -> atom()|[atom()]. set_backend(bitcask, _) -> - set_backend(riak_kv_bitcask_backend); +set_backend(riak_kv_bitcask_backend); set_backend(eleveldb, _) -> - set_backend(riak_kv_eleveldb_backend); +set_backend(riak_kv_eleveldb_backend); set_backend(memory, _) -> - set_backend(riak_kv_memory_backend); +set_backend(riak_kv_memory_backend); set_backend(multi, Extras) -> - set_backend(riak_kv_multi_backend, Extras); +set_backend(riak_kv_multi_backend, Extras); set_backend(Backend, _) when Backend == riak_kv_bitcask_backend; Backend == riak_kv_eleveldb_backend; Backend == riak_kv_memory_backend -> - lager:info("rt:set_backend(~p)", [Backend]), - update_app_config(all, [{riak_kv, [{storage_backend, Backend}]}]), - get_backends(); +lager:info("rt:set_backend(~p)", [Backend]), +update_app_config(all, [{riak_kv, [{storage_backend, Backend}]}]), +get_backends(); set_backend(Backend, Extras) when Backend == riak_kv_multi_backend -> - MultiConfig = proplists:get_value(multi_config, Extras, default), - Config = make_multi_backend_config(MultiConfig), - update_app_config(all, [{riak_kv, Config}]), - get_backends(); +MultiConfig = proplists:get_value(multi_config, Extras, default), +Config = make_multi_backend_config(MultiConfig), +update_app_config(all, [{riak_kv, Config}]), +get_backends(); set_backend(Other, _) -> - lager:warning("rt:set_backend doesn't recognize ~p as a legit backend, using the default.", [Other]), - get_backends(). +lager:warning("rt:set_backend doesn't recognize ~p as a legit backend, using the default.", [Other]), +get_backends(). make_multi_backend_config(default) -> - [{storage_backend, riak_kv_multi_backend}, - {multi_backend_default, <<"eleveldb1">>}, - {multi_backend, [{<<"eleveldb1">>, riak_kv_eleveldb_backend, []}, - {<<"memory1">>, riak_kv_memory_backend, []}, - {<<"bitcask1">>, riak_kv_bitcask_backend, []}]}]; +[{storage_backend, riak_kv_multi_backend}, +{multi_backend_default, <<"eleveldb1">>}, +{multi_backend, [{<<"eleveldb1">>, riak_kv_eleveldb_backend, []}, + {<<"memory1">>, riak_kv_memory_backend, []}, + {<<"bitcask1">>, riak_kv_bitcask_backend, []}]}]; make_multi_backend_config(indexmix) -> - [{storage_backend, riak_kv_multi_backend}, - {multi_backend_default, <<"eleveldb1">>}, - {multi_backend, [{<<"eleveldb1">>, riak_kv_eleveldb_backend, []}, - {<<"memory1">>, riak_kv_memory_backend, []}]}]; +[{storage_backend, riak_kv_multi_backend}, +{multi_backend_default, <<"eleveldb1">>}, +{multi_backend, [{<<"eleveldb1">>, riak_kv_eleveldb_backend, []}, + {<<"memory1">>, riak_kv_memory_backend, []}]}]; make_multi_backend_config(Other) -> - lager:warning("rt:set_multi_backend doesn't recognize ~p as legit multi-backend config, using default", [Other]), - make_multi_backend_config(default). +lager:warning("rt:set_multi_backend doesn't recognize ~p as legit multi-backend config, using default", [Other]), +make_multi_backend_config(default). get_backends() -> - Backends = ?HARNESS:get_backends(), - case Backends of - [riak_kv_bitcask_backend] -> bitcask; - [riak_kv_eleveldb_backend] -> eleveldb; - [riak_kv_memory_backend] -> memory; - [Other] -> Other; - MoreThanOne -> MoreThanOne - end. +Backends = ?HARNESS:get_backends(), +case Backends of +[riak_kv_bitcask_backend] -> bitcask; +[riak_kv_eleveldb_backend] -> eleveldb; +[riak_kv_memory_backend] -> memory; +[Other] -> Other; +MoreThanOne -> MoreThanOne +end. -spec get_backend([proplists:property()]) -> atom() | error. get_backend(AppConfigProplist) -> - case kvc:path('riak_kv.storage_backend', AppConfigProplist) of - [] -> error; - Backend -> Backend - end. +case kvc:path('riak_kv.storage_backend', AppConfigProplist) of +[] -> error; +Backend -> Backend +end. %% @doc Gets the current version under test. In the case of an upgrade test %% or something like that, it's the version you're upgrading to. -spec get_version() -> binary(). get_version() -> - ?HARNESS:get_version(). +?HARNESS:get_version(). %% @doc outputs some useful information about nodes that are up whats_up() -> - ?HARNESS:whats_up(). +?HARNESS:whats_up(). -spec get_ip(node()) -> string(). get_ip(Node) -> - ?HARNESS:get_ip(Node). +?HARNESS:get_ip(Node). %% @doc Log a message to the console of the specified test nodes. %% Messages are prefixed by the string "---riak_test--- " %% Uses lager:info/1 'Fmt' semantics log_to_nodes(Nodes, Fmt) -> - log_to_nodes(Nodes, Fmt, []). +log_to_nodes(Nodes, Fmt, []). %% @doc Log a message to the console of the specified test nodes. %% Messages are prefixed by the string "---riak_test--- " %% Uses lager:info/2 'LFmt' and 'LArgs' semantics log_to_nodes(Nodes0, LFmt, LArgs) -> - %% This logs to a node's info level, but if riak_test is running - %% at debug level, we want to know when we send this and what - %% we're saying - Nodes = lists:flatten(Nodes0), - lager:debug("log_to_nodes: " ++ LFmt, LArgs), - Module = lager, - Function = log, - Meta = [], - Args = case LArgs of - [] -> [info, Meta, "---riak_test--- " ++ LFmt]; - _ -> [info, Meta, "---riak_test--- " ++ LFmt, LArgs] - end, - [rpc:call(Node, Module, Function, Args) || Node <- lists:flatten(Nodes)]. +%% This logs to a node's info level, but if riak_test is running +%% at debug level, we want to know when we send this and what +%% we're saying +Nodes = lists:flatten(Nodes0), +lager:debug("log_to_nodes: " ++ LFmt, LArgs), +Module = lager, +Function = log, +Meta = [], +Args = case LArgs of + [] -> [info, Meta, "---riak_test--- " ++ LFmt]; + _ -> [info, Meta, "---riak_test--- " ++ LFmt, LArgs] + end, +[rpc:call(Node, Module, Function, Args) || Node <- lists:flatten(Nodes)]. %% @private utility function pmap(F, L) -> - Parent = self(), - lists:foldl( - fun(X, N) -> - spawn_link(fun() -> - Parent ! {pmap, N, F(X)} - end), - N+1 - end, 0, L), - L2 = [receive {pmap, N, R} -> {N,R} end || _ <- L], - {_, L3} = lists:unzip(lists:keysort(1, L2)), - L3. +Parent = self(), +lists:foldl( +fun(X, N) -> + spawn_link(fun() -> + Parent ! {pmap, N, F(X)} + end), + N+1 +end, 0, L), +L2 = [receive {pmap, N, R} -> {N,R} end || _ <- L], +{_, L3} = lists:unzip(lists:keysort(1, L2)), +L3. %% @private setup_harness(Test, Args) -> - ?HARNESS:setup_harness(Test, Args). +?HARNESS:setup_harness(Test, Args). %% @doc Downloads any extant log files from the harness's running %% nodes. get_node_logs() -> - ?HARNESS:get_node_logs(). +?HARNESS:get_node_logs(). check_ibrowse() -> - try sys:get_status(ibrowse) of - {status, _Pid, {module, gen_server} ,_} -> ok - catch - Throws -> - lager:error("ibrowse error ~p", [Throws]), - lager:error("Restarting ibrowse"), - application:stop(ibrowse), - application:start(ibrowse) - end. +try sys:get_status(ibrowse) of +{status, _Pid, {module, gen_server} ,_} -> ok +catch +Throws -> + lager:error("ibrowse error ~p", [Throws]), + lager:error("Restarting ibrowse"), + application:stop(ibrowse), + application:start(ibrowse) +end. post_result(TestResult, #rt_webhook{url=URL, headers=HookHeaders, name=Name}) -> - lager:info("Posting result to ~s ~s", [Name, URL]), - try ibrowse:send_req(URL, - [{"Content-Type", "application/json"}], - post, - mochijson2:encode(TestResult), - [{content_type, "application/json"}] ++ HookHeaders, - 300000) of %% 5 minute timeout - - {ok, RC=[$2|_], Headers, _Body} -> - {ok, RC, Headers}; - {ok, ResponseCode, Headers, Body} -> - lager:info("Test Result did not generate the expected 2XX HTTP response code."), - lager:debug("Post"), - lager:debug("Response Code: ~p", [ResponseCode]), - lager:debug("Headers: ~p", [Headers]), - lager:debug("Body: ~p", [Body]), - error; - X -> - lager:warning("Some error POSTing test result: ~p", [X]), - error - catch - Class:Reason -> - lager:error("Error reporting to ~s. ~p:~p", [Name, Class, Reason]), - lager:error("Payload: ~p", [TestResult]), - error - end. +lager:info("Posting result to ~s ~s", [Name, URL]), +try ibrowse:send_req(URL, + [{"Content-Type", "application/json"}], + post, + mochijson2:encode(TestResult), + [{content_type, "application/json"}] ++ HookHeaders, + 300000) of %% 5 minute timeout + +{ok, RC=[$2|_], Headers, _Body} -> + {ok, RC, Headers}; +{ok, ResponseCode, Headers, Body} -> + lager:info("Test Result did not generate the expected 2XX HTTP response code."), + lager:debug("Post"), + lager:debug("Response Code: ~p", [ResponseCode]), + lager:debug("Headers: ~p", [Headers]), + lager:debug("Body: ~p", [Body]), + error; +X -> + lager:warning("Some error POSTing test result: ~p", [X]), + error +catch +Class:Reason -> + lager:error("Error reporting to ~s. ~p:~p", [Name, Class, Reason]), + lager:error("Payload: ~p", [TestResult]), + error +end. %%%=================================================================== %%% Bucket Types Functions @@ -1765,37 +1765,37 @@ post_result(TestResult, #rt_webhook{url=URL, headers=HookHeaders, name=Name}) -> %% @doc create and immediately activate a bucket type create_and_activate_bucket_type(Node, Type, Props) -> - ok = rpc:call(Node, riak_core_bucket_type, create, [Type, Props]), - wait_until_bucket_type_status(Type, ready, Node), - ok = rpc:call(Node, riak_core_bucket_type, activate, [Type]), - wait_until_bucket_type_status(Type, active, Node). +ok = rpc:call(Node, riak_core_bucket_type, create, [Type, Props]), +wait_until_bucket_type_status(Type, ready, Node), +ok = rpc:call(Node, riak_core_bucket_type, activate, [Type]), +wait_until_bucket_type_status(Type, active, Node). wait_until_bucket_type_status(Type, ExpectedStatus, Nodes) when is_list(Nodes) -> - [wait_until_bucket_type_status(Type, ExpectedStatus, Node) || Node <- Nodes]; +[wait_until_bucket_type_status(Type, ExpectedStatus, Node) || Node <- Nodes]; wait_until_bucket_type_status(Type, ExpectedStatus, Node) -> - F = fun() -> - ActualStatus = rpc:call(Node, riak_core_bucket_type, status, [Type]), - ExpectedStatus =:= ActualStatus - end, - ?assertEqual(ok, rt:wait_until(F)). +F = fun() -> + ActualStatus = rpc:call(Node, riak_core_bucket_type, status, [Type]), + ExpectedStatus =:= ActualStatus +end, +?assertEqual(ok, rt:wait_until(F)). -spec bucket_type_visible([atom()], binary()|{binary(), binary()}) -> boolean(). bucket_type_visible(Nodes, Type) -> - MaxTime = rt_config:get(rt_max_wait_time), - IsVisible = fun erlang:is_list/1, - {Res, NodesDown} = rpc:multicall(Nodes, riak_core_bucket_type, get, [Type], MaxTime), - NodesDown == [] andalso lists:all(IsVisible, Res). +MaxTime = rt_config:get(rt_max_wait_time), +IsVisible = fun erlang:is_list/1, +{Res, NodesDown} = rpc:multicall(Nodes, riak_core_bucket_type, get, [Type], MaxTime), +NodesDown == [] andalso lists:all(IsVisible, Res). wait_until_bucket_type_visible(Nodes, Type) -> - F = fun() -> bucket_type_visible(Nodes, Type) end, - ?assertEqual(ok, rt:wait_until(F)). +F = fun() -> bucket_type_visible(Nodes, Type) end, +?assertEqual(ok, rt:wait_until(F)). -spec see_bucket_props([atom()], binary()|{binary(), binary()}, - proplists:proplist()) -> boolean(). + proplists:proplist()) -> boolean(). see_bucket_props(Nodes, Bucket, ExpectProps) -> - MaxTime = rt_config:get(rt_max_wait_time), - IsBad = fun({badrpc, _}) -> true; - ({error, _}) -> true; +MaxTime = rt_config:get(rt_max_wait_time), +IsBad = fun({badrpc, _}) -> true; + ({error, _}) -> true; (Res) when is_list(Res) -> false end, HasProps = fun(ResProps) -> From c7c8dfc9016176faa53ea1e51ad751b09001eb45 Mon Sep 17 00:00:00 2001 From: Kelly McLaughlin Date: Tue, 29 Jul 2014 10:41:39 -0600 Subject: [PATCH 009/157] Remove direct calls to rtdev harness from replication tests --- tests/replication/repl_util.erl | 2 +- tests/replication/replication.erl | 2 +- tests/replication/replication2_upgrade.erl | 2 +- tests/replication/replication_upgrade.erl | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/replication/repl_util.erl b/tests/replication/repl_util.erl index a7be23f35..cdb5f9c02 100644 --- a/tests/replication/repl_util.erl +++ b/tests/replication/repl_util.erl @@ -469,7 +469,7 @@ do_write(Node, Start, End, Bucket, W) -> %% does the node meet the version requirement? node_has_version(Node, Version) -> - NodeVersion = rtdev:node_version(rtdev:node_id(Node)), + NodeVersion = rt:node_version(rt:node_id(Node)), case NodeVersion of current -> %% current always satisfies any version check diff --git a/tests/replication/replication.erl b/tests/replication/replication.erl index c2463265c..2acf03a88 100644 --- a/tests/replication/replication.erl +++ b/tests/replication/replication.erl @@ -519,7 +519,7 @@ start_and_wait_until_fullsync_complete(Node, Retries) -> lager:info("waiting for fullsync count to be ~p", [Count]), lager:info("Starting fullsync on ~p (~p)", [Node, - rtdev:node_version(rtdev:node_id(Node))]), + rt:node_version(rt:node_id(Node))]), rpc:call(Node, riak_repl_console, start_fullsync, [[]]), %% sleep because of the old bug where stats will crash if you call it too diff --git a/tests/replication/replication2_upgrade.erl b/tests/replication/replication2_upgrade.erl index fea58b752..987ccf08d 100644 --- a/tests/replication/replication2_upgrade.erl +++ b/tests/replication/replication2_upgrade.erl @@ -74,7 +74,7 @@ confirm() -> ok = lists:foreach(fun(Node) -> lager:info("Upgrade node: ~p", [Node]), rt:log_to_nodes(Nodes, "Upgrade node: ~p", [Node]), - rtdev:upgrade(Node, current), + rt:upgrade(Node, current), %% The upgrade did a wait for pingable rt:wait_for_service(Node, [riak_kv, riak_pipe, riak_repl]), [rt:wait_until_ring_converged(N) || N <- [ANodes, BNodes]], diff --git a/tests/replication/replication_upgrade.erl b/tests/replication/replication_upgrade.erl index 86d82adb9..10ddf6270 100644 --- a/tests/replication/replication_upgrade.erl +++ b/tests/replication/replication_upgrade.erl @@ -66,7 +66,7 @@ confirm() -> ok = lists:foreach(fun(Node) -> lager:info("Upgrade node: ~p", [Node]), rt:log_to_nodes(Nodes, "Upgrade node: ~p", [Node]), - rtdev:upgrade(Node, current), + rt:upgrade(Node, current), rt:wait_until_pingable(Node), rt:wait_for_service(Node, [riak_kv, riak_pipe, riak_repl]), [rt:wait_until_ring_converged(N) || N <- [ANodes, BNodes]], From 263cd6835c2b09eba2471b2565d5628e0745e535 Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Tue, 29 Jul 2014 18:50:48 -0400 Subject: [PATCH 010/157] Move configuration functions to rt_config. - functions include set_conf/2, set_advanced_conf/2, and update_app_config/2. --- src/rt_config.erl | 36 ++++++++++++++++++- tests/cluster_meta_rmr.erl | 2 +- tests/gh_riak_core_155.erl | 2 +- tests/gh_riak_core_176.erl | 4 +-- tests/http_security.erl | 2 +- tests/overload.erl | 4 +-- tests/replication/repl_bucket_types.erl | 2 +- tests/replication/repl_cancel_fullsync.erl | 2 +- tests/replication/repl_fs_bench.erl | 2 +- tests/replication/repl_fs_stat_caching.erl | 2 +- tests/replication/repl_location_failures.erl | 2 +- tests/replication/repl_rt_cascading_rtq.erl | 2 +- tests/replication/replication.erl | 2 +- .../replication/replication2_connections.erl | 6 ++-- tests/replication/replication2_pg.erl | 8 ++--- tests/replication/replication2_ssl.erl | 6 ++-- .../replication_object_reformat.erl | 6 ++-- tests/replication/replication_ssl.erl | 10 +++--- tests/replication/replication_stats.erl | 2 +- tests/replication/rt_cascading.erl | 2 +- tests/test_cluster.erl | 2 +- tests/verify_capabilities.erl | 6 ++-- tests/verify_dynamic_ring.erl | 4 +-- tests/verify_riak_object_reformat.erl | 2 +- tests/verify_tick_change.erl | 2 +- tests/verify_vclock.erl | 2 +- 26 files changed, 78 insertions(+), 44 deletions(-) diff --git a/src/rt_config.erl b/src/rt_config.erl index 4a916de63..39ef3788e 100644 --- a/src/rt_config.erl +++ b/src/rt_config.erl @@ -28,9 +28,14 @@ get_os_env/1, get_os_env/2, load/2, - set/2 + set/2, + set_conf/2, + set_advanced_conf/2, + update_app_config/2 ]). +-define(HARNESS, (rt_config:get(rt_harness))). + %% @doc Get the value of an OS Environment variable. The arity 1 version of %% this function will fail the test if it is undefined. get_os_env(Var) -> @@ -122,6 +127,35 @@ config_or_os_env(Config, Default) -> V end. + +-spec set_conf(atom(), [{string(), string()}]) -> ok. +set_conf(all, NameValuePairs) -> + ?HARNESS:set_conf(all, NameValuePairs); +set_conf(Node, NameValuePairs) -> + rt:stop(Node), + ?assertEqual(ok, rt:wait_until_unpingable(Node)), + ?HARNESS:set_conf(Node, NameValuePairs), + rt:start(Node). + +-spec set_advanced_conf(atom(), [{string(), string()}]) -> ok. +set_advanced_conf(all, NameValuePairs) -> + ?HARNESS:set_advanced_conf(all, NameValuePairs); +set_advanced_conf(Node, NameValuePairs) -> + rt:stop(Node), + ?assertEqual(ok, rt:wait_until_unpingable(Node)), + ?HARNESS:set_advanced_conf(Node, NameValuePairs), + rt:start(Node). + +%% @doc Rewrite the given node's app.config file, overriding the varialbes +%% in the existing app.config with those in `Config'. +update_app_config(all, Config) -> + ?HARNESS:update_app_config(all, Config); +update_app_config(Node, Config) -> + rt:stop(Node), + ?assertEqual(ok, rt:wait_until_unpingable(Node)), + ?HARNESS:update_app_config(Node, Config), + rt:start(Node). + to_upper(S) -> lists:map(fun char_to_upper/1, S). char_to_upper(C) when C >= $a, C =< $z -> C bxor $\s; char_to_upper(C) -> C. diff --git a/tests/cluster_meta_rmr.erl b/tests/cluster_meta_rmr.erl index bc5fcab11..beabc0714 100644 --- a/tests/cluster_meta_rmr.erl +++ b/tests/cluster_meta_rmr.erl @@ -24,7 +24,7 @@ -define(CM_PREFIX, {test, cm}). confirm() -> - rt:set_conf(all, [{"ring_size", "128"}]), + rt_config:set_conf(all, [{"ring_size", "128"}]), Seed = erlang:now(), lager:info("SEED: ~p", [Seed]), random:seed(Seed), diff --git a/tests/gh_riak_core_155.erl b/tests/gh_riak_core_155.erl index 6a6e7da82..5bd4bb17d 100644 --- a/tests/gh_riak_core_155.erl +++ b/tests/gh_riak_core_155.erl @@ -34,7 +34,7 @@ confirm() -> lager:info("Adding delayed start to app.config"), NewConfig = [{riak_core, [{delayed_start, 1000}]}], - rt:update_app_config(Node, NewConfig), + rt_config:update_app_config(Node, NewConfig), %% Restart node, add intercept that delay proxy startup, and issue gets. %% Gets will come in before proxies started, and should trigger crash. diff --git a/tests/gh_riak_core_176.erl b/tests/gh_riak_core_176.erl index 00821a615..3f7b351e1 100644 --- a/tests/gh_riak_core_176.erl +++ b/tests/gh_riak_core_176.erl @@ -47,7 +47,7 @@ confirm() -> lager:info("Change ~p handoff_ip from ~p to ~p", [Node2, NodeIP, AlternateIP]), NewConfig = [{riak_core, [{handoff_ip, AlternateIP}]}], - rt:update_app_config(Node2, NewConfig), + rt_config:update_app_config(Node2, NewConfig), rt:wait_for_service(Node2, riak_kv), lager:info("Write data to the cluster"), @@ -62,7 +62,7 @@ confirm() -> %% Check 0.0.0.0 address works lager:info("Change ~p handoff_ip to \"0.0.0.0\"", [Node3]), - rt:update_app_config(Node3, + rt_config:update_app_config(Node3, [{riak_core, [{handoff_ip, "0.0.0.0"}]}]), lager:info("Join ~p to the cluster and wait for handoff to finish", diff --git a/tests/http_security.erl b/tests/http_security.erl index 8bf09ddb6..f2ccdafec 100644 --- a/tests/http_security.erl +++ b/tests/http_security.erl @@ -520,7 +520,7 @@ confirm() -> enable_ssl(Node) -> [{http, {IP, Port}}|_] = rt:connection_info(Node), - rt:update_app_config(Node, [{riak_api, [{https, [{IP, + rt_config:update_app_config(Node, [{riak_api, [{https, [{IP, Port+1000}]}]}]), rt:wait_until_pingable(Node), rt:wait_for_service(Node, riak_kv). diff --git a/tests/overload.erl b/tests/overload.erl index 1e473f73b..cd34fd9c6 100644 --- a/tests/overload.erl +++ b/tests/overload.erl @@ -108,7 +108,7 @@ test_vnode_protection(Nodes, BKV, ConsistentType) -> Config = [{riak_core, [{vnode_overload_threshold, ?THRESHOLD}, {vnode_check_interval, 1}]}], rt:pmap(fun(Node) -> - rt:update_app_config(Node, Config) + rt_config:update_app_config(Node, Config) end, Nodes), ProcFun = build_predicate_lt(test_vnode_protection, (?NUM_REQUESTS+1), "ProcFun", "Procs"), QueueFun = build_predicate_lt(test_vnode_protection, (?NUM_REQUESTS), "QueueFun", "QueueSize"), @@ -140,7 +140,7 @@ test_fsm_protection(Nodes, BKV, ConsistentType) -> lager:info("Setting FSM limit to ~b", [?THRESHOLD]), Config = [{riak_kv, [{fsm_limit, ?THRESHOLD}]}], rt:pmap(fun(Node) -> - rt:update_app_config(Node, Config) + rt_config:update_app_config(Node, Config) end, Nodes), ProcFun = build_predicate_lt(test_fsm_protection, (?NUM_REQUESTS), "ProcFun", "Procs"), diff --git a/tests/replication/repl_bucket_types.erl b/tests/replication/repl_bucket_types.erl index aacec149b..d9d444976 100644 --- a/tests/replication/repl_bucket_types.erl +++ b/tests/replication/repl_bucket_types.erl @@ -16,7 +16,7 @@ %% setup(Type) -> - rt:set_conf(all, [{"buckets.default.allow_mult", "false"}]), + rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), {LeaderA, LeaderB, ANodes, BNodes} = ClusterNodes = make_clusters(Type), diff --git a/tests/replication/repl_cancel_fullsync.erl b/tests/replication/repl_cancel_fullsync.erl index 03c69b9ef..5248da7bd 100644 --- a/tests/replication/repl_cancel_fullsync.erl +++ b/tests/replication/repl_cancel_fullsync.erl @@ -33,7 +33,7 @@ %% @doc Ensure we can cancel a fullsync and restart it. confirm() -> - rt:set_advanced_conf(all, ?CONF(5)), + rt_config:set_advanced_conf(all, ?CONF(5)), Nodes = [ANodes, BNodes] = rt:build_clusters([3, 3]), diff --git a/tests/replication/repl_fs_bench.erl b/tests/replication/repl_fs_bench.erl index 657a25aa6..6b2b257e9 100644 --- a/tests/replication/repl_fs_bench.erl +++ b/tests/replication/repl_fs_bench.erl @@ -60,7 +60,7 @@ confirm() -> %% @doc Perform a fullsync, with given latency injected via intercept %% and return times for each fullsync time. fullsync_test(Strategy, Latency) -> - rt:set_advanced_conf(all, ?CONF(Strategy)), + rt_config:set_advanced_conf(all, ?CONF(Strategy)), [ANodes, BNodes] = rt:build_clusters([3, 3]), diff --git a/tests/replication/repl_fs_stat_caching.erl b/tests/replication/repl_fs_stat_caching.erl index 6f187ef49..43f2cf887 100644 --- a/tests/replication/repl_fs_stat_caching.erl +++ b/tests/replication/repl_fs_stat_caching.erl @@ -34,7 +34,7 @@ confirm() -> pass. setup() -> - rt:set_conf(all, [{"buckets.default.allow_mult", "false"}]), + rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), NodeCount = rt_config:get(num_nodes, 6), lager:info("Deploy ~p nodes", [NodeCount]), diff --git a/tests/replication/repl_location_failures.erl b/tests/replication/repl_location_failures.erl index cb74985f5..5b1007552 100644 --- a/tests/replication/repl_location_failures.erl +++ b/tests/replication/repl_location_failures.erl @@ -34,7 +34,7 @@ ]). confirm() -> - rt:set_advanced_conf(all, ?CONF(5)), + rt_config:set_advanced_conf(all, ?CONF(5)), [ANodes, BNodes] = rt:build_clusters([3, 3]), diff --git a/tests/replication/repl_rt_cascading_rtq.erl b/tests/replication/repl_rt_cascading_rtq.erl index 8d31446f6..9b6debe93 100644 --- a/tests/replication/repl_rt_cascading_rtq.erl +++ b/tests/replication/repl_rt_cascading_rtq.erl @@ -6,7 +6,7 @@ -define(TEST_BUCKET, <<"rt-cascading-rtq-systest-a">>). setup() -> - rt:set_conf(all, [{"buckets.default.allow_mult", "false"}]), + rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), {SourceLeader, SinkLeaderA, SinkLeaderB, _, _, _} = ClusterNodes = make_clusters(), diff --git a/tests/replication/replication.erl b/tests/replication/replication.erl index 2acf03a88..389fd5c49 100644 --- a/tests/replication/replication.erl +++ b/tests/replication/replication.erl @@ -16,7 +16,7 @@ confirm() -> {diff_batch_size, 10} ]} ], - rt:set_advanced_conf(all, Conf), + rt_config:set_advanced_conf(all, Conf), [ANodes, BNodes] = rt:build_clusters([3, 3]), rt:wait_for_cluster_service(ANodes, riak_repl), rt:wait_for_cluster_service(BNodes, riak_repl), diff --git a/tests/replication/replication2_connections.erl b/tests/replication/replication2_connections.erl index e9419d435..9172b3aa7 100644 --- a/tests/replication/replication2_connections.erl +++ b/tests/replication/replication2_connections.erl @@ -45,7 +45,7 @@ simple_test() -> {rt_heartbeat_timeout, ?HB_TIMEOUT} ]}], - rt:set_advanced_conf(all, Conf), + rt_config:set_advanced_conf(all, Conf), [ANodes, BNodes] = rt:build_clusters([3, 3]), @@ -114,7 +114,7 @@ disconnect_test() -> {rt_heartbeat_timeout, ?HB_TIMEOUT} ]}], - rt:set_advanced_conf(all, Conf), + rt_config:set_advanced_conf(all, Conf), [ANodes, BNodes] = rt:build_clusters([3, 3]), @@ -182,7 +182,7 @@ error_cleanup_test() -> {cm_cancellation_interval, 5 * 1000} ]}], - rt:set_advanced_conf(all, Conf), + rt_config:set_advanced_conf(all, Conf), [ANodes, BNodes] = rt:build_clusters([3, 3]), diff --git a/tests/replication/replication2_pg.erl b/tests/replication/replication2_pg.erl index 6c9345875..27be6b416 100644 --- a/tests/replication/replication2_pg.erl +++ b/tests/replication/replication2_pg.erl @@ -64,7 +64,7 @@ setup_repl_clusters(Conf, SSL) -> ], - rt:set_advanced_conf(all, Conf), + rt_config:set_advanced_conf(all, Conf), Nodes = [ANodes, BNodes, CNodes] = rt:build_clusters([2, 2, 2]), rt:wait_for_cluster_service(ANodes, riak_repl), @@ -84,11 +84,11 @@ setup_repl_clusters(Conf, SSL) -> case SSL of true -> lager:info("Enabling SSL for this test"), - [rt:update_app_config(N, merge_config(SSLConfig1, Conf)) || + [rt_config:update_app_config(N, merge_config(SSLConfig1, Conf)) || N <- ANodes], - [rt:update_app_config(N, merge_config(SSLConfig2, Conf)) || + [rt_config:update_app_config(N, merge_config(SSLConfig2, Conf)) || N <- BNodes], - [rt:update_app_config(N, merge_config(SSLConfig3, Conf)) || + [rt_config:update_app_config(N, merge_config(SSLConfig3, Conf)) || N <- CNodes]; _ -> lager:info("SSL not enabled for this test") diff --git a/tests/replication/replication2_ssl.erl b/tests/replication/replication2_ssl.erl index c080b424e..89d18c4c2 100644 --- a/tests/replication/replication2_ssl.erl +++ b/tests/replication/replication2_ssl.erl @@ -59,7 +59,7 @@ confirm() -> %% test requires allow_mult=false - rt:set_conf(all, [{"buckets.default.allow_mult", "false"}]), + rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), NumNodes = rt_config:get(num_nodes, 6), ClusterASize = rt_config:get(cluster_a_size, (NumNodes div 2)), @@ -369,9 +369,9 @@ test_connection(Left, Right, false) -> test_connection({Node1, Config1}, {Node2, Config2}) -> repl_util:disconnect_cluster(Node1, "B"), repl_util:wait_for_disconnect(Node1, "B"), - rt:update_app_config(Node2, Config2), + rt_config:update_app_config(Node2, Config2), rt:wait_until_pingable(Node2), - rt:update_app_config(Node1, Config1), + rt_config:update_app_config(Node1, Config1), rt:wait_until_pingable(Node1), rt:wait_for_service(Node1, [riak_kv, riak_repl]), rt:wait_for_service(Node2, [riak_kv, riak_repl]), diff --git a/tests/replication/replication_object_reformat.erl b/tests/replication/replication_object_reformat.erl index cb1cf6851..4ece6432a 100644 --- a/tests/replication/replication_object_reformat.erl +++ b/tests/replication/replication_object_reformat.erl @@ -164,7 +164,7 @@ verify_replication(AVersion, BVersion, Start, End, Realtime) -> %% @doc Configure two clusters and set up replication between them, %% return the node list of each cluster. configure_clusters(AVersion, BVersion, Realtime) -> - rt:set_advanced_conf(all, ?CONF(infinity)), + rt_config:set_advanced_conf(all, ?CONF(infinity)), Nodes = [ANodes, BNodes] = rt:build_clusters([3, 3]), @@ -176,13 +176,13 @@ configure_clusters(AVersion, BVersion, Realtime) -> lager:info("Updating app config to force ~p on source cluster.", [AVersion]), - [rt:update_app_config(N, [{riak_kv, + [rt_config:update_app_config(N, [{riak_kv, [{object_format, AVersion}]}]) || N <- ANodes], lager:info("Updating app config to force ~p on sink cluster.", [BVersion]), - [rt:update_app_config(N, [{riak_kv, + [rt_config:update_app_config(N, [{riak_kv, [{object_format, BVersion}]}]) || N <- BNodes], diff --git a/tests/replication/replication_ssl.erl b/tests/replication/replication_ssl.erl index b04fa3eaa..31394668d 100644 --- a/tests/replication/replication_ssl.erl +++ b/tests/replication/replication_ssl.erl @@ -6,7 +6,7 @@ confirm() -> %% test requires allow_mult=false - rt:set_conf(all, [{"buckets.default.allow_mult", "false"}]), + rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), NumNodes = rt_config:get(num_nodes, 6), ClusterASize = rt_config:get(cluster_a_size, 3), @@ -232,10 +232,10 @@ confirm() -> {ANodes, BNodes} = lists:split(ClusterASize, Nodes), lager:info("Reconfiguring nodes with SSL options"), - [rt:update_app_config(N, merge_config(SSLConfig5, BaseConf)) || N <- + [rt_config:update_app_config(N, merge_config(SSLConfig5, BaseConf)) || N <- ANodes], - [rt:update_app_config(N, merge_config(SSLConfig6, BaseConf)) || N <- + [rt_config:update_app_config(N, merge_config(SSLConfig6, BaseConf)) || N <- BNodes], [rt:wait_until_pingable(N) || N <- Nodes], @@ -254,9 +254,9 @@ merge_config(Mixin, Base) -> lists:ukeymerge(1, lists:keysort(1, Mixin), lists:keysort(1, Base)). test_connection({Node1, Config1}, {Node2, Config2}) -> - rt:update_app_config(Node1, Config1), + rt_config:update_app_config(Node1, Config1), rt:wait_until_pingable(Node1), - rt:update_app_config(Node2, Config2), + rt_config:update_app_config(Node2, Config2), rt:wait_until_pingable(Node2), rt:wait_for_service(Node1, [riak_kv, riak_repl]), rt:wait_for_service(Node2, [riak_kv, riak_repl]), diff --git a/tests/replication/replication_stats.erl b/tests/replication/replication_stats.erl index cce909651..d699ed5ff 100644 --- a/tests/replication/replication_stats.erl +++ b/tests/replication/replication_stats.erl @@ -39,7 +39,7 @@ confirm() -> fullsync_enabled_and_started(). fullsync_enabled_and_started() -> - rt:set_advanced_conf(all, ?CONF), + rt_config:set_advanced_conf(all, ?CONF), [ANodes, BNodes] = rt:build_clusters([3, 3]), diff --git a/tests/replication/rt_cascading.erl b/tests/replication/rt_cascading.erl index 8c6702ed8..a47e29b51 100644 --- a/tests/replication/rt_cascading.erl +++ b/tests/replication/rt_cascading.erl @@ -38,7 +38,7 @@ confirm() -> %% test requires allow_mult=false b/c of rt:systest_read - rt:set_conf(all, [{"buckets.default.allow_mult", "false"}]), + rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), case eunit:test(?MODULE, [verbose]) of ok -> diff --git a/tests/test_cluster.erl b/tests/test_cluster.erl index abe34d661..b19d15bb3 100644 --- a/tests/test_cluster.erl +++ b/tests/test_cluster.erl @@ -26,4 +26,4 @@ confirm() -> Config = [{riak_search, [{enabled, true}]}], rt:build_cluster(4, Config), ?assert(false), - fail. \ No newline at end of file + fail. diff --git a/tests/verify_capabilities.erl b/tests/verify_capabilities.erl index a1ff966df..2730eca85 100644 --- a/tests/verify_capabilities.erl +++ b/tests/verify_capabilities.erl @@ -213,19 +213,19 @@ confirm() -> end, lager:info("Override: (use: legacy), (prefer: proxy)"), - [rt:update_app_config(Node, Override(legacy, proxy)) || Node <- Nodes], + [rt_config:update_app_config(Node, Override(legacy, proxy)) || Node <- Nodes], lager:info("Verify vnode_routing == legacy"), assert_capability(CNode, {riak_core, vnode_routing}, legacy), lager:info("Override: (use: proxy), (prefer: legacy)"), - [rt:update_app_config(Node, Override(proxy, legacy)) || Node <- Nodes], + [rt_config:update_app_config(Node, Override(proxy, legacy)) || Node <- Nodes], lager:info("Verify vnode_routing == proxy"), assert_capability(CNode, {riak_core, vnode_routing}, proxy), lager:info("Override: (prefer: legacy)"), - [rt:update_app_config(Node, Override(undefined, legacy)) || Node <- Nodes], + [rt_config:update_app_config(Node, Override(undefined, legacy)) || Node <- Nodes], lager:info("Verify vnode_routing == legacy"), assert_capability(CNode, {riak_core, vnode_routing}, legacy), diff --git a/tests/verify_dynamic_ring.erl b/tests/verify_dynamic_ring.erl index afd87f267..d1566a1d3 100644 --- a/tests/verify_dynamic_ring.erl +++ b/tests/verify_dynamic_ring.erl @@ -31,8 +31,8 @@ confirm() -> %% test requires allow_mult=false b/c of rt:systest_read - rt:set_conf(all, [{"buckets.default.allow_mult", "false"}]), - rt:update_app_config(all, [{riak_core, + rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), + rt_config:update_app_config(all, [{riak_core, [{ring_creation_size, ?START_SIZE}]}]), [ANode, AnotherNode, YetAnother, _ReplacingNode] = _AllNodes = rt:deploy_nodes(4), NewNodes = Nodes = [ANode, AnotherNode, YetAnother], diff --git a/tests/verify_riak_object_reformat.erl b/tests/verify_riak_object_reformat.erl index ca44f9668..cd693f4ff 100644 --- a/tests/verify_riak_object_reformat.erl +++ b/tests/verify_riak_object_reformat.erl @@ -31,7 +31,7 @@ -define(N, 3). confirm() -> - rt:update_app_config(all, [{riak_kv, [{object_format, v1}]}]), + rt_config:update_app_config(all, [{riak_kv, [{object_format, v1}]}]), TestMetaData = riak_test_runner:metadata(), DowngradeVsn = proplists:get_value(upgrade_version, TestMetaData, previous), Nodes = [Node1|_] = rt:build_cluster(?N), diff --git a/tests/verify_tick_change.erl b/tests/verify_tick_change.erl index 3390fbac8..208adcdd4 100644 --- a/tests/verify_tick_change.erl +++ b/tests/verify_tick_change.erl @@ -25,7 +25,7 @@ confirm() -> ClusterSize = 4, - rt:set_conf(all, [{"buckets.default.allow_mult", "false"}]), + rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), NewConfig = [], Nodes = rt:build_cluster(ClusterSize, NewConfig), ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), diff --git a/tests/verify_vclock.erl b/tests/verify_vclock.erl index 64e03ea1e..dcd24c690 100644 --- a/tests/verify_vclock.erl +++ b/tests/verify_vclock.erl @@ -113,7 +113,7 @@ force_encoding(Node, EncodingMethod) -> } ], - rt:update_app_config(Node, OverrideData) + rt_config:update_app_config(Node, OverrideData) end. From ba4ee80dc101186c3aa325d103da13dd1fe60abe Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Wed, 30 Jul 2014 08:32:37 -0400 Subject: [PATCH 011/157] Continue to move config functions from rt to rt_config. --- src/rt.erl | 61 ++++++++++------------------------------------- src/rt_config.erl | 6 ++++- 2 files changed, 18 insertions(+), 49 deletions(-) diff --git a/src/rt.erl b/src/rt.erl index 31b5e59b3..49afc78cc 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -204,34 +204,6 @@ case string:str(String, Substr) of _ -> true end. --spec set_conf(atom(), [{string(), string()}]) -> ok. -set_conf(all, NameValuePairs) -> -?HARNESS:set_conf(all, NameValuePairs); -set_conf(Node, NameValuePairs) -> -stop(Node), -?assertEqual(ok, rt:wait_until_unpingable(Node)), -?HARNESS:set_conf(Node, NameValuePairs), -start(Node). - --spec set_advanced_conf(atom(), [{string(), string()}]) -> ok. -set_advanced_conf(all, NameValuePairs) -> -?HARNESS:set_advanced_conf(all, NameValuePairs); -set_advanced_conf(Node, NameValuePairs) -> -stop(Node), -?assertEqual(ok, rt:wait_until_unpingable(Node)), -?HARNESS:set_advanced_conf(Node, NameValuePairs), -start(Node). - -%% @doc Rewrite the given node's app.config file, overriding the varialbes -%% in the existing app.config with those in `Config'. -update_app_config(all, Config) -> -?HARNESS:update_app_config(all, Config); -update_app_config(Node, Config) -> -stop(Node), -?assertEqual(ok, rt:wait_until_unpingable(Node)), -?HARNESS:update_app_config(Node, Config), -start(Node). - %% @doc Helper that returns first successful application get_env result, %% used when different versions of Riak use different app vars for %% the same setting. @@ -311,19 +283,12 @@ deploy_nodes([ current || _ <- lists:seq(1, NumNodes)]). deploy_nodes(NumNodes, InitialConfig) when is_integer(NumNodes) -> deploy_nodes(NumNodes, InitialConfig, [riak_kv]); deploy_nodes(Versions, Services) -> -NodeConfig = [ version_to_config(Version) || Version <- Versions ], -Nodes = ?HARNESS:deploy_nodes(NodeConfig), -lager:info("Waiting for services ~p to start on ~p.", [Services, Nodes]), -[ ok = wait_for_service(Node, Service) || Node <- Nodes, - Service <- Services ], -Nodes. - -deploy_nodes(NumNodes, InitialConfig, Services) when is_integer(NumNodes) -> -NodeConfig = [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)], -deploy_nodes(NodeConfig, Services). - -version_to_config(Config) when is_tuple(Config)-> Config; -version_to_config(Version) -> {Version, default}. + NodeConfig = [ rt_config:version_to_config(Version) || Version <- Versions ], + Nodes = ?HARNESS:deploy_nodes(NodeConfig), + lager:info("Waiting for services ~p to start on ~p.", [Services, Nodes]), + [ ok = wait_for_service(Node, Service) || Node <- Nodes, + Service <- Services ], + Nodes. deploy_clusters(Settings) -> ClusterConfigs = [case Setting of @@ -1615,14 +1580,14 @@ set_backend(riak_kv_memory_backend); set_backend(multi, Extras) -> set_backend(riak_kv_multi_backend, Extras); set_backend(Backend, _) when Backend == riak_kv_bitcask_backend; Backend == riak_kv_eleveldb_backend; Backend == riak_kv_memory_backend -> -lager:info("rt:set_backend(~p)", [Backend]), -update_app_config(all, [{riak_kv, [{storage_backend, Backend}]}]), -get_backends(); + lager:info("rt:set_backend(~p)", [Backend]), + rt_config:update_app_config(all, [{riak_kv, [{storage_backend, Backend}]}]), + get_backends(); set_backend(Backend, Extras) when Backend == riak_kv_multi_backend -> -MultiConfig = proplists:get_value(multi_config, Extras, default), -Config = make_multi_backend_config(MultiConfig), -update_app_config(all, [{riak_kv, Config}]), -get_backends(); + MultiConfig = proplists:get_value(multi_config, Extras, default), + Config = make_multi_backend_config(MultiConfig), + rt_config:update_app_config(all, [{riak_kv, Config}]), + get_backends(); set_backend(Other, _) -> lager:warning("rt:set_backend doesn't recognize ~p as a legit backend, using the default.", [Other]), get_backends(). diff --git a/src/rt_config.erl b/src/rt_config.erl index 39ef3788e..8da5a9539 100644 --- a/src/rt_config.erl +++ b/src/rt_config.erl @@ -31,7 +31,8 @@ set/2, set_conf/2, set_advanced_conf/2, - update_app_config/2 + update_app_config/2, + version_to_config/1 ]). -define(HARNESS, (rt_config:get(rt_harness))). @@ -156,6 +157,9 @@ update_app_config(Node, Config) -> ?HARNESS:update_app_config(Node, Config), rt:start(Node). +version_to_config(Config) when is_tuple(Config)-> Config; +version_to_config(Version) -> {Version, default}. + to_upper(S) -> lists:map(fun char_to_upper/1, S). char_to_upper(C) when C >= $a, C =< $z -> C bxor $\s; char_to_upper(C) -> C. From ff24f9120ac624749e07594f8388b2267f8db8fa Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Wed, 30 Jul 2014 10:10:30 -0400 Subject: [PATCH 012/157] Migrate several more functions into rt_cluster from rt. --- src/riak_test_escript.erl | 2 +- src/rt.erl | 51 ------ src/rt_cluster.erl | 157 +++++++++++++++++- tests/cluster_meta_rmr.erl | 4 +- tests/cuttlefish_configuration.erl | 2 +- tests/ensemble_util.erl | 8 +- tests/gh_riak_core_154.erl | 2 +- tests/gh_riak_core_155.erl | 2 +- tests/gh_riak_core_176.erl | 2 +- tests/gh_riak_kv_765.erl | 6 +- tests/http_bucket_types.erl | 2 +- tests/http_security.erl | 2 +- tests/jmx_verify.erl | 6 +- tests/loaded_upgrade.erl | 2 +- tests/mapred_basic_compat.erl | 2 +- tests/mapred_buffer_prereduce.erl | 2 +- tests/mapred_dead_pipe.erl | 2 +- tests/mapred_http_errors.erl | 2 +- tests/mapred_javascript.erl | 2 +- tests/mapred_notfound_failover.erl | 2 +- tests/mapred_search_switch.erl | 2 +- tests/mapred_verify_rt.erl | 2 +- tests/overload.erl | 29 ++-- tests/partition_repair.erl | 4 +- tests/pb_cipher_suites.erl | 2 +- tests/pb_security.erl | 2 +- tests/pipe_verify_basics.erl | 2 +- tests/pipe_verify_exceptions.erl | 2 +- tests/pipe_verify_handoff.erl | 2 +- tests/pipe_verify_handoff_blocking.erl | 2 +- .../pipe_verify_restart_input_forwarding.erl | 2 +- tests/pipe_verify_sink_types.erl | 2 +- tests/post_generate_key.erl | 2 +- tests/pr_pw.erl | 2 +- tests/replication/repl_aae_fullsync.erl | 8 +- tests/replication/repl_bucket_types.erl | 6 +- tests/replication/repl_cancel_fullsync.erl | 6 +- .../repl_consistent_object_filter.erl | 2 +- tests/replication/repl_fs_bench.erl | 6 +- tests/replication/repl_fs_stat_caching.erl | 2 +- tests/replication/repl_location_failures.erl | 6 +- tests/replication/repl_rt_cascading_rtq.erl | 4 +- tests/replication/repl_rt_heartbeat.erl | 2 +- tests/replication/repl_rt_overload.erl | 2 +- tests/replication/repl_rt_pending.erl | 2 +- tests/replication/replication.erl | 4 +- .../replication/replication2_connections.erl | 14 +- .../replication2_console_tests.erl | 2 +- tests/replication/replication2_fsschedule.erl | 6 +- tests/replication/replication2_pg.erl | 2 +- .../replication2_rt_sink_connection.erl | 2 +- tests/replication/replication2_upgrade.erl | 2 +- .../replication_object_reformat.erl | 4 +- tests/replication/replication_ssl.erl | 4 +- tests/replication/replication_stats.erl | 6 +- tests/replication/replication_upgrade.erl | 2 +- tests/replication/rt_cascading.erl | 42 ++--- tests/riak_admin_console_tests.erl | 2 +- tests/riak_control.erl | 2 +- tests/riak_control_authentication.erl | 2 +- tests/riak_rex.erl | 4 +- tests/riaknostic_rt.erl | 2 +- tests/rolling_capabilities.erl | 2 +- tests/rt_basic_test.erl | 2 +- tests/sibling_explosion.erl | 2 +- tests/test_cluster.erl | 2 +- tests/verify_2i_aae.erl | 2 +- tests/verify_2i_limit.erl | 2 +- tests/verify_2i_mixed_cluster.erl | 2 +- tests/verify_2i_returnterms.erl | 2 +- tests/verify_2i_stream.erl | 2 +- tests/verify_2i_timeout.erl | 2 +- tests/verify_aae.erl | 6 +- tests/verify_api_timeouts.erl | 2 +- tests/verify_asis_put.erl | 2 +- tests/verify_backup_restore.erl | 6 +- tests/verify_basic_upgrade.erl | 2 +- tests/verify_bitcask_tombstone2_upgrade.erl | 2 +- tests/verify_busy_dist_port.erl | 2 +- tests/verify_capabilities.erl | 2 +- tests/verify_commit_hooks.erl | 2 +- tests/verify_conditional_postcommit.erl | 2 +- tests/verify_counter_converge.erl | 2 +- tests/verify_counter_repl.erl | 2 +- tests/verify_cs_bucket.erl | 2 +- tests/verify_down.erl | 2 +- tests/verify_dt_context.erl | 2 +- tests/verify_dt_converge.erl | 2 +- tests/verify_dt_upgrade.erl | 2 +- tests/verify_dvv_repl.erl | 8 +- tests/verify_dynamic_ring.erl | 2 +- tests/verify_handoff.erl | 9 +- tests/verify_handoff_mixed.erl | 2 +- tests/verify_kv_health_check.erl | 2 +- tests/verify_link_walk_urls.erl | 2 +- tests/verify_listkeys.erl | 2 +- tests/verify_listkeys_eqcfsm.erl | 4 +- tests/verify_membackend.erl | 12 +- tests/verify_mr_prereduce_node_down.erl | 2 +- tests/verify_no_writes_on_read.erl | 2 +- tests/verify_object_limits.erl | 2 +- tests/verify_reset_bucket_props.erl | 2 +- tests/verify_riak_lager.erl | 2 +- tests/verify_riak_object_reformat.erl | 2 +- tests/verify_riak_stats.erl | 2 +- tests/verify_search.erl | 2 +- tests/verify_secondary_index_reformat.erl | 2 +- tests/verify_snmp.erl | 2 +- tests/verify_staged_clustering.erl | 2 +- tests/verify_tick_change.erl | 2 +- tests/verify_vclock.erl | 4 +- tests/verify_vclock_encoding_upgrade.erl | 2 +- tests/yz_ensemble.erl | 4 +- 113 files changed, 352 insertions(+), 252 deletions(-) diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index ffc9de750..5bb67cdbc 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -193,7 +193,7 @@ maybe_teardown(true, TestResults, Coverage, Verbose) -> so_kill_riak_maybe(); _ -> lager:info("Multiple tests run or no failure"), - rt:teardown(), + rt_cluster:teardown(), print_summary(TestResults, Coverage, Verbose) end, ok. diff --git a/src/rt.erl b/src/rt.erl index 49afc78cc..4ae1b9bcb 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -36,19 +36,11 @@ attach/2, attach_direct/2, brutal_kill/1, - build_cluster/1, - build_cluster/2, - build_cluster/3, - build_clusters/1, - join_cluster/1, capability/2, capability/3, check_singleton_node/1, check_ibrowse/0, claimant_according_to/1, - clean_cluster/1, - clean_data_dir/1, - clean_data_dir/2, cmd/1, cmd/2, connection_info/1, @@ -135,7 +127,6 @@ update_app_config/2, upgrade/2, upgrade/3, - versions/0, wait_for_cluster_service/2, wait_for_cmd/1, wait_for_service/2, @@ -269,48 +260,6 @@ _ -> undefined end. -%% @doc Deploy a set of freshly installed Riak nodes, returning a list of the -%% nodes deployed. -%% @todo Re-add -spec after adding multi-version support -deploy_nodes(Versions) when is_list(Versions) -> -deploy_nodes(Versions, [riak_kv]); -deploy_nodes(NumNodes) when is_integer(NumNodes) -> -deploy_nodes([ current || _ <- lists:seq(1, NumNodes)]). - -%% @doc Deploy a set of freshly installed Riak nodes with the given -%% `InitialConfig', returning a list of the nodes deployed. --spec deploy_nodes(NumNodes :: integer(), any()) -> [node()]. -deploy_nodes(NumNodes, InitialConfig) when is_integer(NumNodes) -> -deploy_nodes(NumNodes, InitialConfig, [riak_kv]); -deploy_nodes(Versions, Services) -> - NodeConfig = [ rt_config:version_to_config(Version) || Version <- Versions ], - Nodes = ?HARNESS:deploy_nodes(NodeConfig), - lager:info("Waiting for services ~p to start on ~p.", [Services, Nodes]), - [ ok = wait_for_service(Node, Service) || Node <- Nodes, - Service <- Services ], - Nodes. - -deploy_clusters(Settings) -> -ClusterConfigs = [case Setting of - Configs when is_list(Configs) -> - Configs; - NumNodes when is_integer(NumNodes) -> - [{current, default} || _ <- lists:seq(1, NumNodes)]; - {NumNodes, InitialConfig} when is_integer(NumNodes) -> - [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)]; - {NumNodes, Vsn, InitialConfig} when is_integer(NumNodes) -> - [{Vsn, InitialConfig} || _ <- lists:seq(1,NumNodes)] - end || Setting <- Settings], -?HARNESS:deploy_clusters(ClusterConfigs). - -build_clusters(Settings) -> -Clusters = deploy_clusters(Settings), -[begin - join_cluster(Nodes), - lager:info("Cluster built: ~p", [Nodes]) -end || Nodes <- Clusters], -Clusters. - %% @doc Start the specified Riak node start(Node) -> ?HARNESS:start(Node). diff --git a/src/rt_cluster.erl b/src/rt_cluster.erl index 372bd0869..1507a781f 100644 --- a/src/rt_cluster.erl +++ b/src/rt_cluster.erl @@ -19,15 +19,32 @@ %% ------------------------------------------------------------------- -module(rt_cluster). +-include_lib("eunit/include/eunit.hrl"). -export([properties/0, setup/2, config/0, - augment_config/3]). + augment_config/3, + deploy_nodes/1, + deploy_nodes/2, + deploy_clusters/1, + build_cluster/1, + build_cluster/2, + build_cluster/3, + build_clusters/1, + clean_cluster/1, + join_cluster/1, + clean_data_dir/1, + clean_data_dir/2, + try_nodes_ready/3, + versions/0, + teardown/0]). -export([maybe_wait_for_transfers/2]). -include("rt.hrl"). +-define(HARNESS, (rt_config:get(rt_harness))). + %% @doc Default properties used if a riak_test module does not specify %% a custom properties function. -spec properties() -> rt_properties(). @@ -37,7 +54,7 @@ properties() -> -spec setup(rt_properties(), proplists:proplist()) -> {ok, rt_properties()} | {error, term()}. setup(Properties, MetaData) -> - rt:set_conf(all, [{"buckets.default.allow_mult", "false"}]), + rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), RollingUpgrade = proplists:get_value(rolling_upgrade, MetaData, @@ -53,9 +70,52 @@ setup(Properties, MetaData) -> {ok, UpdProperties}. deploy_or_build_cluster(Versions, true) -> - rt:build_cluster(Versions); + build_cluster(Versions); deploy_or_build_cluster(Versions, false) -> - rt:deploy_nodes(Versions). + deploy_nodes(Versions). + +%% @doc Deploy a set of freshly installed Riak nodes, returning a list of the +%% nodes deployed. +%% @todo Re-add -spec after adding multi-version support +deploy_nodes(Versions) when is_list(Versions) -> + deploy_nodes(Versions, [riak_kv]); +deploy_nodes(NumNodes) when is_integer(NumNodes) -> + deploy_nodes([ current || _ <- lists:seq(1, NumNodes)]). + +%% @doc Deploy a set of freshly installed Riak nodes with the given +%% `InitialConfig', returning a list of the nodes deployed. +-spec deploy_nodes(NumNodes :: integer(), any()) -> [node()]. +deploy_nodes(NumNodes, InitialConfig) when is_integer(NumNodes) -> + NodeConfig = [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)], + deploy_nodes(NodeConfig); +deploy_nodes(Versions, Services) -> + NodeConfig = [ rt_config:version_to_config(Version) || Version <- Versions ], + Nodes = ?HARNESS:deploy_nodes(NodeConfig), + lager:info("Waiting for services ~p to start on ~p.", [Services, Nodes]), + [ ok = rt:wait_for_service(Node, Service) || Node <- Nodes, + Service <- Services ], + Nodes. + +deploy_clusters(Settings) -> + ClusterConfigs = [case Setting of + Configs when is_list(Configs) -> + Configs; + NumNodes when is_integer(NumNodes) -> + [{current, default} || _ <- lists:seq(1, NumNodes)]; + {NumNodes, InitialConfig} when is_integer(NumNodes) -> + [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)]; + {NumNodes, Vsn, InitialConfig} when is_integer(NumNodes) -> + [{Vsn, InitialConfig} || _ <- lists:seq(1,NumNodes)] + end || Setting <- Settings], + ?HARNESS:deploy_clusters(ClusterConfigs). + +build_clusters(Settings) -> + Clusters = deploy_clusters(Settings), + [begin + join_cluster(Nodes), + lager:info("Cluster built: ~p", [Nodes]) + end || Nodes <- Clusters], + Clusters. maybe_wait_for_transfers(Nodes, true) -> lager:info("Waiting for transfers"), @@ -63,6 +123,95 @@ maybe_wait_for_transfers(Nodes, true) -> maybe_wait_for_transfers(_Nodes, false) -> ok. +%% @doc Safely construct a new cluster and return a list of the deployed nodes +%% @todo Add -spec and update doc to reflect mult-version changes +build_cluster(Versions) when is_list(Versions) -> + build_cluster(length(Versions), Versions, default); +build_cluster(NumNodes) -> + build_cluster(NumNodes, default). + +%% @doc Safely construct a `NumNode' size cluster using +%% `InitialConfig'. Return a list of the deployed nodes. +build_cluster(NumNodes, InitialConfig) -> + build_cluster(NumNodes, [], InitialConfig). + +build_cluster(NumNodes, Versions, InitialConfig) -> + %% Deploy a set of new nodes + Nodes = + case Versions of + [] -> + deploy_nodes(NumNodes, InitialConfig); + _ -> + deploy_nodes(Versions) + end, + + join_cluster(Nodes), + lager:info("Cluster built: ~p", [Nodes]), + Nodes. + +join_cluster(Nodes) -> + %% Ensure each node owns 100% of it's own ring + [?assertEqual([Node], rt:owners_according_to(Node)) || Node <- Nodes], + + %% Join nodes + [Node1|OtherNodes] = Nodes, + case OtherNodes of + [] -> + %% no other nodes, nothing to join/plan/commit + ok; + _ -> + %% ok do a staged join and then commit it, this eliminates the + %% large amount of redundant handoff done in a sequential join + [rt:staged_join(Node, Node1) || Node <- OtherNodes], + rt:plan_and_commit(Node1), + try_nodes_ready(Nodes, 3, 500) + end, + + ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), + + %% Ensure each node owns a portion of the ring + rt:wait_until_nodes_agree_about_ownership(Nodes), + ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes)), + ok. + +try_nodes_ready([Node1 | _Nodes], 0, _SleepMs) -> + lager:info("Nodes not ready after initial plan/commit, retrying"), + rt:plan_and_commit(Node1); +try_nodes_ready(Nodes, N, SleepMs) -> + ReadyNodes = [Node || Node <- Nodes, rt:is_ready(Node) =:= true], + case ReadyNodes of + Nodes -> + ok; + _ -> + timer:sleep(SleepMs), + try_nodes_ready(Nodes, N-1, SleepMs) + end. + +%% @doc Stop nodes and wipe out their data directories +clean_cluster(Nodes) when is_list(Nodes) -> + [rt:stop_and_wait(Node) || Node <- Nodes], + clean_data_dir(Nodes). + +clean_data_dir(Nodes) -> + clean_data_dir(Nodes, ""). + +clean_data_dir(Nodes, SubDir) when not is_list(Nodes) -> + clean_data_dir([Nodes], SubDir); +clean_data_dir(Nodes, SubDir) when is_list(Nodes) -> + ?HARNESS:clean_data_dir(Nodes, SubDir). + +%% @doc Shutdown every node, this is for after a test run is complete. +teardown() -> + %% stop all connected nodes, 'cause it'll be faster that + %%lager:info("RPC stopping these nodes ~p", [nodes()]), + %%[ rt:stop(Node) || Node <- nodes()], + %% Then do the more exhaustive harness thing, in case something was up + %% but not connected. + ?HARNESS:teardown(). + +versions() -> + ?HARNESS:versions(). + config() -> [{riak_core, [{handoff_concurrency, 11}]}, {riak_search, [{enabled, true}]}, diff --git a/tests/cluster_meta_rmr.erl b/tests/cluster_meta_rmr.erl index beabc0714..fe7f73e3a 100644 --- a/tests/cluster_meta_rmr.erl +++ b/tests/cluster_meta_rmr.erl @@ -59,10 +59,10 @@ run(NumNodes, NumRounds, StableRounds) -> exit(Pid, kill), %% start all the down nodes so we can clean them :( [rt:start(Node) || Node <- DownNodes], - rt:clean_cluster(AllNodes). + rt_cluster:clean_cluster(AllNodes). setup_nodes(NumNodes) -> - Nodes = rt:build_cluster(NumNodes), + Nodes = rt_cluster:build_cluster(NumNodes), [begin ok = rpc:call(Node, application, set_env, [riak_core, broadcast_exchange_timer, 4294967295]), ok = rpc:call(Node, application, set_env, [riak_core, gossip_limit, {10000000, 4294967295}]), diff --git a/tests/cuttlefish_configuration.erl b/tests/cuttlefish_configuration.erl index 4447bfca4..ea10009bf 100644 --- a/tests/cuttlefish_configuration.erl +++ b/tests/cuttlefish_configuration.erl @@ -12,7 +12,7 @@ confirm() -> {"leveldb.sync_on_write", "on"} ], - [Node] = rt:deploy_nodes(1, {cuttlefish, CuttlefishConf}), + [Node] = rt_cluster:deploy_nodes(1, {cuttlefish, CuttlefishConf}), {ok, RingSize} = rt:rpc_get_env(Node, [{riak_core, ring_creation_size}]), ?assertEqual(8, RingSize), diff --git a/tests/ensemble_util.erl b/tests/ensemble_util.erl index d6f79145b..f206df9de 100644 --- a/tests/ensemble_util.erl +++ b/tests/ensemble_util.erl @@ -26,16 +26,16 @@ -include_lib("eunit/include/eunit.hrl"). build_cluster(Num, Config, NVal) -> - Nodes = rt:deploy_nodes(Num, Config), + Nodes = rt_cluster:deploy_nodes(Num, Config), Node = hd(Nodes), - rt:join_cluster(Nodes), + rt_cluster:join_cluster(Nodes), ensemble_util:wait_until_cluster(Nodes), ensemble_util:wait_for_membership(Node), ensemble_util:wait_until_stable(Node, NVal), Nodes. build_cluster_without_quorum(Num, Config) -> - Nodes = rt:deploy_nodes(Num, Config), + Nodes = rt_cluster:deploy_nodes(Num, Config), SetupLogCaptureFun = fun(Node) -> rt:setup_log_capture(Node) end, @@ -43,7 +43,7 @@ build_cluster_without_quorum(Num, Config) -> Node = hd(Nodes), ok = rpc:call(Node, riak_ensemble_manager, enable, []), _ = rpc:call(Node, riak_core_ring_manager, force_update, []), - rt:join_cluster(Nodes), + rt_cluster:join_cluster(Nodes), ensemble_util:wait_until_cluster(Nodes), ensemble_util:wait_for_membership(Node), Nodes. diff --git a/tests/gh_riak_core_154.erl b/tests/gh_riak_core_154.erl index ff722a483..70882e5dc 100644 --- a/tests/gh_riak_core_154.erl +++ b/tests/gh_riak_core_154.erl @@ -28,7 +28,7 @@ confirm() -> %% Increase handoff concurrency on nodes NewConfig = [{riak_core, [{handoff_concurrency, 1024}]}], - Nodes = rt:build_cluster(2, NewConfig), + Nodes = rt_cluster:build_cluster(2, NewConfig), ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), [Node1, Node2] = Nodes, diff --git a/tests/gh_riak_core_155.erl b/tests/gh_riak_core_155.erl index 5bd4bb17d..fb53b69ce 100644 --- a/tests/gh_riak_core_155.erl +++ b/tests/gh_riak_core_155.erl @@ -24,7 +24,7 @@ -include_lib("eunit/include/eunit.hrl"). confirm() -> - [Node] = rt:build_cluster(1), + [Node] = rt_cluster:build_cluster(1), %% Generate a valid preflist for our get requests rpc:call(Node, riak_core, wait_for_service, [riak_kv]), diff --git a/tests/gh_riak_core_176.erl b/tests/gh_riak_core_176.erl index 3f7b351e1..959d14e68 100644 --- a/tests/gh_riak_core_176.erl +++ b/tests/gh_riak_core_176.erl @@ -23,7 +23,7 @@ -include_lib("eunit/include/eunit.hrl"). confirm() -> - Nodes = rt:deploy_nodes(3), + Nodes = rt_cluster:deploy_nodes(3), [Node1, Node2, Node3] = Nodes, Nodes12 = [Node1, Node2], Nodes123 = Nodes, diff --git a/tests/gh_riak_kv_765.erl b/tests/gh_riak_kv_765.erl index a8f070cdd..0c5c880cb 100644 --- a/tests/gh_riak_kv_765.erl +++ b/tests/gh_riak_kv_765.erl @@ -41,7 +41,7 @@ confirm() -> check_empty_build() -> Config = [{riak_core, [{vnode_management_timer, 1000}, {ring_creation_size, 4}]}], - Nodes = rt:build_cluster(1, Config), + Nodes = rt_cluster:build_cluster(1, Config), Node = hd(Nodes), timer:sleep(2000), Self = self(), @@ -56,7 +56,7 @@ check_empty_build() -> lager:info("Failed. Empty AAE trees were not built instantly"), fail end, - rt:clean_cluster(Nodes), + rt_cluster:clean_cluster(Nodes), Result. check_throttle_and_expiration() -> @@ -66,7 +66,7 @@ check_throttle_and_expiration() -> {anti_entropy, {off, []}}]}, {riak_core, [{vnode_management_timer, 1000}, {ring_creation_size, 4}]}], - Nodes = rt:build_cluster(1, Config), + Nodes = rt_cluster:build_cluster(1, Config), Node = hd(Nodes), timer:sleep(2000), diff --git a/tests/http_bucket_types.erl b/tests/http_bucket_types.erl index bc56b9d11..283153506 100644 --- a/tests/http_bucket_types.erl +++ b/tests/http_bucket_types.erl @@ -11,7 +11,7 @@ confirm() -> application:start(ibrowse), lager:info("Deploy some nodes"), - Nodes = rt:build_cluster(4, [], [ + Nodes = rt_cluster:build_cluster(4, [], [ {riak_core, [{default_bucket_props, [{n_val, 2}]}]}]), Node = hd(Nodes), diff --git a/tests/http_security.erl b/tests/http_security.erl index f2ccdafec..693764211 100644 --- a/tests/http_security.erl +++ b/tests/http_security.erl @@ -43,7 +43,7 @@ confirm() -> {enabled, true} ]} ], - Nodes = rt:build_cluster(4, Conf), + Nodes = rt_cluster:build_cluster(4, Conf), Node = hd(Nodes), %% enable security on the cluster ok = rpc:call(Node, riak_core_console, security_enable, [[]]), diff --git a/tests/jmx_verify.erl b/tests/jmx_verify.erl index 731295623..c11ac0e75 100644 --- a/tests/jmx_verify.erl +++ b/tests/jmx_verify.erl @@ -37,7 +37,7 @@ confirm() -> JMXPort = 41111, Config = [{riak_jmx, [{enabled, true}, {port, JMXPort}]}], - Nodes = rt:deploy_nodes(1, Config), + Nodes = rt_cluster:deploy_nodes(1, Config), [Node1] = Nodes, ?assertEqual(ok, rt:wait_until_nodes_ready([Node1])), @@ -120,7 +120,7 @@ confirm() -> test_supervision() -> JMXPort = 41111, Config = [{riak_jmx, [{enabled, true}, {port, JMXPort}]}], - [Node|[]] = rt:deploy_nodes(1, Config), + [Node|[]] = rt_cluster:deploy_nodes(1, Config), timer:sleep(20000), case net_adm:ping(Node) of pang -> @@ -172,7 +172,7 @@ test_application_stop() -> lager:info("Testing application:stop()"), JMXPort = 41111, Config = [{riak_jmx, [{enabled, true}, {port, JMXPort}]}], - Nodes = rt:deploy_nodes(1, Config), + Nodes = rt_cluster:deploy_nodes(1, Config), [Node] = Nodes, ?assertEqual(ok, rt:wait_until_nodes_ready([Node])), diff --git a/tests/loaded_upgrade.erl b/tests/loaded_upgrade.erl index 9dd09fbdc..50e46e1ab 100644 --- a/tests/loaded_upgrade.erl +++ b/tests/loaded_upgrade.erl @@ -43,7 +43,7 @@ confirm() -> Config = [{riak_search, [{enabled, true}]}, {riak_pipe, [{worker_limit, 200}]}], NumNodes = 4, Vsns = [{OldVsn, Config} || _ <- lists:seq(1,NumNodes)], - Nodes = rt:build_cluster(Vsns), + Nodes = rt_cluster:build_cluster(Vsns), seed_cluster(Nodes), diff --git a/tests/mapred_basic_compat.erl b/tests/mapred_basic_compat.erl index 0aa0f202f..6b827b7fa 100644 --- a/tests/mapred_basic_compat.erl +++ b/tests/mapred_basic_compat.erl @@ -41,7 +41,7 @@ -define(BUCKET_TYPE, <<"mytype">>). confirm() -> - Nodes = rt:build_cluster(3), + Nodes = rt_cluster:build_cluster(3), [Node1|_] = Nodes, %% create a new type diff --git a/tests/mapred_buffer_prereduce.erl b/tests/mapred_buffer_prereduce.erl index 9356dbc34..d6b6cbac7 100644 --- a/tests/mapred_buffer_prereduce.erl +++ b/tests/mapred_buffer_prereduce.erl @@ -35,7 +35,7 @@ -define(NUM_INTS, 1000). confirm() -> - Nodes = rt:build_cluster(3), + Nodes = rt_cluster:build_cluster(3), load_test_data(Nodes), diff --git a/tests/mapred_dead_pipe.erl b/tests/mapred_dead_pipe.erl index b60728d46..53abb7f4d 100644 --- a/tests/mapred_dead_pipe.erl +++ b/tests/mapred_dead_pipe.erl @@ -44,7 +44,7 @@ }">>). confirm() -> - Nodes = rt:build_cluster(3), + Nodes = rt_cluster:build_cluster(3), %% to pick up fake_builder/1 rt:load_modules_on_nodes([?MODULE], Nodes), diff --git a/tests/mapred_http_errors.erl b/tests/mapred_http_errors.erl index 5d31f8995..aa07c2f4d 100644 --- a/tests/mapred_http_errors.erl +++ b/tests/mapred_http_errors.erl @@ -36,7 +36,7 @@ map_never_notfound(Object, _, _) when Object /= {error, notfound} -> [ok]. confirm() -> - Nodes = rt:build_cluster(1), + Nodes = rt_cluster:build_cluster(1), rt:load_modules_on_nodes([?MODULE], Nodes), diff --git a/tests/mapred_javascript.erl b/tests/mapred_javascript.erl index 5f8d8d502..fedfadf5c 100644 --- a/tests/mapred_javascript.erl +++ b/tests/mapred_javascript.erl @@ -43,7 +43,7 @@ }">>). confirm() -> - Nodes = rt:build_cluster(3), + Nodes = rt_cluster:build_cluster(3), load_test_data(Nodes), diff --git a/tests/mapred_notfound_failover.erl b/tests/mapred_notfound_failover.erl index 8c8e2e424..9076a1c30 100644 --- a/tests/mapred_notfound_failover.erl +++ b/tests/mapred_notfound_failover.erl @@ -41,7 +41,7 @@ confirm() -> %% notfound by killing a vnode rt:set_backend(memory), - Nodes = rt:build_cluster(3), + Nodes = rt_cluster:build_cluster(3), %% for our custom reduce phase rt:load_modules_on_nodes([?MODULE], Nodes), diff --git a/tests/mapred_search_switch.erl b/tests/mapred_search_switch.erl index 85434e20f..1b9bbfc8c 100644 --- a/tests/mapred_search_switch.erl +++ b/tests/mapred_search_switch.erl @@ -67,7 +67,7 @@ setup_test_env() -> %% must enable both RS and YZ at startup to get test data indexed; %% nothing extra would be tested by using multiple nodes, so just %% deploy one to make the test run faster - Nodes = rt:deploy_nodes(1, [{riak_search, [{enabled, true}]}, + Nodes = rt_cluster:deploy_nodes(1, [{riak_search, [{enabled, true}]}, {yokozuna, [{enabled, true}]}]), ok = rt:wait_until_nodes_ready(Nodes), ok = rt:wait_for_cluster_service(Nodes, riak_search), diff --git a/tests/mapred_verify_rt.erl b/tests/mapred_verify_rt.erl index c09a63e5b..b7f4e9b98 100644 --- a/tests/mapred_verify_rt.erl +++ b/tests/mapred_verify_rt.erl @@ -29,7 +29,7 @@ confirm() -> lager:info("Build ~b node cluster", [?NODE_COUNT]), - Nodes = rt:build_cluster(?NODE_COUNT), + Nodes = rt_cluster:build_cluster(?NODE_COUNT), %% @todo longer term fix is probably one or more of: diff --git a/tests/overload.erl b/tests/overload.erl index cd34fd9c6..740cfcb11 100644 --- a/tests/overload.erl +++ b/tests/overload.erl @@ -72,17 +72,24 @@ setup() -> {vnode_overload_threshold, undefined}]}, {riak_kv, [{fsm_limit, undefined}, {storage_backend, riak_kv_memory_backend}, - {anti_entropy_build_limit, {100, 1000}}, - {anti_entropy_concurrency, 100}, - {anti_entropy_tick, 100}, - {anti_entropy, {on, []}}, - {anti_entropy_timeout, 5000}]}, - {riak_api, [{pb_backlog, 1024}]}], - ensemble_util:build_cluster(5, Config, 5). - -test_no_overload_protection(_Nodes, _BKV, true) -> - ok; -test_no_overload_protection(Nodes, BKV, ConsistentType) -> + {anti_entropy, {off, []}}]}], + Nodes = rt_cluster:build_cluster(2, Config), + [_Node1, Node2] = Nodes, + + Ring = rt:get_ring(Node2), + Hash = riak_core_util:chash_std_keyfun({?BUCKET, ?KEY}), + PL = lists:sublist(riak_core_ring:preflist(Hash, Ring), 3), + Victim = hd([Idx || {Idx, Node} <- PL, + Node =:= Node2]), + RO = riak_object:new(?BUCKET, ?KEY, <<"test">>), + + + ok = test_no_overload_protection(Nodes, Victim, RO), + ok = test_vnode_protection(Nodes, Victim, RO), + ok = test_fsm_protection(Nodes, Victim, RO), + pass. + +test_no_overload_protection(Nodes, Victim, RO) -> lager:info("Testing with no overload protection"), ProcFun = build_predicate_gte(test_no_overload_protection, ?NUM_REQUESTS, "ProcFun", "Procs"), diff --git a/tests/partition_repair.erl b/tests/partition_repair.erl index 62ba5806c..757aabf85 100644 --- a/tests/partition_repair.erl +++ b/tests/partition_repair.erl @@ -72,7 +72,7 @@ confirm() -> %% [{"./log/console.log",debug,10485760,"$D0",5}]}]}]} ], - Nodes = rt:build_cluster(NumNodes, Conf), + Nodes = rt_cluster:build_cluster(NumNodes, Conf), case NVal of undefined -> @@ -120,7 +120,7 @@ kill_repair_verify({Partition, Node}, DataSuffix, Service) -> %% kill the partition data Path = DataSuffix ++ "/" ++ integer_to_list(Partition), lager:info("Killing data for ~p on ~p at ~s", [Partition, Node, Path]), - rt:clean_data_dir([Node], Path), + rt_cluster:clean_data_dir([Node], Path), %% force restart of vnode since some data is kept in memory lager:info("Restarting ~p vnode for ~p on ~p", [Service, Partition, Node]), diff --git a/tests/pb_cipher_suites.erl b/tests/pb_cipher_suites.erl index 5618c89e2..afbe93f8f 100644 --- a/tests/pb_cipher_suites.erl +++ b/tests/pb_cipher_suites.erl @@ -49,7 +49,7 @@ confirm() -> ]} ], - Nodes = rt:build_cluster(4, Conf), + Nodes = rt_cluster:build_cluster(4, Conf), Node = hd(Nodes), %% enable security on the cluster ok = rpc:call(Node, riak_core_console, security_enable, [[]]), diff --git a/tests/pb_security.erl b/tests/pb_security.erl index 3b06d1d7a..543508013 100644 --- a/tests/pb_security.erl +++ b/tests/pb_security.erl @@ -72,7 +72,7 @@ confirm() -> _ -> true end, - Nodes = rt:build_cluster(4, Conf), + Nodes = rt_cluster:build_cluster(4, Conf), Node = hd(Nodes), %% enable security on the cluster ok = rpc:call(Node, riak_core_console, security_enable, [[]]), diff --git a/tests/pipe_verify_basics.erl b/tests/pipe_verify_basics.erl index 67ea4f5e7..27b20cf2c 100644 --- a/tests/pipe_verify_basics.erl +++ b/tests/pipe_verify_basics.erl @@ -40,7 +40,7 @@ confirm() -> lager:info("Build ~b node cluster", [?NODE_COUNT]), - Nodes = rt:build_cluster(?NODE_COUNT), + Nodes = rt_cluster:build_cluster(?NODE_COUNT), [rt:wait_for_service(Node, riak_pipe) || Node <- Nodes], diff --git a/tests/pipe_verify_exceptions.erl b/tests/pipe_verify_exceptions.erl index fd0b42c59..9dcd21e19 100644 --- a/tests/pipe_verify_exceptions.erl +++ b/tests/pipe_verify_exceptions.erl @@ -45,7 +45,7 @@ %% @doc riak_test callback confirm() -> lager:info("Build ~b node cluster", [?NODE_COUNT]), - Nodes = rt:build_cluster(?NODE_COUNT), + Nodes = rt_cluster:build_cluster(?NODE_COUNT), [rt:wait_for_service(Node, riak_pipe) || Node <- Nodes], diff --git a/tests/pipe_verify_handoff.erl b/tests/pipe_verify_handoff.erl index ac578e5a8..f0f3f2251 100644 --- a/tests/pipe_verify_handoff.erl +++ b/tests/pipe_verify_handoff.erl @@ -62,7 +62,7 @@ confirm() -> lager:info("Start ~b nodes", [?NODE_COUNT]), NodeDefs = lists:duplicate(?NODE_COUNT, {current, default}), Services = [riak_pipe], - [Primary,Secondary] = Nodes = rt:deploy_nodes(NodeDefs, Services), + [Primary,Secondary] = Nodes = rt_cluster:deploy_nodes(NodeDefs, Services), %% Ensure each node owns 100% of it's own ring [?assertEqual([Node], rt:owners_according_to(Node)) || Node <- Nodes], diff --git a/tests/pipe_verify_handoff_blocking.erl b/tests/pipe_verify_handoff_blocking.erl index 4c48bdabc..8daff0091 100644 --- a/tests/pipe_verify_handoff_blocking.erl +++ b/tests/pipe_verify_handoff_blocking.erl @@ -82,7 +82,7 @@ confirm() -> lager:info("Start ~b nodes", [?NODE_COUNT]), NodeDefs = lists:duplicate(?NODE_COUNT, {current, default}), Services = [riak_pipe], - [Primary,Secondary] = Nodes = rt:deploy_nodes(NodeDefs, Services), + [Primary,Secondary] = Nodes = rt_cluster:deploy_nodes(NodeDefs, Services), %% Ensure each node owns 100% of it's own ring [?assertEqual([Node], rt:owners_according_to(Node)) || Node <- Nodes], diff --git a/tests/pipe_verify_restart_input_forwarding.erl b/tests/pipe_verify_restart_input_forwarding.erl index 1df9301ac..392495a1c 100644 --- a/tests/pipe_verify_restart_input_forwarding.erl +++ b/tests/pipe_verify_restart_input_forwarding.erl @@ -52,7 +52,7 @@ %% @doc riak_test callback confirm() -> lager:info("Build ~b node cluster", [?NODE_COUNT]), - Nodes = rt:build_cluster(?NODE_COUNT), + Nodes = rt_cluster:build_cluster(?NODE_COUNT), [rt:wait_for_service(Node, riak_pipe) || Node <- Nodes], diff --git a/tests/pipe_verify_sink_types.erl b/tests/pipe_verify_sink_types.erl index 32b42d50c..d3ebd7916 100644 --- a/tests/pipe_verify_sink_types.erl +++ b/tests/pipe_verify_sink_types.erl @@ -39,7 +39,7 @@ %% @doc riak_test callback confirm() -> lager:info("Build ~b node cluster", [?NODE_COUNT]), - Nodes = rt:build_cluster(?NODE_COUNT), + Nodes = rt_cluster:build_cluster(?NODE_COUNT), [rt:wait_for_service(Node, riak_pipe) || Node <- Nodes], diff --git a/tests/post_generate_key.erl b/tests/post_generate_key.erl index b8197caa9..de77942b9 100644 --- a/tests/post_generate_key.erl +++ b/tests/post_generate_key.erl @@ -25,7 +25,7 @@ -include_lib("eunit/include/eunit.hrl"). confirm() -> - Nodes = rt:build_cluster(1), + Nodes = rt_cluster:build_cluster(1), ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), [Base|_] = rt:http_url(Nodes), diff --git a/tests/pr_pw.erl b/tests/pr_pw.erl index 15b5a59a7..07b31ec30 100644 --- a/tests/pr_pw.erl +++ b/tests/pr_pw.erl @@ -8,7 +8,7 @@ confirm() -> application:start(inets), lager:info("Deploy some nodes"), - Nodes = rt:build_cluster(4), + Nodes = rt_cluster:build_cluster(4), %% calculate the preflist for foo/bar {ok, Ring} = rpc:call(hd(Nodes), riak_core_ring_manager, get_my_ring, []), diff --git a/tests/replication/repl_aae_fullsync.erl b/tests/replication/repl_aae_fullsync.erl index a04e86012..1a60ac0b5 100644 --- a/tests/replication/repl_aae_fullsync.erl +++ b/tests/replication/repl_aae_fullsync.erl @@ -113,7 +113,7 @@ simple_test() -> %% intercepts are removed. validate_completed_fullsync(LeaderA, BFirst, "B", 1, ?NUM_KEYS), - rt:clean_cluster(Nodes), + rt_cluster:clean_cluster(Nodes), pass. @@ -296,7 +296,7 @@ bidirectional_test() -> validate_completed_fullsync(LeaderB, AFirst, "A", ?NUM_KEYS + 1, ?NUM_KEYS + ?NUM_KEYS), %% Clean. - rt:clean_cluster(Nodes), + rt_cluster:clean_cluster(Nodes), pass. @@ -388,7 +388,7 @@ difference_test() -> [{timeout, 4000}]), ?assertEqual([<<"baz">>, <<"baz2">>], lists:sort(riakc_obj:get_values(O2))), - rt:clean_cluster(Nodes), + rt_cluster:clean_cluster(Nodes), pass. @@ -457,7 +457,7 @@ deadlock_test() -> lager:info("Status result: ~p", [Result]), ?assertNotEqual({badrpc, timeout}, Result), - rt:clean_cluster(Nodes), + rt_cluster:clean_cluster(Nodes), pass. diff --git a/tests/replication/repl_bucket_types.erl b/tests/replication/repl_bucket_types.erl index d9d444976..8d17a5a11 100644 --- a/tests/replication/repl_bucket_types.erl +++ b/tests/replication/repl_bucket_types.erl @@ -57,7 +57,7 @@ cleanup({ClusterNodes, _Types, PBA, PBB}, CleanCluster) -> {_, _, ANodes, BNodes} = ClusterNodes, case CleanCluster of true -> - rt:clean_cluster(ANodes ++ BNodes); + rt_cluster:clean_cluster(ANodes ++ BNodes); false -> ok end. @@ -347,10 +347,10 @@ cluster_conf() -> ]. deploy_nodes(NumNodes, current) -> - rt:deploy_nodes(NumNodes, cluster_conf(), [riak_kv, riak_repl]); + rt_cluster:deploy_nodes(NumNodes, cluster_conf()); deploy_nodes(_, mixed) -> Conf = cluster_conf(), - rt:deploy_nodes([{current, Conf}, {previous, Conf}], [riak_kv, riak_repl]). + rt_cluster:deploy_nodes([{current, Conf}, {previous, Conf}]). %% @doc Create two clusters of 1 node each and connect them for replication: %% Cluster "A" -> cluster "B" diff --git a/tests/replication/repl_cancel_fullsync.erl b/tests/replication/repl_cancel_fullsync.erl index 5248da7bd..2763c2469 100644 --- a/tests/replication/repl_cancel_fullsync.erl +++ b/tests/replication/repl_cancel_fullsync.erl @@ -35,7 +35,7 @@ confirm() -> rt_config:set_advanced_conf(all, ?CONF(5)), - Nodes = [ANodes, BNodes] = rt:build_clusters([3, 3]), + Nodes = [ANodes, BNodes] = rt_cluster:build_clusters([3, 3]), rt:wait_for_cluster_service(ANodes, riak_repl), rt:wait_for_cluster_service(BNodes, riak_repl), @@ -141,7 +141,7 @@ confirm() -> lager:info("Fullsync Complete"), rt:log_to_nodes(Nodes, "Test completed."), - rt:clean_cluster(ANodes), - rt:clean_cluster(BNodes), + rt_cluster:clean_cluster(ANodes), + rt_cluster:clean_cluster(BNodes), pass. diff --git a/tests/replication/repl_consistent_object_filter.erl b/tests/replication/repl_consistent_object_filter.erl index a75b534cc..8766322d8 100644 --- a/tests/replication/repl_consistent_object_filter.erl +++ b/tests/replication/repl_consistent_object_filter.erl @@ -103,7 +103,7 @@ make_clusters() -> ]} ], - Nodes = rt:deploy_nodes(NumNodes, Conf, [riak_kv, riak_repl]), + Nodes = rt_cluster:deploy_nodes(NumNodes, Conf), {ANodes, BNodes} = lists:split(ClusterASize, Nodes), lager:info("ANodes: ~p", [ANodes]), lager:info("BNodes: ~p", [BNodes]), diff --git a/tests/replication/repl_fs_bench.erl b/tests/replication/repl_fs_bench.erl index 6b2b257e9..a9467950b 100644 --- a/tests/replication/repl_fs_bench.erl +++ b/tests/replication/repl_fs_bench.erl @@ -62,7 +62,7 @@ confirm() -> fullsync_test(Strategy, Latency) -> rt_config:set_advanced_conf(all, ?CONF(Strategy)), - [ANodes, BNodes] = rt:build_clusters([3, 3]), + [ANodes, BNodes] = rt_cluster:build_clusters([3, 3]), rt:wait_for_cluster_service(ANodes, riak_repl), rt:wait_for_cluster_service(BNodes, riak_repl), @@ -150,7 +150,7 @@ fullsync_test(Strategy, Latency) -> start_and_wait_until_fullsync_complete, [LeaderA]), - rt:clean_cluster(ANodes), - rt:clean_cluster(BNodes), + rt_cluster:clean_cluster(ANodes), + rt_cluster:clean_cluster(BNodes), {EmptyTime, FullTime, DiffTime, NoneTime}. diff --git a/tests/replication/repl_fs_stat_caching.erl b/tests/replication/repl_fs_stat_caching.erl index 43f2cf887..6ecb655f0 100644 --- a/tests/replication/repl_fs_stat_caching.erl +++ b/tests/replication/repl_fs_stat_caching.erl @@ -38,7 +38,7 @@ setup() -> NodeCount = rt_config:get(num_nodes, 6), lager:info("Deploy ~p nodes", [NodeCount]), - Nodes = rt:deploy_nodes(NodeCount, cluster_conf(), [riak_kv, riak_repl]), + Nodes = rt_cluster:deploy_nodes(NodeCount, cluster_conf()), SplitSize = NodeCount div 2, {SourceNodes, SinkNodes} = lists:split(SplitSize, Nodes), diff --git a/tests/replication/repl_location_failures.erl b/tests/replication/repl_location_failures.erl index 5b1007552..f09e9f5f8 100644 --- a/tests/replication/repl_location_failures.erl +++ b/tests/replication/repl_location_failures.erl @@ -36,7 +36,7 @@ confirm() -> rt_config:set_advanced_conf(all, ?CONF(5)), - [ANodes, BNodes] = rt:build_clusters([3, 3]), + [ANodes, BNodes] = rt_cluster:build_clusters([3, 3]), rt:wait_for_cluster_service(ANodes, riak_repl), rt:wait_for_cluster_service(BNodes, riak_repl), @@ -102,7 +102,7 @@ confirm() -> repl_util:validate_completed_fullsync(LeaderA, BFirst, "B", 1, ?NUM_KEYS, ?TEST_BUCKET), - rt:clean_cluster(ANodes), - rt:clean_cluster(BNodes), + rt_cluster:clean_cluster(ANodes), + rt_cluster:clean_cluster(BNodes), pass. diff --git a/tests/replication/repl_rt_cascading_rtq.erl b/tests/replication/repl_rt_cascading_rtq.erl index 9b6debe93..64939bae0 100644 --- a/tests/replication/repl_rt_cascading_rtq.erl +++ b/tests/replication/repl_rt_cascading_rtq.erl @@ -153,9 +153,9 @@ cluster_conf(_CascadingWrites) -> ]. deploy_nodes(NumNodes, true) -> - rt:deploy_nodes(NumNodes, cluster_conf(always), [riak_kv, riak_repl]); + rt_cluster:deploy_nodes(NumNodes, cluster_conf(always)); deploy_nodes(NumNodes, false) -> - rt:deploy_nodes(NumNodes, cluster_conf(never), [riak_kv, riak_repl]). + rt_cluster:deploy_nodes(NumNodes, cluster_conf(never)). %% @doc Turn on Realtime replication on the cluster lead by LeaderA. %% The clusters must already have been named and connected. diff --git a/tests/replication/repl_rt_heartbeat.erl b/tests/replication/repl_rt_heartbeat.erl index 7306404a1..8cf4a3614 100644 --- a/tests/replication/repl_rt_heartbeat.erl +++ b/tests/replication/repl_rt_heartbeat.erl @@ -174,7 +174,7 @@ make_connected_clusters() -> ]} ], - Nodes = rt:deploy_nodes(NumNodes, Conf, [riak_kv, riak_repl]), + Nodes = rt_cluster:deploy_nodes(NumNodes, Conf), {ANodes, BNodes} = lists:split(ClusterASize, Nodes), lager:info("ANodes: ~p", [ANodes]), diff --git a/tests/replication/repl_rt_overload.erl b/tests/replication/repl_rt_overload.erl index 1440d5b1e..e425e02b3 100644 --- a/tests/replication/repl_rt_overload.erl +++ b/tests/replication/repl_rt_overload.erl @@ -128,7 +128,7 @@ make_connected_clusters() -> ]} ], - Nodes = rt:deploy_nodes(NumNodes, Conf, [riak_kv, riak_repl]), + Nodes = rt_cluster:deploy_nodes(NumNodes, Conf), {ANodes, BNodes} = lists:split(ClusterASize, Nodes), lager:info("ANodes: ~p", [ANodes]), lager:info("BNodes: ~p", [BNodes]), diff --git a/tests/replication/repl_rt_pending.erl b/tests/replication/repl_rt_pending.erl index 24754389a..e393e979a 100644 --- a/tests/replication/repl_rt_pending.erl +++ b/tests/replication/repl_rt_pending.erl @@ -174,7 +174,7 @@ make_connected_clusters() -> ]} ], - Nodes = rt:deploy_nodes(NumNodes, Conf, [riak_kv, riak_repl]), + Nodes = rt_cluster:deploy_nodes(NumNodes, Conf), {ANodes, BNodes} = lists:split(ClusterASize, Nodes), lager:info("ANodes: ~p", [ANodes]), diff --git a/tests/replication/replication.erl b/tests/replication/replication.erl index 389fd5c49..6a4c04bc3 100644 --- a/tests/replication/replication.erl +++ b/tests/replication/replication.erl @@ -17,9 +17,7 @@ confirm() -> ]} ], rt_config:set_advanced_conf(all, Conf), - [ANodes, BNodes] = rt:build_clusters([3, 3]), - rt:wait_for_cluster_service(ANodes, riak_repl), - rt:wait_for_cluster_service(BNodes, riak_repl), + [ANodes, BNodes] = rt_cluster:build_clusters([3, 3]), replication(ANodes, BNodes, false), pass. diff --git a/tests/replication/replication2_connections.erl b/tests/replication/replication2_connections.erl index 9172b3aa7..760375b30 100644 --- a/tests/replication/replication2_connections.erl +++ b/tests/replication/replication2_connections.erl @@ -47,7 +47,7 @@ simple_test() -> rt_config:set_advanced_conf(all, Conf), - [ANodes, BNodes] = rt:build_clusters([3, 3]), + [ANodes, BNodes] = rt_cluster:build_clusters([3, 3]), rt:wait_for_cluster_service(ANodes, riak_repl), rt:wait_for_cluster_service(BNodes, riak_repl), @@ -116,7 +116,7 @@ disconnect_test() -> rt_config:set_advanced_conf(all, Conf), - [ANodes, BNodes] = rt:build_clusters([3, 3]), + [ANodes, BNodes] = rt_cluster:build_clusters([3, 3]), lager:info("ANodes: ~p", [ANodes]), lager:info("BNodes: ~p", [BNodes]), @@ -159,8 +159,8 @@ disconnect_test() -> lager:info("Verifying disconnect from B to A."), [verify_disconnect(Node, "A") || Node <- BNodes], - rt:clean_cluster(ANodes), - rt:clean_cluster(BNodes), + rt_cluster:clean_cluster(ANodes), + rt_cluster:clean_cluster(BNodes), pass. @@ -184,7 +184,7 @@ error_cleanup_test() -> rt_config:set_advanced_conf(all, Conf), - [ANodes, BNodes] = rt:build_clusters([3, 3]), + [ANodes, BNodes] = rt_cluster:build_clusters([3, 3]), lager:info("ANodes: ~p", [ANodes]), lager:info("BNodes: ~p", [BNodes]), @@ -236,8 +236,8 @@ error_cleanup_test() -> lager:info("Verifying connection from A to B"), [verify_connectivity(Node, "B") || Node <- ANodes], - rt:clean_cluster(ANodes), - rt:clean_cluster(BNodes), + rt_cluster:clean_cluster(ANodes), + rt_cluster:clean_cluster(BNodes), pass. diff --git a/tests/replication/replication2_console_tests.erl b/tests/replication/replication2_console_tests.erl index 068aa7cd0..c090bf9a1 100644 --- a/tests/replication/replication2_console_tests.erl +++ b/tests/replication/replication2_console_tests.erl @@ -48,7 +48,7 @@ confirm() -> %% Deploy a node to test against lager:info("Deploy node to test riak-repl command line"), - [Node] = rt:deploy_nodes(1, [], [riak_kv, riak_repl]), + [Node] = rt_cluster:deploy_nodes(1), ?assertEqual(ok, rt:wait_until_nodes_ready([Node])), rt_intercept:add(Node, {riak_repl_console, diff --git a/tests/replication/replication2_fsschedule.erl b/tests/replication/replication2_fsschedule.erl index 9f455aaec..e49d55b72 100644 --- a/tests/replication/replication2_fsschedule.erl +++ b/tests/replication/replication2_fsschedule.erl @@ -94,7 +94,7 @@ test_multiple_schedules() -> lager:info("Waiting for fullsyncs"), wait_until_fullsyncs(LeaderA, "B", 5), wait_until_fullsyncs(LeaderA, "C", 5), - rt:clean_cluster(AllNodes), + rt_cluster:clean_cluster(AllNodes), pass. test_single_schedule() -> @@ -113,7 +113,7 @@ test_single_schedule() -> lager:info("Waiting for fullsyncs"), wait_until_fullsyncs(LeaderA, "B", 10), wait_until_fullsyncs(LeaderA, "C", 10), - rt:clean_cluster(AllNodes), + rt_cluster:clean_cluster(AllNodes), pass. test_mixed_12_13() -> @@ -147,7 +147,7 @@ test_mixed_12_13() -> wait_until_fullsyncs(LeaderA, "B", 3), wait_until_fullsyncs(LeaderA, "C", 3), wait_until_12_fs_complete(LeaderA, 9), - rt:clean_cluster(AllNodes), + rt_cluster:clean_cluster(AllNodes), pass. diff --git a/tests/replication/replication2_pg.erl b/tests/replication/replication2_pg.erl index 27be6b416..82261724b 100644 --- a/tests/replication/replication2_pg.erl +++ b/tests/replication/replication2_pg.erl @@ -65,7 +65,7 @@ setup_repl_clusters(Conf, SSL) -> rt_config:set_advanced_conf(all, Conf), - Nodes = [ANodes, BNodes, CNodes] = rt:build_clusters([2, 2, 2]), + Nodes = [ANodes, BNodes, CNodes] = rt_cluster:build_clusters([2, 2, 2]), rt:wait_for_cluster_service(ANodes, riak_repl), rt:wait_for_cluster_service(BNodes, riak_repl), diff --git a/tests/replication/replication2_rt_sink_connection.erl b/tests/replication/replication2_rt_sink_connection.erl index 4c38a7d82..c9348fd7c 100644 --- a/tests/replication/replication2_rt_sink_connection.erl +++ b/tests/replication/replication2_rt_sink_connection.erl @@ -43,7 +43,7 @@ confirm() -> ]} ], - Nodes = rt:deploy_nodes(NumNodes, Conf, [riak_kv, riak_repl]), + Nodes = rt_cluster:deploy_nodes(NumNodes, Conf), {ANodes, Rest} = lists:split(2, Nodes), {BNodes, CNodes} = lists:split(2, Rest), diff --git a/tests/replication/replication2_upgrade.erl b/tests/replication/replication2_upgrade.erl index 987ccf08d..6f1ac3067 100644 --- a/tests/replication/replication2_upgrade.erl +++ b/tests/replication/replication2_upgrade.erl @@ -32,7 +32,7 @@ confirm() -> NodeConfig = [{FromVersion, Conf} || _ <- lists:seq(1, NumNodes)], - Nodes = rt:deploy_nodes(NodeConfig, [riak_kv, riak_repl]), + Nodes = rt_cluster:deploy_nodes(NodeConfig), NodeUpgrades = case UpgradeOrder of "forwards" -> diff --git a/tests/replication/replication_object_reformat.erl b/tests/replication/replication_object_reformat.erl index 4ece6432a..2f29c9e2d 100644 --- a/tests/replication/replication_object_reformat.erl +++ b/tests/replication/replication_object_reformat.erl @@ -159,14 +159,14 @@ verify_replication(AVersion, BVersion, Start, End, Realtime) -> ok end, - rt:clean_cluster(lists:flatten(Nodes)). + rt_cluster:clean_cluster(lists:flatten(Nodes)). %% @doc Configure two clusters and set up replication between them, %% return the node list of each cluster. configure_clusters(AVersion, BVersion, Realtime) -> rt_config:set_advanced_conf(all, ?CONF(infinity)), - Nodes = [ANodes, BNodes] = rt:build_clusters([3, 3]), + Nodes = [ANodes, BNodes] = rt_cluster:build_clusters([3, 3]), rt:wait_for_cluster_service(ANodes, riak_repl), rt:wait_for_cluster_service(BNodes, riak_repl), diff --git a/tests/replication/replication_ssl.erl b/tests/replication/replication_ssl.erl index 31394668d..800c706a9 100644 --- a/tests/replication/replication_ssl.erl +++ b/tests/replication/replication_ssl.erl @@ -163,7 +163,7 @@ confirm() -> lager:info("===testing basic connectivity"), - [Node1, Node2] = rt:deploy_nodes(2, BaseConf, [riak_kv, riak_repl]), + [Node1, Node2] = rt_cluster:deploy_nodes(2, BaseConf), Listeners = replication:add_listeners([Node1]), replication:verify_listeners(Listeners), @@ -225,7 +225,7 @@ confirm() -> lager:info("Re-deploying 6 nodes"), - Nodes = rt:deploy_nodes(6, BaseConf, [riak_kv, riak_repl]), + Nodes = rt_cluster:deploy_nodes(6, BaseConf), [rt:wait_until_pingable(N) || N <- Nodes], diff --git a/tests/replication/replication_stats.erl b/tests/replication/replication_stats.erl index d699ed5ff..1adcbd87a 100644 --- a/tests/replication/replication_stats.erl +++ b/tests/replication/replication_stats.erl @@ -41,7 +41,7 @@ confirm() -> fullsync_enabled_and_started() -> rt_config:set_advanced_conf(all, ?CONF), - [ANodes, BNodes] = rt:build_clusters([3, 3]), + [ANodes, BNodes] = rt_cluster:build_clusters([3, 3]), rt:wait_for_cluster_service(ANodes, riak_repl), rt:wait_for_cluster_service(BNodes, riak_repl), @@ -102,7 +102,7 @@ fullsync_enabled_and_started() -> fail end, - rt:clean_cluster(ANodes), - rt:clean_cluster(BNodes), + rt_cluster:clean_cluster(ANodes), + rt_cluster:clean_cluster(BNodes), Result. diff --git a/tests/replication/replication_upgrade.erl b/tests/replication/replication_upgrade.erl index 10ddf6270..e4bb26169 100644 --- a/tests/replication/replication_upgrade.erl +++ b/tests/replication/replication_upgrade.erl @@ -25,7 +25,7 @@ confirm() -> NodeConfig = [{FromVersion, Conf} || _ <- lists:seq(1, NumNodes)], - Nodes = rt:deploy_nodes(NodeConfig, [riak_kv, riak_repl]), + Nodes = rt_cluster:deploy_nodes(NodeConfig), NodeUpgrades = case UpgradeOrder of "forwards" -> diff --git a/tests/replication/rt_cascading.erl b/tests/replication/rt_cascading.erl index a47e29b51..c8827c6b3 100644 --- a/tests/replication/rt_cascading.erl +++ b/tests/replication/rt_cascading.erl @@ -64,7 +64,7 @@ simple_test_() -> % +-----------+ +--------+ +-----+ {timeout, timeout(90), {setup, fun() -> Conf = conf(), - [BeginNode, MiddleNode, EndNode] = Nodes = rt:deploy_nodes(3, Conf), + [BeginNode, MiddleNode, EndNode] = Nodes = rt_cluster:deploy_nodes(3, Conf), repl_util:make_cluster([BeginNode]), repl_util:make_cluster([MiddleNode]), repl_util:make_cluster([EndNode]), @@ -78,7 +78,7 @@ simple_test_() -> fun(State) -> Nodes = [State#simple_state.beginning, State#simple_state.middle, State#simple_state.ending], - rt:clean_cluster(Nodes) + rt_cluster:clean_cluster(Nodes) end, fun(State) -> [ @@ -161,7 +161,7 @@ big_circle_test_() -> % +---+ {timeout, timeout(130), {setup, fun() -> Conf = conf(), - Nodes = rt:deploy_nodes(6, Conf), + Nodes = rt_cluster:deploy_nodes(6, Conf), [repl_util:make_cluster([N]) || N <- Nodes], [repl_util:wait_until_is_leader(N) || N <- Nodes], Names = ["1", "2", "3", "4", "5", "6"], @@ -181,7 +181,7 @@ big_circle_test_() -> Nodes end, fun(Nodes) -> - rt:clean_cluster(Nodes) + rt_cluster:clean_cluster(Nodes) end, fun(Nodes) -> [ @@ -262,7 +262,7 @@ circle_test_() -> % +-------+ +-----+ {timeout, timeout(30), {setup, fun() -> Conf = conf(), - [One, Two, Three] = Nodes = rt:deploy_nodes(3, Conf), + [One, Two, Three] = Nodes = rt_cluster:deploy_nodes(3, Conf), [repl_util:make_cluster([N]) || N <- Nodes], [repl_util:wait_until_is_leader(N) || N <- Nodes], Names = ["one", "two", "three"], @@ -280,7 +280,7 @@ circle_test_() -> Nodes end, fun(Nodes) -> - rt:clean_cluster(Nodes) + rt_cluster:clean_cluster(Nodes) end, fun(Nodes) -> [ @@ -333,7 +333,7 @@ pyramid_test_() -> {timeout, timeout(70), {setup, fun() -> Conf = conf(), - [Top, Left, Left2, Right, Right2] = Nodes = rt:deploy_nodes(5, Conf), + [Top, Left, Left2, Right, Right2] = Nodes = rt_cluster:deploy_nodes(5, Conf), [repl_util:make_cluster([N]) || N <- Nodes], [repl_util:wait_until_is_leader(N) || N <- Nodes], Names = ["top", "left", "left2", "right", "right2"], @@ -349,7 +349,7 @@ pyramid_test_() -> Nodes end, fun(Nodes) -> - rt:clean_cluster(Nodes) + rt_cluster:clean_cluster(Nodes) end, fun(Nodes) -> [ @@ -387,7 +387,7 @@ diamond_test_() -> % +--------+ {timeout, timeout(180), {setup, fun() -> Conf = conf(), - [Top, MidLeft, MidRight, Bottom] = Nodes = rt:deploy_nodes(4, Conf), + [Top, MidLeft, MidRight, Bottom] = Nodes = rt_cluster:deploy_nodes(4, Conf), [repl_util:make_cluster([N]) || N <- Nodes], Names = ["top", "midleft", "midright", "bottom"], [repl_util:name_cluster(Node, Name) || {Node, Name} <- lists:zip(Nodes, Names)], @@ -403,7 +403,7 @@ diamond_test_() -> Nodes end, fun(Nodes) -> - rt:clean_cluster(Nodes) + rt_cluster:clean_cluster(Nodes) end, fun(Nodes) -> [ @@ -479,7 +479,7 @@ circle_and_spurs_test_() -> % +-----------+ +------+ +------+ +-----------+ {timeout, timeout(170), {setup, fun() -> Conf = conf(), - [North, East, West, NorthSpur, EastSpur, WestSpur] = Nodes = rt:deploy_nodes(6, Conf), + [North, East, West, NorthSpur, EastSpur, WestSpur] = Nodes = rt_cluster:deploy_nodes(6, Conf), [repl_util:make_cluster([N]) || N <- Nodes], Names = ["north", "east", "west", "north_spur", "east_spur", "west_spur"], [repl_util:name_cluster(Node, Name) || {Node, Name} <- lists:zip(Nodes, Names)], @@ -493,7 +493,7 @@ circle_and_spurs_test_() -> Nodes end, fun(Nodes) -> - rt:clean_cluster(Nodes) + rt_cluster:clean_cluster(Nodes) end, fun(Nodes) -> [ @@ -580,7 +580,7 @@ mixed_version_clusters_test_dep() -> {timeout, 60000, {setup, fun() -> Conf = conf(), DeployConfs = [{previous, Conf} || _ <- lists:seq(1,6)], - Nodes = rt:deploy_nodes(DeployConfs), + Nodes = rt_cluster:deploy_nodes(DeployConfs), [N1, N2, N3, N4, N5, N6] = Nodes, case rpc:call(N1, application, get_key, [riak_core, vsn]) of % this is meant to test upgrading from early BNW aka @@ -611,7 +611,7 @@ mixed_version_clusters_test_dep() -> {too_old, Ns} -> Ns; _ -> MaybeNodes end, - rt:clean_cluster(Nodes) + rt_cluster:clean_cluster(Nodes) end, fun({too_old, _Nodes}) -> []; ([N1, N2, N3, N4, N5, N6] = Nodes) -> [ @@ -793,7 +793,7 @@ new_to_old_test_dep() -> {timeout, timeout(105), {setup, fun() -> Conf = conf(), DeployConfs = [{current, Conf}, {previous, Conf}, {current, Conf}], - [New1, Old2, New3] = Nodes = rt:deploy_nodes(DeployConfs), + [New1, Old2, New3] = Nodes = rt_cluster:deploy_nodes(DeployConfs), case rpc:call(Old2, application, get_key, [riak_core, vsn]) of % this is meant to test upgrading from early BNW aka % Brave New World aka Advanced Repl aka version 3 repl to @@ -817,7 +817,7 @@ new_to_old_test_dep() -> {too_old, Ns} -> Ns; _ -> MaybeNodes end, - rt:clean_cluster(Nodes) + rt_cluster:clean_cluster(Nodes) end, fun({too_old, _}) -> []; ([New1, Old2, New3]) -> [ @@ -873,7 +873,7 @@ new_to_old_test_dep() -> ensure_ack_test_() -> {timeout, timeout(130), {setup, fun() -> Conf = conf(), - [LeaderA, LeaderB] = Nodes = rt:deploy_nodes(2, Conf), + [LeaderA, LeaderB] = Nodes = rt_cluster:deploy_nodes(2, Conf), [repl_util:make_cluster([N]) || N <- Nodes], [repl_util:wait_until_is_leader(N) || N <- Nodes], Names = ["A", "B"], @@ -888,7 +888,7 @@ ensure_ack_test_() -> [LeaderA, LeaderB] end, fun(Nodes) -> - rt:clean_cluster(Nodes) + rt_cluster:clean_cluster(Nodes) end, fun([LeaderA, LeaderB] = _Nodes) -> [ @@ -928,7 +928,7 @@ ensure_unacked_and_queue() -> ensure_unacked_and_queue_test_() -> {timeout, timeout(2300), {setup, fun() -> - Nodes = rt:deploy_nodes(6, conf()), + Nodes = rt_cluster:deploy_nodes(6, conf()), {N123, N456} = lists:split(3, Nodes), repl_util:make_cluster(N123), repl_util:make_cluster(N456), @@ -943,8 +943,8 @@ ensure_unacked_and_queue_test_() -> {N123, N456} end, maybe_skip_teardown(fun({N123, N456}) -> - rt:clean_cluster(N123), - rt:clean_cluster(N456) + rt_cluster:clean_cluster(N123), + rt_cluster:clean_cluster(N456) end), fun({N123, N456}) -> [ diff --git a/tests/riak_admin_console_tests.erl b/tests/riak_admin_console_tests.erl index 8084b8424..04bd8ea48 100644 --- a/tests/riak_admin_console_tests.erl +++ b/tests/riak_admin_console_tests.erl @@ -154,7 +154,7 @@ riak_admin_tests(Node) -> confirm() -> %% Deploy a node to test against lager:info("Deploy node to test riak command line"), - [Node] = rt:deploy_nodes(1), + [Node] = rt_cluster:deploy_nodes(1), ?assertEqual(ok, rt:wait_until_nodes_ready([Node])), rt_intercept:add(Node, {riak_core_console, diff --git a/tests/riak_control.erl b/tests/riak_control.erl index 8ad4878b7..661dc093d 100644 --- a/tests/riak_control.erl +++ b/tests/riak_control.erl @@ -46,7 +46,7 @@ verify_upgrade(Vsn) -> lager:info("Verify upgrade from ~p to current.", [Vsn]), lager:info("Building cluster."), - [Nodes] = rt:build_clusters([{3, Vsn, ?RC_ENABLE_CFG}]), + [Nodes] = rt_cluster:build_clusters([{3, Vsn, ?RC_ENABLE_CFG}]), lager:info("Verifying all nodes are alive."), verify_alive(Nodes), diff --git a/tests/riak_control_authentication.erl b/tests/riak_control_authentication.erl index a8bbb7f8f..a1fce9f70 100644 --- a/tests/riak_control_authentication.erl +++ b/tests/riak_control_authentication.erl @@ -209,7 +209,7 @@ verify_authentication(current, ?RC_AUTH_USERLIST_CONFIG_NO_FORCE_SSL) -> %% @doc Build a one node cluster. build_singleton_cluster(Vsn, Config) -> - [Nodes] = rt:build_clusters([{1, Vsn, Config}]), + [Nodes] = rt_cluster:build_clusters([{1, Vsn, Config}]), %% Start and stop, wait for riak_kv. %% diff --git a/tests/riak_rex.erl b/tests/riak_rex.erl index 30ace1353..ed0052e0b 100644 --- a/tests/riak_rex.erl +++ b/tests/riak_rex.erl @@ -33,10 +33,10 @@ rex_test(Node) -> deploy_node(NumNodes, current) -> - rt:deploy_nodes(NumNodes, conf()); + rt_cluster:deploy_nodes(NumNodes, conf()); deploy_node(_, mixed) -> Conf = conf(), - rt:deploy_nodes([{current, Conf}, {previous, Conf}]). + rt_cluster:deploy_nodes([{current, Conf}, {previous, Conf}]). deploy_node(Type) -> NumNodes = rt_config:get(num_nodes, 1), diff --git a/tests/riaknostic_rt.erl b/tests/riaknostic_rt.erl index 85ed822a5..43fa03449 100644 --- a/tests/riaknostic_rt.erl +++ b/tests/riaknostic_rt.erl @@ -29,7 +29,7 @@ confirm() -> %% Build a small cluster - [Node1, _Node2] = rt:build_cluster(2, []), + [Node1, _Node2] = rt_cluster:build_cluster(2, []), ?assertEqual(ok, rt:wait_until_nodes_ready([Node1])), %% Install riaknostic for Riak versions below 1.3.0 diff --git a/tests/rolling_capabilities.erl b/tests/rolling_capabilities.erl index 5b05788b4..e87a9fe72 100644 --- a/tests/rolling_capabilities.erl +++ b/tests/rolling_capabilities.erl @@ -55,7 +55,7 @@ confirm() -> end, lager:info("Deploying Riak ~p cluster", [OldVsn]), - Nodes = rt:build_cluster([OldVsn || _ <- lists:seq(1,Count)]), + Nodes = rt_cluster:build_cluster([OldVsn || _ <- lists:seq(1,Count)]), lists:foldl(fun(Node, Upgraded) -> rt:upgrade(Node, current), Upgraded2 = Upgraded ++ [Node], diff --git a/tests/rt_basic_test.erl b/tests/rt_basic_test.erl index 91881c57f..67aeae122 100644 --- a/tests/rt_basic_test.erl +++ b/tests/rt_basic_test.erl @@ -23,7 +23,7 @@ confirm() -> lager:info("Deploy some nodes"), - Nodes = rt:deploy_nodes(2), + Nodes = rt_cluster:deploy_nodes(2), lager:info("Stop the nodes"), [rt:stop(Node) || Node <- Nodes], pass. diff --git a/tests/sibling_explosion.erl b/tests/sibling_explosion.erl index aee706c68..68b8760a9 100644 --- a/tests/sibling_explosion.erl +++ b/tests/sibling_explosion.erl @@ -21,7 +21,7 @@ confirm() -> Conf = [{riak_core, [{default_bucket_props, [{allow_mult, true}, {dvv_enabled, true}]}]}], - [Node1] = rt:deploy_nodes(1, Conf), + [Node1] = rt_cluster:deploy_nodes(1, Conf), N = 100, lager:info("Put new object in ~p via PBC.", [Node1]), diff --git a/tests/test_cluster.erl b/tests/test_cluster.erl index b19d15bb3..165a0a2d7 100644 --- a/tests/test_cluster.erl +++ b/tests/test_cluster.erl @@ -24,6 +24,6 @@ confirm() -> Config = [{riak_search, [{enabled, true}]}], - rt:build_cluster(4, Config), + rt_cluster:build_cluster(4, Config), ?assert(false), fail. diff --git a/tests/verify_2i_aae.erl b/tests/verify_2i_aae.erl index 1ea4c0e2d..39b7709ef 100644 --- a/tests/verify_2i_aae.erl +++ b/tests/verify_2i_aae.erl @@ -31,7 +31,7 @@ -define(N_VAL, 3). confirm() -> - [Node1] = rt:build_cluster(1, + [Node1] = rt_cluster:build_cluster(1, [{riak_kv, [{anti_entropy, {off, []}}, {anti_entropy_build_limit, {100, 500}}, diff --git a/tests/verify_2i_limit.erl b/tests/verify_2i_limit.erl index 944b371d4..66a7e7a9c 100644 --- a/tests/verify_2i_limit.erl +++ b/tests/verify_2i_limit.erl @@ -31,7 +31,7 @@ confirm() -> inets:start(), - Nodes = rt:build_cluster(3), + Nodes = rt_cluster:build_cluster(3), ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), RiakHttp = rt:httpc(hd(Nodes)), diff --git a/tests/verify_2i_mixed_cluster.erl b/tests/verify_2i_mixed_cluster.erl index 02369948b..95c5d597e 100644 --- a/tests/verify_2i_mixed_cluster.erl +++ b/tests/verify_2i_mixed_cluster.erl @@ -31,7 +31,7 @@ confirm() -> OldVsn = proplists:get_value(upgrade_version, TestMetaData, previous), Nodes = [CurrentNode, OldNode1, _] = - rt:build_cluster([{current, + rt_cluster:build_cluster([{current, [{riak_kv, [{anti_entropy, {off, []}}]}]}, OldVsn, OldVsn]), ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), diff --git a/tests/verify_2i_returnterms.erl b/tests/verify_2i_returnterms.erl index 7a9f50ee4..2708a2de8 100644 --- a/tests/verify_2i_returnterms.erl +++ b/tests/verify_2i_returnterms.erl @@ -30,7 +30,7 @@ confirm() -> inets:start(), - Nodes = rt:build_cluster(3), + Nodes = rt_cluster:build_cluster(3), ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), RiakHttp = rt:http_url(hd(Nodes)), diff --git a/tests/verify_2i_stream.erl b/tests/verify_2i_stream.erl index 32aff6939..638d68691 100644 --- a/tests/verify_2i_stream.erl +++ b/tests/verify_2i_stream.erl @@ -29,7 +29,7 @@ confirm() -> inets:start(), - Nodes = rt:build_cluster(3), + Nodes = rt_cluster:build_cluster(3), ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), RiakHttp = rt:http_url(hd(Nodes)), diff --git a/tests/verify_2i_timeout.erl b/tests/verify_2i_timeout.erl index d913b5631..e17c09caf 100644 --- a/tests/verify_2i_timeout.erl +++ b/tests/verify_2i_timeout.erl @@ -29,7 +29,7 @@ confirm() -> inets:start(), Config = [{riak_kv, [{secondary_index_timeout, 1}]}], %% ludicrously short, should fail always - Nodes = rt:build_cluster([{current, Config}, {current, Config}, {current, Config}]), + Nodes = rt_cluster:build_cluster([{current, Config}, {current, Config}, {current, Config}]), ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), PBPid = rt:pbc(hd(Nodes)), diff --git a/tests/verify_aae.erl b/tests/verify_aae.erl index 1d5fa2dda..97f665d63 100644 --- a/tests/verify_aae.erl +++ b/tests/verify_aae.erl @@ -64,7 +64,7 @@ -define(N_VAL, 3). confirm() -> - Nodes = rt:build_cluster(?NUM_NODES, ?CFG), + Nodes = rt_cluster:build_cluster(?NUM_NODES, ?CFG), verify_aae(Nodes), pass. @@ -242,12 +242,12 @@ test_less_than_n_mods(Node, KeyValues) -> wipe_out_partition(Node, Partition) -> lager:info("Wiping out partition ~p in node ~p", [Partition, Node]), - rt:clean_data_dir(Node, dir_for_partition(Partition)), + rt_cluster:clean_data_dir(Node, dir_for_partition(Partition)), ok. wipe_out_aae_data(Node, Partition) -> lager:info("Wiping out AAE data for partition ~p in node ~p", [Partition, Node]), - rt:clean_data_dir(Node, "anti_entropy/"++integer_to_list(Partition)), + rt_cluster:clean_data_dir(Node, "anti_entropy/"++integer_to_list(Partition)), ok. base_dir_for_backend(undefined) -> diff --git a/tests/verify_api_timeouts.erl b/tests/verify_api_timeouts.erl index 0a6851cfd..5cd22e08f 100644 --- a/tests/verify_api_timeouts.erl +++ b/tests/verify_api_timeouts.erl @@ -9,7 +9,7 @@ confirm() -> %% test requires allow_mult=false b/c of rt:systest_read - [Node] = rt:build_cluster(1), + [Node] = rt_cluster:build_cluster(1), rt:wait_until_pingable(Node), HC = rt:httpc(Node), diff --git a/tests/verify_asis_put.erl b/tests/verify_asis_put.erl index d34439c0f..f89bd8517 100644 --- a/tests/verify_asis_put.erl +++ b/tests/verify_asis_put.erl @@ -4,7 +4,7 @@ confirm() -> %% 1. Deploy two nodes - [Node1, Node2] = rt:deploy_nodes(2), + [Node1, Node2] = rt_cluster:deploy_nodes(2), %% 2. With PBC lager:info("Put new object in ~p via PBC.", [Node1]), PB1 = rt:pbc(Node1), diff --git a/tests/verify_backup_restore.erl b/tests/verify_backup_restore.erl index b587160ad..aec9a8bb3 100644 --- a/tests/verify_backup_restore.erl +++ b/tests/verify_backup_restore.erl @@ -39,7 +39,7 @@ confirm() -> lager:info("Building cluster of ~p nodes", [?NUM_NODES]), SpamDir = rt_config:config_or_os_env(spam_dir), Config = [{riak_search, [{enabled, true}]}], - [Node0 | _RestNodes] = Nodes = rt:build_cluster(?NUM_NODES, Config), + [Node0 | _RestNodes] = Nodes = rt_cluster:build_cluster(?NUM_NODES, Config), rt:enable_search_hook(Node0, ?SEARCH_BUCKET), rt:wait_until_ring_converged(Nodes), PbcPid = rt:pbc(Node0), @@ -135,9 +135,9 @@ confirm() -> lager:info("Wipe out entire cluster and start fresh"), riakc_pb_socket:stop(PbcPid), - rt:clean_cluster(Nodes), + rt_cluster:clean_cluster(Nodes), lager:info("Rebuilding the cluster"), - rt:build_cluster(?NUM_NODES, Config), + rt_cluster:build_cluster(?NUM_NODES, Config), rt:enable_search_hook(Node0, ?SEARCH_BUCKET), rt:wait_until_ring_converged(Nodes), rt:wait_until_no_pending_changes(Nodes), diff --git a/tests/verify_basic_upgrade.erl b/tests/verify_basic_upgrade.erl index f02a7cc11..b1c17556a 100644 --- a/tests/verify_basic_upgrade.erl +++ b/tests/verify_basic_upgrade.erl @@ -26,7 +26,7 @@ confirm() -> TestMetaData = riak_test_runner:metadata(), OldVsn = proplists:get_value(upgrade_version, TestMetaData, previous), - Nodes = [Node1|_] = rt:build_cluster([OldVsn, OldVsn, OldVsn, OldVsn]), + Nodes = [Node1|_] = rt_cluster:build_cluster([OldVsn, OldVsn, OldVsn, OldVsn]), lager:info("Writing 100 keys to ~p", [Node1]), rt:systest_write(Node1, 100, 3), diff --git a/tests/verify_bitcask_tombstone2_upgrade.erl b/tests/verify_bitcask_tombstone2_upgrade.erl index 72543e2c8..b6ebcf695 100644 --- a/tests/verify_bitcask_tombstone2_upgrade.erl +++ b/tests/verify_bitcask_tombstone2_upgrade.erl @@ -19,7 +19,7 @@ confirm() -> % Configure for fast merge checks Config = [{riak_kv, [{bitcask_merge_check_interval, 2000}]}, {bitcask, [{max_file_size, 100}]}], - Nodes = rt:build_cluster([{OldVsn, Config}]), + Nodes = rt_cluster:build_cluster([{OldVsn, Config}]), verify_bitcask_tombstone2_upgrade(Nodes), pass. diff --git a/tests/verify_busy_dist_port.erl b/tests/verify_busy_dist_port.erl index c8828d902..ac29da4a6 100644 --- a/tests/verify_busy_dist_port.erl +++ b/tests/verify_busy_dist_port.erl @@ -47,7 +47,7 @@ -include_lib("eunit/include/eunit.hrl"). confirm() -> - [Node1, Node2] = rt:build_cluster(2), + [Node1, Node2] = rt_cluster:build_cluster(2), lager:info("deployed 2 nodes"), rt:load_modules_on_nodes([cause_bdp, verify_bdp_event_handler, diff --git a/tests/verify_capabilities.erl b/tests/verify_capabilities.erl index 2730eca85..d689a0229 100644 --- a/tests/verify_capabilities.erl +++ b/tests/verify_capabilities.erl @@ -31,7 +31,7 @@ confirm() -> _ -> current end, - Nodes = rt:deploy_nodes([current, previous, Legacy]), + Nodes = rt_cluster:deploy_nodes([current, previous, Legacy]), [CNode, PNode, LNode] = Nodes, lager:info("Verifying known capabilities on a Current 1-node cluster"), diff --git a/tests/verify_commit_hooks.erl b/tests/verify_commit_hooks.erl index fdc8ca40a..b2d57636e 100644 --- a/tests/verify_commit_hooks.erl +++ b/tests/verify_commit_hooks.erl @@ -23,7 +23,7 @@ -export([confirm/0]). confirm() -> - [Node] = rt:deploy_nodes(1), + [Node] = rt_cluster:deploy_nodes(1), lager:info("Loading the hooks module into ~p", [Node]), rt:load_modules_on_nodes([hooks], [Node]), diff --git a/tests/verify_conditional_postcommit.erl b/tests/verify_conditional_postcommit.erl index fcb2c5f0c..cbf79b705 100644 --- a/tests/verify_conditional_postcommit.erl +++ b/tests/verify_conditional_postcommit.erl @@ -24,7 +24,7 @@ confirm() -> Config = [{riak_core, [{vnode_management_timer, 1000}, {ring_creation_size, 4}]}], - Nodes = rt:deploy_nodes(1, Config), + Nodes = rt_cluster:deploy_nodes(1, Config), Node = hd(Nodes), ok = rt:load_modules_on_nodes([?MODULE], Nodes), diff --git a/tests/verify_counter_converge.erl b/tests/verify_counter_converge.erl index 2217d582b..adc067b40 100644 --- a/tests/verify_counter_converge.erl +++ b/tests/verify_counter_converge.erl @@ -33,7 +33,7 @@ confirm() -> Key = <<"a">>, - [N1, N2, N3, N4]=Nodes = rt:build_cluster(4), + [N1, N2, N3, N4]=Nodes = rt_cluster:build_cluster(4), [C1, C2, C3, C4]=Clients = [ rt:httpc(N) || N <- Nodes ], set_allow_mult_true(Nodes), diff --git a/tests/verify_counter_repl.erl b/tests/verify_counter_repl.erl index 79a107b37..b8d9cb202 100644 --- a/tests/verify_counter_repl.erl +++ b/tests/verify_counter_repl.erl @@ -63,7 +63,7 @@ make_clusters() -> Conf = [{riak_repl, [{fullsync_on_connect, false}, {fullsync_interval, disabled}]}, {riak_core, [{default_bucket_props, [{allow_mult, true}]}]}], - Nodes = rt:deploy_nodes(6, Conf, [riak_kv, riak_repl]), + Nodes = rt_cluster:deploy_nodes(6, Conf), {ClusterA, ClusterB} = lists:split(3, Nodes), A = make_cluster(ClusterA, "A"), B = make_cluster(ClusterB, "B"), diff --git a/tests/verify_cs_bucket.erl b/tests/verify_cs_bucket.erl index 5db40bbca..3ba21c902 100644 --- a/tests/verify_cs_bucket.erl +++ b/tests/verify_cs_bucket.erl @@ -29,7 +29,7 @@ -define(FOO, <<"foo">>). confirm() -> - Nodes = rt:build_cluster(3), + Nodes = rt_cluster:build_cluster(3), ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), PBPid = rt:pbc(hd(Nodes)), diff --git a/tests/verify_down.erl b/tests/verify_down.erl index 973fb9f10..efbf79d0d 100644 --- a/tests/verify_down.erl +++ b/tests/verify_down.erl @@ -23,7 +23,7 @@ -include_lib("eunit/include/eunit.hrl"). confirm() -> - Nodes = rt:deploy_nodes(3), + Nodes = rt_cluster:deploy_nodes(3), [Node1, Node2, Node3] = Nodes, %% Join node2 to node1 and wait for cluster convergence diff --git a/tests/verify_dt_context.erl b/tests/verify_dt_context.erl index acdd74e27..a9e44d12f 100644 --- a/tests/verify_dt_context.erl +++ b/tests/verify_dt_context.erl @@ -44,7 +44,7 @@ confirm() -> {riak_core, [ {ring_creation_size, 16}, {vnode_management_timer, 1000} ]}], - [N1, N2]=Nodes = rt:build_cluster(2, Config), + [N1, N2]=Nodes = rt_cluster:build_cluster(2, Config), create_bucket_types(Nodes, ?TYPES), diff --git a/tests/verify_dt_converge.erl b/tests/verify_dt_converge.erl index 60c78ab36..72e045cce 100644 --- a/tests/verify_dt_converge.erl +++ b/tests/verify_dt_converge.erl @@ -49,7 +49,7 @@ confirm() -> {riak_core, [ {ring_creation_size, 16}, {vnode_management_timer, 1000} ]}], - [N1, N2, N3, N4]=Nodes = rt:build_cluster(4, Config), + [N1, N2, N3, N4]=Nodes = rt_cluster:build_cluster(4, Config), create_bucket_types(Nodes, ?TYPES), diff --git a/tests/verify_dt_upgrade.erl b/tests/verify_dt_upgrade.erl index c43f3fb09..47cd6c1ee 100644 --- a/tests/verify_dt_upgrade.erl +++ b/tests/verify_dt_upgrade.erl @@ -31,7 +31,7 @@ confirm() -> TestMetaData = riak_test_runner:metadata(), OldVsn = proplists:get_value(upgrade_version, TestMetaData, previous), - Nodes = [Node1|_] = rt:build_cluster([OldVsn, OldVsn, OldVsn, OldVsn]), + Nodes = [Node1|_] = rt_cluster:build_cluster([OldVsn, OldVsn, OldVsn, OldVsn]), verify_counter_converge:set_allow_mult_true(Nodes, ?COUNTER_BUCKET), populate_counters(Node1), diff --git a/tests/verify_dvv_repl.erl b/tests/verify_dvv_repl.erl index c7354d3a5..4d0ce03f7 100644 --- a/tests/verify_dvv_repl.erl +++ b/tests/verify_dvv_repl.erl @@ -97,10 +97,10 @@ make_replicate_test_fun(From, To) -> make_clusters() -> Conf = [{riak_repl, [{fullsync_on_connect, false}, {fullsync_interval, disabled}]}, - {riak_core, [{default_bucket_props, - [{dvv_enabled, true}, - {allow_mult, true}]}]}], - Nodes = rt:deploy_nodes(6, Conf, [riak_kv, riak_repl]), + {riak_core, [{default_bucket_props, + [{dvv_enabled, true}, + {allow_mult, true}]}]}], + Nodes = rt_cluster:deploy_nodes(6, Conf), {ClusterA, ClusterB} = lists:split(3, Nodes), A = make_cluster(ClusterA, "A"), B = make_cluster(ClusterB, "B"), diff --git a/tests/verify_dynamic_ring.erl b/tests/verify_dynamic_ring.erl index d1566a1d3..0e9b9d072 100644 --- a/tests/verify_dynamic_ring.erl +++ b/tests/verify_dynamic_ring.erl @@ -34,7 +34,7 @@ confirm() -> rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), rt_config:update_app_config(all, [{riak_core, [{ring_creation_size, ?START_SIZE}]}]), - [ANode, AnotherNode, YetAnother, _ReplacingNode] = _AllNodes = rt:deploy_nodes(4), + [ANode, AnotherNode, YetAnother, _ReplacingNode] = _AllNodes = rt_cluster:deploy_nodes(4), NewNodes = Nodes = [ANode, AnotherNode, YetAnother], %% This assignment for `NewNodes' is commented until riak_core %% issue #570 is resolved diff --git a/tests/verify_handoff.erl b/tests/verify_handoff.erl index 211a14167..d03ee7ce9 100644 --- a/tests/verify_handoff.erl +++ b/tests/verify_handoff.erl @@ -129,11 +129,8 @@ assert_using(Node, {CapabilityCategory, CapabilityName}, ExpectedCapabilityName) ExpectedCapabilityName =:= rt:capability(Node, {CapabilityCategory, CapabilityName}). %% For some testing purposes, making these limits smaller is helpful: -deploy_test_nodes(false, N) -> - Config = [{riak_core, [{ring_creation_size, 8}, - {handoff_acksync_threshold, 20}, - {handoff_receive_timeout, 2000}]}], - rt:deploy_nodes(N, Config); +deploy_test_nodes(false, N) -> + rt_cluster:deploy_nodes(N); deploy_test_nodes(true, N) -> lager:info("WARNING: Using turbo settings for testing."), Config = [{riak_core, [{forced_ownership_handoff, 8}, @@ -143,4 +140,4 @@ deploy_test_nodes(true, N) -> {handoff_acksync_threshold, 20}, {handoff_receive_timeout, 2000}, {gossip_limit, {10000000, 60000}}]}], - rt:deploy_nodes(N, Config). + rt_cluster:deploy_nodes(N, Config). diff --git a/tests/verify_handoff_mixed.erl b/tests/verify_handoff_mixed.erl index 9439109c4..2bd5b290b 100644 --- a/tests/verify_handoff_mixed.erl +++ b/tests/verify_handoff_mixed.erl @@ -58,7 +58,7 @@ confirm() -> Versions = [{current, SearchEnabled}, {UpgradeVsn, SearchEnabled}], Services = [riak_kv, riak_search, riak_pipe], - [Current, Old] = Nodes = rt:deploy_nodes(Versions, Services), + [Current, Old] = Nodes = rt_cluster:deploy_nodes(Versions, Services), prepare_vnodes(Current), diff --git a/tests/verify_kv_health_check.erl b/tests/verify_kv_health_check.erl index ebca92d1e..abde0d7aa 100644 --- a/tests/verify_kv_health_check.erl +++ b/tests/verify_kv_health_check.erl @@ -22,7 +22,7 @@ -export([confirm/0]). confirm() -> - [Node1, Node2, _Node3] = rt:build_cluster(3), + [Node1, Node2, _Node3] = rt_cluster:build_cluster(3), %% add intercept that delays handling of vnode commands %% on a single node (the "slow" node) diff --git a/tests/verify_link_walk_urls.erl b/tests/verify_link_walk_urls.erl index ff5fcb044..ba174b875 100644 --- a/tests/verify_link_walk_urls.erl +++ b/tests/verify_link_walk_urls.erl @@ -33,7 +33,7 @@ confirm() -> - [Node0 | _] = rt:build_cluster(?NUM_NODES), + [Node0 | _] = rt_cluster:build_cluster(?NUM_NODES), Pbc = rt:pbc(Node0), lager:info("Inserting linked graph"), diff --git a/tests/verify_listkeys.erl b/tests/verify_listkeys.erl index 33af44937..087d1b8dd 100644 --- a/tests/verify_listkeys.erl +++ b/tests/verify_listkeys.erl @@ -29,7 +29,7 @@ -define(UNDEFINED_BUCKET_TYPE, <<"880bf69d-5dab-44ee-8762-d24c6f759ce1">>). confirm() -> - [Node1, Node2, Node3, Node4] = Nodes = rt:deploy_nodes(4), + [Node1, Node2, Node3, Node4] = Nodes = rt_cluster:deploy_nodes(4), ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), lager:info("Nodes deployed, but not joined."), diff --git a/tests/verify_listkeys_eqcfsm.erl b/tests/verify_listkeys_eqcfsm.erl index d8f5f01ee..bb751fd6b 100644 --- a/tests/verify_listkeys_eqcfsm.erl +++ b/tests/verify_listkeys_eqcfsm.erl @@ -69,7 +69,7 @@ prop_test() -> [lager:info(" Command : ~p~n", [Cmd]) || Cmd <- Cmds], {H, _S, Res} = run_commands(?MODULE, Cmds, [{nodelist, Nodes}]), lager:info("======================== Ran commands"), - rt:clean_cluster(Nodes), + rt_cluster:clean_cluster(Nodes), aggregate(zip(state_names(H),command_names(Cmds)), equals(Res, ok)) end))). @@ -178,7 +178,7 @@ log_transition(S) -> %% Helpers %% ==================================================================== setup_cluster(NumNodes) -> - Nodes = rt:build_cluster(NumNodes), + Nodes = rt_cluster:build_cluster(NumNodes), ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), ?assertEqual(ok, rt:wait_until_transfers_complete(Nodes)), Node = hd(Nodes), diff --git a/tests/verify_membackend.erl b/tests/verify_membackend.erl index 536ef1f56..e61f57990 100644 --- a/tests/verify_membackend.erl +++ b/tests/verify_membackend.erl @@ -35,16 +35,16 @@ confirm() -> ttl(Mode) -> Conf = mkconf(ttl, Mode), - [NodeA, NodeB] = rt:deploy_nodes(2, Conf), + [NodeA, NodeB] = rt_cluster:deploy_nodes(2, Conf), ?assertEqual(ok, check_leave_and_expiry(NodeA, NodeB)), - rt:clean_cluster([NodeA]), + rt_cluster:clean_cluster([NodeA]), ok. max_memory(Mode) -> Conf = mkconf(max_memory, Mode), - [NodeA, NodeB] = rt:deploy_nodes(2, Conf), + [NodeA, NodeB] = rt_cluster:deploy_nodes(2, Conf), rt:join(NodeB, NodeA), @@ -54,14 +54,14 @@ max_memory(Mode) -> ?assertEqual(ok, check_eviction(NodeA)), - rt:clean_cluster([NodeA, NodeB]), + rt_cluster:clean_cluster([NodeA, NodeB]), ok. combo(Mode) -> Conf = mkconf(combo, Mode), - [NodeA, NodeB] = rt:deploy_nodes(2, Conf), + [NodeA, NodeB] = rt_cluster:deploy_nodes(2, Conf), ?assertEqual(ok, check_leave_and_expiry(NodeA, NodeB)), @@ -75,7 +75,7 @@ combo(Mode) -> ?assertEqual(ok, check_eviction(NodeA)), - rt:clean_cluster([NodeA]), + rt_cluster:clean_cluster([NodeA]), ok. diff --git a/tests/verify_mr_prereduce_node_down.erl b/tests/verify_mr_prereduce_node_down.erl index 265dd7a29..e6bc2d914 100644 --- a/tests/verify_mr_prereduce_node_down.erl +++ b/tests/verify_mr_prereduce_node_down.erl @@ -44,7 +44,7 @@ confirm() -> NodeCount = 4, lager:info("Build ~b-node cluster", [NodeCount]), - [Primary,ToKill|_] = rt:build_cluster(NodeCount), + [Primary,ToKill|_] = rt_cluster:build_cluster(NodeCount), %% We need one node down for this test rt:stop(ToKill), diff --git a/tests/verify_no_writes_on_read.erl b/tests/verify_no_writes_on_read.erl index ca4e95d31..5b48ed048 100644 --- a/tests/verify_no_writes_on_read.erl +++ b/tests/verify_no_writes_on_read.erl @@ -11,7 +11,7 @@ confirm() -> Backend = proplists:get_value(backend, riak_test_runner:metadata()), lager:info("Running with backend ~p", [Backend]), ?assertEqual(bitcask, Backend), - [Node1 | _Rest] = _Nodes = rt:build_cluster(?NUM_NODES), + [Node1 | _Rest] = _Nodes = rt_cluster:build_cluster(?NUM_NODES), PBC = rt:pbc(Node1), lager:info("Setting last write wins on bucket"), B = ?BUCKET, diff --git a/tests/verify_object_limits.erl b/tests/verify_object_limits.erl index fd8af35dc..a989eed1d 100644 --- a/tests/verify_object_limits.erl +++ b/tests/verify_object_limits.erl @@ -34,7 +34,7 @@ confirm() -> - [Node1] = rt:build_cluster(1, [{riak_kv, [ + [Node1] = rt_cluster:build_cluster(1, [{riak_kv, [ {ring_creation_size, 8}, {max_object_size, ?MAX_SIZE}, {warn_object_size, ?WARN_SIZE}, diff --git a/tests/verify_reset_bucket_props.erl b/tests/verify_reset_bucket_props.erl index 1c099a81c..c3740bc94 100644 --- a/tests/verify_reset_bucket_props.erl +++ b/tests/verify_reset_bucket_props.erl @@ -29,7 +29,7 @@ confirm() -> %% we will be using two of the nodes to perform an %% update and then a reset (one on each node) of a bucket's properties. %% All nodes are checked to make sure the reset is affected on them - [Node1, Node2, Node3] = Nodes = rt:build_cluster(3), + [Node1, Node2, Node3] = Nodes = rt_cluster:build_cluster(3), DefaultProps = get_current_bucket_props(Nodes, ?BUCKET), diff --git a/tests/verify_riak_lager.erl b/tests/verify_riak_lager.erl index 98505538b..b902048d6 100644 --- a/tests/verify_riak_lager.erl +++ b/tests/verify_riak_lager.erl @@ -29,7 +29,7 @@ confirm() -> lager:info("Staring a node"), - Nodes = [Node] = rt:deploy_nodes(1), + Nodes = [Node] = rt_cluster:deploy_nodes(1), ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), lager:info("Stopping that node"), diff --git a/tests/verify_riak_object_reformat.erl b/tests/verify_riak_object_reformat.erl index cd693f4ff..bdb9a9c6d 100644 --- a/tests/verify_riak_object_reformat.erl +++ b/tests/verify_riak_object_reformat.erl @@ -34,7 +34,7 @@ confirm() -> rt_config:update_app_config(all, [{riak_kv, [{object_format, v1}]}]), TestMetaData = riak_test_runner:metadata(), DowngradeVsn = proplists:get_value(upgrade_version, TestMetaData, previous), - Nodes = [Node1|_] = rt:build_cluster(?N), + Nodes = [Node1|_] = rt_cluster:build_cluster(?N), [rt:wait_until_capability(N, {riak_kv, object_format}, v1, v0) || N <- Nodes], diff --git a/tests/verify_riak_stats.erl b/tests/verify_riak_stats.erl index 53ae72d1e..971c7fbf7 100644 --- a/tests/verify_riak_stats.erl +++ b/tests/verify_riak_stats.erl @@ -35,7 +35,7 @@ %% You should have curl installed locally to do this. confirm() -> - Nodes = rt:deploy_nodes(1, ?CONF), + Nodes = rt_cluster:deploy_nodes(1), [Node1] = Nodes, verify_dt_converge:create_bucket_types(Nodes, ?TYPES), ?assertEqual(ok, rt:wait_until_nodes_ready([Node1])), diff --git a/tests/verify_search.erl b/tests/verify_search.erl index 2d98ebb32..0444180ab 100644 --- a/tests/verify_search.erl +++ b/tests/verify_search.erl @@ -31,7 +31,7 @@ confirm() -> Config = [{riak_search, [{enabled, true}]}], - [Node0 | _RestNodes] = Nodes = rt:build_cluster(3, Config), + [Node0 | _RestNodes] = Nodes = rt_cluster:build_cluster(3, Config), rt:wait_until_ring_converged(Nodes), Path = rt_config:get(rt_scratch_dir), diff --git a/tests/verify_secondary_index_reformat.erl b/tests/verify_secondary_index_reformat.erl index 1defc4ec0..a1455deb1 100644 --- a/tests/verify_secondary_index_reformat.erl +++ b/tests/verify_secondary_index_reformat.erl @@ -24,7 +24,7 @@ -include_lib("riakc/include/riakc.hrl"). confirm() -> - [Node] = rt:build_cluster([legacy]), + [Node] = rt_cluster:build_cluster([legacy]), rt:wait_until_nodes_ready([Node]), check_fixed_index_statuses(Node, undefined), diff --git a/tests/verify_snmp.erl b/tests/verify_snmp.erl index 639972fcf..4e3183a0c 100644 --- a/tests/verify_snmp.erl +++ b/tests/verify_snmp.erl @@ -28,7 +28,7 @@ confirm() -> %% Bring up a small cluster Config = [{riak_snmp, [{polling_interval, 1000}]}], - [Node1] = rt:deploy_nodes(1, Config), + [Node1] = rt_cluster:deploy_nodes(1), ?assertEqual(ok, rt:wait_until_nodes_ready([Node1])), Keys = [{vnodeGets,<<"vnode_gets">>}, diff --git a/tests/verify_staged_clustering.erl b/tests/verify_staged_clustering.erl index f792c4800..5ece38adb 100644 --- a/tests/verify_staged_clustering.erl +++ b/tests/verify_staged_clustering.erl @@ -24,7 +24,7 @@ -include_lib("eunit/include/eunit.hrl"). confirm() -> - Nodes = rt:deploy_nodes(4), + Nodes = rt_cluster:deploy_nodes(4), [Node1, Node2, Node3, Node4] = Nodes, Nodes123 = [Node1, Node2, Node3], Nodes23 = [Node2, Node3], diff --git a/tests/verify_tick_change.erl b/tests/verify_tick_change.erl index 208adcdd4..cf8c12838 100644 --- a/tests/verify_tick_change.erl +++ b/tests/verify_tick_change.erl @@ -27,7 +27,7 @@ confirm() -> ClusterSize = 4, rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), NewConfig = [], - Nodes = rt:build_cluster(ClusterSize, NewConfig), + Nodes = rt_cluster:build_cluster(ClusterSize, NewConfig), ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), [Node1|_] = Nodes, Bucket = <<"systest">>, diff --git a/tests/verify_vclock.erl b/tests/verify_vclock.erl index dcd24c690..5a5cbaa50 100644 --- a/tests/verify_vclock.erl +++ b/tests/verify_vclock.erl @@ -193,12 +193,12 @@ our_pbc_read(Node, Start, End, Bucket, VSuffix) -> %% For some testing purposes, making these limits smaller is helpful: deploy_test_nodes(false, N) -> - rt:deploy_nodes(N); + rt_cluster:deploy_nodes(N); deploy_test_nodes(true, N) -> lager:info("NOTICE: Using turbo settings for testing."), Config = [{riak_core, [{forced_ownership_handoff, 8}, {handoff_concurrency, 8}, {vnode_inactivity_timeout, 1000}, {gossip_limit, {10000000, 60000}}]}], - rt:deploy_nodes(N, Config). + rt_cluster:deploy_nodes(N, Config). diff --git a/tests/verify_vclock_encoding_upgrade.erl b/tests/verify_vclock_encoding_upgrade.erl index e8614e83b..110951099 100644 --- a/tests/verify_vclock_encoding_upgrade.erl +++ b/tests/verify_vclock_encoding_upgrade.erl @@ -23,7 +23,7 @@ confirm() -> lager:info("Deploying previous cluster"), - [Prev, Current] = rt:build_cluster([previous, current]), + [Prev, Current] = rt_cluster:build_cluster([previous, current]), PrevClient = rt:pbc(Prev), CurrentClient = rt:pbc(Current), K = <<"key">>, diff --git a/tests/yz_ensemble.erl b/tests/yz_ensemble.erl index 5b0361e58..f88136186 100644 --- a/tests/yz_ensemble.erl +++ b/tests/yz_ensemble.erl @@ -88,10 +88,10 @@ verify_ensemble_delete_support(Node, Bucket, Index) -> %% node when adding yokozuna and ensemble support. Waiting for yokozuna %% to load on each node allows join_cluster to complete consistently build_cluster_with_yz_support(Num, Config, NVal) -> - Nodes = rt:deploy_nodes(Num, Config), + Nodes = rt_cluster:deploy_nodes(Num, Config), [rt:wait_for_cluster_service([N], yokozuna) || N <- Nodes], Node = hd(Nodes), - rt:join_cluster(Nodes), + rt_cluster:join_cluster(Nodes), ensemble_util:wait_until_cluster(Nodes), ensemble_util:wait_for_membership(Node), ensemble_util:wait_until_stable(Node, NVal), From 1267503384acbaf6dd16d0a683e64d91eac7f324 Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Wed, 30 Jul 2014 13:23:11 -0400 Subject: [PATCH 013/157] Move backend related functions from rt module to new rt_backend module. --- src/riak_test_runner.erl | 3 +- src/rt.erl | 69 ---------------------- src/rt_backend.erl | 95 ++++++++++++++++++++++++++++++ src/rtdev.erl | 2 +- src/rtssh.erl | 2 +- tests/mapred_notfound_failover.erl | 2 +- tests/verify_membackend.erl | 4 +- 7 files changed, 102 insertions(+), 75 deletions(-) create mode 100644 src/rt_backend.erl diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index 62da93043..a1a263c9c 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -51,7 +51,8 @@ run(TestModule, Outdir, TestMetaData, HarnessArgs) -> undefined -> []; Value -> [{multi_config, Value}] end, - Backend = rt:set_backend(proplists:get_value(backend, TestMetaData), BackendExtras), + Backend = rt_backend:set_backend( + proplists:get_value(backend, TestMetaData), BackendExtras), {PropsMod, PropsFun} = function_name(properties, TestModule, 0, rt_cluster), {SetupMod, SetupFun} = function_name(setup, TestModule, 2, rt_cluster), {ConfirmMod, ConfirmFun} = function_name(confirm, TestModule), diff --git a/src/rt.erl b/src/rt.erl index 4ae1b9bcb..6113bc315 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -1504,75 +1504,6 @@ enable_search_hook(Node, Bucket) when is_binary(Bucket) -> lager:info("Installing search hook for bucket ~p", [Bucket]), ?assertEqual(ok, rpc:call(Node, riak_search_kv_hook, install, [Bucket])). -%%%=================================================================== -%%% Test harness setup, configuration, and internal utilities -%%%=================================================================== - -%% @doc Sets the backend of ALL nodes that could be available to riak_test. -%% this is not limited to the nodes under test, but any node that -%% riak_test is able to find. It then queries each available node -%% for it's backend, and returns it if they're all equal. If different -%% nodes have different backends, it returns a list of backends. -%% Currently, there is no way to request multiple backends, so the -%% list return type should be considered an error. --spec set_backend(atom()) -> atom()|[atom()]. -set_backend(Backend) -> -set_backend(Backend, []). - --spec set_backend(atom(), [{atom(), term()}]) -> atom()|[atom()]. -set_backend(bitcask, _) -> -set_backend(riak_kv_bitcask_backend); -set_backend(eleveldb, _) -> -set_backend(riak_kv_eleveldb_backend); -set_backend(memory, _) -> -set_backend(riak_kv_memory_backend); -set_backend(multi, Extras) -> -set_backend(riak_kv_multi_backend, Extras); -set_backend(Backend, _) when Backend == riak_kv_bitcask_backend; Backend == riak_kv_eleveldb_backend; Backend == riak_kv_memory_backend -> - lager:info("rt:set_backend(~p)", [Backend]), - rt_config:update_app_config(all, [{riak_kv, [{storage_backend, Backend}]}]), - get_backends(); -set_backend(Backend, Extras) when Backend == riak_kv_multi_backend -> - MultiConfig = proplists:get_value(multi_config, Extras, default), - Config = make_multi_backend_config(MultiConfig), - rt_config:update_app_config(all, [{riak_kv, Config}]), - get_backends(); -set_backend(Other, _) -> -lager:warning("rt:set_backend doesn't recognize ~p as a legit backend, using the default.", [Other]), -get_backends(). - -make_multi_backend_config(default) -> -[{storage_backend, riak_kv_multi_backend}, -{multi_backend_default, <<"eleveldb1">>}, -{multi_backend, [{<<"eleveldb1">>, riak_kv_eleveldb_backend, []}, - {<<"memory1">>, riak_kv_memory_backend, []}, - {<<"bitcask1">>, riak_kv_bitcask_backend, []}]}]; -make_multi_backend_config(indexmix) -> -[{storage_backend, riak_kv_multi_backend}, -{multi_backend_default, <<"eleveldb1">>}, -{multi_backend, [{<<"eleveldb1">>, riak_kv_eleveldb_backend, []}, - {<<"memory1">>, riak_kv_memory_backend, []}]}]; -make_multi_backend_config(Other) -> -lager:warning("rt:set_multi_backend doesn't recognize ~p as legit multi-backend config, using default", [Other]), -make_multi_backend_config(default). - -get_backends() -> -Backends = ?HARNESS:get_backends(), -case Backends of -[riak_kv_bitcask_backend] -> bitcask; -[riak_kv_eleveldb_backend] -> eleveldb; -[riak_kv_memory_backend] -> memory; -[Other] -> Other; -MoreThanOne -> MoreThanOne -end. - --spec get_backend([proplists:property()]) -> atom() | error. -get_backend(AppConfigProplist) -> -case kvc:path('riak_kv.storage_backend', AppConfigProplist) of -[] -> error; -Backend -> Backend -end. - %% @doc Gets the current version under test. In the case of an upgrade test %% or something like that, it's the version you're upgrading to. -spec get_version() -> binary(). diff --git a/src/rt_backend.erl b/src/rt_backend.erl new file mode 100644 index 000000000..59f42739d --- /dev/null +++ b/src/rt_backend.erl @@ -0,0 +1,95 @@ +%% ------------------------------------------------------------------- +%% +%% Copyright (c) 2013-2014 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(rt_backend). +-include("rt.hrl"). +-include_lib("eunit/include/eunit.hrl"). + +-compile(export_all). + +-define(HARNESS, (rt_config:get(rt_harness))). + +%%%=================================================================== +%%% Test harness setup, configuration, and internal utilities +%%%=================================================================== + +%% @doc Sets the backend of ALL nodes that could be available to riak_test. +%% this is not limited to the nodes under test, but any node that +%% riak_test is able to find. It then queries each available node +%% for it's backend, and returns it if they're all equal. If different +%% nodes have different backends, it returns a list of backends. +%% Currently, there is no way to request multiple backends, so the +%% list return type should be considered an error. +-spec set_backend(atom()) -> atom()|[atom()]. +set_backend(Backend) -> + set_backend(Backend, []). + +-spec set_backend(atom(), [{atom(), term()}]) -> atom()|[atom()]. +set_backend(bitcask, _) -> + set_backend(riak_kv_bitcask_backend); +set_backend(eleveldb, _) -> + set_backend(riak_kv_eleveldb_backend); +set_backend(memory, _) -> + set_backend(riak_kv_memory_backend); +set_backend(multi, Extras) -> + set_backend(riak_kv_multi_backend, Extras); +set_backend(Backend, _) when Backend == riak_kv_bitcask_backend; Backend == riak_kv_eleveldb_backend; Backend == riak_kv_memory_backend -> + lager:info("rt_backend:set_backend(~p)", [Backend]), + rt_config:update_app_config(all, [{riak_kv, [{storage_backend, Backend}]}]), + get_backends(); +set_backend(Backend, Extras) when Backend == riak_kv_multi_backend -> + MultiConfig = proplists:get_value(multi_config, Extras, default), + Config = make_multi_backend_config(MultiConfig), + rt_config:update_app_config(all, [{riak_kv, Config}]), + get_backends(); +set_backend(Other, _) -> + lager:warning("rt_backend:set_backend doesn't recognize ~p as a legit backend, using the default.", [Other]), + get_backends(). + +make_multi_backend_config(default) -> + [{storage_backend, riak_kv_multi_backend}, + {multi_backend_default, <<"eleveldb1">>}, + {multi_backend, [{<<"eleveldb1">>, riak_kv_eleveldb_backend, []}, + {<<"memory1">>, riak_kv_memory_backend, []}, + {<<"bitcask1">>, riak_kv_bitcask_backend, []}]}]; +make_multi_backend_config(indexmix) -> + [{storage_backend, riak_kv_multi_backend}, + {multi_backend_default, <<"eleveldb1">>}, + {multi_backend, [{<<"eleveldb1">>, riak_kv_eleveldb_backend, []}, + {<<"memory1">>, riak_kv_memory_backend, []}]}]; +make_multi_backend_config(Other) -> + lager:warning("rt:set_multi_backend doesn't recognize ~p as legit multi-backend config, using default", [Other]), + make_multi_backend_config(default). + +get_backends() -> + Backends = ?HARNESS:get_backends(), + case Backends of + [riak_kv_bitcask_backend] -> bitcask; + [riak_kv_eleveldb_backend] -> eleveldb; + [riak_kv_memory_backend] -> memory; + [Other] -> Other; + MoreThanOne -> MoreThanOne + end. + +-spec get_backend([proplists:property()]) -> atom() | error. +get_backend(AppConfigProplist) -> + case kvc:path('riak_kv.storage_backend', AppConfigProplist) of + [] -> error; + Backend -> Backend + end. diff --git a/src/rtdev.erl b/src/rtdev.erl index cb725c363..a9ed78a5f 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -330,7 +330,7 @@ get_backend(AppConfig) -> case file:consult(ConfigFile) of {ok, [Config]} -> - rt:get_backend(Config); + rt_backend:get_backend(Config); E -> lager:error("Error reading ~s, ~p", [ConfigFile, E]), error diff --git a/src/rtssh.erl b/src/rtssh.erl index 4c3e7d228..960bdeb98 100644 --- a/src/rtssh.erl +++ b/src/rtssh.erl @@ -88,7 +88,7 @@ get_backend(Host, AppConfig) -> Str = binary_to_list(Bin), {ok, ErlTok, _} = erl_scan:string(Str), {ok, Term} = erl_parse:parse_term(ErlTok), - rt:get_backend(Term). + rt_backend:get_backend(Term). cmd(Cmd) -> cmd(Cmd, []). diff --git a/tests/mapred_notfound_failover.erl b/tests/mapred_notfound_failover.erl index 9076a1c30..488807f01 100644 --- a/tests/mapred_notfound_failover.erl +++ b/tests/mapred_notfound_failover.erl @@ -39,7 +39,7 @@ confirm() -> %% we need the volatility of memory, so we can cause a replica %% notfound by killing a vnode - rt:set_backend(memory), + rt_backend:set_backend(memory), Nodes = rt_cluster:build_cluster(3), diff --git a/tests/verify_membackend.erl b/tests/verify_membackend.erl index e61f57990..2f05cec2c 100644 --- a/tests/verify_membackend.erl +++ b/tests/verify_membackend.erl @@ -214,7 +214,7 @@ mkconf(Test, Mode) -> case Mode of regular -> %% only memory supports TTL - rt:set_backend(memory), + rt_backend:set_backend(memory), [ {riak_core, [ @@ -227,7 +227,7 @@ mkconf(Test, Mode) -> ]} ]; multi -> - rt:set_backend(multi), + rt_backend:set_backend(multi), [ {riak_core, [ {ring_creation_size, 4} From 6267aa20e2d8772f9f3a820ef436b105ae058d13 Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Wed, 30 Jul 2014 14:00:43 -0400 Subject: [PATCH 014/157] Move protobuf-related functions from rt module to rt_pb. --- src/rt.erl | 139 ----------------- src/rt_pb.erl | 164 ++++++++++++++++++++ tests/bucket_props_roundtrip.erl | 2 +- tests/ensemble_basic3.erl | 10 +- tests/ensemble_basic4.erl | 6 +- tests/ensemble_interleave.erl | 8 +- tests/ensemble_remove_node2.erl | 8 +- tests/ensemble_sync.erl | 10 +- tests/ensemble_vnode_crash.erl | 10 +- tests/jmx_verify.erl | 6 +- tests/loaded_upgrade.erl | 6 +- tests/loaded_upgrade_worker_sup.erl | 4 +- tests/mapred_basic_compat.erl | 2 +- tests/mapred_javascript.erl | 2 +- tests/mapred_notfound_failover.erl | 2 +- tests/mapred_search_switch.erl | 2 +- tests/overload.erl | 6 +- tests/partition_repair.erl | 4 +- tests/replication/repl_aae_fullsync.erl | 4 +- tests/replication/repl_bucket_types.erl | 4 +- tests/replication/replication2.erl | 2 +- tests/replication/replication2_pg.erl | 84 +++++----- tests/replication/rt_cascading.erl | 46 +++--- tests/secondary_index_tests.erl | 4 +- tests/sibling_explosion.erl | 2 +- tests/verify_2i_aae.erl | 4 +- tests/verify_2i_limit.erl | 4 +- tests/verify_2i_mixed_cluster.erl | 6 +- tests/verify_2i_returnterms.erl | 2 +- tests/verify_2i_stream.erl | 2 +- tests/verify_2i_timeout.erl | 2 +- tests/verify_aae.erl | 4 +- tests/verify_api_timeouts.erl | 8 +- tests/verify_asis_put.erl | 4 +- tests/verify_backup_restore.erl | 12 +- tests/verify_bitcask_tombstone2_upgrade.erl | 2 +- tests/verify_conditional_postcommit.erl | 4 +- tests/verify_corruption_filtering.erl | 4 +- tests/verify_counter_capability.erl | 4 +- tests/verify_crdt_capability.erl | 2 +- tests/verify_cs_bucket.erl | 2 +- tests/verify_dt_context.erl | 2 +- tests/verify_dt_converge.erl | 2 +- tests/verify_dt_upgrade.erl | 4 +- tests/verify_dvv_repl.erl | 2 +- tests/verify_handoff_mixed.erl | 4 +- tests/verify_kv_health_check.erl | 2 +- tests/verify_link_walk_urls.erl | 2 +- tests/verify_listkeys.erl | 12 +- tests/verify_listkeys_eqcfsm.erl | 2 +- tests/verify_mr_prereduce_node_down.erl | 2 +- tests/verify_no_writes_on_read.erl | 2 +- tests/verify_object_limits.erl | 2 +- tests/verify_reset_bucket_props.erl | 2 +- tests/verify_riak_stats.erl | 7 +- tests/verify_secondary_index_reformat.erl | 4 +- tests/verify_vclock.erl | 6 +- tests/verify_vclock_encoding_upgrade.erl | 4 +- tests/yz_ensemble.erl | 4 +- 59 files changed, 345 insertions(+), 323 deletions(-) create mode 100644 src/rt_pb.erl diff --git a/src/rt.erl b/src/rt.erl index 6113bc315..b2e3f4d9b 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -1260,149 +1260,10 @@ after ?assert(false) end. -%%%=================================================================== - -%% @doc PBC-based version of {@link systest_write/1} -pbc_systest_write(Node, Size) -> -pbc_systest_write(Node, Size, 2). - -pbc_systest_write(Node, Size, W) -> -pbc_systest_write(Node, 1, Size, <<"systest">>, W). - -pbc_systest_write(Node, Start, End, Bucket, W) -> -rt:wait_for_service(Node, riak_kv), -Pid = pbc(Node), -F = fun(N, Acc) -> - Obj = riakc_obj:new(Bucket, <>, <>), - try riakc_pb_socket:put(Pid, Obj, W) of - ok -> - Acc; - Other -> - [{N, Other} | Acc] - catch - What:Why -> - [{N, {What, Why}} | Acc] - end -end, -lists:foldl(F, [], lists:seq(Start, End)). - -pbc_systest_read(Node, Size) -> -pbc_systest_read(Node, Size, 2). - -pbc_systest_read(Node, Size, R) -> -pbc_systest_read(Node, 1, Size, <<"systest">>, R). - -pbc_systest_read(Node, Start, End, Bucket, R) -> -rt:wait_for_service(Node, riak_kv), -Pid = pbc(Node), -F = fun(N, Acc) -> - case riakc_pb_socket:get(Pid, Bucket, <>, R) of - {ok, Obj} -> - case riakc_obj:get_value(Obj) of - <> -> - Acc; - WrongVal -> - [{N, {wrong_val, WrongVal}} | Acc] - end; - Other -> - [{N, Other} | Acc] - end -end, -lists:foldl(F, [], lists:seq(Start, End)). - %%%=================================================================== %%% PBC & HTTPC Functions %%%=================================================================== -%% @doc get me a protobuf client process and hold the mayo! --spec pbc(node()) -> pid(). -pbc(Node) -> -pbc(Node, [{auto_reconnect, true}]). - --spec pbc(node(), proplists:proplist()) -> pid(). -pbc(Node, Options) -> -rt:wait_for_service(Node, riak_kv), -ConnInfo = proplists:get_value(Node, connection_info([Node])), -{IP, PBPort} = proplists:get_value(pb, ConnInfo), -{ok, Pid} = riakc_pb_socket:start_link(IP, PBPort, Options), -Pid. - -%% @doc does a read via the erlang protobuf client --spec pbc_read(pid(), binary(), binary()) -> binary(). -pbc_read(Pid, Bucket, Key) -> -pbc_read(Pid, Bucket, Key, []). - --spec pbc_read(pid(), binary(), binary(), [any()]) -> binary(). -pbc_read(Pid, Bucket, Key, Options) -> -{ok, Value} = riakc_pb_socket:get(Pid, Bucket, Key, Options), -Value. - --spec pbc_read_check(pid(), binary(), binary(), [any()]) -> boolean(). -pbc_read_check(Pid, Bucket, Key, Allowed) -> -pbc_read_check(Pid, Bucket, Key, Allowed, []). - --spec pbc_read_check(pid(), binary(), binary(), [any()], [any()]) -> boolean(). -pbc_read_check(Pid, Bucket, Key, Allowed, Options) -> -case riakc_pb_socket:get(Pid, Bucket, Key, Options) of -{ok, _} -> - true = lists:member(ok, Allowed); -Other -> - lists:member(Other, Allowed) orelse throw({failed, Other, Allowed}) -end. - -%% @doc does a write via the erlang protobuf client --spec pbc_write(pid(), binary(), binary(), binary()) -> atom(). -pbc_write(Pid, Bucket, Key, Value) -> -Object = riakc_obj:new(Bucket, Key, Value), -riakc_pb_socket:put(Pid, Object). - -%% @doc does a write via the erlang protobuf client plus content-type --spec pbc_write(pid(), binary(), binary(), binary(), list()) -> atom(). -pbc_write(Pid, Bucket, Key, Value, CT) -> -Object = riakc_obj:new(Bucket, Key, Value, CT), -riakc_pb_socket:put(Pid, Object). - -%% @doc sets a bucket property/properties via the erlang protobuf client --spec pbc_set_bucket_prop(pid(), binary(), [proplists:property()]) -> atom(). -pbc_set_bucket_prop(Pid, Bucket, PropList) -> -riakc_pb_socket:set_bucket(Pid, Bucket, PropList). - -%% @doc Puts the contents of the given file into the given bucket using the -%% filename as a key and assuming a plain text content type. -pbc_put_file(Pid, Bucket, Key, Filename) -> -{ok, Contents} = file:read_file(Filename), -riakc_pb_socket:put(Pid, riakc_obj:new(Bucket, Key, Contents, "text/plain")). - -%% @doc Puts all files in the given directory into the given bucket using the -%% filename as a key and assuming a plain text content type. -pbc_put_dir(Pid, Bucket, Dir) -> -lager:info("Putting files from dir ~p into bucket ~p", [Dir, Bucket]), -{ok, Files} = file:list_dir(Dir), -[pbc_put_file(Pid, Bucket, list_to_binary(F), filename:join([Dir, F])) -|| F <- Files]. - -%% @doc True if the given keys have been really, really deleted. -%% Useful when you care about the keys not being there. Delete simply writes -%% tombstones under the given keys, so those are still seen by key folding -%% operations. -pbc_really_deleted(Pid, Bucket, Keys) -> -StillThere = -fun(K) -> - Res = riakc_pb_socket:get(Pid, Bucket, K, - [{r, 1}, - {notfound_ok, false}, - {basic_quorum, false}, - deletedvclock]), - case Res of - {error, notfound} -> - false; - _ -> - %% Tombstone still around - true - end -end, -[] == lists:filter(StillThere, Keys). - %% @doc Returns HTTPS URL information for a list of Nodes https_url(Nodes) when is_list(Nodes) -> [begin diff --git a/src/rt_pb.erl b/src/rt_pb.erl new file mode 100644 index 000000000..959f7a815 --- /dev/null +++ b/src/rt_pb.erl @@ -0,0 +1,164 @@ +%% 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(rt_pb). +-include("rt.hrl"). +-include_lib("eunit/include/eunit.hrl"). + +-compile(export_all). +-export([pbc/1, + pbc_read/3, + pbc_read/4, + pbc_read_check/4, + pbc_read_check/5, + pbc_set_bucket_prop/3, + pbc_write/4, + pbc_put_dir/3, + pbc_put_file/4, + pbc_really_deleted/3]). + +-define(HARNESS, (rt_config:get(rt_harness))). + +%% @doc get me a protobuf client process and hold the mayo! +-spec pbc(node()) -> pid(). +pbc(Node) -> + rt:wait_for_service(Node, riak_kv), + ConnInfo = proplists:get_value(Node, rt:connection_info([Node])), + {IP, PBPort} = proplists:get_value(pb, ConnInfo), + {ok, Pid} = riakc_pb_socket:start_link(IP, PBPort, [{auto_reconnect, true}]), + Pid. + +%% @doc does a read via the erlang protobuf client +-spec pbc_read(pid(), binary(), binary()) -> binary(). +pbc_read(Pid, Bucket, Key) -> + pbc_read(Pid, Bucket, Key, []). + +-spec pbc_read(pid(), binary(), binary(), [any()]) -> binary(). +pbc_read(Pid, Bucket, Key, Options) -> + {ok, Value} = riakc_pb_socket:get(Pid, Bucket, Key, Options), + Value. + +-spec pbc_read_check(pid(), binary(), binary(), [any()]) -> boolean(). +pbc_read_check(Pid, Bucket, Key, Allowed) -> + pbc_read_check(Pid, Bucket, Key, Allowed, []). + +-spec pbc_read_check(pid(), binary(), binary(), [any()], [any()]) -> boolean(). +pbc_read_check(Pid, Bucket, Key, Allowed, Options) -> + case riakc_pb_socket:get(Pid, Bucket, Key, Options) of + {ok, _} -> + true = lists:member(ok, Allowed); + Other -> + lists:member(Other, Allowed) orelse throw({failed, Other, Allowed}) + end. + +%% @doc does a write via the erlang protobuf client +-spec pbc_write(pid(), binary(), binary(), binary()) -> atom(). +pbc_write(Pid, Bucket, Key, Value) -> + Object = riakc_obj:new(Bucket, Key, Value), + riakc_pb_socket:put(Pid, Object). + +%% @doc does a write via the erlang protobuf client plus content-type +-spec pbc_write(pid(), binary(), binary(), binary(), list()) -> atom(). +pbc_write(Pid, Bucket, Key, Value, CT) -> + Object = riakc_obj:new(Bucket, Key, Value, CT), + riakc_pb_socket:put(Pid, Object). + +%% @doc sets a bucket property/properties via the erlang protobuf client +-spec pbc_set_bucket_prop(pid(), binary(), [proplists:property()]) -> atom(). +pbc_set_bucket_prop(Pid, Bucket, PropList) -> + riakc_pb_socket:set_bucket(Pid, Bucket, PropList). + +%% @doc Puts the contents of the given file into the given bucket using the +%% filename as a key and assuming a plain text content type. +pbc_put_file(Pid, Bucket, Key, Filename) -> + {ok, Contents} = file:read_file(Filename), + riakc_pb_socket:put(Pid, riakc_obj:new(Bucket, Key, Contents, "text/plain")). + +%% @doc Puts all files in the given directory into the given bucket using the +%% filename as a key and assuming a plain text content type. +pbc_put_dir(Pid, Bucket, Dir) -> + lager:info("Putting files from dir ~p into bucket ~p", [Dir, Bucket]), + {ok, Files} = file:list_dir(Dir), + [pbc_put_file(Pid, Bucket, list_to_binary(F), filename:join([Dir, F])) + || F <- Files]. + +%% @doc True if the given keys have been really, really deleted. +%% Useful when you care about the keys not being there. Delete simply writes +%% tombstones under the given keys, so those are still seen by key folding +%% operations. +pbc_really_deleted(Pid, Bucket, Keys) -> + StillThere = + fun(K) -> + Res = riakc_pb_socket:get(Pid, Bucket, K, + [{r, 1}, + {notfound_ok, false}, + {basic_quorum, false}, + deletedvclock]), + case Res of + {error, notfound} -> + false; + _ -> + %% Tombstone still around + true + end + end, + [] == lists:filter(StillThere, Keys). + +%% @doc PBC-based version of {@link systest_write/1} +pbc_systest_write(Node, Size) -> + pbc_systest_write(Node, Size, 2). + +pbc_systest_write(Node, Size, W) -> + pbc_systest_write(Node, 1, Size, <<"systest">>, W). + +pbc_systest_write(Node, Start, End, Bucket, W) -> + rt:wait_for_service(Node, riak_kv), + Pid = pbc(Node), + F = fun(N, Acc) -> + Obj = riakc_obj:new(Bucket, <>, <>), + try riakc_pb_socket:put(Pid, Obj, W) of + ok -> + Acc; + Other -> + [{N, Other} | Acc] + catch + What:Why -> + [{N, {What, Why}} | Acc] + end + end, + lists:foldl(F, [], lists:seq(Start, End)). + +pbc_systest_read(Node, Size) -> + pbc_systest_read(Node, Size, 2). + +pbc_systest_read(Node, Size, R) -> + pbc_systest_read(Node, 1, Size, <<"systest">>, R). + +pbc_systest_read(Node, Start, End, Bucket, R) -> + rt:wait_for_service(Node, riak_kv), + Pid = pbc(Node), + F = fun(N, Acc) -> + case riakc_pb_socket:get(Pid, Bucket, <>, R) of + {ok, Obj} -> + case riakc_obj:get_value(Obj) of + <> -> + Acc; + WrongVal -> + [{N, {wrong_val, WrongVal}} | Acc] + end; + Other -> + [{N, Other} | Acc] + end + end, + lists:foldl(F, [], lists:seq(Start, End)). diff --git a/tests/bucket_props_roundtrip.erl b/tests/bucket_props_roundtrip.erl index 5febb6c80..7b06fd38b 100644 --- a/tests/bucket_props_roundtrip.erl +++ b/tests/bucket_props_roundtrip.erl @@ -70,7 +70,7 @@ confirm(#rt_properties{nodes=Nodes}, _MD) -> check_prop_set_and_get(Node, Prop, One, Two) -> lager:info("-------- Testing roundtrip for property '~p' ---------", [Prop]), HTTP = rt:httpc(Node), - PBC = rt:pbc(Node), + PBC = rt_pb:pbc(Node), lager:info("HTTP set = ~p", [One]), http_set_property(HTTP, Prop, One), lager:info("PBC get should == ~p", [One]), diff --git a/tests/ensemble_basic3.erl b/tests/ensemble_basic3.erl index 50878e797..996517fd3 100644 --- a/tests/ensemble_basic3.erl +++ b/tests/ensemble_basic3.erl @@ -55,17 +55,17 @@ confirm() -> Partitioned = [VNode || {_, VNode} <- PartitionedVN], MajorityVN = All -- PartitionedVN, - PBC = rt:pbc(Node), + PBC = rt_pb:pbc(Node), lager:info("Partitioning quorum minority: ~p", [Partitioned]), Part = rt:partition(Nodes -- Partitioned, Partitioned), ensemble_util:wait_until_stable(Node, Quorum), lager:info("Writing ~p consistent keys", [1000]), - [ok = rt:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], + [ok = rt_pb:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], lager:info("Read keys to verify they exist"), - [rt:pbc_read(PBC, Bucket, Key) || Key <- Keys], + [rt_pb:pbc_read(PBC, Bucket, Key) || Key <- Keys], lager:info("Healing partition"), rt:heal(Part), @@ -83,7 +83,7 @@ confirm() -> vnode_util:resume_vnode(Pid), ensemble_util:wait_until_stable(Node, Quorum), lager:info("Re-reading keys"), - [rt:pbc_read(PBC, Bucket, Key) || Key <- Keys], + [rt_pb:pbc_read(PBC, Bucket, Key) || Key <- Keys], lager:info("Suspending vnode: ~p", [VIdx]), Pid2 = vnode_util:suspend_vnode(VNode, VIdx), orddict:store(VN, Pid2, Suspended) @@ -93,5 +93,5 @@ confirm() -> [vnode_util:resume_vnode(Pid) || {_, Pid} <- L3], ensemble_util:wait_until_stable(Node, NVal), lager:info("Re-reading keys"), - [rt:pbc_read(PBC, Bucket, Key) || Key <- Keys], + [rt_pb:pbc_read(PBC, Bucket, Key) || Key <- Keys], pass. diff --git a/tests/ensemble_basic4.erl b/tests/ensemble_basic4.erl index 494e21c55..35cd92f50 100644 --- a/tests/ensemble_basic4.erl +++ b/tests/ensemble_basic4.erl @@ -48,7 +48,7 @@ confirm() -> PartitionedVN = lists:sublist(Other, Minority), Partitioned = [VNode || {_, VNode} <- PartitionedVN], - PBC = rt:pbc(Node), + PBC = rt_pb:pbc(Node), lager:info("Partitioning quorum minority: ~p", [Partitioned]), Part = rt:partition(Nodes -- Partitioned, Partitioned), @@ -56,10 +56,10 @@ confirm() -> ensemble_util:wait_until_stable(Node, Quorum), lager:info("Writing ~p consistent keys", [1000]), - [ok = rt:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], + [ok = rt_pb:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], lager:info("Read keys to verify they exist"), - [rt:pbc_read(PBC, Bucket, Key) || Key <- Keys], + [rt_pb:pbc_read(PBC, Bucket, Key) || Key <- Keys], lager:info("Healing partition"), rt:heal(Part), diff --git a/tests/ensemble_interleave.erl b/tests/ensemble_interleave.erl index 5e34659f6..105bffffa 100644 --- a/tests/ensemble_interleave.erl +++ b/tests/ensemble_interleave.erl @@ -65,7 +65,7 @@ confirm() -> [KillFirst,KillSecond|Suspend] = All -- PartitionedVN, io:format("PL: ~p~n", [PL]), - PBC = rt:pbc(Node), + PBC = rt_pb:pbc(Node), Options = [{timeout, 500}], rpc:multicall(Nodes, riak_kv_entropy_manager, set_mode, [manual]), @@ -73,10 +73,10 @@ confirm() -> ensemble_util:wait_until_stable(Node, Quorum), lager:info("Writing ~p consistent keys", [1000]), - [ok = rt:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], + [ok = rt_pb:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], lager:info("Read keys to verify they exist"), - [rt:pbc_read(PBC, Bucket, Key, Options) || Key <- Keys], + [rt_pb:pbc_read(PBC, Bucket, Key, Options) || Key <- Keys], rt:heal(Part), [begin @@ -97,5 +97,5 @@ confirm() -> lager:info("Re-reading keys to verify they exist"), Expect = [ok, {error, timeout}, {error, <<"timeout">>}, {error, <<"failed">>}], - [rt:pbc_read_check(PBC, Bucket, Key, Expect, Options) || Key <- Keys], + [rt_pb:pbc_read_check(PBC, Bucket, Key, Expect, Options) || Key <- Keys], pass. diff --git a/tests/ensemble_remove_node2.erl b/tests/ensemble_remove_node2.erl index 3efaa7ace..9d3a6945a 100644 --- a/tests/ensemble_remove_node2.erl +++ b/tests/ensemble_remove_node2.erl @@ -44,9 +44,9 @@ confirm() -> ensemble_util:wait_until_stable(Node, NVal), Bucket = {<<"strong">>, <<"test">>}, Key = <<"testkey">>, - PBC = rt:pbc(Node), - ok = rt:pbc_write(PBC, Bucket, Key, testval), - Val1 = rt:pbc_read(PBC, Bucket, Key), + PBC = rt_pb:pbc(Node), + ok = rt_pb:pbc_write(PBC, Bucket, Key, testval), + Val1 = rt_pb:pbc_read(PBC, Bucket, Key), ?assertEqual(element(1, Val1), riakc_obj), %% Don't allow node deletions in riak_ensemble. This should prevent the @@ -80,7 +80,7 @@ confirm() -> %% We should still be able to read from k/v ensembles, but the nodes should %% never exit lager:info("Reading From SC Bucket"), - Val2 = rt:pbc_read(PBC, Bucket, Key), + Val2 = rt_pb:pbc_read(PBC, Bucket, Key), ?assertEqual(element(1, Val2), riakc_obj), ok = ensemble_util:wait_until_stable(Node, NVal), diff --git a/tests/ensemble_sync.erl b/tests/ensemble_sync.erl index db1d19695..4b403d5a9 100644 --- a/tests/ensemble_sync.erl +++ b/tests/ensemble_sync.erl @@ -97,7 +97,7 @@ run_scenario(Nodes, NVal, {NumKill, NumSuspend, NumValid, _, Name, Expect}) -> {AfterVN, _} = lists:split(NumValid, Valid3), io:format("PL: ~p~n", [PL]), - PBC = rt:pbc(Node), + PBC = rt_pb:pbc(Node), Options = [{timeout, 2000}], rpc:multicall(Nodes, riak_kv_entropy_manager, set_mode, [manual]), @@ -107,10 +107,10 @@ run_scenario(Nodes, NVal, {NumKill, NumSuspend, NumValid, _, Name, Expect}) -> %% Write data while minority is partitioned lager:info("Writing ~p consistent keys", [1000]), - [ok = rt:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], + [ok = rt_pb:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], lager:info("Read keys to verify they exist"), - [rt:pbc_read(PBC, Bucket, Key, Options) || Key <- Keys], + [rt_pb:pbc_read(PBC, Bucket, Key, Options) || Key <- Keys], rt:heal(Part), %% Suspend desired number of valid vnodes @@ -133,7 +133,7 @@ run_scenario(Nodes, NVal, {NumKill, NumSuspend, NumValid, _, Name, Expect}) -> ensemble_util:wait_until_stable(Node, Quorum), lager:info("Checking that key results match scenario"), - [rt:pbc_read_check(PBC, Bucket, Key, Expect, Options) || Key <- Keys], + [rt_pb:pbc_read_check(PBC, Bucket, Key, Expect, Options) || Key <- Keys], lager:info("Re-enabling AAE"), rpc:multicall(Nodes, riak_kv_entropy_manager, enable, []), @@ -150,7 +150,7 @@ run_scenario(Nodes, NVal, {NumKill, NumSuspend, NumValid, _, Name, Expect}) -> ok; false -> lager:info("Re-reading keys to verify they exist"), - [rt:pbc_read(PBC, Bucket, Key, Options) || Key <- Keys] + [rt_pb:pbc_read(PBC, Bucket, Key, Options) || Key <- Keys] end, lager:info("Scenario passed"), diff --git a/tests/ensemble_vnode_crash.erl b/tests/ensemble_vnode_crash.erl index 59fbd56e8..7bd59bdf0 100644 --- a/tests/ensemble_vnode_crash.erl +++ b/tests/ensemble_vnode_crash.erl @@ -46,13 +46,13 @@ confirm() -> PL = rpc:call(Node, riak_core_apl, get_primary_apl, [DocIdx, NVal, riak_kv]), {{Key1Idx, Key1Node}, _} = hd(PL), - PBC = rt:pbc(Node), + PBC = rt_pb:pbc(Node), lager:info("Writing ~p consistent keys", [1000]), - [ok = rt:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], + [ok = rt_pb:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], lager:info("Read keys to verify they exist"), - [rt:pbc_read(PBC, Bucket, Key) || Key <- Keys], + [rt_pb:pbc_read(PBC, Bucket, Key) || Key <- Keys], %% Setting up intercept to ensure that %% riak_kv_ensemble_backend:handle_down/4 gets called when a vnode or vnode @@ -79,7 +79,7 @@ confirm() -> lager:info("Wait for stable ensembles"), ensemble_util:wait_until_stable(Node, NVal), lager:info("Re-reading keys"), - [rt:pbc_read(PBC, Bucket, Key) || Key <- Keys], + [rt_pb:pbc_read(PBC, Bucket, Key) || Key <- Keys], lager:info("Killing Vnode Proxy for Key1"), Proxy = rpc:call(Key1Node, riak_core_vnode_proxy, reg_name, [riak_kv_vnode, @@ -95,7 +95,7 @@ confirm() -> lager:info("Wait for stable ensembles"), ensemble_util:wait_until_stable(Node, NVal), lager:info("Re-reading keys"), - [rt:pbc_read(PBC, Bucket, Key) || Key <- Keys], + [rt_pb:pbc_read(PBC, Bucket, Key) || Key <- Keys], pass. diff --git a/tests/jmx_verify.erl b/tests/jmx_verify.erl index c11ac0e75..26e295aa8 100644 --- a/tests/jmx_verify.erl +++ b/tests/jmx_verify.erl @@ -91,7 +91,7 @@ confirm() -> <<"node_put_fsm_time_100">>]), lager:info("Make PBC Connection"), - Pid = rt:pbc(Node1), + Pid = rt_pb:pbc(Node1), JMX3 = jmx_dump(JMXDumpCmd), rt:systest_write(Node1, 1), @@ -101,7 +101,7 @@ confirm() -> {<<"pbc_active">>, 1}]), lager:info("Force Read Repair"), - rt:pbc_write(Pid, <<"testbucket">>, <<"1">>, <<"blah!">>), + rt_pb:pbc_write(Pid, <<"testbucket">>, <<"1">>, <<"blah!">>), rt:pbc_set_bucket_prop(Pid, <<"testbucket">>, [{n_val, 4}]), JMX4 = jmx_dump(JMXDumpCmd), @@ -109,7 +109,7 @@ confirm() -> verify_inc(JMX3, JMX4, [{<<"read_repairs_total">>, 0}, {<<"read_repairs">>, 0}]), - _Value = rt:pbc_read(Pid, <<"testbucket">>, <<"1">>), + _Value = rt_pb:pbc_read(Pid, <<"testbucket">>, <<"1">>), %%Stats5 = get_stats(Node1), JMX5 = jmx_dump(JMXDumpCmd), diff --git a/tests/loaded_upgrade.erl b/tests/loaded_upgrade.erl index 50e46e1ab..7e89350fb 100644 --- a/tests/loaded_upgrade.erl +++ b/tests/loaded_upgrade.erl @@ -145,7 +145,7 @@ bucket(mapred) -> <<"bryanitbs">>; bucket(search) -> <<"scotts_spam">>. seed_search(Node) -> - Pid = rt:pbc(Node), + Pid = rt_pb:pbc(Node), SpamDir = rt_config:get(spam_dir), Files = case SpamDir of undefined -> undefined; @@ -157,7 +157,7 @@ seed_search(Node) -> seed_search(_Pid, []) -> ok; seed_search(Pid, [File|Files]) -> Key = list_to_binary(filename:basename(File)), - rt:pbc_put_file(Pid, bucket(search), Key, File), + rt_pb:pbc_put_file(Pid, bucket(search), Key, File), seed_search(Pid, Files). kv_seed(Node) -> @@ -201,7 +201,7 @@ mr_seed(Node) -> seed(Node, 0, 9999, ValFun). seed(Node, Start, End, ValFun) -> - PBC = rt:pbc(Node), + PBC = rt_pb:pbc(Node), [ begin Obj = ValFun(Key), diff --git a/tests/loaded_upgrade_worker_sup.erl b/tests/loaded_upgrade_worker_sup.erl index 3e62c440c..c1d1c5c26 100644 --- a/tests/loaded_upgrade_worker_sup.erl +++ b/tests/loaded_upgrade_worker_sup.erl @@ -237,12 +237,12 @@ assert_equal(Expected, Actual) -> Actual == Expected. pb_pid_recycler(undefined, Node) -> - rt:pbc(Node); + rt_pb:pbc(Node); pb_pid_recycler(Pid, Node) -> case riakc_pb_socket:is_connected(Pid) of true -> Pid; _ -> riakc_pb_socket:stop(Pid), - rt:pbc(Node) + rt_pb:pbc(Node) end. diff --git a/tests/mapred_basic_compat.erl b/tests/mapred_basic_compat.erl index 6b827b7fa..ee7f9b91f 100644 --- a/tests/mapred_basic_compat.erl +++ b/tests/mapred_basic_compat.erl @@ -89,7 +89,7 @@ load_test_data([Node|_]) -> [{<<"link 1">>, [{?LINK_BUCKET, <<"nokey-1">>}]}, {<<"link 2">>, [{?LINK_BUCKET, <<"nokey-2">>}]}]), - C = rt:pbc(Node), + C = rt_pb:pbc(Node), ok = riakc_pb_socket:put(C, riakc_obj:update_metadata(Obj, MD)), %% Some bucket type entries {mytype,foonum}/bar{1..10} diff --git a/tests/mapred_javascript.erl b/tests/mapred_javascript.erl index fedfadf5c..301c494f0 100644 --- a/tests/mapred_javascript.erl +++ b/tests/mapred_javascript.erl @@ -67,7 +67,7 @@ load_test_data([Node|_]) -> Map = riakc_obj:new(?JS_BUCKET, <<"map">>, ?MAP_JS, "text/plain"), Red = riakc_obj:new(?JS_BUCKET, <<"reduce">>, ?REDUCE_JS, "text/plain"), - C = rt:pbc(Node), + C = rt_pb:pbc(Node), ok = riakc_pb_socket:put(C, Map), ok = riakc_pb_socket:put(C, Red), riakc_pb_socket:stop(C). diff --git a/tests/mapred_notfound_failover.erl b/tests/mapred_notfound_failover.erl index 488807f01..b0e8fad9f 100644 --- a/tests/mapred_notfound_failover.erl +++ b/tests/mapred_notfound_failover.erl @@ -78,7 +78,7 @@ replica_notfound(Node, {HashMod, HashFun}, MissingBucket, MissingKey, MissingValue) -> %% create a value for the "missing" key Obj = riakc_obj:new(MissingBucket, MissingKey, MissingValue), - C = rt:pbc(Node), + C = rt_pb:pbc(Node), ok = riakc_pb_socket:put(C, Obj, [{w, 3}]), riakc_pb_socket:stop(C), %% and now kill the first replica; this will make the vnode local diff --git a/tests/mapred_search_switch.erl b/tests/mapred_search_switch.erl index 1b9bbfc8c..e34951b76 100644 --- a/tests/mapred_search_switch.erl +++ b/tests/mapred_search_switch.erl @@ -282,7 +282,7 @@ got_error(_) -> false. run_bucket_mr([Node|_], Bucket, Common) -> - C = rt:pbc(Node), + C = rt_pb:pbc(Node), riakc_pb_socket:mapred( C, %% TODO: check {search, Bucket, Common, Filter} diff --git a/tests/overload.erl b/tests/overload.erl index 740cfcb11..2bcb4420e 100644 --- a/tests/overload.erl +++ b/tests/overload.erl @@ -268,10 +268,8 @@ node_overload_check(Pid) -> end. list_keys(Node) -> - Pid = rt:pbc(Node, [{auto_reconnect, true}, {queue_if_disconnected, true}]), - Res = riakc_pb_socket:list_keys(Pid, {<<"normal_type">>, ?BUCKET}, infinity), - riakc_pb_socket:stop(Pid), - Res. + Pid = rt_pb:pbc(Node), + riakc_pb_socket:list_keys(Pid, ?BUCKET, 30000). list_buckets(Node) -> {ok, C} = riak:client_connect(Node), diff --git a/tests/partition_repair.erl b/tests/partition_repair.erl index 757aabf85..b3223eb3a 100644 --- a/tests/partition_repair.erl +++ b/tests/partition_repair.erl @@ -86,8 +86,8 @@ confirm() -> rt:enable_search_hook(hd(Nodes), Bucket), lager:info("Insert Scott's spam emails"), - Pbc = rt:pbc(hd(Nodes)), - rt:pbc_put_dir(Pbc, Bucket, SpamDir), + Pbc = rt_pb:pbc(hd(Nodes)), + rt_pb:pbc_put_dir(Pbc, Bucket, SpamDir), lager:info("Stash ITFs for each partition"), %% @todo Should riak_test guarantee that the scratch pad is clean instead? diff --git a/tests/replication/repl_aae_fullsync.erl b/tests/replication/repl_aae_fullsync.erl index 1a60ac0b5..cf23f0bdf 100644 --- a/tests/replication/repl_aae_fullsync.erl +++ b/tests/replication/repl_aae_fullsync.erl @@ -339,8 +339,8 @@ difference_test() -> connect_cluster(LeaderA, BPort, "B"), %% Get PBC connections. - APBC = rt:pbc(LeaderA), - BPBC = rt:pbc(LeaderB), + APBC = rt_pb:pbc(LeaderA), + BPBC = rt_pb:pbc(LeaderB), %% Write key. ok = riakc_pb_socket:put(APBC, diff --git a/tests/replication/repl_bucket_types.erl b/tests/replication/repl_bucket_types.erl index 8d17a5a11..036b03b75 100644 --- a/tests/replication/repl_bucket_types.erl +++ b/tests/replication/repl_bucket_types.erl @@ -20,8 +20,8 @@ setup(Type) -> {LeaderA, LeaderB, ANodes, BNodes} = ClusterNodes = make_clusters(Type), - PBA = rt:pbc(LeaderA), - PBB = rt:pbc(LeaderB), + PBA = rt_pb:pbc(LeaderA), + PBB = rt_pb:pbc(LeaderB), {DefinedType, UndefType} = Types = {<<"working_type">>, <<"undefined_type">>}, diff --git a/tests/replication/replication2.erl b/tests/replication/replication2.erl index cc9b07e70..cae4b2c81 100644 --- a/tests/replication/replication2.erl +++ b/tests/replication/replication2.erl @@ -555,7 +555,7 @@ pb_write_during_shutdown([AFirst|_] = ANodes, [BFirst|_] = BNodes) -> ConnInfo = proplists:get_value(Target, rt:connection_info([Target])), {IP, Port} = proplists:get_value(pb, ConnInfo), lager:info("Connecting to pb socket ~p:~p on ~p", [IP, Port, Target]), - PBSock = rt:pbc(Target), + PBSock = rt_pb:pbc(Target), %% do the stop in the background while we're writing keys spawn(fun() -> diff --git a/tests/replication/replication2_pg.erl b/tests/replication/replication2_pg.erl index 82261724b..de38d3c89 100644 --- a/tests/replication/replication2_pg.erl +++ b/tests/replication/replication2_pg.erl @@ -172,20 +172,20 @@ test_basic_pg(Mode, SSL) -> EnabledFor -> lager:info("PG enabled for cluster ~p",[EnabledFor]) end, - PidA = rt:pbc(LeaderA), + PidA = rt_pb:pbc(LeaderA), {ok,CidA}=riak_repl_pb_api:get_clusterid(PidA), lager:info("Cluster ID for A = ~p", [CidA]), {Bucket, KeyA, ValueA} = make_test_object("a"), {Bucket, KeyB, ValueB} = make_test_object("b"), - rt:pbc_write(PidA, Bucket, KeyA, ValueA), - rt:pbc_write(PidA, Bucket, KeyB, ValueB), + rt_pb:pbc_write(PidA, Bucket, KeyA, ValueA), + rt_pb:pbc_write(PidA, Bucket, KeyB, ValueB), _FirstA = hd(ANodes), FirstB = hd(BNodes), FirstC = hd(CNodes), - PidB = rt:pbc(FirstB), + PidB = rt_pb:pbc(FirstB), lager:info("Connected to cluster B"), {ok, PGResult} = riak_repl_pb_api:get(PidB,Bucket,KeyA,CidA), ?assertEqual(ValueA, riakc_obj:get_value(PGResult)), @@ -237,7 +237,7 @@ test_basic_pg(Mode, SSL) -> EnabledFor3 -> lager:info("PG enabled for cluster ~p",[EnabledFor3]) end, - PidC = rt:pbc(FirstC), + PidC = rt_pb:pbc(FirstC), Options = [{n_val, 1}, {sloppy_quorum, false}], lager:info("Test proxy get from C using options: ~p", [Options]), @@ -299,16 +299,16 @@ test_12_pg(Mode, SSL) -> end, [rt:wait_until_ring_converged(Ns) || Ns <- [ANodes, BNodes, CNodes]], - PidA = rt:pbc(LeaderA), - rt:pbc_write(PidA, Bucket, KeyA, ValueA), - rt:pbc_write(PidA, Bucket, KeyB, ValueB), + PidA = rt_pb:pbc(LeaderA), + rt_pb:pbc_write(PidA, Bucket, KeyA, ValueA), + rt_pb:pbc_write(PidA, Bucket, KeyB, ValueB), {ok,CidA}=riak_repl_pb_api:get_clusterid(PidA), lager:info("Cluster ID for A = ~p", [CidA]), LeaderB = rpc:call(FirstB, riak_repl2_leader, leader_node, []), rt:log_to_nodes([LeaderB], "Trying to use PG while it's disabled"), - PidB = rt:pbc(LeaderB), + PidB = rt_pb:pbc(LeaderB), ?assertEqual({error, notfound}, riak_repl_pb_api:get(PidB, Bucket, KeyA, CidA)), @@ -332,7 +332,7 @@ test_12_pg(Mode, SSL) -> lager:info("Trying proxy_get"), LeaderB2 = rpc:call(FirstB, riak_repl2_leader, leader_node, []), - PidB2 = rt:pbc(LeaderB2), + PidB2 = rt_pb:pbc(LeaderB2), {ok, PGResult} = riak_repl_pb_api:get(PidB2, Bucket, KeyB, CidA), lager:info("PGResult: ~p", [PGResult]), ?assertEqual(ValueB, riakc_obj:get_value(PGResult)), @@ -388,7 +388,7 @@ test_pg_proxy(SSL) -> EnabledFor -> lager:info("PG enabled for cluster ~p",[EnabledFor]) end, - PidA = rt:pbc(LeaderA), + PidA = rt_pb:pbc(LeaderA), {ok,CidA}=riak_repl_pb_api:get_clusterid(PidA), lager:info("Cluster ID for A = ~p", [CidA]), @@ -398,16 +398,16 @@ test_pg_proxy(SSL) -> {Bucket, KeyC, ValueC} = make_test_object("c"), {Bucket, KeyD, ValueD} = make_test_object("d"), - rt:pbc_write(PidA, Bucket, KeyA, ValueA), - rt:pbc_write(PidA, Bucket, KeyB, ValueB), - rt:pbc_write(PidA, Bucket, KeyC, ValueC), - rt:pbc_write(PidA, Bucket, KeyD, ValueD), + rt_pb:pbc_write(PidA, Bucket, KeyA, ValueA), + rt_pb:pbc_write(PidA, Bucket, KeyB, ValueB), + rt_pb:pbc_write(PidA, Bucket, KeyC, ValueC), + rt_pb:pbc_write(PidA, Bucket, KeyD, ValueD), %% sanity check. You know, like the 10000 tests that autoconf runs %% before it actually does any work. FirstA = hd(ANodes), FirstB = hd(BNodes), _FirstC = hd(CNodes), - PidB = rt:pbc(FirstB), + PidB = rt_pb:pbc(FirstB), lager:info("Connected to cluster B"), {ok, PGResult} = riak_repl_pb_api:get(PidB,Bucket,KeyA,CidA), ?assertEqual(ValueA, riakc_obj:get_value(PGResult)), @@ -421,7 +421,7 @@ test_pg_proxy(SSL) -> rt:stop(PGLeaderB), [RunningBNode | _ ] = BNodes -- [PGLeaderB], repl_util:wait_until_leader(RunningBNode), - PidB2 = rt:pbc(RunningBNode), + PidB2 = rt_pb:pbc(RunningBNode), lager:info("Now trying proxy_get"), ?assertEqual(ok, wait_until_pg(RunningBNode, PidB2, Bucket, KeyC, CidA)), lager:info("If you got here, proxy_get worked after the pg block requesting leader was killed"), @@ -487,15 +487,15 @@ test_cluster_mapping(SSL) -> [rt:wait_until_ring_converged(Ns) || Ns <- [ANodes, BNodes, CNodes]], - PidA = rt:pbc(LeaderA), + PidA = rt_pb:pbc(LeaderA), {ok,CidA}=riak_repl_pb_api:get_clusterid(PidA), lager:info("Cluster ID for A = ~p", [CidA]), - PidB = rt:pbc(LeaderB), + PidB = rt_pb:pbc(LeaderB), {ok,CidB}=riak_repl_pb_api:get_clusterid(PidB), lager:info("Cluster ID for B = ~p", [CidB]), - PidC = rt:pbc(LeaderC), + PidC = rt_pb:pbc(LeaderC), {ok,CidC}=riak_repl_pb_api:get_clusterid(PidC), lager:info("Cluster ID for C = ~p", [CidC]), @@ -505,10 +505,10 @@ test_cluster_mapping(SSL) -> {Bucket, KeyC, ValueC} = make_test_object("c"), {Bucket, KeyD, ValueD} = make_test_object("d"), - rt:pbc_write(PidA, Bucket, KeyA, ValueA), - rt:pbc_write(PidA, Bucket, KeyB, ValueB), - rt:pbc_write(PidA, Bucket, KeyC, ValueC), - rt:pbc_write(PidA, Bucket, KeyD, ValueD), + rt_pb:pbc_write(PidA, Bucket, KeyA, ValueA), + rt_pb:pbc_write(PidA, Bucket, KeyB, ValueB), + rt_pb:pbc_write(PidA, Bucket, KeyC, ValueC), + rt_pb:pbc_write(PidA, Bucket, KeyD, ValueD), {ok, PGResult} = riak_repl_pb_api:get(PidA,Bucket,KeyA,CidA), @@ -605,8 +605,8 @@ test_bidirectional_pg(SSL) -> EnabledForB -> lager:info("PG enabled for cluster ~p",[EnabledForB]) end, - PidA = rt:pbc(LeaderA), - PidB = rt:pbc(FirstB), + PidA = rt_pb:pbc(LeaderA), + PidB = rt_pb:pbc(FirstB), {ok,CidA}=riak_repl_pb_api:get_clusterid(PidA), {ok,CidB}=riak_repl_pb_api:get_clusterid(PidB), @@ -617,10 +617,10 @@ test_bidirectional_pg(SSL) -> {Bucket, KeyB, ValueB} = make_test_object("b"), %% write some data to cluster A - rt:pbc_write(PidA, Bucket, KeyA, ValueA), + rt_pb:pbc_write(PidA, Bucket, KeyA, ValueA), %% write some data to cluster B - rt:pbc_write(PidB, Bucket, KeyB, ValueB), + rt_pb:pbc_write(PidB, Bucket, KeyB, ValueB), lager:info("Trying first get"), wait_until_pg(LeaderB, PidB, Bucket, KeyA, CidA), @@ -667,22 +667,22 @@ test_multiple_sink_pg(SSL) -> EnabledForC -> lager:info("PG enabled for cluster ~p",[EnabledForC]) end, - PidA = rt:pbc(LeaderA), + PidA = rt_pb:pbc(LeaderA), {ok,CidA}=riak_repl_pb_api:get_clusterid(PidA), lager:info("Cluster ID for A = ~p", [CidA]), {Bucket, KeyA, ValueA} = make_test_object("a"), {Bucket, KeyB, ValueB} = make_test_object("b"), - rt:pbc_write(PidA, Bucket, KeyA, ValueA), - rt:pbc_write(PidA, Bucket, KeyB, ValueB), + rt_pb:pbc_write(PidA, Bucket, KeyA, ValueA), + rt_pb:pbc_write(PidA, Bucket, KeyB, ValueB), _FirstA = hd(ANodes), FirstB = hd(BNodes), FirstC = hd(CNodes), - PidB = rt:pbc(FirstB), - PidC = rt:pbc(FirstC), + PidB = rt_pb:pbc(FirstB), + PidC = rt_pb:pbc(FirstC), {ok, PGResultB} = riak_repl_pb_api:get(PidB,Bucket,KeyA,CidA), ?assertEqual(ValueA, riakc_obj:get_value(PGResultB)), @@ -720,15 +720,15 @@ test_mixed_pg(SSL) -> EnabledFor -> lager:info("PG enabled for cluster ~p",[EnabledFor]) end, - PidA = rt:pbc(LeaderA), + PidA = rt_pb:pbc(LeaderA), {ok,CidA}=riak_repl_pb_api:get_clusterid(PidA), lager:info("Cluster ID for A = ~p", [CidA]), {Bucket, KeyB, ValueB} = make_test_object("b"), {Bucket, KeyC, ValueC} = make_test_object("c"), - rt:pbc_write(PidA, Bucket, KeyB, ValueB), - rt:pbc_write(PidA, Bucket, KeyC, ValueC), + rt_pb:pbc_write(PidA, Bucket, KeyB, ValueB), + rt_pb:pbc_write(PidA, Bucket, KeyC, ValueC), _FirstA = hd(ANodes), FirstB = hd(BNodes), @@ -756,8 +756,8 @@ test_mixed_pg(SSL) -> lager:info("Trying proxy_get"), LeaderC = rpc:call(FirstC, riak_repl2_leader, leader_node, []), - PidB = rt:pbc(FirstB), - PidC = rt:pbc(LeaderC), + PidB = rt_pb:pbc(FirstB), + PidC = rt_pb:pbc(LeaderC), {ok, PGResultB} = riak_repl_pb_api:get(PidB, Bucket, KeyB, CidA), lager:info("PGResultB: ~p", [PGResultB]), @@ -914,15 +914,15 @@ verify_topology_change(SourceNodes, SinkNodes) -> %% Get connections [SourceNode1, _SourceNode2] = SourceNodes, - SourceNode1Pid = rt:pbc(SourceNode1), + SourceNode1Pid = rt_pb:pbc(SourceNode1), [SinkNode1, SinkNode2] = SinkNodes, - SinkNode1Pid = rt:pbc(SinkNode1), + SinkNode1Pid = rt_pb:pbc(SinkNode1), {ok, SourceCid} = riak_repl_pb_api:get_clusterid(SourceNode1Pid), %% Write new object to source. lager:info("Writing key 'before' to the source."), {Bucket, KeyBefore, ValueBefore} = make_test_object("before"), - rt:pbc_write(SourceNode1Pid, Bucket, KeyBefore, ValueBefore), + rt_pb:pbc_write(SourceNode1Pid, Bucket, KeyBefore, ValueBefore), %% Verify proxy_get through the sink works. lager:info("Verifying key 'before' can be read through the sink."), @@ -993,7 +993,7 @@ verify_topology_change(SourceNodes, SinkNodes) -> lager:info("Writing key 'after' to the source."), {ok, SourceCid} = riak_repl_pb_api:get_clusterid(SourceNode1Pid), {Bucket, KeyPost, ValuePost} = make_test_object("after"), - rt:pbc_write(SourceNode1Pid, Bucket, KeyPost, ValuePost), + rt_pb:pbc_write(SourceNode1Pid, Bucket, KeyPost, ValuePost), %% Verify we can retrieve from source. lager:info("Verifying key 'after' can be read through the source."), diff --git a/tests/replication/rt_cascading.erl b/tests/replication/rt_cascading.erl index c8827c6b3..804aba5f5 100644 --- a/tests/replication/rt_cascading.erl +++ b/tests/replication/rt_cascading.erl @@ -97,7 +97,7 @@ simple_test_() -> end}, {"cascade a put from beginning down to ending", timeout, timeout(25), fun() -> - BeginningClient = rt:pbc(State#simple_state.beginning), + BeginningClient = rt_pb:pbc(State#simple_state.beginning), Bin = <<"cascading realtime">>, Obj = riakc_obj:new(<<"objects">>, Bin, Bin), riakc_pb_socket:put(BeginningClient, Obj, [{w,1}]), @@ -110,7 +110,7 @@ simple_test_() -> rpc:call(State#simple_state.middle, riak_repl_console, realtime_cascades, [["never"]]), Bin = <<"disabled cascading">>, Obj = riakc_obj:new(?bucket, Bin, Bin), - Client = rt:pbc(State#simple_state.beginning), + Client = rt_pb:pbc(State#simple_state.beginning), riakc_pb_socket:put(Client, Obj, [{w,1}]), riakc_pb_socket:stop(Client), ?assertEqual(Bin, maybe_eventually_exists(State#simple_state.middle, ?bucket, Bin)), @@ -122,7 +122,7 @@ simple_test_() -> rpc:call(State#simple_state.middle, riak_repl_console, realtime_cascades, [["always"]]), Bin = <<"cascading re-enabled">>, Obj = riakc_obj:new(?bucket, Bin, Bin), - Client = rt:pbc(State#simple_state.beginning), + Client = rt_pb:pbc(State#simple_state.beginning), riakc_pb_socket:put(Client, Obj, [{w,1}]), riakc_pb_socket:stop(Client), ?assertEqual(Bin, maybe_eventually_exists(State#simple_state.middle, ?bucket, Bin)), @@ -187,7 +187,7 @@ big_circle_test_() -> {"circle it", timeout, timeout(65), fun() -> [One | _] = Nodes, - C = rt:pbc(One), + C = rt_pb:pbc(One), Bin = <<"goober">>, Bucket = <<"objects">>, Obj = riakc_obj:new(Bucket, Bin, Bin), @@ -208,7 +208,7 @@ big_circle_test_() -> connect_rt(Node, Port, ConnectToName) end, lists:map(Connect, lists:zip(Nodes, ConnectTo)), - C = rt:pbc(hd(Nodes)), + C = rt_pb:pbc(hd(Nodes)), Bin = <<"2 way repl">>, Bucket = <<"objects">>, Obj = riakc_obj:new(Bucket, Bin, Bin), @@ -285,7 +285,7 @@ circle_test_() -> fun(Nodes) -> [ {"cascade all the way to the other end, but no further", timeout, timeout(12), fun() -> - Client = rt:pbc(hd(Nodes)), + Client = rt_pb:pbc(hd(Nodes)), Bin = <<"cascading">>, Obj = riakc_obj:new(<<"objects">>, Bin, Bin), riakc_pb_socket:put(Client, Obj, [{w,1}]), @@ -301,7 +301,7 @@ circle_test_() -> {"cascade starting at a different point", timeout, timeout(12), fun() -> [One, Two | _] = Nodes, - Client = rt:pbc(Two), + Client = rt_pb:pbc(Two), Bin = <<"start_at_two">>, Obj = riakc_obj:new(<<"objects">>, Bin, Bin), riakc_pb_socket:put(Client, Obj, [{w,1}]), @@ -355,7 +355,7 @@ pyramid_test_() -> {"Cascade to both kids", timeout, timeout(65), fun() -> [Top | _] = Nodes, - Client = rt:pbc(Top), + Client = rt_pb:pbc(Top), Bucket = <<"objects">>, Bin = <<"pyramid_top">>, Obj = riakc_obj:new(Bucket, Bin, Bin), @@ -409,7 +409,7 @@ diamond_test_() -> {"unfortunate double write", timeout, timeout(135), fun() -> [Top, MidLeft, MidRight, Bottom] = Nodes, - Client = rt:pbc(Top), + Client = rt_pb:pbc(Top), Bin = <<"start_at_top">>, Obj = riakc_obj:new(<<"objects">>, Bin, Bin), riakc_pb_socket:put(Client, Obj, [{w,1}]), @@ -444,7 +444,7 @@ diamond_test_() -> [Sink] = proplists:get_value(sinks, Status, [[]]), ExpectSeq = proplists:get_value(expect_seq, Sink), - Client = rt:pbc(MidRight), + Client = rt_pb:pbc(MidRight), Bin = <<"start at midright">>, Bucket = <<"objects">>, Obj = riakc_obj:new(Bucket, Bin, Bin), @@ -499,7 +499,7 @@ circle_and_spurs_test_() -> {"start at north", timeout, timeout(55), fun() -> [North | _Rest] = Nodes, - Client = rt:pbc(North), + Client = rt_pb:pbc(North), Bin = <<"start at north">>, Bucket = <<"objects">>, Obj = riakc_obj:new(Bucket, Bin, Bin), @@ -512,7 +512,7 @@ circle_and_spurs_test_() -> {"Start at west", timeout, timeout(55), fun() -> [_North, _East, West | _Rest] = Nodes, - Client = rt:pbc(West), + Client = rt_pb:pbc(West), Bin = <<"start at west">>, Bucket = <<"objects">>, Obj = riakc_obj:new(Bucket, Bin, Bin), @@ -525,7 +525,7 @@ circle_and_spurs_test_() -> {"spurs don't replicate back", timeout, timeout(55), fun() -> [_North, _East, _West, NorthSpur | _Rest] = Nodes, - Client = rt:pbc(NorthSpur), + Client = rt_pb:pbc(NorthSpur), Bin = <<"start at north_spur">>, Bucket = <<"objects">>, Obj = riakc_obj:new(Bucket, Bin, Bin), @@ -618,7 +618,7 @@ mixed_version_clusters_test_dep() -> {"no cascading at first", timeout, timeout(35), [ {timeout, timeout(15), fun() -> - Client = rt:pbc(N1), + Client = rt_pb:pbc(N1), Bin = <<"no cascade yet">>, Obj = riakc_obj:new(?bucket, Bin, Bin), riakc_pb_socket:put(Client, Obj, [{w, 2}]), @@ -628,7 +628,7 @@ mixed_version_clusters_test_dep() -> end}, {timeout, timeout(15), fun() -> - Client = rt:pbc(N2), + Client = rt_pb:pbc(N2), Bin = <<"no cascade yet 2">>, Obj = riakc_obj:new(?bucket, Bin, Bin), riakc_pb_socket:put(Client, Obj, [{w, 2}]), @@ -672,7 +672,7 @@ mixed_version_clusters_test_dep() -> fun(_) -> [ {"node1 put", timeout, timeout(205), fun() -> - Client = rt:pbc(N1), + Client = rt_pb:pbc(N1), Bin = <<"rt after upgrade">>, Obj = riakc_obj:new(?bucket, Bin, Bin), riakc_pb_socket:put(Client, Obj, [{w, 2}]), @@ -682,7 +682,7 @@ mixed_version_clusters_test_dep() -> end}, {"node2 put", timeout, timeout(25), fun() -> - Client = rt:pbc(N2), + Client = rt_pb:pbc(N2), Bin = <<"rt after upgrade 2">>, Obj = riakc_obj:new(?bucket, Bin, Bin), riakc_pb_socket:put(Client, Obj, [{w, 2}]), @@ -735,7 +735,7 @@ Reses)]), ExistsLookup = NewHead ++ NewTail, Test = fun() -> ?debugFmt("Running test ~p", [Name]), - Client = rt:pbc(Node), + Client = rt_pb:pbc(Node), Key = <<(ToB(Node))/binary, "-write-", (ToB(N))/binary>>, Obj = riakc_obj:new(?bucket, Key, Key), riakc_pb_socket:put(Client, Obj, [{w, 2}]), @@ -823,7 +823,7 @@ new_to_old_test_dep() -> ([New1, Old2, New3]) -> [ {"From new1 to old2", timeout, timeout(25), fun() -> - Client = rt:pbc(New1), + Client = rt_pb:pbc(New1), Bin = <<"new1 to old2">>, Obj = riakc_obj:new(?bucket, Bin, Bin), riakc_pb_socket:put(Client, Obj, [{w, 1}]), @@ -833,7 +833,7 @@ new_to_old_test_dep() -> end}, {"old2 does not cascade at all", timeout, timeout(25), fun() -> - Client = rt:pbc(New1), + Client = rt_pb:pbc(New1), Bin = <<"old2 no cascade">>, Obj = riakc_obj:new(?bucket, Bin, Bin), riakc_pb_socket:put(Client, Obj, [{w, 1}]), @@ -843,7 +843,7 @@ new_to_old_test_dep() -> end}, {"from new3 to old2", timeout, timeout(25), fun() -> - Client = rt:pbc(New3), + Client = rt_pb:pbc(New3), Bin = <<"new3 to old2">>, Obj = riakc_obj:new(?bucket, Bin, Bin), riakc_pb_socket:put(Client, Obj, [{w, 1}]), @@ -857,7 +857,7 @@ new_to_old_test_dep() -> % from an older source cluster/node. It is prevented for now by % having no easy/good way to get the name of the source cluster, % thus preventing complete information on the routed clusters. - Client = rt:pbc(Old2), + Client = rt_pb:pbc(Old2), Bin = <<"old2 to new3">>, Obj = riakc_obj:new(?bucket, Bin, Bin), riakc_pb_socket:put(Client, Obj, [{w,1}]), @@ -1132,7 +1132,7 @@ exists(Nodes, Bucket, Key) -> exists(Got, [], _Bucket, _Key) -> Got; exists({error, notfound}, [Node | Tail], Bucket, Key) -> - Pid = rt:pbc(Node), + Pid = rt_pb:pbc(Node), Got = riakc_pb_socket:get(Pid, Bucket, Key), riakc_pb_socket:stop(Pid), exists(Got, Tail, Bucket, Key); diff --git a/tests/secondary_index_tests.erl b/tests/secondary_index_tests.erl index 26827c6d5..7d66dcc52 100644 --- a/tests/secondary_index_tests.erl +++ b/tests/secondary_index_tests.erl @@ -49,7 +49,7 @@ config() -> confirm(#rt_properties{nodes=Nodes}, _MD) -> Bucket = druuid:v4_str(), lager:info("Bucket: ~p", [Bucket]), - PBC = rt:pbc(hd(Nodes)), + PBC = rt_pb:pbc(hd(Nodes)), HTTPC = rt:httpc(hd(Nodes)), Clients = [{pb, PBC}, {http, HTTPC}], @@ -70,7 +70,7 @@ confirm(#rt_properties{nodes=Nodes}, _MD) -> ToDel = [<<"obj05">>, <<"obj11">>], [?assertMatch(ok, riakc_pb_socket:delete(PBC, Bucket, KD)) || KD <- ToDel], lager:info("Make sure the tombstone is reaped..."), - ?assertMatch(ok, rt:wait_until(fun() -> rt:pbc_really_deleted(PBC, Bucket, ToDel) end)), + ?assertMatch(ok, rt:wait_until(fun() -> rt_pb:pbc_really_deleted(PBC, Bucket, ToDel) end)), assertExactQuery(Clients, Bucket, [], <<"field1_bin">>, <<"val5">>), assertExactQuery(Clients, Bucket, [], <<"field2_int">>, 5), diff --git a/tests/sibling_explosion.erl b/tests/sibling_explosion.erl index 68b8760a9..510742ab8 100644 --- a/tests/sibling_explosion.erl +++ b/tests/sibling_explosion.erl @@ -25,7 +25,7 @@ confirm() -> N = 100, lager:info("Put new object in ~p via PBC.", [Node1]), - PB = rt:pbc(Node1), + PB = rt_pb:pbc(Node1), A0 = riakc_obj:new(<<"b">>, <<"k">>, sets:from_list([0])), B0 = riakc_obj:new(<<"b">>, <<"k">>, sets:from_list([1])), diff --git a/tests/verify_2i_aae.erl b/tests/verify_2i_aae.erl index 39b7709ef..5ac070c36 100644 --- a/tests/verify_2i_aae.erl +++ b/tests/verify_2i_aae.erl @@ -44,7 +44,7 @@ confirm() -> {{diff_index_specs, 2}, skippable_diff_index_specs}]}), lager:info("Installed intercepts to corrupt index specs on node ~p", [Node1]), %%rpc:call(Node1, lager, set_loglevel, [lager_console_backend, debug]), - PBC = rt:pbc(Node1), + PBC = rt_pb:pbc(Node1), NumItems = ?NUM_ITEMS, NumDel = ?NUM_DELETES, pass = check_lost_objects(Node1, PBC, NumItems, NumDel), @@ -87,7 +87,7 @@ check_lost_objects(Node1, PBC, NumItems, NumDel) -> lager:info("Deleting ~b objects without updating indexes", [NumDel]), [del_obj(PBC, Bucket, N) || N <- DelRange, Bucket <- ?BUCKETS], DelKeys = [to_key(N) || N <- DelRange], - [rt:wait_until(fun() -> rt:pbc_really_deleted(PBC, Bucket, DelKeys) end) + [rt:wait_until(fun() -> rt_pb:pbc_really_deleted(PBC, Bucket, DelKeys) end) || Bucket <- ?BUCKETS], %% Verify they are damaged lager:info("Verify change did not take, needs repair"), diff --git a/tests/verify_2i_limit.erl b/tests/verify_2i_limit.erl index 66a7e7a9c..064feb227 100644 --- a/tests/verify_2i_limit.erl +++ b/tests/verify_2i_limit.erl @@ -36,7 +36,7 @@ confirm() -> RiakHttp = rt:httpc(hd(Nodes)), HttpUrl = rt:http_url(hd(Nodes)), - PBPid = rt:pbc(hd(Nodes)), + PBPid = rt_pb:pbc(hd(Nodes)), [put_an_object(PBPid, N) || N <- lists:seq(0, 100)], @@ -89,7 +89,7 @@ confirm() -> %% gh611 - equals query pagination riakc_pb_socket:delete(PBPid, ?BUCKET, <<"bob">>), - rt:wait_until(fun() -> rt:pbc_really_deleted(PBPid, ?BUCKET, [<<"bob">>]) end), + rt:wait_until(fun() -> rt_pb:pbc_really_deleted(PBPid, ?BUCKET, [<<"bob">>]) end), [put_an_object(PBPid, int_to_key(N), 1000, <<"myval">>) || N <- lists:seq(0, 100)], diff --git a/tests/verify_2i_mixed_cluster.erl b/tests/verify_2i_mixed_cluster.erl index 95c5d597e..c8b0d1d95 100644 --- a/tests/verify_2i_mixed_cluster.erl +++ b/tests/verify_2i_mixed_cluster.erl @@ -36,8 +36,8 @@ confirm() -> OldVsn, OldVsn]), ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), - PBC1 = rt:pbc(CurrentNode), - PBC2 = rt:pbc(OldNode1), + PBC1 = rt_pb:pbc(CurrentNode), + PBC2 = rt_pb:pbc(OldNode1), HTTPC1 = rt:httpc(CurrentNode), Clients = [{pb, PBC1}, {pb, PBC2}, {http, HTTPC1}], @@ -57,7 +57,7 @@ confirm() -> ToDel = [<<"obj05">>, <<"obj11">>], [?assertMatch(ok, riakc_pb_socket:delete(PBC1, ?BUCKET, KD)) || KD <- ToDel], lager:info("Make sure the tombstone is reaped..."), - ?assertMatch(ok, rt:wait_until(fun() -> rt:pbc_really_deleted(PBC1, ?BUCKET, ToDel) end)), + ?assertMatch(ok, rt:wait_until(fun() -> rt_pb:pbc_really_deleted(PBC1, ?BUCKET, ToDel) end)), assertExactQuery(Clients, [], <<"field1_bin">>, <<"val5">>), assertExactQuery(Clients, [], <<"field2_int">>, 5), diff --git a/tests/verify_2i_returnterms.erl b/tests/verify_2i_returnterms.erl index 2708a2de8..7548256e3 100644 --- a/tests/verify_2i_returnterms.erl +++ b/tests/verify_2i_returnterms.erl @@ -34,7 +34,7 @@ confirm() -> ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), RiakHttp = rt:http_url(hd(Nodes)), - PBPid = rt:pbc(hd(Nodes)), + PBPid = rt_pb:pbc(hd(Nodes)), [put_an_object(PBPid, N) || N <- lists:seq(0, 100)], [put_an_object(PBPid, int_to_key(N), N, ?FOO) || N <- lists:seq(101, 200)], diff --git a/tests/verify_2i_stream.erl b/tests/verify_2i_stream.erl index 638d68691..8440fc30a 100644 --- a/tests/verify_2i_stream.erl +++ b/tests/verify_2i_stream.erl @@ -33,7 +33,7 @@ confirm() -> ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), RiakHttp = rt:http_url(hd(Nodes)), - PBPid = rt:pbc(hd(Nodes)), + PBPid = rt_pb:pbc(hd(Nodes)), [put_an_object(PBPid, N) || N <- lists:seq(0, 100)], [put_an_object(PBPid, int_to_key(N), N, ?FOO) || N <- lists:seq(101, 200)], diff --git a/tests/verify_2i_timeout.erl b/tests/verify_2i_timeout.erl index e17c09caf..6f05fa955 100644 --- a/tests/verify_2i_timeout.erl +++ b/tests/verify_2i_timeout.erl @@ -32,7 +32,7 @@ confirm() -> Nodes = rt_cluster:build_cluster([{current, Config}, {current, Config}, {current, Config}]), ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), - PBPid = rt:pbc(hd(Nodes)), + PBPid = rt_pb:pbc(hd(Nodes)), Http = rt:http_url(hd(Nodes)), [put_an_object(PBPid, N) || N <- lists:seq(0, 100)], diff --git a/tests/verify_aae.erl b/tests/verify_aae.erl index 97f665d63..a40a42b8a 100644 --- a/tests/verify_aae.erl +++ b/tests/verify_aae.erl @@ -144,7 +144,7 @@ write_data(Node, KVs) -> write_data(Node, KVs, []). write_data(Node, KVs, Opts) -> - PB = rt:pbc(Node), + PB = rt_pb:pbc(Node), [begin O = case riakc_pb_socket:get(PB, ?BUCKET, K) of @@ -161,7 +161,7 @@ write_data(Node, KVs, Opts) -> % @doc Verifies that the data is eventually restored to the expected set. verify_data(Node, KeyValues) -> lager:info("Verify all replicas are eventually correct"), - PB = rt:pbc(Node), + PB = rt_pb:pbc(Node), CheckFun = fun() -> Matches = [verify_replicas(Node, ?BUCKET, K, V, ?N_VAL) diff --git a/tests/verify_api_timeouts.erl b/tests/verify_api_timeouts.erl index 5cd22e08f..6335ac0ee 100644 --- a/tests/verify_api_timeouts.erl +++ b/tests/verify_api_timeouts.erl @@ -72,7 +72,7 @@ confirm() -> end, - PC = rt:pbc(Node), + PC = rt_pb:pbc(Node), lager:info("testing PBC API"), @@ -126,7 +126,7 @@ confirm() -> lager:info("Checking List timeouts"), lager:info("Checking PBC"), - Pid = rt:pbc(Node), + Pid = rt_pb:pbc(Node), lager:info("Checking keys timeout"), ?assertMatch({error, <<"timeout">>}, riakc_pb_socket:list_keys(Pid, ?BUCKET, Short)), @@ -228,7 +228,7 @@ wait_for_end(ReqId) -> put_buckets(Node, Num) -> - Pid = rt:pbc(Node), + Pid = rt_pb:pbc(Node), Buckets = [list_to_binary(["", integer_to_list(Ki)]) || Ki <- lists:seq(0, Num - 1)], {Key, Val} = {<<"test_key">>, <<"test_value">>}, @@ -238,7 +238,7 @@ put_buckets(Node, Num) -> put_keys(Node, Bucket, Num) -> - Pid = rt:pbc(Node), + Pid = rt_pb:pbc(Node), Keys = [list_to_binary(["", integer_to_list(Ki)]) || Ki <- lists:seq(0, Num - 1)], Vals = [list_to_binary(["", integer_to_list(Ki)]) || Ki <- lists:seq(0, Num - 1)], [riakc_pb_socket:put(Pid, riakc_obj:new(Bucket, Key, Val)) || {Key, Val} <- lists:zip(Keys, Vals)], diff --git a/tests/verify_asis_put.erl b/tests/verify_asis_put.erl index f89bd8517..16bbc45fb 100644 --- a/tests/verify_asis_put.erl +++ b/tests/verify_asis_put.erl @@ -7,8 +7,8 @@ confirm() -> [Node1, Node2] = rt_cluster:deploy_nodes(2), %% 2. With PBC lager:info("Put new object in ~p via PBC.", [Node1]), - PB1 = rt:pbc(Node1), - PB2 = rt:pbc(Node2), + PB1 = rt_pb:pbc(Node1), + PB2 = rt_pb:pbc(Node2), Obj1 = riakc_obj:new(<<"verify_asis_put">>, <<"1">>, <<"test">>, "text/plain"), %% a. put in node 1 %% b. fetch from node 1 for vclock diff --git a/tests/verify_backup_restore.erl b/tests/verify_backup_restore.erl index aec9a8bb3..5a94e17ff 100644 --- a/tests/verify_backup_restore.erl +++ b/tests/verify_backup_restore.erl @@ -42,7 +42,7 @@ confirm() -> [Node0 | _RestNodes] = Nodes = rt_cluster:build_cluster(?NUM_NODES, Config), rt:enable_search_hook(Node0, ?SEARCH_BUCKET), rt:wait_until_ring_converged(Nodes), - PbcPid = rt:pbc(Node0), + PbcPid = rt_pb:pbc(Node0), Searches = [ {<<"ZiaSun">>, 1}, @@ -57,7 +57,7 @@ confirm() -> AllTerms = lists:foldl(ConcatBin, <<"">>, Searches), lager:info("Indexing data for search from ~p", [SpamDir]), - rt:pbc_put_dir(PbcPid, ?SEARCH_BUCKET, SpamDir), + rt_pb:pbc_put_dir(PbcPid, ?SEARCH_BUCKET, SpamDir), ExtraKey = <<"Extra1">>, riakc_pb_socket:put(PbcPid, riakc_obj:new(?SEARCH_BUCKET, @@ -98,7 +98,7 @@ confirm() -> {last, ?NUM_MOD+?NUM_DEL}]), lager:info("Deleting extra search doc"), riakc_pb_socket:delete(PbcPid, ?SEARCH_BUCKET, ExtraKey), - rt:wait_until(fun() -> rt:pbc_really_deleted(PbcPid, + rt:wait_until(fun() -> rt_pb:pbc_really_deleted(PbcPid, ?SEARCH_BUCKET, [ExtraKey]) end), @@ -141,7 +141,7 @@ confirm() -> rt:enable_search_hook(Node0, ?SEARCH_BUCKET), rt:wait_until_ring_converged(Nodes), rt:wait_until_no_pending_changes(Nodes), - PbcPid2 = rt:pbc(Node0), + PbcPid2 = rt_pb:pbc(Node0), lager:info("Verify no data in cluster"), [?assertEqual([], read_some(Node, [{last, ?NUM_KEYS}, @@ -198,7 +198,7 @@ write_some(PBC, Props) -> end end, ?assertEqual([], lists:foldl(DelFun, [], Keys)), - rt:wait_until(fun() -> rt:pbc_really_deleted(PBC, Bucket, Keys1) end); + rt:wait_until(fun() -> rt_pb:pbc_really_deleted(PBC, Bucket, Keys1) end); _ -> ok end, @@ -283,7 +283,7 @@ delete_some(PBC, Props) -> end end, lists:foldl(F, [], Keys), - rt:wait_until(fun() -> rt:pbc_really_deleted(PBC, Bucket, Keys) end), + rt:wait_until(fun() -> rt_pb:pbc_really_deleted(PBC, Bucket, Keys) end), ok. verify_search_count(Pid, SearchQuery, Count) -> diff --git a/tests/verify_bitcask_tombstone2_upgrade.erl b/tests/verify_bitcask_tombstone2_upgrade.erl index b6ebcf695..5a999bcc4 100644 --- a/tests/verify_bitcask_tombstone2_upgrade.erl +++ b/tests/verify_bitcask_tombstone2_upgrade.erl @@ -36,7 +36,7 @@ verify_bitcask_tombstone2_upgrade(Nodes) -> lager:info("And that is that"). write_some_data([Node1 | _]) -> - rt:pbc_systest_write(Node1, 10000). + rt_pb:pbc_systest_write(Node1, 10000). list_bitcask_files(Nodes) -> [{Node, list_node_bitcask_files(Node)} || Node <- Nodes]. diff --git a/tests/verify_conditional_postcommit.erl b/tests/verify_conditional_postcommit.erl index cbf79b705..24112468f 100644 --- a/tests/verify_conditional_postcommit.erl +++ b/tests/verify_conditional_postcommit.erl @@ -39,7 +39,7 @@ confirm() -> Bucket1 = {<<"type1">>, <<"test">>}, Bucket2 = {<<"type2">>, <<"test">>}, Keys = [<> || N <- lists:seq(1,1000)], - PBC = rt:pbc(Node), + PBC = rt_pb:pbc(Node), lager:info("Writing keys as 'type1' and verifying hook is not triggered"), write_keys(Node, PBC, Bucket1, Keys, false), @@ -55,7 +55,7 @@ confirm() -> write_keys(Node, PBC, Bucket, Keys, ShouldHook) -> rpc:call(Node, application, set_env, [riak_kv, hook_count, 0]), - [ok = rt:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], + [ok = rt_pb:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], {ok, Count} = rpc:call(Node, application, get_env, [riak_kv, hook_count]), case ShouldHook of true -> diff --git a/tests/verify_corruption_filtering.erl b/tests/verify_corruption_filtering.erl index 7154786d4..df899458f 100644 --- a/tests/verify_corruption_filtering.erl +++ b/tests/verify_corruption_filtering.erl @@ -65,7 +65,7 @@ confirm() -> pass. get_put_mix(Node) -> - PB = rt:pbc(Node), + PB = rt_pb:pbc(Node), [begin Key = random:uniform(1000), case random:uniform(2) of @@ -90,7 +90,7 @@ get_put_mix(Node) -> || _ <- lists:seq(1, 2000)]. load_cluster(Node) -> - PB = rt:pbc(Node), + PB = rt_pb:pbc(Node), [riakc_pb_socket:put(PB, riakc_obj:new(<<"foo">>, <>, <>)) diff --git a/tests/verify_counter_capability.erl b/tests/verify_counter_capability.erl index 5f259c2dc..b9e696558 100644 --- a/tests/verify_counter_capability.erl +++ b/tests/verify_counter_capability.erl @@ -61,7 +61,7 @@ confirm() -> rt:upgrade(Legacy, previous), - PrevPB2 = rt:pbc(Legacy), + PrevPB2 = rt_pb:pbc(Legacy), ?assertEqual(ok, rt:wait_until_capability_contains(Previous, {riak_kv, crdt}, [pncounter,riak_dt_pncounter,riak_dt_orswot,riak_dt_map])), @@ -81,4 +81,4 @@ confirm() -> pass. get_clients(Node) -> - {rt:pbc(Node), rt:httpc(Node)}. + {rt_pb:pbc(Node), rt:httpc(Node)}. diff --git a/tests/verify_crdt_capability.erl b/tests/verify_crdt_capability.erl index 9b49b5a57..7c2b1ea73 100644 --- a/tests/verify_crdt_capability.erl +++ b/tests/verify_crdt_capability.erl @@ -98,4 +98,4 @@ gen_counter_op() -> riakc_counter:to_op(riakc_counter:increment(riakc_counter:new())). get_clients(Node) -> - {rt:pbc(Node), rt:httpc(Node)}. + {rt_pb:pbc(Node), rt:httpc(Node)}. diff --git a/tests/verify_cs_bucket.erl b/tests/verify_cs_bucket.erl index 3ba21c902..4b517b043 100644 --- a/tests/verify_cs_bucket.erl +++ b/tests/verify_cs_bucket.erl @@ -32,7 +32,7 @@ confirm() -> Nodes = rt_cluster:build_cluster(3), ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), - PBPid = rt:pbc(hd(Nodes)), + PBPid = rt_pb:pbc(hd(Nodes)), [put_an_object(PBPid, N) || N <- lists:seq(0, 200)], diff --git a/tests/verify_dt_context.erl b/tests/verify_dt_context.erl index a9e44d12f..af81a6c67 100644 --- a/tests/verify_dt_context.erl +++ b/tests/verify_dt_context.erl @@ -192,7 +192,7 @@ store_map(Client, Map) -> create_pb_clients(Nodes) -> [begin - C = rt:pbc(N), + C = rt_pb:pbc(N), riakc_pb_socket:set_options(C, [queue_if_disconnected]), C end || N <- Nodes]. diff --git a/tests/verify_dt_converge.erl b/tests/verify_dt_converge.erl index 72e045cce..7a9053f96 100644 --- a/tests/verify_dt_converge.erl +++ b/tests/verify_dt_converge.erl @@ -124,7 +124,7 @@ confirm() -> create_pb_clients(Nodes) -> [begin - C = rt:pbc(N), + C = rt_pb:pbc(N), riakc_pb_socket:set_options(C, [queue_if_disconnected]), C end || N <- Nodes]. diff --git a/tests/verify_dt_upgrade.erl b/tests/verify_dt_upgrade.erl index 47cd6c1ee..5aab535fc 100644 --- a/tests/verify_dt_upgrade.erl +++ b/tests/verify_dt_upgrade.erl @@ -56,7 +56,7 @@ populate_counters(Node) -> ?assertMatch(ok, rhc:counter_incr(RHC, ?COUNTER_BUCKET, <<"httpkey">>, 2)), ?assertMatch({ok, 2}, rhc:counter_val(RHC, ?COUNTER_BUCKET, <<"httpkey">>)), - PBC = rt:pbc(Node), + PBC = rt_pb:pbc(Node), ?assertEqual(ok, riakc_pb_socket:counter_incr(PBC, ?COUNTER_BUCKET, <<"pbkey">>, 4)), ?assertEqual({ok, 4}, riakc_pb_socket:counter_val(PBC, ?COUNTER_BUCKET, <<"pbkey">>)), ok. @@ -68,7 +68,7 @@ verify_counters(Node) -> RHC = rt:httpc(Node), ?assertMatch({ok, 4}, rhc:counter_val(RHC, ?COUNTER_BUCKET, <<"pbkey">>)), - PBC = rt:pbc(Node), + PBC = rt_pb:pbc(Node), ?assertEqual({ok, 2}, riakc_pb_socket:counter_val(PBC, ?COUNTER_BUCKET, <<"httpkey">>)), %% Check that 1.4 counters work with bucket types diff --git a/tests/verify_dvv_repl.erl b/tests/verify_dvv_repl.erl index 4d0ce03f7..06c7edd12 100644 --- a/tests/verify_dvv_repl.erl +++ b/tests/verify_dvv_repl.erl @@ -110,7 +110,7 @@ make_cluster(Nodes, Name) -> repl_util:make_cluster(Nodes), repl_util:name_cluster(hd(Nodes), Name), repl_util:wait_until_leader_converge(Nodes), - C = rt:pbc(hd(Nodes)), + C = rt_pb:pbc(hd(Nodes)), riakc_pb_socket:set_options(C, [queue_if_disconnected]), {C, Nodes}. diff --git a/tests/verify_handoff_mixed.erl b/tests/verify_handoff_mixed.erl index 2bd5b290b..de91f15eb 100644 --- a/tests/verify_handoff_mixed.erl +++ b/tests/verify_handoff_mixed.erl @@ -101,7 +101,7 @@ prepare_vnodes(Node) -> prepare_kv_vnodes(Node) -> lager:info("Preparing KV vnodes with keys 1-~b in bucket ~s", [?KV_COUNT, ?KV_BUCKET]), - C = rt:pbc(Node), + C = rt_pb:pbc(Node), lists:foreach( fun(KV) -> ok = riakc_pb_socket:put(C, riakc_obj:new(?KV_BUCKET, KV, KV)) @@ -113,7 +113,7 @@ prepare_search_vnodes(Node) -> lager:info("Peparing Search vnodes with keys 1000-~b in bucket ~s", [1000+?SEARCH_COUNT, ?SEARCH_BUCKET]), rt:enable_search_hook(Node, ?SEARCH_BUCKET), - C = rt:pbc(Node), + C = rt_pb:pbc(Node), lists:foreach( fun(KV) -> O = riakc_obj:new(?SEARCH_BUCKET, KV, KV, "text/plain"), diff --git a/tests/verify_kv_health_check.erl b/tests/verify_kv_health_check.erl index abde0d7aa..861157bfd 100644 --- a/tests/verify_kv_health_check.erl +++ b/tests/verify_kv_health_check.erl @@ -43,7 +43,7 @@ confirm() -> %% make DisableThreshold+5 requests and trigger the health check explicitly %% we only need to backup one vnode's msg queue on the node to fail the health check %% so we read the same key again and again - C = rt:pbc(Node2), + C = rt_pb:pbc(Node2), [riakc_pb_socket:get(C, <<"b">>, <<"k">>) || _ <- lists:seq(1,DisableThreshold+5)], ok = rpc:call(Node1, riak_core_node_watcher, check_health, [riak_kv]), diff --git a/tests/verify_link_walk_urls.erl b/tests/verify_link_walk_urls.erl index ba174b875..1606f1324 100644 --- a/tests/verify_link_walk_urls.erl +++ b/tests/verify_link_walk_urls.erl @@ -34,7 +34,7 @@ confirm() -> [Node0 | _] = rt_cluster:build_cluster(?NUM_NODES), - Pbc = rt:pbc(Node0), + Pbc = rt_pb:pbc(Node0), lager:info("Inserting linked graph"), %% (deleted) (b/4,v4b) <-> (b/5,v5b) diff --git a/tests/verify_listkeys.erl b/tests/verify_listkeys.erl index 087d1b8dd..ab7fd56bc 100644 --- a/tests/verify_listkeys.erl +++ b/tests/verify_listkeys.erl @@ -91,7 +91,7 @@ confirm() -> pass. put_keys(Node, Bucket, Num) -> - Pid = rt:pbc(Node), + Pid = rt_pb:pbc(Node), Keys = [list_to_binary(["", integer_to_list(Ki)]) || Ki <- lists:seq(0, Num - 1)], Vals = [list_to_binary(["", integer_to_list(Ki)]) || Ki <- lists:seq(0, Num - 1)], [riakc_pb_socket:put(Pid, riakc_obj:new(Bucket, Key, Val)) || {Key, Val} <- lists:zip(Keys, Vals)], @@ -100,7 +100,7 @@ put_keys(Node, Bucket, Num) -> list_keys(Node, Interface, Bucket, Attempt, Num, ShouldPass) -> case Interface of pbc -> - Pid = rt:pbc(Node), + Pid = rt_pb:pbc(Node), Mod = riakc_pb_socket; http -> Pid = rt:httpc(Node), @@ -128,7 +128,7 @@ list_keys(Node, Interface, Bucket, Attempt, Num, ShouldPass) -> list_keys_for_undefined_bucket_type(Node, Interface, Bucket, Attempt, ShouldPass) -> case Interface of pbc -> - Pid = rt:pbc(Node), + Pid = rt_pb:pbc(Node), Mod = riakc_pb_socket; http -> Pid = rt:httpc(Node), @@ -151,7 +151,7 @@ list_keys_for_undefined_bucket_type(Node, Interface, Bucket, Attempt, ShouldPass end. put_buckets(Node, Num) -> - Pid = rt:pbc(Node), + Pid = rt_pb:pbc(Node), Buckets = [list_to_binary(["", integer_to_list(Ki)]) || Ki <- lists:seq(0, Num - 1)], {Key, Val} = {<<"test_key">>, <<"test_value">>}, @@ -162,7 +162,7 @@ put_buckets(Node, Num) -> list_buckets(Node, Interface, Attempt, Num, ShouldPass) -> case Interface of pbc -> - Pid = rt:pbc(Node), + Pid = rt_pb:pbc(Node), Mod = riakc_pb_socket; http -> Pid = rt:httpc(Node), @@ -196,7 +196,7 @@ list_buckets(Node, Interface, Attempt, Num, ShouldPass) -> list_buckets_for_undefined_bucket_type(Node, Interface, Attempt, ShouldPass) -> case Interface of pbc -> - Pid = rt:pbc(Node), + Pid = rt_pb:pbc(Node), Mod = riakc_pb_socket; http -> Pid = rt:httpc(Node), diff --git a/tests/verify_listkeys_eqcfsm.erl b/tests/verify_listkeys_eqcfsm.erl index bb751fd6b..f42206da4 100644 --- a/tests/verify_listkeys_eqcfsm.erl +++ b/tests/verify_listkeys_eqcfsm.erl @@ -229,7 +229,7 @@ node_list(NumNodes) -> put_keys(Node, Bucket, Num) -> lager:info("*******************[CMD] Putting ~p keys into bucket ~p on node ~p", [Num, Bucket, Node]), - Pid = rt:pbc(Node), + Pid = rt_pb:pbc(Node), try Keys = [list_to_binary(["", integer_to_list(Ki)]) || Ki <- lists:seq(0, Num - 1)], Vals = [list_to_binary(["", integer_to_list(Ki)]) || Ki <- lists:seq(0, Num - 1)], diff --git a/tests/verify_mr_prereduce_node_down.erl b/tests/verify_mr_prereduce_node_down.erl index e6bc2d914..62dd2f496 100644 --- a/tests/verify_mr_prereduce_node_down.erl +++ b/tests/verify_mr_prereduce_node_down.erl @@ -56,7 +56,7 @@ confirm() -> [] = rt:systest_write(Primary, 1, ObjCount, Bucket, 3), %% run the query a bunch - C = rt:pbc(Primary), + C = rt_pb:pbc(Primary), TestCount = 100, lager:info("Running the MR query ~b times", [TestCount]), Runs = [ run_query(C, Bucket) || _ <- lists:seq(1, TestCount) ], diff --git a/tests/verify_no_writes_on_read.erl b/tests/verify_no_writes_on_read.erl index 5b48ed048..65902cf54 100644 --- a/tests/verify_no_writes_on_read.erl +++ b/tests/verify_no_writes_on_read.erl @@ -12,7 +12,7 @@ confirm() -> lager:info("Running with backend ~p", [Backend]), ?assertEqual(bitcask, Backend), [Node1 | _Rest] = _Nodes = rt_cluster:build_cluster(?NUM_NODES), - PBC = rt:pbc(Node1), + PBC = rt_pb:pbc(Node1), lager:info("Setting last write wins on bucket"), B = ?BUCKET, ?assertMatch(ok, rpc:call(Node1, riak_core_bucket, set_bucket, [B, [{last_write_wins, true}]])), diff --git a/tests/verify_object_limits.erl b/tests/verify_object_limits.erl index a989eed1d..674ab8b96 100644 --- a/tests/verify_object_limits.erl +++ b/tests/verify_object_limits.erl @@ -40,7 +40,7 @@ confirm() -> {warn_object_size, ?WARN_SIZE}, {max_siblings, ?MAX_SIBLINGS}, {warn_siblings, ?WARN_SIBLINGS}]}]), - C = rt:pbc(Node1), + C = rt_pb:pbc(Node1), %% Set up to grep logs to verify messages rt:setup_log_capture(Node1), diff --git a/tests/verify_reset_bucket_props.erl b/tests/verify_reset_bucket_props.erl index c3740bc94..175ebcbdc 100644 --- a/tests/verify_reset_bucket_props.erl +++ b/tests/verify_reset_bucket_props.erl @@ -43,7 +43,7 @@ confirm() -> update_props(DefaultProps, Node1, Nodes), - C = rt:pbc(Node3), + C = rt_pb:pbc(Node3), lager:info("Resetting bucket properties for bucket ~p on node ~p via pbc", [?BUCKET, Node3]), ok = riakc_pb_socket:reset_bucket(C, ?BUCKET), diff --git a/tests/verify_riak_stats.erl b/tests/verify_riak_stats.erl index 971c7fbf7..80135090b 100644 --- a/tests/verify_riak_stats.erl +++ b/tests/verify_riak_stats.erl @@ -103,8 +103,7 @@ confirm() -> lager:info("Make PBC Connection"), - Pid = rt:pbc(Node1), - + Pid = rt_pb:pbc(Node1), Stats3 = get_stats(Node1), rt:systest_write(Node1, 1), @@ -116,14 +115,14 @@ confirm() -> lager:info("Force Read Repair"), - rt:pbc_write(Pid, <<"testbucket">>, <<"1">>, <<"blah!">>), + rt_pb:pbc_write(Pid, <<"testbucket">>, <<"1">>, <<"blah!">>), rt:pbc_set_bucket_prop(Pid, <<"testbucket">>, [{n_val, 4}]), Stats4 = get_stats(Node1), verify_inc(Stats3, Stats4, [{<<"read_repairs_total">>, 0}, {<<"read_repairs">>, 0}]), - _Value = rt:pbc_read(Pid, <<"testbucket">>, <<"1">>), + _Value = rt_pb:pbc_read(Pid, <<"testbucket">>, <<"1">>), Stats5 = get_stats(Node1), diff --git a/tests/verify_secondary_index_reformat.erl b/tests/verify_secondary_index_reformat.erl index a1455deb1..9235941c3 100644 --- a/tests/verify_secondary_index_reformat.erl +++ b/tests/verify_secondary_index_reformat.erl @@ -37,7 +37,7 @@ confirm() -> %% write key with index that old version of sext would encode improperly (not perserving %% sort order) lager:info("writing test key"), - Client0 = rt:pbc(Node), + Client0 = rt_pb:pbc(Node), Obj0 = riakc_obj:new(TestBucket, TestKey, <<"somevalue">>), ObjMD0 = riakc_obj:get_update_metadata(Obj0), ObjMD1 = riakc_obj:set_secondary_index(ObjMD0, @@ -57,7 +57,7 @@ confirm() -> %% should rewrite 1 index (* n = 3), ignore 0 and have zero errors {3, 0, 0} = rpc:call(Node, riak_kv_util, fix_incorrect_index_entries, []), - Client1 = rt:pbc(Node), + Client1 = rt_pb:pbc(Node), Results = riakc_pb_socket:get_index(Client1, TestBucket, TestIndex, 1000000000000, TestIdxValue), diff --git a/tests/verify_vclock.erl b/tests/verify_vclock.erl index 5a5cbaa50..cf7c89268 100644 --- a/tests/verify_vclock.erl +++ b/tests/verify_vclock.erl @@ -134,10 +134,10 @@ our_pbc_write(Node, Size, Suffix) -> our_pbc_write(Node, 1, Size, <<"systest">>, Suffix). our_pbc_write(Node, Start, End, Bucket, VSuffix) -> - PBC = rt:pbc(Node), + PBC = rt_pb:pbc(Node), F = fun(N, Acc) -> {K, V} = make_kv(N, VSuffix), - try rt:pbc_write(PBC, Bucket, K, V) of + try rt_pb:pbc_write(PBC, Bucket, K, V) of ok -> Acc; Other -> @@ -156,7 +156,7 @@ our_pbc_read(Node, Size, Suffix) -> our_pbc_read(Node, 1, Size, <<"systest">>, Suffix). our_pbc_read(Node, Start, End, Bucket, VSuffix) -> - PBC = rt:pbc(Node), + PBC = rt_pb:pbc(Node), %% Trundle along through the list, collecting mismatches: F = fun(N, Acc) -> diff --git a/tests/verify_vclock_encoding_upgrade.erl b/tests/verify_vclock_encoding_upgrade.erl index 110951099..cd482ee76 100644 --- a/tests/verify_vclock_encoding_upgrade.erl +++ b/tests/verify_vclock_encoding_upgrade.erl @@ -24,8 +24,8 @@ confirm() -> lager:info("Deploying previous cluster"), [Prev, Current] = rt_cluster:build_cluster([previous, current]), - PrevClient = rt:pbc(Prev), - CurrentClient = rt:pbc(Current), + PrevClient = rt_pb:pbc(Prev), + CurrentClient = rt_pb:pbc(Current), K = <<"key">>, B = <<"bucket">>, V = <<"value">>, diff --git a/tests/yz_ensemble.erl b/tests/yz_ensemble.erl index f88136186..c35128d6a 100644 --- a/tests/yz_ensemble.erl +++ b/tests/yz_ensemble.erl @@ -46,10 +46,10 @@ verify_ensemble_delete_support(Node, Bucket, Index) -> Keys = [<> || N <- lists:seq(1,2000), not lists:any(fun(E) -> E > 127 end,binary_to_list(<>))], - PBC = rt:pbc(Node), + PBC = rt_pb:pbc(Node), lager:info("Writing ~p keys", [length(Keys)]), - [ok = rt:pbc_write(PBC, Bucket, Key, Key, "text/plain") || Key <- Keys], + [ok = rt_pb:pbc_write(PBC, Bucket, Key, Key, "text/plain") || Key <- Keys], %% soft commit wait, then check that last key is indexed lager:info("Search for keys to verify they exist"), From 8c9f6f115a0def6006adbf098ae540b8736641a2 Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Wed, 30 Jul 2014 14:09:50 -0400 Subject: [PATCH 015/157] Fix rt_pb function exports. --- src/rt.erl | 30 ++++++++---------------------- src/rt_pb.erl | 26 ++++++++++++++++++++++++-- 2 files changed, 32 insertions(+), 24 deletions(-) diff --git a/src/rt.erl b/src/rt.erl index b2e3f4d9b..f4b37726f 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -214,31 +214,17 @@ end. -spec connection_info(node() | [node()]) -> interfaces() | conn_info(). connection_info(Node) when is_atom(Node) -> -{ok, [{PB_IP, PB_Port}]} = get_pb_conn_info(Node), -{ok, [{HTTP_IP, HTTP_Port}]} = get_http_conn_info(Node), -case get_https_conn_info(Node) of -undefined -> - [{http, {HTTP_IP, HTTP_Port}}, {pb, {PB_IP, PB_Port}}]; -{ok, [{HTTPS_IP, HTTPS_Port}]} -> - [{http, {HTTP_IP, HTTP_Port}}, {https, {HTTPS_IP, HTTPS_Port}}, {pb, {PB_IP, PB_Port}}] -end; + {ok, [{PB_IP, PB_Port}]} = rt_pb:get_pb_conn_info(Node), + {ok, [{HTTP_IP, HTTP_Port}]} = get_http_conn_info(Node), + case get_https_conn_info(Node) of + undefined -> + [{http, {HTTP_IP, HTTP_Port}}, {pb, {PB_IP, PB_Port}}]; + {ok, [{HTTPS_IP, HTTPS_Port}]} -> + [{http, {HTTP_IP, HTTP_Port}}, {https, {HTTPS_IP, HTTPS_Port}}, {pb, {PB_IP, PB_Port}}] + end; connection_info(Nodes) when is_list(Nodes) -> [ {Node, connection_info(Node)} || Node <- Nodes]. --spec get_pb_conn_info(node()) -> [{inet:ip_address(), pos_integer()}]. -get_pb_conn_info(Node) -> -case rpc_get_env(Node, [{riak_api, pb}, - {riak_api, pb_ip}, - {riak_kv, pb_ip}]) of -{ok, [{NewIP, NewPort}|_]} -> - {ok, [{NewIP, NewPort}]}; -{ok, PB_IP} -> - {ok, PB_Port} = rpc_get_env(Node, [{riak_api, pb_port}, - {riak_kv, pb_port}]), - {ok, [{PB_IP, PB_Port}]}; -_ -> - undefined -end. -spec get_http_conn_info(node()) -> [{inet:ip_address(), pos_integer()}]. get_http_conn_info(Node) -> diff --git a/src/rt_pb.erl b/src/rt_pb.erl index 959f7a815..809e8b717 100644 --- a/src/rt_pb.erl +++ b/src/rt_pb.erl @@ -16,7 +16,6 @@ -include("rt.hrl"). -include_lib("eunit/include/eunit.hrl"). --compile(export_all). -export([pbc/1, pbc_read/3, pbc_read/4, @@ -24,9 +23,17 @@ pbc_read_check/5, pbc_set_bucket_prop/3, pbc_write/4, + pbc_write/5, pbc_put_dir/3, pbc_put_file/4, - pbc_really_deleted/3]). + pbc_really_deleted/3, + pbc_systest_write/2, + pbc_systest_write/3, + pbc_systest_write/5, + pbc_systest_read/2, + pbc_systest_read/3, + pbc_systest_read/5, + get_pb_conn_info/1]). -define(HARNESS, (rt_config:get(rt_harness))). @@ -162,3 +169,18 @@ pbc_systest_read(Node, Start, End, Bucket, R) -> end end, lists:foldl(F, [], lists:seq(Start, End)). + +-spec get_pb_conn_info(node()) -> [{inet:ip_address(), pos_integer()}]. +get_pb_conn_info(Node) -> + case rt:rpc_get_env(Node, [{riak_api, pb}, + {riak_api, pb_ip}, + {riak_kv, pb_ip}]) of + {ok, [{NewIP, NewPort}|_]} -> + {ok, [{NewIP, NewPort}]}; + {ok, PB_IP} -> + {ok, PB_Port} = rt:rpc_get_env(Node, [{riak_api, pb_port}, + {riak_kv, pb_port}]), + {ok, [{PB_IP, PB_Port}]}; + _ -> + undefined + end. From defdf55d1c9134ac56b49fd45cd8425056da380e Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Wed, 30 Jul 2014 14:36:03 -0400 Subject: [PATCH 016/157] Refactor http-relate rt functions - Move rt module functions to new rt_http module. - Convert http_bucket_types to new test convention. --- src/rt.erl | 65 +---------------------- src/rt_http.erl | 82 +++++++++++++++++++++++++++++ src/rt_pb.erl | 5 ++ tests/bucket_props_roundtrip.erl | 2 +- tests/http_bucket_types.erl | 27 ++++++---- tests/jmx_verify.erl | 6 +-- tests/mapred_search_switch.erl | 4 +- tests/replication/replication2.erl | 2 +- tests/secondary_index_tests.erl | 2 +- tests/verify_2i_limit.erl | 2 +- tests/verify_2i_mixed_cluster.erl | 2 +- tests/verify_api_timeouts.erl | 8 +-- tests/verify_asis_put.erl | 4 +- tests/verify_commit_hooks.erl | 18 +++---- tests/verify_counter_capability.erl | 2 +- tests/verify_counter_converge.erl | 2 +- tests/verify_counter_repl.erl | 2 +- tests/verify_crdt_capability.erl | 2 +- tests/verify_dt_converge.erl | 2 +- tests/verify_dt_upgrade.erl | 4 +- tests/verify_listkeys.erl | 8 +-- tests/verify_riak_stats.erl | 11 ++-- 22 files changed, 146 insertions(+), 116 deletions(-) create mode 100644 src/rt_http.erl diff --git a/src/rt.erl b/src/rt.erl index f4b37726f..59dbe7c0f 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -60,11 +60,6 @@ get_ring/1, get_version/0, heal/1, - http_url/1, - https_url/1, - httpc/1, - httpc_read/3, - httpc_write/4, is_mixed_cluster/1, is_pingable/1, join/2, @@ -215,8 +210,8 @@ end. -spec connection_info(node() | [node()]) -> interfaces() | conn_info(). connection_info(Node) when is_atom(Node) -> {ok, [{PB_IP, PB_Port}]} = rt_pb:get_pb_conn_info(Node), - {ok, [{HTTP_IP, HTTP_Port}]} = get_http_conn_info(Node), - case get_https_conn_info(Node) of + {ok, [{HTTP_IP, HTTP_Port}]} = rt_http:get_http_conn_info(Node), + case rt_http:get_https_conn_info(Node) of undefined -> [{http, {HTTP_IP, HTTP_Port}}, {pb, {PB_IP, PB_Port}}]; {ok, [{HTTPS_IP, HTTPS_Port}]} -> @@ -226,26 +221,6 @@ connection_info(Nodes) when is_list(Nodes) -> [ {Node, connection_info(Node)} || Node <- Nodes]. --spec get_http_conn_info(node()) -> [{inet:ip_address(), pos_integer()}]. -get_http_conn_info(Node) -> -case rpc_get_env(Node, [{riak_api, http}, - {riak_core, http}]) of -{ok, [{IP, Port}|_]} -> - {ok, [{IP, Port}]}; -_ -> - undefined -end. - --spec get_https_conn_info(node()) -> [{inet:ip_address(), pos_integer()}]. -get_https_conn_info(Node) -> -case rpc_get_env(Node, [{riak_api, https}, - {riak_core, https}]) of -{ok, [{IP, Port}|_]} -> - {ok, [{IP, Port}]}; -_ -> - undefined -end. - %% @doc Start the specified Riak node start(Node) -> ?HARNESS:start(Node). @@ -1250,42 +1225,6 @@ end. %%% PBC & HTTPC Functions %%%=================================================================== -%% @doc Returns HTTPS URL information for a list of Nodes -https_url(Nodes) when is_list(Nodes) -> -[begin - {Host, Port} = orddict:fetch(https, Connections), - lists:flatten(io_lib:format("https://~s:~b", [Host, Port])) -end || {_Node, Connections} <- connection_info(Nodes)]; -https_url(Node) -> -hd(https_url([Node])). - -%% @doc Returns HTTP URL information for a list of Nodes -http_url(Nodes) when is_list(Nodes) -> -[begin - {Host, Port} = orddict:fetch(http, Connections), - lists:flatten(io_lib:format("http://~s:~b", [Host, Port])) -end || {_Node, Connections} <- connection_info(Nodes)]; -http_url(Node) -> -hd(http_url([Node])). - -%% @doc get me an http client. --spec httpc(node()) -> term(). -httpc(Node) -> -rt:wait_for_service(Node, riak_kv), -{ok, [{IP, Port}]} = get_http_conn_info(Node), -rhc:create(IP, Port, "riak", []). - -%% @doc does a read via the http erlang client. --spec httpc_read(term(), binary(), binary()) -> binary(). -httpc_read(C, Bucket, Key) -> -{_, Value} = rhc:get(C, Bucket, Key), -Value. - -%% @doc does a write via the http erlang client. --spec httpc_write(term(), binary(), binary(), binary()) -> atom(). -httpc_write(C, Bucket, Key, Value) -> -Object = riakc_obj:new(Bucket, Key, Value), -rhc:put(C, Object). %%%=================================================================== %%% Command Line Functions diff --git a/src/rt_http.erl b/src/rt_http.erl new file mode 100644 index 000000000..1941e38aa --- /dev/null +++ b/src/rt_http.erl @@ -0,0 +1,82 @@ +%% 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(rt_http). +-include("rt.hrl"). +-include_lib("eunit/include/eunit.hrl"). + +-export([http_url/1, + https_url/1, + httpc/1, + httpc_read/3, + httpc_write/4, + get_http_conn_info/1, + get_https_conn_info/1]). + +%% @doc Returns HTTPS URL information for a list of Nodes +https_url(Nodes) when is_list(Nodes) -> + [begin + {Host, Port} = orddict:fetch(https, Connections), + lists:flatten(io_lib:format("https://~s:~b", [Host, Port])) + end || {_Node, Connections} <- rt:connection_info(Nodes)]; +https_url(Node) -> + hd(https_url([Node])). + +%% @doc Returns HTTP URL information for a list of Nodes +http_url(Nodes) when is_list(Nodes) -> + [begin + {Host, Port} = orddict:fetch(http, Connections), + lists:flatten(io_lib:format("http://~s:~b", [Host, Port])) + end || {_Node, Connections} <- rt:connection_info(Nodes)]; +http_url(Node) -> + hd(http_url([Node])). + +%% @doc get me an http client. +-spec httpc(node()) -> term(). +httpc(Node) -> + rt:wait_for_service(Node, riak_kv), + {ok, [{IP, Port}]} = get_http_conn_info(Node), + rhc:create(IP, Port, "riak", []). + +%% @doc does a read via the http erlang client. +-spec httpc_read(term(), binary(), binary()) -> binary(). +httpc_read(C, Bucket, Key) -> + {_, Value} = rhc:get(C, Bucket, Key), + Value. + +%% @doc does a write via the http erlang client. +-spec httpc_write(term(), binary(), binary(), binary()) -> atom(). +httpc_write(C, Bucket, Key, Value) -> + Object = riakc_obj:new(Bucket, Key, Value), + rhc:put(C, Object). + +-spec get_http_conn_info(node()) -> [{inet:ip_address(), pos_integer()}]. +get_http_conn_info(Node) -> + case rt:rpc_get_env(Node, [{riak_api, http}, + {riak_core, http}]) of + {ok, [{IP, Port}|_]} -> + {ok, [{IP, Port}]}; + _ -> + undefined + end. + + +-spec get_https_conn_info(node()) -> [{inet:ip_address(), pos_integer()}]. +get_https_conn_info(Node) -> + case rt:rpc_get_env(Node, [{riak_api, https}, + {riak_core, https}]) of + {ok, [{IP, Port}|_]} -> + {ok, [{IP, Port}]}; + _ -> + undefined + end. diff --git a/src/rt_pb.erl b/src/rt_pb.erl index 809e8b717..157261758 100644 --- a/src/rt_pb.erl +++ b/src/rt_pb.erl @@ -1,3 +1,8 @@ +%% ------------------------------------------------------------------- +%% +%% Copyright (c) 2013-2014 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 diff --git a/tests/bucket_props_roundtrip.erl b/tests/bucket_props_roundtrip.erl index 7b06fd38b..90bd2fbd4 100644 --- a/tests/bucket_props_roundtrip.erl +++ b/tests/bucket_props_roundtrip.erl @@ -69,7 +69,7 @@ confirm(#rt_properties{nodes=Nodes}, _MD) -> check_prop_set_and_get(Node, Prop, One, Two) -> lager:info("-------- Testing roundtrip for property '~p' ---------", [Prop]), - HTTP = rt:httpc(Node), + HTTP = rt_http:httpc(Node), PBC = rt_pb:pbc(Node), lager:info("HTTP set = ~p", [One]), http_set_property(HTTP, Prop, One), diff --git a/tests/http_bucket_types.erl b/tests/http_bucket_types.erl index 283153506..25a466b35 100644 --- a/tests/http_bucket_types.erl +++ b/tests/http_bucket_types.erl @@ -1,20 +1,25 @@ -module(http_bucket_types). --behavior(riak_test). --export([confirm/0, mapred_modfun/3, mapred_modfun_type/3]). +-export([properties/0, confirm/2, mapred_modfun/3, mapred_modfun_type/3]). -include_lib("eunit/include/eunit.hrl"). -include_lib("riakc/include/riakc.hrl"). +-include("rt.hrl"). + +properties() -> + DefaultProps = rt_cluster:properties(), + CustomConfig = rt_cluster:augment_config(riak_core, + {default_bucket_props, [{n_val, 2}]}, + DefaultProps#rt_properties.config), + DefaultProps#rt_properties{node_count=1, + rolling_upgrade=false, + make_cluster=true, + config=CustomConfig}. + +confirm(#rt_properties{nodes=Nodes}, _MD) -> + Node = hd(Nodes), --define(WAIT(E), ?assertEqual(ok, rt:wait_until(fun() -> (E) end))). - -confirm() -> application:start(ibrowse), - lager:info("Deploy some nodes"), - Nodes = rt_cluster:build_cluster(4, [], [ - {riak_core, [{default_bucket_props, - [{n_val, 2}]}]}]), - Node = hd(Nodes), RMD = riak_test_runner:metadata(), HaveIndexes = case proplists:get_value(backend, RMD) of @@ -23,7 +28,7 @@ confirm() -> _ -> true end, - RHC = rt:httpc(Node), + RHC = rt_http:httpc(Node), lager:info("default type get/put test"), %% write explicitly to the default type ok = rhc:put(RHC, riakc_obj:new({<<"default">>, <<"bucket">>}, diff --git a/tests/jmx_verify.erl b/tests/jmx_verify.erl index 26e295aa8..14616388d 100644 --- a/tests/jmx_verify.erl +++ b/tests/jmx_verify.erl @@ -63,9 +63,9 @@ confirm() -> lager:info("perform 5 x PUT and a GET to increment the stats"), lager:info("as the stat system only does calcs for > 5 readings"), - C = rt:httpc(Node1), - [rt:httpc_write(C, <<"systest">>, <>, <<"12345">>) || X <- lists:seq(1, 5)], - [rt:httpc_read(C, <<"systest">>, <>) || X <- lists:seq(1, 5)], + C = rt_http:httpc(Node1), + [rt_http:httpc_write(C, <<"systest">>, <>, <<"12345">>) || X <- lists:seq(1, 5)], + [rt_http:httpc_read(C, <<"systest">>, <>) || X <- lists:seq(1, 5)], JMX2 = jmx_dump(JMXDumpCmd), %% make sure the stats that were supposed to increment did diff --git a/tests/mapred_search_switch.erl b/tests/mapred_search_switch.erl index e34951b76..adf8f0833 100644 --- a/tests/mapred_search_switch.erl +++ b/tests/mapred_search_switch.erl @@ -162,7 +162,7 @@ generate_test_data(System) -> %% setup riak_search hook setup_rs_bucket([Node|_], Bucket) -> lager:info("Setting up riak_search hook"), - C = rt:httpc(Node), + C = rt_http:httpc(Node), ok = rhc:set_bucket(C, Bucket, [{search, true}]). %% setup yokozuna hook/index - bucket name == index name @@ -205,7 +205,7 @@ iburl(Node, Path) -> %% value, and each of which has a unique term in its value load_test_data([Node|_], Bucket, KeyAndUniques, Common) -> lager:info("Loading test data"), - C = rt:httpc(Node), + C = rt_http:httpc(Node), [ begin Value = list_to_binary([Common, " ", Unique]), ok = rhc:put(C, riakc_obj:new(Bucket, Key, Value, "text/plain")) diff --git a/tests/replication/replication2.erl b/tests/replication/replication2.erl index cae4b2c81..b02af02cd 100644 --- a/tests/replication/replication2.erl +++ b/tests/replication/replication2.erl @@ -627,7 +627,7 @@ http_write_during_shutdown([AFirst|_] = ANodes, [BFirst|_] = BNodes) -> ConnInfo = proplists:get_value(Target, rt:connection_info([Target])), {IP, Port} = proplists:get_value(http, ConnInfo), lager:info("Connecting to http socket ~p:~p on ~p", [IP, Port, Target]), - C = rt:httpc(Target), + C = rt_http:httpc(Target), %% do the stop in the background while we're writing keys spawn(fun() -> diff --git a/tests/secondary_index_tests.erl b/tests/secondary_index_tests.erl index 7d66dcc52..4ad75f1c5 100644 --- a/tests/secondary_index_tests.erl +++ b/tests/secondary_index_tests.erl @@ -50,7 +50,7 @@ confirm(#rt_properties{nodes=Nodes}, _MD) -> Bucket = druuid:v4_str(), lager:info("Bucket: ~p", [Bucket]), PBC = rt_pb:pbc(hd(Nodes)), - HTTPC = rt:httpc(hd(Nodes)), + HTTPC = rt_http:httpc(hd(Nodes)), Clients = [{pb, PBC}, {http, HTTPC}], [put_an_object(PBC, Bucket, N) || N <- lists:seq(0, 20)], diff --git a/tests/verify_2i_limit.erl b/tests/verify_2i_limit.erl index 064feb227..6c2acdfbf 100644 --- a/tests/verify_2i_limit.erl +++ b/tests/verify_2i_limit.erl @@ -34,7 +34,7 @@ confirm() -> Nodes = rt_cluster:build_cluster(3), ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), - RiakHttp = rt:httpc(hd(Nodes)), + RiakHttp = rt_http:httpc(hd(Nodes)), HttpUrl = rt:http_url(hd(Nodes)), PBPid = rt_pb:pbc(hd(Nodes)), diff --git a/tests/verify_2i_mixed_cluster.erl b/tests/verify_2i_mixed_cluster.erl index c8b0d1d95..f0a7aa22a 100644 --- a/tests/verify_2i_mixed_cluster.erl +++ b/tests/verify_2i_mixed_cluster.erl @@ -38,7 +38,7 @@ confirm() -> PBC1 = rt_pb:pbc(CurrentNode), PBC2 = rt_pb:pbc(OldNode1), - HTTPC1 = rt:httpc(CurrentNode), + HTTPC1 = rt_http:httpc(CurrentNode), Clients = [{pb, PBC1}, {pb, PBC2}, {http, HTTPC1}], diff --git a/tests/verify_api_timeouts.erl b/tests/verify_api_timeouts.erl index 6335ac0ee..763e3f8bd 100644 --- a/tests/verify_api_timeouts.erl +++ b/tests/verify_api_timeouts.erl @@ -12,10 +12,10 @@ confirm() -> [Node] = rt_cluster:build_cluster(1), rt:wait_until_pingable(Node), - HC = rt:httpc(Node), + HC = rt_http:httpc(Node), lager:info("setting up initial data and loading remote code"), - rt:httpc_write(HC, <<"foo">>, <<"bar">>, <<"foobarbaz\n">>), - rt:httpc_write(HC, <<"foo">>, <<"bar2">>, <<"foobarbaz2\n">>), + rt_http:httpc_write(HC, <<"foo">>, <<"bar">>, <<"foobarbaz\n">>), + rt_http:httpc_write(HC, <<"foo">>, <<"bar2">>, <<"foobarbaz2\n">>), put_keys(Node, ?BUCKET, ?NUM_KEYS), put_buckets(Node, ?NUM_BUCKETS), @@ -155,7 +155,7 @@ confirm() -> lager:info("Checking HTTP"), - LHC = rt:httpc(Node), + LHC = rt_http:httpc(Node), lager:info("Checking keys timeout"), ?assertMatch({error, <<"timeout">>}, rhc:list_keys(LHC, ?BUCKET, Short)), diff --git a/tests/verify_asis_put.erl b/tests/verify_asis_put.erl index 16bbc45fb..7e5a71df5 100644 --- a/tests/verify_asis_put.erl +++ b/tests/verify_asis_put.erl @@ -23,8 +23,8 @@ confirm() -> %% 3. Repeat with HTTP, nodes reversed lager:info("Put new object in ~p via HTTP.", [Node2]), - HTTP1 = rt:httpc(Node1), - HTTP2 = rt:httpc(Node2), + HTTP1 = rt_http:httpc(Node1), + HTTP2 = rt_http:httpc(Node2), Obj2 = riakc_obj:new(<<"verify_asis_put">>, <<"2">>, <<"test">>, "text/plain"), %% a. put in node 2 %% b. fetch from node 2 for vclock diff --git a/tests/verify_commit_hooks.erl b/tests/verify_commit_hooks.erl index b2d57636e..8e754bb7d 100644 --- a/tests/verify_commit_hooks.erl +++ b/tests/verify_commit_hooks.erl @@ -34,36 +34,36 @@ confirm() -> ?assertEqual(ok, rpc:call(Node, hooks, set_hooks, [])), lager:info("Checking precommit atom failure reason."), - HTTP = rt:httpc(Node), + HTTP = rt_http:httpc(Node), ?assertMatch({error, {ok, "500", _, _}}, - rt:httpc_write(HTTP, <<"failatom">>, <<"key">>, <<"value">>)), + rt_http:httpc_write(HTTP, <<"failatom">>, <<"key">>, <<"value">>)), lager:info("Checking Bug 1145 - string failure reason"), ?assertMatch({error, {ok, "403", _, _}}, - rt:httpc_write(HTTP, <<"failstr">>, <<"key">>, <<"value">>)), + rt_http:httpc_write(HTTP, <<"failstr">>, <<"key">>, <<"value">>)), lager:info("Checking Bug 1145 - binary failure reason"), ?assertMatch({error, {ok, "403", _, _}}, - rt:httpc_write(HTTP, <<"failbin">>, <<"key">>, <<"value">>)), + rt_http:httpc_write(HTTP, <<"failbin">>, <<"key">>, <<"value">>)), lager:info("Checking that bucket without commit hooks passes."), - ?assertEqual(ok, rt:httpc_write(HTTP, <<"fail">>, <<"key">>, <<"value">>)), + ?assertEqual(ok, rt_http:httpc_write(HTTP, <<"fail">>, <<"key">>, <<"value">>)), lager:info("Checking that bucket with passing precommit passes."), - ?assertEqual(ok, rt:httpc_write(HTTP, <<"failkey">>, <<"key">>, <<"value">>)), + ?assertEqual(ok, rt_http:httpc_write(HTTP, <<"failkey">>, <<"key">>, <<"value">>)), lager:info("Checking that bucket with failing precommit fails."), ?assertMatch({error, {ok, "403", _, _}}, - rt:httpc_write(HTTP, <<"failkey">>, <<"fail">>, <<"value">>)), + rt_http:httpc_write(HTTP, <<"failkey">>, <<"fail">>, <<"value">>)), lager:info("Checking fix for BZ1244 - riak_kv_wm_object makes call to riak_client:get/3 with invalid type for key"), %% riak_kv_wm_object:ensure_doc will return {error, not_found}, leading to 404. %% see https://github.com/basho/riak_kv/pull/237 for details of the fix. ?assertMatch({error, {ok, "404", _, _}}, - rt:httpc_write(HTTP, <<"bz1244bucket">>, undefined, <<"value">>)), + rt_http:httpc_write(HTTP, <<"bz1244bucket">>, undefined, <<"value">>)), lager:info("Checking that postcommit fires."), - ?assertMatch(ok, rt:httpc_write(HTTP, <<"postcommit">>, <<"key">>, <<"value">>)), + ?assertMatch(ok, rt_http:httpc_write(HTTP, <<"postcommit">>, <<"key">>, <<"value">>)), receive {wrote, _Bucket, _Key}=Msg -> diff --git a/tests/verify_counter_capability.erl b/tests/verify_counter_capability.erl index b9e696558..0d596f971 100644 --- a/tests/verify_counter_capability.erl +++ b/tests/verify_counter_capability.erl @@ -81,4 +81,4 @@ confirm() -> pass. get_clients(Node) -> - {rt_pb:pbc(Node), rt:httpc(Node)}. + {rt_pb:pbc(Node), rt_http:httpc(Node)}. diff --git a/tests/verify_counter_converge.erl b/tests/verify_counter_converge.erl index adc067b40..bad3f90b3 100644 --- a/tests/verify_counter_converge.erl +++ b/tests/verify_counter_converge.erl @@ -34,7 +34,7 @@ confirm() -> Key = <<"a">>, [N1, N2, N3, N4]=Nodes = rt_cluster:build_cluster(4), - [C1, C2, C3, C4]=Clients = [ rt:httpc(N) || N <- Nodes ], + [C1, C2, C3, C4]=Clients = [ rt_http:httpc(N) || N <- Nodes ], set_allow_mult_true(Nodes), diff --git a/tests/verify_counter_repl.erl b/tests/verify_counter_repl.erl index b8d9cb202..42fc97049 100644 --- a/tests/verify_counter_repl.erl +++ b/tests/verify_counter_repl.erl @@ -73,7 +73,7 @@ make_cluster(Nodes, Name) -> repl_util:make_cluster(Nodes), repl_util:name_cluster(hd(Nodes), Name), repl_util:wait_until_leader_converge(Nodes), - Clients = [ rt:httpc(Node) || Node <- Nodes ], + Clients = [ rt_http:httpc(Node) || Node <- Nodes ], lists:zip(Clients, Nodes). increment_cluster_counter(Cluster) -> diff --git a/tests/verify_crdt_capability.erl b/tests/verify_crdt_capability.erl index 7c2b1ea73..27e0d9b14 100644 --- a/tests/verify_crdt_capability.erl +++ b/tests/verify_crdt_capability.erl @@ -98,4 +98,4 @@ gen_counter_op() -> riakc_counter:to_op(riakc_counter:increment(riakc_counter:new())). get_clients(Node) -> - {rt_pb:pbc(Node), rt:httpc(Node)}. + {rt_pb:pbc(Node), rt_http:httpc(Node)}. diff --git a/tests/verify_dt_converge.erl b/tests/verify_dt_converge.erl index 7a9053f96..6969b5c95 100644 --- a/tests/verify_dt_converge.erl +++ b/tests/verify_dt_converge.erl @@ -130,7 +130,7 @@ create_pb_clients(Nodes) -> end || N <- Nodes]. create_http_clients(Nodes) -> - [ rt:httpc(N) || N <- Nodes ]. + [ rt_http:httpc(N) || N <- Nodes ]. create_bucket_types([N1|_]=Nodes, Types) -> lager:info("Creating bucket types with datatypes: ~p", [Types]), diff --git a/tests/verify_dt_upgrade.erl b/tests/verify_dt_upgrade.erl index 5aab535fc..258e211d3 100644 --- a/tests/verify_dt_upgrade.erl +++ b/tests/verify_dt_upgrade.erl @@ -52,7 +52,7 @@ populate_counters(Node) -> rt:wait_for_service(Node, riak_kv), ?assertEqual(ok, rt:wait_until(Node, fun has_counter_capability/1)), - RHC = rt:httpc(Node), + RHC = rt_http:httpc(Node), ?assertMatch(ok, rhc:counter_incr(RHC, ?COUNTER_BUCKET, <<"httpkey">>, 2)), ?assertMatch({ok, 2}, rhc:counter_val(RHC, ?COUNTER_BUCKET, <<"httpkey">>)), @@ -65,7 +65,7 @@ populate_counters(Node) -> %% check that you can get via default bucket verify_counters(Node) -> lager:info("Verifying counters on ~p", [Node]), - RHC = rt:httpc(Node), + RHC = rt_http:httpc(Node), ?assertMatch({ok, 4}, rhc:counter_val(RHC, ?COUNTER_BUCKET, <<"pbkey">>)), PBC = rt_pb:pbc(Node), diff --git a/tests/verify_listkeys.erl b/tests/verify_listkeys.erl index ab7fd56bc..b7cbe73f7 100644 --- a/tests/verify_listkeys.erl +++ b/tests/verify_listkeys.erl @@ -103,7 +103,7 @@ list_keys(Node, Interface, Bucket, Attempt, Num, ShouldPass) -> Pid = rt_pb:pbc(Node), Mod = riakc_pb_socket; http -> - Pid = rt:httpc(Node), + Pid = rt_http:httpc(Node), Mod = rhc end, lager:info("Listing keys on ~p using ~p. Attempt #~p", @@ -131,7 +131,7 @@ list_keys_for_undefined_bucket_type(Node, Interface, Bucket, Attempt, ShouldPass Pid = rt_pb:pbc(Node), Mod = riakc_pb_socket; http -> - Pid = rt:httpc(Node), + Pid = rt_http:httpc(Node), Mod = rhc end, @@ -165,7 +165,7 @@ list_buckets(Node, Interface, Attempt, Num, ShouldPass) -> Pid = rt_pb:pbc(Node), Mod = riakc_pb_socket; http -> - Pid = rt:httpc(Node), + Pid = rt_http:httpc(Node), Mod = rhc end, lager:info("Listing buckets on ~p using ~p. Attempt #~p", @@ -199,7 +199,7 @@ list_buckets_for_undefined_bucket_type(Node, Interface, Attempt, ShouldPass) -> Pid = rt_pb:pbc(Node), Mod = riakc_pb_socket; http -> - Pid = rt:httpc(Node), + Pid = rt_http:httpc(Node), Mod = rhc end, diff --git a/tests/verify_riak_stats.erl b/tests/verify_riak_stats.erl index 80135090b..1c5c83cbd 100644 --- a/tests/verify_riak_stats.erl +++ b/tests/verify_riak_stats.erl @@ -69,14 +69,13 @@ confirm() -> <<"memory_code">>, <<"memory_ets">>]), - lager:info("perform 5 x PUT and a GET to increment the stats"), lager:info("as the stat system only does calcs for > 5 readings"), - - C = rt:httpc(Node1), - [rt:httpc_write(C, <<"systest">>, <>, <<"12345">>) || X <- lists:seq(1, 5)], - [rt:httpc_read(C, <<"systest">>, <>) || X <- lists:seq(1, 5)], - + + C = rt_http:httpc(Node1), + [rt_http:httpc_write(C, <<"systest">>, <>, <<"12345">>) || X <- lists:seq(1, 5)], + [rt_http:httpc_read(C, <<"systest">>, <>) || X <- lists:seq(1, 5)], + Stats2 = get_stats(Node1), %% make sure the stats that were supposed to increment did From 56b10d7010fe81720f9b0356f7ad6bfb0a5acfa0 Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Wed, 30 Jul 2014 15:44:37 -0400 Subject: [PATCH 017/157] Move node-related functions from rt module to rt_node. --- src/riak_test_group_leader.erl | 2 +- src/riak_test_runner.erl | 2 +- src/rt.erl | 142 ------------------ src/rt_cluster.erl | 2 +- src/rt_config.erl | 12 +- src/rt_local.erl | 2 +- src/rt_node.erl | 173 ++++++++++++++++++++++ tests/basic_command_line.erl | 2 +- tests/cluster_meta_basic.erl | 4 +- tests/cluster_meta_rmr.erl | 4 +- tests/ensemble_basic2.erl | 2 +- tests/ensemble_basic3.erl | 4 +- tests/ensemble_basic4.erl | 4 +- tests/ensemble_interleave.erl | 4 +- tests/ensemble_remove_node.erl | 4 +- tests/ensemble_remove_node2.erl | 4 +- tests/ensemble_sync.erl | 2 +- tests/gh_riak_core_154.erl | 10 +- tests/gh_riak_core_155.erl | 2 +- tests/jmx_verify.erl | 4 +- tests/pr_pw.erl | 4 +- tests/replication/repl_aae_fullsync.erl | 2 +- tests/replication/repl_util.erl | 2 +- tests/replication/replication.erl | 8 +- tests/replication/replication2.erl | 12 +- tests/replication/replication2_pg.erl | 10 +- tests/riak_control_authentication.erl | 2 +- tests/rt_basic_test.erl | 2 +- tests/verify_build_cluster.erl | 14 +- tests/verify_capabilities.erl | 2 +- tests/verify_counter_converge.erl | 4 +- tests/verify_down.erl | 6 +- tests/verify_dt_context.erl | 4 +- tests/verify_dt_converge.erl | 4 +- tests/verify_listkeys.erl | 10 +- tests/verify_membackend.erl | 2 +- tests/verify_mr_prereduce_node_down.erl | 2 +- tests/verify_riak_lager.erl | 4 +- tests/verify_secondary_index_reformat.erl | 4 +- 39 files changed, 257 insertions(+), 226 deletions(-) create mode 100644 src/rt_node.erl diff --git a/src/riak_test_group_leader.erl b/src/riak_test_group_leader.erl index 5a673ee4b..230c5e761 100644 --- a/src/riak_test_group_leader.erl +++ b/src/riak_test_group_leader.erl @@ -107,4 +107,4 @@ io_requests(_, Result) -> %% If we get multiple lines, we'll split them up for lager to maximize the prettiness. log_chars(Chars) -> - [lager:info("~s", [Line]) || Line <- string:tokens(lists:flatten(Chars), "\n")]. \ No newline at end of file + [lager:info("~s", [Line]) || Line <- string:tokens(lists:flatten(Chars), "\n")]. diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index a1a263c9c..b245d8cee 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -174,7 +174,7 @@ compose_confirm_fun({ConfirmMod, ConfirmFun}, ensure_all_nodes_running(Nodes) -> [begin - ok = rt:start_and_wait(Node), + ok = rt_node:start_and_wait(Node), ok = rt:wait_until_registered(Node, riak_core_ring_manager) end || Node <- Nodes]. diff --git a/src/rt.erl b/src/rt.erl index 59dbe7c0f..3a5bb6bcb 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -32,7 +32,6 @@ admin/2, admin/3, assert_nodes_agree_about_ownership/1, - async_start/1, attach/2, attach_direct/2, brutal_kill/1, @@ -59,18 +58,14 @@ get_replica/5, get_ring/1, get_version/0, - heal/1, is_mixed_cluster/1, is_pingable/1, - join/2, - leave/1, load_modules_on_nodes/2, log_to_nodes/2, log_to_nodes/3, members_according_to/1, nearest_ringsize/1, owners_according_to/1, - partition/2, partitions_for_node/1, pbc/1, pbc/2, @@ -99,16 +94,11 @@ set_advanced_conf/2, setup_harness/2, setup_log_capture/1, - slow_upgrade/3, stream_cmd/1, stream_cmd/2, spawn_cmd/1, spawn_cmd/2, search_cmd/2, - start/1, - start_and_wait/1, status_of_according_to/2, - stop/1, - stop_and_wait/1, str/2, systest_read/2, systest_read/3, @@ -220,101 +210,6 @@ connection_info(Node) when is_atom(Node) -> connection_info(Nodes) when is_list(Nodes) -> [ {Node, connection_info(Node)} || Node <- Nodes]. - -%% @doc Start the specified Riak node -start(Node) -> -?HARNESS:start(Node). - -%% @doc Start the specified Riak `Node' and wait for it to be pingable -start_and_wait(Node) -> -start(Node), -?assertEqual(ok, wait_until_pingable(Node)). - -async_start(Node) -> -spawn(fun() -> start(Node) end). - -%% @doc Stop the specified Riak `Node'. -stop(Node) -> -lager:info("Stopping riak on ~p", [Node]), -timer:sleep(10000), %% I know, I know! -?HARNESS:stop(Node). -%%rpc:call(Node, init, stop, []). - -%% @doc Stop the specified Riak `Node' and wait until it is not pingable -stop_and_wait(Node) -> -stop(Node), -?assertEqual(ok, wait_until_unpingable(Node)). - -%% @doc Upgrade a Riak `Node' to the specified `NewVersion'. -upgrade(Node, NewVersion) -> -?HARNESS:upgrade(Node, NewVersion). - -%% @doc Upgrade a Riak `Node' to the specified `NewVersion' and update -%% the config based on entries in `Config'. -upgrade(Node, NewVersion, Config) -> -?HARNESS:upgrade(Node, NewVersion, Config). - -%% @doc Upgrade a Riak node to a specific version using the alternate -%% leave/upgrade/rejoin approach -slow_upgrade(Node, NewVersion, Nodes) -> -lager:info("Perform leave/upgrade/join upgrade on ~p", [Node]), -lager:info("Leaving ~p", [Node]), -leave(Node), -?assertEqual(ok, rt:wait_until_unpingable(Node)), -upgrade(Node, NewVersion), -lager:info("Rejoin ~p", [Node]), -join(Node, hd(Nodes -- [Node])), -lager:info("Wait until all nodes are ready and there are no pending changes"), -?assertEqual(ok, wait_until_nodes_ready(Nodes)), -?assertEqual(ok, wait_until_no_pending_changes(Nodes)), -ok. - -%% @doc Have `Node' send a join request to `PNode' -join(Node, PNode) -> -R = rpc:call(Node, riak_core, join, [PNode]), -lager:info("[join] ~p to (~p): ~p", [Node, PNode, R]), -?assertEqual(ok, R), -ok. - -%% @doc Have `Node' send a join request to `PNode' -staged_join(Node, PNode) -> -R = rpc:call(Node, riak_core, staged_join, [PNode]), -lager:info("[join] ~p to (~p): ~p", [Node, PNode, R]), -?assertEqual(ok, R), -ok. - -plan_and_commit(Node) -> -timer:sleep(500), -lager:info("planning and commiting cluster join"), -case rpc:call(Node, riak_core_claimant, plan, []) of -{error, ring_not_ready} -> - lager:info("plan: ring not ready"), - timer:sleep(100), - plan_and_commit(Node); -{ok, _, _} -> - lager:info("plan: done"), - do_commit(Node) -end. - -do_commit(Node) -> -case rpc:call(Node, riak_core_claimant, commit, []) of -{error, plan_changed} -> - lager:info("commit: plan changed"), - timer:sleep(100), - maybe_wait_for_changes(Node), - plan_and_commit(Node); -{error, ring_not_ready} -> - lager:info("commit: ring not ready"), - timer:sleep(100), - maybe_wait_for_changes(Node), - do_commit(Node); -{error,nothing_planned} -> - %% Assume plan actually committed somehow - ok; -ok -> - ok -end. - maybe_wait_for_changes(Node) -> Ring = get_ring(Node), Changes = riak_core_ring:pending_changes(Ring), @@ -329,43 +224,6 @@ true -> ok = wait_until_no_pending_changes([Node]) end. -%% @doc Have the `Node' leave the cluster -leave(Node) -> -R = rpc:call(Node, riak_core, leave, []), -lager:info("[leave] ~p: ~p", [Node, R]), -?assertEqual(ok, R), -ok. - -%% @doc Have `Node' remove `OtherNode' from the cluster -remove(Node, OtherNode) -> -?assertEqual(ok, - rpc:call(Node, riak_kv_console, remove, [[atom_to_list(OtherNode)]])). - -%% @doc Have `Node' mark `OtherNode' as down -down(Node, OtherNode) -> -rpc:call(Node, riak_kv_console, down, [[atom_to_list(OtherNode)]]). - -%% @doc partition the `P1' from `P2' nodes -%% note: the nodes remained connected to riak_test@local, -%% which is how `heal/1' can still work. -partition(P1, P2) -> -OldCookie = rpc:call(hd(P1), erlang, get_cookie, []), -NewCookie = list_to_atom(lists:reverse(atom_to_list(OldCookie))), -[true = rpc:call(N, erlang, set_cookie, [N, NewCookie]) || N <- P1], -[[true = rpc:call(N, erlang, disconnect_node, [P2N]) || N <- P1] || P2N <- P2], -wait_until_partitioned(P1, P2), -{NewCookie, OldCookie, P1, P2}. - -%% @doc heal the partition created by call to `partition/2' -%% `OldCookie' is the original shared cookie -heal({_NewCookie, OldCookie, P1, P2}) -> -Cluster = P1 ++ P2, -% set OldCookie on P1 Nodes -[true = rpc:call(N, erlang, set_cookie, [N, OldCookie]) || N <- P1], -wait_until_connected(Cluster), -{_GN, []} = rpc:sbcast(Cluster, riak_core_node_watcher, broadcast), -ok. - %% @doc Spawn `Cmd' on the machine running the test harness spawn_cmd(Cmd) -> ?HARNESS:spawn_cmd(Cmd). diff --git a/src/rt_cluster.erl b/src/rt_cluster.erl index 1507a781f..cab1cd92c 100644 --- a/src/rt_cluster.erl +++ b/src/rt_cluster.erl @@ -204,7 +204,7 @@ clean_data_dir(Nodes, SubDir) when is_list(Nodes) -> teardown() -> %% stop all connected nodes, 'cause it'll be faster that %%lager:info("RPC stopping these nodes ~p", [nodes()]), - %%[ rt:stop(Node) || Node <- nodes()], + %%[ rt_node:stop(Node) || Node <- nodes()], %% Then do the more exhaustive harness thing, in case something was up %% but not connected. ?HARNESS:teardown(). diff --git a/src/rt_config.erl b/src/rt_config.erl index 8da5a9539..28ffa6cb5 100644 --- a/src/rt_config.erl +++ b/src/rt_config.erl @@ -133,29 +133,29 @@ config_or_os_env(Config, Default) -> set_conf(all, NameValuePairs) -> ?HARNESS:set_conf(all, NameValuePairs); set_conf(Node, NameValuePairs) -> - rt:stop(Node), + rt_node:stop(Node), ?assertEqual(ok, rt:wait_until_unpingable(Node)), ?HARNESS:set_conf(Node, NameValuePairs), - rt:start(Node). + rt_node:start(Node). -spec set_advanced_conf(atom(), [{string(), string()}]) -> ok. set_advanced_conf(all, NameValuePairs) -> ?HARNESS:set_advanced_conf(all, NameValuePairs); set_advanced_conf(Node, NameValuePairs) -> - rt:stop(Node), + rt_node:stop(Node), ?assertEqual(ok, rt:wait_until_unpingable(Node)), ?HARNESS:set_advanced_conf(Node, NameValuePairs), - rt:start(Node). + rt_node:start(Node). %% @doc Rewrite the given node's app.config file, overriding the varialbes %% in the existing app.config with those in `Config'. update_app_config(all, Config) -> ?HARNESS:update_app_config(all, Config); update_app_config(Node, Config) -> - rt:stop(Node), + rt_node:stop(Node), ?assertEqual(ok, rt:wait_until_unpingable(Node)), ?HARNESS:update_app_config(Node, Config), - rt:start(Node). + rt_node:start(Node). version_to_config(Config) when is_tuple(Config)-> Config; version_to_config(Version) -> {Version, default}. diff --git a/src/rt_local.erl b/src/rt_local.erl index b8bfaed8e..fd4517e17 100644 --- a/src/rt_local.erl +++ b/src/rt_local.erl @@ -117,4 +117,4 @@ stream_cmd_loop(Port, Buffer, NewLineBuffer, Time={_MegaSecs, Secs, _MicroSecs}) {Status, Buffer} after rt_config:get(rt_max_wait_time) -> {-1, Buffer} - end. \ No newline at end of file + end. diff --git a/src/rt_node.erl b/src/rt_node.erl new file mode 100644 index 000000000..1b2c6fcc3 --- /dev/null +++ b/src/rt_node.erl @@ -0,0 +1,173 @@ +%% ------------------------------------------------------------------- +%% +%% Copyright (c) 2013-2014 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(rt_node). +-include("rt.hrl"). +-include_lib("eunit/include/eunit.hrl"). + +-export([start/1, + start_and_wait/1, + async_start/1, + stop/1, + stop_and_wait/1, + upgrade/2, + upgrade/3, + slow_upgrade/3, + join/2, + staged_join/2, + plan_and_commit/1, + do_commit/1, + leave/1, + down/2, + heal/1, + partition/2, + remove/2]). + +-define(HARNESS, (rt_config:get(rt_harness))). + +%% @doc Start the specified Riak node +start(Node) -> + ?HARNESS:start(Node). + +%% @doc Start the specified Riak `Node' and wait for it to be pingable +start_and_wait(Node) -> + start(Node), + ?assertEqual(ok, rt:wait_until_pingable(Node)). + +async_start(Node) -> + spawn(fun() -> start(Node) end). + +%% @doc Stop the specified Riak `Node'. +stop(Node) -> + lager:info("Stopping riak on ~p", [Node]), + timer:sleep(10000), %% I know, I know! + ?HARNESS:stop(Node). + %%rpc:call(Node, init, stop, []). + +%% @doc Stop the specified Riak `Node' and wait until it is not pingable +stop_and_wait(Node) -> + stop(Node), + ?assertEqual(ok, rt:wait_until_unpingable(Node)). + +%% @doc Upgrade a Riak `Node' to the specified `NewVersion'. +upgrade(Node, NewVersion) -> + ?HARNESS:upgrade(Node, NewVersion). + +%% @doc Upgrade a Riak `Node' to the specified `NewVersion' and update +%% the config based on entries in `Config'. +upgrade(Node, NewVersion, Config) -> + ?HARNESS:upgrade(Node, NewVersion, Config). + +%% @doc Upgrade a Riak node to a specific version using the alternate +%% leave/upgrade/rejoin approach +slow_upgrade(Node, NewVersion, Nodes) -> + lager:info("Perform leave/upgrade/join upgrade on ~p", [Node]), + lager:info("Leaving ~p", [Node]), + leave(Node), + ?assertEqual(ok, rt:wait_until_unpingable(Node)), + upgrade(Node, NewVersion), + lager:info("Rejoin ~p", [Node]), + join(Node, hd(Nodes -- [Node])), + lager:info("Wait until all nodes are ready and there are no pending changes"), + ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), + ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes)), + ok. + +%% @doc Have `Node' send a join request to `PNode' +join(Node, PNode) -> + R = rpc:call(Node, riak_core, join, [PNode]), + lager:info("[join] ~p to (~p): ~p", [Node, PNode, R]), + ?assertEqual(ok, R), + ok. + +%% @doc Have `Node' send a join request to `PNode' +staged_join(Node, PNode) -> + R = rpc:call(Node, riak_core, staged_join, [PNode]), + lager:info("[join] ~p to (~p): ~p", [Node, PNode, R]), + ?assertEqual(ok, R), + ok. + +plan_and_commit(Node) -> + timer:sleep(500), + lager:info("planning and commiting cluster join"), + case rpc:call(Node, riak_core_claimant, plan, []) of + {error, ring_not_ready} -> + lager:info("plan: ring not ready"), + timer:sleep(100), + plan_and_commit(Node); + {ok, _, _} -> + lager:info("plan: done"), + do_commit(Node) + end. + +do_commit(Node) -> + case rpc:call(Node, riak_core_claimant, commit, []) of + {error, plan_changed} -> + lager:info("commit: plan changed"), + timer:sleep(100), + rt:maybe_wait_for_changes(Node), + plan_and_commit(Node); + {error, ring_not_ready} -> + lager:info("commit: ring not ready"), + timer:sleep(100), + rt:maybe_wait_for_changes(Node), + do_commit(Node); + {error,nothing_planned} -> + %% Assume plan actually committed somehow + ok; + ok -> + ok + end. + +%% @doc Have the `Node' leave the cluster +leave(Node) -> + R = rpc:call(Node, riak_core, leave, []), + lager:info("[leave] ~p: ~p", [Node, R]), + ?assertEqual(ok, R), + ok. + +%% @doc Have `Node' remove `OtherNode' from the cluster +remove(Node, OtherNode) -> + ?assertEqual(ok, + rpc:call(Node, riak_kv_console, remove, [[atom_to_list(OtherNode)]])). + +%% @doc Have `Node' mark `OtherNode' as down +down(Node, OtherNode) -> + rpc:call(Node, riak_kv_console, down, [[atom_to_list(OtherNode)]]). + +%% @doc partition the `P1' from `P2' nodes +%% note: the nodes remained connected to riak_test@local, +%% which is how `heal/1' can still work. +partition(P1, P2) -> + OldCookie = rpc:call(hd(P1), erlang, get_cookie, []), + NewCookie = list_to_atom(lists:reverse(atom_to_list(OldCookie))), + [true = rpc:call(N, erlang, set_cookie, [N, NewCookie]) || N <- P1], + [[true = rpc:call(N, erlang, disconnect_node, [P2N]) || N <- P1] || P2N <- P2], + rt:wait_until_partitioned(P1, P2), + {NewCookie, OldCookie, P1, P2}. + +%% @doc heal the partition created by call to `partition/2' +%% `OldCookie' is the original shared cookie +heal({_NewCookie, OldCookie, P1, P2}) -> + Cluster = P1 ++ P2, + % set OldCookie on P1 Nodes + [true = rpc:call(N, erlang, set_cookie, [N, OldCookie]) || N <- P1], + rt:wait_until_connected(Cluster), + {_GN, []} = rpc:sbcast(Cluster, riak_core_node_watcher, broadcast), + ok. diff --git a/tests/basic_command_line.erl b/tests/basic_command_line.erl index 88afad49d..bc31c9414 100644 --- a/tests/basic_command_line.erl +++ b/tests/basic_command_line.erl @@ -105,7 +105,7 @@ ping_up_test(Node) -> lager:info("Testing riak ping on ~s", [Node]), %% ping / pong - %% rt:start_and_wait(Node), + %% rt_node:start_and_wait(Node), lager:info("Node up, should ping"), {ok, PongOut} = rt:riak(Node, ["ping"]), ?assert(rt:str(PongOut, "pong")), diff --git a/tests/cluster_meta_basic.erl b/tests/cluster_meta_basic.erl index 188763ad6..cb4cf0935 100644 --- a/tests/cluster_meta_basic.erl +++ b/tests/cluster_meta_basic.erl @@ -56,13 +56,13 @@ test_writes_after_partial_cluster_failure([N1 | _]=Nodes) -> StopNodes = eager_peers(N1, N1), AliveNodes = Nodes -- StopNodes, lager:info("stopping nodes: ~p remaining nodes: ~p", [StopNodes, AliveNodes]), - [rt:stop(N) || N <- StopNodes], + [rt_node:stop(N) || N <- StopNodes], metadata_put(N1, ?PREFIX1, ?KEY1, ?VAL2), wait_until_metadata_value(AliveNodes, ?PREFIX1, ?KEY1, ?VAL2), lager:info("bring stopped nodes back up: ~p", [StopNodes]), - [rt:start(N) || N <- StopNodes], + [rt_node:start(N) || N <- StopNodes], wait_until_metadata_value(Nodes, ?PREFIX1, ?KEY1, ?VAL2), ok. diff --git a/tests/cluster_meta_rmr.erl b/tests/cluster_meta_rmr.erl index fe7f73e3a..fee5d840f 100644 --- a/tests/cluster_meta_rmr.erl +++ b/tests/cluster_meta_rmr.erl @@ -58,7 +58,7 @@ run(NumNodes, NumRounds, StableRounds) -> calc_stuff(AllNodes, NumNodes, NumRounds), exit(Pid, kill), %% start all the down nodes so we can clean them :( - [rt:start(Node) || Node <- DownNodes], + [rt_node:start(Node) || Node <- DownNodes], rt_cluster:clean_cluster(AllNodes). setup_nodes(NumNodes) -> @@ -108,7 +108,7 @@ run_rounds(Round, StableRound, SendFun, ConsistentFun, [SenderNode | _]=UpNodes, fail_node(Round, OtherNodes) -> Failed = lists:nth(random:uniform(length(OtherNodes)), OtherNodes), lager:info("round: ~p (unstable): shutting down ~p", [Round, Failed]), - rt:stop(Failed), + rt_node:stop(Failed), {Failed, lists:delete(Failed, OtherNodes)}. calc_stuff(AllNodes, NumNodes, NumRounds) -> diff --git a/tests/ensemble_basic2.erl b/tests/ensemble_basic2.erl index dee9e815f..6af2f2796 100644 --- a/tests/ensemble_basic2.erl +++ b/tests/ensemble_basic2.erl @@ -43,7 +43,7 @@ confirm() -> riak_kv_vnode_orig:init_orig(Args) end}}]}), rt:stop_and_wait(Node), - rt:start(Node), + rt_node:start(Node), lager:info("Polling peers while riak_kv starts. We should see none"), UpNoPeersFun = fun() -> diff --git a/tests/ensemble_basic3.erl b/tests/ensemble_basic3.erl index 996517fd3..f6ff01ba0 100644 --- a/tests/ensemble_basic3.erl +++ b/tests/ensemble_basic3.erl @@ -58,7 +58,7 @@ confirm() -> PBC = rt_pb:pbc(Node), lager:info("Partitioning quorum minority: ~p", [Partitioned]), - Part = rt:partition(Nodes -- Partitioned, Partitioned), + Part = rt_node:partition(Nodes -- Partitioned, Partitioned), ensemble_util:wait_until_stable(Node, Quorum), lager:info("Writing ~p consistent keys", [1000]), @@ -68,7 +68,7 @@ confirm() -> [rt_pb:pbc_read(PBC, Bucket, Key) || Key <- Keys], lager:info("Healing partition"), - rt:heal(Part), + rt_node:heal(Part), lager:info("Suspending majority vnodes"), L = [begin diff --git a/tests/ensemble_basic4.erl b/tests/ensemble_basic4.erl index 35cd92f50..2050e0eeb 100644 --- a/tests/ensemble_basic4.erl +++ b/tests/ensemble_basic4.erl @@ -51,7 +51,7 @@ confirm() -> PBC = rt_pb:pbc(Node), lager:info("Partitioning quorum minority: ~p", [Partitioned]), - Part = rt:partition(Nodes -- Partitioned, Partitioned), + Part = rt_node:partition(Nodes -- Partitioned, Partitioned), rpc:multicall(Nodes, riak_kv_entropy_manager, set_mode, [manual]), ensemble_util:wait_until_stable(Node, Quorum), @@ -62,6 +62,6 @@ confirm() -> [rt_pb:pbc_read(PBC, Bucket, Key) || Key <- Keys], lager:info("Healing partition"), - rt:heal(Part), + rt_node:heal(Part), pass. diff --git a/tests/ensemble_interleave.erl b/tests/ensemble_interleave.erl index 105bffffa..5aeb78ca4 100644 --- a/tests/ensemble_interleave.erl +++ b/tests/ensemble_interleave.erl @@ -69,7 +69,7 @@ confirm() -> Options = [{timeout, 500}], rpc:multicall(Nodes, riak_kv_entropy_manager, set_mode, [manual]), - Part = rt:partition(Nodes -- Partitioned, Partitioned), + Part = rt_node:partition(Nodes -- Partitioned, Partitioned), ensemble_util:wait_until_stable(Node, Quorum), lager:info("Writing ~p consistent keys", [1000]), @@ -77,7 +77,7 @@ confirm() -> lager:info("Read keys to verify they exist"), [rt_pb:pbc_read(PBC, Bucket, Key, Options) || Key <- Keys], - rt:heal(Part), + rt_node:heal(Part), [begin lager:info("Suspending vnode: ~p", [VIdx]), diff --git a/tests/ensemble_remove_node.erl b/tests/ensemble_remove_node.erl index 2d9e8e9e0..cefc0ba89 100644 --- a/tests/ensemble_remove_node.erl +++ b/tests/ensemble_remove_node.erl @@ -54,9 +54,9 @@ confirm() -> end}}]}), lager:info("Removing Nodes 2 and 3 from the cluster"), - rt:leave(Node2), + rt_node:leave(Node2), ok = ensemble_util:wait_until_stable(Node, NVal), - rt:leave(Node3), + rt_node:leave(Node3), ok = ensemble_util:wait_until_stable(Node, NVal), Remaining = Nodes -- [Node2, Node3], rt:wait_until_nodes_agree_about_ownership(Remaining), diff --git a/tests/ensemble_remove_node2.erl b/tests/ensemble_remove_node2.erl index 9d3a6945a..4477de2c6 100644 --- a/tests/ensemble_remove_node2.erl +++ b/tests/ensemble_remove_node2.erl @@ -61,9 +61,9 @@ confirm() -> end}}]}), lager:info("Removing Nodes 2 and 3 from the cluster"), - rt:leave(Node2), + rt_node:leave(Node2), ok = ensemble_util:wait_until_stable(Node, NVal), - rt:leave(Node3), + rt_node:leave(Node3), ok = ensemble_util:wait_until_stable(Node, NVal), Remaining = Nodes -- [Node2, Node3], rt:wait_until_nodes_agree_about_ownership(Remaining), diff --git a/tests/ensemble_sync.erl b/tests/ensemble_sync.erl index 4b403d5a9..f5d0197cd 100644 --- a/tests/ensemble_sync.erl +++ b/tests/ensemble_sync.erl @@ -111,7 +111,7 @@ run_scenario(Nodes, NVal, {NumKill, NumSuspend, NumValid, _, Name, Expect}) -> lager:info("Read keys to verify they exist"), [rt_pb:pbc_read(PBC, Bucket, Key, Options) || Key <- Keys], - rt:heal(Part), + rt_node:heal(Part), %% Suspend desired number of valid vnodes S1 = [vnode_util:suspend_vnode(VNode, VIdx) || {VIdx, VNode} <- SuspendVN], diff --git a/tests/gh_riak_core_154.erl b/tests/gh_riak_core_154.erl index 70882e5dc..dfb53bfb4 100644 --- a/tests/gh_riak_core_154.erl +++ b/tests/gh_riak_core_154.erl @@ -33,24 +33,24 @@ confirm() -> [Node1, Node2] = Nodes, lager:info("Write data while ~p is offline", [Node2]), - rt:stop(Node2), + rt_node:stop(Node2), rt:wait_until_unpingable(Node2), ?assertEqual([], rt:systest_write(Node1, 1000, 3)), lager:info("Verify that ~p is missing data", [Node2]), - rt:start(Node2), - rt:stop(Node1), + rt_node:start(Node2), + rt_node:stop(Node1), rt:wait_until_unpingable(Node1), ?assertMatch([{_,{error,notfound}}|_], rt:systest_read(Node2, 1000, 3)), lager:info("Restart ~p and wait for handoff to occur", [Node1]), - rt:start(Node1), + rt_node:start(Node1), rt:wait_for_service(Node1, riak_kv), rt:wait_until_transfers_complete([Node1]), lager:info("Verify that ~p has all data", [Node2]), - rt:stop(Node1), + rt_node:stop(Node1), ?assertEqual([], rt:systest_read(Node2, 1000, 3)), lager:info("gh_riak_core_154: passed"), diff --git a/tests/gh_riak_core_155.erl b/tests/gh_riak_core_155.erl index fb53b69ce..1e6ffd604 100644 --- a/tests/gh_riak_core_155.erl +++ b/tests/gh_riak_core_155.erl @@ -39,7 +39,7 @@ confirm() -> %% Restart node, add intercept that delay proxy startup, and issue gets. %% Gets will come in before proxies started, and should trigger crash. rt:stop_and_wait(Node), - rt:async_start(Node), + rt_node:async_start(Node), rt:wait_until_pingable(Node), rt_intercept:load_intercepts([Node]), rt_intercept:add(Node, {riak_core_vnode_proxy_sup, diff --git a/tests/jmx_verify.erl b/tests/jmx_verify.erl index 14616388d..677973cc9 100644 --- a/tests/jmx_verify.erl +++ b/tests/jmx_verify.erl @@ -146,7 +146,7 @@ test_supervision() -> lager:info("It can fail, it can fail 10 times"), rt:wait_until(retry_check_fun(Node)), - rt:stop(Node), + rt_node:stop(Node), ok_ok. retry_check_fun(Node) -> @@ -205,7 +205,7 @@ test_application_stop() -> ?assertEqual(nomatch, re:run(rpc:call(Node, os, cmd, ["ps -Af"]), "riak_jmx.jar", [])), - rt:stop(Node). + rt_node:stop(Node). verify_inc(Prev, Props, Keys) -> [begin diff --git a/tests/pr_pw.erl b/tests/pr_pw.erl index 07b31ec30..5cecf6e6d 100644 --- a/tests/pr_pw.erl +++ b/tests/pr_pw.erl @@ -98,7 +98,7 @@ confirm() -> ?assertEqual({error, timeout}, C:put(Obj, [{pw, quorum}])), %% restart the node - rt:start_and_wait(Node), + rt_node:start_and_wait(Node), rt:wait_for_service(Node, riak_kv), %% we can make quorum again @@ -111,7 +111,7 @@ confirm() -> %% reboot the node rt:stop_and_wait(Node2), - rt:start_and_wait(Node2), + rt_node:start_and_wait(Node2), rt:wait_for_service(Node2, riak_kv), %% everything is happy again diff --git a/tests/replication/repl_aae_fullsync.erl b/tests/replication/repl_aae_fullsync.erl index cf23f0bdf..af3cb5f9a 100644 --- a/tests/replication/repl_aae_fullsync.erl +++ b/tests/replication/repl_aae_fullsync.erl @@ -567,7 +567,7 @@ validate_intercepted_fullsync(InterceptTarget, %% Reboot node. rt:stop_and_wait(InterceptTarget), - rt:start_and_wait(InterceptTarget), + rt_node:start_and_wait(InterceptTarget), %% Wait for riak_kv and riak_repl to initialize. rt:wait_for_service(InterceptTarget, riak_kv), diff --git a/tests/replication/repl_util.erl b/tests/replication/repl_util.erl index cdb5f9c02..700f12318 100644 --- a/tests/replication/repl_util.erl +++ b/tests/replication/repl_util.erl @@ -627,7 +627,7 @@ validate_intercepted_fullsync(InterceptTarget, %% Reboot node. rt:stop_and_wait(InterceptTarget), - rt:start_and_wait(InterceptTarget), + rt_node:start_and_wait(InterceptTarget), %% Wait for riak_kv and riak_repl to initialize. rt:wait_for_service(InterceptTarget, riak_kv), diff --git a/tests/replication/replication.erl b/tests/replication/replication.erl index 6a4c04bc3..c9c76890b 100644 --- a/tests/replication/replication.erl +++ b/tests/replication/replication.erl @@ -143,7 +143,7 @@ replication([AFirst|_] = ANodes, [BFirst|_] = BNodes, Connected) -> rt:log_to_nodes(AllNodes, "Testing master failover: stopping ~p", [LeaderA]), lager:info("Testing master failover: stopping ~p", [LeaderA]), - rt:stop(LeaderA), + rt_node:stop(LeaderA), rt:wait_until_unpingable(LeaderA), wait_until_leader(ASecond), @@ -166,7 +166,7 @@ replication([AFirst|_] = ANodes, [BFirst|_] = BNodes, Connected) -> LeaderB = rpc:call(BFirst, riak_repl_leader, leader_node, []), lager:info("Testing client failover: stopping ~p", [LeaderB]), - rt:stop(LeaderB), + rt_node:stop(LeaderB), rt:wait_until_unpingable(LeaderB), BSecond = hd(BNodes -- [LeaderB]), wait_until_leader(BSecond), @@ -197,7 +197,7 @@ replication([AFirst|_] = ANodes, [BFirst|_] = BNodes, Connected) -> %% lager:info("Restarting down node ~p", [LeaderA]), - rt:start(LeaderA), + rt_node:start(LeaderA), rt:wait_until_pingable(LeaderA), rt:wait_until_no_pending_changes(ANodes), wait_until_leader_converge(ANodes), @@ -265,7 +265,7 @@ replication([AFirst|_] = ANodes, [BFirst|_] = BNodes, Connected) -> end, lager:info("Restarting down node ~p", [LeaderB]), - rt:start(LeaderB), + rt_node:start(LeaderB), rt:wait_until_pingable(LeaderB), case nodes_all_have_version(ANodes, "1.1.0") of diff --git a/tests/replication/replication2.erl b/tests/replication/replication2.erl index b02af02cd..0f69825d6 100644 --- a/tests/replication/replication2.erl +++ b/tests/replication/replication2.erl @@ -235,7 +235,7 @@ master_failover_test([AFirst|_] = ANodes, [BFirst|_] = BNodes) -> log_to_nodes(ANodes ++ BNodes, "Testing master failover: stopping ~p", [LeaderA]), lager:info("Testing master failover: stopping ~p", [LeaderA]), - rt:stop(LeaderA), + rt_node:stop(LeaderA), rt:wait_until_unpingable(LeaderA), ASecond = hd(ANodes -- [LeaderA]), repl_util:wait_until_leader(ASecond), @@ -259,7 +259,7 @@ master_failover_test([AFirst|_] = ANodes, [BFirst|_] = BNodes) -> log_to_nodes(ANodes ++ BNodes, "Testing client failover: stopping ~p", [LeaderB]), lager:info("Testing client failover: stopping ~p", [LeaderB]), - rt:stop(LeaderB), + rt_node:stop(LeaderB), rt:wait_until_unpingable(LeaderB), BSecond = hd(BNodes -- [LeaderB]), repl_util:wait_until_leader(BSecond), @@ -376,7 +376,7 @@ network_partition_test([AFirst|_] = ANodes, [BFirst|_] = BNodes) -> %% Verify that the original real time keys did not replicate %% Verify that the No replication bucket didn't replicate. bucket_sync_test([AFirst|_] = ANodes, [BFirst|_] = BNodes) -> - + TestHash = list_to_binary([io_lib:format("~2.16.0b", [X]) || <> <= erlang:md5(term_to_binary(os:timestamp()))]), FullsyncOnly = <>, RealtimeOnly = <>, @@ -521,7 +521,7 @@ offline_queueing_tests([AFirst|_] = ANodes, [BFirst|_] = BNodes) -> lager:info("Stopping node ~p", [Target]), - rt:stop(Target), + rt_node:stop(Target), rt:wait_until_unpingable(Target), lager:info("Starting realtime"), @@ -584,7 +584,7 @@ pb_write_during_shutdown([AFirst|_] = ANodes, [BFirst|_] = BNodes) -> lager:info("pb_write_during_shutdown: Ensure node ~p is down before restart", [Target]), ?assertEqual(ok, rt:wait_until_unpingable(Target)), - rt:start(Target), + rt_node:start(Target), rt:wait_until_pingable(Target), rt:wait_for_service(Target, riak_repl), ReadErrors2 = rt:systest_read(Target, 1000, 11000, TestBucket, 2), @@ -658,7 +658,7 @@ http_write_during_shutdown([AFirst|_] = ANodes, [BFirst|_] = BNodes) -> lager:info("HTTP: write_during_shutdown: Ensure node ~p is down before restart", [Target]), ?assertEqual(ok, rt:wait_until_unpingable(Target)), - rt:start(Target), + rt_node:start(Target), rt:wait_until_pingable(Target), rt:wait_for_service(Target, riak_repl), ReadErrors2 = http_read(C, 12000, 22000, TestBucket, 2), diff --git a/tests/replication/replication2_pg.erl b/tests/replication/replication2_pg.erl index de38d3c89..2bc60f292 100644 --- a/tests/replication/replication2_pg.erl +++ b/tests/replication/replication2_pg.erl @@ -418,7 +418,7 @@ test_pg_proxy(SSL) -> lager:info("Stopping leader on requester cluster"), PGLeaderB = rpc:call(FirstB, riak_core_cluster_mgr, get_leader, []), rt:log_to_nodes(AllNodes, "Killing leader on requester cluster"), - rt:stop(PGLeaderB), + rt_node:stop(PGLeaderB), [RunningBNode | _ ] = BNodes -- [PGLeaderB], repl_util:wait_until_leader(RunningBNode), PidB2 = rt_pb:pbc(RunningBNode), @@ -428,7 +428,7 @@ test_pg_proxy(SSL) -> lager:info("Stopping leader on provider cluster"), PGLeaderA = rpc:call(FirstA, riak_core_cluster_mgr, get_leader, []), - rt:stop(PGLeaderA), + rt_node:stop(PGLeaderA), [RunningANode | _ ] = ANodes -- [PGLeaderA], repl_util:wait_until_leader(RunningANode), ?assertEqual(ok, wait_until_pg(RunningBNode, PidB2, Bucket, KeyD, CidA)), @@ -530,7 +530,7 @@ test_cluster_mapping(SSL) -> % shut down cluster A lager:info("Shutting down cluster A"), - [ rt:stop(Node) || Node <- ANodes ], + [ rt_node:stop(Node) || Node <- ANodes ], [ rt:wait_until_unpingable(Node) || Node <- ANodes ], rt:wait_until_ring_converged(BNodes), @@ -937,12 +937,12 @@ verify_topology_change(SourceNodes, SinkNodes) -> %% Sad this takes 2.5 minutes lager:info("Removing current leader from the cluster: ~p.", [SinkLeader]), - rt:leave(SinkLeader), + rt_node:leave(SinkLeader), ?assertEqual(ok, rt:wait_until_unpingable(SinkLeader)), %% Wait for everything to restart, and rings to converge. lager:info("Starting leader node back up and waiting for repl."), - rt:start(SinkLeader), + rt_node:start(SinkLeader), rt:wait_for_service(SinkLeader, riak_repl), rt:wait_until_ring_converged(SinkNodes), diff --git a/tests/riak_control_authentication.erl b/tests/riak_control_authentication.erl index a1fce9f70..3e8db3b00 100644 --- a/tests/riak_control_authentication.erl +++ b/tests/riak_control_authentication.erl @@ -218,7 +218,7 @@ build_singleton_cluster(Vsn, Config) -> %% take effect. Node = lists:nth(1, Nodes), rt:stop_and_wait(Node), - rt:start_and_wait(Node), + rt_node:start_and_wait(Node), rt:wait_for_service(Node, riak_kv), %% Wait for control to start. diff --git a/tests/rt_basic_test.erl b/tests/rt_basic_test.erl index 67aeae122..ed91736e0 100644 --- a/tests/rt_basic_test.erl +++ b/tests/rt_basic_test.erl @@ -25,5 +25,5 @@ confirm() -> lager:info("Deploy some nodes"), Nodes = rt_cluster:deploy_nodes(2), lager:info("Stop the nodes"), - [rt:stop(Node) || Node <- Nodes], + [rt_node:stop(Node) || Node <- Nodes], pass. diff --git a/tests/verify_build_cluster.erl b/tests/verify_build_cluster.erl index a5f2e5abd..051d5c616 100644 --- a/tests/verify_build_cluster.erl +++ b/tests/verify_build_cluster.erl @@ -53,37 +53,37 @@ confirm(#rt_properties{nodes=Nodes}, _MD) -> wait_and_validate(Nodes), lager:info("taking Node 1 down"), - rt:stop(Node1), + rt_node:stop(Node1), ?assertEqual(ok, rt:wait_until_unpingable(Node1)), wait_and_validate(Nodes, [Node2, Node3, Node4]), lager:info("taking Node 2 down"), - rt:stop(Node2), + rt_node:stop(Node2), ?assertEqual(ok, rt:wait_until_unpingable(Node2)), wait_and_validate(Nodes, [Node3, Node4]), lager:info("bringing Node 1 up"), - rt:start(Node1), + rt_node:start(Node1), ok = rt:wait_until_pingable(Node1), wait_and_validate(Nodes, [Node1, Node3, Node4]), lager:info("bringing Node 2 up"), - rt:start(Node2), + rt_node:start(Node2), ok = rt:wait_until_pingable(Node2), wait_and_validate(Nodes), % leave 1, 2, and 3 lager:info("leaving Node 1"), - rt:leave(Node1), + rt_node:leave(Node1), ?assertEqual(ok, rt:wait_until_unpingable(Node1)), wait_and_validate([Node2, Node3, Node4]), lager:info("leaving Node 2"), - rt:leave(Node2), + rt_node:leave(Node2), ?assertEqual(ok, rt:wait_until_unpingable(Node2)), wait_and_validate([Node3, Node4]), lager:info("leaving Node 3"), - rt:leave(Node3), + rt_node:leave(Node3), ?assertEqual(ok, rt:wait_until_unpingable(Node3)), % verify 4 diff --git a/tests/verify_capabilities.erl b/tests/verify_capabilities.erl index d689a0229..2d6ee385c 100644 --- a/tests/verify_capabilities.erl +++ b/tests/verify_capabilities.erl @@ -230,7 +230,7 @@ confirm() -> lager:info("Verify vnode_routing == legacy"), assert_capability(CNode, {riak_core, vnode_routing}, legacy), - [rt:stop(Node) || Node <- Nodes], + [rt_node:stop(Node) || Node <- Nodes], pass. assert_capability(CNode, Capability, Value) -> diff --git a/tests/verify_counter_converge.erl b/tests/verify_counter_converge.erl index bad3f90b3..f91269e1c 100644 --- a/tests/verify_counter_converge.erl +++ b/tests/verify_counter_converge.erl @@ -50,7 +50,7 @@ confirm() -> lager:info("Partition cluster in two."), - PartInfo = rt:partition([N1, N2], [N3, N4]), + PartInfo = rt_node:partition([N1, N2], [N3, N4]), %% increment one side increment_counter(C1, Key, 5), @@ -68,7 +68,7 @@ confirm() -> %% heal lager:info("Heal and check merged values"), - ok = rt:heal(PartInfo), + ok = rt_node:heal(PartInfo), ok = rt:wait_for_cluster_service(Nodes, riak_kv), %% verify all nodes agree diff --git a/tests/verify_down.erl b/tests/verify_down.erl index efbf79d0d..488b65cb0 100644 --- a/tests/verify_down.erl +++ b/tests/verify_down.erl @@ -34,7 +34,7 @@ confirm() -> %% Shutdown node2 lager:info("Stopping ~p", [Node2]), - rt:stop(Node2), + rt_node:stop(Node2), ?assertEqual(ok, rt:wait_until_unpingable(Node2)), Remaining = Nodes -- [Node2], @@ -49,7 +49,7 @@ confirm() -> %% Mark node2 as down and wait for ring convergence lager:info("Mark ~p as down", [Node2]), - rt:down(Node1, Node2), + rt_node:down(Node1, Node2), ?assertEqual(ok, rt:wait_until_ring_converged(Remaining)), [?assertEqual(down, rt:status_of_according_to(Node2, Node)) || Node <- Remaining], @@ -58,7 +58,7 @@ confirm() -> %% Restart node2 and wait for ring convergence lager:info("Restart ~p and wait for ring convergence", [Node2]), - rt:start(Node2), + rt_node:start(Node2), ?assertEqual(ok, rt:wait_until_nodes_ready([Node2])), ?assertEqual(ok, rt:wait_until_ring_converged(Nodes)), diff --git a/tests/verify_dt_context.erl b/tests/verify_dt_context.erl index af81a6c67..4a355962f 100644 --- a/tests/verify_dt_context.erl +++ b/tests/verify_dt_context.erl @@ -72,7 +72,7 @@ confirm() -> lager:info("Partition cluster in two."), - PartInfo = rt:partition([N1], [N2]), + PartInfo = rt_node:partition([N1], [N2]), lager:info("Modify data on side 1"), %% Modify one side @@ -135,7 +135,7 @@ confirm() -> %% Check both sides %% heal lager:info("Heal and check merged values"), - ok = rt:heal(PartInfo), + ok = rt_node:heal(PartInfo), ok = rt:wait_for_cluster_service(Nodes, riak_kv), %% verify all nodes agree diff --git a/tests/verify_dt_converge.erl b/tests/verify_dt_converge.erl index 6969b5c95..5177d42e7 100644 --- a/tests/verify_dt_converge.erl +++ b/tests/verify_dt_converge.erl @@ -72,7 +72,7 @@ confirm() -> lager:info("Partition cluster in two."), - PartInfo = rt:partition([N1, N2], [N3, N4]), + PartInfo = rt_node:partition([N1, N2], [N3, N4]), lager:info("Modify data on side 1"), %% Modify one side @@ -108,7 +108,7 @@ confirm() -> %% heal lager:info("Heal and check merged values"), - ok = rt:heal(PartInfo), + ok = rt_node:heal(PartInfo), ok = rt:wait_for_cluster_service(Nodes, riak_kv), %% verify all nodes agree diff --git a/tests/verify_listkeys.erl b/tests/verify_listkeys.erl index b7cbe73f7..2a51f2063 100644 --- a/tests/verify_listkeys.erl +++ b/tests/verify_listkeys.erl @@ -57,17 +57,17 @@ confirm() -> check_it_all(Nodes, http), lager:info("Stopping Node1"), - rt:stop(Node1), + rt_node:stop(Node1), rt:wait_until_unpingable(Node1), %% Stop current node, restart previous node, verify lists:foldl(fun(Node, Prev) -> lager:info("Stopping Node ~p", [Node]), - rt:stop(Node), + rt_node:stop(Node), rt:wait_until_unpingable(Node), lager:info("Starting Node ~p", [Prev]), - rt:start(Prev), + rt_node:start(Prev), UpNodes = Nodes -- [Node], lager:info("Waiting for riak_kv service to be ready in ~p", [Prev]), rt:wait_for_cluster_service(UpNodes, riak_kv), @@ -78,11 +78,11 @@ confirm() -> end, Node1, [Node2, Node3, Node4]), lager:info("Stopping Node2"), - rt:stop(Node2), + rt_node:stop(Node2), rt:wait_until_unpingable(Node2), lager:info("Stopping Node3"), - rt:stop(Node3), + rt_node:stop(Node3), rt:wait_until_unpingable(Node3), lager:info("Only Node1 is up, so test should fail!"), diff --git a/tests/verify_membackend.erl b/tests/verify_membackend.erl index 2f05cec2c..9b81eed99 100644 --- a/tests/verify_membackend.erl +++ b/tests/verify_membackend.erl @@ -89,7 +89,7 @@ check_leave_and_expiry(NodeA, NodeB) -> ?assertEqual(ok, rt:wait_until_nodes_ready([NodeA, NodeB])), rt:wait_until_no_pending_changes([NodeA, NodeB]), - rt:leave(NodeB), + rt_node:leave(NodeB), rt:wait_until_unpingable(NodeB), ?assertEqual([], rt:systest_read(NodeA, 1, 100, ?BUCKET, 2)), diff --git a/tests/verify_mr_prereduce_node_down.erl b/tests/verify_mr_prereduce_node_down.erl index 62dd2f496..7df0ae261 100644 --- a/tests/verify_mr_prereduce_node_down.erl +++ b/tests/verify_mr_prereduce_node_down.erl @@ -47,7 +47,7 @@ confirm() -> [Primary,ToKill|_] = rt_cluster:build_cluster(NodeCount), %% We need one node down for this test - rt:stop(ToKill), + rt_node:stop(ToKill), %% store our test data Bucket = <<"verify_mr_prereduce_node_down">>, diff --git a/tests/verify_riak_lager.erl b/tests/verify_riak_lager.erl index b902048d6..a11b6a03e 100644 --- a/tests/verify_riak_lager.erl +++ b/tests/verify_riak_lager.erl @@ -33,9 +33,9 @@ confirm() -> ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), lager:info("Stopping that node"), - rt:stop(Node), + rt_node:stop(Node), - rt:start(Node), + rt_node:start(Node), lager:info("Checking for log files"), {ok, LagerHandlers} = rt:rpc_get_env(Node, [{lager, handlers}]), diff --git a/tests/verify_secondary_index_reformat.erl b/tests/verify_secondary_index_reformat.erl index 9235941c3..ced7f813e 100644 --- a/tests/verify_secondary_index_reformat.erl +++ b/tests/verify_secondary_index_reformat.erl @@ -74,7 +74,7 @@ confirm() -> lager:info("restarting node"), rt:stop_and_wait(Node), - rt:start(Node), + rt_node:start(Node), rt:wait_for_service(Node, riak_kv), check_fixed_index_statuses(Node, true), @@ -85,7 +85,7 @@ confirm() -> check_fixed_index_statuses(Node, false), rt:stop_and_wait(Node), - rt:start(Node), + rt_node:start(Node), rt:wait_for_service(Node, riak_kv), check_fixed_index_statuses(Node, false), From e1f0d456859693e593444c39845b124cbfd82277 Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Wed, 30 Jul 2014 15:52:31 -0400 Subject: [PATCH 018/157] Move rt:brutal_kill to rt_node. --- src/rt.erl | 13 ------------- src/rt_node.erl | 15 ++++++++++++++- tests/verify_handoff.erl | 4 ++-- tests/verify_vclock.erl | 2 +- 4 files changed, 17 insertions(+), 17 deletions(-) diff --git a/src/rt.erl b/src/rt.erl index 3a5bb6bcb..ed9480587 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -34,7 +34,6 @@ assert_nodes_agree_about_ownership/1, attach/2, attach_direct/2, - brutal_kill/1, capability/2, capability/3, check_singleton_node/1, @@ -565,18 +564,6 @@ is_partitioned(Node, Peers) -> AvailableNodes = rpc:call(Node, riak_core_node_watcher, nodes, [riak_kv]), lists:all(fun(Peer) -> not lists:member(Peer, AvailableNodes) end, Peers). -% when you just can't wait -brutal_kill(Node) -> -rt_cover:maybe_stop_on_node(Node), -lager:info("Killing node ~p", [Node]), -OSPidToKill = rpc:call(Node, os, getpid, []), -%% try a normal kill first, but set a timer to -%% kill -9 after 5 seconds just in case -rpc:cast(Node, timer, apply_after, - [5000, os, cmd, [io_lib:format("kill -9 ~s", [OSPidToKill])]]), -rpc:cast(Node, os, cmd, [io_lib:format("kill -15 ~s", [OSPidToKill])]), -ok. - capability(Node, all) -> rpc:call(Node, riak_core_capability, all, []); capability(Node, Capability) -> diff --git a/src/rt_node.erl b/src/rt_node.erl index 1b2c6fcc3..ee57c2d76 100644 --- a/src/rt_node.erl +++ b/src/rt_node.erl @@ -37,7 +37,8 @@ down/2, heal/1, partition/2, - remove/2]). + remove/2, + brutal_kill/1]). -define(HARNESS, (rt_config:get(rt_harness))). @@ -171,3 +172,15 @@ heal({_NewCookie, OldCookie, P1, P2}) -> rt:wait_until_connected(Cluster), {_GN, []} = rpc:sbcast(Cluster, riak_core_node_watcher, broadcast), ok. + +% when you just can't wait +brutal_kill(Node) -> + rt_cover:maybe_stop_on_node(Node), + lager:info("Killing node ~p", [Node]), + OSPidToKill = rpc:call(Node, os, getpid, []), + %% try a normal kill first, but set a timer to + %% kill -9 after 5 seconds just in case + rpc:cast(Node, timer, apply_after, + [5000, os, cmd, [io_lib:format("kill -9 ~s", [OSPidToKill])]]), + rpc:cast(Node, os, cmd, [io_lib:format("kill -15 ~s", [OSPidToKill])]), + ok. diff --git a/tests/verify_handoff.erl b/tests/verify_handoff.erl index d03ee7ce9..d036c4207 100644 --- a/tests/verify_handoff.erl +++ b/tests/verify_handoff.erl @@ -70,11 +70,11 @@ run_test(TestMode, NTestItems, NTestNodes, Encoding) -> %% Prepare for the next call to our test (we aren't polite about it, it's faster that way): lager:info("Bringing down test nodes."), - lists:foreach(fun(N) -> rt:brutal_kill(N) end, TestNodes), + lists:foreach(fun(N) -> rt_node:brutal_kill(N) end, TestNodes), %% The "root" node can't leave() since it's the only node left: lager:info("Stopping root node."), - rt:brutal_kill(RootNode). + rt_node:brutal_kill(RootNode). set_handoff_encoding(default, _) -> lager:info("Using default encoding type."), diff --git a/tests/verify_vclock.erl b/tests/verify_vclock.erl index cf7c89268..44ec38d9a 100644 --- a/tests/verify_vclock.erl +++ b/tests/verify_vclock.erl @@ -118,7 +118,7 @@ force_encoding(Node, EncodingMethod) -> end. stopall(Nodes) -> - lists:foreach(fun(N) -> rt:brutal_kill(N) end, Nodes). + lists:foreach(fun(N) -> rt_node:brutal_kill(N) end, Nodes). make_kv(N, VSuffix) -> K = <>, From 891f366bdf633434c10b9d591c738e3891d7da42 Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Wed, 30 Jul 2014 18:16:06 -0400 Subject: [PATCH 019/157] Move ring-related rt functions to rt_ring; some cleanup. --- src/rt.erl | 147 ++++--------------- src/rt_cluster.erl | 8 +- src/rt_cs_dev.erl | 2 +- src/rt_ring.erl | 114 ++++++++++++++ src/rtdev.erl | 2 +- src/rtperf.erl | 2 +- src/rtssh.erl | 2 +- tests/overload.erl | 2 +- tests/partition_repair.erl | 2 +- tests/replication/repl_aae_fullsync.erl | 2 +- tests/replication/repl_location_failures.erl | 2 +- tests/verify_bitcask_tombstone2_upgrade.erl | 2 +- tests/verify_staged_clustering.erl | 4 +- 13 files changed, 160 insertions(+), 131 deletions(-) create mode 100644 src/rt_ring.erl diff --git a/src/rt.erl b/src/rt.erl index ed9480587..775f7b8d2 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -36,9 +36,7 @@ attach_direct/2, capability/2, capability/3, - check_singleton_node/1, check_ibrowse/0, - claimant_according_to/1, cmd/1, cmd/2, connection_info/1, @@ -55,7 +53,6 @@ get_ip/1, get_node_logs/0, get_replica/5, - get_ring/1, get_version/0, is_mixed_cluster/1, is_pingable/1, @@ -97,7 +94,6 @@ spawn_cmd/1, spawn_cmd/2, search_cmd/2, - status_of_according_to/2, str/2, systest_read/2, systest_read/3, @@ -210,18 +206,18 @@ connection_info(Nodes) when is_list(Nodes) -> [ {Node, connection_info(Node)} || Node <- Nodes]. maybe_wait_for_changes(Node) -> -Ring = get_ring(Node), -Changes = riak_core_ring:pending_changes(Ring), -Joining = riak_core_ring:members(Ring, [joining]), -lager:info("maybe_wait_for_changes, changes: ~p joining: ~p", - [Changes, Joining]), -if Changes =:= [] -> - ok; -Joining =/= [] -> - ok; -true -> - ok = wait_until_no_pending_changes([Node]) -end. + Ring = rt_ring:get_ring(Node), + Changes = riak_core_ring:pending_changes(Ring), + Joining = riak_core_ring:members(Ring, [joining]), + lager:info("maybe_wait_for_changes, changes: ~p joining: ~p", + [Changes, Joining]), + if Changes =:= [] -> + ok; + Joining =/= [] -> + ok; + true -> + ok = wait_until_no_pending_changes([Node]) + end. %% @doc Spawn `Cmd' on the machine running the test harness spawn_cmd(Cmd) -> @@ -460,19 +456,19 @@ wait_until_all_members(Nodes, Nodes). %% @doc Wait until all nodes in the list `Nodes' believes all nodes in the %% list `Members' are members of the cluster. wait_until_all_members(Nodes, ExpectedMembers) -> -lager:info("Wait until all members ~p ~p", [Nodes, ExpectedMembers]), -S1 = ordsets:from_list(ExpectedMembers), -F = fun(Node) -> - case members_according_to(Node) of - {badrpc, _} -> - false; - ReportedMembers -> - S2 = ordsets:from_list(ReportedMembers), - ordsets:is_subset(S1, S2) - end -end, -[?assertEqual(ok, wait_until(Node, F)) || Node <- Nodes], -ok. + lager:info("Wait until all members ~p ~p", [Nodes, ExpectedMembers]), + S1 = ordsets:from_list(ExpectedMembers), + F = fun(Node) -> + case rt_ring:members_according_to(Node) of + {badrpc, _} -> + false; + ReportedMembers -> + S2 = ordsets:from_list(ReportedMembers), + ordsets:is_subset(S1, S2) + end + end, + [?assertEqual(ok, wait_until(Node, F)) || Node <- Nodes], + ok. %% @doc Given a list of nodes, wait until all nodes believe the ring has %% converged (ie. `riak_core_ring:is_ready' returns `true'). @@ -605,12 +601,12 @@ cap_subset(Val, Cap) when is_list(Cap) -> sets:is_subset(sets:from_list(Val), sets:from_list(Cap)). wait_until_owners_according_to(Node, Nodes) -> -SortedNodes = lists:usort(Nodes), -F = fun(N) -> -owners_according_to(N) =:= SortedNodes -end, -?assertEqual(ok, wait_until(Node, F)), -ok. + SortedNodes = lists:usort(Nodes), + F = fun(N) -> + rt_ring:owners_according_to(N) =:= SortedNodes + end, + ?assertEqual(ok, wait_until(Node, F)), + ok. wait_until_nodes_agree_about_ownership(Nodes) -> lager:info("Wait until nodes agree about ownership ~p", [Nodes]), @@ -699,87 +695,6 @@ end. %%% Ring Functions %%%=================================================================== -%% @doc Ensure that the specified node is a singleton node/cluster -- a node -%% that owns 100% of the ring. -check_singleton_node(Node) -> -lager:info("Check ~p is a singleton", [Node]), -{ok, Ring} = rpc:call(Node, riak_core_ring_manager, get_raw_ring, []), -Owners = lists:usort([Owner || {_Idx, Owner} <- riak_core_ring:all_owners(Ring)]), -?assertEqual([Node], Owners), -ok. - -% @doc Get list of partitions owned by node (primary). -partitions_for_node(Node) -> -Ring = get_ring(Node), -[Idx || {Idx, Owner} <- riak_core_ring:all_owners(Ring), Owner == Node]. - -%% @doc Get the raw ring for `Node'. -get_ring(Node) -> -{ok, Ring} = rpc:call(Node, riak_core_ring_manager, get_raw_ring, []), -Ring. - -assert_nodes_agree_about_ownership(Nodes) -> -?assertEqual(ok, wait_until_ring_converged(Nodes)), -?assertEqual(ok, wait_until_all_members(Nodes)), -[ ?assertEqual({Node, Nodes}, {Node, owners_according_to(Node)}) || Node <- Nodes]. - -%% @doc Return a list of nodes that own partitions according to the ring -%% retrieved from the specified node. -owners_according_to(Node) -> -case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of -{ok, Ring} -> - Owners = [Owner || {_Idx, Owner} <- riak_core_ring:all_owners(Ring)], - lists:usort(Owners); -{badrpc, _}=BadRpc -> - BadRpc -end. - -%% @doc Return a list of cluster members according to the ring retrieved from -%% the specified node. -members_according_to(Node) -> -case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of -{ok, Ring} -> - Members = riak_core_ring:all_members(Ring), - Members; -{badrpc, _}=BadRpc -> - BadRpc -end. - -%% @doc Return an appropriate ringsize for the node count passed -%% in. 24 is the number of cores on the bigger intel machines, but this -%% may be too large for the single-chip machines. -nearest_ringsize(Count) -> -nearest_ringsize(Count * 24, 2). - -nearest_ringsize(Count, Power) -> -case Count < trunc(Power * 0.9) of -true -> - Power; -false -> - nearest_ringsize(Count, Power * 2) -end. - -%% @doc Return the cluster status of `Member' according to the ring -%% retrieved from `Node'. -status_of_according_to(Member, Node) -> -case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of -{ok, Ring} -> - Status = riak_core_ring:member_status(Ring, Member), - Status; -{badrpc, _}=BadRpc -> - BadRpc -end. - -%% @doc Return a list of nodes that own partitions according to the ring -%% retrieved from the specified node. -claimant_according_to(Node) -> -case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of -{ok, Ring} -> - Claimant = riak_core_ring:claimant(Ring), - Claimant; -{badrpc, _}=BadRpc -> - BadRpc -end. %%%=================================================================== %%% Cluster Utility Functions diff --git a/src/rt_cluster.erl b/src/rt_cluster.erl index cab1cd92c..a3ccf4ccb 100644 --- a/src/rt_cluster.erl +++ b/src/rt_cluster.erl @@ -151,7 +151,7 @@ build_cluster(NumNodes, Versions, InitialConfig) -> join_cluster(Nodes) -> %% Ensure each node owns 100% of it's own ring - [?assertEqual([Node], rt:owners_according_to(Node)) || Node <- Nodes], + [?assertEqual([Node], rt_ring:owners_according_to(Node)) || Node <- Nodes], %% Join nodes [Node1|OtherNodes] = Nodes, @@ -162,8 +162,8 @@ join_cluster(Nodes) -> _ -> %% ok do a staged join and then commit it, this eliminates the %% large amount of redundant handoff done in a sequential join - [rt:staged_join(Node, Node1) || Node <- OtherNodes], - rt:plan_and_commit(Node1), + [rt_node:staged_join(Node, Node1) || Node <- OtherNodes], + rt_node:plan_and_commit(Node1), try_nodes_ready(Nodes, 3, 500) end, @@ -176,7 +176,7 @@ join_cluster(Nodes) -> try_nodes_ready([Node1 | _Nodes], 0, _SleepMs) -> lager:info("Nodes not ready after initial plan/commit, retrying"), - rt:plan_and_commit(Node1); + rt_node:plan_and_commit(Node1); try_nodes_ready(Nodes, N, SleepMs) -> ReadyNodes = [Node || Node <- Nodes, rt:is_ready(Node) =:= true], case ReadyNodes of diff --git a/src/rt_cs_dev.erl b/src/rt_cs_dev.erl index 10e34b344..9a36d62ce 100644 --- a/src/rt_cs_dev.erl +++ b/src/rt_cs_dev.erl @@ -249,7 +249,7 @@ deploy_nodes(NodeConfig) -> [ok = rt:wait_until_registered(N, riak_core_ring_manager) || N <- Nodes], %% Ensure nodes are singleton clusters - [ok = rt:check_singleton_node(?DEV(N)) || {N, Version} <- VersionMap, + [ok = rt_ring:check_singleton_node(?DEV(N)) || {N, Version} <- VersionMap, Version /= "0.14.2"], lager:info("Deployed nodes: ~p", [Nodes]), diff --git a/src/rt_ring.erl b/src/rt_ring.erl new file mode 100644 index 000000000..25b70eaa4 --- /dev/null +++ b/src/rt_ring.erl @@ -0,0 +1,114 @@ +%% ------------------------------------------------------------------- +%% +%% Copyright (c) 2013-2014 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(rt_ring). +-include("rt.hrl"). +-include_lib("eunit/include/eunit.hrl"). + +-export([assert_nodes_agree_about_ownership/1, + check_singleton_node/1, + claimant_according_to/1, + get_ring/1, + members_according_to/1, + nearest_ringsize/1, + owners_according_to/1, + partitions_for_node/1, + status_of_according_to/2]). + +%% @doc Ensure that the specified node is a singleton node/cluster -- a node +%% that owns 100% of the ring. +check_singleton_node(Node) -> + lager:info("Check ~p is a singleton", [Node]), + {ok, Ring} = rpc:call(Node, riak_core_ring_manager, get_raw_ring, []), + Owners = lists:usort([Owner || {_Idx, Owner} <- riak_core_ring:all_owners(Ring)]), + ?assertEqual([Node], Owners), + ok. + +% @doc Get list of partitions owned by node (primary). +partitions_for_node(Node) -> + Ring = get_ring(Node), + [Idx || {Idx, Owner} <- riak_core_ring:all_owners(Ring), Owner == Node]. + +%% @doc Get the raw ring for `Node'. +get_ring(Node) -> + {ok, Ring} = rpc:call(Node, riak_core_ring_manager, get_raw_ring, []), + Ring. + +assert_nodes_agree_about_ownership(Nodes) -> + ?assertEqual(ok, rt:wait_until_ring_converged(Nodes)), + ?assertEqual(ok, rt:wait_until_all_members(Nodes)), + [ ?assertEqual({Node, Nodes}, {Node, owners_according_to(Node)}) || Node <- Nodes]. + +%% @doc Return a list of nodes that own partitions according to the ring +%% retrieved from the specified node. +owners_according_to(Node) -> + case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of + {ok, Ring} -> + Owners = [Owner || {_Idx, Owner} <- riak_core_ring:all_owners(Ring)], + lists:usort(Owners); + {badrpc, _}=BadRpc -> + BadRpc + end. + +%% @doc Return a list of cluster members according to the ring retrieved from +%% the specified node. +members_according_to(Node) -> + case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of + {ok, Ring} -> + Members = riak_core_ring:all_members(Ring), + Members; + {badrpc, _}=BadRpc -> + BadRpc + end. + +%% @doc Return an appropriate ringsize for the node count passed +%% in. 24 is the number of cores on the bigger intel machines, but this +%% may be too large for the single-chip machines. +nearest_ringsize(Count) -> + nearest_ringsize(Count * 24, 2). + +nearest_ringsize(Count, Power) -> + case Count < trunc(Power * 0.9) of + true -> + Power; + false -> + nearest_ringsize(Count, Power * 2) + end. + +%% @doc Return the cluster status of `Member' according to the ring +%% retrieved from `Node'. +status_of_according_to(Member, Node) -> + case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of + {ok, Ring} -> + Status = riak_core_ring:member_status(Ring, Member), + Status; + {badrpc, _}=BadRpc -> + BadRpc + end. + +%% @doc Return a list of nodes that own partitions according to the ring +%% retrieved from the specified node. +claimant_according_to(Node) -> + case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of + {ok, Ring} -> + Claimant = riak_core_ring:claimant(Ring), + Claimant; + {badrpc, _}=BadRpc -> + BadRpc + end. diff --git a/src/rtdev.erl b/src/rtdev.erl index a9ed78a5f..9530dbb1d 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -433,7 +433,7 @@ deploy_nodes(NodeConfig) -> [ok = rt:wait_until_registered(N, riak_core_ring_manager) || N <- Nodes], %% Ensure nodes are singleton clusters - [ok = rt:check_singleton_node(?DEV(N)) || {N, Version} <- VersionMap, + [ok = rt_ring:check_singleton_node(?DEV(N)) || {N, Version} <- VersionMap, Version /= "0.14.2"], lager:info("Deployed nodes: ~p", [Nodes]), diff --git a/src/rtperf.erl b/src/rtperf.erl index 7b46a5360..2f132d588 100644 --- a/src/rtperf.erl +++ b/src/rtperf.erl @@ -394,7 +394,7 @@ deploy_nodes(NodeConfig, Hosts) -> [ok = rt:wait_until_registered(N, riak_core_ring_manager) || N <- Nodes], %% Ensure nodes are singleton clusters - [ok = rt:check_singleton_node(N) || {N, Version} <- VersionMap, + [ok = rt_ring:check_singleton_node(N) || {N, Version} <- VersionMap, Version /= "0.14.2"], Nodes. diff --git a/src/rtssh.erl b/src/rtssh.erl index 960bdeb98..e5605370a 100644 --- a/src/rtssh.erl +++ b/src/rtssh.erl @@ -205,7 +205,7 @@ deploy_nodes(NodeConfig, Hosts) -> [ok = rt:wait_until_registered(N, riak_core_ring_manager) || N <- Nodes], %% Ensure nodes are singleton clusters - [ok = rt:check_singleton_node(N) || {N, Version} <- VersionMap, + [ok = rt_ring:check_singleton_node(N) || {N, Version} <- VersionMap, Version /= "0.14.2"], Nodes. diff --git a/tests/overload.erl b/tests/overload.erl index 2bcb4420e..a409772c9 100644 --- a/tests/overload.erl +++ b/tests/overload.erl @@ -76,7 +76,7 @@ setup() -> Nodes = rt_cluster:build_cluster(2, Config), [_Node1, Node2] = Nodes, - Ring = rt:get_ring(Node2), + Ring = rt_ring:get_ring(Node2), Hash = riak_core_util:chash_std_keyfun({?BUCKET, ?KEY}), PL = lists:sublist(riak_core_ring:preflist(Hash, Ring), 3), Victim = hd([Idx || {Idx, Node} <- PL, diff --git a/tests/partition_repair.erl b/tests/partition_repair.erl index b3223eb3a..3d21f4c84 100644 --- a/tests/partition_repair.erl +++ b/tests/partition_repair.erl @@ -94,7 +94,7 @@ confirm() -> ?assertCmd("rm -rf " ++ base_stash_path()), %% need to load the module so riak can see the fold fun rt:load_modules_on_nodes([?MODULE], Nodes), - Ring = rt:get_ring(hd(Nodes)), + Ring = rt_ring:get_ring(hd(Nodes)), Owners = riak_core_ring:all_owners(Ring), [stash_data(riak_search, Owner) || Owner <- Owners], diff --git a/tests/replication/repl_aae_fullsync.erl b/tests/replication/repl_aae_fullsync.erl index af3cb5f9a..febf0e4ec 100644 --- a/tests/replication/repl_aae_fullsync.erl +++ b/tests/replication/repl_aae_fullsync.erl @@ -516,7 +516,7 @@ validate_intercepted_fullsync(InterceptTarget, NumIndicies = length(rpc:call(InterceptTarget, riak_core_ring, my_indices, - [rt:get_ring(InterceptTarget)])), + [rt_ring:get_ring(InterceptTarget)])), lager:info("~p owns ~p indices", [InterceptTarget, NumIndicies]), diff --git a/tests/replication/repl_location_failures.erl b/tests/replication/repl_location_failures.erl index f09e9f5f8..9793c0fe6 100644 --- a/tests/replication/repl_location_failures.erl +++ b/tests/replication/repl_location_failures.erl @@ -84,7 +84,7 @@ confirm() -> BIndicies = length(rpc:call(LeaderB, riak_core_ring, my_indices, - [rt:get_ring(LeaderB)])), + [rt_ring:get_ring(LeaderB)])), lager:warning("BIndicies: ~p", [BIndicies]), diff --git a/tests/verify_bitcask_tombstone2_upgrade.erl b/tests/verify_bitcask_tombstone2_upgrade.erl index 5a999bcc4..fe153eb7f 100644 --- a/tests/verify_bitcask_tombstone2_upgrade.erl +++ b/tests/verify_bitcask_tombstone2_upgrade.erl @@ -43,7 +43,7 @@ list_bitcask_files(Nodes) -> list_node_bitcask_files(Node) -> % Gather partitions owned, list *.bitcask.data on each. - Partitions = rt:partitions_for_node(Node), + Partitions = rt_ring:partitions_for_node(Node), {ok, DataDir} = rt:rpc_get_env(Node, [{bitcask, data_root}]), [begin IdxStr = integer_to_list(Idx), diff --git a/tests/verify_staged_clustering.erl b/tests/verify_staged_clustering.erl index 5ece38adb..f4c533bbe 100644 --- a/tests/verify_staged_clustering.erl +++ b/tests/verify_staged_clustering.erl @@ -47,7 +47,7 @@ confirm() -> lager:info("Ensure that ~p now own all partitions", [Nodes123]), ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes123)), ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes123)), - rt:assert_nodes_agree_about_ownership(Nodes123), + rt_ring:assert_nodes_agree_about_ownership(Nodes123), lager:info("Join ~p to the cluster", [Node4]), stage_join(Node4, Node1), @@ -89,7 +89,7 @@ confirm() -> lager:info("Ensure that ~p now own all partitions", [Nodes124]), ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes124)), ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes124)), - rt:assert_nodes_agree_about_ownership(Nodes124), + rt_ring:assert_nodes_agree_about_ownership(Nodes124), lager:info("Stage leave of ~p", [Node2]), stage_leave(Node1, Node2), From 80005cd626f327968dc4573d3fc0a9702508742f Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Wed, 30 Jul 2014 18:32:55 -0400 Subject: [PATCH 020/157] Move command-line oriented rt functions to rt_cmd_line. --- src/rt.erl | 60 -------------- src/rt_cmd_line.erl | 78 +++++++++++++++++++ tests/basic_command_line.erl | 33 ++++---- tests/replication/repl_fs_stat_caching.erl | 2 +- .../replication2_console_tests.erl | 2 +- tests/riak_admin_console_tests.erl | 4 +- tests/riaknostic_rt.erl | 10 +-- tests/verify_2i_aae.erl | 4 +- tests/verify_backup_restore.erl | 6 +- tests/verify_staged_clustering.erl | 16 ++-- 10 files changed, 115 insertions(+), 100 deletions(-) create mode 100644 src/rt_cmd_line.erl diff --git a/src/rt.erl b/src/rt.erl index 775f7b8d2..0e608b307 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -40,7 +40,6 @@ cmd/1, cmd/2, connection_info/1, - console/2, create_and_activate_bucket_type/3, deploy_nodes/1, deploy_nodes/2, @@ -93,7 +92,6 @@ stream_cmd/1, stream_cmd/2, spawn_cmd/1, spawn_cmd/2, - search_cmd/2, str/2, systest_read/2, systest_read/3, @@ -981,64 +979,6 @@ after ?assert(false) end. -%%%=================================================================== -%%% PBC & HTTPC Functions -%%%=================================================================== - - -%%%=================================================================== -%%% Command Line Functions -%%%=================================================================== - -%% @doc Call 'bin/riak-admin' command on `Node' with arguments `Args' -admin(Node, Args) -> -admin(Node, Args, []). - -%% @doc Call 'bin/riak-admin' command on `Node' with arguments `Args'. -%% The third parameter is a list of options. Valid options are: -%% * `return_exit_code' - Return the exit code along with the command output -admin(Node, Args, Options) -> -?HARNESS:admin(Node, Args, Options). - -%% @doc Call 'bin/riak' command on `Node' with arguments `Args' -riak(Node, Args) -> -?HARNESS:riak(Node, Args). - - -%% @doc Call 'bin/riak-repl' command on `Node' with arguments `Args' -riak_repl(Node, Args) -> -?HARNESS:riak_repl(Node, Args). - -search_cmd(Node, Args) -> -{ok, Cwd} = file:get_cwd(), -rpc:call(Node, riak_search_cmd, command, [[Cwd | Args]]). - -%% @doc Runs `riak attach' on a specific node, and tests for the expected behavoir. -%% Here's an example: ``` -%% rt:attach(Node, [{expect, "erlang.pipe.1 \(^D to exit\)"}, -%% {send, "riak_core_ring_manager:get_my_ring()."}, -%% {expect, "dict,"}, -%% {send, [4]}]), %% 4 = Ctrl + D''' -%% `{expect, String}' scans the output for the existance of the String. -%% These tuples are processed in order. -%% -%% `{send, String}' sends the string to the console. -%% Once a send is encountered, the buffer is discarded, and the next -%% expect will process based on the output following the sent data. -%% -attach(Node, Expected) -> -?HARNESS:attach(Node, Expected). - -%% @doc Runs 'riak attach-direct' on a specific node -%% @see rt:attach/2 -attach_direct(Node, Expected) -> -?HARNESS:attach_direct(Node, Expected). - -%% @doc Runs `riak console' on a specific node -%% @see rt:attach/2 -console(Node, Expected) -> -?HARNESS:console(Node, Expected). - %%%=================================================================== %%% Search %%%=================================================================== diff --git a/src/rt_cmd_line.erl b/src/rt_cmd_line.erl new file mode 100644 index 000000000..d4022f1ae --- /dev/null +++ b/src/rt_cmd_line.erl @@ -0,0 +1,78 @@ +%% ------------------------------------------------------------------- +%% +%% Copyright (c) 2014 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(rt_cmd_line). +-include_lib("eunit/include/eunit.hrl"). + +-export([admin/2, + riak/2, + riak_repl/2, + search_cmd/2, + attach/2, + attach_direct/2, + console/2 + ]). + +-include("rt.hrl"). + +-define(HARNESS, (rt_config:get(rt_harness))). + +%% @doc Call 'bin/riak-admin' command on `Node' with arguments `Args' +admin(Node, Args) -> + ?HARNESS:admin(Node, Args). + +%% @doc Call 'bin/riak' command on `Node' with arguments `Args' +riak(Node, Args) -> + ?HARNESS:riak(Node, Args). + + +%% @doc Call 'bin/riak-repl' command on `Node' with arguments `Args' +riak_repl(Node, Args) -> + ?HARNESS:riak_repl(Node, Args). + +search_cmd(Node, Args) -> + {ok, Cwd} = file:get_cwd(), + rpc:call(Node, riak_search_cmd, command, [[Cwd | Args]]). + +%% @doc Runs `riak attach' on a specific node, and tests for the expected behavoir. +%% Here's an example: ``` +%% rt_cmd_line:attach(Node, [{expect, "erlang.pipe.1 \(^D to exit\)"}, +%% {send, "riak_core_ring_manager:get_my_ring()."}, +%% {expect, "dict,"}, +%% {send, [4]}]), %% 4 = Ctrl + D''' +%% `{expect, String}' scans the output for the existance of the String. +%% These tuples are processed in order. +%% +%% `{send, String}' sends the string to the console. +%% Once a send is encountered, the buffer is discarded, and the next +%% expect will process based on the output following the sent data. +%% +attach(Node, Expected) -> + ?HARNESS:attach(Node, Expected). + +%% @doc Runs 'riak attach-direct' on a specific node +%% @see rt_cmd_line:attach/2 +attach_direct(Node, Expected) -> + ?HARNESS:attach_direct(Node, Expected). + +%% @doc Runs `riak console' on a specific node +%% @see rt_cmd_line:attach/2 +console(Node, Expected) -> + ?HARNESS:console(Node, Expected). diff --git a/tests/basic_command_line.erl b/tests/basic_command_line.erl index bc31c9414..3ac2e1bfc 100644 --- a/tests/basic_command_line.erl +++ b/tests/basic_command_line.erl @@ -56,7 +56,7 @@ confirm(#rt_properties{nodes=Nodes}, _MD) -> console_up_test(Node) -> lager:info("Node is already up, `riak console` should fail"), - {ok, ConsoleFail} = rt:riak(Node, ["console"]), + {ok, ConsoleFail} = rt_cmd_line:riak(Node, ["console"]), ?assert(rt:str(ConsoleFail, "Node is already running")), ok. @@ -65,7 +65,7 @@ console_test(Node) -> lager:info("Testing riak console on ~s", [Node]), %% Stop node, to test console working - rt:console(Node, [{expect, "\(abort with ^G\)"}, + rt_cmd_line:console(Node, [{expect, "\(abort with ^G\)"}, {send, "riak_core_ring_manager:get_my_ring()."}, {expect, "dict,"}, {send, "q()."}, @@ -76,7 +76,7 @@ console_test(Node) -> start_up_test(Node) -> %% Try starting again and check you get the node is already running message lager:info("Testing riak start now will return 'already running'"), - {ok, StartOut} = rt:riak(Node, ["start"]), + {ok, StartOut} = rt_cmd_line:riak(Node, ["start"]), ?assert(rt:str(StartOut, "Node is already running!")), ok. @@ -85,16 +85,15 @@ start_test(Node) -> %% Test starting with /bin/riak start lager:info("Testing riak start works on ~s", [Node]), - {ok, StartPass} = rt:riak(Node, ["start"]), - lager:info("StartPass: ~p", [StartPass]), - ?assert(StartPass =:= "" orelse string:str(StartPass, "WARNING") =/= 0), + {ok, StartPass} = rt_cmd_line:riak(Node, ["start"]), + ?assertMatch(StartPass, ""), rt:stop_and_wait(Node), ok. stop_test(Node) -> ?assert(rt:is_pingable(Node)), - {ok, "ok\n"} = rt:riak(Node, "stop"), + {ok, "ok\n"} = rt_cmd_line:riak(Node, "stop"), ?assertNot(rt:is_pingable(Node)), ok. @@ -107,27 +106,27 @@ ping_up_test(Node) -> %% ping / pong %% rt_node:start_and_wait(Node), lager:info("Node up, should ping"), - {ok, PongOut} = rt:riak(Node, ["ping"]), + {ok, PongOut} = rt_cmd_line:riak(Node, ["ping"]), ?assert(rt:str(PongOut, "pong")), ok. ping_down_test(Node) -> %% ping / pang lager:info("Node down, should pang"), - {ok, PangOut} = rt:riak(Node, ["ping"]), + {ok, PangOut} = rt_cmd_line:riak(Node, ["ping"]), ?assert(rt:str(PangOut, "not responding to pings")), ok. attach_down_test(Node) -> lager:info("Testing riak attach while down"), - {ok, AttachOut} = rt:riak(Node, ["attach"]), + {ok, AttachOut} = rt_cmd_line:riak(Node, ["attach"]), ?assert(rt:str(AttachOut, "Node is not running!")), ok. attach_direct_up_test(Node) -> lager:info("Testing riak attach-direct"), - rt:attach_direct(Node, [{expect, "\(^D to exit\)"}, + rt_cmd_line:attach_direct(Node, [{expect, "\(^D to exit\)"}, {send, "riak_core_ring_manager:get_my_ring()."}, {expect, "dict,"}, {send, [4]}]), %% 4 = Ctrl + D @@ -135,14 +134,14 @@ attach_direct_up_test(Node) -> attach_direct_down_test(Node) -> lager:info("Testing riak attach-direct while down"), - {ok, AttachOut} = rt:riak(Node, ["attach-direct"]), + {ok, AttachOut} = rt_cmd_line:riak(Node, ["attach-direct"]), ?assert(rt:str(AttachOut, "Node is not running!")), ok. status_up_test(Node) -> lager:info("Test riak-admin status on ~s", [Node]), - {ok, {ExitCode, StatusOut}} = rt:admin(Node, ["status"], [return_exit_code]), + {ok, StatusOut} = rt_cmd_line:admin(Node, ["status"]), io:format("Result of status: ~s", [StatusOut]), ?assertEqual(0, ExitCode), ?assert(rt:str(StatusOut, "1-minute stats")), @@ -152,20 +151,18 @@ status_up_test(Node) -> status_down_test(Node) -> lager:info("Test riak-admin status while down"), - {ok, {ExitCode, StatusOut}} = rt:admin(Node, ["status"], [return_exit_code]), - ?assertEqual(1, ExitCode), - ?assert(rt:str(StatusOut, "Node is not running!")), + {ok, StatusOut} = rt_cmd_line:admin(Node, ["status"]), ok. getpid_up_test(Node) -> lager:info("Test riak getpid on ~s", [Node]), - {ok, PidOut} = rt:riak(Node, ["getpid"]), + {ok, PidOut} = rt_cmd_line:riak(Node, ["getpid"]), ?assertNot(rt:str(PidOut, "")), ?assert(rt:str(PidOut, rpc:call(Node, os, getpid, []))), ok. getpid_down_test(Node) -> lager:info("Test riak getpid fails on ~s", [Node]), - {ok, PidOut} = rt:riak(Node, ["getpid"]), + {ok, PidOut} = rt_cmd_line:riak(Node, ["getpid"]), ?assert(rt:str(PidOut, "Node is not running!")), ok. diff --git a/tests/replication/repl_fs_stat_caching.erl b/tests/replication/repl_fs_stat_caching.erl index 6ecb655f0..ce03cd7bd 100644 --- a/tests/replication/repl_fs_stat_caching.erl +++ b/tests/replication/repl_fs_stat_caching.erl @@ -25,7 +25,7 @@ confirm() -> % status. {ok, Suspended} = suspend_an_fs_source(SrcCluster), lager:info("Suspended: ~p", [Suspended]), - {ok, Status} = rt:riak_repl(SrcLead, "status"), + {ok, Status} = rt_cmd_line:riak_repl(SrcLead, "status"), FailLine = "RPC to '" ++ atom_to_list(SrcLead) ++ "' failed: timeout\n", ?assertNotEqual(FailLine, Status), diff --git a/tests/replication/replication2_console_tests.erl b/tests/replication/replication2_console_tests.erl index c090bf9a1..5d5c55fae 100644 --- a/tests/replication/replication2_console_tests.erl +++ b/tests/replication/replication2_console_tests.erl @@ -118,6 +118,6 @@ confirm() -> check_cmd(Node, Cmd) -> lager:info("Testing riak-repl ~s on ~s", [Cmd, Node]), - {ok, Out} = rt:riak_repl(Node, [Cmd]), + {ok, Out} = rt_cmd_line:riak_repl(Node, [Cmd]), ?assertEqual("pass", Out). diff --git a/tests/riak_admin_console_tests.erl b/tests/riak_admin_console_tests.erl index 04bd8ea48..95ad3b421 100644 --- a/tests/riak_admin_console_tests.erl +++ b/tests/riak_admin_console_tests.erl @@ -233,7 +233,7 @@ confirm() -> check_admin_cmd(Node, Cmd) -> S = string:tokens(Cmd, " "), lager:info("Testing riak-admin ~s on ~s", [Cmd, Node]), - {ok, Out} = rt:admin(Node, S), + {ok, Out} = rt_cmd_line:admin(Node, S), ?assertEqual("pass", Out). %% Recently we've started calling riak_core_console twice from the @@ -242,5 +242,5 @@ check_admin_cmd(Node, Cmd) -> check_admin_cmd_2x(Node, Cmd) -> S = string:tokens(Cmd, " "), lager:info("Testing riak-admin ~s on ~s", [Cmd, Node]), - {ok, Out} = rt:admin(Node, S), + {ok, Out} = rt_cmd_line:admin(Node, S), ?assertEqual("passpass", Out). diff --git a/tests/riaknostic_rt.erl b/tests/riaknostic_rt.erl index 43fa03449..9193b3bf2 100644 --- a/tests/riaknostic_rt.erl +++ b/tests/riaknostic_rt.erl @@ -47,7 +47,7 @@ confirm() -> riaknostic_bootstrap(Node) -> lager:info("Check if riaknostic is installed"), - {ok, RiaknosticOut1} = rt:admin(Node, ["diag"]), + {ok, RiaknosticOut1} = rt_cmd_line:admin(Node, ["diag"]), riaknostic_install((rt:str(RiaknosticOut1, "is not present!")), Node). %% riaknostic is already installed, move along @@ -69,7 +69,7 @@ riaknostic_install(true, Node) -> check_riaknostic_execute(Node) -> %% Execute lager:info("** Check Riaknostic executes"), - {ok, RiaknosticOut} = rt:admin(Node, ["diag"]), + {ok, RiaknosticOut} = rt_cmd_line:admin(Node, ["diag"]), ?assertNot(rt:str(RiaknosticOut, "is not present!")), ?assertNot(rt:str(RiaknosticOut, "[debug]")), ok. @@ -78,7 +78,7 @@ check_riaknostic_execute(Node) -> check_riaknostic_usage(Node) -> %% Check usage message lager:info("** Run Riaknostic usage message"), - {ok, RiaknosticOut} = rt:admin(Node, ["diag", "--help"]), + {ok, RiaknosticOut} = rt_cmd_line:admin(Node, ["diag", "--help"]), ?assert(rt:str(RiaknosticOut, "Usage: riak-admin")), ok. @@ -86,7 +86,7 @@ check_riaknostic_usage(Node) -> check_riaknostic_command_list(Node) -> %% Check commands list lager:info("** Run Riaknostic commands list message"), - {ok, RiaknosticOut} = rt:admin(Node, ["diag", "--list"]), + {ok, RiaknosticOut} = rt_cmd_line:admin(Node, ["diag", "--list"]), ?assert(rt:str(RiaknosticOut, "Available diagnostic checks")), ?assert(rt:str(RiaknosticOut, " disk ")), ?assert(rt:str(RiaknosticOut, " dumps ")), @@ -102,7 +102,7 @@ check_riaknostic_command_list(Node) -> check_riaknostic_log_levels(Node) -> %% Check log levels lager:info("** Run Riaknostic with a different log level"), - {ok, RiaknosticOut} = rt:admin(Node, ["diag", "--level", "debug"]), + {ok, RiaknosticOut} = rt_cmd_line:admin(Node, ["diag", "--level", "debug"]), ?assert(rt:str(RiaknosticOut, "[debug]")), ok. diff --git a/tests/verify_2i_aae.erl b/tests/verify_2i_aae.erl index 5ac070c36..a2714e773 100644 --- a/tests/verify_2i_aae.erl +++ b/tests/verify_2i_aae.erl @@ -151,7 +151,7 @@ check_kill_repair(Node1) -> lager:info("Test that killing 2i repair works as desired"), spawn(fun() -> timer:sleep(1500), - rt:admin(Node1, ["repair-2i", "kill"]) + rt_cmd_line:admin(Node1, ["repair-2i", "kill"]) end), ExitStatus = run_2i_repair(Node1), case ExitStatus of @@ -168,7 +168,7 @@ check_kill_repair(Node1) -> run_2i_repair(Node1) -> lager:info("Run 2i AAE repair"), - ?assertMatch({ok, _}, rt:admin(Node1, ["repair-2i"])), + ?assertMatch({ok, _}, rt_cmd_line:admin(Node1, ["repair-2i"])), RepairPid = rpc:call(Node1, erlang, whereis, [riak_kv_2i_aae]), lager:info("Wait for repair process to finish"), Mon = monitor(process, RepairPid), diff --git a/tests/verify_backup_restore.erl b/tests/verify_backup_restore.erl index 5a94e17ff..95272ff59 100644 --- a/tests/verify_backup_restore.erl +++ b/tests/verify_backup_restore.erl @@ -82,7 +82,7 @@ confirm() -> lager:info("Backing up the data to ~p", [BackupFile]), Cookie = "riak", - rt:admin(Node0, ["backup", atom_to_list(Node0), Cookie, BackupFile, "all"]), + rt_cmd_line:admin(Node0, ["backup", atom_to_list(Node0), Cookie, BackupFile, "all"]), lager:info("Modifying data on cluster"), ModF = fun(N) -> @@ -114,7 +114,7 @@ confirm() -> verify_searches(PbcPid, Searches, 0), lager:info("Restoring from backup ~p", [BackupFile]), - rt:admin(Node0, ["restore", atom_to_list(Node0), Cookie, BackupFile]), + rt_cmd_line:admin(Node0, ["restore", atom_to_list(Node0), Cookie, BackupFile]), rt:wait_until_no_pending_changes(Nodes), %% When allow_mult=false, the mods overwrite the restored data. When @@ -150,7 +150,7 @@ confirm() -> verify_searches(PbcPid2, EmptySearches, 0), lager:info("Restoring from backup ~p again", [BackupFile]), - rt:admin(Node0, ["restore", atom_to_list(Node0), Cookie, BackupFile]), + rt_cmd_line:admin(Node0, ["restore", atom_to_list(Node0), Cookie, BackupFile]), rt:enable_search_hook(Node0, ?SEARCH_BUCKET), lager:info("Verifying data is back to original backup"), diff --git a/tests/verify_staged_clustering.erl b/tests/verify_staged_clustering.erl index f4c533bbe..3261cc935 100644 --- a/tests/verify_staged_clustering.erl +++ b/tests/verify_staged_clustering.erl @@ -121,28 +121,28 @@ n(Atom) -> stage_join(Node, OtherNode) -> %% rpc:call(Node, riak_kv_console, staged_join, [[n(OtherNode)]]). - rt:admin(Node, ["cluster", "join", n(OtherNode)]). + rt_cmd_line:admin(Node, ["cluster", "join", n(OtherNode)]). stage_leave(Node, OtherNode) -> %% rpc:call(Node, riak_core_console, stage_leave, [[n(OtherNode)]]). - rt:admin(Node, ["cluster", "leave", n(OtherNode)]). + rt_cmd_line:admin(Node, ["cluster", "leave", n(OtherNode)]). stage_remove(Node, OtherNode) -> %% rpc:call(Node, riak_core_console, stage_remove, [[n(OtherNode)]]). - rt:admin(Node, ["cluster", "force-remove", n(OtherNode)]). + rt_cmd_line:admin(Node, ["cluster", "force-remove", n(OtherNode)]). stage_replace(Node, Node1, Node2) -> %% rpc:call(Node, riak_core_console, stage_replace, [[n(Node1), n(Node2)]]). - rt:admin(Node, ["cluster", "replace", n(Node1), n(Node2)]). + rt_cmd_line:admin(Node, ["cluster", "replace", n(Node1), n(Node2)]). stage_force_replace(Node, Node1, Node2) -> %% rpc:call(Node, riak_core_console, stage_force_replace, [[n(Node1), n(Node2)]]). - rt:admin(Node, ["cluster", "force-replace", n(Node1), n(Node2)]). + rt_cmd_line:admin(Node, ["cluster", "force-replace", n(Node1), n(Node2)]). print_staged(Node) -> %% rpc:call(Node, riak_core_console, print_staged, [[]]). F = fun(_) -> - {ok, StdOut} = rt:admin(Node, ["cluster", "plan"]), + {ok, StdOut} = rt_cmd_line:admin(Node, ["cluster", "plan"]), case StdOut of "Cannot" ++ _X -> false; _ -> true @@ -152,11 +152,11 @@ print_staged(Node) -> commit_staged(Node) -> %% rpc:call(Node, riak_core_console, commit_staged, [[]]). - rt:admin(Node, ["cluster", "commit"]). + rt_cmd_line:admin(Node, ["cluster", "commit"]). clear_staged(Node) -> %% rpc:call(Node, riak_core_console, clear_staged, [[]]). - rt:admin(Node, ["cluster", "clear"]). + rt_cmd_line:admin(Node, ["cluster", "clear"]). stage_join_rpc(Node, OtherNode) -> rpc:call(Node, riak_core, staged_join, [OtherNode]). From 110657c305973b95af5c8ed8ed2d4ce07e6dc09a Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Wed, 30 Jul 2014 22:18:38 -0400 Subject: [PATCH 021/157] Move bucket types related functions from rt module to rt_bucket_types. --- src/rt.erl | 62 ------------- src/rt_bucket_types.erl | 88 +++++++++++++++++++ tests/bucket_types.erl | 8 +- tests/ensemble_basic3.erl | 2 +- tests/ensemble_basic4.erl | 2 +- tests/ensemble_interleave.erl | 2 +- tests/ensemble_remove_node2.erl | 2 +- tests/ensemble_sync.erl | 2 +- tests/ensemble_vnode_crash.erl | 2 +- tests/http_bucket_types.erl | 6 +- tests/http_security.erl | 8 +- tests/mapred_basic_compat.erl | 6 +- tests/pb_security.erl | 18 ++-- tests/replication/repl_bucket_types.erl | 18 ++-- .../repl_consistent_object_filter.erl | 12 +-- tests/verify_conditional_postcommit.erl | 4 +- tests/verify_dt_context.erl | 2 +- tests/verify_handoff.erl | 2 +- tests/verify_listkeys_eqcfsm.erl | 6 +- tests/yz_ensemble.erl | 2 +- 20 files changed, 140 insertions(+), 114 deletions(-) create mode 100644 src/rt_bucket_types.erl diff --git a/src/rt.erl b/src/rt.erl index 0e608b307..3d653e6f3 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -116,8 +116,6 @@ wait_until_aae_trees_built/1, wait_until_all_members/1, wait_until_all_members/2, - wait_until_bucket_props/3, - wait_until_bucket_type_visible/2, wait_until_capability/3, wait_until_capability/4, wait_until_connected/1, @@ -133,7 +131,6 @@ wait_until_status_ready/1, wait_until_transfers_complete/1, wait_until_unpingable/1, - wait_until_bucket_type_status/3, whats_up/0 ]). @@ -1094,65 +1091,6 @@ end. %%% Bucket Types Functions %%%=================================================================== -%% @doc create and immediately activate a bucket type -create_and_activate_bucket_type(Node, Type, Props) -> -ok = rpc:call(Node, riak_core_bucket_type, create, [Type, Props]), -wait_until_bucket_type_status(Type, ready, Node), -ok = rpc:call(Node, riak_core_bucket_type, activate, [Type]), -wait_until_bucket_type_status(Type, active, Node). - -wait_until_bucket_type_status(Type, ExpectedStatus, Nodes) when is_list(Nodes) -> -[wait_until_bucket_type_status(Type, ExpectedStatus, Node) || Node <- Nodes]; -wait_until_bucket_type_status(Type, ExpectedStatus, Node) -> -F = fun() -> - ActualStatus = rpc:call(Node, riak_core_bucket_type, status, [Type]), - ExpectedStatus =:= ActualStatus -end, -?assertEqual(ok, rt:wait_until(F)). - --spec bucket_type_visible([atom()], binary()|{binary(), binary()}) -> boolean(). -bucket_type_visible(Nodes, Type) -> -MaxTime = rt_config:get(rt_max_wait_time), -IsVisible = fun erlang:is_list/1, -{Res, NodesDown} = rpc:multicall(Nodes, riak_core_bucket_type, get, [Type], MaxTime), -NodesDown == [] andalso lists:all(IsVisible, Res). - -wait_until_bucket_type_visible(Nodes, Type) -> -F = fun() -> bucket_type_visible(Nodes, Type) end, -?assertEqual(ok, rt:wait_until(F)). - --spec see_bucket_props([atom()], binary()|{binary(), binary()}, - proplists:proplist()) -> boolean(). -see_bucket_props(Nodes, Bucket, ExpectProps) -> -MaxTime = rt_config:get(rt_max_wait_time), -IsBad = fun({badrpc, _}) -> true; - ({error, _}) -> true; - (Res) when is_list(Res) -> false - end, - HasProps = fun(ResProps) -> - lists:all(fun(P) -> lists:member(P, ResProps) end, - ExpectProps) - end, - case rpc:multicall(Nodes, riak_core_bucket, get_bucket, [Bucket], MaxTime) of - {Res, []} -> - % No nodes down, check no errors - case lists:any(IsBad, Res) of - true -> - false; - false -> - lists:all(HasProps, Res) - end; - {_, _NodesDown} -> - false - end. - -wait_until_bucket_props(Nodes, Bucket, Props) -> - F = fun() -> - see_bucket_props(Nodes, Bucket, Props) - end, - ?assertEqual(ok, rt:wait_until(F)). - - %% @doc Set up in memory log capture to check contents in a test. setup_log_capture(Nodes) when is_list(Nodes) -> rt:load_modules_on_nodes([riak_test_lager_backend], Nodes), diff --git a/src/rt_bucket_types.erl b/src/rt_bucket_types.erl new file mode 100644 index 000000000..1443283fb --- /dev/null +++ b/src/rt_bucket_types.erl @@ -0,0 +1,88 @@ +%% ------------------------------------------------------------------- +%% +%% Copyright (c) 2014 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(rt_bucket_types). +-include_lib("eunit/include/eunit.hrl"). + +-export([create_and_activate_bucket_type/3, + wait_until_bucket_type_visible/2, + wait_until_bucket_type_status/3, + wait_until_bucket_props/3]). + +-include("rt.hrl"). + +%% @doc create and immediately activate a bucket type +create_and_activate_bucket_type(Node, Type, Props) -> + ok = rpc:call(Node, riak_core_bucket_type, create, [Type, Props]), + wait_until_bucket_type_status(Type, ready, Node), + ok = rpc:call(Node, riak_core_bucket_type, activate, [Type]), + wait_until_bucket_type_status(Type, active, Node). + +wait_until_bucket_type_status(Type, ExpectedStatus, Nodes) when is_list(Nodes) -> + [wait_until_bucket_type_status(Type, ExpectedStatus, Node) || Node <- Nodes]; +wait_until_bucket_type_status(Type, ExpectedStatus, Node) -> + F = fun() -> + ActualStatus = rpc:call(Node, riak_core_bucket_type, status, [Type]), + ExpectedStatus =:= ActualStatus + end, + ?assertEqual(ok, rt:wait_until(F)). + +-spec bucket_type_visible([atom()], binary()|{binary(), binary()}) -> boolean(). +bucket_type_visible(Nodes, Type) -> + MaxTime = rt_config:get(rt_max_wait_time), + IsVisible = fun erlang:is_list/1, + {Res, NodesDown} = rpc:multicall(Nodes, riak_core_bucket_type, get, [Type], MaxTime), + NodesDown == [] andalso lists:all(IsVisible, Res). + +wait_until_bucket_type_visible(Nodes, Type) -> + F = fun() -> bucket_type_visible(Nodes, Type) end, + ?assertEqual(ok, rt:wait_until(F)). + +-spec see_bucket_props([atom()], binary()|{binary(), binary()}, + proplists:proplist()) -> boolean(). +see_bucket_props(Nodes, Bucket, ExpectProps) -> + MaxTime = rt_config:get(rt_max_wait_time), + IsBad = fun({badrpc, _}) -> true; + ({error, _}) -> true; + (Res) when is_list(Res) -> false + end, + HasProps = fun(ResProps) -> + lists:all(fun(P) -> lists:member(P, ResProps) end, + ExpectProps) + end, + case rpc:multicall(Nodes, riak_core_bucket, get_bucket, [Bucket], MaxTime) of + {Res, []} -> + % No nodes down, check no errors + case lists:any(IsBad, Res) of + true -> + false; + false -> + lists:all(HasProps, Res) + end; + {_, _NodesDown} -> + false + end. + +wait_until_bucket_props(Nodes, Bucket, Props) -> + F = fun() -> + see_bucket_props(Nodes, Bucket, Props) + end, + ?assertEqual(ok, rt:wait_until(F)). + diff --git a/tests/bucket_types.erl b/tests/bucket_types.erl index c08538188..118c96323 100644 --- a/tests/bucket_types.erl +++ b/tests/bucket_types.erl @@ -105,8 +105,8 @@ confirm(#rt_properties{nodes=Nodes}, _MD) -> lager:info("custom type get/put test"), Type = <<"mytype">>, - rt:create_and_activate_bucket_type(Node, Type, [{n_val, 3}]), - rt:wait_until_bucket_type_status(Type, active, Nodes), + rt_bucket_types:create_and_activate_bucket_type(Node, Type, [{n_val, 3}]), + rt_bucket_types:wait_until_bucket_type_status(Type, active, Nodes), lager:info("doing put"), riakc_pb_socket:put(PB, riakc_obj:new({Type, <<"bucket">>}, @@ -195,8 +195,8 @@ confirm(#rt_properties{nodes=Nodes}, _MD) -> %% make sure a newly created type is not affected either %% create a new type Type2 = <<"mynewtype">>, - rt:create_and_activate_bucket_type(Node, Type2, []), - rt:wait_until_bucket_type_status(Type2, active, Nodes), + rt_bucket_types:create_and_activate_bucket_type(Node, Type2, []), + rt_bucket_types:wait_until_bucket_type_status(Type2, active, Nodes), {ok, BProps11} = riakc_pb_socket:get_bucket_type(PB, Type2), diff --git a/tests/ensemble_basic3.erl b/tests/ensemble_basic3.erl index f6ff01ba0..2fd7e5ba7 100644 --- a/tests/ensemble_basic3.erl +++ b/tests/ensemble_basic3.erl @@ -37,7 +37,7 @@ confirm() -> ensemble_util:wait_until_stable(Node, NVal), lager:info("Creating/activating 'strong' bucket type"), - rt:create_and_activate_bucket_type(Node, <<"strong">>, + rt_bucket_types:create_and_activate_bucket_type(Node, <<"strong">>, [{consistent, true}, {n_val, NVal}]), ensemble_util:wait_until_stable(Node, NVal), Bucket = {<<"strong">>, <<"test">>}, diff --git a/tests/ensemble_basic4.erl b/tests/ensemble_basic4.erl index 2050e0eeb..9c1f0abf3 100644 --- a/tests/ensemble_basic4.erl +++ b/tests/ensemble_basic4.erl @@ -32,7 +32,7 @@ confirm() -> Node = hd(Nodes), lager:info("Creating/activating 'strong' bucket type"), - rt:create_and_activate_bucket_type(Node, <<"strong">>, + rt_bucket_types:create_and_activate_bucket_type(Node, <<"strong">>, [{consistent, true}, {n_val, NVal}]), ensemble_util:wait_until_stable(Node, NVal), Bucket = {<<"strong">>, <<"test">>}, diff --git a/tests/ensemble_interleave.erl b/tests/ensemble_interleave.erl index 5aeb78ca4..2db3da566 100644 --- a/tests/ensemble_interleave.erl +++ b/tests/ensemble_interleave.erl @@ -46,7 +46,7 @@ confirm() -> vnode_util:load(Nodes), lager:info("Creating/activating 'strong' bucket type"), - rt:create_and_activate_bucket_type(Node, <<"strong">>, + rt_bucket_types:create_and_activate_bucket_type(Node, <<"strong">>, [{consistent, true}, {n_val, NVal}]), ensemble_util:wait_until_stable(Node, NVal), Bucket = {<<"strong">>, <<"test">>}, diff --git a/tests/ensemble_remove_node2.erl b/tests/ensemble_remove_node2.erl index 4477de2c6..f5232ed05 100644 --- a/tests/ensemble_remove_node2.erl +++ b/tests/ensemble_remove_node2.erl @@ -39,7 +39,7 @@ confirm() -> {ok, _} = riak_ensemble_client:kget(Node, root, testerooni, 1000), lager:info("Creating/activating 'strong' bucket type"), - rt:create_and_activate_bucket_type(Node, <<"strong">>, + rt_bucket_types:create_and_activate_bucket_type(Node, <<"strong">>, [{consistent, true}, {n_val, NVal}]), ensemble_util:wait_until_stable(Node, NVal), Bucket = {<<"strong">>, <<"test">>}, diff --git a/tests/ensemble_sync.erl b/tests/ensemble_sync.erl index f5d0197cd..849c6a0aa 100644 --- a/tests/ensemble_sync.erl +++ b/tests/ensemble_sync.erl @@ -33,7 +33,7 @@ confirm() -> vnode_util:load(Nodes), lager:info("Creating/activating 'strong' bucket type"), - rt:create_and_activate_bucket_type(Node, <<"strong">>, + rt_bucket_types:create_and_activate_bucket_type(Node, <<"strong">>, [{consistent, true}, {n_val, NVal}]), ensemble_util:wait_until_stable(Node, NVal), diff --git a/tests/ensemble_vnode_crash.erl b/tests/ensemble_vnode_crash.erl index 7bd59bdf0..35e666ebf 100644 --- a/tests/ensemble_vnode_crash.erl +++ b/tests/ensemble_vnode_crash.erl @@ -35,7 +35,7 @@ confirm() -> ensemble_util:wait_until_stable(Node, NVal), lager:info("Creating/activating 'strong' bucket type"), - rt:create_and_activate_bucket_type(Node, <<"strong">>, + rt_bucket_types:create_and_activate_bucket_type(Node, <<"strong">>, [{consistent, true}, {n_val, NVal}]), ensemble_util:wait_until_stable(Node, NVal), Bucket = {<<"strong">>, <<"test">>}, diff --git a/tests/http_bucket_types.erl b/tests/http_bucket_types.erl index 25a466b35..668de357a 100644 --- a/tests/http_bucket_types.erl +++ b/tests/http_bucket_types.erl @@ -106,7 +106,7 @@ confirm(#rt_properties{nodes=Nodes}, _MD) -> lager:info("custom type get/put test"), %% create a new type - ok = rt:create_and_activate_bucket_type(Node, <<"mytype">>, [{n_val,3}]), + ok = rt_bucket_types:create_and_activate_bucket_type(Node, <<"mytype">>, [{n_val,3}]), %% allow cluster metadata some time to propogate timer:sleep(1000), @@ -141,7 +141,7 @@ confirm(#rt_properties{nodes=Nodes}, _MD) -> UCBBin = {UnicodeTypeBin, UnicodeBucketBin}, - ok = rt:create_and_activate_bucket_type(Node, UnicodeTypeBin, [{n_val,3}]), + ok = rt_bucket_types:create_and_activate_bucket_type(Node, UnicodeTypeBin, [{n_val,3}]), lager:info("doing put"), ok = rhc:put(RHC, riakc_obj:new(UCBBin, @@ -230,7 +230,7 @@ confirm(#rt_properties{nodes=Nodes}, _MD) -> %% make sure a newly created type is not affected either %% create a new type - ok = rt:create_and_activate_bucket_type(Node, <<"mynewtype">>, []), + ok = rt_bucket_types:create_and_activate_bucket_type(Node, <<"mynewtype">>, []), %% allow cluster metadata some time to propogate timer:sleep(1000), diff --git a/tests/http_security.erl b/tests/http_security.erl index 693764211..6134cdd64 100644 --- a/tests/http_security.erl +++ b/tests/http_security.erl @@ -246,7 +246,7 @@ confirm() -> "default", "from", Username]]), %% list keys with bucket type - rt:create_and_activate_bucket_type(Node, <<"list-keys-test">>, []), + rt_bucket_types:create_and_activate_bucket_type(Node, <<"list-keys-test">>, []), lager:info("Checking that list keys on a bucket-type is disallowed"), ?assertMatch({error, {"403", _}}, rhc:list_keys(C7, {<<"list-keys-test">>, <<"hello">>})), @@ -542,9 +542,9 @@ crdt_tests([Node|_]=Nodes, RHC) -> Types = [{<<"counters">>, counter, riakc_counter:to_op(riakc_counter:increment(5, riakc_counter:new()))}, {<<"sets">>, set, riakc_set:to_op(riakc_set:add_element(<<"foo">>, riakc_set:new()))}], [ begin - rt:create_and_activate_bucket_type(Node, BType, [{allow_mult, true}, {datatype, DType}]), - rt:wait_until_bucket_type_status(BType, active, Nodes), - rt:wait_until_bucket_type_visible(Nodes, BType) + rt_bucket_types:create_and_activate_bucket_type(Node, BType, [{allow_mult, true}, {datatype, DType}]), + rt_bucket_types:wait_until_bucket_type_status(BType, active, Nodes), + rt_bucket_types:wait_until_bucket_type_visible(Nodes, BType) end || {BType, DType, _Op} <- Types ], lager:info("Checking that CRDT fetch is denied"), diff --git a/tests/mapred_basic_compat.erl b/tests/mapred_basic_compat.erl index ee7f9b91f..35d6cba56 100644 --- a/tests/mapred_basic_compat.erl +++ b/tests/mapred_basic_compat.erl @@ -45,9 +45,9 @@ confirm() -> [Node1|_] = Nodes, %% create a new type - rt:create_and_activate_bucket_type(Node1, ?BUCKET_TYPE, [{n_val, 3}]), - rt:wait_until_bucket_type_status(?BUCKET_TYPE, active, Nodes), - rt:wait_until_bucket_type_visible(Nodes, ?BUCKET_TYPE), + rt_bucket_types:create_and_activate_bucket_type(Node1, ?BUCKET_TYPE, [{n_val, 3}]), + rt_bucket_types:wait_until_bucket_type_status(?BUCKET_TYPE, active, Nodes), + rt_bucket_types:wait_until_bucket_type_visible(Nodes, ?BUCKET_TYPE), load_test_data(Nodes), rt:load_modules_on_nodes([?MODULE], Nodes), diff --git a/tests/pb_security.erl b/tests/pb_security.erl index 543508013..ff3da2b5b 100644 --- a/tests/pb_security.erl +++ b/tests/pb_security.erl @@ -547,9 +547,9 @@ confirm() -> %%%%%%%%%%%% %% create a new type - rt:create_and_activate_bucket_type(Node, <<"mytype">>, [{n_val, 3}]), - rt:wait_until_bucket_type_status(<<"mytype">>, active, Nodes), - rt:wait_until_bucket_type_visible(Nodes, <<"mytype">>), + rt_bucket_types:create_and_activate_bucket_type(Node, <<"mytype">>, [{n_val, 3}]), + rt_bucket_types:wait_until_bucket_type_status(<<"mytype">>, active, Nodes), + rt_bucket_types:wait_until_bucket_type_visible(Nodes, <<"mytype">>), lager:info("Checking that get on a new bucket type is disallowed"), ?assertMatch({error, <<"Permission", _/binary>>}, riakc_pb_socket:get(PB, @@ -624,9 +624,9 @@ confirm() -> lager:info("Creating another bucket type"), %% create a new type - rt:create_and_activate_bucket_type(Node, <<"mytype2">>, [{allow_mult, true}]), - rt:wait_until_bucket_type_status(<<"mytype2">>, active, Nodes), - rt:wait_until_bucket_type_visible(Nodes, <<"mytype2">>), + rt_bucket_types:create_and_activate_bucket_type(Node, <<"mytype2">>, [{allow_mult, true}]), + rt_bucket_types:wait_until_bucket_type_status(<<"mytype2">>, active, Nodes), + rt_bucket_types:wait_until_bucket_type_visible(Nodes, <<"mytype2">>), lager:info("Checking that get on the new type is disallowed"), ?assertMatch({error, <<"Permission", _/binary>>}, riakc_pb_socket:get(PB, @@ -775,9 +775,9 @@ crdt_tests([Node|_]=Nodes, PB) -> {<<"sets">>, set, riakc_set:to_op(riakc_set:add_element(<<"foo">>, riakc_set:new()))}, {<<"maps">>, map, riakc_map:to_op(riakc_map:update({<<"bar">>, counter}, fun(In) -> riakc_counter:increment(In) end, riakc_map:new()))}], [ begin - rt:create_and_activate_bucket_type(Node, BType, [{allow_mult, true}, {datatype, DType}]), - rt:wait_until_bucket_type_status(BType, active, Nodes), - rt:wait_until_bucket_type_visible(Nodes, BType) + rt_bucket_types:create_and_activate_bucket_type(Node, BType, [{allow_mult, true}, {datatype, DType}]), + rt_bucket_types:wait_until_bucket_type_status(BType, active, Nodes), + rt_bucket_types:wait_until_bucket_type_visible(Nodes, BType) end || {BType, DType, _Op} <- Types ], lager:info("Checking that CRDT fetch is denied"), diff --git a/tests/replication/repl_bucket_types.erl b/tests/replication/repl_bucket_types.erl index 036b03b75..83d588deb 100644 --- a/tests/replication/repl_bucket_types.erl +++ b/tests/replication/repl_bucket_types.erl @@ -25,28 +25,28 @@ setup(Type) -> {DefinedType, UndefType} = Types = {<<"working_type">>, <<"undefined_type">>}, - rt:create_and_activate_bucket_type(LeaderA, + rt_bucket_types:create_and_activate_bucket_type(LeaderA, DefinedType, [{n_val, 3}, {allow_mult, false}]), - rt:wait_until_bucket_type_status(DefinedType, active, ANodes), - rt:wait_until_bucket_type_visible(ANodes, DefinedType), + rt_bucket_types:wait_until_bucket_type_status(DefinedType, active, ANodes), + rt_bucket_types:wait_until_bucket_type_visible(ANodes, DefinedType), case Type of current -> - rt:create_and_activate_bucket_type(LeaderB, + rt_bucket_types:create_and_activate_bucket_type(LeaderB, DefinedType, [{n_val, 3}, {allow_mult, false}]), - rt:wait_until_bucket_type_status(DefinedType, active, BNodes), - rt:wait_until_bucket_type_visible(BNodes, DefinedType); + rt_bucket_types:wait_until_bucket_type_status(DefinedType, active, BNodes), + rt_bucket_types:wait_until_bucket_type_visible(BNodes, DefinedType); mixed -> ok end, - rt:create_and_activate_bucket_type(LeaderA, + rt_bucket_types:create_and_activate_bucket_type(LeaderA, UndefType, [{n_val, 3}, {allow_mult, false}]), - rt:wait_until_bucket_type_status(UndefType, active, ANodes), - rt:wait_until_bucket_type_visible(ANodes, UndefType), + rt_bucket_types:wait_until_bucket_type_status(UndefType, active, ANodes), + rt_bucket_types:wait_until_bucket_type_visible(ANodes, UndefType), connect_clusters(LeaderA, LeaderB), {ClusterNodes, Types, PBA, PBB}. diff --git a/tests/replication/repl_consistent_object_filter.erl b/tests/replication/repl_consistent_object_filter.erl index 8766322d8..213b459c8 100644 --- a/tests/replication/repl_consistent_object_filter.erl +++ b/tests/replication/repl_consistent_object_filter.erl @@ -27,18 +27,18 @@ confirm() -> BucketType = <<"consistent_type">>, %% Create consistent bucket type on cluster A - rt:create_and_activate_bucket_type(LeaderA, + rt_bucket_types:create_and_activate_bucket_type(LeaderA, BucketType, [{consistent, true}, {n_val, 5}]), - rt:wait_until_bucket_type_status(BucketType, active, ANodes), - rt:wait_until_bucket_type_visible(ANodes, BucketType), + rt_bucket_types:wait_until_bucket_type_status(BucketType, active, ANodes), + rt_bucket_types:wait_until_bucket_type_visible(ANodes, BucketType), %% Create consistent bucket type on cluster B - rt:create_and_activate_bucket_type(LeaderB, + rt_bucket_types:create_and_activate_bucket_type(LeaderB, BucketType, [{consistent, true}, {n_val, 5}]), - rt:wait_until_bucket_type_status(BucketType, active, BNodes), - rt:wait_until_bucket_type_visible(BNodes, BucketType), + rt_bucket_types:wait_until_bucket_type_status(BucketType, active, BNodes), + rt_bucket_types:wait_until_bucket_type_visible(BNodes, BucketType), connect_clusters(LeaderA, LeaderB), diff --git a/tests/verify_conditional_postcommit.erl b/tests/verify_conditional_postcommit.erl index 24112468f..d5e4208fe 100644 --- a/tests/verify_conditional_postcommit.erl +++ b/tests/verify_conditional_postcommit.erl @@ -29,8 +29,8 @@ confirm() -> ok = rt:load_modules_on_nodes([?MODULE], Nodes), lager:info("Creating bucket types 'type1' and 'type2'"), - rt:create_and_activate_bucket_type(Node, <<"type1">>, [{magic, false}]), - rt:create_and_activate_bucket_type(Node, <<"type2">>, [{magic, true}]), + rt_bucket_types:create_and_activate_bucket_type(Node, <<"type1">>, [{magic, false}]), + rt_bucket_types:create_and_activate_bucket_type(Node, <<"type2">>, [{magic, true}]), lager:info("Installing conditional hook"), CondHook = {?MODULE, conditional_hook}, diff --git a/tests/verify_dt_context.erl b/tests/verify_dt_context.erl index 4a355962f..b4b6e11e1 100644 --- a/tests/verify_dt_context.erl +++ b/tests/verify_dt_context.erl @@ -199,7 +199,7 @@ create_pb_clients(Nodes) -> create_bucket_types([N1|_], Types) -> lager:info("Creating bucket types with datatypes: ~p", [Types]), - [rt:create_and_activate_bucket_type(N1, Name, [{datatype, Type}, {allow_mult, true}]) + [rt_bucket_types:create_and_activate_bucket_type(N1, Name, [{datatype, Type}, {allow_mult, true}]) || {Name, Type} <- Types ]. bucket_type_ready_fun(Name) -> diff --git a/tests/verify_handoff.erl b/tests/verify_handoff.erl index d036c4207..322fab087 100644 --- a/tests/verify_handoff.erl +++ b/tests/verify_handoff.erl @@ -60,7 +60,7 @@ run_test(TestMode, NTestItems, NTestNodes, Encoding) -> lager:info("Populating root node."), rt:systest_write(RootNode, NTestItems), %% write one object with a bucket type - rt:create_and_activate_bucket_type(RootNode, <<"type">>, []), + rt_bucket_types:create_and_activate_bucket_type(RootNode, <<"type">>, []), %% allow cluster metadata some time to propogate rt:systest_write(RootNode, 1, 2, {<<"type">>, <<"bucket">>}, 2), diff --git a/tests/verify_listkeys_eqcfsm.erl b/tests/verify_listkeys_eqcfsm.erl index f42206da4..d8ef9d672 100644 --- a/tests/verify_listkeys_eqcfsm.erl +++ b/tests/verify_listkeys_eqcfsm.erl @@ -183,9 +183,9 @@ setup_cluster(NumNodes) -> ?assertEqual(ok, rt:wait_until_transfers_complete(Nodes)), Node = hd(Nodes), [begin - rt:create_and_activate_bucket_type(Node, BucketType, [{n_val, NVal}]), - rt:wait_until_bucket_type_status(BucketType, active, Nodes), - rt:wait_until_bucket_type_visible(Nodes, BucketType) + rt_bucket_types:create_and_activate_bucket_type(Node, BucketType, [{n_val, NVal}]), + rt_bucket_types:wait_until_bucket_type_status(BucketType, active, Nodes), + rt_bucket_types:wait_until_bucket_type_visible(Nodes, BucketType) end || {BucketType, NVal} <- bucket_types()], Nodes. diff --git a/tests/yz_ensemble.erl b/tests/yz_ensemble.erl index c35128d6a..b0e090af6 100644 --- a/tests/yz_ensemble.erl +++ b/tests/yz_ensemble.erl @@ -26,7 +26,7 @@ confirm() -> Node = hd(Nodes), lager:info("Creating/activating 'strong' bucket type"), - rt:create_and_activate_bucket_type(Node, <<"strong">>, + rt_bucket_types:create_and_activate_bucket_type(Node, <<"strong">>, [{consistent, true}, {n_val, NVal}]), Bucket = {<<"strong">>, <<"test">>}, From 10b56a2dfa27475e9d97433dc658911f2ca58359 Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Wed, 30 Jul 2014 22:30:51 -0400 Subject: [PATCH 022/157] Move AAE related functions from rt module to rt_aae. --- src/rt.erl | 79 -------------- src/rt_aae.erl | 103 +++++++++++++++++++ tests/gh_riak_kv_765.erl | 2 +- tests/replication/repl_aae_fullsync.erl | 24 ++--- tests/replication/repl_aae_fullsync_util.erl | 4 +- tests/replication/repl_fs_bench.erl | 8 +- tests/replication/repl_util.erl | 2 +- tests/verify_2i_aae.erl | 4 +- 8 files changed, 125 insertions(+), 101 deletions(-) create mode 100644 src/rt_aae.erl diff --git a/src/rt.erl b/src/rt.erl index 3d653e6f3..3737cb44f 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -113,7 +113,6 @@ wait_until/3, wait_until/2, wait_until/1, - wait_until_aae_trees_built/1, wait_until_all_members/1, wait_until_all_members/2, wait_until_capability/3, @@ -608,84 +607,6 @@ lager:info("Wait until nodes agree about ownership ~p", [Nodes]), Results = [ wait_until_owners_according_to(Node, Nodes) || Node <- Nodes ], ?assert(lists:all(fun(X) -> ok =:= X end, Results)). -%% AAE support -wait_until_aae_trees_built(Nodes) -> -lager:info("Wait until AAE builds all partition trees across ~p", [Nodes]), -BuiltFun = fun() -> lists:foldl(aae_tree_built_fun(), true, Nodes) end, -?assertEqual(ok, wait_until(BuiltFun)), -ok. - -aae_tree_built_fun() -> -fun(Node, _AllBuilt = true) -> - case get_aae_tree_info(Node) of - {ok, TreeInfos} -> - case all_trees_have_build_times(TreeInfos) of - true -> - Partitions = [I || {I, _} <- TreeInfos], - all_aae_trees_built(Node, Partitions); - false -> - some_trees_not_built - end; - Err -> - Err - end; -(_Node, Err) -> - Err -end. - -% It is unlikely but possible to get a tree built time from compute_tree_info -% but an attempt to use the tree returns not_built. This is because the build -% process has finished, but the lock on the tree won't be released until it -% dies and the manager detects it. Yes, this is super freaking paranoid. -all_aae_trees_built(Node, Partitions) -> -%% Notice that the process locking is spawned by the -%% pmap. That's important! as it should die eventually -%% so the lock is released and the test can lock the tree. -IndexBuilts = rt:pmap(index_built_fun(Node), Partitions), -BadOnes = [R || R <- IndexBuilts, R /= true], -case BadOnes of -[] -> - true; -_ -> - BadOnes -end. - -get_aae_tree_info(Node) -> -case rpc:call(Node, riak_kv_entropy_info, compute_tree_info, []) of -{badrpc, _} -> - {error, {badrpc, Node}}; -Info -> - lager:debug("Entropy table on node ~p : ~p", [Node, Info]), - {ok, Info} -end. - -all_trees_have_build_times(Info) -> -not lists:keymember(undefined, 2, Info). - -index_built_fun(Node) -> -fun(Idx) -> - case rpc:call(Node, riak_kv_vnode, - hashtree_pid, [Idx]) of - {ok, TreePid} -> - case rpc:call(Node, riak_kv_index_hashtree, - get_lock, [TreePid, for_riak_test]) of - {badrpc, _} -> - {error, {badrpc, Node}}; - TreeLocked when TreeLocked == ok; - TreeLocked == already_locked -> - true; - Err -> - % Either not_built or some unhandled result, - % in which case update this case please! - {error, {index_not_built, Node, Idx, Err}} - end; - {error, _}=Err -> - Err; - {badrpc, _} -> - {error, {badrpc, Node}} - end -end. - %%%=================================================================== %%% Ring Functions %%%=================================================================== diff --git a/src/rt_aae.erl b/src/rt_aae.erl new file mode 100644 index 000000000..841a2f15d --- /dev/null +++ b/src/rt_aae.erl @@ -0,0 +1,103 @@ +%% ------------------------------------------------------------------- +%% +%% Copyright (c) 2014 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(rt_aae). +-include_lib("eunit/include/eunit.hrl"). + +-export([wait_until_aae_trees_built/1]). + +-include("rt.hrl"). + +wait_until_aae_trees_built(Nodes) -> + lager:info("Wait until AAE builds all partition trees across ~p", [Nodes]), + BuiltFun = fun() -> lists:foldl(aae_tree_built_fun(), true, Nodes) end, + ?assertEqual(ok, rt:wait_until(BuiltFun)), + ok. + +aae_tree_built_fun() -> + fun(Node, _AllBuilt = true) -> + case get_aae_tree_info(Node) of + {ok, TreeInfos} -> + case all_trees_have_build_times(TreeInfos) of + true -> + Partitions = [I || {I, _} <- TreeInfos], + all_aae_trees_built(Node, Partitions); + false -> + some_trees_not_built + end; + Err -> + Err + end; + (_Node, Err) -> + Err + end. + +% It is unlikely but possible to get a tree built time from compute_tree_info +% but an attempt to use the tree returns not_built. This is because the build +% process has finished, but the lock on the tree won't be released until it +% dies and the manager detects it. Yes, this is super freaking paranoid. +all_aae_trees_built(Node, Partitions) -> + %% Notice that the process locking is spawned by the + %% pmap. That's important! as it should die eventually + %% so the lock is released and the test can lock the tree. + IndexBuilts = rt:pmap(index_built_fun(Node), Partitions), + BadOnes = [R || R <- IndexBuilts, R /= true], + case BadOnes of + [] -> + true; + _ -> + BadOnes + end. + +get_aae_tree_info(Node) -> + case rpc:call(Node, riak_kv_entropy_info, compute_tree_info, []) of + {badrpc, _} -> + {error, {badrpc, Node}}; + Info -> + lager:debug("Entropy table on node ~p : ~p", [Node, Info]), + {ok, Info} + end. + +all_trees_have_build_times(Info) -> + not lists:keymember(undefined, 2, Info). + +index_built_fun(Node) -> + fun(Idx) -> + case rpc:call(Node, riak_kv_vnode, + hashtree_pid, [Idx]) of + {ok, TreePid} -> + case rpc:call(Node, riak_kv_index_hashtree, + get_lock, [TreePid, for_riak_test]) of + {badrpc, _} -> + {error, {badrpc, Node}}; + TreeLocked when TreeLocked == ok; + TreeLocked == already_locked -> + true; + Err -> + % Either not_built or some unhandled result, + % in which case update this case please! + {error, {index_not_built, Node, Idx, Err}} + end; + {error, _}=Err -> + Err; + {badrpc, _} -> + {error, {badrpc, Node}} + end + end. diff --git a/tests/gh_riak_kv_765.erl b/tests/gh_riak_kv_765.erl index 0c5c880cb..834b6c964 100644 --- a/tests/gh_riak_kv_765.erl +++ b/tests/gh_riak_kv_765.erl @@ -88,7 +88,7 @@ check_throttle_and_expiration() -> time_build(Node) -> T0 = erlang:now(), - rt:wait_until_aae_trees_built([Node]), + rt_aae:wait_until_aae_trees_built([Node]), Duration = timer:now_diff(erlang:now(), T0), lager:info("Build took ~b us", [Duration]), Duration. diff --git a/tests/replication/repl_aae_fullsync.erl b/tests/replication/repl_aae_fullsync.erl index febf0e4ec..421f3aa3d 100644 --- a/tests/replication/repl_aae_fullsync.erl +++ b/tests/replication/repl_aae_fullsync.erl @@ -90,8 +90,8 @@ simple_test() -> read_from_cluster(BFirst, 1, ?NUM_KEYS, ?NUM_KEYS), %% Wait for trees to compute. - rt:wait_until_aae_trees_built(ANodes), - rt:wait_until_aae_trees_built(BNodes), + rt_aae:wait_until_aae_trees_built(ANodes), + rt_aae:wait_until_aae_trees_built(BNodes), lager:info("Test fullsync from cluster A leader ~p to cluster B", [LeaderA]), @@ -185,9 +185,9 @@ dual_test() -> rt:wait_until_ring_converged(ANodes), %% Wait for trees to compute. - rt:wait_until_aae_trees_built(ANodes), - rt:wait_until_aae_trees_built(BNodes), - rt:wait_until_aae_trees_built(CNodes), + rt_aae:wait_until_aae_trees_built(ANodes), + rt_aae:wait_until_aae_trees_built(BNodes), + rt_aae:wait_until_aae_trees_built(CNodes), %% Flush AAE trees to disk. perform_sacrifice(AFirst), @@ -277,7 +277,7 @@ bidirectional_test() -> perform_sacrifice(AFirst), %% Wait for trees to compute. - rt:wait_until_aae_trees_built(ANodes), + rt_aae:wait_until_aae_trees_built(ANodes), %% Verify A replicated to B. validate_completed_fullsync(LeaderA, BFirst, "B", 1, ?NUM_KEYS), @@ -290,7 +290,7 @@ bidirectional_test() -> perform_sacrifice(BFirst), %% Wait for trees to compute. - rt:wait_until_aae_trees_built(BNodes), + rt_aae:wait_until_aae_trees_built(BNodes), %% Verify B replicated to A. validate_completed_fullsync(LeaderB, AFirst, "A", ?NUM_KEYS + 1, ?NUM_KEYS + ?NUM_KEYS), @@ -349,8 +349,8 @@ difference_test() -> [{timeout, 4000}]), %% Wait for trees to compute. - rt:wait_until_aae_trees_built(ANodes), - rt:wait_until_aae_trees_built(BNodes), + rt_aae:wait_until_aae_trees_built(ANodes), + rt_aae:wait_until_aae_trees_built(BNodes), lager:info("Test fullsync from cluster A leader ~p to cluster B", [LeaderA]), @@ -435,8 +435,8 @@ deadlock_test() -> [ok = rt_intercept:add(Target, Intercept) || Target <- ANodes], %% Wait for trees to compute. - rt:wait_until_aae_trees_built(ANodes), - rt:wait_until_aae_trees_built(BNodes), + rt_aae:wait_until_aae_trees_built(ANodes), + rt_aae:wait_until_aae_trees_built(BNodes), lager:info("Test fullsync from cluster A leader ~p to cluster B", [LeaderA]), @@ -574,7 +574,7 @@ validate_intercepted_fullsync(InterceptTarget, rt:wait_for_service(InterceptTarget, riak_repl), %% Wait until AAE trees are compueted on the rebooted node. - rt:wait_until_aae_trees_built([InterceptTarget]). + rt_aae:wait_until_aae_trees_built([InterceptTarget]). %% @doc Given a node, find the port that the cluster manager is %% listening on. diff --git a/tests/replication/repl_aae_fullsync_util.erl b/tests/replication/repl_aae_fullsync_util.erl index 33714c77e..6376a7e54 100644 --- a/tests/replication/repl_aae_fullsync_util.erl +++ b/tests/replication/repl_aae_fullsync_util.erl @@ -75,6 +75,6 @@ prepare_cluster_data(TestBucket, NumKeysAOnly, _NumKeysBoth, [AFirst|_] = ANodes ?assertEqual(NumKeysAOnly, length(Res2)), %% wait for the AAE trees to be built so that we don't get a not_built error - rt:wait_until_aae_trees_built(ANodes), - rt:wait_until_aae_trees_built(BNodes), + rt_aae:wait_until_aae_trees_built(ANodes), + rt_aae:wait_until_aae_trees_built(BNodes), ok. diff --git a/tests/replication/repl_fs_bench.erl b/tests/replication/repl_fs_bench.erl index a9467950b..c2b0dfc5d 100644 --- a/tests/replication/repl_fs_bench.erl +++ b/tests/replication/repl_fs_bench.erl @@ -125,27 +125,27 @@ fullsync_test(Strategy, Latency) -> ?assertEqual(ok, repl_util:wait_for_connection(LeaderA, "B")), %% Perform fullsync of an empty cluster. - rt:wait_until_aae_trees_built(ANodes ++ BNodes), + rt_aae:wait_until_aae_trees_built(ANodes ++ BNodes), {EmptyTime, _} = timer:tc(repl_util, start_and_wait_until_fullsync_complete, [LeaderA]), %% Write keys and perform fullsync. repl_util:write_to_cluster(AFirst, 0, ?FULL_NUM_KEYS, ?TEST_BUCKET), - rt:wait_until_aae_trees_built(ANodes ++ BNodes), + rt_aae:wait_until_aae_trees_built(ANodes ++ BNodes), {FullTime, _} = timer:tc(repl_util, start_and_wait_until_fullsync_complete, [LeaderA]), %% Rewrite first 10% keys and perform fullsync. repl_util:write_to_cluster(AFirst, 0, ?DIFF_NUM_KEYS, ?TEST_BUCKET), - rt:wait_until_aae_trees_built(ANodes ++ BNodes), + rt_aae:wait_until_aae_trees_built(ANodes ++ BNodes), {DiffTime, _} = timer:tc(repl_util, start_and_wait_until_fullsync_complete, [LeaderA]), %% Write no keys, and perform the fullsync. - rt:wait_until_aae_trees_built(ANodes ++ BNodes), + rt_aae:wait_until_aae_trees_built(ANodes ++ BNodes), {NoneTime, _} = timer:tc(repl_util, start_and_wait_until_fullsync_complete, [LeaderA]), diff --git a/tests/replication/repl_util.erl b/tests/replication/repl_util.erl index 700f12318..567af839e 100644 --- a/tests/replication/repl_util.erl +++ b/tests/replication/repl_util.erl @@ -634,4 +634,4 @@ validate_intercepted_fullsync(InterceptTarget, rt:wait_for_service(InterceptTarget, riak_repl), %% Wait until AAE trees are compueted on the rebooted node. - rt:wait_until_aae_trees_built([InterceptTarget]). + rt_aae:wait_until_aae_trees_built([InterceptTarget]). diff --git a/tests/verify_2i_aae.erl b/tests/verify_2i_aae.erl index a2714e773..9284a4640 100644 --- a/tests/verify_2i_aae.erl +++ b/tests/verify_2i_aae.erl @@ -68,7 +68,7 @@ check_lost_objects(Node1, PBC, NumItems, NumDel) -> ok = rpc:call(Node1, application, set_env, [riak_kv, anti_entropy, {on, [debug]}]), ok = rpc:call(Node1, riak_kv_entropy_manager, enable, []), - rt:wait_until_aae_trees_built([Node1]), + rt_aae:wait_until_aae_trees_built([Node1]), lager:info("AAE trees built, now put the rest of the data"), [put_obj(PBC, Bucket, N, N+1, Index) @@ -125,7 +125,7 @@ do_tree_rebuild(Node) -> ?assertEqual(ok, rpc:call(Node, application, set_env, [riak_kv, anti_entropy_build_limit, {100, 1000}])), - rt:wait_until_aae_trees_built([Node]), + rt_aae:wait_until_aae_trees_built([Node]), ok. %% Write objects without a 2i index. Test that running 2i repair will generate From 3e8928c1f69bd4473299c169ea73fe28699f20f4 Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Wed, 30 Jul 2014 22:53:40 -0400 Subject: [PATCH 023/157] More functions moved to rt_node. --- src/rt.erl | 28 ++------------ src/rt_cluster.erl | 6 +-- src/rt_node.erl | 37 ++++++++++++++++++- tests/ensemble_remove_node.erl | 2 +- tests/ensemble_remove_node2.erl | 2 +- tests/gh_riak_core_154.erl | 2 +- tests/gh_riak_core_176.erl | 8 ++-- tests/jmx_verify.erl | 4 +- tests/mapred_search_switch.erl | 2 +- tests/pipe_verify_handoff.erl | 2 +- tests/pipe_verify_handoff_blocking.erl | 2 +- tests/post_generate_key.erl | 2 +- tests/replication/repl_util.erl | 2 +- .../replication2_console_tests.erl | 2 +- .../replication_object_reformat.erl | 2 +- tests/riak_admin_console_tests.erl | 2 +- tests/riaknostic_rt.erl | 2 +- tests/verify_2i_limit.erl | 2 +- tests/verify_2i_mixed_cluster.erl | 2 +- tests/verify_2i_returnterms.erl | 2 +- tests/verify_2i_stream.erl | 2 +- tests/verify_2i_timeout.erl | 2 +- tests/verify_build_cluster.erl | 4 +- tests/verify_cs_bucket.erl | 2 +- tests/verify_down.erl | 6 +-- tests/verify_dynamic_ring.erl | 2 +- tests/verify_handoff.erl | 2 +- tests/verify_leave.erl | 4 +- tests/verify_listkeys.erl | 2 +- tests/verify_listkeys_eqcfsm.erl | 2 +- tests/verify_membackend.erl | 2 +- tests/verify_riak_lager.erl | 2 +- tests/verify_riak_stats.erl | 2 +- tests/verify_secondary_index_reformat.erl | 2 +- tests/verify_snmp.erl | 2 +- tests/verify_staged_clustering.erl | 6 +-- tests/verify_tick_change.erl | 2 +- 37 files changed, 86 insertions(+), 73 deletions(-) diff --git a/src/rt.erl b/src/rt.erl index 3737cb44f..06d70839b 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -119,10 +119,7 @@ wait_until_capability/4, wait_until_connected/1, wait_until_legacy_ringready/1, - wait_until_owners_according_to/2, wait_until_no_pending_changes/1, - wait_until_nodes_agree_about_ownership/1, - wait_until_nodes_ready/1, wait_until_pingable/1, wait_until_ready/1, wait_until_registered/2, @@ -310,18 +307,6 @@ is_mixed_cluster(Node) -> Nodes = rpc:call(Node, erlang, nodes, []), is_mixed_cluster(Nodes). -%% @private -is_ready(Node) -> -case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of -{ok, Ring} -> - case lists:member(Node, riak_core_ring:ready_members(Ring)) of - true -> true; - false -> {not_ready, Node} - end; -Other -> - Other -end. - %% @private is_ring_ready(Node) -> case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of @@ -367,9 +352,9 @@ end. %% states. A ready node is guaranteed to have current preflist/ownership %% information. wait_until_ready(Node) -> -lager:info("Wait until ~p ready", [Node]), -?assertEqual(ok, wait_until(Node, fun is_ready/1)), -ok. + lager:info("Wait until ~p ready", [Node]), + ?assertEqual(ok, wait_until(Node, fun rt_node:is_ready/1)), + ok. %% @doc Wait until status can be read from riak_kv_console wait_until_status_ready(Node) -> @@ -435,13 +420,6 @@ end, [?assertEqual(ok, wait_until(Node, F)) || Node <- Nodes], ok. -%% @doc Given a list of nodes, wait until all nodes are considered ready. -%% See {@link wait_until_ready/1} for definition of ready. -wait_until_nodes_ready(Nodes) -> -lager:info("Wait until nodes are ready : ~p", [Nodes]), -[?assertEqual(ok, wait_until(Node, fun is_ready/1)) || Node <- Nodes], -ok. - %% @doc Wait until all nodes in the list `Nodes' believe each other to be %% members of the cluster. wait_until_all_members(Nodes) -> diff --git a/src/rt_cluster.erl b/src/rt_cluster.erl index a3ccf4ccb..5a581bf31 100644 --- a/src/rt_cluster.erl +++ b/src/rt_cluster.erl @@ -167,10 +167,10 @@ join_cluster(Nodes) -> try_nodes_ready(Nodes, 3, 500) end, - ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), + ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes)), %% Ensure each node owns a portion of the ring - rt:wait_until_nodes_agree_about_ownership(Nodes), + rt_node:wait_until_nodes_agree_about_ownership(Nodes), ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes)), ok. @@ -178,7 +178,7 @@ try_nodes_ready([Node1 | _Nodes], 0, _SleepMs) -> lager:info("Nodes not ready after initial plan/commit, retrying"), rt_node:plan_and_commit(Node1); try_nodes_ready(Nodes, N, SleepMs) -> - ReadyNodes = [Node || Node <- Nodes, rt:is_ready(Node) =:= true], + ReadyNodes = [Node || Node <- Nodes, rt_node:is_ready(Node) =:= true], case ReadyNodes of Nodes -> ok; diff --git a/src/rt_node.erl b/src/rt_node.erl index ee57c2d76..bdce0d576 100644 --- a/src/rt_node.erl +++ b/src/rt_node.erl @@ -28,6 +28,7 @@ stop_and_wait/1, upgrade/2, upgrade/3, + is_ready/1, slow_upgrade/3, join/2, staged_join/2, @@ -38,7 +39,10 @@ heal/1, partition/2, remove/2, - brutal_kill/1]). + brutal_kill/1, + wait_until_nodes_ready/1, + wait_until_owners_according_to/2, + wait_until_nodes_agree_about_ownership/1]). -define(HARNESS, (rt_config:get(rt_harness))). @@ -184,3 +188,34 @@ brutal_kill(Node) -> [5000, os, cmd, [io_lib:format("kill -9 ~s", [OSPidToKill])]]), rpc:cast(Node, os, cmd, [io_lib:format("kill -15 ~s", [OSPidToKill])]), ok. + +%% @doc Given a list of nodes, wait until all nodes are considered ready. +%% See {@link wait_until_ready/1} for definition of ready. +wait_until_nodes_ready(Nodes) -> + lager:info("Wait until nodes are ready : ~p", [Nodes]), + [?assertEqual(ok, rt:wait_until(Node, fun is_ready/1)) || Node <- Nodes], + ok. + +is_ready(Node) -> + case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of + {ok, Ring} -> + case lists:member(Node, riak_core_ring:ready_members(Ring)) of + true -> true; + false -> {not_ready, Node} + end; + Other -> + Other + end. + +wait_until_owners_according_to(Node, Nodes) -> + SortedNodes = lists:usort(Nodes), + F = fun(N) -> + rt_ring:owners_according_to(N) =:= SortedNodes + end, + ?assertEqual(ok, rt:wait_until(Node, F)), + ok. + +wait_until_nodes_agree_about_ownership(Nodes) -> + lager:info("Wait until nodes agree about ownership ~p", [Nodes]), + Results = [ wait_until_owners_according_to(Node, Nodes) || Node <- Nodes ], + ?assert(lists:all(fun(X) -> ok =:= X end, Results)). diff --git a/tests/ensemble_remove_node.erl b/tests/ensemble_remove_node.erl index cefc0ba89..51f92d13a 100644 --- a/tests/ensemble_remove_node.erl +++ b/tests/ensemble_remove_node.erl @@ -59,7 +59,7 @@ confirm() -> rt_node:leave(Node3), ok = ensemble_util:wait_until_stable(Node, NVal), Remaining = Nodes -- [Node2, Node3], - rt:wait_until_nodes_agree_about_ownership(Remaining), + rt_node:wait_until_nodes_agree_about_ownership(Remaining), ok = rt:wait_until_unpingable(Node2), ok = rt:wait_until_unpingable(Node3), lager:info("Read value from the root ensemble"), diff --git a/tests/ensemble_remove_node2.erl b/tests/ensemble_remove_node2.erl index f5232ed05..ee82e716e 100644 --- a/tests/ensemble_remove_node2.erl +++ b/tests/ensemble_remove_node2.erl @@ -66,7 +66,7 @@ confirm() -> rt_node:leave(Node3), ok = ensemble_util:wait_until_stable(Node, NVal), Remaining = Nodes -- [Node2, Node3], - rt:wait_until_nodes_agree_about_ownership(Remaining), + rt_node:wait_until_nodes_agree_about_ownership(Remaining), %% TODO: How do we wait indefinitely for nodes to never exit here? A 30s %% sleep? diff --git a/tests/gh_riak_core_154.erl b/tests/gh_riak_core_154.erl index dfb53bfb4..37beaba37 100644 --- a/tests/gh_riak_core_154.erl +++ b/tests/gh_riak_core_154.erl @@ -29,7 +29,7 @@ confirm() -> %% Increase handoff concurrency on nodes NewConfig = [{riak_core, [{handoff_concurrency, 1024}]}], Nodes = rt_cluster:build_cluster(2, NewConfig), - ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), + ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes)), [Node1, Node2] = Nodes, lager:info("Write data while ~p is offline", [Node2]), diff --git a/tests/gh_riak_core_176.erl b/tests/gh_riak_core_176.erl index 959d14e68..df6664372 100644 --- a/tests/gh_riak_core_176.erl +++ b/tests/gh_riak_core_176.erl @@ -56,9 +56,9 @@ confirm() -> lager:info("Join ~p to the cluster and wait for handoff to finish", [Node2]), rt:join(Node2, Node1), - ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes12)), + ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes12)), ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes12)), - rt:wait_until_nodes_agree_about_ownership(Nodes12), + rt_node:wait_until_nodes_agree_about_ownership(Nodes12), %% Check 0.0.0.0 address works lager:info("Change ~p handoff_ip to \"0.0.0.0\"", [Node3]), @@ -69,9 +69,9 @@ confirm() -> [Node3]), rt:wait_for_service(Node3, riak_kv), rt:join(Node3, Node1), - ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes123)), + ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes123)), ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes123)), - rt:wait_until_nodes_agree_about_ownership(Nodes123), + rt_node:wait_until_nodes_agree_about_ownership(Nodes123), lager:info("Test gh_riak_core_176 passed"), pass. diff --git a/tests/jmx_verify.erl b/tests/jmx_verify.erl index 677973cc9..3387cd4e2 100644 --- a/tests/jmx_verify.erl +++ b/tests/jmx_verify.erl @@ -39,7 +39,7 @@ confirm() -> Config = [{riak_jmx, [{enabled, true}, {port, JMXPort}]}], Nodes = rt_cluster:deploy_nodes(1, Config), [Node1] = Nodes, - ?assertEqual(ok, rt:wait_until_nodes_ready([Node1])), + ?assertEqual(ok, rt_node:wait_until_nodes_ready([Node1])), [{http, {IP, _Port}}|_] = rt:connection_info(Node1), @@ -174,7 +174,7 @@ test_application_stop() -> Config = [{riak_jmx, [{enabled, true}, {port, JMXPort}]}], Nodes = rt_cluster:deploy_nodes(1, Config), [Node] = Nodes, - ?assertEqual(ok, rt:wait_until_nodes_ready([Node])), + ?assertEqual(ok, rt_node:wait_until_nodes_ready([Node])), %% Let's make sure the java process is alive! lager:info("checking for riak_jmx.jar running."), diff --git a/tests/mapred_search_switch.erl b/tests/mapred_search_switch.erl index adf8f0833..f0c6a89d0 100644 --- a/tests/mapred_search_switch.erl +++ b/tests/mapred_search_switch.erl @@ -69,7 +69,7 @@ setup_test_env() -> %% deploy one to make the test run faster Nodes = rt_cluster:deploy_nodes(1, [{riak_search, [{enabled, true}]}, {yokozuna, [{enabled, true}]}]), - ok = rt:wait_until_nodes_ready(Nodes), + ok = rt_node:wait_until_nodes_ready(Nodes), ok = rt:wait_for_cluster_service(Nodes, riak_search), ok = rt:wait_for_cluster_service(Nodes, yokozuna), diff --git a/tests/pipe_verify_handoff.erl b/tests/pipe_verify_handoff.erl index f0f3f2251..c6bdaaad3 100644 --- a/tests/pipe_verify_handoff.erl +++ b/tests/pipe_verify_handoff.erl @@ -109,7 +109,7 @@ confirm() -> %% Give slave a chance to start and master to notice it. rt:join(Secondary, Primary), rt:wait_until_no_pending_changes(Nodes), - rt:wait_until_nodes_agree_about_ownership(Nodes), + rt_node:wait_until_nodes_agree_about_ownership(Nodes), lager:info("Unpause workers"), Runner ! go, diff --git a/tests/pipe_verify_handoff_blocking.erl b/tests/pipe_verify_handoff_blocking.erl index 8daff0091..36bf69489 100644 --- a/tests/pipe_verify_handoff_blocking.erl +++ b/tests/pipe_verify_handoff_blocking.erl @@ -110,7 +110,7 @@ confirm() -> %% Give slave a chance to start and master to notice it. rt:join(Secondary, Primary), rt:wait_until_no_pending_changes(Nodes), - rt:wait_until_nodes_agree_about_ownership(Nodes), + rt_node:wait_until_nodes_agree_about_ownership(Nodes), lager:info("Unpause workers"), Runner ! go, diff --git a/tests/post_generate_key.erl b/tests/post_generate_key.erl index de77942b9..9ac63ab8c 100644 --- a/tests/post_generate_key.erl +++ b/tests/post_generate_key.erl @@ -26,7 +26,7 @@ confirm() -> Nodes = rt_cluster:build_cluster(1), - ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), + ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes)), [Base|_] = rt:http_url(Nodes), diff --git a/tests/replication/repl_util.erl b/tests/replication/repl_util.erl index 567af839e..251805921 100644 --- a/tests/replication/repl_util.erl +++ b/tests/replication/repl_util.erl @@ -55,7 +55,7 @@ make_cluster(Nodes) -> [First|Rest] = Nodes, - ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), + ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes)), [rt:wait_for_service(N, riak_kv) || N <- Nodes], [rt:join(Node, First) || Node <- Rest], ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes)). diff --git a/tests/replication/replication2_console_tests.erl b/tests/replication/replication2_console_tests.erl index 5d5c55fae..30c64436b 100644 --- a/tests/replication/replication2_console_tests.erl +++ b/tests/replication/replication2_console_tests.erl @@ -49,7 +49,7 @@ confirm() -> %% Deploy a node to test against lager:info("Deploy node to test riak-repl command line"), [Node] = rt_cluster:deploy_nodes(1), - ?assertEqual(ok, rt:wait_until_nodes_ready([Node])), + ?assertEqual(ok, rt_node:wait_until_nodes_ready([Node])), rt_intercept:add(Node, {riak_repl_console, [ diff --git a/tests/replication/replication_object_reformat.erl b/tests/replication/replication_object_reformat.erl index 2f29c9e2d..588a0700b 100644 --- a/tests/replication/replication_object_reformat.erl +++ b/tests/replication/replication_object_reformat.erl @@ -115,7 +115,7 @@ verify_replication(AVersion, BVersion, Start, End, Realtime) -> %% Wait until the sink cluster is in a steady state before %% starting fullsync - rt:wait_until_nodes_ready(BNodes), + rt_node:wait_until_nodes_ready(BNodes), rt:wait_until_no_pending_changes(BNodes), rt:wait_until_registered(BFirst, riak_repl2_fs_node_reserver), diff --git a/tests/riak_admin_console_tests.erl b/tests/riak_admin_console_tests.erl index 95ad3b421..d8928abea 100644 --- a/tests/riak_admin_console_tests.erl +++ b/tests/riak_admin_console_tests.erl @@ -155,7 +155,7 @@ confirm() -> %% Deploy a node to test against lager:info("Deploy node to test riak command line"), [Node] = rt_cluster:deploy_nodes(1), - ?assertEqual(ok, rt:wait_until_nodes_ready([Node])), + ?assertEqual(ok, rt_node:wait_until_nodes_ready([Node])), rt_intercept:add(Node, {riak_core_console, [ diff --git a/tests/riaknostic_rt.erl b/tests/riaknostic_rt.erl index 9193b3bf2..cbcb72285 100644 --- a/tests/riaknostic_rt.erl +++ b/tests/riaknostic_rt.erl @@ -30,7 +30,7 @@ confirm() -> %% Build a small cluster [Node1, _Node2] = rt_cluster:build_cluster(2, []), - ?assertEqual(ok, rt:wait_until_nodes_ready([Node1])), + ?assertEqual(ok, rt_node:wait_until_nodes_ready([Node1])), %% Install riaknostic for Riak versions below 1.3.0 riaknostic_bootstrap(Node1), diff --git a/tests/verify_2i_limit.erl b/tests/verify_2i_limit.erl index 6c2acdfbf..5aa551b35 100644 --- a/tests/verify_2i_limit.erl +++ b/tests/verify_2i_limit.erl @@ -32,7 +32,7 @@ confirm() -> inets:start(), Nodes = rt_cluster:build_cluster(3), - ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), + ?assertEqual(ok, (rt_node:wait_until_nodes_ready(Nodes))), RiakHttp = rt_http:httpc(hd(Nodes)), HttpUrl = rt:http_url(hd(Nodes)), diff --git a/tests/verify_2i_mixed_cluster.erl b/tests/verify_2i_mixed_cluster.erl index f0a7aa22a..2552e2b50 100644 --- a/tests/verify_2i_mixed_cluster.erl +++ b/tests/verify_2i_mixed_cluster.erl @@ -34,7 +34,7 @@ confirm() -> rt_cluster:build_cluster([{current, [{riak_kv, [{anti_entropy, {off, []}}]}]}, OldVsn, OldVsn]), - ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), + ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes)), PBC1 = rt_pb:pbc(CurrentNode), PBC2 = rt_pb:pbc(OldNode1), diff --git a/tests/verify_2i_returnterms.erl b/tests/verify_2i_returnterms.erl index 7548256e3..0f38a14cc 100644 --- a/tests/verify_2i_returnterms.erl +++ b/tests/verify_2i_returnterms.erl @@ -31,7 +31,7 @@ confirm() -> inets:start(), Nodes = rt_cluster:build_cluster(3), - ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), + ?assertEqual(ok, (rt_node:wait_until_nodes_ready(Nodes))), RiakHttp = rt:http_url(hd(Nodes)), PBPid = rt_pb:pbc(hd(Nodes)), diff --git a/tests/verify_2i_stream.erl b/tests/verify_2i_stream.erl index 8440fc30a..5bd3e6871 100644 --- a/tests/verify_2i_stream.erl +++ b/tests/verify_2i_stream.erl @@ -30,7 +30,7 @@ confirm() -> inets:start(), Nodes = rt_cluster:build_cluster(3), - ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), + ?assertEqual(ok, (rt_node:wait_until_nodes_ready(Nodes))), RiakHttp = rt:http_url(hd(Nodes)), PBPid = rt_pb:pbc(hd(Nodes)), diff --git a/tests/verify_2i_timeout.erl b/tests/verify_2i_timeout.erl index 6f05fa955..be0da7e61 100644 --- a/tests/verify_2i_timeout.erl +++ b/tests/verify_2i_timeout.erl @@ -30,7 +30,7 @@ confirm() -> inets:start(), Config = [{riak_kv, [{secondary_index_timeout, 1}]}], %% ludicrously short, should fail always Nodes = rt_cluster:build_cluster([{current, Config}, {current, Config}, {current, Config}]), - ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), + ?assertEqual(ok, (rt_node:wait_until_nodes_ready(Nodes))), PBPid = rt_pb:pbc(hd(Nodes)), Http = rt:http_url(hd(Nodes)), diff --git a/tests/verify_build_cluster.erl b/tests/verify_build_cluster.erl index 051d5c616..7f87e0290 100644 --- a/tests/verify_build_cluster.erl +++ b/tests/verify_build_cluster.erl @@ -94,11 +94,11 @@ confirm(#rt_properties{nodes=Nodes}, _MD) -> wait_and_validate(Nodes) -> wait_and_validate(Nodes, Nodes). wait_and_validate(RingNodes, UpNodes) -> lager:info("Wait until all nodes are ready and there are no pending changes"), - ?assertEqual(ok, rt:wait_until_nodes_ready(UpNodes)), + ?assertEqual(ok, rt_node:wait_until_nodes_ready(UpNodes)), ?assertEqual(ok, rt:wait_until_all_members(UpNodes)), ?assertEqual(ok, rt:wait_until_no_pending_changes(UpNodes)), lager:info("Ensure each node owns a portion of the ring"), - [rt:wait_until_owners_according_to(Node, RingNodes) || Node <- UpNodes], + [rt_node:wait_until_owners_according_to(Node, RingNodes) || Node <- UpNodes], [rt:wait_for_service(Node, riak_kv) || Node <- UpNodes], lager:info("Verify that you got much data... (this is how we do it)"), ?assertEqual([], rt:systest_read(hd(UpNodes), 0, 1000, <<"verify_build_cluster">>, 2)), diff --git a/tests/verify_cs_bucket.erl b/tests/verify_cs_bucket.erl index 4b517b043..470615a54 100644 --- a/tests/verify_cs_bucket.erl +++ b/tests/verify_cs_bucket.erl @@ -30,7 +30,7 @@ confirm() -> Nodes = rt_cluster:build_cluster(3), - ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), + ?assertEqual(ok, (rt_node:wait_until_nodes_ready(Nodes))), PBPid = rt_pb:pbc(hd(Nodes)), diff --git a/tests/verify_down.erl b/tests/verify_down.erl index 488b65cb0..2bc539849 100644 --- a/tests/verify_down.erl +++ b/tests/verify_down.erl @@ -29,7 +29,7 @@ confirm() -> %% Join node2 to node1 and wait for cluster convergence lager:info("Join ~p to ~p", [Node2, Node1]), rt:join(Node2, Node1), - ?assertEqual(ok, rt:wait_until_nodes_ready([Node1, Node2])), + ?assertEqual(ok, rt_node:wait_until_nodes_ready([Node1, Node2])), ?assertEqual(ok, rt:wait_until_no_pending_changes([Node1, Node2])), %% Shutdown node2 @@ -59,10 +59,10 @@ confirm() -> %% Restart node2 and wait for ring convergence lager:info("Restart ~p and wait for ring convergence", [Node2]), rt_node:start(Node2), - ?assertEqual(ok, rt:wait_until_nodes_ready([Node2])), + ?assertEqual(ok, rt_node:wait_until_nodes_ready([Node2])), ?assertEqual(ok, rt:wait_until_ring_converged(Nodes)), %% Verify that all three nodes are ready lager:info("Ensure all nodes are ready"), - ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), + ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes)), pass. diff --git a/tests/verify_dynamic_ring.erl b/tests/verify_dynamic_ring.erl index 0e9b9d072..19cfdbda5 100644 --- a/tests/verify_dynamic_ring.erl +++ b/tests/verify_dynamic_ring.erl @@ -41,7 +41,7 @@ confirm() -> %% NewNodes = [ANode, YetAnother, ReplacingNode], rt:join(AnotherNode, ANode), rt:join(YetAnother, ANode), - rt:wait_until_nodes_agree_about_ownership(Nodes), + rt_node:wait_until_nodes_agree_about_ownership(Nodes), rt:wait_until_ring_converged(Nodes), rt:wait_until_no_pending_changes(Nodes), diff --git a/tests/verify_handoff.erl b/tests/verify_handoff.erl index 322fab087..c87ca6b9c 100644 --- a/tests/verify_handoff.erl +++ b/tests/verify_handoff.erl @@ -112,7 +112,7 @@ test_handoff(RootNode, NewNode, NTestItems) -> lager:info("Joining new node with cluster."), rt:join(NewNode, RootNode), - ?assertEqual(ok, rt:wait_until_nodes_ready([RootNode, NewNode])), + ?assertEqual(ok, rt_node:wait_until_nodes_ready([RootNode, NewNode])), rt:wait_until_no_pending_changes([RootNode, NewNode]), %% See if we get the same data back from the joined node that we added to the root node. diff --git a/tests/verify_leave.erl b/tests/verify_leave.erl index 9bf079f38..52690241e 100644 --- a/tests/verify_leave.erl +++ b/tests/verify_leave.erl @@ -42,7 +42,7 @@ confirm() -> lager:info("Verify ~p no longer owns partitions and all nodes believe " "it is invalid", [Node2]), Remaining1 = Nodes -- [Node2], - rt:wait_until_nodes_agree_about_ownership(Remaining1), + rt_node:wait_until_nodes_agree_about_ownership(Remaining1), [?assertEqual(invalid, status_of_according_to(Node2, Node)) || Node <- Remaining1], %% Have node1 remove node3 @@ -54,6 +54,6 @@ confirm() -> lager:info("Verify ~p no longer owns partitions, and all nodes believe " "it is invalid", [Node3]), Remaining2 = Remaining1 -- [Node3], - rt:wait_until_nodes_agree_about_ownership(Remaining2), + rt_node:wait_until_nodes_agree_about_ownership(Remaining2), [?assertEqual(invalid, status_of_according_to(Node3, Node)) || Node <- Remaining2], pass. diff --git a/tests/verify_listkeys.erl b/tests/verify_listkeys.erl index 2a51f2063..d4edd58e5 100644 --- a/tests/verify_listkeys.erl +++ b/tests/verify_listkeys.erl @@ -30,7 +30,7 @@ confirm() -> [Node1, Node2, Node3, Node4] = Nodes = rt_cluster:deploy_nodes(4), - ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), + ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes)), lager:info("Nodes deployed, but not joined."), diff --git a/tests/verify_listkeys_eqcfsm.erl b/tests/verify_listkeys_eqcfsm.erl index d8ef9d672..d7737f338 100644 --- a/tests/verify_listkeys_eqcfsm.erl +++ b/tests/verify_listkeys_eqcfsm.erl @@ -179,7 +179,7 @@ log_transition(S) -> %% ==================================================================== setup_cluster(NumNodes) -> Nodes = rt_cluster:build_cluster(NumNodes), - ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), + ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes)), ?assertEqual(ok, rt:wait_until_transfers_complete(Nodes)), Node = hd(Nodes), [begin diff --git a/tests/verify_membackend.erl b/tests/verify_membackend.erl index 9b81eed99..ded71c846 100644 --- a/tests/verify_membackend.erl +++ b/tests/verify_membackend.erl @@ -86,7 +86,7 @@ check_leave_and_expiry(NodeA, NodeB) -> rt:join(NodeB, NodeA), - ?assertEqual(ok, rt:wait_until_nodes_ready([NodeA, NodeB])), + ?assertEqual(ok, rt_node:wait_until_nodes_ready([NodeA, NodeB])), rt:wait_until_no_pending_changes([NodeA, NodeB]), rt_node:leave(NodeB), diff --git a/tests/verify_riak_lager.erl b/tests/verify_riak_lager.erl index a11b6a03e..72f5b473e 100644 --- a/tests/verify_riak_lager.erl +++ b/tests/verify_riak_lager.erl @@ -30,7 +30,7 @@ confirm() -> lager:info("Staring a node"), Nodes = [Node] = rt_cluster:deploy_nodes(1), - ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), + ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes)), lager:info("Stopping that node"), rt_node:stop(Node), diff --git a/tests/verify_riak_stats.erl b/tests/verify_riak_stats.erl index 1c5c83cbd..dbc2aff98 100644 --- a/tests/verify_riak_stats.erl +++ b/tests/verify_riak_stats.erl @@ -38,7 +38,7 @@ confirm() -> Nodes = rt_cluster:deploy_nodes(1), [Node1] = Nodes, verify_dt_converge:create_bucket_types(Nodes, ?TYPES), - ?assertEqual(ok, rt:wait_until_nodes_ready([Node1])), + ?assertEqual(ok, rt_node:wait_until_nodes_ready([Node1])), Stats1 = get_stats(Node1), lager:info("Verifying that all expected stats keys are present from the HTTP endpoint"), diff --git a/tests/verify_secondary_index_reformat.erl b/tests/verify_secondary_index_reformat.erl index ced7f813e..99a3d9fce 100644 --- a/tests/verify_secondary_index_reformat.erl +++ b/tests/verify_secondary_index_reformat.erl @@ -25,7 +25,7 @@ confirm() -> [Node] = rt_cluster:build_cluster([legacy]), - rt:wait_until_nodes_ready([Node]), + rt_node:wait_until_nodes_ready([Node]), check_fixed_index_statuses(Node, undefined), diff --git a/tests/verify_snmp.erl b/tests/verify_snmp.erl index 4e3183a0c..7cbc1c7c0 100644 --- a/tests/verify_snmp.erl +++ b/tests/verify_snmp.erl @@ -29,7 +29,7 @@ confirm() -> %% Bring up a small cluster Config = [{riak_snmp, [{polling_interval, 1000}]}], [Node1] = rt_cluster:deploy_nodes(1), - ?assertEqual(ok, rt:wait_until_nodes_ready([Node1])), + ?assertEqual(ok, rt_node:wait_until_nodes_ready([Node1])), Keys = [{vnodeGets,<<"vnode_gets">>}, {vnodePuts,<<"vnode_puts">>}, diff --git a/tests/verify_staged_clustering.erl b/tests/verify_staged_clustering.erl index 3261cc935..7970c6d79 100644 --- a/tests/verify_staged_clustering.erl +++ b/tests/verify_staged_clustering.erl @@ -45,7 +45,7 @@ confirm() -> commit_staged(Node1), lager:info("Ensure that ~p now own all partitions", [Nodes123]), - ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes123)), + ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes123)), ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes123)), rt_ring:assert_nodes_agree_about_ownership(Nodes123), @@ -62,7 +62,7 @@ confirm() -> Nodes134 = [Node1, Node3, Node4], lager:info("Ensure that ~p now own all partitions", [Nodes134]), - ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes134)), + ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes134)), ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes134)), rt:assert_nodes_agree_about_ownership(Nodes134), @@ -87,7 +87,7 @@ confirm() -> Nodes124 = [Node1, Node2, Node4], lager:info("Ensure that ~p now own all partitions", [Nodes124]), - ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes124)), + ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes124)), ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes124)), rt_ring:assert_nodes_agree_about_ownership(Nodes124), diff --git a/tests/verify_tick_change.erl b/tests/verify_tick_change.erl index cf8c12838..32f6d9040 100644 --- a/tests/verify_tick_change.erl +++ b/tests/verify_tick_change.erl @@ -28,7 +28,7 @@ confirm() -> rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), NewConfig = [], Nodes = rt_cluster:build_cluster(ClusterSize, NewConfig), - ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), + ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes)), [Node1|_] = Nodes, Bucket = <<"systest">>, Start = 0, End = 100, From 614661ac0d7e39b005749ab5bd0db6f32e102533 Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Wed, 30 Jul 2014 23:03:40 -0400 Subject: [PATCH 024/157] Move rt:is_pingable to rt_node. --- src/rt.erl | 5 ----- src/rt_cluster.erl | 2 +- src/rt_node.erl | 7 ++++++- tests/basic_command_line.erl | 6 +++--- tests/ensemble_basic2.erl | 2 +- tests/gh_riak_core_155.erl | 2 +- tests/pr_pw.erl | 4 ++-- tests/replication/repl_aae_fullsync.erl | 2 +- tests/replication/repl_util.erl | 2 +- tests/riak_control_authentication.erl | 2 +- tests/verify_secondary_index_reformat.erl | 4 ++-- 11 files changed, 19 insertions(+), 19 deletions(-) diff --git a/src/rt.erl b/src/rt.erl index 06d70839b..d9e043c09 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -54,7 +54,6 @@ get_replica/5, get_version/0, is_mixed_cluster/1, - is_pingable/1, load_modules_on_nodes/2, log_to_nodes/2, log_to_nodes/3, @@ -295,10 +294,6 @@ load_modules_on_nodes(MoreModules, Nodes). %%% Status / Wait Functions %%%=================================================================== -%% @doc Is the `Node' up according to net_adm:ping -is_pingable(Node) -> -net_adm:ping(Node) =:= pong. - is_mixed_cluster(Nodes) when is_list(Nodes) -> %% If the nodes are bad, we don't care what version they are {Versions, _BadNodes} = rpc:multicall(Nodes, init, script_id, [], rt_config:get(rt_max_wait_time)), diff --git a/src/rt_cluster.erl b/src/rt_cluster.erl index 5a581bf31..26470e122 100644 --- a/src/rt_cluster.erl +++ b/src/rt_cluster.erl @@ -189,7 +189,7 @@ try_nodes_ready(Nodes, N, SleepMs) -> %% @doc Stop nodes and wipe out their data directories clean_cluster(Nodes) when is_list(Nodes) -> - [rt:stop_and_wait(Node) || Node <- Nodes], + [rt_node:stop_and_wait(Node) || Node <- Nodes], clean_data_dir(Nodes). clean_data_dir(Nodes) -> diff --git a/src/rt_node.erl b/src/rt_node.erl index bdce0d576..21b204035 100644 --- a/src/rt_node.erl +++ b/src/rt_node.erl @@ -42,7 +42,8 @@ brutal_kill/1, wait_until_nodes_ready/1, wait_until_owners_according_to/2, - wait_until_nodes_agree_about_ownership/1]). + wait_until_nodes_agree_about_ownership/1, + is_pingable/1]). -define(HARNESS, (rt_config:get(rt_harness))). @@ -219,3 +220,7 @@ wait_until_nodes_agree_about_ownership(Nodes) -> lager:info("Wait until nodes agree about ownership ~p", [Nodes]), Results = [ wait_until_owners_according_to(Node, Nodes) || Node <- Nodes ], ?assert(lists:all(fun(X) -> ok =:= X end, Results)). + +%% @doc Is the `Node' up according to net_adm:ping +is_pingable(Node) -> + net_adm:ping(Node) =:= pong. diff --git a/tests/basic_command_line.erl b/tests/basic_command_line.erl index 3ac2e1bfc..c47998586 100644 --- a/tests/basic_command_line.erl +++ b/tests/basic_command_line.erl @@ -87,15 +87,15 @@ start_test(Node) -> {ok, StartPass} = rt_cmd_line:riak(Node, ["start"]), ?assertMatch(StartPass, ""), - rt:stop_and_wait(Node), + rt_node:stop_and_wait(Node), ok. stop_test(Node) -> - ?assert(rt:is_pingable(Node)), + ?assert(rt_node:is_pingable(Node)), {ok, "ok\n"} = rt_cmd_line:riak(Node, "stop"), - ?assertNot(rt:is_pingable(Node)), + ?assertNot(rt_node:is_pingable(Node)), ok. ping_up_test(Node) -> diff --git a/tests/ensemble_basic2.erl b/tests/ensemble_basic2.erl index 6af2f2796..77ec73c9b 100644 --- a/tests/ensemble_basic2.erl +++ b/tests/ensemble_basic2.erl @@ -42,7 +42,7 @@ confirm() -> timer:sleep(Delay), riak_kv_vnode_orig:init_orig(Args) end}}]}), - rt:stop_and_wait(Node), + rt_node:stop_and_wait(Node), rt_node:start(Node), lager:info("Polling peers while riak_kv starts. We should see none"), UpNoPeersFun = diff --git a/tests/gh_riak_core_155.erl b/tests/gh_riak_core_155.erl index 1e6ffd604..522e088f2 100644 --- a/tests/gh_riak_core_155.erl +++ b/tests/gh_riak_core_155.erl @@ -38,7 +38,7 @@ confirm() -> %% Restart node, add intercept that delay proxy startup, and issue gets. %% Gets will come in before proxies started, and should trigger crash. - rt:stop_and_wait(Node), + rt_node:stop_and_wait(Node), rt_node:async_start(Node), rt:wait_until_pingable(Node), rt_intercept:load_intercepts([Node]), diff --git a/tests/pr_pw.erl b/tests/pr_pw.erl index 5cecf6e6d..5e1e5c802 100644 --- a/tests/pr_pw.erl +++ b/tests/pr_pw.erl @@ -68,7 +68,7 @@ confirm() -> ?assertMatch({ok, _}, C:get(<<"foo">>, <<"bar">>, [{pr, quorum}])), - rt:stop_and_wait(Node), + rt_node:stop_and_wait(Node), %% there's now a fallback in the preflist, so PR/PW won't be satisfied %% anymore @@ -110,7 +110,7 @@ confirm() -> ?assertEqual({error, timeout}, C:put(Obj, [{pw, all}])), %% reboot the node - rt:stop_and_wait(Node2), + rt_node:stop_and_wait(Node2), rt_node:start_and_wait(Node2), rt:wait_for_service(Node2, riak_kv), diff --git a/tests/replication/repl_aae_fullsync.erl b/tests/replication/repl_aae_fullsync.erl index 421f3aa3d..f72ed3c17 100644 --- a/tests/replication/repl_aae_fullsync.erl +++ b/tests/replication/repl_aae_fullsync.erl @@ -566,7 +566,7 @@ validate_intercepted_fullsync(InterceptTarget, NumIndicies), %% Reboot node. - rt:stop_and_wait(InterceptTarget), + rt_node:stop_and_wait(InterceptTarget), rt_node:start_and_wait(InterceptTarget), %% Wait for riak_kv and riak_repl to initialize. diff --git a/tests/replication/repl_util.erl b/tests/replication/repl_util.erl index 251805921..eaa4271bd 100644 --- a/tests/replication/repl_util.erl +++ b/tests/replication/repl_util.erl @@ -626,7 +626,7 @@ validate_intercepted_fullsync(InterceptTarget, NumIndicies), %% Reboot node. - rt:stop_and_wait(InterceptTarget), + rt_node:stop_and_wait(InterceptTarget), rt_node:start_and_wait(InterceptTarget), %% Wait for riak_kv and riak_repl to initialize. diff --git a/tests/riak_control_authentication.erl b/tests/riak_control_authentication.erl index 3e8db3b00..88b495aca 100644 --- a/tests/riak_control_authentication.erl +++ b/tests/riak_control_authentication.erl @@ -217,7 +217,7 @@ build_singleton_cluster(Vsn, Config) -> %% the supervisor starts, we need to restart to ensure settings %% take effect. Node = lists:nth(1, Nodes), - rt:stop_and_wait(Node), + rt_node:stop_and_wait(Node), rt_node:start_and_wait(Node), rt:wait_for_service(Node, riak_kv), diff --git a/tests/verify_secondary_index_reformat.erl b/tests/verify_secondary_index_reformat.erl index 99a3d9fce..9ea33f1a2 100644 --- a/tests/verify_secondary_index_reformat.erl +++ b/tests/verify_secondary_index_reformat.erl @@ -73,7 +73,7 @@ confirm() -> rt:systest_write(Node, 10, 1), lager:info("restarting node"), - rt:stop_and_wait(Node), + rt_node:stop_and_wait(Node), rt_node:start(Node), rt:wait_for_service(Node, riak_kv), @@ -84,7 +84,7 @@ confirm() -> check_fixed_index_statuses(Node, false), - rt:stop_and_wait(Node), + rt_node:stop_and_wait(Node), rt_node:start(Node), rt:wait_for_service(Node, riak_kv), check_fixed_index_statuses(Node, false), From 82c3564dbeb60c1ee17cd55fe76b6708ae838c1a Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Thu, 31 Jul 2014 11:52:14 -0400 Subject: [PATCH 025/157] Move systest read/write functions from rt module to rt_systest. --- src/riak_test_runner.erl | 2 +- src/rt.erl | 3 +- src/rt_systest.erl | 179 ++++++++++++++++++ tests/gh_riak_core_154.erl | 6 +- tests/gh_riak_core_176.erl | 6 +- tests/gh_riak_kv_765.erl | 2 +- tests/jmx_verify.erl | 2 +- tests/loaded_upgrade.erl | 4 +- tests/pipe_verify_handoff.erl | 2 +- tests/pipe_verify_handoff_blocking.erl | 2 +- tests/replication/repl_aae_fullsync.erl | 2 +- tests/replication/repl_aae_fullsync_util.erl | 2 +- tests/replication/repl_rt_cascading_rtq.erl | 2 +- tests/replication/repl_rt_overload.erl | 2 +- tests/replication/repl_util.erl | 12 +- tests/replication/replication.erl | 22 +-- tests/replication/replication2.erl | 2 +- tests/replication/replication2_pg.erl | 4 +- .../replication2_rt_sink_connection.erl | 2 +- tests/verify_basic_upgrade.erl | 8 +- tests/verify_build_cluster.erl | 10 +- tests/verify_capabilities.erl | 6 +- tests/verify_down.erl | 4 +- tests/verify_dynamic_ring.erl | 20 +- tests/verify_handoff.erl | 12 +- tests/verify_handoff_mixed.erl | 2 +- tests/verify_listkeys.erl | 2 +- tests/verify_membackend.erl | 18 +- tests/verify_mr_prereduce_node_down.erl | 2 +- tests/verify_riak_object_reformat.erl | 6 +- tests/verify_riak_stats.erl | 2 +- tests/verify_secondary_index_reformat.erl | 2 +- tests/verify_snmp.erl | 4 +- tests/verify_tick_change.erl | 4 +- 34 files changed, 270 insertions(+), 90 deletions(-) create mode 100644 src/rt_systest.erl diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index b245d8cee..54d39007f 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -159,7 +159,7 @@ compose_confirm_fun({ConfirmMod, ConfirmFun}, InitialResult = ConfirmMod:ConfirmFun(SetupData, MetaData), OtherResults = [begin ensure_all_nodes_running(Nodes), - _ = rt:upgrade(Node, UpgradeVersion), + _ = rt_node:upgrade(Node, UpgradeVersion), _ = rt_cluster:maybe_wait_for_transfers(Nodes, WaitForTransfers), ConfirmMod:ConfirmFun(SetupData, MetaData) end || Node <- Nodes], diff --git a/src/rt.erl b/src/rt.erl index d9e043c09..2a4509237 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -88,7 +88,8 @@ set_advanced_conf/2, setup_harness/2, setup_log_capture/1, - stream_cmd/1, stream_cmd/2, + stream_cmd/1, + stream_cmd/2, spawn_cmd/1, spawn_cmd/2, str/2, diff --git a/src/rt_systest.erl b/src/rt_systest.erl new file mode 100644 index 000000000..918496bb0 --- /dev/null +++ b/src/rt_systest.erl @@ -0,0 +1,179 @@ +%% ------------------------------------------------------------------- +%% +%% Copyright (c) 2013-2014 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(rt_systest). +-include("rt.hrl"). +-include_lib("eunit/include/eunit.hrl"). + +-export([read/2, + read/3, + read/5, + read/6, + write/2, + write/3, + write/5, + write/6, + verify_systest_value/4]). + +write(Node, Size) -> + write(Node, Size, 2). + +write(Node, Size, W) -> + write(Node, 1, Size, <<"systest">>, W). + +write(Node, Start, End, Bucket, W) -> + write(Node, Start, End, Bucket, W, <<>>). + +%% @doc Write (End-Start)+1 objects to Node. Objects keys will be +%% `Start', `Start+1' ... `End', each encoded as a 32-bit binary +%% (`<>'). Object values are the same as their keys. +%% +%% The return value of this function is a list of errors +%% encountered. If all writes were successful, return value is an +%% empty list. Each error has the form `{N :: integer(), Error :: term()}', +%% where N is the unencoded key of the object that failed to store. +write(Node, Start, End, Bucket, W, CommonValBin) + when is_binary(CommonValBin) -> + rt:wait_for_service(Node, riak_kv), + {ok, C} = riak:client_connect(Node), + F = fun(N, Acc) -> + Obj = riak_object:new(Bucket, <>, + <>), + try C:put(Obj, W) of + ok -> + Acc; + Other -> + [{N, Other} | Acc] + catch + What:Why -> + [{N, {What, Why}} | Acc] + end + end, + lists:foldl(F, [], lists:seq(Start, End)). + +read(Node, Size) -> + read(Node, Size, 2). + +read(Node, Size, R) -> + read(Node, 1, Size, <<"systest">>, R). + +read(Node, Start, End, Bucket, R) -> + read(Node, Start, End, Bucket, R, <<>>). + +read(Node, Start, End, Bucket, R, CommonValBin) + when is_binary(CommonValBin) -> + read(Node, Start, End, Bucket, R, CommonValBin, false). + +%% Read and verify the values of objects written with +%% `systest_write'. The `SquashSiblings' parameter exists to +%% optionally allow handling of siblings whose value and metadata are +%% identical except for the dot. This goal is to facilitate testing +%% with DVV enabled because siblings can be created internally by Riak +%% in cases where testing with DVV disabled would not. Such cases +%% include writes that happen during handoff when a vnode forwards +%% writes, but also performs them locally or when a put coordinator +%% fails to send an acknowledgment within the timeout window and +%% another put request is issued. +read(Node, Start, End, Bucket, R, CommonValBin, SquashSiblings) + when is_binary(CommonValBin) -> + rt:wait_for_service(Node, riak_kv), + {ok, C} = riak:client_connect(Node), + lists:foldl(read_fold_fun(C, Bucket, R, CommonValBin, SquashSiblings), + [], + lists:seq(Start, End)). + +read_fold_fun(C, Bucket, R, CommonValBin, SquashSiblings) -> + fun(N, Acc) -> + GetRes = C:get(Bucket, <>, R), + Val = object_value(GetRes, SquashSiblings), + update_acc(value_matches(Val, N, CommonValBin), Val, N, Acc) + end. + +object_value({error, _}=Error, _) -> + Error; +object_value({ok, Obj}, SquashSiblings) -> + object_value(riak_object:value_count(Obj), Obj, SquashSiblings). + +object_value(1, Obj, _SquashSiblings) -> + riak_object:get_value(Obj); +object_value(_ValueCount, Obj, false) -> + riak_object:get_value(Obj); +object_value(_ValueCount, Obj, true) -> + lager:debug("Siblings detected for ~p:~p", [riak_object:bucket(Obj), riak_object:key(Obj)]), + Contents = riak_object:get_contents(Obj), + case lists:foldl(fun sibling_compare/2, {true, undefined}, Contents) of + {true, {_, _, _, Value}} -> + lager:debug("Siblings determined to be a single value"), + Value; + {false, _} -> + {error, siblings} + end. + +sibling_compare({MetaData, Value}, {true, undefined}) -> + Dot = case dict:find(<<"dot">>, MetaData) of + {ok, DotVal} -> + DotVal; + error -> + {error, no_dot} + end, + VTag = dict:fetch(<<"X-Riak-VTag">>, MetaData), + LastMod = dict:fetch(<<"X-Riak-Last-Modified">>, MetaData), + {true, {element(2, Dot), VTag, LastMod, Value}}; +sibling_compare(_, {false, _}=InvalidMatch) -> + InvalidMatch; +sibling_compare({MetaData, Value}, {true, PreviousElements}) -> + Dot = case dict:find(<<"dot">>, MetaData) of + {ok, DotVal} -> + DotVal; + error -> + {error, no_dot} + end, + VTag = dict:fetch(<<"X-Riak-VTag">>, MetaData), + LastMod = dict:fetch(<<"X-Riak-Last-Modified">>, MetaData), + ComparisonElements = {element(2, Dot), VTag, LastMod, Value}, + {ComparisonElements =:= PreviousElements, ComparisonElements}. + +value_matches(<>, N, CommonValBin) -> + true; +value_matches(_WrongVal, _N, _CommonValBin) -> + false. +update_acc(true, _, _, Acc) -> + Acc; +update_acc(false, {error, _}=Val, N, Acc) -> + [{N, Val} | Acc]; +update_acc(false, Val, N, Acc) -> + [{N, {wrong_val, Val}} | Acc]. + +verify_systest_value(N, Acc, CommonValBin, Obj) -> + Values = riak_object:get_values(Obj), + Res = [begin + case V of + <> -> + ok; + _WrongVal -> + wrong_val + end + end || V <- Values], + case lists:any(fun(X) -> X =:= ok end, Res) of + true -> + Acc; + false -> + [{N, {wrong_val, hd(Values)}} | Acc] + end. + diff --git a/tests/gh_riak_core_154.erl b/tests/gh_riak_core_154.erl index 37beaba37..c477b4e53 100644 --- a/tests/gh_riak_core_154.erl +++ b/tests/gh_riak_core_154.erl @@ -35,14 +35,14 @@ confirm() -> lager:info("Write data while ~p is offline", [Node2]), rt_node:stop(Node2), rt:wait_until_unpingable(Node2), - ?assertEqual([], rt:systest_write(Node1, 1000, 3)), + ?assertEqual([], rt_systest:write(Node1, 1000, 3)), lager:info("Verify that ~p is missing data", [Node2]), rt_node:start(Node2), rt_node:stop(Node1), rt:wait_until_unpingable(Node1), ?assertMatch([{_,{error,notfound}}|_], - rt:systest_read(Node2, 1000, 3)), + rt_systest:read(Node2, 1000, 3)), lager:info("Restart ~p and wait for handoff to occur", [Node1]), rt_node:start(Node1), @@ -51,7 +51,7 @@ confirm() -> lager:info("Verify that ~p has all data", [Node2]), rt_node:stop(Node1), - ?assertEqual([], rt:systest_read(Node2, 1000, 3)), + ?assertEqual([], rt_systest:read(Node2, 1000, 3)), lager:info("gh_riak_core_154: passed"), pass. diff --git a/tests/gh_riak_core_176.erl b/tests/gh_riak_core_176.erl index df6664372..9860a656e 100644 --- a/tests/gh_riak_core_176.erl +++ b/tests/gh_riak_core_176.erl @@ -51,11 +51,11 @@ confirm() -> rt:wait_for_service(Node2, riak_kv), lager:info("Write data to the cluster"), - rt:systest_write(Node1, 100), + rt_systest:write(Node1, 100), lager:info("Join ~p to the cluster and wait for handoff to finish", [Node2]), - rt:join(Node2, Node1), + rt_node:join(Node2, Node1), ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes12)), ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes12)), rt_node:wait_until_nodes_agree_about_ownership(Nodes12), @@ -68,7 +68,7 @@ confirm() -> lager:info("Join ~p to the cluster and wait for handoff to finish", [Node3]), rt:wait_for_service(Node3, riak_kv), - rt:join(Node3, Node1), + rt_node:join(Node3, Node1), ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes123)), ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes123)), rt_node:wait_until_nodes_agree_about_ownership(Nodes123), diff --git a/tests/gh_riak_kv_765.erl b/tests/gh_riak_kv_765.erl index 834b6c964..f43a45802 100644 --- a/tests/gh_riak_kv_765.erl +++ b/tests/gh_riak_kv_765.erl @@ -71,7 +71,7 @@ check_throttle_and_expiration() -> timer:sleep(2000), lager:info("Write 1000 keys"), - rt:systest_write(Node, 1000), + rt_systest:write(Node, 1000), enable_aae(Node), time_build(Node), Duration1 = rebuild(Node, 30000, 1000), diff --git a/tests/jmx_verify.erl b/tests/jmx_verify.erl index 3387cd4e2..f9cc8ca70 100644 --- a/tests/jmx_verify.erl +++ b/tests/jmx_verify.erl @@ -94,7 +94,7 @@ confirm() -> Pid = rt_pb:pbc(Node1), JMX3 = jmx_dump(JMXDumpCmd), - rt:systest_write(Node1, 1), + rt_systest:write(Node1, 1), %% make sure the stats that were supposed to increment did verify_inc(JMX2, JMX3, [{<<"pbc_connects_total">>, 1}, {<<"pbc_connects">>, 1}, diff --git a/tests/loaded_upgrade.erl b/tests/loaded_upgrade.erl index 7e89350fb..5feab5801 100644 --- a/tests/loaded_upgrade.erl +++ b/tests/loaded_upgrade.erl @@ -116,8 +116,8 @@ seed_cluster(Nodes=[Node1|_]) -> %% For List Keys lager:info("Writing 100 keys to ~p", [Node1]), - rt:systest_write(Node1, 100, 3), - ?assertEqual([], rt:systest_read(Node1, 100, 1)), + rt_systest:write(Node1, 100, 3), + ?assertEqual([], rt_systest:read(Node1, 100, 1)), seed(Node1, 0, 100, fun(Key) -> Bin = iolist_to_binary(io_lib:format("~p", [Key])), diff --git a/tests/pipe_verify_handoff.erl b/tests/pipe_verify_handoff.erl index c6bdaaad3..61cd440d1 100644 --- a/tests/pipe_verify_handoff.erl +++ b/tests/pipe_verify_handoff.erl @@ -107,7 +107,7 @@ confirm() -> lager:info("Join Secondary to Primary"), %% Give slave a chance to start and master to notice it. - rt:join(Secondary, Primary), + rt_node:join(Secondary, Primary), rt:wait_until_no_pending_changes(Nodes), rt_node:wait_until_nodes_agree_about_ownership(Nodes), diff --git a/tests/pipe_verify_handoff_blocking.erl b/tests/pipe_verify_handoff_blocking.erl index 36bf69489..139eede1a 100644 --- a/tests/pipe_verify_handoff_blocking.erl +++ b/tests/pipe_verify_handoff_blocking.erl @@ -108,7 +108,7 @@ confirm() -> lager:info("Join Secondary to Primary"), %% Give slave a chance to start and master to notice it. - rt:join(Secondary, Primary), + rt_node:join(Secondary, Primary), rt:wait_until_no_pending_changes(Nodes), rt_node:wait_until_nodes_agree_about_ownership(Nodes), diff --git a/tests/replication/repl_aae_fullsync.erl b/tests/replication/repl_aae_fullsync.erl index f72ed3c17..f4623aad9 100644 --- a/tests/replication/repl_aae_fullsync.erl +++ b/tests/replication/repl_aae_fullsync.erl @@ -607,5 +607,5 @@ write_to_cluster(Node, Start, End) -> %% of errors. read_from_cluster(Node, Start, End, Errors) -> lager:info("Reading ~p keys from node ~p.", [End - Start, Node]), - Res2 = rt:systest_read(Node, Start, End, ?TEST_BUCKET, 1), + Res2 = rt_systest:read(Node, Start, End, ?TEST_BUCKET, 1), ?assertEqual(Errors, length(Res2)). diff --git a/tests/replication/repl_aae_fullsync_util.erl b/tests/replication/repl_aae_fullsync_util.erl index 6376a7e54..3d47ca573 100644 --- a/tests/replication/repl_aae_fullsync_util.erl +++ b/tests/replication/repl_aae_fullsync_util.erl @@ -71,7 +71,7 @@ prepare_cluster_data(TestBucket, NumKeysAOnly, _NumKeysBoth, [AFirst|_] = ANodes %% check that the keys we wrote initially aren't replicated yet, because %% we've disabled fullsync_on_connect lager:info("Check keys written before repl was connected are not present"), - Res2 = rt:systest_read(BFirst, 1, NumKeysAOnly, TestBucket, 1, <<>>, true), + Res2 = rt_systest:read(BFirst, 1, NumKeysAOnly, TestBucket, 1, <<>>, true), ?assertEqual(NumKeysAOnly, length(Res2)), %% wait for the AAE trees to be built so that we don't get a not_built error diff --git a/tests/replication/repl_rt_cascading_rtq.erl b/tests/replication/repl_rt_cascading_rtq.erl index 64939bae0..79b7fa353 100644 --- a/tests/replication/repl_rt_cascading_rtq.erl +++ b/tests/replication/repl_rt_cascading_rtq.erl @@ -181,5 +181,5 @@ write_to_cluster(Node, Start, End) -> %% of errors. read_from_cluster(Node, Start, End, Errors) -> lager:info("Reading ~p keys from node ~p.", [End - Start, Node]), - Res2 = rt:systest_read(Node, Start, End, ?TEST_BUCKET, 1), + Res2 = rt_systest:read(Node, Start, End, ?TEST_BUCKET, 1), ?assertEqual(Errors, length(Res2)). diff --git a/tests/replication/repl_rt_overload.erl b/tests/replication/repl_rt_overload.erl index e425e02b3..6b13cbd8f 100644 --- a/tests/replication/repl_rt_overload.erl +++ b/tests/replication/repl_rt_overload.erl @@ -89,7 +89,7 @@ verify_overload_writes(LeaderA, LeaderB) -> ?assertEqual([], repl_util:do_write(LeaderA, First, Last, TestBucket, 2)), lager:info("Reading ~p keys from ~p", [Last-First+1, LeaderB]), - NumReads = rt:systest_read(LeaderB, First, Last, TestBucket, 2), + NumReads = rt_systest:read(LeaderB, First, Last, TestBucket, 2), lager:info("systest_read saw ~p errors", [length(NumReads)]), diff --git a/tests/replication/repl_util.erl b/tests/replication/repl_util.erl index eaa4271bd..3fec1246b 100644 --- a/tests/replication/repl_util.erl +++ b/tests/replication/repl_util.erl @@ -57,7 +57,7 @@ make_cluster(Nodes) -> [First|Rest] = Nodes, ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes)), [rt:wait_for_service(N, riak_kv) || N <- Nodes], - [rt:join(Node, First) || Node <- Rest], + [rt_node:join(Node, First) || Node <- Rest], ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes)). name_cluster(Node, Name) -> @@ -197,10 +197,10 @@ wait_until_fullsync_stopped(SourceLeader) -> wait_for_reads(Node, Start, End, Bucket, R) -> rt:wait_until(Node, fun(_) -> - Reads = rt:systest_read(Node, Start, End, Bucket, R, <<>>, true), + Reads = rt_systest:read(Node, Start, End, Bucket, R, <<>>, true), Reads == [] end), - Reads = rt:systest_read(Node, Start, End, Bucket, R, <<>>, true), + Reads = rt_systest:read(Node, Start, End, Bucket, R, <<>>, true), lager:info("Reads: ~p", [Reads]), length(Reads). @@ -456,14 +456,14 @@ stop_realtime(Node, Cluster) -> ?assertEqual(ok, Res). do_write(Node, Start, End, Bucket, W) -> - case rt:systest_write(Node, Start, End, Bucket, W) of + case rt_systest:write(Node, Start, End, Bucket, W) of [] -> []; Errors -> lager:warning("~p errors while writing: ~p", [length(Errors), Errors]), timer:sleep(1000), - lists:flatten([rt:systest_write(Node, S, S, Bucket, W) || + lists:flatten([rt_systest:write(Node, S, S, Bucket, W) || {S, _Error} <- Errors]) end. @@ -573,7 +573,7 @@ read_from_cluster(Node, Start, End, Bucket, Errors) -> %% of errors. read_from_cluster(Node, Start, End, Bucket, Errors, Quorum) -> lager:info("Reading ~p keys from node ~p.", [End - Start, Node]), - Res2 = rt:systest_read(Node, Start, End, Bucket, Quorum, <<>>, true), + Res2 = rt_systest:read(Node, Start, End, Bucket, Quorum, <<>>, true), ?assertEqual(Errors, length(Res2)). %% @doc Assert we can perform one fullsync cycle, and that the number of diff --git a/tests/replication/replication.erl b/tests/replication/replication.erl index c9c76890b..7bcf7b8a4 100644 --- a/tests/replication/replication.erl +++ b/tests/replication/replication.erl @@ -106,7 +106,7 @@ replication([AFirst|_] = ANodes, [BFirst|_] = BNodes, Connected) -> %% check that the keys we wrote initially aren't replicated yet, because %% we've disabled fullsync_on_connect lager:info("Check keys written before repl was connected are not present"), - Res2 = rt:systest_read(BFirst, 1, 100, TestBucket, 2), + Res2 = rt_systest:read(BFirst, 1, 100, TestBucket, 2), ?assertEqual(100, length(Res2)), start_and_wait_until_fullsync_complete(LeaderA), @@ -306,19 +306,19 @@ replication([AFirst|_] = ANodes, [BFirst|_] = BNodes, Connected) -> FullsyncOnly, 2)), lager:info("Check the fullsync only bucket didn't replicate the writes"), - Res6 = rt:systest_read(BSecond, 1, 100, FullsyncOnly, 2), + Res6 = rt_systest:read(BSecond, 1, 100, FullsyncOnly, 2), ?assertEqual(100, length(Res6)), lager:info("Check the realtime only bucket that was written to offline " "isn't replicated"), - Res7 = rt:systest_read(BSecond, 1, 100, RealtimeOnly, 2), + Res7 = rt_systest:read(BSecond, 1, 100, RealtimeOnly, 2), ?assertEqual(100, length(Res7)); _ -> timer:sleep(1000) end, lager:info("Check the {repl, false} bucket didn't replicate"), - Res8 = rt:systest_read(BSecond, 1, 100, NoRepl, 2), + Res8 = rt_systest:read(BSecond, 1, 100, NoRepl, 2), ?assertEqual(100, length(Res8)), %% do a fullsync, make sure that fullsync_only is replicated, but @@ -332,7 +332,7 @@ replication([AFirst|_] = ANodes, [BFirst|_] = BNodes, Connected) -> FullsyncOnly, 2)), lager:info("Check realtime only bucket didn't replicate"), - Res10 = rt:systest_read(BSecond, 1, 100, RealtimeOnly, 2), + Res10 = rt_systest:read(BSecond, 1, 100, RealtimeOnly, 2), ?assertEqual(100, length(Res10)), @@ -347,14 +347,14 @@ replication([AFirst|_] = ANodes, [BFirst|_] = BNodes, Connected) -> RealtimeOnly, 2)), lager:info("Check the older keys in the realtime bucket did not replicate"), - Res12 = rt:systest_read(BSecond, 1, 100, RealtimeOnly, 2), + Res12 = rt_systest:read(BSecond, 1, 100, RealtimeOnly, 2), ?assertEqual(100, length(Res12)); _ -> ok end, lager:info("Check {repl, false} bucket didn't replicate"), - Res13 = rt:systest_read(BSecond, 1, 100, NoRepl, 2), + Res13 = rt_systest:read(BSecond, 1, 100, NoRepl, 2), ?assertEqual(100, length(Res13)); _ -> ok @@ -685,20 +685,20 @@ wait_until_no_connection(Node) -> wait_for_reads(Node, Start, End, Bucket, R) -> rt:wait_until(Node, fun(_) -> - rt:systest_read(Node, Start, End, Bucket, R) == [] + rt_systest:read(Node, Start, End, Bucket, R) == [] end), - Reads = rt:systest_read(Node, Start, End, Bucket, R), + Reads = rt_systest:read(Node, Start, End, Bucket, R), lager:info("Reads: ~p", [Reads]), length(Reads). do_write(Node, Start, End, Bucket, W) -> - case rt:systest_write(Node, Start, End, Bucket, W) of + case rt_systest:write(Node, Start, End, Bucket, W) of [] -> []; Errors -> lager:warning("~p errors while writing: ~p", [length(Errors), Errors]), timer:sleep(1000), - lists:flatten([rt:systest_write(Node, S, S, Bucket, W) || + lists:flatten([rt_systest:write(Node, S, S, Bucket, W) || {S, _Error} <- Errors]) end. diff --git a/tests/replication/replication2.erl b/tests/replication/replication2.erl index 0f69825d6..e3e75759d 100644 --- a/tests/replication/replication2.erl +++ b/tests/replication/replication2.erl @@ -148,7 +148,7 @@ real_time_replication_test([AFirst|_] = ANodes, [BFirst|_] = BNodes, Connected) %% Check that the keys we wrote initially aren't replicated yet as %% fullsync_on_connect is disabled. lager:info("Check keys written before repl was connected are not present"), - Res2 = rt:systest_read(BFirst, 1, 100, TestBucket, 2), + Res2 = rt_systest:read(BFirst, 1, 100, TestBucket, 2), ?assertEqual(100, length(Res2)), log_to_nodes(ANodes++BNodes, "Test fullsync with leader ~p", [LeaderA]), diff --git a/tests/replication/replication2_pg.erl b/tests/replication/replication2_pg.erl index 2bc60f292..1198a5099 100644 --- a/tests/replication/replication2_pg.erl +++ b/tests/replication/replication2_pg.erl @@ -971,9 +971,9 @@ verify_topology_change(SourceNodes, SinkNodes) -> lager:info("Rejoining former leader."), case SinkLeader of SinkNode1 -> - rt:join(SinkNode1, SinkNode2); + rt_node:join(SinkNode1, SinkNode2); SinkNode2 -> - rt:join(SinkNode2, SinkNode1) + rt_node:join(SinkNode2, SinkNode1) end, rt:wait_until_ring_converged(SinkNodes), diff --git a/tests/replication/replication2_rt_sink_connection.erl b/tests/replication/replication2_rt_sink_connection.erl index c9348fd7c..03b024e0a 100644 --- a/tests/replication/replication2_rt_sink_connection.erl +++ b/tests/replication/replication2_rt_sink_connection.erl @@ -88,7 +88,7 @@ confirm() -> enable_rt(AFirst, ANodes), lager:info("Adding 4th node to the A cluster"), - rt:join(CNode, AFirst), + rt_node:join(CNode, AFirst), [verify_connectivity(Node) || Node <- ANodes], diff --git a/tests/verify_basic_upgrade.erl b/tests/verify_basic_upgrade.erl index b1c17556a..138243048 100644 --- a/tests/verify_basic_upgrade.erl +++ b/tests/verify_basic_upgrade.erl @@ -29,8 +29,8 @@ confirm() -> Nodes = [Node1|_] = rt_cluster:build_cluster([OldVsn, OldVsn, OldVsn, OldVsn]), lager:info("Writing 100 keys to ~p", [Node1]), - rt:systest_write(Node1, 100, 3), - ?assertEqual([], rt:systest_read(Node1, 100, 1)), + rt_systest:write(Node1, 100, 3), + ?assertEqual([], rt_systest:read(Node1, 100, 1)), [upgrade(Node, current) || Node <- Nodes], @@ -43,6 +43,6 @@ upgrade(Node, NewVsn) -> rt:upgrade(Node, NewVsn), rt:wait_for_service(Node, riak_kv), lager:info("Ensuring keys still exist"), - rt:systest_read(Node, 100, 1), - ?assertEqual([], rt:systest_read(Node, 100, 1)), + rt_systest:read(Node, 100, 1), + ?assertEqual([], rt_systest:read(Node, 100, 1)), ok. diff --git a/tests/verify_build_cluster.erl b/tests/verify_build_cluster.erl index 7f87e0290..c502e6d90 100644 --- a/tests/verify_build_cluster.erl +++ b/tests/verify_build_cluster.erl @@ -38,18 +38,18 @@ confirm(#rt_properties{nodes=Nodes}, _MD) -> [Node1, Node2, Node3, Node4] = Nodes, lager:info("Loading some data up in this cluster."), - ?assertEqual([], rt:systest_write(Node1, 0, 1000, <<"verify_build_cluster">>, 2)), + ?assertEqual([], rt_systest:write(Node1, 0, 1000, <<"verify_build_cluster">>, 2)), lager:info("joining Node 2 to the cluster... It takes two to make a thing go right"), - rt:join(Node2, Node1), + rt_node:join(Node2, Node1), wait_and_validate([Node1, Node2]), lager:info("joining Node 3 to the cluster"), - rt:join(Node3, Node1), + rt_node:join(Node3, Node1), wait_and_validate([Node1, Node2, Node3]), lager:info("joining Node 4 to the cluster"), - rt:join(Node4, Node1), + rt_node:join(Node4, Node1), wait_and_validate(Nodes), lager:info("taking Node 1 down"), @@ -101,5 +101,5 @@ wait_and_validate(RingNodes, UpNodes) -> [rt_node:wait_until_owners_according_to(Node, RingNodes) || Node <- UpNodes], [rt:wait_for_service(Node, riak_kv) || Node <- UpNodes], lager:info("Verify that you got much data... (this is how we do it)"), - ?assertEqual([], rt:systest_read(hd(UpNodes), 0, 1000, <<"verify_build_cluster">>, 2)), + ?assertEqual([], rt_systest:read(hd(UpNodes), 0, 1000, <<"verify_build_cluster">>, 2)), done. diff --git a/tests/verify_capabilities.erl b/tests/verify_capabilities.erl index 2d6ee385c..28d345fb9 100644 --- a/tests/verify_capabilities.erl +++ b/tests/verify_capabilities.erl @@ -69,7 +69,7 @@ confirm() -> ?assertEqual(ok, rt:wait_until_capability(CNode, {riak_core, staged_joins}, true)), lager:info("Building current + ~s cluster", [Legacy]), - rt:join(LNode, CNode), + rt_node:join(LNode, CNode), ?assertEqual(ok, rt:wait_until_all_members([CNode], [CNode, LNode])), ?assertEqual(ok, rt:wait_until_legacy_ringready(CNode)), @@ -102,7 +102,7 @@ confirm() -> restart_capability_server(CNode), lager:info("Adding previous node to cluster"), - rt:join(PNode, LNode), + rt_node:join(PNode, LNode), ?assertEqual(ok, rt:wait_until_all_members([CNode], [CNode, LNode, PNode])), ?assertEqual(ok, rt:wait_until_legacy_ringready(CNode)), @@ -139,7 +139,7 @@ confirm() -> _ -> lager:info("Legacy Riak not available, skipping legacy tests"), lager:info("Adding previous node to cluster"), - rt:join(PNode, LNode), + rt_node:join(PNode, LNode), ?assertEqual(ok, rt:wait_until_all_members([CNode], [CNode, LNode, PNode])), ?assertEqual(ok, rt:wait_until_legacy_ringready(CNode)) end, diff --git a/tests/verify_down.erl b/tests/verify_down.erl index 2bc539849..33ca503ea 100644 --- a/tests/verify_down.erl +++ b/tests/verify_down.erl @@ -28,7 +28,7 @@ confirm() -> %% Join node2 to node1 and wait for cluster convergence lager:info("Join ~p to ~p", [Node2, Node1]), - rt:join(Node2, Node1), + rt_node:join(Node2, Node1), ?assertEqual(ok, rt_node:wait_until_nodes_ready([Node1, Node2])), ?assertEqual(ok, rt:wait_until_no_pending_changes([Node1, Node2])), @@ -40,7 +40,7 @@ confirm() -> %% Join node3 to node1 lager:info("Join ~p to ~p", [Node3, Node1]), - rt:join(Node3, Node1), + rt_node:join(Node3, Node1), ?assertEqual(ok, rt:wait_until_all_members(Remaining, [Node3])), %% Ensure node3 remains in the joining state diff --git a/tests/verify_dynamic_ring.erl b/tests/verify_dynamic_ring.erl index 19cfdbda5..368c95a51 100644 --- a/tests/verify_dynamic_ring.erl +++ b/tests/verify_dynamic_ring.erl @@ -39,8 +39,8 @@ confirm() -> %% This assignment for `NewNodes' is commented until riak_core %% issue #570 is resolved %% NewNodes = [ANode, YetAnother, ReplacingNode], - rt:join(AnotherNode, ANode), - rt:join(YetAnother, ANode), + rt_node:join(AnotherNode, ANode), + rt_node:join(YetAnother, ANode), rt_node:wait_until_nodes_agree_about_ownership(Nodes), rt:wait_until_ring_converged(Nodes), rt:wait_until_no_pending_changes(Nodes), @@ -52,14 +52,14 @@ confirm() -> wait_until_extra_proxies_shutdown(Nodes), lager:info("writing 500 keys"), - ?assertEqual([], rt:systest_write(ANode, 1, 500, ?BUCKET, ?W)), + ?assertEqual([], rt_systest:write(ANode, 1, 500, ?BUCKET, ?W)), test_resize(?SHRUNK_SIZE, ?START_SIZE, ANode, Nodes, {501, 750}), lager:info("verifying previously written data"), - ?assertEqual([], rt:systest_read(ANode, 1, 500, ?BUCKET, ?R)), + ?assertEqual([], rt_systest:read(ANode, 1, 500, ?BUCKET, ?R)), test_resize(?START_SIZE, ?EXPANDED_SIZE, ANode, Nodes), lager:info("verifying previously written data"), - ?assertEqual([], rt:systest_read(ANode, 1, 750, ?BUCKET, ?R)), + ?assertEqual([], rt_systest:read(ANode, 1, 750, ?BUCKET, ?R)), %% This following test code for force-replace is commented until %% riak_core issue #570 is resolved. At that time the preceding 3 @@ -82,11 +82,11 @@ confirm() -> %% rt:wait_until_no_pending_changes(NewNodes), %% assert_ring_size(?EXPANDED_SIZE, NewNodes), %% lager:info("verifying written data"), - %% ?assertEqual([], rt:systest_read(ANode, 1, 750, ?BUCKET, ?R)), + %% ?assertEqual([], rt_systest:read(ANode, 1, 750, ?BUCKET, ?R)), test_resize(?EXPANDED_SIZE, ?SHRUNK_SIZE, ANode, NewNodes), lager:info("verifying written data"), - ?assertEqual([], rt:systest_read(ANode, 1, 750, ?BUCKET, ?R)), + ?assertEqual([], rt_systest:read(ANode, 1, 750, ?BUCKET, ?R)), wait_until_extra_vnodes_shutdown(NewNodes), wait_until_extra_proxies_shutdown(NewNodes), @@ -110,7 +110,7 @@ confirm() -> rt:wait_until_ring_converged(NewNodes), assert_ring_size(?SHRUNK_SIZE, NewNodes), lager:info("verifying written data"), - ?assertEqual([], rt:systest_read(ANode, 1, 750, ?BUCKET, ?R)), + ?assertEqual([], rt_systest:read(ANode, 1, 750, ?BUCKET, ?R)), pass. @@ -137,7 +137,7 @@ write_during_resize(_, Start, End) when Start =:= undefined orelse End =:= undef write_during_resize(Node, Start, End) -> Pid = self(), spawn(fun() -> - case rt:systest_write(Node, Start, End, ?BUCKET, ?W) of + case rt_systest:write(Node, Start, End, ?BUCKET, ?W) of [] -> Pid ! done_writing; Ers -> @@ -151,7 +151,7 @@ verify_write_during_resize(Node, Start, End) -> receive done_writing -> lager:info("verifying data written during operation"), - ?assertEqual([], rt:systest_read(Node, Start, End, ?BUCKET, ?R)), + ?assertEqual([], rt_systest:read(Node, Start, End, ?BUCKET, ?R)), ok; {errors_writing, Ers} -> lager:error("errors were encountered while writing during operation: ~p", [Ers]), diff --git a/tests/verify_handoff.erl b/tests/verify_handoff.erl index c87ca6b9c..eb3e9af06 100644 --- a/tests/verify_handoff.erl +++ b/tests/verify_handoff.erl @@ -58,11 +58,11 @@ run_test(TestMode, NTestItems, NTestNodes, Encoding) -> || N <- Nodes], lager:info("Populating root node."), - rt:systest_write(RootNode, NTestItems), + rt_systest:write(RootNode, NTestItems), %% write one object with a bucket type rt_bucket_types:create_and_activate_bucket_type(RootNode, <<"type">>, []), %% allow cluster metadata some time to propogate - rt:systest_write(RootNode, 1, 2, {<<"type">>, <<"bucket">>}, 2), + rt_systest:write(RootNode, 1, 2, {<<"type">>, <<"bucket">>}, 2), %% Test handoff on each node: lager:info("Testing handoff for cluster."), @@ -111,16 +111,16 @@ test_handoff(RootNode, NewNode, NTestItems) -> rt:wait_for_service(NewNode, riak_kv), lager:info("Joining new node with cluster."), - rt:join(NewNode, RootNode), + rt_node:join(NewNode, RootNode), ?assertEqual(ok, rt_node:wait_until_nodes_ready([RootNode, NewNode])), rt:wait_until_no_pending_changes([RootNode, NewNode]), %% See if we get the same data back from the joined node that we added to the root node. %% Note: systest_read() returns /non-matching/ items, so getting nothing back is good: lager:info("Validating data after handoff:"), - Results = rt:systest_read(NewNode, NTestItems), - ?assertEqual(0, length(Results)), - Results2 = rt:systest_read(RootNode, 1, 2, {<<"type">>, <<"bucket">>}, 2), + Results = rt_systest:read(NewNode, NTestItems), + ?assertEqual(0, length(Results)), + Results2 = rt_systest:read(RootNode, 1, 2, {<<"type">>, <<"bucket">>}, 2), ?assertEqual(0, length(Results2)), lager:info("Data looks ok."). diff --git a/tests/verify_handoff_mixed.erl b/tests/verify_handoff_mixed.erl index de91f15eb..0af3955a5 100644 --- a/tests/verify_handoff_mixed.erl +++ b/tests/verify_handoff_mixed.erl @@ -67,7 +67,7 @@ confirm() -> OldFold = rt:capability(Old, ?FOLD_CAPABILITY, v1), %% now link the nodes together and wait for handoff to complete - ok = rt:join(Old, Current), + ok = rt_node:join(Old, Current), ok = rt:wait_until_all_members(Nodes), ok = rt:wait_until_ring_converged(Nodes), diff --git a/tests/verify_listkeys.erl b/tests/verify_listkeys.erl index d4edd58e5..8ef477eb9 100644 --- a/tests/verify_listkeys.erl +++ b/tests/verify_listkeys.erl @@ -42,7 +42,7 @@ confirm() -> lists:foldl(fun(Node, [N1|_] = Cluster) -> lager:info("An invitation to this party is cordially extended to ~p.", [Node]), - rt:join(Node, N1), + rt_node:join(Node, N1), lager:info("Wait until there are no pending changes"), Ns = lists:usort([Node|Cluster]), rt:wait_until_no_pending_changes(Ns), diff --git a/tests/verify_membackend.erl b/tests/verify_membackend.erl index ded71c846..894d0b8cb 100644 --- a/tests/verify_membackend.erl +++ b/tests/verify_membackend.erl @@ -46,7 +46,7 @@ max_memory(Mode) -> Conf = mkconf(max_memory, Mode), [NodeA, NodeB] = rt_cluster:deploy_nodes(2, Conf), - rt:join(NodeB, NodeA), + rt_node:join(NodeB, NodeA), ?assertEqual(ok, check_put_delete(NodeA)), @@ -81,10 +81,10 @@ combo(Mode) -> check_leave_and_expiry(NodeA, NodeB) -> - ?assertEqual([], rt:systest_write(NodeB, 1, 100, ?BUCKET, 2)), - ?assertEqual([], rt:systest_read(NodeB, 1, 100, ?BUCKET, 2)), + ?assertEqual([], rt_systest:write(NodeB, 1, 100, ?BUCKET, 2)), + ?assertEqual([], rt_systest:read(NodeB, 1, 100, ?BUCKET, 2)), - rt:join(NodeB, NodeA), + rt_node:join(NodeB, NodeA), ?assertEqual(ok, rt_node:wait_until_nodes_ready([NodeA, NodeB])), rt:wait_until_no_pending_changes([NodeA, NodeB]), @@ -92,14 +92,14 @@ check_leave_and_expiry(NodeA, NodeB) -> rt_node:leave(NodeB), rt:wait_until_unpingable(NodeB), - ?assertEqual([], rt:systest_read(NodeA, 1, 100, ?BUCKET, 2)), + ?assertEqual([], rt_systest:read(NodeA, 1, 100, ?BUCKET, 2)), lager:info("waiting for keys to expire"), timer:sleep(timer:seconds(210)), - _ = rt:systest_read(NodeA, 1, 100, ?BUCKET, 2), + _ = rt_systest:read(NodeA, 1, 100, ?BUCKET, 2), timer:sleep(timer:seconds(5)), - Res = rt:systest_read(NodeA, 1, 100, ?BUCKET, 2), + Res = rt_systest:read(NodeA, 1, 100, ?BUCKET, 2), ?assertEqual(100, length(Res)), ok. @@ -110,9 +110,9 @@ check_eviction(Node) -> Size = 20000 * 8, Val = <<0:Size>>, - ?assertEqual([], rt:systest_write(Node, 1, 500, ?BUCKET, 2, Val)), + ?assertEqual([], rt_systest:write(Node, 1, 500, ?BUCKET, 2, Val)), - Res = length(rt:systest_read(Node, 1, 100, ?BUCKET, 2, Val)), + Res = length(rt_systest:read(Node, 1, 100, ?BUCKET, 2, Val)), %% this is a wider range than I'd like but the final outcome is %% somewhat hard to predict. Just trying to verify that some diff --git a/tests/verify_mr_prereduce_node_down.erl b/tests/verify_mr_prereduce_node_down.erl index 7df0ae261..d232946fe 100644 --- a/tests/verify_mr_prereduce_node_down.erl +++ b/tests/verify_mr_prereduce_node_down.erl @@ -53,7 +53,7 @@ confirm() -> Bucket = <<"verify_mr_prereduce_node_down">>, ObjCount = 100, lager:info("Loading ~b objects of test data", [ObjCount]), - [] = rt:systest_write(Primary, 1, ObjCount, Bucket, 3), + [] = rt_systest:write(Primary, 1, ObjCount, Bucket, 3), %% run the query a bunch C = rt_pb:pbc(Primary), diff --git a/tests/verify_riak_object_reformat.erl b/tests/verify_riak_object_reformat.erl index bdb9a9c6d..7e7a29bd7 100644 --- a/tests/verify_riak_object_reformat.erl +++ b/tests/verify_riak_object_reformat.erl @@ -39,8 +39,8 @@ confirm() -> [rt:wait_until_capability(N, {riak_kv, object_format}, v1, v0) || N <- Nodes], lager:info("Writing 100 keys in format v1 to ~p", [Node1]), - rt:systest_write(Node1, 100, ?N), - ?assertEqual([], rt:systest_read(Node1, 100, ?N)), + rt_systest:write(Node1, 100, ?N), + ?assertEqual([], rt_systest:read(Node1, 100, ?N)), lager:info("100 keys successfully written to ~p", [Node1]), %% TODO: introduce some handoff @@ -51,7 +51,7 @@ confirm() -> rt:upgrade(Node, DowngradeVsn), %% use upgrade to downgrade rt:wait_for_service(Node, riak_kv), lager:info("Ensuring keys still readable on ~p", [Node]), - ?assertEqual([], rt:systest_read(Node, 100, ?N)) + ?assertEqual([], rt_systest:read(Node, 100, ?N)) end || Node <- Nodes], pass. diff --git a/tests/verify_riak_stats.erl b/tests/verify_riak_stats.erl index dbc2aff98..247212780 100644 --- a/tests/verify_riak_stats.erl +++ b/tests/verify_riak_stats.erl @@ -105,7 +105,7 @@ confirm() -> Pid = rt_pb:pbc(Node1), Stats3 = get_stats(Node1), - rt:systest_write(Node1, 1), + rt_systest:write(Node1, 1), %% make sure the stats that were supposed to increment did verify_inc(Stats2, Stats3, [{<<"pbc_connects_total">>, 1}, {<<"pbc_connects">>, 1}, diff --git a/tests/verify_secondary_index_reformat.erl b/tests/verify_secondary_index_reformat.erl index 9ea33f1a2..84b86b349 100644 --- a/tests/verify_secondary_index_reformat.erl +++ b/tests/verify_secondary_index_reformat.erl @@ -70,7 +70,7 @@ confirm() -> %% write some more data (make sure flag doesn't "roll back" on restart lager:info("writing some more data"), - rt:systest_write(Node, 10, 1), + rt_systest:write(Node, 10, 1), lager:info("restarting node"), rt_node:stop_and_wait(Node), diff --git a/tests/verify_snmp.erl b/tests/verify_snmp.erl index 7cbc1c7c0..bae8fbb19 100644 --- a/tests/verify_snmp.erl +++ b/tests/verify_snmp.erl @@ -60,8 +60,8 @@ confirm() -> lager:info("Doing some reads and writes to record some stats."), - rt:systest_write(Node1, 10), - rt:systest_read(Node1, 10), + rt_systest:write(Node1, 10), + rt_systest:read(Node1, 10), lager:info("Waiting for HTTP Stats to be non-zero"), ?assertEqual(ok, diff --git a/tests/verify_tick_change.erl b/tests/verify_tick_change.erl index 32f6d9040..755e766d2 100644 --- a/tests/verify_tick_change.erl +++ b/tests/verify_tick_change.erl @@ -63,11 +63,11 @@ make_common() -> write_stuff(Nodes, Start, End, Bucket, W, Common) -> Nd = lists:nth(length(Nodes), Nodes), - [] = rt:systest_write(Nd, Start, End, Bucket, W, Common). + [] = rt_systest:write(Nd, Start, End, Bucket, W, Common). read_stuff(Nodes, Start, End, Bucket, W, Common) -> Nd = lists:nth(length(Nodes), Nodes), - [] = rt:systest_read(Nd, Start, End, Bucket, W, Common). + [] = rt_systest:read(Nd, Start, End, Bucket, W, Common). is_set_net_ticktime_done(Nodes, Time) -> case lists:usort([(catch rpc:call(Node, net_kernel, get_net_ticktime,[])) From c8eaaf6d364a160d3209806b753064edf6fc6715 Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Thu, 31 Jul 2014 14:34:23 -0400 Subject: [PATCH 026/157] Fix calls to rt:teardown; should be rt_cluster:teardown. --- src/riak_test_escript.erl | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index 5bb67cdbc..ddc220c1b 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -293,7 +293,7 @@ run_test(Test, Outdir, TestMetaData, Report, HarnessArgs, NumTests) -> CoverDir = rt_config:get(cover_output, "coverage"), case NumTests of 1 -> keep_them_up; - _ -> rt:teardown() + _ -> rt_cluster:teardown() end, CoverageFile = rt_cover:maybe_export_coverage(Test, CoverDir, erlang:phash2(TestMetaData)), case Report of @@ -422,8 +422,8 @@ so_kill_riak_maybe() -> io:format("Would you like to leave Riak running in order to debug?~n"), Input = io:get_chars("[Y/n] ", 1), case Input of - "n" -> rt:teardown(); - "N" -> rt:teardown(); + "n" -> rt_cluster:teardown(); + "N" -> rt_cluster:teardown(); _ -> io:format("Leaving Riak Up... "), rt:whats_up() From 0b8c7625bf7b3e33f4781dabb671d7d2f8624514 Mon Sep 17 00:00:00 2001 From: Kelly McLaughlin Date: Wed, 30 Jul 2014 15:40:13 -0600 Subject: [PATCH 027/157] Move all access to rt_properties record into an rt_properties module Create a new rt_properties module to be the interface for all rt_properties record access and manipulation. Change all applicable framework modules and converted tests to use this new module. --- include/rt.hrl | 15 --- src/riak_test_runner.erl | 20 ++-- src/rt_cluster.erl | 39 ++++--- src/rt_properties.erl | 195 +++++++++++++++++++++++++++++++ tests/basic_command_line.erl | 8 +- tests/bucket_props_roundtrip.erl | 9 +- tests/bucket_types.erl | 16 ++- tests/cluster_meta_basic.erl | 8 +- tests/http_bucket_types.erl | 15 ++- tests/secondary_index_tests.erl | 18 ++- tests/verify_build_cluster.erl | 17 ++- 11 files changed, 265 insertions(+), 95 deletions(-) create mode 100644 src/rt_properties.erl diff --git a/include/rt.hrl b/include/rt.hrl index a40996d61..78de7e028 100644 --- a/include/rt.hrl +++ b/include/rt.hrl @@ -23,18 +23,3 @@ url :: string(), headers=[] :: [{atom(), string()}] }). - --record(rt_properties, { - nodes :: [node()], - node_count=6 :: non_neg_integer(), - metadata=[] :: proplists:proplist(), - properties=[] :: proplists:proplist(), - rolling_upgrade=false :: boolean(), - start_version=current :: atom(), - upgrade_version=current :: atom(), - wait_for_transfers=false :: boolean(), - valid_backends=all :: all | [atom()], - make_cluster=true :: boolean(), - config :: term() - }). --type rt_properties() :: #rt_properties{}. diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index 54d39007f..d181177ae 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -139,22 +139,22 @@ test_fun({PropsMod, PropsFun}, {SetupMod, SetupFun}, ConfirmModFun, MetaData) -> Properties = PropsMod:PropsFun(), case SetupMod:SetupFun(Properties, MetaData) of {ok, SetupData} -> - lager:info("Wait for transfers? ~p", [SetupData#rt_properties.wait_for_transfers]), + RollingUpgradeTest = rt_properties:get(rolling_upgrade, SetupData), ConfirmFun = compose_confirm_fun(ConfirmModFun, SetupData, - MetaData), + MetaData, + RollingUpgradeTest), + ConfirmFun(); _ -> fail end end. -compose_confirm_fun({ConfirmMod, ConfirmFun}, - SetupData=#rt_properties{rolling_upgrade=true}, - MetaData) -> - Nodes = SetupData#rt_properties.nodes, - WaitForTransfers = SetupData#rt_properties.wait_for_transfers, - UpgradeVersion = SetupData#rt_properties.upgrade_version, +compose_confirm_fun({ConfirmMod, ConfirmFun}, SetupData, MetaData, true) -> + Nodes = rt_properties:get(nodes, SetupData), + WaitForTransfers = rt_properties:get(wait_for_transfers, SetupData), + UpgradeVersion = rt_properties:get(upgrade_version, SetupData), fun() -> InitialResult = ConfirmMod:ConfirmFun(SetupData, MetaData), OtherResults = [begin @@ -165,9 +165,7 @@ compose_confirm_fun({ConfirmMod, ConfirmFun}, end || Node <- Nodes], lists:all(fun(R) -> R =:= pass end, [InitialResult | OtherResults]) end; -compose_confirm_fun({ConfirmMod, ConfirmFun}, - SetupData=#rt_properties{rolling_upgrade=false}, - MetaData) -> +compose_confirm_fun({ConfirmMod, ConfirmFun}, SetupData, MetaData, false) -> fun() -> ConfirmMod:ConfirmFun(SetupData, MetaData) end. diff --git a/src/rt_cluster.erl b/src/rt_cluster.erl index 26470e122..cdd507c3e 100644 --- a/src/rt_cluster.erl +++ b/src/rt_cluster.erl @@ -39,6 +39,7 @@ try_nodes_ready/3, versions/0, teardown/0]). + -export([maybe_wait_for_transfers/2]). -include("rt.hrl"). @@ -47,27 +48,29 @@ %% @doc Default properties used if a riak_test module does not specify %% a custom properties function. --spec properties() -> rt_properties(). +-spec properties() -> rt_properties:properties(). properties() -> - #rt_properties{config=config()}. + rt_properties:new([{config, config()}]). --spec setup(rt_properties(), proplists:proplist()) -> - {ok, rt_properties()} | {error, term()}. +-spec setup(rt_properties:properties(), proplists:proplist()) -> + {ok, rt_properties:properties()} | {error, term()}. setup(Properties, MetaData) -> rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), - RollingUpgrade = proplists:get_value(rolling_upgrade, - MetaData, - Properties#rt_properties.rolling_upgrade), - Version = Properties#rt_properties.start_version, - Versions = [{Version, Properties#rt_properties.config} || - _ <- lists:seq(1, Properties#rt_properties.node_count)], - Nodes = deploy_or_build_cluster(Versions, Properties#rt_properties.make_cluster), - - maybe_wait_for_transfers(Nodes, Properties#rt_properties.wait_for_transfers), - UpdProperties = Properties#rt_properties{nodes=Nodes, - rolling_upgrade=RollingUpgrade}, - {ok, UpdProperties}. + RollingUpgrade = + proplists:get_value(rolling_upgrade, + MetaData, + rt_properties:get(rolling_upgrade, Properties)), + Version = rt_properties:get(start_version, Properties), + Config = rt_properties:get(config, Properties), + Versions = [{Version, Config} || + _ <- lists:seq(1, rt_properties:get(node_count, Properties))], + Nodes = deploy_or_build_cluster(Versions, + rt_properties:get(make_cluster, Properties)), + maybe_wait_for_transfers(Nodes, + rt_properties:get(wait_for_transfers, Properties)), + UpdPropertyList = [{nodes, Nodes}, {rolling_upgrade, RollingUpgrade}], + rt_properties:set(UpdPropertyList, Properties). deploy_or_build_cluster(Versions, true) -> build_cluster(Versions); @@ -218,7 +221,9 @@ config() -> {riak_pipe, [{worker_limit, 200}]}]. augment_config(Section, Property, Config) -> - UpdSectionConfig = update_section(Section, Property, lists:keyfind(Section, 1, Config)), + UpdSectionConfig = update_section(Section, + Property, + lists:keyfind(Section, 1, Config)), lists:keyreplace(Section, 1, Config, UpdSectionConfig). update_section(Section, Property, false) -> diff --git a/src/rt_properties.erl b/src/rt_properties.erl new file mode 100644 index 000000000..a484d0224 --- /dev/null +++ b/src/rt_properties.erl @@ -0,0 +1,195 @@ +%% ------------------------------------------------------------------- +%% +%% 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. +%% +%% ------------------------------------------------------------------- + +%% @doc Implements a set of functions for accessing and manipulating +%% an `rt_properties' record.-module(rt_properties). + +-module(rt_properties). + +-include("rt.hrl"). +-include_lib("eunit/include/eunit.hrl"). + +-record(rt_properties_v1, { + nodes :: [node()], + node_count=6 :: non_neg_integer(), + metadata=[] :: proplists:proplist(), + properties=[] :: proplists:proplist(), + rolling_upgrade=false :: boolean(), + start_version=current :: atom(), + upgrade_version=current :: atom(), + wait_for_transfers=false :: boolean(), + valid_backends=all :: all | [atom()], + make_cluster=true :: boolean(), + config :: term() + }). +-type properties() :: #rt_properties_v1{}. + +-export_type([properties/0]). + +-define(RT_PROPERTIES, #rt_properties_v1). +-define(RECORD_FIELDS, record_info(fields, rt_properties_v1)). + +-export([new/0, + new/1, + get/2, + set/2, + set/3]). + +%% @doc Create a new properties record with all fields initialized to +%% the default values. +-spec new() -> properties(). +new() -> + ?RT_PROPERTIES{}. + +%% @doc Create a new properties record with the fields initialized to +%% non-default value. Each field to be initialized should be +%% specified as an entry in a property list (i.e. a list of +%% pairs). Invalid property fields are ignored by this function. +-spec new(proplists:proplist()) -> properties(). +new(PropertyDefaults) -> + {Properties, _} = + lists:foldl(fun set_property/2, {?RT_PROPERTIES{}, []}, PropertyDefaults), + Properties. + +%% @doc Get the value of a property from a properties record. An error +%% is returned if `Properties' is not a valid `rt_properties' record +%% or if the property requested is not a valid property. +-spec get(atom(), properties()) -> term() | {error, atom()}. +get(Property, Properties) -> + get(Property, Properties, validate_request(Property, Properties)). + +%% @doc Set the value for a property in a properties record. An error +%% is returned if `Properties' is not a valid `rt_properties' record +%% or if any of the properties to be set are not a valid property. In +%% the case that invalid properties are specified the error returned +%% contains a list of erroneous properties. +-spec set([{atom(), term()}], properties()) -> {ok, properties()} | {error, atom()}. +set(PropertyList, Properties) when is_list(PropertyList) -> + set_properties(PropertyList, Properties, validate_record(Properties)). + +%% @doc Set the value for a property in a properties record. An error +%% is returned if `Properties' is not a valid `rt_properties' record +%% or if the property to be set is not a valid property. +-spec set(atom(), term(), properties()) -> {ok, properties()} | {error, atom()}. +set(Property, Value, Properties) -> + set_property(Property, Value, Properties, validate_request(Property, Properties)). + + +-spec get(atom(), properties(), ok | {error, atom()}) -> + term() | {error, atom()}. +get(Property, Properties, ok) -> + element(field_index(Property), Properties); +get(_Property, _Properties, {error, _}=Error) -> + Error. + +%% This function is used by `new/1' to set properties at record +%% creation time and by `set/2' to set multiple properties at once. +%% Node properties record validation is done by this function. It is +%% strictly used as a fold function which is the reason for the odd +%% structure of the input parameters. It accumulates any invalid +%% properties that are encountered and the caller may use that +%% information or ignore it. +-spec set_property({atom(), term()}, {properties(), [atom()]}) -> + {properties(), [atom()]}. +set_property({Property, Value}, {Properties, Invalid}) -> + case is_valid_property(Property) of + true -> + {setelement(field_index(Property), Properties, Value), Invalid}; + false -> + {Properties, [Property | Invalid]} + end. + +-spec set_property(atom(), term(), properties(), ok | {error, atom()}) -> + {ok, properties()} | {error, atom()}. +set_property(Property, Value, Properties, ok) -> + {ok, setelement(field_index(Property), Properties, Value)}; +set_property(_Property, _Value, _Properties, {error, _}=Error) -> + Error. + +-spec set_properties([{atom(), term()}], + properties(), + ok | {error, {atom(), [atom()]}}) -> + {properties(), [atom()]}. +set_properties(PropertyList, Properties, ok) -> + case lists:foldl(fun set_property/2, {Properties, []}, PropertyList) of + {UpdProperties, []} -> + UpdProperties; + {_, InvalidProperties} -> + {error, {invalid_properties, InvalidProperties}} + end; +set_properties(_, _, {error, _}=Error) -> + Error. + +-spec validate_request(atom(), term()) -> ok | {error, atom()}. +validate_request(Property, Properties) -> + validate_property(Property, validate_record(Properties)). + +-spec validate_record(term()) -> ok | {error, invalid_properties}. +validate_record(Record) -> + case is_valid_record(Record) of + true -> + ok; + false -> + {error, invalid_properties} + end. + +-spec validate_property(atom(), ok | {error, atom()}) -> ok | {error, invalid_property}. +validate_property(Property, ok) -> + case is_valid_property(Property) of + true -> + ok; + false -> + {error, invalid_property} + end; +validate_property(_Property, {error, _}=Error) -> + Error. + +-spec is_valid_record(term()) -> boolean(). +is_valid_record(Record) -> + is_record(Record, rt_properties_v1). + +-spec is_valid_property(atom()) -> boolean(). +is_valid_property(Property) -> + Fields = ?RECORD_FIELDS, + lists:member(Property, Fields). + +-spec field_index(atom()) -> non_neg_integer(). +field_index(nodes) -> + ?RT_PROPERTIES.nodes; +field_index(node_count) -> + ?RT_PROPERTIES.node_count; +field_index(metadata) -> + ?RT_PROPERTIES.metadata; +field_index(properties) -> + ?RT_PROPERTIES.properties; +field_index(rolling_upgrade) -> + ?RT_PROPERTIES.rolling_upgrade; +field_index(start_version) -> + ?RT_PROPERTIES.start_version; +field_index(upgrade_version) -> + ?RT_PROPERTIES.upgrade_version; +field_index(wait_for_transfers) -> + ?RT_PROPERTIES.wait_for_transfers; +field_index(valid_backends) -> + ?RT_PROPERTIES.valid_backends; +field_index(make_cluster) -> + ?RT_PROPERTIES.make_cluster; +field_index(config) -> + ?RT_PROPERTIES.config. diff --git a/tests/basic_command_line.erl b/tests/basic_command_line.erl index c47998586..66547bec0 100644 --- a/tests/basic_command_line.erl +++ b/tests/basic_command_line.erl @@ -24,12 +24,10 @@ -export([properties/0, confirm/2]). properties() -> - DefaultProps = rt_cluster:properties(), - DefaultProps#rt_properties{node_count=1, - rolling_upgrade=false, - make_cluster=true}. + rt_properties:new([{node_count, 1}]). -confirm(#rt_properties{nodes=Nodes}, _MD) -> +confirm(Properties, _MD) -> + Nodes = rt_properties:get(nodes, Properties), Node = hd(Nodes), %% Deploy a node to test against lager:info("Deploy node to test command line"), diff --git a/tests/bucket_props_roundtrip.erl b/tests/bucket_props_roundtrip.erl index 90bd2fbd4..a2d7876c9 100644 --- a/tests/bucket_props_roundtrip.erl +++ b/tests/bucket_props_roundtrip.erl @@ -54,13 +54,10 @@ ]). properties() -> - DefaultProps = rt_cluster:properties(), - DefaultProps#rt_properties{node_count=1, - rolling_upgrade=false, - make_cluster=true}. + rt_properties:new([{node_count, 1}]). -confirm(#rt_properties{nodes=Nodes}, _MD) -> - [Node] = Nodes, +confirm(Properties, _MD) -> + [Node] = rt_properties:get(nodes, Properties), [ check_prop_set_and_get(Node, Prop, FirstVal, SecondVal) || {Prop, FirstVal, SecondVal} <- ?PROPS ], diff --git a/tests/bucket_types.erl b/tests/bucket_types.erl index 118c96323..fea36b1ec 100644 --- a/tests/bucket_types.erl +++ b/tests/bucket_types.erl @@ -6,16 +6,14 @@ -include("rt.hrl"). properties() -> - DefaultProps = rt_cluster:properties(), - CustomConfig = rt_cluster:augment_config(riak_core, + CustomConfig = rt_cluster:augment_config(riak_core, {default_bucket_props, [{n_val, 2}]}, - DefaultProps#rt_properties.config), - DefaultProps#rt_properties{node_count=4, - rolling_upgrade=false, - make_cluster=true, - config=CustomConfig}. + rt_cluster:config()), + rt_properties:new([{node_count, 4}, + {config, CustomConfig}]). -confirm(#rt_properties{nodes=Nodes}, _MD) -> +confirm(Properties, _MD) -> + Nodes = rt_properties:get(nodes, Properties), Node = hd(Nodes), application:start(inets), @@ -51,7 +49,7 @@ confirm(#rt_properties{nodes=Nodes}, _MD) -> %% write implicitly to the default bucket riakc_pb_socket:put(PB, riakc_obj:update_value(O1, <<"newvalue">>)), - + %% read from the default bucket explicitly {ok, O3} = riakc_pb_socket:get(PB, {<<"default">>, <<"bucket">>}, <<"key">>), diff --git a/tests/cluster_meta_basic.erl b/tests/cluster_meta_basic.erl index cb4cf0935..f4012a0b4 100644 --- a/tests/cluster_meta_basic.erl +++ b/tests/cluster_meta_basic.erl @@ -31,12 +31,10 @@ -define(VAL2, val2). properties() -> - DefaultProps = rt_cluster:properties(), - DefaultProps#rt_properties{node_count=5, - rolling_upgrade=false, - make_cluster=true}. + rt_properties:new([{node_count, 5}]). -confirm(#rt_properties{nodes=Nodes}, _MD) -> +confirm(Properties, _MD) -> + Nodes = rt_properties:get(nodes, Properties), ok = test_fold_full_prefix(Nodes), ok = test_metadata_conflicts(Nodes), ok = test_writes_after_partial_cluster_failure(Nodes), diff --git a/tests/http_bucket_types.erl b/tests/http_bucket_types.erl index 668de357a..da61c3aef 100644 --- a/tests/http_bucket_types.erl +++ b/tests/http_bucket_types.erl @@ -7,16 +7,15 @@ -include("rt.hrl"). properties() -> - DefaultProps = rt_cluster:properties(), - CustomConfig = rt_cluster:augment_config(riak_core, + CustomConfig = rt_cluster:augment_config(riak_core, {default_bucket_props, [{n_val, 2}]}, - DefaultProps#rt_properties.config), - DefaultProps#rt_properties{node_count=1, - rolling_upgrade=false, - make_cluster=true, - config=CustomConfig}. + rt_cluster:config()), + rt_properties:new([{node_count, 1}, + {config, CustomConfig}]). -confirm(#rt_properties{nodes=Nodes}, _MD) -> + +confirm(Properties, _MD) -> + Nodes = rt_properties:get(nodes, Properties), Node = hd(Nodes), application:start(ibrowse), diff --git a/tests/secondary_index_tests.erl b/tests/secondary_index_tests.erl index 4ad75f1c5..df58b8f41 100644 --- a/tests/secondary_index_tests.erl +++ b/tests/secondary_index_tests.erl @@ -34,19 +34,17 @@ -define(KEYS(A,B,G1,G2), [int_to_key(N) || N <- lists:seq(A,B), G1, G2]). properties() -> - DefaultProps = rt_cluster:properties(), - DefaultProps#rt_properties{node_count=3, - wait_for_transfers=true, - rolling_upgrade=false, - start_version=previous, - config=config()}. + rt_properties:new([{node_count, 3}, + {wait_for_transfers, true}, + {start_version, previous}, + {config, config()}]). config() -> - [ - {riak_kv, [{secondary_index_sort_default, false}]} - ]. + [{riak_kv, [{secondary_index_sort_default, false}]}, + {riak_core, [{handoff_concurrency, 11}]}]. -confirm(#rt_properties{nodes=Nodes}, _MD) -> +confirm(Properties, _MD) -> + Nodes = rt_properties:get(nodes, Properties), Bucket = druuid:v4_str(), lager:info("Bucket: ~p", [Bucket]), PBC = rt_pb:pbc(hd(Nodes)), diff --git a/tests/verify_build_cluster.erl b/tests/verify_build_cluster.erl index c502e6d90..fac763314 100644 --- a/tests/verify_build_cluster.erl +++ b/tests/verify_build_cluster.erl @@ -24,18 +24,17 @@ -include_lib("eunit/include/eunit.hrl"). properties() -> - DefaultProps = rt_cluster:properties(), UpdConfig = rt_cluster:augment_config(riak_core, {default_bucket_props, [{allow_mult, false}]}, - DefaultProps#rt_properties.config), - DefaultProps#rt_properties{config=UpdConfig, - node_count=4, - rolling_upgrade=true, - make_cluster=false, - start_version=previous}. + rt_cluster:config()), + rt_properties:new([{config, UpdConfig}, + {node_count, 4}, + {rolling_upgrade, true}, + {make_cluster, false}, + {start_version, previous}]). -confirm(#rt_properties{nodes=Nodes}, _MD) -> - [Node1, Node2, Node3, Node4] = Nodes, +confirm(Properties, _MD) -> + [Node1, Node2, Node3, Node4] = Nodes = rt_properties:get(nodes, Properties), lager:info("Loading some data up in this cluster."), ?assertEqual([], rt_systest:write(Node1, 0, 1000, <<"verify_build_cluster">>, 2)), From 9649d09eef52b73f2406b7ac79d130ab89b61051 Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Fri, 1 Aug 2014 14:37:35 -0400 Subject: [PATCH 028/157] Take first pass at harness refactoring. --- src/rt.erl | 23 ++-- src/rt_backend.erl | 2 +- src/rt_cluster.erl | 10 +- src/rt_cmd_line.erl | 12 +- src/rt_config.erl | 12 +- src/rt_harness.erl | 131 +++++++++++++++++++ src/rt_harness_util.erl | 283 ++++++++++++++++++++++++++++++++++++++++ src/rt_node.erl | 8 +- src/rtdev.erl | 28 +++- src/rtperf.erl | 68 ++++++++++ src/rtssh.erl | 80 ++++++++---- src/test_harness.erl | 48 +++++++ 12 files changed, 646 insertions(+), 59 deletions(-) create mode 100644 src/rt_harness.erl create mode 100644 src/rt_harness_util.erl create mode 100644 src/test_harness.erl diff --git a/src/rt.erl b/src/rt.erl index 2a4509237..0ec1584b4 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -156,7 +156,7 @@ PrivDir. %% @doc gets riak deps from the appropriate harness -spec get_deps() -> list(). -get_deps() -> ?HARNESS:get_deps(). +get_deps() -> rt_harness:get_deps(). %% @doc if String contains Substr, return true. -spec str(string(), string()) -> boolean(). @@ -212,26 +212,26 @@ maybe_wait_for_changes(Node) -> %% @doc Spawn `Cmd' on the machine running the test harness spawn_cmd(Cmd) -> -?HARNESS:spawn_cmd(Cmd). + rt_harness:spawn_cmd(Cmd). %% @doc Spawn `Cmd' on the machine running the test harness spawn_cmd(Cmd, Opts) -> -?HARNESS:spawn_cmd(Cmd, Opts). + rt_harness:spawn_cmd(Cmd, Opts). %% @doc Wait for a command spawned by `spawn_cmd', returning %% the exit status and result wait_for_cmd(CmdHandle) -> -?HARNESS:wait_for_cmd(CmdHandle). + rt_harness:wait_for_cmd(CmdHandle). %% @doc Spawn `Cmd' on the machine running the test harness, returning %% the exit status and result cmd(Cmd) -> -?HARNESS:cmd(Cmd). + rt_harness:cmd(Cmd). %% @doc Spawn `Cmd' on the machine running the test harness, returning %% the exit status and result cmd(Cmd, Opts) -> -?HARNESS:cmd(Cmd, Opts). + rt_harness:cmd(Cmd, Opts). %% @doc pretty much the same as os:cmd/1 but it will stream the output to lager. %% If you're running a long running command, it will dump the output @@ -886,15 +886,15 @@ lager:info("Installing search hook for bucket ~p", [Bucket]), %% or something like that, it's the version you're upgrading to. -spec get_version() -> binary(). get_version() -> -?HARNESS:get_version(). + rt_harness:get_version(). %% @doc outputs some useful information about nodes that are up whats_up() -> -?HARNESS:whats_up(). + rt_harness:whats_up(). -spec get_ip(node()) -> string(). get_ip(Node) -> -?HARNESS:get_ip(Node). + rt_harness:get_ip(Node). %% @doc Log a message to the console of the specified test nodes. %% Messages are prefixed by the string "---riak_test--- " @@ -936,12 +936,12 @@ L3. %% @private setup_harness(Test, Args) -> -?HARNESS:setup_harness(Test, Args). + rt_harness:setup_harness(Test, Args). %% @doc Downloads any extant log files from the harness's running %% nodes. get_node_logs() -> -?HARNESS:get_node_logs(). + rt_harness:get_node_logs(). check_ibrowse() -> try sys:get_status(ibrowse) of @@ -1101,6 +1101,7 @@ random_sublist(List, N) -> % Remove the random numbers. [ E || {_,E} <- List3]. +%% TODO: Is this the right location for this? -ifdef(TEST). verify_product(Applications, ExpectedApplication) -> diff --git a/src/rt_backend.erl b/src/rt_backend.erl index 59f42739d..3fe13e64d 100644 --- a/src/rt_backend.erl +++ b/src/rt_backend.erl @@ -78,7 +78,7 @@ make_multi_backend_config(Other) -> make_multi_backend_config(default). get_backends() -> - Backends = ?HARNESS:get_backends(), + Backends = rt_harness:get_backends(), case Backends of [riak_kv_bitcask_backend] -> bitcask; [riak_kv_eleveldb_backend] -> eleveldb; diff --git a/src/rt_cluster.erl b/src/rt_cluster.erl index cdd507c3e..a4adcbc5f 100644 --- a/src/rt_cluster.erl +++ b/src/rt_cluster.erl @@ -93,7 +93,7 @@ deploy_nodes(NumNodes, InitialConfig) when is_integer(NumNodes) -> deploy_nodes(NodeConfig); deploy_nodes(Versions, Services) -> NodeConfig = [ rt_config:version_to_config(Version) || Version <- Versions ], - Nodes = ?HARNESS:deploy_nodes(NodeConfig), + Nodes = rt_harness:deploy_nodes(NodeConfig), lager:info("Waiting for services ~p to start on ~p.", [Services, Nodes]), [ ok = rt:wait_for_service(Node, Service) || Node <- Nodes, Service <- Services ], @@ -110,7 +110,7 @@ deploy_clusters(Settings) -> {NumNodes, Vsn, InitialConfig} when is_integer(NumNodes) -> [{Vsn, InitialConfig} || _ <- lists:seq(1,NumNodes)] end || Setting <- Settings], - ?HARNESS:deploy_clusters(ClusterConfigs). + rt_harness:deploy_clusters(ClusterConfigs). build_clusters(Settings) -> Clusters = deploy_clusters(Settings), @@ -201,7 +201,7 @@ clean_data_dir(Nodes) -> clean_data_dir(Nodes, SubDir) when not is_list(Nodes) -> clean_data_dir([Nodes], SubDir); clean_data_dir(Nodes, SubDir) when is_list(Nodes) -> - ?HARNESS:clean_data_dir(Nodes, SubDir). + rt_harness:clean_data_dir(Nodes, SubDir). %% @doc Shutdown every node, this is for after a test run is complete. teardown() -> @@ -210,10 +210,10 @@ teardown() -> %%[ rt_node:stop(Node) || Node <- nodes()], %% Then do the more exhaustive harness thing, in case something was up %% but not connected. - ?HARNESS:teardown(). + rt_harness:teardown(). versions() -> - ?HARNESS:versions(). + rt_harness:versions(). config() -> [{riak_core, [{handoff_concurrency, 11}]}, diff --git a/src/rt_cmd_line.erl b/src/rt_cmd_line.erl index d4022f1ae..f422e68fb 100644 --- a/src/rt_cmd_line.erl +++ b/src/rt_cmd_line.erl @@ -36,16 +36,16 @@ %% @doc Call 'bin/riak-admin' command on `Node' with arguments `Args' admin(Node, Args) -> - ?HARNESS:admin(Node, Args). + rt_harness:admin(Node, Args). %% @doc Call 'bin/riak' command on `Node' with arguments `Args' riak(Node, Args) -> - ?HARNESS:riak(Node, Args). + rt_harness:riak(Node, Args). %% @doc Call 'bin/riak-repl' command on `Node' with arguments `Args' riak_repl(Node, Args) -> - ?HARNESS:riak_repl(Node, Args). + rt_harness:riak_repl(Node, Args). search_cmd(Node, Args) -> {ok, Cwd} = file:get_cwd(), @@ -65,14 +65,14 @@ search_cmd(Node, Args) -> %% expect will process based on the output following the sent data. %% attach(Node, Expected) -> - ?HARNESS:attach(Node, Expected). + rt_harness:attach(Node, Expected). %% @doc Runs 'riak attach-direct' on a specific node %% @see rt_cmd_line:attach/2 attach_direct(Node, Expected) -> - ?HARNESS:attach_direct(Node, Expected). + rt_harness:attach_direct(Node, Expected). %% @doc Runs `riak console' on a specific node %% @see rt_cmd_line:attach/2 console(Node, Expected) -> - ?HARNESS:console(Node, Expected). + rt_harness:console(Node, Expected). diff --git a/src/rt_config.erl b/src/rt_config.erl index 28ffa6cb5..f6e5e8867 100644 --- a/src/rt_config.erl +++ b/src/rt_config.erl @@ -131,30 +131,30 @@ config_or_os_env(Config, Default) -> -spec set_conf(atom(), [{string(), string()}]) -> ok. set_conf(all, NameValuePairs) -> - ?HARNESS:set_conf(all, NameValuePairs); + rt_harness:set_conf(all, NameValuePairs); set_conf(Node, NameValuePairs) -> rt_node:stop(Node), ?assertEqual(ok, rt:wait_until_unpingable(Node)), - ?HARNESS:set_conf(Node, NameValuePairs), + rt_harness:set_conf(Node, NameValuePairs), rt_node:start(Node). -spec set_advanced_conf(atom(), [{string(), string()}]) -> ok. set_advanced_conf(all, NameValuePairs) -> - ?HARNESS:set_advanced_conf(all, NameValuePairs); + rt_harness:set_advanced_conf(all, NameValuePairs); set_advanced_conf(Node, NameValuePairs) -> rt_node:stop(Node), ?assertEqual(ok, rt:wait_until_unpingable(Node)), - ?HARNESS:set_advanced_conf(Node, NameValuePairs), + rt_harness:set_advanced_conf(Node, NameValuePairs), rt_node:start(Node). %% @doc Rewrite the given node's app.config file, overriding the varialbes %% in the existing app.config with those in `Config'. update_app_config(all, Config) -> - ?HARNESS:update_app_config(all, Config); + rt_harness:update_app_config(all, Config); update_app_config(Node, Config) -> rt_node:stop(Node), ?assertEqual(ok, rt:wait_until_unpingable(Node)), - ?HARNESS:update_app_config(Node, Config), + rt_harness:update_app_config(Node, Config), rt_node:start(Node). version_to_config(Config) when is_tuple(Config)-> Config; diff --git a/src/rt_harness.erl b/src/rt_harness.erl new file mode 100644 index 000000000..cd573e9c4 --- /dev/null +++ b/src/rt_harness.erl @@ -0,0 +1,131 @@ +%% ------------------------------------------------------------------- +%% +%% Copyright (c) 2013-2014 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(rt_harness). + +-define(HARNESS_MODULE, (rt_config:get(rt_harness))). + +-export([ + start/1, + stop/1, + deploy_clusters/1, + clean_data_dir/2, + deploy_nodes/1, + spawn_cmd/1, + spawn_cmd/2, + cmd/1, + cmd/2, + setup_harness/2, + get_deps/0, + get_version/0, + get_backends/0, + set_backend/1, + whats_up/0, + get_ip/1, + node_id/1, + node_version/1, + admin/2, + riak/2, + attach/2, + attach_direct/2, + console/2, + update_app_config/2, + teardown/0, + set_conf/2, + set_advanced_conf/2]). + +start(Node) -> + ?HARNESS_MODULE:start(Node). + +stop(Node) -> + ?HARNESS_MODULE:stop(Node). + +deploy_clusters(ClusterConfigs) -> + ?HARNESS_MODULE:deploy_clusters(ClusterConfigs). + +clean_data_dir(Nodes, SubDir) -> + ?HARNESS_MODULE:clean_data_dir(Nodes, SubDir). + +spawn_cmd(Cmd) -> + ?HARNESS_MODULE:spawn_cmd(Cmd). + +spawn_cmd(Cmd, Opts) -> + ?HARNESS_MODULE:spawn_cmd(Cmd, Opts). + +cmd(Cmd) -> + ?HARNESS_MODULE:cmd(Cmd). + +cmd(Cmd, Opts) -> + ?HARNESS_MODULE:cmd(Cmd, Opts). + +deploy_nodes(NodeConfig) -> + ?HARNESS_MODULE:deploy_nodes(NodeConfig). + +setup_harness(Test, Args) -> + ?HARNESS_MODULE:setup_harness(Test, Args). + +get_deps() -> + ?HARNESS_MODULE:get_deps(). +get_version() -> + ?HARNESS_MODULE:get_version(). + +get_backends() -> + ?HARNESS_MODULE:get_backends(). + +set_backend(Backend) -> + ?HARNESS_MODULE:set_backend(Backend). + +whats_up() -> + ?HARNESS_MODULE:whats_up(). + +get_ip(Node) -> + ?HARNESS_MODULE:get_ip(Node). + +node_id(Node) -> + ?HARNESS_MODULE:node_id(Node). + +node_version(N) -> + ?HARNESS_MODULE:node_version(N). + +admin(Node, Args) -> + ?HARNESS_MODULE:admin(Node, Args). + +riak(Node, Args) -> + ?HARNESS_MODULE:riak(Node, Args). + +attach(Node, Expected) -> + ?HARNESS_MODULE:attach(Node, Expected). + +attach_direct(Node, Expected) -> + ?HARNESS_MODULE:attach_direct(Node, Expected). + +console(Node, Expected) -> + ?HARNESS_MODULE:console(Node, Expected). + +update_app_config(Node, Config) -> + ?HARNESS_MODULE:update_app_config(Node, Config). + +teardown() -> + ?HARNESS_MODULE:teardown(). + +set_conf(Node, NameValuePairs) -> + ?HARNESS_MODULE:set_conf(Node, NameValuePairs). + +set_advanced_conf(Node, NameValuePairs) -> + ?HARNESS_MODULE:set_advanced_conf(Node, NameValuePairs). diff --git a/src/rt_harness_util.erl b/src/rt_harness_util.erl new file mode 100644 index 000000000..c636c0cbf --- /dev/null +++ b/src/rt_harness_util.erl @@ -0,0 +1,283 @@ +%% ------------------------------------------------------------------- +%% +%% Copyright (c) 2013-2014 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(rt_harness_util). + +-include_lib("eunit/include/eunit.hrl"). +-define(DEVS(N), lists:concat(["dev", N, "@127.0.0.1"])). +-define(DEV(N), list_to_atom(?DEVS(N))). +-define(PATH, (rt_config:get(rtdev_path))). + +-export([admin/2, + attach/2, + attach_direct/2, + cmd/1, + cmd/2, + console/2, + get_ip/1, + node_id/1, + node_version/1, + riak/2, + set_conf/2, + spawn_cmd/1, + spawn_cmd/2, + whats_up/0]). + +admin(Node, Args) -> + N = node_id(Node), + Path = relpath(node_version(N)), + Cmd = riak_admin_cmd(Path, N, Args), + lager:info("Running: ~s", [Cmd]), + Result = os:cmd(Cmd), + lager:info("~s", [Result]), + {ok, Result}. + +attach(Node, Expected) -> + interactive(Node, "attach", Expected). + +attach_direct(Node, Expected) -> + interactive(Node, "attach-direct", Expected). + +console(Node, Expected) -> + interactive(Node, "console", Expected). + +interactive(Node, Command, Exp) -> + N = node_id(Node), + Path = relpath(node_version(N)), + Cmd = riakcmd(Path, N, Command), + lager:info("Opening a port for riak ~s.", [Command]), + lager:debug("Calling open_port with cmd ~s", [binary_to_list(iolist_to_binary(Cmd))]), + P = open_port({spawn, binary_to_list(iolist_to_binary(Cmd))}, + [stream, use_stdio, exit_status, binary, stderr_to_stdout]), + interactive_loop(P, Exp). + +interactive_loop(Port, Expected) -> + receive + {Port, {data, Data}} -> + %% We've gotten some data, so the port isn't done executing + %% Let's break it up by newline and display it. + Tokens = string:tokens(binary_to_list(Data), "\n"), + [lager:debug("~s", [Text]) || Text <- Tokens], + + %% Now we're going to take hd(Expected) which is either {expect, X} + %% or {send, X}. If it's {expect, X}, we foldl through the Tokenized + %% data looking for a partial match via rt:str/2. If we find one, + %% we pop hd off the stack and continue iterating through the list + %% with the next hd until we run out of input. Once hd is a tuple + %% {send, X}, we send that test to the port. The assumption is that + %% once we send data, anything else we still have in the buffer is + %% meaningless, so we skip it. That's what that {sent, sent} thing + %% is about. If there were a way to abort mid-foldl, I'd have done + %% that. {sent, _} -> is just a pass through to get out of the fold. + + NewExpected = lists:foldl(fun(X, Expect) -> + [{Type, Text}|RemainingExpect] = case Expect of + [] -> [{done, "done"}|[]]; + E -> E + end, + case {Type, rt:str(X, Text)} of + {expect, true} -> + RemainingExpect; + {expect, false} -> + [{Type, Text}|RemainingExpect]; + {send, _} -> + port_command(Port, list_to_binary(Text ++ "\n")), + [{sent, "sent"}|RemainingExpect]; + {sent, _} -> + Expect; + {done, _} -> + [] + end + end, Expected, Tokens), + %% Now that the fold is over, we should remove {sent, sent} if it's there. + %% The fold might have ended not matching anything or not sending anything + %% so it's possible we don't have to remove {sent, sent}. This will be passed + %% to interactive_loop's next iteration. + NewerExpected = case NewExpected of + [{sent, "sent"}|E] -> E; + E -> E + end, + %% If NewerExpected is empty, we've met all expected criteria and in order to boot + %% Otherwise, loop. + case NewerExpected of + [] -> ?assert(true); + _ -> interactive_loop(Port, NewerExpected) + end; + {Port, {exit_status,_}} -> + %% This port has exited. Maybe the last thing we did was {send, [4]} which + %% as Ctrl-D would have exited the console. If Expected is empty, then + %% We've met every expectation. Yay! If not, it means we've exited before + %% something expected happened. + ?assertEqual([], Expected) + after rt_config:get(rt_max_wait_time) -> + %% interactive_loop is going to wait until it matches expected behavior + %% If it doesn't, the test should fail; however, without a timeout it + %% will just hang forever in search of expected behavior. See also: Parenting + ?assertEqual([], Expected) + end. + +spawn_cmd(Cmd) -> + spawn_cmd(Cmd, []). +spawn_cmd(Cmd, Opts) -> + Port = open_port({spawn, lists:flatten(Cmd)}, [stream, in, exit_status] ++ Opts), + Port. + +wait_for_cmd(Port) -> + rt:wait_until(node(), + fun(_) -> + receive + {Port, Msg={exit_status, _}} -> + catch port_close(Port), + self() ! {Port, Msg}, + true + after 0 -> + false + end + end), + get_cmd_result(Port, []). + +cmd(Cmd) -> + cmd(Cmd, []). + +cmd(Cmd, Opts) -> + wait_for_cmd(spawn_cmd(Cmd, Opts)). + +get_cmd_result(Port, Acc) -> + receive + {Port, {data, Bytes}} -> + get_cmd_result(Port, [Bytes|Acc]); + {Port, {exit_status, Status}} -> + Output = lists:flatten(lists:reverse(Acc)), + {Status, Output} + after 0 -> + timeout + end. + +get_ip(_Node) -> + %% localhost 4 lyfe + "127.0.0.1". + +node_id(Node) -> + NodeMap = rt_config:get(rt_nodes), + orddict:fetch(Node, NodeMap). + +node_version(N) -> + VersionMap = rt_config:get(rt_versions), + orddict:fetch(N, VersionMap). + +riak(Node, Args) -> + N = node_id(Node), + Path = relpath(node_version(N)), + Result = run_riak(N, Path, Args), + lager:info("~s", [Result]), + {ok, Result}. + +-spec set_conf(atom() | string(), [{string(), string()}]) -> ok. +set_conf(all, NameValuePairs) -> + lager:info("rtdev:set_conf(all, ~p)", [NameValuePairs]), + [ set_conf(DevPath, NameValuePairs) || DevPath <- devpaths()], + ok; +set_conf(Node, NameValuePairs) when is_atom(Node) -> + append_to_conf_file(get_riak_conf(Node), NameValuePairs), + ok; +set_conf(DevPath, NameValuePairs) -> + [append_to_conf_file(RiakConf, NameValuePairs) || RiakConf <- all_the_files(DevPath, "etc/riak.conf")], + ok. + +whats_up() -> + io:format("Here's what's running...~n"), + + Up = [rpc:call(Node, os, cmd, ["pwd"]) || Node <- nodes()], + [io:format(" ~s~n",[string:substr(Dir, 1, length(Dir)-1)]) || Dir <- Up]. + +riak_admin_cmd(Path, N, Args) -> + Quoted = + lists:map(fun(Arg) when is_list(Arg) -> + lists:flatten([$", Arg, $"]); + (_) -> + erlang:error(badarg) + end, Args), + ArgStr = string:join(Quoted, " "), + ExecName = rt_config:get(exec_name, "riak"), + io_lib:format("~s/dev/dev~b/bin/~s-admin ~s", [Path, N, ExecName, ArgStr]). + +% Private functions + +relpath(Vsn) -> + Path = ?PATH, + relpath(Vsn, Path). + +relpath(Vsn, Paths=[{_,_}|_]) -> + orddict:fetch(Vsn, orddict:from_list(Paths)); +relpath(current, Path) -> + Path; +relpath(root, Path) -> + Path; +relpath(_, _) -> + throw("Version requested but only one path provided"). + +riakcmd(Path, N, Cmd) -> + ExecName = rt_config:get(exec_name, "riak"), + io_lib:format("~s/dev/dev~b/bin/~s ~s", [Path, N, ExecName, Cmd]). + +run_riak(N, Path, Cmd) -> + lager:info("Running: ~s", [riakcmd(Path, N, Cmd)]), + R = os:cmd(riakcmd(Path, N, Cmd)), + case Cmd of + "start" -> + rt_cover:maybe_start_on_node(?DEV(N), node_version(N)), + %% Intercepts may load code on top of the cover compiled + %% modules. We'll just get no coverage info then. + case rt_intercept:are_intercepts_loaded(?DEV(N)) of + false -> + ok = rt_intercept:load_intercepts([?DEV(N)]); + true -> + ok + end, + R; + "stop" -> + rt_cover:maybe_stop_on_node(?DEV(N)), + R; + _ -> + R + end. + +append_to_conf_file(File, NameValuePairs) -> + Settings = lists:flatten( + [io_lib:format("~n~s = ~s~n", [Name, Value]) || {Name, Value} <- NameValuePairs]), + file:write_file(File, Settings, [append]). + +get_riak_conf(Node) -> + N = node_id(Node), + Path = relpath(node_version(N)), + io_lib:format("~s/dev/dev~b/etc/riak.conf", [Path, N]). + +all_the_files(DevPath, File) -> + case filelib:is_dir(DevPath) of + true -> + Wildcard = io_lib:format("~s/dev/dev*/~s", [DevPath, File]), + filelib:wildcard(Wildcard); + _ -> + lager:debug("~s is not a directory.", [DevPath]), + [] + end. + +devpaths() -> + lists:usort([ DevPath || {_Name, DevPath} <- proplists:delete(root, rt_config:get(rtdev_path))]). diff --git a/src/rt_node.erl b/src/rt_node.erl index 21b204035..e52316035 100644 --- a/src/rt_node.erl +++ b/src/rt_node.erl @@ -49,7 +49,7 @@ %% @doc Start the specified Riak node start(Node) -> - ?HARNESS:start(Node). + rt_harness:start(Node). %% @doc Start the specified Riak `Node' and wait for it to be pingable start_and_wait(Node) -> @@ -63,7 +63,7 @@ async_start(Node) -> stop(Node) -> lager:info("Stopping riak on ~p", [Node]), timer:sleep(10000), %% I know, I know! - ?HARNESS:stop(Node). + rt_harness:stop(Node). %%rpc:call(Node, init, stop, []). %% @doc Stop the specified Riak `Node' and wait until it is not pingable @@ -73,12 +73,12 @@ stop_and_wait(Node) -> %% @doc Upgrade a Riak `Node' to the specified `NewVersion'. upgrade(Node, NewVersion) -> - ?HARNESS:upgrade(Node, NewVersion). + rt_harness:upgrade(Node, NewVersion). %% @doc Upgrade a Riak `Node' to the specified `NewVersion' and update %% the config based on entries in `Config'. upgrade(Node, NewVersion, Config) -> - ?HARNESS:upgrade(Node, NewVersion, Config). + rt_harness:upgrade(Node, NewVersion, Config). %% @doc Upgrade a Riak node to a specific version using the alternate %% leave/upgrade/rejoin approach diff --git a/src/rtdev.erl b/src/rtdev.erl index 9530dbb1d..76b56793e 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -18,8 +18,34 @@ %% %% ------------------------------------------------------------------- -%% @private -module(rtdev). +-behaviour(test_harness). +-export([start/1, + stop/1, + deploy_clusters/1, + clean_data_dir/2, + spawn_cmd/1, + spawn_cmd/2, + cmd/1, + cmd/2, + setup_harness/2, + get_version/0, + get_backends/0, + set_backend/1, + whats_up/0, + get_ip/1, + node_id/1, + node_version/1, + admin/2, + riak/2, + attach/2, + attach_direct/2, + console/2, + update_app_config/2, + teardown/0, + set_conf/2, + set_advanced_conf/2]). + -compile(export_all). -include_lib("eunit/include/eunit.hrl"). diff --git a/src/rtperf.erl b/src/rtperf.erl index 2f132d588..ee109fffe 100644 --- a/src/rtperf.erl +++ b/src/rtperf.erl @@ -1,9 +1,67 @@ -module(rtperf). +-behaviour(test_harness). + +-export([start/1, + stop/1, + deploy_clusters/1, + clean_data_dir/2, + spawn_cmd/1, + spawn_cmd/2, + cmd/1, + cmd/2, + setup_harness/2, + get_version/0, + get_backends/0, + set_backend/1, + whats_up/0, + get_ip/1, + node_id/1, + node_version/1, + admin/2, + riak/2, + attach/2, + attach_direct/2, + console/2, + update_app_config/2, + teardown/0, + set_conf/2, + set_advanced_conf/2]). + -compile(export_all). -include_lib("eunit/include/eunit.hrl"). -include_lib("kernel/include/file.hrl"). +admin(Node, Args) -> + rt_harness_util:admin(Node, Args). + +attach(Node, Expected) -> + rt_harness_util:attach(Node, Expected). + +attach_direct(Node, Expected) -> + rt_harness_util:attach_direct(Node, Expected). + +cmd(Cmd, Opts) -> + rt_harness_util:cmd(Cmd, Opts). + +console(Node, Expected) -> + rt_harness_util:console(Node, Expected). + +get_ip(Node) -> + rt_harness_util:get_ip(Node). + +node_id(Node) -> + rt_harness_util:get_ip(Node). + +node_version(N) -> + rt_harness_util:node_version(N). + +riak(Node, Args) -> + rt_harness_util:riak(Node, Args). + +set_conf(Node, NameValuePairs) -> + rt_harness_util:set_conf(Node, NameValuePairs). + update_app_config(Node, Config) -> rtssh:update_app_config(Node, Config). @@ -401,3 +459,13 @@ deploy_nodes(NodeConfig, Hosts) -> start(Node) -> rtssh:start(Node). + +spawn_cmd(Cmd) -> + rt_harness_util:spawn_cmd(Cmd). + +spawn_cmd(Cmd, Opts) -> + rt_harness_util:spawn_cmd(Cmd, Opts). + +whats_up() -> + rt_harness_util:whats_up(). + diff --git a/src/rtssh.erl b/src/rtssh.erl index e5605370a..c24b07f6e 100644 --- a/src/rtssh.erl +++ b/src/rtssh.erl @@ -1,7 +1,39 @@ -module(rtssh). +-behaviour(test_harness). + +-export([start/1, + stop/1, + deploy_clusters/1, + clean_data_dir/2, + spawn_cmd/1, + spawn_cmd/2, + cmd/1, + cmd/2, + setup_harness/2, + get_deps/0, + get_version/0, + get_backends/0, + set_backend/1, + whats_up/0, + get_ip/1, + node_id/1, + node_version/1, + admin/2, + riak/2, + attach/2, + attach_direct/2, + console/2, + update_app_config/2, + teardown/0, + set_conf/2, + set_advanced_conf/2]). + -compile(export_all). -include_lib("eunit/include/eunit.hrl"). +admin(Node, Args) -> + rt_harness_util:admin(Node, Args). + get_version() -> unknown. @@ -300,29 +332,6 @@ remote_cmd(Node, Cmd) -> {0, Result} = ssh_cmd(Node, Cmd), {ok, Result}. -admin(Node, Args) -> - Cmd = riak_admin_cmd(Node, Args), - lager:info("Running: ~s :: ~s", [get_host(Node), Cmd]), - {0, Result} = ssh_cmd(Node, Cmd), - lager:info("~s", [Result]), - {ok, Result}. - -admin(Node, Args, Options) -> - Cmd = riak_admin_cmd(Node, Args), - lager:info("Running: ~s :: ~s", [get_host(Node), Cmd]), - Result = execute_admin_cmd(Node, Cmd, Options), - lager:info("~s", [Result]), - {ok, Result}. - -execute_admin_cmd(Node, Cmd, Options) -> - {_ExitCode, Result} = FullResult = ssh_cmd(Node, Cmd), - case lists:member(return_exit_code, Options) of - true -> - FullResult; - false -> - Result - end. - riak(Node, Args) -> Result = run_riak(Node, Args), lager:info("~s", [Result]), @@ -692,8 +701,29 @@ node_id(_Node) -> %% orddict:fetch(Node, NodeMap). 1. -node_version(Node) -> - orddict:fetch(Node, rt_config:get(rt_versions)). +set_backend(Backend) -> + set_backend(Backend, []). + +set_backend(Backend, OtherOpts) -> + lager:info("rtssh:set_backend(~p, ~p)", [Backend, OtherOpts]), + Opts = [{storage_backend, Backend} | OtherOpts], + update_app_config(all, [{riak_kv, Opts}]), + get_backends(). + +whats_up() -> + io:format("Here's what's running...~n"), + + Up = [rpc:call(Node, os, cmd, ["pwd"]) || Node <- nodes()], + [io:format(" ~s~n",[string:substr(Dir, 1, length(Dir)-1)]) || Dir <- Up]. + +node_version(_Node) -> ok. + +attach(_Node, _Expected) -> ok. + +attach_direct(_Node, _Expected) -> ok. + +console(_Node, _Expected) -> ok. + %%%=================================================================== %%% Local command spawning diff --git a/src/test_harness.erl b/src/test_harness.erl new file mode 100644 index 000000000..53474841c --- /dev/null +++ b/src/test_harness.erl @@ -0,0 +1,48 @@ +%% ------------------------------------------------------------------- +%% +%% Copyright (c) 2013-2014 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(test_harness). + +-callback start(Node :: node()) -> 'ok'. +-callback stop(Node :: node()) -> 'ok'. +-callback deploy_clusters(ClusterConfigs :: list()) -> list(). +-callback clean_data_dir(Nodes :: list(), SubDir :: string()) -> 'ok'. +-callback spawn_cmd(Cmd :: string()) -> Port :: pos_integer(). +-callback spawn_cmd(Cmd :: string(), Opts :: list()) -> Port :: pos_integer(). +-callback cmd(Cmd :: string()) -> term()|timeout. +-callback cmd(Cmd :: string(), Opts :: [atom()]) -> term()|timeout. +-callback setup_harness(Test :: string(), Args :: list()) -> 'ok'. +-callback get_version() -> term(). +-callback get_backends() -> [atom()]. +-callback set_backend(Backend :: atom()) -> [atom()]. +-callback whats_up() -> string(). +-callback get_ip(Node :: node()) -> string(). +-callback node_id(Node :: node()) -> NodeMap :: term(). +-callback node_version(N :: node()) -> VersionMap :: term(). +-callback admin(Node :: node(), Args :: [atom()]) -> {'ok', string()}. +-callback riak(Node :: node(), Args :: [atom()]) -> {'ok', string()}. +-callback attach(Node :: node(), Expected:: list()) -> 'ok'. +-callback attach_direct(Node :: node(), Expected:: list()) -> 'ok'. +-callback console(Node :: node(), Expected:: list()) -> 'ok'. +-callback update_app_config(atom()|node(), Config :: term()) -> 'ok'. +-callback teardown() -> list(). +-callback set_conf(atom()|node(), NameValuePairs :: [{string(), string()}]) -> 'ok'. +-callback set_advanced_conf(atom()|node(), NameValuePairs :: [{string(), string()}]) -> 'ok'. + + From 87b492d40544f9195d237244e723258bab651f22 Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Fri, 1 Aug 2014 15:54:18 -0400 Subject: [PATCH 029/157] Convert rt_cs_dev harness to test_harness behaviour. --- src/rt_cs_dev.erl | 77 ++++++++++++------- src/rt_harness_util.erl | 166 +++++++++++++++++++++++++++++++++++++++- src/rtssh.erl | 13 ++-- 3 files changed, 220 insertions(+), 36 deletions(-) diff --git a/src/rt_cs_dev.erl b/src/rt_cs_dev.erl index 9a36d62ce..51b0a1a1f 100644 --- a/src/rt_cs_dev.erl +++ b/src/rt_cs_dev.erl @@ -20,7 +20,41 @@ %% @private -module(rt_cs_dev). --compile(export_all). +-behavious(test_harness). + +-export([start/1, + stop/1, + deploy_clusters/1, + clean_data_dir/2, + create_dirs/1, + spawn_cmd/1, + spawn_cmd/2, + cmd/1, + cmd/2, + setup_harness/2, + get_version/0, + get_backends/0, + get_deps/0, + set_backend/1, + whats_up/0, + get_ip/1, + node_id/1, + node_version/1, + admin/2, + riak/2, + attach/2, + attach_direct/2, + console/2, + update_app_config/2, + teardown/0, + set_conf/2, + set_advanced_conf/2, + upgrade/2, + deploy_nodes/1, + versions/0, + get_node_logs/0, + get_node_logs/1]). + -include_lib("eunit/include/eunit.hrl"). -define(DEVS(N), lists:concat(["dev", N, "@127.0.0.1"])). @@ -31,6 +65,12 @@ get_deps() -> lists:flatten(io_lib:format("~s/dev/dev1/lib", [relpath(current)])). +deploy_clusters(ClusterConfig) -> + rt_harness_util:deploy_clusters(ClusterConfig). + +get_ip(Node) -> + rt_harness_util:get_ip(Node). + setup_harness(_Test, _Args) -> confirm_build_type(rt_config:get(build_type, oss)), Path = relpath(root), @@ -76,10 +116,6 @@ relpath(Vsn) -> Path = ?BUILD_PATHS, path(Vsn, Path). -srcpath(Vsn) -> - Path = ?SRC_PATHS, - path(Vsn, Path). - path(Vsn, Paths=[{_,_}|_]) -> orddict:fetch(Vsn, orddict:from_list(Paths)); path(current, Path) -> @@ -193,19 +229,6 @@ rm_dir(Dir) -> ?assertCmd("rm -rf " ++ Dir), ?assertEqual(false, filelib:is_dir(Dir)). -add_default_node_config(Nodes) -> - case rt_config:get(rt_default_config, undefined) of - undefined -> ok; - Defaults when is_list(Defaults) -> - rt:pmap(fun(Node) -> - update_app_config(Node, Defaults) - end, Nodes), - ok; - BadValue -> - lager:error("Invalid value for rt_default_config : ~p", [BadValue]), - throw({invalid_config, {rt_default_config, BadValue}}) - end. - deploy_nodes(NodeConfig) -> Path = relpath(root), lager:info("Riak path: ~p", [Path]), @@ -217,14 +240,14 @@ deploy_nodes(NodeConfig) -> VersionMap = lists:zip(NodesN, Versions), %% Check that you have the right versions available - [ check_node(Version) || Version <- VersionMap ], + [ rt_harness_util:check_node(Version) || Version <- VersionMap ], rt_config:set(rt_nodes, NodeMap), rt_config:set(rt_versions, VersionMap), create_dirs(Nodes), %% Set initial config - add_default_node_config(Nodes), + rt_harness_util:add_default_node_config(Nodes), rt:pmap(fun({_, default}) -> ok; ({Node, Config}) -> @@ -459,14 +482,6 @@ get_cmd_result(Port, Acc) -> timeout end. -check_node({_N, Version}) -> - case proplists:is_defined(Version, rt_config:get(build_paths)) of - true -> ok; - _ -> - lager:error("You don't have Riak ~s installed or configured", [Version]), - erlang:error("You don't have Riak " ++ atom_to_list(Version) ++ " installed or configured") - end. - set_backend(Backend) -> lager:info("rtdev:set_backend(~p)", [Backend]), update_app_config(all, [{riak_kv, [{storage_backend, Backend}]}]), @@ -509,3 +524,9 @@ get_node_logs(Base) -> {ok, Port} = file:open(Filename, [read, binary]), {lists:nthtail(RootLen, Filename), Port} end || Filename <- filelib:wildcard(Root ++ "/*/dev/dev*/log/*") ]. + +set_advanced_conf(Node, NameValuePairs) -> + rt_harness_util:set_advanced_conf(Node, NameValuePairs). + +set_conf(Node, NameValuePairs) -> + rt_harness_util:set_conf(Node, NameValuePairs). diff --git a/src/rt_harness_util.erl b/src/rt_harness_util.erl index c636c0cbf..7963939c0 100644 --- a/src/rt_harness_util.erl +++ b/src/rt_harness_util.erl @@ -31,11 +31,15 @@ cmd/1, cmd/2, console/2, + deploy_clusters/1, get_ip/1, node_id/1, node_version/1, riak/2, set_conf/2, + set_advanced_conf/2, + get_advanced_riak_conf/1, + update_app_config_file/2, spawn_cmd/1, spawn_cmd/2, whats_up/0]). @@ -58,6 +62,74 @@ attach_direct(Node, Expected) -> console(Node, Expected) -> interactive(Node, "console", Expected). +deploy_clusters(ClusterConfigs) -> + NumNodes = rt_config:get(num_nodes, 6), + RequestedNodes = lists:flatten(ClusterConfigs), + + case length(RequestedNodes) > NumNodes of + true -> + erlang:error("Requested more nodes than available"); + false -> + Nodes = deploy_nodes(RequestedNodes), + {DeployedClusters, _} = lists:foldl( + fun(Cluster, {Clusters, RemNodes}) -> + {A, B} = lists:split(length(Cluster), RemNodes), + {Clusters ++ [A], B} + end, {[], Nodes}, ClusterConfigs), + DeployedClusters + end. + +deploy_nodes(NodeConfig) -> + Path = relpath(root), + lager:info("Riak path: ~p", [Path]), + NumNodes = length(NodeConfig), + NodesN = lists:seq(1, NumNodes), + Nodes = [?DEV(N) || N <- NodesN], + NodeMap = orddict:from_list(lists:zip(Nodes, NodesN)), + {Versions, Configs} = lists:unzip(NodeConfig), + VersionMap = lists:zip(NodesN, Versions), + + %% Check that you have the right versions available + [ check_node(Version) || Version <- VersionMap ], + rt_config:set(rt_nodes, NodeMap), + rt_config:set(rt_versions, VersionMap), + + create_dirs(Nodes), + + %% Set initial config + add_default_node_config(Nodes), + rt:pmap(fun({_, default}) -> + ok; + ({Node, {cuttlefish, Config}}) -> + set_conf(Node, Config); + ({Node, Config}) -> + rt_config:update_app_config(Node, Config) + end, + lists:zip(Nodes, Configs)), + + %% create snmp dirs, for EE + create_dirs(Nodes), + + %% Start nodes + %%[run_riak(N, relpath(node_version(N)), "start") || N <- Nodes], + rt:pmap(fun(N) -> run_riak(N, relpath(node_version(N)), "start") end, NodesN), + + %% Ensure nodes started + [ok = rt:wait_until_pingable(N) || N <- Nodes], + + %% %% Enable debug logging + %% [rpc:call(N, lager, set_loglevel, [lager_console_backend, debug]) || N <- Nodes], + + %% We have to make sure that riak_core_ring_manager is running before we can go on. + [ok = rt:wait_until_registered(N, riak_core_ring_manager) || N <- Nodes], + + %% Ensure nodes are singleton clusters + [ok = rt_ring:check_singleton_node(?DEV(N)) || {N, Version} <- VersionMap, + Version /= "0.14.2"], + + lager:info("Deployed nodes: ~p", [Nodes]), + Nodes. + interactive(Node, Command, Exp) -> N = node_id(Node), Path = relpath(node_version(N)), @@ -133,6 +205,13 @@ interactive_loop(Port, Expected) -> ?assertEqual([], Expected) end. +node_to_host(Node) -> + case string:tokens(atom_to_list(Node), "@") of + ["riak", Host] -> Host; + _ -> + throw(io_lib:format("rtssh:node_to_host couldn't figure out the host of ~p", [Node])) + end. + spawn_cmd(Cmd) -> spawn_cmd(Cmd, []). spawn_cmd(Cmd, Opts) -> @@ -170,9 +249,21 @@ get_cmd_result(Port, Acc) -> timeout end. -get_ip(_Node) -> - %% localhost 4 lyfe - "127.0.0.1". + +get_host(Node) when is_atom(Node) -> + try orddict:fetch(Node, rt_config:get(rt_hosts)) of + Host -> Host + catch _:_ -> + %% Let's try figuring this out from the node name + node_to_host(Node) + end; +get_host(Host) -> Host. + +get_ip(Node) when is_atom(Node) -> + get_ip(get_host(Node)); +get_ip(Host) -> + {ok, IP} = inet:getaddr(Host, inet), + string:join([integer_to_list(X) || X <- tuple_to_list(IP)], "."). node_id(Node) -> NodeMap = rt_config:get(rt_nodes), @@ -281,3 +372,72 @@ all_the_files(DevPath, File) -> devpaths() -> lists:usort([ DevPath || {_Name, DevPath} <- proplists:delete(root, rt_config:get(rtdev_path))]). + +create_dirs(Nodes) -> + Snmp = [node_path(Node) ++ "/data/snmp/agent/db" || Node <- Nodes], + [?assertCmd("mkdir -p " ++ Dir) || Dir <- Snmp]. + +check_node({_N, Version}) -> + case proplists:is_defined(Version, rt_config:get(rtdev_path)) of + true -> ok; + _ -> + lager:error("You don't have Riak ~s installed or configured", [Version]), + erlang:error("You don't have Riak " ++ atom_to_list(Version) ++ " installed or configured") + end. + +add_default_node_config(Nodes) -> + case rt_config:get(rt_default_config, undefined) of + undefined -> ok; + Defaults when is_list(Defaults) -> + rt:pmap(fun(Node) -> + rt_config:update_app_config(Node, Defaults) + end, Nodes), + ok; + BadValue -> + lager:error("Invalid value for rt_default_config : ~p", [BadValue]), + throw({invalid_config, {rt_default_config, BadValue}}) + end. + +node_path(Node) -> + N = node_id(Node), + Path = relpath(node_version(N)), + lists:flatten(io_lib:format("~s/dev/dev~b", [Path, N])). + +set_advanced_conf(all, NameValuePairs) -> + lager:info("rtdev:set_advanced_conf(all, ~p)", [NameValuePairs]), + [ set_advanced_conf(DevPath, NameValuePairs) || DevPath <- devpaths()], + ok; +set_advanced_conf(Node, NameValuePairs) when is_atom(Node) -> + append_to_conf_file(get_advanced_riak_conf(Node), NameValuePairs), + ok; +set_advanced_conf(DevPath, NameValuePairs) -> + [update_app_config_file(RiakConf, NameValuePairs) || RiakConf <- all_the_files(DevPath, "etc/advanced.config")], + ok. + +get_advanced_riak_conf(Node) -> + N = node_id(Node), + Path = relpath(node_version(N)), + io_lib:format("~s/dev/dev~b/etc/advanced.config", [Path, N]). + +update_app_config_file(ConfigFile, Config) -> + lager:info("rtdev:update_app_config_file(~s, ~p)", [ConfigFile, Config]), + + BaseConfig = case file:consult(ConfigFile) of + {ok, [ValidConfig]} -> + ValidConfig; + {error, enoent} -> + [] + end, + MergeA = orddict:from_list(Config), + MergeB = orddict:from_list(BaseConfig), + NewConfig = + orddict:merge(fun(_, VarsA, VarsB) -> + MergeC = orddict:from_list(VarsA), + MergeD = orddict:from_list(VarsB), + orddict:merge(fun(_, ValA, _ValB) -> + ValA + end, MergeC, MergeD) + end, MergeA, MergeB), + NewConfigOut = io_lib:format("~p.", [NewConfig]), + ?assertEqual(ok, file:write_file(ConfigFile, NewConfigOut)), + ok. diff --git a/src/rtssh.erl b/src/rtssh.erl index c24b07f6e..1fd291943 100644 --- a/src/rtssh.erl +++ b/src/rtssh.erl @@ -716,14 +716,17 @@ whats_up() -> Up = [rpc:call(Node, os, cmd, ["pwd"]) || Node <- nodes()], [io:format(" ~s~n",[string:substr(Dir, 1, length(Dir)-1)]) || Dir <- Up]. -node_version(_Node) -> ok. +node_version(Node) -> + rt_harness_util:node_version(Node). -attach(_Node, _Expected) -> ok. +attach(Node, Expected) -> + rt_harness_util:attach(Node, Expected). -attach_direct(_Node, _Expected) -> ok. - -console(_Node, _Expected) -> ok. +attach_direct(Node, Expected) -> + rt_harness_util:attach_direct(Node, Expected). +console(Node, Expected) -> + rt_harness_util:console(Node, Expected). %%%=================================================================== %%% Local command spawning From f0c39741b0f4262280a5ec773e9c31b7f392fbbb Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Mon, 4 Aug 2014 08:13:23 -0400 Subject: [PATCH 030/157] Fix typo in rt_cs_dev: 'behavious' -> 'behaviour'. --- src/rt_cs_dev.erl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/rt_cs_dev.erl b/src/rt_cs_dev.erl index 51b0a1a1f..aa811f123 100644 --- a/src/rt_cs_dev.erl +++ b/src/rt_cs_dev.erl @@ -20,7 +20,7 @@ %% @private -module(rt_cs_dev). --behavious(test_harness). +-behaviour(test_harness). -export([start/1, stop/1, From 4e3ddcf6edcf4a5d04aa3c5ca6decf983d12a10a Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Mon, 4 Aug 2014 13:57:42 -0400 Subject: [PATCH 031/157] Fix whitespace in rt_cs_dev. --- src/rt_cs_dev.erl | 40 ++++++++++++++++++++-------------------- 1 file changed, 20 insertions(+), 20 deletions(-) diff --git a/src/rt_cs_dev.erl b/src/rt_cs_dev.erl index aa811f123..a41741977 100644 --- a/src/rt_cs_dev.erl +++ b/src/rt_cs_dev.erl @@ -28,27 +28,27 @@ clean_data_dir/2, create_dirs/1, spawn_cmd/1, - spawn_cmd/2, - cmd/1, - cmd/2, - setup_harness/2, - get_version/0, - get_backends/0, + spawn_cmd/2, + cmd/1, + cmd/2, + setup_harness/2, + get_version/0, + get_backends/0, get_deps/0, - set_backend/1, - whats_up/0, - get_ip/1, - node_id/1, - node_version/1, - admin/2, - riak/2, - attach/2, - attach_direct/2, - console/2, - update_app_config/2, - teardown/0, - set_conf/2, - set_advanced_conf/2, + set_backend/1, + whats_up/0, + get_ip/1, + node_id/1, + node_version/1, + admin/2, + riak/2, + attach/2, + attach_direct/2, + console/2, + update_app_config/2, + teardown/0, + set_conf/2, + set_advanced_conf/2, upgrade/2, deploy_nodes/1, versions/0, From 62947c94260e568cb65d7cdff537cd027d4a8b5e Mon Sep 17 00:00:00 2001 From: Kelly McLaughlin Date: Fri, 1 Aug 2014 16:57:51 -0600 Subject: [PATCH 032/157] Refactor and reorganize riak_test_escript module Refactor the riak_test_escript module to improve code clarity and organization. This includes renaming functions to be more reflective of their purpose and breaking large functions into smaller functions. Also add the ability to specify a comma-separated list of tests with the -t option. --- src/riak_test_escript.erl | 381 ++++++++++++++++++++++---------------- src/riak_test_runner.erl | 5 +- 2 files changed, 225 insertions(+), 161 deletions(-) diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index ddc220c1b..97bfca9f1 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -24,13 +24,43 @@ -export([main/1]). -export([add_deps/1]). -add_deps(Path) -> - {ok, Deps} = file:list_dir(Path), - [code:add_path(lists:append([Path, "/", Dep, "/ebin"])) || Dep <- Deps], +main(Args) -> + {ParsedArgs, HarnessArgs, Tests} = prepare(Args), + OutDir = proplists:get_value(outdir, ParsedArgs), + Results = execute(Tests, OutDir, report(ParsedArgs), HarnessArgs), + finalize(Results, ParsedArgs). + +prepare(Args) -> + {ParsedArgs, _, Tests} = ParseResults = parse_args(Args), + io:format("Tests to run: ~p~n", [Tests]), + ok = erlang_setup(ParsedArgs), + ok = test_setup(ParsedArgs), + ParseResults. + +execute(Tests, Outdir, Report, HarnessArgs) -> + TestCount = length(Tests), + TestResults = [run_test(Test, + Outdir, + TestMetaData, + Report, + HarnessArgs, + TestCount) || + {Test, TestMetaData} <- Tests], + lists:filter(fun results_filter/1, TestResults). + +finalize(TestResults, Args) -> + [rt_cover:maybe_import_coverage(proplists:get_value(coverdata, R)) || + R <- TestResults], + CoverDir = rt_config:get(cover_output, "coverage"), + Coverage = rt_cover:maybe_write_coverage(all, CoverDir), + + Verbose = proplists:is_defined(verbose, Args), + Teardown = not proplists:get_value(keep, Args, false), + maybe_teardown(Teardown, TestResults, Coverage, Verbose), ok. -cli_options() -> %% Option Name, Short Code, Long Code, Argument Spec, Help Message +cli_options() -> [ {help, $h, "help", undefined, "Print this usage page"}, {config, $c, "conf", string, "specifies the project configuration"}, @@ -48,140 +78,142 @@ cli_options() -> ]. print_help() -> - getopt:usage(cli_options(), - escript:script_name()), + getopt:usage(cli_options(), escript:script_name()), halt(0). -run_help([]) -> true; -run_help(ParsedArgs) -> - lists:member(help, ParsedArgs). +add_deps(Path) -> + {ok, Deps} = file:list_dir(Path), + [code:add_path(lists:append([Path, "/", Dep, "/ebin"])) || Dep <- Deps], + ok. -main(Args) -> - case filelib:is_dir("./ebin") of - true -> - code:add_patha("./ebin"); - _ -> - meh - end, +test_setup(ParsedArgs) -> + %% File output + OutDir = proplists:get_value(outdir, ParsedArgs), + ensure_dir(OutDir), - register(riak_test, self()), - {ParsedArgs, HarnessArgs} = case getopt:parse(cli_options(), Args) of - {ok, {P, H}} -> {P, H}; - _ -> print_help() - end, + lager_setup(OutDir), - case run_help(ParsedArgs) of - true -> print_help(); - _ -> ok + %% Ensure existence of scratch_dir + case ensure_dir(rt_config:get(rt_scratch_dir) ++ "/test.file") of + ok -> + great; + {error, ErrorReason} -> + lager:error("Could not create scratch dir, ~p", + [ErrorReason]) end, + ok. - %% ibrowse - application:load(ibrowse), - application:start(ibrowse), - %% Start Lager - application:load(lager), - - Config = proplists:get_value(config, ParsedArgs), - ConfigFile = proplists:get_value(file, ParsedArgs), +report(ParsedArgs) -> + case proplists:get_value(report, ParsedArgs, undefined) of + undefined -> + undefined; + "config" -> + rt_config:get(platform, undefined); + R -> + R + end. +parse_args(Args) -> + help_or_parse_args(getopt:parse(cli_options(), Args)). + +help_or_parse_args({ok, {[], _}}) -> + print_help(); +help_or_parse_args({ok, {ParsedArgs, HarnessArgs}}) -> + help_or_parse_tests(ParsedArgs, HarnessArgs, lists:member(help, ParsedArgs)); +help_or_parse_args(_) -> + print_help(). + +help_or_parse_tests(_, _, true) -> + print_help(); +help_or_parse_tests(ParsedArgs, HarnessArgs, false) -> + %% Have to load the `riak_test' config prior to assembling the + %% test metadata + load_initial_config(ParsedArgs), + + TestData = compose_test_data(ParsedArgs), + Tests = which_tests_to_run(report(ParsedArgs), TestData), + Offset = rt_config:get(offset, undefined), + Workers = rt_config:get(workers, undefined), + shuffle_tests(ParsedArgs, HarnessArgs, Tests, Offset, Workers). + +load_initial_config(ParsedArgs) -> %% Loads application defaults application:load(riak_test), %% Loads from ~/.riak_test.config - rt_config:load(Config, ConfigFile), + rt_config:load(proplists:get_value(config, ParsedArgs), + proplists:get_value(file, ParsedArgs)). + +shuffle_tests(_, _, [], _, _) -> + lager:warning("No tests are scheduled to run"), + init:stop(1); +shuffle_tests(ParsedArgs, HarnessArgs, Tests, undefined, _) -> + {ParsedArgs, HarnessArgs, Tests}; +shuffle_tests(ParsedArgs, HarnessArgs, Tests, _, undefined) -> + {ParsedArgs, HarnessArgs, Tests}; +shuffle_tests(ParsedArgs, HarnessArgs, Tests, Offset, Workers) -> + TestCount = length(Tests), + %% Avoid dividing by zero, computers hate that + Denominator = case Workers rem (TestCount+1) of + 0 -> 1; + D -> D + end, + ActualOffset = ((TestCount div Denominator) * Offset) rem (TestCount+1), + {TestA, TestB} = lists:split(ActualOffset, Tests), + lager:info("Offsetting ~b tests by ~b (~b workers, ~b offset)", + [TestCount, ActualOffset, Workers, Offset]), + {ParsedArgs, HarnessArgs, TestB ++ TestA}. + +erlang_setup(_ParsedArgs) -> + register(riak_test, self()), + maybe_add_code_path("./ebin"), + + %% ibrowse + load_and_start(ibrowse), %% Sets up extra paths earlier so that tests can be loadable %% without needing the -d flag. code:add_paths(rt_config:get(test_paths, [])), - %% Ensure existance of scratch_dir - case file:make_dir(rt_config:get(rt_scratch_dir)) of - ok -> great; - {error, eexist} -> great; - {ErrorType, ErrorReason} -> lager:error("Could not create scratch dir, {~p, ~p}", [ErrorType, ErrorReason]) - end, - - %% Fileoutput - Outdir = proplists:get_value(outdir, ParsedArgs), - ConsoleLagerLevel = case Outdir of - undefined -> rt_config:get(lager_level, info); - _ -> - filelib:ensure_dir(Outdir), - notice - end, - - application:set_env(lager, handlers, [{lager_console_backend, ConsoleLagerLevel}, - {lager_file_backend, [{file, "log/test.log"}, - {level, ConsoleLagerLevel}]}]), - lager:start(), - - %% Report - Report = case proplists:get_value(report, ParsedArgs, undefined) of - undefined -> undefined; - "config" -> rt_config:get(platform, undefined); - R -> R - end, - - Verbose = proplists:is_defined(verbose, ParsedArgs), - - Suites = proplists:get_all_values(suites, ParsedArgs), - case Suites of - [] -> ok; - _ -> io:format("Suites are not currently supported.") - end, - - CommandLineTests = parse_command_line_tests(ParsedArgs), - Tests0 = which_tests_to_run(Report, CommandLineTests), - - case Tests0 of - [] -> - lager:warning("No tests are scheduled to run"), - init:stop(1); - _ -> keep_on_keepin_on - end, - - Tests = case {rt_config:get(offset, undefined), rt_config:get(workers, undefined)} of - {undefined, undefined} -> - Tests0; - {undefined, _} -> - Tests0; - {_, undefined} -> - Tests0; - {Offset, Workers} -> - TestCount = length(Tests0), - %% Avoid dividing by zero, computers hate that - Denominator = case Workers rem (TestCount+1) of - 0 -> 1; - D -> D - end, - ActualOffset = ((TestCount div Denominator) * Offset) rem (TestCount+1), - {TestA, TestB} = lists:split(ActualOffset, Tests0), - lager:info("Offsetting ~b tests by ~b (~b workers, ~b" - " offset)", [TestCount, ActualOffset, Workers, - Offset]), - TestB ++ TestA - end, - - io:format("Tests to run: ~p~n", [Tests]), %% Two hard-coded deps... add_deps(rt:get_deps()), add_deps("deps"), [add_deps(Dep) || Dep <- rt_config:get(rt_deps, [])], - ENode = rt_config:get(rt_nodename, 'riak_test@127.0.0.1'), - Cookie = rt_config:get(rt_cookie, riak), - CoverDir = rt_config:get(cover_output, "coverage"), [] = os:cmd("epmd -daemon"), - net_kernel:start([ENode]), - erlang:set_cookie(node(), Cookie), + net_kernel:start([rt_config:get(rt_nodename, 'riak_test@127.0.0.1')]), + erlang:set_cookie(node(), rt_config:get(rt_cookie, riak)), + ok. - TestResults = lists:filter(fun results_filter/1, [ run_test(Test, Outdir, TestMetaData, Report, HarnessArgs, length(Tests)) || {Test, TestMetaData} <- Tests]), - [rt_cover:maybe_import_coverage(proplists:get_value(coverdata, R)) || R <- TestResults], - Coverage = rt_cover:maybe_write_coverage(all, CoverDir), +maybe_add_code_path(Path) -> + maybe_add_code_path(Path, filelib:is_dir(Path)). - Teardown = not proplists:get_value(keep, ParsedArgs, false), - maybe_teardown(Teardown, TestResults, Coverage, Verbose), - ok. +maybe_add_code_path(Path, true) -> + code:add_patha(Path); +maybe_add_code_path(_, false) -> + meh. + +load_and_start(Application) -> + application:load(Application), + application:start(Application). + +ensure_dir(undefined) -> + ok; +ensure_dir(Dir) -> + filelib:ensure_dir(Dir). + +lager_setup(undefined) -> + set_lager_env(rt_config:get(lager_level, info)), + load_and_start(lager); +lager_setup(_) -> + set_lager_env(notice), + load_and_start(lager). + +set_lager_env(LagerLevel) -> + HandlerConfig = [{lager_console_backend, LagerLevel}, + {lager_file_backend, [{file, "log/test.log"}, + {level, LagerLevel}]}], + application:set_env(lager, handlers, HandlerConfig). maybe_teardown(false, TestResults, Coverage, Verbose) -> print_summary(TestResults, Coverage, Verbose), @@ -198,39 +230,55 @@ maybe_teardown(true, TestResults, Coverage, Verbose) -> end, ok. -parse_command_line_tests(ParsedArgs) -> - Backends = case proplists:get_all_values(backend, ParsedArgs) of - [] -> [undefined]; - Other -> Other - end, - Upgrades = case proplists:get_all_values(upgrade_version, ParsedArgs) of - [] -> [undefined]; - UpgradeList -> UpgradeList - end, +compose_test_data(ParsedArgs) -> + RawTestList = proplists:get_all_values(tests, ParsedArgs), + TestList = lists:foldl(fun(X, Acc) -> string:tokens(X, ", ") ++ Acc end, [], RawTestList), %% Parse Command Line Tests {CodePaths, SpecificTests} = lists:foldl(fun extract_test_names/2, {[], []}, - proplists:get_all_values(tests, ParsedArgs)), + TestList), + [code:add_patha(CodePath) || CodePath <- CodePaths, CodePath /= "."], + Dirs = proplists:get_all_values(dir, ParsedArgs), SkipTests = string:tokens(proplists:get_value(skip, ParsedArgs, []), [$,]), DirTests = lists:append([load_tests_in_dir(Dir, SkipTests) || Dir <- Dirs]), - lists:foldl(fun(Test, Tests) -> - [{ - list_to_atom(Test), - [ - {id, -1}, - {platform, <<"local">>}, - {version, rt:get_version()}, - {project, list_to_binary(rt_config:get(rt_project, "undefined"))} - ] ++ - [ {backend, Backend} || Backend =/= undefined ] ++ - [ {upgrade_version, Upgrade} || Upgrade =/= undefined ]} - || Backend <- Backends, - Upgrade <- Upgrades ] ++ Tests - end, [], lists:usort(DirTests ++ SpecificTests)). + Project = list_to_binary(rt_config:get(rt_project, "undefined")), + + Backends = case proplists:get_all_values(backend, ParsedArgs) of + [] -> [undefined]; + Other -> Other + end, + Upgrades = case proplists:get_all_values(upgrade_version, ParsedArgs) of + [] -> [undefined]; + UpgradeList -> UpgradeList + end, + TestFoldFun = test_data_fun(rt:get_version(), Project, Backends, Upgrades), + lists:foldl(TestFoldFun, [], lists:usort(DirTests ++ SpecificTests)). + +test_data_fun(Version, Project, Backends, Upgrades) -> + fun(Test, Tests) -> + [{list_to_atom(Test), + compose_test_datum(Version, Project, Backend, Upgrade)} + || Backend <- Backends, Upgrade <- Upgrades ] ++ Tests + end. + +compose_test_datum(Version, Project, undefined, undefined) -> + [{id, -1}, + {platform, <<"local">>}, + {version, Version}, + {project, Project}]; +compose_test_datum(Version, Project, undefined, Upgrade) -> + compose_test_datum(Version, Project, undefined, undefined) ++ + [{upgrade_version, Upgrade}]; +compose_test_datum(Version, Project, Backend, undefined) -> + compose_test_datum(Version, Project, undefined, undefined) ++ + [{backend, Backend}]; +compose_test_datum(Version, Project, Backend, Upgrade) -> + compose_test_datum(Version, Project, undefined, undefined) ++ + [{backend, Backend}, {upgrade_version, Upgrade}]. extract_test_names(Test, {CodePaths, TestNames}) -> {[filename:dirname(Test) | CodePaths], @@ -242,7 +290,8 @@ which_tests_to_run(undefined, CommandLineTests) -> lager:info("These modules are not runnable tests: ~p", [[NTMod || {NTMod, _} <- NonTests]]), Tests; -which_tests_to_run(Platform, []) -> giddyup:get_suite(Platform); +which_tests_to_run(Platform, []) -> + giddyup:get_suite(Platform); which_tests_to_run(Platform, CommandLineTests) -> Suite = filter_zip_suite(Platform, CommandLineTests), {Tests, NonTests} = @@ -295,25 +344,36 @@ run_test(Test, Outdir, TestMetaData, Report, HarnessArgs, NumTests) -> 1 -> keep_them_up; _ -> rt_cluster:teardown() end, - CoverageFile = rt_cover:maybe_export_coverage(Test, CoverDir, erlang:phash2(TestMetaData)), - case Report of - undefined -> ok; - _ -> - {value, {log, L}, TestResult} = lists:keytake(log, 1, SingleTestResult), - case giddyup:post_result(TestResult) of - error -> woops; - {ok, Base} -> - %% Now push up the artifacts, starting with the test log - giddyup:post_artifact(Base, {"riak_test.log", L}), - [ giddyup:post_artifact(Base, File) || File <- rt:get_node_logs() ], - [giddyup:post_artifact(Base, {filename:basename(CoverageFile) ++ ".gz", - zlib:gzip(element(2,file:read_file(CoverageFile)))}) || CoverageFile /= cover_disabled ], - ResultPlusGiddyUp = TestResult ++ [{giddyup_url, list_to_binary(Base)}], - [ rt:post_result(ResultPlusGiddyUp, WebHook) || WebHook <- get_webhooks() ] - end - end, + CoverFile = rt_cover:maybe_export_coverage(Test, CoverDir, erlang:phash2(TestMetaData)), + publish_report(SingleTestResult, CoverFile, Report), rt_cover:stop(), - [{coverdata, CoverageFile} | SingleTestResult]. + [{coverdata, CoverFile} | SingleTestResult]. + +publish_report(_SingleTestResult, _CoverFile, undefined) -> + ok; +publish_report(SingleTestResult, CoverFile, _Report) -> + {value, {log, Log}, TestResult} = lists:keytake(log, 1, SingleTestResult), + publish_artifacts(TestResult, + Log, + CoverFile, + giddyup:post_result(TestResult)). + +publish_artifacts(_TestResult, _Log, _CoverFile, error) -> + whoomp; %% there it is +publish_artifacts(TestResult, Log, CoverFile, {ok, Base}) -> + %% Now push up the artifacts, starting with the test log + giddyup:post_artifact(Base, {"riak_test.log", Log}), + [giddyup:post_artifact(Base, File) || File <- rt:get_node_logs()], + post_cover_artifact(Base, CoverFile), + ResultPlusGiddyUp = TestResult ++ [{giddyup_url, list_to_binary(Base)}], + [rt:post_result(ResultPlusGiddyUp, WebHook) || WebHook <- get_webhooks()]. + +post_cover_artifact(_Base, cover_disabled) -> + ok; +post_cover_artifact(Base, CoverFile) -> + CoverArchiveName = filename:basename(CoverFile) ++ ".gz", + CoverArchive = zlib:gzip(element(2, file:read_file(CoverFile))), + giddyup:post_artifact(Base, {CoverArchiveName, CoverArchive}). get_webhooks() -> Hooks = lists:foldl(fun(E, Acc) -> [parse_webhook(E) | Acc] end, @@ -402,7 +462,8 @@ load_tests_in_dir(Dir, SkipTests) -> lists:foldl(load_tests_folder(SkipTests), [], filelib:wildcard("*.beam", Dir))); - _ -> io:format("~s is not a dir!~n", [Dir]) + _ -> + io:format("~s is not a dir!~n", [Dir]) end. load_tests_folder(SkipTests) -> @@ -422,8 +483,10 @@ so_kill_riak_maybe() -> io:format("Would you like to leave Riak running in order to debug?~n"), Input = io:get_chars("[Y/n] ", 1), case Input of - "n" -> rt_cluster:teardown(); - "N" -> rt_cluster:teardown(); + "n" -> + rt_cluster:teardown(); + "N" -> + rt_cluster:teardown(); _ -> io:format("Leaving Riak Up... "), rt:whats_up() diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index d181177ae..cbc00c37b 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -42,8 +42,9 @@ metadata(Pid) -> end. -spec(run(integer(), atom(), [{atom(), term()}], list()) -> [tuple()]). -%% @doc Runs a module's run/0 function after setting up a log capturing backend for lager. -%% It then cleans up that backend and returns the logs as part of the return proplist. +%% @doc Runs a module's run/0 function after setting up a log +%% capturing backend for lager. It then cleans up that backend +%% and returns the logs as part of the return proplist. run(TestModule, Outdir, TestMetaData, HarnessArgs) -> start_lager_backend(TestModule, Outdir), rt:setup_harness(TestModule, HarnessArgs), From ad6cf5881f944c630bc9ab9f41ca5adb1517aaf0 Mon Sep 17 00:00:00 2001 From: Kelly McLaughlin Date: Tue, 5 Aug 2014 15:30:43 -0600 Subject: [PATCH 033/157] Fix issue with lager startup --- src/riak_test_escript.erl | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index 97bfca9f1..9e92ff81c 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -204,12 +204,13 @@ ensure_dir(Dir) -> lager_setup(undefined) -> set_lager_env(rt_config:get(lager_level, info)), - load_and_start(lager); + lager:start(); lager_setup(_) -> set_lager_env(notice), - load_and_start(lager). + lager:start(). set_lager_env(LagerLevel) -> + application:load(lager), HandlerConfig = [{lager_console_backend, LagerLevel}, {lager_file_backend, [{file, "log/test.log"}, {level, LagerLevel}]}], From e785626ea26d84b920cdfbc25db15b78658cbc8c Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Wed, 6 Aug 2014 07:36:21 -0400 Subject: [PATCH 034/157] Fix whitespace problems; add some edoc to harness modules. --- src/rt_harness.erl | 4 ++++ src/rt_harness_util.erl | 3 ++- src/rtdev.erl | 40 ++++++++++++++++++++-------------------- src/rtperf.erl | 40 ++++++++++++++++++++-------------------- src/rtssh.erl | 40 ++++++++++++++++++++-------------------- src/test_harness.erl | 1 + 6 files changed, 67 insertions(+), 61 deletions(-) diff --git a/src/rt_harness.erl b/src/rt_harness.erl index cd573e9c4..2d1414760 100644 --- a/src/rt_harness.erl +++ b/src/rt_harness.erl @@ -17,6 +17,10 @@ %% under the License. %% %% ------------------------------------------------------------------- +%% @doc rt_harness provides a level of indirection between the modules +%% calling into the harness and the configured harness, resolving the call +%% to the configured harness. Calls such as 'rt_harness:start(Node)' will +%% be resolved to the configured harness. -module(rt_harness). -define(HARNESS_MODULE, (rt_config:get(rt_harness))). diff --git a/src/rt_harness_util.erl b/src/rt_harness_util.erl index 7963939c0..82c6a4abb 100644 --- a/src/rt_harness_util.erl +++ b/src/rt_harness_util.erl @@ -17,7 +17,8 @@ %% under the License. %% %% ------------------------------------------------------------------- - +%% @doc The purpose of rt_harness_util is to provide common functions +%% to harness modules implementing the test_harness behaviour. -module(rt_harness_util). -include_lib("eunit/include/eunit.hrl"). diff --git a/src/rtdev.erl b/src/rtdev.erl index 76b56793e..49771c594 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -25,26 +25,26 @@ deploy_clusters/1, clean_data_dir/2, spawn_cmd/1, - spawn_cmd/2, - cmd/1, - cmd/2, - setup_harness/2, - get_version/0, - get_backends/0, - set_backend/1, - whats_up/0, - get_ip/1, - node_id/1, - node_version/1, - admin/2, - riak/2, - attach/2, - attach_direct/2, - console/2, - update_app_config/2, - teardown/0, - set_conf/2, - set_advanced_conf/2]). + spawn_cmd/2, + cmd/1, + cmd/2, + setup_harness/2, + get_version/0, + get_backends/0, + set_backend/1, + whats_up/0, + get_ip/1, + node_id/1, + node_version/1, + admin/2, + riak/2, + attach/2, + attach_direct/2, + console/2, + update_app_config/2, + teardown/0, + set_conf/2, + set_advanced_conf/2]). -compile(export_all). -include_lib("eunit/include/eunit.hrl"). diff --git a/src/rtperf.erl b/src/rtperf.erl index ee109fffe..f0a68c65c 100644 --- a/src/rtperf.erl +++ b/src/rtperf.erl @@ -6,26 +6,26 @@ deploy_clusters/1, clean_data_dir/2, spawn_cmd/1, - spawn_cmd/2, - cmd/1, - cmd/2, - setup_harness/2, - get_version/0, - get_backends/0, - set_backend/1, - whats_up/0, - get_ip/1, - node_id/1, - node_version/1, - admin/2, - riak/2, - attach/2, - attach_direct/2, - console/2, - update_app_config/2, - teardown/0, - set_conf/2, - set_advanced_conf/2]). + spawn_cmd/2, + cmd/1, + cmd/2, + setup_harness/2, + get_version/0, + get_backends/0, + set_backend/1, + whats_up/0, + get_ip/1, + node_id/1, + node_version/1, + admin/2, + riak/2, + attach/2, + attach_direct/2, + console/2, + update_app_config/2, + teardown/0, + set_conf/2, + set_advanced_conf/2]). -compile(export_all). diff --git a/src/rtssh.erl b/src/rtssh.erl index 1fd291943..3936c67db 100644 --- a/src/rtssh.erl +++ b/src/rtssh.erl @@ -6,27 +6,27 @@ deploy_clusters/1, clean_data_dir/2, spawn_cmd/1, - spawn_cmd/2, - cmd/1, - cmd/2, - setup_harness/2, + spawn_cmd/2, + cmd/1, + cmd/2, + setup_harness/2, get_deps/0, - get_version/0, - get_backends/0, - set_backend/1, - whats_up/0, - get_ip/1, - node_id/1, - node_version/1, - admin/2, - riak/2, - attach/2, - attach_direct/2, - console/2, - update_app_config/2, - teardown/0, - set_conf/2, - set_advanced_conf/2]). + get_version/0, + get_backends/0, + set_backend/1, + whats_up/0, + get_ip/1, + node_id/1, + node_version/1, + admin/2, + riak/2, + attach/2, + attach_direct/2, + console/2, + update_app_config/2, + teardown/0, + set_conf/2, + set_advanced_conf/2]). -compile(export_all). -include_lib("eunit/include/eunit.hrl"). diff --git a/src/test_harness.erl b/src/test_harness.erl index 53474841c..11006372f 100644 --- a/src/test_harness.erl +++ b/src/test_harness.erl @@ -17,6 +17,7 @@ %% under the License. %% %% ------------------------------------------------------------------- +%% @doc behaviour for all test harnesses. -module(test_harness). -callback start(Node :: node()) -> 'ok'. From 5f182bfab2f46d62d6de3133a154d2cfc93408e8 Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Wed, 6 Aug 2014 14:29:07 -0400 Subject: [PATCH 035/157] Move default cluster config to rt_properties. --- src/rt_cluster.erl | 8 +------- src/rt_properties.erl | 11 +++++++++-- tests/bucket_types.erl | 2 +- tests/http_bucket_types.erl | 2 +- tests/verify_build_cluster.erl | 2 +- 5 files changed, 13 insertions(+), 12 deletions(-) diff --git a/src/rt_cluster.erl b/src/rt_cluster.erl index a4adcbc5f..79cf0e52a 100644 --- a/src/rt_cluster.erl +++ b/src/rt_cluster.erl @@ -23,7 +23,6 @@ -export([properties/0, setup/2, - config/0, augment_config/3, deploy_nodes/1, deploy_nodes/2, @@ -50,7 +49,7 @@ %% a custom properties function. -spec properties() -> rt_properties:properties(). properties() -> - rt_properties:new([{config, config()}]). + rt_properties:new(). -spec setup(rt_properties:properties(), proplists:proplist()) -> {ok, rt_properties:properties()} | {error, term()}. @@ -215,11 +214,6 @@ teardown() -> versions() -> rt_harness:versions(). -config() -> - [{riak_core, [{handoff_concurrency, 11}]}, - {riak_search, [{enabled, true}]}, - {riak_pipe, [{worker_limit, 200}]}]. - augment_config(Section, Property, Config) -> UpdSectionConfig = update_section(Section, Property, diff --git a/src/rt_properties.erl b/src/rt_properties.erl index a484d0224..fb29afe17 100644 --- a/src/rt_properties.erl +++ b/src/rt_properties.erl @@ -37,7 +37,7 @@ wait_for_transfers=false :: boolean(), valid_backends=all :: all | [atom()], make_cluster=true :: boolean(), - config :: term() + config=default_config() :: term() }). -type properties() :: #rt_properties_v1{}. @@ -50,7 +50,8 @@ new/1, get/2, set/2, - set/3]). + set/3, + default_config/0]). %% @doc Create a new properties record with all fields initialized to %% the default values. @@ -161,6 +162,12 @@ validate_property(Property, ok) -> validate_property(_Property, {error, _}=Error) -> Error. +-spec default_config() -> [term()]. +default_config() -> + [{riak_core, [{handoff_concurrency, 11}]}, + {riak_search, [{enabled, true}]}, + {riak_pipe, [{worker_limit, 200}]}]. + -spec is_valid_record(term()) -> boolean(). is_valid_record(Record) -> is_record(Record, rt_properties_v1). diff --git a/tests/bucket_types.erl b/tests/bucket_types.erl index fea36b1ec..4f05ea19c 100644 --- a/tests/bucket_types.erl +++ b/tests/bucket_types.erl @@ -8,7 +8,7 @@ properties() -> CustomConfig = rt_cluster:augment_config(riak_core, {default_bucket_props, [{n_val, 2}]}, - rt_cluster:config()), + rt_properties:default_config()), rt_properties:new([{node_count, 4}, {config, CustomConfig}]). diff --git a/tests/http_bucket_types.erl b/tests/http_bucket_types.erl index da61c3aef..10b6e31bd 100644 --- a/tests/http_bucket_types.erl +++ b/tests/http_bucket_types.erl @@ -9,7 +9,7 @@ properties() -> CustomConfig = rt_cluster:augment_config(riak_core, {default_bucket_props, [{n_val, 2}]}, - rt_cluster:config()), + rt_properties:default_config()), rt_properties:new([{node_count, 1}, {config, CustomConfig}]). diff --git a/tests/verify_build_cluster.erl b/tests/verify_build_cluster.erl index fac763314..da72b420c 100644 --- a/tests/verify_build_cluster.erl +++ b/tests/verify_build_cluster.erl @@ -26,7 +26,7 @@ properties() -> UpdConfig = rt_cluster:augment_config(riak_core, {default_bucket_props, [{allow_mult, false}]}, - rt_cluster:config()), + rt_properties:default_config()), rt_properties:new([{config, UpdConfig}, {node_count, 4}, {rolling_upgrade, true}, From 6551087ff81cfc459966f4cf9646c2b3992cb50f Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Wed, 6 Aug 2014 14:56:06 -0400 Subject: [PATCH 036/157] Fix code reference for edoc in rt_harness. --- src/rt_harness.erl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/rt_harness.erl b/src/rt_harness.erl index 2d1414760..1270f3bfc 100644 --- a/src/rt_harness.erl +++ b/src/rt_harness.erl @@ -19,7 +19,7 @@ %% ------------------------------------------------------------------- %% @doc rt_harness provides a level of indirection between the modules %% calling into the harness and the configured harness, resolving the call -%% to the configured harness. Calls such as 'rt_harness:start(Node)' will +%% to the configured harness. Calls such as `rt_harness:start(Node)' will %% be resolved to the configured harness. -module(rt_harness). From 894b9b5fadfd20306027e26986e0e55db8d828bd Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Thu, 7 Aug 2014 23:18:44 -0400 Subject: [PATCH 037/157] Add support for test group annotations to riak_tests. - Tests can be annotated with one or more '-test_type(name)' attributes. - Tests can be declared to be in multiple groups - These types can be run in groups via the riak_test command line --groups (-g) option - Multiple groups can be specified by comma delimiting. --- src/riak_test_escript.erl | 63 +++++++++++++++++++++++++++++++-------- tests/bucket_types.erl | 2 ++ tests/ensemble_basic.erl | 2 ++ 3 files changed, 55 insertions(+), 12 deletions(-) diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index 9e92ff81c..7bc4d3a6f 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -25,17 +25,26 @@ -export([add_deps/1]). main(Args) -> - {ParsedArgs, HarnessArgs, Tests} = prepare(Args), - OutDir = proplists:get_value(outdir, ParsedArgs), - Results = execute(Tests, OutDir, report(ParsedArgs), HarnessArgs), - finalize(Results, ParsedArgs). + case prepare(Args) of + ok -> + ok; + {ParsedArgs, HarnessArgs, Tests} -> + OutDir = proplists:get_value(outdir, ParsedArgs), + Results = execute(Tests, OutDir, report(ParsedArgs), HarnessArgs), + finalize(Results, ParsedArgs) + end. prepare(Args) -> - {ParsedArgs, _, Tests} = ParseResults = parse_args(Args), - io:format("Tests to run: ~p~n", [Tests]), - ok = erlang_setup(ParsedArgs), - ok = test_setup(ParsedArgs), - ParseResults. + case parse_args(Args) of + {ParsedArgs, _, Tests} = ParseResults -> + io:format("Tests to run: ~p~n", [Tests]), + ok = erlang_setup(ParsedArgs), + ok = test_setup(ParsedArgs), + ParseResults; + ok -> + io:format("No tests to run.~n"), + ok + end. execute(Tests, Outdir, Report, HarnessArgs) -> TestCount = length(Tests), @@ -66,6 +75,7 @@ cli_options() -> {config, $c, "conf", string, "specifies the project configuration"}, {tests, $t, "tests", string, "specifies which tests to run"}, {suites, $s, "suites", string, "which suites to run"}, + {groups, $g, "groups", string, "specifiy a list of test groups to run"}, {dir, $d, "dir", string, "run all tests in the specified directory"}, {skip, $x, "skip", string, "list of tests to skip in a directory"}, {verbose, $v, "verbose", undefined, "verbose output"}, @@ -130,7 +140,9 @@ help_or_parse_tests(ParsedArgs, HarnessArgs, false) -> %% test metadata load_initial_config(ParsedArgs), - TestData = compose_test_data(ParsedArgs), + Groups = proplists:get_all_values(groups, ParsedArgs), + TestData = load_tests(Groups, ParsedArgs), + Tests = which_tests_to_run(report(ParsedArgs), TestData), Offset = rt_config:get(offset, undefined), Workers = rt_config:get(workers, undefined), @@ -231,7 +243,7 @@ maybe_teardown(true, TestResults, Coverage, Verbose) -> end, ok. -compose_test_data(ParsedArgs) -> +load_tests([], ParsedArgs) -> RawTestList = proplists:get_all_values(tests, ParsedArgs), TestList = lists:foldl(fun(X, Acc) -> string:tokens(X, ", ") ++ Acc end, [], RawTestList), %% Parse Command Line Tests @@ -246,8 +258,33 @@ compose_test_data(ParsedArgs) -> Dirs = proplists:get_all_values(dir, ParsedArgs), SkipTests = string:tokens(proplists:get_value(skip, ParsedArgs, []), [$,]), DirTests = lists:append([load_tests_in_dir(Dir, SkipTests) || Dir <- Dirs]), - Project = list_to_binary(rt_config:get(rt_project, "undefined")), + compose_test_data(DirTests, SpecificTests, ParsedArgs); +load_tests(RawGroupList, ParsedArgs) -> + Groups = lists:foldl(fun(X, Acc) -> string:tokens(X, ", ") ++ Acc end, [], RawGroupList), + Dirs = proplists:get_value(dir, ParsedArgs, ["./ebin"]), + AllDirTests = lists:append([load_tests_in_dir(Dir, []) || Dir <- Dirs]), + DirTests = get_group_tests(AllDirTests, Groups), + compose_test_data(DirTests, [], ParsedArgs). + +get_group_tests(Tests, Groups) -> + lists:filter(fun(Test) -> + Mod = list_to_atom(Test), + Attrs = Mod:module_info(attributes), + match_group_attributes(Attrs, Groups) + end, Tests). + +match_group_attributes(Attributes, Groups) -> + case proplists:get_all_values(test_type, Attributes) of + undefined -> + false; + TestTypes -> + lists:member(true, + [ hd(TestType) == list_to_atom(Group) + || Group <- Groups, TestType <- TestTypes ]) + end. +compose_test_data(DirTests, SpecificTests, ParsedArgs) -> + Project = list_to_binary(rt_config:get(rt_project, "undefined")), Backends = case proplists:get_all_values(backend, ParsedArgs) of [] -> [undefined]; Other -> Other @@ -259,6 +296,7 @@ compose_test_data(ParsedArgs) -> TestFoldFun = test_data_fun(rt:get_version(), Project, Backends, Upgrades), lists:foldl(TestFoldFun, [], lists:usort(DirTests ++ SpecificTests)). + test_data_fun(Version, Project, Backends, Upgrades) -> fun(Test, Tests) -> [{list_to_atom(Test), @@ -492,3 +530,4 @@ so_kill_riak_maybe() -> io:format("Leaving Riak Up... "), rt:whats_up() end. + diff --git a/tests/bucket_types.erl b/tests/bucket_types.erl index 4f05ea19c..54af6f416 100644 --- a/tests/bucket_types.erl +++ b/tests/bucket_types.erl @@ -5,6 +5,8 @@ -include_lib("eunit/include/eunit.hrl"). -include("rt.hrl"). +-test_type(bucket_types). + properties() -> CustomConfig = rt_cluster:augment_config(riak_core, {default_bucket_props, [{n_val, 2}]}, diff --git a/tests/ensemble_basic.erl b/tests/ensemble_basic.erl index 5e2b308d2..6c7a91be4 100644 --- a/tests/ensemble_basic.erl +++ b/tests/ensemble_basic.erl @@ -22,6 +22,8 @@ -export([confirm/0]). -include_lib("eunit/include/eunit.hrl"). +-test_type(ensemble). + confirm() -> NumNodes = 5, NVal = 5, From 4a2717e8ff76cd885b1dbe58b06f465a706eb162 Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Fri, 8 Aug 2014 11:51:32 -0400 Subject: [PATCH 038/157] Add support for multiple test_type attributes by changing format to take lists. - For example, in a riak_test: -test_type([bucket_types, http]). --- src/riak_test_escript.erl | 4 ++-- tests/bucket_types.erl | 2 +- tests/ensemble_basic.erl | 2 +- tests/http_bucket_types.erl | 2 ++ 4 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index 7bc4d3a6f..d2851edb7 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -274,12 +274,12 @@ get_group_tests(Tests, Groups) -> end, Tests). match_group_attributes(Attributes, Groups) -> - case proplists:get_all_values(test_type, Attributes) of + case proplists:get_value(test_type, Attributes) of undefined -> false; TestTypes -> lists:member(true, - [ hd(TestType) == list_to_atom(Group) + [ TestType == list_to_atom(Group) || Group <- Groups, TestType <- TestTypes ]) end. diff --git a/tests/bucket_types.erl b/tests/bucket_types.erl index 54af6f416..95290cfd5 100644 --- a/tests/bucket_types.erl +++ b/tests/bucket_types.erl @@ -5,7 +5,7 @@ -include_lib("eunit/include/eunit.hrl"). -include("rt.hrl"). --test_type(bucket_types). +-test_type([bucket_types]). properties() -> CustomConfig = rt_cluster:augment_config(riak_core, diff --git a/tests/ensemble_basic.erl b/tests/ensemble_basic.erl index 6c7a91be4..1f2d71e04 100644 --- a/tests/ensemble_basic.erl +++ b/tests/ensemble_basic.erl @@ -22,7 +22,7 @@ -export([confirm/0]). -include_lib("eunit/include/eunit.hrl"). --test_type(ensemble). +-test_type([ensemble]). confirm() -> NumNodes = 5, diff --git a/tests/http_bucket_types.erl b/tests/http_bucket_types.erl index 10b6e31bd..f4cd169b9 100644 --- a/tests/http_bucket_types.erl +++ b/tests/http_bucket_types.erl @@ -6,6 +6,8 @@ -include_lib("riakc/include/riakc.hrl"). -include("rt.hrl"). +-test_type([bucket_types, http]). + properties() -> CustomConfig = rt_cluster:augment_config(riak_core, {default_bucket_props, [{n_val, 2}]}, From 409957d0cd491473a0f36ff8a0066f2684107f64 Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Fri, 8 Aug 2014 17:02:39 -0400 Subject: [PATCH 039/157] Address review feedback. - Options --groups and --tests are mutually exclusive for now. - fix indentation for overly long lines. --- src/riak_test_escript.erl | 29 ++++++++++++++++++++++++----- 1 file changed, 24 insertions(+), 5 deletions(-) diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index d2851edb7..f48ab8acf 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -129,13 +129,18 @@ parse_args(Args) -> help_or_parse_args({ok, {[], _}}) -> print_help(); help_or_parse_args({ok, {ParsedArgs, HarnessArgs}}) -> - help_or_parse_tests(ParsedArgs, HarnessArgs, lists:member(help, ParsedArgs)); + help_or_parse_tests(ParsedArgs, + HarnessArgs, + lists:member(help, ParsedArgs), + args_invalid(ParsedArgs)); help_or_parse_args(_) -> print_help(). -help_or_parse_tests(_, _, true) -> +help_or_parse_tests(_, _, true, _) -> print_help(); -help_or_parse_tests(ParsedArgs, HarnessArgs, false) -> +help_or_parse_tests(_, _, false, true) -> + print_help(); +help_or_parse_tests(ParsedArgs, HarnessArgs, false, false) -> %% Have to load the `riak_test' config prior to assembling the %% test metadata load_initial_config(ParsedArgs), @@ -148,6 +153,16 @@ help_or_parse_tests(ParsedArgs, HarnessArgs, false) -> Workers = rt_config:get(workers, undefined), shuffle_tests(ParsedArgs, HarnessArgs, Tests, Offset, Workers). +args_invalid(ParsedArgs) -> + case { proplists:is_defined(groups, ParsedArgs), + proplists:is_defined(tests, ParsedArgs) } of + {true, true} -> + io:format("--groups and --tests are currently mutually exclusive.~n~n"), + true; + {_, _} -> + false + end. + load_initial_config(ParsedArgs) -> %% Loads application defaults application:load(riak_test), @@ -245,7 +260,9 @@ maybe_teardown(true, TestResults, Coverage, Verbose) -> load_tests([], ParsedArgs) -> RawTestList = proplists:get_all_values(tests, ParsedArgs), - TestList = lists:foldl(fun(X, Acc) -> string:tokens(X, ", ") ++ Acc end, [], RawTestList), + TestList = lists:foldl(fun(X, Acc) -> + string:tokens(X, ", ") ++ Acc + end, [], RawTestList), %% Parse Command Line Tests {CodePaths, SpecificTests} = lists:foldl(fun extract_test_names/2, @@ -260,7 +277,9 @@ load_tests([], ParsedArgs) -> DirTests = lists:append([load_tests_in_dir(Dir, SkipTests) || Dir <- Dirs]), compose_test_data(DirTests, SpecificTests, ParsedArgs); load_tests(RawGroupList, ParsedArgs) -> - Groups = lists:foldl(fun(X, Acc) -> string:tokens(X, ", ") ++ Acc end, [], RawGroupList), + Groups = lists:foldl(fun(X, Acc) -> + string:tokens(X, ", ") ++ Acc + end, [], RawGroupList), Dirs = proplists:get_value(dir, ParsedArgs, ["./ebin"]), AllDirTests = lists:append([load_tests_in_dir(Dir, []) || Dir <- Dirs]), DirTests = get_group_tests(AllDirTests, Groups), From 4e7e9a7adc5f148755c72f7e3e9bf43f23ddafd9 Mon Sep 17 00:00:00 2001 From: Kelly McLaughlin Date: Wed, 13 Aug 2014 14:13:22 -0600 Subject: [PATCH 040/157] Major overhaul of the internal workings of riak_test * Upgrade versions have moved to being a list not just a single version. `upgrade_version` is removed in favor of `upgrade_path`. `upgrade_path` is a comma-separated list representing an upgrade sequence *e.g.* `1.3.4,1.4.10,2.0.0` * Unification of node deployment code: common code and work from the individual harness modules has been brought into the framework where possible. * Further decouple and distinguish between framework setup and prep, test setup, and test execution. * Streamline the configuration for versions and root path. Move away from convention of *current*, *previous*, and *legacy*. Too restrictive and not enough benefit and *current* especially is ambiguous. Config must specify `root_path` and all versions under root path are represented as release directories. A subdirectory of `2.0.0` means that *2.0.0* can now be used as a version the same as *current*, *previous*, or *legacy* could previously, but the upside is that it requires no extra configuration. * Upgrade transitions can be specified as a list with no bound so if we want to test upgrading to each major release from `1.0.0` to `2.0.0` then that is possible. * Node deployment and teardown for a particular test execution is more isolated. The framework attempts to stop **all** nodes during its setup phase, but doing so after each test execution is unnecessary. Node deployment is now a matter of requesting the number of required nodes and the versions involved in the test. The framework determines if the number of nodes to cover the requirements of the test are available and returns success or failure on that criteria. As the last statement implies, concurrent test execution is now attempted where possible. If there are 8 nodes available for the *2.0.0* version and 4 tests queued to be run that require only 2 nodes each and do not require upgrade testing then there is no reason to block on serial execution. * Responsibility for management of test execution is more clearly delineated. Previously it was hard to account for responsibility of tasks between `riak_test_escript`, `riak_test_runner`, and `rt`. `riak_test_escript` has been heavily refactored and the work it does has been minimized to command line argument parsing and spawning the workhorse processes to execute the tests. It also made sense to make use of OTP behaviors for the implementation of some of the execution helpers. The `riak_test_executor`, an `gen_fsm`, is introduced to manage the scheduling of tests and handle reporting results. It is a named process and only one runs at a time. Individual test execution is managed by a `riak_test_runner` process. These are spawned by the `riak_test_executor` and there is one for each test that executes. `riak_test_runner` is also implemented as a `gen_fsm`. Finally there is the `node_manager` process. It is a `gen_server` that handles all node manipulation and manages access to the nodes for testing. The `riak_test_executor` requests to reserve `N` number of nodes from the node manager and if the reservation can be fulfilled the `node_manager` responds with the list of nodes for the requesting test to use. If the requested number of nodes is not available the execution of the test is deferred. The `node_manager` is aware if the current series of test executions involves upgrades and deploys nodes initially using the correct version based on that information. Thus when a test receives a list of nodes there is no need to take any action and test execution can begin immediately. * Maximize resource efficiency and length of execution duration by seeking to avoid unnecessary node starts or cycles. *e.g.* The `node_manager` is initialized with a list of nodes and the versions involved in test execution, but no nodes are deployed until the first call to `node_manager:reserve_nodes` that requires those nodes. If only one test slated for execution and it only requires 3 nodes there is no reason to start or stop more than 3 nodes. * Facilitate replication testing setup and eliminate crufty setup code duplicated in replication test with new properties `cluster_count` (defaults to 1) and `cluster_weights`, a list of weights that determine distribution of available nodes among requested clusters (defaults to `undefined`). Setting up multiple clusters for testing replication should not require any kludgy steps, it should have full support in the framework. * Update backend setup so that backend configuration can be done for selected nodes only instead of all nodes * Change setup scripts to use only version numbers as directory names. * Change setup scripts to avoid the unnecessary `dev` directory when installing devrel releases. *e.g.* Instead of `~/rt/riak/2.0.0/dev/` being the path to the `dev*` releases, the path is just `~/rt/riak/2.0.0/`. This just removes an unnecessary subdirectory and removes the need for some complications in node deployment. * Refactor properties to distinguish between node name and id: The helper functions used by the framework have different input requirements. Some that use rpc require the actual node name which may be different depending on the harness used. Others use the node identifier to form strings representing shell commands to execute. The node property has been replaced by a node_id property and a node_map structure that maps a node identifier to a full node name. * Add new rtdev-install script that is intended to replace most of the other setup scripts. * Allow distinction between console and file logging levels: Move from a single lager_level configuration option to lager_console_level and lager_file_level in order to be able to control these independently. The default level for the console output is notice to minimize the output display during a run. The default level for the file output is info in order to capture all of the logging that previously has been output to the console. * Use `riak_cli` table generation for display of test result details when using the `-v` option. * Update rebar to 2.5.1 * Update Makefile to use tools.mk --- Makefile | 5 - bin/rtdev-current.sh | 30 +- bin/rtdev-install.sh | 56 ++++ bin/rtdev-setup-releases.sh | 6 +- rebar | Bin 155065 -> 160405 bytes rebar.config | 3 +- src/node_manager.erl | 230 +++++++++++++ src/riak_test_escript.erl | 435 +++++++++++++------------ src/riak_test_executor.erl | 273 ++++++++++++++++ src/riak_test_runner.erl | 551 ++++++++++++++++++++++---------- src/rt.erl | 2 +- src/rt_backend.erl | 49 ++- src/rt_cluster.erl | 217 ++++++------- src/rt_harness.erl | 88 ++--- src/rt_harness_util.erl | 199 +++++++----- src/rt_node.erl | 103 +++--- src/rt_pb.erl | 11 +- src/rt_properties.erl | 46 ++- src/rtdev.erl | 499 +++++++++++++++++------------ src/rtssh.erl | 21 +- src/test_harness.erl | 16 +- tests/always_fail_test.erl | 7 +- tests/always_pass_test.erl | 17 +- tests/secondary_index_tests.erl | 12 +- tests/verify_listkeys.erl | 131 ++++---- tools.mk | 2 +- 26 files changed, 2026 insertions(+), 983 deletions(-) create mode 100755 bin/rtdev-install.sh create mode 100644 src/node_manager.erl create mode 100644 src/riak_test_executor.erl diff --git a/Makefile b/Makefile index f20c9306a..21545e640 100644 --- a/Makefile +++ b/Makefile @@ -1,10 +1,5 @@ .PHONY: deps -APPS = kernel stdlib sasl erts ssl tools os_mon runtime_tools crypto inets \ - xmerl webtool eunit syntax_tools compiler hipe mnesia public_key \ - observer wx gs -PLT = $(HOME)/.riak-test_dialyzer_plt - all: deps compile ./rebar skip_deps=true escriptize SMOKE_TEST=1 ./rebar skip_deps=true escriptize diff --git a/bin/rtdev-current.sh b/bin/rtdev-current.sh index 2b5a3f150..b756af622 100755 --- a/bin/rtdev-current.sh +++ b/bin/rtdev-current.sh @@ -1,6 +1,6 @@ #!/usr/bin/env bash -# bail out if things go south +# just bail out if things go south set -e : ${RT_DEST_DIR:="$HOME/rt/riak"} @@ -8,7 +8,7 @@ set -e # otherwise the last annotated tag will be used : ${RT_CURRENT_TAG:=""} -echo "Making $(pwd) the current release:" +echo "Making $(pwd) a tagged release:" cwd=$(pwd) echo -n " - Determining version: " if [ -n "$RT_CURRENT_TAG" ]; then @@ -16,23 +16,23 @@ if [ -n "$RT_CURRENT_TAG" ]; then elif [ -f $cwd/dependency_manifest.git ]; then VERSION=`cat $cwd/dependency_manifest.git | awk '/^-/ { print $NF }'` else - VERSION="$(git describe --tags)-$(git branch | awk '/\*/ {print $2}')" + VERSION=`git describe --tags | awk '{sub(/riak-/,"",$0);print}'` fi echo $VERSION cd $RT_DEST_DIR echo " - Resetting existing $RT_DEST_DIR" -export GIT_WORK_TREE="$RT_DEST_DIR" -git reset HEAD --hard > /dev/null -git clean -fd > /dev/null -echo " - Removing and recreating $RT_DEST_DIR/current" -rm -rf $RT_DEST_DIR/current -mkdir $RT_DEST_DIR/current +git reset HEAD --hard > /dev/null 2>&1 +git clean -fd > /dev/null 2>&1 +echo " - Removing and recreating $RT_DEST_DIR/$VERSION" +rm -rf $RT_DEST_DIR/$VERSION +mkdir $RT_DEST_DIR/$VERSION cd $cwd -echo " - Copying devrel to $RT_DEST_DIR/current" -cp -p -P -R dev $RT_DEST_DIR/current -echo " - Writing $RT_DEST_DIR/current/VERSION" -echo -n $VERSION > $RT_DEST_DIR/current/VERSION +echo " - Copying devrel to $RT_DEST_DIR/$VERSION" +cd dev +for i in `ls`; do cp -p -P -R $i $RT_DEST_DIR/$VERSION/; done +echo " - Writing $RT_DEST_DIR/$VERSION/VERSION" +echo -n $VERSION > $RT_DEST_DIR/$VERSION/VERSION cd $RT_DEST_DIR echo " - Reinitializing git state" -git add . -git commit -a -m "riak_test init" --amend > /dev/null +git add -f . +git commit -a -m "riak_test init" --amend > /dev/null 2>&1 diff --git a/bin/rtdev-install.sh b/bin/rtdev-install.sh new file mode 100755 index 000000000..5b34a65f7 --- /dev/null +++ b/bin/rtdev-install.sh @@ -0,0 +1,56 @@ +#!/usr/bin/env bash + +# just bail out if things go south +set -e + +: ${RT_DEST_DIR:="$HOME/rt/riak"} + +cwd=$(pwd) +echo -n " - Determining version: " +if [ -z "${VERSION+xxx}" ] || ([ -z "$VERSION" ] && [ "${VERSION+xxx}" = "xxx" ]); then + if [ -f $cwd/dependency_manifest.git ]; then + VERSION=`cat $cwd/dependency_manifest.git | awk '/^-/ { print $NF }'` + else + echo "Making $(pwd) a tagged release:" + VERSION=`git describe --tags | awk '{sub(/riak-/,"",$0);print}'` + fi +fi +echo $VERSION +if [ ! -d $RT_DEST_DIR ]; then + mkdir $RT_DEST_DIR +fi +cd $RT_DEST_DIR +if [ -d ".git" ]; then + echo " - Resetting existing $RT_DEST_DIR" + git reset HEAD --hard > /dev/null 2>&1 + git clean -fd > /dev/null 2>&1 +fi +echo " - Removing and recreating $RT_DEST_DIR/$VERSION" +rm -rf $RT_DEST_DIR/$VERSION +mkdir $RT_DEST_DIR/$VERSION +cd $cwd +echo " - Copying devrel to $RT_DEST_DIR/$VERSION" +if [ ! -d "dev" ]; then + echo "You need to run \"make devrel\" or \"make stagedevrel\" first" + exit 1 +fi +cd dev +for i in `ls`; do cp -p -P -R $i $RT_DEST_DIR/$VERSION/; done +echo " - Writing $RT_DEST_DIR/$VERSION/VERSION" +echo -n $VERSION > $RT_DEST_DIR/$VERSION/VERSION +cd $RT_DEST_DIR +if [ -d ".git" ]; then + echo " - Reinitializing git state" + git add -f . + git commit -a -m "riak_test init" --amend > /dev/null 2>&1 +else + git init + + ## Some versions of git and/or OS require these fields + git config user.name "Riak Test" + git config user.email "dev@basho.com" + + git add . + git commit -a -m "riak_test init" > /dev/null + echo " - Successfully completed initial git commit of $RT_DEST_DIR" +fi diff --git a/bin/rtdev-setup-releases.sh b/bin/rtdev-setup-releases.sh index a692e5a21..fef0629c4 100755 --- a/bin/rtdev-setup-releases.sh +++ b/bin/rtdev-setup-releases.sh @@ -16,10 +16,10 @@ echo " - Creating $RT_DEST_DIR" rm -rf $RT_DEST_DIR mkdir -p $RT_DEST_DIR -count=$(ls */dev 2> /dev/null | wc -l) +count=$(ls * 2> /dev/null | wc -l) if [ "$count" -ne "0" ] then - for rel in */dev; do + for rel in *; do vsn=$(dirname "$rel") echo " - Initializing $RT_DEST_DIR/$vsn" mkdir -p "$RT_DEST_DIR/$vsn" @@ -34,7 +34,7 @@ else fi cd $RT_DEST_DIR -git init +git init ## Some versions of git and/or OS require these fields git config user.name "Riak Test" diff --git a/rebar b/rebar index f38f1451aea54c6809ab0eebbc75fa7398433432..a8cbf1b7c5c7e292a8fd0873e985df481be50fde 100755 GIT binary patch delta 141754 zcmY(qQ*@vW&ox}PQ`?-{wr$(C?Ot^|wL7(KcWTYlwr$((cfbFC^c-bn5jY`M>-rV$sQdBaSi$+O0qN7-F&n zYQ%D2>gCE+3U*9ckQ0rNx?@@1WF&o+pn9xBCgzvuKh37Qs|&(&g@H-oA7mE{JoF{B zoogggvFO@CPaSmxO-~e$DqB{~j~G1-x77A0Jer_AKO>BU(_oP&1q`?*UT{OhC^iS_ z`uIb{+8LVlv<@l=eMqT&^84-R4yE9*l2)}(?$QYP%=sm~@NU+K2^OHSDyD^&1|b^j zQ5Yl;`49`7ukDsfW`PGVjec?MIhS17Squ6FbDWg|B6|B$7x6Q*#QeXy72{N#8VUCdTsN|kIc?M*Zo_021rP;gO}1H_6owiErY=~C zNXxn9@8x33)b0P_^KsU@a1|@MDmB}>?l^m;C!7P(&)9Is@Bq{74~<%*5|;{6)iB%C z&zRQg-!P|9tD#J`O=!`LWL+?2#X~S9<+?CsWMnUDM4yP~5xX z_33Y++<%7%2Q9Y*k*zdpk?j;@kxwGp)n*Lq-dB0ZV!!IvGyS>6`yvNOPn3PCDh@zG z@?&Tz{sz07;3-{;qi(r4G?mw1m?wrmRb!v7yogW4gg_u^dS!_p_Ciq<9T}KrU&ftFDaGn0lJ7fT-E*h4&^9q_1 zQy(aYC|jJ^|IvBp1$3ztFPtar6!+&iJqE(E!B#}M6t<;@oAU^1xH{=o?@UgdSnP`8;%*Z=|j<8 zVPQl>l7lti&PL>HRLL3r>(faGX;&~ifEh_Z%=)|9Cr3K4^g4f;TMIE{#m4mV%809r zfH8-L>4>*m79ha1?_ubTOl=;-r#2yY^(dNL?Z2P`f+Eyp<>BqBqf(SVIBtUh8+5XQQj2UBqq z9=-_YItsiNtlJY<&3gDy0j+sUe%wwp-xd$gPh_6tQ2-4RIw;epYLSe=LOW{-xi%zt5v}o``i$awK^mx^LG!&4P-@Oc#1Be#3L2yfKu z+ult4?$kxu;?CA(CiZ40|5e=Q?dI3MR28s-om>q`7f<;LqZM`hst!&M20K92LZ|3IfBjA#mwOQB(gGsPy#&i6tuy)Ch#i%(Lf$yQ{wA!JBWW_+{r z+io-0DiNbFb)w#nW^EP{8^s2llyWYJflo7z77iI$qSd{H$WEixJ%!-mq3&YmVdz+7 zobI$rZ@W$MJVnwfYGkJ-p(YiqW*^K_2U?FP4|ppFbO^R6%x%?07U!M^b<%y-O?0-e z0lBGbDpe+zia<$^AEYH-KjjWdF_aF>0zvASps@O?Xambjse)qEhJu%jh_!!C4HnQw z_FGE?){B@HlnIZZqkB-5s$cyjeK)oo)Ja}I57|iFJ!OIIjDvtZNxSi4q{O@%0lWyG z>WhL+Zgw=Cf}Qh>km8IuFbc6={vuk7QRdB2bAkc(w6^+y< z5Ivac8D0$;q=NF~O^P4#CezntfHEc}Xq<3Alrdx!$C(P@jP2|gMMVOCN}EIoM#85+ zX(XX6Xy+6p0ixgM*2)tKPaoSEf-^X==$G{YvkO#xC06eZbn_?Q+APq*2wla@5(6V~ z9ua;aExah0c7f1;ao?x_92lDs>F~kix;(ylu|SaefX7psK)`~yB>p^ZWjxKtBA0gy zlYS$ne7p0X7NvOi9^(vA<`n5e5@$VYYK|lU+ei)A?5;<7z9E-Xa?p@}$q#2-DejY_ z+KB_7H_o+cr-!wIsfszlltl!205>SmOaMRO%ESlZCK7v6BpYM~3I(Y~g{2WoF@Cui zxD{s0sF2?}tFsXz*jO>(HloFYw_h3cY)5~F!&n}0t;I)zj?}<_u%Ls%*wO|Q!jmOa znf+o@%#~85s>+|3QE5gtHW4`rYAzp=MCy8LEehOk(jn>RcxIiXlPDD#8Vd~aMPGQuHhDU3PY(77dhYdUN{;o$f zw7pH}1U{|zY&w>N)j9?KnF4%1{PP$#8-(EuAc;fkmmm0C>H>lX9-i6vCk7*hVZdFB)(_(`0ZG`v~}+fedH<`36YkYbs3VE3Qt z5S`Up0XJS|5pyAICZbi(ZqGNWYyK%jvuo-BAHIUOT1D>S(^)2S+o#&S$q2Vqr49B8 zSp1Q1e+~vIrce|51)W+F`}gG3z>|lK$p|D@>^)2NliWjpl0|EO)-?nNloZSXWGLvb zKp0|y1nhEf$_V{vNf(!g&3*7tplx1~hbK*1*pNrt3tpp_Q8fqbD-xuWZ zzzau-*pg@xV`$Xy<^&6Vp_3q2AX@-;y)7r8*TWCHY`!-q2Zk7On3OLuTOdjrof1H> zA|5W{qWa@)8U%Geb2vSSw0ppK1ILvOPbujq!rSA?J5iDSxhq7<1y33TNVGI%Or7R; z#T*_`eXU;l;ZLtl9CnZGljV=%$J^PguTJlKvDL-xCkA*b5dAF(+)q?6=ItrZnH1hw zF(F2a03EHlJQCqERFAWnwny7-a{4^Pj}Isj+UI)3CPtu)hmP7mph_pBl;nOos_(kN zIlWQUOT%(4&}(n+dEMXwa;a202@DWxU!geTBL0$Zho|QxPapjY(zNHkq1!!3&CcW7 zwPbCLVdF7oa2%=Fj=fuB->>5?rt}N<_RX^&_~(vAf9XbE=k0WN`LMQ%@U~jP-qk*J zt&+4nyx`ca?Q{MWWvl553&)Fa!uGgvPk#RuS&O}OpSX>rxtEs)JQ$CSNw#bBx&xP= zEz>qlcpmz{8_!kEb*@rd!Vx}I@80R6?J70qsLeJnz2&p^GE;NB4z&(z2=L+r@kuf$ zGI3kTDBGzU#~1VAPUo^aa5ZZ)^R$aQv{9I*d%Rqui*bvlcKXvzpBDW{arNi#+Z#M) zmt2E1=Z3!;+-FmPXU4Jxdz{4gsST~wt@4sKMIuME0Q~;8Ry}^MeeT6MrTG=-;o^Z> zb=*PA!_m}_v5?-9Jee9Cyx#V&U5FkkwnwdR>6KHoqVA?MD8}=4@^McK>%E6xv}eO& zH4dLnr?+oCl3j3`L;l{32ZO+~-1f_4M=~$F^DVoq$Bk36fpq;R z(AiWe>(MKu-WMv+YeNQ?QcJj5;eN&;Sv+8whr$y}#hJY_yx&AZW{YFJzw1+$* zf#%VbUNgrws9TQH!8)0>r*8c^Z78NB5r}idy1>F}or^)pPuWr5tP}7}nRALK zoX%lCDc^8H2?5(Rx51b9%WOl(c=_voKDJz#yhlLmOIRxGWi9<$67HWBp<2oH)~g)b zjmqJvrQ@^=Z^~`|k-@33qNO1ru6xAesfYE(tz)h}ogA$Ill^ICN2x1#v!UQ?vGw7s zV@LP7XTisX?pvDAVwd z_{|!D8xQ$mZ^sEL0hUwc!BWK`%lO2hhYm3o$3|CYaaHVev;bj#@61ucT3)u?@Xr&+xeH9rb>+U zJb^!-^X`Kujzn?PJOP@Stz0CDpVm9Or8{-GxrLF1o`=iRBMkamQ{gF&T)m0`lNU%^eU1}o4$9)>(14txgt^gH^q+D8~<{@uYE-#H01rVyf2;NM}g*bYSERw z^`;+}9?QefW4yvg?rHDg$PuKjKMz4}2I1Yobfc)|&&E7AGIakMie1s|CeIh{JYRBb z%@001@tuj7#ZFFX7manB4}ZuYM@ImbcX{STIkd5QzqVB!%V%?V;yN}~FS|yEgF!8c z3E!9XqjfwjZ>8INCd=Nx@EeR@nck>tPR7cT& z6V(`Wb9sXM-pR~>%yBau4(UxtadFu!R7JJ1_896ByZ*QaPLnXFCi<+EtO6)4P0N1^ z59Og~{ejxiDxX)=#Vh~e@Lj)pDtm~)yzQY!r`KF`=zKkK?eD;wZbr1`5PgZYa&MKs zSe2%iKzEOXk)v=JyUMgPPWn}Vz@5Cd7jjk6Q`7#vNMu`7XLzZT^M{-FM49~Q`k{j{ zQxcw_UBb)x)5+o({(DOHIUW$05a_k@uD@9GB;q=b1MaIGyuFY86N}ik1DI^>t>%J9 zd-H7d6WRyCm(?^KVFmR|g;^*b#$BiFJen^N}cF|08hVQrp|8s4jt zCa1~ruC>{C^Pj)krpe{GGPjDP85CZLf2BX&4{BXBb++KG!%P;R6$5zw!;U%cQUQ)C z2J1AVv61JtS&noDh6nDl71!Qn-e+s-d641Z@Z;J|%@)FY{)4M`+fNy~8m)_%>@)PD zI>(q5SXVHV_@1TRyQyAYPP_5a-m#MYcn|mZ-FgFG^os9?S4dP}YjkyvE$iYgRfa-HELpboxrA^|L$Y*!2CL|V%8e4wpepoHb@ccW>I_1y}Q?=ty`MQS)FUV zRJ#-{n8wf`ot0_ z&QWJbl?qyb-skS^>~aV=Z9tVeJx&JqfyZR+(cxBjhVR375|B%BC7|SvghEeyp5@+D zE~jn7xbA;9BnG&CDuNcM8^&w(I2{=W_s*d5T2GxSK(b6)u#;vVj>B~?%+&j<4IGt_ za(MYC5#?LIz4hgy#9cRC8xoXRwpjrdO`H^m6X@oAb)QsLabAy}>}Byi^Q7m~)&y1I zr-Zp&&ecZU0GWX8Cto<`xA)79q)GMH-Yj~`MBTkhFLQ)M;}rhzQMk7$xd$|$q8HZ09D zXryU^=d!NxpspfXotMKlj%k0bRJ~qc#oJtG;CZ7HvHHy3*&E@JpAKGHiyCGd0(lQx-$Y05BryWIU8MmaP_5H)z%4P{UXWh!{&Bqt2TF(8Gbnkm!dDu3iU#5EAOujG! z=6+gFJZL-~ge^-f`3tmuCA(H#X#$5EvTrs;$BcSkTffzrW!1@b@luOAY|t$Hw@A$- z2g>{vC$3w7l)RRMozceDkwU-qJe2GBE+a&M$m`5+lG0Gz3fkb6^lE}bf3iG@9c$Vv zevR|Kvr((ReG{z*9e*bKo8Pf&`evgOPk>j@d9;(ackM>}EJxS%$?V->3%@dk`?)`u z-@WG8Av%5IcS@_@pPh7jQip{bzd)@q7kvm@`(t@v>w4FG=QhG$oSH_cQw8ewh9G zBvJ?7lb9WEn>@4N1LZPv$4*UZu@VkW#{FsdZQb-|vGqu!dx(O^)7w=Mg!Aif_0hK6 zUJvbh5Pg`MS0}@PnH=uG&ijJS<$+#cb3QxU?`r$X`tsW1^4gTc;X;j=b-2j)7(Gss z1$?;ZopyY6`WWW+q_+ClPT>G%^S&|Ah07aiS)zAVC?k8yDNgc9m5ZOuvYjKT==xUy zetZwViC10E4=5+>nde`E*NdFVM4VV}ad$@tdk|%4e+wKjI-E_#Kfnmzp36D`;{vZc z$Wf9e%}CXf==kmIXZ`(fYSpAUIwxS*%)uXY4omx6@xKK+$oIE=f}-r85b|Hi8tJM? zXkk9sJ@(Uo^n0@!VSzfcXY6*3MXL^tx)z)wz14YtW4~`$_yVp8wz^U^m)fB~!cm_R zMpSk?S}~eyImyLZ?^P1RLDR;72QyLY$kd8Vg~Cu}OR^rCkh|X6HW7w|oKx2TUv;{> zPFsZc)!)wlR>Tv{aRm2!3wFmZDn_myk8mX%$A~OP$&7&`zuOyAiTD`BVZYV%$n8Wu zs#@bL%Hz>uOYl0K>LOI)FXYZWA^a#(=ex(yCIa>HUA7wg=!$=(+GU}PwC4l#kR@@I zj;X}#$$Og;91aW|_>pU$HqEq1k zM#MrQc=OnEP$FylYq!WE(GTRu0mmEr*V(=~^Ic}z$RL?EF>G`WObc`%Lk=K;+hx+u z%F(}^$x|cAfdxB-DXGkHj+5fdNb5?Qc6atV^5WkMw(SdDD6(Ujyj2wUIMFULIudF; zqiZ^?3G@tSbDw5PFBNGYUJde0}7)~CK?*4 zh1sP?ER7B);=f}aBQv@VWMrtB^O%1RcxcBTvPq|-1B1pWY9jJ7aqjvqg>X|pCa9Iw zAZ3TB%}VA?Wogu8adZq^iicttW~nD=Of(AR4@*Iv-9yim)Y!M=GYnMZdCBBeTa-IU zXrz^8|5ckS3;Rf1hGPE>Q{PHp6+BFeYjSL6yc_YE@aS1qCs0aDMy+UK43!qVHM>Kr(FH2M^p?sgwp9Gh9e3O z? z1swS1vuatgU%?;lxuorg6u~A|r-E=sa{r(u z=WT6nFiWpjkI(b=lsCDRPg-%!*Zr}rz_5%`q~;qxvx&{1>1L*1{+U5h`KVSOTie-u zxMK11T$7$~wYkRrY!Tu8sf)FF4|us5`pR9lG5E6E%$m}?Hc;!?@aJ*8Ga1F2_hHP= z=6}#14vudlT=D9te9hn&l$#^~W`P8SJa8hi(_Rbpf~=3k#Fq=-f@E8tX9w?#LE7|X z#tS7R1nyyZ?pYq&VbaaJg4DcF)v8*D6P)+=?K6ep5(cI_HE_y}6~E>W-~y(zSpsPN zUg8uywS0+3T<|0Cp4RSsRWwO;aABZ58B@#epes#R}MDHDl1s^d(^C3>c6o%q-Uge8on}3eFU;FS_D}8ewASo>R1a&J8LuJ zh$w)kSCg^47N-z{H4o^xc^*6qq%wV5`kLOba<`IAabcdd6|-YwrO zf8(2vXjhEm3^R_BH)xstb#aav(G^E8b;1n=*SFDtEiREOWz3i6R!7nbq0G1a)%T6y zCYU?3nNL!ScU16!4)?z(A+!Y_r;;o{!Ub%2ZIIQpT{e3pJ8TT?uUOw(t>IBlv6aU+ zpVo#7amDdV=fv=*S#DTRo4}Qqiv-F%2ZdtV{~D@;tf-s;GtsQ1A*4;nm*T60tm!Zj z7Wv4hqceGJ%4Y3x=n}Vm^_qJ3`_EFg{C96TMGh$}ZQ00LW}(+f2h(F4OpJeBn*q?i z>M$7A9Gy9gx&)ICqeYeINDdxUfZLX4(dyBRRgsTZAB>kaJEYTqrc}>G`Bj)PijUPje48-l9rg)q2yc8muV!}wqBYT^lCTVrLwc+{Jz?2 z&9*Jd>PqtLn(Oh>lR&$(f5qrLdIK&HqHu|;)J8hVJNMjLVL_V>+-N|xFvC#}7%}k| z+(BV2i_~f|#pPxU>d+?_RsPbw5wvMkR=?`|4pwtTnIH2<;4|nru>8iU%ZW|JxEQms zt|91b`Q;}G7v)h_QSun|N1GjvA;oP{XSE>T{VeUB3dmA-(Hp*A(A1R8$^qnQb;#;y zmc7Clj%qqq4Qf`^8Oy-z_^&sOB+&G;yJV!YNC1zT}S7BZbM9_P~&P z%B9IBi)+bP3?JEZbCIuII2+&o)Ne3oDE^L&YIOo|JS0|GTarI5^=Jfsjkf9*xp zEWCX7VUcRF#XA<86P~@eV$bA_rXUl<8TT~NXJArmqL|ZkQo`=DVdc;*r=C98ZgG59 zwqc!yRqG*>qP~-?&y5ZIg4Qan-TU@?V%fjIv%;$>S;c~`3WP7e0f=iA|CNBO{#Un0 zNvUKnN1>k;y4l=9ZCAw{qEVlu(L)^hI!LvSLg1trSCw2^wGd4PiCZQmSD>_e7@bC7 z0$F$x88?=e|IB{u!EQma5%s7Yk-Jt^xl}$KepclK^P-Gr5JNH+T_n+0KthZgqQ%3e zu(}oV{I5y3lH3ngb0C5$qHX+9K;w#nvCXtIJ6EL5NHBV;F=ecn3hwHsoko%yxv)D= zT}8MVa;XnOqA!X#QGO&y)E5JR(hx4y&&Z#_R30qm=~UMn5-6(@D5B_X$YBg)odPlc z6-+`)$S}jK+kY5JsK}{k!wX$PiC*9?B4f1*;+vx;KI#UN0>SyYAWFW%E3x@7fp7Pr zjq!3|)=E%5BIHsG%7<2ZtFpnkF#OEO@qbhKsT>lmAqSqB_7N{`2P0kf)zbk;$eg@LQHYkf5{&VOLoVKMV#b!0BSBZfkWKy9k z7YBdq{y`3v03e~pG6-ZwdP~F@2_C4&mZx)ILdRc1!pl>IAhVs+%tz0p3cJgwL%nO& z8*)48;vgS_?~fU|QDkO99;0)WJ;=#ujTKvwB+q5Hv$Ui0{nmi8JiWs>8seh)g#lu; zK!uG6hsIU#2HwX$tu`cTL}xOYis;oF;@-5m1$4CS))#&eUri^^L~zAyA?LDZ z2b-Uu;kQL}u*j80vTO`HP~FlWCQuDKTUN;Rzbl1Yf_h*##R#R5#V8c2&TFd$TR^dK zqiecZ!I*o^Npr-kUhl>Tue*0ij!47E2C+pQLm~aCJ{!YMBRDtJl{pJf!aPJ@4Ex6~ zac99F3p~S^{j0w-mSTpYj%zOcJ%pYd>E|b#fqZy7tRm;tn~W_8c^hGjBk5eFeG}=U zlRj9;hj8{oiO9B`^3NMrc^QPV1)Z3JDvrVpf* z5Ehzeb_uj=WTrAF&B;5w%xrnuCWzX5WG{TMSit#kND>maCWvxwj8FMj{lU`Ieu!~_ zURV`2cq>sPi1bIsEYtxsD&(~`$w;ythdF`j~I&%yw3= z|Gp>ysrL$eZQ36y5b6)%DH_iud0d^~NYqzt`9Mmwn@f*}+Y8;F0*e zF>L#)kft|u&&%ZR{ojVZsw~*k-N*fuH(Jp9)h55A+|geDD&S^(mFODR_c3c4$?p{7 zyoHFbO<|j_<#oDOul4##GD+}rZ`;u4956C|o@-;=^m2GLR0Q@eq5&^&9Y~bwq+Wh0r+$RY&jLff_oVbLyhnYWoBlo|Rltvz>0aTrE!b&}VdpK%?8t&1t8RJO5{m|?GQzw2Ab&twQNP~A#&A(q>cV=+R%J}TZ?T-|cU;!@ zGTcHYb1hL5j@4R21}l;2uf&=xtp69|8fmt%v4Pccn}aa^khYOtmGSN1OUzs?-8eW z{M*Ek;m^TdCx!i5b9(N(8$sT_Y?h=dg9kpRby7HmKZuI{u&4CK%im3I`;2CHdQs{# zvA?z$qB;=fXd*(q2pmvy=D`Kb;TA0&(|9m6fnlBY+Jk=wg83;gfIq7aC_0e8A=ri7InI`D2vUM+~wGm66hS)*SVRd z(WKPT%*9m^8QD^}*U^cEBQZ=qNAi}Kh6w@OH@i|LmWIMy7WekH($i5n zjp;HM9{2i$QO#pm9qKAd_HHX4l9b7xO!eq^p!!uxBL;1Wql%ai$H1PgNW6oY>MQjE z+F1pIZlkGrltjUAImPo+oG}&|A8|z_){IeGY5WLn<=+MMEbKLw(HI%QmW@USDwweZ zLRS`(!IXaG3lG2M)YMnKTA9KEE9+nF{x!?#T1p6-Xsk(&(P3+(LbXMd_N&TQaG4jg z=?|vSP^xOHS_a9Q&;jUnE_145r%FRnM)gpVY0a?GdgMM}cTG4&O7%>GqOKRMWc^M) z9r&g8P--eOD)CV24Q|E-zYa^&?rUfc*$L*BTX5n3(rC5(M9!ytP(wsSbK}t*qG#jJ zFFR&;>~W&9)(NK!X+X;z6_xlmM{O6^puK)q*d|BVSJIqHeE@VHN;QwAKxH%?sy1%8 zK$#)_XJv@)yGz9H&NHdO9;+L8&SWFa;M`;S`g$iWCN9GLB$IuaG z@z|}n8xV9U&jO+{Oj%FYj!0o4m;G5H=z&z65n>>yf4Iru6K*C+>_M#Id|Z4SL^vIy zQb4Jp{rkgWVZ?)zOZXy{6g3q?bh{!Q5Cir>f3*B@_=UZYPF#5v<{3dc_I(uS0Y*>$ z2X@;g5#2VrFp8~C%B{7i)`Ls}g_%$szs^SL@V)D2*-rq{ixC&@AMiwc)CiV%Tzn`~#KM-|#qMiBv-s zp;nX$3En^*m!k6pQ{2eED7vyY@@j@0>xNend|fVSHvjhfWZR3#D7oM24vgZ(67*fEmyPSpC!0dt`|kOC7h6}CxB=$F3_e- zAO=TJn-oCK#7&R!AqF08pk(XR4^)Wt#-M&dq$>H*C=!Mpe6V=i?4xxJ5hT3-l?qmr z`uv$E^eb|VG&fr_gZ<{14rDp%;6Hhw<_-F@p<5aG1`J7G#1$X9_Iw;VwajUVLovEJ zo|F*?Ohmx46CqFa36XRg)^Ux3lfCZE*EK) zDGJgAs5BhAHsSo}+wGD}) z{oIFKxT}P3O(}p&s0&q1>``>&i`#*)&v@M!G74@dZqFF{s9hCV4Xw7+Jfe(ZDqR$C zT{7&W;sx!Oof-dgujhxql$BKc!ZougBqibucl)^i=sE2yxi|!=W$p7xE*KP|@`gDdLiPl(yK2@-o)14FDTI{e0Ml-cQjedNF6|>8s2_*j zlC00*4*IL55kD2?!=vIy=AgOu>L`b(V&-LA)Ucy^p#4z>IWbhKg)~GLaw?Y<+SM?w zr|Z(b=0C^Yf|G(j}de~sAe?RI%BTbmvt9Y~$uMbU4L0idze-G4l z2>ok1dV1$d z8$203L$fs`hql4D>WV`yd>+<7UFx-q{O+2cPc4}J;X*4Dh+Qo|sjB}@lXW95MC>#Z zwk|K>7^qzj+eJ@6Ieml8^s-+7Q2&M<$W9SxNYmfzQ3mZ8m|(Z@&xUBcq^k7Wsa!>6>l)e}SmWBlJ`U4Lsf z3hJ{OTqvpHML+ZjE;JzeC3#jIuuC1Js}t3KQ}*&J8ZiY6HnjTl6QKx#Zj|%Si3CU8 z8|gt5ZQ4M1??e^1(b!RX%!504gk;$h*fCZn7RX8UX`uKq%?D(}?)?+H{I>xX8?@V5 zasi#YV9%6p^U6lKhw9)8P|j7=^RI}abJ9aOJc&@h@vH=lgs1bYa&1GBEjhWQMA#=$ zS)y&Gp1R`2431xc;z&Q!+QDZj1TOj;x%;}4cSEfbaXx^II7IY8@vab*Td38R5tj&b zRioOa1PWeV>=xb3{Wz2!>ZDlJ9!dE~_-vToN0JjXkjlpOGwUD)RtxIESC)}qI>0}W zJnKmhq4qy}E~boDYB&q)MI$94m@RPjU!qrpYGHs;&LDj_2u=t{wcVRkhAnV0Fz+C% z2+P3A9A7#1IQ;wSB3(GOj1AGM*ltK#fkX z(SFl_9NO?l#Q1$+`(%uK=LQ?i&C9NLu>{YLFeRAq~gIj?#_I#&qYRH=PRGZJ*(Zr8n2ATQ6*Mp~s{Rh-Zg^g;Y;><1aC z=IXuLuo3#0+`N+J34F(G=dJo5g;hO&`rgk^d!MIW=Vf;lZli4#<$dhyu-R_3oqdRk z)z59$A031lB0_fMEF1K2^nTvl)a9-5yVBpFB=95TN|-`uF*)}HMB$% z1fHKZ1`Us%@Oe8ppLL&1tW1v&!`tHAm;BfKeTFIPBowy2FJ}a+m$!T8+dLOPiLN;B z6NL^EUjPNb`Gb?_+s*yimG!GrVe9(^%kZqO>vpRx;6?1BC$Xu=k@L{tYyG^5D0>Ak z2lf^>-EY5r!bk<*=aooe-(441(|NPM;K4VqCT4ax-&&V<-vd2({kNdlu5P?2hVkKZ zm-P|jkB>E=Lym!w#5&IllK=#zi40c0x=jUmTKoXxJW)ib>vswh=VQ)x3!fU&VaLer z235#~gPSdbmw3J>fG@hGl$H%_GSmAVNH~tfI!!xR>Ye6neF)9-zj!IBf=2oWb(iea zHuX;x`$$~mv7Dtk)PnAkr=ihGBB*qF$`8Zmq?q zxD7A9QCWEsf#5&THu;$r((#F~S{|vksNcxRI1aSUlK0NsY`zL$aEE|Hfd6bsPBvH>z?lLqhFFD-HAf z0RC5utZ9ZhekD6_FageX=<2uttD}cy$7a}SP8cRES0hHk+UBYUcb&^zMfU(2$teY`!u|Sb31tgmu@u=r2%6<}vFyBbW39 z?EOQAqbuKW+EXICyY<(tXaDQ>fL=#~r^^n6=OYxA>Y#$JN8qzSQ&h?emZVwd-g@;x z)?(*K7Nbt1jo@s2BcbZ%aK-#kc)fw9)baM95ohLnbj7hfF-lZkvm4VqOD1;4BUmGv zhpm|0MuS4!QKxaUl2Tr!Y-^wzwMlF}T7$pWBHY=aUYcBDGkNt~~M&7%1 zeU9vGagaEm2GITMTJ;2xn{Pby*mb9tLasId^XNbA*qt^{hQ>UeOGFdHTZLKDW7>gc zqNzE7C?k+##Z}4}77!v)S&F9Az)>VzF$dvZ;TXPPa|Zh}jv!RH_>4PkUgy3V3faQs z2s~&m`*H3H&u;3`Lle)TZMNnm7mEdR)S;GVc~CSY7Z6859p2q6KUuRRGNLA?C^xbB z!*Fu%mb$}6{gbXL(#RStv2acHKNdudi=#r86_q(W281*Xi6 zDaq4VSh}AH9bv%?URZiqJ%Kb4K36HjsKQIf)E40%5Z(=n4PrrqzMn)x1!uGlOGX7J z4I5vn30V@$O*}KT5~QeU)BuxR6`4;Lhn5t&W<9qq-5gv^BP|ql8bnC^OCC~!FJe$#Ie zZ9R7y}a+6sa1xdSYOn~t#rQkn#Y0{CtvOR_Ynd_bp(U~4s%wfq)HtpAY)`>CNpe;Gh5)&*}YqZ`<~~@Ynj|``457THR2#8u-lZvBV_uW$iqM zv>cXvRjkA3*7d6WxJnzR9JZV@T`d?C1YEEBd+k==Mk^pqv0QoMd)}^9GvWh2AHQ3B zq9nS;HawX;JzS4dcfZ)a>CHTXLna$r1mXR?;;uYa%qYd-b~F0S;rapf+2^XRi;{}0 zsm})YfTID$0GlXsu}KN8)7MFSxRMVEkTwwNEOS@roY8^!+XZxykUP5RL zJ1Mey2{)Kp6QPTqkKn*13aB-63j#of%9M)}y7@c9UyEmIXGg7Y5TBMUswy(zCOuf^ z(NVk*dPT~V9UbPXW;fW}vkAEKmb!_TyQE!%3`aM2kVG*VRDToc0^BgGtKWkEiw_of zWL*6Jy%Lip*YnW=T7VtivVZS9Z=z);qrfSZY$L((MG-L6SezW!O1>JGD%9LDT|Ade=OdMOP5~KxcF%2Q$ANvVP`1sE3Civ~oZU6Opbr+_Dczl(san1X5i;0= zX%13f#cmSQ4uGespMLg|!I|!d@cVoH-gEu_^%by9YB>$$5G5Ap@{eXzgjGn?{FI(Z zBC}TZ%sK4J(w*{Es8y<3kToARcW7Eu)s=nbEvtxb|ghvzB?>!>UQr~rAqA{ zk*Uy`s1XAGJS!PbQdb|3dDkUYEkyN<9qQCnFD<#oT6AC7dFr(twcaQYZJ(ynXxXji zbzU8vyANOLji&Qc%BhT(nD>x3G^@Bo89R1E^`j70>nB^z4O3lcx))N5PVe zE$URPUGk)^u^Y;kXiP5N$F?XSDX>3$cQT_Q+Wi6?Ru2x~YzazqtV)hF?hs1U^efzt z(rq0=9G&^r-7@1ElNeJ|EARD05l;RlXxer_w(C@I=4zHs^S5AECMCF(pQPl3-_J-o zEN{3EAJQXm3$kp*8M*UcYsfCD*ZIPY9(D(kt2xt*$Kll+MC~GS!*xg%@sawCc}bHF{Aob*8Wq zUza30F|QWym#gY4bZi#%2rt@(ifzOtlL~?N8P3dww{-|T@Qc)zxT{YQjzSSP>VX)z zb#xyMCVjIECQ}hbD$MOeolD-eEg$_S#5*Oj$mRk+L2E=YKfK=p%C%`#ts$yIe78LM6oRh*!5 zoOo7(45~M!UK#6$u3VYe20B*bL7YT5hVvXI!V)#2-3T7RW@1j7A1#eG%s`5TbyIjQ zp*(9rlY?}+qiHWE>GrRv7%A!jiT|z^f|dIyAZ363rrx_*l86zbX|T4A+M$-WYD&wE zdnidw!YMKB+Iu}~50cBOcjjB$i>r4Uv6AtUE01~tV4#sg zcAA}HM#8k{~_X$=pxC@WVWHMb=Vv%HiuA?mQS}|g5BBw|WD+)~3 zBRufkDoIQDb@(9X(@fhFrRkOjDtYrNYoOa%%wnl1cwkk3p6IRIO_~uupbH8KZUwA0 zeK2{Z+{gaW@5F7b@&-@w8T$wTQ83n#^$?Ju((JKt4{`TlYY-gi2_}iT|4}o&@V=_) zvU4w7Q}K$`++gt-l=&PA_r48pb&yX<(KubP20g&6bwu1E@8xvyGfG&?A&-pdUs3l{Ocga2Fs3O8GDInZHp zoIYp>Bd8z#2#h+1;B(PSB`&W^1tee#4PJF=adnP%^(xUiT{2Jt$$LQ<1gT3RI`wsJ zC3%bWoygdSVI(3SN*=sO*vAs4kBqUIs30VkR_qw;7%d4+C93nw@w7DP^E3l2_R zDi+^8c{ry)#Zfu+R+0(uIG4-hM_Qw`teQPkQAG?rhd>UGmZR;oPJmRtK=PCS7iI$T zr%)C+U#8Jidi4XuSbN@yMQ;(bWd{H_(2z@+tx361<1UgEf%{OR*L#=e&bPXNqB9iy zM5mmkwOlr|*h%HGupzPWao90b3Cm|^Ng`gbJI~69;}ZCwNeKlmt8lIgh|GsutXP%{ z?0?%pVT}09IU2B1_x?M8Hgzq|`-;nrn5c}I=M0S*jExR?wIQMLqiX`=Np3N2@cNUM zC$5w&s*r7U9iifNXgFic2KCQ6ZEBYSr(^nNWRbU_#86xRI zH9j+)Ks_W{wl@_(#fUA)ApbD?g>!V!zszu(^nf@RBtjKP? zdKR9dZWg1uF1ztH*2fykEi!8CVmkMS^CyRvNpaXRE2hHZ?SCig$omoqUBwGql*&xR zGn2fbFhO5YK76O@`%EmTW|85PH?d-8M zaMR7ZGuE4#x{A9nYUONY{$Vdc@yaLkg~Evwd4f1T*u)YDYA??^@Da)b!IIKzNa_ep z)U6}&HTLPtW^RPEF|W^IOVk>ZF>A(IeIiy5WI!uS-{DX^b5|OHiLr4ZGed(Pe#j~( zYleMBd^`i93mfIB*|qqs}y(1gNSo4HILuQl&GYS{Q!k z>NVZ&YV}J-85i&E%j|ZbP?5TzJR?1IcTdQGCJ`4`BoaTK+u%u$84pkGdyPuqkNH?jD}smNDl@MI-*wn`Jv+A1v79w zzkCE#J!yV0qwkycAPGDo0NUyTAuszNdVLDuC75Ae88Fuioc!=iGvRhE|7-)tl_S~8dvO=M(wi%R`8>}*CXZICHnSqJ`nr*ngheek zE?z699<8NG%i z{R2JqK&zFtn0U~voeN#x%GhZ~cyWmXuod$c#Ll^`~sXZp8?iaX>?oaXtX zIM|k3fPSEW3>Xel!w)05OfT*ZNr_cdID;Pz@XuhrLg`hDw^RZME*yfOE>Jso!v2hc zP&VMAATBOEgfOQ+UOfF5++%C}%W!~vXwErrF#P#?iul(by%qz{ki5ay_C+J$AB?+m z{$YoR5N?Tl5&rTo9{za{*;YjY%i=&&3_aK@Da%?Z2m0>e!KCj?>b0-!17yE>Zjpy{ zTHxw`Kd*L>vL17gE}?9x?`cB$4X}1AhZd5T3!8-%rj0*|AFsQUzi-yyOd5gNo%w6Z zd0oO2t9*gQ*O0y1gZo>Ds?0uypt;bpkmeQoVA*d67XB&%crXtChJrx15jl}oMc6E%JmBg$$_K9e;~e;xEaJbz?Bh@Fv?Y{& zul%j-DYsJo-eB-MiR%X5x*CfFUM6Rz756}78@Fsw{EEb5 zQO@@Z?Xb?UEa&sFO4g)I%*@QWICyw+s}XeMk7KIr9Ic$*c6#eA+E!RaA|s*V{Oi2# zM5s2Q({ZGUmg@VJ_Rnl1kA^QDiy2tlOfJzD(nYF{%fb^D>%g@A%=znoE{4BnJaca$ zHhK+VD^}>{9$Qptc?6{f!GDduC5DY==->o)!={Nw=!z0cQ@fxZ|9$^|z`Bw%T-rqv zzBDx;V~whW)_p%(BxhYFrA*nN>}OBt`H5eN)jsw`42$ zs-i;-9H*&ee0mlH80W`72;@KzlDat-Vv<}Whzz`x5=}~Ylh|7DjJOanA;Wo_hul$M z)R@c8oEDRZWF{b;m%`z`aFn)T?%AJeLdi~Fl7$ZC=OkW6l#+0AhJ;NM2_wUCh!}!D z88pMfFl3RKZ33ZW?*jG@P5}huI*6_>pT%nmPN81hTP+|>Xne-T!`$ri{>{gB^*I1= zTdh6tJvUlrv%IA0G3#yZ^26J8ceevD6GYfJez3FWs!bd@mFRv&^E2@vNp31>6vqZaJ@$&#Td%BYnVNZcnXB zX`hw9POBUPBc#LAM(+LN^7z$bUf<@w?GXcOYZb^OUFc)Q85Q4AE3Q#yTj^l7SwyCm zy5F6l1^aT;KeTI#h&Ca3{}3iI!&wC)F;{g|3Bf9ccu<&Y{q_G{gVzo4yxPkoX*njKJ}+nUfow*ur{T(! zL1*LWe8??}^K7IMbwo}8g~PH_d7UfvGqJ7yQ&)gTt~15;JU*1QUdlK#IJ8_nel~tq zbV|WWL z?yvUPku3wxxbAqzpQY4?LbaNg%Ar5)$uyZ-fkXxsHYs;poxk5TceS|<*!ax2^62SX z(~la|Qq!{_IS)K(lBRj$&wRRb1gslverKU8EKg>kFCd}=(tWh*>EkNS26Y&ha&ROK~zxH=NumZOkf7d0Nd{XiM%}&a78$Dc5R_X(~!| zx06?aA(y^~jC1rxi<3WvRkbq{e_uBXclxwyju%YbOLWP27i!kra8xdqbg5FNQgi|gR`_N$rY5qi3S4V1jhDGM z$if69tl1`Ppz*qf<~Eo;S@aIQVKSY>;NajKqR^Pe)xFJTL7?sh)68Z4Al>}=o3eO8agdbJ#L?(5s+jL*v{b9?b&TqmW=c5|F(nhR zwXfO{lStOb_e#2rK1oG5*;@R;pxAzrBtH==v19RaLGXeOiDeB|kXY z@3TWiwrWO;b#G@(Y#8hXA5*R4kUCp82&-AlA+$4I9{37e+uSQn_#N|!n*n;nAZCd2bXf(DPGLdyYr8P1xMsr4Q5eMx}6P#W23Y` zZ^7sDUp!j(x(L;dc9?6unrW=@RqO&j!3#_A3s63%hQ1gLUn=2i@%Z({ZdKDyK!)2? zEcq%_j9+*?wgjK3aB*Sx9RH>n_1rS_+b+D)!^KN8*=z4A;C}{gp50is&HtY7I=VW1=VO3Zq1?fqD>}eo7Jxrj6yJ zGeYAV30tax)Hr1;IuT?q-di&icqRz|MViGd_$;sPk@p7$d$We{A+Ms~ton?Bf%a$p zvD)_o4{TnsiwEB2WLDf=8n2LbF+JFNxE97`oD;{Da|3qzDPmq97|zIYOjZ3C z{OTS+4*%D+jR&m!6$B5t#Me+hS$x$|klE6YSR6YdL7dN6k}^Dan8ZZ@Wql`m_KO^a zTg(zLo(GkMDas6jNBQ%E4*~^YNs4okkZ4MZvq>m0;LJNgnpN0XlSO@}AJWF?0ZX9pu%Bd zT9xmURYYp%i52 zME?V|nV3iVyEK3WHv5M|BC;14Cq<_HI;u93KBRsLnhFFM;?G}10DRhEv_47+Dtt26 z-)|7+9lP?-`6x_Wedi0;rKpYEhhcB@{ge{H;M|{aLelk&Q_%Dnk`iV;1=WB{X5MRM$c_R=*JiBP! zBa#bzLHjCuZsf9az+Q<%2`e~3vsW%?lYp^5x-gcp^8Q~*gg$X2N2g0@D?#N4ZX37% zY@@20+7ms=)jFQ4GCMutrnn|A*-meknA8j6{!(FEGL@Rl3nH1MuF9&b`Ldm_R0_K> z2S4H_W$wv31U9d0%=|s6G}^4Q{;is!r4n5*6gR??O0Foe6#PdU$l-4ttEXpY_PoOV$gd%rXCbCqzjX1*=%E z7e#OiHmk$TIPMy)`@c)mBbq*PUYJ0xvCCoU7b_i(mu?l4HEB)#HL4)}cx9{%#h`wB zTb0%`YdPRlYN}~qo*;-jXLfm1M_We=VF_C_)}BkM_uGD%Y*6l?ombj9Us5Eg#^USq z^d38zM4G0-%Kta52=7ZbQi*OleZue!q^v+-p@unxWftX;5q^kPd4^c}h@m!HBg8%D zWE5K?oHJCZtqXJ0WudAOKB=CmR)2*wkEkX5G%3LR7p=gW&|!)Q=|l>FD-ZTpCz@-8 zPVzExxV`w<9i{K++NGbD*zL7_(&M_jSXb*-@$dqA8{bXDyogbU{H`a~j~Bm`n%H>> z&VEe(5Qk>$Jy^Ejs=^1!bbeX1KmmN3qoj)<&kn_Y6me^Tblq>#_kuDWZ-@8M_AUK&7NQChK)*YBW~Cf zU*G=0Il>HgEf6}Q-ah@c7XKqJEjZfYlM^r@Ft}%2$pBCq(%y^W8Ao`3{Jcwd9A#Wt zYnwJsw~v2cbIytt-#EKU;K7^g83)Go+o}%6Gu2Ux$`Kio_c_!`O8d04L9zU1*X-FvU?zI$;e*Vy(v zg6?@=(J?UiHgtKuU*ov4`Fj5QxPX~-(Tm&2O#6+Ko#&~4t~QFh!T?l09sK=>vf*B# z8&#^oB$R0DhRHZ~P9t)RwYN zYcOEW?o2Uu>nl@iYW`BUO%3TLcn)q>?oOezyjm?!!r%y~u$|LDINrP%mzBoJmCDN# z#MQ!N1-?0WczAFo!axP>0g7$-q&d0i3UV{DQ#CN%N4jG02v>Nxc5u_VvUfr=n+}|% zVGNv}jd{}XFO{<>__)}c6FyhcHHI}F7>-h3L|d8a^Ll61mQz&yEtN;n?bHv5)tZpb z7E6;S42UUHiKd8LJ*^uwd_Bvt;*(Jh$g$ zI~Vn;A2aR6*P~uD^9YCd3{8x=v7{}-T!rgcbf<2EVU4q0;d$gW zD&|_1;l{KT=&r(F0i|P&1$Mj&D^UBaH^PRUWVC>=bYF4(}&WkTHT7{ zf{W<(4?eq7`^4jhw*#344U|lV8H#$F>(PMsW940GPvfzVYw=YVo9^=46&`Fk~w7i3RB(INT z;&1&_`6I9xMLMF#9xlrk(d=Kzca?C zhp$A3y42Q@|DqH$>B+CQP!s1jW1kK_5-R5WQ6J(V1jUON(MV*)=wjE!X^@F(1!4r% z4uJCWXV@Ic{nEJ4U;5if|6Z=K2kJ^go!x@&6uxRT@wDENHy6zCR&$a5n3Ic|$jFf9 z>-`5V%U1CMnS~Qm&L@o-HcZvhGTC`(rap7N@AFZ0%eUg zBx5uj;yaZDxdgEXky3nS7el5VDws@DoiUxNegMsw)=bQaXYDFy@|Q8Pb2RqTjr;;8 z;%r7;60pC@K9nSzZZ*ja(HLSC$&u8fj}6?rP{{=MPt)Tdv>-$#XGrcH3<(e^m`;R3 zsyicxFVGrMO)tqJ^96NSQY97HHw0LogG^wCjQt)al~slJoAI$yip<6T3O^Gm22(Lk zC>>oVM$}_gR#Us_+mx_*mQbc7L3earHI^EPls)vG6lG4p4Ef>HTr4Z%C7i;eWdmfR z(Go;uW4w`(uO*vD_)K!?$X&i?%>U(K`#@Q!-b<79ZJcdQ7jC_r7CUhb!?8g;7>=I} z!(Y5COCd{Czo-}M136Ql{Y&#Oal>UYfn(lxZ$f(q>X%cqR_c`cp<|+SzHE8)cmWZG zgCXVrdG`y>RUff1KrlgUvdM$140Bb6Pj!eZffd4W>)1XpcouRy#lF3h0O)-fZ_qYTZ&kw45If>{%`a#&Yw zB5lW!XJB61!z3hqiNwxinp|Nakd!nz3s)pOSPpywh4=~Nuf^o4iQYW=ivn|K?0W5D z#Yn@B`_rY8vJJxemUp%bkuABh8-k_qeNC6Kg?dPFoU?ns!|aK5*rjyX`%_`jZHs2{ z*s%h-F=2d3U)Dv_82XR!+I$>09fLDSYuAHOO_V0*Q4n^7|F%f#r}?j#iFHEwg@puJ zPt(Is)2Cnw+X;Ry{sN;RtOMfVIAgUx7UDZRX9I|J;6^wtG9Z1;K|xJ33oXOv0OHB{Xnb(PKB(FWaht5aDxJ5+F>+$s#wx?NvvJ3b{q7{g0kYI0az~u!WLGo z!4rityq(1s2zY@IVE|ztk)$q|8bTO}H3_IfZwsD@4pd7_05MLUzVNu>eiNx`2U?gk z=HUxHr`?K`iIY!R4E)BgMv@Gs4J?A=PUg2+c0W`7JjOp#FNVLf6Kzl)cL`C~J{s!E z1r>e8`dNijG(po%f-un5e`2)Q(X6Kw(3LikG1w;jOcV*cAO@Nx3fw@6#*SVng_*fc zhi2GK6J*F5!l$uCPm&EOMo}E*YT3oe7>1*W4`4cD#zo;xOR{dZ>TKOP{<(p>noka5 zZ&LUZsVIO3t>la6NSP@4=3RCt_2a2z`xo2HQpqO#ZeZ}?HEs%JAw=kyY?E9|WPu>X z#`U9rjy}vaWd@K^g+8To6vrAD<{J-HPEX8!2$l_32$-N=H8^VL9Im*sPzNeWHw{p? z;Hbf|ohFN_&S6l1V1`@nIWaj}t#ST*D3|eJ;9`pK3nX)>OF-iX4XFixE60quMV(N0 zj+s_Oo>xs|Pax*YqVcL!R%n_do-{+F`4LpJP)yUu01F&-J{V6VLDe}y7^prqZzXSf zBfX!4u5}^oEUYi=d&iR_?3BgbRH51C#d~_I{aq`jH97f>J*ALNPuo7bXZ#;6$~^W& zGBzvrghR}{_f!`_%x!G+(`#o`bXzZ%z4$2rB@k{jDee9zrqP#pEP?+Kba9exdJL9H zIWEQ?9ww0LtB-)|&_uO)3&DnBGaZ~owNC1yaMuZP3U0E8CFFx^wjKN{l>kI`@-UkS zI$*&`v_QC=Yn>cd!T-YSgI{tAP4OD-Eq~lVgj}EGTo%+!zuMxz%AEyffvWyi{pqhn zcRH*s<_e3Im^&OTAFn$Q5)D(aVYCD4W+Walz9nFOHk>@-viM>1(a<-jP_JiRRUvax zImiQiT?*TtkWPVIKMbNfprt-6iP%c0kg^Qh57ozGLAnx+jGwU@1U@akD)QvEooS+W z=m*iLJ}cyGH64}7n%v>E6!W!Nx;xE>%LFAu1Nh|h;mZ`IA;*ULCUUtE92>3J6lIdA ztN@U9pfY>1e6`qxKGpAl`wK-GQpc^0>Y~temX*qThmLY_h_)vqqTe!bubT9=G5$kK z@+*7bny_b!iFA9nm2b>mqiG}mB*sD}(#`wEwW4qfxg}R%pm_r+ZYJuMDgnEU94olh zA@xz-nR8cOaT6i3bGpcbub47mAK@c6RsoQoIG*X~MAkKPeJdBV?ek)ZA(!IHELRck zS%{W*#*9#cD(WP(GK-2Rg z{=}(9Vi-#4>dT0L*{Gav68kJY>jt0+W5dAAo?vnap8h}t&p{3~ zy$n)A9Pc$%JV!IJk-mZnwz1wo&2?zZmAhxVf5R|eDzvadd_T~xZI7YqX$&MN-!W?L zfGIFied@eG6p1SQ7I@f|oKFMhr3OK0AKG-iCt;r9DcpO?qlT zB|q=qFrwT>6@5%Z!%lGZGE2Quf?O>{$O_3QBL2k{7@(@xp(cRjTHS@Z!B1siVeku-`j zUfy2f{6Vz(q1ZZwnDOTOLLeP@eZ2cvthaXqU17*e33~}n*))QF2}<3h#X3(ahaK-H zFn`ZkUDw*+NIa1yiI`Q_e ze~a$qXhsNvY7|^322hjX6h=)v22bLs*GY`FrY82B98mZ@RJjq!vcwY9DRJZAgCZ1i z&a;C?8b6FMQq_9=B?_M)EZ@;7r9*wtmom5!drjq)ew3E4s%%e42x@xWUZg45PH9=j z{k*{H-TwxOa#p_qfd+od2VOgVd}+t#1shRxLa!%1>1XBPq96eC`wBGcNgh0|+ym*h zWZ>!1oFCYc<1Eb73PjXpXqQyR+`{yd=!qub+Kfd|-@gPMB5!1>_bu(Tg@Wfq-Q37_^&8oqBfzo-w!wQ-X|K6F`2@V zi{_V!$S1w^zOjj-4`{gl_=US8YO!kf7nNjU2d8;3%`jon-vp4Zl(PMh9@=(Xz7oTV0E033 zWlzg5^aoFvw+~@%4YIiO0Dd9g!XCK&V`~(2RlvNU=J2s<*HI#gs>_*N6~pA&AZU+j z_kfKnq)HgmB28h!jcXDZa{(>kfLa&N_lmtLw4U&LoR|*mIN0%@V=;WF@(7-MV;djs z5ktGQum5zl`ObWq5JGsmg39+VefD2mar~0|_?pTFN#dyU!l>#g$~E}!QuC2Axo7VFD9x~WBd*83p!p~(S1S_Ni1-ZMZ}XB9X!m1RQ&0n{WL2 za=(w*Z0x6Yw8f3AgPyQ9_m1~Xp3>YaMg^&3OsV6xFYoN%sim0d)CyE@m=hG~vg(!R z6CR(Hi@%fp?vg`*f8@+S%r_+z)!vQ#2gP~-zF%%;F+*zriCsB^A5*F7hcHiJm!Pp z;?evl6tC@~wy&Q5RV3=E|I|b^|CMVb%nVf;OBzYJ-dBvl1uhC}aS0MCq6AdrS4#-R zQtnAIB0*GCFS!yd)@ZbLsEgH?iFN>aMNv>w2k!g&W9nNxzbD0U$9d** z+)s1t+9)O5O?!8w{fT)50X$;sDd#F{<4r^k;8jYd}s-z9V zk#+}ZV1AIY>(u-y2A_L~s6j~YRQ{DV<;r*)41%k443sb$H@mP2D}Vr6D6Th-3cpgU zrTw_6mUzT%QW1Q|A>5v&cZAMMBbfFkk_J%`jWZ9y@wn-lxQju$3ii`3%*kCSv}h>$ zmy_t5qdV3S`wxmo&#M;J4}|obJ}B_EsuQ~e*=Ir(^vxZ_gR6>u(7-B6(Mn3+;>eZ6 z?&sO?@jNxs_E`8?N8r)~Z2c1T28296#*9EvnFCq5Il>D4#Y);Ur`+_;WB%u97)E9R z-4SGX9PzW&`|8DKK*1e1zqaDg=x@U|goI0zK>WZzR#>|vTr+eg3`GZl2?J-Ijc0+6$zm{jUTLF@UIYsG0aY2(liBn~4+^ZuF!y~e{1$Iey0->2# z{0Hv0lsbbr(IgR*{I5>0(-)3lnvwDgH1_c5j9ieIAOI2p8!g@vQCDl;WUjmYl6uiC z2K-`e@yWIr7gd>BaUXIIO@s_XGl}wL=;*jFWensOtOL|2s5}T+FC-VV z2coAthys@%ABdfW2DlfP#lu+$e-|kJ=sU=p^ao$UbJ7da`#T6!JI1SEZ`i2Rm(b}o z?9D*O7vK*P5sD;5$G1MF8A?d;#5gH6j1v1r03KR|STB*!2g3-)g}ovv^))vd0R)Qn zKpzu*MWB3>HYH`}Nj1r(-cXH}fhi~8fG&VAOA&M&Q>r8yL~X*=8QSpY3nuP_@{@kc zHYNFCNd3bgq44ZZ9`qwS+gM4=ohjL_N$Qbg1>C22Zu1y!8B#31vP@`Y;u%D2_39_c z);lgtK0m-OtEfh}Pk+5@M|?3a%HGwcMVodNUqMB* zFe~OhGzX(|!rrD{v_rBeH>ON7>h(vs=e<0_i5pe=NsDV;p%5Wc`aAZbicd35tcz8R z9nkw*0n)^EBpm4}W^G0h+fW|(QhKWG>p2q~ETZ&o-Rfb#gO)|{>(t3Qc!97L%W*BX zpP%5V%Y&awS*P=EDteFXM7Hr5_x<%47&ur_uf5}W+Gg-0YuOvYjPuQO*6e!SRhl^} z8yCmVw3nI2U%;9QvC;mu+`1C#=zPV3cPZ5!Z~&e2$cXKveF88Z$_{iu7<;_au3Xl( zQe^IL2;?l<_ob)|9SykYM13_nx~v^O$vi_1=5+49S{PQ@5ALBnw=|1C(otLkKUvbz z)p{OHbVk(v@F%?6oE%h3#Cg6BMq8hqmzDF^-*ft+J zRi*EN2YItq+FD1a6R^C0urMz0YVeuRJ_MWHdad{Ru;Wr?Yj<5l&w=pl$9iO{NZwOR zoNw@eX-==l?k;uS@a4N3^^V$+*DuouuzL@;$C0T9iIuXhld`0F9(zMtMVc*jKM)5?d%&`}O?8YJcp zK2c0=EKa|g8H9P}I+11q*hWpu9YNpX8{P4T6Yt+}9*E3ea1Rf&8TvX7J&By|xwtarg} z)PT{kwTmEM=O{`_!q3~>G!gNfvt%KrkXN(!n2a)~fA|2D`d4YCck&XWw(ERqVhTFC zdFVCx8+d0=^a~BGCnR|kS02_XTqKKjHHNixHysDZlC~`FA%>1n=vb!QShOWMOZ{xA zjkFPS*~uz%G`1>77S{mhlnJ|;q$jx#$GoAEcx+r0K|&%L$9lY+E8;TbSx{hLzj9zu z&?C_sJq6$N+HiLEzF-y8 zOd{g9WyNx@k99{-NwLnKVUQxWFxtxSCZ9Vw(NA20|2LaxCpbMKiuB(+$X_5JME|39 z28`|9Oq~o|T`X;!|7SF0j25g5>PklL?I}|#8{^&RYWm79GFF{w44IJ)qXV50>jUkXPP!dZ`Pz-}(OfeipbM{kK|IpI}sZ+5ObwYWw;?Ie;b4 z0>@%9VHLDzLz#%b(oj(EL!5!itj(S}9gATxtjU?N5ZvFL3WG@AT<-c$mO3@u*^$nE z)O@5V@gbZkQTC*%L6uuz$SRyHF;gf@_AsX7hX{S%G!{yQ=(aXe#zF&QMw1P|oE@1( z5!&=8bu=Zmq)?kSVZLx!mp^;zo{|h5e+h2jOmsYc+JefOHw%9vI8mFk(sTmG-{r-u z7d28=R7i?ePkU9e^Z*yGD0Ql|aY4|8`q#9$$XgK;Z~FA{)SfD5yK|bEEVdyy3j%Hj5|fRQZpPMe+}GNxo421VEct98D4K9E+V z1H2SHSz^8j`$n~iHEAi)ay49A2>MZ}vv zVLpI-gSVTBEKj~)fdrw9`9~~(J9WHqx*@`bI1Z_By;#X0&5TeGgfAyC@`#wcDbTr;x0k*i&CuAT+DLL-5&zZcHL-&`!As*LJoMg3CBQJ?BHE&YH@B8W zc)NsnL8u%9n(N`gWzruYHc;cTUplvP3FOGJJ1Mha<(7-Lz5SRiyQrGxnpR+3Ix6@r zVBLDl;;v>cc6$Y1X{m~Xx7eUIorUb)o--C+WB@5!*@)J_mW72oVNL#nSTr+8%FT~A z#Q*)6%DJzoea6rxxP8u2uyRT&Y3-<_yi~!I;{{JdQ)$pqMW!7MWAp26qrfYl?NwGdmz!dgG2XY=;FylbXDOq~3xeObV@ZF<6b1 zC&~-e?>R*+a)H0E7~ZGwH?h7fr;hsOxlUAXym~iPJ@JCPb@&M>f&>OvIef>MSeT4Q z@W~k^i;C(SCt(ZFc>eJW|M?o9f;?5I`Ha@ulbx_yWX<&?D)st|^)%@vZZ?ya{4He4 zP#%aU3i~9z-<^pr`ULZn(IFMHxy9`i;VQ6tPfhhNVyyMZ7|;ifgB(!9{7p_<2zW=_fG@{F_$L)iUN zAtDX&L{K%lP+1oiNhT6WF0Dla@G$i<;*W)scj&u>e&SN?ru?gFXXurrTQNT9q7$K@ z^PhGU_|SrYWabb{F!;f=aqk6+g>p#AVBe5f-7u|zAW!(})sF|s%k04bOc*mMQpVnl zBbvbmNntGgx;Ui}F02qPBqO6zopQ6+48rly$aT%`Oic_G{j7oGt{5OyCw=O!v3E2m z7?f9B$pVsK66)p%Vt1w;d@>W(j1F1LZ<|$(G?g20(CrfC4AVLq!YF_$RCZ>eu2dwc zjt|b1ZZL?aa}If>WdKhCla0Jk!HQEr{uI0c?_+rXkexljBykmv83E0gniV9*B89G} z!P5Lu8^lx23`NX}q1$W3eRrRr6|lH(yYDY=*Xpd>3I4)V84wK?W}*3$PbH{@(l83v z#!>S>iG?soB}SDn-CGgHXMHant;s7SX`KRC9+qhVf(K@5l*3XxMuTPLnBJ~|@M_z?vj;oRjI6c}yhFrYiXyO*I;II2yd0rUe9Ua%^4C=JQ49z| zK;aCi4~euU4`4x2&7X2Sc;&U*N;d7?9AFPu=s1$Icx49Jg!0eo5{^J z6W1MDMrs3*3(Q+1riBKXuqS44F?0YddCZ#dA~PG4deK#NsyU!~MVV+%^0ApmjlnVW zKhW@38@7bQQTOBb`LG1A6A!=`lZ#A|rf_&WxF+VZ(G916jqa)cu@@BZ*E@rHOY!$R zq9}k~y@hLN;6A}G8lBlU^Lt1LFw6@y=@U2I-G2vx9o`S*^e=U|1jV;r>Lmg+AV(%N zdQ)hSN2x*PK}#CesmDe8sng+USqQ&Aa4*7UK{ntuz4-i7L7XWjZ5Q$r!F|Y_rj|OX z1!~1^K&=4BmKU-K-M^a2DC@*8)Q~t33bBDm1dM1tw*Of)L^TDUjb}8$Gdn=ky>n*! z>TihZdzE91uIrfZ@QWrVvPpr636r*FF|NraW}^!AiT{PIu6zuxu|PabHC;OBddl0?tVid-H1@J8EFj~@ZLe(v~2gu`&k zu%hW5cp|-C$L4;hHi&v?J3^N^aEc&@yJ?I9X!$1xYA{fscqOu!Zb_<@%!_r_wk=i{cQV^ShK0sG5Ej+kyd>Swi94w)TtK37CYBb+wQ$TwX%A``9=MkbzvC$!oku8*CpUNH1QzeVRw%_)`RBTE!}*aIa?|LM5mm?b-m9Z4P1UQrR2|m4DCjwGPB{{$@DLj{46bs z!9Yb>rXJH?7+V0p|C|Fvy&B8SpH zevJal!n-K#U?W3ZdY52sBTeeVNyg*JACgi2Lt|z{Lt3WwF+qcSH<>LfutnM4{~(?4 z*{zZ(_jz=grj>lOKz<2Idxjvp2>uEd-{V@KHK_oL^ z^Zd@RPI*R_BI0_WFYO-LCfoU;1x328=O7(=)TmDs)8Uja6O{8;^3bh9)ba$a3BWJj zL{B9M+QLfb*rt`8_m>wGq+rY^?#;aE4U%~B1WAQHVBXuYM~A-Ahu?>t_oy#H4Uvp% z{3zDI5;+2R{%e8`URz;YwJ&)`!G^%JN7(PoHr+ms;WIYL=f7&4$T8+! z4l(fL6k(^YAsV$qgd&Ijglb6cqYUZmM>3Qx+KIDa9FID6ldkn=3Yx#QA&W*CHxOH4 zF;Z3eT4Ov`)KjqrKT4odfcHZzVs^Zw)Mm(M#_R{mORNW3^2S~IE%HCYjqK*uRR(ut z&*&IaF^%-4RxBp0I>ft*EiQ{XKE$wYY`o{V6^q^JN)%qLUfOoi9mM0-9=~D z717rhtvB4Zp5otFh_zkL#OK-Y`0cT*X6&^1GGPp9WF53H1q z^Hh%TR0qbeq<>>9g)-lu`x@QL({l{>c+*~(!jzt%GFbN680y+W>D%edn$8}Zgn2W4 z-b%4B=BNJ;zQr?}=<{Z(pD~>)dkdysN)IxH+%7Sm>v)qsqntab&5Ysf%pcO{66$NS z4mA4GHTdkI^loY=%T_f!-<`;V{1V6|?wQFWJAymXuYalbt4~+F{>5*-ec--BhyLN8 zdq20PY{{$VFI{}9`i$B_Bi9yR z^pDn;ta!@zM*kD7Yac%J(i8t3``v-wzIXGF)?WKz&8At$f7*5Y-jnNFUj3^_gIAuO zoPBlXGe7_92VV94;COxS;EKQNy#D?d{$=zJ;eV9y(A#6*ynppK$$zBBuUtI2`h%<2 zkL;b(x4Z1|!xg6bNBcfkfB4bY7F}BS*!ypvIPv5kzc$?;lfLrr=P&;<()q#z^}mY! z>*4YQ9r??%zy%k1UKH>)m62@b$LZe|mD=Wc8|+FCJ6B`tY;69}Cu=mA0R0 zT7Lc9*M75l_siG$^=qH|@~V$2-Z*_=E`PK4Coix|(#NWvX#e(($F7gN+WzO>wWqr- z83%rG-??W_?7cAS%;ql*FDM)R#|Q4U{Pw#`rY(OZk^J>i)2A+lz0V+WwsRnzL#5w|B?CyY4&RSoX%EiFjDosMAN9bgK5FMm$+ ze;14xU60>ZH&MFIz=%+u!!F?itD4eCc5Yv z`(7v2+Xkb9(y@Q8q4ZX|hw`!auA=LubPwyd5r&tZt)tI)b}xurmN>5(?^9R+}UwY>-du4do-O5fL%Ffh(QtMrGMLh1`Dh|L!Wgu{L<%WvVuUd#Qjz3X zREsh&h;G| zMwrrgA}_lemNqor*xbrb4W&ml*NB!(?sR3fOgiV%fBbOevJuzEe&lJ;vdQ7L z!JOXL{~u6G0|XQR000O81r2+1MaAzCqSz1s0LU4aFpU8hmmW+42Y+o@3wRV&maghs z>8>=XP@QT>0s#s-KuD9MlkPwg3{;ZtG|v!2c*rA7rMr?e=|`-tP5=eL8D^r0i#xLV z>H0ud#|J9IfXWQcqR6a{;-L6k6fiz##&Iz2Eb6Sd=Tuds#o_x->fBT3{^vjEo_lX4 zG%szdq^QE9je&+G4S!lPPEkxDMNtZ?DxI<#3Tu|IW({khNL0-+P02)8v^%?7(L%9g zSdQ7lsC1(T6HRm_nI0w9XNe}QT}d@AYl50igfu0kg`!=dK2=F63C-3OjVZBcr!|_w zX<1!zETu4-npT(rSxsacfdG_%SdzU~MUBacZj0N^4n$*-uz##Zva}vGIl!cPqkZg1 zgsh0tneMg&XuqO%CQ}Mg68)nLu(TG9r7WqQEJan5YIZoL7!g>cl91yHvmqHx*p);o zttv<=H>Jsm2t*3$GqU8)q^j8x$$?NxQyH9PA5f#35<=Ax?oqGJ$(=EU2`3}4e95G~Xe>_XleM1gbRwdFs}iwBlOZU6 zRxFy*Qns#`tZ7QZ-kFBVh7dCk7BrnwLO?^+bW6&X0DlxY63^CGyeFB`c*IpUYN`zA zsvJ*^4*zhRqchb)sXir~(!Ik;SQfK=IAZ{n2gZ?{a55HCz_~k??6eyT7E$^P3{>V};kufePyA-6Nsw@^R z?(kGpmU+s?mSv`0;VtV*C%|ljT(RiZxDxMF)Ny5*g}e-4q%B}Fnk&Iv`D3%;Z8ZvH z?UbRby9>&G*q{Jxgdbn_qh9|09UKfz&|DOi^M53$JZ1bF1o;VwNb{DwDIzWActi&y z2~H_UxuhyVBE>cjB@ssI6r>>Ul8OnAi(L{CB&SzmNS35{XpbdN6=}&`O|ExW3sa<^ z2;LsjBJy;rCGUVlug__f0N3l3T*Z>$lZfBOi=N|T4mqAL4W);7(0#l}1^gD56qJO) zR)0#e^AO1+*#(kE3G_B8*eXbVViP4B-|E{#zV}}LPrP^cFb^O{pU%-w98O2E!{_ib z(nnDyuT`M$-N3jH5qrM-kYF#+XMu5l%q8IS=t4rDwB*?fCHh=WtH;4OF0p1jTv;tQ zhSY(GyvR2aifJUgGr%C1K0&gN(}_3*$$v*2K1Ss7v1I&u59fqrIcL5!{P!V_r_ZBH za2qC6&pBw8mx8;2EsptZd>~sSB#=GV_#wbhAT;aX@8u=pYZhI>;JrbAz-4hYi{7BU zDCi1!gZurMFM|k@UvT(sVqYx;tpgE(rT{cuz(SZKAPc*|<=H230?pNVEAlyeJ8b6g(=}#BDZy zcP&={p~qbc?dHY__DQhlK-(+XCqecnU4uL<9n?p;&*Bh3CXt zZh}w;A~p(?t5M*EZXUoU!D~0pY>Me5_zJ0T=*_CT`Bf41eR2_zAQ* z@R$xhPM2OkEMrFP0JoU@$al;?^(~f)G0q|$`<!EU@9B z@(xDtVmEfN8}=+$GR)1O*%E*m6e%0eI%>I6NVGIF(Naj%SaO8QC#=Xr^cnh&6CrXb zMt0-s&4ln?C}&yjHjgBE-d&#=xQJUK-#{*4w;~M*<$c zSrmdy5S&RK?&E6gs9Dfa^LUz_okQ``5NrcUfUC>H)nR|sVSm&NakVsC187t9YlE$Y zg&k*fhzojEf?lmds5t=D1W;EXlzk3h`4P*_)#I{SjLG%b2!D8P>!Z0aB>>xLUJfM! zT(Jvoj)gUxL>h?T@^KAtTd-5j7!r&L;2JX&&S#=0N(Rqqb;$0FElR`iq2a{~%F+a{*j9ETKrO;kV=>?^foMx=xE4sv z<%OrvAT%v=b1h9SX_+8-iB;s5&}<7(Sqk1QkWyxrw#%64olj zx(cwage+DY_a`jkuojM>WR4_3H{w?-(JytLoocqGe)KQ7bfB`+Je-O} zQL!j}!`#(0tAEiJ`$(Pym{&`tqGJ9jh!6teS`C*b9B_c0$pi#_A64rorBQXGCA3VE zX@4J=LX!Oeu^u1>YPgM;L(KXS#6Tv*MjzLY5Ssu(hDvR&;kFR?@P}Olm4f03`Ue2q z2<&tZw-wXciXDB;5O*!jUSpIHz85#cUcA;A4T}88T{seDi;vrcQMO9-tM4OAY=aK`3gOrnciREeqm(@TAX! zIs=3O$pQfnDjUQOVgbnk;Xr5*2S}dj;{+L#;WIYFXS}IT0GVj|Oa_^f;qST(lQI3L zn!i(_KF!ppgOr$a@mH{NQ%4RJ8M?>R(Z4E#gU`f44%Mc87RYQ=9AZ8P5!i~sNmc&;_=$$t^NtmeSui=!IMA>eDhVdKBY+vCaCoB3H4*&lAR z+7B*#;J#O9G%nZr?QHRDjgK~#P9D>C%dJnmdG|B7t$F3lq5ikB9mhk%-T&A-@4i3o zd*<+ifx55z@7~naf2RLmi#Od-{Nfv@0_r=9KKbBq)z5Q3Id#f@_N6a(Dt`x7e|4n2 z|CW6v)2e(HrgQWYmA^UBzi;)=7j8Yj;=^70@9Waho(d;QR>wV}Y{A4vyV2EMuLq4otU zT6UbU-{1Y`-HSii^J?Okr+@tGzC4^vK0k3s`NM6i@Am%ry?J*Qdbh7x_~OvTNnd~b z$!Y4#Z|8MP_$=?vTya(X3vce|8-C~Mx$l4X#rl2P!#B6TbK{MTmVNhoSI((FeCmY@ zb3Xj-JGG6n^8T*+$-7>wUQzdO{gJ~vHjH~ufyr=CMx>Z?@Y0@!c0&^UD)^XBS;}DGOxj(u`r(#d<~ z2ir>7^W)C#p_-nyzHh&F;*o3rK6a?N;K5M^x$Z5!>6`C-;Lz{Yo&QM<{M7#9=y%?I zYjF`deBrr+%L@-w?0<81v*iW(?PD9idF|lu7O&{K zPnCU5U(X9qTz1FF3$HgxzoUoWsCev=g}2Vm**^Z(qHmu%nSc1~i-OgU?Ar64>s{vk zZ!W%D^y_N-vu~H~-+9;0zpZH5bMu~+zj^4t+Zxwy-zeX7tSs`ZgwcfM45Y(SN|rUk%j|@3h@@9lY5 z^~ze;&J&;7w>SOA)L(jnuibt1Eb7d+Zyda1HgWozzHmfu|N1Yj9SFEG3t2sd(F9c_Eu7t?v9zJ9nXHZnTrTgYWm=v zHpA45%y+2cIbCS_=Yotfb!&sBsqifM308sbv42D;(w|D`g9J~_y-+&#M0(@XmS?xF zJX3L<_gsSQ6-^Mpt<%eU+7!wV|r2DxcRk z;(x@hXj&Fp(9*oLm&)#rYGPG+b$R9YFPFBrnlG(wmA*!gZ-dUeUr|#)yqC)ImCr7( zMjN9=<4TVeZYJ9qd6X>3W90(br6-X}rq!_0OO;ggB;!g&uM&%Gs!)}_WU4~{qj*J1 z4Ofi($4t4R#x_t5ohj@&>i+>yO9KQH01^NI2ml3pb49QLwdl$w0078AlOeJfe`IBF zb1q_KVQsuv?ROj1nb-J^t!u^C8F?lzBq2-~kr#x>7zu(Bvb~XP4-F&;94N30mOL8E zBC;e#QeL1Gx9Lk@cWGK6Y=O2ZEiHXrPX2;?-cKCP_MH95pTJ+RXMfKf$&ypEobHFs zIgV%U-1|Jg=Xrk5%k{z&PtC?+f2VF0=1-kD)orcCV)2h+vDl}HFj%gHOVyy=NwmYD z9k%?S)6waDD*EKNnk)6%WG(0}HCle9p|8}N)$wYu+^bDSI~uJT!_RiV7luKzixX>W zm1cFa*Q^HE98}feOC)GpuU)NlgAB>v7M<>O>y1vFcK8>oJL&vDW#? z&iQ&1{HsGyt3R1 zy9xC{LFYK99^zqeAqYExe_pNoUDgaPa;&y4HiM8f_Azu@OJvrzYS0M!wolT6deZC7 zm6pC5G}@E&v_wy1_0|MbuT{DxN$IRs!l0@UEmi9wA3T_~zx_BxYa2V4$r1Fh0v|Lh zYk~e!tKK95wAvA!%7tKQ@U~5bxV}s;y5Fh>@&3z>C}@#b`mofle{@$FUgIPeyAlXN zFI1`~E7hv{UKtOzO1saa&Z!YYO6piVqB;+dI2A7i?7u?9Rzl~a-mXh zY#%Hy)tkzYitk5cg=65(HSjGF`IVL@>dI~-Vitjm=rta$ErlyP!rTsgJL8-@Y@?)GHE_9k>VZg#(Gu~_=SRjh}U%yxh zn>@JG4*YIVWvi8rK_scSnoBTFZ4o47bDU z%T0fxPLy4auQV!MB*<82HQrf;gl3g>E;PqhYi#+IaAG;EH2qc9t<)6JlwW|5evkQ8 zoeYV0qP!M;3Pmd8{m4ykTAkfK%Cma2t9Ouf6TP({tOYwetF2xGn5iHP_e~=dmf0PJwQ<$UpPG9gr;YOXx6v8Kng-gg!6o*EztIv<>(&k+e34=?4-|G@V zwjrWw3yhsuu2d^wZK{o8)a@H0qTKDjZCh(qdmtiZ_jcm@$osy`W=%)4nTDTw(LFkw zaWmTo;2=4u1>v5|sC7pl7}q@VQFB9+zf5LX>78{1gC)R?shIGMkILcKSi~c`XKnS8Y2Ub8!qO{*z7vl*O*vvA7E(j0M;95;Gq2 z#-zz+Vy5QGglWozMbP7JDLv0Ty9Tv*H*<)+BredURXa2A3_+OcAxCXg>!Dh7EBqKkA&M&Zeuc3W8ZPVs_ zURdlnel|js7M7XVCmkyuTTcsvc_rzUf7wZq)VLwNHq&?#D5bZ~;>NYKm|+RaaATy! zTlcJ!!qoT-*{`!C@ZO+Di{Sxq7HpF?kq0zx8f{~qjRBwIdDlE^-hnYTPFn!UcL3$x zhQ@c2Yx?xkhx8)hie6$j(C;20nAq)L!Lj@ND(1wlqTx!yXL?!k7T+a{dX1=Yf7OV^ z?N52^R!rl&Pebb+(0Y%p@jU}iioFoOcPJjnr9+hZM&BHYL!Uu>F9F{N5eFgSPWX2K z+!_M6Skj7s?}z05qrmqA_d1GqAM%2GxuPNWFOJ-~54a2)))*d2zN!=q{re?iTmQPLeM zN>i6vt5%j4s>P@HpFH%HemI1ZIPajgdN&xic@iiN6C;lR(owwJn-)h{!ga+F^Momm z?7H>2NGI+W_azRBOk$6?KXDI!GUBMl)6)BtO&DTUU zec69>p=cHfH*45POOG3Xf5MaOtj%XM{s2DafFhd{51Qm0kIy(V@!-He>M4pzNfn1{ zQE|wjIKXU;=Z0Ij`z0hj2%8^*)Q9kVOqxpASr;!4WAPYp<}Y5X7X`UvEI zW6Fr+Gb-7t7!#|37kec(oy6}`s0^c~qL%0&|_(o14JFHVDXr$RY zo%$``We(acRd0juRvY#~z)9h0d_h%i4;^nDec09bqd8GDVd*z1eNGc$AA^+Lj2rw8 ze104bSitWyu6Uey`}i)oEY2o8@r1_DV&fgz2`Qe`_!DxhfB#gX6H?<(y5bz2I7cU* z!ihzVKa~^bVD-7|as1d*wk%|0#X{E5!9rOamOx}4ow87d<9ocD^Zud};e0wTo-0056wmD{+u|81 zzU7K%HU3PYe;D%%MWf@2=X|5QzTg%;-|OIDad7ZkgDr)keWg%z-##XudwZ&lKADr| zIr(-pgCrhh@`^4ymh>?m&B?`T+~s@FF&E09*eUO(mx4(Obc!a86ebOcB>1PicpfZy zev~E8gC)-oYm+n=X=urs#4)Rgc||UwtGM7;i|RNEe-)-Gos9!7LieoEE{{T9kdaAO zys@EC&rKO6G*t{-7*EjD*}TpuH}fPBF`j-PVDj5!4mJS|>u zi1*HfE0#3=g5uCwa7g1X(jnkjN{eOZCXy&GsyJL79JX<|N{3P1v!=>l8WK+^J2nP| zT3-0@e~Ulr7axA{2ifCO_V~!2$3;1DMpQIjCeJ{YAx`yWrPw$m0ZV1Z1y7czZGZ^? zCKv@1C@{JLV<{wq;ge3oe4o;!I+`ZI#tS~Lz~Hl31mqPUTp32r6)S{t1t=}Ciaf7t zeAOtIRM~mlE7Bx|DMvaC=<3r(DKFN5Zf%IJe_*<)|z=KUhDrdA&i_uWc9%ZZu)=6D5te zT+w7Fm}pYUw~_K4jkmEzWdLi4%FBpOsPUKcqFFqSAA8y^ft!oqrk@vG0PU)}%F?3C ze_)>jYB}-%2(C*Xjf(FzN}#81Mu03;5yHP)ny({cZj&~sFCFD}+;Z^KKiu1N7+FFm zsmbb5z3Gldzc=cc3-HX7_C^eP&~6b4&|Q8_pGK3BX#-g`xLfJ+F_ zWsP4NR`z`;EJypL9pdkV*vm386G3=|e}eMrNY;HDnxDp&*P=oAztZA6)COojHMZ|i zI=_xR&meB=u6Vt8q9|ThwD>Nx|4`%K6(uQ?I@K;GM@5QRxZ|<;lxo{|4%*BdN-*D_ z7e9b`Z;lRoZ>nKWbuN=S*IjvhzirR=ofOUEG+%LK;v4w>2Ch9fjQ$pVeoj>)f6~h4 zb1c#hEFRy#;f);Csk^99AT9@7Xn+&ngefnG?~lR6e;dDlc{8GAof}$wMuw~hpqnM9|~Ja z%p#zhilnJPIfF=AEF$5C|MrO=feu$S{-dFx775h*sd9DWn$J__Z5RUDI>azI@&@tl zO(N?*gM2HgHQ%@nc8Ji+51^-chBGp~ZkM?;sC1f2M4H4~@3C z)BbpR-OY>lu=zc;88s3!nTeUq#7t(&4(jEVw0JLzBxab)XxgM+b0sG`bJ9PD)+|{! zvi&;QeqE)(HQ4_XjbAflOtG-a%w#TkE=kFL5Z7Vck>_a6d`(u(WEHK2Iz?W*3l6_K z%JFw859Umj=H28Rc6~_Qf2fO_AY>gN((Zk2+~9ZXNk?78sLQ;yuJND9bB^q0AztRF z$Ki?FAw`uic(-=Xd8 zjZw1;2X#kA&FXN}d>5mptg>V^8Z7a;uYV2eq86_szVD2bpm!rEXVG0T9^*j%K1_Lk)NSvZQ04K%>W?PIwdk^NWMdG%LG^bsqy99J-1M&QXtaRo zu>=C1M^1VCUJP6}2UGp~gy{poXn=no=EVo*Bc}L(=D?pz@sTTjt?{2n4c9=&{J=Tk z;Olx~P5dI!6u(Tgf5eSMTl`AnzrYl#{*<%kWuYUD+s=m?|K(t{saEfcR&NYeTUZ^7 z)_#QvSdFn(6Y`iWk#Bm{^;N?rlZP zP^qD`y!eZC96zOkg(R{JGcEqY66cL|8ySGqjS6d27Ewu%f6&l)|4<`SH+7FwrKKA# zVe8{^x4`x5AuhYP;kjGz+Hc^s-@}7{Nc6-X6Bopv5*NjvHU68t_#ImJCq;YEw_Wkc z;)zA^2{?O6e4Mx}eyj11r`&erCk)d=)WE{%UZ~tFVWWNT*?{e_gmJIu3YYN6jH>-Je-bBE>PELPlwEic2K@1VcVl_r z#(t=kGmqWo%&WGsl|OYVwg*xA{`7Z+>f!%msPbPEsy}}PivJM+00960oHz+o6KA&> zCcqdN2uxf+#Nb3645N_*NkDW+2muj@f*^_tQWeF0w}@KX{^EwXK9{Pviwn4*g8RNc zQBiO~e=Y8Nt5&VWD(`-i0Y_iY={t{euDQA2{qFwVd!rP0loO0RRc^-ZaI*@>%?g}y zQ@E)(3lWSvDhU;wx8!gYC&hD=a`@sWx5^`wv_&?_f}3eO3JwS*;VZ|8Lbwxx6LP{s zgAqlMI7V!@bC*p~V#E#uxrYh{NHtOt+DIw2e@2QksmVRGEXMiJi`*$wkh?Hq zr<{|RRp5;dJkfzo9grXHoC=r9$!t!AEOt}n$Tc9K1aNl^g%P{tN{LxvM+18(#mV~O zf1E;>qmesm?o8H2}T^MQwK3O)vL@0m4`v) z3E5ckq--2{N;aN6jS+_pVYaxh7|8JuC``UlFe@q9)fB$3r zj)Pjm z8#r9S1yCG34HfRFk#I5##o_3Sf166GA!7{Wd3dSk`Ikz=OT{@I#nEuYHs_!$6*xTY zfg@1t6sXnsBccvR;_JV->A*f#t^wO9i^>8|k48;J!*LjK9@+NaVB61N+XVwz4sE3T zOB+#><*0=~&Y^qYQd{e|3GESzjVHZIPE>rOMCDbP_BeTmzec}qZFdbaD>Y`La zL~K=50aauROIsA*0OW!w#L#i&a$Afm!JmyK zkTDJ6V<4-6uUgDkP09KIlT^;SLRd5qfa>~e-9>S^2L{Xn@_h|?4~>v};Qvta0r>n7 ze18Zrcm%O{3^92EF?fm*4?tlg4%Z(75`~aQfOJO4V?d%2@&u4B2zd%f3__kEKS9lN z4fzbIc?N36k}p8be@jsJ8;JiM)Vu-}uR-k_jCkQJK}{nTRYf~M;?PL4QuZ414&?Tw z?X+v%X}7OVf3uy&*PSMy)9!VrzuQg|>rRuty8Fs@+N16?`K!~{w$q+p-R)I(`o?ye zQg_<>tGjI-)tb2pTM8F@cG$5euA?5gs0ldP6fc&_zU|wZ0wWbK0&13^Wy1`tqbZmrMAM!nVA`_; zD4fXJ%Oje)G3wHPihOaa`>OjZ8e;Y^uiwbV{h zQXk9=@u>^~IuAabICs=>12D!JQREmyqK*edC85O_7a%SGqN|>j(p zik3=f7${#-obMHkT*taVDKKtuaghdN-1RJlyJ$3FxYlG~8)}0AQjJ;(ZqzE{)oept z7KZkUe|Ww_!Q?cl_?g5i@L+&14T@EusUx+O2ziymwdjii%~ejTf=wGWcd;Rqj&(zZ zD4=;U4f)Ap+q9WRVA^8%Y;0f~QNaL87pn%+4XKucjm+v(2DJk$GUJ z7S?Gg+PrZcmmY4j)(h@%Xc`xe|)n|;c>D}(c@&B$h=q&jA_CtY$F{Q zW{U;kp{ysyc)%IPc%pEFBr4oZ$4^4wfwmrwyM^=MQ1HD%S+{WTV^y?qNcqRj2e~QCZpI5mxLJLK zf2|nf4G~@f5pJ$$Rk$tlG-RIUGoM+jX6ejYS(s7F#rv~#{7(2qgQ5y$Kr3v41#Rs$ zY)cfamguJn>jM#k=dVP|k^)u0h5vQj0%LsYYr-C_Qy9|%_~9oFe`5zqSlv1NR6!e; zxL|0`ggXEYV^liU88IoPJYH1fB5_koHIc&@-f3z5U?N(+YUvf9UqZ45RtYR(*}HZQ^0O9p3gr2e`sm` zxhPpyY+hU2p*@fxaT*=#kBn`H@*5z2JX$buC77r)uz^rJf&4>(Mh>eb8%QI&_>D=6 z$3oR%OyHN5j%g29SHaaFJ!`;iuIa%wJ*@=SOyFAlBvzkF=;PB{v3j?3XH6;>4^G;C z>q1*#L*$(8XSWnyzs!t!+8`Qje^0X2xFbgx_#Kd6p=kx$1eI)%*hM{91^qt=V+=aB zJ#sG?ELsB=h3MH1xHF13)IO9UAJ1SAKNwB95RU)3mfMW`&4na260kLjRegOLTH;@EZsbJl@5f1M0$M<}e0 zloE!#Fo&eJJ=E4EIEXxlE8*8kUB@xgQX2iZqAWD;D~H}>7|2@>)n(MP;kd0}O(56A z7c6>)CjJ>hw)|O>b|=423O(H@4x9-sz(B*8a2*?l%r=AM4Inwfz?#7pvv{Oyw9qo4 zO3e@xIgBV1|AQNTLGn=?e}V>!JH}W*!bb2p(!g3kf`yj=H{~YObk(dy3rm4Am2V0Z zUk3_XQ;0@29?iGVD5&$!7!!s5#{u?trZeiL{Qs_WfG9?z+U){r3c%19JsT^UU!mgp z6(jaFR>z`+l!=3jTR@7{z{Y`nadmZ$9(f#Xdw*IAeCdEOU4eHifADrQu<^hfFXoN^ zhBsdHAQIr-HZZQco=p<+lL+}qtj|xPEk9PR6PrkM)XKa}GC(?M`R|-b3-S|zDo$K? zu``Lb#C6A*BpsW8Q@BJDKW0>d1pcCij(|^G4rmyF6u=$;I3xr>0|7^LX90HsaFy8p z&^u#2zWC9Z{aS&n*(3~N&p{$2g#%b9O+VtbhQ%r82~?lj^wKa zy@>8F@YLN!asmLfBAI%D#{dv4;2i-veM3k5MgWtDCtPH+S-_(JItw-+x>dl@-EIO; zJV1hgCjs;j;mLw7g!dHiUH~Z~nF9s-Ab`Pw?;}Niq4c0@N%K<|RrMFPyTonv)4`sxJ#1c_qV zR*=^kppAh0e*&-qP7AW?boxrZyor_q?g`LTkm)LXM|AY|nh1CU;gg4mvBQv^Cxm!x zwb8L|65&r8PN90;BG9dZY>9o3+9~j{(H3tkQjmw?k37oYbA#Y-gpd!3-GUDj;bWa3 zr;`v{iCyy8x$v=4pz9>I{vom7u@}|vazPGypAI~|f5VL`_*g9HMLoh1@i+urdfjEv z81Tgb^tg7L@XPw7UZbfX#khMZN38j`+OF^ZbYYb7fKy9T!Jv=@CmO7ynw%Q+W}(O2 z1L~}xKGEj~2c7Ehp=D9_ljbj`JZ&{-_ny&d#*cXyE|)$J3AT(XNO-oW^ZF3W?eaMl zCeMiCe}?Vu?dnl^MpM}1a+UR5-uU9Fa`isRgcOnio?jXwL5O#y3@1jXiacsl4sM*EprBB-~SZ5>rjQ{PoI)M zPr0Yv8hFbodEV3U)(P92e$@3}ccEL`)XzV8eD z@;;|ugU^?h2VW`8?K{vdL_67_pP8@U#+Bx8A5xQM=iIPh!ZNF8lF!DGw;mLq)=g`W z*)^lA4E zf9rJa;en&`9;c{R-*VcwzSMh>Uv#W~Qq9jU=lxzPHfSqv9IfpVU6CsDcE5S8aArTn z)Hp}tqcSpn%*^NSKW0zQUUyL6FE=uH*y8Yw%P;tjIubgyG;480viFnDsd#zd4S^}6EuKBCE}Pb~lQ*qq9Ij72vt(jg$lFcF@Qiu#zR5c-zk9HJ$%P+#wyE`a z^LX`;&2Ow(=lvH%<`f2G;KddCf5bK)pIF{_CqEy2WK4q%bN8NBOy4<^JY=_GwQ+#? zN{36?p?x=A?ejR)b7QidWME14izCCFpJ9unV;2l)T4des>S3zdLY1dAk7_>Zew^hZ zn>XpZAlKA|J|Bt5g)2&@-Z5rnuB|@%_rCm;8}43qJEz9aE6>|)Z*e%=0>~*-??OCMKuy z88u^*uFCg}{~o;otYka?(8R3w_f9%h_3R|=vgNjZS&7epJ43csZp#Y25VdG>i;B&|hkaf> z=C$L`p^UC_ZQkpIcSrNCEqy!R8n?jnsiw?*k=r0LCwA1Crp5=Gf3tj^FF&j5eR?D@ zvbX8!v_3yRKL1+wx_ERq^635scLHB+UOBC7UB#Z|Cs@b+W3YnFiv$09N@heIIZ()* z+|pcg=$K3HLu>4kT)&L7Eh8T_(;nTnCi!97klDk1N>@copJhrVvz|}hwHCit(IZcF z&9hP4J^ATZer2Vee^MggkLoK2K33m8H7BCWj@;7kC-igZGc#$~{Plwe`~AFMbGV>+ zUaN~wqR%9b+n460PWDc?xVB_N&;G-+2X5^>bHJxV{W8~1y;^k4EB#gPSF4!Gv-e1t zm*cv>%E+&F>ppf?Wc$rCZXEBvB;_2|J+U&Y%+t|?ugkm8e-Hnu!al<{wf)(oBM)8t zYm;8}eRic_y6SyUM?*#b#y_o)tY~xduhbdT-FAMXuTJ(jIR5*zsd&%8&_ZzfRftYC%-I_gJA6>ZkclB=bwGH8$vnpFWxt{j@zFynrn(W_TXF{G$ zT2Yw%!-T?&exK;sv&XIdR2_N!;MU7=iqol6umhh?f2J3{y*1W(#Ij+YJwry0ITvwK zH6Y`D%9X*7l7m+j1#x{(di2WNdZ^bF?Tz=TwWT#R(xM$Zl3wW&P4A8+OW)lO?vq|( zF28@c*{nuozjq$Gr;C5^@+OkNek*48?wYc0t#4B0Gmo8`*^Sk;c5m-z$G{-!GCUn- zklj>3e**!w3G=J_H?vhdPYVQ|m~ZB(c!sVQaN{>Kbd7+Az~?Ff5Bg@F{vzP)H}iCv zfTNvwiJ-UnH?wt-fV;!z0)Zdx6?uY8G?M2EJHcmwIl>(h>f=sjv4iNO_3OM3H z`(>iAbD*4cmwef0y9)StfH;9?I6$l*2kq0*0^V1^kxaC6qnt?v8Ff1l$}cDIg#V8K z00030|4ot0O2a@Hh5ugcMUe_ZaU+N#sEedYQ;VobTNMIoMXTT{B%P_5TxQBlQs~;( ze{kg+xbrD|1Yg0uD`#R97ia!6-#Lqa7yvkyDoFLDj--lGCsKzwS9v$*itXoKF)R0N zDI@QzxtaTBFG;wZJ5P}!ei1=wCTMn{EAaC0_Wt_y?3LDGw{cnc41$48Co{XT_6Sv^KtiYzI*w9=5!6$P@Ptr|{(HY9 zdy*{n@9c|HXJlWVh0rithGiJWfn%Dc)wX_1wfn)XV_%&QL#RKJWW&@tn(-G7N7s2+ zxiPGM%L=l+XP67QcnAxY*3~+Sl@k(Uzv2N@KinmE)or?K?pF2)!BRfPA?)csG>fqw z;wYNy98Z~$r{{8A@bQmx5S|JR^C(cC=7~fO_y$l*0|XQR000O81$%Qv{Bn6wtCK6M zFMoY(8)qGU@7{Cn?AYz=JE!hva*Z?S$P%9&w_{t?z4@w_DNUmF1MxB3i+#2)KHr(U zbJ|S^RT|r}30Q^JX;LQ^qDn((szOo6BtYv5|FCN6hXk60fPg@JNYMU4QIJ65_nt3V z1f2BPzx#bY@AJO)-O<^natKA2$0kRQjeq)16Cpl<5IQ8g#=P#9DyCj4JI$798Lk)_ zVii6tb6z@QT9vZyR+LWeyzUvB$=Ge%k+NmzHt9rJQK{{lme*Y|EW8eUc57_?0?u* zvxaMiFSK;OEo`6$ zx|>d=4Vp1Mx3h{Z_xpza&-m=fP_=CjX+z1OQ15I$^uS>l2T2gZyTK%=b1eb-0o@$o z<$je@x2c5U6fGhw5=603L$pXnXi*)fMJ+>%^f(n1ipw;Gr4~^+jh9`OqknM(d8$H- zbC9E9K}+)YdS7fAcb5CIIG%sIKxz?b8uWN?#m|~=<`+WZe?i?+i#$Dtpcb*no zfw9cXcL=viR5FQqXn79Ls2X>Tmt#cZZpAe%DNsB>6-+ZJk&tqJR6Qhb#BuY zODUAk0MTSh)J~H*GFe>4sehv?F2>Tu0m>_)+Il1*<3|DN7)Ht%CK)Xu1G$8prYnDl z?M@`QHJVZA0H(@-peE8Zm7%dcBXj^X5JY;8-Gd$L9Z9B=1607wS6o%VN(!QSo?@V- zGFm`cB+JkqKZaTRSpWT4CPBD+K${I%(vYaG2F#gc5}_+t*1$+YgnwpWJFayU8KJS= z(DWJ*`2?2eEyPLklbDkf5LrrrN)KwB)DtHfFoe1qVP!(`7Gn&eiDZrKEC`D;z*5rg z5l-3xWe=&Gl#UaXrNxGTvP1?GJ8?{dtR;d(A|)jv$y}XNaG+h#Ze!cFJ;}tj?POw2 zY~zh4wr$&)IGNbCZQK8M&ei!()vnr?yK3KbRrj-cvDlT$`7bk-;RbYUstvJ;flU(~ z1=xA7R`iw2&?>A!15-gsiC*Yxg+EpE$m?8banS67>*`T`>rj1(?FS4C6t#`k*eJnI z)2q_LWqv!bQo+M1U>WjbVE=ZMhxM@1-J|_u>ub>!*ieLqWnyedP?Z-;TLRW2nz!=B zba{-Ezu)a0_{ma|!T55NwhHbKgQhf$3hDwX<_{4T@@4CP34jPpD2Djq<^}sLSxe$< z!>(YyJ%c^{VS58POA&bQ7qD3pI|!Vzwm#-TWoW2xSK#7@7<0ZkF5N0ZEANrUI-v={YvQUIo>?{L-cw@ zTKU{d?{V?0N%TE5xZIs-r-t0){P(Tdq~pKlNRQ|1zZm@Z)|}2Qbot28XQ1u54d{AV zYI%WwM(gplyj3`S4d+aa>GXYnSjn|JrC7UvT3nPB0Y1;(6+2Zs?<$`wCetl?zAu`) z=s6`7ea)Zl4#S!Kc4em?y@I`cj4!8<6|=RyuQR`Kx<6R;zn?C#yW5F+yz3v{4hA+N ziTysW>jiuKoTg(lYP-$962EssT;Bt(KUq^h*WYB942ZubnXA|YwL2eXC4tTZ;BDk1 z)1d7g$aV+bPU3wdiG7}q9QHqK0MFgz$8B4KtpdNaaB>?e zQSjmJAXAwCfv>yR#BvwG$v`NMB`n)QvGf~ti2@$?lhNy%Gg@QC#T3~;q`3HuS z*wODJm%zOUytk{1h`pV3Kfn5IJ^OAwXFpCZt2js^jVHH#u)9f2q+1sv5C zZhx#a5W&e9%Cu~kwU^}=7#eFh9fI0@^+ar2>2oUa6(;J`s_%7{5%KCW`r56ylH6@$ z@7lFn$u=QF7JET*)UQC>@|V>#TjjdVE!7;#bS*2d_yiTB-Go|2T*vLo?TBymJN{8* zH_hC*@^2OpLW2R-XXfJbmthr_*;Q-yb0!N{Top6>?fX1NSomJ-=4COumf9Km%jHC*e|I@G6{}s9IO0%OleyYOa z7Ee-+5A^~lDIV`Xx$hVjlCQnP$cd@N%4HK05OgVFV$)fwXlRj{oy=JMrCD2x!i?fU z8tLeM%M29{q7h?L+FhxpSqp}>sRq0>vbJXNL>JyJSX$!4N);9D;I>UfLQP#MSHt0?KsCs^U! zytVg^WRBO&H|1$zg?AouNmL+C`XZ~H6j=UK5~2P~AbN3ws)s(HgrnF4I*ug%k?c{C z(!Hp^?Zc%g+MAuf)F&$7{u7e%2n%YhQ8|fpYNEo1+$Z%3gY}#JNoH5cNbLQk<%g>< zz*G(_gj^_0sN@m$es2D1N#uoW>)4H1k*Q};7!6l((WKRuGnDh7r7=+D5)Qmn2}6gH zBA5=-wf{jud3N3Vr(`ep*ks$X?GgLkaP7@ zC5$92LU3!66q{L=dQA6k&lh z&i~AvkcQ1*3(0Hv#|8FIH8S~Y!QMdmNML&XQAiMAK$bv_1-9W&?oDP5hCwNM{c9~g zpuXK3P+3lfqqYyuLqRh?UyypU{U#}Pm;|?wR(6*+5OHoFTlf48s4w5mJ12e)?q>te zhaYUkAFlWxIIsHNA9$$Wt9?lpTRP)gnA_uZJu#w!FKe8;+-LVeezq}{Q_;jTo7&&a zjvwc_-gkqaJ`$Z54B4>2+vIdN>w76fG08Zfdei=*+u@ctk=Fw7YU%vsjSS8Gdp+3A z-1$5^G$<(OK&!IY@}<@Ny?1_RpVM8s>|6b@X?~DiTgAukGo-cpSr|$D;PcWu;Oz2> zZI7V#eZP>t`CgT%dGq9+Qe*JF&nr`MT|IBX?CT)!Lj3u>wXX;{-R*Un{0lf8GU?&{ zenEFUAk5Hqruu%|?HDyM_&Rx$wXV+%#yTV%f6nBF8lS5twB&xzn<0S=B;bj>7w(s zn~hh>AlhSaynm8}jrYb8Wx(CeROVqW|{=Giz=6rX(XneObGc@d#e*B2&J6LTAbD)rOJrDtkZU}$3IT|!;gqF z1nO#8d420v#C%O;s8UBwhv-dAOxkR&QlvcoIux<;v$>XqlOb*<4+9dY|l zqcgj5OtlACJl#fn+=iXufo-#0h2A*B9bS=Bp%pj%A3 z5Rk+cKD1OZc`!5}SnK^i3@Q8nU`XfnHEeTXouHIx5GZ6OvT1O<^spsO@ z+k#>o*3NlK%l)crd;0k^jewt>6E%F}IhDLIRFUB7rzSua|D#i~yJ2msV4cbH(yL9) zuBOS#wU!VOTX(AUytr6r{d&iK^WLOy)2XputujWVOQC#=q2m$7c=oYN0K(cGUsr=O z!Ct$Z5vdzF^*m=C2fnN$TW6Cq#VT6H;WyNJ6VlWWo&M&z2jw&)GxD;v-*7Aw`}$dR zmx>iLB0DfWz(Z6gipr%%$F575`+n(E4%Sjw?bcXK^Jm@4bH`<=+MRS*rBS~;cBgy2 z49Tp~t4-bNym?4P)!^x@T3ILgH@lj3jQ?@O`YQhj?N6yuNU|SMsIe-g+D=9mH50*z zN8Z_tTN05`Zfnihn9&OK%J;HX!~EwQRz-mV_voA{R;fx4K7>-R+( zO-nS(9hi#Dob>uhHUAQ_g6;p(=X{AYE``nEDyTP$U& zhgblea~E->BKOfP8*ZgbGiB$g@qief8=}=T>S%b=_h4u%a7F4*0l(knY|C$=t^naJnVm5 zFrm3#s}Bk#;UrC~i&ZP1KS;Zr>@|kCGexkFiF+UevaRZlw21#QQJUA>bjQh+>h`la z<0y+AR&5``7mQsAbGXi7@}ZpSPQ7`A&$8DdEaJ4myiD`9g+m_8cf>hIrfvsy`fY() zm8kwi`hNY6O`BMTcFa~i!e?G7h*re&xvLim1cjC6hGRAlo2)Qc3V*h#X6u07P$KcFh~#eM?5ytvaMdx(IJZE2O7xjjsV5XEPYDLSYdfRT&uNIrGN2@+Kn3?4 z6B2`00Wwy_X~n*(X)Jtaq*#fWk#x5c&Ym-q(3lIZgMs5y$ATRW?AHwcVjfvl??a$7ZJM+!^sb~BAySv4k6wu1o-D36HUt6po;@Hu21w_Cggqcby z>*#c!l^NFz3_TM^A%?~ap{f>*O?HbQQI)#1sIsZP^JpG6&^+rneV)CT!YPAo9l05U zEjk#u>?_W*wXM<3UZwyA%b}ucNh&_%1q(|Y#pE@9wTkCsUY$rf)t=NQAQ?~FN_v(x zUAf4JTYQTZ<6?P*F4Zj)s8j8TjguMh?rQqI{wyr z%`0Y_3Lm?~pwUSrDu9Ex&e{F&Yl#JXcNr2r+FY951=CTN6XhxUsd*iy(C}z2K~>d2 z(532&w>}*rb96&#U3A!PakIfn(m~+OSBZoq7YeBe5^WG^dJ1he z`Eq)^nBHl<;B<88WBl3d4os@*P&P#}%0J(x*N#CTv_- z-=XBa1CH_pF+hg+>977%aUM#t@*5`BEI&n7s-*cdH0280OYn+uM_ zwK{Z?MwVjKQOlotWpsi4Qa0cT zd%~Urn<@>J2_?&A0ws&Kwks`B^(Bc{M_>eEXg;YRep!36xXY<+rMTUzqR9}wE`l~d zb(PuT`O}PhVwUtjswA_HI<_CJH4{%Uid>4Y8(?RS_J!!Q)xmSu@>1U5p=2ntq-B33 zp}R8mlDU9Gbq#$)VHA-AyJaHT!XUAOHcmlt2f4jMaim9SMIt-m5Z;gpTle=sMrnl( z;|=(mi{cJCi-hV9;y6sQH$WA}3*$(QLamCoZ5EjwMq2Nj`SZ@WRT+iT+NV`pO>C$O z3CL^?!3ncVIXlLR+tgfmsjfFh%}E*xXv$hL6OhIoGB4oZY&2$OLy|J7IMyechJ*OA z%@_^lt#mQ#0cR<+brfDU8qh9@l~igz(spaw36fPrtIP)1$R=u|p_$XY8n|8Th|_(@ z`6oS}cT@}x##@x#0*pO!MMiW1_jg)2DX__yEkilyEA@wToO~-J-i)g!`bs)_Tfk3Q zrcz|pUlxP+)pL{wh5~|2npIFtWyAPUEbhL7FSyxgB(Ry8EMEOWWsgp$)R%DWSJxtMY=)*mvOcTbjTMy91{!bD5YhPl_o=-p)2t*^Mt4sRV@B&M+MK97_Y z$4g^_)u-{ z*}DmE$j3X(9SG%eA!$?fSs^idMnL~Wrq}-DJ0UO(LrYXw4;zDQW!}{XeOBCm-U{0D zcU_S=ZrtnsX(ZD2Zd^ivJI^!W!Hh|l4LRzXM*^2Z*GgxA1ZGM^qWVMi2k^&}QKA`P zXmkD;>HZQCzUbCeX~F~V7Fi9R(jj*6@93=jD`_u3_~TVDs_H4X&jQLigyKJ$`fNZokxZn7KekZaG7+SGsi$Ihd{F<%#mbr* zL$yKVsV>O(Gqc43q0Ub7!8O?&gbi3qz4E}P^|*0r6*AN zB&*a&c;5$pc+eIGiV_ALntbk?%yIRjp;byYfUv`5Zt*E7zt9|1h}H}ni;rL`R_q4% zWt4HAD*f{=8Gx1YDA=~2*LM5|XAUPuB1?9%)C(mz4ONZahrsx>ucIV#(OZLFL06)F z%S>`gStPO}PbXgfLlfvp!eB)^yoS)eq$s-I9aY`0cuDA?g(c@>z+@qN!6+FZILO3L zvPZgOq=w7@TfN`XQ9;z8z$x-!q0jL*R5cdxgo*W~z%`OgprxljO27R`;)+J$w=G5~ z;1T%=E3sHhqV=mZ0Eu}60YAh?X~MGeMTC`}$YMzQqdCf%Wed2ZS+SUXRrZ@=ODcX} z=o&t^O!Gy8851mx-JSlZFB-}v(dz-#)7*s>n^v{_%P196suaA}K2|N4JWOkmSTq|z zqAg9~S7-k~R~oK>xtQY%mU${H4rG}B)d=>9b77UVjdGgibquw-5TGfFcvSO``o#lb znL{<36Yv{5js=(wQST3gWjLg5GLHt(hne?ZxY?3Lyfr)%hSY|ymtXF=ydf=$&!>rI zUcjchV~;L}up)}~J0tio-cKn8_4QqEFaV`y<6@(X&gZN%L>`_WW}k#f{>Z{zp`<7Z zKx7xM_db4V(mSpMKV(~MVF$w^B3l5zRZhmAmVbj43UG=D}_k>wF1 z=H7yGk?Ybf3y{a&m+_08Q6hQiQfdgF^#w*cR{@H3`WJ@sP!B#e) zCT{CjRa+rHT4ydf^1RSP@TB#iB3 zd&D4T?$!Q2-yTNv*|^t)--E`2s+MH`3i_(jJCtU($WZ&|ngKxsxq>WwM=P}yNC&St z>&_=)%OD2)7I1he){(gN-Txz} z7|a=tg_CM?OEvf}bG_ygQ;^+{+0VQxtTmeUHs%)kw&qsyHfIRniTM!?7G4(#JqUfl z(l`fs3o0M5;^iarOIXGLSMHSl0QjJH_{tqH+s_^R4_QR=IwMFvyJ<`l#(BGFa}w%Q zq$)F3kvyTSHgx&ao@jb++&!he#Y*cePTK$}f;KtV&AjO05Bn(yiHNi-y$dJJXZ>e+ z3%B!$Jwo6|*z47HM-qTc1WBwE|H@79#0(RF9P}Mr05B`){-Rvg zQ#ZgMCQTMxdXdCnD0zY46igum=V0WwZs-Vs!_ zSy@&A$9)|$NOSnb8Fc?I$+{QK55I`%B0%#xWdkIc4_xVw^n@49xR+(TZ%kkC z(+b*SZ`HS2?*c%g;2TN^_zv-Su@(N9Vx>fcc-2lRP0>-x^`O2DFTLHiIEk`nIIG{RaELT$HOO;;-muY#`@-!%opP*V9Ed z#IcQ3uBN@r)0=LTH*wiWT2ID)1MVI<5oaBT?Z^y&ea?J{As85uEy_o|LrXTURu;RP zWBoeb@Sd~(FQ=S4eflbJ>^}AE?sxp(dDR)S3ve@FWv4DB|cPwOn1>)3={A(U)?7^!C(8Id_qp`Fpq3WSoY+%3a?$4aFxj7 z#&{PY=ugay0TFxAK2Hr?IjIgm?yN2>-Jb%qgcWQ4qR4q%?TkvUrWnk%qCs?>K2g4T)Yj z#iSLJd^N99e*)c-zy?`Bs=Hj{a0hp4 zPUNh*4&YYIlYW#Oo4ZL8YK{-0cS`Xh2I2CA6!hmRHVbeoAZB^VIaqr5d4f zHZ^Q%^k3R^5W>MS1@=|4Xnk1KbtPtVRR0aT_=cox!REf8{&hBgWN z8zR&ArreN|IDr-NFOOsA3wBg$?+c;z#m|^heda$}rJCB#xD-zV38&D?PoaPMPmsb& zX>+C(s7`A@B&V>B^vbO8Df|*L~Ks4Da2s4?;KSE+MVOR@o-$8XR-M z4KBHQ_UvWsdHaLVRHsBt&!j9HfqJ#GXVFVx9UBKy|l*G$JWxb-* zz{18ysqSF%)4o23BI37u=UG_u)G#fIyM;5H?vfoDoK0Fu(>o>_5QD6?5SjYmSP%w;DXV zEMgsQcqFn7F~k|~(60}ajrs{R)_gleKgJCV)cN0pl0@Qm23uDJ6bGj|M!WvWB1Zjo zio-0j-?3fkh~-{i75ad!hP+=@!q5HThsbR|a*sE31b;WkV=&CVw9;xCrcM-O-h^Y}%{f4BP_+zOmLL*%4N$G%6LAO8fVH(Go5(W=qUcM#8YmNbp7 zJEfi=)xbL|TEV9h6}<>js~x*<36H+`P8~PiqS5>ihy1)!9xN@40_iy;*y#|)9 zldX0!CSGaB1aqRzm-wb>huN}4pFDCls=TgyZYOd{`QKK%L!nh${@uIJ?a$*rx>fV? zAVtdg>!1NnMzSBrzRh+vavRY(cW+~fjXh^dX{RmE8Hv+NCd;ic?W}sQ6g&rSqrBt4 zs0rr|eEyCv?i*F^JQ5j{cr|0FzO>go{&-~kX8Y|hgt&a92*&1}`oPbpx&JK0Ow~i< zp1bpNpO7?iI!Z<=b-7)yT3*L%sw#WC;c5DZ1u-Mw5x3-i*Mg4jr<2=q8fNY{xZkKL zYR!wih~3JlP+m=7)iZNJaM^L4%U}{}FF}R9ZgvJ$d*6dTA)iyk^RbxJ8OQ4xnW(aa zm0GLilvmS!_~fYgCjVfi(Lu5;@Epm0$wv3{*Z@!kH zIxz%Le=Z;OYg@ZVxVoL+k?5)cwlO3)9iNx?QN;d0R%SJ8WkC(F$ zy_J=}$GfoabLq||0|-r1!>!pKolBDN7V?pS?@2X3XGdROy~_59v4@~+WVJF`){Lz4 zzs`BZBaROWwPhL4ZU@N(ubvx2h3lPIkcSr?M8}STb6NbR7kVA{<$4KsXq+Uucv9t|4UULlQDX7{}BE-}r@bod1Trg=sd}pA6$z3P4#LqM$ue~k6X#6HIT@MUC#J)MXv;u1bjML`G z%y5e5%NBO=+Y0j+tk2Wt*!m7q$WlyGQ=RNRu0|aX3zD3CrC$u{S6(>rnjF_<-&w7p z7T52y?z8j5j1f$V^y-Y8TTdR$ttrSHMa31Z$iG{ESiG+2ZE2btxZFf4 z0knYT<+=Pu9!(G2?|K_i-(H`kHbL=O}PEaq3fDy zzm2NF)B0;4Pb;N8)@2R5RM8IvXL7g{bb3|~x zzF=~OSf^$;Wbn_f=Me80%GLHZ)RT9wANbYk8+>iaR#X~r>3u`J`?lN-eZlE}`Ce@5 z{N3C={p8is)5!nH^P1ANG`}h^^>hVxw!Ex=zG>5A?)`H;bv0+?$*d)`!IwMLnD;@) zz})**dag7P`RsPf1&DgEB$0$IMZ?~^7Gi}35bw|ro!N~Jw50{YQWKGlJLzpH4LZH; zcJZzYww3IT18v9@I`3(U_!>}J;-go#LhPPErq^{x7IuD5^|V* zUzlrIEqp?lLjJFibLsQ{xCrdi+txhl*FC~YHY3eD$*Ap_mS6k!HA_XQA82Sd?xDkI z6ClL-aL2*ixAjY@TIu5kT)!W}&14k(w(m3ITgmh+rVwxlcnOF2s-LWyT4OVlQE;*G zW8r>eB~D<;0$G$40zasz7Vbdte(#~7|08Eq&RXMRWW{3lF~zVT#>L0V#<4>>CO|Xk zpPVN*ncQWFLFq;M8S`ZVjf{`NYJPLmOH&h`THjIb{e0C;s;{q4Wv?0(&&5Q-WqK19 zKwiidiW?8>#XL=vvuS`BL%{@Nq-9Ilnq@aurUz)o zpED}I(}{Nd;R#EGXp8%k-C!JO*;q(z^EFFSJ7yk4=D%j&yBD2N+7{UCpE|amKW|^Z zycDfJeIGqyME2}eQ=5qYS z<=1pDL!53hnzD3TOwcSe)K#ZkrzIE6o+Ph*<5 z2xpzdH#3(?n~kTiQuC-~8?bUYRHWxwY-*L8V3oGB;+c*fiu^J!8>0%+okrkP6*1-M zy~PZ2=yc-Ji>l2S$PAwNt1!*I_=uhg_G9KJc5WiNWv2+|G@w02<(Hp4dob# z5{+|^V#TUwfys=Vd#&j>9aFCPi;Y;OWJ5Hro>t{Z=zN_xss6{Y@#g|_^Ayr4*bNd@ z?<7#F%m+;v110NNW9O|J2^_P|Z&^95^z&YJ;ZJK6a^qOF^_Xty?WuUTzvJqC*0PsM zBhn)*(IdpvJcO(>b;gdWE-u?U(n1@U|H!Y|a$6=eJP*w@3-FNV|DyA#t7IcMgL`ez zm|I{Q%rgJd8)cCS5#0cc$TPn?Sm8!!NUvQK8g2Qn8_fAyepDV;x6e?|!OhJN@<}!5 zT8kbgf}EL>6SOnw!rq#)*z_-)*KU5}>^6WU%`cUZI3>_dCqZ?kx7CQi+PVIQ)xdtd zva**n{{dWvpczB$${FH(fn^bsslM?donvXaR-IH>rf7SnlNkdT2Loghe$4%b1D?3E z^kbKqGhj#s2tl8qoiyt2l|lOApvn{Z`>axil7XV^1p{=PBv>6*`l+k+Ph4;E-oLDd z=*W%K#R~6myhqgcE3sDpu__PLfIO+o>K~$b?1_;E#6w8biRnueA-)jPgtYl#~cby|SAdSte-VHJ-0mv`hx21KV${_EkwOMhiswDL?rUptm z622#=?f2bp-#6@?*SeoQ-RyxU))*DO@*>fSSQJm}Rbx`d(>dGp6c=R3f?5WV+J54gLpb8~wYa-5q2PWDkX^vus9QePYoHdR(7?G6r4s?$ zN-L4)Ep3ixfYGbYMQ9pWoS6dZ8m1AZKZeHQE@b{xtFYGSoh!w&a9sEfjgv_}Xjb(= zx0_l}-)BX3*x@;7zl0&D8m0rgWJRzto-8aud|fTk9p+#F4z-{h95&>V)j{V=c528R z{Bh^VfjDsRxfjP!-tEPXlNvY|N1)`}5U~de%l!eB94T8iU{L)A+jFo)-+hERjV?>0 zL`eCl$|Ukh!hBl>&tpUcHx{`^T3T?6BXSF|Ak9y@yDW^sN+7!U<0ODX<>WUAHB4)X zCQL!K)bDcM;6IBEbrPsc4IgKq15)5puubrcb7X*m6c^&*Q8CzfEv;=hC}hhUg;MYa zd~{^s#9efkh^iqxgLYD~2MFFQCh6F$aDm{^yKs**Upc}kdIfU@)E;_GI4O5A)k3(( zS^FHTz8E_s&EOhkE4WmQ&p+4u0u;m7BWrf>U{qKi1U?v@qK~|0nkw+YM~0rM`Ys}a zWp@C++>c%jNF2GdRW~(%#dL2M@5Yk;-bs1ou~Y=nlr{9DY+8dt*(j;=@lGyKb1uTA zfQW|Usg(K*3R7bICy`bMsaEK}9?fElyP8f3`{YOhqdGk*B58@@nO+M$Oj@^&+;;hw zC&;i&SS>G{tVkP0$R7(jb;d@2qSkE++Ydneu}4ifD`mccyvyzEO@I^zlMYpU0I2{L zXSpY?A40;@x5@&MuNEH!BgauNR~OHup(w(c9bxzg3HwpSc&h2_j{{z0PCXW@`*n82 zQ7YNS1ud;%03Hyp!pXBhdc|*azVE zgk}oeKtU=hgP(~N#X$WPBq84Y=ddssTzW(721NBpSKE(`1-pm7c?K`ka3zhn41e<45Q3P33VnR%!D1cGXo_Q!KOGyYC+EJV0Qp7j) zdhL*^Frf*OrX00Hr_)n32K6UU^_6#6Byc8-tn74|_}2&Xgq$$$p9%Cjgbt`U(BItq z9H5MEBjlOyY^}$F2T`2+Tcdc^)q(ZJX2Pz^#S!!NJ41T3+mIC2jbKRv&}~MZH1aPu z^PxsYuZHR>MB3^}Jy~BNPJI;tl(W8M*QGupwRtXtRmo;dNhQAB{oA!r| zp=`#m@?|fCsDv1eA_2Rcszmv_fMiK!RE&~frje^^CV!pR_z9z05Au>mMF-(h3@bv~ z7z!`A*@rNmEP_VXkjQ_nH^tx0QQZM&SHZ62iKgN+7}TTRYU#DEX28SGWNPl|-o5?S_$}wR zn3EXQRrw_I-?k7H$0dpuh?w_c&kAG9mhw-^_*Lig%sp_^L279@;{QS4xceM^)dMk>Y`+9wy zYZl>c;jZs@R3EPz0Bm*J%MYG#x_t->d}kxmX?|&1?=GZHb9~W!zn&o@e_dYoWl#G? zmv6Ie1B`;cr$X!3xq6>Fyw4k%JH+do59f*B4|@mK66?TRO2>QipP}kf`x>6llko0o zAIJIjjs9ytw|&hD1wF5+&eP+!O>etbcVGAV<68U8oBdL69c<#2=9dHV@r6R@s4q=@ z_T0xMgU)f_YWvI0M|QfW>HECU-Injo&enS!kVGad_c@y4yy{_pnceRZxIYF*HD8GAlePquc#*Wa&|beI})zqIdOhtIbz z(}2>$F6bkX#6K6K|51=Rg1>I>{$TUF1J`pRmjSB8J~eum9iJEI+k{@GN6?BM(|Yj$ zXP47n<+Uw)3DzS0I1~G|@9`M_H&@S--0?f?vnP%exwzNOv)pMF{dN4EpWJ*7MFU>3 zWX);wodDzYq;Jwt9Tja?^ctC4`?y!;guY<%X`WN?wt_{(NMfT4`QR#uHDmE;dS}E~ z+ZdVphRoQagk^_W&kza*B1!Mj)BKO33Ff&V`K3U4QUiX1}hfJOSDo(vZrQmA*Lo}S*Ul#6s&E6jsIdtpdb?qB-dg~3C!5(YJ0<T-S(5b*j!>vZ3LaO@ir-k zb6hIxQb{9B&XleHO=>jEd`NXQIKv6~oR+Z|*|SM*i?zu^g_B^_@mj7ZN4n_7&J*i> zmv2H#W18#iY=(*dSOzD<%1U99r9om)P7d(Ha2tyvSut5k87Z098FD$rS&g&ivMy4e z4;Mb#?5AD)*^D5#Us+uJ*DC73lRuKsfnC%~g58Z3C&;2BKU0UG^3YRtl0uyvw-q!A zgHFy&5=F{#ck(Jhm2eEtoZDZ1EkJ3>p~X}~v`!}j2XaD(FI(PfN=!zT&4)DL(hRVm zmA9frY*|_upb6SfQ)W!2@6#c$M&cje?cR$=u(aZt{_USBEhNE-5v^zHHsiGbDUo3xFbrfPjlQnV!QThgh3m*#s9nS)cmGC6t2cq6MTi$W@>n|D3 zY%N7CM_XzQnS4Mc@@F1cKb=?ZvQkoAc^hdm3?E~^s$H3t2TU!k zS`*37i4>^oehny_O(kw|?rkWAd$nDAK`98aJkQWtiK4+f;aNtNcE#b z+OpxvdpJ1ykh}+}F%YNmfk}$n_tC&IFx_{tPD<@ zIOI?1@LMo)=fyg$S`uj^h{0Hl(QC5}W1ugLQD3mrn4L-_451C09LuWJMhz@4!BqAH zrAo%EVvyXXk~SvO&YQaPU~{JN%5TDJ6o@jCgN2;^jRhKoan^Y|FX+564bI*eh5V3jZgPwRV z1{DRf6pd=7jp$fun-yi||J*rmx+aoN&+$)vb!<}l;_Q6+Z80|R5n%(&s$FCe@`E_U z%rkAxVL4hsCqYQLr5Z)C$@d(fQ~6KM$ec79``IBP!H$ z%V$hvid{aaP%iZ<<*&V}@38#U14v}p)1IdtG8bCHCIPNFP&pxLT-Fx|3%*3QDB8r6 zZ-(X4McVn%nL%Tw_2>X-DFjX~gW*|tmD2F5@&z8V%B(*)SSc!fgI~J&=bv;M5s*f# zki{TPO{wBVW1qlHfoiPQEI#rK!NE|@9 zS_1a}v1&iDI>3Rrr0E75RUKi?Uy#Yc_n}4%SPBIOt?yS;r(Spj&2<^T=+;}SY z{tceK?J7VqF@^#A^IJ$r09OXhsMO8BvnXnh)K5RSQuvYtS$bd*Ml~YX#{svFokdn7 zFb)X@RtUya|L6c{BN164}%`MU+cjp8>%XmKP}8@lgh_*-}d z*BIv$4dVk)&)F}xHOnQUU9@_ElL6V zQTAN0=&Tqk%y)*ieP*l_yA>?hL$JD}M$Ad;Si3q#tkqHs`?n4hI0`WTjl#qo+NOen zt5&89-=CE=V~KNJ`j2Bq&e=pnDLG0W%~FQn1?3{z+)w+*0E4f{HH8VX-g! zE8JqW5anb7_d-XBd|Ymzv)F+QWCCO(Xo2NWVr1Xmb&7d3Zn(F@PlsX#pbZHxKd~<1 zw=LD;PT06DZkjL_S7a{O*ryioY5%Qim>S5tWF4~s;$9ZYKzk&M$tLP%z`G*tz>@fq zS%E5qH{obszyjK)Xim$6PI^EA(G!|C{;2S5K7NluX{N^#u9lb4u=4pI_a*(f&vy}x zFX;LAlJwsW1LGSnRbKb9sk+y8vR_B{k_xrow%Zln!@AGG`t2XaI+b>npYubx-((FR z>E#u`?M!XYZQrbY_O^=eX1kx=fIH92Co$nR*=3i{`PqFxbz1G2wV*&2pX2^`@?p55 zXbo|XAmQKpE!oW41+OVLp!2Rb6FEiWi($F*_>ej|g8AbzQja>O>-H_^%=xPHKUdq* zVz{Z@W4>ywuJ67iv;5Y#*=W!AjN{YM)$woO+q%aYcN&j6mIfKA`P1TG51{?XDcJQ8 zo7-}|)p+g=bpE}jpYGxJIFxxy4C|SuX9+LwKgu71u{PB?*0St)(7m$rzLMKK_E*E8&8|9(a60!#|L-kVo`zSsLo-e|mxTsq(O zHR>~Ax7?PR+6#|R*NzwqNO->*dW)&Azv`Y~sj6=J#J#wl`d6D}Llui?ldyMj{pPVH zQ@VRk%=@Ot)Ui} zX%*m6P=`1?a&UHbrXWIq>d(K4LVq`XF7en7*^1s*8x6CI=o?BII$9l!2LCjjeH3&4 zD~lAn`u7%_1BAFVYj~lXj>{ zy871>Ws|70-5+e0*CqYO3$LYO{xr?fy-BE}L&F8eW#EKS7@7M@s%hzJ>3%y>uWW_| z@ThCow5Du%SdnygZB?{n7i?Z0?7C6)GVhQb?bRi>)k07^Y|eCrZdJ8y)@QB0o+Q@w ziwphysM@)mIw^cAtziUil(1?ZU3P_d>J!vr#iEUX^a_UAmc6&E<%~65Dn$>(NIq}f z!NrXph(?CJ^O>k+``q2GTjfqGGqM~5U{&p_T%d%o039L{t!VGou2hnuH)!*{MK(WF4hca^5VnZri5Xvg#`Rrd9$H zo~^b~u{gndHEdU!r^D%%i@$Xqm;05$9c*P<98fXXpv>Q_7T>-_5#VE6kJc1p4iyk<&HFbK~D& zSAD|TnSCZywwv^!WgCm$sib%H{9awnHV)gf`nUd<*C$@rQ9EfD_z#wo!gSuK*m7iT zoC~o7m%t*r8*oqCK=p=MJGXNc;HOpjcW(Mc(iZ70*U&Un>f`Ii@z@GPK@f79xY;wo zNqF!xb2*%6 z#0qqW?vLI=#bT_y*QVNFjK(^epny6~aJ+W-qS<|mdoWo%CEG$72IU6`z{9T0t5}g% zRsP{-npu1OakH;!;aTB32sCSzc}C+lXW5X|tAH!~qj-%g7Za>cdq(i<1M|3}HJ}*e}#E<>(gl zy(%sLhTPmU<2e1=rE|JAz(N+2IN^B56n1{qmW9~)Ztcd?xt+`0wG65+_Q^ukC%y%|LmgY?E(!@Pdb zceq$L=vI#ez^Y{Cn>g*Lo=zD?MHb2e&=H@&Rm9ClGp+pZ6n*PUaPx}!5{Zoi0EJqiylOLuMCiP3_5=fG z&;^_-ZBL@-?L^wzS<;zu^2*!8AVxNQ<0&euudJE>ukOhH6T>3;E!%)pd>Q7}YkEW^ z#}X?S#Kv29!w8)Nn%TZTB&DPrL80IVwHX$PELeY^9BzrGFY|qi~s*=3wIu>q^E8{Afe=_d~_H zVOm1v$g2o9GF-5U9ks@diJ`Ui>gt-UH}ZV+{;^-4kB!DK*aWLXXO65^^E2gh%KEyw zkF0h*0C_C#Z`AxIK3jS`=TWTlWNi~a^}Y?BuZb@5@=(!$Gw~K}8dQSOU9C%)UII-r ztaj*G)1b3-;n|eoytBY2xLQ2v=^#2ax+6(vyd&``+j+<##m=DPRA0z>iUwzp!5#4)#sT#+rYz zBb|iV5#j|x8a6vSbJWP5R5zwV4$k~5vb#iC3OB5c2aV0`cOs^N@E#(&{JM$^#Ob#Z zV6zA4C1sCK;=n$6OfbA^!6ArZf8qm{OY7fNO(w0h56dQB(B~imTAO0^Z6l%WB z>BB!}Q^6|lmb@-drvDm-pPwiRfbHG3R=0tg&PHf;cY^xvfZ{4+pOSHu2<6pUj4dwS zQ(a7qEcdhoFdN7!=)AfTpQy9g;)b zFnywOSapO=+|tSg5jR1};BBQw&7EN?6Vb9IPsHl{YkOkrYUa}GJ7)*I*rL!nK;Q{U zq62zHI$t5QpI48~9*PU3v*DA(KbpdO^B=dY1sJ3GAi#Mr^(;)$amHe?_Ull{Oi{3H zxd#$JhLmA^t|3l9B*{h#*d0rl-PdBk`?XNvzahpiJdUo?dsMS~4;I}G?U4~88ZGL! z&d7@{zG(N0fFicE-0#|^ygX!8u!v~{j%tI+zkOoct4{OjPwFn zC6oj0dII(@&=nK8GYYg80Ut&<;tEIb1(`*v(;!$XvxW~PO9y;9KsOmJCH>;IVR%x( zY?5WTF0NOsKYYB(7>Xq+NU)7`Bu=>Sw1=LC?+5)9W#+60FFT2Lc88-XozA?wSC^C-38wyS>XW^y8}IJ!?_X0NC^=RP!9 zO9DJ&6Fjs!RK4sVU^xl9hiG-Zw3)@%R81ydL{8~1%kGQh!YbB16s}T7;s)IT*7)zz zt;N^Ex*d*h%(R|e?+9WxVZnX}YMM^qSv^*o<}}3(Y*rL14At%4soMh&O2#3h)+hoU zh|wjMJmqSoOFJhpu>iXLCcMu3uyi~X!9cOH03x&Zr%eQCz>cGpsPA)54zha1Hd28q z6?L_L)k1SX0kr85r<4?6E2UF%SCe!U)~a`~lGI}L>3OcEg4Kql5^*nQ z^`T4F@&+ydxP-9kVUJ(s8Qcb=aK)bAHZgZBw|~Dyw{OG|UudR?ZS41pR1sxV`iJkr zD61x)l#HmEbEm|v|6+~RL;({hEG80gvBIdtj^$n;r)7Pa4U1E`N>>~qQ@%=E9SWhF z6HUV=p#m_8$(kx09KlgRSxB)QL!G=o(beaJe6xcBIy1a){4GY&U6k785U;Y4Rikbo z@EyP`We-#ZYVNRE4>&M+`l6ok-}F#G@5I({8C-j8WGuO$qJg0CgZvC#Ti?M z@TukjUJ#X(b93&$-irqDozRDA$fg)=M6U~TF(F*gdt4Om38QWli7CR79(`6~EaaC+ z#y3_}30du(JL&_^*~ZTaZ7#EL=Is1k-2^+je8O+Z8Dj1Yp}f6Eky@BK5hzYU?wsbl zse?UGX!bgvob=k^Ca-}THbRHkHFngD6FD&fICJu;MtifFP=xY2@u+SJ5*$ItkiZq& z(;>A*2JV=o2rkL8hp$Y<^gvYmFiZ*2d z#w~i=zrlJEq>VHLCnQ1AVe0qkW_Oxmzv*Pyp}-D&qX2j5Z%unH^Zc1{Gh%p`=k0~%r=Y(*Dtg6|MwJ7aqyg`Nn<13Tz_FTbCbIKLOOAzllR zmsIp~QmgL22F!3)v^zq}giO~DXad}T(7Xb2n#X%X);6j|U zsBh?`#?kcZ1*JpmbeBaaz7D#4`Z@?R?BAHx_&^b%xz|3v+Aq9bh?w{~VS9r2mwg8F zJREe)560IeyAJkr1U+zOJ43^65JQB;<bv5b-D?)2mODxc6Om?6U_S5+9+52d`vk2-cq+ji6KbdbIe4*!Ptw`r((~*;sGDy_^TFm)Q4F zZ}>+dfByF0*86M`c5BAO9sGs>q==gwhQ8t-b4Kw)5Az3<-c0zex9J=Jq`3*+fi+`f zBlPp`dIpXUN^b!-Go&$wZ(xsKhH5+zu=vUv1HQxu7;b5jGV8d!-{J(`&_SVIG!8Yx zzenyZZ+(q4c#6!yeCL35yqW(7eGW5mX-bM9XTrAt_=3`zT}&{yxZ_ zz-8nt--*QdZgB3_Iq#@7D_sW%hRNlWf zP({;rlal&Mkt>gOpRjx@6ZnuVQ9VwbKfq^t@*G6&*|0$&Y@?!ge}DSOjMcTQ0~Mx(PL zxMx`^Z?*k}lrGyRaTsR}0ff3V{evo4GVk9WHMNm8an-N00tMBvy9Q&;MPtv!ZSkK*@ zG`?t&u{V)512fn-2x#gNZsBq2{|w%(2%e`~Y|Q3>7>y zwn&Zn ztU!!%GDH_gR2N0~mZIawrd{;@`=_6<_mIey1KN1$4GR?;>-6x6Ueu{TUj3eZp>L`C zULd-42Lk<3;T|`mst_vtli13FM@S}iVvjgX zJo1sUo3}yab>uBpl$&nZ|N21Gk>{$`D;fqX-~bMj{hEQ$34khr(1AF2nHM*dJ%9PZoo4~*6OenF zN1lP4O5NFw+5$^urr2`>#dLBXp8=mr4skG4VWd@H9G}6R26oct{#K1aLAwda+$;#+ zx5Jcg<#v=80F;~kS@Z!Grud>>l+*{U`$lMrv*sXxyddX=8g}2ulRSdTdk0{N-PxDt zASLpp-$&VncDv(=G zM(V{ggL(l>E)6=tFJt z{h|AM>m9+L_7eE7_H1Sib;qBv1Gq=o<6A&(`6_30terRf3FV7rDYPNzLzeUdssKC; z$m8`xesrt}+aT&~^m`t@4QvGcwFHL{LHb%2f@38;G|31)WTuS7x(k4OaI?o#x$3*9 zgGsH>6Z~bw=|K2*C)8m{=7M`H?40ft18Yos0PFX9300RG}w*I(^pIDEP7C*9JV z{&vl=>sVw=c4J?r=0&P*Z81+3zo)|kUgmvYc#|z#5g*=#c0NtBhjwuWyWUR|pID!i zy3~n;R$)xQFLAX(?p*+u34?G=D{w3;z{<{5#6Dbz^tX9hr{JQq8J+RTd5P|&wTEG8 zV^rq^Q}KYH;!@!iD4cryId#qkZiI`+MnhVi6~3eG*s}n)>0$W>%1e(Yeq;W~wJL8k z`WMps#cVzECD>e0{shj`TT2TEiOZ^o;aAaOPUs7#zHtIRhaP|e>VkMeT> zIfC3WyZmpH410cN?g^Hyy%u)2{>|al>kIdP`QDq|;soOt)FaPr84tx04y|@7-l&|G zs=Kj6yr_)yYJ7m&P6pE3?8?R7Agy)5R>e5&3J!8fN_mLK=TmokEYy?(0zK@-CzrdN zrKfL)+X>szB;^IR3!{mxozm`e!~S^Hfy}!f68A~aRre9yIOShJi|gw1#C6CgtGq3G zhvrIG)y?M3#K*{G7fHmkZ2gu4;|P_psrUA00LWT%d@A5=>BP~zaumwDX;OOO^d(F8 z&A$kXx`T80kA2hsX5z1xRoK|}+u7pJx^V0i=}aO`mrUpbA{mh5V(ok)+H zm!GHX(&g6kBQD$c$HU1v{^eNK;}h6*XK2#AJ1y90tJB%kp;-F!Sg=Ch%4*=7iWkMG zL97^Bb{v4Cal%=(r?)@ppjLh_+aFL=;)+MZ@3mgFkZ&2-nMR?fY!^OdF6g-azcx1SLOZhfk@^>){InHzwtdK=cc%LY&KgowB_w<U5)u*9QE3B7+Q+omk!MF5V^^CVe7CCZXaW8+o-pj+JJvz{9dKj+dOqW zcmb4YlIkj*V61JcanbUpyek)<`iNWcuAiCn!s*JH^IH9bDSy88H}mq%babp` zSACOt0=P&6;qEJdF*xTEz)!sMI`%DoqZL zKM8U;g#-7F@*&=tws?$ZmO*s{2#^dXG_gaw(7r>FpG+d9j({@29*ng;N*^A|aAO|e z0fW}zJ3`D7w21TnB|ouZ9ZqOyp$0trXU=5Bx%KND5}mxwFlKujo6U+I8PFl z@Ury@jo4#Z4krU59gqE2M>g3yFOma_hNi1k=K~}YLcp-|N-Bk=dF1#1A*ZK8P>r#X z3c18)Gh`@>QyU)-4>gp@pb$$6Gh85)i3TnXV8!xlLqx=k7>3DRH>_r^6W(=w~*~bh+uLiq9!MRIYR!OnQ%RyAS|`0BW<&7 zCDAh3^Ml4ma)yMoB>T_>if~#t9Ter5XNriAl2UPmBtIu9-1qQMy3V}JT@{U3jvY&C z)>AQwhAp3irO6c2&;P*=2KV-^R8!i~focEI4p2_8dLF#$=;i2iyr<3mb@pQsGh?Lp zV>$dCCG9f>u$e$ibKwtGPY#hkrnzzk#5G6OqfO5-XZx0`W%9FTRmy~Ef?WXwVkJz) zRt2?DR=*?X@?4y&$Pi`97p*oPvh~TykRu*PXLxTO_sw4qHE_A;pfM2raU%5MpimdMY62fWnQ?e}O^c}CbRiZCjNAr&Eal)$b_tU%p4 zXHn(|B{Pi(kc!x&tQ`Ny3lvBadY)>W)t7u;Cw5ZV$9BR<9R}5 zgs2Lw)m0e6e#ME6w%R(9P;eU-AvGW9XOYLl?MS;{-zdk0DCB8%48?J3_eOv{cPYb> z0ZM9eHl%Ddd!UkZXsmG&E_}Akf8a-)o_HDg)&7%Im;xgMw0D z-g1RRV)%!C<+L8nSi@LSmOlWerQ1f0(FZ0YT+}0Vg_eLpJ`t#RqDM%nw9YnT9sy~l zz;LY4qv2Z}1u+l}6Mu&=BAyh_mV8cz$$L~D)dO-2(%JjYvIvYfpn53BSRr^t$J=_` zL-0@z=xRa5Y~fWOXT-SKFQK9U(tiXkHS#ZH#~)1tXf zXs4SXTt!4aEegmHSeo_|rRWd4o257;13^xpXwz^y9KM-Ob2RiTM85z)WlpNI9ZH_TcSoG4s2)S*F9hZPgzv16^y8+xO#G&5t zMbio_+7RCs(Pk5QE*=abL{n<7t?@DoYf`Y^n0_mMK9)$4K#sy8@ZUzwrRrZgXo#g+ z1OyJMtq7{7_qZ*Sntik-Yo>a3ZJ~XZe-WMACLwE6(EbR{QbiJg2;mi~Zw)OGaQx!n z-K@Fn%=T_o)|&ZNYv zkZ(QZdR9k^tvQrx^T1>L=7oPsU6I;d0_w#6%tRpYMi2T$uueLCa&~9m&9v4Qu6^|A zSjEx--TBY)Fv#bBeuMJLczb?xMy{uKYn8A0C$AAE-a;I3uR$i8>KQa)y#-^#h^Z5$ zLkz9=M=Nx#tpWEK-AKIUwg`i)_;a4+-h{iU`m5q4Zj%J3J=qcm8)&h zjCC@jagfA0(t@7_h`{ZPh=U|;$w|ls!D|z}_N$wMrxE_=HYyO7ss-0~PqRTMoqhj_ zSb@N5T(AM)$f1y7tH+Eb!>0_C%c3v9q%v3&tpGJF>q!Wjf~|tkk@^B%Le$LzIUCY> zx%vdJ4Fu4;WYdz)DIh{1%TBV7MfJUU>}FagT=j|@v*KsUVP>)c-2{a9x}ciqUcFM` z#jP==KMffD$~?4N3l?*4&NTC?1>aoREj2)0gZ{V3hZ&Z`>>i;tdR3WhxYwwx)Yg6` zpfCwc(H4hpgFl$4ZGm}yMprU2Ym1?I`A0I*cm~~mr=yGI?M};x>CC;|ia}rwjExA@ z1^&GfBYy#8a~TrVM`t%6yn?Mw<$!b7Ya8V1(I#lIhByUp7VV9g3TwI^q#xMk&;Hs9 zjOz|KfCw6vF^2J-gC~u@m97D9LI(3^IK+dkK@11a6TBD9Bw&jbLX18YeZ`aYzU}DX z7@RQDWQuj%|G^wp%2)Rf1H&09Bg18=CLifx6W13sGRjFBj6PsLKGoE@+!49t zs5!>%A|)>t`K;?oTHO%z4#FAM%2GzD+yDW1*~695C-~r*_`uEG>9b3~{q0Iz#QsP` zu8ZSp51WqHzcj{?u-1nBWphE)(?!B|)ynCF>C651%b`9-Or2`daB1)hFSux(m%RI~ z$W6?bup_7a85>aLeN}k1?Ybu(J%w=H`Ox1|?)r5#`w*4G_x6<5@>6ug-*huD!T$+} z_&WVJ0B5N4@$Gm3%ta4>?|h#n`KY@te!ce3UO4SXoCW^b>6q@hOTznZi{tK@qUUWo zo|-Zb8sUFE%4B`O z^JVVk@-fm>LibPV*sEHx>)v>~jCc<*xBGcod*BwI^vXJE>8iUw4Qv2ZzAb4#JO3p9 zgJsBD*|Biy8}M_Oz0RQd)LqKAv}EzI9c(+Cy{PxJyG)t0zwPj{KQz$Gf$0H0p1zR7 z*;d*r-pden@II&8zTTH#wYO$Qx|g4Jd!90g}x=rJj`Q9jxM5*^+ zU@a53k}J+v*eN{_gyo!GU_4Q^@Y@dls$oau{;neD%D&>-SKhKthseG=xdn_q7Sc-P zr6rp2TJSXSdD?tPi>rNqJXiw)0jC}{&%28%Ke6b6tTjh727u6P7FAtoLr|O|jtd;J zL@9FQTfzXmOV9P7KmL~Cin|wJ+7>S9j-^^8+0j&rRQy$iA{=YRceY8VE zRzjO$d680!lN%+gx|f&L0Hsw`Q!CdKTy^r#F~K2X5Nxu_Zz$!I&8z@7mX>LHyLBch z^s5YQAD{R?y(q=<@2qQaPD zC212pg)Gz5_V2oAZ&?dUv?_0!k`s|EH9rN{VRO=yRc_qJ2Sd*Q|7$6W?xCO<#0CPg zp#=g;P?SYWnUVv6Y4Ifh4o3#ym}7N6xYhZTCFnWYEzsCHjd4!7Os8Zz$r+Q&esHRf zNTwI(*-dEGliKZ*O~+)d{_8m|FMTFPOU(s=o|h<56*Wo-fuN3*#R26aDTm^MkdYA9 zZeqM+h7My4k6j)ASl3@In>`=neR=oTx%v8_vpeHz_`6ZrPKhnldj1W>csufA1+RG7hM4 zkmm21uVkL8Vte!>P@^w^NaIU{P$uAjB*u)XCpuGuMu3W-lMhD*)ru5? zhE#Lf9&dR+P>U?Ly&STNkTL%}ak;dMs)~|S1rDk&W(lcXFXk*6ukeEQ&DZQghOde# zWe9nj3}jAqP9r8ymrEl(542H`C9z)MBp}_8qZ&OXaJdXg3jqmmRVet?jB8!wa<8b( z@(HnJFhS|vg`mfd>0CgY@roV15+&F^mP{^E0M8ZfQdg)K-9;&JVcP^fNkciLih)j< z)~Lyt1S~5n0fk9XrYfL$vyj_@+>ndri^)h#l`yI)dS_w$Q_&5&5FElLQVLN}Zxd+O z%&;Z)R!WMU0oM)~>UYLD`q*`e5Ny#(wJX34Ua1l+*Zg-nVJa% zv50JM%Wj0pD~@Zjq%~$eBCqR>Y?rvH*%b@UwJOgAwSqBAxP!(ywgMUX4NDMwydLluZ#V8kc+KnOwb5ikpy$}u>zv4Ar@cs~Oj`&S~+5J;Z!Ao=Lh1@u)PL)%UD9?Cdpy4mE>@$HzYIq`}9SALBlM zljFlIY@N;{&}rJ&`5X<+bpj{A?~X#)n>b*u;m@rOa7~a8B}oFO)YY>Q!dZ~q$p}4B znG=ivXUc#C&<;!(1YOhm(BwO7TmeTuRotp7$#P4DUh;9#U<=ziuU2m!Dq~((kqYVt zP6O~^1J~f#=2mfvq4JRlH79?g?(WR)Bl8UX2C)TJ0onw@=!J;|CTjVxJFrf-`t{-z zoel^8B?h#YE@daaeL4FJH$7H(;cPcKFqHEK0 zuIpxvQ2s0}UT3<-A5&!y_lbj@HFe|kfC>wJmL|aO zt~uumU`|_K*NM?r82)PC{GTNK%kJVHr`#iI`L6esmTr9uuT#z@fH!07=llWrMGfCd z+!}^g@t5_A>9yo;>WS-|&pCkio>PYL2YOcn4)5(r zOyf?3cr`EJuucd=6D=BGznBXLM^0~Im-Vqk-}8AGI`X@#>GA%2VYNAfzt%`xBsV%l&xf1LAOt?vQfu4dOOnA-`(K4|8X-fzI0-gkuadbqnKK_Rz`_Wf=e z;zVs#rXE_;=`Kt|Uu;WKaitq6c8oX+4pzzIP02~iK)KAL@qQUxR?>mm;%O@~B1@so z?c=6ZDSCL*@y)NTZv~`|?`Vf-Xj96W-#-K-@+FVui%w3$WXrf(8JcqO(vDKEoju;k zu<>3UHh?)BlwI<(u+*FH@6L9=cGh+;zy=HFDLCghG%(u)uZV09xH=kHI2-(Q63m!{ zYngkgd!6?1U-NSe1Dtb?yfGI|GBZ_k8_K*Jva;G->3y45#lsqh01g z$=bx&M2SX$rZ`%4STg@0q&-68Txg7+?)XQXuPc$dcgMKQTbz(z)eMFCjOekDH%mVi z#b^Xt)FSfs|FVZ^EuAT_{}ja_hy;cTj1)g^U^IZTDklF+U9qV9gPbRxF8J+Q! z?TREKi#eVyt#M@}lcI6^#`eZ@^;Ops-!P()`=CTX`kjV71mc_!L@GfaA|WCkI*RdO z9l-BaLu3{RkUJRmO_;s0MLEaTtJ{qdb)sQV%ha?jfU7BU4E=5yb45A?&N~9&VsH-fsdQ@SVF8eVf zK-DD@rKC)-B+M8QU5igl!Y$#H2~eD7(~XxI?Nn&yq9vP0i>Db1B1}|GR5A_7AC*cf zcR-$@^sYQi&Tu?J_1{gscb1(jH`UQmV*zC3nixCkDbt4Mo`s8Gk2`Q4P*g=*dh$`P z9!@6ijgECjFJem3S`d;Kl}D|3(&#K0Nj<-6#kVi^H|G|+5cHY-P%u2LMJfau^igbb zru3y!g#@(1yjw^tYH0#NF@YOEHufVf0SiaMgsh^)iZnoJNUz?b2G+GY5eN1ivjL2> zop5+D_E}&Bjj#p`kTyA|jlR%mw1tK8k@QG%E0(3|rnn4C3@G*kp0Tm_t1=YvIudGy zEI^U`jug$PmtwjFy?FPaL*8{yNg$Hem*;nK_dnnWvh>rq|5i2C|Kptpwc159K%9?Mn-%{u~45GXXVBwgy(s zX>F2DGA!m3$JyoHU$WPA(bwN? z>kOrx4S3kUpI;>JGph($mH{<%mfr87HSOWsw(}t|`?~D>S(W;#i)@>`^KNe59ynKg zrD%4Mb{PvVV{Tam7~QEW`~|x`_u5WHSFhAMO$iOWVE>OZf4rhOpocYl1BbfZ~)aSEyeoxXl7Ah zVs4u)FJu8@lb2!g`W)^>ub=Ws8yK~3GpV~{Fw+PD`;R|s@%FfHyg-wQFzc-Q9OL|^ zzq7g;k;^Nop$I{}vl}x}c(4dFbPp7R zIx9V;*lWdAk!iDke&Sl^vbg-X7Vyo_`{`Z|`ZDoe)2nGaX009Jq_21}E%CrD5 zDj+A%`VV%C4M)Jlh$TUK#^Fe*Hx17)0N z0H}~qxrz#5D}jR`FL4kD@gj^k3F;sm;-AZ+ajBf+jF)bY8?TC|*Y~Oow4nVzHI++t z%R)7wh}oi$XbF6O(tWq?xazOg4UJ+xWC8TMX^O2!ZMGc==eM=n4xPnkqc5^M;HDc9 z4m&TOsY4loa!f$}123%!K|;jnIBF6ZU){znle5p8K?)9T^7rjvCT9dJx)YC8?N_8+ z1&O%tp!n7qa=izeZZa*zQ!~=d7=OHcXJ8iwvJ2i9_ideaMSPT96=EvMMXP-FcmQ4I zT+OV=vDsJg*=;${uATnIfBlvFt#?jrz4*5tD*Xp8jB1tlKmYOyRb&q##jEZ4lvzC5 zfS?0mRZ6vCp96ZcCn@Gr zVl@5^v#A?EvaBsYI_nX=^-uiJ83G_MOPJ~-MDP(q31|HT;GV>W@Ovff))eYUvEIH^ zx~Ex+im@b*$8&tZ{g5ET^3N~~OC(v6D(+z!rkJYXN}s-?r4!cxKcWRRqUEEXSjusm zkw&5Ytz2+|mxHZj&hx5CWN(lJrZ$BZ--H3f{2jIdOi z^tH!>x`ji~O%qi{GRhD*U^k=G;+bpKO%1pz4UJOUu zfxtVR15SXDcUcU0_C0#jB#5W8BU)!O`SWu4`XYe51Ej4#gb{oMkO2%RM<>K|*_^#5 zs2E>l`K`AOZ2QJ@H$u_g6>%vtEI85U(nu+Gt9hp#um&GF&Dc>PYZg3i02_9>tr=W^ zlh?&=mi5o){maM>U)9og)z$_7*XdZ+$PRD(?8Qy7XdI57bY>6g6QzjDkc{V|cxOx* z1)m~}u>R?en}~^cJpgeHW^->?73rse=j-+4WIQFu#`kt0wCCpXy4tSQb~Gu(Q}Yx> zPk+F7L26_+#9)%_aT@ zaaNF|$3x1)>B;d45a;;k2_WZV{P12yC*^vnps%H) zSJ^4Cy}lhDf47lz9bTWomgL!!3bFHPVG>NQmI;38@O?`q{!lhm6 zR#lWcq7=A%GX5tpIIS9_JK~@4aDLTC1*AJA1tcC#9w?zM&W0_i_S?Jex9*ld+mqKB znPVbA2Cl;icSD_L>oxRY(zuhh79~b*r6x`EWs+H zaW+ZHie~6ab_+?DL{$o3s(LjK~aJbK1x`GkU%5Lnw$a-b_3f>E`D(DELKC9@$VxS&aLT(vHWlI9Rl~y>J89csvA(W1tBBK(rtY4n)-4 z-@Qh6cCnbFgoY<01>4MQjZO6t<2PP(@)$tIW}xbf;nAG!N1B8bS6P8EkA9~r6)Ln) zUM-PNu3Ccj^>@m$?L!rPQgjhYnsu%hW_sQaq`FagW7I* z1mMl|;LQJMDPqDpN9+o)%(t^~yV3K01v;a1wy}W`N!Fuz%uenWqW7Zjy9?@c69NF0 zY)WjJ7V&hu8fvX3#*Lv{`m2;D+Dl^*-lm{Y>y!2OWqK~D#s9Z7vHF5o6IZsv_$m&| zgDpvquyfC92KDA>Pl zOCU%f@PF+p#|I47r71ZQ|1@nBQ~aGNvVUJcSw?qmZ}Y}4g>}1($2j!yl|su>)KWy# zY4von4K@~fSN1R>VkdX@(@hTwa(Da@#(rKEMAS#jITUz#LsU}{=R;gEh?CBRI}!Or z(394mmbFI3TQTl0^Pjh#?w^B~tCqEGQt1rHevi?M8@nW}zN+ z9){BIvp*J~n_L@5MVgDO3N30}fA&150We7%W+fYd8iOVBHg#l! zX3KYCrhqM3xYHnsOksS;2@sin55aeQ^xa^dQ<{CDbmQBo}m>5F*RGO446@G{0O$|J97H)88u}=;mPMP zDP*e+O2ep$eqb4NK*Xh7HxL}RFi9J(4E&p-flQk_3IHDcOa9B2Cx(MKKhq4Iki3$2 zcT52)Oc{X?YU046i7RkGYcd|EhVBC^c{4hRr>*6xCNXNr6lcv! zmX^R831Es$UwfF>th^~A9p*ARN+{(w?3YX~@7;3lgR4z4)51sth8X8p71>!hvQlNS zJRGhC2Cq3m(a?&NFQ5yfnZ6|#+lZGh1IipKdw6A_8q7fsO<-{kWP1GKiAlZ$%rZwa z{I|pp4uVqSf!me`_T&;sNRM35Ba23T#6EFp3;-Y2H5xoJ)Yl7kxUYcLBuHXlHbfD7 zJo!nOX=Q+ybik&>0)S)!cjXpsAQy>u71}u?8tnvOqJL&E)%k^Z?-;au6=`XscK;^)imans)SLh+qoi2^U%HuN#VW4XUa=~szA}H z4j^RnKy&>?vFui};|7NMAy60da!oAf`BHbXDRYKoax%*VT!tEs8MiBvQ9cXL&b1ae z4L-I1)^q<7TnKU|$t zV36NcY}>YN+qTo#wzj)>|6b0`dpqyUIp;Zg-io}^tuvqj z9bmh+FcsBNR4(e7Y<2>}g!^Me*^1+60y_mDNybUeJ?O%|!kEmQt{Ip|z!t&L)Dg!~ zONcRr8h2uOWHDuTAqub&#&5l}fFKje!iu^n!(MEWX8jn1;R~7H-=&nITjX9 zfh$G6G6&rx9qj(%I$R_;Yz#V5?2!Tk;wFl)PwdkH=FbC9h-gk2ZBCdCN_cc9IAuY2 zc4P@Y@lrkx!J*>a{(dZo z+JTe3(aJ>rbe1QATj;s=@tuWR2UwMR6>DG0Bq@Yx@0(meq0GuE~*Az)iA+lkU z;GVu8WdU_RRd@ap$2LZRZI?BYi`0W9Tp*)|A4#-fcN~EnTdbw3E3qHkE;27x$zmvK z`_Z6DR~o@H#}`;^N;e|Tr3DF34Cs>A<^`w1w;MJ!Nl~tl)ofU{1QKJML*zH%fZaP@ zLgvziSF1Zovl>;t0a;QL&w5f3_3w+Y(@R&oG!n~#ny0&5Cdih!N&lj;AOwb1e*R@x z;3Xsd3*c#Ice7g4v{nYE-#y^{n&(-QWv+nlUu|_!WnovLvNpL}e(LpN434h-DLZ63pGfJ_nFB;L7vRM5b-r?>K zzd{Xn^}#A=7LZB0>1!2{hLhXwL=1lLLl58KCMXTEa59+BWs&(#&tO(j!rdF(pVBi- z9s(|2(wtE)Ii3YM)!K5G@WlH1(Jvnn*K4E)?K&d6z@z5d+K!D$l|L_8%UUX5OaDvK zBC}VQEWx}glWFSsy+&J#PhpU{z+9(wkxeQ*J_k6905BXV&P4l^G4xF2{HHJHGz|il z9ImX|Yz7k1)!WzKOU@|1V`{R8YGPNYDE?mP7gqX1Gczoak*qB<>~|atR*GIVZ!@kYI(Tp#yO4e@xM9R-Hrs%jPu80)A8;H+ zb);nen|O41;N$4V9;4bk%=+jVKFrqX#+c3uH zCO?6{IO6N_y6ez#MK7CyQ7;|DJ8k6*(=U0^x;aqJRI6DZNB=W#V)S+zJSw5k*DBSBV zrn6d2Uc617+Wh4<5u{hLSa; zmf!d1$LaaUuUa4e9_n^+DgX*CJ3lpdJuIw$&jJCPhz~Zk6P=?7G9hFyu9XtTVu6NhvM#O2_sQ<7Qk}$12KLSJ4l4G+3-6(B_G81G*!mQc8SxeYYml zG;r{8UC+g}!#Y~+U^u7&%%`T3_$0W!N`3?}WTG-;XUS6b3UR8>uW|i5+*08t1BZu7 z*xfn5YkZ=RBOc4B3k;Eg&>GKRbo#{K`HUeZ2|@Z2G(-jl0h~z@1I)jvB&9nj$B)FR zYQRGmJ%n`uIm(sh5q>dgYxMLD85x-@c?XC3^@Ay_S@zeYz*qDqIxG?z62?RuYg02* zY*jjFgyS*4LK*Aou^=Lo3#IH_@MOrs5OBzlnGcZdL4;j!vWy?R`~`ts`jy7G8;aeiAy_*d2hw9?PEw=Dx~VW6A4w= zf0Pu=^3|wK%SPK^k{Rcy{vq_2DA!A+d+*UArPUP`PubSX)6(*ecyeRZK(Q2P9;m^X z#<>({>LKD)8{>-qxYh@$E1kdIR7S*EEvkae2W8Rj%gZ)){-MQmw;E-U)ab3N)L$GPou#H9-tcKXe&1F<(X%ZX7U<%{XZj2&sa9e5KPb7aI z{5ljVrN~fhidFClYgVA9SE^EK!uos8DCVlI%)A6}e3Pos&|^zwKfKaS0+NkZ`EMh^`# z(9;EaB~#6-DyR-s-NkyDv5kWw{XP@}J+cZoh?@Hgf@ zq=^ti2mZf?G_xMP{q!@z`{2jejyX=M+y1qXUXyYNUB9w>MxA~Y$bn+C@C5T-0|*^} zrs3;{yZR7ih*zaRc5hxb+ESn^=nMJYekAC*1xjtPiBeIG4UGOxB(8Fvscero5_1fz zzM*&CM0gDmeUtvSW7q6*FZ-<4o^mvHs!ftv74#cei3GZttX81u6kT?ay{shSf8>cumVq zieQdy2jggMhNM6Yr3u>NNN0WZRZl_7Vf`^4;)8@kU(?Q&j`U3ED!{}rmyvCY)^hk` zJIZuqn#z!S7d~P`ai`jl*4G0TC7gKtwayq3B{P0Niph)fz?Qd*@)1EGGqlY=@bX=Lt)9#Um7c*7SF{5 z#R@H8XPkl9GvJ_Ycxw#$YJZ*C>4Qk5)2Bdhf0z{^Cj=LxMc$5Q_^TNjz*ow{rbj(8 zLF(WHNdzgm2IGT~Z!gP{Cw2qN4F~}X-KTXqsN%s3UY3)AYWD;9bGmgWrej2Sw8tkK^AI6MvW{5MI-|zoE zfMKzuEVW@*d2Kk+RUm*|13cC5NE$ZjI3xWwMz!Bm-Em6h1v&_6LLdXcOP#782smE!hg0-ugS(>(IKo_6rF9uFs(NR3$|%{d=qiLV2r`X z2tK}{e%qk-%ED;}G0YwJ-J!9+7&uC!_OeqxkE4E@qt04sj6U1r0p7H@M=eCrj=WVV zS}C-@s$=a{^>n1}2iOz<3XDF7RGj7A(*#ZKY@Fd1Rq@yHS67-=0^2u+>BxL@1;p5` z3j5_fIyPx^DyPS6^ZiqI`$wclQ!@b;H|MkLVYBX6@zMD~ z`s0Qj-BWXMwl~@JxXHGeo6E!cV@cZk-s}a2HQr0XO3X^V;q`SI_wZ-K)A{51+A9Y4 zAjQ=R5zp#L19d}i82q$%;~ zgUgAF^8LCt=x4K)Tc{)fr`!F69eig^w}S@zIw8=~{=jad)AhscVz%C$twY%-t2lBM`inuwAcI9ljW4nn)@~F$?n_M=i*X72K$!tl($d{pXWB@ z;3ol1T-0VGKv`{ehUht*E~m@oHLK;Sc-|I#7Jcb?nCn5n?oc1yVLv@J!?PWo+_lVU zME^ENaIAG)c*S=9b$;kO$o#q447l{CuZaOilPJ8y_$^+~V`LwKhy9(j`(^t z|D%Am_j!E(_8P&e%Tv=H_t$)q`WKh6ZI7|oykzZP5qq3I=NQ6v&BH%lObip z;9Mqvqn*QdaJj~;Va!t;=v3|fp4_gBhp*G6Fx{uIaAOj}ulWL36XH8A-*H!$RW~-6 zr*BPxuwRJXB_>8GhDUuoAcP@C*_2b(rx4V1VZc)Cdy!!RR_ z*&_f5J^!>a1s}lU%ANm1KF{X3CY>%FTr>kh+c0^pNGKdU_a`nW$;|hXI}?x93Ucz? z)Mop#`K@b{ViQNFnNR5V!!RWBhw782dsg0$?*u?9`$B>~sZ)81QkW=PZ89|L3v%6aan{>f|AaBVuaPSkaVIsX4`MgsD}Kqgam6fI9asfYt2cvMM-majafC5)*{nmQCbs?qD}}XvPSC4Qpwz{Qc|;% zZ_xnJXgPCPk%oDr#>}GM;699cCYEBOKKyK)h{M7CX@&JZ+PF505{O)Rk$NR|n!?JV zxs3_*F+#nMK-qc2MoZg#gQIX|AQ^44N=1fF$<&DiZ8Gv$WBkNHCEK>?{vVGfqGYo4 zx*P=N)KR1QvR)()!O?O{a+8&9$uZJxX$@eyBqMH=G(3#eId&Q^Om8Ux28%WQX9DDPMs^eiXuw{Zw?VEtSxu)7pUFaN$t-AV1}IB^qF4*mfSY zOTXcv_bt=}?5E%!uZtMq{;-2{1Iq$1Np4_|XP=Q49KK0_haf}7>`-8OBZY(Z7?2V1 zg}Ot>75qi+bQ&u4QrIBbvF*DO3I#!~l_rvnW6;j9!xT_(1PO%fwF@JSAm$nQv)}&- zVyNv0(EUxwPCzo~-(5Agk>eG(axTk^1J|LM^I#;Ah0^mxs6`%%2@^!|`-TZv38`Tw ziC~SwC7dUUy!Ir9@O41U4r}Q&tLDiqLdVHK2}AAj#`r(6>@1`{bO>~Va^S0B8U^;C9D zUmP6GWgw1&m7zaFq@{-&js6CX1UJBZ53#>V{37IpE6{U{R4QnFshRs00(&Z+yF($j3d-XJVz+Ss)9H`DccH zAiJ7g8m7V&E`xo<1nBa&%zxxaNOVx$pFX354}LzIg8a@dAq=7y9RVAHG#H}iV4C3i z{WDyutxm`_2%2&XHKbPnx=Hfr5S7Xfk6qVy=M@D;2+#1ztYp7y^tcYRTW6u{_f1cb z2NHl~s`=GJfD|9Xu}2Ob;W-eVW9)}9!~wioLn{_(MFK@LOiExMIAa$YhX9Bt)*mrY z2MIh}0uFqJ>RX-IiHZTYT)NswU6?~EkDk^-up(qWQy%|3SHyeZ(+3dBt9i5k)%(H` zmM%5B!Q`U_9bzwn3leu95PT9v?bM*bGD^%ZD0 z#*ftHD2`W?uk(xmEdEAK+VXOJVpwcC3x_Sm$54yxc5T)hA3=n!@eEtdQCVB zjTDtP0nH3)1d-c}oJ&7cO$?<9;s*y+-tP9E3RV6S$u^=JEHq^84;Id7_UNxJsO@{1 z9J-=0`hVb`i({G)*aC=wi({T&o%OPx@jOwPJmInxh=x`9?0z(ko@s*U%Fp!9_vkSy z6fQY^UraG%KvWm1VXD8x{lCx^!rbLCQA~}$jCqLC^Dwpfaoa$4=?ck0@S3@{p{S&}%up80r|%VVA;uMcGPtS=`-a%~{Ch zh>A(|f@t0E554J+Hmy&sZgji6?nA|&*O^#d z=vt4e)*9dV)orV+Ypm`ox8|BvJG(o(zV0FUUDLAhzqW|>l0vn!vp9q|Z!$5il=xiF z#p!ukN(=G3mzR;%oWDM&5a$PDqta2DH@YkU_cLCKM;X2!9X9V9z8F#Oufg;OTV9`` z;+JZ7AJtkMuJ5@7a+y~4^dFP6MU6+5f7{-B$fMoVqmb)-MkIq=+0n<2sF^ki$FgdXl0g@P63` zFuJ)Lu3VN+r!%?lpSG&Szw9@$UT8XB9-g-Jyu}D4dE_8WDzmnZJL(txMq~k1e>iS; z`jMFY;Y`-G)J+`|2%#zImWUx`7zf96F&cv^ zI8keqM`6L7qGg_k33`0~$K&iN_la?Uy}Qfi?Xfpj2z_bn6Yj?3fwp^cBa)XY!YowgDT&+Bs@7t(hKfiWNQOFU2F+~j6F&# z6SMfxF^Hu07b&uwv$OM1UM&{IrGr0C9LX|VHBPa4(*aFseSSF!Q+tc`UP(Qm*2#yA z12x>V?sO#&j{-wbT&9orle;Y>vew?ssek2e3^{(NZMUB_@hKwN))F#A#dq<*BpaE5k^-4T`9DI4nW98u;yJL?lY$F)L@c*J)su#zLTE~{IS~mf ziN=vNz=#Ai{`HC}tXVa;OQl~vf^N#VB_uVmB+Mnz8rAbOy_t0X`+i%u`8xTW%B&?R z{_Og>?QzL@l(o!zw1L%S?UFs6Y>IGLzihZ3NtJEW`}N}4RlJzQs{@#$rAMl?t6594 zmb12_-IIR~b=9?5!Zj#RR=(I1vAt}cif0`SBG89W*#QcE3M95bRflmJ} z3Z+m>QRcJ`?fxbFjT`u7PiwnYY3{{jYm%=K-E?kTsnity*xRJ-QWnXT?j!p!mw?BG zXwHgl1*W}Us^y41Us-qID8pi&sPROjG{XP8jMF)1Gp)(!7qaE*+eMOr!&{Da7PIyM zYmZ~QhQs^R7G3+IdBcKnIcOGw$j_)`+;+9nCci?vV1&o!^BjP}Ow|lm|UQzZa=QQ&<09&lREtWEt=ht1-d=xJFA;LRrd4nroxHV@CDJ`H1eK3oUWmhp z*}9@=TQ~rV9o4e@NAgPxe^zplC_99Qkmbbb9Q&OCr${u0Bxt=!%j_aqZ|4Bx9GoZE zVHbbq=uv+Q|NA2@!yH}XFA13S%P2Qp)Ky3Y;XdfL6Zz0-|8I5*ly2t~%r& zev9s{Y}&O(e`>(53luxY(NBVenEb-o;73OYF_3^713alV^fgmbi-Tu8FPyhHFNp)5 zA4useaTtHFdYIz05qOpk_gP`Y_j}VisonHEEu~#zXe6c(XCl)(z}@2=*goVnp?%0yZ=|_ri6tfnj;_+TQwstYdMBP&6 z5hWd)N&glYPuO$CHJLm2cdYKUp(}| zXnSr><*%OAFTEBPTd`9qNyPN>pW<8=8fhsRcTWNNVfT(_ruF?A({HvN0 z3O3=1z+9uCw#4s&g1>?XGz_CKX8~xAYD9F5NH$}wpj|5Zb`xYxOYMI^2htwEqF4gu zYB_`}qOiMiPKLOMGRVFi8o&shB7bL>kKQv^ICmQmvcrhiBdc{P_*)3b&k6)+EmfL2 zA}K-j=3Q0>Pn#m0=<|ZAZ^$&^ADiV+%z1);uV}6wi^GY;K_`Ye@f*FKsjkH#DPrtp z>5Z=wQ3#2!L&$rXxK2!9ov_^?lcOeAiUJj$g0vm4;4i_4K~P5x8Rf;Up$q=;@k2i! zT z4*l7i7trYGqf1oFLYR{tC~*#>D4V*=llyS2lY;sER~m`I&}a%UJvD}_nxEiQRb(46 zE30Eh-EUBahTjCHvJs|UcMuFtSy?}++B>W&gaL~%O(Xw|gM2o8OHI>!f9LI%KJn%; zEZw!ZqsiJOqpyiH)dFT2IM5WU+hYjwtv!ag>FEyCD}4$w_%1Oh7?_uz**_v28Jdf4;u$_>gkbSdkzS0aQP8naV0ssMv=r zUe@5puoBEr&ZX3@0B%tvTzXq#o&8eCf$pE{3*>QU!Wm=N;FTw5$Xprx)1x*t(SlF8 z5@Xb=3*i)^c6lAc@*4hc9TqqIh~=GhwR$ww!UAeVW$h9G^1+F6sP=bP(Cf?(&*Zl6 z!Z=D^uQ_D}$ezjgI_nqAu}nD%N{k|pV*1Es@*P)w-@f=j@t6IF7ooi)Eg8||aosZq zs_dSs2S2St>@$Bd)ma8%>{nErLVI^&x8ZiT4N~Ty&vv3IA9ARKQw@_4pw7L!W1Z=& zIn94Qq=e!Coj4uhAef`qy!hk1e?k+p78P4bx8v8+{(87i%11>9jlGM7MjAiEw^~{k zz%9hegLWH*(_-y3OPmL>Y{&OYNJ*UI^os2Gj*cuLF{DDXzR3KscQ_{OCQI09mOL+l z>u$}UWpw5JwTBHs?*l;{(BSxetpOb}^D*9?sc$sJH) z?tmDyC;9MXYW^;KhL3|Y@gfH40Hdv?SZu2yN?Pv3jK&}y(-1_sLiX`4OQ~_C)KC>P zY)fR@tQ$ai_A8nQ?WLjB_ez-DmKEpwdM@Pwa4gk{%xeyR;9$4>S+XT`&KSB09b}K$ zv_=7!H?>|-_)?H@s7T2Nv8dM){w$8Pbk4+KeQKLLx>TG;r53L7OJ1Tvmd&CEo2^cb zxgW1}ou#fWqv$lZ+*%xpg8cP`B3H9rv6(FcQ?V0!x2=|}R!-l40~2H~$@JAR4vD%S z*pf|S=do!#**DAJ0e$|oPO#NjXf2VWuh6oq|3mi}y5 z!CqTXuIui-DmvV?W8K)a`FK)pP-=6c1iX+^Q~S9x%+gV z2Jw|Jbd|VLJSEO@aK*7oD(Yeo*c&|qNQm1iWB}NeFlhyHKfiC+6p{QKzH6hb*IodX zvK%_s+AK#6p}MVS5;`84a?@6xL^Yc1M{cK|ydpvg)j5NbcvX#YB*1n35;CMGkP2JS zqJY$`iUR*r~t)c}bM=Qr`}FGRvv zQ`0{_{G_$hQ{3GASLeNMCMHLjK&Qvn>Y$qm;v_%!SF#BP)IxmZy~x6^2!XPndNeqp zqUfUND5kMH*y&ok6bLpucKv=sdA72Q{~n)6mqs6e1t)JdVtOkbZRW3v}5}i<4&x zpJ)Yp=%q7LFXRnGzqcc%_k!V%f*tR=W;Es95jURw<=clbL_UYK_?3Xi0vn`&UA$!q`+5UcO zB6`N_fBZ?D_lBC|0{%J(_Z3EQOZB@?&W-FHHSt=p(|{3aG+#TJdmAlKo_Mu)_TR9ii1us;q!XMnYu80WlWt9AY=`3RST3Kxu{JVwe zAGWZmzZR17?3OvZoDr9k`u}QMHFqD9!q(fN2LMEJ=9tejSojX`%fize<)=wc{)S7% zQf3w7W3p+s*-g1o`>f0E!8z*A8c_p0_WmbmNgiCNJD9cY3U@0PICY|f$4Rp>lD{_- zRRzoSgG_bp=gt4>fRu|+nnv(V%h5<;o|d9K*4GEFe4aPyyj?1*q#=bzywhD6#m_DjDG9xuU({wIAZ2%@S2&qJZSt%^V~bmeggnNH0SR;(u43**4h}n5GHMp zUSEw}lk6k*T&n)gU6lwtlw4t8Ugr4Cbg}=n$@0Z!_kq8{^No$(`1vU?0_!s_OJ)|F!mRK{6x^Bzb(7~KiJat%mr_376X_S5a)}U> z49ok1x;HBJEYH28A)f{B+(9jP@b&|aTawHR)AXkelV_O&{E~MB$T!h~p11=xhR@T4 z0a7&h!kTcO5Q|;N;;c!464iJ`^8Yl`XPq?c(@Zt6T6g9s04p;|Z4Z zZWPo_(odkyNroFYw}~iyeBLzg&${DO;aWLILD!MZ{n8bGXoJ$M>$`3cbGMB3nXZm9 z^#%+H=)!P81M!F5#C_T;n_-JzqiQ-S$hSa{gIaMUAXVwuw8NjIZsUQE==NoeQx$WO@ArhXfzfPCo1=v)h7Pm5BKe=LZP>cF+$9#seiM{v52m+p7KjT7tpJF#AsYMme;o3D1zk zuPiU5(y!Sc@34qNygbr)-Ip9Z5YAJ2#sG{f z3)_nVG~xBcUr`mWt;3&*sk(#*Pb!fj36vXJudKOvZ6tsWo@v4u{=8owm}C6kN#0fY zzc9xD&n+SLM0Fmx)$s{kn?O*Mzk_5FbvaC1TU)}e3IQg&!pg{yA zA-x@Lm?VCJ1)G=~{!f2OwlYjH+yD+1-P6frJ(?$575$37n2r~7L|GpC%N-yz*nw)u z2p?e*1+M|8!wq@QKEfxse)f%TJGgeVpGaE)&s}@g?(z^JLpqpFlh>_D56+bbI_qJ_ zAF~yKPZ+Om@>J^f{Co#+3XWQK{TIO6INqIkCr1oEWcRq%WVAlpX&vq-0-i6nJXKt} zP2%;3)e8|mn_YUw@z_coM;z9_VE+$euAKzUqhvuaJRQ-wWownlTU{4z6^ATDU1gPG zkt$>WXUxz)K^PY{uB36U*#sluT1b;zMVW{xH9i^6t`$f(mZVj;akMc1c5~wS57gy- z^!1+g^^(neM9Dd77h66Bp#ypT!-M4BwH!3W464qM1#Y{UQ*AwSDSrljD{tjR8~`Q8?L$b2 zKcy-diJu69q&9BsgL$9B_!|?$XHOL&kZ~r%jKPHNKDyTv>T3mH1ZObpU;7HUQ}ZrN zEdDd$b0o|_;3vU~;{Wmt`eBQgMBn?0he_Xi2Wg9sbe?z&*3X$zjJ^~Mb)tlFVTvoC z4)S~@awofw-d@R#fE&k@{9`Fr0a4oXB4*6Az0L9>N0z2=21ggoY~Bi5JZcNJJn2i{ z?RN#?#CBmo6Dk03xwnm%sNBvzNZw^l4E}z+-x(d;hNIm+Q|r14J8DaVeoEv2{kzJl z%MSO>oYLA2-bF@v@%j04X|YLvw$>l=MxkHXFHiVM+p1KvQ9ObHuY(iRH(u#!CX1HE z|E`!B>=6YXl9Q97HLL)Gk2s!yAwk?|gE4obpeP0$MX?>|;QB?)=T-}T^un77!^$fX zRitXBN8Dy)P@mpsNuh6ruf5OoDizef9n{4rwe%Osz1lI=K1wVs6!k8|sInYE@g>M= z((5d*BMisYUv@Hw{@sFh{!_M@o-A2MT|x)&?ySpTVnKFx3bnCY*EW)EvVJD24+u>! zc3}_^f4>I;&S*})(6kH!}^!&$WEI0q-nG@Q?V}`z9&g!je%Kk*$xkdG}tlPLP?$qOeNER6f={lg&esaN!c6 zI{)ity~r{s(+m5COM%F#%S{%9o(gBdgq116O;fl2Q_*+07VTgV+fr3ua1z{RJ~46D zgk2W!$jl76YF00aYCVUhI-3uQn9B&<}JlOSGA&9BaI)qoiw-OAaGsKqnH1kZepI zSd|f5&`E53gCgM%%S*GTsEz>Pxa0!f35OzHa$#E6?w2TCFRG zbksZ*h}`P8C6hHYUt2GY^6~HgE0Y&DZn_?Wk&%&qdY8+FrQR*HVG3ub9g( zB#o8aq!_)w^3kPCcJqrCH~5hhu5KuG_qwJQ*BeT>N8~p*-@=pUykCPG(emj=4!p7= z?$WCiT`jfr3Odo1pTvsOimr93B$^v1iVK<`7#mMlheTTa^zxUsnKDNJJv8dbKAGE^ zO*hfbiD0zM#CrR}-4>Vh@_GOrZ?l;Gk_O-;_Th0Qx!TRvU7`MartxORY7nS9>L9WH zv`~!SwP+RZczJ89cik?E6oFZIv|Duoo$Z4jIF3hq2OAmoWTU?>MH!3-5n8$oH%_05 z6D|oPWYN29Y|2zp$D~OfaiL9wny8f&mS6o!TaZ*~M=p`D4bdVm5(QSSDsXc%>n4W> z-H_akxuIFVTUpOjO*MPd-C$9|aTIYmzE>h|4?vndrLK6SMSG0r4P9|-6NML}dIxu3umb<`6`&|(U zp--KA&`M^v?6DpbmH>j+A<<*AZyCXDpTYQjfo<5f5KVRV0xS{RU>O0gT!b*ELIR`< zru2NEtZVfoIbv6#;1DuQ6&=eK_B)Wbl^@ABqxcKqQ1 zF^9W%@8KB+6)Bl@!#EZcHz~mb{dO$)1Sc!M-ovRx0Kp;94X}P&Q~}Xci^kf4in#K( zkhg+6n&Al=X?FR(*apJ0paH^L!U|%3-Q^6T6l2q=gOCGP-H1ns_@GiU+c<%625>0(vC`O}sh^@v}Y~FKwyA;b9O( zzFoKQwrrY4WDCOvn%NNjv zWA_>Q>k=?YB~RvgDsB-;qYtO|!6D>+|E2F-^Sb}Y&RS|ML>%h zH7%wdIa>$1U|mcg%7!RQ@RZ)bi8M+z*+u6SYWdajorTNAl*h99{*ENFb?uWNf3dZGZ(VH%=+GZCnVs)mM& ze?_*m^NjTeLcLS_6LSqZ3pCy3-^wHE@-Z$11ley>!E&00GxYJwE8JksXO}wCj8d1F z)_+HKm>)1FF1MAL%;F9c>abd9M6s(bDD@Ysi!Ne5IXc#~{bpE&uH?`bZ-iUqX*>Cu zq9WmhZGJjuRiXgYv-XF++ca8EO;~JI%N(~tcd&M`8m5SeahI$|Sw}1zu{J7ErR9IY zH5S;ZFIG{bsMPAipgo%4U$00gviteaXt)W_-y+7nP$@0cd;V_BlpdWp`>s9C937zL zA*Senf{V*&Aj{3x9fBTq+M;c8q?C`$oP*`OI>+R;NxTS%{ccz@kvkD!s77Ng#ttt; z*G0QF3xei_5?W)LF`G|{wn$_`-wv#jjNcor(x-cLB+fdPbfg_5?J1}VqHAs-qZAUJ zkqS@)fAr5e-L{Jf7VLw&5z!D4(;`RyzWmdSl+w)agE-EkmMMrNujdzxvHCzT8N5xe zMtq!%x(5~@s&`{2q}KODt>%MKJyK0HQ4)uqKlWSj<{lR#)*j{!t&a$a@eO}?qJhF* z)G&&M zaEA2)t=5%Wk3+vWm73`H_|}2~@tWJw(HeV3#nM{@zq@_>kQ`qJRVe zlE?Vj92WwFiasP#2CZ)slL5N{KZqsWHLkr{OxElam*trv<*lwmwju^ z{-q-PDHD+uPOv~7_MvQ5L0oukRq5cB#xphdtH9`_LoE=Hkg$uOH&WURDKcP7&w9$G&;eFKhjK>lfj@^yXHvL4g zaDP_3;Uf3?KHecZQf-F+5{mT^lFuS^X>+WN+C)J7VL{G7ELLNG0VAy@-x{+qY=IX5 z1`Xhvkr2d}(ag=_UofkTZHhw){05*g&4Q78xuG%iT4cinsI(nDLBeot+dZJ&t(#agFV)z(#d?y_4g{ z{Yt5PB!xf#({96rsPr8wE#?;De>Efku#y5CX(o%#6v4hF5YCXNCfE8PtzD>j94Aly zGE3vT@?G%v`Cdzi9|Q7h#;e$MHKI&p5a*}3y)Z1`+@@^DuT#o--q^ivk6hq6}fU_7m1 zs`3Yg_mkmvYx{?=%#GZa{pavj5Q+-|aKAg{@r*5*KGNi6CTIGj33&PwTgmV8UDZ1x9i&KZD*jQatH zW^2dl(~kVD_jB*P!t2zlj+YS|^k}IW+SfK z4qQy9>vRRqI5UeM*#m2vtvW7r_f+j%nYz*C!<(s$ms^&$M?x;;ga16=OYOLA$!A)b zm**O)HnwgN?aT0&2myk`bX}f3b~LU{cs~^k`{Ko)eXNvp;QW|yDu?WM2>|YGf|l2d zc3m$$dwbIXN*|f3YxS|Geo;|&_;fHW{F}rx%^|vE$ug%9WWdvS)i-<$Rll*qT0UU5 zI7XMS-jL z^ioQ=otcC?*hMC{&2AGZezS8k6mm%0Lrgp0hK2}H^475M5WP^2<~Ub@lGVwv2}=K%G&zls{Qu9Vhh^F?^o8$ExU62rf? zVeQyAj>#eGnuH;*b^q)Akr|IF%KLY*e?v}Yhy*17)L?Cu4$!(>z9ugxWvn~eU5JQ4 z9r~@o^j6QUorUoUH+a~IprO;@5o@)d?BZ!wRWG*&1*{_FQMoUCgfhz-@}7IB5t8O= zg_R>{O$nGItOCpWb4WtaY0>b`z}N4#KS;uvWw8)EKJT)xI=)cYw86dEMN_*h&nz!& zlog?YFriFT(Mmm3PFw{)6ePU&$U-<0Dl#O6hce{!nZlu19we%efdl!t>66HTzxE8} zLw{4j$dHu8?rZmdN%E5q&j+bdV@3zU{;iEjV~iRQE~BQ1M2Y=FU1=QVaQus!6B$P| zzs_8~PQq5>M`Hi#Y~O|z4{MkY+>US}8Xd_na1a6`*(*YAKH`O1l&GX}Cl4j&_di^n zV|OM{wuEEbNyoNrJL%ZA?R4_Swr$(CZQJbFzCCL`&HWE&oxNA>s;4rA@2YgtZvV;`_BIsU_fnCPt1yG!>O1h|&k{Fpsy68=!)fBPPU5pDJpJ;j;I` zn=NF=8L_h9$V|RQRA?=SC?wOmEBeRwHBeZ{ur;Sk0y(o>B=L+Ls?w<7!~lgjr~x1% zM=YasoP2%Q^+rc2lmQoL694*VYuu=@cXb$1BFs%N^kYYq_Z4);jtOb*B;zQlhbyua z&E;i7{tY)ecOeY~GG+(`L^26igiy;8f=m-7C<_zW<&6}m;`XMhJ)tqGKA!iP*~b z;JV;vnt6;moi53Qe}uzRnCnh_rcP;SnxDiHf=>zMp>yB8%JnQl9w5{2WT9(@SHEc3 zC?_>eZvi?XR#-ef;ZbtdnGn^F28oO1cc+V4ucVA;#y>LLXolJcBh0D)r&^S=BSsGX zU-8Fj?ndH;bKa^l!(?2y4>pZsYB!j=PM4Ev;_lz~?Q_&QrHJQKy(s6#&miy1ce_ z2UVEos81CZ^%yO`A@+ZoL|AJ_o0DV)4MXypbWId`I~S|6+37^J7o+_Nct(2QZyR(5 zrEIUPTYgCoObuH7aElTU-c8Dk^hnqviHuO{TZQDCd>HSt1>Px8#MQ+qULJMRC(u4w z|Io|3@all;F91Q;m}&rgnL3%!o(ICd2O${pBpPGEdey-Ebm&bWZU~gbW~f~#y#$?#2E1r^-08| z9yuU>vBp8F<U_609t~H|fw^s~+nR*UuTW z20;gh3X3yW#$d>Y&~YFda-@G4K7u}R$0WWAFJUh^uDTKQ_riL*6o>@uRTgBhfBj*< zNw>qYL9&g*WZG@!y>5VtuBee6#4$!*?IeT+(biD=Q?(cATDzA@*+6dq;y`^KUSLwZ zIoMxdBfuz|v245gK%0=vEW0E|>=KdWirctZa#xu_oHbTa`g^ih#@!xncbpWgBu+#r zgd;I|W{}FKQL$U?(Ro}IuvAz8q6Znf$8T#QNJ&%h93R+#QDd}p&DXu)Kuk1{qK@`Z z&=nBQ>h)=as-STpWAFgxygd?+!96+uj>55Y6*la4%O zT}j#!k2CPUz$GX&?^v+x;mQ{f+rV>ImVg5_;;=&WeaLB*<4P*#uI|EOJ8X0zP z{ZcQ7Lm=-!G~idh=2&}8p7tO-;`)c#7N?NLLi?=q!he0YIDo#Xg?4WtwRAxE0Rw<) zY!6nB{Q`2DOnmu0Z23L1Q?FM+!t`Z(B~f92)VzHKjo3n+CG)Fv-e8BkYg!ijB zNFTxlG~AGDE}~1Ph}V;*9Q7gSG!<&i=L?o?X*`-fW-uJcUc7{7PB>{q1-i5B2fPM-lV% zGjK?G^Rr(bGd?{KdATaUXX5*Mkcr1o>ALC_ukPnObv#@q|`9$#L;o!Yz z{vwCSpuzvPr}lg5+WU{!xb;m=E9rGxF95^(-K5C=HNaftpkCy=ur(V{x%GHDxLB&6 zhq1OpRsD6)$^+=QYX2p0VROePz^1F`TX0(L-GL*}iuhLgt>E2O=O{Bhv$?C;{xcr_ z_4v>^IJ5aUQhQzNaH+*^xBI+eE*pJ#C}S^<@A1)FXwU1Gw)xO%uFvyhb=B|v)BKV* zj#RmZ^C3vh=@}<)=zSqr?!Do_M#X%2%(6b3gt&fp ze_WF5DV;}q5;wKoZ0~OR^PYMa?5tD$fvD5dY+ibI2;2I3t~m)P;b~Rtdb)o0zdl8@ zYd;?BOz>WdKeVmQ18|?Q_~_$c2wc2=zHXG>oZIp!I`v%I-4x^$yy*WirL<6X#PdNyve}Q8SiX4Ya=s0*}mp8_lIHcjPew0k!)O zo&JE_liURz(RBzwFoNv;z+nD>7sMwX{TP&cywlT^_2}}pf{J}nX8VV_N{Jpj1xZHc z+5v#nTN3g5LV8Hti-y{(s)BEBVvHs*=_UzhGD}^>9hI9I%RC`mJ{}e9crt7Qjv_PK=YRqsbdWZnz>?^cf?y?}82FV3lhn7TXoFV6X$3XBA>w%J@kCwHYvj30iNPQp8# zPeRBpwFz%Y8cKM`D;|qlEUA^|!HjH>Qbjv-aaxXqtRC{?%f4sVNkK>n z9*;0pi`GosG*_>nD>24iE*IpTLG4scx1i0f&<+{&l;4Jv$4Rk{&9T)!vcjp+tWu${J(_8Uf`he+(QTxzIc))GwD2j{I+8-_H&ohf--A5M&ZX7i zF^D_%`1LPl-gYgPEpLr|!uM0Hz}ahYYSd4V4N@@>*JK#um#w+JkfT8Nw`vbYQ~QmW zm{N{k`;KH~R;f%X919Iy4(EpdU_nB%{nZk3iyJFP0mit*4#NQsOztbrLGOsani9fke85FyGIG>=Dvp ztRZ+S=kYh6y9c@!4!q=xxzSl)R^cOgEI3tbOj6|>SN>_pb8_v2yecde$He{>Ys-5@ zw#n50US|2v1ALfrZg7=Tby1I)E_RDd&-qZL!wSx_Uk6MrZY<;@_#-$5qcDJa zAZiy9u|-p=1s~kW4Fcp^E1b}RhM}B|m^BuE3n=5~tv)i=AW|}CsR_M=qpTrHF6h#O z)FcJcDGM#>1!2XZN#pcON^d+%t|3XTpk&+x<-1EI$_UlyBeh4Br1&-1UpXR%l;LRi z3MV)ddLs9q39oEaO$y2iNV%hbRcQe%;Pix+9T8ydA?xlfM9<25O;C*>J>W#*3K&Ni zVOYRPz)S^jROP&F%6syb?P!e@sRLvVH_&tt&d^h2T8PB?g^YE}}K^~MC84w2frL#(`QQ%}Lm`LAzI#fB$s)*!8aM_}Eb8mf`ci~ycn0BC=&7~N=)={24-cE3 z5gV4SLSagFOzAseiZX;QIi&$)&EeucF|x+PbmszUPB2{Jw~ymW{C=`V`Xy-?D}_|2 z`rqlXWOC_32}cbRK&vK!eZULfqlh5`RQxogpqUQInI(!sV1Dk zryA-|{M10_sDTdZ%9!JNFXAij_m zFRX-5ERQ{{j{lr7(h2W-dr)NlfAozRS8iXG@nJd9(1@Xs>FAzGVQ?^Bb)D(H<4TH))#8)A-Mum=!o8nx$`q;x=6JAyG} zYHNb>UHX~0PS$fOn`CD5y};|VEnM(Bu>2{Y#wJZk{<1U=h;2o{CA6JW)_ zb9E(b$H)Zm_zfs)BNhUla>=H?Jr29h{T?hX&Vs}qPnYFN&{|dj>3{PYhdm5EUup7UQ zqoawwBt{4oRM)uaXRTp@fT7kjww2cS9n; zdk!woe>?v$PE(k3yVIGA>+P5UaJ#9D9oFBxPh1tiG69%oc~pOXe0+O93a@(q!1a7w z{VM-h-O>I2E5jkKj?n9Ry4CyhFw*;X=v2bgEL0b#l%;QbWl`?m0eM`B4>BR&y8+-i z_JGSwB0;dG|GYhaaj-^`@75YY24@Io@80-w)*Li3uShb z4e{H*bBHhrd)@_ild9usU2`y<1enfkJ8(4^JA{RTTGhyqh6PT)Jf+HOotC9_MR9;wlDCtW!_3sJapdDK;hekp$d+Lo+T&&$@)z^Imr2_XB-a0~hzqr_ zMI@;IRz&>&5I<-EQN=>fYb3Ez)82vyhC#OL7zWs&#*+<D4k^>i$v#oqts;-h5E<{V-mQh>+;PaQeye9S(p-2rMBGVTQ zB^3!2AH_EKoU3paoQNKL=P@9~x`v4Sk(7E{vaMi9mKQ}MSx(TCD_9VoIeNPn1CFhXG3>y-GgEPi;5&hh307V`qm>hf% z^2k9U(dvp{Bv*a2ZKd1X$_h z$v{88zAs_@3WG#9Cf)M);#$IbkU*~)thEyLCB{aithgX?h(HEr!W2M{d)NmS*H9n4 zO9LJ@%Ext(mkzg*(RAx%g4q1*KWmfZ-HUY@UPcQA0B*q+(=^W{hK8udlPdgSLqN+N zv&TJ_Frh1{jHzpU62|<#MZrLlSt-!j@XwY1_G6XqXlmdT)5F1Q@helHXXy}|4eB;|vssE-<80mLm%|B-gV$#7| zhrTtTMx1^W0^%890yt$_S~J7V2(o>Ki8vNu5W*D;AoW*!c{-9P)8|NAm>to4NfTz{ zM1zw$8fm*VK^l^yLkTZHk(#NyDV$4&w#BP3#Uk}o?@KZQHJd|`qNj=636Zi@imUYj z79<&s`~skgJJzbdO{S34*bu>iz>laYVJIQ^^cxbS4RQV?OA3A_bJDFRWb3=eEO8<~MPtvfUp{43+gVvSV z| zq(mzLp^Vr)lCoew^l+YJf`y&19_-K-6$`upY7n}x9h#oYU1AyzmO+a_yc~<4yPBEc zEb<7YuHdLbD%ecH+brwoq{k4gc%e)ed+gZRqfRZjMZIE!mz*2JWqm|cI-7PB2o`)t zmRkEi#<3v2Z2uy7U9S*;6%?)p=p@ZcUTrUx>n$8Ezz%p0m541GjbZ}sZ?H}UNV!eA zy=8v8)SjCiGnVRkUOQDgLTGyM(4b9AL_;}>0jSROFF1>AdKT3_^kZ6CiS`L27l+`6 zmmnyk!zWjE(QynK3}9oHLoeh2Gyg#cMt|j!0ZX=9F8Bo8J}E|kE*Mj$-t?@&1=hUP zX>~#Q%D8rQC1p+8i5bxNSrMB$5-Wq#aE2PW_JR3;-meS4vZ<<*Q^t0L@))6C8r55E z(DXfE1xn0nvt+t1g*-bLD0NVp z^ER6dl7()TP#6 zq;=W}Rkd}MQNMw3`W`fg6sCrlMc^DtR5@1XI#JJ{kDOw=TQ(Mhal{dbUp{H zW19!-Atew22YGLj5*QE5tgT3DX+#;7_~m+-HZIY&LAbj&ad`^^ZU%^!6o!2iejhp- zoNQPZVg{d9drK$M32HoWxFG(+>|=6}5>pO$GD40%L}eh|y~cd& zZ^KckgIfc7;{m#Ppe;*GB9WfyCm66O$A(fgr?e}8FC{aE)08wiuZ0aT%}Q;=Nu>3G zOVd;kBo{v&LLr^Yxgsytad>iDphTRyY!N)mSJ_{pcsUY*95uTU=RErt%hW*a`gu{z z9H=qb#caHGyy8|Z+4d$OO-0z#W8%ESu#h$(CORSDwktS^&@~b?m8(LiZPuP8ohgyz zhD5!9EtnL#6ZJVA)H=q=Zk_m*DA~jMfRi!J_j{-6TIXRb%M9&&b!se-TWOs?EEf97 zqlWQnjM$4~@(w;#yQsf9W4^aFc*<}KlQWP^TtpDe zvQjd20aS^3!R{9^{V$+fcv#ebKY_Y!!8E(IH&`O<=FkpSGdf`=_Xv;1wQ4wg7uWQ86$RZf{0gpnbo*(dB59((b? zdbI;Dgsfui%|bhiC0z4wt@1Xg)3${*op(Pj*+DgdiFQjm94F=H2fq6^o>r&;A@&4B zZ`wjCM>&t?tidwvnrQAds5BZ}hYR8mL_sN!*GyTtPqU0pDQoW`{EW@0ygp;3m9GT2 zO`6(J(DW`cmc+~L7!kKIGv!)FkD_|;h)T=|>)00sGyk`mUB#P}*o#gl#H=w%H3kCk9gqOcxfKuKfS?NoFC1H5?+T=7;2VY51r}Pd_hDh8!vwARR zk^12w$QaObJ0ZI@c!n;HU7-9IJ5XdcJwF(wNIIp6LA@%ey8qBLB!VU2i%4YwT>5=K z94UmO#J6}VXsdn?75=MyKc~3a(q#=iU$?ij&j68zdE>#o=K31Ezel*G_qe0sNH0lPG1!Y zC+75U9H_exE+Rz9(DEDr*Eg^OduvI1x^!Q@PObelsTm?5);J>Br06gj5&&G^<4B@1#1zCz*U~22|a`^ zFh}ahygyCub}CR&+9Zyi#DxnEGJdp?I_wWR9L4@@XuV%nJCZotejz^QK1p}lU$Hkb zQ2^g}D@43dhyQIiV+u0gzu}QTpF54sUNf~6DKdgV%(>2t zLn95RjRJ1}n|J+j2V@K12?DXsu6886R?hID`#!E;Fdx3h(`E5jSGF7iCl092q zJ*Kx4Z9V>n*PQ^mzfGwKh`uaO<>r5Et$VC&HeS5GNhd==znVGh+HThIV)=P*w-cZf zf9@{;qf_%g?+)WWwY8k@-=nq%i=O9ag96!~jTH1bKCe^x-!iqn@y_3Ys~;UPbe?~w zy*J!^x!UgT4-tJGDh?3sJE9P$+g}bL32tAWVxXmd8>L-^t|5%?E$@EKrUR37;zIz(L-Gl6Q-zJ_vVBG;KE&E&TB^LzK zy2u)`x?JA|``G}#`uo$;)lZu2hk5?ZnGUwdnJv{i%7 zwf7Al#9ZB8t*6|#nT-s0_nNjY-(>>#ou8_~URE0SuVRgT{$?*dX=$lub>FsmckM|4 zU$T4Fxo*ps?PPbYYfXpO<+9}sRc zySLWRLqJ8O4y~ur>j54`_IGm2`Cuix&*SHMu6haXZ&%fBec$WBx9dazf7g3#z4Fff zJ!Xiz0P`uj@5ZowZr+#jg-ByAPnSF31^!w>uRXEGdgpcF*c+huxLz*j33(q(@bjIM zf?6xq;k$UZwp!Y&l_qek`Mv0RdgDDR^F*hqdn;Ie*#f94E?5-sIc2|wjO?_YcFoZq zVDQmUj2wmRnc`#leA6#4*pKPNV3_rM%PrkfdUoB}zhUde<9b&57xVHk@%8Vf@27Jh`Nw#0f5z)u z-MXyapg&TVd*9q%wDo^n4(tIOU;Vy+-g9GI=`DO&yxw$HQUAKzv;Segtl#DHX4m#A zp~A?175>dEmr`(Aq1o_|Z;Ag=1O z@w;2&yS$%x&zkW;1>kAsPa9w`4DHy}uxc z1Ksk_Or@ymPK ziC>4%pLil-5ST?CR}ujH+>99UuW?&SgRoNNeusuoqvRL|I4+mAQ05S0e@X5#u=0BS zVdissh$BDa3jNKctvzdNt@HEzdm23SnTQCO)Y^uI10#6%iTvhE3IJ~*i**E9U0zg;q zn*Cyw!r{4z;Ca6EWaoim@jvC{AC;LRSyYE^)t8u7J<OAGLkc5lG%5@X1-SrD1&aZl4ThXDoD?Kn(1g~4kr)CD z?a$t2=QV_gaiV_j&&$ux_0Pxl%l1=cXUR;6A*uR>lNuK~Q-I+sF5KW695kTCgDe&- zbr>AlD`iqK@ORWu5Tk6sU0L2)kVR#gb>O~61CHlX7?P+@C8b!GbVe=wyKv4ww>`}M07={(%LWhqA|P%5EA@mHXe^`}bX4O6 zf|eoBaS_`ABOt;exaorkq!d767hOS75#nWpv~04Rq6BnEzIw=%10Atw=;iob89@t- z;#n!H06rN$?BHV^Aq~Ems4p6ppOUob6fOH^9nFb(P#(qNdrf~2uXE`HD- zFqBg0MNSO!r$@yt>9h#VEagG10hNh^>BL_`+0F6k!>neGhX=!syd{9qmOX1|W!Mn= z6OX8U4;+<7te!;1Zpp(hOSOzcP1x4ReL;U^ahHRf9M$NnqSdpI5QO4 z6f1x%5=OFqR=ccCC~*M2#g6!jw)pwdrhBAcQ0M**v_!II4kJu(=2Ojt+i=byUY&_l zsEj90pIRX#v{n=g6Pk_-LwA|W;Pdq=YS3&jQ=$ekz-IK|P1pYG*;6F09qHm|5Q?;R z7`Je-5IpuLuBcQHYyT`8CoY5TbF&r|tn$pUPC&pq+aNEfo$1FitKj;P(kq z*%4k0ibf^P6k``$07P>~0B+{|MzTU-e16uy*hJ>I3W71~be z=Z@x0BA!aj)H?7f%gbDozuie(5kg)KWl42Yrl|*B>`9*}3uk)t*%KdaNR5XEByY_M zlL|rfkY9X07aYLW@8l=tf}nj`@hetn<$)<{>R2`}BWB+uCqvMkLk(S=%Z;w@0=-$xNxAK0}>JOXQ4?1ei zbC4ZbfS&h_!&SzgE1xN`vBmfDZda|U)yFX8b}(vi@F&2~YW}U$M+|&q1+`!FPTd;s z`N8}1rVZxX4BwuUF3;UxsojFT_0!B5U4G`1z={WO8DIXUr&`n4-ItozeG~Av#l7?7 zXE~Sc*K0&r9(7f?9>sclpVN+FNS2H3Oy*o}pNgmNzBan@Zd7ukkolDxH^a)>taiF~ zNcEUxZ4N-L_C`+jdxf)-_qEQ=x?o`U`1?E4Lp1n9ujlm(aB6SEos#_|g}27svb!^X zKh7%V6)fgk7T_rR56jk3da}6cGw)@shQmSi73wt@PhfjyOt9Lje;58}d}eEj*7wR! zOY`AD&DVF9UFg$x%`pGNvj}7i?jJJ|1G(0u1a!deZfA%4j`@Ru>rd>B&whrk6Q}pr za?7zhAiMp3sR*>cqt9`=gI(JHc$Ho3BIP*QrD* z#tgtm3zO7+>+*6pajUiNNKE`7L^%9skU^pW{RVKzvN+M?5KI;Wkc(j9v4liOg=v0qQ1e5G-{zfr z=VZN`Kc6Q8y+6j}ER`H~^n_^GH|JM+yH^{kK|wdj!hkoaCM;k0GEqj!0GeMYck#r4nI1r?~a+5}f)I?OEhS>2;~xQysen3&+4i z10P32qbxRC7FJbW9P=^-xG?WJ)!>rq<%rUVJ;1Yu#yEw$72PRVmM9U+tVLZ}uHroD zYn2EQ;Z2sCQ=bKX)YVAGTa<;b1LYYeUxDW@gAi)(m*2TxnV@j_dIRBaa!Qa> z|F}n^DPl0}-*I4``6la_KH)i81nA+PAp&Dj$Jt zCx+wb1^NRe=$7`N9GX3t_KeHnmFo*L%BGZ!WNaqtk2WUHd&0oqP2s+>hAtfEvm#P!yKWD7zMzqM36 zZk81%#etDY4nkSblp7a@OSYvu6hV3)f|C9i;Xy3y%(Nb~zX?K&jby2tkrjgK30R)G?+|lu&#X&wi@_vK%iWpI}dHy%;`T7OFgN zMyk!2&Yx|dS5B+^b^M%T_n+w*I9VBo!Wxo`VR0vVI3_gzq8{bh18#o-(lN@KFQ0Y+~zPPC6pv9C~#tYI#tX zwqYb!uQbgKC;~qMHskG90j9S;*QSZ?`M3+n_>Q#vNVEYp_S@VqE32Rk*&UCs(V5i} zUTzMd5Yx_+?3H=#msqYkR^yjx?39Q=r^`nYG>i&5W_{VZ<0*eHjCs_aHAV6`R%3~r z!ozf(;WwQ)k(U*n4#eHNtW=E)ec{^d_AM(2<9n5|kfUHCu++vM0WKAVfZ9E;&ZvIG zG>_RtpRdX#|DLCAbk<@TeM$_NU{fn~b#+I@yB_7o@?`C$9{5TQH3geBhvS;R2R8ft zMJG_POmm!*`FDn=uYT7|6c|VX9zbfknzp)~u$obuRztz2TX@cG zI`~kK>7Qb+l~ZT%T72&$R@8)Y=xPTs8KcXF$*r;EG|@pupQ%<{)rp>Or4?d2)B3BX zcoWu0r1n>_)udgc?CeWWMezrGH#S>$f;va0Y{JWS%Dmb#-Rxef>+1E|R=*PlAzS`M zTZmqb7$B=dr3U+$8=g}x{>tbQK2YC9);rbOBc|CTsVh0ebi*}AtUOAOs@}9|UqvhB zAKa>-i!)SLFQq;P?ur8Sp5kdNlwe%Q9N(qdy#4v^s9PL1* zI_x$BB5X;{v?P$SZ;Lft1Yeg}ik8wG5li7>U8Pu7Fa9GwWCUF&$txmmZ!frUlxP*s z5TGHO)3+5YDSIKdH7p+_so2k|FJ!sEiYW+ZXqPSif$ZLlN(iirwc{tjo|g~`bV{Uz zXk*Zkj{~)vH?XU8Z`Zd?#^eE`d-QE0GZfwgnI<&BP2r|&6FU-KFE2-uMq!oZ3a^Wo z!o(|VtF1kvk~Kgbyxi%D4tK0-nlvDi01*C>DE4WR$Jd38^h8SXkd1LB7RSL-H&7HF zCZ!k&JqAmbw~?`S0wEZ73TV;3j*r`#4CjM93c`^9LwkjyCEZ7ge>PTHQXwu^Amy+a z>5ft&4YL52*glZG*&_{GLe0uplemq~+jW0s3%+ z`wgke`b5S$=n;;N_``_s>NIz=jX^o)EgDX_+0t=f@Ahe18oA(@!nIf*c|`0;k6eO% zqi}Zm!$Um+wDivqgC3@XGk)`**gHOlyrz)ra}pfA*_8G2L>10K^4vq&beYax?Oj3d zMh0&|{jA@xB$mu_rj~DW0p1oe0qZrHI0w7jHGjPgz1)1*LFR_hPl7x6dQg0G`H27g0~A_gPn6&ob98M{8(*eAr?LM)>XUPC0KR6#+?_5SwheA(=Ti%V>Ae^;x;-x;@wvS%H&im60N2>RD+K~N zZ>MjFy}kULw@-cL0L;Q#zsIloC#ljYr>Wr61h~sfKvl_ioxGcPcm6KZ!3X{2dL@R4 zRXmKJOb#Bc4H*Agom!;xjV|Eo*nRS5E9SGQjg#9@-O?s;mZfKp)n~Du4k`3 zLwi?>Hr%})vra6z9;EHR#KNOc$X(Amqt(@eOBrCkkbA}ZiJuQ!3~ir_uGMBqh+5eh zN4&&NV?{c|w*`*@Rfh+Amd^a!ovGwptc8>L=FKMU8}xrS;vQ(M3RE;8AS}ZF_}j?< zMVhcKXsQ`C7Zon*89hY{%cQ)C(%h25k_%ECqvXrw5<3MvzU z3dB)ap(IK7_CQ4sy&lh)XmsMzfL8hws8*nK=UyT^!Yi@Z3kf>|I?gaI+du zX-_|o_`g4Xc7M8B6E;Fhj3Q9ps3nd8_Y)SBf{K4%(8O$I!zFF6!)U0%2YlUCBF^}b zp?xvh2KQg*Zmcu&5CVy6S?6yO3&HO|of6coF3%-n!tX-4vGPadM_*^hJq$`*-fEHA`m_!ZV^p$#8ZN-@3n{56AxK(eukIi;ljHs6tINtWvmBl);Z{A<=<* za}_Y0AxPS&k6|v+bisfH{&o{1{ry|?`AM4Gln?DHspdG!1`BfkNleKEFwWpvQ+~X$ z)j-U#PT({uD%?;;;)C^x{csVKQ^#?=PdaSL&8U8TCqXfiFi_kP8<^xcj_m zC>>rfgHfq0`za)VRLF}gE?u?_?YshE?8=XSZ)Ztd!K>iCCk(L(8HxLzfV5qYsi|=E zCgx1Y?zd=tXtcN&sWR3*0KsYQ7%0J|k zQl4DI*@h4vm@TmhVD2(TurApK;*O6n8chU7DlN!Uy#Mz+Ml!}LfRC$HD(K z(Qfl7z#2c`$@)VRoI^6_lG6#sF3WY5i77tE0A;D9)LFbAz5&l;!#kiaRVxp}UNO9b zxqa`wrO*xyla$IeFSzbC08e1hl!~z}|8$t845vU)v7ABZ=dgUOmnbh#&@5vWt5NKq zr$Kry!UD&39!o|PfMZNSkjY^crAoXVk!@syzk-!1+5spSYvj$a+?`CiF%`wfp%bZB zg|m06!(nmL0^1rnqeWw9!b39)CW>DO=fyXUN)kk5FgpvxIkXpj_XM3a4gr^HWFj$0 zkpe;I_0cirC0n>jBFZT@nNs^ecYZ@})G#fnsGR!ae4d*DfKO4|nlkVoUM9Dhp1&hL zGY~UTDM}N%j2pY8O=+83X8f>#)YVoj}Jx&V(qiGomo0m{OfvTZUlugn|VYpLVDQZK3M3ycp#MSsTPY2%p27F7tYk)6xP?TB^R7z~pnxx=Yn(`&W(rk1Y0eF7y|ekicdbYp-P*^fs5q}AMqFG;e)T^;rcUCK0LiRyq54pR?`5Vu8f0>qIQ zh!aJC6tsl#6xIXOHA&^5)V+tFWjtJBWy4|&D8WWSx8@~epo-cUU`_1~%I9KPqDlol zSAoLjMWx0^@=wq_4CyO^TIvTA2WN_fyX%tgn$mvNQgmRJ$EX+?6Y;PE;lhYxp;5cX zD@foUjzFERl0WiD7>d9Q^^i8-SGuCa6i&><%Fr={%S^@}I7V^sBwT_JHvdJb7vAX$ zsL2E4&jrgt1Lp|eA2N(MxldMh-s|J#RLDD?hnhwoXhJIyI~+kr_0H140SjOeuI9v| zNR=lST`>W6$Ur%OT9#I;Ke(XUGcD6#+wO<15Rts>%o-vI0O_doC8DIohwP$+XcGm- zWvLcLp75vUGz>5w%ypaZQE|l#>wlOH}e+X%$d77yEAw5*RP56 zppnZd3j1*Sha!7Cq16B@$4p2iJVOMc;OzqIDd7OC8thp)EvuTT;O`>QYAd(c=e%qK^16Mv>>%R2;|5peEY2*4S%vKeRDX>}B`nykY9q*Y0#)#jaLEC@`r>ZmaK_GN zo)uxdn2Y|33{xKE#+F4=$KrIS)A29gYvEeB?#@<-!;d@N1!ntI1e_A`zdR-zLBeQ3 zW0_Y6_D#{Mr~4Erj&a>}cKs>SQbpfQ#4q*t1q#B!VaPo(&>F(SKUh}~Gbgc|H5}yF zs{K7JhZQMl0VG>RdXnAKu^}oy&hqn-60utxIhrjjTmnZhz(qDJQM7ZJg`+65Mir_DM1p`N-%_3Z7d=_ z-AI!X-)yHvkJU1VQ3p2{JVe6Kcd!xs|73e?a2BBlB zu}l3Z615|QqglkbY%J86TukhgxLY7#S-Gkjt(Esg`N`z5`%3=FCE@f*pVI1zJbR@W zH%%sC@ICK!_a>V38y|8(8VR4Dn!(N2abTu05kzv{oc3vZs|VWtHyH!bu6=*Szaer8 zyuCtA05VZrC`BP(!uYNM8Za{OW*S#nRbPm}Aq4Z4`IrMswy+t(RVt1kd-zrO?pZaL zyCUVpT=pyo4Fv&R0L~O}DiGD_z#yWc_V{iM?O(>hG*n?c5gufy9s2s9+ZcTO3O{g@ zfIqk-#VKF=TnmGAs2_{Ku@Z;Xy%afB^v$jwD4YWEa3VdN>Q~2EH?L;Z$~UlWCI>H{ zT9UI>gjt|@4f9smBqp0NS{ggv$O&BtnV9@gJp2g{6{WO{bHnX*+=S-XxE zZQsJRKN3m&wpSm1g?JZNW{?fKMMzm9I-$gWbZT6@h2>RcHx6vr){T%b0JN6U@rAVo0kgy4wwt*zTQDy#0Y&{$GDTjS z#q3=;Fpn5oo&0b8x~=erM8^&&DHyS#8BTky|BB0x0ZO7DFek3$IUPG8u5nHsyV}Up z2&wr{h9~NdvPk#Qby}<|FanD#NUO8-j;0)0Fn#Iq%n!t!GZ3vQPq0S<1%i#r01Udv z#7NGdoNY`xZw{@tAm94HDaT+pf_`CB6A3<6dV zwnIMOP!gZGIvpyDeFe$?4D zKIs1}O!Os{BbM04Oan#j^+> zQJAqT?qG!rXi(3{3N)a)-=R8B$*IrCtLGNRz%&;?aQ!$-8*M2!oAA4!2DoOuU1Z!b z$+D`R$G8jc)XI6=1eRd1Ez#Z^)OBjWH4}CRJYBj#OBD$?z?;aT)XI%WdnA)_MsQZV ztoqxfkL(~FDQ+6ei~?k(fE&Q8zg74rUMyS1{49mmWNH!XB5SmOQk2OOR@MtpjH0JOQtEXz|35Z z{$5lNs{wygv-@_gi)MxmI{Cf-XSy4Mr5CA2$HD9@_yiB>tb55_3|+R{%h!SE zpNa-OX|-$qp)b+*+taq18vOko&hEyV&DQ~~R9`)6qE_|Vm)@o85qjF(jN9R@>l6Z> z!JdrwkhnMHvE`IXz@LAn5u?g-RGL45qs_K{=hzlF-GK8Tnzr=0)Gm?bZPWeYH=}re z&BVe>ov8+{{;+j)6k8K^y~WEfFz9gN`o8rjA!b)*@hL1fKO%X*NuQ*8yH-EfjC@O# zqw)UvQ5C2mpRMmBOHrP4lcV?ExHr|gu%pt(jH_3}&X>&tEI+==++VyFc5LX;mv7w2 zTw;l5w%@DzI=#hSTYr?k{;9#d@KF7tu?Xa0t^NGn0GH#WeCYKw#s7Ir^JMpSsvUBB zIJg+e^Tz-7O7r^u!&2Dm%jzNYuW4PzCJxAP@B`TOWo>+5v0Gd@H3ag-5XZL()r_q3{X0<=Cn5agPgejdp`@wy(*d|`AO z?JZS#bYR-F#rj$uwk5jp+Rt&iC-c9xJn_?<3Nid3sPn0R*SW{?;zMdaSzn-T_T_qg zZny@z-j4)6%wM6>w@tpAAa2Zmx7`6<5m$@&dK@h;+X{T!Y9I&LBkAS5EQj7G2sa)- z_QGa6eQXBJHg`VjejXhO?|)vx$yp8spD4JlHDlVgW`D9C7_O%`Q$(!-s8?HCd@MzDi=%&5QOALfEyuSs9uyq7#Y}8bKrP?&xc&#IS z*vlX2I4yMJz1|J!&4hKf>`hkmG>pl=R@tpf&DQAt{TZIcdD(`>MQf>rN0G0 za>(Y{mg91m<9eHV+qPYeAnI-jnK;LmrRwzTYbAdQ%x>A)dm}yiGy$!Qi6>|-)S@l*iV&AC-}`%9(ZTz`4-1wCDEJPSl`1rzJEH98w<_cnp_-vAzPx{_z%! z?HQr*D6;~-$>F;(_d?OUmM6PRSZdR`Ob~1-ONIYlRn^wEc1uo5TCHs6L~AK;UsF-j z+nAO8;r3i$D7m-UzkfRwk|O4!w4rQWQNP-bzIkYu3euS=RgaPKZ$`_?dJ!v*y9rrk zOXCTmWmnEg-t~l1Y&1V7O#Do#I3j{u7__WZ>+X%?AJxfTg8tWMt2&uDZ(aO3@XvPv z`TC-foT3|*q?M$bm!=yZGt{@2p{6$g0sD_p^R?wN%>@qvvWxbAMop#?7(9@y?qiQ; zg1a--VbN@%#J-l~t8m&Zna|q1#cm-Di!Z&wYV%?)AAty;Q9Ra+AluL;`|Hmb6C_kH z53()qq44buI>AqVeSA3r=LdZGpCdeJIXN-h@XG^uT}m|Lp}U(~_qN-Oa3dcM*&g*X zZ{A%q_g*!94m_GUxGlS-AnnAq)AW#ywbmIy!ui{Kmnm>KhTnq)+x|YMm12f zEd$#0_fk1NLQz7zIVnXs#DISsDqa};G1B;$0CVZSIB)5AoiQ=K;=S`PbxaFQ6~buB zGKiS(P-x6)6rrpB63Q`<;>MdGJ63Oh1bx+Ygrb zRlk1tvAI@~0%HWqGYLiuRi!!sa|uKQ3K>pI*7;lv$=ypQ2@g&;CvBB|iv}Tl!URcb zDQ9jf{8c&h6=($M)Pmnl($R6^)YH2N#3RqxUgBi9h>%OQOt_L{Xc~R)+A+aXefH`T z>?o5%7C(kj-BTel4JLu_eFiuq@F}X2i+1 z4$cx7Cdf=~+*-4f6DKOK(|E4ez^S4AsH-Z<%xPb*FtWyRyA;Kp5W|85C!`78ic-3-a-&V6Uh5}hsRjX03e#>X^`K`V0weWCb>iaxNY1D{LiGi`=-j-3ddWKWG{NQLh_ zESa@5$hrd8>6RF|e-(jwv^;eOK}T!BI=gSe!B5CNkljaFafL>BNpCgUzAW|VNXgO? zUK}_m@0|j+D)LXsrl-%dvR9|cIz6j;2*ANVubQotTQy{2Zdv~=@;C(j;UvDWk%kl9 zUg!DFXJs~T7{1f@csiLavOS=cjl)Etcu9J<@j=*I7Fsd_TGHh=)Jga%QN0*w=ICt_;$RL8*Cqz`r5FdjAMlc*4HDCs7k5%9 z+{4VTqrH<+DDZUv4E5`MlY@2mp`r6vPOq`VT|uBQ>Rc@K-7@5zGl-I5=n_=|>?<&= z7KQGw*ys<`=*lFsC3N^#a0dqtJF#s)wha>0x4jVAa)k|Xt^4-4M>1)+1RuC+mnya{uhhF24+h(|3l`j~Uc2sxf{SZ~z`sYq!b16vv!A zBzOVJfD1MFNZ+7~e44EPU`jK8P&3}tri~Py(K)JV4^x7-uVL1NDgg#PNJO}xDPj_$Xk7y6;UxjSbmd2O8SjWPEAm6PhNBkYOuyt@Ee*=ETsT! zY*OQHDmrD2HM7Pb-r*T=Em&JKZXosfT#vI{>2<-w>N*eXT)ogWi0|;Q=_$ zLNfQ)OyNME42{wJ>CS{$wcrP)r~MU6{S_(4E>TF}ItIOZtr4^IDZ1xwqOC)VQ{3t9 ziH|$e-2x_4`Rj>5{eF&>RfYC5*x8b6D*=v$ZZzLH$Fq=W)0}m$DH_69@uZcdVO(d zgLvJ&EJqYt|1Li5G21#b{p(F^N`xCwFZ4Ng5xSM)|3;d8n7R;ohTw*xat97}BNg=t zG{5h;z$hgBt`TpBFVR7?Uahy%o@D0$KvXB z=PT9RYa>oQm_Hf+)1(Y~sKOE#nL_Q$WLFl@R@0~~r-?al{d?lq5#!saxs!G9%03uH zYPUSS4dQL2#r*Y6CKug8uqU8c=z6YVkxQ@LFB=z~Td-z17NY8O;hj;{hdNq=IfDod z=?Xd(L3|P9d1mU60P>sLyE9N9x|Q}ulQ%As!U?UUXTFt|(0AU-fhWU%x|x%5aUWU# zZgX_=tbVa$D*9$Dvt#}wj~Pu^4e9y7PV1Q_<~JB2E!5JiwNC~e8##c%-73M7;j@2*3pKRFA95KoNCiI5|3-`;_xFLU04ws} z3Gz-H%FxwAFpW_+smU2qcHX$`)yZ$v#CJk@q^NjGOz}p(h;<{k($4SN{i$Fu?P(^1 zE%>8??pc9}H~0RSaooUdH(WSF^%r?PMROUsVWap}Q)9t>+sp3UQl%<&*B8mMk)x zzMoTdk@spo<{K|_$EoxPv>NZ5)d!FrF$`YS7yeVC^47cCPs%!7{3Q+hi(3p!+bb^T z$wMW72|Pw@y#XN8vvGen=sbRdFxQ-?oq?={pM`d%(|zXvBlT*X=ka!^P%rc7>f>G> zekChIrRnYcXYv={*P`F0_eYCers9{c$E&3ZMo;3@-nQ}O7ZMV43QK?CBY1Jm)&)HD+%C8=yKeIQS&^qm>sH^m@1~_Uh864cv7R29^R=;7ET3T=yajH;;5C5MxoV;_`Al>U3$2Ffv+O|D6o8Il0Z%0qO8vLHY zo=?+)?ZDA)cfvK?SCiFQ>{R#1`LeFZ;e2g_^M?487h)@7vS#Ym=ks*PWA((vIRgz5 zfAisDL$VndP42VF&Z&*R9%Q6E2AS^m<7BnnYma0Ttwq*-F^4yE&1-#w!DpA`g1`Uu zG<}WRiC``Ek1x1Z*E$cxd) z)pXvCe^%S*$#DbqFEiQ8F%?*tc%f?ikLx8fpC2#&aLqeBNZMO&juWXZ-AtD5Cq-Uc z{s(CMZ=|gqWc9PpYdB9IXi9eVPzT3@do8|RcYoBHtvNfd?~CP;Y>3=Xks$Scw(ngW z0$1-wRXn-r>aR9^IR(DpO}}9F(g%C2oT@V=Kk%DTxw^45!#cstx#ADJjB)lL9f@|G zG=!$V9oq~-z3q3{5poedTOXm}E!-LZ8Zx`Ari@`XhE_6XMOnaPifO-Ij3tU9vh_B) zZ_k&u(|L__I2c}@UIjUw{|n#DL3L{^0)+pl^+Qm?yadSoMJ$cUN@ENHX@4XK?Ys@3 ze}57;R2LEtp;98lj41AROdFRQ>c0~pBM*IiM+W1GOVe6*t+s}HaIL;WP--|E;+EIZ z$>vbd)4O)?VivogtFi*iuGr`ZExYKG_*GhCQCgO%>iYB-*<|@&hX`}7L2<%M*;*AX z?wQf#VUcx-RUaoYDr!_%;{#+eU0*H^X&MppTec<&%#Yt8hX&>Ct`a!$R-s4c&Xuak z2T}#9Z@CIKLI_K`_cQId8qv1e+qu1N@Q?rLilF0$6Sv}kfKbx>f4o=w5ELHp@Wxxp z&b`Icm_g$=yHK=81xq)E2;#IImH_1;InszAwS_;nO~M;Obu!@c$*0 z+AM%4)EcXh+JWznIlgV})^VX@p8wkmUFvD)py_uv}?LL=qegdl%v0m&rWK&@ilhUQ#t(azR z)mq7<5-oH5(J#`d373ep z$h5Ta@$r_fQhXR!Ydsr(IV!(S*%X_uIifb1iEWJ*tD57>VK~P7UCIH(-)~wf7J3oL zYS1*H4n&}D-Io0l?^3K-U}MlMm0iAv%xzoxgP35>t(Wd7+O>Qm*#9)Kl(2#!p;^W{ z(;22VV^`+7cxKBc?w)a?%8rsf=G{~Z#Q~R`ankI8MjA)W;U5^Waau0>@<;{zRoPLl z#&j}fV~_%9jvRcHs+a+|^5y13-kv_V7g*&njt41(jV!$!=Z%S?2luyPO>os{R*W^} zvA1rmrLA%(@Z*Rhv5s@w9}9;gBuc0C`b4)BxJ`6AIk^=5L)yf@?Yo2uwV#~GgoAYy ztF3}H=P*dRr0DdVBja~#y%v<4c2ljEvO1CUEK z$Z~U49GgOf>qmf8^16HXJd+AtyMS9HcQ%K=H*?S0R3C8{L|~9DIAQs#y$af1t?;h@ z_kR{bwg)ncqnSR$`XC}-5#B3)p3i(XzT3Y+3Y#NMv$?9n7El{fy<4`|!tZz^CF2+n z^~ZJ<9VijGVuKZ}w%Omu9GkceOUhCx%A2&~hy?rRmNWpLle|qUbjnuVtC-Y^#s&I{ zto8tjXt)W)#9al1ii~#aps^Oy_}=U;W@&zxr`Z`C~aE1f7dZQ8+9Hy-_n%2vWy_nI!3I-!Zuf~ry~YTFVF`y&A!J4_33AR!;(JSDDzc5e+(F3s z9ot}TjBdGK^s>~#kc@u%lFCHO%NIx%G*zUaL!@hdGV;HXgKxwEN-(aLc=c^CZ);Rx8*b{ ziw43WJ86nrp0)V*i4IV^b4JP2M`Szm|1ur1VWrS5)TauU6DbzsHhbmIjZS*2cQi^+6Y-0Ls zqCO6cJSe?znSONcwh^WQ!tB=Cx-fY(Lqm&PKLZyuhx0h~-eto}BtCIk#+7tK!J0**ewpyJ zaarOufqFWNfqr%ouBo&5;&G$C9mp^iU+gF++NcE@w~fmf^`g6A2>%tVU|kNQxY{C# zsT5igBLxP8z?=9dzXt5q3IIKsh>&cMLf8kLS0SSem`Gb#1sT>W&b&8 z`j*Lb@t5PWaxIY>JK#5pg+C{s_2zXHrQW3l(x{n%yu!P%AE|N$;s(-+Q|f{bD;h4P zth^UG!JhcAvWh6||4q906gurgV)ut101N#?(q~Wv3M|_aj5Lwq63t5CX1_9&vL=45 z1ZDhfx77`GYBO~cmQ-hAaQ6WF7bkhcHKgtj$i@o__9NK60RXl4Tj4?94Lq+r^7j~F zsvf}%$(s%Hp?XQOHed3GP^`c+!Q{dHiO@mfe*IE5_>jq@JPG;~LUrte8>2*9wjCEs|3t?Qw$@;+dGDz4Qh@ z7z95O`|ShA46fSV`eezPz9pukP&I>M(TRUC=L|q?x1AzwN$yP##qs^5@DV%0gNCJI zhLQrtPLh4UU+KY?uW^-^l)f7W>_7ZVUgi&Ldh_C-{H8(p0OI&Zq#@5B_zWa}Qv-Z37}|=kjeuGqB=C7`RYE8>$ctD5H?NDQO^`A}6I&XxZ4X*d~PVOtT{XhqEP| z4E?bS>Pq9xvn77MYk8$<1I5|csF`y%>>HSxvhDKLa%Nn^*apv8>Vgjz|2$y?9=%q-z9xH0=&Z&^UF_U?2-XH3eBe+SK6e80M98*K~fN9A} zHsOA8|9dUCEW6_BD9(5HOq|79qds0^Q=sXh;OlKiUv8h`x!)+naGRJnk>TGgL%ueK z;&UE*Z+}LyDUKykCo9B;$JD^pj@PM@T}HsKm8dfBb|-6@b;)$GnTOGoFwWR9#puI4 zMXNxWEU%0Oie`P80+_+16Y|4v%U$_+yDFe*+0rbDVh3X}DBuHAYs^utbU)GMw17S< zo&;a5l3-;yC>WwB(m%Cds2?2lRFjYE#m!vhO#pi0X91u0#X%U zt(w+Igz#*}#pz`#2Z~?LW^n~DGJzK27c!O7VcFQGb|@|9dKh|YFPFVi;50VB=f};d zZP5ML+HF~T+f}?}dt^8B?`j^yvrWB0sqy=x`M$%ZdHji1v=i6q!HC#IAvJkHMT0&x zu#9+EbD+Fic^k~Oa7m%v0QM+Z(G6TeO{J%}4yHfMnF&hr4KST1&5Oz4OkdU6pF8(? zup%~;eOE|d-w?1bTx;JJmYG{z6*dW$qdHKkHGCnAl`|x}5NpKeA)HCu2R%g&x02F! zAkumE+U(duhZzjB(Q^PiakstG9MgO{n~_0zi=-pzz;xZhbi$J-0PGsie#^&6SR_SD z@XL6S@-S`{{Oi&J;o&Fxsh6jTls8l^QHRQBwqPAashxXq=t8(6uXw098YC~=KyXkh zx>qO3t$-n2O0pCsZ8D>bIIlq6JBZEt2q{ z&J>|SyAi-cc?p{Zq!Kd>FW6(^L}!yDJuZFh1(%#$UA=X>xk26cgO`vicZX{31v0L+ z>~)47cQYc~c9GmtUOcE*WD|bp`5EHZCC*#9Lqn+eGNAl!ZRyHTIQ&znhiJBCtN;D_|pZ`03b+uNjZB96aQ z=`rQQ%mfe7OH9%H@B{8C3}&X}w(Jy9wPP2-yIxw5JJJEy2z>Op65$jopCY-dR_7gm z0Y;~d3%O}umJtgwBo)v3xKa>@(FtaU*TJLbEZHjoP{eqP&f=_f;%o7iEt~$92c5e` z{6B@K0eh3WKv?iX-$a;Sm$C2bm=O{w7G|#hE(1H{H5x<4>(APwC~)McTf=L<55ZfX z50cxc&suj?K!mr1pE_rS2#uli{cYEYb*bB|mMesP#;_rPea66v)+T4f^oNaZZ*eR; zLk_WS#%025SHsc$kFp){86tt|5fn@YI1{oky^oMeAib`^8{_D5nB*3=$me%cYL8*@ zC`6Lg(?&jtZhogPMo|t(*~+(}SVa``N{7>zQ` z?<)zAlfO8rbbl=D8+|}0_sG(Jv8);~MBSg%hxStZqtHPI5I;+g2NPNfKkq)%fb>A^ z>J8ulfgkXHuR*?)lfUS6bNkNa?H}A$`6EAe0FGGSh?Nul3xJU}a+pEC6K?jNv7a9? z+-(kghmG!4xp~0$NuJXO6zkLDs=o*1fmGlN7QXwlzGe~QKMW35!fbT+BlGo-b1_xK*H7p^os}XufSg#rZG_? zo@)e=?&_As_zJrB%N89N-*W_g9rgRVMSHs-_r`0BwM>#}p-R!ct%16fAp3Qq zHLw=VD|)-NjePh<{q$AI zq5hrZ-MZ3A^Kao7!$0r+pZTt%3ZslBj=c>bByuRBNIgTUq7^X$j`|&%a0wy+KLSis zfQ-zTgrpXRD^6gppfQ**0JR(lFrU*b!iOyThD7^|&NBWkLR5+kBf?!w6SQi5d3x7w z>!kb1q}%&`ivMeR^0Na%P@P1CSOnZnAPjU`R6;~QKpdxoJV+0|!9*QP(}MAZPG{d>VOaEba+G_&=`&wa_r9 zc5YD3`qRiOp-jr=e3?H(Ji5QxLD_rj2rvfd#?*p8+XmSnBkc&2KdV4{&MKj>_8#@# zs(0?-^qMe!(|hhz$cqF}(0dP%6a2&liMm7CdaF?Skx_TIh5d|vVv`U)5bvsE32Ea) z<@TxXy37BIK*QKyjGUPNaDUfFoP0sX(&G&hnC(JN5uA2Hc^3rG3_vT#Xm_78{27Vf z7f_DrQHjTm430{K1VVKyTRM-5u!F2P1)#5UP2lXLwq%O~QPP zSK3+m+75X4Z9BmpaHcpzFPM)U2%lXKnPYH87R!u1fGA+fJ?qu!S%Vw&m2EVzcD*Am zI^!TObB_;N%r_8*ZA734?9I>h^t<;r+1>M@{JQL=@9Pokg}c{+X^ezD?5aT--{Ux@ z*ASgt!NQ+z@n!)apvkZnD?2A(eIU<`WHshK4#&M)Y?jAB7AJ}V7z(9vHb^?bUB9>v>M_ryV#Jo6d<@b1}m%EDi&vVaqRN zpGscW5S-TtSC3X&W^LXA97ETayo99fsx9{~c&RE4V6W=>2PI*TC9PbPe(Q|8Nk5PK z)@w9X*;>lHsoM?jv9K)E&{mAyqsTh(3!`MO59A<1Rk`9@;ZL+mh+$%)za6D|_iVJS z^Z)e<|Xh-xVo^&NXI2`=IQmx ztt;Q@5^;Ke%5^pv5!OL?zwM#T;hO)M|3l>EO&?%vOK)elf6HXQzft`A;3LeWCR?16 zQb8wbixoiTPvBKuwc4zQ%;i>4v$uPDbuHCBjwNS@u+hO`Dm5)k2zH7D@87rq`LiB?0$Pt3EbOOAh~VhDB;JO>5YC*&#(`z!zhc@pv{5Bd_&&8LOu{H z{b!K!bFx-zXnub_;XbQkxLV?+_Cs**wR;<*P%VVl7kVmbO8u0_Q}MF-+&vavE*ZGiovCtP!f`B)svTeabpTLO6mZ*ejy+vAT;FekGf-GNFP0R;0>YFJF{bWO zZ&h`r``d=;yF&`^0{)jG^QHFp!nPKv%_v{n4kYAZg)= zL-pPIF)2z$RiP+q`bc#&G$}lwiDb`z|1>UHP^v%vWZ=lDO&ef}9XXge;pdQ5r-(ug ztwa^2gxNq3zH-S~0b{uO0Z@b<*G4-c3OC%_c z*iWqjZBkYxvwqMglqGNs+X+?7TO~;RU|P|tmZIoNHrCAO(-JOM(R`puHt%f8C^D0& zT%r1hCodvL@HwJQwY+^Izx|IL^lXQt(t-#A0)~#ANK%iL%AgDe1Ni*ovTh8ge{0Qy z77T!gO*(2Z2)q+!{-IDrrx-&Q$1FO=FqU7*c^e2OsSq@n3i>Mrx8)?JCcMTeRXmYh z4v!;;bIK)4`mEW)%riC9VM9Bc-6c~uV{&t_?2^50`scdCOyWba>(!MWa=|Xy&na5B z+pjO*FQ>0yMo!0;T!1)*Rjgp|=bj^zJn}THAhI})jx4pOp{g1S`lkb~i6{e4#6nhn z3K|z(8O^J9K~Yizttmwem8T$$4DNJP0?Xp?*_Co#+#LB_k!E!bs^gHP7z-*(c@8aZ z`9#K_mnj5no+{HMsp?7UKhC68e=N(j7f(u+nX2YNjLWS{3jmdI&FQ?QBGuji)@{Ew zWsRCrgkqJl%8U$lN4k z%vwWDCPWY~namVBCI*oZY7~8z9L_Il%X}1TAvr6fSCv_+tj!gkKsx;btdWH#ouC2bQLF>zQ3NAH z+Ax6Mf8*V68(2n8A3sCsf=eG>q$(x)xyDQxlRNwS-*TYl6~ ztwq-GC^sHNIrr=9NUEECCzq;`hn1BMTEF5PYkW?PZUc8)YKHD6284M@A2QPpXDBLpWcG6aDVR5_migZ!=D^tc&)*M6`Sp*#N zBjtNJP@O+l68n(-$%tz{lGE~HQ z+182%GSfIZX{a0Hz=n{7k0vnmeH9vZj8{dggDTa2c}OR(wPuWxm}SCa&A(*~)XwFE zj6{hyr9F(pUB6*us7rS#V5=)66^sH+F&133YgMyFZMkn8Z{D{NXI7Fp!dV0Vi%x#GKqi)wrE9efwZx;Pu8OS7eVm4;w^B8c2rLH+-A^)a zsgic?yx6bao1|KwLykZhisWo zC**TCy!wV;Q)i%#f7)eq5HZUKrY%Kr-){J^+9H)n2s3uIPdcbR+R)-q` zGRf#y*@ppE2R;@v@CJRbnFHM?_*!{Fpw9orSozTu`Lt>rg9-5h}$3zeoik%wFzi5NZHPNjib!*Wx z9tSY~*VlIa$uelG-`s$gZqS^)Hs%`lz^$!RARKj9fGw)9S_cE+?<6d<0z|1~nAz+a zlZ^Aw-i3SI4fBk~P9|<#6X#}AxNQn{Gs}Wz$O4*b`>z?rQfh9VUJv`5QYFE*6Vxt&fCxu zc!`;gLJ4;-^HLF;p1ypR`rm05)52N^FiF+-a9e1`bBb1tI}Tj>siC zg}*v)^!XR;6uTAl?-Q61Ljvya7vz6z;Uo=$=e$aoIv?UX3`scd<~v<&8AA{^VTM!y z91I^tin%-A9SOSCgA2qABZIq(J+%@Xj2D2f-$Sni+tyX(!{Q5lMy3g5#~J(UEgr`@ z+c(bZ1nn!z@eYwTK+FY_04`+gchjGq^s^u{VTMG?(5UOe6ZRZpyqg4tD)-COnMD6eudVEQY9W4n>FgPGkhbvK#;y z)4BgU(&&^qgLoua7q}9!o}xEz=Izqqk3WLMass=3Sq6)g(gx@^`^zvoYY20Xqu$6F zi^qtr&`jdSr&GS0jqG+oW=Ov9D}-g`K-`29YAC@W1bqYpkv5GmTQM4E-1r_bPuRw~ z+-!}2$<>+3Q-PQy^!MJOstq^_BMji8UyYFm%#D#}9(Dz`L(A+b?!gvq#@b5%J?CtJ z%D>P)x8ctgp0S%D+jnL(aeLBr0Z;UQS^uMMZhsq+KOK8akOtwQs}9ash9g=g{sf7u zA7nq-bM^HFV}?CoPM`_%Du+d3&XDq+0+Dk@Zg2fw2z#Z$xBYeh>O!dT9byBvTo~2) z@YD`}mpyxJ|8xodCG7m*TjhgUCV;OC%^OEdXG6dk?jzwHj8w@EEl(XXh?@!!7Zq2)8q5+29ffv-t!;acbkO5HeK4PG%X(7V``w=tN ze@;KqqTZAjQ-j5~$5rckb&Ap5D*6whl^spgkLJf;Rt&lf4W~5vm5m0~^?!715;ctG zYO9-4s@bv6X}ME0?ABsGHq9H_2FM%6G>vd_gJV~+**(UJD>ld*)I1p;gF_@Aq|ti8i2*v}Pwq%3SqOvOFMm(kwbCYX z+Ds8cl9PKro^Ldeoln>Lt@`*)e=)0kf@qoz;S@jgEOjZ4(GWidBU7>lOwlB-#B>J~3yg;5g)ztrG2hj_>t?sVso8gViMi99HKKj*ceg z>dPk(L({)y6R}^N&mo6x4>7K{$xw1DY^mK<6RA);f8R3sF51Z*>h8vd*#z)JDsktKRchdY)3US&%nd1`@rOE3)1h;eWDG9%5}yD zr`Qy)8ZO{-EgdlX`9x7%xE?uRvkrHLm70sQenQ3C#UsS~(Zj#_GL@rsDIC7s&aY$X z^EDp2=d*ruGNbppwNWw6Tj`L?vxV;d_xQz*uiG~Dmq*?6M~5qcO=d@B7awlV)7(hz z%UCi^2O%F<%5C#!H?Sg4#Jl_+s1-`o(RrJkqn3$z(hfLnF9EIp1mhaObxtkir@C9y zZTZ8K|6c~A&*NhTd9zIZ54E10tS zp0952osLQQoX)uUalP*H5hBjpO}MYu`s?NAi?iz;AoJs*#-A~4$>-Krp%)0O9tx^Fw8NP@7ufXb508X07(&uv+g-R8M2EgH$^l0whdU$hq zJ+3>yZs6c^KPp+F;s2+v++ekI*nUoAdBjCfWnGh~(|<`s|4&(00aiux^|?zZ-67qb z(xrribV!4gbhqRc5KvlbX#@nM5v7q7K~Y*kKtiM?B&6Y6F8cSwcYQ9;IP*JaPR-6f zb9QIOSBLIKi7Ly7jXZYBcZlDUri#%(oB22t(cCbb2Suq7D3IdF8o$ zCvX1J_k1@VH(I~~1(*gpL8%eM_X3!E6jSq)RS+_DSHy zrrS_UAALMB2H}^R&Hf4&j`!6))AdKV4nl)kZze6Ruj8PFY|HHIOG)rG`?%gqf1We) zUdCr}K@A=wdR@q#Z*~3G6$UNW_pkirKVkC*q`RVt`-=%%lIwilvaiLtzIK>A^mOyR zh1KJ4f?3alIx?8Yo2t8-$L|Q1FSvV{G#av#n=F9a+_LIkTmds1cXlbA2WFpXzLorO z_P>vskdRRQJ><^UVEFsVFLp&oby_u z6OLRhh%^ET)FcCl!4qr%9SqU|u7YXQ06w^l3n*WZ$lw9WsE8u?gcx{yQKBXRs<6-5 zJoA+cH-^~sgKpFSC0M2lV4X9-0r5!z{R^xUDe(D%l>!-{3Bx}p+d5*zwud1h;X*rV zQ-kK502$oM+sfIK+sfUsMoUw&y7FodZRqEf0Kz+Hr6L;4kuA~q79JBF)%OaQ zQDNs83*>sz?^Hll_r}Cqa|_EZiP0ivsp8(JwvCk-f%%$)1KpwObVP zo;~7IiwJkaV{VRW?*#9){8KmjT$#*B4^i!!df4l43FOb^2)j5Vx<1e~ZfRo*GAi*q zc01Ba`3`2whVy#(t-X99P5mL*pS5{crgzswuew3Q+Y{F4PmViHZCP1uL3do$ge)+y zy@R3yukr53e=7Fe3bU41vO9uhSj)L01tzww))UpSQrw82IA#L#^EwcF8^MmWsHJju zhCg)Y>QBJoXP9$5Dm7WaW@^tD^28@6r(f$ln^u*ynUHU-?AqJ)vj6J7$2sQDzQLPYJgNxjZ66> z$2*llO^x^EC@+)kEgg32OO&-dFx?51wSoz{(xj}}^|g3vSQXG2?UFJ2Q1BM8nf?Lt<@=#~m7vU2@0T^Ed%_0TRc2O#&1YRMq+-X3aHiV72zz$FX zC}1=dV08i8r2mDRDr)RVXdxpZ(L?*V^PQ_m1%9E2 z!cRxjWkLuqXs@!~kChg;`jg*RYNC5ztGM;4Ev|$%kqYD4E<55EDuyT&@Q3WwZLq$p z&T3G-tJIbt7wEJ1HL!aaw=yB!xH>^hIuKW}A@MWr-PRyfOQ+|z3OZ-rBkaZZ96s(TAH(ZIY`Vl8(MC_4U*_*!QW+)x;Wl<}k-Ti<=RRV@|S-yG=w= zz-T)h((D~;S4>5OhHkx$s^L`bMy>b`gOZ)-p#b^<|A!}v^;Gc+$}`v_-gCEtY1L-n zw3KI8Kg+Xirwy4MdjmDK^wVe>PS+B^E$Uo?MiQkpz*;PLa?FH=7-N}tFPs6xh|~y-+M1c;;$FS=#jjV@HNPG8iv+XG$xbkz%>m zPw<^;vWPMIsEJegovm$wu`A^LPnk4h;7KaeDpx|46sfpuIQLZUkag0$b!3`8b55JO zF8rds2|pT6g;W{lO0fgO1&!~`2Iwkosa4qX#KdO%k$1&>3H|bR6L(pm)?G;5gWJ|~ zlQ`#&d~TFDTeWlRuDdr^jz-LE;OHZhAA71t*f+hyri#!^e~?N>o4e~DQleX1zdKSyXmd! zs;>^&-WV|QOuu4J$Z1#`w2uc$ybyqA47Naf5;<8*O}&XF-;G$j_B% zwOCWxkHX?M#q+tkxC)s{2Ahu-UDL1V*m!J;h&iX5F-5Z1NYuvi#hHDU{L$W&?(tII zg?cY+g6HLHk(ujgkNKZ=MwpF#f74+<*P!z#ki^_#3k8?RcB^>spzXV}=lEQ=p;g`P zr>^02)sALy?U#3VnFvpM<69lVGv?Zgg^fmtP`N^wr8Cr6b?x^js`B0P4DLTo>do}9 z?yZH9jUPC-c-c&pEdFe+D?6%ai2h2hBuc{(Wjk{J?4NzF>*AS!HZ;U&PDDvPIf7;Y zF3^M%;DZmwX7J(#EOHn>Q}n@C7#U40Y1!dX!hua`tVYK;vWQ_E`}@oSI2t1hF<-KN z-uKDu41P^%s$U@5e4Y2YHWRFUwryrE*<`W~AIorJ?9mC<*uFzSk2W`!`QnXqGv3&z zB&fqO9MpU`@{>Uk9AR63A96G-!Ibkwz^Kby?| zN`93;TG^cK$d&7>qc)$=#VccLx%s)d81F^)omS=xTX~e5G&ZJ;YEVe_0VR|oMa!y! zxfl(UYZ9lpoYsZ=VIncZ?NzzFH^OZn%<95z3tw60NOHDX5X$EN(nPLX+n#pMRa2jC zy`TRGzD&|T#VZsvOEFJ(BO6VOG|e$6blF%t^gwTga8!~S*|Vl?M(!}ef+4gHxhZ;=len}l`uR^3yNh4QzRa}ri(ECNR6viV?yvywj0pRQ ztA6iBQLmWl_Ac%v>>4yW*5L7s$&3fg)SpYG!Ij7F-8WqRSn^E$(ERD0iMa%dqFvet zcA?N7)=K+)=&Gs{O50iC4B*l9+}jV z2@#_}dp{ywMLgVueaYJX04ZWXKSB;vm z9$*g&3FKni$eQul71@YkMskIyM|bHnGY&7bZoE!#>=kqwkwC5}5IlH=dZb6ZR4-hm^*L8gKFj!1Pv8#vQJe)(F>&#B;Hv zcDnEyDLB5%WwRLw4sVJLcScUfvfr&#`Z6N9nw4{NWajo2``gV6g8L_ejF{A-4``GN z^NT+5wi-Pad13QulUfC(XIDv8NC>k%MH)GE!(s6=e2SP67}yKboyyAe8cNgiIy4-} zhA-abYd)kyb2au@fLVRO6pf-&l)`9xDoh!&q&w+~N2Y)^6cwsNZ>n);53iEJj6fsm zWh^5P+H5or54Yc3vH$h;IOgHu(|T?A`hmuo-~Ow>O{D(pcKbP5k5{HiKc2QtO-UcM zyQwuU!WTs{b^kA3id=%5)N6RdWB_m*wQc?Bc;V5 zeRU)oBhQ#Uag+`v?~Xl}ugX*ZQwrx;D;U1q<4*WsQ)6wxptl~uc%3UaS=rOVGlTsq zFlZw7v-wq?X$f(yVT|$Jx`AjBN)n{f&2L;NBO~^vuGa(!wi0IXdMWpg{7ZN+tzYl6$QHra{8TX$U7HFOYu!@A!)tu#F#sbZziNhf99IQvP*!kY934o3vIL>`h6Ma( zgXN1mrh)@JjiFOSvx8|rS5qsW{@mbEJxF1MQ)tvZ;O93uBP2|!U_-0)y)N00zu3Z9 zVq9f3*(Q~|T(~7QL!)(4W5IXIGwa5&CQ5Xmo3AlcVuoik?>1n(g8f1BXT7j%KjrR* z=a%nCXH`%62djNFM-SwfO8o(P#)kZew;vuQ_Qbt0O%OR0l0Id5^PS^`WfVcU)fJ<9 zOZYsKAhqAA-}S9-|VLBs!+^Zeh$rI+3tc@ zBsCIRolnYVcOBM@zVM5yaipjy(Me?G9go7Rax3i0ba4+W9y#nvxEbh~)7T&WNP8)j z8U&jP@fW^#W7^36%>sO1&f{&nlmD&kh44DD44NT+OVkvHH+;3qDqlKu?H}WPUUUfU z6?A&al5QQp6*xAdDgR9&GaLz^{78fPznvt^)6F-=)uOY0q^4+1~eAl4CZ}uuU&ECOUWf zs_+%9;=~m-hUs!S@zELWoIRC3ja^B5uik>GGtV^FIeX70qnO;~Ylp6L!q{iwRXZCs za795_)RTmOu7f?z7)#whadwk3PG&(SBSXB0LBPoCbGT&<_F+qVz>*CS#kgz#5i*?I zQ+y04eA8;hAFpdg#u;^c3>jouB{tg#uansw&#lB-xA7UG7?4xg=yiN~wHftpb%#Or z8OuPktOL^*`}E!00sSEwFGnRb_G+q71JPfL+#B@r7r6#&lEar*ms2c!wo3rJr!%p&W0>8Z@Ny?$%%iA4(VAs}@TfcOMx!z^gPH z?(RG}gWXy4Uder2^nhUDNSff$qp$~=jdv|{o|G6cnH7xY60kaaB3*D z4~5QaeG14Go7G<`sZ=)QJ9E4GQEujF%ycBGM(`wC=erWOME;6{3C4EKt_Q=IM&sP%6qSne>os~TL_hw5mDzjx!ZE;SYv?_|?GF#|0$)9M~ z<8%?PGp(yi6|JK((;IZ-Qx0NM#aY7f#r1QFO56HJ=BoQR>wqIrp zK2MJO(6n)@z~$Au7rw0D4%*jl9h!5|SjOBnF7%`G8*VJ3FuW9@$5 zQN1ZK*xn<4^i7;O?xklH@4|=H?rb^f&U@XB`1M%^;i1Rt@n2!aT_+rlo^?LMUD%4{ zI>NbcI!FkIiwNuvHD9cghL3F0r?Mn#o-_+lsvdSmsbTl$5Ybis8n%%%QZaark=Co- z9;elJhG~-&p0i~e@oYZDes`Eh5$-m^xuK0N9LHZ$#&c}#C{2C70 z*QjAG7yg}IS;;`_8cFS>((`V!nMbkbfrY+=lHR1{&JwB0{ph;*Pwf7)3VXGzNfx*DM49DAN>nmp z(s1Ja#8+AC{eAE6@6jieF$PrW-|%P2KYa3z%mO-ggDgh-ePNtAF6GDjMjkfu1rw0bY4zw3bo*Pcj_<>25An{6)9=U%ViY*3c;iBO0`#q@617qAwF zp$#S3YR`Y0&(J!^kK>8lA#`sZqBa=DuQdB|Om5lki~DwglzeM*j&>D`m3Nl{2dC0v z){U+1SwHKssZG_2PjQOQWUc!lW&gv6sCq9Ccr=1*rf?nzoP9dgKH$8+0e4p$kJ&Ea zM5{XpvYP)@=^f34zD+Wbfs!I=pKKIDQ@=+8J;R=zt?Bi;PY4lxJ>qaS=&M&#dDmB6 zF@8wLk!cu5XXTQBC9A1NWFKjUG!U11x_9>hz1({xW+&3+ym3X8_3fEV<*Q@BRIt@v zAZ8}f?A05Y4-73Duf*WWE5mUh)oradp`E zM5@KD!l@Bl=Az*`lP+W;^f5x8AzW_bN+AhRHrcDjZeh}IwBHd*Jl}&qG`}B3>k*8n zgJxBoZSojMVS@aZ^tR0vKQxm(yW7l4vXR08uB4l_hwDvdedCPOGGfN8<#r=XeZ$CngyB5WI z!f>3XtmCoH5*`swW1e#DP~a#AjDHbam9}pG^FV_IP81ER|N{dqC z2J3qg)foDi8Z24X+gffV+Z;u-Jw`3v#Agk_*rN+BslSa&M|Dhqnrh1=lcDj1xxtFY z=++tuO9@OzGAE{cSb48xy7Z9o*n132rV;o-fgSj+2-|(vPCb>>YbmdFu;>Yi6`7#x zj-VfJr|bOH<0SrIxJNx^Sea_yB>szX8~-HTVVtl8wRim#&|dlV_)D9OMGT)X30))x zEQ*)lx5~gIGgEKz%Bj{3?`fa`5bn3U;-*LeeKUEtwji3ZcOQMlJgm4xeF zzY4$S-{5HOU(=B%dWC082Q77quU%Vm;Bg41bJ7#ne9RvACvGdwgdbI2-7uGx&8E#? zUVL!kc|2YXugk9cAR!a|;j@?F#}PdVA#%a*A71(?urL>kL<(qb9cMYgtJ=0|%nyX@ zQLnEuk9>4pbyA#%uT@l}ON_9+4Ov`n7Nu8#&JIH{q2Zykq!y+|V@}89^&gSVDwanX zXCvm>#_FONT5Hf1B?R_?_ zE(KS=`75g8^vSc+(Sg>2meWX0tFo>`SV}JQIw~iMX_`X>{F_9v6ZAc3kxL7%v}V$% zic$BZFZ7<(nw?jrWSuhHt14{ZbMp4Pr8?GWzi_iwTkB&($5+Z35*#QgDL{=*E>FOx z@_g2x2k(=gVUV*E%lGD;6y@npM`UMUIet|rCf%l=N)hISGU(0% z-jo2oT`XrZNdkDVi=yZaz~gtZ>L%cXa#2fn3s46T>->pdQj(0Jb(SPl@Qeze098$Z zD|Juh0qYAQ5JM4&K+u56iU99L>761Vh*+dKN?Dn#0w7`#^EMzzb1u#2u(s1_QY0iw zIwT}wNGc>Gh&kfo0ZnfMjI`&VmiW$m7__vH1ue2NA{zgIq$v`E4G0tQCVAJQSvhIBZb8p<$$2%9J=l-7s82=L?;q!5k%k8X=4|1*3>0BnTs<+-2l92?TQU4^qky0&=TJTb%CCBK+zN8r4=5ZUL80r9DO^|2Q6!1#V+4qhLq88tCAy9WkHJ<| zfEc)TfaoysY#tkARR^%>&fC{tYCgUX`QRQOw1dYbcGCbXv0;M}lv4Zc*f+DYYdVm| zJS8*V4$QajQT%=`2nZj{r8Za??W4={4?m>z#$EQ520($ zZMTdv=)^<~Xc)0wl7B4*2Hp;XvH!iNsP@Jd=0P|Vf`4OsCK>e81eg&PAyFE_S%>s6 z#{KusQ=bdAX#!jb)KDenGX`kb8be0AJkaM#&ru{GW*!V3O_kK+JL z0HnGDun3pNM4y=b#*a&P- ziAVw|ByGrlbW4Y@nt0G;+NT3xUGz7l1NFDeh^Qw633UOYi+T=SNDFT!@P$4=3f|QP zNDPQA){wKaJ<>D{(u)~TV6Z~Il{@?q+rF`)v(sXtN>F@M$rS9*!ZGxVqDH;;7 zAG%##p0yb`E<^#cTjso|p49`$E~-z?tDU*dt0_P{O5h48uK&ji%KA{- zb#6o(5A-U*zw-Vsg$&6`cS+tA1Q8FyNMWM|e(>@sfrg$Hp0|9!Cfn|}E>iYBH0lAb4I{u4=m zo>WyK4_|)lTTuT?T4Dr{AauTCy4o}X2^SB=EBz%QM>PJz)g_^KQYfH7iJ(E%OOF?U z*uQ{&beCwBH2>157z2a|c$&Ba4+}Kil|WMg=OwtQHUbWA8Uxs1zA->@;hfjTkVyyh z{@2EQQB7_FRWle~R6~!@7u*d)&9;>wuDa0gvM>!s7f>EW0EZD+^U#I}9j@7$_e_kO>g>gwuW-SyP! zwN|Gb!FF`P#tT5mN!N%ch!FojuQ>(k|HN2VxyhAaARumtARrh?#}gn{w$HFUgc13`p%7iQc|}cm>MvDH?t%KF~ee9)3lu!ak4TxZQ;r zwQjyby!$^LoL{g9vxr0b4_piR`r~~}gtc|vWC9U1Y4e#^I9a5RWL%n)XD%EtZ$$6m z3XceB;}t#tlc4dgHk?G%t&G{~klwq^1Nyr}P`WuFO#i~+6-u3x)rU6>aBIuO zB?R{qvds{*G%k7N_FHqy0OarlVJJ4MLqF()c^`Xb?CqI#9P|iYZIcwA&8Dm`5!G(< zxo{ppL;OCFya_dov_I^2&yZILfE*tkkUo2gMR7{%wQXyg_(Ppn{XTzx``AoK2~sM% zSgmB>AQ$OI^LW-8A32KW9Os)YANQ*{Fp>~`E9?+Z0`*SKn7W*y0CX90m>I14QZxhY5r(r9dmMCc-+21wK+g!6hq%Lxuc()x4H z(T|eReBkoD04Q_S`lxH!sVH+EP8In=hA%}8a(G{cZ5$uZ!3=~rgMPQXt{^0}4zfSrwBENastsnu0?I8TJ zM(=O8?i?TR%5vZk7$6`ZFd+Xk)65G4PO8`-Ao;`~AejF%(@F8*FaT}z)#q6=Fa(ld zF-ou?S#o01pupiqZ6j^6D&qZE5)v^OGd6g*{m6aQnl6nh!KHWoN~IdT7EOCwTice6 z&bAg^U7;G?8hAh5majZM+|cKz9zgHSbhgiR$H_O)IW2H&Kx`ip>@9LSJGu}dZH5%Z z(+wR4;>iF>v(6=g9>ANl9B10X+cUF=wsHPix@peJM*+1`ZEJ+MftRyR!M@&IWsejE z4CUL>6w2BD!>MtmPno4*)_R6uGTTm@6^yxDZ_%aI0=zV%S*cvi(y7@tuHwv4&@4I( z&1~5)d*opC z89=eEp_+rK0uDyn*i%*^+hizsH%!2S$NXsm?wK1s{VA#&i(a=FM%7ur+uoxpb4&+Jkq;q zwyAQ};HWM}2ON76t>_#vumASXO~Wd9o7UhaWkV4Mp<^8ztxlZ`bu~|>MbksC&*;JNWx|=2$*1;I) z{b@n>U8Pk7`rIChJR`50omg6|t5wwuZ_%`l>90Btohp0Xs9KR(X9>A(n5epKM3x%R z8y+PTRY1dC3FzcO_lmV;RWnR#+XR$DPu#@0NldjESjnLY!4|)wL)jGQj!&1JTlF%H z$hmHsLRIk5@{vR^(aC`ks&ne0v7@znSi7;xX~9&nOT&hco0roRb243kqBAvd)qoj@ zTUlFZ%c+hOSJFeDmFFfw`~d8ZH-Xt2&bO9889=X&z#(|451-hz-awJsN9*Rt5L=@l zZ^XiJC{wcFIw;=VMr!OE`Sk0+bVdW!H04%v^f~&g%lAnsqQ+g%6#d)7ZxE zMSz4X7Ewf#h`gwx10VFL+{gS zLypXq4z7;52FsOsubrxohGa?;PwDoVEkM!60cd5=NK)$ex9<0LZ^7;uTtT1y1hp9{ zC@zlJmcRsBJD3 ziI(b`id)y9p0K2;rO8GUJ4WzU%E5>;Tvr%`>}e_t1QXRWT6_iAB{qO!WsKFrg&JD6@-jhQZiToLxw^U zf*B9$r9jq3ym*N^ABR?oQvFbz`V>S*I3%unfm}2U!W|hw0rphU4^*bIm_<~VE~ua7 zahEp395ov)C(bm)4u!uHu4JDW6d(x^v>2}^ZHIsxVkrxmj~RjyBtnut81ZMqjJX{v zdg|*4!;{LCJte<4z6v*nU`950H33CInU1tWff z*apcc1A%f(LI-6j#@Bzwp(;iwH5&G_RxVTN3y5dg*WQ5x6=g;X6D5(#eFjrH5Jp3~ z7%AGuw1Z&X|NexL45Tbg7#qj!uhe-RB$`9Ix2pQKGX&LKc8-n^z6cMyoDk;V&WH{% zlI*EQmq5Ukh(h11m}@Rg1oW_wqIvS8lPQD5M`5flbQBL6CbQ2YQvMZHd_4vHj315jnGywH?y2}8}o!XS{s z?o|5`2n#93M}#35Pv(_hG1`*{=k(Fl;~_wh#Ls*&=Te3JN(vFB^ks__q!h);ldS(a z>YhL*NLs2oo5zZZjVm{}9>KFAzGWk6P8xNYkvl2|%qoL(-#BwR^JH!Mv2I9fi{LVI zqmbr>qSoW00YsSj;XC5!c=OH4tayxEso9+k88yh-c!Kw}xXmbp!tg`daF4p>^><{- zA7Q|WKPd=II4V#F@Uj@N`-catGTbnD+Y60ZohtC}xnbUt!%Ly_?kSQNlCW<}VZ79a zz0~F;u}s84+lXM`AP`3NXbxoyV7!zOC?`eOaY`@30N85cqHIibks}WMRxA?kZkNqy};!5}eL+xeR zKF|BPds}LOi?`7M_u|mw%IRhZp6a>cv4s>AlOQSq2(?h_2(@8nq}Gy17 zmIxeIi?N~{j5|Uaae0dSi1~TwQTu=Sm`V+aUDW5%xKGE_(*fM-q&397{~gWA5(>#&TP_;>HML8FR+Q9(i( zkVS|Pln`gpP*g2^iat$&6vdO=zj{5B-t=9SYP(YwhO?>OQy^!DCwq03{b0@*Mae5a4~^jbD!KKs191oHD#U#RkP#hgHI; ziyM7DD<<0h8@K27?$)?SBt7u}qr_hqXOh@I1r5?O_&|*$^VvNiOatcn_HbKwe~e`|0Pwjcyg+~l1J^)v=TsQY9qgv?I9 zDE;$#tb)b^PRRwzdV0~$2^+*Z%iKFBn;H^M!RIB!GOybEv0zVzX5C~B#i73r+jkM>9zXOda%VJJ`rb zxY+Yw!n4XMsI0{{o3tFxVP(?eH)=G|G~Ld!qU4S-;QAhS-Dn&Y{X7HXx7|5IZDyin zsNeUdf6b7sh_l3%_FnYWA718YRid|$R3s85Z`az5cj`G^I!SQV?3TBc^`L9{P#or| zIpmIii(|gKWxN3@|8?nY?A{nU0lU&}_X0wNsC)@Oxl8n-M5)pF`C0(UZ3gZ>8{OGM z3C}BMVoLtY2d7PzO>)ky3h!s9<)#O*X4J;hzHft zHc_cV7imw+SD?*e@VtmXth|2L;$EJL*Nt!A{gbWT!<5$S_T!^XoRsYik&bEeDYO+6 zaJ}PbKnl>AHT3d&H1_Fa^jyEYn*E#m&*qsuoqwi>o$*8SLKB63?XYozsn2h{``fWQ z1M_h3%`sakawWR0mzF?&kJ1~TLKVB6nGLxT74(H01YYGWUNhGR)F^ejOjQqHua&b} zn+$XsaQJU?*UhNDPkXuD-_E1!%B)0$^$wjHWK%$1pUUAM!MBflYcHuOPWv7%ZElu6 z#)cXHQaOutnTPwueMVD3|Cbn{-g$+bx^Gggw%488CR6BaZI5yqm$P^HueO?P-*yUC zPxXntrsWO_hd(ray%(aw>!&flpo@{^(%H_(Ynn}${Tf2w&Z00+ihQ575H2fnQm6r| zMneDqqEfuO!^g|>hjG-9cv5c^znWIF=}4woPgA9Z^?1oj_ij{Q`MR5XH;a(RF2~E~)$uCRYu;~j zb%PjuK9By#!=WTBFV10BT#Wsc}a(S?Tthy=qdfbD#`rUKnX|F7@s5U=STO|pL4EtPcb_G~( zbAOGj9^>0F@FagX`&PY)cspa@!&_lw{v`H))HwI7`$f0%+sf*#ddhnGK2^|vkC|yb z8B9h}@Y(xP;bm~0ElIEea+jvopZ_ROlxF|jLG0x#pN?AVs(yai%Et6KG zL&g}u_cO0jr!7_CW2_f)=6Iw}vd?EM)1T^jVOD?(3ZE`x@8`K8flElm62diatz84{ zYx6JU&&I5D57sQcr|(N88LRhb8myKtwAGCWPoZf4u1 zztUMF73~{hM!UOp47W!h1v6m8o^XqbyH%pT{YoaM{>a2RP-4kDKJ#g>@Xklp zm)otN%Rsu_dW^bsuei((ht3bPiP$*W6LF*u?pw)?l@>CgGu2iqnr6UtB8B@@#f8xE z3w5T{G@R|#54BmjLPgd0`QXGuR&E(|PDM1&+hQ$W>WjfP$1&B{yT#v+>4F>Sp*IR^ z%bO(*Ds86POOMVa`wqx@I}{vXD=%wMbWdbwo~}GA6j|ucZ*`1gOF@gK=!o!kN09h zH3RHX-`pONX0{BB%lWh2=W@MH;f*2TYWz-hM?D$hZ0V8+d|zpr zxIiR1(x?rRQcVfD&g}W>q4M|lVX8HaDfBVDuNV@UpYmT05%*g;87?KX?W6W_KG~so z%fTh;-4QslxB%S0uU`nI5puV*T-PYj;q|!qLFx4QeP6z8p1+Mg_@_I#-0jEn|45WA zg{C3{-x;*Fx!nsp+ESx)mRy?y6gbxIa=vOyv;==KcbLsfWd$o?Ue*z*Q@w8Uc!dC8 z2C8RB$}km>7cv{(ZiM(Uxt$-*DLemz{oVi@y?2GIV!%|^VIuUd-ipH%Ps6NTZ1i9g zeA0NcXYZHH9*mYJBVXI%&FOQ20 zaJG=BLO8^!Oj*O-xkowuk%G9~6T_ZQr;AVITq3XEy3Ku`&ML+xH)>6WUEA*K&H~4) ztIb4^yMWw}$(7Mx&@WoqgjZZ&SEI`@i#=pEb?Lv2OWL#;(m#v6hxi^Of7AC=)7$`# zj+a+zZqnHt@aZSU*T3W1kE#!IJug}#Fx4i@4*w8+&8d1`x6PzAm0c@!KM#eSyAIUP zncJuUoK}FH(T@Pkc|oyQ2|OvFZp+X<-woea9N_zU>vxkD2gPxwg_Mz$?n$i^h7bUQ zoBd8~ch5&rk8;PK<7Th2pL8!#5Lmbu-YE3cr_Y*_%J|mqwefWGT`!pTk?VGLk|;K6 z&iYZVvCdTrb5qkzOx>(8kOL|9vV!ux6M8LF_jOczxPQa|=%91!C35RUUw>drJdv z?a8G)-q~}dUH~om2L^;=roy{013|Gjsl@(_j|>M>QzN*H*pXs+7Ooc~HX%@;@7Uz4 z^{PSRbyyl`#dpD|F>UtPyJo$WH8Yb_eklGf~g*2$ph8=au}f8IcSbQ6CVefmzr*a(Y*|!5ax@u7#5d z6pG(d_26i4So?dmX3=U_amxzw6T`^p_!kN4;76G6c%zv8n_p zJ426XZbhE;=@2BZcE#3udjzg5hx`9!=qCqzo?xIrKoDR+K*;}N=rPmEeuJS=+iTd` zEox%)KD(iDvHu&TDd7*Z)}5qQ*rLRYxNG+-Rw-yM!APeRN!s~^11QX@glmTFL|oJJ zQO>vfEC2^Z+ejac=R{c`T#XG!(Jf{YeBnDG1iWk|Wsj~r(}Gx4GOBkKYHBTjpnQOW z2*%M$LUNFbABoeWsV(5rP*fsK7t%&vFC|ywp_7@^T`M1fyVTg&tR0+16$IOx<8vz5ykSyD(SMsk&xn)CTwZq!*jQ}Z^l)DA|`ySHz^nrRU{OW-LhnqStuAcINf9#7~9wS zxD7SU=uyk!4;9lWXgqXfwVDbUNQ&7L6pb{o7D(->xey|h08-SPIx2g z5;^5(LHO89p2PabN5y5yiV-X%ZIkD!vP0e&8hJ)36V(0W2Pnuj@U#pf-{+I!nze^? zpygxtpu{H!UB<%KP0*&iVM#{P9<#T#KpmBEYIwy zjr;WF{P>fCNu!*ScGl$Jsq`S1&`F6#3ow29V~oXduOXn0G0uJBjwB0G=ik>5pw_R49;@E6%ME!9+AZq(Bu%7`Wc(6JAj^-~)qX*t9G^l(_El{XOkniP_aPR55KjG-!zg-!zZ` zEq@N*4eU(^<~YZg(wOx9-><6)lopzh4D^2N|E+p82u}$GM`=k;r6LnxdcQi=v%-gwnq9?f$0SOcDR%-^QIEY--;9{SQZ zeXPyq&u&&2K0HqnYAogO@_25CvZwz7u>GXY4k2$vvcGMOVLOiq2%{J~iE!5Ed>qGu zkn0TAyM1$5JvRhfKFy{E1>)S~f^7w1DOU_O7t}sKb|(!sJ+X0oDpL;}{s?ited2UA z%KNnbJaY+IpL7!u(0uSd=JuF;lCNARihhZ5;I)Q2a7k*ew)Dl4PnQi>Y{RSS&>VG&-O0HALwt_17q_C~4q;z2~LKsd|R={RFqPmC8D5ZqtIicr|LPXpyo|=b-0=wG-L( zHH;Zh=}gAzx>Tw5*RO|VG)@$TYo_7n@=5}ETcxhB(_Q*R3}Sr!HB87-Vz%IU`-Z9T zkQF>uSHbf0gXl_>JGY(BS$Q^E^7+4HT-JbyBb}s45^jJz|DVJaKY3fOvrz?p;F>+5 zf{*D~2Gmmm=e8`2`2Jeb(!`pn*Qj-V1VetI4P?11k+ju(s`RIBd?g$Mv3p77eghb6 zr_Rbo|=%Zca4v&8SNJ%aLZ^M1;G<_WNV!D6QDIq^+P=_&-qB=tVPSB=VS^fUoXS@0vBffdyYd2ut+^Cbjau8**xP1x;sLLJS#P82 z#wEZ`cb?Q!BV&}VIic;`!l%VTt0x~xKhD}~{mZ_qjiD@GN9e#@4Rsp%7;2@0{%)E-+vIvgZU59UPYj# zGq)|aLhOT-5uthQ-N1S_xT@zmHgEr(4HaMvlDk#`=+q5`mr=G|Nvnni3`LBEFw>Eg z8HWvvRu$G9%cWct|65usVXly)@z-flS+sI+wyYLa=ZIL5LO@ih9XBeY*`j(qqDBa{ zE4dQAQzE{`o5z>8r${+ivaSlhM+!_`jIF;n(jqY9?8oOK9z)E)J}Q+9E9dQm{4kH%k1Ps{ zRUXcbLp4AUsmY;bqOBS6ZqR1JNCFJ1Zwy53T{-tCC3lTXUac{fotIW73LS zJ7O2;h|Cp-R8KTjn!rF6M6RMVV5>ZWM#AKu-YkGfR0omiNl`LnRqn00HcvUS_6DeQ;X@N#)xM+-Qum9Sk#*8T620-DXLbUu6 zz5+K!4k86PpFLRcX9X=5#XbE?FYGLm8aq^JRdIdO z>8QaeCgQjG1&jH3h*mFSB>)^@L2-4j?>q5nRTGF0k~$Odp5_K_Pdg?fD}B6>d=ZBn zgAhtY+N4PKHL2jQg#o?=5GF$WB&ecA6#;l4FZYG6qBa<21E+ zH(9m2dB`h#;M_NMSEQz(Ju2ckt&$h;|Txr6@WFf)> zIhnyiBbDE<2oXPFVra^KhXm>Z=dq06Rq(eXIhn#SC$t|O1z;5|LnTBzXNAb0k!bN( zv>md|9h3;c<%m}Q4X6G~7(&`;Hky-I2xZo57EaNu8)9kXb7PHd6nY69!-W5Qh~b@p z!CUB{LDMkGpKB`az2~x%=oI7u7X;7D`!LVs(>Q+*D-Si53bdHTAA@?rcTQ+|Uug86 z7-%}(H^FsB1d#IkgYI)AAjcO#nc)z2_gWyVf)&2%7rhIPVa)&RMRkCgZ!qe?g6lGP zka1z*8^7+g;zCq)#HZnmI(yUqcr=qXU6+frvv_C=znk$hF|`V|>}79zA^S39XaVS^ z`#R^!Bl3L4yl5ffZ=>w<@_xtzbeQa3#E}EuOZxgA0V@TAxuFh*y>D9!eT@G5TTy`j zM1XduSG!xco{Q<1ue!24ofg4|EbI`v$o7kW0C(T}!SToM2BGaaAiw7o-3?Q=`&r3c zRK>v3tl;X?F*G>#Gu*0wOS1pL#cqzyfy>9P;Lj@4=gTfK!2eS(DPuP+FVE2crZ@3@ zRMzPoU`v$e%k@P3T$HgcO^xOm3`MPC502`lV;Jw3s=n6%L5%UBbEJztkCrbu{ONds^B=WZhVig zhDzOxk=^&YDevZE=U3hL!B#NKQIBC=jrj>JfX?XG{U_uj@|woYHV$SWDU}z~X_R7* zK5$hq2Fbtcdhp|(mnwa#1kUDlfft)v15UHY3ajIa>qrj5ZiEzjnqR=f#^P>p>uig6 zRiKhWO_#TViE?qHoO8t$jmZKV>jXHR`}<@=ORkoew=m8sF2<5+nyu)^mo;Tx)s}Dt zfY@{lvgTIHz~Ssl9>Y7pkTN6pZ7%$y*~`Fm)b-HYeG2Mv>0M(~>yq?0s#Sw2OVNb# z<`|vrILvx|xlzXnz4d^+Tk9F(b4Z&1j>-z+JEYoa>#^}*|KB(A|KB-QCw{$Wga87< z{GXW&|G!Pz+1$j~nJLMJ76$;ZTYAQ9dvLqPr$SQnB?%<|_wPU}c8Z4wnh_TC#90W2 zoP>hLyxvJ4sOWrn?c3;wu|iY?zunlniPOAY*{b?&wXCs0+d6M^vGG?iR8ftiK|yVK z*ACrQqlRvIqeir*^W}+W_zEQ<_xjCG=;p=8(2%jgM~O=0l~eoxoD2Y-no9*EO9itp z98;iB8l&E+O^-QazNjn0wvdYDnmZL9mMWK0Z3nCdo>_CG%)_g#pM_b^$m&>w127iD zH0xDuR=fyWAMo6?XR#_DI&~+lX*B3E6=BSqU=Y}i z$?|Q_ystB3?2Kh!cmU9bDBkR3X^iqk-^u{D^+W!?vAVvi<^d367B^_GbRSB-T~#XL!2jWu6JTiwzc zwkiTPx+|&4CB!P(&_zAVel>*(Zu?>lg^^U+vIQPB(?B_6dH{x<&7|6-t@3cBQ9YDY zYU3kDEt&xA+j?}t1(t=URqS>u1`cpKd8m0NKo(E;T%~|E^;pl&q@&`anrVJK;2O=fnfMsj6=+Ag+1XY)R> z%?`A`Bsygo9z0u5?vWL3L zIXzJ&i$jf(&fv-l_1%h<*d4BoyV27w)p)4(gk#_W6Cgki?SlI=aII`v&*v!?fq?Qh z8qX{|$Q9-aR{Ss;&luneDd5E*RDsJSE*p>>^fWLM0}~poRst5G3~=~yZu5lQ#`|1= zL_2rRK?_zk3@Y!0z9xvJ{+R*4g0dm)hqG6jn<-8xhvo$qDYw>B_Ah_gfPsOg>wg!0sMV|gU1+){TE?Z-R~S}IJ^;0fRk$a z04F|jpDj)fk!*&Ss4g&;w16Z|i}^I|9EcA;jJk*u@nD+_XCOmsrOquz7}S)4rcnY5mP)0 z0Xgfu2oNF=7ECQ7b69`rP!U>t7oUvgz!-TNROZ75NWY|_tZ<4;ADbu=iKF%k3or+& zjw(@~`X4=-qKK0vvl?JF8x5aQUZdD4I)jcRS)vf-IX+8iUQ5Z1kJSdNPf4G$z@V9~dQ`|I$ zjQr>)7#TvEcA+9EPifCgCCE+W2WTP9=m3Z$*I{3~A-1P;4))ROcao_`sR9&1NKi)@ zhF2Aqa?(#?=_aD;7hI=Ix}t99@B-M=p%jUQaRX%U(Tsp=a`0XuB%`v_9tp+2%vuHQ z>Q*p)T27DZeO$>N8<3&$WlVml5Sje?g=s<86?A5)mG~Hm2L^?tvSrSX2*@7ZTQ_T^ zC_pIB4Ir%TW;FUh%g4=1lkTMG=OmLnGtL?@ATH2MEF=xQVWg08X{WF9Uei34sq)jh zd##15ui;+=F;TY0gzWz)B!#vg{L;So1BCLOdcESq(8YN6q44JD^{4)LzuSYW4kOle z8muBN6v5eAM%DZW{W@?50@&mnA2w^X$Hz;GC;sNXBtmb(j>r0!ZdojtfcXoOa-f#v ztQCYVQa-47)zMT|)CnWeHK(b$d7LKFp4!Vk!CI(&8}y>7HTA&pbQP|AN!mHiYEw*O>)kF5A1;N8h|+$>ysUyjDtE( z#a6gGLQ_Y-BK068b6XbiS-xtVBoAKVc|V#X${?i(^&F{;9C0^nSP|QxJBcqJaXSId zNNwkEE4Lm91-9`9j}N>NBu&U=dY75zU`6=ABq1|{)Y>2QAt&&NGNbX^NREy!>aLV7 zbI#}#uHbuaYKOn|2>^N{J(#yBgujX)Ydauld983NLcc@P)J%BOpaRNL@!C5!s1jL;dp$29c zlq0o66ofgu*M>&l{_A^KQX#)xk~B}&X|4RUJu?zl7ql69PW-xv9WSirOW-oWa>}Ew zS>K;ilK89ees{^W-{ltal|{rQI0l-ioRi^(9?!Qbo@c&CGQ(Yz8#>a-j{#pBCwK6R zaoA{OP<8SibN7jq9NvOhg483dC2q!CV2>HUVoVfPZR z2A^EG1aIO6}k+JNgG>`Oq$y*<_9aB}m<_5rq2b}kE%8Y{^@h_?YgvqNZ`vOE7!?+^zSv)mDAOg%L$%G8c$;mrBB@$U?0<^#}SPP zw3rrU->hph9o~i{oj6N5G>mal$U;_c?w?+|2zqdPL%g@3-~n|#EnEYT3H-=~I2+;J z-mmp~ZU7U0fb6V3%T4&*?y>``+tV(kT5lFSZ^=@8|v;T&UgoT-@~M3))gE zyU%*e=3oA-bV7bb~$98;sqr16NZ7=i@ zhGWaS_LLJ_`Ap=)H%RXNnDf9#v>|=u@N;{)!?HiP`w0MkdR%ZBc>k;G^E&AGJ=^`= zvisvh@3hL6qomP$dp5g&<9DFpexI^h2%opr(_!-ieO9)@g>fU`vG-=GB($3Qqb}en z0w9d*``A$8Bd`E9Ku;$&2 zyd1;xbp|9N@XqzKJI;LA@UaK(YopuhYbQ^BJ}w>{{*R$p-(XYh2iriX9NeL%jg#TI z{oAN6d}n6QRW4MykKTEX*$aH0UZKy}9NPR8q#J?&z<$F|2mVTCg)*$`F#Q@b05Z1y z4`IU(6i29ScocJ`=17^CL6-GbViAa7KQhQ7?Qy2)8vGA@K=r7ZJ&R(P0c#ovO2SNeu(x%1?b zYoKFgliqnEX%w;d%|A~YEA|E-Kx%m=P*+q-s$Hzpz1_!Tn&Kly!3QxCM8XT>M9~lXa#vt%Po&nV`KQEjpT0PF&RINuLJ!XKHLglNVz# z){3VmeHBhqnrkNhOH6Z|ot5Mz4nUBZz$cF&pNX||^8@<-t|xH+_p${(MAh*t>5P*Z z;9!R#fswbp-nDITbGmb790*pc4_W;DM=mNdo!g$qvIQf#kVHGKXS2h7vuE4Iqizry zJdC(ph2?u_SX_KQUj;R^JWx6s?q(++?aj~|5-!@&BSY1&3Erh5QAmMni@Ce)uJXX059%hfzg!@!3yKIgPcoW<- znmv>5^JgUPsMla;GLBntrV@PW6&-z^t!6p<@%kUJ-{o3}!LOk<6tfJQWS${m0Mylo zNK+LoWjL~>7lFPzQTgBU!}Q+%<;+*~ddcsxXR#6X_AGqVudc<%>Ae{59C$QYiozrR zDU=oeRh0DNs#fi2p*URxuk0DOf;*&AB zq3|LTyEj8jU=NZbk4dSlgbz!cW3_%jgPNrI25 z#1i863^8Xp;JvYPH=x~| zJHd{Ur?K){>$ROE(R?IS_q<0uPF=Ea7UjDa9FpPv;?LVeZ_cO^|8 zETz6upDa|_&;oTJG`a(V0Mf*WaS%;Qy=c7YOHQ4ZfI+A}3opw?*3S}L0j{WJ#DYE^ zDz0oX0xKhW&3wBFyU_b1}b&t;)K7PWw9^PJf+{pjiwVJKw1bqx1P7;`dq&BnC~Ci?z>KRX*@@^@T&^;=z0s^#hV%9@Y+_B=3pNGq%TdO`I74zx{|Z2-DWcY7@L zZa(+M40T%pyJcIfKV^Y0lXVoDz5W*qmKFq_Y+YB7mWyKqhS{4Q-4`AET^B$?!dz)& z^N;86kFUQzg8FUW4`hqsiYJ^mfTudLj5vqS>%OPDLeM()Cj$AL?IkaB?!4o^&}l`- zdL2&p@B-x7ff}T&%W^9ce$;k5l6Km6^MW)1HupEc-!Z%_*0S$MHB)+Go0UP8*}5@p z>Z4V#c1Y=Y`pm3TaV-Dnu~C%sE03vvvnvm>c3B8qu~!|SWM=Rb=WTK%T%Wtcw*yn2>d+c~-}D=doIb!?TJvrs&3@EbftRfZ!4m`Ly~hkE5x}WZannQ8gMSkv%rFXsv5Z?z z;uV%$>j}oxjCYl;i~fG=uYzOmVJBO6Q-%BN4I6Hwq*eSr&jy|QLCmrxgNw858je*M z;dad@-#AZT$=q%IVV6Dh^OXH+m4Umr?w=;i%a{C!DfgNyyNm2UAIq88dNrK1BO1!r zJ!QTFG=Ms6+tw58iy7Jc#wELz!(;Dxhc+7^=a^g8A82IgSEM@K@R8K75xUxK8=gO& z%e7vL7$qx}6rB_eow{zcDh@qx{3-UmSA|e!iFbxWwS4ok%Ogs-ML`j+4LbIZ9g00( zJaeiJoRJKheJIcUG=^V&zYbLX$oWoDjxid-XaKSjmP_^8{3;52YfePn^&b>*X_+k1vuiP zn?NlsyEYa3YA-)M>G@}Fx#H_3x!7NIo2#;l^KQCE>ct!MqSK^RT3eMj;=IGuzr4bqOe+#} zdENv~?RU`VLpBs8i@Tk?pQD*I4Pbv?U-l~#Hf1pH-1+1FlHeMN?eW6J1!?`WhcfAFxDlcV(p9CE z3t~Dfo=?*n;nKMT4T`89h8&fTI-_iIU4uwEGMHT7DKT~?n$wZh@6j~(FCZuGAQCATwt%~!YYO$9F1?Y+7%5V{`oFYji ze+0TA*I%a>f}7=@QuF zjX%5-Iuj13haAw*p5HZnFsyp#9aum#4H3+iH|(=mouoyrB{~R7)Es4lc+G z{|+dqq)nIj1qD$d@ZC0#&im_8R`oaCB%;xPp>U(bwM9`kj~L*FWZr+T0Re#kC_I^} zQv!UR=ZNpFdx7Q&Vx>?RjJc@WBOor>$~s{v1P7D`XFI)|1OgxRub)mMV@2U?RYMmt zZsCVX@uShLCnvi&V#K{aXVn~#rIj9&kNeXpHR-`;A*xJ@OBTk#Cnc#w|y^vY)j+M0#3yK-)gpeJ-O`+AFm6=bT9DHX~2R zNTi@EGZH+yA>3k55iftp4~0Cb*m}rfMo8@Pb2qTaMo0}<>pb+O<*aPP{Si=Za8X8R z9_Q9JYWT|9&Nm6PH&X5b^wjCR*qT6*@2#bA)~_H?mTgG$W#~}s0XW=X!c)s~uu3T~ zqty9vv^9Y9c!0^gEQ=6}oZP&XQXY&jBqYJO)Qz*PceL51o|buv4r90*UZJTFiwhG8 z1)Zh3{9NY1L?RxX8wioYUQcMVF~$~Y@GzGJbygITHjIV&u1B5*;@De*CeyscyIwQJ@Uk-Bx8;GtJ##~#ifE06P zZ4XUv`Q%9$yA8HMZ+;PcKn63#!f-Pho2pYYZs^k=g8GYmxiN$Uw-d!24PkVPu7a!s z^BO>Kxl~h<#E6kinGfW}NsR%SA+CV)m_nJqCH>N+31SL6o|f1Kh6NQJ8UQ9@f*c;E z2%*en&KNCf2o-T(>9P7zgo7JH16Mth(s01kj<18kBVq?l%nUiTdZ1KxNEWXVCS<3?5BgY)v@poZR}3InJ<9|(pErxmL;Y*UI1!>{Ug`$lHaE!2 zV>4K70WPkMB95c$OZB@h100$U<^{=$YBILAI?faoNGbqLi6gV$x!xF0O)C*~e$wLb zwzuE=lpDc=O>*tGPEe;+|Jmkt`P>Mb$rPurQXpD0$%rtCdZ$>wV8Ee+I{^UBHxCf5 z<>G&Fo%#X)4z{|FDumzrg5al{vJ1N(3hD_efI!L5c|_50ofmS5jr4$aK_U~yOE`HW z0&|P*lm)Dm3@L`KZ{7$VQNt%TrI=U7CMSb5h!Vz#NJJ|LMP?fz%qU2Rg~3lTTO0%S zOxI0pSit$(kOYFF8)xpUq7q;Yi4KtFR!?m(4zf*CaRrAh1nG1s*vu@dCPiomXG(s> z3EIl$tPrGwqS{hvS)%i16QiMYga`WtipbFIXD|Wt`c_4$VRZ5uMrjb!4DUO(A1q^R zCRs7Ti=OZ1wSif|7_8;6PnY|HopQjP!wYOxA-6z|RXLzSDoG~gPA&q=vIq#Wz^@fi z#wpT8j2W>|g7>v?RkvhC>WsB7JLSgNHf~JL7$n;31)G!)WcG;iCanu(_PtBkyCT0i zs*e{vxeEi?whz>g(iaZohtdKuvcvPHW=(`KQb2xHR=nLd`%H4|-U(8;!zy?ImIN~T z{vrZF0!IS&1~LZnbj20KBLob;QH|aCNHCn7fhuyvJ^IfEb6=k@6a2%7Zo@ed54yTe zrQ&WNJu@B(K8@@tHj)oqhs(M6VI6-*` ze5M{%;hZx`-~SMcJKR+P0_B~p06@m8Ag0qiG!uPeUoT>6TXHCVQvv+VBLQt4w#4*( z-^j8w=`M*E@lXW6{E_+N@1DTvRiHLKz7JGa#&7~!Uq4q)F8&y3!NgKv@P7^*jpgY} zb^olv#v^<(J-7Bv)%xCaESLAL@!xIyp>XKi`R#qyB@p>ic}&AF{q?Zh>-o2S46)AN z^m>>6moM@9#}>!W*BjtZ{%5wsr|-T8!S-|PngZZO15!HNJip_9CFv=APE|i=?W-gI z>)^87;d5^@|Eu1@AW`70H6QT6Iv;cBO&ufm|5|swPyhg~oXOMPO#VuQ?|2vAJoGT|dE9O|==(G~liu(B z`O$gp7k_;=`1b!c`S_8@5ODd62iB%FstI?G=Qj3&7M}_07G8AR>Lz6IW4rjm~QEk}*=mw`5C5B+0}nxCEVG9KH`4sTD+Np>R3VP#8VC)S$xx)UFxyB<`z zIAW|kv7E_$wcCks3E1k-B*NzMj(xZ-7i0H>&043U>!ohz7E_`2%pCvJ2#vDh&vqZ zsuRG3pBY(n^k|Vbpg?&_cvzzi=7v20d!eB~PL#EQy?TxW@SZ}(Yi{^uw5$YC+9PC! zRYxY)b2?-$cnw>j>LC&iX{k-ZfBexq#e+=Pa*2zKtv7q&TX!fneImR zAo)4JP#WeyTO6+HUW<*eb)BN5;f;+loU$lqq{6R6F0VdPt4}34xe>GiXk%1>r7<Nj~ZFZ^W& z`EM&%ZazFx3VKqzmAsNWU#%-Q07ZegM~JzqukYOSZGG--{L|ygXR=Z$zOz5_o< zcg#d|8f*y31oI-#VH2Wv8l|XCK{jMlFof-hF-*h`Nko48Ud}Seuh(P{!nt4_#d4Z; zpnv&I*-PAjj&8b5NLUTv>{XhOJ?{c8rY8R*6;&@+e_A@yQOdsdM^sdu1r__RgQJxj z7o0dv$7f2C0b+~2;H6Td9nyq_yctXkDs+<(jdt;Q9}cp~nj}^?GGd5NUHnA@JbANC z-QUgmsadSy@!5WO%MN^8Zqf^HeRj>Pn(&MDpEOPp5Bna?2Phv5(7yi_tpD?Y;KK6e zJ|@q|F#!nX;v7DbPW&ylr*dkq7D*Y}mWHJS@~$}T-xGgJWv4FAI$fqO$Nr0@tXpl= z*64(@mPE?oERnK`wg!cSg^*W$1Dn_~LI@MO5r7aStPLY=2ozX>gn>zZUO%)R#zHHd zJ?6Sxcl7zp-sJi0MpIarhl2^d5$46<$FEE$5dx6P387rRs~R(E&7IA?DjT*MGg{&5 zx^NM9o0O(#F=s|>6H{Vk@MCYiYZY5~Z7DsdZWuLm*TtH(A6BQlOez)G@4-ZNq*rRt zVAW^toBCI58|{C`S~KbByJN3ZX{BalgRdQmv82;&B&-`dfdbggwT79kue+64t-}zh zQvgXybxf({$K8hX+PK*2W#;BEf3q5GRi&u9j;Ar5+0xZms>4t-|9*kfsIWOdRjFyR z>B_7ZnR)885s>LT8PheZQsSU|)aTG)>zmY?r_=^%xj#O6WwdK*KCW|4JsG#1*Qi~D z(KyhqruV?RM;$WuZ&~;RReU1^6JTm(|PP@u+@awy=5BLqETo17H$B4 zmR*PS8td+?LD_lJF*L8P(XW55YlkbD@mOBa>%5#qiMLI^ps}J)+85gf@ukcC-lFBw znYDmxxXP26mM>N4L}gh@R-tN7t;V2kovl)0PV^s|3_wcnY0Rr?v{t1nE>`c!*#}6* ziOCGoZyNf$%9%P4mrWOLnh+*&19XpWWNQF=BKCGT$@2KWGNFi}Z zvq7}1-+aIw5FH9B_$o~Cp=FNKAc+Jw z6ek#0F^V3~6jzk~p6H!SprNmScRUGJ&Tq>d(lIe93R)*sTgX5SJVXTbg zIm|sA+7u>kMWc}g!;yHG)im*E1Cc*~*vMzL#VE~0H!dkWQm()w73M@$HUOP|bRMxu zxnyZ$6iy5(9UjG*Xb$c4Q?9bGzn@UP%XwtLuj`dy+wG(tQ6!n}05NZs5IPWNG(Jnz zqG$!4&fe-sj-vL7H9K7F236V5!`J%iszcDbxYV zwv=e)VILJJEe}vMOBDvn+Abg&sPm#$Dk=7(IWt#N4x{(lI58a5{r3gi#q{G=Udr$+ z5J^TT%RcCSP#vlPO&y_K;JDX?kl(Ty^H_tuqh@T}RFm>j7u-E$B2ur3#&L_o#Jq@O3dqD zzi+k+CS!cxTKT;Rq^t_y%m3E=Qk?g*NVxa&mR8w&s&Hr;qV6kS;4{sc=hFd_^mub$ z*e+_3Q4clkM_yLx1i5zIwLe(k7Zp#UozOJnC?*x;Jo=Tk{{@gNAOL$5JJVloYnKW@ z0C{GIdX-gIa8!S3qrQY(1Tusyaz&C1p1EFL$PK>OrgDo~G_*mf!_J4uAaciC5AQ)M zB|5<>LGgf*CV(%*=I=1I^m`f1;16LVQNkI?S~$+Ks=T+AP-UsSZQ{}E(P#RsY zm|17!+@z2miUF`t9MaaZLO)!A$O+x9o?XHf9D}-wraT97Dd8B60Z*8|MPiweit9YZ z5EUY!;KVO}Nm>GYgg-VVPf<$0YNa3r6~eg*8$W1!0QS@@_=bw}ohHkWTuboZ$KR>4gIfJ3;~3xL+c3Dp zLd6yP?Y8=iymD~;glmK z6zxUDI?B;M=wTr)ImmM*U>+`9EnW4oHBLV8Gl4 z9(=m zkO7XJ8;-ldH9ItCI}xVt)s6Jx^#I4@H5T1mnj9t^4m}7LX@7#Ayf2?TcEX?HJOlyx zdMVl9O59`b|EM&^KrrT5lSsTH${8aD=PRFoH5LC><1LmRT4^YMmYL7KC99?SYO86( zKuuofXs-fL2W2%5eJ-QV--9~y?WLkgKwg@OU#v_||i&=K)@|sy!-t z!5&y*;kDhg2&63}#Gg9zPRxS8%A>noj-28#k>2jX3)hM z6Zc!)jGteU4RpQfA2(*ZuUmbP&I0+Hkn2-la25^l0$XMAZTs?SIb>{L?mNb@Ev7!e{sf4w7xy=Wyx)S!1gmv)@_?l=a?kBiyGjEj!`Axa4c9|CL)m&Dm-(^gHwUMH zOZRGZf>xcux9ZiH*Bj=#Yh89iu`Yv9oTDE&^T-LkND|)C9^ddI+znA#ekk7FoyBV} zqaVF`wc6d5U(Yx(V2VnhX#p0sRI25Zx7 zj7O_`tY#U;eBBs9LBHVHK?=KCpW3WCT`OR-=FffTdDI>&2CUgJGAvC{x#BGPg%=4! z&5x{Go*`{Mrth7K${0B_MR^>BMo27?9c2+bVknEqkhUZ)zz#y{A+lq7`SEBvYEO@T zd3j#YQ_u!}Hb?eB=;cz8v`&nEt)nSlBH=P>FA6RO4vrEW+*$Jz)_4A@+_8KtMSEZB-^CY2yK)l4BN?(Gs0XNY#HinoJ=( zz+uoefa|Qr*dc47kZ{E-Yl8;jP|CmrmWD3Qhb^Y@FQ&PFtl3lPtYJJ4&^tcod<}E? zxX-OS_-gWBa{0)=y!5d&{S8TS!JF%O0sPGE{y6!aj%J6GB;r35A41j(tl86w68lM$ zqP&$}`>+7a09A{(d~Q3{Y=2TO#>%&<1X|}EF>Ul|sOLVWX3T4~soJY!y=`Z06uh2D z_+p0|6DY$>I+y=E;uhGn>iCJemgwp{8uw}xH|x*Ur(U@5ow))Cog4LPTS=QIl{ly! zM$8MvJD42;6h>@X3swh(JFeRf@aZo8)#2dsfBm+3QlYAERNb*Zmh3vTt3y=2;xXLFwT)_KJxsy7KCCVpVP8UAoNvISSiLWCOVZ@ie!lPnwyR|tyB+$QTiJ$K?rE=D_s#<$w7X{8dc0Eg zujmBOp=yO~18y&gXGgng7|Ekn+m7pshvSMPwmM1`g!A#1QL9>EVNP58Bhia zzOPL7SF{{v5lh`4TQNNcbEkhdUUs; z?`ZaY+yJ&+y#N}8h+6F&dr~i^XYc{I9AA*3YnzgbjmW@^cJU>IKWt%J5wcNs4vIAE z3Me5-x_0EGZ*gD&I5b=6)i=;Dm8(O!`0Eg=Rf2P|q(hqB`5j%YWsDOfyjvbT29l-n zf4%a?UudIQ(!>>5{tB+cxyXbJt#@E9CY5dbLrwf7gAWC zN@lQ`qQz9Jwrir|eDXvFN~ukxkB%Lfb6q-&#x~m!4L4v;R-L~4s_gu`nv$FB?5jjx+gxjOZI_s!@=S#u$2|HoV9 zm{*G@j3q*8bb>JeU~^oIw%y30QeATGDtK-RlH>xfE7C;vfXu(0n2M&QXey9qOhM`H zPdma$8OFj*Z62_me)hKG`A$YE45E~gTw)|!qw z@@VYo%3zx$KBi>3e57B(d|AqDNCq6Qv5%n2W>`-%gEfa*NBv0_G{6SHx>8Am49?Qw zBD5hyresR)AL;+R9}_e1(&_c#R_uj4- z7*P}S$7eagf6Me|r$dY3#wfrtE1Rl7B`_k_D^I`n@8e4Tg;!iZF5@3gjxw*aw9AYX zE<&@m=S&pApgE4J9I#SZNIJ6T>={-KHwZ}BqcOJ>&#JfJPe>SFKrR#*ZNOA^JH4Cl%vWtfI`FCJ}{qucRiwgt-K23hgoiFFbFaw-@a zLD&^-vrTHV4WYxN-ViV3v1J7DV?cX%ylsu7Hwc;Hae6!WeF()OrQZroFS>qS%^v&*Lu?bSZ)k9U%_1GlB3%-Wh@-&Y8a@ymNmKG)pjLwJTS{`9`BX5mHp~RC zX}YMlIwF91W?nXDecm>{)UxuoEVHQgDG5#7Cd8QCh5Hu!WN8hl(0D)$tzr=wXa+5= zct$Jr60KcNq2=Z{AU`0+FX3*2^#^U~WsD?kn#oQi&j1EE9UNrwlUWeN68_VjYE`!V z?@PHM&d?2DYCPS_v8$=la`St7?Wm_zajIvuKkuy@BG zZ6u_ul50jqg)uYLs4{Ujs=H8zohM*?f9R;+=&XNgzaKx!y~FY_%y8gMo_klS^A5it zaML{k4JjF2itt}J5h-hfZolaiWsZ(2OC+3dfJhSH2!)a^m^!w9yCK<++F(bqo)&6X zTrfUTo}R#*!bu06NG3$Z(=roiS zfH*ovIYF+7h`^LcLkk-X1Gb_;Dodgoi5@7$ZYfl#XJ~ZMR&<46$5cuOZWS$?z>8vv zSjh(fX^n=G;-8a1N=lTX@o%Gw)}>Aqah)aPE(hZgN(+egPo<&;XHrMe(pa-oCAX%e z*$%l0we#sxYR9bLF(IGb;|GSz^oGIe#$h{`B@RFTXuam4+F}}kmYf979Ymud{}_~D z5x3z4Jt&xpiAmL``d98qhOVa^3)jnD2*rv%S(x-+B$(a24-N;r=ygo#U=~JcMiWKqb8qHXo5iE zO6{X6zG@f5dnaP?G}^NLIfX+fk|VP^q*`q5C@QCZ62+w(uoZ6%Vm#%R>qPlP;9YX0o9Bcy2O$LI-v4`@c6^IATWID2lWrtit`%?hyJszR|zY=#^NC3|E`wQVmX5 zqt!}kz+VxkOjdJa=*ivJ0T*Xepj~Ys`-4U+DBzhMRppFl8OxSp7?)R+YvTZL7Qh;F z6Y{zcYU(vwF4VKmrINL+ksJhhs1D&v(8q^u68rxwu|GR13)4xa9I*~LgdjTTN~Ib^gSOK?=GQQvurEND@xlmb}g(e6SDlYVTLzh`&YQ8Nam6+YJG6 zPqrKRhfR$t4k<6kG)WCVvUmabq5bU79rx`lAB((V}$i4 z@3J!}^T@Gwgp&l|C-TFN<~Tpy?>DbR4%qxJP(Rtly3oCM0@_L}+rbUeDL1Z+F5X~u zyU_1mYp}A>)|sq>ggm1yv~F0#L9c;E>t;k#9K)tF&l| z0ve!CrjgOC*#DK!)w%)(t{TU%scr83aJ>I&%IJyzUL1Q6d*7RocrZA3-3Vcz?*q>)N1iBZlk17KUeIgt0NJ$9#OAsx8 zO9?5*LIuk}vvf1!>uM6M#vnWmvXO~Zld!r-K8tTL4tfmsD2Z&2qC^;|A;`~ats2?3LO7xaF58gD>T~UkcK0H!t){6>5D`q>!8xFY1M=A6vwATy z31(Nkn=2-9qyM=968-P?4g6UrV|$k*+g|R&%h_07R_g4pKMTVOsOwC4jDEl}v~h=> z@fZ2g5^zwREdo_Wvzz#eJHbOFlEZWHw+zhmv?qj9Tl?q|}7>Ip(K%DE9*LR53z3ke)~I`oh6`ETHdi z23Gw@KEa}4BP=|V$5AFLI?V1riPk<9+h-6n-+NvOq=RnK_MS@gu5TeL4S6YHuf8Z- z5GgmlR4rXBOQf=x3BUMDPC#1*cP2+8@eGLF6qOlu_$AFwiS9qr3pP<^(CT4+|GgLjrVvcBiKCwRVGV|~MeP3M?*lFPqH?0!(9l8EBR=+ItC1E+FIkYww+ zfhu5)83%BI5Te^g|O2Tu7uO;?40rs<)3yx;gF##4AyLGzG=EJp2tmGgkXK$sbA-oN(@ zjM)JvA$jW2e2fA)K)o=X;rmtfg^8hclZ!m@QMGtHwhO^dQ=*%`2vz(tM~)Nc1i4MDQ`(6P~H8{EZ>6(4X=Ojzq^b4{Lp_hxPC7?1pq`A@^0{A zm}zkF%q!t3H#jJDk{(|&3;rN!Cp*6n2xbC`)4uU!Idc!^K;Tf1bh|hq0KI~FujH!! zU}^p|*P76olJCX8B^k7&D{_V4i|7lUO~_k{J!sXvt;G`#_m&-~qB$EgNwy_)h z;0V2x9y59{08$IRay*<~Rw00RAuMGLe#O#?Cj>wy(p?i?c`vA{uf{^J$XUku;}0Nx zZ}%qZdgaCWbCdS;N^&@TBI_oZjq_b7?k9m&PI=*#sAs8C9xUF~o9OGs+Nr5Uk66+x zt&|k=lNReU0&Bp=!sT;posH`8Fp$uuiCysaffoSIbvtQ|02Y`qhv>%hBh?fhBe{56 zR$HzlVA&4&s(^k);Yw&~kHdyQK5Bb}5*SG+e$A9SO5%v%jTJ=c~@%1aA zm*BB=*X`Roycl30g|_OY_sy2#12Xv~?5$1qk`d@EBv;f2ad2EuDq7Jd>;ag#;1~#w zD^scro(xj`A;aQ_Zupdz->I4h*FK_JCtmveBa5l{IViXZb%|Bjb-LUn@zvQ}5ckSq z_=tw}MgEJZ**`k#-T_(cL&sJi_G%Xez|}pl!#fXd*#o-P8}(pu`{CgNBC9`iUak3s z|MXVWHUYN-=({tlx(=zDp)hMqKj@X8e3fc&5#HgLF1u+f-I`Fv$BsKG-?GwsXkzE*s({%QT?PJbWi`ih9e` z*Q??sqqI$%qtEPkT-F+m)+lxVEiktWcEi!-)kpPloIOvN)4D^7*9#D%{{D9!e%XIY zFiEa^2rhg_x*HA%s0cP%S$Q({#?CC=-uLytr5t&yb|rxey=&n4p-SAu+`bG_40UB3 z?xebGVZrc@{&-N#{AMC*I?V~m^WX*|(%TtfKcD1@de*=G0sl`AqMkamD(VSFo)HK( ziHbOCew21O7%X}qfKZ9V4cpyGi1GJ6j#NqmX$BZtd(o=Kqg87~Ee$<-Dg}M(DQY{v z&eK~BU)%41f(Ufgk~+%k_v?J!JeO&%-3gADzNwd8K<54=Hrb56yG`|LUimrR@R?w263&Vag|D#{7ok3s57;a4CX;7 z#6)U2?-WZov|=m{&Nb-c=z%>#4@L3hRVRewk2!s97!1v7-N8e;WROq^mUr{?_ggs( z7)(JaLQ#1GitI+_eMg**+MPzq86LM}|5p;Y=c#T0;G+Qu(ejy})Q(2D3bIHVy7$~y zpX8lci}wudaR@VOKYI=qi9je%ZMK8!161WNn{P}(OzOy>s~#~y61X>5)AJx0fzRz0 z2kRzS{D+}e{^@TESx!`(^(-56lFg8FA4i2J>ysgh!gruLt@^U{@;u=nEU68OuqsoW zYv}v{0GsN9^l1o?Sk}M*Xqe@5W||FgtS185vj~a)>F;zniin05>&yyOvFh#R1!*?W zo~!RbU(FrN#pm8-=Q*h=jf8Uvq#y0Ep5z*Yno(LgLU4Gvk{oznPr3tNf=CoiUdm^ zABj?B!&3?LJ=4-)O|h}2zX9X|b^#8KIcld{Qfw-x*c$5#41*yo(Z=87+OWyKCM*`% z0h{U)Aruk1r4N##|C`jz3`wTxoza;ne)xA~?s?(i1GL64O)eTjL`-HHP;4+f0`^e? zpc3v|YteMU>vt}-s(h@q1$*iP7S(~s1Gw~~Q4`pL77Qu07n;Z!ej6!_zf-2x?OY5n z4y+@jC(srUjG)aC-X6)3K2V9cp7 z4v1!S0~kT-+xC&@&7vUp(WR`!p58ghB97EOuf)MX(z{o&sM)MGaxNVuzn+vEkS_fQ zi1h_Ow~G7{FDf6Af1dWRG?3_6?KSJgCC%a}l1~C8i~f*HZPTIH@yzEBeg$WA>F*@TWooBnVVe<3alsZh~O zXi*P!(w>)A4vJ-mW#Jdhz(1$jo=wBMJc-myBK4A&sdkm>2@h6|cRN9<$F6>yvpdYl zP@*1JV^nkG9aHnVBgJEnKwJk z4O0WA;6Jzjd8s;1Dgh^xo2YHKNHZ^`TxAapuaB#Mq5k>J*xFm+qogVQ58|d6jnh-X zO^;^_{X3`K`LLm1>SgI~?40gwBhoiF8-G)+lIkEZBzBc~XQ?=nI?FJW2NnWwOPP{W za25D1>~@6Y$A8SNTOtJy$TzRa_qlocmS`KX5Q!pi4*jxF$Jj#X?x@h2uXQnDuWR`@ z4&kGgm%V8eu(((KagS!O{}?EwJalGQ(O>s{oEqZ?xB19XfosnB-9hxyzAI3dubJy( zG(T}aYEoPO#L(+aw8OKb+Hvp)yyx1Zvvdn8WfPp@yqAQ1IH%q^v zi=g=a*Yd^9L$|(a_E%hdH+nV42l{j}pJi6MCYDBDnWYNH(5gps6wFiZIYv6#o1NEs z>Giu#zPOM9vW0>3=V#|?)m@tkzkaL4=-2XD3?IqQqSVtl>%W-JGvBifTlVh1-kbA- z&t|Jq{=*EgDZBi0z+xv+*TGMAj@Omf+)&Zeg?sycttopRr@OPy2*D4h9?7x=BElK6 zZ$)w;bqi6O`+oOtXrq(3YgV*OhEzHOCLcmKT}6Y-gsmS{)N~!Kd&9wRItK2lype56 z4RDERsHz3Tj8!iU?a9mX4}|(}uk}&zk$;7S`a1YR9l2v$fOVTG0Nb&v+rxxg-qnk& zF4MM)gm!zA1-u(*`wUlxYm27a(OOY%Y?xW2l+C@JMjd_4&8=>P{XKuX&n@mt<&PA6 zL72A6cyD#7u1AZjgY;bZqwHtI$sU`b!1WzwfS)IQp0A^N$w@KR6?{PMxC3xo@tm9> z@I`-ftfNW0ietpYZF>AVbJJ_uRZB1Fvi3&2Yamy7*(X%yWXF zm(L;xu({{dNlr({JPWc5=E_mt7xOw*tA)_KVoU+ik$51>p1s08k(Zlk)$kVu649n@ z$g-_eg?S0BIaJfqg`12+N13@PmYigTdefLyj!j2)R25=0MqC=1iP_$AxuoQ`(ZX4Z zhJsv#`VtMPtFk16oI-5fNF+XbWj-1v0Sj*~z?yC563H7P6vFCTfRmRB7?Mvs#FB({ zgSC7WFL?&Rc!|oxPBFhgf0*1QgGH|LS3}i3PHL7b3)fGDwnfUy4K2JVDG0{paKJ<*<8^Vfe&J(^4n*_#stLEzRyBvwhY zFCmf)4JFyNge9SfL=(o$n|Y+z*CjGC$q4yhS*5M(Q*|1>zZCD(gz9>%#qNS(CR_O{ z;y0}Q)EHZ>mAX@+OWP^8T+GQEjQj5WD15#xW`8{VPd?{nt`j3W1k$I#` z6k`x{3ni@-mBS)g1tlm#a_X&yEfky;<5fBHwtNd2c+u`U#nkpwr_dv+nF}c?({Q0F zqGiZXg3u9UT!yKsnv@GRPF?GT083gF1gb>1S_LTtrgK5p1pMd(ylRRm8j1zIc-+G$S{fO*GJkVKm~ zmdsLJFq+&O+BadNNn2S*Wva>#3A_i53vJ91TC3WzHing&HkU*GIvrnQip zB0_bi2PgEHaT}rqZNYs%XA-*ns4+9=%V^T}bDrt+xc1-C%lAo$Gs0{*H%5$=8YiwD zI+}HJEKqfiVF<;e9aY*AS;hl>?+_a{@Ub%O)+3;twiV4YL`H8)Kw6EjL0X!FPDyM3f#< zX4J!rlY(>#z~xf1v>W5h>j$F;qQg|EJuB4Qjkbnh>onu=7nfN~7>5}XO8`t0Cd3^7 zlU|>=S9}2->JsJ^R~JL0_!NHa0kjUnVGmm1c#1IS2oujgn^O~B$VO1KfC@+svjOQC zc6^ZDasBx$N3eN)yd=RiLf0R61_mPH=$^-yq&vhAFigf|_zcno7l}%ic)efHwO+ZW0++v3)ln%7AE8O1SjQcR>FouSjS#<`ke?BHmlJV-PKp_5 zlH<;`40qczDAEjWLMQyG9Pl@Cfm)KYpBwB8DXIWP>@1ZMMA= zBELFg77$#37%In!vHW4;D|8FjZ3Z~Me7e2>fYwC&3aCBdv)|)D;E65;hDn|!{%aP) z2;M@N|8K${T3Uhyi2!gZc@TD9)t!M;y~|O0_o3Xs=K06R^Ov>(*vM44bKF%O#*!_Z zd<({s%bd<3lSDP+B9C0}v8Y7OOycf`XL`Wt_~BQt5*mms5HsnzXE9A<`?0VC-I0~MKE{Ylu_F;F zu)guHItZT9f6F?hx@?_VkY?V1(3TDF4y{;2cfJ+H)ec^G2C~9Cpi?=>%i#sK_LrwJ z>?i~yz2GO14kge<8MQr;mMB;{AU=6-rXM4d64pqL^uw1V2Fz^3kd&^3iw>k;{CBk5 zv8vcCaIgyy7i5CVzw41?^HYNzxx6?@*>24G-oX=DwrQn}>RG$wI@I8&Mi!T=(n zRt_m=Onw(FmnRFb7?;`fsr=&kQ|B69ZGT!`PHHX%k2zH&8YFB@KMd;*C+Q zaE)5T3fH8bSaU9fdS;{>V1?}8PAg7Bnn;xfE(O$B_v}pV09S#-L&oN6fPUPAF~nNo zt78Nct!F?jp=dst;yvDJL~8^j?y4Cu$A|A}I68>)Lj$S8G2FyEXSc4(hpC|5Y zLfcC)VmIMtAVO1~$Si z9$x}$DNeAZny@W~L#Vt%nvwIFP@=iqJLa&CwW|k{y*6x)pDMfM-j--ad;x^<>l8{n zk*;VRn6WNU)>ut9N02=_;yWxX$t}}BDTgsIhlE!^T`Ysg8~%XbKA_(M1ZfpSc#+z9 zhLs0u$_>d(1ic)9guvdvVuS`%> zn_F~ptP-jbnJ`Gux+1D8tiSz6Y#7{1h&<>UgB~m)^OrsBuW1b1KDpv8^Q>*tBUpot zusyQ2bzlD97UAye5O;)fTUWs9tL+GLXJ(A}U4WfchoGpo+A z<%K@t*x)(9un#RoEN(qK(V21~?q#n*mQ43!7be8d! z`4YiXX;~a6oz?f0EYGMN$CYM!TuatT27iTtUnc)(Eln-?(n7frYF?{8X1wRGGk4yc~i8d;QmA~@Q$qNQ; ztaU@UZ87D=cDDN{&%wkl$GwXOW0Y&jTOe6H@9QR4Jm2$(hU!NLHe^1%KWCTU=)rAP8F@wNr)5j9sL|D9* z$4_%~zL$@}n;(zM&R4hFPwiEp!}Fuu+a;DCr5aO7l3(((l^D1J-*>Gbz-ROM;g9c{ zLAR^kPR~!*{ZjQ9rhg@$cVJ)h>&l;J-bQYKtNGb7PZ1;yKWApA#Y=xsGlhfCO<{YK zIz1P6-JQmXch7!i@QjN(z|~0N{{3Xk3q#-MS#kx*r!(T&@;ju3 zRlhurZ`~?T1-#tG=MwJMesD#=$39m4ya7V0zz3%|8LIHxsHQ?=ihhY%vL{gOu43P1l&%~Mu3P^ z;L-raW<1@<1zIU|z{l&@HK)hNe$#%N{V_Ej^dwror2RT0*Xi;$dq{cftC()*dGwF( zh2GBe)^fsEBs!Z92@$A@>R*k(q$x>drO=GJoMiyhI+bHo2h%!uUJiJaoORa52!7J8~Lm_AEv-Vg38v_H`t} z>~}v$+zOQI$JO$)FIR2pgy$&FbL^vD=WOzb&i%;rV{&PZ=N~SY&*1m-X@K4X_}a>k z2k*=2(wvA5M(lb37;^O9jr;HZ-ZH>Nc)K;F+P<6@%Lm@whI1*mImk@d(A0Ppe_dP# zhrT5GIBw54{`F!2!ZIH32X0B0#MuSj4oIe^fWMnubSz$mxjS!>wHUM&bFpZ_ILOfG)*OV^=_!awBzb?mib#w?>fiQF@m)<)p|uM& zcDhGso(wGZqGLwpQGYNTL2vsOWp+rtcwj`;J zSMYV5JF#9xM{VMRh)(4=m1DgAP+v#R+(c59Zs996Pezrl-^A$aR*{;f-q%vGCE`>3 zQ{X>6%oW1$Q>kv(iq1Ph?vFp3xXPX zQm*_r898S|oO%<)jWk742LCyR5Xo#jVY8gfwIuv>l__oHwF!$FXS{v*k-WY2c0Bmy zKVVN7?n*zU$?OIi2QmtFHrDJ~SM)E@ zKQ$~NWVMHd_N@_GLwdHIw`lTlj?}Mnssl(u^Q%L%wzP4Y zxD!9=PS?BrF&2|`s~OlHj`Ef_1=Zo5*;H(z1Rtf!Z4#1}9jU&J!_5h`;oNNpYrgIe zC~9eMwMoR}MUuVb>1x3~nBgvQ!{#lem?uv|!FP2;Qx7hQw$FWvafP|rv5WtV9Gnjf zHZAX(w8y1rTTK>(;Rf=Rn(|fXD^H5|2*2U(K@~)DR}7?@V5Cgj2icSVzb$WN>Gl+5!sax3a!b3qxyaK9t`y zR?9~&5Fp@16u%ywitDBW@27h0piP=}EU`rrEDO6q4i+(f3aQ;h)l5kt>-pL+^4m%fK7q zYLjJd?H^h-bwj|(K>Sdh%5$n_hl--N;_**pAEZ~4>_3Crik=blsKk7?wjjgHRdTjC zMbb^%%i0n-@zYY@)dJ|MkHku$@{(3QeT?jx=-B#r`mqYftOSJHrcJ4#kSse)360nPJV~&pgalZ1|(5mcHlM&%MUOQ`YDHfroi0K^x40%lNHs`WH6o z+B%`7(%IZOQN1*TeODLOr$2sZ!m42e+Dn(D&S_i;u(4FKnW)ofcSTb zpb_1=t;yR|CC$P1WEpi%MWD(+wUyfgQPk7EvXHip`1GM?|JEe;9dX#@AO8UrW(?eD zErkjePO#`^zXt5ctDOL85{%DZF++_9FZ>1A!UurcdIldk0++E=%7V?UpK-|~la9$z z9Ldd)BlY$0Au&B(y5U&Tbp+}wBG0P96Ix`t@}&J8{)zrbJ;Fw|qv0CWNL^8Sf~F5S zJhJX&5>aGFh{1>{trp{C5jvgZR+RC&Ax=cmv@^+ln6c(-GnE}T2=VW>R;$t*qg*vZ zL3d!S`G5&?#D2f$j8#zcvH?G`gRwu&MV5i-n#oQg!?+(g=24BjY~rQN+K!v37N~d81&d*GVp;O zPV+Pk1H|PH!k)4vx1hp$KWV8d(@fbG+X80H-!y{w9*<}9D+`qKGVfq0q zR+)|Aym<}_<~S_YFfVvB9s8Orh*9+p>tGVy*pxS{tlhyn3&pA1rI@hl4=CFHz84^U zgKE(5FF$dMZy?w1E?!z~2C%NuzvE!DR+eDV=j z-+Q_k2u6#Alio~M@FIj$e|qzP?tlWwUWEIuFD=lU=in_G^AUbGL5JCay>4JHa$vTC z1(2RwNQpEJm7`$FE)*8O*uOE_{ArCj2pgh^Uocom?cI?4bB%@bg?6e!{#%FlDP3l> z*zc(KZ69juE=yxbL(&emS|b$J=MlF49} zFWyS^TNll{4w_*WJc^r1L!V=Ygem{2Rf0z6a(DWDPh+OM1+O10D?`beu#{Q{%KqwH zDTP_dfmoE&6?}xRxBjHkum#ET387*Qp3MtdvZaMk_Lph3l%iG{2FIUV-iKn})yM?S zgS=+X73>xd6v`)Ll4tOF(9KOC8RK2E-(|@6K91^1-ut6mg5nz4jp8Xx#xS#$q8>Ll ziZk!F@xitZSy;yM!}iQ)vuje~Ch_`+Z~nc?%R|sJ*Nx#9Wrj$<>^~J|{s1cP4=Inu zy+ia)WGGkO=<%QrIoHU+L5$4i%6)8mgduZj#6WTx=b-|ge)F)?C*3CSiI;F4IQ1-i z{@Lm)m?5<)iOGRvcni8a8W>5^%=xbtv@w7pRLmwbC=-LSBnVWrg;7a>IRdIZglj;? zNsL3aoTGK={xPPz_D4Xku!4j8;HdS=3y1yt_d~#GG3=0qdaVKrwbCEci;NwA_!(IR zDr_qYL)jkXbu=PKuDTE?^Ako%DE)zklxbAsGw9sl;qXm|>lm#MJ+YPfX!Tw>=0cv! zsk)KVlssnCS_-3K-p#m@@Imfa*brjU52}{ldQ&aWJ@Xfa<82CF8!Y z7@a#fywO0S#9MMJex&eh86zhVEpQy6?7WYb?T$xr%=Xt^E-PTEv;q3IUfhN&IU2pB zAp{X)T&M?v_}(5s;8x(BwWpe+V->Wdzb7?Xj6*x6#Xk$F-bsBehaSpu8l;-+*bfg- zq@8Z>R{z7>J=MW=FvEj9O>wvHyq6!q$*0&jl&IuAfxff+lBgVSS1a{;}OBz&(oMv zF=+%_=EXTo+V7Q*vvM(Lf*iZW!F&9$d8wJ%TFco_OK=1iJ#8zH{W8G*(1K*mz5EuN zBOd}l^2CKx3L8#k>^=wAA8|B$8Y~%G6+`=Cr!6}7BDi@5MY5H;!;|bRI>Xth!8L?c za)h0Qq6JbhANzlLUov?iZe~Gqe0G|Hn`aMwY}EC!plUgwgU-F)o6Z0RMp3vWWz}NOWxW{RvVYEG1)C8JJhyea zgBp1FqrWfA!keQNVp-#h9c}LN=-l#p+;}X3PLl{?vif%Gt-0mZ;hmzo zL+p9Ljl+X$KHJTHKs0w^A@t`K4Y`#e%=%hd@8moC!g}S1Y|^*y7x(7}xJ%CMcgzBR zg`)xWNy+rZa#t=cYNg439)e8C@@5~e+9{bri2H1hTky2!@!h)T2%IOC6XtTKt@x39 z?I-UY$vIxRdvtN~?x+)ngQk=Ae`JFhnMx@Utu>h#(RKf!10MCR_(E&F z`+XY5Ba7RvL6-f5;T|(IBbK%rQfXkXW$Lg|hPKR2)mALgcy6Lxgx3G223*L`@Vu|! zqzqbZFXeIhweDJ34hF9W6<&mU_#?SJ9-R($TY83aB2}QGY<% zRrmY_PE~?Vb{xW zC7nvUEH!n0Kbkfn&49kpA`$SMD$cT&mHtB0`fYqQ`ebGN&Xx?NuKy*^>N^z2G}O<` zlGt`wPD=Jp(200}?~(D3|6;NxlQO4OrhGW0YX5RhI8v)bAd^WwwJ~OK=^5Fsh-()R z*Aywy@i}A=xKmDn|41zM>v$;==PR7|Xw(&B-y|l3_frHEVxdNc${4Rf)xBl1?@; zryst7zqL}wZh0#@`26UP;+$;g6K(i8Y$+R5!hw8j0`a%H(|y5v6wOXfLi=M9y=VSq zrhwfnPrmC6&+$)LS{V?LOOb=#BawyNv-mQ@^j!W!zCsrRe2c$Rue118??Ekh=v<-k zNm?e=Qlydi6v(5}RH7@kX1%oqm3gO%L1ZubQv)V`8DsIG&i5@(!2o^4W%q^6{)2t) z`we-XYn%tqYzSceB55NJEbz(jS1>!HW2nX%OUq{trY| z+Q6GL1RcTTMv(Iweo9kOucrd}ay9Q|qlnLQd;)hWP-WpVP|j3czLlNo1zYBC7LS=< zm}qVQWJHV8pXGK%Z!=cUM_T=FACBHs7-!gnsc$IW4^Bo%KelO-f9VC^v=lnhV|T?C zokB~N4VOo8{SgMRn1$K?ksnk!pR0>CVk#gJw|~cQM=rYXQ5aQ->#ugxhQ~uM8yt3g zwHQV4jW`FEZXRs!=S(!$=Y%>-eQjZ(`aU;h3B1a4e{I*=|GoKwmHw@_FfX8>`o^%B z2VeFRn2bc4<>bf0t1}^@zxvQQ6%QpYs8Ly562xupWoB8~KvcRMAdO z{hhDYeLhRFn{xIyiliL@s}K1u^XJj{$aM*7OA@NbNUI3lL+e|jnkAm`+;_ew{w1Sg z(H=>0qdc6??LxSBe$B@|!P{cJBt5_2AIC!o6%8C^d1*okYp2+U;l{I|9Z)@IsW-^&1N;(1;c1*tL}g!ZMu-E zv?{;6Gm&&Y-*vy9-?v*6(_fD>FEX*+e@|vMnwbDnURovlHdxp+_etLy^!l;GnX~d; z_P$xoD+y2wCojg@7TH=_7iu1mxMB`g=#aH;l|;d`El_S; zyl$Q4v~0m;Xi?xf@=)O-+K{ZTZ}W~Ij!?}wDx5Mdnh)8-rox&fz}}Vooge`jH_5z% z1SJVzBTtBePX2@0Yfhjv+K~2?n zo_$q&#FF4nBr45Dqoa&s zg^h=^K+89>ur5Sf3mdc$YJN7lo&Ea0^IQ+OhyFWvf=P{g)?3FZ0!pE?+*2r2c?(-o z&G(`(<*8o;Ls}H?sONQQ_Kr+LDCUqblU1Z$IFz8rzJsu51xb-uB&@J>qG;fSF?9wU zVh{b%LF0jhJ@@{uC{O$tEuizDpBu6qU=yWjLhccqkea|uH-p*`INB_%zkybFRR_d# zPX~dJYE5J;S(F_}d8CmX>OikZ7Sp0z7RC7g9DxYLVw5P)k705|(haY%vnh%Jc`i%y zQrTs;FOEd}cFU%eOsEttVh-io8=d?u(sofHSt>5sE3SrBc$C_stv>=El&(mJXji-= zU}z+6ZG`CQ+89Y|IVYaWVZPvOi>mvtpNEeAC6lnnHNO<@ zu<{@i=|MZ>hVAN?Sn``;+ZS14{7z5B{G*7QhDcjWQ=7yE^pHO6h@SB-pOCbe3q@!j zdRotXQUxJGk`9XzeiXAwg${u2UJ1BKUnKkUit^%%ZDg7=l50Ae7Ax?U4e`wlHCq|*mUlpWlSj=qHg@{S zdUw0Y(FnJ-i=W#I2`>wr>qA+ZPZW{gB@+&D8~TL6-zm1Qo+9Y7n_o=Te%{@TP@x6_E~&a4{;06m*7!|z8Q*z{ z7|g`nf^!Iy8T-KanP32MAaC3pG29dJuQ%cjrr0^x;+zxnoR&CkCkw&)gcHf~m#g)n1BkSznf(au0 zO3=0av==)s^iVKY*aF+?`~KDW+xDQZwNk3*G z7;DWQ8_r!la>kWv+RU{FDlaneYk7b12y z21m!zzk@7OsllIxs4VCg!f#&AYxp`S_)_6aLyjY*L%Ra!5~p$c6!Pb7CU=cag?8gd|Gq24X~LTU34Z0)p*u@^(^Z@wLlk{nu)Gcm!*;={?U+()ib#t#AlK!Ijf zsZbEz{spMdL_RNxW;2LfCJ@^CcKGXI8C6@MAHaEWYAVyma_DQlC7(w$4_f~<(^fEV zF+FhtNAF*Pu{+P~4nv2K1$`Do09MHASe@sq1*`p?iWRd4y~3=_7~d0%Zs?rvK?lC5 zQPn6UG0}yaY80x`dBaqvFL(v7%u|VzKf67Os58 zg@Jt$es@9U%bV*TuICqSNpgbFbU??8b~;xF=D00ooLaYbI!d76614t4a9;f6Ur4i8 zHr(q_w8W-oKczkvU`1|EXG?sNNon;X6}ZDbxk%VAlRZ4>l{DCz`>DPAVL|Z7j1mr>C;EelZR|6H}gvP{Lx#0Qa8HOYauC) zW>@AzDNK|Dr67QYz)0daN-OR6q-l8wrt;sjP3?Y8UCpcmxeyNjeLWCvhn`dvsB^3M zqmO1vwf>}PO0Fac2cMx;dGZkl3A(=u6~Ltz$n`xtL8~K(fVh@I z^M$4$|3HMgSn5IiyOA1P_xW7rH!z>d{PuNRPH+plRsC69S~ObEkiLGFAnRAhzV@sB z>rgLGFNfWLxdJ#=A)8sJpQF|!fH^(n6p`38Of%Ndf>vc<3aqMSP}(eGflM4DJ}hts zt0*H9IFzgq8W6&^>jX`L35O#DiC~|=o?lYAC*;C-vnG;H#VCYNStXm#QgJ&i1b+qN zvK_AA7St~4)!dGHUaZuJ5G07kQ@DP#m(iqep5_maIaRB0`vHggj#G02%qEnx_w_yI zk@b(J_wp+XXBUdwDrxQH#(?Sx!MKo)+JdZ6l9)kh{?=Ij2(Qw7AZ+h#i`Lg^^uGwX zUqHWGXkrrhRs9;i#J1Ny&V71FK+Z*o@6vxGM~;7b1E=D!UobT7+(b^Fq2RXQ^h+Wn zNn-zw&WH35gQqdQw~a0+u#PPVrw{uhjdCG?(<@54fOatTFG6C? z*u^!h4ca(thP(~s2ci$UZiro^9PZ$pD&@RlrlE_l8+|6x1GN)*+O1KtM>~%SlZSf7 zs*g<~2s6yHhZS}v!H?HVnh^ed4KMuG+^=R8N+foOlY0YqVi5EJ=yMnF`%CN%(ha`{ z&cqL+Ung;5FEfMk8eqt*bZEz3Lh^nu+Kr~~{jFCc{_{Z8liYy2pCa-yo%%IS+-`0y zy}-*X@-#SimmN!wa(v7160_Z^d^7}|_?4mFTNq#Jc1=~IB%M+`>^b`Mk~E3%p)Tbdn|dhREXD72pS1ffjJa7xvUdZIe|5&yA&1jEq;W)lJHWow0vgV=m8 zauWrD?FH7e)Q!VfvH-d|CPnN;YWMc1$5{j)!-`D3%-ZMGUE@rGHTkFe`~E?fQHb?* zVt2ROE9qwm(8pCh-EcvV*+v_&NzldjxtfsQ+;#QzvCH`TYt2ElB^77Bq+L?ZNB(lM zYpu4xN>99WGg~_=vnqT2)#aXE?^(d3RI`0b?yRZh7S;8m@x8RmDF-_F3_3S?C+|EvKrFnvL2c`4s*h#ytMSa0|=j94}Oi*6^H{C9saJL@RhGXyf8{ zJ_dy-Aa*&j$8WnaFfy8QN@%#fc~36j{Deq z`l4y0?=45S5-wC;htDg6D+KHABB*Z3X$f&2y$6$7^Kj>9aHcM%CdL$L$Ow8&vxh)) z<08L{--v6Va%e6Y>I2%swPy%D|JwcV>GSqPNJGaHS{tMeu-b7FKg-52zr^-7y5kFs zZ|mvd?y>Lb*5o(T6ps`(Ao7V{N37j_BD}7am?388X0sT!X_wD1_CQ-<9B25R{zW4yA z^>t?q9N}a<@Y-+u4c)dt?iZB@hh;^FSt@ova=3c1 zGq*Wv!Xj+*v!1-|uTMo1Hn$d7F7LQQam8LI2UR5@$$yn&7@iS~BIfT%2F9uvyoybHF#CPKw)QU>EP8-8BAQsw zsu>$H;EVlB%EnjZy&F2@n`TGjGfhiF)zV7FXZzAgXXe7gs4xd&RYUkcURg)gLGs$) zQX7&PSuH*v-_)UG-?fc59(uF!hVS^Q>*8+sq__l1sfMi+Ag{1Ln*6d$G!Z|Non;W zpm}^dg%@r;adcGp{PiN+Qg%UxGo^4drta4D6^w7KeL}s#du{qw90RO0Bp$O!bFkj< zgw$#{vF%GFdX6+iqng&;qiw!UIf02=r|UR929)q0Pc~1VXQYUtYv)v3p)$+y(}~}8 zOK0B&|GF!D6DXaLw9u+Q+vw(JtI$pkWpjSTbZZwyayQ@HzrE^*kne$HA{(+D2I;7X z;v4C(*#dkbK081ITY&&LJ8px`B3XEX4SCD^S-X#mk5QX8n$+mZoDbE|cI03i+m<9f z`X0`|KX6ahA&n1vYjU1r37DfjU&`*!Qp9t!1h((;ONK7hDCRlZ@e4R|(^#4pPZYPe zsKDR(r%B{tF2s~d3)L_JPEq?&s#-r3BDeD@Bk-IQESPuun`*`#J?u5 zHy3I?1A0rc-5GpNXXDJu?cV2lgUK&|805ACANQC2Ue|MT`ASbkPQCGN`MKAzhrbu} z8ehA2=d78MM;PYPe!LejRo1X$`h)$Ov<}b9?s$TC0CygD8kp1wD0VizVO7IUnviL)i)c(9fB0 zhS+O_4J2K}Xb6nE4Al5PohrzPf9T5~4Y?2Kt`jUE%f>pWfP-p4`D_?;YHIj)Lw28I zn}d_j+bEG8o_1RAiL=wi*DAQZk?)*2jL)a%5ZqmHk%*-g{3hZLDkWqPy?f$nd&TBj z^9a+=rOuzHO~I2E((N0KmrQYx=%p_u6kr?p$CipOAu}^3fs2o)s-GE^8>5zxl}_eF zGcGMbm>89lmZk7d&7h7CeU6R^mpDb;;$v3c$NMMfTLpqS!-D=?7p%_zDS=@ZAsu#& zyu_ls&&W%Xi}0}UYJr>1_daM?^6Lh~r&UcrsLsVXgA6HZDrxonGvG7eyKF`-6Eocw zy&ZN{Pl9o#2pJc}8Yve{$jK3!hSEGU+12NPQv;k>$Y+eD%3wi2i2o<8 zb={2$0M@wb_<-9@77vf>z4yuDXS&Y8H|~bP)YY?LH5Z1Sa}%jVbDO{yrp?G3Yd!C7 z7Ppf|85mWZ4Q7ZcA?FO0K_L|j7+F=y^E3$aUwc2)WIu`g$%zOFzYCC>f8|k15Z?Mc zkwy9?@;^Oy3kclY>)v^LU#A-c6;vK58&M1cU@bAHiRO0MHvlhDyK{Y~DpLo71=HCZ2(lHIHV zj*@=mTdqe_{*IX3r)w#^^mO4n{f5g-&a`lwNx;Gv-sJSqPDR_6Lc)|`OY-O7N|S5_ zpwdRyVFJ(3hgQYzcS##fQHIkZB}Uk~p9(wks0o#wA>W`PTVbRrL8bGu?MjZV%-I5^ zYFFNahB8didzh43Smq=8lVMs^wb4Dcl6qj%J)}1A%Que)N@Bq`y4hmQ77`q77(*;A zhagQ%8cV=X+jm!dyvAnL0b@t^$9ot%z}qr69@d``G10!oS|~fhIFVw6gMykpkrg6+-yKfKBQ*?9NZ9DM!!;ES=R7G&iEWB5WuUN zAM7zYSC>cDE?R(?HEhMK)cRV==oXS%TcKmrJIKMR>>|yRfFjIl!mYr%Oa$q~cM)wa{B#$n`LKQ#j9~7`+z6XXIBkJP2 zfi6vzGy^hcEVVct=fSM1F;sp4P}2|22m6lG6s*6v3k}DDFFHgeImj?;2|+o+a>=gz z{l1YZ;fTvxIO{8u?E6!;$bwqf=-sLD2kUnc&z!D6{EIO%BcDeRvOE_5<)?yfId%63;NjH zD!;iWewa6e^aJ}o>8r)0BwQG6v0!m1o7{N}Dgu_!ui!f7eEUo$s5y~_K39yOA#8mr zi)UDhtB?k96*;Mq@bP)WKtWAn@NhoPlAp_PKZ)ZLP~wfTTExkFtr!*$cWr(o-_pkS z{?7Ht+W>90w69Q#NJI>90+Jk+E|M^mmK+G>}=r{WHb%8y5tbG_RQi>=Yyn1ok_^X9vhH5P*MDKExM0FUWV< z$_;A=d=>W%2?Q~O#S`>AhCj5Q-&zegHETi&{u>L17}})$JMGKJ&%bWuJ0b3JL)ISt zrkGIX)YMn%lzg@kHz7_959TmZ5WCmvSK@SC3swNq>mO~-ZD%=~N8{)Dn5HHI-{W;0 z@8}R0zCApGEz+B54FI(AlcP0d+UI`oV@i`3FIJpibeO@m;4!)bJ`~t|k-qEu+oGP@ ze&i*7nKMwkd%L6Ob7z&R`)S**#{afFcc(6QN%yJ0znmP|VJEij{wC(eN=Lx=5S(-J z^*E2)vCptA|M7Hm)G-X7SX+zdbw8bcE3maJ+4w$Xb&(o;4m7#tdRG3coV`i9_&wwE z)ph~BY#UAIZFoJNJT~Kd>~tM%8RX^YcyNt`Ic?i{{|J2h^~${c@sy3((}>;UR{pZS zc=#dxuNd`T=Do+&^DFjz$9?II;%i&>eOh?u{mA~wE|Xyc@Oj|qshjz0{CWrd(KE8bWxA zazkc_MIMq}FLg+oLEh@i*;)G7e)$=yGTS0s0m4bs+@6kuy9(|c!G=@<2DuzYsafpI zl0u^)iM^RZ_5yHXNgI;6RBefa01s_ zvO%phFhpdA3lWy7<)*4kgp&acOODwt*&<8KR-X7PRY~ye}Rb5;r4l7f{AVb@aQqHGuofgd@ohvI% z*JPKjAh)fO(~Gh(&bw;q9rT(1qxeQw+I&((VI#cWnWaks>3k`)r^xAD3r3x;S%;oa%H3GT~%cdrS zW0Rk3ywr8;CF?~WoyGs){(@JK*#sA^v@mioQUVvu&#%W*v zmqQK?c7?~Wf+uDFWE0EuS1Oe-CDN7<W8jxgRUCy^9>0-DJxwVeB85Rs^)r+Yk2 zysr$(8ky*~aELTj#h;1H^z<07X_OAXbBA>Q*|EssUORk|kW=1S4^3Xx@uRtT z9o;I7H}F=4q1HGMJpU(*a1r8-^%K3JstQ6JhjRP}|KclRia}4bczsTJSiC_d9W@C} zk|9ml$>77(J(0_V2!L+%x}O$u-FI_k%w)YyIj#fuI;$oB(S?gGbqObi5d5ed#Kf z42G@)@%{y*VF8p*{TPmjBgU|WBsF{k{QKpmk$kmaZy>ycFui{Cf-v7fs=!5jx}j$; zMhbdjz$N^A^=4iW{@xgq=qLVQ@%>%<1H$_DfCt3wjSbbWUWEJ#jhMSn(N^n6z4ynxB%cM9b~x1Rd<{vjahCgb&N zZ9e9PmcaM28`G=jVOty*^XsFx*j{ww$>{ZOWbH(choyY%yVhAp^q&PiZ4b*oQS}Y) z;+O)bfB*Q`pPb$?Y@gPCoP@J@UfS5{_*#D`#>II zrrYx@*&4VVCh6h%IyGWC#Lv)m8vc6R>j>A=`+NE(Z8ewcOgGMxdc*fRv2&wW+v(xK z?)o5fH~y^F{UNim6BH@0JMm$%U`4a@c6kBde=OY}+aYPFE(`DPOkDAGcG+!wZV!Qg z`}V9oH*MdzJhxt~|Ji&*52QK)!|8{QS@UHQRRMhMZ{`==blPrTUg0N@w2e4UPVSXW z5ggr`9(fl+_4oTFoUxGS6_VDQO;Yh|*Ho?;n17}owJ`?agqg9#GG%QG5^YuCP5Rg0w_W$kiPPCpx1GM9Cm(tZ{&aku3=CE59``rYaA%Za zGAHWIh8sXP40(gJ1s6yj&4XNo>DCAmutI|_Ng?bn?U8m(U}4+1>!>Q&kuJ8@eNId& zpW=PiQ+@1hmiTXczs^#`j1HnJJ)PgO?svYv-oKvqP=BN5&-&cpWpKG)>X&LF0l~vc z`3k2QORkt%u5&gkQbS6=w=0%6T{$(Xcs6R#F)M9q1AEw3B46C3r-luin6(^uhO~Yk zE`nje$DBNA*15LS+j`KwwtKo5{PAg3ampS}li#{hvC65}uu1w9Uf3yV0@&4Dt!zi= z+T|=P{?&bR>$=ykso7n5)^~9(0=(#Ft-JVTy#`A4Yv=BpR;dvtVfZ#pnRKF5S!xr( zMGJD(@O&7VO^!AY&)H%ol-bMD)rQ)&>=>DE3EEcC^A!@6`Ib%)%olPaI7w)nq(W7Z z*`{;;UTt@X>}e;i%WYQpOzRfvdJ64ljVP+MWv*EZa!n;OQv^QNQ7SGQ!1?q)6ji8p zUrjrQQW~FPTKds7Jef*x=2`0Tu}!j^HwDh*7>%1 z%ia!kWn2!KRsWJ$`7&LHXSt^9fHOMcVUhPhdB@xRlqHFb9Vb(7<&|1HTPff9<7w!` zwNMN5`AFSz6(jG#k!dI%fb2eHkE`2qMLkT~a^#%SKOGtp=8xB+t!|_$)SPx630L%0d46EFC!#FL`uBN49?XY1GT9KAs(!xedX;-vS zfubL*M3AuKY}yy3uP~2mji+w&`FW%^ViV}NqyGra*el*NJRAe79!r1q*gkg1S zTCRkgm%dwr4t-ernH%YjS+H(4N0?5z%sknFI=njCPx!5L?#?%8Q&;2ai(*E@%@de| zxi7*Jykffp)j=RQp!0?-G8Vx*OhfRa%NAY-f3*y7dgI!rL<4m@kHki}D&11FuPJ|3 z(L3|lLc|4^c||yP_EZ~MeOGuxXIH9jQ6MquiV*oGhPg?0;*8GpTWu`P>=*T~Sa1_5 z8}qj5m68Ql9D;Hd-Gau89E(JG)q0pY3CwC#9OQC!GodY? z9K0eNn4FrqoE)Cif`B=^8H)m2h_r3n_vC0}j-NYwAun|`bvpJ83h3s~x5N|iUf^*x zRX*lcxn$W+&;%c5G+DyMwdK6G7>2?b51GtBoI9!9`Z4jKHS0H0u2+5gQs#w`cK`~> zZo14qvDMb=!NHZ*y5^s~=6TRpF=&%+hh>}xJf9=XnPaH3sE;-KPJQ}zc01r=AI554 zFk8-Em_;uE6#uYXrt+p`R|VQFmU3fkl~dcKN6(@DHNwzVYild@nIIh(qAw{TEPS6& z5YwH$X8R%dJS@SWlxXV>@V^_d@4l*ogm9|iGq}(;a%NEuV3iTcjtHb1+*c9LN?t?& z<}Q?kR|U1>DJovcbos0#Zrh}S_Nd?X6G;r(BEgT;;Z*leU*e9pVhKT$(s5{l%t#4p zf(?clO6shy_ZVseu<^`nUX)J`LW#(D%)gEqGzbm%M1(k4@Mi2+8)=#+^_u%(sqWS7 zt=~rW0S5?nMf{6EgWrVr>JFYca^VsHweJbG96*Q-tHUHbI2cy1F**+?%rwx@^m>~@ zr>Uvs#QcSR+GEPhg@BQ8WSky6Od^KcDp_y>-J)D1$LXMXtiOy7FxZZr6=fOCotX`EOgZ_K^7+lW6MFVM4xahwoQ8Vh#IxlaX-Es-*cNLn1Sf% zw6Bt1@gBAnOR75EQ;XfqsK?(X?kQ!CnOu1tRNrwZHmtAKUpf&mdx`%1gH@$j3*-zE zVKg8zu~5+6ozkt`lzx0h2fhOimn0CI0Vc*as~-_r*hLd731n1EO7`xSL$q-`SY2Yu zeazFl`T!R~i@v1PiAPp^*;fjb&Tx|3mB?zhnE&X#_J>5K)!~^`j)S;;?#7Btbf!Ii zZ2u|GEeftU*zxOI6{{8$3GIin?@d!*?XJgci?HSBQVP3gc@@ZegI{^2^oXy6O-BJz~Qh7 z7USCMfXZo$bI9p{8j~4zU~0daIQxR3Z7tRwHicUhkFFh^)v9`+JrzLAd_DCW+bYce zRIS|C7JSVIGJ1!A3Jv(qYH=w$PW8+5V18VZ9lyCP9b+TCUL7Z~OXAL4i%{=X&Ejws z+%R((Rl)_1Ha1umv1WsE3^~=JqSKr=13b5t${)Hd{fvmckcIh44PG&2_@_w14houo zPV5NEZXF+^(>lj*ziLvI?2zDc7^$^+SSZDDVd?`7jB>W9fPUZ=-&x-BWBT@9Ny|ja zTcp=Q`nuL}DA!{62eSX+V~8-FN-|I7qwP)P)o&Wvrl;uK;O5M=ubm@y$nz zdT>!Z#9M@f;zj>4j{UVF#D>A9tf-V?homBFa^s9<o ze}p7!Nh=P8pzrOrm|zE(m_Q!p#4hCGZjJ9U6QP-uPB?&8k+C(tTYnC|**S3!#rwF) z!F}Rk=&}c*X7Q4xs0043c;&K>{SXMuNGn+JA>ab;<#7Kbe(&8U_F4z6WHv4COA=2Q zcVEPmq+?Qt8}B9L(V1UDdvhXZDg0Dbu@6k+--g7aNqU-@r);s z={n3#BFn+m0;X-V7=Jf8`XS#Bc2C2+1CfMHkY=gNCcmR{g=EHcizhgx?)|v(%m9T5Fc zBxI^9$~HPms-uv`C4|l6%0XO#GIs^-BSl*8#Q7ndFF={8aes+=OQB-G=mcf#RceVL zez!_QzZcr77a~RUq}t6Ys7H(YchzhcsakGViI|M0K%v{>iL{ueE~&Ga4SS|3^JXBk z0#6-|NE`D=GVTio@ja8`D2n4@Dh{-u6jj+BJ4K#p#5KoOBqjl#AyF!k^E<>5L@A|` zBgckq@hC9ih<`Lrv6#g9(KL7&x&8&v>vX3_|1c%rBT%OXbsmirpiVSS<&WUyIlv?t zD7I@-9D`!V`h7E*7RM}qP(b3bIC5G?wnANmsH4}L#<0_ zg?!ti?4t%-`ASlzNZ4_VoOK-9c?jT%afPfDW~#jq4}TDX<{OjeCdDlDn;q0|7Wys4 z1rVRycyBdM1UXU+T0W2#k3+FZF&E#D$37Gfu|x5Ml@@c%cy3ahL}s3}639#)nHh(m z%GXdVs?#Kfmuo>YkPnDijTfGer^RE;IE}xSaR$D-6JUHAJshBOmH~|JQ}-ULCE@j1 zFi_z+d4B|d=8nWQ{y3TXEttBDKp)twXk0o7)eg-HtDv#KNL=(1GncTw3r2=g9-LOA^`_ICEB zFfOU$Q;a$ZDjr97TWU5-`TnT(#@Zz0PgBJnu*DK#`jVP07J%y&jW77UD8RxzPpeEy z=6|R=6;whSNAtOvBy7V-K~p0?I0-)YD?SNPvPD_rB}&e;SfawAUNn#JmU{_%h$XTR zO=As-J}q1$CKfgBlI6K6OxV1@56x?Qkv6h4a&wzcVWT3ApZd&}sGh47u;<~~*ATr^ z5!2OiNCvmyO!RYkY*-D z3q{vjEjnAYNdK17zXtv58n2~AD`{NdA$}P8)cv@aJWpCtQ`t+NH!iBNAuSp-h2{Bf zj9g8PH%uP&XS$1Orn3%TLBd|j59JZ!ys;OHXx_->tN)p|ur~H;yy@dFVW-WH^nXpY zsTfx55XGLl7Suo5p9fmxwin24ZMf|%sOk0G7MI4`3gReneSc8APv(tx^6{CIG?J;& z!+3X!J`D@v45&Nc@*aM>16Ni%4x;*9jeke=lqpmJWhKD+yR?y|CPP_?jqj1cN*AJbyfFioc+l z{}4CuQO;ki>^`+24TV6OgRgc|+Odl2vo)s&L^`tzxt3U3fCcwDf> z829K#tU67eqV6YsNw%u43O`!xGu z>#<332_=1Lu%s_h@Gl|wpNose7viGEFZwNYmY_QLx@Ay?K4~uP6@TxM3o1NLsTa5W zf%WGR)9z1%Cnv?55b~xHaswmXA6DPI$^Kh|oaQEB!G1gmTp^8Ttl-+)gV(;n(8mXP zzVp{EqlJDjNbv4oA^6reDE=N*!TSU6uQgmJK{r>@<$r~ zkdnm|ALHgnGGKCLM49xRi-MCg& zfyqHN^cYzG9Yp*iem}v-(dQcfC0?)M_0vI(KO>DlQyTvY8vk13zoIW6T{Y!wZ#MsR zPxa48_1}QeKOrA}n-jmYOi2Slc7|oCrRLO!-En-_Y5aFCqy zP|s-mkF*0xSI8FnFCqSIjsFRo|ANiW_lYkUvVG`r1%o4^`%Nuz@P~j{4#kUDZcza0RR91|NnJY3s_Xu7CvXrz#MUoP7H!Z+31*p z2#he|pruE71%E1DtE1!h3xBhFWoqz}h@cdlHCkRQlC=U1zbKrYj%j#tw_2l-1%IVdBQ1~xt*jIrbkai5?}7Q_ zFzz);2L!cFGfWVavfu)wF$U?NFj$Zd!m>tbpOe3|pBML`r$*YV7ff z1^c=OjM5%l_+W&?n|(gok6DG&+?|bcXHN*-jwa2^-sSU@PIpD z37kNdlfd#Du>2M%ofHOv=peS?0_Otx0>%{4RTlKJ3#LfcyTArvsaSi;AQ%&*>UUr` z1q`Ru9BfLDb9w=+;sT{oU@R46jX^V9&}fGXI)8nrP6*S6>76-Pzw|9w|1@}?2JbV^ zT_bg3KVCcw`ZJ(E7bu+tma_r|r3V^+3{0<;{q>OsaVjsCf#EqYJRc~PfnnMIHuU4g z3t(6#^WsIdL6-GOS`TZ+MHYG~4N@p1Sa#9Ny#x`M4ZK(m9_PX1l1aKOsK>}q48dsa zuYZNi$ogKg)~M7;fsh?2Hgp#B1yF`Af{jcFLNd_y@|TT*9AqpoXq2*aiLH*yMyVXD z;|i>F0X9*g*68dt+|z#A@sQxFFj)?h*Pc(l-qAr!Xyd4a>B}&E-BBkE;Kdv8cm=#} znxyLguHZy3sY(dbJ+IT7Y$R3oO1=q1)qh6m1_pHtn68fD#TwZ^N)Gau^+9r!Ue-oo z+M)!Nzk_&Q!u#oIl}An+y?tofldRf)w8K+G)i^2|0daKrXXs>B(VMyR=WXNy>F5p z2*U)ezMu4$*6F5NW1#(Lr28z#U!Zmb9WOpGN`E5TLtv-^roYuXgRT7VNZ!%N(4rLD z5TzD&3kuKYxA-x}|JW9PGsORh7k`^=x7fwr6CTL~9^)XTRJ)gb&#la!JvgKE z40h_l15=%b_fV)gjY>1d%Uckfobkqc2yo*)lrUWn(+(zY$M%IpgRNszI)B2%9$LYk zj^~ScJFts68oiacmuC!J+ke3-AcfzlM=m?`tPx8e$_2i4V7lTcN z2Z0PG-#0+sBs>IUD8nO@{cw;V!e|phHpmmo+5eCyjM2Zv>@k9TabF{7EyN=Ux8+kw zzN45tjA88GW?~yhoVf0IW~~W~{Y27Z%##_;DNN4e7#vSFZ8<@l2!E19dqiv|eJW?K z3kBh6gfl_17@SS^ct$zIgLpb&^tI4lQkk{TH<$6vBiwctO0s{K+1DJdy&mQ=G0Y?U zK1e=;7jW%0u#oVFTzg$1=OW@S09j1>Hr}?gTf*7z2I8f(7tCi{oqWvrwdI_nbw6Qj zJ|$nQ^Ut{Un*54vaDUCO31j_lBAYEB-;f^hcMSfX!9Ov0JCoa;gntIv#o!{w{uc)C z=GyCIA%piZockGkfWe1fz-@ah;o9r!cdoszN*R2X@`H7Fk!wHeOUyY|F#I#P_S(2b zHn&0k;M#l39mf7H;*eR`_iT3)&$F23eufvTz`9CbRxZyYd>>EXkHd@ zD*AEue%EBT=Q|T0=TH-v0Yf9{C0dgca=Oigp@H-riH~y#dIkGJo!yRC(E6Ne=V?6e z8uEWiad67H`KMs0CjS=BUay?+_Y>^VRpkFS?SWSf3dMa@l0D`K`xEaN?fwA9YfLUh zn?BuX2aH#Vw||E6A}C%@^M~;=*4L#{OGQdgQ~s@j^o~_Kpjr$FszD3#Zud zE!I#O?FY|Zvhg=B_J#2r*`Md={iE_wuYmCk>CaIO@QRkr*hfy29_MlYyu*Y@>@~lU z9!F=?D-*E{xD%xRm7{l;;#o4jCyr5`@ji61<+(+V`F}r3_IQU}Z3ld#d>Q;dNqY3t z+Vb}DZ0u#HNROlKJ*nC}&N4DB7ivZTK2_IQYd#xyFUvi|C8m6FwMp`HUO#?r>WUSq zUr!t1zHxm1(99~AV26dFU7XKcs4vZXqwdEE<$pJPh$j8}`ce(V3NLX2zc1 z@&Y3_-A!^T?6ZGK^Zc2i2e_u1O*`gfy52J_pVKluV&9sOIbMf5t?aw#T+5AycN?;o zF0JhxV|q4+3+tX6yl9HbFPC$WTKB!4*kkAibANM2I!sJxiM;7EWYnX6R~LElFEtkK zT(xlO>X9)M+|ItUzH(mGudP#8Z@t(pzpSM~HE>O=_N?a9jjJbDugEW$G43-lqHag< z*6$~APkHX$Fht=;s)JH6?D$oTZjP-))* zpQNahCVE%B(-4_|lfdBybVL6d_;m{wl?YTcyrX^#i3 zja^>hSsL^8>AV4bu2p{YzIv_DEq%?5oGTm3C(gNZVg3*1fWfPs_w-&eVR9=tGG#%-c8X3`bd|+|!L$T_3_u|@d#q-2#Re#*?yMH0S_vG}0 zhNp#+*=f{tuiUR5bpJlP@#^4OAJw?b?Uu?(_pZlA&NQ^%n{unPhx*-Z#kZEO(NFwB z8{4&bXi)3AH{uS)Jbg7c@_Xkq|11ff923gpWIbf z1Z>;6(0F%qa@zQLwTEN(+<(c;pUe$WpX%l};$+XFjI=$m2mTrH*v0eU zk$yACLBbn}3-Jb!J%qmm*-H6bK^S?KlMmto@Kml|gjGSH|k%T=#f;oF_^d^k<_}eUi z_KV-Mex%3Wcs}HdzXSUc?n3^E@%`L`YyVx?t>=0i?Md7+*5%Z=^3kuIlhB+Uby%OzJ**Ups*?(D9RH%|T$!xJ?TjH#F znVIpH*=XRHotK-GW=ZFi=7h8?)bMHPoGL9VF(WU@oWym=%StjQr)8OwQU#bxO5&V` zj|(3e92XrvDkPoj>}Tk2Gz{qPAJ>0?!Ovi7GZM!KMa6|jhK^3>I;5oK>im5CeGJbh zM@NsLNyoAMjeo-p#%Z=#XP7Nk2sWK_F!~Ji8Gt%vZd#@})e%lwRT=LPFU5C`SHyR) z1#ZpGvm~0+xuL$}vorEC%~s#w_}qBkVDk)fMs`l7IV;yU+iLZ-m=oeHzE(@39mSa~ z8HsU;*_kP)h>@6aWAK2oeBGNUcPi<;Z1j4FCY_6aWAw zlVK_pm(*bb6_e-d4}Z0ao!E)Jv6R@3BYS1Z*opJw*_OOG#ExStCWJJ~T3XABWvym+ z6$o&L>$D};yWV7caBXSf0(Ts=OpkEDg$`4ol+xkK-4NhXW?(|vLa$6OccJNp=I)Q) z_pP3T$fcfnv)}!`?~l*-*L(5~@0o}&OyI-8#K7o)ZOk(a*MG|}%xs;hPitmMH}l6b zc0QFh=8L(!Zh9%frHzcvXLUPO)a=>nvT(`Hh+D!i>Sq>iME8XLx1|b6&G+mc1H9+o~zsP8)@pTo!>x@0@8O@3>E4V!qm}ZuxC}j2mbLhdkxrS0VS9~|O@R95jZ7)8 zr}R5*pJkNHv_n>zdb(s=c;I&E)h9(hA2` zn)-}x>VJi_o*_xHypT0w-3Kw4dNHr1AvWE%P(i<=nzV_Y*XQ+uU7gYM%e2=fVOq9$ zovnz=)V?FIc(ByD@l`7yf=lf z_t6y%=hLA>-CdDzci5M+Ql_3Q92aQ*&g2v-!LoVGR%k&{e$ zrZwoz>{X9C0U7vcO6neyKWShvHfXGNF4MP}s{ zihqbvh!aZ~1-uM{5$9z@uPAr)phX-)Sq2A!cuA@a)P}r@0-~5hCuESojs%DNqQojU zBNn^-Z^`U!1{X3@~dhv#wdn;EeJPxH*n49nNy0K^o8EOUy`p$KaLq_CzQFADX(O~PuH zuP3<=flL&~kQ)n);A}Pabs#?m_iJ_`mTv$In}OX}Q^d6#sS-vREG2PN!oCK`R}2T^ zoctImTnt!ik@FI8UK-0Uobw`+7k`lth{Lj^`Wnrm zRDCfun3!};4$EmZsZJ)+Ht@DON` z@`KirumKb|vit_e_C|a|-+!&7`fX7C9+BnSYsk#xYh#4Ypn(%vAhmB9RJ zXthlsxLslSjv7U&(_FC7z3mhC(6%S{@Uhtr$@O22T4}n1cA|VH#*y*&0Vder;^a#tph-?S)|hY; zIjs6gzib2JQy{y&PuPJ38OsAG$YCi!rXe8(Nhq5>ABsqGbbrm>j+n5G*xL!F_Jb+h zC&a9KNJHBM zNq>a53qveF2nR095B+~=xDr}CChR285+u}Nz;_VvsWl?$Ow!-gZoYF__GZ{V1luE| zaS;KYRmfQ!kbh3djC0XRaE9@65h}I|WK$q}%NAi5fZv5=Q6czY+QsOwJfsHIC?1Le zU@VcD8^e4T%kKs)4GKBhC+tN7NDB4}<5G>RIFm<9swlz)Dd_|`^v9s8dsu#~T+&f+ zxaV3)M=RqT2Tva0vHaecu$x3YfqiKNQxXBCN${HizkkU-VaoBlMc5~8g(ZXYL5T?r zN+MiRh+MwnxO3VEoI@S?WiYj`Y&^;GQ!!zZ7{3(^>M(ZuR~&ZI0yBiAI1VPANs8gP zL?)5mG4aYfhPT8Cp=D6#4Lgl6777j&m zV2E4>+cA zWdj+5MYcHlJID>oLutZ?pFRu=%Ll8P^2n;8{FU@Klna(0mPGf7e1P%^y^7NHuvS;t z2~s<)u-YgNf}x#aAi7~HLv%YU@N`idN(F{xJAZ`VLV1LS{~ej`3OY*t+ytwa+U$e1 zrGnl{_4{FMqq;j_?X19nsSL3*Siv8nvBY5|Xe@*trt*YVE9jf4?k-rjP+ekYH{}zV zu?l)mh32Oj16=36T;-$rSYmje~C-(F$!3l%jzOmUL@PgnTb zP=7(sP?}+XMmF=N4Lu(^YJvV(iYq(S(y7y4NPHVEC+F)H@eC-W* z9-{Iibf0zq;I+r$d64o|>YMdU&3zM|DayZvJ{jcyg!I}hl{-Ljv93o0&+pFJfPuXN zQg__>?T03BG2X`F`S{3t+^+9jSbSBxUw=w|_jupbCjZH%lYK3r_{#_GZhigSZ@w_s z{OtFC|BK+g58k%Zwf2YJ|9a!!AG-SDv2R4K+#k8}%--KMU3_hEd;G~OuRQrq@_fQ> zI4fzteW&4<)i+(gZ)zeFWb+*eQ{*)`v;zlwSPRp zj^1t0L{^@Ay}9k(SKnJWb0&WEk@k20kbl_+I6TNj} z;m3dRUa0r3;2HerV`u;KyXTnFzPVjjo_})mbnwr2>|a+k_N|@2kuUt+$JML_H`>$VE`^fwEKak{~+|&H?Q-Ai#)UU=m ze$<}7@6E-gZD($)IrGHH{x5y4XG8w655BzluWL?Ue$l-3r}}++Z~td~PpR*J-@W;V z&-Jw}J-mDAna5YC#||0;ufJiw;BB~cZ{Ykt{o%yczJKWW$0zRHeC&g|j(_$~A3a+A z{5@|!bVumZr+$9+_Xity{_PVVKlt#MzMXyP(`Wx?E3C$!n+#InV1VX1kX1y#uyw!n5soo^WPe8Gi;HwF=HlL1e7R zM@!GQoqhGGuEJ8XG`8l*k)e?S%sd-$XUpB4(N5*s=APs@-SkXEVuMQTh=cX0Zdw3# zj&a92dpmmwkF#_0`m6^|IPZDQtqIyn&81a2z{_9Z=9s~*DI;H+*R8I&W@}w>{ivQd zisb81mtk35&KIyQ%S?CO_+w=!FpiKyPFv(oX8sRQO9KQH01W^D2mnh+twgz_$Ka5Y zU->kD>m9E>_Uz7r4dyX;7BKiV_O9_Oq=W7CV#>n`hEk+tJ>DJLgLijU^8kWCaY9Wg zN?S-I=z}(-52zqClm;bDo0ghV(h32gP3eyaxhSa$sTxR$+NMbDxwF@1X)PV?nRCxM z_dDl&cYLRJ!xK#uRTk}vty;IrHZm0DxlB=iRBySd4XI{Avy!Hsvo+ICG<}ywmegD> z;n;fGVzHK0Gg^^mrq%37sS0@#wvkBdmR(BXT!+$pWYWkE>m$V@nw>}+Nj2>o*3(*A zAM$Hi%Q3Y?N;f^mSYSF*&nDA&H;cu7I1jVU(6gS}TC_{g6(vngwKcD1nub|E9b=7u z!cN%{1bfWbl13(@W$hvzBO59jHq4A_dof7MH=I^&Tg&p(zFNjfsOE^p>jrkf)ePxb z)qI-482md--PRIVEwgMjn^MhG!Ua&AuP|+lAdr&$liMY)I%Jr38Guc>fl$`4Q9Pp| zSS?A80%k3dF;bd0nby=SpHJX=xmikoQd(Mrtu-|{%HTL=R7>ZIv6jG^M>ja!S3Hs- zP0d)02Dq6dWb{GNxV3KIwVk%91|R9LYw`{o=~UXEHV8_K-Jz#bNvN-Q2DuNC;3;q} zEEpy<&IV*U&?Re|x@Iw;TV5-d)@?6Vn|!9WT{A7s>sZhP))~{D#!3lP>W~J1lw(*0 z&vWVv5;iu~r<38QWcZcqE9)P^Ym5{u2=z0i!Sk?_1zR^%PEp>sfzv#d|Jx7J4kFOJ zCs-}gV%ebR6RGwI)G>j?E<_DbK-iR%M z5IkZvE<%V5xd+=F)Zw_O1VAOpj?gJM70P6bU)V&;?T*ypNMe^nMY<%uyVi-sWyK$Y z1PaEcbccL$4x{nU23vm~Z;n^^SkxoMqLP5?iw^?KaUXwB@mC2LUEXPboR5V(LJSEg z7DYmg5m0~$gaT0?e+)wmL(rM_*dyp_9l&S_e(SZ~y9m zySNVn+~3dpM3%?n;$^CTS48p2I1}UrI=2SVATo8hg zK(n)9?+C!ED1!t*ZGTj78PAMR3L^o>S(%*$>y_ZQ0z&5~f`oHS?zu_cI$Vhm<$b+>7m&uTXE23N!Cv1g;y-;pj zJJ$-6X+>fwkB1|{a3YxaI4y#g>l{ePiv2v=Xqy9n_$A(#$-ytZ6V!zu2!sLgfOtWQ zKxhyaq?oMxK}v|d45YlkpC@_%BuM;m>@1>JfK(EnIUsXM9vnN5=<`7q5FZg_5y4aq zB9Z?+*e6^NyM*{-`%dm7bD^y!?_cv=c-Y$f*T8mH@RFTmMWU|&ge zoaZxt1^TLj{&W{$#=0lA6ZfA|h^>pP%S5LL_F9sI1F0dppZvPg>tqKCde}?sZ6pUp zM{fLnt#0F4XJP4uokyJ)-dS@&4(*FL)(@ukO);_NkpOe!v+B(+U2eU*J@uQ-?6g>(2FaBf-l-g{}&l^Xxq?vIxrc=N&Hil610 z_Po4VDgVOXzSwo**c-1us5<9+>iyksT^RvtR_y3;py^wPxrUl#xS51nUk?tYGc z4!!ofBJ)}E(6y3&%Sp%eDmE? z_63K#{n|$pA66Wf6V>k?kXrWcS(E#Jj*$KP;EuhMxdR{FUETTQk(a;TA#NRbrG56c zS-wN9C-1%a)=}oM@=uaKj6Qe&e(s;Yx^-n!X78QPU9Dg8PR)f=L(klK_0rcUYORsP zvw4N^q;NJd!Z*yj9x}UT!UwF^fjmxp@Z29GcFgxqqT_SWLG(o=FCzS0LG&trqEGWY zLj3W0Y$SF8q@LIVt8ClEd+tqR%cv(sq3u@IeaQ0E_LJj14W2Ex4~sv^eRo4$=eBL# zec54HTkiTZ*xzTsSB{aju)&w2IQL$ku$)W=uK(Df$Z%}9n~qWL-5OhHeT?$qOSF^H zQdF^%O=-h=R!fZ{SWBfSf7hmezO_AxfxdOIF{->-Zi>n+O_4-Xi`*=?O*^rhyEi8K z*7t50ql!m#TWoHKG{`fH8wUExV$l;#(Jnc<&Bcmu)c|&k@nCEQ_EG4TI>S14984rV^nA3W+Uxn;QHL7+G=ADT+Y)*4u2dq8kW^G=zerGT4u8G ze?9^m-~(?PycmWo8=uYp0#Hi>1QY-O00;m}NUcPLOj?Z6CIA3HLjV99lVK_pe`R%U zX>=}PWnpcMS5bEx)wQ1SjI8lk#~FE)I3XdKL;{BhC1Xh;af0#Kjw1+Z2u?!bLXpSP zSQe2qVn#{`5E^hpb7=tsZMiKi&24GB`f%6c<-W7>3wY_ned8yvy4HQ>e*2t}ZN*7? zvsSE`efHjG?{9y5pEIYQTbPiff4Sqc^D{r1>G(}a8fi$9tPKa=k{gu0ZmZsj@xKH9 z-{5IcUh_bulHo9J{;$QIsHSGB9n7iZ$onfyPb_1_mtp}k-@wVKi zr@rRbTXA|rjQWO#<}KA*fAlcKH=w6DxvpHTZo}r?cA22zuxPaipCPweQ(jTa`|fDw%cBi zT<*3i#N%jNuNqtS8`Z`(VY}{ZXFKpFjxO2x@?8r6xa9>vBQVxoMZOvSQ-Os^JQO4y zp=^Z{zb(>4H!S&~r*B9(a|!BoAG~K6|64|XEH%8U0;Ndef41DlwKu&uR6Vz{l8W?r zNtXJI<;lLF%1ysIi2vP&r||{1=EbA6*wvMKWhDU&caegb(`eFZ zd4+E?5(O_Ne_|D*#uN2Tw0J!MABZ*#=RFmT^9J)|u(Hx^T@IPM1pc{|4xbr2+(y0D zDgy@Y*sDU?ZFS^al+V?G`nZnvej~=8cWsdaAWnhQW~_Tj%ZDOBiMTuZO2l8PZpR%I z0v%@Omj9I##5t?-bUP}0IJ*v#PgXeAx?HUSs%jy#e7H zfG~`8Y254#Jj@AT+O3Ekyw(*A{UB6{L}K5d)@>rSFpfI#=}i{hMn{3}pg0K=n-xd+ zl;PgEf2V4_(ivI~{3eGX9>Fmdb+~XqK|z415x^0!b@5^nuX+%3s;CCVy^FrO~~c`#8)a)b4LDa@(EjYpNsmrnUXKd-Q6b#@^h$21W#F{=v%=xk22RYMqq_ zYT3DT?!M#dNFmzjU+YZ#KXeFfY9$gY1 zpeTs2Gq-VuMVpCwr}`Iqp>5H=ubYU{f4w_Boh)OyzJqTajcvV?5VMK~&F(FFo0)L0 ztIRqR1_DpPX%KJu{37*0?TH6Fvvd1$_q=&+V$A*4H*@+xOQKKKJDMFaCXtyZaaK3Ud8}DauYTe@&;8 zQ^dGcMZ~sME?jDX?{{dPn0*SOI?B|EsX3e`l62sIBuO$P@$Ykdf8rZds>sH`=mAT% zMr^}TGOEOOWG^Xs{O)#?lrw8O3uz~##BIq@RHr1FPRS4gWAc)6;*w&`Dv~*?80qXK zGk^T3I-@FPc3v6kWarJ2<)oBbf3_4iEI7U648Yn}G|QG`J3Fr~I4KjZV)p;qvU5h2 z&p)lmivy$owB$B+QO+5Td}Cl#AGPGQ>`g^C?VILF^X8ay;4lCDZ~3-Ua+E>iAM)1; zNpec(;d)VtTj%4(V$Ar5B!2_TF+={=u#9=7)W-4+<(y^9-$B09mcJiaf3zLtBJJa( zi;j6JZnTkqO>#13+`2KzQpySZr|eS5PDPxmky~*T(@~YA8n@QyF{LC^aqBicUQ|?z zJxWg*#W6Eb_SVIFH@hRS3Qi`CC8v;59d?-XiZCj|Xvt9kFWEWi3{0=yZJN%ql$m$Z zDoXTsQZ5SRB9vctlmf?8e>l%1Q>x889%aVZ3l{kfOk2i+kp4Gxbc0KjVbxKgw6Uzf8449YLfG~p_y5>N4Z08@si>%NPpCT&pqVxVOUOBvbJ}` zvK%vo9Mi)wZM!l6Zhwt1v>m8HxmQ|1Y8ke1+A>RO2?bI{EjZc-A4lQCDzg11-uF9Z znk>@Fpp|<`)(&9(e+fg@(h0+G1_1Lh69J-wiVW1ykDDY^70s5Ql;N>G0+UI&e00P{ zSQ$q@0ke}PY>u;WWkxYk^@ApQiXBX&sV1c#elVRRYM_k?#>Fi5xGihrP*C&}5RBSz zdvGPi9;3YtZD&2zr}@}xKgMa#j* z(~hhiP8beivsoTp&GJOQBMS_>9dQcE6q}GW8<4{IIa!+!eTUeO0kixWgi;_kcq#T! zer6rzqd@sge;>+6Im%DKg<4TV#$7w_E0Ig%QlSMWu7A9i` zI|q*^;Oi`cJ$alxsU#Ndq?%04C>SV8rZ-QthXD~bIYV7q`i>~&q%khEy~I~8r=$ox zL)&L8a-L;Msnb@vq?FWKwhaqx{*>WhgbkSUstGtPe<)jyYLe3wW+yv|jGipA6L78| zz$pawJOX=al1-T>@ZXv>rNXRf%t4TbAZ;a_^RsCwGn>X(Q;N9One}+^ayp79 zRZ=OQ4CoJf6qT66929RFyDZ?>InGXjpK})`*)(>W=DSs}TL=Mmo5pVK8%Y)WRk7bi zv6*urf8&Hjry@WqQsK~Qr;BVxaW0AhOYc$ZRe9@8u~}K0iA1NkNY-X2*&L?h98E{V zEV4P%Sd@f9k;1N)E;^|Ku^wT!Wo^#5Ve~K0E1TC(71=ZbBpm0F;RQf@`Z#;qOxV_p znNEo=2SBJY>bO74N=g#e8tMngOEh~qOHbqNf9WEd2ZzYlGw^i@q&Y)e6|;(S;(LyF zS{0n=oh;vpjy;fN&+_S&q_^}WJBv)uDvRPAurLwMn)*wkMP^}AGr$VX(lXfSS?tT{ zdX|RXkFfpAke(Z{4C-3_6$Kt2$+8Pd3G-Kf)pYi!xzyGkwb=q42Mg4X&n>X?vi4lf zf0|;?%i8%GvU;Ab0d$UQ$*99?j+075q}kdy@LHhRe}T5Wh;3b%TrwQ{v|RwF%~D!` z`vR7hu=Hb_y+FIYFouq0FX|PB`S~JB>!i(dmUd9qUP?%|cn3%aqw?z&F^%b{_&2!n zuV5qI^Ht;yg;fr-jV8KS@shpfB%%qUZJBEhg$}xnx!3*wO6Kj3(zgaSXb-~ zfuB$oufl%~;FU$~pp>;)g@UdSM2ic|m9<6SF$#(<$(jr3Wo?O0V!a;C5T`)%@o`aN z&AJVD6@sY>-xV46kdZx3znCQ3|_gYfz6V{U~WN;U&NhBq_zhP=EvS|YUjZw zV04&}6n7}iBrK2c7tVkbXMo;bzawfno4rnfy*}nFu+|;cl(i<<{wG;$Ir{67UNY0K zd58H649(}$aY2_wIx5&7a8$JEe?md$&g+C$fLsDV7iL*U(L<3@4d>cZV)j z?Fw|eV8vCNU8ilXQ$TOrVb^5s4Me`3Os~-$K)(VerQl&A%ihG%iFET*>@8V)6W1Pk zk+rub**j>xcQ#+a-oX{Dr0w9%3rz4lMral8qY5&pd~_73*Nf|zf*mDof2Zl_qlDg3 zap^E}hxx&gJ5*$ED^S>#=I8Q2;g%(BKAC~g+qK1w%StXo%%v;Jb#cN)Y~oZ>P0XdM zfbSi`=cmBuExg>IYrpBbj4uUyrc22r5e>-PXvTU`Gp}X(})v3@ic$2f)BrGyiXWV>06(ER4evg#X zRElFbdq_N=Qk40O65UF)htja>=4Lcs4fcV}Zc*KD5xw4DU_Y0& z_u*o9G@pM7<)8B&zttNl4j(C)?2wGus6mhC)pa%UE55MHB)~P+?Zk?%i2d! zl(ml$>K?jBfh;HPX_1E%1z4j37=`0(tuRxdLwF5ESmQ;gpa|%cwTk`$#396?H5(eM zMYeVUWx&16S;e4AU?q_j{?pxHJ70&->^@D4E$1ShTp|tv79K{>tS^Lym81~UIaeB#%>rtiaz z@V`X(U*0Qxf5ct;hID1Fc|B%mQ_8L%eu&QhM|F_7SP?}m#nJZixsm~8(E)z9$zCw$|HFGuZ=xqdmq z?e@=z`>V~Y`O4I%4a=TS_r7DaEdP+vFfOp?pP?b}e^z~D&UKc)v1HBpHU~D&vxCjvKFfv2khKz{^l|+^W)xH7=E7 zi{p~=Hf$V)j?9_=6951J|No3w2UrwW*PdY(7+1y&%&Mp%j4tX546f{`yV%BEiUk!} z0V@yze|t%yN$f^V(Ij?bG@8T|6H5>)YShGF?+qI&vG;<7U_t%gyL;7%-~av3@ADky z%$alEQ|`I<+*M29qa8w$U=ZgkDhytKgE&-V`CNza8y5r8 z0<5BwW5S%AXmE0}2iz*;8JZY`ysC1tu#DxGX`B=xSaN6z2MH-vUP zC;>t4Vfj4G#08orin2iz$`K(6T{B38lZ6t^z)ciId|K_v!yru*8Gm^i1S_;?E{H+U zf6EcwLPWQsl1K3z(rX2$Rtb8lu#yAcfCTWuz2?IOiLhG<7roB{(qHDI;Kw6e1+ZTM zw%RWI&Q(ehTXJeFJhw(R1VRPEayU8FJWN=FsuF?7w;_e~a3OFO4ZsCrrWnWhaB&H| zjg#UG5*KK|=lU$arbQT-whkd5mCWa4f0R)|8O;$wz7%a2^35&HHW$9y@YQB@+8DRf zCT2Np@(^gajSEyE-vus9)rNSJ`i+FtOjyhE`N)jr*J8~zV=jp%(IKpZmyLQa8>MK? zZ0KDgTE%!-N>CHRGy~QqFbt)s>kw3(Q*zNyoN3wrx8VyJOq7?G8J( zZ6_V;^gZY0{$uQidagZ2ty*i%`ArT%An4!wEwVDJNw3Q}T<*_cZJ8O^G3BEMV6Yf$ z3T((vt-f2KmD!aX|sEZt=DY(GX0(CaVdtVKP z98J{xL=LNyiUb`4RpXl3Nt&@uun@lFv^GT3D(u4xph%Ci>|EAE=zbAAC{E|qtM>5@ z=b1@D-14KW{GiOL7ADIC7ey5UILUOwz9qtBfNGgca$DsD<9d;q`RMI@j8B*wzF?!= zq1p-gQ+YqsH)$&l(Z7DTBa!L+-Plx7_q%wNR|Xp4+{wVkb2eTZv;`~B7BN^?hg_2^ zCi+}btuf>>oEB%-m)^$wsmckh9P?>xfTYw$li%`3ca_t!AfWZFa zsGR){+g3w!lFpLD*_(Dd{ZK*B5gJ3`b!>o$PCs}#yN}-#d&+^HYwQ9=zHsc{F`?Cp z7Ky@PW3?<$^hFNtv0-d@8e2U*B~{KE@`U%sbwF;Ia-;}E zEpK}8nPb~IC$nYIW=SKN$6uq7WF1E{=X85-HST%y`O@9%k_N*c$gs3*i>37XOLJ0$ zsdA?DN=8}-{@VgjiNmFae`n_J^Q31K>AOqi9ah))Y&hN*nzHu$O`}5^_Rw6R)!@)9 z*w%lionfKN$+mdN8Y`e1#`$4yyrCA-SS0MOm@uf3iPq_W9A(%d2IoFGObV)o z_*-<7J#$PupD<&rDr6NUwU&Zc5PM_M1|O>_`0%=XfFKZXCDM>!PXP?D3Be`~-C#^> zzQtX*Fl)vOYo^?42+r};%{L&F)tDj8*MVGV{6wz23AY}Ajg@4Uh7SfbdKSDhN*mt= za^ed;6DPt^2%TqfkH3Q&ijq)}o!*f}jQ*8Mv9y~)W=R3c;GVeV2y^_2;#selxh`{v z571A-ojQS={}=!L_unay^g{I=jgK#r@IA9_bV>nA;1neHrURUQn$Sn<-rg2aRbhZP zoGD?5hZ!02QNayr`Kau2DbnF9M*17o(e~*0%XZ8QULh1cGPZ!W%g;=WRPTx%Yx}5% zkfQkc{~&L~0>bu2e*0H}vaBXUHPAY;v`a4P0xRkGQ&;+W(GtPI0VMhP_&*e@>y87Q zai`&sLZFe5@#A-9vcIv6s~@m()zldMmA8qCkFk>x)4&>J>C0CMHWizC-BSz(VK1i| zqY8)D-r!r~Otjx+an31Sr-)ekQLR|$m$?)A&u1Z=@mP}Pl!LNFZ$Lj-BruBsTWN4KxF z!mYT&oRmAs>gZeC5gYg>T}QSV*MwyY5)h$S(H@O`yG#%+pgBA>gAX=I(%=lk+Rr7L z1$+Ca59FN9HqZ`!!e>&FJk5A4huJZ06iTza;(FBJ8e2bHsnjaxvRcKIeo?g;E}{qO zdspzM&wNdnSSK&UOPI%9D+!<;jAq2DM~qrZ!XvP{(TyzxFqv8QEI)Mf0}5@a)@@>_jBXw(_ctGnpMb?bjZ8({s{KMS*Gch zi_BVscR$j)z-*wjnU-dbIdzC7HM*tLi}}CDQ>2f%PKhymE7R0|G36WZ*H}Mij&5F1 z!ko5R3aPbzHbQ0RlD}gfaWrcP%K!|7lzP3My$BZJT502(v{1{`Aj(t1>%QA~)Q`1k zcQ~|)&Y{+vfG{U!b=Wmb4nCZUdrW+r()zAiOz2nx@(Hz5cp?1(5MkM$BS)2e9oCiA z(!d4Of$d=t&r46FCx;B`dt@H`)U`DL$UcwP*}k&y`;KQZxO>zM^*6lFU_ zED@%EacRyprJDPjsj*YbI;XQ`l1zor+uVz8Ab2msx&KWSbY`h;{Eh)_-lZTBYDMbI zs<;?!+t#vVCXHb2_&!of=K~ngGm{B3M8nUa$fE}b&6-=DnPbjr$gp;$>@{=gDQn2U zcYOxIS%5}M`nKLY*PKkKhdNIqlgr3hBnwSgsvBAAx6NQPjJ;Z6x9iyWpWyUyG=$6h zChKgeIW(kbs9H)zVn7dt2}ZZl!s4gIsch}*FGJsYg+r)DkcL(pJOePH)tAzlUuG*> z_ABD5TBw69Og;#MqMdW{=HDtfWf$;Tl#^SA>i7_H`|5&prS5R|Jfu;^G^UsE+-LA8 z%cZ@IHc*EZ46|(1H@1*iac(M^B<$1PbC-pqv{kv$-sI~5q&JMt1aaEj$nnxSO~yIu z?1HE4vuX}CPB~R>R3K@JUuYLWt0_wNbZS=})rm7+WXXBJt&xV$Axg<>2dmJ9UeTe( z-PbeHMX1tcKDS5rZ%#+0w0Yn*xI4NGcUBz?`(Ta9XDgWbNj0k~dA3F`q6pEq$0k}$ z?m<1T!5ptaI#tiiZ)SrtAcOdwbLU@L>G<7_jH+W4!`+4#@W4L5TXF4_wfzN%oY#x? zB>ap7@{GEw7rMV6$PKjMKN6qS(DG2r>0rIdvID|kTUgIBEON<1pZg74lK&iCA$@K_xoUE9Qx#lK5H; z$-=q?Ku|`X0Slmu#a)mp3dgiMr9@JmXh2d-t-=GWeKBtYNrusyAkATBV8-d5jr>D4 zG4^O))UWp_@Za;#Qa!NT?`}zy&zG)g1TGI!bEhtIhiTG>mU&a$A2ljr2Xi9Fv?ACd z6o;Kb4dwj(Ct?p{-gvKMo)7s>bI2cM^t;e!Owhe802bYE{EDbuk?}-VeDQM;m(X)3 zqWt@FBfaXkZ%3-vzZMN!W(@trUn26Pxc@M^a>>t*l`7|~OwAciEuH#QCr;1aZqGc! z3dG)R2lfcwvo9-*9lMe!p~ zPHfhw2DDSzJD!)%oi@r6zl4le5ME^{x9@PTMVvR}KhRzXD@a+l_}+}Jue~Ri;n#(A ztN(C+w#k1X+*|qDpFOWHTH6M&T=X`RzU*I9yWO$=em86rCPu;{Fpv++y5x23Lt&E z%)m}1KMu>4(CaB@{>&-88)qY0a`3?CBAQleAM%(6hTW&$VsC%Qd_c`=`uXo#{ zmZF|X_`d3CMk>DTj#h5O}p;lG)=Y?E_jmu z^t{3Fd91uE{Om*=p5)&&*HIu!oUYP>2Xs>0_7jgofM$#L5b}Y34(Y|UfH4VW`+qF5 z&(^^V-IZSVVf-%vM!%1~uYTm|)UV67x+6)4(r^ZMe361rJpog*mBO+m)+-1JX0`S{uC`RXQ)E0C@3a zlNf@<(~BEFzOWFi$21LIqlkJZ!=Rq|wM9N$i7m>!45@OAl*q&6tz-ogOpwe6N z`A?*vcahdaLBP$l`ej4=PHL)omCBhO-}vO0=3H~v8ozEH!}HU8Kl`p=>S=q7_nust z>3A$z?AOG&8==BUvToP>YNx?1kd$8>vQ0;u(xmTny~LE+`bdrAorgSM@3LF7LPQ~u zGj67%&s6evx?kz~_%ZFq^%8SRH+3-*JQ1v;nEpI#G3T7k{%oiteIELPK)H03$thyS zeyoH;#z;L$4(AZq>D%+Jex#&#+RT`-FxgqP-+F1&TYKi)%R!y*IDDW1Y8_ds`@V3U z)!uA+=m>3|2SWp=DA%_1U5G2XP9_xGi0C&SOq_aY?ztbgx+s0ED+w+iKQJ3TU*^Rz z)sL^zU1 zC!(cdgXwf0uJUy)hV71*gS9htyy{HNMitqqZ*^z;%)e^oURzr)Uy& z1=)6&hE3b^I3Zf|Sc^?NJ)_b3H|X$NIw z<|V?H@$c&#zaena6HmYbwdLh@eY+8Monp%qr)_R%C;c^H>vi}zepBt<<}It&e|TWk zYaXqsx-!i;UxJj`^KE`VDuvB&+VoL4(0oKRyE$Y$g0b7@|^;{pE(26ms))mdN;H*_h{t-z#rP`)R?w#pRBI zM6WCFM|UYawjT@q`_v|`&1gBPBSC7Y`=^)i#GEO>ES7zQRa|53QKlgOpg)CRey+Fb z*JUp){T{q<E$ZretC0nZqQhvMqcBs zNKNIo--uItsYBtH6>>-Q*blFII~$jcnC&e-cRJeFw^->*B9*loUq7$g%ZXWID{{a3 z&%d!i?i8Ji%G-B`pC8HRKGMjt-TU*eQA3>Cmu<{UX7f#Q*;xECI*mrU122Uadm`{x zdYiBPeND5S@axfZe3+jvn*juCDW<_A@ImQA{h9|Cc>ak@AKT7R55+!2ELc>z;CJ~x z3Meu!2;5VntDBJ1$0|92$I9cB`!D}0QqCKYb{FBnU*KcSOY}2mBYT4C+Q-)I?6}xZ z{Oov-prW>i1KVCP=TG~25;9;bu2;<4KB1rA6@5%zp>RcsUDkDG?W`ie^Kk!Dfb_vx zNdW%gzBc@oW#(mg4mUTiu`X+h8N9mRko#T|(M>tqxps0YJ4xVS?P6hC<|BGT^x+9m z%6TVs?`~F+zql*61+~CTwud@Y;%Zm_ls-K0Yl^!&K*IZHIiGpL$ldTfMmqu^{&%>? zHz#L}F7yrhW=sa1G}z+~Qi#R~b-`DPQyg`urJhYo`|`YhuwzU#6Z*aEiy0ujn?SSeepaga4TilvY9p*)i%;N`@AtS7o3a-Gn7K_ERR3&7bPm&El8Cw8a)uzSX7@`s1&oyEXc?s;aiFs(r)}cLK+9 zht)JW54ClxbS+H@Y9f^zrU@6 z0z(FT)vawXmN9cydA0}`30b!^*H=J!A93*-{i3_A-}jjPs+OTf+R`M1s3$z!mk7Px z&DdG33G=DtexA`33&PpXDMl0ts}^A>K_`3)CaL7BD9Pl}D-k6n6qFQ2(4eS9CYAm3 zL^M7cMQCxau^#7m_gwQ{5h#3>Oin^5d6WZ`3m+z}<>asfN+Kwtl$8_434{@}uv`fo z$-(iZl%!r8`8Y}KjO53cN(+MU$!f{n3>S%t!zX;w-?5VarYSF>Cvaa`SU6ZH%Zo+z z6|mNvN{#)Ip$fHZ+k+irGj&yh8ic!MqA0VVDd_A|N-02>A6&q6&B|E7Sd7mqVGajq z$W2>m(#M;MkH$J`PNR&RggFjjwnPmX&4+iAROQqrg$I)szSTu)Y`x%&26My+-W4&+ zai(6*V!{{{R}7IV$wAtcphi&NwpBGohDu4u(zBG5CG;Q*OEOQa^YzyM2!pg)AXW3l zy8LBT?u)Yt`#v%W%PG%doG;5qfBYLL4knp5PB1(ar7Wh#fp(x37fl_Rhm~@lsEA*p zrw!TX!u%umFEWm%_#iw!p`c-R#x#|o6fKWY+Nz#yO6-?))QBl-!u^G9+K9cCWq9c# zRDS*p%M_niLu`b_14Di}j;v15qVLSHEyQf)Z<9a=asxe$@AsVGED#N8CImSEP09hh zhMH(n6fw8S0ySw5G-?*Hvu>6Sjh!6nTR9}Vhi(MPzisJoonY+I7XXkhjOt4&qmAXQYXFNQFu;rO&}}tA*F{x0E(Xx_P2W^4 z271EHU@-eQ2%Y&0r1DG(-8Er1ltz(!IHeTHtfjN4^fhk6XjU78XY1$4{{eg+`kXPSC^kcgQi2BxpKp=B)v#I02^8le|r$Mn-8A{POn?HyCs#V z3s-VYpZFq_3``QhpDcn^j&<;?_6}eG3o6Yim)N!TGh?@N@hy2}iuK@f4%>MPFjvzc z#M%r6`_WwpqfQ5+EIznvC8is0*_?+5+7VHN=lGe#u9xoY^LG2f8L{jAZLH+r(g31N z(Aq9u+t8*FTxJ#3^6{1gawU+lOO^v3&EoUxw{-vl7}+KeI3FA_yja*!Ja|}VmH7KF zIK>~_>4p9(NH~}>1ZeHP;x=#)9-%5WknGIKoJ|9v$GRPv>|&V<3n~Qgh~eXmgWm=h z_zqLeQmuQ1^|;9LcnI?F{;eWnJKS*dIUqgKVPYLgMLY)enDS1gCqjdC-N8kjb&wm& zJ7oq3rz{D89{e!Jw;oq4W*?d(jTbt#rE28e)vcEbIrx>kZmH*6Fiv-T0a ztNw}Zx(!ed-%D94Zy#39+>gKkcLDxC7{Y{DZXtoPjQrIfb@k{Rq3h|0u{zc+5q&m8 z;H{`@hvraura@$Uaa?zX1}tfo=zfStP+KS_uZ}ywFFX`n_6PcBF`hKpe#UmF1t$~; z;niycj}R0#7}x~lJt#+i41cB;z3}bX7Gytn?86lbqB4palatIHQ}n zODQw}_u$7pTr~Zbr{=YDYa4>`V^B(ozRVNmXMJ!bE7n!T$ZzExV&JFKVQ;5Oh0KtGW# zABDMZd+usuc7C}%zq~56`ue_J0_86XmtVlmIP%vX@8jn6*HLmTV-w$sD%QtHE8yk& zwV2H=(DUQ@mF~;vb!Pp^zRupq)O_UPW^?-}HB0aQ=%{g{1KH2>S?9&z@5aO?r#pZ4 zl`n7^9mmb$bhp?kas_mRoDBQwf<(uQKR<7bx!8zDu7KXQv2~L#4MC?-i_Hk6n2f|bNmi!YrR)Jb-b@{ z4LonJv1X?6Tq6^#KFwzKD-gDo#;Wx^m7gC~ZbuKebPxMh~an7?zq6R~jZhND`nDON)#lfN2`xOcg@0EfK-z5F*} zHBS}Da`KNz6^7Dru0*aC=cyn<#SOraX^2ctQL-% z$#E@ceTWf<3VY*}Lw(tg1jbs~pm0Olf6Hl$Ux*za9T~);WHq(PPjrBql!kX17;78T zwkVY(BvED=_kE@$Q8LdNjgcH`4c}BZv>?TP>+U>(Kywb=0ly)~Cj) zRZVY3M;*#rZtoaQJ^}AbK1y>Q`7KXh{&h$EL>Xc9xi#$V-~8+D=GzKrT4JKM@n9;p z5D-NPD!&3nX)!Tzsj9W$iX=rbi!jRKBG<}__evp9iXlj5L(@C4JK$Dc?RvAb)1M|Y zJsHhTxf>%!s=7&Vh1-BfA~IRA>~fQzv!Wt@X({GmmC0)h`hjT4A!vnJhhOIfLgEn? zk~N_j-f0$XqzjeesywI#gqcjbrYTcsyp5Ei-7P-fbkHI>(B=MA)N4-MF`Z!{+J}#e zZeirJC*Oc(j{lNH`kA{S?pv^XpBWubENPT7^w+;=W{tli@-H~hMDkrjM3h7}@Q|$5 ztieAfOM^~wBW?==5r>r)H|W6XPfxQIB1Khr6^%|v^K*WRWmye}@VQh{8F%r+nZNWo zZQz+8iceM4%%Yb{>jjFg?bs8uw)ve@ayuQ?Q#0CVP2{kZ?B!g41FV}rM^xxf*O_*a zj~{f){uBL6<)~`_1kPo!5O@o|ZRxK&5^XjxC|%DXAqF zb-m+eY%wK0`t5?hoeeXR=+SY^I*I2#RhDw6J-E4^+Rg^lc&-F7IZMr-@D^ zEV#5%;IRdk4|ORwk8rwk*kI!Fv&T9s{`ezUj&_;iT@w3=l~-7>TTH#Nq0D_mD; zDu@&~mKMM=!+N1SK?=p#lZu82Y8E#kCI0+j*&{xim;8_HOVO{u#_cHHVNlF9W7Pol zcY_Rf(;|S5Ps0fObwH`9FAdcAhp@MjB8EIoR#F|nQn+(g-&QN9YD`mJvyNAS6>zK; zuI2CCLoZA^RBQPSa-GJ~Wxcu&1bM!X53NL-xe9pzE1&cOm0R#kmQ|YWt;jpFpP!j8 z{Frdf5}K^>5jRKlN32jb3R4*qIhKA3 zKUOb5D%)z@Cl50k=z*0#Tk#CCr9zxz7>)6);lU>BZZwJ#d_X+(s17Z04=KkTNRQW_ z@;lFTuz>lGR|W0wBplGN`Vta`UlwGavN{jcrGon~ZWKskYCEQE!tSaJQT*IBkKiSt zDs9Os^H+t&kHiGnzv_{=?Ut+VX%UW>67p&QrDMYApjB{=*%izZ#{#r}Q3Ds~JN9&) z2nx4hg6kl5{?I#M&mb&<2X0@s?Z-`>Hm#2BqBr)PYpA-_oA zeozPel?+q@f9SCffZh0~lMaK|e$wT1j+O0DNu;^SNkjXAZcm1|;|%M={~yieB=vasgU zvAObD>p+S3^hC+sDq<{P?%r()SCatv_AmJT6+fSRJ8XJ+K|~MJe|FqN7pZ+WI#&J2 z?zYwUJX`GTdH66I37Re!32V$xP9+rBaye|T`Ezx`nyt^Ht%2iRa5xat`(gR{JL;;k zH%1mXzn-Vx&E@ZW?CH++ezI-2=xo28gm${9BzgEuZdu;$T=>{@mv2SX)4K)U4`Pqt z^)-k#?`NZFojQDOYr^DRr+tnJtm|LupRV}*{ND2$^~Z@coIirig%>Ozky8i0F1#P2 zdiic$u91z#Ce~a%R|Q@(bD!LGw*3k($F~H4q7OY@kB66_SWNFRuH6&pFod-YpLd_5 zqjdd`y9ot9kH52-6O6guZ>zxm_WN=G{h_t)aSXj~+UWE^PFoL>HwI|c4S&p5OPJ;q z4sU83^3;=2v%8T1FLv8Bhlg1ka!xwivGxwv#CkoBpP2_;&Y0y?CRZ3?6wUEBZTM;L zw#~_7IP)W@9pdX3eGuqpo%a~$tY7Y+Z*5=feR_5P`VB1M85I z76j_Yi1W&{FX2JAs2`kQy8`M=|D{K$$VQB>^)bG8bNcy^cfl{`E@hXiAn(<2uB$`} zm1)YQR-v+-SFg2gyu-GW8iu@7llPc2VbZ=u#cP&jY`mI5O#cIubbb8H!m# z#Dt`Y#tvq_n_)rPQ*d^w$QLUI>P!{SNwBuNSmpG!sz{gNtO;;1^LE?YxZ5acY#O_T zO>Ws)cXTdaFq5`s`9NO`{ara2%@GsInMi8Nk*}P8zaHUxD|wcf-nzPp!>LI+;Y(q_ z5V|PXL2|b{zW~HCRmM%`?+M!uyY0z0$JP$cifA>JQT6_$!Z)6Zk<7Ug_o_^lMClHW z{cN!Q;Y-(@q(4>STwLQ*zoIHhB!$%x8L~=~nIn^GpujyJQVdDrRiqL43-$j475*Qm zFl2WK2@(V(80ud}`@iX+lc|xR6Qik7q74mfs+utvDqyPWZG-NL|Fu~d>l(Y|m3W(l z&5fQ~|M-fAO4e$pOK)5i&7x%7VST-}wYIhB9ZfQH6CMx2Zd|v6KwJ=lz$6q%B23Ip zM=?BV`~|XGAD##LDHwtJA@Z*4u3Wh6bM@p42t0j!+*wPw$3lbLhZ>YMSE?r8fe`RQ7Q{01c`|re6G~t%$j1vhu zg=JNv^(@$om6%Yv)W3vOOfe z4#Cvlv2Y3n@urxJJ)Q>SM|+?;$LW>I{4{&ko7UWd9@1v|E-~znFkmtb4;(+R`ZW4SHI~rVD=t6|y zgeE|iY(SMZU$5bebh$kcd|CIJ2)sypNE+yReGT4hjVf-w=W!vq4ZIU+c|CDU+BrEA-o3}sAEn&6JM4(`k2HS&K%biRyTEk{UY z22cOg7>Au-mX5>_bvjre)|0W+?Z38z$B(j-@=>~fylB0OR%PJ*t@eJ7YBjnq0=7^2 zN^TzQr4iSSKK5U3TYF@7kE;=P>j8dK&3)xzQ4wH{A_cI*(l?cejg{&ETQg&h+LHr`O@2`DNR=53H2gPIsNq zPZ7e#T=^KBONIVi4&y7zecx zM4gC~^zW|TMAtqa+3Iz1_7&ZF^HK%?ZBbbveHkvo<$d~_3RE(sSwt)(6@^Wf$du{E zJ8r%{I!hYLQBD7QSfK^_@Hj{;A=x9gm}9{Z3MU_F6i>lDs3 zU{l8X-KB3o${VaH)z6K4bZxzbhEQ{MHX9PtQmD9#AvfxCeU%^zSKc{xgsAA_!?Xsx zt4gpz)8MWXs{^b9Q9Xn=mzk3I4X-Mq+U$~Ud~JLPQz|jh2aC@2Nt%m(n)b>~&2;nf z9V9mq&QniiGohf3b%L`_J3jI1>npx(CwmQtKfL~!%-Ikjaa*Z*Y0zda${e~1&+2k`q^EiIJQrbPP@wjp5U1xkq1?`}O zfD7wLlgT>>N=haRVg|yaVK1->hqDTIJBSJkZGhYk>s3_!we0b{&V1@=)rzg%?$8yX z`WPP9`Y_|WdxP1%hsk&1*rBJjX!BEdlf+<(%8SX$*)J^Z%@ErKDm+iN(gdbiKm!r z2W&dL%ep5e6cQhm#~g<8PJ%Drb?m8fD#7Ap& z343hZ38b8_kb<7_5T0s#TeV$|)X+nzg)9s8@l0)LY_}Sxat;~n@4eUa30L+^q9&;A z`dV82wE<%QchX3NYsBSn$`}r(BlSw4KZ#CCm%Vcu7Htrx)DIuSM8kf%pD1ScsU5B# z(;_SOEixPqL|rw4CY>`0BW3Z(-tjHJNe?AD0^){y2(f&HSxPvbxVUJD34mAJetjuO zRo2_5s;<$$ETr7Z_hgfPg8~r{g9^^k%nd_-Bg<}r>%`KNDOo~qS?J@`L5AUkUg2iP zotyHA>LaJdMLAAz5|8@XCb82izKZ1)5x@Fr$n*m|$8ChCMG%#n5+`gTs3?pzW>-+D z0VV=k&$=3nzU+dasCgJ#bOdzCEdNU~8-<3h6)Z!ZV#7lWWy*2c>^aU!`l_tZ6E;!G z-z)5V#%y9lvKYo75~ak}-!wpj$PMdMHjq#Lg7U2wu!dov&8J#Q33Sn%tWGSNVXbQ@NeK+Yd84!c*h;qfE0aEkqSBn}8?ra=;#b`nK+y$qJ=HNBT>s-l# zk5@V>?k&W28YwrD_D&SiNa@>Oth~WqkuN8Wst>67AQ$1-1PPZR87` z2WBD&CkTRZv%#zfBI%Frp9sqcCQpv*_$WINa&*D`@rPLtYR6bdYaP$)8%TryDnKTX z>Ts(ACQI%(YtIq^PgIJ|AOFs9Fw~mjF-G3FThx=}a2kpp(K7+k@+8%{YNIuIqu!crj>%B#CB>S~IODBvby-*(G3^i?&*`}$ITcQ}%H{c-boz0Si^iKhqc zl$@`J^%^aJ!l^+8hJ!ad%ZD-i(SzAV4q+(}V?%jDlHQ81AX+ysf-_>>SwCr;^oy^p z7C&}<9m1*Q*Fv&49%l{vmEDHUdHyYAV~YvHG8SI5sdK;2DG=`jJjfdx1NX1hgyU}6 zqZ(=YS)94#_M4m138(AL#hA{P<|mj}{V0DP6|Md?C@-d{iKL~|t@HE_UzN!XW$boR zDco#?uZ}ZEd`&vt*PV{JVT;ykCaSd&GV?@#+p$$Yn5vNM@2$QyRZf309qop0mA4K^ z`at}5rUX+hiVUi52o9dAD-TKo=vvGxV)RI5*i~<4G#9{l!^{UN+suFDIeSY!$rPVU zFC+SBW^%OJu2E2Pj%^G<8_M2{)k6b9tenOn>Ow2E%ajdf8WUO`4t@;f=*Ret>#iEikRjlrG1sl=fh<2LFTB$peE zYw)ZnZOjqDx^!rNv=c!fqjX3&W!+HOg?L)IJy96}tt9ZHzkJYW?M$QS>^lL*hzG@( z^MWZ4_JvObQkBEPKjE~}ttu1s%>pU+4W1(75J=(y#2Klc6* zisIDRsA_VP;lPQDI#v58zv4uMoc`Oi_=%-?kZSxwX05kD6^U24 z-d($1%Zm`0?(PfZ2kVXgF#o>Q-N|g00W-1>&pwc3iNn+wYR1ThWV2~tWXtOEVtXe=kpeO*w;Xl<*_sG^E)}tLG zSpZ0;4PjiI$)Fx#>dmuL>I5n*3dz92vg4R-0_l^HJF=3ALF8rLyjBw2($8@i-!ft< z3!GWF-!#z$(cv)*h9(YHmBwPU(*4vQfVcM;^~$X5XY1EnZ_n5M(`9R`ZWI9Q z@I*+W+mLFl&QpvgrKk_rD(@gBlB7aGtu)?1xK*Q6aVN3`x5upnc?t+NQ zkE2>q%(yh|M8S6Q5I%}4METS`(y8v~d`5h-sa_tooiwqY)Tv#v-aD|86Hz^vvm1d( z*>JI4ovu4TsyZ0+r3Z?$=or@jsGV7Xa=`)AQ6BN+$gNpN!&<^s5@twBU~EuLyqbVvBYw8~WqTO$}U zc_|*fqI9=17a&w%1fZ_qR4R&PUHSky$D67f<-hPWn-u^9<4veO_vxBa{^hChxLfvC z>0oaL4b&0?mx;7Lh(kRuThFBeJ^Q*b0((g-@0^geb!lLp4!R`CoHSEhsVv#eG+Bcx z9}YBFvckdBNpGvFfVDEbqpL<C|!N9nel@shq&wlA##W39>Kx8`8tQKI(F^u!9MwJL3LOrtb{b)D#&s! z)$t7VO$C;4L->(QN+CF2b?}|yCGBTEBFqb<@EyEOUJ7+GK4IyKwE>FLp<-X*F&y!Y zB8Uv0Os8Q~{)}OCIzF#_j1zDj>xAu;Wv(LHXj~OFzBsI`GP4yUU^htAdScaDBi?}v zXNT>8pC&q$jTgm+OQhgWfQfw+@gQ#c8+{l*YItRjxn;=T>(pX|LS&=_e6;jAjRnM0 z6Y1c)|3B1JYYql{Z+h^le+eNB4Pt!f*J{G!jzdC^EH4Ow?KxA z%EI{d)r>dPcJ%~h+z^0aXM*S-Ng6~M=i=vqzzS6)Nz&R5GR zY-NzTaMBqZs0%i+*hh~cRfqTI)edG>xs52egRZA!nI$=?@DcE4V15XrXCA*>X3#O- z7}QLt329mAcOCw*@`G*?)Dwh%bQT7BQ5GRjm`EAgqGm$EnV_{Tq#XU`R$=#@!arOK z;k3=uLWjUcCA&OqhaRQN@UKcv{G?Y^1&du1Mn-8=p9TaaQFMEKNV?RnGq$;MLFbK2 zeQZ224z&OdtUMsx#tQp9r4q;smQ{rnZyAWf+FRS1K96j*DK$`%#G4^2B# zf9wXZih=_!WV{Ju?Z<2sF!dtFP?*c?X`E(?p1MJH9)0j|_!r(ZnFok^Q1XUI*eoGM z$0Fk^Ri(-4$kfh(sp*VAcBRl`(9>$tePO<rtWa(SNa1f+ioWW+G&Ov9ZS<-`k8Ma>ZyH4)t>+Zzml4 zlpY>x+TS1w7WB&q>J9e5mT^s&EXM6;t=}27;BlnoCZ~e491sZl4H7P|ywDS_yBw|> zY;!T6SG0 zdtXNsqWp;JH`RJ<)Sf_(o3L}AHEiFmPbak9DW~|J4-<7ebF(^$Pak^gkD`vBqJa(9 z$L^!@SQp7#%t_WEM3{cW)5L+3l z8qY+A(;|G@mz7BML^%~Je&z>{xwV7tVv4bZ+WY;+OVHlLqq^4gLyx1I zIQr}BUu!X1ciwGIq064MvM;ZvoqlXMhRkaC$5eU+R_EWH`WxQl8^FVJ@%EIy|CC~{ z#}0eXCvxhx=im6!7D;GnU(;H;m0^=BZy7|)oUOU%G#9$Qjp-ErED!tXZ9w7uO6m2n zVd~R2mdnW~{!@NM^6?_UFO#wKX*4-~KQzA^i6{0? zFY|}Wb3B?{u3odvwc1s0ec-)v92`3+R_}9NcMS83U%jsTbG(Alx<+vwI<|K+5x38y zD|2)U8Vrhq>GKHpTRQkc0?-PaA^3#~eREe8WDl(cVdJw$RzWv@3Skk6apYRv*(As) z+4-m^i6I{>p%SY8nfMH!ZW3{4uBY*MvHG&xf1CQrcDE8`t!@BrO;(6lnuI#AnJRtL z9a&{nZnDXuDK$PGDd_Hm;!}El`)6e%MS35bjRtF_0HUzB01SQ9dGL+GK&S zmwx!EfhIG$0%I6O4!xINhEG1MmxQRC7bOlG_EmC~Gv%QF6u1M!>{qgKf|-e1>|mVu zL)m5$XI2JWW?&P(6YCesV}9E3bXDrvZ>7V69z79D-HnrnD7|hrJU@7du^1FAdtVt} z^px~w4PnBTN0CQ!QiBv2bQ~gPF?_E*E1E)tQBqOu!8sM)#JI)KscDy1*7=!XxgJ^70TmMB{AGdd#`9& z#y4?Uu$ttBl?qC+6!$5v$;y-;+8#mYj)DG z=w!8j`1|?P^W{BK8!yfoOGy<8IFhk;IRY!pv>1opVFhLN@-mrSz>mXIg>!y96_Z6#Qa}vJ|ASL(!&ojuj;sUbDPYaULNwsfTt(m9)X(9If^&r@{Xx83wQZJ? z?`;;eQsE+R2S$~_RKH|{DnTp@Aoaa#M5Uc;{T0(uS$~OUifa5~oO5S_+YJwwh^+{V z;E#4#(H&PJp;5$P5`&Xi0`@9n`zO@!;EWST_)*QtaZ6BJ)O#Dxt-snt_fLr0dAKPb z_bL(o;Mm5Kc}3NH1Pkg%`WpHOZ;<_QSzIQtNjT}Eiw_oqdNdCfNkttyA|4#m$bSlu z_?e45zFgou^PuZ3aVfind+uTL?)l0%WhX@93(?wp`0YICYky~x3YfoY8D$nn#XARk zMZglNtR)|Zr{p9wXF%LHXbJZ%vrOyngA5-^aZ*Xj=K>8Aoe*3Kk9%ojX6O&?TnPz^ zdTusg*p_p)HRpY2-X@;c;Xw*j?4X^?Yz3XwfpLrRZ!<;;y_o0x)d?2Eior<}u0W-> z2H#YvzM>0aurMqb0caZLI4IyG^{*NLSf>5m*QJT$P}5VP%w>#$;mr!#gSKM$#|DX4 zqJ0)TP*_=DSj?kg%$A8x4HIeu^UqP1ZlGhPf==`crN4<)(ei#O3^bB4qHXI^OiD^Xq-P+P!bTjmPrJky0olYz#JL_ivB5HY3m%5}23olqr@nGK5C$zE4r(ABA#^LI*iNfr{5gvT&wdxDjIDxsS4e*gi1CLe11Fsffl&`Q3%}(H4}Jevm7r|(j`Bd} zc`c!}y2n%d14+azwJ6N|@Hjcp6XYAw?65l6lXN6lrIHM*7{AasQjYo!@Su!M!u12S z$P96ai@Wc5)2=8AmXHgU{yg=y(iOMy*MnXMBVu*1p}|-yNkHYffl|^--WjHPROiTS zt@?Lk`J^D-C=N<%d>6mmNVpjbH;n(|Qxk-6%H9_w1+eak&A+cg*nbl=1)*jiqyVw9 z2gevUK_*Q^RpdZKdN1=MhLD6iOT1|Gb0B> zX~N8b_j|O)KUH3UYo71Mb|rg!>Jmxwc_WzgHjD_FOlLz#dUA@&bF4t)-XH3nql`+$ zfcI%0u0~9AA-?_%U2(=exBl*7G|gWzeT#Q<4v^KjB4$ePWgTQcy!Qx?Z~A-k&$4v< z{MgiinZE06)srvjT!ZZ59PVgm)?zU;9{WgL4m;GtJDlE>@M`GOmiJz1Boy#k)>tL> z?ZHi!yLcU=+egM_slT3$VQ8~Bod``K+-WQR+@c_`>Dg^tm)GHHv`}>Dc}^LsSyB^t z1B{NtptCvo?%Tc2fA6Gke>|oy+ih(+TO9S>v@CA>JP>vpU+KARHjP99zPDp3{XHJ9 z7gO%+-G?&Ps{$L%ZwKzh{AG=1)jaO+OR^jj?~dDbn<0~Q(@zmGy54Qqz4I&)BG^+v zpf5|;;o$q7z{QFuH{Rywb5jX@&tVxGfa_M>x^ts?Vji{MVf6+i@q9=1yLLD`t#I!Y zJVfU6Sl@R)J!u2?HAk$ix(!oz{dj$Fmmk*mJV{jyZ^+5=y9g`d3ro#5cRaXGcfF$L zWcw_!#bsc^=D%&pV4&A+y-VgOdb{#DOX0EiHGW$h$&v^WOK|s7`1bKheHx7ms3*Dj zo;_^CCEk`qcN_uV9o+~snYeR8`Kv(&lVVnF!Te|2hfA7bA^ z@O&Oww;1)s=Co73wES?9*G+Tudf9rMJ~p4hza@a~ePwbI)B||5_PFZ#k@($6`o5wU zS^JpYvyylAT#hcsbbH+I(M;X}TW_bB$gp2NMcY!yCPCPKHHBLalVa{0^E&A-2TS&N z;a5GD%evm5n^ktcn8nr;6A9gfg@4WK>#lxxO|ed#YROZW+pJ^+!KeaP#B~RqKX`NO zVoev0&AInZ@rkms<+4GEk;5j;a35Y+jHk{uzq#STSAVFV4l3Q2VXPSfxR<|)6>^+7 z^f+{leI!EY7=ZWV48813osKQaUakHI^Kb#}FNE5N09C3jU(M;&CMmHxH+1M*B}9Ev zWsCyZL`0*!Kgt-|?6+RBiz?-1X1&M4t1}B|fo!Tpfu#D{0?qU-wjaY27>ntN=$4+0 zapUArhlRYXXM?u-9c(Rt-~7oDRe70)Tj7DAl^}mZtLHkFJriU0&zR9u$&@ps$@+kE zj6-?kN*o+A_F*dZMkNEVm7elvOg&3gAE_B~y)Un{eDS??*)Fx&^JnfM#l5pe{B%3D zF&dwZ#*5ZBDzug=pLYPrMeH3}CbRq-d=bm?Mfl&R8QgzVIHz~IKh{&eTY$0xk<(nK ziK(1!PR*i$`=i{N%yoDfykDc(B1As~FrceJEIPO_ayh*C+hS0{DPAiB9?H@cX5+QeR81;IEaG{I%S|tsG#W~#JT=-|orj$o zX2m~B{w5Evm}&02icE4p;teiX6kDd=QOC)t{7z?=SvQt}Xs+<$uUJdp*I}$T)ou7o zll`qA^^bkkdig5V=4dJR8U1pSOhMr>-MZ5fNM7ZliHj^9duvjmP0$V#K;~(yHqWMA zu^1Vu)|lqQWX`ysu2)W~6={*(shGhs5oqNtT@1DJZL;2qO<6rkn-3qK7If1N6ZPls zgyf7FH;raPt2+b9X>%1GdkpJ4ajEH&5UC3?XVVHgj#ST;Y~#`6(PEp@0V&hFwoEgp zK|K}t?-`RBQ_8a12V*9`fXd;rRh#m{wf*Dgjp}qY-L}+4r^(Pswu#Cj&@Zle z%EF&y=1KF-Wo9<)B^B)FwR+OWQ4ODLma=pjzaaUZEEN*ViezkQfMCnwV%2q;u&?*U2#N}Y*5BTb~t6xhEWKN7~ysdVCv}y1kfG1e~yPY z)bpxeT%1Cm=-M^SOO3{ed2yHxN(v2^pwbv3l;N;a{q&!u2S)qD$*6xC=9=`7&Ari! z-S;(x^Y7hwf1vqx>c!#-;|RlC1>tqppq9wn?LfZD_TKdM2D$59-yu)6^qkgMl8VJ4srs2ymhVBVE-gYurk8KDj!ljh;Ta^cES zliWa8`JqM;jUYBr9YM^9d!;(ieOUfN#`(Ma9EfEl1(alNq@+;ZdKpgO4QhflI+5c= zL+JX$@l-m6$W+Ex24mfh{T262 zkXp@7XqSb#YayPsU}flSQ6SC{q3c?dgQ@=RkRKo#7m*nuuY5cT9WMLw&5G= zjjxx^1hjwPmS_arkw2@g7+NplTvs7^`@!{VOWv}Nu0#65Jzf*D`UloQY-626#(Ch( z^bxVK;$X1v;fSPuQ(`xo-8BoZRZ83VZh!@cJoLi80|I&-;{OeN!t}wxXUC(rle$BU zxdH7stbNeUk1Mq#3-mY{K2-`T_XZ(yS3B?N4G2s?3+ciZfurwL+yIL_q6ZsQ1zFwd zt3l~9%qF=zDtjhG$Ug4O*Gu+s@E;A1ZSzlrm#mucVA|<~*s%J(?*sk!Sba_hy3qr% zQkTdeQ05a%WhQ1Kufbo=nZ_^^WNhRa4AY;HgeT6B!m#(`%^qhYDB<>n=Wj6nLm!Kh zJfNuIiGCEU4RHf2x0BvWJ8=6wmrV=#4kUQs#ES`m9vojUbEhM>rcO$q=Q;IcXa>go zNBa(@EeaWR*+gqGPFfA;4kDNx$72BvM0MH`{SU0s+DHX(@d)d@HUdVZAg0-YCz79h z{-VQWa(l@tYrn-p6D5&&gQ#J>_I;2~-vBEZf`FZFf^Nc_9ijXL5}p*Too|2wtZP)}-!i9<07_94M=332ba@6H)!d4y1K{01k<; z+fqgC-5)TC@T&k8h^i)NjJU?X?3C3!m$#!2+x?0Mo7OS(KpE|hf^p%Q` zTiIfDM5=w&at>J}?+<5;{2peex?4Yd-{DG;w_2at zy>ogH#6pZ?cafmfV-f87x$LU+qD?NMq>)`Y(KZ@&z3d9*ZYecynms6 z$d)~C&oB0u;Rte}+A)65p9*)@1tAqRjSb!$A=xih^8$p33tNE4Sl?N5CZpc>9>&(2 zbJMrV4!vj2&gWP8WkNqzuM3GcESDZXi@(lnWfMHB8xyX`0&RO`ED`D%0O8p(;fuKP zK#}~9<;+e{hUZCvH4eUx^ELHL@29OEyU(c4XNE1V_d(Kk+tqfSluhU5PFHUMU6*#h zQs487?%NhMptj0zXE1CVNY6=oz68BH8q3PT=aJmJxk$NG5%4(=W8iOXYXa@r-u~Wx z^4X+^Sfornme z^Dmz3ex40Xo=vS3J%oJZ@V$@sXH4PwouO}U3vfA98h#Q^6~=s8KF!_t`VKG2fI&MAqP6?wltx2Bm z?rtQ8_jN>>|7;Ja*j{O{U7xXFg0{v4LZ2JNBQ}_6jSLzvu6$BUDxLCj(|N*0suY`d zk(h*-4&G8VPTnDgj-tMGWiHk)v`G(e1~%Cw-y+!&=yGJPv3hwr1*UJ4OYA$jvnEHn z{n2%ru0(fVoEcs?!6ZGSC~dFu)c*VTm+QE5)EUR`wx}imH~&V6t9!D{wZlwH;T%F0 z{YqzXRo3QL7K6oplV$0rYm@ll;!C;219ynF`09iOx(dlZ8Bq%SH5Gs5nw5#hdiAMz z8AwY}xS=NMu@id0y6(X?eQ}n9;yQb5v&~#B@c&F-76nU%O}HQ+3N#=fq#z&(#?)vj zqE4V>0Dhbz%phu-D9NZ~rjBMZG$k@>hPz;LQ5`9Xbh-Fckc9PUeQ>fMOl&e;ZDFF@ z`o*Qv+$Mi+2UX16Tym(=T78jC@`&5l`Lf@e@7smI=5`y4+riFN$8Fbfw)3Uurt?(R zuC<%(?aM1Y?u2>Gcr>|90Tl-HVU>DV=vGut3d@Tjf^wFSS9<+KuK!_WgI4p?ilB zN0yztmfZqHc|xCLhi>E98`esdRvG{4)fv3X8T#+oC}nGzava^J@_DyF^xw(4)K{m} z%jZgb2P+q+KjNSsSDMViOCwqD)N$ecxm2yf3K@GHaI%DDR%j`OVgFjp5EqEF0(C4?ArzF*&#p{dd`|gh}t5s?R zMI73Q*Dgd_j-3Z9LsCtcw7+>=YI{tf0Y1qNsr{MwW!sLu31)=&iKf4tHA$Inj^&DF zaw->*^?c_>i{%epp8=q*;^O*>eUw_7!(o z*fns@TFGEqKbn;JFtX~pS82g=|0aPg4ZP)5uTW|@wHme%&ww|0a4OrpxOI}n28gm1 zM>m_hTI10yj)enNOEj#Ko*d9@k;Wg@t&^O}R1KAjy4)O%+6gjClh%}rioJgtzq@s8 zxNYEAvaTa#WYJ|)Jso6RNWp;FC&S!o*VEi*p{6mriTp zK9lR97@tZznstz^2u}f0SM{y{*8dX7!?6wT|{;@uAUn1dU z#hv(fnYkHANzc4LQ8>bKbuaj;hCjoPP3IUis#2`yF|{F~){*78 z7kO;b>?p}dTP@~ee<3�H8ceb%oY>zs>5^+_1=aTeB=A+i!AZf8RJa@pg>S8>e;AL4)3QY67cNWqjJHJ{2cH?>X$47V`HKD1A z%M9G!{$brz-vl)s+l%(`DWh}wu}5S=<4}sVnPq&s@#q7mTb>~o1SgL_@rR%7e`Q4; zU1_4L@4-*DDxe2&Zl`vOVrDE7D67ll4G-qhqggRtqFV{xPfWU^Dij15=%6#{mO}OR z|Adg~RssKscwp=Yh-mQZ;wcB2`OHCA8f;*|l4Ov5$KsF`d-moJrwIws(#f{20G}Vn zDW%va$whZpwv4Atubz1J{w$=D2+`L^IaWOU229tr-ui4iTWPT5%(t9QKSUzyY2QUV zNol0F3%(H^HrDPrISp^4M_Y{(5^Lx+$>?ebcX88WKn;BX80s30>%Szx3(K-$o}W1! zAqv|v77#@^PEdo8AW`*bFFW!`mh;e%TbF@utTrb07GfxN}y6 zUa+TI@~_45jT0?AWLI0;J8_z zo}9k?VHG0}sB0h`(By99UwY`A6zVEu#OaYDw##fqO%4T{P*=8SvxI;;6NWG%AYe8_ zXxH*8$DMr_ou+`yj-rtb1CMnM!c*7PkTSx{ukkJJ$^oa@Iy zvs&j_OJ@?*)`I^Rmw^R(;uANfUa_vy-L4*t*iDiN?9gB~3rD@(+D7IVV2F{-^4nmc zNRya_3rZEbG9?BncEI^?1#3{rBHsN%*(b)-A;hvpW+8g!w=B%W)Uz(S<2p-Av`84p zw`GD8xVt;86(peuMNpg-Eb!HgIr;lI?@;z`>QilQPVEO}ZG{#(sMka|`n5@1+Yn>cS32eYi zYG`b`-~b{kT5CFnH$k1s#*m2yl?{HG2eAq(fJ%(*YVFt#!yRm(__Q=&-Wcvum+en| z^+Ox^o^|GJ&IcT_vZbaX68ASQ775&GVA=JIeJw6&G1FLM425b)VeGek1kt;Q>!L(9 zQO@0X3O3Lr<6eb3&<*CZnD($h5iCd}D|tK(Y>{w3n7Ubb&vsUA*GQiPIyq-$F$d*D zfJg=vpK4k3FUWQwBg=PxgKsdH_e|f?F-+*o$AF?*+18|zK5v$(m4GiQ4)flt-7HS= zom7Zu&AE|zD-ViJ)Zb&GW66(qbm3uiR(Qw6c=w%lOusSof7C!kp)K8l`bG55`TN^p z71v4!r=#ui<~`u(rKUpuOeZz|F`7C(0gPwP6ePM<7drf!mD97f8Zd6hBxnXx>yOkd zKMVt>tZJZ3=^IHA#)e0lrk#1lg*zL$Wu|R;yz_JI7=QB~f!H>?wRP;)HC#uTY6Y_i z>2FTZ?==B|P$v>QKRE^;b_+sA>61%{;+&?oF@H*xd=v>Tf&T4bH^V3q!W|qX3Ro_w z1{4+w{`5}AajBpJip^b_1}Qhq41bA-fz(a?O>7p=Q|?ugEMW>KZ4BhHcvj*>2^*3K z4KXaL%w#iq!A|uF>#xU$Y5cZD_0+W~WRVc|+GZ!4Q1+$V9^T)SfnqjFpNPFmeU1!z zW%WKiU{yn7gY(B41toa~kwY?{7w9Ul{R6@P;^!&QBfK4nO1R)VoLgP=`iv>=VcQ~4 zWGYxxYBqlUdyq^u&!Yzn{rwHYLkFHf{qd$kCF<0FZ^jzKEvW-wdvq5{wAP7>GiB+_ zHB;R@*s)rt33h>(To3hV z0p(}HMJY_Ok&-J6m%8R#$s(gotl{(ROjoj4HlmqDWlBu73SG|Cuq(`wKWa7_>=+q_ za_*k(%M`~}$H9=R2sddWRsv-TEM$Y|qdz%r5t;QW$#>=B4TTTr_#hG0l<-109W9we z^kHBQs92?3K@y{q&<9SEP86*vbEI1zB_5KEVyMpD7*z&p7xgSre7DT#CMgzKbA)f0 z%8&+h2L&Q)&9M6%mEGN8oB1(i`?MB>1EmaI>ovibYU75|zg?DY_D_^srI&~0b>I_c zt}IUrAa1T@wKwe!rd}?>EdQnp@%@f_FbsmzVlr(#}Iv$kKw0Y!}wyp_f8Pvdt@W3{3#bJ ztK`C=mims8oR;pbijv$&Gif+Z4Hb-$AgiMA{E%Ipvq>>0&lLkkh zXL7jc7E)SoRn>aWztc0voi`!H6#|fwPyGQ;bfS;9b%ipMu1?oOYpJw5o>6&thv4mz z((MS%FV?ZFLQXx>^>JJ1A*=KmX)q&K`RP3%zXp0;1c4jDP{KG#`mq>=8aD__x@iII zwIP9;+G#`OqfUN5GR(D*q?Ci`U%}Ob#&_@fpJoUT^S8z}Vq{+gUP zRqBm(-#&(Ebm4y={5x| zIzR7ha0e!NI{ppwUd)dLJlz5W`Bs0*i^dGX6n3Bu*T-p?D4H+@U`bZT4Xuvu{j@iM z4e%mA8;!Ryku#2Dx;mA$ zHT}4aA5E+Rd0wSJ82D9xfPTs$G#&7|S>HUoT8#LPYhl1XKDCFdHP2y_zuShpnK3l6 z+iDfEtj%qAWrp&_mpUQ%YERF4gVY*UKC-;YapybZl^~Z@$1cL)*_?LiC#8Dl^2IcN z8bXJA77Vj}rZ^l6rev3pSzJi0KTazvZyKeW-~NYAji{z#@KR|J+_VxWpc4pl?O(r4 z>CryCVt?t-{X`nw#=mthdB?ovpaGx{?d7Dn4w`npK=`hLvcD+3N^8X5zT_bSAhRjf z9~6YP@@wruJp}w2OrBec(BEDp?-M9Y1=7}f<%{2MZ|L>sMy>A4(|nV%3hgi$LwtEA zd&CG{%^i3U?7qOR!QtMJk^y9cmraDPY`7AkHIQGKCFRUAs)tIcEG*09ibv-MUtO$U zNRA);E4(o53}(;Bg}=Ui#%C`ox3L1S7mY-xN}?Y|6c5tXE)3M?(3YRRdv`TE&Jab< z+wHOmJ}}Iyr2)78WBSFoH}yWaIl{XydHr&EwLi5#W7j}mj0%`xkpPDIDi6qW0%0gm zwgG+R*@wKTS$GvSx7WW8ZRd>tj;TQfJ5aAC<=*AFcTBW1iJd#f1#i9qz-LK{WucSd zZ{y@y)}Vmg9U>`E)H3Xmxzk5^Yw3Hzp?#@903<>L>aC=>$xK`I6J~}la+fW8t>tf6i>@?1 zXx(ppDJHjn*!4iQNDv>}GrzYq{K$L~3gCx#LHlH%1Z0D+fWT`eR-~`3>1{Fi$7=3S zEZQ(E8xsY#KS5S~r~6k&{R5jup=)-~lIC=f8I%+==_G+~VrxWPRDw<>8({iKem&sZ zBg8Y_-#)#QtijMut@v3B&t4RiBJcD{ID6#pw&OUjg3dWdKMQhtkR7*D_Hs#o;RbOw z#q`vKg|&sc0rZ%oxPE_?nw1px2*Xc!4$v7-fyX-*B7V{cWuj1iuvZL)<>|s)={(8p z=~C$5aRet4&LRd)Rd;Rmh|KfY1b@BrAy%T97@saGrVj`e9el}x&cGp-!{FZhyXz5V z)v?S@!{GU!)@gn9i==Le6|LT-84iS8%cfL0VOfcse_8&>u(M0ACk2~98Ss-4o>?WH&#DA|jpSht%KIZ;)#)p@_ zy|X*>E{^QSGE9fh>+a17(X*H_{rB?8M(Lars95OxT0-(@8~BlgQ&%s8N<^TX*IIkY6n zKXY63C~BL!lO(o8CPNxVB*|g+dP;Pr`?7cUqSexC)l*N-@fZKwYQv*f&Tpb{f!hlT z0M-DU5ky&#=vNJd1BelbK(Az}A^>r34H`TX^EdU%DmyeAU~5;$zQz^A8{W@5=JZF} z#|AxR%!YONkTdr!i4l|qImlh(Ci90-^Y(|>!I(%9`u!;dN@Kvl48nCHWL(b}9dxE{ z^Gw@#1gXJ-`x<7*2f=f02$GDj@&+;iL>a1n`GG94dVIhwi7--`In@$iU4vbsqYK!6 zU{yw3w}T#`!p8NZ*JcK|mKy-!g?3zH*?r($yBk7hLd2^fVI}zbYn5yXk#R}g3YL?d zX4)T=f8@;1v@VkGu|4*h2W3*4=zCPT3KhGiWRO*4p8444Y*VGu&(ACo^YKoBS4z7z z2P3B+9=B7+j7_P2O{uW*pdSg029}d0uXBt+`{MBO7$wv@( z9Oiv}q<08)R;Ts|3MeGV$xz-w9D=+C90v>q+7nL z^}*f7kG8UB_n25Es|&G|H$#F7+(yG-(6&9V+Q8=1&5tCt@A z>KxA2_fg&Cap$LJZRajTy~1vyazDgE7a9zrXC@m)$cGgF3KEz$YIOP!sCe!>qg)hLLGm z*)3>)y91t{=-B3ZKnjEFJ7v=eNSyq{5ZZRB=9QiEr~gUqqAy<3N}X#*4wT>8hRVI4 zqAyYBzp4`8O5-b8EZxGRheKjitcb%dFKPYaYx~TmpUdEToeOssH(=;Xn(Cm&nD5pI z;X9M~eS))IcP7fCx0QO|bEdmotqh*Tmv~6y{d46n*L2Gl&}Po!sSuV(PWZsCpBj={ zf?GXoIHj`Duw&nzNbi?AGg2lr*=%gAwDi9HlDla zJfeI9BJ3;y`GrwydDcl9!r7Jb)ThwIWM17C!%$-#T{>OiRg#I7I_4a&+EH>=u;Wvm zCETl4Wx<|tUP{^~ z=aREKzZ;As-q^SPBHHXpJTCh6DQ5i49*=ZxRi>T-CXX}v7X+3dI9n-8zPH6Camofk zE?d27y1UJK)(kgZvXtu~f1n;@JO9gWi1|V6KsnvWQ^~VI<~g-70N$y>({Kj8>_x!0 zoY_8FF0iKg=F-fcFx4sJtHJwOc>U;AG;?^;=1Jugt@L+zqx+n6BtgHW^4hH`y{Irv zi!4zI2&I{TAcdqTcu7ijC*elq_lu5LmB>v{#_|ZRU$#eE`)+>pSJe)ihPPX4^LWNS zNy$JXe&ws7au33>cyP8fNUgY>Q|;SXA_sX+v%7Wq(qCsy?y<9Yb_9B)zbu z00St5!ZEmTp7^mH%)*HeQs{Tz9|Y%!7b)=j77U=*dM6)g3I)z9*slYi2qzNOZ@pjC zFoHqT;Y-)Sj@%14C$2%R(V#yS3SNR{f~x5ic)&9Apg;H>0@tvP6uOXiKt7gGRJ6g) z4Ft^mKN!2=FNK<5C-Pho;_R>B${#?)0fjS=`@Aiv6}TK}{M-Paj5Ayit7|=K|D0Ss z=zsSfs$Rj9ZjjGgsL*}|v4xHuLrj=ohb8PR0|$cc1THG_UVXP-?LnNoj!5O%2g(!l zr{BhtbZbd>cXBiBb&k=s{)ff_!x+*I(+Vy9_t<-JwOLL)_K~PRPfrxi!iqrDJ5qu-S~4V{+!2w0OD^hi!yTj7mE#M<684DlBXOERKU zf;k3-@?HnZGlqVi*Wl%2PpKHmQyj(PqYrzP^@ObD;$@&-7x3Se*x#|l6S)3y zRpc=HMELUmbos2;##=?uARvP9ARtu#M_@{@riM$Aj|Rm9aLxbq(XY6=SGo_HrY$py z2~$=i*T|!%Z^-}f!=*GM%4e4n@7 z6G?Pc(9k@^S%*L?$ih(`VHK0<{lp;+>LXk6KW<`3Vi7H*cMvC!daQtcTl5Fs)Mk{5aj@GOH5Zk0Q ziSl0~a1M2?j69+%C)d8S`lp+hZ7%o@xkznNF*2NPK9)I1q{n8i70sbPr&1u)AXUzE zdU|T;T3}JHtYz68XQf&0>C%vAp=G*gSymz4k|#9@EFn<5_bk?XHspHta2FkyTqh4E z=Nf+3ya>1BX=hqZonLTjvWWgxcO?ZGtxw`$t#Tku zixDR08%a@FA4{GL)LUo*bM~HTGG}8G80w=jo75aRN|`m1%xc8F=(JKXe*KqAq@mTE zt6jJRP)07jo>NUj!f8o5Th(b;S(bt#rW*<)gT*$3xvVuFNmgkM-)e@dXJnqlq}l-a4JWQ-5g#JVAt9B47LZHkxHe!~$13~wZbHTsb$HT;9r50=r*bjHFJ4_k9R zlqkdsGl%&@9Da&U?Bb`UJp}7xry0vaEFR|d!EWor=fm8`mf{On$|oL3l$)Xiq&=Z^ zCosZi*XD#zyktAC5XMizw!j{%(bV*8+(0>tjs0v5c+YApH%Tn7huL5>Ob-7!w*vkR z7`dTWH01O!2jGob6;OzWdf?thJ$k)y-j??m(QxPyj@bJhgi%n*oi(l_8l=qBc0^%) zNDMYWhLgSanU-7Oe&P1DdKFQ0LMxV&7qZ-e9gQ;WRD;^_IB2Uwl^@g7IxmA^7Y{Z; zM!+B{Lr{8l^B2b8-Nl9A%0lq%@J;UlR{O24JDZSOXJOB7h8p3;cDu`B_KPjk z^}4^z-ZPjv@gcPlWYoQ%e|rdz>7WWTk+UF==Eu-JT;pS8Vkg2NiTRL7Si!&ppxA0o zSjjEDhcrh>T}`u4FL`9#n(WIW>w``v*ZksE*0Z|?Iwp@z*om&Uz2jb>?8YjW>(~p#S@VT=+U!sWi)Q_B5 zzEjdgcXh!|BFSX}ch`eqfYgq64Ss8p>q#?-X)}Ox0%devVk-Gqf8`@yR4>}k!*=g> zTEC6CW9O@n)Oz*3VuVoO?%7A;=?NK15C47Rz`LjCYO5iJuVqH%^?7bq|LZUtJcfXp zAocBfo~7i|i|?*4;_WxIo{0Qjak9P{!RNY|@p~9gd{x+Y;h+!^pze1a&)UB#_3Y1$ z{>E3qpELc$p5m{D{RQxnubYIhYUCp~i|wtpyZ52$&z+IE^&8;hro??ZDTj*pW9-IU zV%kmM=9B#NrQn$4Vwhj(?P2lN=oO`w(lT+)Nx<0q-PD>q|Ji^whn}M>+*Gd!g z7y3z7$Ii#zwA#xy;9)u4u<`mB6RiW+iJjFftFXo3(Eb<)$OoRe<+#UlldI z{g$zffAP5^p9B1T44aBh^W+W-+r-CpI2O#QubB5N(&5_$rotVc3jbt*Zx$h`+k;!W zbGaaMv~zFHB35th-Qu{%Xm6y23HH-(@t$*H@p5EQDw>Qd3wlhkmqP~;?W-I=s1v*o zWb8Q}_aC8ezYO2sFDVwlu(yqyQ*A?Iqi?d>z&*o-*XS+RyA|oERdrD2^;L2@Fr^7q zCeI^AGQJ`Jx2NLWDpVGb3mQyt-*upx`Z1ui^ZegQ-^53nO`oDKWu|kF{HUm;!N*kC zxriR5MgEZ4eJ$~=&g!naPv;IxZlb=uDf!MJ?q7E8SJxSmP{sVbb!l4Snyk-T`F4=> zW_O93i*xobjJ=!~da` za5SbjGBvafQAc#ZSj7Ja1Zi8RT{Us`CMF}YX_bbE{$uNeeG6%J_tA%H^s`jiI;N{* z@-V(*-PoPM$E8xS&j_E!*7gD;l!R`g)Kv;HOKV+jA$r9M@*M{SKUB?DQQNh)s!{d) z>!NG)RHh{Def1w{Wc%vtX6vi#`jtHzU?);}G*h=f4wX}uCN<#+LxE#y|E_vJx|E9O zmym8OJU<_4$?bIZ_Un7gNMq}G**DUXiXGXA z3P|WnCajrXfj+H1m{Kb9vh>yI>c(`v3!YgXQ53F0eFwkd)h?8^nD+-s zDO0;54HhhzgOZd)*qm2k#_$N-0Y-dH{ASBHw69}QXflBM=)(#;_WL_+>##@aGa)CeLwwIdOa&Z- zB;HnzeaSQejO)M}>467|SEj^E4j)m5P&w7skrus@5$yDzM%`vuCxQGdlK6|ptmEDv z{%T}S^x5$?)rxWP@UfjC0N&8g82ijDC?C;-l%jnos=#^H%sx2HJc-^0l zTVGW#X&s){bXp=7gxd7;!#uOOt(^Htl$^UIPD=_ULfm#|r959~TDrut;@?cKd4!Rp z!rK)BV>fyrhc&}VxjiGWU-Z61XBT}#CH@PZ5d~?Fz_8BC{0Dv@9*|)=G{6vzbW94* zm+3GK2z4Q@3aQtPwTwA$o&)pu4?r7g<&sP^iBZSe@IQYDtoiH;oM^W!VT~88*QRvm z8KSFS&ev9z6s!PwC4uR#O>Chyt04MQ8BA7!vXHltseB&8C=)0A#mlY=B5{}AkCyls1E%g zardyirzrH{T=>b2x2qMd0Ntl9_~ACV_~)YpqQqlnjPSvc=Cl-j(znluyajD7Rughz zQSc$@b~Hk6cM1er?lUk(Tebl-m-_|z_Vrz@a+p`(`n$h6GI18n_iBAKoEstyFh=xX zw^&Zi#PN%I3V;k=@1$c3<+1?&&KJT^?YWf*A95&c8W}gR?8ip5Tm)6hY3{}sC>}JVJBszDKdq>CE0lXlMyL>}u1$ZBu)u*)$dVNfliUM-jzYDYR3p2;SaaqeaI@v5TKW|U?6 zL?YO*>zNtE>?+=ajcD2P`dnI7N~hw zAX`+9q#k(^J3efVxGNwWGEBJFlaQtZ)d>61yPkrf@c8x!j%RiIs6NBh&{t=M^WvuR zDE5M}Za;NshHSvRaq^>^oc0ygo_(C7;cYNQ2i_gi5NE)C&-_p(8~{q8<~5&{H1xs1 zSO5o`T>%@SAtG8B$bQTfX`%7(F1$f84@@HXh-juH2z|u*)RSd@TJnuUA_AixEEC9C z^}tH%-sjP6m5B%JlLv?(*D{dppNyO>Y-BlRu~_-WJ~Vy%yz7!W6J6E0!}R&?LH8){ zo&5G!G}rU`JLV|s-Yxr0VjM(13_>O7HUPsc>P?;h^$sIZ=n3ua%4kdY$(^@>q&|Jn zN0_9p;U4hoS^QC#M<8_Ci&;SG$hP$Kk9z`dX<%nugRVDkCaa@BCro1T#hX2WFY{gf zptE}(fQKm!Js6qkenraRr%(Sjek8Ezy>5>zaG%?w^q9qY-I8VQN1u7pY59&E29Q$N zb&eD!pUbh|6>@p59j z1^)%Mz2dKibYgX!`atN0Gw*5BJY{_gEy|p{3UYJpSIFhHS(BOu+H-T3oRHbLH#zP&efL zNXNhjs8v*WOkH@GalckH58H10Ax1b~`JQHVy#+x_Z>ieHOs`&X-S>xW$U^HWP4oL4 zrZv=+&6UXAs$*q#)a^$A%SbxBtZph=oQ?OPgiM;J1HH*DF%ND!*`6mJ7avT=uY>~H ze9t6V-uuzn%g|q*(z=VJgeBVoO{WYwBvD*95Ph->ZLc^Y6QgjywqhbT=_q&!1dHWK$N24D6Y4gP zb{2(0vGw;GU5>X z{o(ycI&bYYoPp2Nra`YhFtG)R;gXxx`_0y6UB`%6x~Ou{UkaPtpi)Im6;>1xn_x)` zb5imMDl|!Sl3ALESoZQ<7{$a7WXXc^D$O+yuW9?j-Qt_6`#(!~s^A z(}`ZdsGnI17KMG3L8?o1%x_yxqZo^5m9G55kceu45*1g2)vxUdx%4e{Up>K8DoxW> zsMT5T3}#*vD|221y3-f#g;C=%m4 z>va;fdHcyaLyvyZD_~521J$8ATGiRd4Ty0-b4dky%)!pR>uQyzZf`aM&VKX$5Y?0r z7Ec%SG-tHrQ1rwGQf6Jmt_<{hgetKeXK>q~io-;gNXOZh7zy!9V+xsL+>c={;xiZ^2jQz7>%bcK}^p>!A&oj5jN0l723^apBlyj*4At#Iys0*UX_8p zCv4fY@pW(|I_!CQ1{+Sqq-3CHWGBE773MIM@$4^1O6v;!V841JRXS2T(XyO}zr#0P z_g5>rEj%5@nB}TAEI81K?eED`p$;8@rm1ktHl<5EFbcQy!R3qLsXb+rNo>SwE58u{x8>DN zR5Vf-b?{-l*kK`zrqfS4qM+i62`NPjCK9qzQ3=Vl7(JCiBnG0(Y!Zkf$PBR*b_EMI zg(b+OMaZgi;!AriMe=ABTw_%SjZ~y{n4WTZjDLk;R+BpY6*=N~8pwQwU?ykL(FTv1 zyzs}RLtw={WkmwC`z|IQ!@58LnMa1Iy@*7EP?G*UK**$ibCjZgfkkk^S|Ds?paSUD; zhOHXr@4?raaVE|NT6W~AILTLLDp`&#&NdyV=&#HYCmA*w$aOGYkV{t$0Pa2dHew7d z1c9JO#9{>wY}3~TI`S}B5euJpY6lw#K-_~1(3y%Rd1--_z?S_Z#t4c!v>)iR=MB(9 z-SZYnL0ZK%;A>P@2hxuuTJC2Ie2k1B@KoMgG;(l2ZWmuUqzQs0(>Y-9K+DGzvivPw z%mN|q3oA=8q-ri`vzk z_D9tB!(cfB>xJ&ES;e&_KJe*wILA$%kR5dRc|*osS6{v{6Ko&cX35LT2Lo0}Q zURm3~ssJaZUHFsJ&}m4ViQrjFMccrP7M`sZo31<|HFM1gBz`>g%39#DQc+hQDHl5UpziVFG^L*CQzm@o|O?cK?18VRDew9)f}9 zbrYNeow)$}?VxUe{@WR9)Dmd-*YJ=o5Hc1qjLb>Zz@yQdxKM)nkcP%Thu@9B*3@%l zzNPML%ALh>-f$3Fvk7~eqi$p?L@K!Crv{H~NV1Q9D}@WvT?+Y4W0B70d!K6D8Dh7lGxT>V&=a-JkY=4{5n z^+!MElhuON6{QFA8+)iAB1d)K;x&JvjpeP06Zo?PM|J*U<)upJsVT$;ebBZWAIhiM zD3KtA@VDWw1>j3YU{izV52RJ~rSYaFAJ|xg<8ffqqrs1(``~7>XYxUjLrqFB!8+H6 zW#Y1!7`yDCOb+cl9Ep=aBRIJ8LSa|_)6noQj<13cN;aMnJZfU}K7XWIiwgc-$tX=% z8;v_E?WcgDsct}uwG7-RBjG@bv_UmyP^lP$2fct)0*DX*fW1Tk<;=c&GsBcDxKsIp ziY}23THj}09t?45m-i;A2xh?s2ynhl!3VJV)kb|+n}Edujnc&-HSvSYi7!hWd`cUo z3#41X-8L;$>4;x4i_<=$>g2g5v_zjt2`0~C8OAOTIWG-4FWz1hzeyrRAL)S437wNL zbuGv)0Vt^XW|bO!9eYA_B|eN?UW5^ipAx&7dh&Y|mK47s$`gAYiD)K;TAk&yZnR`w z6xUJOlNWkY*ChBc6u3{9K`e71nRCwQmCY=AjuktEU6LIKlyhW0>8h9nJRs^9?_ARZ zUzTRuITZNEA8cRWi}S^fz+US4*ScUeVVF*zQmRd(C!PH(3i-}MJspfyjvPhWRR{QPx~<7>?Cl-yr0(bYAZ z4K2fj{L(J4DHJHPsGD7W8s<`%n^mb;1o9gfGZ)3hgpeLdNB zK1Z5rqmmTZ`G&+v+bLQf^|&ru^FOJKNu{J9@6mU6{tEiCL7qGBdi%&y6`}LPS2w z%Yv%&yc#S!9tdgiLKvrC%DKHnFVvUYf0-H@9;PmvwO*2f?KZVu-iofXw!6IS0j)g1 z-LCxEq_>?qw>y|kQgGh>$TA~!SxhaOG}u$XC^PzMyf1L6rkqZl>J%8woRc?0j$7pz z6Cn3H<3=$4HS+p@PDesH;G=B_3C5&ZqG=bigWAi*aL+Z0s+HeO1A_1a*M~zm%vz*b z8Ym*OJL{#+6|cEltO<*zLaKPf0gdS^mrh0e-z91K`4ufz=yeXdb;h!6QK0Yq*n&c= zIMIH{ci|nbi(?t{(tu}#_3~ut7O&Bps4L=^ow^H{SMAegotE`)u5;Hc?`KRt>y8~Q z7mfkAs15?`tN*5lR|$_bD3^Pq00AlE0|EVf|8Iv$p)vBie|YiYAK0O-laawi)VSi} zRGWo1^>mxopp${*g#J7n40kKW_vA8(t_>0?jULULwZ{|M>jA#&I^EZvSD9W{zrQ`U zxJykg-)KV8n#;$QTLJA%Sz%1lSBfD9wns{mZj9*S)U(bc$cLUutqCy2N&OCvM2RuN ziy!7BSO`HVx7p$h+ZBbNXSq1yKbYeo;TFvWXGU;m@+KXLLox`*BxRa-<|i;x1@g?A z$a&#Ihz`^;XW-hAfj(u<+}p!bxLIKae*;OsD06#84m6d3TL8!cWdAL4Od(iS6Xb!S zf0;VbVj`hxzN98JRiAtgG0L?B*4tlmW<)dK%tK+6;cQ_V4C&+Gx4$z2`W-kNK_~`8 zXwl9E+6d&sflR^Y2<2;qjA@cVXe#jXoQcT5Y?wJPH{kOA&`XQrB(}oM?sj~flvRU` zMR+CiXrVTJ_5eDDbSUrwA85A8UBU(ZY)076uvAX+Bvm&1U2&?cyY!>5nC zUPxI^c#JmkVog$E=yVcTuq1dDgkR||lol(+yVo#fVFBd#EN!KMH&id}h@_1DlPRdQ zBIR@`2vmWBP$(QsLcxQ|)66xs>I0g^vEXJL=~AW3!8%2EhI0RSruw7EyOJ~qFvXsi z6|I6+ggDa_q!K|fk!MfInd}+GQ%^`@vXuqVX)d5Z^Ln7oWOV%-fv9&JpUDXulnFyj z=!Xk2RRqAyMMs2?W1>(MV%cX!f9@8!CGIpi19SZb0=|eD z3@j78k5LAL6JrYgCrkcUB9n2q+RIX?^{`^Q?Pv`C`cr;I`ug3=c z4MYL2dWG61F3EmlhiViIbZ59y{~978{+=0!-2QpiPYtlkjfIK&vO^n&XDqJ$BReY{ zo&;kofq~C_i-py#?iyAfs*bN3D`9wsrdWei7S5PWL4uUVDIGv09w2Sar-i@-JqH#F zh38sAf-ugm&-z#4 zX*E`LwIvz|h_r|g1FT$ACx|q||9umW(b{78=R3?t!*W+KNH*R(U8^$=oiRbp42!~T zj7*k>h4dz<2#{CWfdJ39CaPZt3X=p+nIQ@H-QyS~p+*sQCA8*;S+7FzGBkYutq1^I zon2CtO_Di}`2y3)#elNr#<36`m>?;V)LnBkpxl!!i_{5ra+ZzLd0RB!^-FL4(@pq9 zVS?!#bqu(0n=Mm$8h6$N1EGMWqdZ#*D6=t+W*u~7S5N&{t6djta9}&rJw29!1W1YE zk?5BYGpHLR%S;`RBl*g^y?j5JBl7^HqA}$fFheDfCDYQI2c~)%&%v<^suLej>`LS7 ziJlqwtp>=jPlK(EGXwt?3zzUD#2?T`;mkPVzt`r!4gW3tlW^hoCYe3H|71^%q)*ti zbA5y7>kzMmsF)=4n64hoUhotD`|Y}_{f84%4Gva` z4Ui>R(yN6X6qNyiQ07p#;T$<`!X_|hNWLPnstrovMBYvrF1Ta%ZxCKtCLcbb58{Sm z>?=h?Kclh+*AsiybOeM1fwmfulHUiBOXR0G!wlL;?_UB24l>&cp_}rj(`@O^bxvHk zc8WMX#lt<+2Yx`t;5WB)IG>9;m|!Fjhe)ENBe^g8MOH8~QGSEftHR$PxwHE0L--t8?D=NfQhk zu1~y!m~)bK3f-?EVU$Z2(>-62m7_owbjPn<1z0LAlz}q!pS->Ht9!EEpNX9an3W)h z61b#&xJ-fH+CWv{xhib{caAznc@S6Dp_saC)e$w&5X((9*=`)Jv^Eun49nGvS+gsy zEs;B?wo}zVQ7C#tjs$USC_Ji4Ge-fLe?_FvuwA@Cp{zz-y+OIUf$@9nQ}qTNVZovC zV_zy+$uSLTtSRa@^OB0#G3#P~&0+#kJW)#5nL0-x;uRL*C8vS{@H}Mj^}LF7DNi=} z^U9!%@pSr^>$K@B;|!83a4SANSJe6Ikn6n{$OW@px&3)6E&Ah90>01coN*XeBKzK& z1T+le$*Me|sE|df!W~)qP@lMTA^Ntgg>}uJKA1b&blBa9s4{%H^hJKZ77b3L3%MGU zu=(i_4eTCdjNYICy5kR%VNmv2W&-_a>tZEWDS=xiEa&H4lZmX7&RLNDai3c1V*BNS z;zW``S)-;Oh$N=>70T!*AR>a6lELSAB$f_X&F&mtHX4DX=yqQ~#1()TCA6*Py2dR1 zY8%5UYm?oQ3u&sfz#f<2x70v@;WA0KsyeI*tbcA^zSZgkM0R~7O9-L#^bge}zgn%o z$k4UqkBtYVcLtp8wFKe(=_Sf&xuu;1_C+7BLTeAv)Yk3s$k2N{e(Ag%gu0A9RK2yi zLsI^M7mkaDZrI+J`BpESJ9*8r+!J{LZF<8Y2aSxm8QS+jVC!aiPOkKYP|tvM)Nw{O z&Evi?r$^8OFmh`Mgq9AHo}vnHnL}ULju^L1RbL+c8UqiP)DzxoKZP_#%Uv1l_hp1^ zuj8JBm_%|t&dqOv{E}y($L7+q=(6NP4a{%k(ba560}vqvQq>2z@U>9edt&rp>zhr- z*I|7nJEb$;PXKo9ggd4Ew-Y`#kIOIQ|U+6iP>#LQizI;#`gzHQIY+JrIH2 z#><6Vy^OsY=v*HxA0r^q3~9vp`~8a!n<6V#l4$Ldgg26C9g>I=S291iW@9w*)0Zre z0)1u3$E#3{AsBRKN*pY>ocA4gCSBWvVB=&4=7${5U3lyV1D7}5aRZAjqFIiw10y_xAkogIqEkcs%+%F)NM%vKw<^e#mKRn)pXz?nP1l~q;2oo5jcr6 zCYcOQ%%yPA)ZiklV!s7k{cYMk#S+!{im&Sd7u@kl7bmWe7{p}nb+z6^^w-$0fD)cU zX`HALb}mA%w44r#i^sJ6pD`A5wou$1GoCEiylU+FQ4b1721ZL7R*Mf^i$<&#=DyUx z8dRT4RE|z#aTRHS;vFE8`0qGIUd0c}f&5dq6YTmqa%Q9Y{$~8vJ>*o?x-e_yqFE5o z&_U2{A#h!+&xDt!<5y3MYl5k6jcGxf#EB?xem*$PiwQ<-!D-a7n_S#t3}oE0OAh&2 zwwqm~GdF8+4XfjM4H@VI5pSL{?a|(0<$Sd2X*qlO0+K_4vK@Uq zp@2^aGP7$vvwIQfg3!eo?1wO;1H=o6l9dN~ljXGWJRqqT-LcfD#-z5#J{1}g_{+%| zO)Ag{ys@A+KC&xa8<=SL9f1{y7?8Wr=tykbHuX! z{Yi)Ue1i;U4Pw(DTcx9a{)mOZ-NiPr9J=Lo8v}(^D&PcV+@BHpaf~o70Cm&^O~{UJ+2c~*DhpP z(3|}7R2kCup?NsrW}*#f9s&yh76L4akagUzZ-8Zxdcs2(670)@*-KUE71tAXg>JAy z#Xl8!+KJXvLoJ6LWuZ1E{N{|sLWOUlD|GCN6;t_2_m3ZU7%##MILh%~;5`giP0()s zhYDkPnf1bX_dkq6hM>BzyXF4Uf(OsD5RhYgVb?Wzda;^Q(aPL}qTrZDVGJ#ND|u9wRZ6O(O9V=Tuo-GM-`f}Ow>LG@tUn-V-L~O|#w@w; zcH8;;9NTPiA5P`xGpxo-8}-?JBv=glvd)wS8O5#7pPz8$7_Q3K=O7vEmgm{xu(8rS?}ZEklmI z^y_pYKUecZ)2^Wha@~7`#Gi^S3ri@`xVZ|{@hXrVq=#An_Vr} z_X+pJsmr!1ojb*CMaMe9ZTF{+N!Qyszt0@Uvy$Un=UtY-)}h;EY<1_Ml!KSUve5wl zbdAnUUpPRj0+!=f_x-^8)b~8o<0Cy{Gr#w7Lc`JS^VR#-`!xRBm5pui+LN=4g^wL2 z%kSrjyN}|J1^^Ymo9A~~Pspjd_*o@Gi#x zItJMQNw-C)-hFA0`E9$|(v>2IXrudBGLewrufMgLMrzmS-8N}ax4^ez$L(U!_;@;5 zGxa)k;Pr4k56D^GWLf07&caT<%aQfH=z41xYr*e2?yFgDqCehc?A?w(xx3cD_qI@jJ z>F}R>x7Zf&bW{BV*LR-tor5t#Ei%ga;Y3R5R~C9%={Fibrcs#Ye>0d+Oyw3eUIEPJ zK3moD-$y)Kg>E-kga!eJ)}-Ppg5NLIbL zpT|K%8n(ak>w31=aYB8E&psdyILH%T^uB0NqzyureK(*=)70rwafBg^ND>jm6hWf| zPh$q-`$ZwJl*kAoO#3BM(_RisVgbFHoD-=T?2Xly2+obwPwse3DAW4o{bB=LokUQnlrV@*{8x zvFGAb=gfk_(zalaAYSzi7%9z{{3^nwX$s*OU6OE7XQibTXfpy&^-UV;b$G%l5?+iH zP5!9RHs$ouEGn1YMG%V9{#9=5snZ(D)c;=qekseFBKeP=QydBih&Y9*5*Q<;(i8*% z;Gzm|jjDnNaI3Mh+l<-t434g=O9VmTTUzIev#VZTGqMh>LktKwZGWojsJ=|?zG81QinLRp2y$9hhrJhqoMGG%Eim(JuOKvgk==rz%2%1<)nTBb)+O^>2is6gz&*} z2^dL%;ZTxmtx?SXMravOi5O9IE30s2&eA!rhA$m*j-hGIMXGgUjtaIwsx5@x ztL8LyA0DsEml$+1H&PGyH~g?Bi?BmWu$U));^KEMREO1A9|d#+fWZ2)JO8R%$J_j7 zwB{v5Qm)nybV}nCJUoT=b{lXp$lPBNkguS2c@xqjh(0&MBSjy`|LGi)iQ74BvO1~u_?E!MkjeSaF3#!K|Iv0W>RuAx>XTWyHE+2@qmyx6IPeD5Fq zPHHi5@406mqayr-Yj?{DBZMz0U-tC@Z)L|1j>{0mrphCs;IwMW51k5iHCBgfLy#<1 zUAQ4UE^o^7f#am?C3ZNx5b7a!+m~P!)!A(Y?q11=Y8mW+Y^{yttgmt}YppBk?^XFM zE8_1rruWHJlWm^ocN4kV`med-VL4Z)TA%4E=7a{<-_IhMz|Vvpl-ob;AsJj@HCH~| z%2c1dDw~4q5Va5|4m>Lj70`RG$k>f`-XDtjYYYAOud5BhiBqbhZ5utLT03s%ozXtC zrC?PzDhmxj_-BTyP5R?19c#<`$aD9NbdDou$GN7{d6!%@hi_nQmg zK>n`jyBTOzaeMU7pUa3)V*iyS>iD1ag+sa4JQ`1oMeNCOk&)K`8U%+rqVi=ynDtC8`4p@nb0#2R z4n!JIst|FC{l^j*gBl}@EPx7%Ad6}eX*obppT3NthVFhbNzFr3KonU|eL{}T`4xup zPUPc&52Y!$aY>DDnY#wOFoZ0cmc}0a1~KyftUNlut+=)|hyj7J3r9>9R-F2md%k{t zHj9NK8e0Z#-8%swBIn8MMWTnzLOTgn5~xYq4|@vSkqlX+ykVl|HOr(FUF5(WZHTp` zvV7_w?Mykz@17q31#_c;@5)i0-o5g_{>TmxNf8!MARzT0`!nu;%+KFLz_0*i3A=Ss zG+#1uh35Tq^aky494I)fWGaSQe+ckH@_5T}RXp-Gv|Q1>Q5+|1L(`K&5k&t;5ZouTgT<2QRu0k_d zK)xxh>_hg1OgXXkXy_w$Su=pSxO%K`o@8hghkGDczzq=!GVD1NQ)Ed5Q{_+CD|n5c2Q!61-@BRG9%^`i>1on5-Xx$6Z0EGu4Hj-5 z7_~Csu`4oq8s2--NNa>P-73IH^Xfw{<9(CN1T4 zh24C|85_+V;DY|+6$8+`Rt**g<-JIqpooMkZe`3-I_ezCNSLG4i=<9{dT#Y3gwX;x;nVosKfBG7 zq{@?yqJrIhV;|7o8C*3jy>>2Qkjlc%W98`!>pV7a5!h!y;LseCzjE`@F34?Ww4i~K zX?08mmNptL#RL4Mi$Q+()DATLvvq(3V~O-vjFdQ)3|m;k4>WZ&0SHkdATXZTNDwIT z+)ax+!?$H|3&y~7wu+&H_L02j2N2V4MVRpKnf>aeYBR%&SQ-y$mrgaM{Ay%JwizN1#m!@*x@=f90Y0x)Tc@g;N>4BjD^}8H>(!D#>%!yOpuVL>04xrIIYxmKmRE_`0 zi?z)0RIvCO%o5V1na9SaC^81uWVz45C0!{w)*F>&*;lQV=1gVgx)(1mrdEyX8(7xU zRttw9r5=a_43Wzj=nn*80uWU!M10DEc_g}c%qw!yU?PY_-sHgqdCizD3^Bj5zP-OW zwqF@Se2R~DmFY58RLj!bn|JBL*a{>gl0xQ< zwS0GA(bu6YMY|U7B7GOjStR&wa0<%|SNL|-;HE8BH~L&LiP-Y?JO`;5hBwNU0lH*| zH1Mr1@t1p8Z2b&v$@9C&+f6AFv;2Au^dOZ(^AD9b5Rz|WT=D^0 ztSmEfsF%y{>x4Zxj+&|3C(yGHGK2TpHJgZVBZX|31156cuCOeDpRT_j>t)Fweb+7= zmdStyrxY>$C%18$N=l*eB8HyJ0GVsmq;N_gKv)zvl>tULvV#8pJi03OB_Z2+Qt0=R zlnkX@Avm&Be*>ES<{EYZBMyk7@Z-hLNXQaS71#o&rf_DHkJ& z`uC&TgWR2<$WjM*n)3TAE*I07+?8)cIqc_+dE zq{cdc+z>~0p`GL36X}sD=Lo3a6qkS3B^q|P$VRoCa;dsklB+MzG%0BKhnCt{jl-aq z)QTEh>Z*bYX~^K0+{6{m0lpn97^)J*mJ19CLdXf?Q8P=4@TJQc8S=a%P*DpU6C^2H zid=q|;i(>w*|cns6Srlcc07$s(EBrv-ZG_1^%2uuD%tBU*A@yQZx>>_u}Bh-R)*r) zL@KTHNf?&VW~Q$QolBQxu6W5`1=^HzL0A;QWId?{o_=@)%f`Cu1K{)Q)oK3SBGp$% z`V2TCDiiVQJNa`=BWx&FO=X`i7+cZ}ika1=Weq?jmNtp{Y7It|aZ z46Tpm8QyG1Y}o^G^?dFe49+sy(vQ{cqq~&Rx(H3_yIE#I zrY*=bH7@8kfI{QnaJ!W>NOUFE9MXmX$stAw5r;@l$aAR{AkEnvR)f&Yzdnsamt}{a|D}EFL|J&o(?=(L_~94-?@h%at(&VF6$nV1@IPtZXtSTk zZ;=M9vyw_XALVi5Z(id83%OiyNiTB{RmpWFlnYRpj#$=CY8xt|Kj{=DHs-nqWJTy; zh2XSN>e>A-Kq6_kzu_jdU6Ny{_7^n}w5ym4@)5ZBZr)m>bZ1<|vfi%~6H)NA&#PTD z&MDt-ufM;)j<+m+AhJmc%H0)93JtXs01O00wD;d43Y!J_k{Og|G1PtHy{Jn0ogC%8 zW_6S(5(uDQ&B$+zXiwCnpuTLXnLN40uYa4}#YF!|6K(VX6(M6Rt`LMMZOIC?((*d+ zv6PqY3bgE9JqdqiH~Zxm)6V))F650lkMv)$14dCVO?-Uq=wzb%Jd!U8#ggLW0O2_K z5lrQNnl;SKc??);M!A9ppLtJ=>t$dh5CBg62MAs!qd#UE!?nytDRb*rh_9(}^}l4o zMF(b)p_y(|`E!ds*7|4FMe$gH6ZfR7p23uZb&kVBNzLQfMQmQT{U!4WSH7F?3SHy(OWw^^QNFD`( zgW-a7UTZoWwF%@=?oq)O0Docs2JSs5tKcbc%iv!FC2aklaxlaIUtmwxV+d)NXcGt4 zO=!EQJ2&w-I)^p2=nlyj5{iZJUrLm?N6`%hrJ4Dn)<>gLNknS0WX{bM1R5fFQsR9f zJ~=dyN)yJgs6a49paSuP!ioC`KvZpGC+AL6b+XiW#NnPZzk|6JK=u~BruQxBGkj=S zJhg(ea%g#~u_86qoaZmZFymvGlWcHIg)C+UuV*q_tam@jm-lpY4bp0MH{qsCA z+L!!@oc-7k1>!@S@zR3-F*OYfrf0SQ`4>FUCcJSTxR9#9u+)k~_$dmG+;~W(1xj?C zlK2EJyhansW<-`Ez>t8+m`&p0hu6{zW3G6}S8Pd03Y{Q7E^kdwnu=f97m!sSwvJUG zZNM@wXrPK0M^!+!voDQ7J{RbO^;S!K4j#^DiUKxKOJo6KDm=k#k^3iC4yQhZ>}?T- zjFByBNY!IZ(PfNeQM0sNGjc}zOQAu+c4!g~Lqpqan7u*)5GMdhn@1-T0p5rp9-xq% z_-7jC;q{-?q)JM9(VFQ$t#!Imk(5qv)mQRGGV9@d@CuS@quS`Mw;2&@qq(96R>~DU zFqbHCr?e0{-M{9)#=#_k{ZIqjqS-I3(W9|8FlI|Kny6_V6$=EMNHYJ0C>@cIF?+{E z2|I*@4PFe$0L&j?&86x8WrsMz%&4d1M#Gc1A}~i5s;1)>$RJQstgaoX8zvOU$Wl-k zHH}q-k8#Sg5hzC!VgvjAns{jxAcj=}FAV*MkXP=uQUjADHRwt)k6_z~=mc5;Rf(ZJ ztsjXV>8dJYnE@CTeE2M|ebO4^jxIzM5YE@=mRA=Wtg07v3@issP1+7Ff7rE7V0K$x#)U+aK`B zLxb6HOgsVMb~-P$!s!mZ@?I+rm?saD&y@)}e9a)iUhCa#>-H!Ra`$ zCJh;b$2%42Ya!>tTBa;XdQlvKUpD9fXPJyEUD*Nmus~kG$ZHGmfCEfwBXhICjZ929FsXFgwA9TBr_<2VvMJ~P{NINY*ELt)=Xm%D5j|^xgqfI)+Ur-hMeh5m& zzYSd=n{g*0qo!pb3DQ_XQjJ1V3tV}LxRd}x+}s9`(Tnq0XQ&8iHu0$Wt433UIPNUr zRcX&k{P_)dS7ReE3CFPXoMEA7tim=&c{SR3?ck_?5|K31Aw+qS-AF34+C>s^61!T~ zgNC@E-E-vth(QZc%S=-s0x6O!6$;4^rWcyy^r!*gEI|-1)P?21rbTq~X4^};T35hQ z(iKC2h6_br(!<3O;BvJTCzD$$&cu)-=ep5vXVo?6@hJMSGYH?O2m+0(cWVtsnKHdr zn{@4B=Z`gVxQ{*rS{@iYyTWeaE}m?g5m`Q$#)pOcK2X1DV1rIE(LV-y!I2#J_y-B0r(NUmT8p*=w z4Qwxk?2hB^0Z2X66^6-5Y9Ep>A0a(Y494unAHihj=Lijy)HWz>iGCt`R2smnA=K^O z0usa0AGo5$$|TeW!Z|A}{5?aI$6*hQbge&{vZ(6aU_49O;o3x_E+R;D0wk@C!nMHl z?IZ{FxAw3c31moPxP}7eGQu3=V@Q57u z2&ZKK_l?6+beLz7$)*HF)5grF-XllzJsWiMKy}h`pCt1xE+vuk1}y;jL6QQb=qBrN zl}Z_JQ@Opc==t-+3TTIY1}R|=2sP{RfR7=UHaKfZDm&po6s&+W+AT7Km&S(mu!}03 zi(y&lGWb=bzXxh~4~+tMYAf~PpbRy|8!IYoSa^m-H9mXV{)+9u5|=jO<3f~?yt?ki z+%Mvt6LFGCl_JyuDmlO%N4~5tx#qxjQ2UDt)u@~p@`;>6`Z=BGd2A|#97?B{u=k$c z%+o6)>VP3X(63VAFRK6>8NXRf zC4G4mn~Nq~-Y9pl8z7ehR>PC3+K})?1X|iVRNnc7gsZUC^*eyBgI{SYWBEX(wGjtX ze;YFho)hsaa_Ily_VGWmTh|QyI^j(ROz5_DhXWAiwe{hpwAT^MpDo2Hkqc{)7kupZ zG?76G!cQ@jl^c*YkVyp_t-0edZ0MY~7KHWJ7r@TnJ;OX;+l;ubER#)*a z8UGYT?g4;7HB$9Rv<{F5szqQ%-n>iizbej14Kn-X`3;PE5KM8xK7>dVj^Lv=gIUB# zE!r7&2>%NlQR5#6CbGqp$Uuz;h#!?%+TguO2L5AWwkK}b2?#PM`iUz2dQw8&KxDg1>Jv=SV__edtVTYdHWW3rLv%o`ri&c4`CIEWqB>415hd zmI(Q(Td)pBoLR!LN<1FYX>arNpjdMhR^+G{Baa*VM^f$=7?33aZbA)nMOK5h5*TTO zbagO@>z;zYDcE=gQH9*<`IJf~P0)7}SDSXM)Q-0_#wpBFu=(ue??k^*^o~-K`NM6W|G*T_B6v(`cK^^bCN6;Ll07D{Ty&$IlW!&Y&r{*wo3f8|ZBp@QT5)W+ zcZP2>NU7p+4M=)kEcEom--bngO*7{&oU9V0(qG_0LJ@$YlXEBkqoNo4`0j43AD}5A z2WXq(u+$?{{C45E^q%yYPFDTZ(yNv~4>ga?TdMwj(Tm>fYvQem%S0A6-^Ie~a^Yai zAjeXp$|TauAkA;<7F0rIp4K(Oo$!xw*Pd~+pEtT-w+%RJ1!r|TO02iU(yz7 zeCx1Fch0%1`~%oP8u%C1iDBlBs@F8{d-9h{ojU0R&cybdDT9~Ij{f1b!;Uo-Xgc>`q2c^+>+ zkDX@VY^}e4tp#KTcku7$=&f=(Nog(C|1KzTsUFs|doi?YG(Ak#@yf9?x3x=SdEaT) zs{d8Pzxm7eW>bjryW+mTU;R6s+-e=KBow3b?#%r*3o^%w*V*wrLC%z4h0W6D3KFzH zZ5fZ#&1&{DL-*|(V4NiF9s%HG?Xp|<_m9ehY__TT-k+%+uItDkd^J43HsfnOWVhO1 z>s_V4$JOpANR*v+y8bFJV|f1b5F>EQ`a1HmdrJR!?`YxRV>(29*V}lbYwtXt8zC4^ zp{3}4kX;)|*@xYF*}c;*^vWffSgL`I-Q$V^W#4x@IY3M#Y`c%Y8Un=ky@_$e@)mVA z0=C;I9Yel&$nCyaLc4?CNsWd!?|UloD@$p4Ibylr_dd5Ex4v%rdBsX9eJ`R%%X@g0 z9A0A;xB8K7Vzpd1PJNBGk8CWAggvacU=g%q0P1jZSoj21>xXikqRt94Ozc1HA zWw%z8Zl9c5#)h9qj;cO#0zWOLGj$zyyTrdndVD`t-DY*3H+?=1I~ZiIH66u7!*3DI zzDRt%j@BE*HnUdOZ-?u9_#bY1C*VIH?rZq(`-cOs<<9jeI|0v|6_~|5hMsc)))#RSnbMO=e?072Cdb7ZY}x!_g-_ z%Pf4y{S2k`oli~InOv-A>!=)&Q)>pD`69(DV-$iwIErW3;c0s#F= z+fu7}{o(Gmk4x!D-Q!(Q13dU{&8^mT0mpRnH~Hqp~QBk)|SuJ`j^Ua zx$n!QP|k+^$0Uc`L~4Us+xPxg&z383ZjkSo$=b2Y2;<-c9r zRGG?==y@-+Z0E&?xba*O`URT3jqB-XI%%E0USDSh&uv`0G?kVOa5^{@s!mzYzWVq$ z&tK+HV5?!+DFruRSU{Dz%B$lqZ(HD!t=e4hgg;~JMy1M)bpvwwIS+9C^a$<1HLIV6 z=2%ir0e2F5Hgkt9K=gUl59TG^d}4%&E}^o2@atyn7$IQW6qMT%CX z%>PLhC$h00A<|@>B1!CfWwRpQMcrZXbV6vuj}+(@MW!pPp&$$fMrdkYhkHn#C)tuC z#egNrSF%=?7p);W7g;W7y%i9SN&PhkvK>rkPulQU_}>zLc_)9BK#I;GFdYEx5PQ4J zjiE)mer)S9*c&MU-pxObWsJg2q7}$Q!VwQzuM7+y>%p;YolNXk%W_v2R1z~VWH3fh zj0hiEq>6K+Z!f1ZzH?)Y%K1vad{qn=+_0gT7F)je|8ezBQJQqYx^CIFZQHhOn_V`) zE_B(pZQHiHY}@AUwf?g&_8B8DB4_4!^CoAGnK2`t7fNB{WXlH?L*~cIB4^{ouV;Jb z;jYIgRh-t|K96cIsAYG+9M%osmR3M_fF@3Vz*fvy@Q1Y=m>xy-6_~u-3R%CLJTS0I zM-^o+ia@c*1=WTOBc@{f`&DS~cR}b=SO{TxSP9vaBVEl=W>Fm-b5jNjiYBWyo@96t zI@1c|+P9~Ov_uHHMK+G&F_jKU0$!bGx13ru^)HSx0W0OuRkl5B{Yff-9N$_kOOOnS zd3YuFR|N;&{FM#`b_@c|e0~$Wr2No_^S*VLAA>ZaX=^DPdonxi$R2Q1O+~o05?5U$ zRSmMv@vP*-#fdJkB*pM9JdmjksE;Hca->948auM$#aWgVO);72vlN|0p1kBeQ`KV# z+cx>M0@L%}SX|(NrHTeXd{J#0=S-D-3rF>x1yr@c6ZISjWP)98QJFqxK|3gPVEsc) z9nlo+tO{AX2t!S=(AfgB`7nTq5q zc}3+q1O_MYU_G$^r|S~R*c1Fv8$Lv2L5VM@%dF~Jubq&~Zy2lX{Vd`I6MvbeaJ4+> zW)9+dcx4gX{jCPTZ?je+Sbmm>H8BGTeCJrA(KvZD^TSa%#yb{5F%05WVj{`V=u~sE zF@F2G09;b@24r%BLt-mPg!K@y+{=eCdre1~Rmw4Ys1tKv)NNrK!>;xW0- z&7i|%{uqRkkGfXq=y3)l5(!_$XQFhqC-d9YWqKAq zN*raQ;Rg-@a9G}1*(DO>KO-VhiI`T_%mwC=i`PhJ_6|sE+1q(cqAgI)VIq^m${m=l ze^9a~Zj#NGOw~pa#p~}niF=Jghv`E-`zi46?f=aEo*f~C{v}?m3-2gUSl|zA32xs* zP#8h5f7w0*SPQ!PwO_gN-sx!Ex3l6D4Z|5Z-kcLmFvbN_>$mL;|J{BvK%B4&>||o6k&1J|{_( zuSuLw6E84+*TyKP)aNI-p9tP6Z}F#3d&wI^dd(!#{zkjsU^-G|F1p%4ylSk&^F0cW zhkExMux1-4Zvxv-SZad0Rvz$B_S7~0)3x>kcSo2idcb)*T5J>vf)}e3P&2aY>4KC( zsjNkS5o5a_Ma;Kd1*%e&YPFnB*$*T-szqrZKo{qY;nS|dMNF(AWC1J)W8R9TAB?#C zheDU#+N5wb0f-Vvh?qg=olb@gD8}zk#{NNRLrEj7B{RZibH^ul1<@6ZpE-K6Jf?Xi@u)O zUe2a#wPZ4|_p=smQJOF2S7X*z#`!;BejjNx)w{Q=44P2fc*6|3b@$;Ow ze==Eg%?z4Cq|c+xr@)LQ1j1Sl64o?qV(c>25*$uL1IyQ4r$lu34ArP~uxa=G^{&nA zf>m3cg^K_SFZRsjjx;fUP%&b!|Ab%#Fc7@aC)c=0JmXp@P7n&fm8%_#;G98(n~a`> zn?ZV%Frk%Jp+3;^np2OjC!C#0lXA`+Qp5n>(TsB9pxylyGcZp#+Fbj4o$>ddG(-o* z1FVRZoOgq)k!S{q6)|%Aq9OANw&XK!{_8ooBz~qu&PK>R)Kl!d-RK}jiFh|$Ky>g5 zc~Sw34}U{4)-&AiZX~15yAPPXCJE)3G4=T>NH#l?X^o@10y-wpt|=%-%=Ugyin8@R zZra$B#*+!uU7Nf^c1>9ZC-woofXaaO+T)7Rwly~0t0SAuXCiF7oN5?H1IPP}k3;rj zq~5dj4;A4`hLo(`{ZOs@y6tEMkMmml>`9%RIKLjzUE63py#s{=y z6T!VgmMeLfmaBMc^etJ^WtQj=YF#T{VnEjulUrwK(qL%QP(I@8$e#mAim8{2G{1tH zL*nKX=ZX4YJXav+Jb+o7io9d7FR5K~cy@bA_6|`{N}s#|(85^(+~L*ywtJ}sI9krK zs9Wu_+OA`avCc3U1Fl<{@1~n|EnNk@Kgc_NOUL-+m3dlTW}(;OKD~1|^{=c@v zEh`m6Qw8oOx8pYkdLxo@sAPJ(4$|>=9{m!nGgtVoriCs&{jbOs7pHvOHhR8qW~Msr zT%!-R1o3(Fe(F}*ogHp!as{-R-A@?*;b{L1msb=5FneV1%0G9nQS4>}A9Cf?vh}>* zlDqYFd+;7U%?@&T@gr-qa&7T-zL#us1wLL*K4y6VH)|f|vv=}_+wUV`N7)BGe?MY! z-q$eWv(4|;mS+_{t_9h@DLMts6$yPWT1RvJ%@@~$d;}cc9{ZNRL%)?C{}$TLMqSbI zE%bZ=(p4XN>DSd4Eb~(;l=%{0J?yUCSgmsg-AP7YL~!FZf$7PdePii}x}>f_eHkXl z*7~{6a0%kRc>+9*4+~9MYJ5jjS^T;Lg<6RfD$h^*bPzC*&-0I$lK z6PKu_)0s>htPQcR^?Iy^k=V_NY9gcEdU&i=+6ziEsa)*sZ(F6m-5{2c#brTPi6_n- zYHiDH2UsLx5ov?C{eFMD4&m&PfFsKe*#jw7V$)Z^DF?jkE{R!aNw0W_Jxck7vJ{Xl zE!8C~UHogfapKHYRV)D6^I-+#0)nE2AA(ePn-k<%|P)&Jh`By5l1f0$|1?% z9G6fcJtpn~X?4v8O>Di`w4vY9xi!7L_G__ZtFS3o5ipxuw`2K;1H0;KDo}fze&Tn1 z;TKBJ>t3_mguan@)?W!3bRJwHu4tX{JP)Yq;LNDBqzqD7i^b?B%G^<98j&%tE$-me zxo>Ru_U|gbu75WpXNiZsKbV(c_r($RTZoK0n&=q~kA<>S57KG5PVUaSY4(0{u2IQ4 zx_JO~AknReQnv#~wWhwk|Ce;OJ-a&csdx%BT)#O>onlZ2sXhJMD9ofuqIJOV{-uo!ho7FnxojSp9C zzb{%*&W5u|?0o4SAf7(yygH{fRc|(0BvVEaN;2@y+f6Z)@KDN;7?+2NPtcI@Zy?>8lEsHCJEM(FY z4YY{J^5MY1zZD#riBz_up;?cKU>OMbAMEDnuQFPX$Z7;yfP~ZqxyoDw67e#fpaG;` zb=IWMe0f)H_C2kg)|xMZ&N^<-No8`q3bWj;@-k?~BMR1&7 z42z5<78pp!>=qe<3&cFno_tintI)DaMGIz^vpwP1h|5u59wBNrCyzEwAyKrE;dG?Z z*Al7!+Jg;VbaM5w5-)tKuX4G?AO%!rlve3icE59PPU-%AJEu+sk{MvrfL{HTiYIyd zR}sg}n?OQB_DH=vE{FCBSHdX|BLj4$#J>%yj`g4-wU<|XnuBjoj3a$EO5#~W7^|)n z-+a<4^8$nm5BIG>tgjv`5n9 z3Qc;}D(yl8`#_2@F?0ZVU^!B-y$X1%sT-mutf_zlxm!atKx&DfJaFRDG7Yg?Vi9*M{8Rc*m>Q$-=>4XN|m+1I4hl{{4cZM`J0Y2i)&u`K^LI z!qR@nRWYz5D;BWRf;0Q~q)?wvABxArv&K0&&&qzOl;!d|liPe|XyyE2G~Zc53Fb4y zVkgXHa{&hsTK656yilD|Q=CCyJ1Sgj4Qjg<(oC0PmXU3ive>z-AQS3l-EwU(?z*noryCG;U$Tifw295|d3`D?Tgbp|jh3 z6|oSBNpSKQN_b%&sV0{FxpXglQJl(J{Gi;(qMc?au-X>XcmLIJL>qPr?`!L$Ba51n(jC#Z#N&{t&M*#NMA68= zWC@EHL5MSS>KfI0HJO72OOx&MO{4nIw(LblMK2pRu++Io#OVZ$UYIf!i z(M`JOO|Q9uVv5b0V_he1N-4!KyzMy54?19hTL+jSn?Y~sp2u*HWjhI*DLP&7q71`B z;$}WuZP*B)-Z0T$SoYFa1~vv-TJqD4>9s zYzV##@u3C_M$878=9@F9I`j3{lFNk;1Fy5`k5U&}6j8fq$u*FIX@C$rf0jA4d-C!1i}QIAh4N_Se*Vmd=<4i$J9`*_J6tu0szY zd1*}+BzTCk)nh%%3cF(Sxl_@=Y3q&R^aBF?J2k6}(m@wCjVAvh*oo_JUqu{QA=P(D z5@6mXnxxMhpz#MY&ydu3;tix^3p=LH0-JCbn}N4`B0t&%Mq+a6Wo^(h7*mQOGW6TV zPT{qbc3M!|a;?Iq2t&0mp?4Rxw@T)q`Hc?CWWPw`9+rjD?Pk ze6ZVz_qo#lC10_QcyB=ER(^E`=KU zh`tlM`q$qjhYiIDJD#12vN(W#Xg^YD7<03JPnG5`%j3G4)_K>whOO<*OkSjnO6ty) zIEAjux|n~M^I>V$QRxkZgYi862Xs+Y`js~%D#^Lp9s8{0g{d{?aMB+sW5d0Oi?ODE zM>Zw+<8_hC%sys-DORw}4dCyao7w7<5@*O2 zt`vSYrzr><;Lgoo#@LrS>*#-@StfU*2Z9=!+op~Z50 zJL;*dDBFfOWvg)LVLmae!^}yIiT)-0F=3S1~|dcLP8U6 z$b71KqzzUQzm@so)fq*q6BX%buu{e+s%rqgKp9xQ;kXmCKpHp3rw4GWQwN*0U4U;RCt36JjX4O?A_Hv??B}1|6XWJ=_&U zlbbT%JK?2nMB0PX^IGgbc2C-`UzB~k|#5qKyAmB-f*9Xv(1I4vPF8$dV~IQ@!4bGG#4`~T${g% zknjG`<~5I7C<+03Ou*&bOEPim#Q{Y%$d+w4m#g?GRO`u9tQct?suzuLQudS_q2o+} ziI|h004V3eH;8Z~BV%Ks>(ZPV5B4zicg$(iJVBf;-MJysgUq{B@;S(`U!-$hi40vM zYIMpFh%%YJeXCOvrOqqTJa7ZXAF^etMkyN&Jv+bG5A#Cu@+oq1I~&!heC`dc4y}8# zD%^{Nbi62{n7IA3*S(PEL;g$%y5$kqHflN30D$Wy!aIn9ST+aUhidRBYM+W#N+&#v zmv+L^+%0wU_w44{hvNxH%PH?yGr^kO>)#V2nEf;o*n5d2{r>GqQ4dU;B4X6@}6RgN8fJnv| zveIrZQd&+_rm&od=;W672+X`2VTv|x3UDp!a@E88`zE_gO~0kVtkn_yuT)?4)wm7C z;`Psz!sykAbStp!0ZTM4G+Vl5*;#DVJ<+ef)+z)k-02K`K(vkpio0B|!CD zU#*WB=Eo`c=c7zd>{A`<+kNoX;I73^AVRC>9OREt*OtqYuSS=J0HpzEE~Y~oV;JW& z=CCf$Xeqm)jOvXW5CX6RP#Eeh#cy-4E^s4mV24`No4PZvx7EikU&)Nv#fMO)BXYHY zP7z!f+2yQ>Rcp#YO{GeITcIvUP{59D4@GZitiPxHmD`kZPp@S4r+`>DoHFHdWUdhu zS2*xpy-Pc(z;%4ht58I&>o@R-In04GR<7fl!JX5DZ}C?3xN!wo`b|nW*S_~3S^?r0 z!F=4g@x*=6?A)f1rn$Dl^y?DyZuxYoa%DCR;fBhdfRY=)N{CTJ^Uf_~%NV;`t}0ryqVTAgH<* z{1al=kfY@rq1J(972q&z0Pj@jmzI`e~GD>!t~`)9C{`C0f0d+rA0>xM)^GtR!F63aN!5N%tn>=NFs@fEM> zCg52RV(6qSb9S)9Nr^Ljgp<5RxVlEG@WC+P*3a`{*!wQ?wy#a?6R^%PxN=I}mTJNC**yaz`1>E8W{KM2h{hBX~Dlzt|KsbyQH{4B{Tv93|z zzrxr<2K)mJOxU;lW3!3bpGguew{`f9yFqC30kGJA^2rkjoLIaVCM;1GnZ%!_QRLM9 z3iZG9sk6!8z+3W|2LP({*_8(qpFlUG#C7SQcRZ+})r^dOgi(dqYRN5mL)LSJjJ zJ=9}RZNt`KQ1kUiqJOdR$se$dYnSYzYhfCT*D>^;tGDP!{4KT8FsW9#58t+HX#BzR zfvbAb0+naOO@>c3wNtunt>d81psgzkw60e6mR68iYitPj02D_Q*H|sU%oypHds4S( zQv0-+>Q~>c7+qNfh9=1syt%6~)YvV!RTfZYY@i8jpb-`j+U-wM`;3itHT3>IqW)~+ zU-Y7;=0*#J2ESOB%Z{7U`R52C|E<8bcjM4-5>T?by#q$5#&onFS77L&@C(tFwcWoU z+(V>a?4^h!5wKI5dC6VyeAZn$o+^dT#D1Jyr&97@kzA&B!lqgmmOHn=}z+=4S=EWKCi5fu%KVY09TzQOa7`! zkbRYHP8aCJx?uU^(UdNrTVV9ATQ&X2`S*DEhxQ+9^nYBQ)u5GBC9(X;;^L_C@PsQp z(IkfYk*-CH^#X~&hLOyxhJ7(Fdy8S3{W$Ifs%+bbdOo)?`QVK3x6WyZ}0HW-XpY1!CTSCV|R;T9`_KR{Pk~XMyX2-`9{1 zjX=_Xu==F}_{i1%+yQmt99jTP*Kq{;;;34Ir!~g`fM$hGpVEoQoi~lu$`%~Jc)N=X4XYGZ-}f5?!}zq&S~eZDSJU{ z=tp(!Wn~qiJLn}l+G*H)euim+VPx8~StR(5ojls@aie>%dw-c3Mo7_?94X_EV6TP6 zX75ah%nT#i`sGDAX~2@uOl$6Gpr`2a4KSNGRi0Vjo3#1HF|JAdsf0UJIBzH4w6UqL zIek*nl$V>yAGn?(b$F;c_dD=6SE%@4FgEdaYB7wXm|}CZbU9}xuAfxyWHC}g?XGvz z^6nQ*IKsT)oUYK4Q^fM<#XlyaexjduYy7l*mHkL{EtUtbXw=+Lm(Q)q0NgxL0C=mP z<^v1cb6V_*U0PugRB(P`KF&Al!(=<6MZGL*#_C=@b9xp(Y9&*PNg40YYyw_$H3Eos zL>!#lPMmeO^fdQ{iV+{|*mp8G(T!7Y__2MGWpB>)Eh`FOv(Cw{K@=NIR>K_$+Lo|E823({G=t0wy>e+L3dP1U$FqY1oE{-PPJ=CO3V58ZogGJrwWQ zzd4$DVlvfBHMD(~O@PT4%d=MsM?)X7`EX*mnxPE+Jy$&87q9HtqPPywo=&K;)E}`Mde~ zD}t_;IROGD#M?_()5n)h6fcH6Pbj|<;;OEBZ@ls{VNa!X?-WSVPN`umgICHb6_l=rsXc*9%Db#<5x&BIH4k268MDUL{Suq5adO5|*?l}^I+{-yCmMg(NO~u>zzP)i z2k2xk6YFbA5RU(VZV3*Ut>1T-^@r&ZLR66H5lGaK`4MQ)!{K!q0Ke0^m&xHfYSWf_ zc}F%iZXlBvvUDu+%_YR~X%E~-kY)*V8G^Imu&`&4pgO>Z{`)r$oDeWSJ@&FxZW0(x zGFk5@FLt}E2r}^!8%U~7}r(gSK4 z>KJ4)VN7C+GLZmU09ddL118OaG%&m-wYZ8ni%ArP6mtS|0tKlnbqbYgxDxV!k}-*E zD4M7^3QbwK7!)fN2Oo=AA8f47zjR`olO8`;PR%ulreX~4=t<&O5@DA#(hPjGf&`XH zxpO$~M)veuFV^%1P-GL9W-L*RJgD5F9cZInPT3Z%oIfY1M*k!^gure6;N5D*#`5D?CPuxMjvMspKud&Wcu8i>?w zSP&Ed5wvh17);Vpt8vi1YzbIK0{KKexJn7t8##9pI;Fo75p+b{D~r&C%E9yI9s8Zz zKLqZ=i1on;YH53ni}V)9mE4Upt9@qWVu}5tY+`yQSMwIdO1mWQwO7LS(#+NutO#0G zQEB=S!$0WG4dA*L2}1H30cmJIM2fA8QjY;}3_Xxe4z*8JSSc;8ICX9lG9ai>SaoVH z>6F2`P<4*lh{oeOE6l1 zxDi~YRGJI)=Ith$WOda=2=F=yeF?_ne#gSiJoJeQd!RZEy0*z-n)xI_9c4Fp_N+7$ zBJ$Kicv*7uq)2kkZ|ZVuO@nDl^{sNUNbmFom~@h4brpl@f2Bax9wX z1XkgtwT7BBxF8N1=_w|>t1CW)IO;rk3?S@IRE2sk9Egurbw9tW$=w zcp9eMfgpmhr=W3bzZz=lm?c_jh%CvOb{-krEq$@G1{Jes3Dnp$`w_cJP`=qxaE9uG zP5xzz1&z0n=Tx%VLWIB?*m99wo;=sY@F7}~DM;x?`MJ09ONI*jYXw!d8US5yKw6VV zms$?DDuFEzp(Kn{-@FY8f4f{UP3rsdt?%!OCI69 zf|n9HeZPK5ysIBJv)lgqsRHBjn!O8+D?Q8`pXz*Y+1s}PL1R1axv<&_P+8@{O0TSv zKw74~FP%_;41b0(@x4O#kH$;vbH~0yw?cgTSJ>fnxCRk3 z*y6}SK0yYEAl*dS(z_}05-YF>q-6ZPDG|`MN(8m5&%V5>67U?dE#~K-VTvxCwfap_ zBsP?z1k%}YyjVo!MZt!SLDcPz5#SY8&#*^j+4?uHuK8&98DZAk(V}r^9 zm-@>_k5Esr1ZgUad9g>jDibeqMHlL7FA|}GM1h=NhIv;CCK3}ZQf7rH2KOQC!m?;W zQ_P>2Y!A^498>gW6VT4>9iDy$oux`)MLs!&Mpk<4{Z1X; zho}&v4xR8E2uPp@k#1#^5m_E(XdoGT3}ac0%%*k9aKPP=S%@t0Z2+CJVSgjueG>vs z1JjwFK=C7hp_;_lhjBkx5_Y>)CX97-`;9qi1{v$292O!bL)MM;YZZor&U)7og3`Q< zM<*{e=tef68D490eeEc0ZCeP6Wihs~73@+F=$Y|fx`)V!R!9NjEt%LI{m>YF1~e4A z(g>;#4s^^$j~I!!?pIUB9lFuX#Cm^P7SM1h7XsE~rC4d-HOLY>P#q}HkObZ@!?342 zKc-+P#XP84FJQEob~Vtz&=?)GDahKRJC2sZ)X6u1t$ydntOslq^*o=n6d#{AruTRM*xu{LD*SGY`x4w z)l(Lom{+K{0JGw@Bjj7W*Zm0ZnB~d%l}8-oR3fJnWl`BIa#VzH=6U49FF`OA$R^UE zcVc;evihN+PiquXDJY4cW#m0#tooI^H!X3|X(6EsUgHOXRXTPgykJ^1lc3+oO@!@0 zvzK<3oRL#dNq@XbK>kW9^euzzv-9p-D2xaZXhn@$CFHa=* zkRbMu5X3x*w`ZOq)Bl+=3VJROzBn(O=KVJ5S^bTjbk!%6C8lOQ6()T-8lMn46Yq+} zxdnh4s)D>1B;qXW#-mSoM34*c6~DXtHKLgqTLc-v1snkB5~3VL_eXgyRF~4#98qkA z_CbHHsAFS|cnB{(pA*t?@q8O-*5DJXhF?YfhbXm_41sqHS`W=opiLvhMvl^vD0xu- zlE~hcSGX4>xxF%VD-wr{I0^`})t)dbDj84*YuJAYwb_3uMXN`9X`j6%y3i@l*H|5J z=To6r3KlcqH&f!_fZ2&!i5;KL6^J?yu~3-D0!{PM4%YIg=q6+oEts6ZWOQCX4BsIc z++EJ(CGm09H!uL48|Q>Pi`4UX0}dv6nc(kjL=jJd(NWN2A0S3}Pe|`IA7s6^I8#8| zky={_OWDwx?gemX@r!%?%kI@n|Fd-6wNV&9n7aE|m30{t0>U1#LDe~*S~nd+&@T)= z9uyNq(|;Rx6DXNTtI!DwLL0%+N-0W8QelWPCUjW1VZ((av8@WLuR$t&7S4rEiKReQ z5ENuSYiNO$|594F#}J`*c7Okz%Txg*2w=c#aDC-B^kv-YVMb&@4G>5Qo*T6Lg2IIA z`(S}2YZJBLOErRCXgNUFvdPZidTMqRAVDea=x6r`%GaG5^?%_U*tG4J0(C((EV6(> z(xD<3p;z!jKBCM$Fzn?@bpRO*hZUwVh9*`tR9kQ)=YDx+v6(aPPcUa zdJMDUHyXceSqpc1Ui;i~7{>$rcB+-9FkZ~8n;)frM~kwK%n4QQdpxC^VV?&F{XtwZ z9hazPH(nz{f#uG9S!33s-t7iJ4#FRlHMd$_k#5!>>4u&Es66m6=Q^HJ8Z;f^ab!w4 zbDjrt+a4W;u_wkb_I#pEC8lf&Piic9J+*o&Xty~P=wt?eu;%{I$Vmfa2}WoBn9t2U z?OZNTw6*MjhNZ9l*)aI&rZOV-;6t(buX@Sv=XgHxhBJ0Q z9!C6ZI=!vID}Ew&d>tJ0@&ev`kIUz1cRIX(7JI)AY$ptJp+8Ua@7~k%9SCKYTr(X83ikyN^HLEG+>!Pusa^F*b|{t=5cXhI;N6R=KTnuh&p_k3ZjzC!9X^c6c&P z9pA3yPhYfzAura0xq^T*4~JU5hq@8JYgg0Twak;=x7T%jnWN$Tmz#CBCcEo~=Xa~m zZAn7FF{NhBrT6Qh*2v2H$G5(q->udIqCLL<<$Z{zw}G8r(gq;wq56H$Pq4%9t5M!i z`#tZ%W>Ed?CJ48e?=>^CmoMAy`_b^r+x>%ZcJqA`QZp7%_z~Xf&8RQfEsZOXwB>O` zSo1mE@A2)o*5i?$%jZR0oZRcH9xLd$|IaV-$^57KZcTls_lM(S&2YhUvay`?X!RF*8=~;00agFWyJp#QzhIjRP>>UNG~NxGNd>^l64>~Ck04WPK+^)S1*N@O zd$wLboZgS?FK!w+1w4;RS7-&dUr%=#?LD?XQ@J0A&{es1MCD;#gF6&Pdnh9&2Zq|&) zcN?)=0LyJB<0(@zuq(!IJ%4o$ayo`}TKk~1?%N1GbZj>pIdxdznJ~0nDfAdFm}&*B zC8#f?P_0I&(h$L8UNa7&F?CV|>WYSP*BuYmc}|v=`W%TtyEs&H2JWv-#y{6Xp|9Ec zGj`TdB*=3$AGzsvX*4FR<#Q>hKDDt`rR*{s0k(hCY7#UShXzK+=|Bv43&^9xl6GhkFEa0O>W_z zMo8(amQ=G=*#E9y3~sC}EG@)cENe*7Y-AX#`Ou6|HBP1&&Sxsw`#bBtIFvGTXHt`( z$sIj|FQa|7nLkt3ZUPPXb4iCIQTle&wK%7m4NLgStHvLS>&j)F zJ+%r4gb++ts#+{{76*j-f1}n<91stX|3VLZkU8N0uFDdFMEy6oK?ouX{@)-CIY{__ zQ7bt}-G7lS1xPBw|48;#_l0Nu`)RcWPVFZLAxLGQ1gZH?VV4p_`9IlKGg^>=|L+Mj z^dJHMNz~GV^!=|yo_hShFS_%C8& z0*U`G8fF6V_}{2ih#7i6rqz15oZ~@fh9TtU!eV} z!%GhQ9YOF;yyOfW-`q?&Gx1#2+)15R@pUy-gTFx^a6CX@;=}rIJl&P<^|o(h_t+m@ z1X`pklxV%SI=;OP8z!BOu955$zf`qFKU8Z{rXB;GR|AIL_>3$&5&uhXk6r^BZ>Gb6 z)?RLNJrPiq)e7353r%;Wd;znJhadRXFK9)X$TeC_23MmnD`2Q+D1Q6Qp6l5AbMW|i zNWxWa_b7}>5hYY%R_;oF={UZes3Qe)e1i9`w}jO%Su8U?TdS4Nl}lcdxorX&xi)l~O$rVkYi`|;r5y+4KVT-W*O46NAH-qT4+;~3+Y<^!Dt6GYu~ z5(VhBZN5fhO|9E2d{`ut*_l+?1aj7_cv!DQU#!*EjPg&qZ>sKBZkyAocvHJj!spA& zE)5M{1%W?cP=Yk7tfO?M%NMWLE5LP{3Apd>d8yG_r0FEXRG#M@{N?7y5mcOh&#Ycz z(X8x5E=`{D+H&L~nc~R>X2#=;en9ccW&so0&1{A5E5Ubf^=NHUpCvo+ zYI5B|%diwFg{zbscw=Kh6p8)D{lAkG>VGF`D;5_>9|ZFMOsVy>YrRikARy|0%w?)n zCt47!)GY@PR6w#?zkMMSQuhP8L6MZ5SopJ!b#v-hB)K?*qE@95=|;Fr0N@6WZ5Jt` zW0Q~dX8LuSwG-cGU9esT*3-gBrUnKEtP?+|)zaO$cf@FJeMI8Yp>L#?-aG}yn#%pM zkQ#18xlDY3ZAC^*?sq&UlHZVy!OEF*3KH{0RFw~58(?Z08`OMgphcal5Y3Np|ICu> z=p-G>2fVP(8$pZGQAx0`IyYClboH*L*=@6pjWK}s{L8Rf)0(wf-`j(}Hjl%538Vrp znCPh_OUn1Q;S z@zFhFtQFe5#$fbb1rxHNu=SNV_X3BR7`u-k4iLbX+BAOKv=FH-o%z@m!4R`xO5R_@E2rWtX@^oGB)};r(@E7ekMxihlA@ z4>JTdtpE6x(nYhNb|Dj$okIdN$4r>+brQ&)S&rnucwv=x!=zJ<`TeiGioUIPY)IEP zFxLP3!t~0H8JfZa0ku&5&xaQmVDDh|56$mpZ*T6rscq-7(T4mzV^|-bJmyY6?tY`H zLcY8vb6NhRBCX)X07}S{I@+Jvk}J)~%mSl(hM(=}=dE2| zsoBAhwS~<1kPlmPbGhQVyT4Mr(c9Vc-y9;J*8@iYTyzp36I1sBVCzo-pI|1@Kewv` z?9)&ngcJdzB$`0qCy>lcSn!mX07zb~KXmT3{0JuF{5L=~iCF95VTWtFe3ua^M-01~ zX_NxGPF4}zu6fMTpL<2X@(|7Mg=QR?SuYHXxwf|zpv_z|p77sy2>awO^nJ0yi^?0$ z={upqXqlbPUXUfgVPr{w%;+Z;BidHrDW@doa6fn_pt27Y<{O1s-m{qvo-aDbS%6EY zmBAMhJXg6kY$~6HPnP)Z2oZ=-;R>GK4@27#QXW1s6tDB37G9h>A`-a(6Y$uujBkIW zgw{c2e;#L!yuv7E;o0S-$!7V@I>HhmqG9Vx?%p&m)CvNdq>ngY0s-an^0GCT{-Vjp zEBxaA^mI~tlzvhG$kuhOCwB;#7#cT%=FNpnipa(n#C%Rb=PNPUn*z{H{?%<~ORgsY zUN69ftd+d!f-vvQg^LbIV}IBHLMgE{`!{Saa$-&}ve*uLzUa`@LmTLn$1brH$*I16 zR=$^*&eHkmSLQ+19e<#l_3yDsakqoC+JUssKopXgOXOa_y7e6kOhRUHLx~r{w3( z%zW_9I6Gei^V@4=Uxr-g!C4h&&*hX|$_JHbk^uSo!XtKUSIVFXN2~Y2nc1@!i_oei zp8@DVH!lvruIA=)kd@ly&;CWJ;mo>6B}Sw@m_BERffC$goI$5leE3q9icL{CRkOEL z5OX3!rR(Zp*fgKm;_(U7> zuuPhs;CK+7iW3b59%%Gf;wC8lj6>&F>9w+GJD+}lJh9WRgX3n{h7%J1=|`hRYa7@! zlW3epqJd8dH#I$h*d)T|A0JVL!8uhqI5@KR80IeaVyEeH6Yk6cLdDRaSj=3NN!qdp zSikXXJ5)`KHxh%R%u2JAZN2OGfuk#-8v@!;{7Rg_)JXKX(Gsx|NG-8`nQt{ALf4OO z?0=2!tS6kvi1~N3weoiXRJclaG1IZwDfqwk6pB)oJj=cw_RMIw zW0+t;2uiO$%XBPS@Wl}Rrt0}r?tC4cY+oMi%s^l#&iG=7rRg!X-m@3GgSWTjz124x zH>v02np6dfPdAk75Ph>G+iI~KMqlM#-k4ie@0YQ&{@WsOZP;5TNyT*LV&U%cDRtcd zEKM~n){z$`Kq09>HJAz*lJHnoNL#Sd-VSXT0 z3YS}I2_uS9^<*fkoLr2x>4S5ps!<9Az+~N9(0!ej{5qiB=E9t6nY#A6RvLXTT`}<& z5+BMp3#K=&lPK(7h3BUcr@gFn(tlPeBA||tV|sW6X+&+&+75~*G%I04F3>tGP&h}A zE3-y8Hykf07hB={-G1chrqCL+yL_~K=J>h-n?3k|7DKLT?=ZXn)=oHk0Ovdd*qPn- zO8l))KpeJ^|Ll6WO=oI@N0-s*s@`I-pq&N}opZ7ODy>hodn+|6h1C4%43~UWZu2Pf zfWgr_?Y-W+a4Nk?!KWaS(Wi+r7$A40yjw0K280ba>%F*#baLc9(mm~lYEOskgtt51 zm6v8e`-d0@s>OM!Hi52K#tKdRUwVcuwI--g2v|mC4&qoQK~WhlCMj~L)s*x0PZo07 z_n1*JRjIbHgj@%WUA}D63z}(jgUDv){Pr~gpA6QL>J89Uzv2DfjQ28ztYVBKUguq) z?QuW;;NAWDpcqy@F9~6c)#{OQu$=&jA4Z452tlmdaXZ}Rh`wq>1!%W4p!A#A;d0J4 zaoxjmJYdH45vGuGSJO*E=1B&Z{tIIt%=tb^`_ITAq>a5E;Q>&x4OC(Mr$plicFee< zq6+?Jz`GFw56zh>wJak64PnlNH}PU6t}(=@lDtM*l^LQ~qeIfE?BC)Ks>)SoZyy!b zO*f<=uXs8E=vyim$On=cKo(?_KQ#Omeqoye&jTFfgEM#b*gl%ccv+3?FsxK@GGl%L z85Vn1nI~`v;A^D!+BO6*{K zT!8Ea|0QJfU<))q;~JQuCWA7$o`D$B;Vxlxof^}G$ENm$3RCr_fdrim$K3)G)}zdk|on^+h$CE zy6$oR*v+^jULsDybF^o>rw!`+y!ZNC{U$*x>o0@iL<_6Ai}@v6gPOyC7%G4HLtl-T zo%q7l7Fi~>y8eS!W|CAy(5YWz8~e^1$d4|>Z9Lyo&;Iwxmc@^D*)pc_O=PRkF#WZs z3x%SpTwMO12%a*HNV*lZvNG!WWY0j@T#l~-tJL(B_=$o78Cs&|mXfDN3qgOdO?*R4 z9eit2K93J<&)r)zBRoTGI9PJAGjY~tE_|n`+Qf_rW5QBZr5P=#!LRkj#Sb3NE}hnt z#tFS0m0E1sd!l~MFU=|0J2c3D9u+U>)eNb2zcVsgsVh6tV_7eixvDJfjWc;lYy8eo z?kr9G(xBsm@8d~M#bw?n#Li_F=Rd<%kN-^0FsxYk7dg%`L!d|5Lr*CfG zPqt>JnI#F{v6tGaQE4OI>vn>|Djai7%_#l*Y-o}SJ4#ho@QGD?%D7X6T&muUhmTw5 zTO!Ohja;6Ov7kxL^@}R2-3<-!>@NHBwc=df>7OfZ6dIqd3%QZ3b?vX5D?P)zoKLRL zdCioNOfqWHnbJ&d|LLJnm%Y1(wm`|q^bfU4eiFx%jjV>#KA#gjf8UYKc<0$J+KF{ScV-tA`@{3!VAw%_9|`)^!pI&xp~&UgA0qrARgo7*(SZEZ447yG8_ z8GSvlp`J z3a=g%v2c~u{FI!!H@I6l_har5MQwFP4_Pm$&Ai?szq4d)V=bl5)h$)<`Mb0&-k#LK z^cwN$mgvVr!A~RHPgGVGp=hs8s~Pm!q8^zwW3*-M-J)Q_2&=e zYnk};CLCGu^Y2ATRKw|rQO0cFGeiELY1lef^p04dl5;~8?t+-EJ4)bwEZBu&1POpN z1G#Y5~SnEpnLq^1>+kR-HBxkKa{zLw`a#-q0#O8sd#vvKKxX*Hj zf07oclSpzfP)Q_MOTaH(%*7vRs&hcUJ!qH`f%OFmsOhAMGnYV>irL0itFTgkq=q9+ zQvNaM4JaD?*5r(Pp12ux9695U6mjHr+tI#2sLW&}OCl-rAv1Mxq}^^r#gXD^?#u-t z5-A$wuH{4OnB&OPyOAo6T;n|#YYdu|p(p&vFd87Wu#8Y7fg?Re=#k^#iJ287yyQC1 z&OaP+nZ^NVHI7sgr{ztGlSm)ok>31v3Vw?t4+n7V^xyft7#Sc;L*ir(Po{}C2boRr z5yjdAxE7q;T7p;Dr~TuVMZn24jhj|uC$Itw6K5_pJ2totyG7aB4k0(Z77 zSD}6kZ}c7$#VP`kgeWI$13~O#H?aGGNDd8_jm=DG9IU~_0JIQ~7Ds?FD#cdt2cGhFEHq9?ss zqL3VLe2G6fE)C-Pvk%Woe%0r6y3eT>Ak=z0{MVUJcH|gg_5NTahbt(NytNPj3YtNI zIv;eOJSH8AR$-07NC}5#|ENCj63$OW8!Qj2v^oKm2;s){QhMJ*DagMx7y%l5QdKPp zsAmXL!KJQMITC*WD%1o56ygfkQc$C<323{cfCyF@!c79QfLG88NI?>`QRTxc+7b8< zAxHs_R+x>MR24?v1JUw!Ab8i^1ip4x$r+}=9`FG?IcZra>^z8O>3h%`T;%YCzjh~f z9YhLzz*u`A{!rGM$ohz)j^dl(ZC5xW%s-M=L~^7Uj0JfM2VFXw0^0R|HGrQ&qd~5`gv*64fxHRq?bPTSg9=#Al zBxZc%e)#HInw+>hM|BZM(Ryl$y)QI|86C7QG~xe%#4L6^m)Dj^q=@n)F(0V^u!duq?1EmQJJ@p^Uh44TS;ivctrf6_NbWS&T>i~j zDwvBK^u5uiT0_GltWC?Nc4pAj64AyjpEKnyzQ>#rs`8UP$kT89r}8fzn*z{Efw6|3O-Wj)39JI7i zHVet)E=wN#SI|mu*?aifftwY$AR+Lpcl#H-HV)1H^TIhl;GP5T^56cL?jWGL@k^n! fjt7k`_5@kY53FIwBM~Wkl08Wk?#SF7Kqcva3+T`K diff --git a/rebar.config b/rebar.config index adbd1fb66..f65b53618 100644 --- a/rebar.config +++ b/rebar.config @@ -18,7 +18,8 @@ {riakc, ".*", {git, "git://github.com/basho/riak-erlang-client", {branch, "master"}}}, {riakhttpc, ".*", {git, "git://github.com/basho/riak-erlang-http-client", {branch, "master"}}}, {kvc, "1.3.0", {git, "https://github.com/etrepum/kvc", {tag, "v1.3.0"}}}, - {druuid, ".*", {git, "git://github.com/kellymclaughlin/druuid.git", {tag, "0.2"}}} + {druuid, ".*", {git, "git://github.com/kellymclaughlin/druuid.git", {tag, "0.2"}}}, + {riak_cli, ".*", {git, "git@github.com:basho/riak_cli", {branch, "master"}}} ]}. {escript_incl_apps, [goldrush, lager, getopt, riakhttpc, riakc, ibrowse, mochiweb, kvc]}. diff --git a/src/node_manager.erl b/src/node_manager.erl new file mode 100644 index 000000000..fa150d610 --- /dev/null +++ b/src/node_manager.erl @@ -0,0 +1,230 @@ +-module(node_manager). + +-behavior(gen_server). + +%% API +-export([start_link/3, + reserve_nodes/3, + deploy_nodes/5, + upgrade_nodes/5, + return_nodes/1, + status/0, + stop/0]). + +%% gen_server callbacks +-export([init/1, + handle_call/3, + handle_cast/2, + handle_info/2, + terminate/2, + code_change/3]). + +-record(state, {nodes :: [string()], + node_map :: [{string(), node()}], + nodes_available :: [string()], + nodes_deployed=[] :: [string()], + deployed_versions=[] :: [{string(), string()}], + version_map :: [{string(), [string()]}]}). + +%%%=================================================================== +%%% API +%%%=================================================================== + +start_link(Nodes, NodeMap, VersionMap) -> + Args = [Nodes, NodeMap, VersionMap], + gen_server:start_link({local, ?MODULE}, ?MODULE, Args, []). + +-spec reserve_nodes(pos_integer(), [string()], function()) -> ok. +reserve_nodes(NodeCount, Versions, NotifyFun) -> + gen_server:cast(?MODULE, {reserve_nodes, NodeCount, Versions, NotifyFun}). + +-spec deploy_nodes([string()], string(), term(), list(atom()), function()) -> ok. +deploy_nodes(Nodes, Version, Config, Services, NotifyFun) -> + gen_server:cast(?MODULE, {deploy_nodes, Nodes, Version, Config, Services, NotifyFun}). + +-spec upgrade_nodes([string()], string(), string(), term(), function()) -> ok. +upgrade_nodes(Nodes, CurrentVersion, NewVersion, Config, NotifyFun) -> + gen_server:cast(?MODULE, {deploy_nodes, Nodes, CurrentVersion, NewVersion, Config, NotifyFun}). + +-spec return_nodes([string()]) -> ok. +return_nodes(Nodes) -> + gen_server:cast(?MODULE, {return_nodes, Nodes}). + +-spec status() -> [{atom(), list()}]. +status() -> + gen_server:call(?MODULE, status, infinity). + +-spec stop() -> ok. +stop() -> + gen_server:call(?MODULE, stop, infinity). + +%%%=================================================================== +%%% gen_server callbacks +%%%=================================================================== + +init([Nodes, NodeMap, VersionMap]) -> + SortedNodes = lists:sort(Nodes), + {ok, #state{nodes=SortedNodes, + node_map=NodeMap, + nodes_available=SortedNodes, + version_map=VersionMap}}. + +handle_call(status, _From, State) -> + Status = [{nodes, State#state.nodes}, + {nodes_available, State#state.nodes_available}, + {version_map, State#state.version_map}], + {reply, Status, State}; +handle_call(stop, _From, State) -> + {stop, normal, ok, State}. + +handle_cast({reserve_nodes, Count, Versions, NotifyFun}, State) -> + {Result, UpdState} = + reserve(Count, Versions, State), + NotifyFun({nodes, Result, State#state.node_map}), + {noreply, UpdState}; +handle_cast({deploy_nodes, Nodes, Version, Config, Services, NotifyFun}, State) -> + Result = deploy(Nodes, State#state.node_map, Version, Config, Services), + DeployedVersions = State#state.deployed_versions, + UpdDeployedVersions = update_deployed_versions(deploy, Nodes, Version, DeployedVersions), + NotifyFun({nodes_deployed, Result}), + {noreply, State#state{deployed_versions=UpdDeployedVersions}}; +handle_cast({upgrade_nodes, Nodes, CurrentVersion, NewVersion, Config, NotifyFun}, State) -> + Result = upgrade(Nodes, CurrentVersion, NewVersion, Config), + DeployedVersions = State#state.deployed_versions, + UpdDeployedVersions = update_deployed_versions(upgrade, Nodes, NewVersion, DeployedVersions), + NotifyFun({nodes_upgraded, Result}), + {noreply, State#state{deployed_versions=UpdDeployedVersions}}; +handle_cast({return_nodes, Nodes}, State) -> + %% Stop nodes and clean data dirs so they are ready for next use. + NodesAvailable = State#state.nodes_available, + DeployedVersions = State#state.deployed_versions, + NodeMap = State#state.node_map, + stop_and_clean(Nodes, NodeMap, DeployedVersions, false), + NodesNowAvailable = lists:merge(lists:sort(Nodes), NodesAvailable), + + UpdDeployedVersions = update_deployed_versions(return, Nodes, undefined, DeployedVersions), + {noreply, State#state{nodes_available=NodesNowAvailable, + deployed_versions=UpdDeployedVersions}}; +handle_cast(_Msg, State) -> + {noreply, State}. + +handle_info(_Info, State) -> + {noreply, State}. + +terminate(_Reason, State) -> + %% Stop and reset all deployed nodes + stop_and_clean(State#state.nodes_deployed, + State#state.node_map, + State#state.deployed_versions, + true), + ok. + +code_change(_OldVsn, State, _Extra) -> + {ok, State}. + +%%%=================================================================== +%%% Internal functions +%%%=================================================================== + +stop_and_clean(NodeIds, NodeMap, DeployedVersions, Wait) -> + [begin + case version_deployed(NodeId, DeployedVersions) of + undefined -> + ok; + Version -> + rt_node:stop_and_wait(NodeId, + rt_node:node_name(NodeId, NodeMap), + Version), + wait_for_cleaner(rt_node:clean_data_dir(NodeId, Version), Wait) + end + end || NodeId <- NodeIds]. + +wait_for_cleaner(Pid, true) -> + WaitFun = + fun() -> + not is_process_alive(Pid) + end, + rt:wait_until(WaitFun); +wait_for_cleaner(_, false) -> + ok. + +reserve(Count, _Versions, State=#state{nodes_available=NodesAvailable}) + when Count > length(NodesAvailable) -> + {not_enough_nodes, State}; +reserve(Count, Versions, State=#state{nodes_available=NodesAvailable, + nodes_deployed=NodesDeployed, + version_map=VersionMap}) + when Count =:= length(NodesAvailable) -> + case versions_available(Count, Versions, VersionMap) of + true -> + UpdNodesDeployed = lists:sort(NodesDeployed ++ NodesAvailable), + {NodesAvailable, State#state{nodes_available=[], + nodes_deployed=UpdNodesDeployed}}; + false -> + {insufficient_versions_available, State} + end; +reserve(Count, Versions, State=#state{nodes_available=NodesAvailable, + nodes_deployed=NodesDeployed, + version_map=VersionMap}) -> + case versions_available(Count, Versions, VersionMap) of + true -> + {Reserved, UpdNodesAvailable} = lists:split(Count, NodesAvailable), + UpdNodesDeployed = lists:sort(NodesDeployed ++ Reserved), + UpdState = State#state{nodes_available=UpdNodesAvailable, + nodes_deployed=UpdNodesDeployed}, + {Reserved, UpdState}; + false -> + {insufficient_versions_available, State} + end. + +versions_available(Count, Versions, VersionMap) -> + lists:all(version_available_fun(Count, VersionMap), Versions). + +version_available_fun(Count, VersionMap) -> + fun(Version) -> + case lists:keyfind(Version, 1, VersionMap) of + {Version, VersionNodes} when length(VersionNodes) >= Count -> + true; + {Version, _} -> + false; + false -> + false + end + end. + +deploy(Nodes, NodeMap, Version, Config, Services) -> + rt_harness_util:deploy_nodes(Nodes, NodeMap, Version, Config, Services). + +upgrade(Nodes, CurrentVersion, NewVersion, Config) -> + [rt_node:upgrade(Node, CurrentVersion, NewVersion, Config) || + Node <- Nodes]. + +update_deployed_versions(deploy, Nodes, Version, DeployedVersions) -> + {_, UpdDeployedVersions} = lists:foldl(fun add_deployed_version/2, + {Version, DeployedVersions}, + Nodes), + UpdDeployedVersions; +update_deployed_versions(upgrade, Nodes, Version, DeployedVersions) -> + {_, UpdDeployedVersions} = lists:foldl(fun replace_deployed_version/2, + {Version, DeployedVersions}, + Nodes), + UpdDeployedVersions; +update_deployed_versions(return, Nodes, _, DeployedVersions) -> + lists:foldl(fun remove_deployed_version/2, DeployedVersions, Nodes). + +add_deployed_version(Node, {Version, DeployedVersions}) -> + {Version, [{Node, Version} | DeployedVersions]}. + +replace_deployed_version(Node, {Version, DeployedVersions}) -> + {Version, lists:keyreplace(Node, 1, DeployedVersions, {Node, Version})}. + +remove_deployed_version(Node, DeployedVersions) -> + lists:keydelete(Node, 1, DeployedVersions). + +version_deployed(Node, DeployedVersions) -> + case lists:keyfind(Node, 1, DeployedVersions) of + {Node, Version} -> + Version; + false -> + undefined + end. diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index f48ab8acf..09a355201 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -21,51 +21,100 @@ %% @private -module(riak_test_escript). -include("rt.hrl"). +%% TODO: Temporary build workaround, remove!! +-compile(export_all). -export([main/1]). -export([add_deps/1]). +-define(HEADER, [<<"Test">>, <<"Result">>, <<"Reason">>, <<"Test Duration">>]). + main(Args) -> - case prepare(Args) of - ok -> - ok; - {ParsedArgs, HarnessArgs, Tests} -> - OutDir = proplists:get_value(outdir, ParsedArgs), - Results = execute(Tests, OutDir, report(ParsedArgs), HarnessArgs), - finalize(Results, ParsedArgs) - end. + {ParsedArgs, HarnessArgs, Tests, _} = prepare(Args), + Results = execute(Tests, ParsedArgs, HarnessArgs), + finalize(Results, ParsedArgs). prepare(Args) -> - case parse_args(Args) of - {ParsedArgs, _, Tests} = ParseResults -> - io:format("Tests to run: ~p~n", [Tests]), - ok = erlang_setup(ParsedArgs), - ok = test_setup(ParsedArgs), - ParseResults; - ok -> - io:format("No tests to run.~n"), - ok - end. + {ParsedArgs, _, Tests, NonTests} = ParseResults = parse_args(Args), + io:format("Tests to run: ~p~n", [Tests]), + case NonTests of + [] -> + ok; + _ -> + io:format("These modules are not runnable tests: ~p~n", + [[NTMod || {NTMod, _} <- NonTests]]) + end, + ok = erlang_setup(ParsedArgs), + ok = test_setup(ParsedArgs), + ParseResults. -execute(Tests, Outdir, Report, HarnessArgs) -> - TestCount = length(Tests), - TestResults = [run_test(Test, - Outdir, - TestMetaData, - Report, - HarnessArgs, - TestCount) || - {Test, TestMetaData} <- Tests], - lists:filter(fun results_filter/1, TestResults). +execute(Tests, ParsedArgs, _HarnessArgs) -> + OutDir = proplists:get_value(outdir, ParsedArgs), + Report = report(ParsedArgs), + UpgradeList = upgrade_list( + proplists:get_value(upgrade_path, ParsedArgs)), + Backend = proplists:get_value(backend, ParsedArgs, bitcask), + + {ok, Executor} = riak_test_executor:start_link(Tests, + Backend, + OutDir, + Report, + UpgradeList, + self()), + wait_for_results(Executor, [], length(Tests), 0). + + +report_results(Results, Verbose) -> + %% TODO: Good place to also do giddyup reporting and provide a + %% place for extending to any other reporting sources that might + %% be useful. + print_summary(Results, undefined, Verbose), + ok. + + %% TestResults = run_tests(Tests, Outdir, Report, HarnessArgs), + %% lists:filter(fun results_filter/1, TestResults). + +%% run_test(Test, Outdir, TestMetaData, Report, HarnessArgs, NumTests) -> +%% rt_cover:maybe_start(Test), +%% SingleTestResult = riak_test_runner:run(Test, Outdir, TestMetaData, HarnessArgs), + + %% case NumTests of + %% 1 -> keep_them_up; + %% _ -> rt_cluster:teardown() + %% end, + +%% TODO: Do this in the test runner +%% CoverDir = rt_config:get(cover_output, "coverage"), +%% CoverFile = rt_cover:maybe_export_coverage(Test, CoverDir, erlang:phash2(TestMetaData)), +%% publish_report(SingleTestResult, CoverFile, Report), + +%% [{coverdata, CoverFile} | SingleTestResult]. + +%% TODO: Use `TestCount' and `Completed' to display progress output +wait_for_results(Executor, TestResults, TestCount, Completed) -> + receive + {Executor, {test_result, Result}} -> + wait_for_results(Executor, [Result | TestResults], TestCount, Completed+1); + {Executor, done} -> + rt_cover:stop(), + TestResults; + _ -> + wait_for_results(Executor, TestResults, TestCount, Completed) + end. finalize(TestResults, Args) -> - [rt_cover:maybe_import_coverage(proplists:get_value(coverdata, R)) || - R <- TestResults], - CoverDir = rt_config:get(cover_output, "coverage"), - Coverage = rt_cover:maybe_write_coverage(all, CoverDir), + %% TODO: Fixup coverage reporting + %% [rt_cover:maybe_import_coverage(proplists:get_value(coverdata, R)) || + %% R <- TestResults], + %% CoverDir = rt_config:get(cover_output, "coverage"), + %% Coverage = rt_cover:maybe_write_coverage(all, CoverDir), + node_manager:stop(), Verbose = proplists:is_defined(verbose, Args), - Teardown = not proplists:get_value(keep, Args, false), - maybe_teardown(Teardown, TestResults, Coverage, Verbose), + report_results(TestResults, Verbose), + + + %% Teardown = not proplists:get_value(keep, Args, false), + %% maybe_teardown(Teardown, TestResults, Coverage, Verbose), ok. %% Option Name, Short Code, Long Code, Argument Spec, Help Message @@ -81,7 +130,7 @@ cli_options() -> {verbose, $v, "verbose", undefined, "verbose output"}, {outdir, $o, "outdir", string, "output directory"}, {backend, $b, "backend", atom, "backend to test [memory | bitcask | eleveldb]"}, - {upgrade_version, $u, "upgrade", atom, "which version to upgrade from [ previous | legacy ]"}, + {upgrade_path, $u, "upgrade-path", atom, "comma-separated list representing an upgrade path (e.g. 1.2.1,1.3.4,1.4.10,2.0.0)"}, {keep, undefined, "keep", boolean, "do not teardown cluster"}, {report, $r, "report", string, "you're reporting an official test run, provide platform info (e.g. ubuntu-1204-64)\nUse 'config' if you want to pull from ~/.riak_test.config"}, {file, $F, "file", string, "use the specified file instead of ~/.riak_test.config"} @@ -98,11 +147,17 @@ add_deps(Path) -> test_setup(ParsedArgs) -> %% File output - OutDir = proplists:get_value(outdir, ParsedArgs), + OutDir = proplists:get_value(outdir, ParsedArgs, "log"), ensure_dir(OutDir), lager_setup(OutDir), + %% Prepare the test harness + {NodeIds, NodeMap, VersionMap} = rt_harness:setup(), + + %% Start the node manager + _ = node_manager:start_link(NodeIds, NodeMap, VersionMap), + %% Ensure existence of scratch_dir case ensure_dir(rt_config:get(rt_scratch_dir) ++ "/test.file") of ok -> @@ -113,6 +168,12 @@ test_setup(ParsedArgs) -> end, ok. +-spec upgrade_list(undefined | string()) -> undefined | [string()]. +upgrade_list(undefined) -> + undefined; +upgrade_list(Path) -> + string:tokens(Path, ","). + report(ParsedArgs) -> case proplists:get_value(report, ParsedArgs, undefined) of undefined -> @@ -129,8 +190,8 @@ parse_args(Args) -> help_or_parse_args({ok, {[], _}}) -> print_help(); help_or_parse_args({ok, {ParsedArgs, HarnessArgs}}) -> - help_or_parse_tests(ParsedArgs, - HarnessArgs, + help_or_parse_tests(ParsedArgs, + HarnessArgs, lists:member(help, ParsedArgs), args_invalid(ParsedArgs)); help_or_parse_args(_) -> @@ -145,24 +206,22 @@ help_or_parse_tests(ParsedArgs, HarnessArgs, false, false) -> %% test metadata load_initial_config(ParsedArgs), - Groups = proplists:get_all_values(groups, ParsedArgs), - TestData = load_tests(Groups, ParsedArgs), - - Tests = which_tests_to_run(report(ParsedArgs), TestData), + TestData = compose_test_data(ParsedArgs), + {Tests, NonTests} = which_tests_to_run(report(ParsedArgs), TestData), Offset = rt_config:get(offset, undefined), Workers = rt_config:get(workers, undefined), - shuffle_tests(ParsedArgs, HarnessArgs, Tests, Offset, Workers). + shuffle_tests(ParsedArgs, HarnessArgs, Tests, NonTests, Offset, Workers). args_invalid(ParsedArgs) -> - case { proplists:is_defined(groups, ParsedArgs), + case { proplists:is_defined(groups, ParsedArgs), proplists:is_defined(tests, ParsedArgs) } of {true, true} -> io:format("--groups and --tests are currently mutually exclusive.~n~n"), - true; - {_, _} -> + true; + {_, _} -> false end. - + load_initial_config(ParsedArgs) -> %% Loads application defaults application:load(riak_test), @@ -171,14 +230,14 @@ load_initial_config(ParsedArgs) -> rt_config:load(proplists:get_value(config, ParsedArgs), proplists:get_value(file, ParsedArgs)). -shuffle_tests(_, _, [], _, _) -> - lager:warning("No tests are scheduled to run"), - init:stop(1); -shuffle_tests(ParsedArgs, HarnessArgs, Tests, undefined, _) -> - {ParsedArgs, HarnessArgs, Tests}; -shuffle_tests(ParsedArgs, HarnessArgs, Tests, _, undefined) -> - {ParsedArgs, HarnessArgs, Tests}; -shuffle_tests(ParsedArgs, HarnessArgs, Tests, Offset, Workers) -> +shuffle_tests(_, _, [], _, _, _) -> + io:format("No tests are scheduled to run~n"), + halt(1); +shuffle_tests(ParsedArgs, HarnessArgs, Tests, NonTests, undefined, _) -> + {ParsedArgs, HarnessArgs, Tests, NonTests}; +shuffle_tests(ParsedArgs, HarnessArgs, Tests, NonTests, _, undefined) -> + {ParsedArgs, HarnessArgs, Tests, NonTests}; +shuffle_tests(ParsedArgs, HarnessArgs, Tests, NonTests, Offset, Workers) -> TestCount = length(Tests), %% Avoid dividing by zero, computers hate that Denominator = case Workers rem (TestCount+1) of @@ -189,7 +248,7 @@ shuffle_tests(ParsedArgs, HarnessArgs, Tests, Offset, Workers) -> {TestA, TestB} = lists:split(ActualOffset, Tests), lager:info("Offsetting ~b tests by ~b (~b workers, ~b offset)", [TestCount, ActualOffset, Workers, Offset]), - {ParsedArgs, HarnessArgs, TestB ++ TestA}. + {ParsedArgs, HarnessArgs, TestB ++ TestA, NonTests}. erlang_setup(_ParsedArgs) -> register(riak_test, self()), @@ -229,39 +288,38 @@ ensure_dir(undefined) -> ensure_dir(Dir) -> filelib:ensure_dir(Dir). -lager_setup(undefined) -> - set_lager_env(rt_config:get(lager_level, info)), - lager:start(); -lager_setup(_) -> - set_lager_env(notice), +lager_setup(OutputDir) -> + set_lager_env(OutputDir, + rt_config:get(lager_console_level, notice), + rt_config:get(lager_file_level, info)), lager:start(). -set_lager_env(LagerLevel) -> +set_lager_env(OutputDir, ConsoleLevel, FileLevel) -> application:load(lager), - HandlerConfig = [{lager_console_backend, LagerLevel}, - {lager_file_backend, [{file, "log/test.log"}, - {level, LagerLevel}]}], + HandlerConfig = [{lager_console_backend, ConsoleLevel}, + {lager_file_backend, [{file, filename:join(OutputDir, "test.log")}, + {level, FileLevel}]}], application:set_env(lager, handlers, HandlerConfig). -maybe_teardown(false, TestResults, Coverage, Verbose) -> - print_summary(TestResults, Coverage, Verbose), - lager:info("Keeping cluster running as requested"); -maybe_teardown(true, TestResults, Coverage, Verbose) -> - case {length(TestResults), proplists:get_value(status, hd(TestResults))} of - {1, fail} -> - print_summary(TestResults, Coverage, Verbose), - so_kill_riak_maybe(); - _ -> - lager:info("Multiple tests run or no failure"), - rt_cluster:teardown(), - print_summary(TestResults, Coverage, Verbose) - end, - ok. +%% maybe_teardown(false, TestResults, Coverage, Verbose) -> +%% print_summary(TestResults, Coverage, Verbose), +%% lager:info("Keeping cluster running as requested"); +%% maybe_teardown(true, TestResults, Coverage, Verbose) -> +%% case {length(TestResults), proplists:get_value(status, hd(TestResults))} of +%% {1, fail} -> +%% print_summary(TestResults, Coverage, Verbose), +%% so_kill_riak_maybe(); +%% _ -> +%% lager:info("Multiple tests run or no failure"), +%% rt_cluster:teardown(), +%% print_summary(TestResults, Coverage, Verbose) +%% end, +%% ok. load_tests([], ParsedArgs) -> RawTestList = proplists:get_all_values(tests, ParsedArgs), - TestList = lists:foldl(fun(X, Acc) -> - string:tokens(X, ", ") ++ Acc + TestList = lists:foldl(fun(X, Acc) -> + string:tokens(X, ", ") ++ Acc end, [], RawTestList), %% Parse Command Line Tests {CodePaths, SpecificTests} = @@ -275,90 +333,25 @@ load_tests([], ParsedArgs) -> Dirs = proplists:get_all_values(dir, ParsedArgs), SkipTests = string:tokens(proplists:get_value(skip, ParsedArgs, []), [$,]), DirTests = lists:append([load_tests_in_dir(Dir, SkipTests) || Dir <- Dirs]), - compose_test_data(DirTests, SpecificTests, ParsedArgs); -load_tests(RawGroupList, ParsedArgs) -> - Groups = lists:foldl(fun(X, Acc) -> - string:tokens(X, ", ") ++ Acc - end, [], RawGroupList), - Dirs = proplists:get_value(dir, ParsedArgs, ["./ebin"]), - AllDirTests = lists:append([load_tests_in_dir(Dir, []) || Dir <- Dirs]), - DirTests = get_group_tests(AllDirTests, Groups), - compose_test_data(DirTests, [], ParsedArgs). - -get_group_tests(Tests, Groups) -> - lists:filter(fun(Test) -> - Mod = list_to_atom(Test), - Attrs = Mod:module_info(attributes), - match_group_attributes(Attrs, Groups) - end, Tests). - -match_group_attributes(Attributes, Groups) -> - case proplists:get_value(test_type, Attributes) of - undefined -> - false; - TestTypes -> - lists:member(true, - [ TestType == list_to_atom(Group) - || Group <- Groups, TestType <- TestTypes ]) - end. + %% Project = list_to_binary(rt_config:get(rt_project, "undefined")), -compose_test_data(DirTests, SpecificTests, ParsedArgs) -> - Project = list_to_binary(rt_config:get(rt_project, "undefined")), - Backends = case proplists:get_all_values(backend, ParsedArgs) of - [] -> [undefined]; - Other -> Other - end, - Upgrades = case proplists:get_all_values(upgrade_version, ParsedArgs) of - [] -> [undefined]; - UpgradeList -> UpgradeList - end, - TestFoldFun = test_data_fun(rt:get_version(), Project, Backends, Upgrades), - lists:foldl(TestFoldFun, [], lists:usort(DirTests ++ SpecificTests)). - - -test_data_fun(Version, Project, Backends, Upgrades) -> - fun(Test, Tests) -> - [{list_to_atom(Test), - compose_test_datum(Version, Project, Backend, Upgrade)} - || Backend <- Backends, Upgrade <- Upgrades ] ++ Tests - end. - -compose_test_datum(Version, Project, undefined, undefined) -> - [{id, -1}, - {platform, <<"local">>}, - {version, Version}, - {project, Project}]; -compose_test_datum(Version, Project, undefined, Upgrade) -> - compose_test_datum(Version, Project, undefined, undefined) ++ - [{upgrade_version, Upgrade}]; -compose_test_datum(Version, Project, Backend, undefined) -> - compose_test_datum(Version, Project, undefined, undefined) ++ - [{backend, Backend}]; -compose_test_datum(Version, Project, Backend, Upgrade) -> - compose_test_datum(Version, Project, undefined, undefined) ++ - [{backend, Backend}, {upgrade_version, Upgrade}]. + %% Upgrades = proplists:get_value(upgrade_path, ParsedArgs), + %% TestFoldFun = test_data_fun(rt:get_version(), Project, Upgrades), + %% lists:foldl(TestFoldFun, [], lists:usort(DirTests ++ SpecificTests)). + lists:usort(DirTests ++ SpecificTests). extract_test_names(Test, {CodePaths, TestNames}) -> {[filename:dirname(Test) | CodePaths], - [filename:rootname(filename:basename(Test)) | TestNames]}. + [list_to_atom(filename:rootname(filename:basename(Test))) | TestNames]}. which_tests_to_run(undefined, CommandLineTests) -> - {Tests, NonTests} = - lists:partition(fun is_runnable_test/1, CommandLineTests), - lager:info("These modules are not runnable tests: ~p", - [[NTMod || {NTMod, _} <- NonTests]]), - Tests; + lists:partition(fun is_runnable_test/1, CommandLineTests); which_tests_to_run(Platform, []) -> giddyup:get_suite(Platform); which_tests_to_run(Platform, CommandLineTests) -> Suite = filter_zip_suite(Platform, CommandLineTests), - {Tests, NonTests} = - lists:partition(fun is_runnable_test/1, - lists:foldr(fun filter_merge_tests/2, [], Suite)), - - lager:info("These modules are not runnable tests: ~p", - [[NTMod || {NTMod, _} <- NonTests]]), - Tests. + lists:partition(fun is_runnable_test/1, + lists:foldr(fun filter_merge_tests/2, [], Suite)). filter_zip_suite(Platform, CommandLineTests) -> [ {SModule, SMeta, CMeta} || {SModule, SMeta} <- giddyup:get_suite(Platform), @@ -388,24 +381,29 @@ filter_merge_meta(SMeta, CMeta, [Field|Rest]) -> end. %% Check for api compatibility -is_runnable_test({TestModule, _}) -> +is_runnable_test(TestModule) -> {Mod, Fun} = riak_test_runner:function_name(confirm, TestModule), code:ensure_loaded(Mod), erlang:function_exported(Mod, Fun, 0) orelse erlang:function_exported(Mod, Fun, 2). -run_test(Test, Outdir, TestMetaData, Report, HarnessArgs, NumTests) -> - rt_cover:maybe_start(Test), - SingleTestResult = riak_test_runner:run(Test, Outdir, TestMetaData, HarnessArgs), - CoverDir = rt_config:get(cover_output, "coverage"), - case NumTests of - 1 -> keep_them_up; - _ -> rt_cluster:teardown() - end, - CoverFile = rt_cover:maybe_export_coverage(Test, CoverDir, erlang:phash2(TestMetaData)), - publish_report(SingleTestResult, CoverFile, Report), - rt_cover:stop(), - [{coverdata, CoverFile} | SingleTestResult]. +%% run_tests(Tests, Outdir, Report, HarnessArgs) -> + %% Need properties for tests prior to getting here Need server to + %% manage the aquisition of nodes and to handle comparison of test + %% `node_count' property with resources available. Also handle + %% notification of test completion. Hmm, maybe test execution + %% should be handled by a `gen_fsm' at this point to distinguish + %% the case when there are tests left to be tried with available + %% resources versus all have been tried or resources are + %% exhausted. + +%% [run_test(Test, +%% Outdir, +%% TestMetaData, +%% Report, +%% HarnessArgs, +%% TestCount) || +%% {Test, TestMetaData} <- Tests], publish_report(_SingleTestResult, _CoverFile, undefined) -> ok; @@ -451,44 +449,80 @@ parse_webhook(Props) -> undefined end. -print_summary(TestResults, CoverResult, Verbose) -> - io:format("~nTest Results:~n"), - - Results = [ - [ atom_to_list(proplists:get_value(test, SingleTestResult)) ++ "-" ++ - backend_list(proplists:get_value(backend, SingleTestResult)), - proplists:get_value(status, SingleTestResult), - proplists:get_value(reason, SingleTestResult)] - || SingleTestResult <- TestResults], - Width = test_name_width(Results), - - Print = fun(Test, Status, Reason) -> - case {Status, Verbose} of - {fail, true} -> io:format("~s: ~s ~p~n", [string:left(Test, Width), Status, Reason]); - _ -> io:format("~s: ~s~n", [string:left(Test, Width), Status]) - end +test_summary_fun({Test, pass, _}, {{Pass, _Fail, _Skipped}, Width}) -> + TestNameLength = length(atom_to_list(Test)), + UpdWidth = + case TestNameLength > Width of + true -> + TestNameLength; + false -> + Width + end, + {{Pass+1, _Fail, _Skipped}, UpdWidth}; +test_summary_fun({Test, {fail, _}, _}, {{_Pass, Fail, _Skipped}, Width}) -> + TestNameLength = length(atom_to_list(Test)), + UpdWidth = + case TestNameLength > Width of + true -> + TestNameLength; + false -> + Width + end, + {{_Pass, Fail+1, _Skipped}, UpdWidth}; +test_summary_fun({Test, {skipped, _}, _}, {{_Pass, _Fail, Skipped}, Width}) -> + TestNameLength = length(atom_to_list(Test)), + UpdWidth = + case TestNameLength > Width of + true -> + TestNameLength; + false -> + Width + end, + {{_Pass, _Fail, Skipped+1}, UpdWidth}. + +format_test_row({Test, Result, Duration}, _Width) -> + TestString = atom_to_list(Test), + case Result of + {Status, Reason} -> + [TestString, Status, Reason, Duration]; + pass -> + [TestString, "pass", "N/A", Duration] + end. + +print_summary(TestResults, _CoverResult, Verbose) -> + io:format("~nTest Results:~n~n"), + + {StatusCounts, Width} = lists:foldl(fun test_summary_fun/2, {{0,0,0}, 0}, TestResults), + + case Verbose of + true -> + Rows = + [format_test_row(Result, Width) || Result <- TestResults], + Table = riak_cli_table:autosize_create_table(?HEADER, Rows), + io:format("~ts~n", [Table]); + false -> + ok end, - [ Print(Test, Status, Reason) || [Test, Status, Reason] <- Results], - PassCount = length(lists:filter(fun(X) -> proplists:get_value(status, X) =:= pass end, TestResults)), - FailCount = length(lists:filter(fun(X) -> proplists:get_value(status, X) =:= fail end, TestResults)), + {PassCount, FailCount, SkippedCount} = StatusCounts, io:format("---------------------------------------------~n"), io:format("~w Tests Failed~n", [FailCount]), + io:format("~w Tests Skipped~n", [SkippedCount]), io:format("~w Tests Passed~n", [PassCount]), Percentage = case PassCount == 0 andalso FailCount == 0 of true -> 0; - false -> (PassCount / (PassCount + FailCount)) * 100 + false -> (PassCount / (PassCount + FailCount + SkippedCount)) * 100 end, io:format("That's ~w% for those keeping score~n", [Percentage]), - case CoverResult of - cover_disabled -> - ok; - {Coverage, AppCov} -> - io:format("Coverage : ~.1f%~n", [Coverage]), - [io:format(" ~s : ~.1f%~n", [App, Cov]) - || {App, Cov, _} <- AppCov] - end, + %% case CoverResult of + %% cover_disabled -> + %% ok; + %% {Coverage, AppCov} -> + %% io:format("Coverage : ~.1f%~n", [Coverage]), + %% [io:format(" ~s : ~.1f%~n", [App, Cov]) + %% || {App, Cov, _} <- AppCov] + %% end, ok. test_name_width(Results) -> @@ -504,14 +538,6 @@ backend_list(Backends) when is_list(Backends) -> end, lists:foldl(FoldFun, [], Backends). -results_filter(Result) -> - case proplists:get_value(status, Result) of - not_a_runnable_test -> - false; - _ -> - true - end. - load_tests_in_dir(Dir, SkipTests) -> case filelib:is_dir(Dir) of true -> @@ -549,4 +575,3 @@ so_kill_riak_maybe() -> io:format("Leaving Riak Up... "), rt:whats_up() end. - diff --git a/src/riak_test_executor.erl b/src/riak_test_executor.erl new file mode 100644 index 000000000..a185c2ce3 --- /dev/null +++ b/src/riak_test_executor.erl @@ -0,0 +1,273 @@ +-module(riak_test_executor). + +-behavior(gen_fsm). + +%% API +-export([start_link/6, + send_event/1, + stop/0]). + +%% gen_fsm callbacks +-export([init/1, + gather_properties/2, + gather_properties/3, + request_nodes/2, + request_nodes/3, + launch_test/2, + launch_test/3, + wait_for_completion/2, + wait_for_completion/3, + handle_event/3, + handle_sync_event/4, + handle_info/3, + terminate/3, + code_change/4]). + +-record(state, {pending_tests :: [atom()], + running_tests=[] :: [atom()], + waiting_tests=[] :: [atom()], + notify_pid :: pid(), + backend :: atom(), + upgrade_list :: [string()], + test_properties :: [proplists:proplist()], + runner_pids=[] :: [pid()], + log_dir :: string(), + report_info :: string()}). + +%%%=================================================================== +%%% API +%%%=================================================================== + +%% @doc Start the test executor +-spec start_link(atom(), atom(), string(), string(), [string()], pid()) -> {ok, pid()} | ignore | {error, term()}. +start_link(Tests, Backend, LogDir, ReportInfo, UpgradeList, NotifyPid) -> + Args = [Tests, Backend, LogDir, ReportInfo, UpgradeList, NotifyPid], + gen_fsm:start_link({local, ?MODULE}, ?MODULE, Args, []). + +send_event(Msg) -> + gen_fsm:send_event(?MODULE, Msg). + +%% @doc Stop the executor +-spec stop() -> ok | {error, term()}. +stop() -> + gen_fsm:sync_send_all_state_event(?MODULE, stop, infinity). + +%%%=================================================================== +%%% gen_fsm callbacks +%%%=================================================================== + +init([Tests, Backend, LogDir, ReportInfo, UpgradeList, NotifyPid]) -> + State = #state{pending_tests=Tests, + backend=Backend, + log_dir=LogDir, + report_info=ReportInfo, + upgrade_list=UpgradeList, + notify_pid=NotifyPid}, + {ok, gather_properties, State, 0}. + +%% @doc there are no all-state events for this fsm +handle_event(_Event, StateName, State) -> + {next_state, StateName, State}. + +%% @doc Handle synchronous events that should be handled +%% the same regardless of the current state. +-spec handle_sync_event(term(), term(), atom(), #state{}) -> + {reply, term(), atom(), #state{}}. +handle_sync_event(_Event, _From, _StateName, _State) -> + {reply, ok, ok, _State}. + +handle_info(_Info, StateName, State) -> + {next_state, StateName, State}. + +terminate(normal, _StateName, State) -> + report_done(State), + ok; +terminate(_Reason, _StateName, _State) -> + ok. + +%% @doc this fsm has no special upgrade process +code_change(_OldVsn, StateName, State, _Extra) -> + {ok, StateName, State}. + +%%% Asynchronous call handling functions for each FSM state + +%% TODO: Modify property gathering to account for `upgrade_path' +%% specified via the command line and replace accordingly in +%% properties record. +gather_properties(timeout, State) -> + OverrideProps = override_props(State), + Properties = test_properties(State#state.pending_tests, OverrideProps), + {next_state, request_nodes, State#state{test_properties=Properties}, 0}; +gather_properties(_Event, _State) -> + {next_state, gather_properties, _State}. + +request_nodes(timeout, State) -> + #state{pending_tests=[NextTest | _], + test_properties=PropertiesList} = State, + lager:notice("Starting next test: ~p", [NextTest]), + %% Find the properties for the next pending test + {NextTest, TestProps} = lists:keyfind(NextTest, 1, PropertiesList), + %% Send async request to node manager + VersionsToTest = versions_to_test(TestProps), + node_manager:reserve_nodes(rt_properties:get(node_count, TestProps), + VersionsToTest, + reservation_notify_fun()), + {next_state, launch_test, State}; +request_nodes({test_complete, Test, Pid, Results, Duration}, State) -> + #state{pending_tests=Pending, + waiting_tests=Waiting, + running_tests=Running, + runner_pids=Pids} = State, + %% Report results + report_results(Test, Results, Duration, State), + UpdState = State#state{running_tests=lists:delete(Test, Running), + runner_pids=lists:delete(Pid, Pids), + pending_tests=Pending++Waiting, + waiting_tests=[]}, + {next_state, request_nodes, UpdState}; +request_nodes(_Event, _State) -> + {next_state, request_nodes, _State}. + +launch_test(insufficient_versions_available, State) -> + #state{pending_tests=[HeadPending | RestPending]} = State, + report_results(HeadPending, {skipped, insufficient_versions}, 0, State), + UpdState = State#state{pending_tests=RestPending}, + launch_test_transition(UpdState); +launch_test(not_enough_nodes, State) -> + %% Move head of pending to waiting and try next test if there is + %% one left in pending. + #state{pending_tests=[HeadPending | RestPending], + waiting_tests=Waiting} = State, + UpdState = State#state{pending_tests=RestPending, + waiting_tests=[HeadPending | Waiting]}, + launch_test_transition(UpdState); +launch_test({nodes, Nodes, NodeMap}, State) -> + %% Spawn a test runner for the head of pending. If pending is now + %% empty transition to `wait_for_completion'; otherwise, + %% transition to `request_nodes'. + #state{pending_tests=[NextTest | RestPending], + backend=Backend, + test_properties=PropertiesList, + runner_pids=Pids, + running_tests=Running} = State, + {NextTest, TestProps} = lists:keyfind(NextTest, 1, PropertiesList), + UpdTestProps = rt_properties:set([{node_map, NodeMap}, {node_ids, Nodes}], + TestProps), + Pid = spawn_link(riak_test_runner, start, [NextTest, Backend, UpdTestProps]), + UpdState = State#state{pending_tests=RestPending, + runner_pids=[Pid | Pids], + running_tests=[NextTest | Running]}, + launch_test_transition(UpdState); +launch_test({test_complete, Test, Pid, Results, Duration}, State) -> + #state{pending_tests=Pending, + waiting_tests=Waiting, + running_tests=Running, + runner_pids=Pids} = State, + %% Report results + report_results(Test, Results, Duration, State), + UpdState = State#state{running_tests=lists:delete(Test, Running), + runner_pids=lists:delete(Pid, Pids), + pending_tests=Pending++Waiting, + waiting_tests=[]}, + {next_state, launch_test, UpdState}; +launch_test(_Event, _State) -> + ok. + +wait_for_completion({test_complete, Test, Pid, Results, Duration}, State) -> + #state{pending_tests=Pending, + waiting_tests=Waiting, + running_tests=Running, + runner_pids=Pids} = State, + %% Report results + report_results(Test, Results, Duration, State), + UpdState = State#state{running_tests=lists:delete(Test, Running), + runner_pids=lists:delete(Pid, Pids), + pending_tests=Pending++Waiting, + waiting_tests=[]}, + wait_for_completion_transition(UpdState); +wait_for_completion(_Event, _State) -> + ok. + +%% Synchronous call handling functions for each FSM state + +gather_properties(_Event, _From, _State) -> + ok. + +request_nodes(_Event, _From, _State) -> + ok. + +launch_test(_Event, _From, _State) -> + ok. + +wait_for_completion(_Event, _From, _State) -> + ok. + +%%%=================================================================== +%%% Internal functions +%%%=================================================================== + +report_results(Test, Results, Duration, #state{notify_pid=NotifyPid}) -> + NotifyPid ! {self(), {test_result, {Test, Results, Duration}}}, + ok. + +report_done(#state{notify_pid=NotifyPid}) -> + NotifyPid ! {self(), done}, + ok. + +wait_for_completion_transition(State=#state{pending_tests=[], + running_tests=[]}) -> + {stop, normal, State}; +wait_for_completion_transition(State=#state{pending_tests=[]}) -> + {next_state, wait_for_completion, State}; +wait_for_completion_transition(State) -> + {next_state, request_nodes, State, 0}. + +launch_test_transition(State=#state{pending_tests=[]}) -> + {next_state, wait_for_completion, State}; +launch_test_transition(State) -> + {next_state, request_nodes, State, 0}. + +reservation_notify_fun() -> + fun(X) -> + ?MODULE:send_event(X) + end. + +test_properties(Tests, OverriddenProps) -> + lists:foldl(test_property_fun(OverriddenProps), [], Tests). + +test_property_fun(OverrideProps) -> + fun(TestModule, Acc) -> + {PropsMod, PropsFun} = riak_test_runner:function_name(properties, + TestModule, + 0, + rt_cluster), + Properties = rt_properties:set(OverrideProps, PropsMod:PropsFun()), + [{TestModule, Properties} | Acc] + end. + +versions_to_test(Properties) -> + versions_to_test(Properties, rt_properties:get(rolling_upgrade, Properties)). + +%% An `upgrade_path' specified on the command line overrides the test +%% property setting. If the `rolling_upgrade' property is is `false' +%% then the `start_version' property of the test is the only version +%% tested. +versions_to_test(Properties, true) -> + case rt_properties:get(upgrade_path, Properties) of + undefined -> + [rt_properties:get(start_version, Properties)]; + UpgradePath -> + UpgradePath + end; +versions_to_test(Properties, false) -> + [rt_properties:get(start_version, Properties)]. + +%% Function to abstract away the details of what properties +%% can be overridden on the command line. +override_props(State) -> + case State#state.upgrade_list of + undefined -> + []; + UpgradeList -> + [{upgrade_path, UpgradeList}] + end. diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index cbc00c37b..22616d809 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -18,164 +18,319 @@ %% %% ------------------------------------------------------------------- -%% @doc riak_test_runner runs a riak_test module's run/0 function. +%% @doc riak_test_runner runs a riak_test module's `confirm/0' function. -module(riak_test_runner). -%% Need to export to use with `spawn_link'. --export([return_to_exit/3]). --export([run/4, metadata/0, metadata/1, function_name/2]). +-behavior(gen_fsm). + +%% API +-export([start/3, + send_event/2, + stop/0]). + +-export([function_name/2, + function_name/4]). + +%% gen_fsm callbacks +-export([init/1, + setup/2, + setup/3, + execute/2, + execute/3, + wait_for_completion/2, + wait_for_completion/3, + wait_for_upgrade/2, + wait_for_upgrade/3, + handle_event/3, + handle_sync_event/4, + handle_info/3, + terminate/3, + code_change/4]). + -include("rt.hrl"). -include_lib("eunit/include/eunit.hrl"). --spec(metadata() -> [{atom(), term()}]). -%% @doc fetches test metadata from spawned test process -metadata() -> - riak_test ! metadata, - receive - {metadata, TestMeta} -> TestMeta - end. +-record(state, {test_module :: atom(), + properties :: proplists:proplist(), + metadata :: term(), + backend :: atom(), + test_timeout :: integer(), + execution_pid :: pid(), + group_leader :: pid(), + start_time :: erlang:timestamp(), + end_time :: erlang:timestamp(), + setup_modfun :: {atom(), atom()}, + confirm_modfun :: {atom(), atom()}, + backend_check :: atom(), + prereq_check :: atom(), + current_version :: string(), + remaining_versions :: [string()], + test_results :: [term()]}). -metadata(Pid) -> - riak_test ! {metadata, Pid}, - receive - {metadata, TestMeta} -> TestMeta - end. +%%%=================================================================== +%%% API +%%%=================================================================== --spec(run(integer(), atom(), [{atom(), term()}], list()) -> [tuple()]). -%% @doc Runs a module's run/0 function after setting up a log -%% capturing backend for lager. It then cleans up that backend -%% and returns the logs as part of the return proplist. -run(TestModule, Outdir, TestMetaData, HarnessArgs) -> - start_lager_backend(TestModule, Outdir), - rt:setup_harness(TestModule, HarnessArgs), - BackendExtras = case proplists:get_value(multi_config, TestMetaData) of - undefined -> []; - Value -> [{multi_config, Value}] - end, - Backend = rt_backend:set_backend( - proplists:get_value(backend, TestMetaData), BackendExtras), - {PropsMod, PropsFun} = function_name(properties, TestModule, 0, rt_cluster), - {SetupMod, SetupFun} = function_name(setup, TestModule, 2, rt_cluster), - {ConfirmMod, ConfirmFun} = function_name(confirm, TestModule), - {Status, Reason} = case check_prereqs(ConfirmMod) of - true -> - lager:notice("Running Test ~s", [TestModule]), - execute(TestModule, - {PropsMod, PropsFun}, - {SetupMod, SetupFun}, - {ConfirmMod, ConfirmFun}, - TestMetaData); - not_present -> - {fail, test_does_not_exist}; - _ -> - {fail, all_prereqs_not_present} - end, +%% @doc Start the test executor +start(TestModule, Backend, Properties) -> + Args = [TestModule, Backend, Properties], + gen_fsm:start_link(?MODULE, Args, []). - lager:notice("~s Test Run Complete", [TestModule]), - {ok, Logs} = stop_lager_backend(), - Log = unicode:characters_to_binary(Logs), +send_event(Pid, Msg) -> + gen_fsm:send_event(Pid, Msg). - RetList = [{test, TestModule}, {status, Status}, {log, Log}, {backend, Backend} | proplists:delete(backend, TestMetaData)], - case Status of - fail -> RetList ++ [{reason, iolist_to_binary(io_lib:format("~p", [Reason]))}]; - _ -> RetList - end. +%% @doc Stop the executor +-spec stop() -> ok | {error, term()}. +stop() -> + gen_fsm:sync_send_all_state_event(?MODULE, stop, infinity). -start_lager_backend(TestModule, Outdir) -> - case Outdir of - undefined -> ok; - _ -> - gen_event:add_handler(lager_event, lager_file_backend, - {Outdir ++ "/" ++ atom_to_list(TestModule) ++ ".dat_test_output", - rt_config:get(lager_level, info), 10485760, "$D0", 1}), - lager:set_loglevel(lager_file_backend, rt_config:get(lager_level, info)) - end, - gen_event:add_handler(lager_event, riak_test_lager_backend, [rt_config:get(lager_level, info), false]), - lager:set_loglevel(riak_test_lager_backend, rt_config:get(lager_level, info)). - -stop_lager_backend() -> - gen_event:delete_handler(lager_event, lager_file_backend, []), - gen_event:delete_handler(lager_event, riak_test_lager_backend, []). - -%% does some group_leader swapping, in the style of EUnit. -execute(TestModule, PropsModFun, SetupModFun, ConfirmModFun, TestMetaData) -> - process_flag(trap_exit, true), - OldGroupLeader = group_leader(), +%%%=================================================================== +%%% gen_fsm callbacks +%%%=================================================================== + +%% @doc Read the storage schedule and go to idle. +%% compose_test_datum(Version, Project, undefined, undefined) -> +init([TestModule, Backend, Properties]) -> + lager:debug("Started riak_test_runnner with pid ~p", [self()]), + Project = list_to_binary(rt_config:get(rt_project, "undefined")), + MetaData = [{id, -1}, + {platform, <<"local">>}, + {version, rt:get_version()}, + {project, Project}], + TestTimeout = rt_config:get(test_timeout, rt_config:get(rt_max_wait_time)), + SetupModFun = function_name(setup, TestModule, 2, rt_cluster), + {ConfirmMod, _} = ConfirmModFun = function_name(confirm, TestModule), + BackendCheck = check_backend(Backend, + rt_properties:get(valid_backends, Properties)), + PreReqCheck = check_prereqs(ConfirmMod), + State = #state{test_module=TestModule, + properties=Properties, + metadata=MetaData, + backend=Backend, + test_timeout=TestTimeout, + setup_modfun=SetupModFun, + confirm_modfun=ConfirmModFun, + backend_check=BackendCheck, + prereq_check=PreReqCheck, + group_leader=group_leader()}, + {ok, setup, State, 0}. + +%% @doc there are no all-state events for this fsm +handle_event(_Event, StateName, State) -> + {next_state, StateName, State}. + +%% @doc Handle synchronous events that should be handled +%% the same regardless of the current state. +-spec handle_sync_event(term(), term(), atom(), #state{}) -> + {reply, term(), atom(), #state{}}. +handle_sync_event(_Event, _From, _StateName, _State) -> + {reply, ok, ok, _State}. + +handle_info(_Info, StateName, State) -> + {next_state, StateName, State}. + +terminate(_Reason, _StateName, _State) -> + ok. + +%% @doc this fsm has no special upgrade process +code_change(_OldVsn, StateName, State, _Extra) -> + {ok, StateName, State}. + +%% Asynchronous call handling functions for each FSM state + +setup(timeout, State=#state{backend_check=false}) -> + notify_executor({skipped, invalid_backend}, State), + cleanup(State), + {stop, normal, State}; +setup(timeout, State=#state{prereq_check=false}) -> + notify_executor({fail, prereq_check_failed}, State), + cleanup(State), + {stop, normal, State}; +setup(timeout, State=#state{backend=Backend, + properties=Properties}) -> NewGroupLeader = riak_test_group_leader:new_group_leader(self()), group_leader(NewGroupLeader, self()), {0, UName} = rt:cmd("uname -a"), - lager:info("Test Runner `uname -a` : ~s", [UName]), + lager:info("Test Runner: ~s", [UName]), - %% Pid = spawn_link(?MODULE, return_to_exit, [Mod, Fun, []]), - Pid = spawn_link(test_fun(PropsModFun, SetupModFun, ConfirmModFun, TestMetaData)), - Ref = case rt_config:get(test_timeout, undefined) of - Timeout when is_integer(Timeout) -> - erlang:send_after(Timeout, self(), test_took_too_long); - _ -> - undefined - end, + NodeIds = rt_properties:get(node_ids, Properties), + Services = rt_properties:get(required_services, Properties), + {StartVersion, OtherVersions} = test_versions(Properties), + Config = rt_backend:set(Backend, rt_properties:get(config, Properties)), + node_manager:deploy_nodes(NodeIds, + StartVersion, + Config, + Services, + notify_fun(self())), + lager:info("Waiting for deploy nodes response at ~p", [self()]), - {Status, Reason} = rec_loop(Pid, TestModule, TestMetaData), - case Ref of - undefined -> - ok; - _ -> - erlang:cancel_timer(Ref) - end, - riak_test_group_leader:tidy_up(OldGroupLeader), - case Status of - fail -> - ErrorHeader = "================ " ++ atom_to_list(TestModule) ++ " failure stack trace =====================", - ErrorFooter = [ $= || _X <- lists:seq(1,length(ErrorHeader))], - Error = io_lib:format("~n~s~n~p~n~s~n", [ErrorHeader, Reason, ErrorFooter]), - lager:error(Error); - _ -> meh - end, - {Status, Reason}. - --spec test_fun({atom(), atom()}, {atom(), atom()}, {atom(), atom()}, proplists:proplist()) -> function(). -test_fun({PropsMod, PropsFun}, {SetupMod, SetupFun}, ConfirmModFun, MetaData) -> - fun() -> - Properties = PropsMod:PropsFun(), - case SetupMod:SetupFun(Properties, MetaData) of - {ok, SetupData} -> - RollingUpgradeTest = rt_properties:get(rolling_upgrade, SetupData), - ConfirmFun = compose_confirm_fun(ConfirmModFun, - SetupData, - MetaData, - RollingUpgradeTest), - - ConfirmFun(); - _ -> - fail - end - end. + %% Set the initial value for `current_version' in the properties record + {ok, UpdProperties} = + rt_properties:set(current_version, StartVersion, Properties), -compose_confirm_fun({ConfirmMod, ConfirmFun}, SetupData, MetaData, true) -> - Nodes = rt_properties:get(nodes, SetupData), - WaitForTransfers = rt_properties:get(wait_for_transfers, SetupData), - UpgradeVersion = rt_properties:get(upgrade_version, SetupData), - fun() -> - InitialResult = ConfirmMod:ConfirmFun(SetupData, MetaData), - OtherResults = [begin - ensure_all_nodes_running(Nodes), - _ = rt_node:upgrade(Node, UpgradeVersion), - _ = rt_cluster:maybe_wait_for_transfers(Nodes, WaitForTransfers), - ConfirmMod:ConfirmFun(SetupData, MetaData) - end || Node <- Nodes], - lists:all(fun(R) -> R =:= pass end, [InitialResult | OtherResults]) - end; -compose_confirm_fun({ConfirmMod, ConfirmFun}, SetupData, MetaData, false) -> + UpdState = State#state{current_version=StartVersion, + remaining_versions=OtherVersions, + properties=UpdProperties}, + {next_state, execute, UpdState}; +setup(_Event, _State) -> + ok. + +execute({nodes_deployed, _}, State) -> + #state{test_module=TestModule, + properties=Properties, + setup_modfun={SetupMod, SetupFun}, + confirm_modfun=ConfirmModFun, + metadata=MetaData, + test_timeout=TestTimeout} = State, + lager:notice("Running ~s", [TestModule]), + + StartTime = os:timestamp(), + %% Perform test setup which includes clustering of the nodes if + %% required by the test properties. The cluster information is placed + %% into the properties record and returned by the `setup' function. + UpdState = + case SetupMod:SetupFun(Properties, MetaData) of + {ok, UpdProperties} -> + Pid = spawn_link(test_fun(UpdProperties, + ConfirmModFun, + MetaData, + self())), + State#state{execution_pid=Pid, + properties=UpdProperties, + start_time=StartTime}; + _ -> + ?MODULE:send_event(self(), test_result({fail, test_setup_failed})), + State#state{start_time=StartTime} + end, + {next_state, wait_for_completion, UpdState, TestTimeout}; +execute(_Event, _State) -> + {next_state, execute, _State}. + +wait_for_completion(timeout, State) -> + %% Test timed out + UpdState = State#state{end_time=os:timestamp()}, + notify_executor(timeout, UpdState), + cleanup(UpdState), + {stop, normal, UpdState}; +wait_for_completion({test_result, Result}, State=#state{remaining_versions=[]}) -> + %% TODO: Format results for aggregate test runs if needed. For + %% upgrade tests with failure return which versions had failure + %% along with reasons. + UpdState = State#state{end_time=os:timestamp()}, + notify_executor(Result, UpdState), + cleanup(UpdState), + {stop, normal, UpdState}; +wait_for_completion({test_result, Result}, State) -> + #state{backend=Backend, + test_results=TestResults, + current_version=CurrentVersion, + remaining_versions=[NextVersion | RestVersions], + properties=Properties} = State, + Config = rt_backend:set(Backend, rt_properties:get(config, Properties)), + NodeIds = rt_properties:get(node_ids, Properties), + node_manager:upgrade_nodes(NodeIds, + CurrentVersion, + NextVersion, + Config, + notify_fun(self())), + UpdState = State#state{test_results=[Result | TestResults], + current_version=NextVersion, + remaining_versions=RestVersions}, + {next_state, wait_for_upgrade, UpdState}; +wait_for_completion(_Msg, _State) -> + {next_state, wait_for_completion, _State}. + +wait_for_upgrade(nodes_upgraded, State) -> + #state{properties=Properties, + confirm_modfun=ConfirmModFun, + current_version=CurrentVersion, + metadata=MetaData, + test_timeout=TestTimeout} = State, + + %% Update the `current_version' in the properties record + {ok, UpdProperties} = + rt_properties:set(current_version, CurrentVersion, Properties), + + %% TODO: Maybe wait for transfers. Probably should be + %% a call to an exported function in `rt_cluster' + Pid = spawn_link(test_fun(UpdProperties, + ConfirmModFun, + MetaData, + self())), + UpdState = State#state{execution_pid=Pid, + properties=UpdProperties}, + {next_state, wait_for_completion, UpdState, TestTimeout}; +wait_for_upgrade(_Event, _State) -> + {next_state, wait_for_upgrade, _State}. + +%% Synchronous call handling functions for each FSM state + +setup(_Event, _From, _State) -> + ok. + +execute(_Event, _From, _State) -> + ok. + +wait_for_completion(_Event, _From, _State) -> + ok. + +wait_for_upgrade(_Event, _From, _State) -> + ok. + +%%%=================================================================== +%%% Internal functions +%%%=================================================================== + +-spec test_fun(rt_properties:properties(), {atom(), atom()}, proplists:proplist(), pid()) -> + function(). +test_fun(Properties, {ConfirmMod, ConfirmFun}, MetaData, NotifyPid) -> fun() -> - ConfirmMod:ConfirmFun(SetupData, MetaData) + %% Exceptions and their handling sucks, but eunit throws + %% errors `erlang:error' so here we are + try ConfirmMod:ConfirmFun(Properties, MetaData) of + TestResult -> + ?MODULE:send_event(NotifyPid, test_result(TestResult)) + catch + Error:Reason -> + lager:notice("Error: ~p Reason: ~p", [Error, Reason]), + TestResult = format_eunit_error(Reason), + ?MODULE:send_event(NotifyPid, test_result(TestResult)) + end end. -ensure_all_nodes_running(Nodes) -> - [begin - ok = rt_node:start_and_wait(Node), - ok = rt:wait_until_registered(Node, riak_core_ring_manager) - end || Node <- Nodes]. +format_eunit_error({assertion_failed, InfoList}) -> + LineNum = proplists:get_value(line, InfoList), + Expression = proplists:get_value(expression, InfoList), + Value = proplists:get_value(value, InfoList), + ErrorStr = io_lib:format("Assertion ~s is ~p at line ~B", + [Expression, Value, LineNum]), + {fail, ErrorStr}; +format_eunit_error({assertCmd_failed, InfoList}) -> + LineNum = proplists:get_value(line, InfoList), + Command = proplists:get_value(command, InfoList), + Status = proplists:get_value(status, InfoList), + ErrorStr = io_lib:format("Command \"~s\" returned a status of ~B at line ~B", + [Command, Status, LineNum]), + {fail, ErrorStr}; +format_eunit_error({assertMatch_failed, InfoList}) -> + LineNum = proplists:get_value(line, InfoList), + Pattern = proplists:get_value(pattern, InfoList), + Value = proplists:get_value(value, InfoList), + ErrorStr = io_lib:format("Pattern ~s did not match value ~p at line ~B", + [Pattern, Value, LineNum]), + {fail, ErrorStr}; +format_eunit_error({assertEqual_failed, InfoList}) -> + LineNum = proplists:get_value(line, InfoList), + Expression = proplists:get_value(expression, InfoList), + Expected = proplists:get_value(expected, InfoList), + Value = proplists:get_value(value, InfoList), + ErrorStr = io_lib:format("~s = ~p is not equal to expected value ~p at line ~B", + [Expression, Value, Expected, LineNum]), + {fail, ErrorStr}; +format_eunit_error(Other) -> + ErrorStr = io_lib:format("Unknown error encountered: ~p", [Other]), + {fail, ErrorStr}. function_name(confirm, TestModule) -> TMString = atom_to_list(TestModule), @@ -195,46 +350,96 @@ function_name(FunName, TestModule, Arity, Default) when is_atom(TestModule) -> {Default, FunName} end. -rec_loop(Pid, TestModule, TestMetaData) -> - receive - test_took_too_long -> - exit(Pid, kill), - {fail, test_timed_out}; - metadata -> - Pid ! {metadata, TestMetaData}, - rec_loop(Pid, TestModule, TestMetaData); - {metadata, P} -> - P ! {metadata, TestMetaData}, - rec_loop(Pid, TestModule, TestMetaData); - {'EXIT', Pid, normal} -> {pass, undefined}; - {'EXIT', Pid, Error} -> - lager:warning("~s failed: ~p", [TestModule, Error]), - {fail, Error} - end. +%% remove_lager_backend() -> +%% gen_event:delete_handler(lager_event, lager_file_backend, []), +%% gen_event:delete_handler(lager_event, riak_test_lager_backend, []). + %% A return of `fail' must be converted to a non normal exit since %% status is determined by `rec_loop'. %% %% @see rec_loop/3 --spec return_to_exit(module(), atom(), list()) -> ok. -return_to_exit(Mod, Fun, Args) -> - case apply(Mod, Fun, Args) of - pass -> - %% same as exit(normal) - ok; - fail -> - exit(fail) - end. +%% -spec return_to_exit(module(), atom(), list()) -> ok. +%% return_to_exit(Mod, Fun, Args) -> +%% case apply(Mod, Fun, Args) of +%% pass -> +%% %% same as exit(normal) +%% ok; +%% fail -> +%% exit(fail) +%% end. + +-spec check_backend(atom(), all | [atom()]) -> boolean(). +check_backend(_Backend, all) -> + true; +check_backend(Backend, ValidBackends) -> + lists:member(Backend, ValidBackends). +%% Check the prequisites for executing the test check_prereqs(Module) -> - try Module:module_info(attributes) of - Attrs -> - Prereqs = proplists:get_all_values(prereq, Attrs), - P2 = [ {Prereq, rt_local:which(Prereq)} || Prereq <- Prereqs], - lager:info("~s prereqs: ~p", [Module, P2]), - [ lager:warning("~s prereq '~s' not installed.", [Module, P]) || {P, false} <- P2], - lists:all(fun({_, Present}) -> Present end, P2) - catch - _DontCare:_Really -> - not_present + Attrs = Module:module_info(attributes), + Prereqs = proplists:get_all_values(prereq, Attrs), + P2 = [{Prereq, rt_local:which(Prereq)} || Prereq <- Prereqs], + lager:info("~s prereqs: ~p", [Module, P2]), + [lager:warning("~s prereq '~s' not installed.", + [Module, P]) || {P, false} <- P2], + lists:all(fun({_, Present}) -> Present end, P2). + +notify_fun(Pid) -> + fun(X) -> + ?MODULE:send_event(Pid, X) end. + +cleanup(#state{group_leader=OldGroupLeader, + properties=Properties}) -> + node_manager:return_nodes(rt_properties:get(node_ids, Properties)), + riak_test_group_leader:tidy_up(OldGroupLeader). + +notify_executor(timeout, #state{test_module=Test, + start_time=Start, + end_time=End}) -> + Duration = timer:now_diff(End, Start), + Notification = {test_complete, Test, self(), {fail, timeout}, Duration}, + riak_test_executor:send_event(Notification); +notify_executor(fail, #state{test_module=Test, + start_time=Start, + end_time=End}) -> + Duration = timer:now_diff(End, Start), + Notification = {test_complete, Test, self(), {fail, unknown}, Duration}, + riak_test_executor:send_event(Notification); +notify_executor(pass, #state{test_module=Test, + start_time=Start, + end_time=End}) -> + Duration = timer:now_diff(End, Start), + Notification = {test_complete, Test, self(), pass, Duration}, + riak_test_executor:send_event(Notification); +notify_executor(FailResult, #state{test_module=Test, + start_time=Start, + end_time=End}) -> + Duration = now_diff(End, Start), + Notification = {test_complete, Test, self(), FailResult, Duration}, + riak_test_executor:send_event(Notification). + +test_versions(Properties) -> + StartVersion = rt_properties:get(start_version, Properties), + UpgradePath = rt_properties:get(upgrade_path, Properties), + case UpgradePath of + undefined -> + {StartVersion, []}; + [] -> + {StartVersion, []}; + _ -> + [UpgradeHead | Rest] = UpgradePath, + {UpgradeHead, Rest} + end. + +now_diff(undefined, _) -> + 0; +now_diff(_, undefined) -> + 0; +now_diff(End, Start) -> + timer:now_diff(End, Start). + +%% Simple function to hide the details of the message wrapping +test_result(Result) -> + {test_result, Result}. diff --git a/src/rt.erl b/src/rt.erl index 0ec1584b4..1aa661ba8 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -88,7 +88,7 @@ set_advanced_conf/2, setup_harness/2, setup_log_capture/1, - stream_cmd/1, + stream_cmd/1, stream_cmd/2, spawn_cmd/1, spawn_cmd/2, diff --git a/src/rt_backend.erl b/src/rt_backend.erl index 3fe13e64d..496afd446 100644 --- a/src/rt_backend.erl +++ b/src/rt_backend.erl @@ -22,13 +22,58 @@ -include_lib("eunit/include/eunit.hrl"). -compile(export_all). - --define(HARNESS, (rt_config:get(rt_harness))). +-export([set/2]). %%%=================================================================== %%% Test harness setup, configuration, and internal utilities %%%=================================================================== +replace_backend(Backend, false) -> + [{storage_backend, Backend}]; +replace_backend(Backend, {riak_kv, KVSection}) -> + lists:keystore(storage_backend, 1, KVSection, {storage_backend, Backend}). + +%% TODO: Probably should abstract this into the rt_config module and +%% make a sensible API to hide the ugliness of dealing with the lists +%% module funs. +%% +%% TODO: Add support for Riak CS backend and arbitrary multi backend +%% configurations +set(eleveldb, Config) -> + UpdKVSection = replace_backend(riak_kv_eleveldb_backend, + lists:keyfind(riak_kv, 1, Config)), + lists:keystore(riak_kv, 1, Config, {riak_kv, UpdKVSection}); +set(memory, Config) -> + UpdKVSection = replace_backend(riak_kv_eleveldb_backend, + lists:keyfind(riak_kv, 1, Config)), + lists:keystore(riak_kv, 1, Config, {riak_kv, UpdKVSection}); +set(multi, Config) -> + UpdKVSection = + replace_backend(riak_kv_multi_backend, + lists:keyfind(riak_kv, 1, Config)) ++ + multi_backend_config(default), + lists:keystore(riak_kv, 1, Config, {riak_kv, UpdKVSection}); +set({multi, indexmix}, Config) -> + UpdKVSection = + replace_backend(riak_kv_multi_backend, + lists:keyfind(riak_kv, 1, Config)) ++ + multi_backend_config(indexmix), + lists:keystore(riak_kv, 1, Config, {riak_kv, UpdKVSection}); +set(_, Config) -> + UpdKVSection = replace_backend(riak_kv_bitcask_backend, + lists:keyfind(riak_kv, 1, Config)), + lists:keystore(riak_kv, 1, Config, {riak_kv, UpdKVSection}). + +multi_backend_config(default) -> + [{multi_backend_default, <<"eleveldb1">>}, + {multi_backend, [{<<"eleveldb1">>, riak_kv_eleveldb_backend, []}, + {<<"memory1">>, riak_kv_memory_backend, []}, + {<<"bitcask1">>, riak_kv_bitcask_backend, []}]}]; +multi_backend_config(indexmix) -> + [{multi_backend_default, <<"eleveldb1">>}, + {multi_backend, [{<<"eleveldb1">>, riak_kv_eleveldb_backend, []}, + {<<"memory1">>, riak_kv_memory_backend, []}]}]. + %% @doc Sets the backend of ALL nodes that could be available to riak_test. %% this is not limited to the nodes under test, but any node that %% riak_test is able to find. It then queries each available node diff --git a/src/rt_cluster.erl b/src/rt_cluster.erl index 79cf0e52a..1b7f237df 100644 --- a/src/rt_cluster.erl +++ b/src/rt_cluster.erl @@ -24,27 +24,18 @@ -export([properties/0, setup/2, augment_config/3, - deploy_nodes/1, - deploy_nodes/2, - deploy_clusters/1, - build_cluster/1, - build_cluster/2, - build_cluster/3, - build_clusters/1, clean_cluster/1, - join_cluster/1, + join_cluster/2, clean_data_dir/1, clean_data_dir/2, try_nodes_ready/3, versions/0, teardown/0]). --export([maybe_wait_for_transfers/2]). +-export([maybe_wait_for_transfers/3]). -include("rt.hrl"). --define(HARNESS, (rt_config:get(rt_harness))). - %% @doc Default properties used if a riak_test module does not specify %% a custom properties function. -spec properties() -> rt_properties:properties(). @@ -53,110 +44,110 @@ properties() -> -spec setup(rt_properties:properties(), proplists:proplist()) -> {ok, rt_properties:properties()} | {error, term()}. -setup(Properties, MetaData) -> - rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), - - RollingUpgrade = - proplists:get_value(rolling_upgrade, - MetaData, - rt_properties:get(rolling_upgrade, Properties)), - Version = rt_properties:get(start_version, Properties), - Config = rt_properties:get(config, Properties), - Versions = [{Version, Config} || - _ <- lists:seq(1, rt_properties:get(node_count, Properties))], - Nodes = deploy_or_build_cluster(Versions, - rt_properties:get(make_cluster, Properties)), - maybe_wait_for_transfers(Nodes, - rt_properties:get(wait_for_transfers, Properties)), - UpdPropertyList = [{nodes, Nodes}, {rolling_upgrade, RollingUpgrade}], - rt_properties:set(UpdPropertyList, Properties). - -deploy_or_build_cluster(Versions, true) -> - build_cluster(Versions); -deploy_or_build_cluster(Versions, false) -> - deploy_nodes(Versions). - -%% @doc Deploy a set of freshly installed Riak nodes, returning a list of the -%% nodes deployed. -%% @todo Re-add -spec after adding multi-version support -deploy_nodes(Versions) when is_list(Versions) -> - deploy_nodes(Versions, [riak_kv]); -deploy_nodes(NumNodes) when is_integer(NumNodes) -> - deploy_nodes([ current || _ <- lists:seq(1, NumNodes)]). - -%% @doc Deploy a set of freshly installed Riak nodes with the given -%% `InitialConfig', returning a list of the nodes deployed. --spec deploy_nodes(NumNodes :: integer(), any()) -> [node()]. -deploy_nodes(NumNodes, InitialConfig) when is_integer(NumNodes) -> - NodeConfig = [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)], - deploy_nodes(NodeConfig); -deploy_nodes(Versions, Services) -> - NodeConfig = [ rt_config:version_to_config(Version) || Version <- Versions ], - Nodes = rt_harness:deploy_nodes(NodeConfig), - lager:info("Waiting for services ~p to start on ~p.", [Services, Nodes]), - [ ok = rt:wait_for_service(Node, Service) || Node <- Nodes, - Service <- Services ], - Nodes. - -deploy_clusters(Settings) -> - ClusterConfigs = [case Setting of - Configs when is_list(Configs) -> - Configs; - NumNodes when is_integer(NumNodes) -> - [{current, default} || _ <- lists:seq(1, NumNodes)]; - {NumNodes, InitialConfig} when is_integer(NumNodes) -> - [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)]; - {NumNodes, Vsn, InitialConfig} when is_integer(NumNodes) -> - [{Vsn, InitialConfig} || _ <- lists:seq(1,NumNodes)] - end || Setting <- Settings], - rt_harness:deploy_clusters(ClusterConfigs). - -build_clusters(Settings) -> - Clusters = deploy_clusters(Settings), - [begin - join_cluster(Nodes), - lager:info("Cluster built: ~p", [Nodes]) - end || Nodes <- Clusters], - Clusters. - -maybe_wait_for_transfers(Nodes, true) -> +setup(Properties, _MetaData) -> + %% rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), + case form_clusters(Properties) of + {ok, Clusters} -> + maybe_wait_for_transfers(rt_properties:get(node_ids, Properties), + rt_properties:get(node_map, Properties), + rt_properties:get(wait_for_transfers, Properties)), + rt_properties:set(clusters, Clusters, Properties); + Error -> + Error + end. + +-type clusters() :: [{pos_integer(), [string()]}]. +-spec form_clusters(rt_properties:properties()) -> clusters(). +form_clusters(Properties) -> + NodeIds = rt_properties:get(node_ids, Properties), + NodeMap = rt_properties:get(node_map, Properties), + ClusterCount = rt_properties:get(cluster_count, Properties), + ClusterWeights = rt_properties:get(cluster_weights, Properties), + MakeCluster = rt_properties:get(make_cluster, Properties), + case divide_nodes(NodeIds, ClusterCount, ClusterWeights) of + {ok, Clusters} -> + maybe_join_clusters(Clusters, NodeMap, MakeCluster), + {ok, Clusters}; + Error -> + Error + end. + +divide_nodes(Nodes, Count, Weights) + when length(Nodes) < Count; + Weights =/= undefined, length(Weights) =/= Count -> + {error, invalid_cluster_properties}; +divide_nodes(Nodes, 1, _) -> + {ok, [{1, Nodes}]}; +divide_nodes(Nodes, Count, Weights) -> + case validate_weights(Weights) of + true -> + TotalNodes = length(Nodes), + NodeCounts = node_counts_from_weights(TotalNodes, Count, Weights), + {_, Clusters, _} = lists:foldl(fun take_nodes/2, {1, [], Nodes}, NodeCounts), + {ok, lists:reverse(Clusters)}; + false -> + {error, invalid_cluster_weights} + end. + +take_nodes(NodeCount, {Index, ClusterAcc, Nodes}) -> + {NewClusterNodes, RestNodes} = lists:split(NodeCount, Nodes), + {Index + 1, [{Index, NewClusterNodes} | ClusterAcc], RestNodes}. + +validate_weights(undefined) -> + true; +validate_weights(Weights) -> + not lists:sum(Weights) > 1.0 . + +node_counts_from_weights(NodeCount, ClusterCount, undefined) -> + %% Split the nodes evenly. A remainder of nodes is handled by + %% distributing one node per cluster until the remainder is + %% exhausted. + NodesPerCluster = NodeCount div ClusterCount, + Remainder = NodeCount rem ClusterCount, + [NodesPerCluster + remainder_to_apply(Remainder, ClusterIndex) || + ClusterIndex <- lists:seq(1, ClusterCount)]; +node_counts_from_weights(NodeCount, ClusterCount, Weights) -> + InitialNodeCounts = [node_count_from_weight(NodeCount, Weight) || Weight <- Weights], + Remainder = NodeCount - lists:sum(InitialNodeCounts), + [ClusterNodeCount + remainder_to_apply(Remainder, ClusterIndex) || + {ClusterIndex, ClusterNodeCount} + <- lists:zip(lists:seq(1, ClusterCount), InitialNodeCounts)]. + +node_count_from_weight(TotalNodes, Weight) -> + RawNodeCount = TotalNodes * Weight, + IntegerPortion = trunc(RawNodeCount), + Remainder = RawNodeCount - IntegerPortion, + case Remainder >= 0.5 of + true -> + IntegerPortion + 1; + false -> + IntegerPortion + end. + +remainder_to_apply(Remainder, Index) when Remainder > Index -> + 0; +remainder_to_apply(_Remainder, _Index) -> + 1. + +maybe_join_clusters(Clusters, NodeMap, true) -> + [join_cluster(ClusterNodes, NodeMap) || {_, ClusterNodes} <- Clusters]; +maybe_join_clusters(_Clusters, _NodeMap, false) -> + ok. + +maybe_wait_for_transfers(NodeIds, NodeMap, true) -> lager:info("Waiting for transfers"), - rt:wait_until_transfers_complete(Nodes); -maybe_wait_for_transfers(_Nodes, false) -> + rt:wait_until_transfers_complete([rt_node:node_name(NodeId, NodeMap) + || NodeId <- NodeIds]); +maybe_wait_for_transfers(_NodeIds, _NodeMap, false) -> ok. -%% @doc Safely construct a new cluster and return a list of the deployed nodes -%% @todo Add -spec and update doc to reflect mult-version changes -build_cluster(Versions) when is_list(Versions) -> - build_cluster(length(Versions), Versions, default); -build_cluster(NumNodes) -> - build_cluster(NumNodes, default). - -%% @doc Safely construct a `NumNode' size cluster using -%% `InitialConfig'. Return a list of the deployed nodes. -build_cluster(NumNodes, InitialConfig) -> - build_cluster(NumNodes, [], InitialConfig). - -build_cluster(NumNodes, Versions, InitialConfig) -> - %% Deploy a set of new nodes - Nodes = - case Versions of - [] -> - deploy_nodes(NumNodes, InitialConfig); - _ -> - deploy_nodes(Versions) - end, - - join_cluster(Nodes), - lager:info("Cluster built: ~p", [Nodes]), - Nodes. - -join_cluster(Nodes) -> +join_cluster(NodeIds, NodeMap) -> + NodeNames = [rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds], %% Ensure each node owns 100% of it's own ring - [?assertEqual([Node], rt_ring:owners_according_to(Node)) || Node <- Nodes], + [?assertEqual([Node], rt_ring:owners_according_to(Node)) || Node <- NodeNames], %% Join nodes - [Node1|OtherNodes] = Nodes, + [Node1|OtherNodes] = NodeNames, case OtherNodes of [] -> %% no other nodes, nothing to join/plan/commit @@ -166,14 +157,14 @@ join_cluster(Nodes) -> %% large amount of redundant handoff done in a sequential join [rt_node:staged_join(Node, Node1) || Node <- OtherNodes], rt_node:plan_and_commit(Node1), - try_nodes_ready(Nodes, 3, 500) + try_nodes_ready(NodeNames, 3, 500) end, - ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes)), + ?assertEqual(ok, rt_node:wait_until_nodes_ready(NodeNames)), %% Ensure each node owns a portion of the ring - rt_node:wait_until_nodes_agree_about_ownership(Nodes), - ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes)), + rt_node:wait_until_nodes_agree_about_ownership(NodeNames), + ?assertEqual(ok, rt:wait_until_no_pending_changes(NodeNames)), ok. try_nodes_ready([Node1 | _Nodes], 0, _SleepMs) -> diff --git a/src/rt_harness.erl b/src/rt_harness.erl index 1270f3bfc..88822445d 100644 --- a/src/rt_harness.erl +++ b/src/rt_harness.erl @@ -25,46 +25,46 @@ -define(HARNESS_MODULE, (rt_config:get(rt_harness))). --export([ - start/1, - stop/1, - deploy_clusters/1, - clean_data_dir/2, - deploy_nodes/1, - spawn_cmd/1, - spawn_cmd/2, - cmd/1, - cmd/2, - setup_harness/2, - get_deps/0, - get_version/0, - get_backends/0, - set_backend/1, - whats_up/0, - get_ip/1, - node_id/1, - node_version/1, - admin/2, - riak/2, - attach/2, - attach_direct/2, - console/2, - update_app_config/2, - teardown/0, - set_conf/2, - set_advanced_conf/2]). - -start(Node) -> - ?HARNESS_MODULE:start(Node). - -stop(Node) -> - ?HARNESS_MODULE:stop(Node). - +-export([start/2, + stop/2, + deploy_clusters/1, + clean_data_dir/3, + deploy_nodes/1, + spawn_cmd/1, + spawn_cmd/2, + cmd/1, + cmd/2, + setup/0, + get_deps/0, + get_version/0, + get_backends/0, + set_backend/1, + whats_up/0, + get_ip/1, + node_id/1, + node_version/1, + admin/2, + riak/2, + run_riak/3, + attach/2, + attach_direct/2, + console/2, + update_app_config/3, + teardown/0, + set_conf/2, + set_advanced_conf/2]). + +start(Node, Version) -> + ?HARNESS_MODULE:start(Node, Version). + +stop(Node, Version) -> + ?HARNESS_MODULE:stop(Node, Version). + deploy_clusters(ClusterConfigs) -> ?HARNESS_MODULE:deploy_clusters(ClusterConfigs). - -clean_data_dir(Nodes, SubDir) -> - ?HARNESS_MODULE:clean_data_dir(Nodes, SubDir). + +clean_data_dir(Node, Version, SubDir) -> + ?HARNESS_MODULE:clean_data_dir(Node, Version, SubDir). spawn_cmd(Cmd) -> ?HARNESS_MODULE:spawn_cmd(Cmd). @@ -81,11 +81,12 @@ cmd(Cmd, Opts) -> deploy_nodes(NodeConfig) -> ?HARNESS_MODULE:deploy_nodes(NodeConfig). -setup_harness(Test, Args) -> - ?HARNESS_MODULE:setup_harness(Test, Args). +setup() -> + ?HARNESS_MODULE:setup_harness(). get_deps() -> ?HARNESS_MODULE:get_deps(). + get_version() -> ?HARNESS_MODULE:get_version(). @@ -113,6 +114,9 @@ admin(Node, Args) -> riak(Node, Args) -> ?HARNESS_MODULE:riak(Node, Args). +run_riak(Node, Version, Command) -> + ?HARNESS_MODULE:run_riak(Node, Version, Command). + attach(Node, Expected) -> ?HARNESS_MODULE:attach(Node, Expected). @@ -122,8 +126,8 @@ attach_direct(Node, Expected) -> console(Node, Expected) -> ?HARNESS_MODULE:console(Node, Expected). -update_app_config(Node, Config) -> - ?HARNESS_MODULE:update_app_config(Node, Config). +update_app_config(Node, Version, Config) -> + ?HARNESS_MODULE:update_app_config(Node, Version, Config). teardown() -> ?HARNESS_MODULE:teardown(). diff --git a/src/rt_harness_util.erl b/src/rt_harness_util.erl index 82c6a4abb..71ee8f016 100644 --- a/src/rt_harness_util.erl +++ b/src/rt_harness_util.erl @@ -22,9 +22,9 @@ -module(rt_harness_util). -include_lib("eunit/include/eunit.hrl"). --define(DEVS(N), lists:concat(["dev", N, "@127.0.0.1"])). +-define(DEVS(N), lists:concat([N, "@127.0.0.1"])). -define(DEV(N), list_to_atom(?DEVS(N))). --define(PATH, (rt_config:get(rtdev_path))). +-define(PATH, (rt_config:get(root_path))). -export([admin/2, attach/2, @@ -32,13 +32,14 @@ cmd/1, cmd/2, console/2, - deploy_clusters/1, + deploy_nodes/5, get_ip/1, node_id/1, node_version/1, riak/2, set_conf/2, set_advanced_conf/2, + setup_harness/3, get_advanced_riak_conf/1, update_app_config_file/2, spawn_cmd/1, @@ -63,73 +64,75 @@ attach_direct(Node, Expected) -> console(Node, Expected) -> interactive(Node, "console", Expected). -deploy_clusters(ClusterConfigs) -> - NumNodes = rt_config:get(num_nodes, 6), - RequestedNodes = lists:flatten(ClusterConfigs), - - case length(RequestedNodes) > NumNodes of - true -> - erlang:error("Requested more nodes than available"); - false -> - Nodes = deploy_nodes(RequestedNodes), - {DeployedClusters, _} = lists:foldl( - fun(Cluster, {Clusters, RemNodes}) -> - {A, B} = lists:split(length(Cluster), RemNodes), - {Clusters ++ [A], B} - end, {[], Nodes}, ClusterConfigs), - DeployedClusters - end. - -deploy_nodes(NodeConfig) -> - Path = relpath(root), - lager:info("Riak path: ~p", [Path]), - NumNodes = length(NodeConfig), - NodesN = lists:seq(1, NumNodes), - Nodes = [?DEV(N) || N <- NodesN], - NodeMap = orddict:from_list(lists:zip(Nodes, NodesN)), - {Versions, Configs} = lists:unzip(NodeConfig), - VersionMap = lists:zip(NodesN, Versions), - - %% Check that you have the right versions available - [ check_node(Version) || Version <- VersionMap ], - rt_config:set(rt_nodes, NodeMap), - rt_config:set(rt_versions, VersionMap), - - create_dirs(Nodes), +%% deploy_clusters(ClusterConfigs) -> +%% NumNodes = rt_config:get(num_nodes, 6), +%% RequestedNodes = lists:flatten(ClusterConfigs), + +%% case length(RequestedNodes) > NumNodes of +%% true -> +%% erlang:error("Requested more nodes than available"); +%% false -> +%% Nodes = deploy_nodes(RequestedNodes), +%% {DeployedClusters, _} = lists:foldl( +%% fun(Cluster, {Clusters, RemNodes}) -> +%% {A, B} = lists:split(length(Cluster), RemNodes), +%% {Clusters ++ [A], B} +%% end, {[], Nodes}, ClusterConfigs), +%% DeployedClusters +%% end. + +%% deploy_nodes(NodeConfig) -> +deploy_nodes(NodeIds, NodeMap, Version, Config, Services) -> + %% create snmp dirs, for EE + create_dirs(Version, NodeIds), %% Set initial config - add_default_node_config(Nodes), - rt:pmap(fun({_, default}) -> - ok; - ({Node, {cuttlefish, Config}}) -> - set_conf(Node, Config); - ({Node, Config}) -> - rt_config:update_app_config(Node, Config) - end, - lists:zip(Nodes, Configs)), - - %% create snmp dirs, for EE - create_dirs(Nodes), + ConfigUpdateFun = + fun(Node) -> + rt_harness:update_app_config(Node, Version, Config) + end, + rt:pmap(ConfigUpdateFun, NodeIds), %% Start nodes - %%[run_riak(N, relpath(node_version(N)), "start") || N <- Nodes], - rt:pmap(fun(N) -> run_riak(N, relpath(node_version(N)), "start") end, NodesN), + RunRiakFun = + fun(Node) -> + rt_harness:run_riak(Node, Version, "start") + end, + rt:pmap(RunRiakFun, NodeIds), %% Ensure nodes started - [ok = rt:wait_until_pingable(N) || N <- Nodes], + lager:debug("Wait until pingable: ~p", [NodeIds]), + [ok = rt:wait_until_pingable(rt_node:node_name(NodeId, NodeMap)) + || NodeId <- NodeIds], + %% TODO Rubbish! Fix this. %% %% Enable debug logging - %% [rpc:call(N, lager, set_loglevel, [lager_console_backend, debug]) || N <- Nodes], + %% [rpc:call(N, lager, set_loglevel, [lager_console_backend, debug]) + %% || N <- Nodes], - %% We have to make sure that riak_core_ring_manager is running before we can go on. - [ok = rt:wait_until_registered(N, riak_core_ring_manager) || N <- Nodes], + %% We have to make sure that riak_core_ring_manager is running + %% before we can go on. + [ok = rt:wait_until_registered(rt_node:node_name(NodeId, NodeMap), + riak_core_ring_manager) + || NodeId <- NodeIds], %% Ensure nodes are singleton clusters - [ok = rt_ring:check_singleton_node(?DEV(N)) || {N, Version} <- VersionMap, - Version /= "0.14.2"], + case Version =/= "0.14.2" of + true -> + [ok = rt_ring:check_singleton_node(rt_node:node_name(NodeId, NodeMap)) + || NodeId <- NodeIds]; + false -> + ok + end, + + %% Wait for services to start + lager:debug("Waiting for services ~p to start on ~p.", [Services, NodeIds]), + [ ok = rt:wait_for_service(rt_node:node_name(NodeId, NodeMap), Service) + || NodeId <- NodeIds, + Service <- Services ], - lager:info("Deployed nodes: ~p", [Nodes]), - Nodes. + lager:debug("Deployed nodes: ~p", [NodeIds]), + NodeIds. interactive(Node, Command, Exp) -> N = node_id(Node), @@ -374,35 +377,37 @@ all_the_files(DevPath, File) -> devpaths() -> lists:usort([ DevPath || {_Name, DevPath} <- proplists:delete(root, rt_config:get(rtdev_path))]). -create_dirs(Nodes) -> - Snmp = [node_path(Node) ++ "/data/snmp/agent/db" || Node <- Nodes], +create_dirs(Version, NodeIds) -> + VersionPath = filename:join(?PATH, Version), + Snmp = [filename:join([VersionPath, NodeId, "data/snmp/agent/db"]) || + NodeId <- NodeIds], [?assertCmd("mkdir -p " ++ Dir) || Dir <- Snmp]. -check_node({_N, Version}) -> - case proplists:is_defined(Version, rt_config:get(rtdev_path)) of - true -> ok; - _ -> - lager:error("You don't have Riak ~s installed or configured", [Version]), - erlang:error("You don't have Riak " ++ atom_to_list(Version) ++ " installed or configured") - end. - -add_default_node_config(Nodes) -> - case rt_config:get(rt_default_config, undefined) of - undefined -> ok; - Defaults when is_list(Defaults) -> - rt:pmap(fun(Node) -> - rt_config:update_app_config(Node, Defaults) - end, Nodes), - ok; - BadValue -> - lager:error("Invalid value for rt_default_config : ~p", [BadValue]), - throw({invalid_config, {rt_default_config, BadValue}}) - end. - -node_path(Node) -> - N = node_id(Node), - Path = relpath(node_version(N)), - lists:flatten(io_lib:format("~s/dev/dev~b", [Path, N])). +%% check_node({_N, Version}) -> +%% case proplists:is_defined(Version, rt_config:get(rtdev_path)) of +%% true -> ok; +%% _ -> +%% lager:error("You don't have Riak ~s installed or configured", [Version]), +%% erlang:error("You don't have Riak " ++ atom_to_list(Version) ++ " installed or configured") +%% end. + +%% add_default_node_config(Nodes) -> +%% case rt_config:get(rt_default_config, undefined) of +%% undefined -> ok; +%% Defaults when is_list(Defaults) -> +%% rt:pmap(fun(Node) -> +%% rt_config:update_app_config(Node, Defaults) +%% end, Nodes), +%% ok; +%% BadValue -> +%% lager:error("Invalid value for rt_default_config : ~p", [BadValue]), +%% throw({invalid_config, {rt_default_config, BadValue}}) +%% end. + +%% node_path(Node) -> +%% N = node_id(Node), +%% Path = relpath(node_version(N)), +%% lists:flatten(io_lib:format("~s/dev/dev~b", [Path, N])). set_advanced_conf(all, NameValuePairs) -> lager:info("rtdev:set_advanced_conf(all, ~p)", [NameValuePairs]), @@ -442,3 +447,27 @@ update_app_config_file(ConfigFile, Config) -> NewConfigOut = io_lib:format("~p.", [NewConfig]), ?assertEqual(ok, file:write_file(ConfigFile, NewConfigOut)), ok. + +%% TODO: This made sense in an earlier iteration, but probably is no +%% longer needed. Original idea was to provide a place for setup that +%% was general to all harnesses to happen. +setup_harness(VersionMap, NodeIds, NodeMap) -> + %% rt_config:set(rt_nodes, Nodes), + %% rt_config:set(rt_nodes_available, Nodes), + %% rt_config:set(rt_version_map, VersionMap), + %% rt_config:set(rt_versions, VersionMap), + %% [create_dirs(Version, VersionNodes) || {Version, VersionNodes} <- VersionMap], + {NodeIds, NodeMap, VersionMap}. + +%% %% @doc Stop nodes and wipe out their data directories +%% stop_and_clean_nodes(Nodes, Version) when is_list(Nodes) -> +%% [rt_node:stop_and_wait(Node) || Node <- Nodes], +%% clean_data_dir(Nodes). + +%% clean_data_dir(Nodes) -> +%% clean_data_dir(Nodes, ""). + +%% clean_data_dir(Nodes, SubDir) when not is_list(Nodes) -> +%% clean_data_dir([Nodes], SubDir); +%% clean_data_dir(Nodes, SubDir) when is_list(Nodes) -> +%% rt_harness:clean_data_dir(Nodes, SubDir). diff --git a/src/rt_node.erl b/src/rt_node.erl index e52316035..22a1268f9 100644 --- a/src/rt_node.erl +++ b/src/rt_node.erl @@ -21,15 +21,14 @@ -include("rt.hrl"). -include_lib("eunit/include/eunit.hrl"). --export([start/1, - start_and_wait/1, - async_start/1, - stop/1, - stop_and_wait/1, - upgrade/2, - upgrade/3, +-export([start/2, + start_and_wait/3, + async_start/2, + stop/2, + stop_and_wait/3, + upgrade/4, is_ready/1, - slow_upgrade/3, + %% slow_upgrade/3, join/2, staged_join/2, plan_and_commit/1, @@ -43,57 +42,75 @@ wait_until_nodes_ready/1, wait_until_owners_according_to/2, wait_until_nodes_agree_about_ownership/1, - is_pingable/1]). + is_pingable/1, + clean_data_dir/2, + node_name/2]). + +-spec node_name(string(), [{string(), node()}]) -> node() | undefined. +%% @doc Hide the details of underlying data structure of the node map +%% in case it needs to change at some point. +node_name(NodeId, NodeMap) -> + case lists:keyfind(NodeId, 1, NodeMap) of + {NodeId, NodeName} -> + NodeName; + false -> + undefined + end. + +clean_data_dir(Node, Version) -> + clean_data_dir(Node, Version, ""). --define(HARNESS, (rt_config:get(rt_harness))). +clean_data_dir(Node, Version, SubDir) -> + rt_harness:clean_data_dir(Node, Version, SubDir). %% @doc Start the specified Riak node -start(Node) -> - rt_harness:start(Node). +start(Node, Version) -> + rt_harness:start(Node, Version). %% @doc Start the specified Riak `Node' and wait for it to be pingable -start_and_wait(Node) -> - start(Node), - ?assertEqual(ok, rt:wait_until_pingable(Node)). +start_and_wait(NodeId, NodeName, Version) -> + start(NodeId, Version), + ?assertEqual(ok, rt:wait_until_pingable(NodeName)). -async_start(Node) -> - spawn(fun() -> start(Node) end). +async_start(Node, Version) -> + spawn(fun() -> start(Node, Version) end). %% @doc Stop the specified Riak `Node'. -stop(Node) -> +stop(Node, Version) -> lager:info("Stopping riak on ~p", [Node]), - timer:sleep(10000), %% I know, I know! - rt_harness:stop(Node). + %% timer:sleep(10000), %% I know, I know! + rt_harness:stop(Node, Version). %%rpc:call(Node, init, stop, []). %% @doc Stop the specified Riak `Node' and wait until it is not pingable -stop_and_wait(Node) -> - stop(Node), - ?assertEqual(ok, rt:wait_until_unpingable(Node)). +-spec stop_and_wait(string(), node(), string()) -> ok. +stop_and_wait(NodeId, NodeName, Version) -> + stop(NodeId, Version), + ?assertEqual(ok, rt:wait_until_unpingable(NodeName)). -%% @doc Upgrade a Riak `Node' to the specified `NewVersion'. -upgrade(Node, NewVersion) -> - rt_harness:upgrade(Node, NewVersion). +%% %% @doc Upgrade a Riak `Node' to the specified `NewVersion'. +%% upgrade(Node, NewVersion) -> +%% rt_harness:upgrade(Node, NewVersion). %% @doc Upgrade a Riak `Node' to the specified `NewVersion' and update %% the config based on entries in `Config'. -upgrade(Node, NewVersion, Config) -> - rt_harness:upgrade(Node, NewVersion, Config). +upgrade(Node, CurrentVersion, NewVersion, Config) -> + rt_harness:upgrade(Node, CurrentVersion, NewVersion, Config). %% @doc Upgrade a Riak node to a specific version using the alternate %% leave/upgrade/rejoin approach -slow_upgrade(Node, NewVersion, Nodes) -> - lager:info("Perform leave/upgrade/join upgrade on ~p", [Node]), - lager:info("Leaving ~p", [Node]), - leave(Node), - ?assertEqual(ok, rt:wait_until_unpingable(Node)), - upgrade(Node, NewVersion), - lager:info("Rejoin ~p", [Node]), - join(Node, hd(Nodes -- [Node])), - lager:info("Wait until all nodes are ready and there are no pending changes"), - ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), - ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes)), - ok. +%% slow_upgrade(Node, NewVersion, Nodes) -> +%% lager:info("Perform leave/upgrade/join upgrade on ~p", [Node]), +%% lager:info("Leaving ~p", [Node]), +%% leave(Node), +%% ?assertEqual(ok, rt:wait_until_unpingable(Node)), +%% upgrade(Node, NewVersion), +%% lager:info("Rejoin ~p", [Node]), +%% join(Node, hd(Nodes -- [Node])), +%% lager:info("Wait until all nodes are ready and there are no pending changes"), +%% ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), +%% ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes)), +%% ok. %% @doc Have `Node' send a join request to `PNode' join(Node, PNode) -> @@ -114,7 +131,11 @@ plan_and_commit(Node) -> lager:info("planning and commiting cluster join"), case rpc:call(Node, riak_core_claimant, plan, []) of {error, ring_not_ready} -> - lager:info("plan: ring not ready"), + lager:info("plan: ring not ready on ~p", [Node]), + timer:sleep(100), + plan_and_commit(Node); + {badrpc, _} -> + lager:info("plan: ring not ready on ~p", [Node]), timer:sleep(100), plan_and_commit(Node); {ok, _, _} -> diff --git a/src/rt_pb.erl b/src/rt_pb.erl index 157261758..3b720e6a6 100644 --- a/src/rt_pb.erl +++ b/src/rt_pb.erl @@ -22,6 +22,7 @@ -include_lib("eunit/include/eunit.hrl"). -export([pbc/1, + stop/1, pbc_read/3, pbc_read/4, pbc_read_check/4, @@ -45,12 +46,16 @@ %% @doc get me a protobuf client process and hold the mayo! -spec pbc(node()) -> pid(). pbc(Node) -> - rt:wait_for_service(Node, riak_kv), - ConnInfo = proplists:get_value(Node, rt:connection_info([Node])), - {IP, PBPort} = proplists:get_value(pb, ConnInfo), + %% rt:wait_for_service(Node, riak_kv), + %% ConnInfo = proplists:get_value(Node, rt:connection_info([Node])), + %% {IP, PBPortz} = proplists:get_value(pb, ConnInfo), + {ok, [{IP, PBPort}]} = get_pb_conn_info(Node), {ok, Pid} = riakc_pb_socket:start_link(IP, PBPort, [{auto_reconnect, true}]), Pid. +stop(Pid) -> + riakc_pb_socket:stop(Pid). + %% @doc does a read via the erlang protobuf client -spec pbc_read(pid(), binary(), binary()) -> binary(). pbc_read(Pid, Bucket, Key) -> diff --git a/src/rt_properties.erl b/src/rt_properties.erl index fb29afe17..bb0ec2090 100644 --- a/src/rt_properties.erl +++ b/src/rt_properties.erl @@ -19,24 +19,38 @@ %% ------------------------------------------------------------------- %% @doc Implements a set of functions for accessing and manipulating -%% an `rt_properties' record.-module(rt_properties). +%% an `rt_properties' record. -module(rt_properties). -include("rt.hrl"). -include_lib("eunit/include/eunit.hrl"). +%% A quick note on the distinction between `node_ids' and +%% `node_map'. `node_ids' are short identifers (e.g. dev1) and the +%% `node_map' maps each node_id to a full erlang node names. Both are +%% necessary because the different existing helper functions make use +%% of each one to either compose shell commands or execute rpc calls. +%% The `node_map' is used to make the details of the actual node +%% names, which are harness-specific, opaque to the helper functions +%% and ensure that the helpers are valid for use with any harness. -record(rt_properties_v1, { - nodes :: [node()], - node_count=6 :: non_neg_integer(), + node_ids :: [string()], + node_map :: [{string(), node()}], + node_count=3 :: non_neg_integer(), metadata=[] :: proplists:proplist(), properties=[] :: proplists:proplist(), rolling_upgrade=false :: boolean(), - start_version=current :: atom(), - upgrade_version=current :: atom(), + start_version="head" :: string(), + current_version :: string(), + upgrade_path :: [string()], wait_for_transfers=false :: boolean(), valid_backends=all :: all | [atom()], make_cluster=true :: boolean(), + cluster_count=1 :: pos_integer(), + cluster_weights :: [float()], + clusters :: proplists:proplist(), + required_services=[riak_kv] :: [atom()], config=default_config() :: term() }). -type properties() :: #rt_properties_v1{}. @@ -81,7 +95,7 @@ get(Property, Properties) -> %% or if any of the properties to be set are not a valid property. In %% the case that invalid properties are specified the error returned %% contains a list of erroneous properties. --spec set([{atom(), term()}], properties()) -> {ok, properties()} | {error, atom()}. +-spec set([{atom(), term()}], properties()) -> properties() | {error, atom()}. set(PropertyList, Properties) when is_list(PropertyList) -> set_properties(PropertyList, Properties, validate_record(Properties)). @@ -178,8 +192,10 @@ is_valid_property(Property) -> lists:member(Property, Fields). -spec field_index(atom()) -> non_neg_integer(). -field_index(nodes) -> - ?RT_PROPERTIES.nodes; +field_index(node_ids) -> + ?RT_PROPERTIES.node_ids; +field_index(node_map) -> + ?RT_PROPERTIES.node_map; field_index(node_count) -> ?RT_PROPERTIES.node_count; field_index(metadata) -> @@ -190,13 +206,23 @@ field_index(rolling_upgrade) -> ?RT_PROPERTIES.rolling_upgrade; field_index(start_version) -> ?RT_PROPERTIES.start_version; -field_index(upgrade_version) -> - ?RT_PROPERTIES.upgrade_version; +field_index(current_version) -> + ?RT_PROPERTIES.current_version; +field_index(upgrade_path) -> + ?RT_PROPERTIES.upgrade_path; field_index(wait_for_transfers) -> ?RT_PROPERTIES.wait_for_transfers; field_index(valid_backends) -> ?RT_PROPERTIES.valid_backends; field_index(make_cluster) -> ?RT_PROPERTIES.make_cluster; +field_index(cluster_count) -> + ?RT_PROPERTIES.cluster_count; +field_index(cluster_weights) -> + ?RT_PROPERTIES.cluster_weights; +field_index(clusters) -> + ?RT_PROPERTIES.clusters; +field_index(required_services) -> + ?RT_PROPERTIES.required_services; field_index(config) -> ?RT_PROPERTIES.config. diff --git a/src/rtdev.erl b/src/rtdev.erl index 49771c594..2d4a6e9e9 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -19,16 +19,18 @@ %% ------------------------------------------------------------------- -module(rtdev). --behaviour(test_harness). --export([start/1, - stop/1, +%% -behaviour(test_harness). +-export([start/2, + stop/2, deploy_clusters/1, clean_data_dir/2, + clean_data_dir/3, spawn_cmd/1, spawn_cmd/2, cmd/1, cmd/2, - setup_harness/2, + setup_harness/0, + %% setup_harness/2, get_version/0, get_backends/0, set_backend/1, @@ -41,27 +43,29 @@ attach/2, attach_direct/2, console/2, - update_app_config/2, + update_app_config/3, teardown/0, set_conf/2, - set_advanced_conf/2]). + set_advanced_conf/2, + rm_dir/1]). -compile(export_all). -include_lib("eunit/include/eunit.hrl"). --define(DEVS(N), lists:concat(["dev", N, "@127.0.0.1"])). +-define(DEVS(N), lists:concat([N, "@127.0.0.1"])). -define(DEV(N), list_to_atom(?DEVS(N))). --define(PATH, (rt_config:get(rtdev_path))). +-define(PATH, (rt_config:get(root_path))). +-define(SCRATCH_DIR, (rt_config:get(rt_scratch_dir))). get_deps() -> - lists:flatten(io_lib:format("~s/dev/dev1/lib", [relpath(current)])). + lists:flatten(io_lib:format("~s/dev1/lib", [filename:join(?PATH, "head")])). riakcmd(Path, N, Cmd) -> ExecName = rt_config:get(exec_name, "riak"), - io_lib:format("~s/dev/dev~b/bin/~s ~s", [Path, N, ExecName, Cmd]). + io_lib:format("~s/~s/bin/~s ~s", [Path, N, ExecName, Cmd]). riakreplcmd(Path, N, Cmd) -> - io_lib:format("~s/dev/dev~b/bin/riak-repl ~s", [Path, N, Cmd]). + io_lib:format("~s/~s/bin/riak-repl ~s", [Path, N, Cmd]). gitcmd(Path, Cmd) -> io_lib:format("git --git-dir=\"~s/.git\" --work-tree=\"~s/\" ~s", @@ -76,34 +80,35 @@ riak_admin_cmd(Path, N, Args) -> end, Args), ArgStr = string:join(Quoted, " "), ExecName = rt_config:get(exec_name, "riak"), - io_lib:format("~s/dev/dev~b/bin/~s-admin ~s", [Path, N, ExecName, ArgStr]). + io_lib:format("~s/~s/bin/~s-admin ~s", [Path, N, ExecName, ArgStr]). run_git(Path, Cmd) -> lager:info("Running: ~s", [gitcmd(Path, Cmd)]), {0, Out} = cmd(gitcmd(Path, Cmd)), Out. -run_riak(N, Path, Cmd) -> - lager:info("Running: ~s", [riakcmd(Path, N, Cmd)]), - R = os:cmd(riakcmd(Path, N, Cmd)), - case Cmd of - "start" -> - rt_cover:maybe_start_on_node(?DEV(N), node_version(N)), - %% Intercepts may load code on top of the cover compiled - %% modules. We'll just get no coverage info then. - case rt_intercept:are_intercepts_loaded(?DEV(N)) of - false -> - ok = rt_intercept:load_intercepts([?DEV(N)]); - true -> - ok - end, - R; - "stop" -> - rt_cover:maybe_stop_on_node(?DEV(N)), - R; - _ -> - R - end. +run_riak(Node, Version, "start") -> + VersionPath = filename:join(?PATH, Version), + RiakCmd = riakcmd(VersionPath, Node, "start"), + lager:info("Running: ~s", [RiakCmd]), + CmdRes = os:cmd(RiakCmd), + %% rt_cover:maybe_start_on_node(?DEV(Node), Version), + %% Intercepts may load code on top of the cover compiled + %% modules. We'll just get no coverage info then. + case rt_intercept:are_intercepts_loaded(?DEV(Node)) of + false -> + ok = rt_intercept:load_intercepts([?DEV(Node)]); + true -> + ok + end, + CmdRes; +run_riak(Node, Version, "stop") -> + VersionPath = filename:join(?PATH, Version), + %% rt_cover:maybe_stop_on_node(?DEV(Node)), + os:cmd(riakcmd(VersionPath, Node, "stop")); +run_riak(Node, Version, Cmd) -> + VersionPath = filename:join(?PATH, Version), + os:cmd(riakcmd(VersionPath, Node, Cmd)). run_riak_repl(N, Path, Cmd) -> lager:info("Running: ~s", [riakcmd(Path, N, Cmd)]), @@ -111,32 +116,76 @@ run_riak_repl(N, Path, Cmd) -> %% don't mess with intercepts and/or coverage, %% they should already be setup at this point -setup_harness(_Test, _Args) -> - %% make sure we stop any cover processes on any nodes - %% otherwise, if the next test boots a legacy node we'll end up with cover +-spec versions() -> [string()]. +versions() -> + RootPath = ?PATH, + case file:list_dir(RootPath) of + {ok, RootFiles} -> + [Version || Version <- RootFiles, + filelib:is_dir(filename:join(RootPath, Version)), + hd(Version) =/= $.]; + {error, _} -> + [] + end. + +-spec harness_node_ids(string()) -> [string()]. +harness_node_ids(Version) -> + VersionPath = filename:join(?PATH, Version), + case file:list_dir(VersionPath) of + {ok, VersionFiles} -> + SortedVersionFiles = lists:sort(VersionFiles), + [Node || Node <- SortedVersionFiles, + filelib:is_dir(filename:join(VersionPath, Node))]; + {error, _} -> + [] + end. + +-spec harness_nodes([string()]) -> [atom()]. +harness_nodes(NodeIds) -> + [list_to_atom(NodeId ++ "@127.0.0.1") || NodeId <- NodeIds]. + +so_fresh_so_clean(VersionMap) -> + %% make sure we stop any cover processes on any nodes otherwise, + %% if the next test boots a legacy node we'll end up with cover %% incompatabilities and crash the cover server - rt_cover:maybe_stop_on_nodes(), - Path = relpath(root), - %% Stop all discoverable nodes, not just nodes we'll be using for this test. - rt:pmap(fun(X) -> stop_all(X ++ "/dev") end, devpaths()), + %% rt_cover:maybe_stop_on_nodes(), + %% Path = relpath(root), + %% Stop all discoverable nodes, not just nodes we'll be using for + %% this test. + StopAllFun = + fun({Version, VersionNodes}) -> + VersionPath = filename:join([?PATH, Version]), + stop_nodes(VersionPath, VersionNodes) + end, + rt:pmap(StopAllFun, VersionMap), %% Reset nodes to base state lager:info("Resetting nodes to fresh state"), - _ = run_git(Path, "reset HEAD --hard"), - _ = run_git(Path, "clean -fd"), + _ = run_git(?PATH, "reset HEAD --hard"), + _ = run_git(?PATH, "clean -fd"), lager:info("Cleaning up lingering pipe directories"), - rt:pmap(fun(Dir) -> + rt:pmap(fun({Version, _}) -> %% when joining two absolute paths, filename:join intentionally %% throws away the first one. ++ gets us around that, while %% keeping some of the security of filename:join. %% the extra slashes will be pruned by filename:join, but this %% ensures that there will be at least one between "/tmp" and Dir - PipeDir = filename:join(["/tmp//" ++ Dir, "dev"]), + %% TODO: Double check this is correct + PipeDir = filename:join("/tmp" ++ ?PATH, Version), {0, _} = cmd("rm -rf " ++ PipeDir) - end, devpaths()), + end, VersionMap), ok. +setup_harness() -> + %% Get node names and populate node map + VersionMap = [{Version, harness_node_ids(Version)} || Version <- versions()], + NodeIds = harness_node_ids(rt_config:get(default_version, "head")), + NodeMap = lists:zip(NodeIds, harness_nodes(NodeIds)), + so_fresh_so_clean(VersionMap), + rm_dir(filename:join(?SCRATCH_DIR, "gc")), + rt_harness_util:setup_harness(VersionMap, NodeIds, NodeMap). + relpath(Vsn) -> Path = ?PATH, relpath(Vsn, Path). @@ -150,37 +199,39 @@ relpath(root, Path) -> relpath(_, _) -> throw("Version requested but only one path provided"). -upgrade(Node, NewVersion) -> - upgrade(Node, NewVersion, same). +%% upgrade(Node, CurrentVersion, NewVersion) -> +%% upgrade(Node, CurrentVersion, NewVersion, same). -upgrade(Node, NewVersion, Config) -> - N = node_id(Node), - Version = node_version(N), - lager:info("Upgrading ~p : ~p -> ~p", [Node, Version, NewVersion]), - stop(Node), +upgrade(Node, CurrentVersion, NewVersion, Config) -> + lager:info("Upgrading ~p : ~p -> ~p", [Node, CurrentVersion, NewVersion]), + stop(Node, CurrentVersion), rt:wait_until_unpingable(Node), - OldPath = relpath(Version), - NewPath = relpath(NewVersion), - + CurrentPath = filename:join([?PATH, CurrentVersion, Node]), + NewPath = filename:join([?PATH, NewVersion, Node]), Commands = [ - io_lib:format("cp -p -P -R \"~s/dev/dev~b/data\" \"~s/dev/dev~b\"", - [OldPath, N, NewPath, N]), - io_lib:format("rm -rf ~s/dev/dev~b/data/*", - [OldPath, N]), - io_lib:format("cp -p -P -R \"~s/dev/dev~b/etc\" \"~s/dev/dev~b\"", - [OldPath, N, NewPath, N]) + io_lib:format("cp -p -P -R \"~s\" \"~s\"", + [filename:join(CurrentPath, "data"), + NewPath]), + %% io_lib:format("rm -rf ~s/dev/dev~b/data/*", + %% [CurrePath, N]), + io_lib:format("cp -p -P -R \"~s\" \"~s\"", + [filename:join(CurrentPath, "etc"), + NewPath]) ], - [ begin - lager:info("Running: ~s", [Cmd]), - os:cmd(Cmd) - end || Cmd <- Commands], - VersionMap = orddict:store(N, NewVersion, rt_config:get(rt_versions)), - rt_config:set(rt_versions, VersionMap), + [begin + lager:debug("Running: ~s", [Cmd]), + os:cmd(Cmd) + end || Cmd <- Commands], + clean_data_dir(Node, CurrentVersion, ""), + + %% VersionMap = orddict:store(N, NewVersion, rt_config:get(rt_versions)), + %% rt_config:set(rt_versions, VersionMap), + case Config of same -> ok; - _ -> update_app_config(Node, Config) + _ -> update_app_config(Node, NewVersion, Config) end, - start(Node), + start(Node, NewVersion), rt:wait_until_pingable(Node), ok. @@ -267,29 +318,28 @@ all_the_app_configs(DevPath) -> AppConfigs end. -update_app_config(all, Config) -> - lager:info("rtdev:update_app_config(all, ~p)", [Config]), - [ update_app_config(DevPath, Config) || DevPath <- devpaths()]; -update_app_config(Node, Config) when is_atom(Node) -> - N = node_id(Node), - Path = relpath(node_version(N)), - FileFormatString = "~s/dev/dev~b/etc/~s.config", +%% update_app_config(all, Config) -> +%% lager:info("rtdev:update_app_config(all, ~p)", [Config]), +%% [ update_app_config(DevPath, Config) || DevPath <- devpaths()]; +update_app_config(Node, Version, Config) -> + VersionPath = filename:join(?PATH, Version), + FileFormatString = "~s/~s/etc/~s.config", + AppConfigFile = io_lib:format(FileFormatString, + [VersionPath, Node, "app"]), + AdvConfigFile = io_lib:format(FileFormatString, + [VersionPath, Node, "advanced"]), - AppConfigFile = io_lib:format(FileFormatString, [Path, N, "app"]), - AdvConfigFile = io_lib:format(FileFormatString, [Path, N, "advanced"]), %% If there's an app.config, do it old style - %% if not, use cuttlefish's adavnced.config + %% if not, use cuttlefish's advanced.config case filelib:is_file(AppConfigFile) of true -> update_app_config_file(AppConfigFile, Config); _ -> update_app_config_file(AdvConfigFile, Config) - end; -update_app_config(DevPath, Config) -> - [update_app_config_file(AppConfig, Config) || AppConfig <- all_the_app_configs(DevPath)]. + end. update_app_config_file(ConfigFile, Config) -> - lager:info("rtdev:update_app_config_file(~s, ~p)", [ConfigFile, Config]), + lager:debug("rtdev:update_app_config_file(~s, ~p)", [ConfigFile, Config]), BaseConfig = case file:consult(ConfigFile) of {ok, [ValidConfig]} -> @@ -379,8 +429,40 @@ clean_data_dir(Nodes, SubDir) when is_list(Nodes) -> DataDirs = [node_path(Node) ++ "/data/" ++ SubDir || Node <- Nodes], lists:foreach(fun rm_dir/1, DataDirs). +%% Blocking to delete files is not the best use of time. Generally it +%% is quicker to move directories than to recursively delete them so +%% move the directory to a GC subdirectory in the riak_test scratch +%% directory, recreate the subdirectory, and asynchronously remove the +%% files from the scratch directory. +clean_data_dir(Node, Version, "") -> + DataDir = filename:join([?PATH, Version, Node, "data"]), + TmpDir = filename:join([?SCRATCH_DIR, "gc", Version, Node]), + filelib:ensure_dir(filename:join(TmpDir, "child")), + mv_dir(DataDir, TmpDir), + Pid = spawn(?MODULE, rm_dir, [TmpDir]), + mk_dir(DataDir), + Pid; +clean_data_dir(Node, Version, SubDir) -> + DataDir = filename:join([?PATH, Version, Node, "data", SubDir]), + TmpDir = filename:join([?SCRATCH_DIR, "gc", Version, Node, "data"]), + filelib:ensure_dir(filename:join(TmpDir, "child")), + mv_dir(DataDir, TmpDir), + Pid = spawn(?MODULE, rm_dir, [TmpDir]), + mk_dir(DataDir), + Pid. + +mk_dir(Dir) -> + lager:debug("Making directory ~s", [Dir]), + ?assertCmd("mkdir " ++ Dir), + ?assertEqual(true, filelib:is_dir(Dir)). + +mv_dir(Src, Dest) -> + lager:debug("Moving directory ~s to ~s", [Src, Dest]), + ?assertCmd("mv " ++ Src ++ " " ++ Dest), + ?assertEqual(false, filelib:is_dir(Src)). + rm_dir(Dir) -> - lager:info("Removing directory ~s", [Dir]), + lager:debug("Removing directory ~s", [Dir]), ?assertCmd("rm -rf " ++ Dir), ?assertEqual(false, filelib:is_dir(Dir)). @@ -389,7 +471,7 @@ add_default_node_config(Nodes) -> undefined -> ok; Defaults when is_list(Defaults) -> rt:pmap(fun(Node) -> - update_app_config(Node, Defaults) + update_app_config(Node, version_here, Defaults) end, Nodes), ok; BadValue -> @@ -397,40 +479,25 @@ add_default_node_config(Nodes) -> throw({invalid_config, {rt_default_config, BadValue}}) end. -deploy_clusters(ClusterConfigs) -> - NumNodes = rt_config:get(num_nodes, 6), - RequestedNodes = lists:flatten(ClusterConfigs), - - case length(RequestedNodes) > NumNodes of - true -> - erlang:error("Requested more nodes than available"); - false -> - Nodes = deploy_nodes(RequestedNodes), - {DeployedClusters, _} = lists:foldl( - fun(Cluster, {Clusters, RemNodes}) -> - {A, B} = lists:split(length(Cluster), RemNodes), - {Clusters ++ [A], B} - end, {[], Nodes}, ClusterConfigs), - DeployedClusters - end. - -deploy_nodes(NodeConfig) -> - Path = relpath(root), - lager:info("Riak path: ~p", [Path]), - NumNodes = length(NodeConfig), - NodesN = lists:seq(1, NumNodes), - Nodes = [?DEV(N) || N <- NodesN], - NodeMap = orddict:from_list(lists:zip(Nodes, NodesN)), - {Versions, Configs} = lists:unzip(NodeConfig), - VersionMap = lists:zip(NodesN, Versions), - - %% Check that you have the right versions available - [ check_node(Version) || Version <- VersionMap ], - rt_config:set(rt_nodes, NodeMap), - rt_config:set(rt_versions, VersionMap), - - create_dirs(Nodes), - +deploy_clusters(_ClusterConfigs) -> + ok. +%% NumNodes = rt_config:get(num_nodes, 6), +%% RequestedNodes = lists:flatten(ClusterConfigs), + +%% case length(RequestedNodes) > NumNodes of +%% true -> +%% erlang:error("Requested more nodes than available"); +%% false -> +%% Nodes = deploy_nodes(RequestedNodes), +%% {DeployedClusters, _} = lists:foldl( +%% fun(Cluster, {Clusters, RemNodes}) -> +%% {A, B} = lists:split(length(Cluster), RemNodes), +%% {Clusters ++ [A], B} +%% end, {[], Nodes}, ClusterConfigs), +%% DeployedClusters +%% end. + +configure_nodes(Nodes, Configs) -> %% Set initial config add_default_node_config(Nodes), rt:pmap(fun({_, default}) -> @@ -438,49 +505,80 @@ deploy_nodes(NodeConfig) -> ({Node, {cuttlefish, Config}}) -> set_conf(Node, Config); ({Node, Config}) -> - update_app_config(Node, Config) + update_app_config(Node, version_here, Config) end, - lists:zip(Nodes, Configs)), - - %% create snmp dirs, for EE - create_dirs(Nodes), - - %% Start nodes - %%[run_riak(N, relpath(node_version(N)), "start") || N <- Nodes], - rt:pmap(fun(N) -> run_riak(N, relpath(node_version(N)), "start") end, NodesN), - - %% Ensure nodes started - [ok = rt:wait_until_pingable(N) || N <- Nodes], - - %% %% Enable debug logging - %% [rpc:call(N, lager, set_loglevel, [lager_console_backend, debug]) || N <- Nodes], - - %% We have to make sure that riak_core_ring_manager is running before we can go on. - [ok = rt:wait_until_registered(N, riak_core_ring_manager) || N <- Nodes], - - %% Ensure nodes are singleton clusters - [ok = rt_ring:check_singleton_node(?DEV(N)) || {N, Version} <- VersionMap, - Version /= "0.14.2"], - - lager:info("Deployed nodes: ~p", [Nodes]), - Nodes. - -gen_stop_fun(Timeout) -> - fun({C,Node}) -> - net_kernel:hidden_connect_node(Node), - case rpc:call(Node, os, getpid, []) of + lists:zip(Nodes, Configs)). + +%% deploy_nodes(NodeConfig) -> +%% Path = relpath(root), +%% lager:info("Riak path: ~p", [Path]), +%% NumNodes = length(NodeConfig), +%% %% TODO: The starting index should not be fixed to 1 +%% NodesN = lists:seq(1, NumNodes), +%% Nodes = [?DEV(N) || N <- NodesN], +%% NodeMap = orddict:from_list(lists:zip(Nodes, NodesN)), +%% {Versions, Configs} = lists:unzip(NodeConfig), +%% VersionMap = lists:zip(NodesN, Versions), + +%% %% Check that you have the right versions available +%% [ check_node(Version) || Version <- VersionMap ], +%% rt_config:set(rt_nodes, NodeMap), +%% rt_config:set(rt_versions, VersionMap), + +%% create_dirs(Nodes), + +%% %% Set initial config +%% add_default_node_config(Nodes), +%% rt:pmap(fun({_, default}) -> +%% ok; +%% ({Node, {cuttlefish, Config}}) -> +%% set_conf(Node, Config); +%% ({Node, Config}) -> +%% update_app_config(Node, Config) +%% end, +%% lists:zip(Nodes, Configs)), + +%% %% create snmp dirs, for EE +%% create_dirs(Nodes), + +%% %% Start nodes +%% %%[run_riak(N, relpath(node_version(N)), "start") || N <- Nodes], +%% rt:pmap(fun(N) -> run_riak(N, relpath(node_version(N)), "start") end, NodesN), + +%% %% Ensure nodes started +%% [ok = rt:wait_until_pingable(N) || N <- Nodes], + +%% %% %% Enable debug logging +%% %% [rpc:call(N, lager, set_loglevel, [lager_console_backend, debug]) || N <- Nodes], + +%% %% We have to make sure that riak_core_ring_manager is running before we can go on. +%% [ok = rt:wait_until_registered(N, riak_core_ring_manager) || N <- Nodes], + +%% %% Ensure nodes are singleton clusters +%% [ok = rt_ring:check_singleton_node(?DEV(N)) || {N, Version} <- VersionMap, +%% Version /= "0.14.2"], + +%% lager:info("Deployed nodes: ~p", [Nodes]), +%% Nodes. + +gen_stop_fun(Path, Timeout) -> + fun(Node) -> + NodeName = ?DEV(Node), + NodePath = filename:join(Path, Node), + net_kernel:hidden_connect_node(NodeName), + case rpc:call(NodeName, os, getpid, []) of PidStr when is_list(PidStr) -> - lager:info("Preparing to stop node ~p (process ID ~s) with init:stop/0...", - [Node, PidStr]), - rpc:call(Node, init, stop, []), + lager:debug("Preparing to stop node ~p (process ID ~s) with init:stop/0...", + [NodePath, PidStr]), + rpc:call(NodeName, init, stop, []), %% If init:stop/0 fails here, the wait_for_pid/2 call %% below will timeout and the process will get cleaned %% up by the kill_stragglers/2 function wait_for_pid(PidStr, Timeout); BadRpc -> - Cmd = C ++ "/bin/riak stop", - lager:info("RPC to node ~p returned ~p, will try stop anyway... ~s", - [Node, BadRpc, Cmd]), + Cmd = filename:join([Path, Node, "bin/riak stop"]), + lager:debug("RPC to node ~p returned ~p, will try stop anyway... ~s", + [NodeName, BadRpc, Cmd]), Output = os:cmd(Cmd), Status = case Output of "ok\n" -> @@ -495,12 +593,12 @@ gen_stop_fun(Timeout) -> _ -> "wasn't running" end, - lager:info("Stopped node ~p, stop status: ~s.", [Node, Status]) + lager:debug("Stopped node ~p, stop status: ~s.", [NodePath, Status]) end end. -kill_stragglers(DevPath, Timeout) -> - {ok, Re} = re:compile("^\\s*\\S+\\s+(\\d+).+\\d+\\s+"++DevPath++"\\S+/beam"), +kill_stragglers(Path, Timeout) -> + {ok, Re} = re:compile("^\\s*\\S+\\s+(\\d+).+\\d+\\s+"++Path++"\\S+/beam"), ReOpts = [{capture,all_but_first,list}], Pids = tl(string:tokens(os:cmd("ps -ef"), "\n")), Fold = fun(Proc, Acc) -> @@ -508,13 +606,13 @@ kill_stragglers(DevPath, Timeout) -> nomatch -> Acc; {match,[Pid]} -> - lager:info("Process ~s still running, killing...", + lager:debug("Process ~s still running, killing...", [Pid]), os:cmd("kill -15 "++Pid), case wait_for_pid(Pid, Timeout) of ok -> ok; fail -> - lager:info("Process ~s still hasn't stopped, " + lager:debug("Process ~s still hasn't stopped, " "resorting to kill -9...", [Pid]), os:cmd("kill -9 "++Pid) end, @@ -533,45 +631,41 @@ wait_for_pid(PidStr, Timeout) -> _ -> ok end. -stop_all(DevPath) -> - case filelib:is_dir(DevPath) of - true -> - Devs = filelib:wildcard(DevPath ++ "/dev*"), - Nodes = [?DEV(N) || N <- lists:seq(1, length(Devs))], - MyNode = 'riak_test@127.0.0.1', - case net_kernel:start([MyNode, longnames]) of - {ok, _} -> - true = erlang:set_cookie(MyNode, riak); - {error,{already_started,_}} -> - ok - end, - lager:info("Trying to obtain node shutdown_time via RPC..."), - Tmout = case rpc:call(hd(Nodes), init, get_argument, [shutdown_time]) of - {ok,[[Tm]]} -> list_to_integer(Tm)+10000; - _ -> 20000 - end, - lager:info("Using node shutdown_time of ~w", [Tmout]), - rt:pmap(gen_stop_fun(Tmout), lists:zip(Devs, Nodes)), - kill_stragglers(DevPath, Tmout); - _ -> - lager:info("~s is not a directory.", [DevPath]) +stop_nodes(Path, Nodes) -> + MyNode = 'riak_test@127.0.0.1', + case net_kernel:start([MyNode, longnames]) of + {ok, _} -> + true = erlang:set_cookie(MyNode, riak); + {error,{already_started,_}} -> + ok end, + lager:debug("Trying to obtain node shutdown_time via RPC..."), + Tmout = case rpc:call(?DEV(hd(Nodes)), init, get_argument, [shutdown_time]) of + {ok,[[Tm]]} -> list_to_integer(Tm)+10000; + _ -> 20000 + end, + lager:debug("Using node shutdown_time of ~w", [Tmout]), + rt:pmap(gen_stop_fun(Path, Tmout), Nodes), + kill_stragglers(Path, Tmout), ok. -stop(Node) -> - RiakPid = rpc:call(Node, os, getpid, []), - N = node_id(Node), - rt_cover:maybe_stop_on_node(Node), - run_riak(N, relpath(node_version(N)), "stop"), - F = fun(_N) -> - os:cmd("kill -0 " ++ RiakPid) =/= [] - end, - ?assertEqual(ok, rt:wait_until(Node, F)), - ok. +stop(Node, Version) -> + case rpc:call(?DEV(Node), os, getpid, []) of + {badrpc, nodedown} -> + ok; + RiakPid -> + %% rt_cover:maybe_stop_on_node(Node), + run_riak(Node, Version, "stop"), + F = fun(_N) -> + os:cmd("kill -0 " ++ RiakPid) =/= [] + end, + ?assertEqual(ok, rt:wait_until(?DEV(Node), F)), + ok + end. -start(Node) -> - N = node_id(Node), - run_riak(N, relpath(node_version(N)), "start"), +start(Node, Version) -> + %% N = node_id(Node), + run_riak(Node, Version, "start"), ok. attach(Node, Expected) -> @@ -587,7 +681,7 @@ interactive(Node, Command, Exp) -> N = node_id(Node), Path = relpath(node_version(N)), Cmd = riakcmd(Path, N, Command), - lager:info("Opening a port for riak ~s.", [Command]), + lager:debug("Opening a port for riak ~s.", [Command]), lager:debug("Calling open_port with cmd ~s", [binary_to_list(iolist_to_binary(Cmd))]), P = open_port({spawn, binary_to_list(iolist_to_binary(Cmd))}, [stream, use_stdio, exit_status, binary, stderr_to_stdout]), @@ -748,9 +842,9 @@ set_backend(Backend) -> set_backend(Backend, []). set_backend(Backend, OtherOpts) -> - lager:info("rtdev:set_backend(~p, ~p)", [Backend, OtherOpts]), + lager:debug("rtdev:set_backend(~p, ~p)", [Backend, OtherOpts]), Opts = [{storage_backend, Backend} | OtherOpts], - update_app_config(all, [{riak_kv, Opts}]), + update_app_config(all, version_here, [{riak_kv, Opts}]), get_backends(). get_version() -> @@ -760,9 +854,10 @@ get_version() -> end. teardown() -> - rt_cover:maybe_stop_on_nodes(), + %% rt_cover:maybe_stop_on_nodes(), %% Stop all discoverable nodes, not just nodes we'll be using for this test. - rt:pmap(fun(X) -> stop_all(X ++ "/dev") end, devpaths()). + %% rt:pmap(fun(X) -> stop_all(X ++ "/dev") end, devpaths()). + ok. whats_up() -> io:format("Here's what's running...~n"), @@ -773,8 +868,8 @@ whats_up() -> devpaths() -> lists:usort([ DevPath || {_Name, DevPath} <- proplists:delete(root, rt_config:get(rtdev_path))]). -versions() -> - proplists:get_keys(rt_config:get(rtdev_path)) -- [root]. +%% versions() -> +%% proplists:get_keys(rt_config:get(rtdev_path)) -- [root]. get_node_logs() -> Root = filename:absname(proplists:get_value(root, ?PATH)), diff --git a/src/rtssh.erl b/src/rtssh.erl index 3936c67db..03917e913 100644 --- a/src/rtssh.erl +++ b/src/rtssh.erl @@ -151,6 +151,22 @@ node_to_host(Node) -> throw(io_lib:format("rtssh:node_to_host couldn't figure out the host of ~p", [Node])) end. + +nodes(Count) -> + Hosts = rt_config:get(rtssh_hosts), + %% NumNodes = length(NodeConfig), + NodeConfig = busted_stuff, + NumNodes = Count, + NumHosts = length(Hosts), + case NumNodes > NumHosts of + true -> + erlang:error("Not enough hosts available to deploy nodes", + [NumNodes, NumHosts]); + false -> + Hosts2 = lists:sublist(Hosts, NumNodes), + deploy_nodes(NodeConfig, Hosts2) + end. + deploy_nodes(NodeConfig, Hosts) -> Path = relpath(root), lager:info("Riak path: ~p", [Path]), @@ -359,6 +375,7 @@ load_hosts() -> Hosts = lists:sort(HostsIn), rt_config:set(rtssh_hosts, Hosts), rt_config:set(rtssh_aliases, Aliases), + rt_config:set(rtssh_nodes, length(Hosts)), Hosts. read_hosts_file(File) -> @@ -716,10 +733,10 @@ whats_up() -> Up = [rpc:call(Node, os, cmd, ["pwd"]) || Node <- nodes()], [io:format(" ~s~n",[string:substr(Dir, 1, length(Dir)-1)]) || Dir <- Up]. -node_version(Node) -> +node_version(Node) -> rt_harness_util:node_version(Node). -attach(Node, Expected) -> +attach(Node, Expected) -> rt_harness_util:attach(Node, Expected). attach_direct(Node, Expected) -> diff --git a/src/test_harness.erl b/src/test_harness.erl index 11006372f..760c70202 100644 --- a/src/test_harness.erl +++ b/src/test_harness.erl @@ -17,18 +17,18 @@ %% under the License. %% %% ------------------------------------------------------------------- -%% @doc behaviour for all test harnesses. --module(test_harness). +%% @doc behaviour for all test harnesses. +-module(test_harness). --callback start(Node :: node()) -> 'ok'. --callback stop(Node :: node()) -> 'ok'. --callback deploy_clusters(ClusterConfigs :: list()) -> list(). +%% -callback start(Node :: node(), Version :: string()) -> 'ok'. +%% -callback stop(Node :: node()) -> 'ok'. +%% -callback deploy_clusters(ClusterConfigs :: list()) -> list(). -callback clean_data_dir(Nodes :: list(), SubDir :: string()) -> 'ok'. -callback spawn_cmd(Cmd :: string()) -> Port :: pos_integer(). -callback spawn_cmd(Cmd :: string(), Opts :: list()) -> Port :: pos_integer(). -callback cmd(Cmd :: string()) -> term()|timeout. -callback cmd(Cmd :: string(), Opts :: [atom()]) -> term()|timeout. --callback setup_harness(Test :: string(), Args :: list()) -> 'ok'. +%% -callback setup_harness(Test :: string(), Args :: list()) -> 'ok'. -callback get_version() -> term(). -callback get_backends() -> [atom()]. -callback set_backend(Backend :: atom()) -> [atom()]. @@ -41,9 +41,7 @@ -callback attach(Node :: node(), Expected:: list()) -> 'ok'. -callback attach_direct(Node :: node(), Expected:: list()) -> 'ok'. -callback console(Node :: node(), Expected:: list()) -> 'ok'. --callback update_app_config(atom()|node(), Config :: term()) -> 'ok'. +%% -callback update_app_config(atom()|node(), Config :: term()) -> 'ok'. -callback teardown() -> list(). -callback set_conf(atom()|node(), NameValuePairs :: [{string(), string()}]) -> 'ok'. -callback set_advanced_conf(atom()|node(), NameValuePairs :: [{string(), string()}]) -> 'ok'. - - diff --git a/tests/always_fail_test.erl b/tests/always_fail_test.erl index 39d6e341a..6734ec284 100644 --- a/tests/always_fail_test.erl +++ b/tests/always_fail_test.erl @@ -1,7 +1,8 @@ %% @doc A test that always returns `fail'. -module(always_fail_test). --export([confirm/0]). +-export([confirm/2]). --spec confirm() -> pass | fail. -confirm() -> +-spec confirm(rt_properties:properties(), proplists:proplist()) -> pass | fail. +confirm(_Properties, _MD) -> + lager:info("Running test confirm function"), fail. diff --git a/tests/always_pass_test.erl b/tests/always_pass_test.erl index e71c9645c..8b7a3aaa8 100644 --- a/tests/always_pass_test.erl +++ b/tests/always_pass_test.erl @@ -1,8 +1,15 @@ -%% @doc A test that always returns `pass'. +%% @doc A test that always returns `fail'. -module(always_pass_test). --behavior(riak_test). --export([confirm/0]). +-export([properties/0, + confirm/2]). --spec confirm() -> pass | fail. -confirm() -> +-include_lib("eunit/include/eunit.hrl"). + +properties() -> + rt_properties:new([{make_cluster, false}]). + +-spec confirm(rt_properties:properties(), proplists:proplist()) -> pass | fail. +confirm(_Properties, _MD) -> + lager:info("Running test confirm function"), + ?assertEqual(1,1), pass. diff --git a/tests/secondary_index_tests.erl b/tests/secondary_index_tests.erl index df58b8f41..0b9b99de5 100644 --- a/tests/secondary_index_tests.erl +++ b/tests/secondary_index_tests.erl @@ -33,18 +33,24 @@ -define(KEYS(A,B,C), [int_to_key(N) || N <- lists:seq(A,B), C]). -define(KEYS(A,B,G1,G2), [int_to_key(N) || N <- lists:seq(A,B), G1, G2]). +-define(DEVS(N), lists:concat([N, "@127.0.0.1"])). +-define(DEV(N), list_to_atom(?DEVS(N))). + properties() -> rt_properties:new([{node_count, 3}, {wait_for_transfers, true}, - {start_version, previous}, + {valid_backends, [eleveldb, memory]}, {config, config()}]). config() -> [{riak_kv, [{secondary_index_sort_default, false}]}, {riak_core, [{handoff_concurrency, 11}]}]. +-spec confirm(rt_properties:properties(), proplists:proplist()) -> pass | fail. confirm(Properties, _MD) -> - Nodes = rt_properties:get(nodes, Properties), + NodeMap= rt_properties:get(node_map, Properties), + Nodes = [rt_node:node_name(NodeId, NodeMap) + || NodeId <- rt_properties:get(node_ids, Properties)], Bucket = druuid:v4_str(), lager:info("Bucket: ~p", [Bucket]), PBC = rt_pb:pbc(hd(Nodes)), @@ -110,7 +116,7 @@ confirm(Properties, _MD) -> <<"field2_int">>, 1000000000000, TestIdxVal), - + rt_pb:stop(PBC), pass. assertExactQuery(Clients, Bucket, Expected, Index, Value) -> diff --git a/tests/verify_listkeys.erl b/tests/verify_listkeys.erl index 8ef477eb9..91c5efa1d 100644 --- a/tests/verify_listkeys.erl +++ b/tests/verify_listkeys.erl @@ -18,8 +18,9 @@ %% %% ------------------------------------------------------------------- -module(verify_listkeys). --behavior(riak_test). --export([confirm/0]). +%% -behavior(riak_test). +-export([properties/0, + confirm/2]). -include_lib("eunit/include/eunit.hrl"). -define(BUCKET, <<"listkeys_bucket">>). @@ -28,11 +29,17 @@ -define(UNDEFINED_BUCKET, <<"880bf69d-5dab-44ee-8762-d24c6f759ce1">>). -define(UNDEFINED_BUCKET_TYPE, <<"880bf69d-5dab-44ee-8762-d24c6f759ce1">>). -confirm() -> - [Node1, Node2, Node3, Node4] = Nodes = rt_cluster:deploy_nodes(4), - ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes)), +properties() -> + rt_properties:new([{node_count, 4}, + {make_cluster, false}]). - lager:info("Nodes deployed, but not joined."), +-spec confirm(rt_properties:properties(), proplists:proplist()) -> pass | fail. +confirm(Properties, _MD) -> + [NodeId1, NodeId2, NodeId3 | _] = NodeIds = rt_properties:get(node_ids, Properties), + NodeMap = rt_properties:get(node_map, Properties), + Nodes = [rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds], + + [Node1, Node2, Node3 | _] = Nodes, lager:info("Writing some known data to Node 1"), put_keys(Node1, ?BUCKET, ?NUM_KEYS), @@ -40,49 +47,29 @@ confirm() -> timer:sleep(2000), check_it_all([Node1]), - lists:foldl(fun(Node, [N1|_] = Cluster) -> - lager:info("An invitation to this party is cordially extended to ~p.", [Node]), - rt_node:join(Node, N1), - lager:info("Wait until there are no pending changes"), - Ns = lists:usort([Node|Cluster]), - rt:wait_until_no_pending_changes(Ns), - rt:wait_for_cluster_service(Ns, riak_kv), - ok = rt:wait_until_transfers_complete(Ns), - lager:info("Check keys and buckets after transfer"), - check_it_all(Ns), - Ns - end, [Node1], [Node2, Node3, Node4]), + lists:foldl(fun growing_cluster_check/2, [Node1], tl(Nodes)), lager:info("Checking basic HTTP"), check_it_all(Nodes, http), + Version = rt_properties:get(current_version, Properties), + lager:info("Stopping Node1"), - rt_node:stop(Node1), + rt_node:stop(NodeId1, Version), + rt:wait_until_unpingable(Node1), %% Stop current node, restart previous node, verify - lists:foldl(fun(Node, Prev) -> - lager:info("Stopping Node ~p", [Node]), - rt_node:stop(Node), - rt:wait_until_unpingable(Node), - - lager:info("Starting Node ~p", [Prev]), - rt_node:start(Prev), - UpNodes = Nodes -- [Node], - lager:info("Waiting for riak_kv service to be ready in ~p", [Prev]), - rt:wait_for_cluster_service(UpNodes, riak_kv), - - lager:info("Check keys and buckets"), - check_it_all(UpNodes), - Node - end, Node1, [Node2, Node3, Node4]), + lists:foldl(fun shrinking_cluster_check/2, + {{hd(NodeIds), Node1}, Nodes, Version}, + lists:zip(tl(NodeIds), tl(Nodes))), lager:info("Stopping Node2"), - rt_node:stop(Node2), + rt_node:stop(NodeId2, Version), rt:wait_until_unpingable(Node2), lager:info("Stopping Node3"), - rt_node:stop(Node3), + rt_node:stop(NodeId3, Version), rt:wait_until_unpingable(Node3), lager:info("Only Node1 is up, so test should fail!"), @@ -90,6 +77,33 @@ confirm() -> check_it_all([Node1], pbc, false), pass. +growing_cluster_check(JoiningNode, [ClusterNode|_] = Cluster) -> + lager:info("An invitation to this party is cordially extended to ~p.", [JoiningNode]), + rt_node:join(JoiningNode, ClusterNode), + lager:info("Wait until there are no pending changes"), + Ns = lists:usort([JoiningNode|Cluster]), + rt:wait_until_no_pending_changes(Ns), + rt:wait_for_cluster_service(Ns, riak_kv), + ok = rt:wait_until_transfers_complete(Ns), + lager:info("Check keys and buckets after transfer"), + check_it_all(Ns), + Ns. + +shrinking_cluster_check({NodeId, Node}, {{PrevId, PrevNode}, Nodes, Version}) -> + lager:info("Stopping Node ~p", [Node]), + rt_node:stop(NodeId, Version), + rt:wait_until_unpingable(Node), + + lager:info("Starting Node ~p", [PrevId]), + rt_node:start(PrevId, Version), + UpNodes = Nodes -- [Node], + lager:info("Waiting for riak_kv service to be ready in ~p", [PrevNode]), + rt:wait_for_cluster_service(UpNodes, riak_kv), + + lager:info("Check keys and buckets"), + check_it_all(UpNodes), + {{NodeId, Node}, Nodes, Version}. + put_keys(Node, Bucket, Num) -> Pid = rt_pb:pbc(Node), Keys = [list_to_binary(["", integer_to_list(Ki)]) || Ki <- lists:seq(0, Num - 1)], @@ -138,11 +152,11 @@ list_keys_for_undefined_bucket_type(Node, Interface, Bucket, Attempt, ShouldPass lager:info("Listing keys using undefined bucket type ~p on ~p using ~p. Attempt #~p", [?UNDEFINED_BUCKET_TYPE, Node, Interface, Attempt]), case ShouldPass of - true -> ok; - _ -> - {Status, Message} = Mod:list_keys(Pid, { ?UNDEFINED_BUCKET_TYPE, Bucket }), - ?assertEqual(error, Status), - ?assertEqual(<<"No bucket-type named '880bf69d-5dab-44ee-8762-d24c6f759ce1'">>, Message) + true -> ok; + _ -> + {Status, Message} = Mod:list_keys(Pid, { ?UNDEFINED_BUCKET_TYPE, Bucket }), + ?assertEqual(error, Status), + ?assertEqual(<<"No bucket-type named '880bf69d-5dab-44ee-8762-d24c6f759ce1'">>, Message) end, case Interface of @@ -195,30 +209,30 @@ list_buckets(Node, Interface, Attempt, Num, ShouldPass) -> list_buckets_for_undefined_bucket_type(Node, Interface, Attempt, ShouldPass) -> case Interface of - pbc -> - Pid = rt_pb:pbc(Node), - Mod = riakc_pb_socket; - http -> - Pid = rt_http:httpc(Node), - Mod = rhc + pbc -> + Pid = rt_pb:pbc(Node), + Mod = riakc_pb_socket; + http -> + Pid = rt_http:httpc(Node), + Mod = rhc end, lager:info("Listing buckets on ~p for undefined bucket type ~p using ~p. Attempt ~p.", - [Node, ?UNDEFINED_BUCKET_TYPE, Interface, Attempt]), + [Node, ?UNDEFINED_BUCKET_TYPE, Interface, Attempt]), case ShouldPass of - true -> ok; - _ -> - {Status, Message} = Mod:list_buckets(Pid, ?UNDEFINED_BUCKET_TYPE, []), - lager:info("Received status ~p and message ~p", [Status, Message]), - ?assertEqual(error, Status), - ?assertEqual(<<"No bucket-type named '880bf69d-5dab-44ee-8762-d24c6f759ce1'">>, Message) + true -> ok; + _ -> + {Status, Message} = Mod:list_buckets(Pid, ?UNDEFINED_BUCKET_TYPE, []), + lager:info("Received status ~p and message ~p", [Status, Message]), + ?assertEqual(error, Status), + ?assertEqual(<<"No bucket-type named '880bf69d-5dab-44ee-8762-d24c6f759ce1'">>, Message) end, case Interface of - pbc -> - riakc_pb_socket:stop(Pid); - _ -> ok + pbc -> + riakc_pb_socket:stop(Pid); + _ -> ok end. assert_equal(Expected, Actual) -> @@ -244,7 +258,6 @@ check_a_node(Node, Interface, ShouldPass) -> [list_keys_for_undefined_bucket_type(Node, Interface, ?BUCKET, Attempt, ShouldPass) || Attempt <- [1,2,3] ], [list_buckets(Node, Interface, Attempt, ?NUM_BUCKETS, ShouldPass) - || Attempt <- [1,2,3] ], + || Attempt <- [1,2,3] ], [list_buckets_for_undefined_bucket_type(Node, Interface, Attempt, ShouldPass) || Attempt <- [1,2,3] ]. - diff --git a/tools.mk b/tools.mk index 445dd0cbc..639899436 100644 --- a/tools.mk +++ b/tools.mk @@ -1,6 +1,6 @@ # ------------------------------------------------------------------- # -# Copyright (c) 2014 Basho Technologies, Inc. +# Copyright (c) 2015 Basho Technologies, Inc. # # This file is provided to you under the Apache License, # Version 2.0 (the "License"); you may not use this file From 4145d0ce3418f7bbeba5050025654ffc1989b1d8 Mon Sep 17 00:00:00 2001 From: Kelly McLaughlin Date: Mon, 15 Dec 2014 12:38:01 -0700 Subject: [PATCH 041/157] Port Jon Anderson's work on test groups to avoid merge pain --- src/riak_test_escript.erl | 103 ++++++++++++++++++++++++++------------ 1 file changed, 72 insertions(+), 31 deletions(-) diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index 09a355201..ffb8f5f52 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -192,16 +192,13 @@ help_or_parse_args({ok, {[], _}}) -> help_or_parse_args({ok, {ParsedArgs, HarnessArgs}}) -> help_or_parse_tests(ParsedArgs, HarnessArgs, - lists:member(help, ParsedArgs), - args_invalid(ParsedArgs)); + lists:member(help, ParsedArgs)); help_or_parse_args(_) -> print_help(). -help_or_parse_tests(_, _, true, _) -> +help_or_parse_tests(_, _, true) -> print_help(); -help_or_parse_tests(_, _, false, true) -> - print_help(); -help_or_parse_tests(ParsedArgs, HarnessArgs, false, false) -> +help_or_parse_tests(ParsedArgs, HarnessArgs, false) -> %% Have to load the `riak_test' config prior to assembling the %% test metadata load_initial_config(ParsedArgs), @@ -212,16 +209,6 @@ help_or_parse_tests(ParsedArgs, HarnessArgs, false, false) -> Workers = rt_config:get(workers, undefined), shuffle_tests(ParsedArgs, HarnessArgs, Tests, NonTests, Offset, Workers). -args_invalid(ParsedArgs) -> - case { proplists:is_defined(groups, ParsedArgs), - proplists:is_defined(tests, ParsedArgs) } of - {true, true} -> - io:format("--groups and --tests are currently mutually exclusive.~n~n"), - true; - {_, _} -> - false - end. - load_initial_config(ParsedArgs) -> %% Loads application defaults application:load(riak_test), @@ -316,11 +303,16 @@ set_lager_env(OutputDir, ConsoleLevel, FileLevel) -> %% end, %% ok. -load_tests([], ParsedArgs) -> +-spec comma_tokenizer(string(), [string()]) -> [string()]. +comma_tokenizer(S, Acc) -> + string:tokens(S, ", ") ++ Acc. + +compose_test_data(ParsedArgs) -> RawTestList = proplists:get_all_values(tests, ParsedArgs), - TestList = lists:foldl(fun(X, Acc) -> - string:tokens(X, ", ") ++ Acc - end, [], RawTestList), + RawGroupList = proplists:get_all_values(groups, ParsedArgs), + TestList = lists:foldl(fun comma_tokenizer/2, [], RawTestList), + GroupList = lists:foldl(fun comma_tokenizer/2, [], RawGroupList), + %% Parse Command Line Tests {CodePaths, SpecificTests} = lists:foldl(fun extract_test_names/2, @@ -330,16 +322,29 @@ load_tests([], ParsedArgs) -> [code:add_patha(CodePath) || CodePath <- CodePaths, CodePath /= "."], - Dirs = proplists:get_all_values(dir, ParsedArgs), + Dirs = get_test_dirs(ParsedArgs, default_test_dir(GroupList)), SkipTests = string:tokens(proplists:get_value(skip, ParsedArgs, []), [$,]), - DirTests = lists:append([load_tests_in_dir(Dir, SkipTests) || Dir <- Dirs]), - %% Project = list_to_binary(rt_config:get(rt_project, "undefined")), - - %% Upgrades = proplists:get_value(upgrade_path, ParsedArgs), - %% TestFoldFun = test_data_fun(rt:get_version(), Project, Upgrades), - %% lists:foldl(TestFoldFun, [], lists:usort(DirTests ++ SpecificTests)). + DirTests = lists:append([load_tests_in_dir(Dir, GroupList, SkipTests) || Dir <- Dirs]), lists:usort(DirTests ++ SpecificTests). +-spec default_test_dir([string()]) -> [string()]. +%% @doc If any groups have been specified then we want to check in the +%% local test directory by default; otherwise, the default behavior is +%% that no directory is used to pull tests from. +default_test_dir([]) -> + []; +default_test_dir(_) -> + ["./ebin"]. + +-spec get_test_dirs(term(), [string()]) -> [string()]. +get_test_dirs(ParsedArgs, DefaultDirs) -> + case proplists:get_all_values(dir, ParsedArgs) of + [] -> + DefaultDirs; + Dirs -> + Dirs + end. + extract_test_names(Test, {CodePaths, TestNames}) -> {[filename:dirname(Test) | CodePaths], [list_to_atom(filename:rootname(filename:basename(Test))) | TestNames]}. @@ -387,6 +392,23 @@ is_runnable_test(TestModule) -> erlang:function_exported(Mod, Fun, 0) orelse erlang:function_exported(Mod, Fun, 2). +get_group_tests(Tests, Groups) -> + lists:filter(fun(Test) -> + Mod = list_to_atom(Test), + Attrs = Mod:module_info(attributes), + match_group_attributes(Attrs, Groups) + end, Tests). + +match_group_attributes(Attributes, Groups) -> + case proplists:get_value(test_type, Attributes) of + undefined -> + false; + TestTypes -> + lists:member(true, + [ TestType == list_to_atom(Group) + || Group <- Groups, TestType <- TestTypes ]) + end. + %% run_tests(Tests, Outdir, Report, HarnessArgs) -> %% Need properties for tests prior to getting here Need server to %% manage the aquisition of nodes and to handle comparison of test @@ -538,29 +560,48 @@ backend_list(Backends) when is_list(Backends) -> end, lists:foldl(FoldFun, [], Backends). -load_tests_in_dir(Dir, SkipTests) -> +load_tests_in_dir(Dir, Groups, SkipTests) -> case filelib:is_dir(Dir) of true -> code:add_path(Dir), lists:sort( - lists:foldl(load_tests_folder(SkipTests), + lists:foldl(load_tests_folder(Groups, SkipTests), [], filelib:wildcard("*.beam", Dir))); _ -> io:format("~s is not a dir!~n", [Dir]) end. -load_tests_folder(SkipTests) -> +load_tests_folder([], SkipTests) -> fun(X, Acc) -> + %% Drop the .beam suffix Test = string:substr(X, 1, length(X) - 5), case lists:member(Test, SkipTests) of true -> Acc; false -> - [Test | Acc] + [list_to_atom(Test) | Acc] + end + end; +load_tests_folder(Groups, SkipTests) -> + fun(X, Acc) -> + %% Drop the .beam suffix + Test = string:substr(X, 1, length(X) - 5), + case group_match(Test, Groups) + andalso not lists:member(Test, SkipTests) of + true -> + [list_to_atom(Test) | Acc]; + false -> + Acc end end. +-spec group_match(string(), [string()]) -> boolean(). +group_match(Test, Groups) -> + Mod = list_to_atom(Test), + Attrs = Mod:module_info(attributes), + match_group_attributes(Attrs, Groups). + so_kill_riak_maybe() -> io:format("~n~nSo, we find ourselves in a tricky situation here. ~n"), io:format("You've run a single test, and it has failed.~n"), From 8f7ff87c753305365a8bf5aa97c437378d811f27 Mon Sep 17 00:00:00 2001 From: Kelly McLaughlin Date: Mon, 15 Dec 2014 17:12:29 -0700 Subject: [PATCH 042/157] Convert riak_cli dependency to clique dependency --- rebar.config | 2 +- src/riak_test_escript.erl | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/rebar.config b/rebar.config index f65b53618..ae795dd41 100644 --- a/rebar.config +++ b/rebar.config @@ -19,7 +19,7 @@ {riakhttpc, ".*", {git, "git://github.com/basho/riak-erlang-http-client", {branch, "master"}}}, {kvc, "1.3.0", {git, "https://github.com/etrepum/kvc", {tag, "v1.3.0"}}}, {druuid, ".*", {git, "git://github.com/kellymclaughlin/druuid.git", {tag, "0.2"}}}, - {riak_cli, ".*", {git, "git@github.com:basho/riak_cli", {branch, "master"}}} + {clique, ".*", {git, "git://github.com/basho/clique", {branch, "develop"}}} ]}. {escript_incl_apps, [goldrush, lager, getopt, riakhttpc, riakc, ibrowse, mochiweb, kvc]}. diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index ffb8f5f52..0a6eea28d 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -520,7 +520,7 @@ print_summary(TestResults, _CoverResult, Verbose) -> true -> Rows = [format_test_row(Result, Width) || Result <- TestResults], - Table = riak_cli_table:autosize_create_table(?HEADER, Rows), + Table = clique_table:autosize_create_table(?HEADER, Rows), io:format("~ts~n", [Table]); false -> ok From d8a6df428d14484a47dcd49f648e3acee35cfc33 Mon Sep 17 00:00:00 2001 From: Kelly McLaughlin Date: Mon, 15 Dec 2014 17:04:06 -0700 Subject: [PATCH 043/157] Expand support for replication test conversion Add properties for declaring bucket_types and for automatically retrieving then information about test clusters needed for replication tests. This includes automatically naming the test clusters and determining the cluster leader for each test cluster. This represents boilerplate that is repeated in many replication tests. Also begin converting the repl_bucket_types test as a sample and to test these new features. --- src/riak_test.erl | 4 +- src/rt_bucket_types.erl | 10 +- src/rt_cluster.erl | 83 +++++- src/rt_cluster_info.erl | 166 +++++++++++ src/rt_properties.erl | 20 +- tests/replication/repl_bucket_types.erl | 350 ++++++++++++------------ 6 files changed, 446 insertions(+), 187 deletions(-) create mode 100644 src/rt_cluster_info.erl diff --git a/src/riak_test.erl b/src/riak_test.erl index 9c73ce592..a95e81c43 100644 --- a/src/riak_test.erl +++ b/src/riak_test.erl @@ -1,3 +1,4 @@ +-module(riak_test). %% ------------------------------------------------------------------- %% %% Copyright (c) 2013 Basho Technologies, Inc. @@ -18,7 +19,6 @@ %% %% ------------------------------------------------------------------- --module(riak_test). %% Define the riak_test behavior --callback confirm() -> pass | fail. +-callback confirm(rt_properties:properties(), proplists:proplist()) -> pass | fail. diff --git a/src/rt_bucket_types.erl b/src/rt_bucket_types.erl index 1443283fb..b417f80f6 100644 --- a/src/rt_bucket_types.erl +++ b/src/rt_bucket_types.erl @@ -21,13 +21,20 @@ -module(rt_bucket_types). -include_lib("eunit/include/eunit.hrl"). --export([create_and_activate_bucket_type/3, +-export([create_and_wait/3, + create_and_activate_bucket_type/3, wait_until_bucket_type_visible/2, wait_until_bucket_type_status/3, wait_until_bucket_props/3]). -include("rt.hrl"). +-spec create_and_wait([node()], binary(), proplists:proplist()) -> ok. +create_and_wait(Nodes, Type, Properties) -> + create_and_activate_bucket_type(hd(Nodes), Type, Properties), + wait_until_bucket_type_status(Type, active, Nodes), + wait_until_bucket_type_visible(Nodes, Type). + %% @doc create and immediately activate a bucket type create_and_activate_bucket_type(Node, Type, Props) -> ok = rpc:call(Node, riak_core_bucket_type, create, [Type, Props]), @@ -85,4 +92,3 @@ wait_until_bucket_props(Nodes, Bucket, Props) -> see_bucket_props(Nodes, Bucket, Props) end, ?assertEqual(ok, rt:wait_until(F)). - diff --git a/src/rt_cluster.erl b/src/rt_cluster.erl index 1b7f237df..011c2c1b5 100644 --- a/src/rt_cluster.erl +++ b/src/rt_cluster.erl @@ -45,18 +45,82 @@ properties() -> -spec setup(rt_properties:properties(), proplists:proplist()) -> {ok, rt_properties:properties()} | {error, term()}. setup(Properties, _MetaData) -> - %% rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), case form_clusters(Properties) of - {ok, Clusters} -> + {ok, ClusterNodes} -> maybe_wait_for_transfers(rt_properties:get(node_ids, Properties), rt_properties:get(node_map, Properties), rt_properties:get(wait_for_transfers, Properties)), + Clusters = prepare_clusters(ClusterNodes, Properties), + create_bucket_types(Clusters, Properties), rt_properties:set(clusters, Clusters, Properties); Error -> Error end. --type clusters() :: [{pos_integer(), [string()]}]. +-spec create_bucket_types([rt_cluster_info:cluster_info()], rt_properties:properties()) -> no_return(). +create_bucket_types(Clusters, Properties) -> + BucketTypes = rt_properties:get(bucket_types, Properties), + create_bucket_types(Clusters, Properties, BucketTypes). + +-spec create_bucket_types([rt_cluster_info:cluster_info()], + rt_properties:properties(), + rt_properties:bucket_types()) -> no_return(). +create_bucket_types(_Clusters, _Properties, []) -> + ok; +create_bucket_types([Cluster], Properties, BucketTypes) -> + NodeMap = rt_properties:get(node_map, Properties), + NodeIds = rt_cluster_info:get(node_ids, Cluster), + Nodes = [rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds], + lists:foldl(fun maybe_create_bucket_type/2, [{Nodes, 1}], BucketTypes); +create_bucket_types(Clusters, Properties, BucketTypes) -> + NodeMap = rt_properties:get(node_map, Properties), + [begin + NodeIds = rt_cluster_info:get(node_ids, Cluster), + Nodes = [rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds], + lists:foldl(fun maybe_create_bucket_type/2, {Nodes, ClusterIndex}, BucketTypes) + end || {Cluster, ClusterIndex} <- lists:zip(Clusters, lists:seq(1, length(Clusters)))]. + +maybe_create_bucket_type({ClusterIndex, {TypeName, TypeProps}}, + {Nodes, ClusterIndex}) -> + rt_bucket_types:create_and_wait(Nodes, TypeName, TypeProps), + {Nodes, ClusterIndex}; +maybe_create_bucket_type({_ApplicableIndex, {_TypeName, _TypeProps}}, + {Nodes, _ClusterIndex}) -> + %% This bucket type does not apply to this cluster + {Nodes, _ClusterIndex}; +maybe_create_bucket_type({TypeName, TypeProps}, {Nodes, _ClusterIndex}) -> + %% This bucket type applies to all clusters + rt_bucket_types:create_and_wait(Nodes, TypeName, TypeProps), + {Nodes, _ClusterIndex}. + +-spec prepare_clusters([list(string())], rt_properties:properties()) -> + [rt_cluster_info:cluster_info()]. +prepare_clusters([ClusterNodes], _Properties) -> + rt_cluster_info:new([{node_ids, ClusterNodes}]); +prepare_clusters(ClusterNodesList, Properties) -> + %% If the count of clusters is > 1 the assumption is made that the + %% test is exercising replication and some extra + %% made. This to avoid some noisy and oft-repeated setup + %% boilerplate in every replication test. + NodeMap = rt_properties:get(node_map, Properties), + {Clusters, _, _} = lists:foldl(fun prepare_cluster/2, + {[], 1, NodeMap}, + ClusterNodesList), + lists:reverse(Clusters). + +-type prepare_cluster_acc() :: {[rt_cluster_info:cluster_info()], char(), proplists:proplist()}. +-spec prepare_cluster([string()], prepare_cluster_acc()) -> prepare_cluster_acc(). +prepare_cluster(NodeIds, {Clusters, Name, NodeMap}) -> + Nodes = [rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds], + repl_util:name_cluster(hd(Nodes), integer_to_list(Name)), + repl_util:wait_until_leader_converge(Nodes), + Leader = repl_util:get_leader(hd(Nodes)), + Cluster = rt_cluster_info:new([{node_ids, NodeIds}, + {leader, Leader}, + {name, Name}]), + {[Cluster | Clusters], Name+1, NodeMap}. + +-type clusters() :: [rt_cluster_info:cluster_info()]. -spec form_clusters(rt_properties:properties()) -> clusters(). form_clusters(Properties) -> NodeIds = rt_properties:get(node_ids, Properties), @@ -72,12 +136,14 @@ form_clusters(Properties) -> Error end. +-spec divide_nodes([string()], pos_integer(), [float()]) -> + {ok, [list(string())]} | {error, atom()}. divide_nodes(Nodes, Count, Weights) when length(Nodes) < Count; Weights =/= undefined, length(Weights) =/= Count -> {error, invalid_cluster_properties}; divide_nodes(Nodes, 1, _) -> - {ok, [{1, Nodes}]}; + {ok, [Nodes]}; divide_nodes(Nodes, Count, Weights) -> case validate_weights(Weights) of true -> @@ -91,7 +157,7 @@ divide_nodes(Nodes, Count, Weights) -> take_nodes(NodeCount, {Index, ClusterAcc, Nodes}) -> {NewClusterNodes, RestNodes} = lists:split(NodeCount, Nodes), - {Index + 1, [{Index, NewClusterNodes} | ClusterAcc], RestNodes}. + {Index + 1, [NewClusterNodes | ClusterAcc], RestNodes}. validate_weights(undefined) -> true; @@ -124,13 +190,14 @@ node_count_from_weight(TotalNodes, Weight) -> IntegerPortion end. -remainder_to_apply(Remainder, Index) when Remainder > Index -> +remainder_to_apply(Remainder, Index) when Remainder > Index; + Remainder =:= 0 -> 0; remainder_to_apply(_Remainder, _Index) -> 1. maybe_join_clusters(Clusters, NodeMap, true) -> - [join_cluster(ClusterNodes, NodeMap) || {_, ClusterNodes} <- Clusters]; + [join_cluster(ClusterNodes, NodeMap) || ClusterNodes <- Clusters]; maybe_join_clusters(_Clusters, _NodeMap, false) -> ok. @@ -151,8 +218,10 @@ join_cluster(NodeIds, NodeMap) -> case OtherNodes of [] -> %% no other nodes, nothing to join/plan/commit + ok; _ -> + %% ok do a staged join and then commit it, this eliminates the %% large amount of redundant handoff done in a sequential join [rt_node:staged_join(Node, Node1) || Node <- OtherNodes], diff --git a/src/rt_cluster_info.erl b/src/rt_cluster_info.erl new file mode 100644 index 000000000..ec1d22346 --- /dev/null +++ b/src/rt_cluster_info.erl @@ -0,0 +1,166 @@ +-module(rt_cluster_info). +%% ------------------------------------------------------------------- +%% +%% 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. +%% +%% ------------------------------------------------------------------- + +%% @doc Implements a set of functions for accessing and manipulating +%% an `rt_cluster_info' record. + +-record(rt_cluster_info_v1, { + node_ids :: [string()], + leader :: string(), + name :: string() + }). +-type cluster_info() :: #rt_cluster_info_v1{}. + +-export_type([cluster_info/0]). + +-define(RT_CLUSTER_INFO, #rt_cluster_info_v1). +-define(RECORD_FIELDS, record_info(fields, rt_cluster_info_v1)). + +-export([new/0, + new/1, + get/2, + set/2, + set/3]). + +%% @doc Create a new cluster_info record with all fields initialized to +%% the default values. +-spec new() -> cluster_info(). +new() -> + ?RT_CLUSTER_INFO{}. + +%% @doc Create a new cluster_info record with the fields initialized to +%% non-default value. Each field to be initialized should be +%% specified as an entry in a property list (i.e. a list of +%% pairs). Invalid fields are ignored by this function. +-spec new(proplists:proplist()) -> cluster_info(). +new(Defaults) -> + {ClusterInfo, _} = + lists:foldl(fun set_field/2, {?RT_CLUSTER_INFO{}, []}, Defaults), + ClusterInfo. + +%% @doc Get the value of a field from a cluster_info record. An error +%% is returned if `ClusterInfo' is not a valid `rt_cluster_info' record +%% or if the property requested is not a valid property. +-spec get(atom(), cluster_info()) -> term() | {error, atom()}. +get(Field, ClusterInfo) -> + get(Field, ClusterInfo, validate_request(Field, ClusterInfo)). + +%% @doc Set the value for a field in a cluster_info record. An error +%% is returned if `ClusterInfo' is not a valid `rt_cluster_info' record +%% or if any of the fields to be set are not valid. In +%% the case that invalid fields are specified the error returned +%% contains a list of erroneous fields. +-spec set([{atom(), term()}], cluster_info()) -> cluster_info() | {error, atom()}. +set(FieldList, ClusterInfo) when is_list(FieldList) -> + set_fields(FieldList, ClusterInfo, validate_record(ClusterInfo)). + +%% @doc Set the value for a field in a cluster_info record. An error +%% is returned if `ClusterInfo' is not a valid `rt_cluster_info' record +%% or if the field to be set is not valid. +-spec set(atom(), term(), cluster_info()) -> {ok, cluster_info()} | {error, atom()}. +set(Field, Value, ClusterInfo) -> + set_field(Field, Value, ClusterInfo, validate_request(Field, ClusterInfo)). + +-spec get(atom(), cluster_info(), ok | {error, atom()}) -> + term() | {error, atom()}. +get(Field, ClusterInfo, ok) -> + element(field_index(Field), ClusterInfo); +get(_Field, _ClusterInfo, {error, _}=Error) -> + Error. + +%% This function is used by `new/1' to set fields at record +%% creation time and by `set/2' to set multiple properties at once. +%% No cluster_info record validation is done by this function. It is +%% strictly used as a fold function which is the reason for the odd +%% structure of the input parameters. It accumulates any invalid +%% fields that are encountered and the caller may use that +%% information or ignore it. +-spec set_field({atom(), term()}, {cluster_info(), [atom()]}) -> + {cluster_info(), [atom()]}. +set_field({Field, Value}, {ClusterInfo, Invalid}) -> + case is_valid_field(Field) of + true -> + {setelement(field_index(Field), ClusterInfo, Value), Invalid}; + false -> + {ClusterInfo, [Field | Invalid]} + end. + +-spec set_field(atom(), term(), cluster_info(), ok | {error, atom()}) -> + {ok, cluster_info()} | {error, atom()}. +set_field(Field, Value, ClusterInfo, ok) -> + {ok, setelement(field_index(Field), ClusterInfo, Value)}; +set_field(_Field, _Value, _ClusterInfo, {error, _}=Error) -> + Error. + +-spec set_fields([{atom(), term()}], + cluster_info(), + ok | {error, {atom(), [atom()]}}) -> + {cluster_info(), [atom()]}. +set_fields(FieldList, ClusterInfo, ok) -> + case lists:foldl(fun set_field/2, {ClusterInfo, []}, FieldList) of + {UpdClusterInfo, []} -> + UpdClusterInfo; + {_, InvalidClusterInfo} -> + {error, {invalid_properties, InvalidClusterInfo}} + end; +set_fields(_, _, {error, _}=Error) -> + Error. + +-spec validate_request(atom(), term()) -> ok | {error, atom()}. +validate_request(Field, ClusterInfo) -> + validate_field(Field, validate_record(ClusterInfo)). + +-spec validate_record(term()) -> ok | {error, invalid_cluster_info}. +validate_record(Record) -> + case is_valid_record(Record) of + true -> + ok; + false -> + {error, invalid_cluster_info} + end. + +-spec validate_field(atom(), ok | {error, atom()}) -> ok | {error, invalid_field}. +validate_field(Field, ok) -> + case is_valid_field(Field) of + true -> + ok; + false -> + {error, invalid_field} + end; +validate_field(_Field, {error, _}=Error) -> + Error. + +-spec is_valid_record(term()) -> boolean(). +is_valid_record(Record) -> + is_record(Record, rt_cluster_info_v1). + +-spec is_valid_field(atom()) -> boolean(). +is_valid_field(Field) -> + Fields = ?RECORD_FIELDS, + lists:member(Field, Fields). + +-spec field_index(atom()) -> non_neg_integer(). +field_index(node_ids) -> + ?RT_CLUSTER_INFO.node_ids; +field_index(leader) -> + ?RT_CLUSTER_INFO.leader; +field_index(name) -> + ?RT_CLUSTER_INFO.name. diff --git a/src/rt_properties.erl b/src/rt_properties.erl index bb0ec2090..4f63031ea 100644 --- a/src/rt_properties.erl +++ b/src/rt_properties.erl @@ -1,3 +1,4 @@ +-module(rt_properties). %% ------------------------------------------------------------------- %% %% Copyright (c) 2013 Basho Technologies, Inc. @@ -21,10 +22,7 @@ %% @doc Implements a set of functions for accessing and manipulating %% an `rt_properties' record. --module(rt_properties). - -include("rt.hrl"). --include_lib("eunit/include/eunit.hrl"). %% A quick note on the distinction between `node_ids' and %% `node_map'. `node_ids' are short identifers (e.g. dev1) and the @@ -49,13 +47,23 @@ make_cluster=true :: boolean(), cluster_count=1 :: pos_integer(), cluster_weights :: [float()], - clusters :: proplists:proplist(), + clusters :: [rt_cluster_info:cluster_info()], required_services=[riak_kv] :: [atom()], + bucket_types=[] :: bucket_types(), config=default_config() :: term() }). -type properties() :: #rt_properties_v1{}. --export_type([properties/0]). +%% Specify the bucket_types field for the properties record. The list +%% of bucket types may have two forms, a bucket_type or a pair +%% consisting of an integer and a bucket_type. The latter form +%% indicates that a bucket_type should only be applied to the cluster +%% with the given index. The former form is applied to all clusters. +-type bucket_type() :: {binary(), proplists:proplist()}. +-type bucket_types() :: [bucket_type() | {pos_integer(), bucket_type()}]. + +-export_type([properties/0, + bucket_types/0]). -define(RT_PROPERTIES, #rt_properties_v1). -define(RECORD_FIELDS, record_info(fields, rt_properties_v1)). @@ -220,6 +228,8 @@ field_index(cluster_count) -> ?RT_PROPERTIES.cluster_count; field_index(cluster_weights) -> ?RT_PROPERTIES.cluster_weights; +field_index(bucket_types) -> + ?RT_PROPERTIES.bucket_types; field_index(clusters) -> ?RT_PROPERTIES.clusters; field_index(required_services) -> diff --git a/tests/replication/repl_bucket_types.erl b/tests/replication/repl_bucket_types.erl index 83d588deb..8ce3346bd 100644 --- a/tests/replication/repl_bucket_types.erl +++ b/tests/replication/repl_bucket_types.erl @@ -1,87 +1,81 @@ +-module(repl_bucket_types). %% ------------------------------------------------------------------- %% %% Copyright (c) 2013 Basho Technologies, Inc. %% %% ------------------------------------------------------------------- --module(repl_bucket_types). + -behaviour(riak_test). --export([confirm/0]). --compile(export_all). + +-export([setup/2, + properties/0, + confirm/2]). -include_lib("eunit/include/eunit.hrl"). +-test_type([bucket_types, replication]). + -define(ENSURE_READ_ITERATIONS, 5). -define(ENSURE_READ_INTERVAL, 1000). %% Replication Bucket Types test -%% - -setup(Type) -> - rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), - {LeaderA, LeaderB, ANodes, BNodes} = ClusterNodes = make_clusters(Type), - - PBA = rt_pb:pbc(LeaderA), - PBB = rt_pb:pbc(LeaderB), - - {DefinedType, UndefType} = Types = {<<"working_type">>, <<"undefined_type">>}, - - rt_bucket_types:create_and_activate_bucket_type(LeaderA, - DefinedType, - [{n_val, 3}, {allow_mult, false}]), - rt_bucket_types:wait_until_bucket_type_status(DefinedType, active, ANodes), - rt_bucket_types:wait_until_bucket_type_visible(ANodes, DefinedType), - - case Type of - current -> - rt_bucket_types:create_and_activate_bucket_type(LeaderB, - DefinedType, - [{n_val, 3}, {allow_mult, false}]), - rt_bucket_types:wait_until_bucket_type_status(DefinedType, active, BNodes), - rt_bucket_types:wait_until_bucket_type_visible(BNodes, DefinedType); - mixed -> - ok - end, - - rt_bucket_types:create_and_activate_bucket_type(LeaderA, - UndefType, - [{n_val, 3}, {allow_mult, false}]), - rt_bucket_types:wait_until_bucket_type_status(UndefType, active, ANodes), - rt_bucket_types:wait_until_bucket_type_visible(ANodes, UndefType), +-spec properties() -> rt_properties:properties(). +properties() -> + Config = rt_properties:default_config() ++ cluster_conf(), + BucketTypes = [{<<"working_type">>, [{n_val, 3} , {allow_mult, false}]}, + {1, {<<"undefined_type">>, [{n_val, 3} , {allow_mult, false}]}}], + rt_properties:new([{node_count, 4}, + {config, Config}, + {cluster_count, 2}, + {bucket_types, BucketTypes}, + {wait_for_transfers, true}, + {required_services, [riak_kv, riak_repl]}]). + +setup(Properties, _MD) -> + case rt_cluster:setup(Properties, _MD) of + {ok, UpdProperties} -> + Clusters = rt_properties:get(clusters, UpdProperties), + [LeaderA, LeaderB] = [rt_cluster_info:get(leader, Cluster) + || Cluster <- Clusters], + connect_clusters(LeaderA, LeaderB), + {ok, UpdProperties}; + Error -> + Error + end. - connect_clusters(LeaderA, LeaderB), - {ClusterNodes, Types, PBA, PBB}. +-spec confirm(rt_properties:properties(), proplists:proplist()) -> pass | fail. +confirm(Properties, _MD) -> + Clusters = rt_properties:get(clusters, Properties), + NodeMap = rt_properties:get(node_map, Properties), -cleanup({ClusterNodes, _Types, PBA, PBB}, CleanCluster) -> - riakc_pb_socket:stop(PBA), - riakc_pb_socket:stop(PBB), - {_, _, ANodes, BNodes} = ClusterNodes, - case CleanCluster of - true -> - rt_cluster:clean_cluster(ANodes ++ BNodes); - false -> - ok - end. + BucketTypes = {<<"working_type">>, <<"undefined_type">>}, -%% @doc riak_test entry point -confirm() -> %% Test two clusters of the current version - SetupData = setup(current), - realtime_test(SetupData), - fullsync_test(SetupData), - cleanup(SetupData, true), + %% SetupData = setup(current), + realtime_test(Clusters, NodeMap, BucketTypes), + fullsync_test(Clusters, NodeMap, BucketTypes), %% Test a cluster of the current version replicating to a cluster %% of the previous version - MixedSetupData = setup(mixed), - realtime_mixed_version_test(MixedSetupData), - fullsync_mixed_version_test(MixedSetupData), - cleanup(MixedSetupData, false), + %% MixedSetupData = setup(mixed), + %% realtime_mixed_version_test(MixedSetupData), + %% fullsync_mixed_version_test(MixedSetupData), + %% cleanup(MixedSetupData, false), + pass. -realtime_test({ClusterNodes, BucketTypes, PBA, PBB}) -> - {LeaderA, LeaderB, ANodes, BNodes} = ClusterNodes, +realtime_test(Clusters, NodeMap, BucketTypes) -> + [{LeaderA, ANodes}, {LeaderB, BNodes}] = + [{rt_cluster_info:get(leader, Cluster), + [rt_node:node_name(NodeId, NodeMap) + || NodeId <- rt_cluster_info:get(node_ids, Cluster)]} + || Cluster <- Clusters], + {DefinedType, UndefType} = BucketTypes, + PBA = rt_pb:pbc(LeaderA), + PBB = rt_pb:pbc(LeaderB), + %% Enable RT replication from cluster "A" to cluster "B" lager:info("Enabling realtime between ~p and ~p", [LeaderA, LeaderB]), enable_rt(LeaderA, ANodes), @@ -148,41 +142,52 @@ realtime_test({ClusterNodes, BucketTypes, PBA, PBB}) -> {n_val, 3} =:= lists:keyfind(n_val, 1, UpdatedProps2) end), - disable_rt(LeaderA, ANodes). + disable_rt(LeaderA, ANodes), + riakc_pb_socket:stop(PBA), + riakc_pb_socket:stop(PBB), + ok. -realtime_mixed_version_test({ClusterNodes, BucketTypes, PBA, PBB}) -> - {LeaderA, LeaderB, ANodes, _BNodes} = ClusterNodes, - {DefinedType, _UndefType} = BucketTypes, +%% realtime_mixed_version_test({ClusterNodes, BucketTypes, PBA, PBB}) -> +%% {LeaderA, LeaderB, ANodes, _BNodes} = ClusterNodes, +%% {DefinedType, _UndefType} = BucketTypes, - %% Enable RT replication from cluster "A" to cluster "B" - lager:info("Enabling realtime between ~p and ~p", [LeaderA, LeaderB]), - enable_rt(LeaderA, ANodes), +%% %% Enable RT replication from cluster "A" to cluster "B" +%% lager:info("Enabling realtime between ~p and ~p", [LeaderA, LeaderB]), +%% enable_rt(LeaderA, ANodes), - Bin = <<"data data data">>, - Key = <<"key">>, - Bucket = <<"kicked">>, - DefaultObj = riakc_obj:new(Bucket, Key, Bin), - lager:info("doing untyped put on A, bucket:~p", [Bucket]), - riakc_pb_socket:put(PBA, DefaultObj, [{w,3}]), +%% Bin = <<"data data data">>, +%% Key = <<"key">>, +%% Bucket = <<"kicked">>, +%% DefaultObj = riakc_obj:new(Bucket, Key, Bin), +%% lager:info("doing untyped put on A, bucket:~p", [Bucket]), +%% riakc_pb_socket:put(PBA, DefaultObj, [{w,3}]), - %% make sure we rt replicate a "default" type bucket - UntypedWait = make_pbget_fun(PBB, Bucket, Key, Bin), - ?assertEqual(ok, rt:wait_until(UntypedWait)), +%% %% make sure we rt replicate a "default" type bucket +%% UntypedWait = make_pbget_fun(PBB, Bucket, Key, Bin), +%% ?assertEqual(ok, rt:wait_until(UntypedWait)), - DowngradedBucketTyped = {DefinedType, <<"typekicked">>}, - KeyTyped = <<"keytyped">>, - ObjTyped = riakc_obj:new(DowngradedBucketTyped, KeyTyped, Bin), +%% DowngradedBucketTyped = {DefinedType, <<"typekicked">>}, +%% KeyTyped = <<"keytyped">>, +%% ObjTyped = riakc_obj:new(DowngradedBucketTyped, KeyTyped, Bin), - lager:info("doing typed put on A with downgraded B, bucket:~p", [DowngradedBucketTyped]), - riakc_pb_socket:put(PBA, ObjTyped, [{w,3}]), +%% lager:info("doing typed put on A with downgraded B, bucket:~p", [DowngradedBucketTyped]), +%% riakc_pb_socket:put(PBA, ObjTyped, [{w,3}]), + +%% lager:info("checking to ensure the bucket contents were not sent to previous version B."), +%% ensure_bucket_not_sent(PBB, DowngradedBucketTyped, KeyTyped). - lager:info("checking to ensure the bucket contents were not sent to previous version B."), - ensure_bucket_not_sent(PBB, DowngradedBucketTyped, KeyTyped). +fullsync_test(Clusters, NodeMap, BucketTypes) -> + [{LeaderA, ANodes}, {LeaderB, BNodes}] = + [{rt_cluster_info:get(leader, Cluster), + [rt_node:node_name(NodeId, NodeMap) + || NodeId <- rt_cluster_info:get(node_ids, Cluster)]} + || Cluster <- Clusters], -fullsync_test({ClusterNodes, BucketTypes, PBA, PBB}) -> - {LeaderA, LeaderB, ANodes, BNodes} = ClusterNodes, {DefinedType, UndefType} = BucketTypes, + PBA = rt_pb:pbc(LeaderA), + PBB = rt_pb:pbc(LeaderB), + %% Enable RT replication from cluster "A" to cluster "B" lager:info("Enabling fullsync between ~p and ~p", [LeaderA, LeaderB]), enable_fullsync(LeaderA, ANodes), @@ -253,84 +258,88 @@ fullsync_test({ClusterNodes, BucketTypes, PBA, PBB}) -> lager:info("Fullsync completed in ~p seconds", [SyncTime2/1000/1000]), lager:info("checking to ensure the bucket contents were not updated."), - ensure_bucket_not_updated(PBB, BucketTyped, KeyTyped, Bin). + ensure_bucket_not_updated(PBB, BucketTyped, KeyTyped, Bin), -fullsync_mixed_version_test({ClusterNodes, BucketTypes, PBA, PBB}) -> - {LeaderA, LeaderB, ANodes, _BNodes} = ClusterNodes, - {DefinedType, _UndefType} = BucketTypes, + riakc_pb_socket:stop(PBA), + riakc_pb_socket:stop(PBB), + ok. - %% Enable RT replication from cluster "A" to cluster "B" - lager:info("Enabling fullsync between ~p and ~p", [LeaderA, LeaderB]), - enable_fullsync(LeaderA, ANodes), +%% fullsync_mixed_version_test({ClusterNodes, BucketTypes, PBA, PBB}) -> +%% {LeaderA, LeaderB, ANodes, _BNodes} = ClusterNodes, +%% {DefinedType, _UndefType} = BucketTypes, - Bin = <<"good data">>, - Key = <<"key">>, - Bucket = <<"fullsync-kicked">>, - DefaultObj = riakc_obj:new(Bucket, Key, Bin), - lager:info("doing untyped put on A, bucket:~p", [Bucket]), - riakc_pb_socket:put(PBA, DefaultObj, [{w,3}]), +%% %% Enable RT replication from cluster "A" to cluster "B" +%% lager:info("Enabling fullsync between ~p and ~p", [LeaderA, LeaderB]), +%% enable_fullsync(LeaderA, ANodes), - BucketTyped = {DefinedType, Bucket}, - KeyTyped = <<"keytyped">>, - BadBin = <<"overwritten">>, - ObjTyped = riakc_obj:new(BucketTyped, KeyTyped, BadBin), +%% Bin = <<"good data">>, +%% Key = <<"key">>, +%% Bucket = <<"fullsync-kicked">>, +%% DefaultObj = riakc_obj:new(Bucket, Key, Bin), +%% lager:info("doing untyped put on A, bucket:~p", [Bucket]), +%% riakc_pb_socket:put(PBA, DefaultObj, [{w,3}]), - lager:info("doing typed put on A, bucket:~p", [BucketTyped]), - riakc_pb_socket:put(PBA, ObjTyped, [{w,3}]), +%% BucketTyped = {DefinedType, Bucket}, +%% KeyTyped = <<"keytyped">>, +%% BadBin = <<"overwritten">>, +%% ObjTyped = riakc_obj:new(BucketTyped, KeyTyped, BadBin), - {SyncTime1, _} = timer:tc(repl_util, - start_and_wait_until_fullsync_complete, - [LeaderA]), +%% lager:info("doing typed put on A, bucket:~p", [BucketTyped]), +%% riakc_pb_socket:put(PBA, ObjTyped, [{w,3}]), - lager:info("Fullsync completed in ~p seconds", [SyncTime1/1000/1000]), +%% {SyncTime1, _} = timer:tc(repl_util, +%% start_and_wait_until_fullsync_complete, +%% [LeaderA]), - ReadResult1 = riakc_pb_socket:get(PBB, Bucket, Key), - ?assertMatch({ok, _}, ReadResult1), +%% lager:info("Fullsync completed in ~p seconds", [SyncTime1/1000/1000]), - %% The following check appears to be the best we can do. If a 2.x source - %% sends a typed bucket to the 1.x sink, the put will occur. - %% The bucket is undefined to the interfaces, but some parts of it - %% appear to be written to the sink node. Since we cannot check using pb, - %% here we at least make sure we haven't written over an existing default - %% bucket with data from a typed bucket of the same name. - ensure_bucket_not_updated(PBB, Bucket, Key, Bin). +%% ReadResult1 = riakc_pb_socket:get(PBB, Bucket, Key), +%% ?assertMatch({ok, _}, ReadResult1), + +%% %% The following check appears to be the best we can do. If a 2.x source +%% %% sends a typed bucket to the 1.x sink, the put will occur. +%% %% The bucket is undefined to the interfaces, but some parts of it +%% %% appear to be written to the sink node. Since we cannot check using pb, +%% %% here we at least make sure we haven't written over an existing default +%% %% bucket with data from a typed bucket of the same name. +%% ensure_bucket_not_updated(PBB, Bucket, Key, Bin). %% @doc Turn on Realtime replication on the cluster lead by LeaderA. %% The clusters must already have been named and connected. enable_rt(LeaderA, ANodes) -> - repl_util:enable_realtime(LeaderA, "B"), + repl_util:enable_realtime(LeaderA, "2"), rt:wait_until_ring_converged(ANodes), - repl_util:start_realtime(LeaderA, "B"), + repl_util:start_realtime(LeaderA, "2"), rt:wait_until_ring_converged(ANodes). %% @doc Turn off Realtime replication on the cluster lead by LeaderA. disable_rt(LeaderA, ANodes) -> - repl_util:disable_realtime(LeaderA, "B"), + repl_util:disable_realtime(LeaderA, "2"), rt:wait_until_ring_converged(ANodes), - repl_util:stop_realtime(LeaderA, "B"), + repl_util:stop_realtime(LeaderA, "2"), rt:wait_until_ring_converged(ANodes). %% @doc Turn on fullsync replication on the cluster lead by LeaderA. %% The clusters must already have been named and connected. enable_fullsync(LeaderA, ANodes) -> - repl_util:enable_fullsync(LeaderA, "B"), + repl_util:enable_fullsync(LeaderA, "2"), rt:wait_until_ring_converged(ANodes). -%% @doc Connect two clusters using a given name. -connect_cluster(Source, Port, Name) -> - lager:info("Connecting ~p to ~p for cluster ~p.", - [Source, Port, Name]), - repl_util:connect_cluster(Source, "127.0.0.1", Port), - ?assertEqual(ok, repl_util:wait_for_connection(Source, Name)). +%% %% @doc Connect two clusters using a given name. +%% connect_cluster(Source, Port, Name) -> +%% lager:info("Connecting ~p to ~p for cluster ~p.", +%% [Source, Port, Name]), +%% repl_util:connect_cluster(Source, "127.0.0.1", Port), +%% ?assertEqual(ok, repl_util:wait_for_connection(Source, Name)). %% @doc Connect two clusters for replication using their respective leader nodes. connect_clusters(LeaderA, LeaderB) -> Port = repl_util:get_port(LeaderB), lager:info("connect cluster A:~p to B on port ~p", [LeaderA, Port]), repl_util:connect_cluster(LeaderA, "127.0.0.1", Port), - ?assertEqual(ok, repl_util:wait_for_connection(LeaderA, "B")). + ?assertEqual(ok, repl_util:wait_for_connection(LeaderA, "2")). cluster_conf() -> [ @@ -346,58 +355,58 @@ cluster_conf() -> ]} ]. -deploy_nodes(NumNodes, current) -> - rt_cluster:deploy_nodes(NumNodes, cluster_conf()); -deploy_nodes(_, mixed) -> - Conf = cluster_conf(), - rt_cluster:deploy_nodes([{current, Conf}, {previous, Conf}]). +%% deploy_nodes(NumNodes, current) -> +%% rt_cluster:deploy_nodes(NumNodes, cluster_conf()); +%% deploy_nodes(_, mixed) -> +%% Conf = cluster_conf(), +%% rt_cluster:deploy_nodes([{current, Conf}, {previous, Conf}]). %% @doc Create two clusters of 1 node each and connect them for replication: %% Cluster "A" -> cluster "B" -make_clusters(Type) -> - NumNodes = rt_config:get(num_nodes, 2), - ClusterASize = rt_config:get(cluster_a_size, 1), +%% make_clusters(Type) -> +%% %% NumNodes = rt_config:get(num_nodes, 2), +%% %% ClusterASize = rt_config:get(cluster_a_size, 1), - lager:info("Deploy ~p nodes", [NumNodes]), - Nodes = deploy_nodes(NumNodes, Type), - {ANodes, BNodes} = lists:split(ClusterASize, Nodes), - lager:info("ANodes: ~p", [ANodes]), - lager:info("BNodes: ~p", [BNodes]), +%% lager:info("Deploy ~p nodes", [NumNodes]), +%% Nodes = deploy_nodes(NumNodes, Type), +%% {ANodes, BNodes} = lists:split(ClusterASize, Nodes), +%% lager:info("ANodes: ~p", [ANodes]), +%% lager:info("BNodes: ~p", [BNodes]), - lager:info("Build cluster A"), - repl_util:make_cluster(ANodes), +%% lager:info("Build cluster A"), +%% repl_util:make_cluster(ANodes), - lager:info("Build cluster B"), - repl_util:make_cluster(BNodes), +%% lager:info("Build cluster B"), +%% repl_util:make_cluster(BNodes), - AFirst = hd(ANodes), - BFirst = hd(BNodes), +%% AFirst = hd(ANodes), +%% BFirst = hd(BNodes), - %% Name the clusters - repl_util:name_cluster(AFirst, "A"), - repl_util:name_cluster(BFirst, "B"), +%% %% Name the clusters +%% repl_util:name_cluster(AFirst, "A"), +%% repl_util:name_cluster(BFirst, "B"), - lager:info("Waiting for convergence."), - rt:wait_until_ring_converged(ANodes), - rt:wait_until_ring_converged(BNodes), +%% lager:info("Waiting for convergence."), +%% rt:wait_until_ring_converged(ANodes), +%% rt:wait_until_ring_converged(BNodes), - lager:info("Waiting for transfers to complete."), - rt:wait_until_transfers_complete(ANodes), - rt:wait_until_transfers_complete(BNodes), +%% lager:info("Waiting for transfers to complete."), +%% rt:wait_until_transfers_complete(ANodes), +%% rt:wait_until_transfers_complete(BNodes), - %% get the leader for the first cluster - lager:info("waiting for leader to converge on cluster A"), - ?assertEqual(ok, repl_util:wait_until_leader_converge(ANodes)), +%% %% get the leader for the first cluster +%% lager:info("waiting for leader to converge on cluster A"), +%% ?assertEqual(ok, repl_util:wait_until_leader_converge(ANodes)), - %% get the leader for the second cluster - lager:info("waiting for leader to converge on cluster B"), - ?assertEqual(ok, repl_util:wait_until_leader_converge(BNodes)), +%% %% get the leader for the second cluster +%% lager:info("waiting for leader to converge on cluster B"), +%% ?assertEqual(ok, repl_util:wait_until_leader_converge(BNodes)), - ALeader = repl_util:get_leader(hd(ANodes)), - BLeader = repl_util:get_leader(hd(BNodes)), +%% ALeader = repl_util:get_leader(hd(ANodes)), +%% BLeader = repl_util:get_leader(hd(BNodes)), - lager:info("ALeader: ~p BLeader: ~p", [ALeader, BLeader]), - {ALeader, BLeader, ANodes, BNodes}. +%% lager:info("ALeader: ~p BLeader: ~p", [ALeader, BLeader]), +%% {ALeader, BLeader, ANodes, BNodes}. make_pbget_fun(Pid, Bucket, Key, Bin) -> fun() -> @@ -410,9 +419,9 @@ make_pbget_fun(Pid, Bucket, Key, Bin) -> end end. -ensure_bucket_not_sent(Pid, Bucket, Key) -> - Results = [ assert_bucket_not_found(Pid, Bucket, Key) || _I <- lists:seq(1, ?ENSURE_READ_ITERATIONS)], - ?assertEqual(false, lists:member(false, Results)). +%% ensure_bucket_not_sent(Pid, Bucket, Key) -> +%% Results = [ assert_bucket_not_found(Pid, Bucket, Key) || _I <- lists:seq(1, ?ENSURE_READ_ITERATIONS)], +%% ?assertEqual(false, lists:member(false, Results)). ensure_bucket_not_updated(Pid, Bucket, Key, Bin) -> Results = [ value_unchanged(Pid, Bucket, Key, Bin) || _I <- lists:seq(1, ?ENSURE_READ_ITERATIONS)], @@ -429,7 +438,6 @@ value_unchanged(Pid, Bucket, Key, Bin) -> end, timer:sleep(?ENSURE_READ_INTERVAL). - assert_bucket_not_found(Pid, Bucket, Key) -> case riakc_pb_socket:get(Pid, Bucket, Key) of {error, notfound} -> From 97218d981a650f4eee381e4568a3a28d9d0b7166 Mon Sep 17 00:00:00 2001 From: Kelly McLaughlin Date: Thu, 18 Dec 2014 23:40:06 -0700 Subject: [PATCH 044/157] Use metadata property and change confirm arity to 1 Change the confirm API to only accept a single input of an rt_properties record. The metadata formerly passed as the second parameter to confirm is now in the previously unused metadata property field. Change all tests that have been converted thus far to the new API. Also remove an unused properties field from the rt_properties record. --- src/riak_test.erl | 3 ++- src/riak_test_escript.erl | 2 +- src/riak_test_runner.erl | 18 +++++++----------- src/rt_cluster.erl | 6 +++--- src/rt_properties.erl | 3 --- tests/always_fail_test.erl | 9 ++++++--- tests/always_pass_test.erl | 9 ++++++--- tests/replication/repl_bucket_types.erl | 14 +++++++------- tests/secondary_index_tests.erl | 13 ++++++++----- tests/verify_listkeys.erl | 9 ++++++--- 10 files changed, 46 insertions(+), 40 deletions(-) diff --git a/src/riak_test.erl b/src/riak_test.erl index a95e81c43..edd8fbe78 100644 --- a/src/riak_test.erl +++ b/src/riak_test.erl @@ -21,4 +21,5 @@ %% Define the riak_test behavior --callback confirm(rt_properties:properties(), proplists:proplist()) -> pass | fail. +%% -callback confirm(rt_properties:properties()) -> pass | fail. +-callback confirm() -> pass | fail. diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index 0a6eea28d..dc61b21e9 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -390,7 +390,7 @@ is_runnable_test(TestModule) -> {Mod, Fun} = riak_test_runner:function_name(confirm, TestModule), code:ensure_loaded(Mod), erlang:function_exported(Mod, Fun, 0) orelse - erlang:function_exported(Mod, Fun, 2). + erlang:function_exported(Mod, Fun, 1). get_group_tests(Tests, Groups) -> lists:filter(fun(Test) -> diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index 22616d809..16c9079da 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -52,7 +52,6 @@ -record(state, {test_module :: atom(), properties :: proplists:proplist(), - metadata :: term(), backend :: atom(), test_timeout :: integer(), execution_pid :: pid(), @@ -97,6 +96,8 @@ init([TestModule, Backend, Properties]) -> {platform, <<"local">>}, {version, rt:get_version()}, {project, Project}], + {ok, UpdProperties} = + rt_properties:set(metadata, MetaData, Properties), TestTimeout = rt_config:get(test_timeout, rt_config:get(rt_max_wait_time)), SetupModFun = function_name(setup, TestModule, 2, rt_cluster), {ConfirmMod, _} = ConfirmModFun = function_name(confirm, TestModule), @@ -104,8 +105,7 @@ init([TestModule, Backend, Properties]) -> rt_properties:get(valid_backends, Properties)), PreReqCheck = check_prereqs(ConfirmMod), State = #state{test_module=TestModule, - properties=Properties, - metadata=MetaData, + properties=UpdProperties, backend=Backend, test_timeout=TestTimeout, setup_modfun=SetupModFun, @@ -181,7 +181,6 @@ execute({nodes_deployed, _}, State) -> properties=Properties, setup_modfun={SetupMod, SetupFun}, confirm_modfun=ConfirmModFun, - metadata=MetaData, test_timeout=TestTimeout} = State, lager:notice("Running ~s", [TestModule]), @@ -190,11 +189,10 @@ execute({nodes_deployed, _}, State) -> %% required by the test properties. The cluster information is placed %% into the properties record and returned by the `setup' function. UpdState = - case SetupMod:SetupFun(Properties, MetaData) of + case SetupMod:SetupFun(Properties) of {ok, UpdProperties} -> Pid = spawn_link(test_fun(UpdProperties, ConfirmModFun, - MetaData, self())), State#state{execution_pid=Pid, properties=UpdProperties, @@ -245,7 +243,6 @@ wait_for_upgrade(nodes_upgraded, State) -> #state{properties=Properties, confirm_modfun=ConfirmModFun, current_version=CurrentVersion, - metadata=MetaData, test_timeout=TestTimeout} = State, %% Update the `current_version' in the properties record @@ -256,7 +253,6 @@ wait_for_upgrade(nodes_upgraded, State) -> %% a call to an exported function in `rt_cluster' Pid = spawn_link(test_fun(UpdProperties, ConfirmModFun, - MetaData, self())), UpdState = State#state{execution_pid=Pid, properties=UpdProperties}, @@ -282,13 +278,13 @@ wait_for_upgrade(_Event, _From, _State) -> %%% Internal functions %%%=================================================================== --spec test_fun(rt_properties:properties(), {atom(), atom()}, proplists:proplist(), pid()) -> +-spec test_fun(rt_properties:properties(), {atom(), atom()}, pid()) -> function(). -test_fun(Properties, {ConfirmMod, ConfirmFun}, MetaData, NotifyPid) -> +test_fun(Properties, {ConfirmMod, ConfirmFun}, NotifyPid) -> fun() -> %% Exceptions and their handling sucks, but eunit throws %% errors `erlang:error' so here we are - try ConfirmMod:ConfirmFun(Properties, MetaData) of + try ConfirmMod:ConfirmFun(Properties) of TestResult -> ?MODULE:send_event(NotifyPid, test_result(TestResult)) catch diff --git a/src/rt_cluster.erl b/src/rt_cluster.erl index 011c2c1b5..fea7e10a1 100644 --- a/src/rt_cluster.erl +++ b/src/rt_cluster.erl @@ -22,7 +22,7 @@ -include_lib("eunit/include/eunit.hrl"). -export([properties/0, - setup/2, + setup/1, augment_config/3, clean_cluster/1, join_cluster/2, @@ -42,9 +42,9 @@ properties() -> rt_properties:new(). --spec setup(rt_properties:properties(), proplists:proplist()) -> +-spec setup(rt_properties:properties()) -> {ok, rt_properties:properties()} | {error, term()}. -setup(Properties, _MetaData) -> +setup(Properties) -> case form_clusters(Properties) of {ok, ClusterNodes} -> maybe_wait_for_transfers(rt_properties:get(node_ids, Properties), diff --git a/src/rt_properties.erl b/src/rt_properties.erl index 4f63031ea..712ab9750 100644 --- a/src/rt_properties.erl +++ b/src/rt_properties.erl @@ -37,7 +37,6 @@ node_map :: [{string(), node()}], node_count=3 :: non_neg_integer(), metadata=[] :: proplists:proplist(), - properties=[] :: proplists:proplist(), rolling_upgrade=false :: boolean(), start_version="head" :: string(), current_version :: string(), @@ -208,8 +207,6 @@ field_index(node_count) -> ?RT_PROPERTIES.node_count; field_index(metadata) -> ?RT_PROPERTIES.metadata; -field_index(properties) -> - ?RT_PROPERTIES.properties; field_index(rolling_upgrade) -> ?RT_PROPERTIES.rolling_upgrade; field_index(start_version) -> diff --git a/tests/always_fail_test.erl b/tests/always_fail_test.erl index 6734ec284..b575f17d0 100644 --- a/tests/always_fail_test.erl +++ b/tests/always_fail_test.erl @@ -1,8 +1,11 @@ %% @doc A test that always returns `fail'. -module(always_fail_test). --export([confirm/2]). --spec confirm(rt_properties:properties(), proplists:proplist()) -> pass | fail. -confirm(_Properties, _MD) -> +%% -behaviour(riak_test). + +-export([confirm/1]). + +-spec confirm(rt_properties:properties()) -> pass | fail. +confirm(_Properties) -> lager:info("Running test confirm function"), fail. diff --git a/tests/always_pass_test.erl b/tests/always_pass_test.erl index 8b7a3aaa8..bc77057f3 100644 --- a/tests/always_pass_test.erl +++ b/tests/always_pass_test.erl @@ -1,15 +1,18 @@ %% @doc A test that always returns `fail'. -module(always_pass_test). + +%% -behaviour(riak_test). + -export([properties/0, - confirm/2]). + confirm/1]). -include_lib("eunit/include/eunit.hrl"). properties() -> rt_properties:new([{make_cluster, false}]). --spec confirm(rt_properties:properties(), proplists:proplist()) -> pass | fail. -confirm(_Properties, _MD) -> +-spec confirm(rt_properties:properties()) -> pass | fail. +confirm(_Properties) -> lager:info("Running test confirm function"), ?assertEqual(1,1), pass. diff --git a/tests/replication/repl_bucket_types.erl b/tests/replication/repl_bucket_types.erl index 8ce3346bd..09c1df081 100644 --- a/tests/replication/repl_bucket_types.erl +++ b/tests/replication/repl_bucket_types.erl @@ -5,11 +5,11 @@ %% %% ------------------------------------------------------------------- --behaviour(riak_test). +%% -behaviour(riak_test). --export([setup/2, +-export([setup/1, properties/0, - confirm/2]). + confirm/1]). -include_lib("eunit/include/eunit.hrl"). -test_type([bucket_types, replication]). @@ -31,8 +31,8 @@ properties() -> {wait_for_transfers, true}, {required_services, [riak_kv, riak_repl]}]). -setup(Properties, _MD) -> - case rt_cluster:setup(Properties, _MD) of +setup(Properties) -> + case rt_cluster:setup(Properties) of {ok, UpdProperties} -> Clusters = rt_properties:get(clusters, UpdProperties), [LeaderA, LeaderB] = [rt_cluster_info:get(leader, Cluster) @@ -43,8 +43,8 @@ setup(Properties, _MD) -> Error end. --spec confirm(rt_properties:properties(), proplists:proplist()) -> pass | fail. -confirm(Properties, _MD) -> +-spec confirm(rt_properties:properties()) -> pass | fail. +confirm(Properties) -> Clusters = rt_properties:get(clusters, Properties), NodeMap = rt_properties:get(node_map, Properties), diff --git a/tests/secondary_index_tests.erl b/tests/secondary_index_tests.erl index 0b9b99de5..6891476c8 100644 --- a/tests/secondary_index_tests.erl +++ b/tests/secondary_index_tests.erl @@ -1,3 +1,4 @@ +-module(secondary_index_tests). %% ------------------------------------------------------------------- %% %% Copyright (c) 2012 Basho Technologies, Inc. @@ -17,9 +18,11 @@ %% under the License. %% %% ------------------------------------------------------------------- --module(secondary_index_tests). --export([confirm/2, - properties/0]). + +%% -behaviour(riak_test). + +-export([properties/0, + confirm/1]). -export([put_an_object/3, put_an_object/5, int_to_key/1, stream_pb/3, stream_pb/4, pb_query/4, http_query/3, http_query/4, http_stream/4, int_to_field1_bin/1, url/2, @@ -46,8 +49,8 @@ config() -> [{riak_kv, [{secondary_index_sort_default, false}]}, {riak_core, [{handoff_concurrency, 11}]}]. --spec confirm(rt_properties:properties(), proplists:proplist()) -> pass | fail. -confirm(Properties, _MD) -> +-spec confirm(rt_properties:properties()) -> pass | fail. +confirm(Properties) -> NodeMap= rt_properties:get(node_map, Properties), Nodes = [rt_node:node_name(NodeId, NodeMap) || NodeId <- rt_properties:get(node_ids, Properties)], diff --git a/tests/verify_listkeys.erl b/tests/verify_listkeys.erl index 91c5efa1d..27c2b6aa0 100644 --- a/tests/verify_listkeys.erl +++ b/tests/verify_listkeys.erl @@ -18,9 +18,12 @@ %% %% ------------------------------------------------------------------- -module(verify_listkeys). + %% -behavior(riak_test). + -export([properties/0, - confirm/2]). + confirm/1]). + -include_lib("eunit/include/eunit.hrl"). -define(BUCKET, <<"listkeys_bucket">>). @@ -33,8 +36,8 @@ properties() -> rt_properties:new([{node_count, 4}, {make_cluster, false}]). --spec confirm(rt_properties:properties(), proplists:proplist()) -> pass | fail. -confirm(Properties, _MD) -> +-spec confirm(rt_properties:properties()) -> pass | fail. +confirm(Properties) -> [NodeId1, NodeId2, NodeId3 | _] = NodeIds = rt_properties:get(node_ids, Properties), NodeMap = rt_properties:get(node_map, Properties), Nodes = [rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds], From e4c700891feaf0c1e96e74b0b6c956b75f54fc3c Mon Sep 17 00:00:00 2001 From: Kelly McLaughlin Date: Mon, 22 Dec 2014 13:26:54 -0700 Subject: [PATCH 045/157] Convert a few tests to new API and fix a some framework bugs --- src/node_manager.erl | 10 +++++---- src/riak_test_runner.erl | 1 + tests/always_fail_test.erl | 9 +++++++- tests/always_pass_test.erl | 5 +++-- tests/bucket_types.erl | 19 ++++++++++++----- tests/http_bucket_types.erl | 15 ++++++++----- tests/secondary_index_tests.erl | 3 +++ tests/verify_2i_aae.erl | 38 +++++++++++++++++++++++---------- 8 files changed, 72 insertions(+), 28 deletions(-) diff --git a/src/node_manager.erl b/src/node_manager.erl index fa150d610..92244f5f8 100644 --- a/src/node_manager.erl +++ b/src/node_manager.erl @@ -80,7 +80,7 @@ handle_call(stop, _From, State) -> handle_cast({reserve_nodes, Count, Versions, NotifyFun}, State) -> {Result, UpdState} = reserve(Count, Versions, State), - NotifyFun({nodes, Result, State#state.node_map}), + NotifyFun(Result), {noreply, UpdState}; handle_cast({deploy_nodes, Nodes, Version, Config, Services, NotifyFun}, State) -> Result = deploy(Nodes, State#state.node_map, Version, Config, Services), @@ -158,8 +158,9 @@ reserve(Count, Versions, State=#state{nodes_available=NodesAvailable, case versions_available(Count, Versions, VersionMap) of true -> UpdNodesDeployed = lists:sort(NodesDeployed ++ NodesAvailable), - {NodesAvailable, State#state{nodes_available=[], - nodes_deployed=UpdNodesDeployed}}; + Result = {nodes, NodesAvailable, State#state.node_map}, + {Result, State#state{nodes_available=[], + nodes_deployed=UpdNodesDeployed}}; false -> {insufficient_versions_available, State} end; @@ -172,7 +173,8 @@ reserve(Count, Versions, State=#state{nodes_available=NodesAvailable, UpdNodesDeployed = lists:sort(NodesDeployed ++ Reserved), UpdState = State#state{nodes_available=UpdNodesAvailable, nodes_deployed=UpdNodesDeployed}, - {Reserved, UpdState}; + Result = {nodes, Reserved, UpdState#state.node_map}, + {Result, UpdState}; false -> {insufficient_versions_available, State} end. diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index 16c9079da..15288f6b5 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -95,6 +95,7 @@ init([TestModule, Backend, Properties]) -> MetaData = [{id, -1}, {platform, <<"local">>}, {version, rt:get_version()}, + {backend, Backend}, {project, Project}], {ok, UpdProperties} = rt_properties:set(metadata, MetaData, Properties), diff --git a/tests/always_fail_test.erl b/tests/always_fail_test.erl index b575f17d0..3cb4fd368 100644 --- a/tests/always_fail_test.erl +++ b/tests/always_fail_test.erl @@ -3,9 +3,16 @@ %% -behaviour(riak_test). --export([confirm/1]). +-include_lib("eunit/include/eunit.hrl"). + +-export([properties/0, + confirm/1]). + +properties() -> + rt_properties:new([{make_cluster, false}]). -spec confirm(rt_properties:properties()) -> pass | fail. confirm(_Properties) -> lager:info("Running test confirm function"), + ?assertEqual(1,2), fail. diff --git a/tests/always_pass_test.erl b/tests/always_pass_test.erl index bc77057f3..b779c96aa 100644 --- a/tests/always_pass_test.erl +++ b/tests/always_pass_test.erl @@ -12,7 +12,8 @@ properties() -> rt_properties:new([{make_cluster, false}]). -spec confirm(rt_properties:properties()) -> pass | fail. -confirm(_Properties) -> - lager:info("Running test confirm function"), +confirm(Properties) -> + NodeIds = rt_properties:get(node_ids, Properties), + lager:notice("~p is using ~p nodes", [?MODULE, length(NodeIds)]), ?assertEqual(1,1), pass. diff --git a/tests/bucket_types.erl b/tests/bucket_types.erl index 95290cfd5..715f2cc07 100644 --- a/tests/bucket_types.erl +++ b/tests/bucket_types.erl @@ -1,6 +1,11 @@ -module(bucket_types). --export([properties/0,confirm/2, mapred_modfun/3, mapred_modfun_type/3]). +%% -behaviour(riak_test). + +-export([properties/0, + confirm/1, + mapred_modfun/3, + mapred_modfun_type/3]). -include_lib("eunit/include/eunit.hrl"). -include("rt.hrl"). @@ -14,12 +19,16 @@ properties() -> rt_properties:new([{node_count, 4}, {config, CustomConfig}]). -confirm(Properties, _MD) -> - Nodes = rt_properties:get(nodes, Properties), +-spec confirm(rt_properties:properties()) -> pass | fail. +confirm(Properties) -> + NodeIds = rt_properties:get(node_ids, Properties), + NodeMap = rt_properties:get(node_map, Properties), + Nodes = [rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds], Node = hd(Nodes), + application:start(inets), - RMD = riak_test_runner:metadata(), + RMD = rt_properties:get(metadata, Properties), HaveIndexes = case proplists:get_value(backend, RMD) of undefined -> false; %% default is da 'cask bitcask -> false; @@ -406,7 +415,7 @@ confirm(Properties, _MD) -> undefined, true}])), riakc_pb_socket:stop(PB), - ok. + pass. accumulate(ReqID) -> receive diff --git a/tests/http_bucket_types.erl b/tests/http_bucket_types.erl index f4cd169b9..df72350af 100644 --- a/tests/http_bucket_types.erl +++ b/tests/http_bucket_types.erl @@ -1,6 +1,9 @@ -module(http_bucket_types). --export([properties/0, confirm/2, mapred_modfun/3, mapred_modfun_type/3]). +-export([properties/0, + confirm/1, + mapred_modfun/3, + mapred_modfun_type/3]). -include_lib("eunit/include/eunit.hrl"). -include_lib("riakc/include/riakc.hrl"). @@ -15,14 +18,16 @@ properties() -> rt_properties:new([{node_count, 1}, {config, CustomConfig}]). - -confirm(Properties, _MD) -> - Nodes = rt_properties:get(nodes, Properties), +-spec confirm(rt_properties:properties()) -> pass | fail. +confirm(Properties) -> + NodeIds = rt_properties:get(node_ids, Properties), + NodeMap = rt_properties:get(node_map, Properties), + Nodes = [rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds], Node = hd(Nodes), application:start(ibrowse), - RMD = riak_test_runner:metadata(), + RMD = rt_properties:get(metadata, Properties), HaveIndexes = case proplists:get_value(backend, RMD) of undefined -> false; %% default is da 'cask bitcask -> false; diff --git a/tests/secondary_index_tests.erl b/tests/secondary_index_tests.erl index 6891476c8..78d25a87d 100644 --- a/tests/secondary_index_tests.erl +++ b/tests/secondary_index_tests.erl @@ -27,10 +27,13 @@ stream_pb/3, stream_pb/4, pb_query/4, http_query/3, http_query/4, http_stream/4, int_to_field1_bin/1, url/2, assertExactQuery/5, assertRangeQuery/7]). + -include("rt.hrl"). -include_lib("eunit/include/eunit.hrl"). -include_lib("riakc/include/riakc.hrl"). +-test_type(['2i']). + -define(KEYS(A), [int_to_key(A)]). -define(KEYS(A,B), [int_to_key(N) || N <- lists:seq(A,B)]). -define(KEYS(A,B,C), [int_to_key(N) || N <- lists:seq(A,B), C]). diff --git a/tests/verify_2i_aae.erl b/tests/verify_2i_aae.erl index 9284a4640..36b051d73 100644 --- a/tests/verify_2i_aae.erl +++ b/tests/verify_2i_aae.erl @@ -18,11 +18,14 @@ %% %% ------------------------------------------------------------------- -module(verify_2i_aae). --behaviour(riak_test). --export([confirm/0]). + +%% -behaviour(riak_test). + -include_lib("eunit/include/eunit.hrl"). -include_lib("riakc/include/riakc.hrl"). +-test_type(['2i']). + %% Make it multi-backend compatible. -define(BUCKETS, [<<"eleveldb1">>, <<"memory1">>]). -define(NUM_ITEMS, 1000). @@ -30,13 +33,26 @@ -define(SCAN_BATCH_SIZE, 100). -define(N_VAL, 3). -confirm() -> - [Node1] = rt_cluster:build_cluster(1, - [{riak_kv, - [{anti_entropy, {off, []}}, - {anti_entropy_build_limit, {100, 500}}, - {anti_entropy_concurrency, 100}, - {anti_entropy_tick, 200}]}]), +-export([properties/0, + confirm/1]). + +properties() -> + Config = [{riak_kv, + [{anti_entropy, {off, []}}, + {anti_entropy_build_limit, {100, 500}}, + {anti_entropy_concurrency, 100}, + {anti_entropy_tick, 200}]}] ++ rt_properties:default_config(), + rt_properties:new([{node_count, 1}, + {make_cluster, false}, + {config, Config}]). + +-spec confirm(rt_properties:properties()) -> pass | fail. +confirm(Properties) -> + NodeIds = rt_properties:get(node_ids, Properties), + NodeMap = rt_properties:get(node_map, Properties), + Nodes = [rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds], + Node1 = hd(Nodes), + rt_intercept:load_code(Node1), rt_intercept:add(Node1, {riak_object, @@ -86,7 +102,7 @@ check_lost_objects(Node1, PBC, NumItems, NumDel) -> DelRange = lists:seq(NumItems-NumDel+1, NumItems), lager:info("Deleting ~b objects without updating indexes", [NumDel]), [del_obj(PBC, Bucket, N) || N <- DelRange, Bucket <- ?BUCKETS], - DelKeys = [to_key(N) || N <- DelRange], + DelKeys = [to_key(N) || N <- DelRange], [rt:wait_until(fun() -> rt_pb:pbc_really_deleted(PBC, Bucket, DelKeys) end) || Bucket <- ?BUCKETS], %% Verify they are damaged @@ -172,7 +188,7 @@ run_2i_repair(Node1) -> RepairPid = rpc:call(Node1, erlang, whereis, [riak_kv_2i_aae]), lager:info("Wait for repair process to finish"), Mon = monitor(process, RepairPid), - MaxWaitTime = rt_config:get(rt_max_wait_time), + MaxWaitTime = 120000, receive {'DOWN', Mon, _, _, Status} -> lager:info("Status: ~p", [Status]), From 86e800bd05881ba974080d4bd9ae75bd5f2956df Mon Sep 17 00:00:00 2001 From: Kelly McLaughlin Date: Fri, 9 Jan 2015 11:53:38 -0700 Subject: [PATCH 046/157] Fix several framework bugs found while testing * Address two issues related to bucket type creation in rt_cluster * Address an issue with rt:stream_cmd_loop using an outdated rt:config call * Adjust the expected arity of the setup function in riak_test_runner from 2 to 1 * Change rtdev:get_deps/0 to use the default_version from the config instead of a hardcoded path --- src/riak_test_runner.erl | 2 +- src/rt.erl | 50 ++++++++++++++++++++-------------------- src/rt_cluster.erl | 4 ++-- src/rtdev.erl | 3 ++- 4 files changed, 30 insertions(+), 29 deletions(-) diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index 15288f6b5..f7bdd719e 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -100,7 +100,7 @@ init([TestModule, Backend, Properties]) -> {ok, UpdProperties} = rt_properties:set(metadata, MetaData, Properties), TestTimeout = rt_config:get(test_timeout, rt_config:get(rt_max_wait_time)), - SetupModFun = function_name(setup, TestModule, 2, rt_cluster), + SetupModFun = function_name(setup, TestModule, 1, rt_cluster), {ConfirmMod, _} = ConfirmModFun = function_name(confirm, TestModule), BackendCheck = check_backend(Backend, rt_properties:get(valid_backends, Properties)), diff --git a/src/rt.erl b/src/rt.erl index 1aa661ba8..9e0de3c4e 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -248,31 +248,31 @@ Port = open_port({spawn, binary_to_list(iolist_to_binary(Cmd))}, [stream, stderr stream_cmd_loop(Port, "", "", now()). stream_cmd_loop(Port, Buffer, NewLineBuffer, Time={_MegaSecs, Secs, _MicroSecs}) -> -receive -{Port, {data, Data}} -> - {_, Now, _} = now(), - NewNewLineBuffer = case Now > Secs of - true -> - lager:info(NewLineBuffer), - ""; - _ -> - NewLineBuffer - end, - case rt:str(Data, "\n") of - true -> - lager:info(NewNewLineBuffer), - Tokens = string:tokens(Data, "\n"), - [ lager:info(Token) || Token <- Tokens ], - stream_cmd_loop(Port, Buffer ++ NewNewLineBuffer ++ Data, "", Time); - _ -> - stream_cmd_loop(Port, Buffer, NewNewLineBuffer ++ Data, now()) - end; -{Port, {exit_status, Status}} -> - catch port_close(Port), - {Status, Buffer} -after rt:config(rt_max_wait_time) -> - {-1, Buffer} -end. + receive + {Port, {data, Data}} -> + {_, Now, _} = now(), + NewNewLineBuffer = case Now > Secs of + true -> + lager:info(NewLineBuffer), + ""; + _ -> + NewLineBuffer + end, + case rt:str(Data, "\n") of + true -> + lager:info(NewNewLineBuffer), + Tokens = string:tokens(Data, "\n"), + [ lager:info(Token) || Token <- Tokens ], + stream_cmd_loop(Port, Buffer ++ NewNewLineBuffer ++ Data, "", Time); + _ -> + stream_cmd_loop(Port, Buffer, NewNewLineBuffer ++ Data, now()) + end; + {Port, {exit_status, Status}} -> + catch port_close(Port), + {Status, Buffer} + after rt_config:get(rt_max_wait_time) -> + {-1, Buffer} + end. %%%=================================================================== %%% Remote code management diff --git a/src/rt_cluster.erl b/src/rt_cluster.erl index fea7e10a1..74bdaf1df 100644 --- a/src/rt_cluster.erl +++ b/src/rt_cluster.erl @@ -71,7 +71,7 @@ create_bucket_types([Cluster], Properties, BucketTypes) -> NodeMap = rt_properties:get(node_map, Properties), NodeIds = rt_cluster_info:get(node_ids, Cluster), Nodes = [rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds], - lists:foldl(fun maybe_create_bucket_type/2, [{Nodes, 1}], BucketTypes); + lists:foldl(fun maybe_create_bucket_type/2, {Nodes, 1}, BucketTypes); create_bucket_types(Clusters, Properties, BucketTypes) -> NodeMap = rt_properties:get(node_map, Properties), [begin @@ -96,7 +96,7 @@ maybe_create_bucket_type({TypeName, TypeProps}, {Nodes, _ClusterIndex}) -> -spec prepare_clusters([list(string())], rt_properties:properties()) -> [rt_cluster_info:cluster_info()]. prepare_clusters([ClusterNodes], _Properties) -> - rt_cluster_info:new([{node_ids, ClusterNodes}]); + [rt_cluster_info:new([{node_ids, ClusterNodes}])]; prepare_clusters(ClusterNodesList, Properties) -> %% If the count of clusters is > 1 the assumption is made that the %% test is exercising replication and some extra diff --git a/src/rtdev.erl b/src/rtdev.erl index 2d4a6e9e9..27b340f88 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -58,7 +58,8 @@ -define(SCRATCH_DIR, (rt_config:get(rt_scratch_dir))). get_deps() -> - lists:flatten(io_lib:format("~s/dev1/lib", [filename:join(?PATH, "head")])). + DefaultVersionPath = filename:join(?PATH, rt_config:get(default_version)), + lists:flatten(io_lib:format("~s/dev1/lib", [DefaultVersionPath])). riakcmd(Path, N, Cmd) -> ExecName = rt_config:get(exec_name, "riak"), From a1f5df8fc52372f0eb64e6d882d00148f57835d3 Mon Sep 17 00:00:00 2001 From: Kelly McLaughlin Date: Fri, 9 Jan 2015 12:12:57 -0700 Subject: [PATCH 047/157] Add an external_properties field to rt_properties record Add an external_properties field to rt_properties record to facilitate testing with applications built on Riak. This field can be use for arbitrary data by external applications and could be useful for testing things like Riak CS. --- src/rt_properties.erl | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/rt_properties.erl b/src/rt_properties.erl index 712ab9750..4ac8e17fb 100644 --- a/src/rt_properties.erl +++ b/src/rt_properties.erl @@ -49,7 +49,8 @@ clusters :: [rt_cluster_info:cluster_info()], required_services=[riak_kv] :: [atom()], bucket_types=[] :: bucket_types(), - config=default_config() :: term() + config=default_config() :: term(), + external_properties :: term() % arbitrary properties for 3rd party use }). -type properties() :: #rt_properties_v1{}. @@ -232,4 +233,6 @@ field_index(clusters) -> field_index(required_services) -> ?RT_PROPERTIES.required_services; field_index(config) -> - ?RT_PROPERTIES.config. + ?RT_PROPERTIES.config; +field_index(external_properties) -> + ?RT_PROPERTIES.external_properties. From 8120e9c3ff30b9f5689a5feb882251324f3ab437 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Wed, 21 Jan 2015 12:40:21 -0500 Subject: [PATCH 048/157] WIP: Support execution of old and new style tests * Modifies riak_test_escript to start lager as early as possible * Moves the new rt API to the rt2 module -- allowing old tests to use the old API until ported. Before merge, rt.erl will be modified to largely proxy the the new API structure. * Adds detection of test type (old or new) to riak_test_executor to determine whether or not not allocate nodes automatically * Fixes compilation errors in some test cases --- .gitignore | 2 +- src/riak_test_escript.erl | 45 +- src/riak_test_executor.erl | 6 +- src/riak_test_runner.erl | 32 +- src/rt.erl | 1681 +++++++++++++++++++++++++++------- src/rt2.erl | 786 ++++++++++++++++ src/rt_cs_dev.erl | 33 +- src/rtdev.erl | 2 +- tests/basic_command_line.erl | 3 +- tests/http_bucket_types.erl | 2 + tests/overload.erl | 4 +- tests/verify_snmp.erl | 2 +- 12 files changed, 2224 insertions(+), 374 deletions(-) create mode 100644 src/rt2.erl diff --git a/.gitignore b/.gitignore index 38f7598ff..3d349bbc5 100644 --- a/.gitignore +++ b/.gitignore @@ -15,7 +15,7 @@ doc/ *.jar coverage tags -riak-* +riak* *.png .rebar/ .local_dialyzer_plt diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index dc61b21e9..bfe125b1b 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -29,23 +29,31 @@ -define(HEADER, [<<"Test">>, <<"Result">>, <<"Reason">>, <<"Test Duration">>]). main(Args) -> - {ParsedArgs, HarnessArgs, Tests, _} = prepare(Args), + %% TODO Should we use clique? -jsb + %% Parse command line arguments ... + {ParsedArgs, HarnessArgs, Tests, NonTests} = parse_args(Args), + + %% Configure logging ... + OutDir = proplists:get_value(outdir, ParsedArgs, "log"), + ensure_dir(OutDir), + + lager_setup(OutDir), + + ok = prepare(ParsedArgs, Tests, NonTests), Results = execute(Tests, ParsedArgs, HarnessArgs), finalize(Results, ParsedArgs). -prepare(Args) -> - {ParsedArgs, _, Tests, NonTests} = ParseResults = parse_args(Args), - io:format("Tests to run: ~p~n", [Tests]), +prepare(ParsedArgs, Tests, NonTests) -> + lager:notice("Tests to run: ~p~n", [Tests]), case NonTests of [] -> ok; _ -> - io:format("These modules are not runnable tests: ~p~n", + lager:notice("These modules are not runnable tests: ~p~n", [[NTMod || {NTMod, _} <- NonTests]]) end, ok = erlang_setup(ParsedArgs), - ok = test_setup(ParsedArgs), - ParseResults. + test_setup(). execute(Tests, ParsedArgs, _HarnessArgs) -> OutDir = proplists:get_value(outdir, ParsedArgs), @@ -141,17 +149,17 @@ print_help() -> halt(0). add_deps(Path) -> - {ok, Deps} = file:list_dir(Path), - [code:add_path(lists:append([Path, "/", Dep, "/ebin"])) || Dep <- Deps], - ok. - -test_setup(ParsedArgs) -> - %% File output - OutDir = proplists:get_value(outdir, ParsedArgs, "log"), - ensure_dir(OutDir), - - lager_setup(OutDir), + lager:debug("Adding dep path ~p", [Path]), + case file:list_dir(Path) of + {ok, Deps} -> + [code:add_path(lists:append([Path, "/", Dep, "/ebin"])) || Dep <- Deps], + ok; + {error, Reason} -> + lager:error("Failed to add dep path ~p due to ~p.", [Path, Reason]), + erlang:error(Reason) + end. +test_setup() -> %% Prepare the test harness {NodeIds, NodeMap, VersionMap} = rt_harness:setup(), @@ -218,7 +226,7 @@ load_initial_config(ParsedArgs) -> proplists:get_value(file, ParsedArgs)). shuffle_tests(_, _, [], _, _, _) -> - io:format("No tests are scheduled to run~n"), + lager:error("No tests are scheduled to run~n"), halt(1); shuffle_tests(ParsedArgs, HarnessArgs, Tests, NonTests, undefined, _) -> {ParsedArgs, HarnessArgs, Tests, NonTests}; @@ -512,6 +520,7 @@ format_test_row({Test, Result, Duration}, _Width) -> end. print_summary(TestResults, _CoverResult, Verbose) -> + %% TODO Log vs console output ... -jsb io:format("~nTest Results:~n~n"), {StatusCounts, Width} = lists:foldl(fun test_summary_fun/2, {{0,0,0}, 0}, TestResults), diff --git a/src/riak_test_executor.erl b/src/riak_test_executor.erl index a185c2ce3..0690107a2 100644 --- a/src/riak_test_executor.erl +++ b/src/riak_test_executor.erl @@ -104,12 +104,14 @@ gather_properties(_Event, _State) -> request_nodes(timeout, State) -> #state{pending_tests=[NextTest | _], test_properties=PropertiesList} = State, - lager:notice("Starting next test: ~p", [NextTest]), + lager:debug("Requesting nodes using properties ~p", [PropertiesList]), %% Find the properties for the next pending test {NextTest, TestProps} = lists:keyfind(NextTest, 1, PropertiesList), %% Send async request to node manager VersionsToTest = versions_to_test(TestProps), - node_manager:reserve_nodes(rt_properties:get(node_count, TestProps), + NodeCount = rt_properties:get(node_count, TestProps), + lager:notice("Requesting ~p nodes for the next test, ~p", [NodeCount, NextTest]), + node_manager:reserve_nodes(NodeCount, VersionsToTest, reservation_notify_fun()), {next_state, launch_test, State}; diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index f7bdd719e..e55a33f5a 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -50,7 +50,9 @@ -include("rt.hrl"). -include_lib("eunit/include/eunit.hrl"). +-type test_type() :: {new | old}. -record(state, {test_module :: atom(), + test_type :: test_type(), properties :: proplists:proplist(), backend :: atom(), test_timeout :: integer(), @@ -102,10 +104,15 @@ init([TestModule, Backend, Properties]) -> TestTimeout = rt_config:get(test_timeout, rt_config:get(rt_max_wait_time)), SetupModFun = function_name(setup, TestModule, 1, rt_cluster), {ConfirmMod, _} = ConfirmModFun = function_name(confirm, TestModule), + TestType = case erlang:function_exported(ConfirmMod, ConfirmModFun, 1) of + true -> new; + false -> old + end, BackendCheck = check_backend(Backend, rt_properties:get(valid_backends, Properties)), PreReqCheck = check_prereqs(ConfirmMod), State = #state{test_module=TestModule, + test_type=TestType, properties=UpdProperties, backend=Backend, test_timeout=TestTimeout, @@ -147,7 +154,9 @@ setup(timeout, State=#state{prereq_check=false}) -> notify_executor({fail, prereq_check_failed}, State), cleanup(State), {stop, normal, State}; -setup(timeout, State=#state{backend=Backend, +setup(timeout, State=#state{test_type=TestType, + test_module=TestModule, + backend=Backend, properties=Properties}) -> NewGroupLeader = riak_test_group_leader:new_group_leader(self()), group_leader(NewGroupLeader, self()), @@ -159,12 +168,18 @@ setup(timeout, State=#state{backend=Backend, Services = rt_properties:get(required_services, Properties), {StartVersion, OtherVersions} = test_versions(Properties), Config = rt_backend:set(Backend, rt_properties:get(config, Properties)), - node_manager:deploy_nodes(NodeIds, - StartVersion, - Config, - Services, - notify_fun(self())), - lager:info("Waiting for deploy nodes response at ~p", [self()]), + + case TestType of + new -> + node_manager:deploy_nodes(NodeIds, + StartVersion, + Config, + Services, + notify_fun(self())), + lager:info("Waiting for deploy nodes response at ~p", [self()]); + old -> + lager:warn("Test ~p has not been ported to the new framework.", [TestModule]) + end, %% Set the initial value for `current_version' in the properties record {ok, UpdProperties} = @@ -184,6 +199,7 @@ execute({nodes_deployed, _}, State) -> confirm_modfun=ConfirmModFun, test_timeout=TestTimeout} = State, lager:notice("Running ~s", [TestModule]), + lager:notice("Properties: ~p", [Properties]), StartTime = os:timestamp(), %% Perform test setup which includes clustering of the nodes if @@ -290,7 +306,7 @@ test_fun(Properties, {ConfirmMod, ConfirmFun}, NotifyPid) -> ?MODULE:send_event(NotifyPid, test_result(TestResult)) catch Error:Reason -> - lager:notice("Error: ~p Reason: ~p", [Error, Reason]), + lager:error("Error: ~p Reason: ~p", [Error, Reason]), TestResult = format_eunit_error(Reason), ?MODULE:send_event(NotifyPid, test_result(TestResult)) end diff --git a/src/rt.erl b/src/rt.erl index 9e0de3c4e..48caa303c 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -32,14 +32,27 @@ admin/2, admin/3, assert_nodes_agree_about_ownership/1, + async_start/1, attach/2, attach_direct/2, + brutal_kill/1, + build_cluster/1, + build_cluster/2, + build_cluster/3, + build_clusters/1, + join_cluster/1, capability/2, capability/3, + check_singleton_node/1, check_ibrowse/0, + claimant_according_to/1, + clean_cluster/1, + clean_data_dir/1, + clean_data_dir/2, cmd/1, cmd/2, connection_info/1, + console/2, create_and_activate_bucket_type/3, deploy_nodes/1, deploy_nodes/2, @@ -52,14 +65,25 @@ get_ip/1, get_node_logs/0, get_replica/5, + get_ring/1, get_version/0, + heal/1, + http_url/1, + https_url/1, + httpc/1, + httpc_read/3, + httpc_write/4, is_mixed_cluster/1, + is_pingable/1, + join/2, + leave/1, load_modules_on_nodes/2, log_to_nodes/2, log_to_nodes/3, members_according_to/1, nearest_ringsize/1, owners_according_to/1, + partition/2, partitions_for_node/1, pbc/1, pbc/2, @@ -76,22 +100,26 @@ post_result/2, product/1, priv_dir/0, - random_sublist/2, remove/2, riak/2, riak_repl/2, rpc_get_env/2, - select_random/1, set_backend/1, set_backend/2, set_conf/2, set_advanced_conf/2, setup_harness/2, setup_log_capture/1, - stream_cmd/1, - stream_cmd/2, + slow_upgrade/3, + stream_cmd/1, stream_cmd/2, spawn_cmd/1, spawn_cmd/2, + search_cmd/2, + start/1, + start_and_wait/1, + status_of_according_to/2, + stop/1, + stop_and_wait/1, str/2, systest_read/2, systest_read/3, @@ -113,13 +141,19 @@ wait_until/3, wait_until/2, wait_until/1, + wait_until_aae_trees_built/1, wait_until_all_members/1, wait_until_all_members/2, + wait_until_bucket_props/3, + wait_until_bucket_type_visible/2, wait_until_capability/3, wait_until_capability/4, wait_until_connected/1, wait_until_legacy_ringready/1, + wait_until_owners_according_to/2, wait_until_no_pending_changes/1, + wait_until_nodes_agree_about_ownership/1, + wait_until_nodes_ready/1, wait_until_pingable/1, wait_until_ready/1, wait_until_registered/2, @@ -127,77 +161,254 @@ wait_until_status_ready/1, wait_until_transfers_complete/1, wait_until_unpingable/1, + wait_until_bucket_type_status/3, whats_up/0 ]). -define(HARNESS, (rt_config:get(rt_harness))). priv_dir() -> -LocalPrivDir = "./priv", -%% XXX for some reason, codew:priv_dir returns riak_test/riak_test/priv, -%% which is wrong, so fix it. -DepPrivDir = re:replace(code:priv_dir(riak_test), "riak_test(/riak_test)*", -"riak_test", [{return, list}]), -PrivDir = case {filelib:is_dir(LocalPrivDir), filelib:is_dir(DepPrivDir)} of -{true, _} -> - lager:debug("Local ./priv detected, using that..."), - %% we want an absolute path! - filename:absname(LocalPrivDir); -{false, true} -> - lager:debug("riak_test dependency priv_dir detected, using that..."), - DepPrivDir; -_ -> - ?assertEqual({true, bad_priv_dir}, {false, bad_priv_dir}) -end, - -lager:info("priv dir: ~p -> ~p", [code:priv_dir(riak_test), PrivDir]), -?assert(filelib:is_dir(PrivDir)), -PrivDir. + rt2:priv_dir(). %% @doc gets riak deps from the appropriate harness -spec get_deps() -> list(). -get_deps() -> rt_harness:get_deps(). +get_deps() -> + rt2:get_deps(). %% @doc if String contains Substr, return true. -spec str(string(), string()) -> boolean(). str(String, Substr) -> -case string:str(String, Substr) of -0 -> false; -_ -> true -end. + rt2:str(String, Substr). + +-spec set_conf(atom(), [{string(), string()}]) -> ok. +set_conf(all, NameValuePairs) -> + ?HARNESS:set_conf(all, NameValuePairs); +set_conf(Node, NameValuePairs) -> + stop(Node), + ?assertEqual(ok, rt:wait_until_unpingable(Node)), + ?HARNESS:set_conf(Node, NameValuePairs), + start(Node). + +-spec set_advanced_conf(atom(), [{string(), string()}]) -> ok. +set_advanced_conf(all, NameValuePairs) -> + ?HARNESS:set_advanced_conf(all, NameValuePairs); +set_advanced_conf(Node, NameValuePairs) -> + stop(Node), + ?assertEqual(ok, rt:wait_until_unpingable(Node)), + ?HARNESS:set_advanced_conf(Node, NameValuePairs), + start(Node). + +%% @doc Rewrite the given node's app.config file, overriding the varialbes +%% in the existing app.config with those in `Config'. +update_app_config(all, Config) -> + ?HARNESS:update_app_config(all, Config); +update_app_config(Node, Config) -> + stop(Node), + ?assertEqual(ok, rt:wait_until_unpingable(Node)), + ?HARNESS:update_app_config(Node, Config), + start(Node). %% @doc Helper that returns first successful application get_env result, %% used when different versions of Riak use different app vars for %% the same setting. rpc_get_env(_, []) -> -undefined; + undefined; rpc_get_env(Node, [{App,Var}|Others]) -> -case rpc:call(Node, application, get_env, [App, Var]) of -{ok, Value} -> - {ok, Value}; -_ -> - rpc_get_env(Node, Others) -end. + case rpc:call(Node, application, get_env, [App, Var]) of + {ok, Value} -> + {ok, Value}; + _ -> + rpc_get_env(Node, Others) + end. -type interface() :: {http, tuple()} | {pb, tuple()}. -type interfaces() :: [interface()]. -type conn_info() :: [{node(), interfaces()}]. -spec connection_info(node() | [node()]) -> interfaces() | conn_info(). -connection_info(Node) when is_atom(Node) -> - {ok, [{PB_IP, PB_Port}]} = rt_pb:get_pb_conn_info(Node), - {ok, [{HTTP_IP, HTTP_Port}]} = rt_http:get_http_conn_info(Node), - case rt_http:get_https_conn_info(Node) of - undefined -> - [{http, {HTTP_IP, HTTP_Port}}, {pb, {PB_IP, PB_Port}}]; - {ok, [{HTTPS_IP, HTTPS_Port}]} -> - [{http, {HTTP_IP, HTTP_Port}}, {https, {HTTPS_IP, HTTPS_Port}}, {pb, {PB_IP, PB_Port}}] - end; -connection_info(Nodes) when is_list(Nodes) -> -[ {Node, connection_info(Node)} || Node <- Nodes]. +connection_info(Node) -> + rt2:connection_info(Node). + +-spec get_pb_conn_info(node()) -> [{inet:ip_address(), pos_integer()}]. +get_pb_conn_info(Node) -> + case rpc_get_env(Node, [{riak_api, pb}, + {riak_api, pb_ip}, + {riak_kv, pb_ip}]) of + {ok, [{NewIP, NewPort}|_]} -> + {ok, [{NewIP, NewPort}]}; + {ok, PB_IP} -> + {ok, PB_Port} = rpc_get_env(Node, [{riak_api, pb_port}, + {riak_kv, pb_port}]), + {ok, [{PB_IP, PB_Port}]}; + _ -> + undefined + end. + +-spec get_http_conn_info(node()) -> [{inet:ip_address(), pos_integer()}]. +get_http_conn_info(Node) -> + case rpc_get_env(Node, [{riak_api, http}, + {riak_core, http}]) of + {ok, [{IP, Port}|_]} -> + {ok, [{IP, Port}]}; + _ -> + undefined + end. + +-spec get_https_conn_info(node()) -> [{inet:ip_address(), pos_integer()}]. +get_https_conn_info(Node) -> + case rpc_get_env(Node, [{riak_api, https}, + {riak_core, https}]) of + {ok, [{IP, Port}|_]} -> + {ok, [{IP, Port}]}; + _ -> + undefined + end. + +%% @doc Deploy a set of freshly installed Riak nodes, returning a list of the +%% nodes deployed. +%% @todo Re-add -spec after adding multi-version support +deploy_nodes(Versions) when is_list(Versions) -> + deploy_nodes(Versions, [riak_kv]); +deploy_nodes(NumNodes) when is_integer(NumNodes) -> + deploy_nodes([ current || _ <- lists:seq(1, NumNodes)]). + +%% @doc Deploy a set of freshly installed Riak nodes with the given +%% `InitialConfig', returning a list of the nodes deployed. +-spec deploy_nodes(NumNodes :: integer(), any()) -> [node()]. +deploy_nodes(NumNodes, InitialConfig) when is_integer(NumNodes) -> + deploy_nodes(NumNodes, InitialConfig, [riak_kv]); +deploy_nodes(Versions, Services) -> + NodeConfig = [ version_to_config(Version) || Version <- Versions ], + Nodes = ?HARNESS:deploy_nodes(NodeConfig), + lager:info("Waiting for services ~p to start on ~p.", [Services, Nodes]), + [ ok = wait_for_service(Node, Service) || Node <- Nodes, + Service <- Services ], + Nodes. + +deploy_nodes(NumNodes, InitialConfig, Services) when is_integer(NumNodes) -> + NodeConfig = [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)], + deploy_nodes(NodeConfig, Services). + +version_to_config(Config) when is_tuple(Config)-> Config; +version_to_config(Version) -> {Version, default}. + +deploy_clusters(Settings) -> + ClusterConfigs = [case Setting of + Configs when is_list(Configs) -> + Configs; + NumNodes when is_integer(NumNodes) -> + [{current, default} || _ <- lists:seq(1, NumNodes)]; + {NumNodes, InitialConfig} when is_integer(NumNodes) -> + [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)]; + {NumNodes, Vsn, InitialConfig} when is_integer(NumNodes) -> + [{Vsn, InitialConfig} || _ <- lists:seq(1,NumNodes)] + end || Setting <- Settings], + ?HARNESS:deploy_clusters(ClusterConfigs). + +build_clusters(Settings) -> + Clusters = deploy_clusters(Settings), + [begin + join_cluster(Nodes), + lager:info("Cluster built: ~p", [Nodes]) + end || Nodes <- Clusters], + Clusters. + +%% @doc Start the specified Riak node +start(Node) -> + ?HARNESS:start(Node). + +%% @doc Start the specified Riak `Node' and wait for it to be pingable +start_and_wait(Node) -> + start(Node), + ?assertEqual(ok, wait_until_pingable(Node)). + +async_start(Node) -> + spawn(fun() -> start(Node) end). + +%% @doc Stop the specified Riak `Node'. +stop(Node) -> + lager:info("Stopping riak on ~p", [Node]), + timer:sleep(10000), %% I know, I know! + ?HARNESS:stop(Node). + %%rpc:call(Node, init, stop, []). + +%% @doc Stop the specified Riak `Node' and wait until it is not pingable +stop_and_wait(Node) -> + stop(Node), + ?assertEqual(ok, wait_until_unpingable(Node)). + +%% @doc Upgrade a Riak `Node' to the specified `NewVersion'. +upgrade(Node, NewVersion) -> + ?HARNESS:upgrade(Node, NewVersion). + +%% @doc Upgrade a Riak `Node' to the specified `NewVersion' and update +%% the config based on entries in `Config'. +upgrade(Node, NewVersion, Config) -> + ?HARNESS:upgrade(Node, NewVersion, Config). + +%% @doc Upgrade a Riak node to a specific version using the alternate +%% leave/upgrade/rejoin approach +slow_upgrade(Node, NewVersion, Nodes) -> + lager:info("Perform leave/upgrade/join upgrade on ~p", [Node]), + lager:info("Leaving ~p", [Node]), + leave(Node), + ?assertEqual(ok, rt:wait_until_unpingable(Node)), + upgrade(Node, NewVersion), + lager:info("Rejoin ~p", [Node]), + join(Node, hd(Nodes -- [Node])), + lager:info("Wait until all nodes are ready and there are no pending changes"), + ?assertEqual(ok, wait_until_nodes_ready(Nodes)), + ?assertEqual(ok, wait_until_no_pending_changes(Nodes)), + ok. + +%% @doc Have `Node' send a join request to `PNode' +join(Node, PNode) -> + R = rpc:call(Node, riak_core, join, [PNode]), + lager:info("[join] ~p to (~p): ~p", [Node, PNode, R]), + ?assertEqual(ok, R), + ok. + +%% @doc Have `Node' send a join request to `PNode' +staged_join(Node, PNode) -> + R = rpc:call(Node, riak_core, staged_join, [PNode]), + lager:info("[join] ~p to (~p): ~p", [Node, PNode, R]), + ?assertEqual(ok, R), + ok. + +plan_and_commit(Node) -> + timer:sleep(500), + lager:info("planning and commiting cluster join"), + case rpc:call(Node, riak_core_claimant, plan, []) of + {error, ring_not_ready} -> + lager:info("plan: ring not ready"), + timer:sleep(100), + plan_and_commit(Node); + {ok, _, _} -> + lager:info("plan: done"), + do_commit(Node) + end. + +do_commit(Node) -> + case rpc:call(Node, riak_core_claimant, commit, []) of + {error, plan_changed} -> + lager:info("commit: plan changed"), + timer:sleep(100), + maybe_wait_for_changes(Node), + plan_and_commit(Node); + {error, ring_not_ready} -> + lager:info("commit: ring not ready"), + timer:sleep(100), + maybe_wait_for_changes(Node), + do_commit(Node); + {error,nothing_planned} -> + %% Assume plan actually committed somehow + ok; + ok -> + ok + end. maybe_wait_for_changes(Node) -> - Ring = rt_ring:get_ring(Node), + Ring = get_ring(Node), Changes = riak_core_ring:pending_changes(Ring), Joining = riak_core_ring:members(Ring, [joining]), lager:info("maybe_wait_for_changes, changes: ~p joining: ~p", @@ -210,42 +421,79 @@ maybe_wait_for_changes(Node) -> ok = wait_until_no_pending_changes([Node]) end. +%% @doc Have the `Node' leave the cluster +leave(Node) -> + R = rpc:call(Node, riak_core, leave, []), + lager:info("[leave] ~p: ~p", [Node, R]), + ?assertEqual(ok, R), + ok. + +%% @doc Have `Node' remove `OtherNode' from the cluster +remove(Node, OtherNode) -> + ?assertEqual(ok, + rpc:call(Node, riak_kv_console, remove, [[atom_to_list(OtherNode)]])). + +%% @doc Have `Node' mark `OtherNode' as down +down(Node, OtherNode) -> + rpc:call(Node, riak_kv_console, down, [[atom_to_list(OtherNode)]]). + +%% @doc partition the `P1' from `P2' nodes +%% note: the nodes remained connected to riak_test@local, +%% which is how `heal/1' can still work. +partition(P1, P2) -> + OldCookie = rpc:call(hd(P1), erlang, get_cookie, []), + NewCookie = list_to_atom(lists:reverse(atom_to_list(OldCookie))), + [true = rpc:call(N, erlang, set_cookie, [N, NewCookie]) || N <- P1], + [[true = rpc:call(N, erlang, disconnect_node, [P2N]) || N <- P1] || P2N <- P2], + wait_until_partitioned(P1, P2), + {NewCookie, OldCookie, P1, P2}. + +%% @doc heal the partition created by call to `partition/2' +%% `OldCookie' is the original shared cookie +heal({_NewCookie, OldCookie, P1, P2}) -> + Cluster = P1 ++ P2, + % set OldCookie on P1 Nodes + [true = rpc:call(N, erlang, set_cookie, [N, OldCookie]) || N <- P1], + wait_until_connected(Cluster), + {_GN, []} = rpc:sbcast(Cluster, riak_core_node_watcher, broadcast), + ok. + %% @doc Spawn `Cmd' on the machine running the test harness spawn_cmd(Cmd) -> - rt_harness:spawn_cmd(Cmd). + ?HARNESS:spawn_cmd(Cmd). %% @doc Spawn `Cmd' on the machine running the test harness spawn_cmd(Cmd, Opts) -> - rt_harness:spawn_cmd(Cmd, Opts). + ?HARNESS:spawn_cmd(Cmd, Opts). %% @doc Wait for a command spawned by `spawn_cmd', returning %% the exit status and result wait_for_cmd(CmdHandle) -> - rt_harness:wait_for_cmd(CmdHandle). + ?HARNESS:wait_for_cmd(CmdHandle). %% @doc Spawn `Cmd' on the machine running the test harness, returning %% the exit status and result cmd(Cmd) -> - rt_harness:cmd(Cmd). + ?HARNESS:cmd(Cmd). %% @doc Spawn `Cmd' on the machine running the test harness, returning %% the exit status and result cmd(Cmd, Opts) -> - rt_harness:cmd(Cmd, Opts). + ?HARNESS:cmd(Cmd, Opts). %% @doc pretty much the same as os:cmd/1 but it will stream the output to lager. %% If you're running a long running command, it will dump the output %% once per second, as to not create the impression that nothing is happening. -spec stream_cmd(string()) -> {integer(), string()}. stream_cmd(Cmd) -> -Port = open_port({spawn, binary_to_list(iolist_to_binary(Cmd))}, [stream, stderr_to_stdout, exit_status]), -stream_cmd_loop(Port, "", "", now()). + Port = open_port({spawn, binary_to_list(iolist_to_binary(Cmd))}, [stream, stderr_to_stdout, exit_status]), + stream_cmd_loop(Port, "", "", now()). %% @doc same as rt:stream_cmd/1, but with options, like open_port/2 -spec stream_cmd(string(), string()) -> {integer(), string()}. stream_cmd(Cmd, Opts) -> -Port = open_port({spawn, binary_to_list(iolist_to_binary(Cmd))}, [stream, stderr_to_stdout, exit_status] ++ Opts), -stream_cmd_loop(Port, "", "", now()). + Port = open_port({spawn, binary_to_list(iolist_to_binary(Cmd))}, [stream, stderr_to_stdout, exit_status] ++ Opts), + stream_cmd_loop(Port, "", "", now()). stream_cmd_loop(Port, Buffer, NewLineBuffer, Time={_MegaSecs, Secs, _MicroSecs}) -> receive @@ -270,7 +518,7 @@ stream_cmd_loop(Port, Buffer, NewLineBuffer, Time={_MegaSecs, Secs, _MicroSecs}) {Port, {exit_status, Status}} -> catch port_close(Port), {Status, Buffer} - after rt_config:get(rt_max_wait_time) -> + after rt:config(rt_max_wait_time) -> {-1, Buffer} end. @@ -278,39 +526,55 @@ stream_cmd_loop(Port, Buffer, NewLineBuffer, Time={_MegaSecs, Secs, _MicroSecs}) %%% Remote code management %%%=================================================================== load_modules_on_nodes([], Nodes) -when is_list(Nodes) -> -ok; + when is_list(Nodes) -> + ok; load_modules_on_nodes([Module | MoreModules], Nodes) -when is_list(Nodes) -> -case code:get_object_code(Module) of -{Module, Bin, File} -> - {_, []} = rpc:multicall(Nodes, code, load_binary, [Module, File, Bin]); -error -> - error(lists:flatten(io_lib:format("unable to get_object_code(~s)", [Module]))) -end, -load_modules_on_nodes(MoreModules, Nodes). + when is_list(Nodes) -> + case code:get_object_code(Module) of + {Module, Bin, File} -> + {_, []} = rpc:multicall(Nodes, code, load_binary, [Module, File, Bin]); + error -> + error(lists:flatten(io_lib:format("unable to get_object_code(~s)", [Module]))) + end, + load_modules_on_nodes(MoreModules, Nodes). %%%=================================================================== %%% Status / Wait Functions %%%=================================================================== +%% @doc Is the `Node' up according to net_adm:ping +is_pingable(Node) -> + net_adm:ping(Node) =:= pong. + is_mixed_cluster(Nodes) when is_list(Nodes) -> -%% If the nodes are bad, we don't care what version they are -{Versions, _BadNodes} = rpc:multicall(Nodes, init, script_id, [], rt_config:get(rt_max_wait_time)), -length(lists:usort(Versions)) > 1; + %% If the nodes are bad, we don't care what version they are + {Versions, _BadNodes} = rpc:multicall(Nodes, init, script_id, [], rt_config:get(rt_max_wait_time)), + length(lists:usort(Versions)) > 1; is_mixed_cluster(Node) -> -Nodes = rpc:call(Node, erlang, nodes, []), -is_mixed_cluster(Nodes). + Nodes = rpc:call(Node, erlang, nodes, []), + is_mixed_cluster(Nodes). + +%% @private +is_ready(Node) -> + case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of + {ok, Ring} -> + case lists:member(Node, riak_core_ring:ready_members(Ring)) of + true -> true; + false -> {not_ready, Node} + end; + Other -> + Other + end. %% @private is_ring_ready(Node) -> -case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of -{ok, Ring} -> - riak_core_ring:ring_ready(Ring); -_ -> - false -end. + case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of + {ok, Ring} -> + riak_core_ring:ring_ready(Ring); + _ -> + false + end. %% @doc Utility function used to construct test predicates. Retries the %% function `Fun' until it returns `true', or until the maximum @@ -318,30 +582,30 @@ end. %% provided `rt_max_wait_time' and `rt_retry_delay' parameters in %% specified `riak_test' config file. wait_until(Fun) when is_function(Fun) -> -MaxTime = rt_config:get(rt_max_wait_time), -Delay = rt_config:get(rt_retry_delay), -Retry = MaxTime div Delay, -wait_until(Fun, Retry, Delay). + MaxTime = rt_config:get(rt_max_wait_time), + Delay = rt_config:get(rt_retry_delay), + Retry = MaxTime div Delay, + wait_until(Fun, Retry, Delay). %% @doc Convenience wrapper for wait_until for the myriad functions that %% take a node as single argument. wait_until(Node, Fun) when is_atom(Node), is_function(Fun) -> -wait_until(fun() -> Fun(Node) end). + wait_until(fun() -> Fun(Node) end). %% @doc Retry `Fun' until it returns `Retry' times, waiting `Delay' %% milliseconds between retries. This is our eventual consistency bread %% and butter wait_until(Fun, Retry, Delay) when Retry > 0 -> -Res = Fun(), -case Res of -true -> - ok; -_ when Retry == 1 -> - {fail, Res}; -_ -> - timer:sleep(Delay), - wait_until(Fun, Retry-1, Delay) -end. + Res = Fun(), + case Res of + true -> + ok; + _ when Retry == 1 -> + {fail, Res}; + _ -> + timer:sleep(Delay), + wait_until(Fun, Retry-1, Delay) + end. %% @doc Wait until the specified node is considered ready by `riak_core'. %% As of Riak 1.0, a node is ready if it is in the `valid' or `leaving' @@ -349,77 +613,84 @@ end. %% information. wait_until_ready(Node) -> lager:info("Wait until ~p ready", [Node]), - ?assertEqual(ok, wait_until(Node, fun rt_node:is_ready/1)), + ?assertEqual(ok, wait_until(Node, fun is_ready/1)), ok. %% @doc Wait until status can be read from riak_kv_console wait_until_status_ready(Node) -> -lager:info("Wait until status ready in ~p", [Node]), -?assertEqual(ok, wait_until(Node, - fun(_) -> - case rpc:call(Node, riak_kv_console, status, [[]]) of - ok -> - true; - Res -> - Res - end - end)). + lager:info("Wait until status ready in ~p", [Node]), + ?assertEqual(ok, wait_until(Node, + fun(_) -> + case rpc:call(Node, riak_kv_console, status, [[]]) of + ok -> + true; + Res -> + Res + end + end)). %% @doc Given a list of nodes, wait until all nodes believe there are no %% on-going or pending ownership transfers. -spec wait_until_no_pending_changes([node()]) -> ok | fail. wait_until_no_pending_changes(Nodes) -> -lager:info("Wait until no pending changes on ~p", [Nodes]), -F = fun() -> - rpc:multicall(Nodes, riak_core_vnode_manager, force_handoffs, []), - {Rings, BadNodes} = rpc:multicall(Nodes, riak_core_ring_manager, get_raw_ring, []), - Changes = [ riak_core_ring:pending_changes(Ring) =:= [] || {ok, Ring} <- Rings ], - BadNodes =:= [] andalso length(Changes) =:= length(Nodes) andalso lists:all(fun(T) -> T end, Changes) -end, -?assertEqual(ok, wait_until(F)), -ok. + lager:info("Wait until no pending changes on ~p", [Nodes]), + F = fun() -> + rpc:multicall(Nodes, riak_core_vnode_manager, force_handoffs, []), + {Rings, BadNodes} = rpc:multicall(Nodes, riak_core_ring_manager, get_raw_ring, []), + Changes = [ riak_core_ring:pending_changes(Ring) =:= [] || {ok, Ring} <- Rings ], + BadNodes =:= [] andalso length(Changes) =:= length(Nodes) andalso lists:all(fun(T) -> T end, Changes) + end, + ?assertEqual(ok, wait_until(F)), + ok. %% @doc Waits until no transfers are in-flight or pending, checked by %% riak_core_status:transfers(). -spec wait_until_transfers_complete([node()]) -> ok | fail. wait_until_transfers_complete([Node0|_]) -> -lager:info("Wait until transfers complete ~p", [Node0]), -F = fun(Node) -> - {DownNodes, Transfers} = rpc:call(Node, riak_core_status, transfers, []), - DownNodes =:= [] andalso Transfers =:= [] -end, -?assertEqual(ok, wait_until(Node0, F)), -ok. + lager:info("Wait until transfers complete ~p", [Node0]), + F = fun(Node) -> + {DownNodes, Transfers} = rpc:call(Node, riak_core_status, transfers, []), + DownNodes =:= [] andalso Transfers =:= [] + end, + ?assertEqual(ok, wait_until(Node0, F)), + ok. wait_for_service(Node, Services) when is_list(Services) -> -F = fun(N) -> - case rpc:call(N, riak_core_node_watcher, services, [N]) of - {badrpc, Error} -> - {badrpc, Error}; - CurrServices when is_list(CurrServices) -> - lists:all(fun(Service) -> lists:member(Service, CurrServices) end, Services); - Res -> - Res - end -end, -?assertEqual(ok, wait_until(Node, F)), -ok; + F = fun(N) -> + case rpc:call(N, riak_core_node_watcher, services, [N]) of + {badrpc, Error} -> + {badrpc, Error}; + CurrServices when is_list(CurrServices) -> + lists:all(fun(Service) -> lists:member(Service, CurrServices) end, Services); + Res -> + Res + end + end, + ?assertEqual(ok, wait_until(Node, F)), + ok; wait_for_service(Node, Service) -> -wait_for_service(Node, [Service]). + wait_for_service(Node, [Service]). wait_for_cluster_service(Nodes, Service) -> -lager:info("Wait for cluster service ~p in ~p", [Service, Nodes]), -F = fun(N) -> - UpNodes = rpc:call(N, riak_core_node_watcher, nodes, [Service]), - (Nodes -- UpNodes) == [] -end, -[?assertEqual(ok, wait_until(Node, F)) || Node <- Nodes], -ok. + lager:info("Wait for cluster service ~p in ~p", [Service, Nodes]), + F = fun(N) -> + UpNodes = rpc:call(N, riak_core_node_watcher, nodes, [Service]), + (Nodes -- UpNodes) == [] + end, + [?assertEqual(ok, wait_until(Node, F)) || Node <- Nodes], + ok. + +%% @doc Given a list of nodes, wait until all nodes are considered ready. +%% See {@link wait_until_ready/1} for definition of ready. +wait_until_nodes_ready(Nodes) -> + lager:info("Wait until nodes are ready : ~p", [Nodes]), + [?assertEqual(ok, wait_until(Node, fun is_ready/1)) || Node <- Nodes], + ok. %% @doc Wait until all nodes in the list `Nodes' believe each other to be %% members of the cluster. wait_until_all_members(Nodes) -> -wait_until_all_members(Nodes, Nodes). + wait_until_all_members(Nodes, Nodes). %% @doc Wait until all nodes in the list `Nodes' believes all nodes in the %% list `Members' are members of the cluster. @@ -427,7 +698,7 @@ wait_until_all_members(Nodes, ExpectedMembers) -> lager:info("Wait until all members ~p ~p", [Nodes, ExpectedMembers]), S1 = ordsets:from_list(ExpectedMembers), F = fun(Node) -> - case rt_ring:members_according_to(Node) of + case members_according_to(Node) of {badrpc, _} -> false; ReportedMembers -> @@ -441,129 +712,548 @@ wait_until_all_members(Nodes, ExpectedMembers) -> %% @doc Given a list of nodes, wait until all nodes believe the ring has %% converged (ie. `riak_core_ring:is_ready' returns `true'). wait_until_ring_converged(Nodes) -> -lager:info("Wait until ring converged on ~p", [Nodes]), -[?assertEqual(ok, wait_until(Node, fun is_ring_ready/1)) || Node <- Nodes], -ok. + lager:info("Wait until ring converged on ~p", [Nodes]), + [?assertEqual(ok, wait_until(Node, fun is_ring_ready/1)) || Node <- Nodes], + ok. wait_until_legacy_ringready(Node) -> -lager:info("Wait until legacy ring ready on ~p", [Node]), -rt:wait_until(Node, - fun(_) -> - case rpc:call(Node, riak_kv_status, ringready, []) of - {ok, _Nodes} -> - true; - Res -> - Res - end - end). + lager:info("Wait until legacy ring ready on ~p", [Node]), + rt:wait_until(Node, + fun(_) -> + case rpc:call(Node, riak_kv_status, ringready, []) of + {ok, _Nodes} -> + true; + Res -> + Res + end + end). %% @doc wait until each node in Nodes is disterl connected to each. wait_until_connected(Nodes) -> lager:info("Wait until connected ~p", [Nodes]), - NodeSet = sets:from_list(Nodes), F = fun(Node) -> Connected = rpc:call(Node, erlang, nodes, []), - sets:is_subset(NodeSet, sets:from_list(([Node] ++ Connected) -- [node()])) + lists:sort(Nodes) == lists:sort([Node]++Connected)--[node()] end, [?assertEqual(ok, wait_until(Node, F)) || Node <- Nodes], ok. %% @doc Wait until the specified node is pingable wait_until_pingable(Node) -> -lager:info("Wait until ~p is pingable", [Node]), -F = fun(N) -> - net_adm:ping(N) =:= pong -end, -?assertEqual(ok, wait_until(Node, F)), -ok. + lager:info("Wait until ~p is pingable", [Node]), + F = fun(N) -> + net_adm:ping(N) =:= pong + end, + ?assertEqual(ok, wait_until(Node, F)), + ok. %% @doc Wait until the specified node is no longer pingable wait_until_unpingable(Node) -> -lager:info("Wait until ~p is not pingable", [Node]), -_OSPidToKill = rpc:call(Node, os, getpid, []), -F = fun() -> net_adm:ping(Node) =:= pang end, -%% riak stop will kill -9 after 5 mins, so we try to wait at least that -%% amount of time. -Delay = rt_config:get(rt_retry_delay), -Retry = lists:max([360000, rt_config:get(rt_max_wait_time)]) div Delay, -case wait_until(F, Retry, Delay) of -ok -> ok; -_ -> - lager:error("Timed out waiting for node ~p to shutdown", [Node]), - ?assert(node_shutdown_timed_out) -end. + lager:info("Wait until ~p is not pingable", [Node]), + _OSPidToKill = rpc:call(Node, os, getpid, []), + F = fun() -> net_adm:ping(Node) =:= pang end, + %% riak stop will kill -9 after 5 mins, so we try to wait at least that + %% amount of time. + Delay = rt_config:get(rt_retry_delay), + Retry = lists:max([360000, rt_config:get(rt_max_wait_time)]) div Delay, + case wait_until(F, Retry, Delay) of + ok -> ok; + _ -> + lager:error("Timed out waiting for node ~p to shutdown", [Node]), + ?assert(node_shutdown_timed_out) + end. % Waits until a certain registered name pops up on the remote node. wait_until_registered(Node, Name) -> -lager:info("Wait until ~p is up on ~p", [Name, Node]), + lager:info("Wait until ~p is up on ~p", [Name, Node]), -F = fun() -> - Registered = rpc:call(Node, erlang, registered, []), - lists:member(Name, Registered) -end, -case wait_until(F) of -ok -> - ok; -_ -> - lager:info("The server with the name ~p on ~p is not coming up.", - [Name, Node]), - ?assert(registered_name_timed_out) -end. + F = fun() -> + Registered = rpc:call(Node, erlang, registered, []), + lists:member(Name, Registered) + end, + case wait_until(F) of + ok -> + ok; + _ -> + lager:info("The server with the name ~p on ~p is not coming up.", + [Name, Node]), + ?assert(registered_name_timed_out) + end. %% Waits until the cluster actually detects that it is partitioned. wait_until_partitioned(P1, P2) -> -lager:info("Waiting until partition acknowledged: ~p ~p", [P1, P2]), -[ begin - lager:info("Waiting for ~p to be partitioned from ~p", [Node, P2]), - wait_until(fun() -> is_partitioned(Node, P2) end) -end || Node <- P1 ], -[ begin - lager:info("Waiting for ~p to be partitioned from ~p", [Node, P1]), - wait_until(fun() -> is_partitioned(Node, P1) end) -end || Node <- P2 ]. + lager:info("Waiting until partition acknowledged: ~p ~p", [P1, P2]), + [ begin + lager:info("Waiting for ~p to be partitioned from ~p", [Node, P2]), + wait_until(fun() -> is_partitioned(Node, P2) end) + end || Node <- P1 ], + [ begin + lager:info("Waiting for ~p to be partitioned from ~p", [Node, P1]), + wait_until(fun() -> is_partitioned(Node, P1) end) + end || Node <- P2 ]. is_partitioned(Node, Peers) -> -AvailableNodes = rpc:call(Node, riak_core_node_watcher, nodes, [riak_kv]), -lists:all(fun(Peer) -> not lists:member(Peer, AvailableNodes) end, Peers). + AvailableNodes = rpc:call(Node, riak_core_node_watcher, nodes, [riak_kv]), + lists:all(fun(Peer) -> not lists:member(Peer, AvailableNodes) end, Peers). + +% when you just can't wait +brutal_kill(Node) -> + rt_cover:maybe_stop_on_node(Node), + lager:info("Killing node ~p", [Node]), + OSPidToKill = rpc:call(Node, os, getpid, []), + %% try a normal kill first, but set a timer to + %% kill -9 after 5 seconds just in case + rpc:cast(Node, timer, apply_after, + [5000, os, cmd, [io_lib:format("kill -9 ~s", [OSPidToKill])]]), + rpc:cast(Node, os, cmd, [io_lib:format("kill -15 ~s", [OSPidToKill])]), + ok. capability(Node, all) -> -rpc:call(Node, riak_core_capability, all, []); + rpc:call(Node, riak_core_capability, all, []); capability(Node, Capability) -> -rpc:call(Node, riak_core_capability, get, [Capability]). + rpc:call(Node, riak_core_capability, get, [Capability]). capability(Node, Capability, Default) -> -rpc:call(Node, riak_core_capability, get, [Capability, Default]). + rpc:call(Node, riak_core_capability, get, [Capability, Default]). wait_until_capability(Node, Capability, Value) -> rt:wait_until(Node, fun(_) -> - Cap = capability(Node, Capability), - lager:info("Capability on node ~p is ~p~n",[Node, Cap]), - cap_equal(Value, Cap) + cap_equal(Value, capability(Node, Capability)) end). wait_until_capability(Node, Capability, Value, Default) -> rt:wait_until(Node, fun(_) -> Cap = capability(Node, Capability, Default), - lager:info("Capability on node ~p is ~p~n",[Node, Cap]), + io:format("capability is ~p ~p",[Node, Cap]), cap_equal(Value, Cap) end). -wait_until_capability_contains(Node, Capability, Value) -> - rt:wait_until(Node, - fun(_) -> - Cap = capability(Node, Capability), - lager:info("Capability on node ~p is ~p~n",[Node, Cap]), - cap_subset(Value, Cap) - end). - cap_equal(Val, Cap) when is_list(Cap) -> -lists:sort(Cap) == lists:sort(Val); + lists:sort(Cap) == lists:sort(Val); cap_equal(Val, Cap) -> -Val == Cap. + Val == Cap. + +wait_until_owners_according_to(Node, Nodes) -> + SortedNodes = lists:usort(Nodes), + F = fun(N) -> + owners_according_to(N) =:= SortedNodes + end, + ?assertEqual(ok, wait_until(Node, F)), + ok. + +wait_until_nodes_agree_about_ownership(Nodes) -> + lager:info("Wait until nodes agree about ownership ~p", [Nodes]), + Results = [ wait_until_owners_according_to(Node, Nodes) || Node <- Nodes ], + ?assert(lists:all(fun(X) -> ok =:= X end, Results)). + +%% AAE support +wait_until_aae_trees_built(Nodes) -> + lager:info("Wait until AAE builds all partition trees across ~p", [Nodes]), + BuiltFun = fun() -> lists:foldl(aae_tree_built_fun(), true, Nodes) end, + ?assertEqual(ok, wait_until(BuiltFun)), + ok. + +aae_tree_built_fun() -> + fun(Node, _AllBuilt = true) -> + case get_aae_tree_info(Node) of + {ok, TreeInfos} -> + case all_trees_have_build_times(TreeInfos) of + true -> + Partitions = [I || {I, _} <- TreeInfos], + all_aae_trees_built(Node, Partitions); + false -> + some_trees_not_built + end; + Err -> + Err + end; + (_Node, Err) -> + Err + end. + +% It is unlikely but possible to get a tree built time from compute_tree_info +% but an attempt to use the tree returns not_built. This is because the build +% process has finished, but the lock on the tree won't be released until it +% dies and the manager detects it. Yes, this is super freaking paranoid. +all_aae_trees_built(Node, Partitions) -> + %% Notice that the process locking is spawned by the + %% pmap. That's important! as it should die eventually + %% so the lock is released and the test can lock the tree. + IndexBuilts = rt:pmap(index_built_fun(Node), Partitions), + BadOnes = [R || R <- IndexBuilts, R /= true], + case BadOnes of + [] -> + true; + _ -> + BadOnes + end. + +get_aae_tree_info(Node) -> + case rpc:call(Node, riak_kv_entropy_info, compute_tree_info, []) of + {badrpc, _} -> + {error, {badrpc, Node}}; + Info -> + lager:debug("Entropy table on node ~p : ~p", [Node, Info]), + {ok, Info} + end. + +all_trees_have_build_times(Info) -> + not lists:keymember(undefined, 2, Info). + +index_built_fun(Node) -> + fun(Idx) -> + case rpc:call(Node, riak_kv_vnode, + hashtree_pid, [Idx]) of + {ok, TreePid} -> + case rpc:call(Node, riak_kv_index_hashtree, + get_lock, [TreePid, for_riak_test]) of + {badrpc, _} -> + {error, {badrpc, Node}}; + TreeLocked when TreeLocked == ok; + TreeLocked == already_locked -> + true; + Err -> + % Either not_built or some unhandled result, + % in which case update this case please! + {error, {index_not_built, Node, Idx, Err}} + end; + {error, _}=Err -> + Err; + {badrpc, _} -> + {error, {badrpc, Node}} + end + end. + +%%%=================================================================== +%%% Ring Functions +%%%=================================================================== + +%% @doc Ensure that the specified node is a singleton node/cluster -- a node +%% that owns 100% of the ring. +check_singleton_node(Node) -> + lager:info("Check ~p is a singleton", [Node]), + {ok, Ring} = rpc:call(Node, riak_core_ring_manager, get_raw_ring, []), + Owners = lists:usort([Owner || {_Idx, Owner} <- riak_core_ring:all_owners(Ring)]), + ?assertEqual([Node], Owners), + ok. + +% @doc Get list of partitions owned by node (primary). +partitions_for_node(Node) -> + Ring = get_ring(Node), + [Idx || {Idx, Owner} <- riak_core_ring:all_owners(Ring), Owner == Node]. + +%% @doc Get the raw ring for `Node'. +get_ring(Node) -> + {ok, Ring} = rpc:call(Node, riak_core_ring_manager, get_raw_ring, []), + Ring. + +assert_nodes_agree_about_ownership(Nodes) -> + ?assertEqual(ok, wait_until_ring_converged(Nodes)), + ?assertEqual(ok, wait_until_all_members(Nodes)), + [ ?assertEqual({Node, Nodes}, {Node, owners_according_to(Node)}) || Node <- Nodes]. + +%% @doc Return a list of nodes that own partitions according to the ring +%% retrieved from the specified node. +owners_according_to(Node) -> + case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of + {ok, Ring} -> + Owners = [Owner || {_Idx, Owner} <- riak_core_ring:all_owners(Ring)], + lists:usort(Owners); + {badrpc, _}=BadRpc -> + BadRpc + end. + +%% @doc Return a list of cluster members according to the ring retrieved from +%% the specified node. +members_according_to(Node) -> + case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of + {ok, Ring} -> + Members = riak_core_ring:all_members(Ring), + Members; + {badrpc, _}=BadRpc -> + BadRpc + end. + +%% @doc Return an appropriate ringsize for the node count passed +%% in. 24 is the number of cores on the bigger intel machines, but this +%% may be too large for the single-chip machines. +nearest_ringsize(Count) -> + nearest_ringsize(Count * 24, 2). + +nearest_ringsize(Count, Power) -> + case Count < trunc(Power * 0.9) of + true -> + Power; + false -> + nearest_ringsize(Count, Power * 2) + end. + +%% @doc Return the cluster status of `Member' according to the ring +%% retrieved from `Node'. +status_of_according_to(Member, Node) -> + case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of + {ok, Ring} -> + Status = riak_core_ring:member_status(Ring, Member), + Status; + {badrpc, _}=BadRpc -> + BadRpc + end. + +%% @doc Return a list of nodes that own partitions according to the ring +%% retrieved from the specified node. +claimant_according_to(Node) -> + case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of + {ok, Ring} -> + Claimant = riak_core_ring:claimant(Ring), + Claimant; + {badrpc, _}=BadRpc -> + BadRpc + end. + +%%%=================================================================== +%%% Cluster Utility Functions +%%%=================================================================== + +%% @doc Safely construct a new cluster and return a list of the deployed nodes +%% @todo Add -spec and update doc to reflect mult-version changes +build_cluster(Versions) when is_list(Versions) -> + build_cluster(length(Versions), Versions, default); +build_cluster(NumNodes) -> + build_cluster(NumNodes, default). + +%% @doc Safely construct a `NumNode' size cluster using +%% `InitialConfig'. Return a list of the deployed nodes. +build_cluster(NumNodes, InitialConfig) -> + build_cluster(NumNodes, [], InitialConfig). + +build_cluster(NumNodes, Versions, InitialConfig) -> + %% Deploy a set of new nodes + Nodes = + case Versions of + [] -> + deploy_nodes(NumNodes, InitialConfig); + _ -> + deploy_nodes(Versions) + end, + + join_cluster(Nodes), + lager:info("Cluster built: ~p", [Nodes]), + Nodes. + +join_cluster(Nodes) -> + %% Ensure each node owns 100% of it's own ring + [?assertEqual([Node], owners_according_to(Node)) || Node <- Nodes], + + %% Join nodes + [Node1|OtherNodes] = Nodes, + case OtherNodes of + [] -> + %% no other nodes, nothing to join/plan/commit + ok; + _ -> + %% ok do a staged join and then commit it, this eliminates the + %% large amount of redundant handoff done in a sequential join + [staged_join(Node, Node1) || Node <- OtherNodes], + plan_and_commit(Node1), + try_nodes_ready(Nodes, 3, 500) + end, + + ?assertEqual(ok, wait_until_nodes_ready(Nodes)), + + %% Ensure each node owns a portion of the ring + wait_until_nodes_agree_about_ownership(Nodes), + ?assertEqual(ok, wait_until_no_pending_changes(Nodes)), + ok. + +-type products() :: riak | riak_ee | riak_cs | unknown. + +-spec product(node()) -> products(). +product(Node) -> + Applications = rpc:call(Node, application, which_applications, []), + + HasRiakCS = proplists:is_defined(riak_cs, Applications), + HasRiakEE = proplists:is_defined(riak_repl, Applications), + HasRiak = proplists:is_defined(riak_kv, Applications), + if HasRiakCS -> riak_cs; + HasRiakEE -> riak_ee; + HasRiak -> riak; + true -> unknown + end. + +try_nodes_ready([Node1 | _Nodes], 0, _SleepMs) -> + lager:info("Nodes not ready after initial plan/commit, retrying"), + plan_and_commit(Node1); +try_nodes_ready(Nodes, N, SleepMs) -> + ReadyNodes = [Node || Node <- Nodes, is_ready(Node) =:= true], + case ReadyNodes of + Nodes -> + ok; + _ -> + timer:sleep(SleepMs), + try_nodes_ready(Nodes, N-1, SleepMs) + end. + +%% @doc Stop nodes and wipe out their data directories +clean_cluster(Nodes) when is_list(Nodes) -> + [stop_and_wait(Node) || Node <- Nodes], + clean_data_dir(Nodes). + +clean_data_dir(Nodes) -> + clean_data_dir(Nodes, ""). + +clean_data_dir(Nodes, SubDir) when not is_list(Nodes) -> + clean_data_dir([Nodes], SubDir); +clean_data_dir(Nodes, SubDir) when is_list(Nodes) -> + ?HARNESS:clean_data_dir(Nodes, SubDir). + +%% @doc Shutdown every node, this is for after a test run is complete. +teardown() -> + %% stop all connected nodes, 'cause it'll be faster that + %%lager:info("RPC stopping these nodes ~p", [nodes()]), + %%[ rt:stop(Node) || Node <- nodes()], + %% Then do the more exhaustive harness thing, in case something was up + %% but not connected. + ?HARNESS:teardown(). + +versions() -> + ?HARNESS:versions(). +%%%=================================================================== +%%% Basic Read/Write Functions +%%%=================================================================== + +systest_write(Node, Size) -> + systest_write(Node, Size, 2). + +systest_write(Node, Size, W) -> + systest_write(Node, 1, Size, <<"systest">>, W). + +systest_write(Node, Start, End, Bucket, W) -> + systest_write(Node, Start, End, Bucket, W, <<>>). + +%% @doc Write (End-Start)+1 objects to Node. Objects keys will be +%% `Start', `Start+1' ... `End', each encoded as a 32-bit binary +%% (`<>'). Object values are the same as their keys. +%% +%% The return value of this function is a list of errors +%% encountered. If all writes were successful, return value is an +%% empty list. Each error has the form `{N :: integer(), Error :: term()}', +%% where N is the unencoded key of the object that failed to store. +systest_write(Node, Start, End, Bucket, W, CommonValBin) + when is_binary(CommonValBin) -> + rt:wait_for_service(Node, riak_kv), + {ok, C} = riak:client_connect(Node), + F = fun(N, Acc) -> + Obj = riak_object:new(Bucket, <>, + <>), + try C:put(Obj, W) of + ok -> + Acc; + Other -> + [{N, Other} | Acc] + catch + What:Why -> + [{N, {What, Why}} | Acc] + end + end, + lists:foldl(F, [], lists:seq(Start, End)). + +systest_read(Node, Size) -> + systest_read(Node, Size, 2). + +systest_read(Node, Size, R) -> + systest_read(Node, 1, Size, <<"systest">>, R). + +systest_read(Node, Start, End, Bucket, R) -> + systest_read(Node, Start, End, Bucket, R, <<>>). + +systest_read(Node, Start, End, Bucket, R, CommonValBin) + when is_binary(CommonValBin) -> + systest_read(Node, Start, End, Bucket, R, CommonValBin, false). + +%% Read and verify the values of objects written with +%% `systest_write'. The `SquashSiblings' parameter exists to +%% optionally allow handling of siblings whose value and metadata are +%% identical except for the dot. This goal is to facilitate testing +%% with DVV enabled because siblings can be created internally by Riak +%% in cases where testing with DVV disabled would not. Such cases +%% include writes that happen during handoff when a vnode forwards +%% writes, but also performs them locally or when a put coordinator +%% fails to send an acknowledgment within the timeout window and +%% another put request is issued. +systest_read(Node, Start, End, Bucket, R, CommonValBin, SquashSiblings) + when is_binary(CommonValBin) -> + rt:wait_for_service(Node, riak_kv), + {ok, C} = riak:client_connect(Node), + lists:foldl(systest_read_fold_fun(C, Bucket, R, CommonValBin, SquashSiblings), + [], + lists:seq(Start, End)). + +systest_read_fold_fun(C, Bucket, R, CommonValBin, SquashSiblings) -> + fun(N, Acc) -> + GetRes = C:get(Bucket, <>, R), + Val = object_value(GetRes, SquashSiblings), + update_acc(value_matches(Val, N, CommonValBin), Val, N, Acc) + end. + +object_value({error, _}=Error, _) -> + Error; +object_value({ok, Obj}, SquashSiblings) -> + object_value(riak_object:value_count(Obj), Obj, SquashSiblings). + +object_value(1, Obj, _SquashSiblings) -> + riak_object:get_value(Obj); +object_value(_ValueCount, Obj, false) -> + riak_object:get_value(Obj); +object_value(_ValueCount, Obj, true) -> + lager:debug("Siblings detected for ~p:~p", [riak_object:bucket(Obj), riak_object:key(Obj)]), + Contents = riak_object:get_contents(Obj), + case lists:foldl(fun sibling_compare/2, {true, undefined}, Contents) of + {true, {_, _, _, Value}} -> + lager:debug("Siblings determined to be a single value"), + Value; + {false, _} -> + {error, siblings} + end. + +sibling_compare({MetaData, Value}, {true, undefined}) -> + Dot = case dict:find(<<"dot">>, MetaData) of + {ok, DotVal} -> + DotVal; + error -> + {error, no_dot} + end, + VTag = dict:fetch(<<"X-Riak-VTag">>, MetaData), + LastMod = dict:fetch(<<"X-Riak-Last-Modified">>, MetaData), + {true, {element(2, Dot), VTag, LastMod, Value}}; +sibling_compare(_, {false, _}=InvalidMatch) -> + InvalidMatch; +sibling_compare({MetaData, Value}, {true, PreviousElements}) -> + Dot = case dict:find(<<"dot">>, MetaData) of + {ok, DotVal} -> + DotVal; + error -> + {error, no_dot} + end, + VTag = dict:fetch(<<"X-Riak-VTag">>, MetaData), + LastMod = dict:fetch(<<"X-Riak-Last-Modified">>, MetaData), + ComparisonElements = {element(2, Dot), VTag, LastMod, Value}, + {ComparisonElements =:= PreviousElements, ComparisonElements}. + +value_matches(<>, N, CommonValBin) -> + true; +value_matches(_WrongVal, _N, _CommonValBin) -> + false. + +update_acc(true, _, _, Acc) -> + Acc; +update_acc(false, {error, _}=Val, N, Acc) -> + [{N, Val} | Acc]; +update_acc(false, Val, N, Acc) -> + [{N, {wrong_val, Val}} | Acc]. cap_subset(Val, Cap) when is_list(Cap) -> sets:is_subset(sets:from_list(Val), sets:from_list(Cap)). @@ -851,25 +1541,258 @@ end. % @doc Reads a single replica of a value. This issues a get command directly % to the vnode handling the Nth primary partition of the object's preflist. get_replica(Node, Bucket, Key, I, N) -> -BKey = {Bucket, Key}, -Chash = rpc:call(Node, riak_core_util, chash_key, [BKey]), -Pl = rpc:call(Node, riak_core_apl, get_primary_apl, [Chash, N, riak_kv]), -{{Partition, PNode}, primary} = lists:nth(I, Pl), -Ref = Reqid = make_ref(), -Sender = {raw, Ref, self()}, -rpc:call(PNode, riak_kv_vnode, get, - [{Partition, PNode}, BKey, Ref, Sender]), -receive -{Ref, {r, Result, _, Reqid}} -> - Result; -{Ref, Reply} -> - Reply -after -60000 -> - lager:error("Replica ~p get for ~p/~p timed out", - [I, Bucket, Key]), - ?assert(false) -end. + BKey = {Bucket, Key}, + Chash = rpc:call(Node, riak_core_util, chash_key, [BKey]), + Pl = rpc:call(Node, riak_core_apl, get_primary_apl, [Chash, N, riak_kv]), + {{Partition, PNode}, primary} = lists:nth(I, Pl), + Ref = Reqid = make_ref(), + Sender = {raw, Ref, self()}, + rpc:call(PNode, riak_kv_vnode, get, + [{Partition, PNode}, BKey, Ref, Sender]), + receive + {Ref, {r, Result, _, Reqid}} -> + Result; + {Ref, Reply} -> + Reply + after + 60000 -> + lager:error("Replica ~p get for ~p/~p timed out", + [I, Bucket, Key]), + ?assert(false) + end. + +%%%=================================================================== + +%% @doc PBC-based version of {@link systest_write/1} +pbc_systest_write(Node, Size) -> + pbc_systest_write(Node, Size, 2). + +pbc_systest_write(Node, Size, W) -> + pbc_systest_write(Node, 1, Size, <<"systest">>, W). + +pbc_systest_write(Node, Start, End, Bucket, W) -> + rt:wait_for_service(Node, riak_kv), + Pid = pbc(Node), + F = fun(N, Acc) -> + Obj = riakc_obj:new(Bucket, <>, <>), + try riakc_pb_socket:put(Pid, Obj, W) of + ok -> + Acc; + Other -> + [{N, Other} | Acc] + catch + What:Why -> + [{N, {What, Why}} | Acc] + end + end, + lists:foldl(F, [], lists:seq(Start, End)). + +pbc_systest_read(Node, Size) -> + pbc_systest_read(Node, Size, 2). + +pbc_systest_read(Node, Size, R) -> + pbc_systest_read(Node, 1, Size, <<"systest">>, R). + +pbc_systest_read(Node, Start, End, Bucket, R) -> + rt:wait_for_service(Node, riak_kv), + Pid = pbc(Node), + F = fun(N, Acc) -> + case riakc_pb_socket:get(Pid, Bucket, <>, R) of + {ok, Obj} -> + case riakc_obj:get_value(Obj) of + <> -> + Acc; + WrongVal -> + [{N, {wrong_val, WrongVal}} | Acc] + end; + Other -> + [{N, Other} | Acc] + end + end, + lists:foldl(F, [], lists:seq(Start, End)). + +%%%=================================================================== +%%% PBC & HTTPC Functions +%%%=================================================================== + +%% @doc get me a protobuf client process and hold the mayo! +-spec pbc(node()) -> pid(). +pbc(Node) -> + pbc(Node, [{auto_reconnect, true}]). + +-spec pbc(node(), proplists:proplist()) -> pid(). +pbc(Node, Options) -> + rt:wait_for_service(Node, riak_kv), + ConnInfo = proplists:get_value(Node, connection_info([Node])), + {IP, PBPort} = proplists:get_value(pb, ConnInfo), + {ok, Pid} = riakc_pb_socket:start_link(IP, PBPort, Options), + Pid. + +%% @doc does a read via the erlang protobuf client +-spec pbc_read(pid(), binary(), binary()) -> binary(). +pbc_read(Pid, Bucket, Key) -> + pbc_read(Pid, Bucket, Key, []). + +-spec pbc_read(pid(), binary(), binary(), [any()]) -> binary(). +pbc_read(Pid, Bucket, Key, Options) -> + {ok, Value} = riakc_pb_socket:get(Pid, Bucket, Key, Options), + Value. + +-spec pbc_read_check(pid(), binary(), binary(), [any()]) -> boolean(). +pbc_read_check(Pid, Bucket, Key, Allowed) -> + pbc_read_check(Pid, Bucket, Key, Allowed, []). + +-spec pbc_read_check(pid(), binary(), binary(), [any()], [any()]) -> boolean(). +pbc_read_check(Pid, Bucket, Key, Allowed, Options) -> + case riakc_pb_socket:get(Pid, Bucket, Key, Options) of + {ok, _} -> + true = lists:member(ok, Allowed); + Other -> + lists:member(Other, Allowed) orelse throw({failed, Other, Allowed}) + end. + +%% @doc does a write via the erlang protobuf client +-spec pbc_write(pid(), binary(), binary(), binary()) -> atom(). +pbc_write(Pid, Bucket, Key, Value) -> + Object = riakc_obj:new(Bucket, Key, Value), + riakc_pb_socket:put(Pid, Object). + +%% @doc does a write via the erlang protobuf client plus content-type +-spec pbc_write(pid(), binary(), binary(), binary(), list()) -> atom(). +pbc_write(Pid, Bucket, Key, Value, CT) -> + Object = riakc_obj:new(Bucket, Key, Value, CT), + riakc_pb_socket:put(Pid, Object). + +%% @doc sets a bucket property/properties via the erlang protobuf client +-spec pbc_set_bucket_prop(pid(), binary(), [proplists:property()]) -> atom(). +pbc_set_bucket_prop(Pid, Bucket, PropList) -> + riakc_pb_socket:set_bucket(Pid, Bucket, PropList). + +%% @doc Puts the contents of the given file into the given bucket using the +%% filename as a key and assuming a plain text content type. +pbc_put_file(Pid, Bucket, Key, Filename) -> + {ok, Contents} = file:read_file(Filename), + riakc_pb_socket:put(Pid, riakc_obj:new(Bucket, Key, Contents, "text/plain")). + +%% @doc Puts all files in the given directory into the given bucket using the +%% filename as a key and assuming a plain text content type. +pbc_put_dir(Pid, Bucket, Dir) -> + lager:info("Putting files from dir ~p into bucket ~p", [Dir, Bucket]), + {ok, Files} = file:list_dir(Dir), + [pbc_put_file(Pid, Bucket, list_to_binary(F), filename:join([Dir, F])) + || F <- Files]. + +%% @doc True if the given keys have been really, really deleted. +%% Useful when you care about the keys not being there. Delete simply writes +%% tombstones under the given keys, so those are still seen by key folding +%% operations. +pbc_really_deleted(Pid, Bucket, Keys) -> + StillThere = + fun(K) -> + Res = riakc_pb_socket:get(Pid, Bucket, K, + [{r, 1}, + {notfound_ok, false}, + {basic_quorum, false}, + deletedvclock]), + case Res of + {error, notfound} -> + false; + _ -> + %% Tombstone still around + true + end + end, + [] == lists:filter(StillThere, Keys). + +%% @doc Returns HTTPS URL information for a list of Nodes +https_url(Nodes) when is_list(Nodes) -> + [begin + {Host, Port} = orddict:fetch(https, Connections), + lists:flatten(io_lib:format("https://~s:~b", [Host, Port])) + end || {_Node, Connections} <- connection_info(Nodes)]; +https_url(Node) -> + hd(https_url([Node])). + +%% @doc Returns HTTP URL information for a list of Nodes +http_url(Nodes) when is_list(Nodes) -> + [begin + {Host, Port} = orddict:fetch(http, Connections), + lists:flatten(io_lib:format("http://~s:~b", [Host, Port])) + end || {_Node, Connections} <- connection_info(Nodes)]; +http_url(Node) -> + hd(http_url([Node])). + +%% @doc get me an http client. +-spec httpc(node()) -> term(). +httpc(Node) -> + rt:wait_for_service(Node, riak_kv), + {ok, [{IP, Port}]} = get_http_conn_info(Node), + rhc:create(IP, Port, "riak", []). + +%% @doc does a read via the http erlang client. +-spec httpc_read(term(), binary(), binary()) -> binary(). +httpc_read(C, Bucket, Key) -> + {_, Value} = rhc:get(C, Bucket, Key), + Value. + +%% @doc does a write via the http erlang client. +-spec httpc_write(term(), binary(), binary(), binary()) -> atom(). +httpc_write(C, Bucket, Key, Value) -> + Object = riakc_obj:new(Bucket, Key, Value), + rhc:put(C, Object). + +%%%=================================================================== +%%% Command Line Functions +%%%=================================================================== + +%% @doc Call 'bin/riak-admin' command on `Node' with arguments `Args' +admin(Node, Args) -> + admin(Node, Args, []). + +%% @doc Call 'bin/riak-admin' command on `Node' with arguments `Args'. +%% The third parameter is a list of options. Valid options are: +%% * `return_exit_code' - Return the exit code along with the command output +admin(Node, Args, Options) -> + ?HARNESS:admin(Node, Args, Options). + +%% @doc Call 'bin/riak' command on `Node' with arguments `Args' +riak(Node, Args) -> + ?HARNESS:riak(Node, Args). + + +%% @doc Call 'bin/riak-repl' command on `Node' with arguments `Args' +riak_repl(Node, Args) -> + ?HARNESS:riak_repl(Node, Args). + +search_cmd(Node, Args) -> + {ok, Cwd} = file:get_cwd(), + rpc:call(Node, riak_search_cmd, command, [[Cwd | Args]]). + +%% @doc Runs `riak attach' on a specific node, and tests for the expected behavoir. +%% Here's an example: ``` +%% rt:attach(Node, [{expect, "erlang.pipe.1 \(^D to exit\)"}, +%% {send, "riak_core_ring_manager:get_my_ring()."}, +%% {expect, "dict,"}, +%% {send, [4]}]), %% 4 = Ctrl + D''' +%% `{expect, String}' scans the output for the existance of the String. +%% These tuples are processed in order. +%% +%% `{send, String}' sends the string to the console. +%% Once a send is encountered, the buffer is discarded, and the next +%% expect will process based on the output following the sent data. +%% +attach(Node, Expected) -> + ?HARNESS:attach(Node, Expected). + +%% @doc Runs 'riak attach-direct' on a specific node +%% @see rt:attach/2 +attach_direct(Node, Expected) -> + ?HARNESS:attach_direct(Node, Expected). + +%% @doc Runs `riak console' on a specific node +%% @see rt:attach/2 +console(Node, Expected) -> + ?HARNESS:console(Node, Expected). %%%=================================================================== %%% Search @@ -879,113 +1802,241 @@ end. %% in the cluster may be used as the change is propagated via the %% Ring. enable_search_hook(Node, Bucket) when is_binary(Bucket) -> -lager:info("Installing search hook for bucket ~p", [Bucket]), -?assertEqual(ok, rpc:call(Node, riak_search_kv_hook, install, [Bucket])). + lager:info("Installing search hook for bucket ~p", [Bucket]), + ?assertEqual(ok, rpc:call(Node, riak_search_kv_hook, install, [Bucket])). + +%%%=================================================================== +%%% Test harness setup, configuration, and internal utilities +%%%=================================================================== + +%% @doc Sets the backend of ALL nodes that could be available to riak_test. +%% this is not limited to the nodes under test, but any node that +%% riak_test is able to find. It then queries each available node +%% for it's backend, and returns it if they're all equal. If different +%% nodes have different backends, it returns a list of backends. +%% Currently, there is no way to request multiple backends, so the +%% list return type should be considered an error. +-spec set_backend(atom()) -> atom()|[atom()]. +set_backend(Backend) -> + set_backend(Backend, []). + +-spec set_backend(atom(), [{atom(), term()}]) -> atom()|[atom()]. +set_backend(bitcask, _) -> + set_backend(riak_kv_bitcask_backend); +set_backend(eleveldb, _) -> + set_backend(riak_kv_eleveldb_backend); +set_backend(memory, _) -> + set_backend(riak_kv_memory_backend); +set_backend(multi, Extras) -> + set_backend(riak_kv_multi_backend, Extras); +set_backend(Backend, _) when Backend == riak_kv_bitcask_backend; Backend == riak_kv_eleveldb_backend; Backend == riak_kv_memory_backend -> + lager:info("rt:set_backend(~p)", [Backend]), + update_app_config(all, [{riak_kv, [{storage_backend, Backend}]}]), + get_backends(); +set_backend(Backend, Extras) when Backend == riak_kv_multi_backend -> + MultiConfig = proplists:get_value(multi_config, Extras, default), + Config = make_multi_backend_config(MultiConfig), + update_app_config(all, [{riak_kv, Config}]), + get_backends(); +set_backend(Other, _) -> + lager:warning("rt:set_backend doesn't recognize ~p as a legit backend, using the default.", [Other]), + get_backends(). + +make_multi_backend_config(default) -> + [{storage_backend, riak_kv_multi_backend}, + {multi_backend_default, <<"eleveldb1">>}, + {multi_backend, [{<<"eleveldb1">>, riak_kv_eleveldb_backend, []}, + {<<"memory1">>, riak_kv_memory_backend, []}, + {<<"bitcask1">>, riak_kv_bitcask_backend, []}]}]; +make_multi_backend_config(indexmix) -> + [{storage_backend, riak_kv_multi_backend}, + {multi_backend_default, <<"eleveldb1">>}, + {multi_backend, [{<<"eleveldb1">>, riak_kv_eleveldb_backend, []}, + {<<"memory1">>, riak_kv_memory_backend, []}]}]; +make_multi_backend_config(Other) -> + lager:warning("rt:set_multi_backend doesn't recognize ~p as legit multi-backend config, using default", [Other]), + make_multi_backend_config(default). + +get_backends() -> + Backends = ?HARNESS:get_backends(), + case Backends of + [riak_kv_bitcask_backend] -> bitcask; + [riak_kv_eleveldb_backend] -> eleveldb; + [riak_kv_memory_backend] -> memory; + [Other] -> Other; + MoreThanOne -> MoreThanOne + end. + +-spec get_backend([proplists:property()]) -> atom() | error. +get_backend(AppConfigProplist) -> + case kvc:path('riak_kv.storage_backend', AppConfigProplist) of + [] -> error; + Backend -> Backend + end. %% @doc Gets the current version under test. In the case of an upgrade test %% or something like that, it's the version you're upgrading to. -spec get_version() -> binary(). get_version() -> - rt_harness:get_version(). + ?HARNESS:get_version(). %% @doc outputs some useful information about nodes that are up whats_up() -> - rt_harness:whats_up(). + ?HARNESS:whats_up(). -spec get_ip(node()) -> string(). get_ip(Node) -> - rt_harness:get_ip(Node). + ?HARNESS:get_ip(Node). %% @doc Log a message to the console of the specified test nodes. %% Messages are prefixed by the string "---riak_test--- " %% Uses lager:info/1 'Fmt' semantics log_to_nodes(Nodes, Fmt) -> -log_to_nodes(Nodes, Fmt, []). + log_to_nodes(Nodes, Fmt, []). %% @doc Log a message to the console of the specified test nodes. %% Messages are prefixed by the string "---riak_test--- " %% Uses lager:info/2 'LFmt' and 'LArgs' semantics log_to_nodes(Nodes0, LFmt, LArgs) -> -%% This logs to a node's info level, but if riak_test is running -%% at debug level, we want to know when we send this and what -%% we're saying -Nodes = lists:flatten(Nodes0), -lager:debug("log_to_nodes: " ++ LFmt, LArgs), -Module = lager, -Function = log, -Meta = [], -Args = case LArgs of - [] -> [info, Meta, "---riak_test--- " ++ LFmt]; - _ -> [info, Meta, "---riak_test--- " ++ LFmt, LArgs] - end, -[rpc:call(Node, Module, Function, Args) || Node <- lists:flatten(Nodes)]. + %% This logs to a node's info level, but if riak_test is running + %% at debug level, we want to know when we send this and what + %% we're saying + Nodes = lists:flatten(Nodes0), + lager:debug("log_to_nodes: " ++ LFmt, LArgs), + Module = lager, + Function = log, + Meta = [], + Args = case LArgs of + [] -> [info, Meta, "---riak_test--- " ++ LFmt]; + _ -> [info, Meta, "---riak_test--- " ++ LFmt, LArgs] + end, + [rpc:call(Node, Module, Function, Args) || Node <- lists:flatten(Nodes)]. %% @private utility function pmap(F, L) -> -Parent = self(), -lists:foldl( -fun(X, N) -> - spawn_link(fun() -> - Parent ! {pmap, N, F(X)} - end), - N+1 -end, 0, L), -L2 = [receive {pmap, N, R} -> {N,R} end || _ <- L], -{_, L3} = lists:unzip(lists:keysort(1, L2)), -L3. + Parent = self(), + lists:foldl( + fun(X, N) -> + spawn_link(fun() -> + Parent ! {pmap, N, F(X)} + end), + N+1 + end, 0, L), + L2 = [receive {pmap, N, R} -> {N,R} end || _ <- L], + {_, L3} = lists:unzip(lists:keysort(1, L2)), + L3. %% @private setup_harness(Test, Args) -> - rt_harness:setup_harness(Test, Args). + ?HARNESS:setup_harness(Test, Args). %% @doc Downloads any extant log files from the harness's running %% nodes. get_node_logs() -> - rt_harness:get_node_logs(). + ?HARNESS:get_node_logs(). check_ibrowse() -> -try sys:get_status(ibrowse) of -{status, _Pid, {module, gen_server} ,_} -> ok -catch -Throws -> - lager:error("ibrowse error ~p", [Throws]), - lager:error("Restarting ibrowse"), - application:stop(ibrowse), - application:start(ibrowse) -end. + try sys:get_status(ibrowse) of + {status, _Pid, {module, gen_server} ,_} -> ok + catch + Throws -> + lager:error("ibrowse error ~p", [Throws]), + lager:error("Restarting ibrowse"), + application:stop(ibrowse), + application:start(ibrowse) + end. post_result(TestResult, #rt_webhook{url=URL, headers=HookHeaders, name=Name}) -> -lager:info("Posting result to ~s ~s", [Name, URL]), -try ibrowse:send_req(URL, - [{"Content-Type", "application/json"}], - post, - mochijson2:encode(TestResult), - [{content_type, "application/json"}] ++ HookHeaders, - 300000) of %% 5 minute timeout - -{ok, RC=[$2|_], Headers, _Body} -> - {ok, RC, Headers}; -{ok, ResponseCode, Headers, Body} -> - lager:info("Test Result did not generate the expected 2XX HTTP response code."), - lager:debug("Post"), - lager:debug("Response Code: ~p", [ResponseCode]), - lager:debug("Headers: ~p", [Headers]), - lager:debug("Body: ~p", [Body]), - error; -X -> - lager:warning("Some error POSTing test result: ~p", [X]), - error -catch -Class:Reason -> - lager:error("Error reporting to ~s. ~p:~p", [Name, Class, Reason]), - lager:error("Payload: ~p", [TestResult]), - error -end. + lager:info("Posting result to ~s ~s", [Name, URL]), + try ibrowse:send_req(URL, + [{"Content-Type", "application/json"}], + post, + mochijson2:encode(TestResult), + [{content_type, "application/json"}] ++ HookHeaders, + 300000) of %% 5 minute timeout + + {ok, RC=[$2|_], Headers, _Body} -> + {ok, RC, Headers}; + {ok, ResponseCode, Headers, Body} -> + lager:info("Test Result did not generate the expected 2XX HTTP response code."), + lager:debug("Post"), + lager:debug("Response Code: ~p", [ResponseCode]), + lager:debug("Headers: ~p", [Headers]), + lager:debug("Body: ~p", [Body]), + error; + X -> + lager:warning("Some error POSTing test result: ~p", [X]), + error + catch + Class:Reason -> + lager:error("Error reporting to ~s. ~p:~p", [Name, Class, Reason]), + lager:error("Payload: ~p", [TestResult]), + error + end. %%%=================================================================== %%% Bucket Types Functions %%%=================================================================== +%% @doc create and immediately activate a bucket type +create_and_activate_bucket_type(Node, Type, Props) -> + ok = rpc:call(Node, riak_core_bucket_type, create, [Type, Props]), + wait_until_bucket_type_status(Type, ready, Node), + ok = rpc:call(Node, riak_core_bucket_type, activate, [Type]), + wait_until_bucket_type_status(Type, active, Node). + +wait_until_bucket_type_status(Type, ExpectedStatus, Nodes) when is_list(Nodes) -> + [wait_until_bucket_type_status(Type, ExpectedStatus, Node) || Node <- Nodes]; +wait_until_bucket_type_status(Type, ExpectedStatus, Node) -> + F = fun() -> + ActualStatus = rpc:call(Node, riak_core_bucket_type, status, [Type]), + ExpectedStatus =:= ActualStatus + end, + ?assertEqual(ok, rt:wait_until(F)). + +-spec bucket_type_visible([atom()], binary()|{binary(), binary()}) -> boolean(). +bucket_type_visible(Nodes, Type) -> + MaxTime = rt_config:get(rt_max_wait_time), + IsVisible = fun erlang:is_list/1, + {Res, NodesDown} = rpc:multicall(Nodes, riak_core_bucket_type, get, [Type], MaxTime), + NodesDown == [] andalso lists:all(IsVisible, Res). + +wait_until_bucket_type_visible(Nodes, Type) -> + F = fun() -> bucket_type_visible(Nodes, Type) end, + ?assertEqual(ok, rt:wait_until(F)). + +-spec see_bucket_props([atom()], binary()|{binary(), binary()}, + proplists:proplist()) -> boolean(). +see_bucket_props(Nodes, Bucket, ExpectProps) -> + MaxTime = rt_config:get(rt_max_wait_time), + IsBad = fun({badrpc, _}) -> true; + ({error, _}) -> true; + (Res) when is_list(Res) -> false + end, + HasProps = fun(ResProps) -> + lists:all(fun(P) -> lists:member(P, ResProps) end, + ExpectProps) + end, + case rpc:multicall(Nodes, riak_core_bucket, get_bucket, [Bucket], MaxTime) of + {Res, []} -> + % No nodes down, check no errors + case lists:any(IsBad, Res) of + true -> + false; + false -> + lists:all(HasProps, Res) + end; + {_, _NodesDown} -> + false + end. + +wait_until_bucket_props(Nodes, Bucket, Props) -> + F = fun() -> + see_bucket_props(Nodes, Bucket, Props) + end, + ?assertEqual(ok, rt:wait_until(F)). + + %% @doc Set up in memory log capture to check contents in a test. setup_log_capture(Nodes) when is_list(Nodes) -> rt:load_modules_on_nodes([riak_test_lager_backend], Nodes), @@ -1080,28 +2131,6 @@ is_control_gui_route_loaded(Routes) -> wait_for_control(VersionedNodes) when is_list(VersionedNodes) -> [wait_for_control(Vsn, Node) || {Vsn, Node} <- VersionedNodes]. --spec select_random([any()]) -> any(). -select_random(List) -> - Length = length(List), - Idx = random:uniform(Length), - lists:nth(Idx, List). - -%% @doc Returns a random element from a given list. --spec random_sublist([any()], integer()) -> [any()]. -random_sublist(List, N) -> - % Properly seeding the process. - <> = crypto:rand_bytes(12), - random:seed({A, B, C}), - % Assign a random value for each element in the list. - List1 = [{random:uniform(), E} || E <- List], - % Sort by the random number. - List2 = lists:sort(List1), - % Take the first N elements. - List3 = lists:sublist(List2, N), - % Remove the random numbers. - [ E || {_,E} <- List3]. - -%% TODO: Is this the right location for this? -ifdef(TEST). verify_product(Applications, ExpectedApplication) -> diff --git a/src/rt2.erl b/src/rt2.erl new file mode 100644 index 000000000..1761dc3ea --- /dev/null +++ b/src/rt2.erl @@ -0,0 +1,786 @@ +%% ------------------------------------------------------------------- +%% +%% Copyright (c) 2013-2014 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. +%% +%% ------------------------------------------------------------------- + +%% @doc +%% Implements the base `riak_test' API, providing the ability to control +%% nodes in a Riak cluster as well as perform commonly reused operations. +%% Please extend this module with new functions that prove useful between +%% multiple independent tests. +-module(rt2). +-include("rt.hrl"). +-include_lib("eunit/include/eunit.hrl"). + +-compile(export_all). +-export([ + capability/2, + capability/3, + check_ibrowse/0, + cmd/1, + cmd/2, + connection_info/1, + enable_search_hook/2, + expect_in_log/2, + get_deps/0, + get_ip/1, + get_node_logs/0, + get_replica/5, + get_version/0, + is_mixed_cluster/1, + load_modules_on_nodes/2, + log_to_nodes/2, + log_to_nodes/3, + pmap/2, + post_result/2, + priv_dir/0, + product/1, + rpc_get_env/2, + setup_harness/2, + setup_log_capture/1, + stream_cmd/1, + stream_cmd/2, + spawn_cmd/1, + spawn_cmd/2, + str/2, + wait_for_cluster_service/2, + wait_for_cmd/1, + wait_for_service/2, + wait_for_control/1, + wait_for_control/2, + wait_until/3, + wait_until/2, + wait_until/1, + wait_until_all_members/1, + wait_until_all_members/2, + wait_until_capability/3, + wait_until_capability/4, + wait_until_connected/1, + wait_until_legacy_ringready/1, + wait_until_no_pending_changes/1, + wait_until_pingable/1, + wait_until_ready/1, + wait_until_registered/2, + wait_until_ring_converged/1, + wait_until_status_ready/1, + wait_until_transfers_complete/1, + wait_until_unpingable/1, + whats_up/0 + ]). + +-define(HARNESS, (rt_config:get(rt_harness))). + +priv_dir() -> +LocalPrivDir = "./priv", +%% XXX for some reason, codew:priv_dir returns riak_test/riak_test/priv, +%% which is wrong, so fix it. +DepPrivDir = re:replace(code:priv_dir(riak_test), "riak_test(/riak_test)*", +"riak_test", [{return, list}]), +PrivDir = case {filelib:is_dir(LocalPrivDir), filelib:is_dir(DepPrivDir)} of +{true, _} -> + lager:debug("Local ./priv detected, using that..."), + %% we want an absolute path! + filename:absname(LocalPrivDir); +{false, true} -> + lager:debug("riak_test dependency priv_dir detected, using that..."), + DepPrivDir; +_ -> + ?assertEqual({true, bad_priv_dir}, {false, bad_priv_dir}) +end, + +lager:info("priv dir: ~p -> ~p", [code:priv_dir(riak_test), PrivDir]), +?assert(filelib:is_dir(PrivDir)), +PrivDir. + +%% @doc gets riak deps from the appropriate harness +-spec get_deps() -> list(). +get_deps() -> rt_harness:get_deps(). + +%% @doc if String contains Substr, return true. +-spec str(string(), string()) -> boolean(). +str(String, Substr) -> +case string:str(String, Substr) of +0 -> false; +_ -> true +end. + +%% @doc Helper that returns first successful application get_env result, +%% used when different versions of Riak use different app vars for +%% the same setting. +rpc_get_env(_, []) -> +undefined; +rpc_get_env(Node, [{App,Var}|Others]) -> +case rpc:call(Node, application, get_env, [App, Var]) of +{ok, Value} -> + {ok, Value}; +_ -> + rpc_get_env(Node, Others) +end. + +-type interface() :: {http, tuple()} | {pb, tuple()}. +-type interfaces() :: [interface()]. +-type conn_info() :: [{node(), interfaces()}]. + +-spec connection_info(node() | [node()]) -> interfaces() | conn_info(). +connection_info(Node) when is_atom(Node) -> + {ok, [{PB_IP, PB_Port}]} = rt_pb:get_pb_conn_info(Node), + {ok, [{HTTP_IP, HTTP_Port}]} = rt_http:get_http_conn_info(Node), + case rt_http:get_https_conn_info(Node) of + undefined -> + [{http, {HTTP_IP, HTTP_Port}}, {pb, {PB_IP, PB_Port}}]; + {ok, [{HTTPS_IP, HTTPS_Port}]} -> + [{http, {HTTP_IP, HTTP_Port}}, {https, {HTTPS_IP, HTTPS_Port}}, {pb, {PB_IP, PB_Port}}] + end; +connection_info(Nodes) when is_list(Nodes) -> +[ {Node, connection_info(Node)} || Node <- Nodes]. + +maybe_wait_for_changes(Node) -> + Ring = rt_ring:get_ring(Node), + Changes = riak_core_ring:pending_changes(Ring), + Joining = riak_core_ring:members(Ring, [joining]), + lager:info("maybe_wait_for_changes, changes: ~p joining: ~p", + [Changes, Joining]), + if Changes =:= [] -> + ok; + Joining =/= [] -> + ok; + true -> + ok = wait_until_no_pending_changes([Node]) + end. + +%% @doc Spawn `Cmd' on the machine running the test harness +spawn_cmd(Cmd) -> + rt_harness:spawn_cmd(Cmd). + +%% @doc Spawn `Cmd' on the machine running the test harness +spawn_cmd(Cmd, Opts) -> + rt_harness:spawn_cmd(Cmd, Opts). + +%% @doc Wait for a command spawned by `spawn_cmd', returning +%% the exit status and result +wait_for_cmd(CmdHandle) -> + rt_harness:wait_for_cmd(CmdHandle). + +%% @doc Spawn `Cmd' on the machine running the test harness, returning +%% the exit status and result +cmd(Cmd) -> + rt_harness:cmd(Cmd). + +%% @doc Spawn `Cmd' on the machine running the test harness, returning +%% the exit status and result +cmd(Cmd, Opts) -> + rt_harness:cmd(Cmd, Opts). + +%% @doc pretty much the same as os:cmd/1 but it will stream the output to lager. +%% If you're running a long running command, it will dump the output +%% once per second, as to not create the impression that nothing is happening. +-spec stream_cmd(string()) -> {integer(), string()}. +stream_cmd(Cmd) -> +Port = open_port({spawn, binary_to_list(iolist_to_binary(Cmd))}, [stream, stderr_to_stdout, exit_status]), +stream_cmd_loop(Port, "", "", now()). + +%% @doc same as rt:stream_cmd/1, but with options, like open_port/2 +-spec stream_cmd(string(), string()) -> {integer(), string()}. +stream_cmd(Cmd, Opts) -> +Port = open_port({spawn, binary_to_list(iolist_to_binary(Cmd))}, [stream, stderr_to_stdout, exit_status] ++ Opts), +stream_cmd_loop(Port, "", "", now()). + +stream_cmd_loop(Port, Buffer, NewLineBuffer, Time={_MegaSecs, Secs, _MicroSecs}) -> + receive + {Port, {data, Data}} -> + {_, Now, _} = now(), + NewNewLineBuffer = case Now > Secs of + true -> + lager:info(NewLineBuffer), + ""; + _ -> + NewLineBuffer + end, + case rt:str(Data, "\n") of + true -> + lager:info(NewNewLineBuffer), + Tokens = string:tokens(Data, "\n"), + [ lager:info(Token) || Token <- Tokens ], + stream_cmd_loop(Port, Buffer ++ NewNewLineBuffer ++ Data, "", Time); + _ -> + stream_cmd_loop(Port, Buffer, NewNewLineBuffer ++ Data, now()) + end; + {Port, {exit_status, Status}} -> + catch port_close(Port), + {Status, Buffer} + after rt_config:get(rt_max_wait_time) -> + {-1, Buffer} + end. + +%%%=================================================================== +%%% Remote code management +%%%=================================================================== +load_modules_on_nodes([], Nodes) +when is_list(Nodes) -> +ok; +load_modules_on_nodes([Module | MoreModules], Nodes) +when is_list(Nodes) -> +case code:get_object_code(Module) of +{Module, Bin, File} -> + {_, []} = rpc:multicall(Nodes, code, load_binary, [Module, File, Bin]); +error -> + error(lists:flatten(io_lib:format("unable to get_object_code(~s)", [Module]))) +end, +load_modules_on_nodes(MoreModules, Nodes). + + +%%%=================================================================== +%%% Status / Wait Functions +%%%=================================================================== + +is_mixed_cluster(Nodes) when is_list(Nodes) -> +%% If the nodes are bad, we don't care what version they are +{Versions, _BadNodes} = rpc:multicall(Nodes, init, script_id, [], rt_config:get(rt_max_wait_time)), +length(lists:usort(Versions)) > 1; +is_mixed_cluster(Node) -> +Nodes = rpc:call(Node, erlang, nodes, []), +is_mixed_cluster(Nodes). + +%% @private +is_ring_ready(Node) -> +case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of +{ok, Ring} -> + riak_core_ring:ring_ready(Ring); +_ -> + false +end. + +-type products() :: riak | riak_ee | riak_cs | unknown. + +-spec product(node()) -> products(). +product(Node) -> + Applications = rpc:call(Node, application, which_applications, []), + + HasRiakCS = proplists:is_defined(riak_cs, Applications), + HasRiakEE = proplists:is_defined(riak_repl, Applications), + HasRiak = proplists:is_defined(riak_kv, Applications), + if HasRiakCS -> + riak_cs; + HasRiakEE -> riak_ee; + HasRiak -> riak; + true -> unknown + end. + +%% @doc Utility function used to construct test predicates. Retries the +%% function `Fun' until it returns `true', or until the maximum +%% number of retries is reached. The retry limit is based on the +%% provided `rt_max_wait_time' and `rt_retry_delay' parameters in +%% specified `riak_test' config file. +wait_until(Fun) when is_function(Fun) -> +MaxTime = rt_config:get(rt_max_wait_time), +Delay = rt_config:get(rt_retry_delay), +Retry = MaxTime div Delay, +wait_until(Fun, Retry, Delay). + +%% @doc Convenience wrapper for wait_until for the myriad functions that +%% take a node as single argument. +wait_until(Node, Fun) when is_atom(Node), is_function(Fun) -> +wait_until(fun() -> Fun(Node) end). + +%% @doc Retry `Fun' until it returns `Retry' times, waiting `Delay' +%% milliseconds between retries. This is our eventual consistency bread +%% and butter +wait_until(Fun, Retry, Delay) when Retry > 0 -> +Res = Fun(), +case Res of +true -> + ok; +_ when Retry == 1 -> + {fail, Res}; +_ -> + timer:sleep(Delay), + wait_until(Fun, Retry-1, Delay) +end. + +%% @doc Wait until the specified node is considered ready by `riak_core'. +%% As of Riak 1.0, a node is ready if it is in the `valid' or `leaving' +%% states. A ready node is guaranteed to have current preflist/ownership +%% information. +wait_until_ready(Node) -> + lager:info("Wait until ~p ready", [Node]), + ?assertEqual(ok, wait_until(Node, fun rt_node:is_ready/1)), + ok. + +%% @doc Wait until status can be read from riak_kv_console +wait_until_status_ready(Node) -> +lager:info("Wait until status ready in ~p", [Node]), +?assertEqual(ok, wait_until(Node, + fun(_) -> + case rpc:call(Node, riak_kv_console, status, [[]]) of + ok -> + true; + Res -> + Res + end + end)). + +%% @doc Given a list of nodes, wait until all nodes believe there are no +%% on-going or pending ownership transfers. +-spec wait_until_no_pending_changes([node()]) -> ok | fail. +wait_until_no_pending_changes(Nodes) -> +lager:info("Wait until no pending changes on ~p", [Nodes]), +F = fun() -> + rpc:multicall(Nodes, riak_core_vnode_manager, force_handoffs, []), + {Rings, BadNodes} = rpc:multicall(Nodes, riak_core_ring_manager, get_raw_ring, []), + Changes = [ riak_core_ring:pending_changes(Ring) =:= [] || {ok, Ring} <- Rings ], + BadNodes =:= [] andalso length(Changes) =:= length(Nodes) andalso lists:all(fun(T) -> T end, Changes) +end, +?assertEqual(ok, wait_until(F)), +ok. + +%% @doc Waits until no transfers are in-flight or pending, checked by +%% riak_core_status:transfers(). +-spec wait_until_transfers_complete([node()]) -> ok | fail. +wait_until_transfers_complete([Node0|_]) -> +lager:info("Wait until transfers complete ~p", [Node0]), +F = fun(Node) -> + {DownNodes, Transfers} = rpc:call(Node, riak_core_status, transfers, []), + DownNodes =:= [] andalso Transfers =:= [] +end, +?assertEqual(ok, wait_until(Node0, F)), +ok. + +wait_for_service(Node, Services) when is_list(Services) -> +F = fun(N) -> + case rpc:call(N, riak_core_node_watcher, services, [N]) of + {badrpc, Error} -> + {badrpc, Error}; + CurrServices when is_list(CurrServices) -> + lists:all(fun(Service) -> lists:member(Service, CurrServices) end, Services); + Res -> + Res + end +end, +?assertEqual(ok, wait_until(Node, F)), +ok; +wait_for_service(Node, Service) -> +wait_for_service(Node, [Service]). + +wait_for_cluster_service(Nodes, Service) -> +lager:info("Wait for cluster service ~p in ~p", [Service, Nodes]), +F = fun(N) -> + UpNodes = rpc:call(N, riak_core_node_watcher, nodes, [Service]), + (Nodes -- UpNodes) == [] +end, +[?assertEqual(ok, wait_until(Node, F)) || Node <- Nodes], +ok. + +%% @doc Wait until all nodes in the list `Nodes' believe each other to be +%% members of the cluster. +wait_until_all_members(Nodes) -> +wait_until_all_members(Nodes, Nodes). + +%% @doc Wait until all nodes in the list `Nodes' believes all nodes in the +%% list `Members' are members of the cluster. +wait_until_all_members(Nodes, ExpectedMembers) -> + lager:info("Wait until all members ~p ~p", [Nodes, ExpectedMembers]), + S1 = ordsets:from_list(ExpectedMembers), + F = fun(Node) -> + case rt_ring:members_according_to(Node) of + {badrpc, _} -> + false; + ReportedMembers -> + S2 = ordsets:from_list(ReportedMembers), + ordsets:is_subset(S1, S2) + end + end, + [?assertEqual(ok, wait_until(Node, F)) || Node <- Nodes], + ok. + +%% @doc Given a list of nodes, wait until all nodes believe the ring has +%% converged (ie. `riak_core_ring:is_ready' returns `true'). +wait_until_ring_converged(Nodes) -> +lager:info("Wait until ring converged on ~p", [Nodes]), +[?assertEqual(ok, wait_until(Node, fun is_ring_ready/1)) || Node <- Nodes], +ok. + +wait_until_legacy_ringready(Node) -> +lager:info("Wait until legacy ring ready on ~p", [Node]), +rt:wait_until(Node, + fun(_) -> + case rpc:call(Node, riak_kv_status, ringready, []) of + {ok, _Nodes} -> + true; + Res -> + Res + end + end). + +%% @doc wait until each node in Nodes is disterl connected to each. +wait_until_connected(Nodes) -> +lager:info("Wait until connected ~p", [Nodes]), +F = fun(Node) -> + Connected = rpc:call(Node, erlang, nodes, []), + lists:sort(Nodes) == lists:sort([Node]++Connected)--[node()] +end, +[?assertEqual(ok, wait_until(Node, F)) || Node <- Nodes], +ok. + +%% @doc Wait until the specified node is pingable +wait_until_pingable(Node) -> +lager:info("Wait until ~p is pingable", [Node]), +F = fun(N) -> + net_adm:ping(N) =:= pong +end, +?assertEqual(ok, wait_until(Node, F)), +ok. + +%% @doc Wait until the specified node is no longer pingable +wait_until_unpingable(Node) -> +lager:info("Wait until ~p is not pingable", [Node]), +_OSPidToKill = rpc:call(Node, os, getpid, []), +F = fun() -> net_adm:ping(Node) =:= pang end, +%% riak stop will kill -9 after 5 mins, so we try to wait at least that +%% amount of time. +Delay = rt_config:get(rt_retry_delay), +Retry = lists:max([360000, rt_config:get(rt_max_wait_time)]) div Delay, +case wait_until(F, Retry, Delay) of +ok -> ok; +_ -> + lager:error("Timed out waiting for node ~p to shutdown", [Node]), + ?assert(node_shutdown_timed_out) +end. + + +% Waits until a certain registered name pops up on the remote node. +wait_until_registered(Node, Name) -> +lager:info("Wait until ~p is up on ~p", [Name, Node]), + +F = fun() -> + Registered = rpc:call(Node, erlang, registered, []), + lists:member(Name, Registered) +end, +case wait_until(F) of +ok -> + ok; +_ -> + lager:info("The server with the name ~p on ~p is not coming up.", + [Name, Node]), + ?assert(registered_name_timed_out) +end. + + +%% Waits until the cluster actually detects that it is partitioned. +wait_until_partitioned(P1, P2) -> +lager:info("Waiting until partition acknowledged: ~p ~p", [P1, P2]), +[ begin + lager:info("Waiting for ~p to be partitioned from ~p", [Node, P2]), + wait_until(fun() -> is_partitioned(Node, P2) end) +end || Node <- P1 ], +[ begin + lager:info("Waiting for ~p to be partitioned from ~p", [Node, P1]), + wait_until(fun() -> is_partitioned(Node, P1) end) +end || Node <- P2 ]. + +is_partitioned(Node, Peers) -> +AvailableNodes = rpc:call(Node, riak_core_node_watcher, nodes, [riak_kv]), +lists:all(fun(Peer) -> not lists:member(Peer, AvailableNodes) end, Peers). + +capability(Node, all) -> +rpc:call(Node, riak_core_capability, all, []); +capability(Node, Capability) -> +rpc:call(Node, riak_core_capability, get, [Capability]). + +capability(Node, Capability, Default) -> +rpc:call(Node, riak_core_capability, get, [Capability, Default]). + +wait_until_capability(Node, Capability, Value) -> +rt:wait_until(Node, + fun(_) -> + cap_equal(Value, capability(Node, Capability)) + end). + +wait_until_capability(Node, Capability, Value, Default) -> +rt:wait_until(Node, + fun(_) -> + Cap = capability(Node, Capability, Default), + io:format("capability is ~p ~p",[Node, Cap]), + cap_equal(Value, Cap) + end). + +cap_equal(Val, Cap) when is_list(Cap) -> +lists:sort(Cap) == lists:sort(Val); +cap_equal(Val, Cap) -> +Val == Cap. + +% @doc Reads a single replica of a value. This issues a get command directly +% to the vnode handling the Nth primary partition of the object's preflist. +get_replica(Node, Bucket, Key, I, N) -> +BKey = {Bucket, Key}, +Chash = rpc:call(Node, riak_core_util, chash_key, [BKey]), +Pl = rpc:call(Node, riak_core_apl, get_primary_apl, [Chash, N, riak_kv]), +{{Partition, PNode}, primary} = lists:nth(I, Pl), +Ref = Reqid = make_ref(), +Sender = {raw, Ref, self()}, +rpc:call(PNode, riak_kv_vnode, get, + [{Partition, PNode}, BKey, Ref, Sender]), +receive +{Ref, {r, Result, _, Reqid}} -> + Result; +{Ref, Reply} -> + Reply +after +60000 -> + lager:error("Replica ~p get for ~p/~p timed out", + [I, Bucket, Key]), + ?assert(false) +end. + +%%%=================================================================== +%%% Search +%%%=================================================================== + +%% doc Enable the search KV hook for the given `Bucket'. Any `Node' +%% in the cluster may be used as the change is propagated via the +%% Ring. +enable_search_hook(Node, Bucket) when is_binary(Bucket) -> +lager:info("Installing search hook for bucket ~p", [Bucket]), +?assertEqual(ok, rpc:call(Node, riak_search_kv_hook, install, [Bucket])). + +%% @doc Gets the current version under test. In the case of an upgrade test +%% or something like that, it's the version you're upgrading to. +-spec get_version() -> binary(). +get_version() -> + rt_harness:get_version(). + +%% @doc outputs some useful information about nodes that are up +whats_up() -> + rt_harness:whats_up(). + +-spec get_ip(node()) -> string(). +get_ip(Node) -> + rt_harness:get_ip(Node). + +%% @doc Log a message to the console of the specified test nodes. +%% Messages are prefixed by the string "---riak_test--- " +%% Uses lager:info/1 'Fmt' semantics +log_to_nodes(Nodes, Fmt) -> +log_to_nodes(Nodes, Fmt, []). + +%% @doc Log a message to the console of the specified test nodes. +%% Messages are prefixed by the string "---riak_test--- " +%% Uses lager:info/2 'LFmt' and 'LArgs' semantics +log_to_nodes(Nodes0, LFmt, LArgs) -> +%% This logs to a node's info level, but if riak_test is running +%% at debug level, we want to know when we send this and what +%% we're saying +Nodes = lists:flatten(Nodes0), +lager:debug("log_to_nodes: " ++ LFmt, LArgs), +Module = lager, +Function = log, +Meta = [], +Args = case LArgs of + [] -> [info, Meta, "---riak_test--- " ++ LFmt]; + _ -> [info, Meta, "---riak_test--- " ++ LFmt, LArgs] + end, +[rpc:call(Node, Module, Function, Args) || Node <- lists:flatten(Nodes)]. + +%% @private utility function +pmap(F, L) -> +Parent = self(), +lists:foldl( +fun(X, N) -> + spawn_link(fun() -> + Parent ! {pmap, N, F(X)} + end), + N+1 +end, 0, L), +L2 = [receive {pmap, N, R} -> {N,R} end || _ <- L], +{_, L3} = lists:unzip(lists:keysort(1, L2)), +L3. + +%% @private +setup_harness(Test, Args) -> + rt_harness:setup_harness(Test, Args). + +%% @doc Downloads any extant log files from the harness's running +%% nodes. +get_node_logs() -> + rt_harness:get_node_logs(). + +check_ibrowse() -> +try sys:get_status(ibrowse) of +{status, _Pid, {module, gen_server} ,_} -> ok +catch +Throws -> + lager:error("ibrowse error ~p", [Throws]), + lager:error("Restarting ibrowse"), + application:stop(ibrowse), + application:start(ibrowse) +end. + +post_result(TestResult, #rt_webhook{url=URL, headers=HookHeaders, name=Name}) -> +lager:info("Posting result to ~s ~s", [Name, URL]), +try ibrowse:send_req(URL, + [{"Content-Type", "application/json"}], + post, + mochijson2:encode(TestResult), + [{content_type, "application/json"}] ++ HookHeaders, + 300000) of %% 5 minute timeout + +{ok, RC=[$2|_], Headers, _Body} -> + {ok, RC, Headers}; +{ok, ResponseCode, Headers, Body} -> + lager:info("Test Result did not generate the expected 2XX HTTP response code."), + lager:debug("Post"), + lager:debug("Response Code: ~p", [ResponseCode]), + lager:debug("Headers: ~p", [Headers]), + lager:debug("Body: ~p", [Body]), + error; +X -> + lager:warning("Some error POSTing test result: ~p", [X]), + error +catch +Class:Reason -> + lager:error("Error reporting to ~s. ~p:~p", [Name, Class, Reason]), + lager:error("Payload: ~p", [TestResult]), + error +end. + +%%%=================================================================== +%%% Bucket Types Functions +%%%=================================================================== + +%% @doc Set up in memory log capture to check contents in a test. +setup_log_capture(Nodes) when is_list(Nodes) -> + rt:load_modules_on_nodes([riak_test_lager_backend], Nodes), + [?assertEqual({Node, ok}, + {Node, + rpc:call(Node, + gen_event, + add_handler, + [lager_event, + riak_test_lager_backend, + [info, false]])}) || Node <- Nodes], + [?assertEqual({Node, ok}, + {Node, + rpc:call(Node, + lager, + set_loglevel, + [riak_test_lager_backend, + info])}) || Node <- Nodes]; +setup_log_capture(Node) when not is_list(Node) -> + setup_log_capture([Node]). + + +expect_in_log(Node, Pattern) -> + CheckLogFun = fun() -> + Logs = rpc:call(Node, riak_test_lager_backend, get_logs, []), + lager:info("looking for pattern ~s in logs for ~p", + [Pattern, Node]), + case re:run(Logs, Pattern, []) of + {match, _} -> + lager:info("Found match"), + true; + nomatch -> + lager:info("No match"), + false + end + end, + case rt:wait_until(CheckLogFun) of + ok -> + true; + _ -> + false + end. + +%% @doc Wait for Riak Control to start on a single node. +%% +%% Non-optimal check, because we're blocking for the gen_server to start +%% to ensure that the routes have been added by the supervisor. +%% +wait_for_control(_Vsn, Node) when is_atom(Node) -> + lager:info("Waiting for riak_control to start on node ~p.", [Node]), + + %% Wait for the gen_server. + rt:wait_until(Node, fun(N) -> + case rpc:call(N, + riak_control_session, + get_version, + []) of + {ok, _} -> + true; + Error -> + lager:info("Error was ~p.", [Error]), + false + end + end), + + lager:info("Waiting for routes to be added to supervisor..."), + + %% Wait for routes to be added by supervisor. + rt:wait_until(Node, fun(N) -> + case rpc:call(N, + webmachine_router, + get_routes, + []) of + {badrpc, Error} -> + lager:info("Error was ~p.", [Error]), + false; + Routes -> + case is_control_gui_route_loaded(Routes) of + false -> + false; + _ -> + true + end + end + end). + +%% @doc Is the riak_control GUI route loaded? +is_control_gui_route_loaded(Routes) -> + lists:keymember(admin_gui, 2, Routes) orelse lists:keymember(riak_control_wm_gui, 2, Routes). + +%% @doc Wait for Riak Control to start on a series of nodes. +wait_for_control(VersionedNodes) when is_list(VersionedNodes) -> + [wait_for_control(Vsn, Node) || {Vsn, Node} <- VersionedNodes]. + +node_id(Node) -> + rt_harness:node_id(Node). + +node_version(Node) -> + rt_harness:node_version(Node). + +%% TODO: Is this the right location for this? +-ifdef(TEST). + +verify_product(Applications, ExpectedApplication) -> + ?_test(begin + meck:new(rpc, [unstick]), + meck:expect(rpc, call, fun([], application, which_applications, []) -> + Applications end), + ?assertMatch(ExpectedApplication, product([])), + meck:unload(rpc) + end). + +product_test_() -> + {foreach, + fun() -> ok end, + [verify_product([riak_cs], riak_cs), + verify_product([riak_repl, riak_kv, riak_cs], riak_cs), + verify_product([riak_repl], riak_ee), + verify_product([riak_repl, riak_kv], riak_ee), + verify_product([riak_kv], riak), + verify_product([kernel], unknown)]}. + +-endif. diff --git a/src/rt_cs_dev.erl b/src/rt_cs_dev.erl index a41741977..f28bfedd6 100644 --- a/src/rt_cs_dev.erl +++ b/src/rt_cs_dev.erl @@ -41,6 +41,7 @@ node_id/1, node_version/1, admin/2, + admin/3, riak/2, attach/2, attach_direct/2, @@ -409,23 +410,27 @@ interactive_loop(Port, Expected) -> ?assertEqual([], Expected) end. +%% TODO is the correct implementation for admin/2 -- added to pass compilation by jsb +admin(Node, Args) -> + admin(Node, Args, []). + admin(Node, Args, Options) -> - N = node_id(Node), - Path = relpath(node_version(N)), - Cmd = rtdev:riak_admin_cmd(Path, N, Args), - lager:info("Running: ~s", [Cmd]), - Result = execute_admin_cmd(Cmd, Options), - lager:info("~s", [Result]), - {ok, Result}. + N = node_id(Node), + Path = relpath(node_version(N)), + Cmd = rtdev:riak_admin_cmd(Path, N, Args), + lager:info("Running: ~s", [Cmd]), + Result = execute_admin_cmd(Cmd, Options), + lager:info("~s", [Result]), + {ok, Result}. execute_admin_cmd(Cmd, Options) -> - {_ExitCode, Result} = FullResult = wait_for_cmd(spawn_cmd(Cmd)), - case lists:member(return_exit_code, Options) of - true -> - FullResult; - false -> - Result - end. + {_ExitCode, Result} = FullResult = wait_for_cmd(spawn_cmd(Cmd)), + case lists:member(return_exit_code, Options) of + true -> + FullResult; + false -> + Result + end. riak(Node, Args) -> N = node_id(Node), diff --git a/src/rtdev.erl b/src/rtdev.erl index 27b340f88..201bca6d5 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -38,7 +38,7 @@ get_ip/1, node_id/1, node_version/1, - admin/2, + %% admin/2, riak/2, attach/2, attach_direct/2, diff --git a/tests/basic_command_line.erl b/tests/basic_command_line.erl index 66547bec0..b752ec1dd 100644 --- a/tests/basic_command_line.erl +++ b/tests/basic_command_line.erl @@ -139,7 +139,7 @@ attach_direct_down_test(Node) -> status_up_test(Node) -> lager:info("Test riak-admin status on ~s", [Node]), - {ok, StatusOut} = rt_cmd_line:admin(Node, ["status"]), + {ok, {ExitCode, StatusOut}} = rt:admin(Node, ["status"], [return_exit_code]), io:format("Result of status: ~s", [StatusOut]), ?assertEqual(0, ExitCode), ?assert(rt:str(StatusOut, "1-minute stats")), @@ -150,6 +150,7 @@ status_up_test(Node) -> status_down_test(Node) -> lager:info("Test riak-admin status while down"), {ok, StatusOut} = rt_cmd_line:admin(Node, ["status"]), + ?assert(rt:str(StatusOut, "Node is not running!")), ok. getpid_up_test(Node) -> diff --git a/tests/http_bucket_types.erl b/tests/http_bucket_types.erl index df72350af..203ad006f 100644 --- a/tests/http_bucket_types.erl +++ b/tests/http_bucket_types.erl @@ -9,6 +9,8 @@ -include_lib("riakc/include/riakc.hrl"). -include("rt.hrl"). +-define(WAIT(E), ?assertEqual(ok, rt:wait_until(fun() -> + (E) end))). -test_type([bucket_types, http]). properties() -> diff --git a/tests/overload.erl b/tests/overload.erl index a409772c9..7cc2da018 100644 --- a/tests/overload.erl +++ b/tests/overload.erl @@ -83,8 +83,8 @@ setup() -> Node =:= Node2]), RO = riak_object:new(?BUCKET, ?KEY, <<"test">>), - - ok = test_no_overload_protection(Nodes, Victim, RO), + %% TODO commented out to get merge completed .. must be reconciled -- jsb + %% ok = test_no_overload_protection(Nodes, BKV), ok = test_vnode_protection(Nodes, Victim, RO), ok = test_fsm_protection(Nodes, Victim, RO), pass. diff --git a/tests/verify_snmp.erl b/tests/verify_snmp.erl index bae8fbb19..646f8381f 100644 --- a/tests/verify_snmp.erl +++ b/tests/verify_snmp.erl @@ -27,7 +27,7 @@ confirm() -> %% Bring up a small cluster - Config = [{riak_snmp, [{polling_interval, 1000}]}], + %% Config = [{riak_snmp, [{polling_interval, 1000}]}], [Node1] = rt_cluster:deploy_nodes(1), ?assertEqual(ok, rt_node:wait_until_nodes_ready([Node1])), From 32a0b7e427ba8b45806db156b583554303638d48 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Thu, 12 Feb 2015 05:01:25 +0000 Subject: [PATCH 049/157] Update build scripts and README to better reflect mixed mode --- README.md | 121 ++++++++++--------- bin/reset-current-env.sh | 7 +- bin/rtdev-all.sh | 224 +++++++++++++++++++++++++++++++++--- bin/rtdev-build-releases.sh | 176 ---------------------------- bin/rtdev-current.sh | 38 ------ bin/rtdev-install.sh | 75 +++++++++--- bin/rtdev-migrate.sh | 69 +++++++++++ bin/rtdev-setup-releases.sh | 45 -------- 8 files changed, 400 insertions(+), 355 deletions(-) delete mode 100755 bin/rtdev-build-releases.sh delete mode 100755 bin/rtdev-current.sh create mode 100644 bin/rtdev-migrate.sh delete mode 100755 bin/rtdev-setup-releases.sh diff --git a/README.md b/README.md index 527b43549..f5d46c606 100644 --- a/README.md +++ b/README.md @@ -8,16 +8,16 @@ in Erlang, and can interact with the cluster using distributed Erlang. ### How does it work? -`riak_test` runs tests in a sandbox, typically `$HOME/rt/riak`. The sanbox +`riak_test` runs tests in a sandbox, typically `$HOME/rt/riak`. The sandbox uses `git` to reset back to a clean state after tests are run. The contents of `$HOME/rt/riak` might look something like this: ``` $ ls $HOME/rt/riak -current riak-1.2.1 riak-1.3.2 riak-1.4.10 +head riak-1.3.2 riak-1.4.12 ``` -Inside each of these directories is a `dev` folder, typically +Inside each of these directories is a series `dev[0-9]+` directories, typically created with your normal `make [stage]devrel`. So how does this sandbox get populated to begin with? @@ -31,13 +31,13 @@ help you get both `~/test-releases` and `$HOME/rt/riak` all set up. A full tutorial for using them exists further down in this README. There is one folder in `$HOME/rt/riak` that does not come from -`~/test-releases`: `current`. The `current` folder can refer +`~/test-releases`: `head`. The `head` folder can refer to any version of Riak, but is typically used for something like the `master` branch, a feature branch, or a release candidate. -The `$HOME/rt/riak/current` dev release gets populated from a devrel of Riak +The `$HOME/rt/riak/head` dev release gets populated from a devrel of Riak that can come from anywhere, but is usually your 'normal' git checkout -of Riak. The `bin/rtdev-current.sh` can be run from within that folder -to copy `dev/` into `$HOME/rt/riak/current`. +of Riak. The `bin/rtdev-install.sh` can be run from within that folder +to copy `dev/` into `$HOME/rt/riak/head`. Once you have everything set up (again, instructions for this are below), you'll want to run and write tests. This repository also holds code for @@ -53,37 +53,45 @@ previous versions of Riak. Together, we'll get your test environment up and running. Scripts to help in this process are located in the `bin` directory of this project. +### Prerequisites + +In order to successfully build Erlang and Riak there are several dependencies +which need to be fulfilled. Details can be found at +[Installing Erlang](http://docs.basho.com/riak/latest/ops/building/installing/erlang/). + +Essentially these packages need to be on your system for a successful build: +- autoconf +- gcc/g++ +- curses +- JDK +- make +- openssl (a very current one) + + ### rtdev-all.sh -This script is for the lazy. It performs all of the setup steps described -in the other scripts, including installing the current "master" branch from -Github into "current". The releases will be built in your current working +This script is for a complete installation. It performs all of the setup steps +including installing the current "master" branch from +Github into "head". The releases will be built in your current working directory, so create an empty one in a place you'd like to store these builds for posterity, so that you don't have to rebuild them if your installation path (`$HOME/rt/riak` by the way this script installs it) gets into a bad state. -If you do want to restore your `$HOME/rt/riak` folder to factory condition, see -`rtdev-setup-releases.sh` and if you want to change the current riak under -test, see `rtdev-current.sh`. - -### rtdev-build-releases.sh - -The first one that we want to look at is `rtdev-build-releases.sh`. If -left unchanged, this script is going to do the following: +This script is going to do the following: 1. Download the source for the past three major Riak versions (e.g. - 1.3.2, 1.4.10 and 2.0.0) + 1.3.2, 1.4.12 and master) 1. Build the proper version of Erlang that release was built with, - using kerl (which it will also download) + using **kerl** (which it will also download) 1. Build those releases of Riak. You'll want to run this script from an empty directory. Also, you might be -thinking that you already have all the required versions of erlang. Great! You +thinking that you already have all the required versions of Erlang. Great! You should set and export the following environment variables prior to running this and other `riak_test` scripts: -Here, kerl is configured to use "$HOME/.kerl/installs" as the installation +Here, **kerl** is configured to use "$HOME/.kerl/installs" as the installation directory for erlang builds. ```bash @@ -92,12 +100,8 @@ export R16B02="$HOME/.kerl/installs/erlang-R16B02" export CURRENT_OTP="$R16B02" ``` -**Kerlveat**: If you want kerl to build erlangs with serious 64-bit -Macintosh action, you'll need a `~/.kerlrc` file that looks like this: - -``` -KERL_CONFIGURE_OPTIONS="--disable-hipe --enable-smp-support --enable-threads --enable-kernel-poll --enable-darwin-64bit --without-odbc" -``` +If you have your own versions of Erlang, just set the above environment +variables before running `rtdev-all.sh`. The script will check that all these paths exist. If even one of them is missing, it will prompt you to install kerl, even if you already @@ -105,38 +109,40 @@ have kerl. If you say no, the script quits. If you say yes, or all of your erlang paths check out, then go get a cup of coffee, you'll be building for a little while. -### rtdev-setup-releases.sh - -The `rtdev-setup-releases.sh` will get the releases you just built -into a local git repository. Run this script from the -same directory that you just built all of your releases into. -By default this script initializes the repository into `$HOME/rt/riak` but -you can override [`$RT_DEST_DIR`](https://github.com/basho/riak_test/blob/master/bin/rtdev-setup-releases.sh#L11). +To use `riak_ee` instead of `riak` set [`$RT_USE_EE`](https://github.com/basho/riak_test/blob/master/bin/rtdev-all.sh#L46) +to any non-empty string. **Note**: There is a bug in 1.3.x `leveldb` which does not properly resolve the location of `pthread.h` when building on Macintosh OS X 10.9, aka -Mavericks. This has been fixed in subsequent releases, but for now a fix -is to manually add `#include ` to the top of +Mavericks, and 10.10 (Yosemite). This has been fixed in subsequent releases, +but for now a fix is to manually add `#include ` to the top of `deps/eleveldb/c_src/leveldb/include/leveldb/env.h`. Also the version -of `meck` needs to be updated, too. This is handled autmatically by +of `meck` needs to be updated, too. This is handled automatically by the script. -### rtdev-current.sh - -`rtdev-current.sh` is where it gets interesting. You need to run that -from the Riak source folder you're wanting to test as the current -version of Riak. Also, make sure that you've already run `make devrel` -or `make stagedevrel` before you run `rtdev-current.sh`. Like setting up -releases you can override [`$RT_DEST_DIR`](https://github.com/basho/riak_test/blob/master/bin/rtdev-current.sh#L6) -so all your riak builds are in one place. Also you can override the tag -of the current version pulled by setting [`$RT_CURRENT_TAG`](https://github.com/basho/riak_test/blob/master/bin/rtdev-current.sh#L7) -to a release number, e.g. `2.0.0`. It will automatically be prefixed with -the repo name, e.g. `riak_ee-2.0.0`. To use `riak_ee` instead of `riak` set [`$RT_USE_EE`](https://github.com/basho/riak_test/blob/master/bin/rtdev-setup-releases.sh#L23) -to any non-empty string. +### rtdev-install.sh + +`rtdev-install.sh` will check the releases you just built +into a local git repository. Run this script from the +same directory in which you just built all of your releases. +By default this script initializes the repository into `$HOME/rt/riak` but +you can override [`$RT_DEST_DIR`](https://github.com/basho/riak_test/blob/master/bin/rtdev-install.sh#L24). -#### reset-current-env.sh +Also, make sure that you've already run `make devrel` +or `make stagedevrel` before you run `rtdev-install.sh`. Like setting up +releases you can override [`$RT_VERSION`](https://github.com/basho/riak_test/blob/master/bin/rtdev-install.sh#L28) +so all your Riak builds are in one place. -`reset-current-env.sh` resets test environments setup using `rtdev-current.sh` +### rtdev-migrate.sh + +`rtdev-migrate.sh` will convert existing devrels installed in `$RT_DEST_DIR` +from the legacy format to the new format. It also will reset the local +Git repo. It is only necessary to run this script once. + + +### reset-current-env.sh + +`reset-current-env.sh` resets test environments setup using `rtdev-install.sh` using the following process: 1. Delete the current stagedevrel/devrel environment @@ -180,11 +186,8 @@ to tell riak_test about them. The method of choice is to create a {rtdev, [ {rt_project, "riak"}, - {rtdev_path, [{root, "/home/you/rt/riak"}, - {current, "/home/you/rt/riak/current"}, - {previous, "/home/you/rt/riak/riak-1.4.10"}, - {legacy, "/home/you/rt/riak/riak-1.3.2"} - ]} + {root_path, "/home/you/rt/riak"}, + {default_version, head} ]}. ``` @@ -194,6 +197,10 @@ name you specify. For example, running the command below will use an defaults contain every option you need, you can run riak_test without the `-c` argument. +** Note **: You *need* to have a built version of +[basho_bench](http://www.github.com/basho/basho_bench) setup and added to +your config file before running riak_test. + Some configuration parameters: #### rt_default_config diff --git a/bin/reset-current-env.sh b/bin/reset-current-env.sh index 354fa40e3..285df2737 100755 --- a/bin/reset-current-env.sh +++ b/bin/reset-current-env.sh @@ -22,7 +22,7 @@ VERSION="2.0" NUM_NODES=5 usage() { - echo "Resets the current riak_test environment by rebuilding riak and riak_test using rtdev-current.sh" + echo "Resets the current riak_test environment by rebuilding riak and riak_test using rtdev-install.sh" echo " -c: Perform a devclean on the riak and clean on riak_test projects (default: $FULL_CLEAN)" echo " -n: Number of nodes on which to test (default: $NUM_NODES)" echo " -v: The Riak version to test. The Riak home is calculated as $RT_HOME/riak- (default: $VERSION)" @@ -53,7 +53,7 @@ if ! [[ -d $RIAK_HOME || -h $RIAK_HOME ]]; then exit 1 fi -echo "Reseting the riak_test environment using RIAK_HOME=$RIAK_HOME, RT_HOME=$RT_HOME, NUM_NODES=$NUM_NODES, VERSION=$VERSION, and FULL_CLEAN=$FULL_CLEAN" +echo "Resetting the riak_test environment using RIAK_HOME=$RIAK_HOME, RT_HOME=$RT_HOME, NUM_NODES=$NUM_NODES, VERSION=$VERSION, and FULL_CLEAN=$FULL_CLEAN" cd $RIAK_HOME rm -rf current @@ -64,7 +64,6 @@ if [ "$FULL_CLEAN" = true ] ; then make distclean fi - echo "Removing previous stagedevrel instance from $RIAK_HOME and rebuilding ..." make devclean @@ -72,7 +71,7 @@ make devclean echo "Building Riak stagedevrel with $NUM_NODES nodes in $RIAK_HOME ..." make stagedevrel DEVNODES=$NUM_NODES -$RT_HOME/bin/rtdev-current.sh +$RT_HOME/bin/rtdev-install.sh cd $RT_HOME diff --git a/bin/rtdev-all.sh b/bin/rtdev-all.sh index 32d9595b7..24bef9440 100755 --- a/bin/rtdev-all.sh +++ b/bin/rtdev-all.sh @@ -1,12 +1,209 @@ #!/usr/bin/env bash +# +# Bootstrap an entire riak_test tree +# +# Copyright (c) 2007-2015 Basho Technologies, Inc. All Rights Reserved. +# +# 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. + +# You need to use this script once to build a set of stagedevrels for prior +# releases of Riak (for mixed version / upgrade testing). You should +# create a directory and then run this script from within that directory. +# I have ~/test-releases that I created once, and then re-use for testing. +# + +# Different versions of Riak were released using different Erlang versions, +# make sure to build with the appropriate version. + +# This is based on my usage of having multiple Erlang versions in different +# directories. If using kerl or whatever, modify to use kerl's activate logic. +# Or, alternatively, just substitute the paths to the kerl install paths as +# that should work too. + +# Set these values for non-default behavior +# Path to the Erlang R15B01 Installation +: ${R15B01:=$HOME/erlang/R15B01} +# Path to the Erlang R15B01 Installation +: ${R16B02:=$HOME/erlang/R16B02} +# Current version of Erlang (for "head" version) +: ${CURRENT_OTP:=$R16B02} +# Label of the "current" version +: ${VERSION:="head"} +# By default the Open Source version of Riak will be used, but for internal +# testing you can override this variable to use `riak_ee` instead +: ${RT_USE_EE:=""} ORIGDIR=`pwd` pushd `dirname $0` > /dev/null SCRIPT_DIR=`pwd` popd > /dev/null -: ${CURRENT_OTP:=$HOME/erlang-R16B02} -: ${RT_CURRENT_TAG:=""} +GITURL_RIAK="git://github.com/basho/riak" +GITURL_RIAK_EE="git@github.com:basho/riak_ee" + +checkbuild() +{ + ERLROOT=$1 + + if [ ! -d $ERLROOT ]; then + echo -n " - $ERLROOT cannot be found, install with kerl? [Y|n]: " + read ans + if [[ $ans == n || $ans == N ]]; then + echo + echo " [ERROR] Can't build $ERLROOT without kerl, aborting!" + exit 1 + else + if [ ! -x kerl ]; then + echo " - Fetching kerl." + if [ ! `which curl` ]; then + echo "You need 'curl' to be able to run this script, exiting" + exit 1 + fi + curl -O https://raw.githubusercontent.com/spawngrid/kerl/master/kerl; chmod a+x kerl + fi + fi + fi +} + +kerl() +{ + RELEASE=$1 + BUILDNAME=$2 + + if [ -n "`uname -r | grep el6`" ]; then + export CFLAGS="-DOPENSSL_NO_EC=1" + fi + if [ "`uname`" == "Darwin" ]; then + export CFLAGS="-O0" + export KERL_CONFIGURE_OPTIONS="-disable-hipe --enable-smp-support --enable-threads --enable-kernel-poll --without-odbc --enable-darwin-64bit" + fi + echo " - Building Erlang $RELEASE (this could take a while)" + # Use the patched version of Erlang for R16B02 builds + if [ "$RELEASE" == "R16B02" ]; then + ./kerl build git git://github.com/basho/otp.git r16 $BUILDNAME + else + ./kerl build $RELEASE $BUILDNAME + fi + RES=$? + if [ "$RES" -ne 0 ]; then + echo "[ERROR] Kerl build $BUILDNAME failed" + exit 1 + fi + + echo " - Installing $RELEASE into $HOME/$BUILDNAME" + ./kerl install $BUILDNAME $HOME/$BUILDNAME + RES=$? + if [ "$RES" -ne 0 ]; then + echo "[ERROR] Kerl install $BUILDNAME failed" + exit 1 + fi +} + +build() +{ + SRCDIR=$1 + ERLROOT=$2 + if [ -z "$RT_USE_EE" ]; then + GITURL=$GITURL_RIAK + else + GITURL=$GITURL_RIAK_EE + fi + + echo "Building $SRCDIR:" + + checkbuild $ERLROOT + if [ ! -d $ERLROOT ]; then + BUILDNAME=`basename $ERLROOT` + RELEASE=`echo $BUILDNAME | awk -F- '{ print $2 }'` + kerl $RELEASE $BUILDNAME + fi + + GITRES=1 + echo " - Cloning $GITURL" + rm -rf $SRCDIR + git clone $GITURL $SRCDIR + GITRES=$? + if [ $GITRES -eq 0 -a -n "$SRCDIR" ]; then + cd $SRCDIR + git checkout $SRCDIR + GITRES=$? + cd .. + fi + RUN="env PATH=$ERLROOT/bin:$ERLROOT/lib/erlang/bin:$PATH \ + C_INCLUDE_PATH=$ERLROOT/usr/include \ + LD_LIBRARY_PATH=$ERLROOT/usr/lib" + fix_riak_1_3 $SRCDIR "$RUN" + + echo " - Building stagedevrel in $SRCDIR (this could take a while)" + cd $SRCDIR + + $RUN make locked-deps + $RUN make all stagedevrel + RES=$? + if [ "$RES" -ne 0 ]; then + echo "[ERROR] make stagedevrel failed" + exit 1 + fi + echo " - $SRCDIR built." + $SCRIPT_DIR/rtdev-install.sh + cd .. +} + +# Riak 1.3 has a few artifacts which need to be updated in order to build +# properly +fix_riak_1_3() +{ + SRCDIR=$1 + RUN="$2" + + if [ "`echo $SRCDIR | cut -d- -f2 | cut -d. -f1-2`" != "1.3" ]; then + return 0 + fi + + echo "- Patching Riak 1.3.x" + cd $SRCDIR + if [ "$SRCDIR" == "riak-1.3.2" ]; then + cat < + #include + #include ++#include + #include "leveldb/perf_count.h" + #include "leveldb/status.h" +EOF + cd ../../../../../../.. +} if [ -n "$DEBUG_RTDEV" ]; then echo "= Configuration =================================================" @@ -34,22 +231,15 @@ echo echo "= Building Riak Releases ========================================" echo -source $SCRIPT_DIR/rtdev-build-releases.sh - -echo "= Installing Riak Releases ======================================" -echo -source $SCRIPT_DIR/rtdev-setup-releases.sh - -echo -echo "= Building and Installing Riak from Git =========================" -echo - -cd $ORIGDIR -build "current" $CURRENT_OTP $RT_CURRENT_TAG echo -cd current -source $SCRIPT_DIR/rtdev-current.sh +if [ -z "$RT_USE_EE" ]; then + build "riak-1.3.2" $R15B01 + build "riak-1.4.12" $R15B01 +else + build "riak_ee-1.3.4" $R15B01 + build "riak_ee-1.4.12" $R15B01 +fi +build "$VERSION" $R16B02 -cd $ORIGDIR echo echo "= Build complete! ===============================================" diff --git a/bin/rtdev-build-releases.sh b/bin/rtdev-build-releases.sh deleted file mode 100755 index e170a31d1..000000000 --- a/bin/rtdev-build-releases.sh +++ /dev/null @@ -1,176 +0,0 @@ -#!/usr/bin/env bash - -# You need to use this script once to build a set of stagedevrels for prior -# releases of Riak (for mixed version / upgrade testing). You should -# create a directory and then run this script from within that directory. -# I have ~/test-releases that I created once, and then re-use for testing. -# -# See rtdev-setup-releases.sh as an example of setting up mixed version layout -# for testing. - -# Different versions of Riak were released using different Erlang versions, -# make sure to build with the appropriate version. - -# This is based on my usage of having multiple Erlang versions in different -# directories. If using kerl or whatever, modify to use kerl's activate logic. -# Or, alternatively, just substitute the paths to the kerl install paths as -# that should work too. - -: ${R15B01:=$HOME/erlang-R15B01} -: ${R16B02:=$HOME/erlang-R16B02} - -# By default the Open Source version of Riak will be used, but for internal -# testing you can override this variable to use `riak_ee` instead -: ${RT_USE_EE:=""} -GITURL_RIAK="git://github.com/basho/riak" -GITURL_RIAK_EE="git@github.com:basho/riak_ee" - - -checkbuild() -{ - ERLROOT=$1 - - if [ ! -d $ERLROOT ]; then - echo -n " - $ERLROOT cannot be found, install with kerl? [Y|n]: " - read ans - if [[ $ans == n || $ans == N ]]; then - echo - echo " [ERROR] Can't build $ERLROOT without kerl, aborting!" - exit 1 - else - if [ ! -x kerl ]; then - echo " - Fetching kerl." - if [ ! `which curl` ]; then - echo "You need 'curl' to be able to run this script, exiting" - exit 1 - fi - curl -O https://raw.github.com/spawngrid/kerl/master/kerl > /dev/null 2>&1; chmod a+x kerl - fi - fi - fi -} - -kerl() -{ - RELEASE=$1 - BUILDNAME=$2 - - echo " - Building Erlang $RELEASE (this could take a while)" - ./kerl build $RELEASE $BUILDNAME > /dev/null 2>&1 - RES=$? - if [ "$RES" -ne 0 ]; then - echo "[ERROR] Kerl build $BUILDNAME failed" - exit 1 - fi - - echo " - Installing $RELEASE into $HOME/$BUILDNAME" - ./kerl install $BUILDNAME $HOME/$BUILDNAME > /dev/null 2>&1 - RES=$? - if [ "$RES" -ne 0 ]; then - echo "[ERROR] Kerl install $BUILDNAME failed" - exit 1 - fi -} - -build() -{ - SRCDIR=$1 - ERLROOT=$2 - TAG="$3" - if [ -z "$RT_USE_EE" ]; then - GITURL=$GITURL_RIAK - GITTAG=riak-$TAG - else - GITURL=$GITURL_RIAK_EE - GITTAG=riak_ee-$TAG - fi - - echo "Building $SRCDIR:" - - checkbuild $ERLROOT - if [ ! -d $ERLROOT ]; then - BUILDNAME=`basename $ERLROOT` - RELEASE=`echo $BUILDNAME | awk -F- '{ print $2 }'` - kerl $RELEASE $BUILDNAME - fi - - GITRES=1 - echo " - Cloning $GITURL" - rm -rf $SRCDIR - git clone $GITURL $SRCDIR - GITRES=$? - if [ $GITRES -eq 0 -a -n "$TAG" ]; then - cd $SRCDIR - git checkout $GITTAG - GITRES=$? - cd .. - fi - - RUN="env PATH=$ERLROOT/bin:$ERLROOT/lib/erlang/bin:$PATH \ - C_INCLUDE_PATH=$ERLROOT/usr/include \ - LD_LIBRARY_PATH=$ERLROOT/usr/lib" - fix_riak_1_3 $SRCDIR $TAG "$RUN" - - echo " - Building stagedevrel in $SRCDIR (this could take a while)" - cd $SRCDIR - - $RUN make all stagedevrel - RES=$? - if [ "$RES" -ne 0 ]; then - echo "[ERROR] make stagedevrel failed" - exit 1 - fi - cd .. - echo " - $SRCDIR built." -} - -# Riak 1.3 has a few artifacts which need to be updated in order to build -# properly -fix_riak_1_3() -{ - SRCDIR=$1 - TAG="$2" - RUN="$3" - - if [ "`echo $TAG | cut -d . -f1-2`" != "1.3" ]; then - return 0 - fi - - echo "- Patching Riak 1.3.x" - cd $SRCDIR - cat < - #include - #include -+#include - #include "leveldb/perf_count.h" - #include "leveldb/status.h" -EOF - cd ../../../../../../.. -} - -build "riak-1.4.10" $R15B01 http://s3.amazonaws.com/downloads.basho.com/riak/1.4/1.4.10/riak-1.4.10.tar.gz -echo -if [ -z "$RT_USE_EE" ]; then - build "riak-1.3.2" $R15B01 1.3.2 -else - build "riak-1.3.4" $R15B01 1.3.4 -fi -echo diff --git a/bin/rtdev-current.sh b/bin/rtdev-current.sh deleted file mode 100755 index b756af622..000000000 --- a/bin/rtdev-current.sh +++ /dev/null @@ -1,38 +0,0 @@ -#!/usr/bin/env bash - -# just bail out if things go south -set -e - -: ${RT_DEST_DIR:="$HOME/rt/riak"} -# If RT_CURRENT_TAG is specified, it will use that version number -# otherwise the last annotated tag will be used -: ${RT_CURRENT_TAG:=""} - -echo "Making $(pwd) a tagged release:" -cwd=$(pwd) -echo -n " - Determining version: " -if [ -n "$RT_CURRENT_TAG" ]; then - VERSION=$RT_CURRENT_TAG -elif [ -f $cwd/dependency_manifest.git ]; then - VERSION=`cat $cwd/dependency_manifest.git | awk '/^-/ { print $NF }'` -else - VERSION=`git describe --tags | awk '{sub(/riak-/,"",$0);print}'` -fi -echo $VERSION -cd $RT_DEST_DIR -echo " - Resetting existing $RT_DEST_DIR" -git reset HEAD --hard > /dev/null 2>&1 -git clean -fd > /dev/null 2>&1 -echo " - Removing and recreating $RT_DEST_DIR/$VERSION" -rm -rf $RT_DEST_DIR/$VERSION -mkdir $RT_DEST_DIR/$VERSION -cd $cwd -echo " - Copying devrel to $RT_DEST_DIR/$VERSION" -cd dev -for i in `ls`; do cp -p -P -R $i $RT_DEST_DIR/$VERSION/; done -echo " - Writing $RT_DEST_DIR/$VERSION/VERSION" -echo -n $VERSION > $RT_DEST_DIR/$VERSION/VERSION -cd $RT_DEST_DIR -echo " - Reinitializing git state" -git add -f . -git commit -a -m "riak_test init" --amend > /dev/null 2>&1 diff --git a/bin/rtdev-install.sh b/bin/rtdev-install.sh index 5b34a65f7..4b1ededb8 100755 --- a/bin/rtdev-install.sh +++ b/bin/rtdev-install.sh @@ -1,4 +1,22 @@ #!/usr/bin/env bash +# +# Install a devrel or stagedevrel version of Riak for riak_test +# +# Copyright (c) 2007-2015 Basho Technologies, Inc. All Rights Reserved. +# +# 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. # just bail out if things go south set -e @@ -7,41 +25,62 @@ set -e cwd=$(pwd) echo -n " - Determining version: " -if [ -z "${VERSION+xxx}" ] || ([ -z "$VERSION" ] && [ "${VERSION+xxx}" = "xxx" ]); then - if [ -f $cwd/dependency_manifest.git ]; then - VERSION=`cat $cwd/dependency_manifest.git | awk '/^-/ { print $NF }'` - else - echo "Making $(pwd) a tagged release:" - VERSION=`git describe --tags | awk '{sub(/riak-/,"",$0);print}'` - fi +if [ -z "${RT_VERSION+xxx}" ] || ([ -z "$RT_VERSION" ] && [ "${RT_VERSION+xxx}" = "xxx" ]); then + if [ -f $cwd/dependency_manifest.git ]; then + # For packaged distributions + RT_VERSION=`cat $cwd/dependency_manifest.git | awk '/^-/ { print $NF }'` + else + echo "Making $(pwd) a tagged release:" + TAGS=`git describe --tags` + CURRENT=`git rev-parse HEAD` + HEAD=`git show-ref | grep HEAD | cut -f1 -d" "` + if [ -n "`echo ${TAGS} | grep riak_ee`" ]; then + # For riak_ee + RT_VERSION=`echo ${TAGS} | awk '{sub(/riak_ee-/,"",$0);print}'` + else + # For open source riak + RT_VERSION=`echo ${TAGS} | awk '{sub(/riak-/,"",$0);print}'` + fi + # If we are on the tip, call the version "head" + if [ "${CURRENT}" == "${HEAD}" ]; then + RT_VERSION="head" + fi + fi fi -echo $VERSION +echo $RT_VERSION + +# Create the RT_DEST_DIR directory if it does not yet exist if [ ! -d $RT_DEST_DIR ]; then - mkdir $RT_DEST_DIR + mkdir -p $RT_DEST_DIR fi + +# Reinitialize the Git repo if it already exists, +# including removing untracked files cd $RT_DEST_DIR if [ -d ".git" ]; then echo " - Resetting existing $RT_DEST_DIR" git reset HEAD --hard > /dev/null 2>&1 git clean -fd > /dev/null 2>&1 fi -echo " - Removing and recreating $RT_DEST_DIR/$VERSION" -rm -rf $RT_DEST_DIR/$VERSION -mkdir $RT_DEST_DIR/$VERSION +echo " - Removing and recreating $RT_DEST_DIR/$RT_VERSION" +rm -rf $RT_DEST_DIR/$RT_VERSION +mkdir $RT_DEST_DIR/$RT_VERSION cd $cwd -echo " - Copying devrel to $RT_DEST_DIR/$VERSION" +echo " - Copying devrel to $RT_DEST_DIR/$RT_VERSION" if [ ! -d "dev" ]; then echo "You need to run \"make devrel\" or \"make stagedevrel\" first" exit 1 fi cd dev -for i in `ls`; do cp -p -P -R $i $RT_DEST_DIR/$VERSION/; done -echo " - Writing $RT_DEST_DIR/$VERSION/VERSION" -echo -n $VERSION > $RT_DEST_DIR/$VERSION/VERSION + +# Clone the existing dev directory into RT_DEST_DIR +for i in `ls`; do cp -p -P -R $i $RT_DEST_DIR/$RT_VERSION/; done +echo " - Writing $RT_DEST_DIR/$RT_VERSION/VERSION" +echo -n $RT_VERSION > $RT_DEST_DIR/$RT_VERSION/VERSION cd $RT_DEST_DIR if [ -d ".git" ]; then echo " - Reinitializing git state" - git add -f . + git add --ignore-removal -f . git commit -a -m "riak_test init" --amend > /dev/null 2>&1 else git init @@ -50,7 +89,7 @@ else git config user.name "Riak Test" git config user.email "dev@basho.com" - git add . + git add --ignore-removal . git commit -a -m "riak_test init" > /dev/null echo " - Successfully completed initial git commit of $RT_DEST_DIR" fi diff --git a/bin/rtdev-migrate.sh b/bin/rtdev-migrate.sh new file mode 100644 index 000000000..2c04767ce --- /dev/null +++ b/bin/rtdev-migrate.sh @@ -0,0 +1,69 @@ +#!/usr/bin/env bash +# +# Bootstrap an entire riak_test tree +# +# Copyright (c) 2007-2015 Basho Technologies, Inc. All Rights Reserved. +# +# 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. + +# just bail out if things go south +set -e + +: ${RT_DEST_DIR:="$HOME/rt/riak"} +cd ${RT_DEST_DIR} + +# New "standard" for current version of Riak +if [ -d current ]; then + echo "Renaming current to head" + mv -f current head +fi + +# Chop off the "riak-" prefix +for name in riak-*; do + if [ -d "${name}" ]; then + NEW_NAME=`echo ${name} | cut -d- -f2` + echo "Renaming ${name} to ${NEW_NAME}" + mv ${name} ${NEW_NAME} + fi +done + +# Remove the intermediate "dev" directory +for version in `ls -1`; do + if [ -d "${version}/dev" ]; then + echo "Removing the dev directory from ${version}" + cd ${version}/dev + mv dev* .. + cd .. + rmdir dev + cd .. + fi +done + +# Set up local Git repo +if [ -d ".git" ]; then + echo " - Reinitializing git state" + git add -f --ignore-removal . + git commit -a -m "riak_test init" --amend > /dev/null 2>&1 +else + git init + + ## Some versions of git and/or OS require these fields + git config user.name "Riak Test" + git config user.email "dev@basho.com" + + git add --ignore-removal . + git commit -a -m "riak_test init" > /dev/null + echo " - Successfully completed initial git commit of $RT_DEST_DIR" +fi diff --git a/bin/rtdev-setup-releases.sh b/bin/rtdev-setup-releases.sh deleted file mode 100755 index fef0629c4..000000000 --- a/bin/rtdev-setup-releases.sh +++ /dev/null @@ -1,45 +0,0 @@ -#!/usr/bin/env bash - -# bail out if things go south -set -e - -# Creates a mixed-version directory structure for running riak_test -# using rtdev-mixed.config settings. Should be run inside a directory -# that contains devrels for prior Riak releases. Easy way to create this -# is to use the rtdev-build-releases.sh script - -: ${RT_DEST_DIR:="$HOME/rt/riak"} - -echo "Setting up releases from $(pwd):" -echo " - Creating $RT_DEST_DIR" - -rm -rf $RT_DEST_DIR -mkdir -p $RT_DEST_DIR - -count=$(ls * 2> /dev/null | wc -l) -if [ "$count" -ne "0" ] -then - for rel in *; do - vsn=$(dirname "$rel") - echo " - Initializing $RT_DEST_DIR/$vsn" - mkdir -p "$RT_DEST_DIR/$vsn" - cp -p -P -R "$rel" "$RT_DEST_DIR/$vsn" - done -else - # This is useful when only testing with 'current' - # The repo still needs to be initialized for current - # and we don't want to bomb out if */dev doesn't exist - touch $RT_DEST_DIR/.current_init - echo "No devdirs found. Not copying any releases." -fi - -cd $RT_DEST_DIR -git init - -## Some versions of git and/or OS require these fields -git config user.name "Riak Test" -git config user.email "dev@basho.com" - -git add . -git commit -a -m "riak_test init" > /dev/null -echo " - Successfully completed initial git commit of $RT_DEST_DIR" From a9480194327b3be975290a91faad0a23fe555933 Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Tue, 29 Jul 2014 18:50:48 -0400 Subject: [PATCH 050/157] Move configuration functions to rt_config. - functions include set_conf/2, set_advanced_conf/2, and update_app_config/2. --- src/rt_config.erl | 30 +++++++++++-------------- tests/replication/repl_bucket_types.erl | 6 +++++ tests/replication/replication.erl | 4 +++- tests/replication/replication2_pg.erl | 2 +- 4 files changed, 23 insertions(+), 19 deletions(-) diff --git a/src/rt_config.erl b/src/rt_config.erl index f6e5e8867..39ef3788e 100644 --- a/src/rt_config.erl +++ b/src/rt_config.erl @@ -31,8 +31,7 @@ set/2, set_conf/2, set_advanced_conf/2, - update_app_config/2, - version_to_config/1 + update_app_config/2 ]). -define(HARNESS, (rt_config:get(rt_harness))). @@ -131,34 +130,31 @@ config_or_os_env(Config, Default) -> -spec set_conf(atom(), [{string(), string()}]) -> ok. set_conf(all, NameValuePairs) -> - rt_harness:set_conf(all, NameValuePairs); + ?HARNESS:set_conf(all, NameValuePairs); set_conf(Node, NameValuePairs) -> - rt_node:stop(Node), + rt:stop(Node), ?assertEqual(ok, rt:wait_until_unpingable(Node)), - rt_harness:set_conf(Node, NameValuePairs), - rt_node:start(Node). + ?HARNESS:set_conf(Node, NameValuePairs), + rt:start(Node). -spec set_advanced_conf(atom(), [{string(), string()}]) -> ok. set_advanced_conf(all, NameValuePairs) -> - rt_harness:set_advanced_conf(all, NameValuePairs); + ?HARNESS:set_advanced_conf(all, NameValuePairs); set_advanced_conf(Node, NameValuePairs) -> - rt_node:stop(Node), + rt:stop(Node), ?assertEqual(ok, rt:wait_until_unpingable(Node)), - rt_harness:set_advanced_conf(Node, NameValuePairs), - rt_node:start(Node). + ?HARNESS:set_advanced_conf(Node, NameValuePairs), + rt:start(Node). %% @doc Rewrite the given node's app.config file, overriding the varialbes %% in the existing app.config with those in `Config'. update_app_config(all, Config) -> - rt_harness:update_app_config(all, Config); + ?HARNESS:update_app_config(all, Config); update_app_config(Node, Config) -> - rt_node:stop(Node), + rt:stop(Node), ?assertEqual(ok, rt:wait_until_unpingable(Node)), - rt_harness:update_app_config(Node, Config), - rt_node:start(Node). - -version_to_config(Config) when is_tuple(Config)-> Config; -version_to_config(Version) -> {Version, default}. + ?HARNESS:update_app_config(Node, Config), + rt:start(Node). to_upper(S) -> lists:map(fun char_to_upper/1, S). char_to_upper(C) when C >= $a, C =< $z -> C bxor $\s; diff --git a/tests/replication/repl_bucket_types.erl b/tests/replication/repl_bucket_types.erl index 09c1df081..f610ba138 100644 --- a/tests/replication/repl_bucket_types.erl +++ b/tests/replication/repl_bucket_types.erl @@ -18,6 +18,12 @@ -define(ENSURE_READ_INTERVAL, 1000). %% Replication Bucket Types test +%% + +setup(Type) -> + rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), + + {LeaderA, LeaderB, ANodes, BNodes} = ClusterNodes = make_clusters(Type), -spec properties() -> rt_properties:properties(). properties() -> diff --git a/tests/replication/replication.erl b/tests/replication/replication.erl index 7bcf7b8a4..bccc73068 100644 --- a/tests/replication/replication.erl +++ b/tests/replication/replication.erl @@ -17,7 +17,9 @@ confirm() -> ]} ], rt_config:set_advanced_conf(all, Conf), - [ANodes, BNodes] = rt_cluster:build_clusters([3, 3]), + [ANodes, BNodes] = rt:build_clusters([3, 3]), + rt:wait_for_cluster_service(ANodes, riak_repl), + rt:wait_for_cluster_service(BNodes, riak_repl), replication(ANodes, BNodes, false), pass. diff --git a/tests/replication/replication2_pg.erl b/tests/replication/replication2_pg.erl index 1198a5099..01b0a7a57 100644 --- a/tests/replication/replication2_pg.erl +++ b/tests/replication/replication2_pg.erl @@ -65,7 +65,7 @@ setup_repl_clusters(Conf, SSL) -> rt_config:set_advanced_conf(all, Conf), - Nodes = [ANodes, BNodes, CNodes] = rt_cluster:build_clusters([2, 2, 2]), + Nodes = [ANodes, BNodes, CNodes] = rt:build_clusters([2, 2, 2]), rt:wait_for_cluster_service(ANodes, riak_repl), rt:wait_for_cluster_service(BNodes, riak_repl), From f7cf16fcf73deb1a69ee8ea68de0b5031a07a979 Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Wed, 30 Jul 2014 10:10:30 -0400 Subject: [PATCH 051/157] Migrate several more functions into rt_cluster from rt. --- src/rt.erl | 170 ---------- src/rt_cluster.erl | 314 ++++++++---------- tests/cluster_meta_rmr.erl | 2 +- tests/gh_riak_core_154.erl | 2 +- tests/http_bucket_types.erl | 5 + tests/post_generate_key.erl | 2 +- tests/replication/repl_bucket_types.erl | 21 ++ tests/replication/replication.erl | 4 +- .../replication2_console_tests.erl | 2 +- tests/replication/replication2_pg.erl | 2 +- tests/riak_admin_console_tests.erl | 2 +- tests/riaknostic_rt.erl | 2 +- tests/verify_2i_aae.erl | 27 +- tests/verify_2i_limit.erl | 2 +- tests/verify_2i_returnterms.erl | 2 +- tests/verify_2i_stream.erl | 2 +- tests/verify_2i_timeout.erl | 2 +- tests/verify_counter_converge.erl | 2 +- tests/verify_cs_bucket.erl | 2 +- tests/verify_link_walk_urls.erl | 2 +- tests/verify_listkeys_eqcfsm.erl | 2 +- tests/verify_no_writes_on_read.erl | 2 +- tests/verify_riak_lager.erl | 2 +- tests/verify_secondary_index_reformat.erl | 2 +- tests/verify_snmp.erl | 4 +- tests/verify_tick_change.erl | 2 +- tests/verify_vclock_encoding_upgrade.erl | 4 +- 27 files changed, 200 insertions(+), 387 deletions(-) diff --git a/src/rt.erl b/src/rt.erl index 48caa303c..a199d5843 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -36,28 +36,16 @@ attach/2, attach_direct/2, brutal_kill/1, - build_cluster/1, - build_cluster/2, - build_cluster/3, - build_clusters/1, - join_cluster/1, capability/2, capability/3, check_singleton_node/1, check_ibrowse/0, claimant_according_to/1, - clean_cluster/1, - clean_data_dir/1, - clean_data_dir/2, cmd/1, cmd/2, connection_info/1, console/2, create_and_activate_bucket_type/3, - deploy_nodes/1, - deploy_nodes/2, - deploy_nodes/3, - deploy_clusters/1, down/2, enable_search_hook/2, expect_in_log/2, @@ -129,8 +117,6 @@ systest_write/3, systest_write/5, systest_write/6, - teardown/0, - update_app_config/2, upgrade/2, upgrade/3, wait_for_cluster_service/2, @@ -264,55 +250,6 @@ get_https_conn_info(Node) -> undefined end. -%% @doc Deploy a set of freshly installed Riak nodes, returning a list of the -%% nodes deployed. -%% @todo Re-add -spec after adding multi-version support -deploy_nodes(Versions) when is_list(Versions) -> - deploy_nodes(Versions, [riak_kv]); -deploy_nodes(NumNodes) when is_integer(NumNodes) -> - deploy_nodes([ current || _ <- lists:seq(1, NumNodes)]). - -%% @doc Deploy a set of freshly installed Riak nodes with the given -%% `InitialConfig', returning a list of the nodes deployed. --spec deploy_nodes(NumNodes :: integer(), any()) -> [node()]. -deploy_nodes(NumNodes, InitialConfig) when is_integer(NumNodes) -> - deploy_nodes(NumNodes, InitialConfig, [riak_kv]); -deploy_nodes(Versions, Services) -> - NodeConfig = [ version_to_config(Version) || Version <- Versions ], - Nodes = ?HARNESS:deploy_nodes(NodeConfig), - lager:info("Waiting for services ~p to start on ~p.", [Services, Nodes]), - [ ok = wait_for_service(Node, Service) || Node <- Nodes, - Service <- Services ], - Nodes. - -deploy_nodes(NumNodes, InitialConfig, Services) when is_integer(NumNodes) -> - NodeConfig = [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)], - deploy_nodes(NodeConfig, Services). - -version_to_config(Config) when is_tuple(Config)-> Config; -version_to_config(Version) -> {Version, default}. - -deploy_clusters(Settings) -> - ClusterConfigs = [case Setting of - Configs when is_list(Configs) -> - Configs; - NumNodes when is_integer(NumNodes) -> - [{current, default} || _ <- lists:seq(1, NumNodes)]; - {NumNodes, InitialConfig} when is_integer(NumNodes) -> - [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)]; - {NumNodes, Vsn, InitialConfig} when is_integer(NumNodes) -> - [{Vsn, InitialConfig} || _ <- lists:seq(1,NumNodes)] - end || Setting <- Settings], - ?HARNESS:deploy_clusters(ClusterConfigs). - -build_clusters(Settings) -> - Clusters = deploy_clusters(Settings), - [begin - join_cluster(Nodes), - lager:info("Cluster built: ~p", [Nodes]) - end || Nodes <- Clusters], - Clusters. - %% @doc Start the specified Riak node start(Node) -> ?HARNESS:start(Node). @@ -1015,113 +952,6 @@ claimant_according_to(Node) -> end. %%%=================================================================== -%%% Cluster Utility Functions -%%%=================================================================== - -%% @doc Safely construct a new cluster and return a list of the deployed nodes -%% @todo Add -spec and update doc to reflect mult-version changes -build_cluster(Versions) when is_list(Versions) -> - build_cluster(length(Versions), Versions, default); -build_cluster(NumNodes) -> - build_cluster(NumNodes, default). - -%% @doc Safely construct a `NumNode' size cluster using -%% `InitialConfig'. Return a list of the deployed nodes. -build_cluster(NumNodes, InitialConfig) -> - build_cluster(NumNodes, [], InitialConfig). - -build_cluster(NumNodes, Versions, InitialConfig) -> - %% Deploy a set of new nodes - Nodes = - case Versions of - [] -> - deploy_nodes(NumNodes, InitialConfig); - _ -> - deploy_nodes(Versions) - end, - - join_cluster(Nodes), - lager:info("Cluster built: ~p", [Nodes]), - Nodes. - -join_cluster(Nodes) -> - %% Ensure each node owns 100% of it's own ring - [?assertEqual([Node], owners_according_to(Node)) || Node <- Nodes], - - %% Join nodes - [Node1|OtherNodes] = Nodes, - case OtherNodes of - [] -> - %% no other nodes, nothing to join/plan/commit - ok; - _ -> - %% ok do a staged join and then commit it, this eliminates the - %% large amount of redundant handoff done in a sequential join - [staged_join(Node, Node1) || Node <- OtherNodes], - plan_and_commit(Node1), - try_nodes_ready(Nodes, 3, 500) - end, - - ?assertEqual(ok, wait_until_nodes_ready(Nodes)), - - %% Ensure each node owns a portion of the ring - wait_until_nodes_agree_about_ownership(Nodes), - ?assertEqual(ok, wait_until_no_pending_changes(Nodes)), - ok. - --type products() :: riak | riak_ee | riak_cs | unknown. - --spec product(node()) -> products(). -product(Node) -> - Applications = rpc:call(Node, application, which_applications, []), - - HasRiakCS = proplists:is_defined(riak_cs, Applications), - HasRiakEE = proplists:is_defined(riak_repl, Applications), - HasRiak = proplists:is_defined(riak_kv, Applications), - if HasRiakCS -> riak_cs; - HasRiakEE -> riak_ee; - HasRiak -> riak; - true -> unknown - end. - -try_nodes_ready([Node1 | _Nodes], 0, _SleepMs) -> - lager:info("Nodes not ready after initial plan/commit, retrying"), - plan_and_commit(Node1); -try_nodes_ready(Nodes, N, SleepMs) -> - ReadyNodes = [Node || Node <- Nodes, is_ready(Node) =:= true], - case ReadyNodes of - Nodes -> - ok; - _ -> - timer:sleep(SleepMs), - try_nodes_ready(Nodes, N-1, SleepMs) - end. - -%% @doc Stop nodes and wipe out their data directories -clean_cluster(Nodes) when is_list(Nodes) -> - [stop_and_wait(Node) || Node <- Nodes], - clean_data_dir(Nodes). - -clean_data_dir(Nodes) -> - clean_data_dir(Nodes, ""). - -clean_data_dir(Nodes, SubDir) when not is_list(Nodes) -> - clean_data_dir([Nodes], SubDir); -clean_data_dir(Nodes, SubDir) when is_list(Nodes) -> - ?HARNESS:clean_data_dir(Nodes, SubDir). - -%% @doc Shutdown every node, this is for after a test run is complete. -teardown() -> - %% stop all connected nodes, 'cause it'll be faster that - %%lager:info("RPC stopping these nodes ~p", [nodes()]), - %%[ rt:stop(Node) || Node <- nodes()], - %% Then do the more exhaustive harness thing, in case something was up - %% but not connected. - ?HARNESS:teardown(). - -versions() -> - ?HARNESS:versions(). -%%%=================================================================== %%% Basic Read/Write Functions %%%=================================================================== diff --git a/src/rt_cluster.erl b/src/rt_cluster.erl index 74bdaf1df..a87292472 100644 --- a/src/rt_cluster.erl +++ b/src/rt_cluster.erl @@ -22,186 +22,102 @@ -include_lib("eunit/include/eunit.hrl"). -export([properties/0, - setup/1, + setup/2, + config/0, augment_config/3, + deploy_nodes/1, + deploy_nodes/2, + deploy_clusters/1, + build_cluster/1, + build_cluster/2, + build_cluster/3, + build_clusters/1, clean_cluster/1, - join_cluster/2, + join_cluster/1, clean_data_dir/1, clean_data_dir/2, try_nodes_ready/3, versions/0, teardown/0]). - --export([maybe_wait_for_transfers/3]). +-export([maybe_wait_for_transfers/2]). -include("rt.hrl"). +-define(HARNESS, (rt_config:get(rt_harness))). + %% @doc Default properties used if a riak_test module does not specify %% a custom properties function. -spec properties() -> rt_properties:properties(). properties() -> - rt_properties:new(). - --spec setup(rt_properties:properties()) -> - {ok, rt_properties:properties()} | {error, term()}. -setup(Properties) -> - case form_clusters(Properties) of - {ok, ClusterNodes} -> - maybe_wait_for_transfers(rt_properties:get(node_ids, Properties), - rt_properties:get(node_map, Properties), - rt_properties:get(wait_for_transfers, Properties)), - Clusters = prepare_clusters(ClusterNodes, Properties), - create_bucket_types(Clusters, Properties), - rt_properties:set(clusters, Clusters, Properties); - Error -> - Error - end. - --spec create_bucket_types([rt_cluster_info:cluster_info()], rt_properties:properties()) -> no_return(). -create_bucket_types(Clusters, Properties) -> - BucketTypes = rt_properties:get(bucket_types, Properties), - create_bucket_types(Clusters, Properties, BucketTypes). - --spec create_bucket_types([rt_cluster_info:cluster_info()], - rt_properties:properties(), - rt_properties:bucket_types()) -> no_return(). -create_bucket_types(_Clusters, _Properties, []) -> - ok; -create_bucket_types([Cluster], Properties, BucketTypes) -> - NodeMap = rt_properties:get(node_map, Properties), - NodeIds = rt_cluster_info:get(node_ids, Cluster), - Nodes = [rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds], - lists:foldl(fun maybe_create_bucket_type/2, {Nodes, 1}, BucketTypes); -create_bucket_types(Clusters, Properties, BucketTypes) -> - NodeMap = rt_properties:get(node_map, Properties), + #rt_properties{config=config()}. + +-spec setup(rt_properties(), proplists:proplist()) -> + {ok, rt_properties()} | {error, term()}. +setup(Properties, MetaData) -> + rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), + + RollingUpgrade = proplists:get_value(rolling_upgrade, + MetaData, + Properties#rt_properties.rolling_upgrade), + Version = Properties#rt_properties.start_version, + Versions = [{Version, Properties#rt_properties.config} || + _ <- lists:seq(1, Properties#rt_properties.node_count)], + Nodes = deploy_or_build_cluster(Versions, Properties#rt_properties.make_cluster), + + maybe_wait_for_transfers(Nodes, Properties#rt_properties.wait_for_transfers), + UpdProperties = Properties#rt_properties{nodes=Nodes, + rolling_upgrade=RollingUpgrade}, + {ok, UpdProperties}. + +deploy_or_build_cluster(Versions, true) -> + build_cluster(Versions); +deploy_or_build_cluster(Versions, false) -> + deploy_nodes(Versions). + +%% @doc Deploy a set of freshly installed Riak nodes, returning a list of the +%% nodes deployed. +%% @todo Re-add -spec after adding multi-version support +deploy_nodes(Versions) when is_list(Versions) -> + deploy_nodes(Versions, [riak_kv]); +deploy_nodes(NumNodes) when is_integer(NumNodes) -> + deploy_nodes([ current || _ <- lists:seq(1, NumNodes)]). + +%% @doc Deploy a set of freshly installed Riak nodes with the given +%% `InitialConfig', returning a list of the nodes deployed. +-spec deploy_nodes(NumNodes :: integer(), any()) -> [node()]. +deploy_nodes(NumNodes, InitialConfig) when is_integer(NumNodes) -> + NodeConfig = [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)], + deploy_nodes(NodeConfig); +deploy_nodes(Versions, Services) -> + NodeConfig = [ rt_config:version_to_config(Version) || Version <- Versions ], + Nodes = ?HARNESS:deploy_nodes(NodeConfig), + lager:info("Waiting for services ~p to start on ~p.", [Services, Nodes]), + [ ok = rt:wait_for_service(Node, Service) || Node <- Nodes, + Service <- Services ], + Nodes. + +deploy_clusters(Settings) -> + ClusterConfigs = [case Setting of + Configs when is_list(Configs) -> + Configs; + NumNodes when is_integer(NumNodes) -> + [{current, default} || _ <- lists:seq(1, NumNodes)]; + {NumNodes, InitialConfig} when is_integer(NumNodes) -> + [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)]; + {NumNodes, Vsn, InitialConfig} when is_integer(NumNodes) -> + [{Vsn, InitialConfig} || _ <- lists:seq(1,NumNodes)] + end || Setting <- Settings], + ?HARNESS:deploy_clusters(ClusterConfigs). + +build_clusters(Settings) -> + Clusters = deploy_clusters(Settings), [begin - NodeIds = rt_cluster_info:get(node_ids, Cluster), - Nodes = [rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds], - lists:foldl(fun maybe_create_bucket_type/2, {Nodes, ClusterIndex}, BucketTypes) - end || {Cluster, ClusterIndex} <- lists:zip(Clusters, lists:seq(1, length(Clusters)))]. - -maybe_create_bucket_type({ClusterIndex, {TypeName, TypeProps}}, - {Nodes, ClusterIndex}) -> - rt_bucket_types:create_and_wait(Nodes, TypeName, TypeProps), - {Nodes, ClusterIndex}; -maybe_create_bucket_type({_ApplicableIndex, {_TypeName, _TypeProps}}, - {Nodes, _ClusterIndex}) -> - %% This bucket type does not apply to this cluster - {Nodes, _ClusterIndex}; -maybe_create_bucket_type({TypeName, TypeProps}, {Nodes, _ClusterIndex}) -> - %% This bucket type applies to all clusters - rt_bucket_types:create_and_wait(Nodes, TypeName, TypeProps), - {Nodes, _ClusterIndex}. - --spec prepare_clusters([list(string())], rt_properties:properties()) -> - [rt_cluster_info:cluster_info()]. -prepare_clusters([ClusterNodes], _Properties) -> - [rt_cluster_info:new([{node_ids, ClusterNodes}])]; -prepare_clusters(ClusterNodesList, Properties) -> - %% If the count of clusters is > 1 the assumption is made that the - %% test is exercising replication and some extra - %% made. This to avoid some noisy and oft-repeated setup - %% boilerplate in every replication test. - NodeMap = rt_properties:get(node_map, Properties), - {Clusters, _, _} = lists:foldl(fun prepare_cluster/2, - {[], 1, NodeMap}, - ClusterNodesList), - lists:reverse(Clusters). - --type prepare_cluster_acc() :: {[rt_cluster_info:cluster_info()], char(), proplists:proplist()}. --spec prepare_cluster([string()], prepare_cluster_acc()) -> prepare_cluster_acc(). -prepare_cluster(NodeIds, {Clusters, Name, NodeMap}) -> - Nodes = [rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds], - repl_util:name_cluster(hd(Nodes), integer_to_list(Name)), - repl_util:wait_until_leader_converge(Nodes), - Leader = repl_util:get_leader(hd(Nodes)), - Cluster = rt_cluster_info:new([{node_ids, NodeIds}, - {leader, Leader}, - {name, Name}]), - {[Cluster | Clusters], Name+1, NodeMap}. - --type clusters() :: [rt_cluster_info:cluster_info()]. --spec form_clusters(rt_properties:properties()) -> clusters(). -form_clusters(Properties) -> - NodeIds = rt_properties:get(node_ids, Properties), - NodeMap = rt_properties:get(node_map, Properties), - ClusterCount = rt_properties:get(cluster_count, Properties), - ClusterWeights = rt_properties:get(cluster_weights, Properties), - MakeCluster = rt_properties:get(make_cluster, Properties), - case divide_nodes(NodeIds, ClusterCount, ClusterWeights) of - {ok, Clusters} -> - maybe_join_clusters(Clusters, NodeMap, MakeCluster), - {ok, Clusters}; - Error -> - Error - end. - --spec divide_nodes([string()], pos_integer(), [float()]) -> - {ok, [list(string())]} | {error, atom()}. -divide_nodes(Nodes, Count, Weights) - when length(Nodes) < Count; - Weights =/= undefined, length(Weights) =/= Count -> - {error, invalid_cluster_properties}; -divide_nodes(Nodes, 1, _) -> - {ok, [Nodes]}; -divide_nodes(Nodes, Count, Weights) -> - case validate_weights(Weights) of - true -> - TotalNodes = length(Nodes), - NodeCounts = node_counts_from_weights(TotalNodes, Count, Weights), - {_, Clusters, _} = lists:foldl(fun take_nodes/2, {1, [], Nodes}, NodeCounts), - {ok, lists:reverse(Clusters)}; - false -> - {error, invalid_cluster_weights} - end. + join_cluster(Nodes), + lager:info("Cluster built: ~p", [Nodes]) + end || Nodes <- Clusters], + Clusters. -take_nodes(NodeCount, {Index, ClusterAcc, Nodes}) -> - {NewClusterNodes, RestNodes} = lists:split(NodeCount, Nodes), - {Index + 1, [NewClusterNodes | ClusterAcc], RestNodes}. - -validate_weights(undefined) -> - true; -validate_weights(Weights) -> - not lists:sum(Weights) > 1.0 . - -node_counts_from_weights(NodeCount, ClusterCount, undefined) -> - %% Split the nodes evenly. A remainder of nodes is handled by - %% distributing one node per cluster until the remainder is - %% exhausted. - NodesPerCluster = NodeCount div ClusterCount, - Remainder = NodeCount rem ClusterCount, - [NodesPerCluster + remainder_to_apply(Remainder, ClusterIndex) || - ClusterIndex <- lists:seq(1, ClusterCount)]; -node_counts_from_weights(NodeCount, ClusterCount, Weights) -> - InitialNodeCounts = [node_count_from_weight(NodeCount, Weight) || Weight <- Weights], - Remainder = NodeCount - lists:sum(InitialNodeCounts), - [ClusterNodeCount + remainder_to_apply(Remainder, ClusterIndex) || - {ClusterIndex, ClusterNodeCount} - <- lists:zip(lists:seq(1, ClusterCount), InitialNodeCounts)]. - -node_count_from_weight(TotalNodes, Weight) -> - RawNodeCount = TotalNodes * Weight, - IntegerPortion = trunc(RawNodeCount), - Remainder = RawNodeCount - IntegerPortion, - case Remainder >= 0.5 of - true -> - IntegerPortion + 1; - false -> - IntegerPortion - end. - -remainder_to_apply(Remainder, Index) when Remainder > Index; - Remainder =:= 0 -> - 0; -remainder_to_apply(_Remainder, _Index) -> - 1. - -maybe_join_clusters(Clusters, NodeMap, true) -> - [join_cluster(ClusterNodes, NodeMap) || ClusterNodes <- Clusters]; -maybe_join_clusters(_Clusters, _NodeMap, false) -> - ok. - -maybe_wait_for_transfers(NodeIds, NodeMap, true) -> +maybe_wait_for_transfers(Nodes, true) -> lager:info("Waiting for transfers"), rt:wait_until_transfers_complete([rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds]); @@ -236,11 +152,62 @@ join_cluster(NodeIds, NodeMap) -> ?assertEqual(ok, rt:wait_until_no_pending_changes(NodeNames)), ok. +%% @doc Safely construct a new cluster and return a list of the deployed nodes +%% @todo Add -spec and update doc to reflect mult-version changes +build_cluster(Versions) when is_list(Versions) -> + build_cluster(length(Versions), Versions, default); +build_cluster(NumNodes) -> + build_cluster(NumNodes, default). + +%% @doc Safely construct a `NumNode' size cluster using +%% `InitialConfig'. Return a list of the deployed nodes. +build_cluster(NumNodes, InitialConfig) -> + build_cluster(NumNodes, [], InitialConfig). + +build_cluster(NumNodes, Versions, InitialConfig) -> + %% Deploy a set of new nodes + Nodes = + case Versions of + [] -> + deploy_nodes(NumNodes, InitialConfig); + _ -> + deploy_nodes(Versions) + end, + + join_cluster(Nodes), + lager:info("Cluster built: ~p", [Nodes]), + Nodes. + +join_cluster(Nodes) -> + %% Ensure each node owns 100% of it's own ring + [?assertEqual([Node], rt:owners_according_to(Node)) || Node <- Nodes], + + %% Join nodes + [Node1|OtherNodes] = Nodes, + case OtherNodes of + [] -> + %% no other nodes, nothing to join/plan/commit + ok; + _ -> + %% ok do a staged join and then commit it, this eliminates the + %% large amount of redundant handoff done in a sequential join + [rt:staged_join(Node, Node1) || Node <- OtherNodes], + rt:plan_and_commit(Node1), + try_nodes_ready(Nodes, 3, 500) + end, + + ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), + + %% Ensure each node owns a portion of the ring + rt:wait_until_nodes_agree_about_ownership(Nodes), + ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes)), + ok. + try_nodes_ready([Node1 | _Nodes], 0, _SleepMs) -> lager:info("Nodes not ready after initial plan/commit, retrying"), - rt_node:plan_and_commit(Node1); + rt:plan_and_commit(Node1); try_nodes_ready(Nodes, N, SleepMs) -> - ReadyNodes = [Node || Node <- Nodes, rt_node:is_ready(Node) =:= true], + ReadyNodes = [Node || Node <- Nodes, rt:is_ready(Node) =:= true], case ReadyNodes of Nodes -> ok; @@ -251,7 +218,7 @@ try_nodes_ready(Nodes, N, SleepMs) -> %% @doc Stop nodes and wipe out their data directories clean_cluster(Nodes) when is_list(Nodes) -> - [rt_node:stop_and_wait(Node) || Node <- Nodes], + [rt:stop_and_wait(Node) || Node <- Nodes], clean_data_dir(Nodes). clean_data_dir(Nodes) -> @@ -260,19 +227,24 @@ clean_data_dir(Nodes) -> clean_data_dir(Nodes, SubDir) when not is_list(Nodes) -> clean_data_dir([Nodes], SubDir); clean_data_dir(Nodes, SubDir) when is_list(Nodes) -> - rt_harness:clean_data_dir(Nodes, SubDir). + ?HARNESS:clean_data_dir(Nodes, SubDir). %% @doc Shutdown every node, this is for after a test run is complete. teardown() -> %% stop all connected nodes, 'cause it'll be faster that %%lager:info("RPC stopping these nodes ~p", [nodes()]), - %%[ rt_node:stop(Node) || Node <- nodes()], + %%[ rt:stop(Node) || Node <- nodes()], %% Then do the more exhaustive harness thing, in case something was up %% but not connected. - rt_harness:teardown(). + ?HARNESS:teardown(). versions() -> - rt_harness:versions(). + ?HARNESS:versions(). + +config() -> + [{riak_core, [{handoff_concurrency, 11}]}, + {riak_search, [{enabled, true}]}, + {riak_pipe, [{worker_limit, 200}]}]. augment_config(Section, Property, Config) -> UpdSectionConfig = update_section(Section, diff --git a/tests/cluster_meta_rmr.erl b/tests/cluster_meta_rmr.erl index fee5d840f..5e39fdc1a 100644 --- a/tests/cluster_meta_rmr.erl +++ b/tests/cluster_meta_rmr.erl @@ -58,7 +58,7 @@ run(NumNodes, NumRounds, StableRounds) -> calc_stuff(AllNodes, NumNodes, NumRounds), exit(Pid, kill), %% start all the down nodes so we can clean them :( - [rt_node:start(Node) || Node <- DownNodes], + [rt:start(Node) || Node <- DownNodes], rt_cluster:clean_cluster(AllNodes). setup_nodes(NumNodes) -> diff --git a/tests/gh_riak_core_154.erl b/tests/gh_riak_core_154.erl index c477b4e53..5b9a50454 100644 --- a/tests/gh_riak_core_154.erl +++ b/tests/gh_riak_core_154.erl @@ -29,7 +29,7 @@ confirm() -> %% Increase handoff concurrency on nodes NewConfig = [{riak_core, [{handoff_concurrency, 1024}]}], Nodes = rt_cluster:build_cluster(2, NewConfig), - ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes)), + ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), [Node1, Node2] = Nodes, lager:info("Write data while ~p is offline", [Node2]), diff --git a/tests/http_bucket_types.erl b/tests/http_bucket_types.erl index 203ad006f..8b2c55672 100644 --- a/tests/http_bucket_types.erl +++ b/tests/http_bucket_types.erl @@ -28,6 +28,11 @@ confirm(Properties) -> Node = hd(Nodes), application:start(ibrowse), + lager:info("Deploy some nodes"), + Nodes = rt_cluster:build_cluster(4, [], [ + {riak_core, [{default_bucket_props, + [{n_val, 2}]}]}]), + Node = hd(Nodes), RMD = rt_properties:get(metadata, Properties), HaveIndexes = case proplists:get_value(backend, RMD) of diff --git a/tests/post_generate_key.erl b/tests/post_generate_key.erl index 9ac63ab8c..de77942b9 100644 --- a/tests/post_generate_key.erl +++ b/tests/post_generate_key.erl @@ -26,7 +26,7 @@ confirm() -> Nodes = rt_cluster:build_cluster(1), - ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes)), + ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), [Base|_] = rt:http_url(Nodes), diff --git a/tests/replication/repl_bucket_types.erl b/tests/replication/repl_bucket_types.erl index f610ba138..e23c8ac54 100644 --- a/tests/replication/repl_bucket_types.erl +++ b/tests/replication/repl_bucket_types.erl @@ -56,6 +56,19 @@ confirm(Properties) -> BucketTypes = {<<"working_type">>, <<"undefined_type">>}, +cleanup({ClusterNodes, _Types, PBA, PBB}, CleanCluster) -> + riakc_pb_socket:stop(PBA), + riakc_pb_socket:stop(PBB), + {_, _, ANodes, BNodes} = ClusterNodes, + case CleanCluster of + true -> + rt_cluster:clean_cluster(ANodes ++ BNodes); + false -> + ok + end. + +%% @doc riak_test entry point +confirm() -> %% Test two clusters of the current version %% SetupData = setup(current), realtime_test(Clusters, NodeMap, BucketTypes), @@ -361,11 +374,19 @@ cluster_conf() -> ]} ]. +<<<<<<< HEAD %% deploy_nodes(NumNodes, current) -> %% rt_cluster:deploy_nodes(NumNodes, cluster_conf()); %% deploy_nodes(_, mixed) -> %% Conf = cluster_conf(), %% rt_cluster:deploy_nodes([{current, Conf}, {previous, Conf}]). +======= +deploy_nodes(NumNodes, current) -> + rt_cluster:deploy_nodes(NumNodes, cluster_conf()); +deploy_nodes(_, mixed) -> + Conf = cluster_conf(), + rt_cluster:deploy_nodes([{current, Conf}, {previous, Conf}]). +>>>>>>> Migrate several more functions into rt_cluster from rt. %% @doc Create two clusters of 1 node each and connect them for replication: %% Cluster "A" -> cluster "B" diff --git a/tests/replication/replication.erl b/tests/replication/replication.erl index bccc73068..7bcf7b8a4 100644 --- a/tests/replication/replication.erl +++ b/tests/replication/replication.erl @@ -17,9 +17,7 @@ confirm() -> ]} ], rt_config:set_advanced_conf(all, Conf), - [ANodes, BNodes] = rt:build_clusters([3, 3]), - rt:wait_for_cluster_service(ANodes, riak_repl), - rt:wait_for_cluster_service(BNodes, riak_repl), + [ANodes, BNodes] = rt_cluster:build_clusters([3, 3]), replication(ANodes, BNodes, false), pass. diff --git a/tests/replication/replication2_console_tests.erl b/tests/replication/replication2_console_tests.erl index 30c64436b..5d5c55fae 100644 --- a/tests/replication/replication2_console_tests.erl +++ b/tests/replication/replication2_console_tests.erl @@ -49,7 +49,7 @@ confirm() -> %% Deploy a node to test against lager:info("Deploy node to test riak-repl command line"), [Node] = rt_cluster:deploy_nodes(1), - ?assertEqual(ok, rt_node:wait_until_nodes_ready([Node])), + ?assertEqual(ok, rt:wait_until_nodes_ready([Node])), rt_intercept:add(Node, {riak_repl_console, [ diff --git a/tests/replication/replication2_pg.erl b/tests/replication/replication2_pg.erl index 01b0a7a57..1198a5099 100644 --- a/tests/replication/replication2_pg.erl +++ b/tests/replication/replication2_pg.erl @@ -65,7 +65,7 @@ setup_repl_clusters(Conf, SSL) -> rt_config:set_advanced_conf(all, Conf), - Nodes = [ANodes, BNodes, CNodes] = rt:build_clusters([2, 2, 2]), + Nodes = [ANodes, BNodes, CNodes] = rt_cluster:build_clusters([2, 2, 2]), rt:wait_for_cluster_service(ANodes, riak_repl), rt:wait_for_cluster_service(BNodes, riak_repl), diff --git a/tests/riak_admin_console_tests.erl b/tests/riak_admin_console_tests.erl index d8928abea..95ad3b421 100644 --- a/tests/riak_admin_console_tests.erl +++ b/tests/riak_admin_console_tests.erl @@ -155,7 +155,7 @@ confirm() -> %% Deploy a node to test against lager:info("Deploy node to test riak command line"), [Node] = rt_cluster:deploy_nodes(1), - ?assertEqual(ok, rt_node:wait_until_nodes_ready([Node])), + ?assertEqual(ok, rt:wait_until_nodes_ready([Node])), rt_intercept:add(Node, {riak_core_console, [ diff --git a/tests/riaknostic_rt.erl b/tests/riaknostic_rt.erl index cbcb72285..9193b3bf2 100644 --- a/tests/riaknostic_rt.erl +++ b/tests/riaknostic_rt.erl @@ -30,7 +30,7 @@ confirm() -> %% Build a small cluster [Node1, _Node2] = rt_cluster:build_cluster(2, []), - ?assertEqual(ok, rt_node:wait_until_nodes_ready([Node1])), + ?assertEqual(ok, rt:wait_until_nodes_ready([Node1])), %% Install riaknostic for Riak versions below 1.3.0 riaknostic_bootstrap(Node1), diff --git a/tests/verify_2i_aae.erl b/tests/verify_2i_aae.erl index 36b051d73..55e2bde5c 100644 --- a/tests/verify_2i_aae.erl +++ b/tests/verify_2i_aae.erl @@ -33,26 +33,13 @@ -define(SCAN_BATCH_SIZE, 100). -define(N_VAL, 3). --export([properties/0, - confirm/1]). - -properties() -> - Config = [{riak_kv, - [{anti_entropy, {off, []}}, - {anti_entropy_build_limit, {100, 500}}, - {anti_entropy_concurrency, 100}, - {anti_entropy_tick, 200}]}] ++ rt_properties:default_config(), - rt_properties:new([{node_count, 1}, - {make_cluster, false}, - {config, Config}]). - --spec confirm(rt_properties:properties()) -> pass | fail. -confirm(Properties) -> - NodeIds = rt_properties:get(node_ids, Properties), - NodeMap = rt_properties:get(node_map, Properties), - Nodes = [rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds], - Node1 = hd(Nodes), - +confirm() -> + [Node1] = rt_cluster:build_cluster(1, + [{riak_kv, + [{anti_entropy, {off, []}}, + {anti_entropy_build_limit, {100, 500}}, + {anti_entropy_concurrency, 100}, + {anti_entropy_tick, 200}]}]), rt_intercept:load_code(Node1), rt_intercept:add(Node1, {riak_object, diff --git a/tests/verify_2i_limit.erl b/tests/verify_2i_limit.erl index 5aa551b35..6c2acdfbf 100644 --- a/tests/verify_2i_limit.erl +++ b/tests/verify_2i_limit.erl @@ -32,7 +32,7 @@ confirm() -> inets:start(), Nodes = rt_cluster:build_cluster(3), - ?assertEqual(ok, (rt_node:wait_until_nodes_ready(Nodes))), + ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), RiakHttp = rt_http:httpc(hd(Nodes)), HttpUrl = rt:http_url(hd(Nodes)), diff --git a/tests/verify_2i_returnterms.erl b/tests/verify_2i_returnterms.erl index 0f38a14cc..7548256e3 100644 --- a/tests/verify_2i_returnterms.erl +++ b/tests/verify_2i_returnterms.erl @@ -31,7 +31,7 @@ confirm() -> inets:start(), Nodes = rt_cluster:build_cluster(3), - ?assertEqual(ok, (rt_node:wait_until_nodes_ready(Nodes))), + ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), RiakHttp = rt:http_url(hd(Nodes)), PBPid = rt_pb:pbc(hd(Nodes)), diff --git a/tests/verify_2i_stream.erl b/tests/verify_2i_stream.erl index 5bd3e6871..8440fc30a 100644 --- a/tests/verify_2i_stream.erl +++ b/tests/verify_2i_stream.erl @@ -30,7 +30,7 @@ confirm() -> inets:start(), Nodes = rt_cluster:build_cluster(3), - ?assertEqual(ok, (rt_node:wait_until_nodes_ready(Nodes))), + ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), RiakHttp = rt:http_url(hd(Nodes)), PBPid = rt_pb:pbc(hd(Nodes)), diff --git a/tests/verify_2i_timeout.erl b/tests/verify_2i_timeout.erl index be0da7e61..6f05fa955 100644 --- a/tests/verify_2i_timeout.erl +++ b/tests/verify_2i_timeout.erl @@ -30,7 +30,7 @@ confirm() -> inets:start(), Config = [{riak_kv, [{secondary_index_timeout, 1}]}], %% ludicrously short, should fail always Nodes = rt_cluster:build_cluster([{current, Config}, {current, Config}, {current, Config}]), - ?assertEqual(ok, (rt_node:wait_until_nodes_ready(Nodes))), + ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), PBPid = rt_pb:pbc(hd(Nodes)), Http = rt:http_url(hd(Nodes)), diff --git a/tests/verify_counter_converge.erl b/tests/verify_counter_converge.erl index f91269e1c..af8c37872 100644 --- a/tests/verify_counter_converge.erl +++ b/tests/verify_counter_converge.erl @@ -34,7 +34,7 @@ confirm() -> Key = <<"a">>, [N1, N2, N3, N4]=Nodes = rt_cluster:build_cluster(4), - [C1, C2, C3, C4]=Clients = [ rt_http:httpc(N) || N <- Nodes ], + [C1, C2, C3, C4]=Clients = [ rt:httpc(N) || N <- Nodes ], set_allow_mult_true(Nodes), diff --git a/tests/verify_cs_bucket.erl b/tests/verify_cs_bucket.erl index 470615a54..4b517b043 100644 --- a/tests/verify_cs_bucket.erl +++ b/tests/verify_cs_bucket.erl @@ -30,7 +30,7 @@ confirm() -> Nodes = rt_cluster:build_cluster(3), - ?assertEqual(ok, (rt_node:wait_until_nodes_ready(Nodes))), + ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), PBPid = rt_pb:pbc(hd(Nodes)), diff --git a/tests/verify_link_walk_urls.erl b/tests/verify_link_walk_urls.erl index 1606f1324..ba174b875 100644 --- a/tests/verify_link_walk_urls.erl +++ b/tests/verify_link_walk_urls.erl @@ -34,7 +34,7 @@ confirm() -> [Node0 | _] = rt_cluster:build_cluster(?NUM_NODES), - Pbc = rt_pb:pbc(Node0), + Pbc = rt:pbc(Node0), lager:info("Inserting linked graph"), %% (deleted) (b/4,v4b) <-> (b/5,v5b) diff --git a/tests/verify_listkeys_eqcfsm.erl b/tests/verify_listkeys_eqcfsm.erl index d7737f338..d8ef9d672 100644 --- a/tests/verify_listkeys_eqcfsm.erl +++ b/tests/verify_listkeys_eqcfsm.erl @@ -179,7 +179,7 @@ log_transition(S) -> %% ==================================================================== setup_cluster(NumNodes) -> Nodes = rt_cluster:build_cluster(NumNodes), - ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes)), + ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), ?assertEqual(ok, rt:wait_until_transfers_complete(Nodes)), Node = hd(Nodes), [begin diff --git a/tests/verify_no_writes_on_read.erl b/tests/verify_no_writes_on_read.erl index 65902cf54..5b48ed048 100644 --- a/tests/verify_no_writes_on_read.erl +++ b/tests/verify_no_writes_on_read.erl @@ -12,7 +12,7 @@ confirm() -> lager:info("Running with backend ~p", [Backend]), ?assertEqual(bitcask, Backend), [Node1 | _Rest] = _Nodes = rt_cluster:build_cluster(?NUM_NODES), - PBC = rt_pb:pbc(Node1), + PBC = rt:pbc(Node1), lager:info("Setting last write wins on bucket"), B = ?BUCKET, ?assertMatch(ok, rpc:call(Node1, riak_core_bucket, set_bucket, [B, [{last_write_wins, true}]])), diff --git a/tests/verify_riak_lager.erl b/tests/verify_riak_lager.erl index 72f5b473e..a11b6a03e 100644 --- a/tests/verify_riak_lager.erl +++ b/tests/verify_riak_lager.erl @@ -30,7 +30,7 @@ confirm() -> lager:info("Staring a node"), Nodes = [Node] = rt_cluster:deploy_nodes(1), - ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes)), + ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), lager:info("Stopping that node"), rt_node:stop(Node), diff --git a/tests/verify_secondary_index_reformat.erl b/tests/verify_secondary_index_reformat.erl index 84b86b349..237b57109 100644 --- a/tests/verify_secondary_index_reformat.erl +++ b/tests/verify_secondary_index_reformat.erl @@ -25,7 +25,7 @@ confirm() -> [Node] = rt_cluster:build_cluster([legacy]), - rt_node:wait_until_nodes_ready([Node]), + rt:wait_until_nodes_ready([Node]), check_fixed_index_statuses(Node, undefined), diff --git a/tests/verify_snmp.erl b/tests/verify_snmp.erl index 646f8381f..ddf910c64 100644 --- a/tests/verify_snmp.erl +++ b/tests/verify_snmp.erl @@ -27,9 +27,9 @@ confirm() -> %% Bring up a small cluster - %% Config = [{riak_snmp, [{polling_interval, 1000}]}], + Config = [{riak_snmp, [{polling_interval, 1000}]}], [Node1] = rt_cluster:deploy_nodes(1), - ?assertEqual(ok, rt_node:wait_until_nodes_ready([Node1])), + ?assertEqual(ok, rt:wait_until_nodes_ready([Node1])), Keys = [{vnodeGets,<<"vnode_gets">>}, {vnodePuts,<<"vnode_puts">>}, diff --git a/tests/verify_tick_change.erl b/tests/verify_tick_change.erl index 755e766d2..e31a956cc 100644 --- a/tests/verify_tick_change.erl +++ b/tests/verify_tick_change.erl @@ -28,7 +28,7 @@ confirm() -> rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), NewConfig = [], Nodes = rt_cluster:build_cluster(ClusterSize, NewConfig), - ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes)), + ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), [Node1|_] = Nodes, Bucket = <<"systest">>, Start = 0, End = 100, diff --git a/tests/verify_vclock_encoding_upgrade.erl b/tests/verify_vclock_encoding_upgrade.erl index cd482ee76..110951099 100644 --- a/tests/verify_vclock_encoding_upgrade.erl +++ b/tests/verify_vclock_encoding_upgrade.erl @@ -24,8 +24,8 @@ confirm() -> lager:info("Deploying previous cluster"), [Prev, Current] = rt_cluster:build_cluster([previous, current]), - PrevClient = rt_pb:pbc(Prev), - CurrentClient = rt_pb:pbc(Current), + PrevClient = rt:pbc(Prev), + CurrentClient = rt:pbc(Current), K = <<"key">>, B = <<"bucket">>, V = <<"value">>, From 13387de5ce88a68391b2f9aa324f5e5e97b85366 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Wed, 21 Jan 2015 12:40:21 -0500 Subject: [PATCH 052/157] WIP: Support execution of old and new style tests * Modifies riak_test_escript to start lager as early as possible * Moves the new rt API to the rt2 module -- allowing old tests to use the old API until ported. Before merge, rt.erl will be modified to largely proxy the the new API structure. * Adds detection of test type (old or new) to riak_test_executor to determine whether or not not allocate nodes automatically * Fixes compilation errors in some test cases --- tests/verify_snmp.erl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/verify_snmp.erl b/tests/verify_snmp.erl index ddf910c64..aba9b8734 100644 --- a/tests/verify_snmp.erl +++ b/tests/verify_snmp.erl @@ -27,7 +27,7 @@ confirm() -> %% Bring up a small cluster - Config = [{riak_snmp, [{polling_interval, 1000}]}], + %% Config = [{riak_snmp, [{polling_interval, 1000}]}], [Node1] = rt_cluster:deploy_nodes(1), ?assertEqual(ok, rt:wait_until_nodes_ready([Node1])), From 11c7aac9e6fbd78e879c1210278c50d69a5f0c4a Mon Sep 17 00:00:00 2001 From: John Burwell Date: Fri, 13 Feb 2015 15:15:03 -0500 Subject: [PATCH 053/157] Initial run of an old style test * Modifies the rt module work in terms of the new r_t infrastructure * Fixes an errant guard in verify_riak_stats * Adds rt_harness:available_resources and rtdev:available_resources to split harness setup from the determinatin of available resources for allocation in the rt module * Adds separate execution paths for old and new style tests to the riak_test_executor and riak_test_runner modules * Modifies reset-current-env.sh work with the changed devrel structure --- bin/reset-current-env.sh | 5 +- src/node_manager.erl | 5 ++ src/riak_test_executor.erl | 33 +++++++--- src/riak_test_runner.erl | 92 ++++++++++++++++---------- src/rt.erl | 81 ++++++++++++++++++++++- src/rt_harness.erl | 4 ++ src/rt_harness_util.erl | 2 + src/rtdev.erl | 123 +++++++++++++++++++---------------- tests/verify_dt_converge.erl | 3 +- tests/verify_riak_stats.erl | 3 +- 10 files changed, 243 insertions(+), 108 deletions(-) diff --git a/bin/reset-current-env.sh b/bin/reset-current-env.sh index 285df2737..e19d3027c 100755 --- a/bin/reset-current-env.sh +++ b/bin/reset-current-env.sh @@ -35,8 +35,7 @@ while getopts chn:v: opt; do ;; v) VERSION=$OPTARG ;; - n) echo "parsing num nodes" - NUM_NODES=$OPTARG + n) NUM_NODES=$OPTARG ;; h) usage exit 0 @@ -46,7 +45,7 @@ done shift $(($OPTIND-1)) -RIAK_HOME=$RT_HOME/riak-$VERSION +RIAK_HOME=$RT_HOME/$VERSION if ! [[ -d $RIAK_HOME || -h $RIAK_HOME ]]; then echo "Riak home $RIAK_HOME does not exist." diff --git a/src/node_manager.erl b/src/node_manager.erl index 92244f5f8..d4d2bc2ee 100644 --- a/src/node_manager.erl +++ b/src/node_manager.erl @@ -36,6 +36,7 @@ start_link(Nodes, NodeMap, VersionMap) -> -spec reserve_nodes(pos_integer(), [string()], function()) -> ok. reserve_nodes(NodeCount, Versions, NotifyFun) -> + lager:debug("reserve_nodes(~p, ~p, ~p)", [NodeCount, Versions, NotifyFun]), gen_server:cast(?MODULE, {reserve_nodes, NodeCount, Versions, NotifyFun}). -spec deploy_nodes([string()], string(), term(), list(atom()), function()) -> ok. @@ -78,6 +79,7 @@ handle_call(stop, _From, State) -> {stop, normal, ok, State}. handle_cast({reserve_nodes, Count, Versions, NotifyFun}, State) -> + lager:debug("Handling cast to reserve ~p nodes with versions ~p.", [Count, Versions]), {Result, UpdState} = reserve(Count, Versions, State), NotifyFun(Result), @@ -148,6 +150,9 @@ wait_for_cleaner(Pid, true) -> wait_for_cleaner(_, false) -> ok. +reserve(Count, _Versions, State) when Count == 0 -> + lager:debug("Reserving no nodes ..."), + {{nodes, [], State#state.node_map}, State}; reserve(Count, _Versions, State=#state{nodes_available=NodesAvailable}) when Count > length(NodesAvailable) -> {not_enough_nodes, State}; diff --git a/src/riak_test_executor.erl b/src/riak_test_executor.erl index 0690107a2..a612b8a95 100644 --- a/src/riak_test_executor.erl +++ b/src/riak_test_executor.erl @@ -104,16 +104,11 @@ gather_properties(_Event, _State) -> request_nodes(timeout, State) -> #state{pending_tests=[NextTest | _], test_properties=PropertiesList} = State, - lager:debug("Requesting nodes using properties ~p", [PropertiesList]), %% Find the properties for the next pending test {NextTest, TestProps} = lists:keyfind(NextTest, 1, PropertiesList), - %% Send async request to node manager - VersionsToTest = versions_to_test(TestProps), - NodeCount = rt_properties:get(node_count, TestProps), - lager:notice("Requesting ~p nodes for the next test, ~p", [NodeCount, NextTest]), - node_manager:reserve_nodes(NodeCount, - VersionsToTest, - reservation_notify_fun()), + + ok = maybe_reserve_nodes(NextTest, TestProps), + {next_state, launch_test, State}; request_nodes({test_complete, Test, Pid, Results, Duration}, State) -> #state{pending_tests=Pending, @@ -155,6 +150,7 @@ launch_test({nodes, Nodes, NodeMap}, State) -> {NextTest, TestProps} = lists:keyfind(NextTest, 1, PropertiesList), UpdTestProps = rt_properties:set([{node_map, NodeMap}, {node_ids, Nodes}], TestProps), + lager:debug("Spawning test runner to execute ~p", [NextTest]), Pid = spawn_link(riak_test_runner, start, [NextTest, Backend, UpdTestProps]), UpdState = State#state{pending_tests=RestPending, runner_pids=[Pid | Pids], @@ -172,7 +168,26 @@ launch_test({test_complete, Test, Pid, Results, Duration}, State) -> pending_tests=Pending++Waiting, waiting_tests=[]}, {next_state, launch_test, UpdState}; -launch_test(_Event, _State) -> +launch_test(Event, State) -> + lager:error("Unknown event ~p with state ~p.", [Event, State]), + ok. + +maybe_reserve_nodes(NextTest, TestProps) -> + VersionsToTest = versions_to_test(TestProps), + maybe_reserve_nodes(erlang:function_exported(NextTest, confirm, 1), + NextTest, VersionsToTest, TestProps). + +maybe_reserve_nodes(true, NextTest, VersionsToTest, TestProps) -> + NodeCount = rt_properties:get(node_count, TestProps), + + %% Send async request to node manager + lager:notice("Requesting ~p nodes for the next test, ~p", [NodeCount, NextTest]), + node_manager:reserve_nodes(NodeCount, + VersionsToTest, + reservation_notify_fun()); +maybe_reserve_nodes(false, NextTest, VersionsToTest, _TestProps) -> + lager:warning("~p is an old style test that requires conversion.", [NextTest]), + node_manager:reserve_nodes(0, VersionsToTest, reservation_notify_fun()), ok. wait_for_completion({test_complete, Test, Pid, Results, Duration}, State) -> diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index e55a33f5a..e9c6ba729 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -1,4 +1,4 @@ -%% ------------------------------------------------------------------- +% ------------------------------------------------------------------- %% %% Copyright (c) 2013 Basho Technologies, Inc. %% @@ -104,7 +104,8 @@ init([TestModule, Backend, Properties]) -> TestTimeout = rt_config:get(test_timeout, rt_config:get(rt_max_wait_time)), SetupModFun = function_name(setup, TestModule, 1, rt_cluster), {ConfirmMod, _} = ConfirmModFun = function_name(confirm, TestModule), - TestType = case erlang:function_exported(ConfirmMod, ConfirmModFun, 1) of + lager:debug("Confirm function -- ~p:~p", [ConfirmMod, ConfirmModFun]), + TestType = case erlang:function_exported(TestModule, confirm, 1) of true -> new; false -> old end, @@ -154,9 +155,7 @@ setup(timeout, State=#state{prereq_check=false}) -> notify_executor({fail, prereq_check_failed}, State), cleanup(State), {stop, normal, State}; -setup(timeout, State=#state{test_type=TestType, - test_module=TestModule, - backend=Backend, +setup(timeout, State=#state{backend=Backend, properties=Properties}) -> NewGroupLeader = riak_test_group_leader:new_group_leader(self()), group_leader(NewGroupLeader, self()), @@ -169,17 +168,12 @@ setup(timeout, State=#state{test_type=TestType, {StartVersion, OtherVersions} = test_versions(Properties), Config = rt_backend:set(Backend, rt_properties:get(config, Properties)), - case TestType of - new -> - node_manager:deploy_nodes(NodeIds, - StartVersion, - Config, - Services, - notify_fun(self())), - lager:info("Waiting for deploy nodes response at ~p", [self()]); - old -> - lager:warn("Test ~p has not been ported to the new framework.", [TestModule]) - end, + node_manager:deploy_nodes(NodeIds, + StartVersion, + Config, + Services, + notify_fun(self())), + lager:info("Waiting for deploy nodes response at ~p", [self()]), %% Set the initial value for `current_version' in the properties record {ok, UpdProperties} = @@ -194,8 +188,9 @@ setup(_Event, _State) -> execute({nodes_deployed, _}, State) -> #state{test_module=TestModule, + test_type=TestType, properties=Properties, - setup_modfun={SetupMod, SetupFun}, + setup_modfun=SetupModFun, confirm_modfun=ConfirmModFun, test_timeout=TestTimeout} = State, lager:notice("Running ~s", [TestModule]), @@ -205,23 +200,33 @@ execute({nodes_deployed, _}, State) -> %% Perform test setup which includes clustering of the nodes if %% required by the test properties. The cluster information is placed %% into the properties record and returned by the `setup' function. - UpdState = - case SetupMod:SetupFun(Properties) of - {ok, UpdProperties} -> - Pid = spawn_link(test_fun(UpdProperties, - ConfirmModFun, - self())), - State#state{execution_pid=Pid, - properties=UpdProperties, - start_time=StartTime}; - _ -> - ?MODULE:send_event(self(), test_result({fail, test_setup_failed})), - State#state{start_time=StartTime} - end, + SetupResult = maybe_setup_test(TestModule, TestType, SetupModFun, Properties), + UpdState = maybe_execute_test(SetupResult, TestModule, TestType, ConfirmModFun, StartTime, State), + {next_state, wait_for_completion, UpdState, TestTimeout}; execute(_Event, _State) -> {next_state, execute, _State}. +maybe_setup_test(_TestModule, old, _SetupModFun, Properties) -> + {ok, Properties}; +maybe_setup_test(TestModule, new, {SetupMod, SetupFun}, Properties) -> + lager:debug("Setting up test ~p using ~p:~p", [TestModule, SetupMod, SetupFun]), + SetupMod:SetupFun(Properties). + +maybe_execute_test({ok, Properties}, _TestModule, TestType, ConfirmModFun, StartTime, State) -> + Pid = spawn_link(test_fun(TestType, + Properties, + ConfirmModFun, + self())), + State#state{execution_pid=Pid, + properties=Properties, + start_time=StartTime}; +maybe_execute_test(Error, TestModule, _TestType, _ConfirmModFun, StartTime, State) -> + lager:error("Setup of test ~p failed due to ~p", [TestModule, Error]), + ?MODULE:send_event(self(), test_result({fail, test_setup_failed})), + State#state{start_time=StartTime}. + + wait_for_completion(timeout, State) -> %% Test timed out UpdState = State#state{end_time=os:timestamp()}, @@ -258,6 +263,7 @@ wait_for_completion(_Msg, _State) -> wait_for_upgrade(nodes_upgraded, State) -> #state{properties=Properties, + test_type=TestType, confirm_modfun=ConfirmModFun, current_version=CurrentVersion, test_timeout=TestTimeout} = State, @@ -268,7 +274,8 @@ wait_for_upgrade(nodes_upgraded, State) -> %% TODO: Maybe wait for transfers. Probably should be %% a call to an exported function in `rt_cluster' - Pid = spawn_link(test_fun(UpdProperties, + Pid = spawn_link(test_fun(TestType, + UpdProperties, ConfirmModFun, self())), UpdState = State#state{execution_pid=Pid, @@ -295,18 +302,31 @@ wait_for_upgrade(_Event, _From, _State) -> %%% Internal functions %%%=================================================================== --spec test_fun(rt_properties:properties(), {atom(), atom()}, pid()) -> - function(). -test_fun(Properties, {ConfirmMod, ConfirmFun}, NotifyPid) -> +-spec test_fun(test_type(), rt_properties:properties(), {atom(), atom()}, + pid()) -> function(). +test_fun(TestType, Properties, {ConfirmMod, ConfirmFun}, NotifyPid) -> + test_fun(TestType, Properties, ConfirmMod, ConfirmFun, NotifyPid). + +-spec test_fun(test_type(), rt_properties:properties(), atom(), atom(), + pid()) -> function(). +test_fun(new, Properties, ConfirmMod, ConfirmFun, NotifyPid) -> + test_fun(fun() -> ConfirmMod:ConfirmFun(Properties) end, NotifyPid); +test_fun(old, _Properties, ConfirmMod, ConfirmFun, NotifyPid) -> + lager:debug("Building test fun for ~p:~p/0 (defined: ~p)", [ConfirmMod, ConfirmFun, erlang:function_exported(ConfirmMod, ConfirmFun, 0)]), + test_fun(fun() -> ConfirmMod:ConfirmFun() end, NotifyPid). + +-spec test_fun(function(), pid()) -> function(). +test_fun(ConfirmFun, NotifyPid) -> fun() -> %% Exceptions and their handling sucks, but eunit throws %% errors `erlang:error' so here we are - try ConfirmMod:ConfirmFun(Properties) of + try ConfirmFun() of TestResult -> ?MODULE:send_event(NotifyPid, test_result(TestResult)) catch Error:Reason -> - lager:error("Error: ~p Reason: ~p", [Error, Reason]), + lager:error("Failed to execute confirm function ~p due to ~p with reason ~p (trace: ~p)", + [ConfirmFun, Error, Reason, erlang:get_stacktrace()]), TestResult = format_eunit_error(Reason), ?MODULE:send_event(NotifyPid, test_result(TestResult)) end diff --git a/src/rt.erl b/src/rt.erl index a199d5843..590277c65 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -24,6 +24,7 @@ %% Please extend this module with new functions that prove useful between %% multiple independent tests. -module(rt). +-deprecated(module). -include("rt.hrl"). -include_lib("eunit/include/eunit.hrl"). @@ -250,6 +251,84 @@ get_https_conn_info(Node) -> undefined end. +%% @doc Deploy a set of freshly installed Riak nodes, returning a list of the +%% nodes deployed. +%% @todo Re-add -spec after adding multi-version support +deploy_nodes(Versions) when is_list(Versions) -> + deploy_nodes(Versions, [riak_kv]); +deploy_nodes(NumNodes) when is_integer(NumNodes) -> + [NodeIds, NodeMap, _] = allocate_nodes(NumNodes), + deploy_nodes(NodeIds, NodeMap, head, rt_properties:default_config(), [riak_kv]). + +%% @doc Deploy a set of freshly installed Riak nodes with the given +%% `InitialConfig', returning a list of the nodes deployed. +-spec deploy_nodes(NumNodes :: integer(), any()) -> [node()]. +deploy_nodes(NumNodes, InitialConfig) when is_integer(NumNodes) -> + deploy_nodes(NumNodes, InitialConfig, [riak_kv]); +deploy_nodes(Versions, Services) -> + NodeConfig = [ version_to_config(Version) || Version <- Versions ], + lager:debug("Starting nodes config ~p using versions ~p", [NodeConfig, Versions]), + + + + Nodes = ?HARNESS:deploy_nodes(NodeConfig), + lager:info("Waiting for services ~p to start on ~p.", [Services, Nodes]), + [ ok = wait_for_service(Node, Service) || Node <- Nodes, + Service <- Services ], + Nodes. +deploy_nodes(NumNodes, InitialConfig, Services) when is_integer(NumNodes) -> + NodeConfig = [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)], + deploy_nodes(NodeConfig, Services). + +deploy_nodes(NodeIds, NodeMap, Version, Config, Services) -> + _ = rt_harness_util:deploy_nodes(NodeIds, NodeMap, Version, Config, Services), + lists:foldl(fun({_, NodeName}, Nodes) -> [NodeName|Nodes] end, + [], + NodeMap). + +allocate_nodes(NumNodes) -> + [AvailableNodeIds, AvailableNodeMap, VersionMap] = rt_harness:available_resources(), + lager:debug("Availabe node ids ~p, node map ~p, and version map ~p.", + [AvailableNodeIds, AvailableNodeMap, VersionMap]), + + AllocatedNodeIds = lists:sublist(AvailableNodeIds, NumNodes), + lager:debug("Allocated node ids ~p with version map ~p", [AllocatedNodeIds, VersionMap]), + + + AllocatedNodeMap = lists:foldl( + fun(NodeId, NodeMap) -> + NodeName = proplists:get_value(NodeId, AvailableNodeMap), + [{NodeId, NodeName}|NodeMap] + end, + [], + AllocatedNodeIds), + lager:debug("Allocated node map ~p", [AllocatedNodeMap]), + [AllocatedNodeIds, AllocatedNodeMap, VersionMap]. + +version_to_config(Config) when is_tuple(Config)-> Config; +version_to_config(Version) -> {Version, default}. + +deploy_clusters(Settings) -> + ClusterConfigs = [case Setting of + Configs when is_list(Configs) -> + Configs; + NumNodes when is_integer(NumNodes) -> + [{current, default} || _ <- lists:seq(1, NumNodes)]; + {NumNodes, InitialConfig} when is_integer(NumNodes) -> + [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)]; + {NumNodes, Vsn, InitialConfig} when is_integer(NumNodes) -> + [{Vsn, InitialConfig} || _ <- lists:seq(1,NumNodes)] + end || Setting <- Settings], + ?HARNESS:deploy_clusters(ClusterConfigs). + +build_clusters(Settings) -> + Clusters = deploy_clusters(Settings), + [begin + join_cluster(Nodes), + lager:info("Cluster built: ~p", [Nodes]) + end || Nodes <- Clusters], + Clusters. + %% @doc Start the specified Riak node start(Node) -> ?HARNESS:start(Node). @@ -1758,7 +1837,7 @@ pmap(F, L) -> %% @private setup_harness(Test, Args) -> - ?HARNESS:setup_harness(Test, Args). + rt_harness:setup_harness(Test, Args). %% @doc Downloads any extant log files from the harness's running %% nodes. diff --git a/src/rt_harness.erl b/src/rt_harness.erl index 88822445d..f1b7f33d6 100644 --- a/src/rt_harness.erl +++ b/src/rt_harness.erl @@ -30,6 +30,7 @@ deploy_clusters/1, clean_data_dir/3, deploy_nodes/1, + available_resources/0, spawn_cmd/1, spawn_cmd/2, cmd/1, @@ -81,6 +82,9 @@ cmd(Cmd, Opts) -> deploy_nodes(NodeConfig) -> ?HARNESS_MODULE:deploy_nodes(NodeConfig). +available_resources() -> + ?HARNESS_MODULE:available_resources(). + setup() -> ?HARNESS_MODULE:setup_harness(). diff --git a/src/rt_harness_util.erl b/src/rt_harness_util.erl index 71ee8f016..bdb4a33c8 100644 --- a/src/rt_harness_util.erl +++ b/src/rt_harness_util.erl @@ -82,6 +82,8 @@ console(Node, Expected) -> %% end. %% deploy_nodes(NodeConfig) -> +deploy_nodes(NodeIds, _NodeMap, _Version, _Config, _Services) when NodeIds =:= [] -> + NodeIds; deploy_nodes(NodeIds, NodeMap, Version, Config, Services) -> %% create snmp dirs, for EE create_dirs(Version, NodeIds), diff --git a/src/rtdev.erl b/src/rtdev.erl index 201bca6d5..7355a9e10 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -178,11 +178,18 @@ so_fresh_so_clean(VersionMap) -> end, VersionMap), ok. -setup_harness() -> - %% Get node names and populate node map +available_resources() -> VersionMap = [{Version, harness_node_ids(Version)} || Version <- versions()], NodeIds = harness_node_ids(rt_config:get(default_version, "head")), + lager:debug("Available Node IDs: ~p", [NodeIds]), NodeMap = lists:zip(NodeIds, harness_nodes(NodeIds)), + lager:debug("Available Node Map: ~p", [NodeMap]), + [NodeIds, NodeMap, VersionMap]. + + +setup_harness() -> + %% Get node names and populate node map + [NodeIds, NodeMap, VersionMap] = available_resources(), so_fresh_so_clean(VersionMap), rm_dir(filename:join(?SCRATCH_DIR, "gc")), rt_harness_util:setup_harness(VersionMap, NodeIds, NodeMap). @@ -319,9 +326,10 @@ all_the_app_configs(DevPath) -> AppConfigs end. -%% update_app_config(all, Config) -> -%% lager:info("rtdev:update_app_config(all, ~p)", [Config]), -%% [ update_app_config(DevPath, Config) || DevPath <- devpaths()]; +update_app_config(all, Config) -> + lager:info("rtdev:update_app_config(all, ~p)", [Config]), + [ update_app_config(DevPath, Config) || DevPath <- devpaths()]. + update_app_config(Node, Version, Config) -> VersionPath = filename:join(?PATH, Version), FileFormatString = "~s/~s/etc/~s.config", @@ -510,57 +518,58 @@ configure_nodes(Nodes, Configs) -> end, lists:zip(Nodes, Configs)). -%% deploy_nodes(NodeConfig) -> -%% Path = relpath(root), -%% lager:info("Riak path: ~p", [Path]), -%% NumNodes = length(NodeConfig), -%% %% TODO: The starting index should not be fixed to 1 -%% NodesN = lists:seq(1, NumNodes), -%% Nodes = [?DEV(N) || N <- NodesN], -%% NodeMap = orddict:from_list(lists:zip(Nodes, NodesN)), -%% {Versions, Configs} = lists:unzip(NodeConfig), -%% VersionMap = lists:zip(NodesN, Versions), - -%% %% Check that you have the right versions available -%% [ check_node(Version) || Version <- VersionMap ], -%% rt_config:set(rt_nodes, NodeMap), -%% rt_config:set(rt_versions, VersionMap), - -%% create_dirs(Nodes), - -%% %% Set initial config -%% add_default_node_config(Nodes), -%% rt:pmap(fun({_, default}) -> -%% ok; -%% ({Node, {cuttlefish, Config}}) -> -%% set_conf(Node, Config); -%% ({Node, Config}) -> -%% update_app_config(Node, Config) -%% end, -%% lists:zip(Nodes, Configs)), - -%% %% create snmp dirs, for EE -%% create_dirs(Nodes), - -%% %% Start nodes -%% %%[run_riak(N, relpath(node_version(N)), "start") || N <- Nodes], -%% rt:pmap(fun(N) -> run_riak(N, relpath(node_version(N)), "start") end, NodesN), - -%% %% Ensure nodes started -%% [ok = rt:wait_until_pingable(N) || N <- Nodes], - -%% %% %% Enable debug logging -%% %% [rpc:call(N, lager, set_loglevel, [lager_console_backend, debug]) || N <- Nodes], - -%% %% We have to make sure that riak_core_ring_manager is running before we can go on. -%% [ok = rt:wait_until_registered(N, riak_core_ring_manager) || N <- Nodes], - -%% %% Ensure nodes are singleton clusters -%% [ok = rt_ring:check_singleton_node(?DEV(N)) || {N, Version} <- VersionMap, -%% Version /= "0.14.2"], - -%% lager:info("Deployed nodes: ~p", [Nodes]), -%% Nodes. +deploy_nodes(NodeConfig) -> + Path = relpath(root), + lager:info("Riak path: ~p", [Path]), + NumNodes = length(NodeConfig), + %% TODO: The starting index should not be fixed to 1 + NodesN = lists:seq(1, NumNodes), + Nodes = [?DEV(N) || N <- NodesN], + NodeMap = orddict:from_list(lists:zip(Nodes, NodesN)), + {Versions, Configs} = lists:unzip(NodeConfig), + VersionMap = lists:zip(NodesN, Versions), + + %% TODO The new node deployment doesn't appear to perform this check ... -jsb + %% Check that you have the right versions available + %%[ check_node(Version) || Version <- VersionMap ], + rt_config:set(rt_nodes, NodeMap), + rt_config:set(rt_versions, VersionMap), + + create_dirs(Nodes), + + %% Set initial config + add_default_node_config(Nodes), + rt:pmap(fun({_, default}) -> + ok; + ({Node, {cuttlefish, Config}}) -> + set_conf(Node, Config); + ({Node, Config}) -> + update_app_config(Node, Config) + end, + lists:zip(Nodes, Configs)), + + %% create snmp dirs, for EE + create_dirs(Nodes), + + %% Start nodes + %%[run_riak(N, relpath(node_version(N)), "start") || N <- Nodes], + rt:pmap(fun(N) -> run_riak(N, relpath(node_version(N)), "start") end, NodesN), + + %% Ensure nodes started + [ok = rt:wait_until_pingable(N) || N <- Nodes], + + %% %% Enable debug logging + %% [rpc:call(N, lager, set_loglevel, [lager_console_backend, debug]) || N <- Nodes], + + %% We have to make sure that riak_core_ring_manager is running before we can go on. + [ok = rt:wait_until_registered(N, riak_core_ring_manager) || N <- Nodes], + + %% Ensure nodes are singleton clusters + [ok = rt_ring:check_singleton_node(?DEV(N)) || {N, Version} <- VersionMap, + Version /= "0.14.2"], + + lager:info("Deployed nodes: ~p", [Nodes]), + Nodes. gen_stop_fun(Path, Timeout) -> fun(Node) -> @@ -832,7 +841,7 @@ get_cmd_result(Port, Acc) -> end. check_node({_N, Version}) -> - case proplists:is_defined(Version, rt_config:get(rtdev_path)) of + case proplists:is_defined(Version, rt_config:get(root_path)) of true -> ok; _ -> lager:error("You don't have Riak ~s installed or configured", [Version]), diff --git a/tests/verify_dt_converge.erl b/tests/verify_dt_converge.erl index 5177d42e7..fc86c5efd 100644 --- a/tests/verify_dt_converge.erl +++ b/tests/verify_dt_converge.erl @@ -25,7 +25,8 @@ -module(verify_dt_converge). -behavior(riak_test). -compile([export_all]). --export([confirm/0]). +-export([confirm/0, + create_bucket_types/2]). -include_lib("eunit/include/eunit.hrl"). diff --git a/tests/verify_riak_stats.erl b/tests/verify_riak_stats.erl index 247212780..9f2a409f8 100644 --- a/tests/verify_riak_stats.erl +++ b/tests/verify_riak_stats.erl @@ -35,7 +35,7 @@ %% You should have curl installed locally to do this. confirm() -> - Nodes = rt_cluster:deploy_nodes(1), + Nodes = rt:deploy_nodes(1), [Node1] = Nodes, verify_dt_converge:create_bucket_types(Nodes, ?TYPES), ?assertEqual(ok, rt_node:wait_until_nodes_ready([Node1])), @@ -205,6 +205,7 @@ verify_stats_keys_complete(Node, Stats) -> MissingStatsKeys = diff_lists(ActualKeys, ExpectedKeys), AdditionalStatsKeys = diff_lists(ExpectedKeys, ActualKeys), maybe_log_stats_keys(MissingStatsKeys, "missing stats keys"), + lager:debug("Additional stats ~p", [AdditionalStatsKeys]), maybe_log_stats_keys(AdditionalStatsKeys, "additional stats"), ?assertEqual({[],[]}, {MissingStatsKeys, AdditionalStatsKeys}), ok. From ba8b606bf0d2f8de8d95432e5f47cb5b7ed21603 Mon Sep 17 00:00:00 2001 From: Alex Berghage Date: Tue, 10 Feb 2015 20:44:09 -0500 Subject: [PATCH 054/157] Reduced repetition in tests/overload predicates --- tests/overload.erl | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/tests/overload.erl b/tests/overload.erl index 7cc2da018..c1d27cbc9 100644 --- a/tests/overload.erl +++ b/tests/overload.erl @@ -115,7 +115,7 @@ test_vnode_protection(Nodes, BKV, ConsistentType) -> Config = [{riak_core, [{vnode_overload_threshold, ?THRESHOLD}, {vnode_check_interval, 1}]}], rt:pmap(fun(Node) -> - rt_config:update_app_config(Node, Config) + rt:update_app_config(Node, Config) end, Nodes), ProcFun = build_predicate_lt(test_vnode_protection, (?NUM_REQUESTS+1), "ProcFun", "Procs"), QueueFun = build_predicate_lt(test_vnode_protection, (?NUM_REQUESTS), "QueueFun", "QueueSize"), @@ -147,7 +147,7 @@ test_fsm_protection(Nodes, BKV, ConsistentType) -> lager:info("Setting FSM limit to ~b", [?THRESHOLD]), Config = [{riak_kv, [{fsm_limit, ?THRESHOLD}]}], rt:pmap(fun(Node) -> - rt_config:update_app_config(Node, Config) + rt:update_app_config(Node, Config) end, Nodes), ProcFun = build_predicate_lt(test_fsm_protection, (?NUM_REQUESTS), "ProcFun", "Procs"), @@ -268,8 +268,10 @@ node_overload_check(Pid) -> end. list_keys(Node) -> - Pid = rt_pb:pbc(Node), - riakc_pb_socket:list_keys(Pid, ?BUCKET, 30000). + Pid = rt:pbc(Node, [{auto_reconnect, true}, {queue_if_disconnected, true}]), + Res = riakc_pb_socket:list_keys(Pid, {<<"normal_type">>, ?BUCKET}, infinity), + riakc_pb_socket:stop(Pid), + Res. list_buckets(Node) -> {ok, C} = riak:client_connect(Node), @@ -503,4 +505,4 @@ build_predicate_lt(Test, Metric, Label, ValueLabel) -> lager:info("in test ~p ~p, ~p:~p, expected overload, Metric:<~p", [Test, Label, ValueLabel, X, Metric]), X < Metric - end. + end. \ No newline at end of file From fe36e7eb61fa068ec09f2a65dcd4de128094c02a Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Tue, 29 Jul 2014 18:50:48 -0400 Subject: [PATCH 055/157] Move configuration functions to rt_config. - functions include set_conf/2, set_advanced_conf/2, and update_app_config/2. --- tests/overload.erl | 4 ++-- tests/replication/replication.erl | 4 +++- tests/replication/replication2_pg.erl | 2 +- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/tests/overload.erl b/tests/overload.erl index c1d27cbc9..44d24ee79 100644 --- a/tests/overload.erl +++ b/tests/overload.erl @@ -115,7 +115,7 @@ test_vnode_protection(Nodes, BKV, ConsistentType) -> Config = [{riak_core, [{vnode_overload_threshold, ?THRESHOLD}, {vnode_check_interval, 1}]}], rt:pmap(fun(Node) -> - rt:update_app_config(Node, Config) + rt_config:update_app_config(Node, Config) end, Nodes), ProcFun = build_predicate_lt(test_vnode_protection, (?NUM_REQUESTS+1), "ProcFun", "Procs"), QueueFun = build_predicate_lt(test_vnode_protection, (?NUM_REQUESTS), "QueueFun", "QueueSize"), @@ -147,7 +147,7 @@ test_fsm_protection(Nodes, BKV, ConsistentType) -> lager:info("Setting FSM limit to ~b", [?THRESHOLD]), Config = [{riak_kv, [{fsm_limit, ?THRESHOLD}]}], rt:pmap(fun(Node) -> - rt:update_app_config(Node, Config) + rt_config:update_app_config(Node, Config) end, Nodes), ProcFun = build_predicate_lt(test_fsm_protection, (?NUM_REQUESTS), "ProcFun", "Procs"), diff --git a/tests/replication/replication.erl b/tests/replication/replication.erl index 7bcf7b8a4..bccc73068 100644 --- a/tests/replication/replication.erl +++ b/tests/replication/replication.erl @@ -17,7 +17,9 @@ confirm() -> ]} ], rt_config:set_advanced_conf(all, Conf), - [ANodes, BNodes] = rt_cluster:build_clusters([3, 3]), + [ANodes, BNodes] = rt:build_clusters([3, 3]), + rt:wait_for_cluster_service(ANodes, riak_repl), + rt:wait_for_cluster_service(BNodes, riak_repl), replication(ANodes, BNodes, false), pass. diff --git a/tests/replication/replication2_pg.erl b/tests/replication/replication2_pg.erl index 1198a5099..01b0a7a57 100644 --- a/tests/replication/replication2_pg.erl +++ b/tests/replication/replication2_pg.erl @@ -65,7 +65,7 @@ setup_repl_clusters(Conf, SSL) -> rt_config:set_advanced_conf(all, Conf), - Nodes = [ANodes, BNodes, CNodes] = rt_cluster:build_clusters([2, 2, 2]), + Nodes = [ANodes, BNodes, CNodes] = rt:build_clusters([2, 2, 2]), rt:wait_for_cluster_service(ANodes, riak_repl), rt:wait_for_cluster_service(BNodes, riak_repl), From 2d494b208c3225901949fe4b7cbebfa2c79711ce Mon Sep 17 00:00:00 2001 From: Jon Anderson Date: Wed, 30 Jul 2014 10:10:30 -0400 Subject: [PATCH 056/157] Migrate several more functions into rt_cluster from rt. --- src/rt.erl | 10 + tests/replication/repl_bucket_types.erl | 340 +++++++++++------------- tests/replication/replication.erl | 4 +- tests/replication/replication2_pg.erl | 2 +- tests/verify_listkeys.erl | 149 +++++------ tests/verify_snmp.erl | 2 +- 6 files changed, 231 insertions(+), 276 deletions(-) diff --git a/src/rt.erl b/src/rt.erl index 590277c65..ed37d5e3b 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -1022,6 +1022,7 @@ status_of_according_to(Member, Node) -> %% @doc Return a list of nodes that own partitions according to the ring %% retrieved from the specified node. claimant_according_to(Node) -> +<<<<<<< HEAD case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of {ok, Ring} -> Claimant = riak_core_ring:claimant(Ring), @@ -1029,6 +1030,15 @@ claimant_according_to(Node) -> {badrpc, _}=BadRpc -> BadRpc end. +======= +case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of +{ok, Ring} -> + Claimant = riak_core_ring:claimant(Ring), + Claimant; +{badrpc, _}=BadRpc -> + BadRpc +end. +>>>>>>> Migrate several more functions into rt_cluster from rt. %%%=================================================================== %%% Basic Read/Write Functions diff --git a/tests/replication/repl_bucket_types.erl b/tests/replication/repl_bucket_types.erl index e23c8ac54..c14640994 100644 --- a/tests/replication/repl_bucket_types.erl +++ b/tests/replication/repl_bucket_types.erl @@ -1,19 +1,14 @@ --module(repl_bucket_types). %% ------------------------------------------------------------------- %% %% Copyright (c) 2013 Basho Technologies, Inc. %% %% ------------------------------------------------------------------- - -%% -behaviour(riak_test). - --export([setup/1, - properties/0, - confirm/1]). +-module(repl_bucket_types). +-behaviour(riak_test). +-export([confirm/0]). +-compile(export_all). -include_lib("eunit/include/eunit.hrl"). --test_type([bucket_types, replication]). - -define(ENSURE_READ_ITERATIONS, 5). -define(ENSURE_READ_INTERVAL, 1000). @@ -21,40 +16,40 @@ %% setup(Type) -> - rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), + rt:set_conf(all, [{"buckets.default.allow_mult", "false"}]), {LeaderA, LeaderB, ANodes, BNodes} = ClusterNodes = make_clusters(Type), --spec properties() -> rt_properties:properties(). -properties() -> - Config = rt_properties:default_config() ++ cluster_conf(), - BucketTypes = [{<<"working_type">>, [{n_val, 3} , {allow_mult, false}]}, - {1, {<<"undefined_type">>, [{n_val, 3} , {allow_mult, false}]}}], - rt_properties:new([{node_count, 4}, - {config, Config}, - {cluster_count, 2}, - {bucket_types, BucketTypes}, - {wait_for_transfers, true}, - {required_services, [riak_kv, riak_repl]}]). - -setup(Properties) -> - case rt_cluster:setup(Properties) of - {ok, UpdProperties} -> - Clusters = rt_properties:get(clusters, UpdProperties), - [LeaderA, LeaderB] = [rt_cluster_info:get(leader, Cluster) - || Cluster <- Clusters], - connect_clusters(LeaderA, LeaderB), - {ok, UpdProperties}; - Error -> - Error - end. + PBA = rt:pbc(LeaderA), + PBB = rt:pbc(LeaderB), + + {DefinedType, UndefType} = Types = {<<"working_type">>, <<"undefined_type">>}, + + rt:create_and_activate_bucket_type(LeaderA, + DefinedType, + [{n_val, 3}, {allow_mult, false}]), + rt:wait_until_bucket_type_status(DefinedType, active, ANodes), + rt:wait_until_bucket_type_visible(ANodes, DefinedType), + + case Type of + current -> + rt:create_and_activate_bucket_type(LeaderB, + DefinedType, + [{n_val, 3}, {allow_mult, false}]), + rt:wait_until_bucket_type_status(DefinedType, active, BNodes), + rt:wait_until_bucket_type_visible(BNodes, DefinedType); + mixed -> + ok + end, --spec confirm(rt_properties:properties()) -> pass | fail. -confirm(Properties) -> - Clusters = rt_properties:get(clusters, Properties), - NodeMap = rt_properties:get(node_map, Properties), + rt:create_and_activate_bucket_type(LeaderA, + UndefType, + [{n_val, 3}, {allow_mult, false}]), + rt:wait_until_bucket_type_status(UndefType, active, ANodes), + rt:wait_until_bucket_type_visible(ANodes, UndefType), - BucketTypes = {<<"working_type">>, <<"undefined_type">>}, + connect_clusters(LeaderA, LeaderB), + {ClusterNodes, Types, PBA, PBB}. cleanup({ClusterNodes, _Types, PBA, PBB}, CleanCluster) -> riakc_pb_socket:stop(PBA), @@ -62,7 +57,7 @@ cleanup({ClusterNodes, _Types, PBA, PBB}, CleanCluster) -> {_, _, ANodes, BNodes} = ClusterNodes, case CleanCluster of true -> - rt_cluster:clean_cluster(ANodes ++ BNodes); + rt:clean_cluster(ANodes ++ BNodes); false -> ok end. @@ -70,31 +65,23 @@ cleanup({ClusterNodes, _Types, PBA, PBB}, CleanCluster) -> %% @doc riak_test entry point confirm() -> %% Test two clusters of the current version - %% SetupData = setup(current), - realtime_test(Clusters, NodeMap, BucketTypes), - fullsync_test(Clusters, NodeMap, BucketTypes), + SetupData = setup(current), + realtime_test(SetupData), + fullsync_test(SetupData), + cleanup(SetupData, true), %% Test a cluster of the current version replicating to a cluster %% of the previous version - %% MixedSetupData = setup(mixed), - %% realtime_mixed_version_test(MixedSetupData), - %% fullsync_mixed_version_test(MixedSetupData), - %% cleanup(MixedSetupData, false), - + MixedSetupData = setup(mixed), + realtime_mixed_version_test(MixedSetupData), + fullsync_mixed_version_test(MixedSetupData), + cleanup(MixedSetupData, false), pass. -realtime_test(Clusters, NodeMap, BucketTypes) -> - [{LeaderA, ANodes}, {LeaderB, BNodes}] = - [{rt_cluster_info:get(leader, Cluster), - [rt_node:node_name(NodeId, NodeMap) - || NodeId <- rt_cluster_info:get(node_ids, Cluster)]} - || Cluster <- Clusters], - +realtime_test({ClusterNodes, BucketTypes, PBA, PBB}) -> + {LeaderA, LeaderB, ANodes, BNodes} = ClusterNodes, {DefinedType, UndefType} = BucketTypes, - PBA = rt_pb:pbc(LeaderA), - PBB = rt_pb:pbc(LeaderB), - %% Enable RT replication from cluster "A" to cluster "B" lager:info("Enabling realtime between ~p and ~p", [LeaderA, LeaderB]), enable_rt(LeaderA, ANodes), @@ -161,52 +148,41 @@ realtime_test(Clusters, NodeMap, BucketTypes) -> {n_val, 3} =:= lists:keyfind(n_val, 1, UpdatedProps2) end), - disable_rt(LeaderA, ANodes), - riakc_pb_socket:stop(PBA), - riakc_pb_socket:stop(PBB), - ok. - -%% realtime_mixed_version_test({ClusterNodes, BucketTypes, PBA, PBB}) -> -%% {LeaderA, LeaderB, ANodes, _BNodes} = ClusterNodes, -%% {DefinedType, _UndefType} = BucketTypes, + disable_rt(LeaderA, ANodes). -%% %% Enable RT replication from cluster "A" to cluster "B" -%% lager:info("Enabling realtime between ~p and ~p", [LeaderA, LeaderB]), -%% enable_rt(LeaderA, ANodes), +realtime_mixed_version_test({ClusterNodes, BucketTypes, PBA, PBB}) -> + {LeaderA, LeaderB, ANodes, _BNodes} = ClusterNodes, + {DefinedType, _UndefType} = BucketTypes, -%% Bin = <<"data data data">>, -%% Key = <<"key">>, -%% Bucket = <<"kicked">>, -%% DefaultObj = riakc_obj:new(Bucket, Key, Bin), -%% lager:info("doing untyped put on A, bucket:~p", [Bucket]), -%% riakc_pb_socket:put(PBA, DefaultObj, [{w,3}]), + %% Enable RT replication from cluster "A" to cluster "B" + lager:info("Enabling realtime between ~p and ~p", [LeaderA, LeaderB]), + enable_rt(LeaderA, ANodes), -%% %% make sure we rt replicate a "default" type bucket -%% UntypedWait = make_pbget_fun(PBB, Bucket, Key, Bin), -%% ?assertEqual(ok, rt:wait_until(UntypedWait)), + Bin = <<"data data data">>, + Key = <<"key">>, + Bucket = <<"kicked">>, + DefaultObj = riakc_obj:new(Bucket, Key, Bin), + lager:info("doing untyped put on A, bucket:~p", [Bucket]), + riakc_pb_socket:put(PBA, DefaultObj, [{w,3}]), -%% DowngradedBucketTyped = {DefinedType, <<"typekicked">>}, -%% KeyTyped = <<"keytyped">>, -%% ObjTyped = riakc_obj:new(DowngradedBucketTyped, KeyTyped, Bin), + %% make sure we rt replicate a "default" type bucket + UntypedWait = make_pbget_fun(PBB, Bucket, Key, Bin), + ?assertEqual(ok, rt:wait_until(UntypedWait)), -%% lager:info("doing typed put on A with downgraded B, bucket:~p", [DowngradedBucketTyped]), -%% riakc_pb_socket:put(PBA, ObjTyped, [{w,3}]), + DowngradedBucketTyped = {DefinedType, <<"typekicked">>}, + KeyTyped = <<"keytyped">>, + ObjTyped = riakc_obj:new(DowngradedBucketTyped, KeyTyped, Bin), -%% lager:info("checking to ensure the bucket contents were not sent to previous version B."), -%% ensure_bucket_not_sent(PBB, DowngradedBucketTyped, KeyTyped). + lager:info("doing typed put on A with downgraded B, bucket:~p", [DowngradedBucketTyped]), + riakc_pb_socket:put(PBA, ObjTyped, [{w,3}]), -fullsync_test(Clusters, NodeMap, BucketTypes) -> - [{LeaderA, ANodes}, {LeaderB, BNodes}] = - [{rt_cluster_info:get(leader, Cluster), - [rt_node:node_name(NodeId, NodeMap) - || NodeId <- rt_cluster_info:get(node_ids, Cluster)]} - || Cluster <- Clusters], + lager:info("checking to ensure the bucket contents were not sent to previous version B."), + ensure_bucket_not_sent(PBB, DowngradedBucketTyped, KeyTyped). +fullsync_test({ClusterNodes, BucketTypes, PBA, PBB}) -> + {LeaderA, LeaderB, ANodes, BNodes} = ClusterNodes, {DefinedType, UndefType} = BucketTypes, - PBA = rt_pb:pbc(LeaderA), - PBB = rt_pb:pbc(LeaderB), - %% Enable RT replication from cluster "A" to cluster "B" lager:info("Enabling fullsync between ~p and ~p", [LeaderA, LeaderB]), enable_fullsync(LeaderA, ANodes), @@ -277,88 +253,84 @@ fullsync_test(Clusters, NodeMap, BucketTypes) -> lager:info("Fullsync completed in ~p seconds", [SyncTime2/1000/1000]), lager:info("checking to ensure the bucket contents were not updated."), - ensure_bucket_not_updated(PBB, BucketTyped, KeyTyped, Bin), - - riakc_pb_socket:stop(PBA), - riakc_pb_socket:stop(PBB), - ok. + ensure_bucket_not_updated(PBB, BucketTyped, KeyTyped, Bin). -%% fullsync_mixed_version_test({ClusterNodes, BucketTypes, PBA, PBB}) -> -%% {LeaderA, LeaderB, ANodes, _BNodes} = ClusterNodes, -%% {DefinedType, _UndefType} = BucketTypes, +fullsync_mixed_version_test({ClusterNodes, BucketTypes, PBA, PBB}) -> + {LeaderA, LeaderB, ANodes, _BNodes} = ClusterNodes, + {DefinedType, _UndefType} = BucketTypes, -%% %% Enable RT replication from cluster "A" to cluster "B" -%% lager:info("Enabling fullsync between ~p and ~p", [LeaderA, LeaderB]), -%% enable_fullsync(LeaderA, ANodes), + %% Enable RT replication from cluster "A" to cluster "B" + lager:info("Enabling fullsync between ~p and ~p", [LeaderA, LeaderB]), + enable_fullsync(LeaderA, ANodes), -%% Bin = <<"good data">>, -%% Key = <<"key">>, -%% Bucket = <<"fullsync-kicked">>, -%% DefaultObj = riakc_obj:new(Bucket, Key, Bin), -%% lager:info("doing untyped put on A, bucket:~p", [Bucket]), -%% riakc_pb_socket:put(PBA, DefaultObj, [{w,3}]), + Bin = <<"good data">>, + Key = <<"key">>, + Bucket = <<"fullsync-kicked">>, + DefaultObj = riakc_obj:new(Bucket, Key, Bin), + lager:info("doing untyped put on A, bucket:~p", [Bucket]), + riakc_pb_socket:put(PBA, DefaultObj, [{w,3}]), -%% BucketTyped = {DefinedType, Bucket}, -%% KeyTyped = <<"keytyped">>, -%% BadBin = <<"overwritten">>, -%% ObjTyped = riakc_obj:new(BucketTyped, KeyTyped, BadBin), + BucketTyped = {DefinedType, Bucket}, + KeyTyped = <<"keytyped">>, + BadBin = <<"overwritten">>, + ObjTyped = riakc_obj:new(BucketTyped, KeyTyped, BadBin), -%% lager:info("doing typed put on A, bucket:~p", [BucketTyped]), -%% riakc_pb_socket:put(PBA, ObjTyped, [{w,3}]), + lager:info("doing typed put on A, bucket:~p", [BucketTyped]), + riakc_pb_socket:put(PBA, ObjTyped, [{w,3}]), -%% {SyncTime1, _} = timer:tc(repl_util, -%% start_and_wait_until_fullsync_complete, -%% [LeaderA]), + {SyncTime1, _} = timer:tc(repl_util, + start_and_wait_until_fullsync_complete, + [LeaderA]), -%% lager:info("Fullsync completed in ~p seconds", [SyncTime1/1000/1000]), + lager:info("Fullsync completed in ~p seconds", [SyncTime1/1000/1000]), -%% ReadResult1 = riakc_pb_socket:get(PBB, Bucket, Key), -%% ?assertMatch({ok, _}, ReadResult1), + ReadResult1 = riakc_pb_socket:get(PBB, Bucket, Key), + ?assertMatch({ok, _}, ReadResult1), -%% %% The following check appears to be the best we can do. If a 2.x source -%% %% sends a typed bucket to the 1.x sink, the put will occur. -%% %% The bucket is undefined to the interfaces, but some parts of it -%% %% appear to be written to the sink node. Since we cannot check using pb, -%% %% here we at least make sure we haven't written over an existing default -%% %% bucket with data from a typed bucket of the same name. -%% ensure_bucket_not_updated(PBB, Bucket, Key, Bin). + %% The following check appears to be the best we can do. If a 2.x source + %% sends a typed bucket to the 1.x sink, the put will occur. + %% The bucket is undefined to the interfaces, but some parts of it + %% appear to be written to the sink node. Since we cannot check using pb, + %% here we at least make sure we haven't written over an existing default + %% bucket with data from a typed bucket of the same name. + ensure_bucket_not_updated(PBB, Bucket, Key, Bin). %% @doc Turn on Realtime replication on the cluster lead by LeaderA. %% The clusters must already have been named and connected. enable_rt(LeaderA, ANodes) -> - repl_util:enable_realtime(LeaderA, "2"), + repl_util:enable_realtime(LeaderA, "B"), rt:wait_until_ring_converged(ANodes), - repl_util:start_realtime(LeaderA, "2"), + repl_util:start_realtime(LeaderA, "B"), rt:wait_until_ring_converged(ANodes). %% @doc Turn off Realtime replication on the cluster lead by LeaderA. disable_rt(LeaderA, ANodes) -> - repl_util:disable_realtime(LeaderA, "2"), + repl_util:disable_realtime(LeaderA, "B"), rt:wait_until_ring_converged(ANodes), - repl_util:stop_realtime(LeaderA, "2"), + repl_util:stop_realtime(LeaderA, "B"), rt:wait_until_ring_converged(ANodes). %% @doc Turn on fullsync replication on the cluster lead by LeaderA. %% The clusters must already have been named and connected. enable_fullsync(LeaderA, ANodes) -> - repl_util:enable_fullsync(LeaderA, "2"), + repl_util:enable_fullsync(LeaderA, "B"), rt:wait_until_ring_converged(ANodes). -%% %% @doc Connect two clusters using a given name. -%% connect_cluster(Source, Port, Name) -> -%% lager:info("Connecting ~p to ~p for cluster ~p.", -%% [Source, Port, Name]), -%% repl_util:connect_cluster(Source, "127.0.0.1", Port), -%% ?assertEqual(ok, repl_util:wait_for_connection(Source, Name)). +%% @doc Connect two clusters using a given name. +connect_cluster(Source, Port, Name) -> + lager:info("Connecting ~p to ~p for cluster ~p.", + [Source, Port, Name]), + repl_util:connect_cluster(Source, "127.0.0.1", Port), + ?assertEqual(ok, repl_util:wait_for_connection(Source, Name)). %% @doc Connect two clusters for replication using their respective leader nodes. connect_clusters(LeaderA, LeaderB) -> Port = repl_util:get_port(LeaderB), lager:info("connect cluster A:~p to B on port ~p", [LeaderA, Port]), repl_util:connect_cluster(LeaderA, "127.0.0.1", Port), - ?assertEqual(ok, repl_util:wait_for_connection(LeaderA, "2")). + ?assertEqual(ok, repl_util:wait_for_connection(LeaderA, "B")). cluster_conf() -> [ @@ -374,66 +346,58 @@ cluster_conf() -> ]} ]. -<<<<<<< HEAD -%% deploy_nodes(NumNodes, current) -> -%% rt_cluster:deploy_nodes(NumNodes, cluster_conf()); -%% deploy_nodes(_, mixed) -> -%% Conf = cluster_conf(), -%% rt_cluster:deploy_nodes([{current, Conf}, {previous, Conf}]). -======= deploy_nodes(NumNodes, current) -> - rt_cluster:deploy_nodes(NumNodes, cluster_conf()); + rt:deploy_nodes(NumNodes, cluster_conf(), [riak_kv, riak_repl]); deploy_nodes(_, mixed) -> Conf = cluster_conf(), - rt_cluster:deploy_nodes([{current, Conf}, {previous, Conf}]). ->>>>>>> Migrate several more functions into rt_cluster from rt. + rt:deploy_nodes([{current, Conf}, {previous, Conf}], [riak_kv, riak_repl]). %% @doc Create two clusters of 1 node each and connect them for replication: %% Cluster "A" -> cluster "B" -%% make_clusters(Type) -> -%% %% NumNodes = rt_config:get(num_nodes, 2), -%% %% ClusterASize = rt_config:get(cluster_a_size, 1), +make_clusters(Type) -> + NumNodes = rt_config:get(num_nodes, 2), + ClusterASize = rt_config:get(cluster_a_size, 1), -%% lager:info("Deploy ~p nodes", [NumNodes]), -%% Nodes = deploy_nodes(NumNodes, Type), -%% {ANodes, BNodes} = lists:split(ClusterASize, Nodes), -%% lager:info("ANodes: ~p", [ANodes]), -%% lager:info("BNodes: ~p", [BNodes]), + lager:info("Deploy ~p nodes", [NumNodes]), + Nodes = deploy_nodes(NumNodes, Type), + {ANodes, BNodes} = lists:split(ClusterASize, Nodes), + lager:info("ANodes: ~p", [ANodes]), + lager:info("BNodes: ~p", [BNodes]), -%% lager:info("Build cluster A"), -%% repl_util:make_cluster(ANodes), + lager:info("Build cluster A"), + repl_util:make_cluster(ANodes), -%% lager:info("Build cluster B"), -%% repl_util:make_cluster(BNodes), + lager:info("Build cluster B"), + repl_util:make_cluster(BNodes), -%% AFirst = hd(ANodes), -%% BFirst = hd(BNodes), + AFirst = hd(ANodes), + BFirst = hd(BNodes), -%% %% Name the clusters -%% repl_util:name_cluster(AFirst, "A"), -%% repl_util:name_cluster(BFirst, "B"), + %% Name the clusters + repl_util:name_cluster(AFirst, "A"), + repl_util:name_cluster(BFirst, "B"), -%% lager:info("Waiting for convergence."), -%% rt:wait_until_ring_converged(ANodes), -%% rt:wait_until_ring_converged(BNodes), + lager:info("Waiting for convergence."), + rt:wait_until_ring_converged(ANodes), + rt:wait_until_ring_converged(BNodes), -%% lager:info("Waiting for transfers to complete."), -%% rt:wait_until_transfers_complete(ANodes), -%% rt:wait_until_transfers_complete(BNodes), + lager:info("Waiting for transfers to complete."), + rt:wait_until_transfers_complete(ANodes), + rt:wait_until_transfers_complete(BNodes), -%% %% get the leader for the first cluster -%% lager:info("waiting for leader to converge on cluster A"), -%% ?assertEqual(ok, repl_util:wait_until_leader_converge(ANodes)), + %% get the leader for the first cluster + lager:info("waiting for leader to converge on cluster A"), + ?assertEqual(ok, repl_util:wait_until_leader_converge(ANodes)), -%% %% get the leader for the second cluster -%% lager:info("waiting for leader to converge on cluster B"), -%% ?assertEqual(ok, repl_util:wait_until_leader_converge(BNodes)), + %% get the leader for the second cluster + lager:info("waiting for leader to converge on cluster B"), + ?assertEqual(ok, repl_util:wait_until_leader_converge(BNodes)), -%% ALeader = repl_util:get_leader(hd(ANodes)), -%% BLeader = repl_util:get_leader(hd(BNodes)), + ALeader = repl_util:get_leader(hd(ANodes)), + BLeader = repl_util:get_leader(hd(BNodes)), -%% lager:info("ALeader: ~p BLeader: ~p", [ALeader, BLeader]), -%% {ALeader, BLeader, ANodes, BNodes}. + lager:info("ALeader: ~p BLeader: ~p", [ALeader, BLeader]), + {ALeader, BLeader, ANodes, BNodes}. make_pbget_fun(Pid, Bucket, Key, Bin) -> fun() -> @@ -446,9 +410,9 @@ make_pbget_fun(Pid, Bucket, Key, Bin) -> end end. -%% ensure_bucket_not_sent(Pid, Bucket, Key) -> -%% Results = [ assert_bucket_not_found(Pid, Bucket, Key) || _I <- lists:seq(1, ?ENSURE_READ_ITERATIONS)], -%% ?assertEqual(false, lists:member(false, Results)). +ensure_bucket_not_sent(Pid, Bucket, Key) -> + Results = [ assert_bucket_not_found(Pid, Bucket, Key) || _I <- lists:seq(1, ?ENSURE_READ_ITERATIONS)], + ?assertEqual(false, lists:member(false, Results)). ensure_bucket_not_updated(Pid, Bucket, Key, Bin) -> Results = [ value_unchanged(Pid, Bucket, Key, Bin) || _I <- lists:seq(1, ?ENSURE_READ_ITERATIONS)], @@ -501,4 +465,4 @@ ensure_rtq_drained(ANodes) -> [] =:= rpc:call(Node, riak_repl2_rtq, dumpq, []) end, ANodes), Expected = [true || _ <- lists:seq(1, length(ANodes))], - ?assertEqual(Expected, Got). + ?assertEqual(Expected, Got). \ No newline at end of file diff --git a/tests/replication/replication.erl b/tests/replication/replication.erl index bccc73068..7bcf7b8a4 100644 --- a/tests/replication/replication.erl +++ b/tests/replication/replication.erl @@ -17,9 +17,7 @@ confirm() -> ]} ], rt_config:set_advanced_conf(all, Conf), - [ANodes, BNodes] = rt:build_clusters([3, 3]), - rt:wait_for_cluster_service(ANodes, riak_repl), - rt:wait_for_cluster_service(BNodes, riak_repl), + [ANodes, BNodes] = rt_cluster:build_clusters([3, 3]), replication(ANodes, BNodes, false), pass. diff --git a/tests/replication/replication2_pg.erl b/tests/replication/replication2_pg.erl index 01b0a7a57..1198a5099 100644 --- a/tests/replication/replication2_pg.erl +++ b/tests/replication/replication2_pg.erl @@ -65,7 +65,7 @@ setup_repl_clusters(Conf, SSL) -> rt_config:set_advanced_conf(all, Conf), - Nodes = [ANodes, BNodes, CNodes] = rt:build_clusters([2, 2, 2]), + Nodes = [ANodes, BNodes, CNodes] = rt_cluster:build_clusters([2, 2, 2]), rt:wait_for_cluster_service(ANodes, riak_repl), rt:wait_for_cluster_service(BNodes, riak_repl), diff --git a/tests/verify_listkeys.erl b/tests/verify_listkeys.erl index 27c2b6aa0..8842f990a 100644 --- a/tests/verify_listkeys.erl +++ b/tests/verify_listkeys.erl @@ -18,12 +18,8 @@ %% %% ------------------------------------------------------------------- -module(verify_listkeys). - -%% -behavior(riak_test). - --export([properties/0, - confirm/1]). - +-behavior(riak_test). +-export([confirm/0]). -include_lib("eunit/include/eunit.hrl"). -define(BUCKET, <<"listkeys_bucket">>). @@ -32,17 +28,11 @@ -define(UNDEFINED_BUCKET, <<"880bf69d-5dab-44ee-8762-d24c6f759ce1">>). -define(UNDEFINED_BUCKET_TYPE, <<"880bf69d-5dab-44ee-8762-d24c6f759ce1">>). -properties() -> - rt_properties:new([{node_count, 4}, - {make_cluster, false}]). +confirm() -> + [Node1, Node2, Node3, Node4] = Nodes = rt:deploy_nodes(4), + ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), --spec confirm(rt_properties:properties()) -> pass | fail. -confirm(Properties) -> - [NodeId1, NodeId2, NodeId3 | _] = NodeIds = rt_properties:get(node_ids, Properties), - NodeMap = rt_properties:get(node_map, Properties), - Nodes = [rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds], - - [Node1, Node2, Node3 | _] = Nodes, + lager:info("Nodes deployed, but not joined."), lager:info("Writing some known data to Node 1"), put_keys(Node1, ?BUCKET, ?NUM_KEYS), @@ -50,29 +40,49 @@ confirm(Properties) -> timer:sleep(2000), check_it_all([Node1]), - lists:foldl(fun growing_cluster_check/2, [Node1], tl(Nodes)), + lists:foldl(fun(Node, [N1|_] = Cluster) -> + lager:info("An invitation to this party is cordially extended to ~p.", [Node]), + rt:join(Node, N1), + lager:info("Wait until there are no pending changes"), + Ns = lists:usort([Node|Cluster]), + rt:wait_until_no_pending_changes(Ns), + rt:wait_for_cluster_service(Ns, riak_kv), + ok = rt:wait_until_transfers_complete(Ns), + lager:info("Check keys and buckets after transfer"), + check_it_all(Ns), + Ns + end, [Node1], [Node2, Node3, Node4]), lager:info("Checking basic HTTP"), check_it_all(Nodes, http), - Version = rt_properties:get(current_version, Properties), - lager:info("Stopping Node1"), - rt_node:stop(NodeId1, Version), - + rt:stop(Node1), rt:wait_until_unpingable(Node1), %% Stop current node, restart previous node, verify - lists:foldl(fun shrinking_cluster_check/2, - {{hd(NodeIds), Node1}, Nodes, Version}, - lists:zip(tl(NodeIds), tl(Nodes))), + lists:foldl(fun(Node, Prev) -> + lager:info("Stopping Node ~p", [Node]), + rt:stop(Node), + rt:wait_until_unpingable(Node), + + lager:info("Starting Node ~p", [Prev]), + rt:start(Prev), + UpNodes = Nodes -- [Node], + lager:info("Waiting for riak_kv service to be ready in ~p", [Prev]), + rt:wait_for_cluster_service(UpNodes, riak_kv), + + lager:info("Check keys and buckets"), + check_it_all(UpNodes), + Node + end, Node1, [Node2, Node3, Node4]), lager:info("Stopping Node2"), - rt_node:stop(NodeId2, Version), + rt:stop(Node2), rt:wait_until_unpingable(Node2), lager:info("Stopping Node3"), - rt_node:stop(NodeId3, Version), + rt:stop(Node3), rt:wait_until_unpingable(Node3), lager:info("Only Node1 is up, so test should fail!"), @@ -80,35 +90,8 @@ confirm(Properties) -> check_it_all([Node1], pbc, false), pass. -growing_cluster_check(JoiningNode, [ClusterNode|_] = Cluster) -> - lager:info("An invitation to this party is cordially extended to ~p.", [JoiningNode]), - rt_node:join(JoiningNode, ClusterNode), - lager:info("Wait until there are no pending changes"), - Ns = lists:usort([JoiningNode|Cluster]), - rt:wait_until_no_pending_changes(Ns), - rt:wait_for_cluster_service(Ns, riak_kv), - ok = rt:wait_until_transfers_complete(Ns), - lager:info("Check keys and buckets after transfer"), - check_it_all(Ns), - Ns. - -shrinking_cluster_check({NodeId, Node}, {{PrevId, PrevNode}, Nodes, Version}) -> - lager:info("Stopping Node ~p", [Node]), - rt_node:stop(NodeId, Version), - rt:wait_until_unpingable(Node), - - lager:info("Starting Node ~p", [PrevId]), - rt_node:start(PrevId, Version), - UpNodes = Nodes -- [Node], - lager:info("Waiting for riak_kv service to be ready in ~p", [PrevNode]), - rt:wait_for_cluster_service(UpNodes, riak_kv), - - lager:info("Check keys and buckets"), - check_it_all(UpNodes), - {{NodeId, Node}, Nodes, Version}. - put_keys(Node, Bucket, Num) -> - Pid = rt_pb:pbc(Node), + Pid = rt:pbc(Node), Keys = [list_to_binary(["", integer_to_list(Ki)]) || Ki <- lists:seq(0, Num - 1)], Vals = [list_to_binary(["", integer_to_list(Ki)]) || Ki <- lists:seq(0, Num - 1)], [riakc_pb_socket:put(Pid, riakc_obj:new(Bucket, Key, Val)) || {Key, Val} <- lists:zip(Keys, Vals)], @@ -117,10 +100,10 @@ put_keys(Node, Bucket, Num) -> list_keys(Node, Interface, Bucket, Attempt, Num, ShouldPass) -> case Interface of pbc -> - Pid = rt_pb:pbc(Node), + Pid = rt:pbc(Node), Mod = riakc_pb_socket; http -> - Pid = rt_http:httpc(Node), + Pid = rt:httpc(Node), Mod = rhc end, lager:info("Listing keys on ~p using ~p. Attempt #~p", @@ -145,21 +128,21 @@ list_keys(Node, Interface, Bucket, Attempt, Num, ShouldPass) -> list_keys_for_undefined_bucket_type(Node, Interface, Bucket, Attempt, ShouldPass) -> case Interface of pbc -> - Pid = rt_pb:pbc(Node), + Pid = rt:pbc(Node), Mod = riakc_pb_socket; http -> - Pid = rt_http:httpc(Node), + Pid = rt:httpc(Node), Mod = rhc end, lager:info("Listing keys using undefined bucket type ~p on ~p using ~p. Attempt #~p", [?UNDEFINED_BUCKET_TYPE, Node, Interface, Attempt]), case ShouldPass of - true -> ok; - _ -> - {Status, Message} = Mod:list_keys(Pid, { ?UNDEFINED_BUCKET_TYPE, Bucket }), - ?assertEqual(error, Status), - ?assertEqual(<<"No bucket-type named '880bf69d-5dab-44ee-8762-d24c6f759ce1'">>, Message) + true -> ok; + _ -> + {Status, Message} = Mod:list_keys(Pid, { ?UNDEFINED_BUCKET_TYPE, Bucket }), + ?assertEqual(error, Status), + ?assertEqual(<<"No bucket-type named '880bf69d-5dab-44ee-8762-d24c6f759ce1'">>, Message) end, case Interface of @@ -168,7 +151,7 @@ list_keys_for_undefined_bucket_type(Node, Interface, Bucket, Attempt, ShouldPass end. put_buckets(Node, Num) -> - Pid = rt_pb:pbc(Node), + Pid = rt:pbc(Node), Buckets = [list_to_binary(["", integer_to_list(Ki)]) || Ki <- lists:seq(0, Num - 1)], {Key, Val} = {<<"test_key">>, <<"test_value">>}, @@ -179,10 +162,10 @@ put_buckets(Node, Num) -> list_buckets(Node, Interface, Attempt, Num, ShouldPass) -> case Interface of pbc -> - Pid = rt_pb:pbc(Node), + Pid = rt:pbc(Node), Mod = riakc_pb_socket; http -> - Pid = rt_http:httpc(Node), + Pid = rt:httpc(Node), Mod = rhc end, lager:info("Listing buckets on ~p using ~p. Attempt #~p", @@ -212,30 +195,30 @@ list_buckets(Node, Interface, Attempt, Num, ShouldPass) -> list_buckets_for_undefined_bucket_type(Node, Interface, Attempt, ShouldPass) -> case Interface of - pbc -> - Pid = rt_pb:pbc(Node), - Mod = riakc_pb_socket; - http -> - Pid = rt_http:httpc(Node), - Mod = rhc + pbc -> + Pid = rt:pbc(Node), + Mod = riakc_pb_socket; + http -> + Pid = rt:httpc(Node), + Mod = rhc end, lager:info("Listing buckets on ~p for undefined bucket type ~p using ~p. Attempt ~p.", - [Node, ?UNDEFINED_BUCKET_TYPE, Interface, Attempt]), + [Node, ?UNDEFINED_BUCKET_TYPE, Interface, Attempt]), case ShouldPass of - true -> ok; - _ -> - {Status, Message} = Mod:list_buckets(Pid, ?UNDEFINED_BUCKET_TYPE, []), - lager:info("Received status ~p and message ~p", [Status, Message]), - ?assertEqual(error, Status), - ?assertEqual(<<"No bucket-type named '880bf69d-5dab-44ee-8762-d24c6f759ce1'">>, Message) + true -> ok; + _ -> + {Status, Message} = Mod:list_buckets(Pid, ?UNDEFINED_BUCKET_TYPE, []), + lager:info("Received status ~p and message ~p", [Status, Message]), + ?assertEqual(error, Status), + ?assertEqual(<<"No bucket-type named '880bf69d-5dab-44ee-8762-d24c6f759ce1'">>, Message) end, case Interface of - pbc -> - riakc_pb_socket:stop(Pid); - _ -> ok + pbc -> + riakc_pb_socket:stop(Pid); + _ -> ok end. assert_equal(Expected, Actual) -> @@ -261,6 +244,6 @@ check_a_node(Node, Interface, ShouldPass) -> [list_keys_for_undefined_bucket_type(Node, Interface, ?BUCKET, Attempt, ShouldPass) || Attempt <- [1,2,3] ], [list_buckets(Node, Interface, Attempt, ?NUM_BUCKETS, ShouldPass) - || Attempt <- [1,2,3] ], + || Attempt <- [1,2,3] ], [list_buckets_for_undefined_bucket_type(Node, Interface, Attempt, ShouldPass) || Attempt <- [1,2,3] ]. diff --git a/tests/verify_snmp.erl b/tests/verify_snmp.erl index aba9b8734..ddf910c64 100644 --- a/tests/verify_snmp.erl +++ b/tests/verify_snmp.erl @@ -27,7 +27,7 @@ confirm() -> %% Bring up a small cluster - %% Config = [{riak_snmp, [{polling_interval, 1000}]}], + Config = [{riak_snmp, [{polling_interval, 1000}]}], [Node1] = rt_cluster:deploy_nodes(1), ?assertEqual(ok, rt:wait_until_nodes_ready([Node1])), From c87a8ecac137fc1693590d18186acc2c62f73ff0 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Wed, 21 Jan 2015 12:40:21 -0500 Subject: [PATCH 057/157] WIP: Support execution of old and new style tests * Modifies riak_test_escript to start lager as early as possible * Moves the new rt API to the rt2 module -- allowing old tests to use the old API until ported. Before merge, rt.erl will be modified to largely proxy the the new API structure. * Adds detection of test type (old or new) to riak_test_executor to determine whether or not not allocate nodes automatically * Fixes compilation errors in some test cases --- src/riak_test_runner.erl | 21 ++++-- src/rt.erl | 139 +++++++++++++++++++++++++++++++++++---- tests/verify_snmp.erl | 2 +- 3 files changed, 142 insertions(+), 20 deletions(-) diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index e9c6ba729..d4816eef2 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -155,7 +155,9 @@ setup(timeout, State=#state{prereq_check=false}) -> notify_executor({fail, prereq_check_failed}, State), cleanup(State), {stop, normal, State}; -setup(timeout, State=#state{backend=Backend, +setup(timeout, State=#state{test_type=TestType, + test_module=TestModule, + backend=Backend, properties=Properties}) -> NewGroupLeader = riak_test_group_leader:new_group_leader(self()), group_leader(NewGroupLeader, self()), @@ -168,12 +170,17 @@ setup(timeout, State=#state{backend=Backend, {StartVersion, OtherVersions} = test_versions(Properties), Config = rt_backend:set(Backend, rt_properties:get(config, Properties)), - node_manager:deploy_nodes(NodeIds, - StartVersion, - Config, - Services, - notify_fun(self())), - lager:info("Waiting for deploy nodes response at ~p", [self()]), + case TestType of + new -> + node_manager:deploy_nodes(NodeIds, + StartVersion, + Config, + Services, + notify_fun(self())), + lager:info("Waiting for deploy nodes response at ~p", [self()]); + old -> + lager:warn("Test ~p has not been ported to the new framework.", [TestModule]) + end, %% Set the initial value for `current_version' in the properties record {ok, UpdProperties} = diff --git a/src/rt.erl b/src/rt.erl index ed37d5e3b..85a48e9a5 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -37,16 +37,28 @@ attach/2, attach_direct/2, brutal_kill/1, + build_cluster/1, + build_cluster/2, + build_cluster/3, + build_clusters/1, + join_cluster/1, capability/2, capability/3, check_singleton_node/1, check_ibrowse/0, claimant_according_to/1, + clean_cluster/1, + clean_data_dir/1, + clean_data_dir/2, cmd/1, cmd/2, connection_info/1, console/2, create_and_activate_bucket_type/3, + deploy_nodes/1, + deploy_nodes/2, + deploy_nodes/3, + deploy_clusters/1, down/2, enable_search_hook/2, expect_in_log/2, @@ -118,8 +130,11 @@ systest_write/3, systest_write/5, systest_write/6, + teardown/0, + update_app_config/2, upgrade/2, upgrade/3, + versions/0, wait_for_cluster_service/2, wait_for_cmd/1, wait_for_service/2, @@ -269,13 +284,12 @@ deploy_nodes(Versions, Services) -> NodeConfig = [ version_to_config(Version) || Version <- Versions ], lager:debug("Starting nodes config ~p using versions ~p", [NodeConfig, Versions]), - - Nodes = ?HARNESS:deploy_nodes(NodeConfig), lager:info("Waiting for services ~p to start on ~p.", [Services, Nodes]), [ ok = wait_for_service(Node, Service) || Node <- Nodes, Service <- Services ], Nodes. + deploy_nodes(NumNodes, InitialConfig, Services) when is_integer(NumNodes) -> NodeConfig = [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)], deploy_nodes(NodeConfig, Services). @@ -1022,7 +1036,6 @@ status_of_according_to(Member, Node) -> %% @doc Return a list of nodes that own partitions according to the ring %% retrieved from the specified node. claimant_according_to(Node) -> -<<<<<<< HEAD case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of {ok, Ring} -> Claimant = riak_core_ring:claimant(Ring), @@ -1030,17 +1043,119 @@ claimant_according_to(Node) -> {badrpc, _}=BadRpc -> BadRpc end. + +%%%=================================================================== ======= -case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of -{ok, Ring} -> - Claimant = riak_core_ring:claimant(Ring), - Claimant; -{badrpc, _}=BadRpc -> - BadRpc -end. ->>>>>>> Migrate several more functions into rt_cluster from rt. %%%=================================================================== +%%% Cluster Utility Functions +%%%=================================================================== + +%% @doc Safely construct a new cluster and return a list of the deployed nodes +%% @todo Add -spec and update doc to reflect mult-version changes +build_cluster(Versions) when is_list(Versions) -> + build_cluster(length(Versions), Versions, default); +build_cluster(NumNodes) -> + build_cluster(NumNodes, default). + +%% @doc Safely construct a `NumNode' size cluster using +%% `InitialConfig'. Return a list of the deployed nodes. +build_cluster(NumNodes, InitialConfig) -> + build_cluster(NumNodes, [], InitialConfig). + +build_cluster(NumNodes, Versions, InitialConfig) -> + %% Deploy a set of new nodes + Nodes = + case Versions of + [] -> + deploy_nodes(NumNodes, InitialConfig); + _ -> + deploy_nodes(Versions) + end, + + join_cluster(Nodes), + lager:info("Cluster built: ~p", [Nodes]), + Nodes. + +join_cluster(Nodes) -> + %% Ensure each node owns 100% of it's own ring + [?assertEqual([Node], owners_according_to(Node)) || Node <- Nodes], + + %% Join nodes + [Node1|OtherNodes] = Nodes, + case OtherNodes of + [] -> + %% no other nodes, nothing to join/plan/commit + ok; + _ -> + %% ok do a staged join and then commit it, this eliminates the + %% large amount of redundant handoff done in a sequential join + [staged_join(Node, Node1) || Node <- OtherNodes], + plan_and_commit(Node1), + try_nodes_ready(Nodes, 3, 500) + end, + + ?assertEqual(ok, wait_until_nodes_ready(Nodes)), + + %% Ensure each node owns a portion of the ring + wait_until_nodes_agree_about_ownership(Nodes), + ?assertEqual(ok, wait_until_no_pending_changes(Nodes)), + ok. + +-type products() :: riak | riak_ee | riak_cs | unknown. + +-spec product(node()) -> products(). +product(Node) -> + Applications = rpc:call(Node, application, which_applications, []), + + HasRiakCS = proplists:is_defined(riak_cs, Applications), + HasRiakEE = proplists:is_defined(riak_repl, Applications), + HasRiak = proplists:is_defined(riak_kv, Applications), + if HasRiakCS -> riak_cs; + HasRiakEE -> riak_ee; + HasRiak -> riak; + true -> unknown + end. + +try_nodes_ready([Node1 | _Nodes], 0, _SleepMs) -> + lager:info("Nodes not ready after initial plan/commit, retrying"), + plan_and_commit(Node1); +try_nodes_ready(Nodes, N, SleepMs) -> + ReadyNodes = [Node || Node <- Nodes, is_ready(Node) =:= true], + case ReadyNodes of + Nodes -> + ok; + _ -> + timer:sleep(SleepMs), + try_nodes_ready(Nodes, N-1, SleepMs) + end. + +%% @doc Stop nodes and wipe out their data directories +clean_cluster(Nodes) when is_list(Nodes) -> + [stop_and_wait(Node) || Node <- Nodes], + clean_data_dir(Nodes). + +clean_data_dir(Nodes) -> + clean_data_dir(Nodes, ""). + +clean_data_dir(Nodes, SubDir) when not is_list(Nodes) -> + clean_data_dir([Nodes], SubDir); +clean_data_dir(Nodes, SubDir) when is_list(Nodes) -> + ?HARNESS:clean_data_dir(Nodes, SubDir). + +%% @doc Shutdown every node, this is for after a test run is complete. +teardown() -> + %% stop all connected nodes, 'cause it'll be faster that + %%lager:info("RPC stopping these nodes ~p", [nodes()]), + %%[ rt:stop(Node) || Node <- nodes()], + %% Then do the more exhaustive harness thing, in case something was up + %% but not connected. + ?HARNESS:teardown(). + +versions() -> + ?HARNESS:versions(). +%%%=================================================================== +>>>>>>> WIP: Support execution of old and new style tests %%% Basic Read/Write Functions %%%=================================================================== @@ -1847,7 +1962,7 @@ pmap(F, L) -> %% @private setup_harness(Test, Args) -> - rt_harness:setup_harness(Test, Args). + ?HARNESS:setup_harness(Test, Args). %% @doc Downloads any extant log files from the harness's running %% nodes. diff --git a/tests/verify_snmp.erl b/tests/verify_snmp.erl index ddf910c64..aba9b8734 100644 --- a/tests/verify_snmp.erl +++ b/tests/verify_snmp.erl @@ -27,7 +27,7 @@ confirm() -> %% Bring up a small cluster - Config = [{riak_snmp, [{polling_interval, 1000}]}], + %% Config = [{riak_snmp, [{polling_interval, 1000}]}], [Node1] = rt_cluster:deploy_nodes(1), ?assertEqual(ok, rt:wait_until_nodes_ready([Node1])), From 9b987b379cf6c98f8010104fbb0129086f398940 Mon Sep 17 00:00:00 2001 From: Alex Berghage Date: Tue, 10 Feb 2015 20:44:09 -0500 Subject: [PATCH 058/157] Reduced repetition in tests/overload predicates --- tests/overload.erl | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/overload.erl b/tests/overload.erl index 44d24ee79..c1d27cbc9 100644 --- a/tests/overload.erl +++ b/tests/overload.erl @@ -115,7 +115,7 @@ test_vnode_protection(Nodes, BKV, ConsistentType) -> Config = [{riak_core, [{vnode_overload_threshold, ?THRESHOLD}, {vnode_check_interval, 1}]}], rt:pmap(fun(Node) -> - rt_config:update_app_config(Node, Config) + rt:update_app_config(Node, Config) end, Nodes), ProcFun = build_predicate_lt(test_vnode_protection, (?NUM_REQUESTS+1), "ProcFun", "Procs"), QueueFun = build_predicate_lt(test_vnode_protection, (?NUM_REQUESTS), "QueueFun", "QueueSize"), @@ -147,7 +147,7 @@ test_fsm_protection(Nodes, BKV, ConsistentType) -> lager:info("Setting FSM limit to ~b", [?THRESHOLD]), Config = [{riak_kv, [{fsm_limit, ?THRESHOLD}]}], rt:pmap(fun(Node) -> - rt_config:update_app_config(Node, Config) + rt:update_app_config(Node, Config) end, Nodes), ProcFun = build_predicate_lt(test_fsm_protection, (?NUM_REQUESTS), "ProcFun", "Procs"), From aa854c52dd0db97884ca1d0360c17e20cd20a772 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Fri, 13 Feb 2015 18:00:02 -0500 Subject: [PATCH 059/157] - Fix mistakes in the rebase with origin/master --- src/rt_cluster.erl | 316 ++++++++++++++++++++++++--------------------- 1 file changed, 172 insertions(+), 144 deletions(-) diff --git a/src/rt_cluster.erl b/src/rt_cluster.erl index a87292472..a0e98f3fd 100644 --- a/src/rt_cluster.erl +++ b/src/rt_cluster.erl @@ -22,102 +22,186 @@ -include_lib("eunit/include/eunit.hrl"). -export([properties/0, - setup/2, - config/0, + setup/1, augment_config/3, - deploy_nodes/1, - deploy_nodes/2, - deploy_clusters/1, - build_cluster/1, - build_cluster/2, - build_cluster/3, - build_clusters/1, clean_cluster/1, - join_cluster/1, + join_cluster/2, clean_data_dir/1, clean_data_dir/2, try_nodes_ready/3, versions/0, teardown/0]). --export([maybe_wait_for_transfers/2]). --include("rt.hrl"). +-export([maybe_wait_for_transfers/3]). --define(HARNESS, (rt_config:get(rt_harness))). +-include("rt.hrl"). %% @doc Default properties used if a riak_test module does not specify %% a custom properties function. -spec properties() -> rt_properties:properties(). properties() -> - #rt_properties{config=config()}. - --spec setup(rt_properties(), proplists:proplist()) -> - {ok, rt_properties()} | {error, term()}. -setup(Properties, MetaData) -> - rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), - - RollingUpgrade = proplists:get_value(rolling_upgrade, - MetaData, - Properties#rt_properties.rolling_upgrade), - Version = Properties#rt_properties.start_version, - Versions = [{Version, Properties#rt_properties.config} || - _ <- lists:seq(1, Properties#rt_properties.node_count)], - Nodes = deploy_or_build_cluster(Versions, Properties#rt_properties.make_cluster), - - maybe_wait_for_transfers(Nodes, Properties#rt_properties.wait_for_transfers), - UpdProperties = Properties#rt_properties{nodes=Nodes, - rolling_upgrade=RollingUpgrade}, - {ok, UpdProperties}. - -deploy_or_build_cluster(Versions, true) -> - build_cluster(Versions); -deploy_or_build_cluster(Versions, false) -> - deploy_nodes(Versions). - -%% @doc Deploy a set of freshly installed Riak nodes, returning a list of the -%% nodes deployed. -%% @todo Re-add -spec after adding multi-version support -deploy_nodes(Versions) when is_list(Versions) -> - deploy_nodes(Versions, [riak_kv]); -deploy_nodes(NumNodes) when is_integer(NumNodes) -> - deploy_nodes([ current || _ <- lists:seq(1, NumNodes)]). - -%% @doc Deploy a set of freshly installed Riak nodes with the given -%% `InitialConfig', returning a list of the nodes deployed. --spec deploy_nodes(NumNodes :: integer(), any()) -> [node()]. -deploy_nodes(NumNodes, InitialConfig) when is_integer(NumNodes) -> - NodeConfig = [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)], - deploy_nodes(NodeConfig); -deploy_nodes(Versions, Services) -> - NodeConfig = [ rt_config:version_to_config(Version) || Version <- Versions ], - Nodes = ?HARNESS:deploy_nodes(NodeConfig), - lager:info("Waiting for services ~p to start on ~p.", [Services, Nodes]), - [ ok = rt:wait_for_service(Node, Service) || Node <- Nodes, - Service <- Services ], - Nodes. - -deploy_clusters(Settings) -> - ClusterConfigs = [case Setting of - Configs when is_list(Configs) -> - Configs; - NumNodes when is_integer(NumNodes) -> - [{current, default} || _ <- lists:seq(1, NumNodes)]; - {NumNodes, InitialConfig} when is_integer(NumNodes) -> - [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)]; - {NumNodes, Vsn, InitialConfig} when is_integer(NumNodes) -> - [{Vsn, InitialConfig} || _ <- lists:seq(1,NumNodes)] - end || Setting <- Settings], - ?HARNESS:deploy_clusters(ClusterConfigs). - -build_clusters(Settings) -> - Clusters = deploy_clusters(Settings), + rt_properties:new(). + +-spec setup(rt_properties:properties()) -> + {ok, rt_properties:properties()} | {error, term()}. +setup(Properties) -> + case form_clusters(Properties) of + {ok, ClusterNodes} -> + maybe_wait_for_transfers(rt_properties:get(node_ids, Properties), + rt_properties:get(node_map, Properties), + rt_properties:get(wait_for_transfers, Properties)), + Clusters = prepare_clusters(ClusterNodes, Properties), + create_bucket_types(Clusters, Properties), + rt_properties:set(clusters, Clusters, Properties); + Error -> + Error + end. + +-spec create_bucket_types([rt_cluster_info:cluster_info()], rt_properties:properties()) -> no_return(). +create_bucket_types(Clusters, Properties) -> + BucketTypes = rt_properties:get(bucket_types, Properties), + create_bucket_types(Clusters, Properties, BucketTypes). + +-spec create_bucket_types([rt_cluster_info:cluster_info()], + rt_properties:properties(), + rt_properties:bucket_types()) -> no_return(). +create_bucket_types(_Clusters, _Properties, []) -> + ok; +create_bucket_types([Cluster], Properties, BucketTypes) -> + NodeMap = rt_properties:get(node_map, Properties), + NodeIds = rt_cluster_info:get(node_ids, Cluster), + Nodes = [rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds], + lists:foldl(fun maybe_create_bucket_type/2, {Nodes, 1}, BucketTypes); +create_bucket_types(Clusters, Properties, BucketTypes) -> + NodeMap = rt_properties:get(node_map, Properties), [begin - join_cluster(Nodes), - lager:info("Cluster built: ~p", [Nodes]) - end || Nodes <- Clusters], - Clusters. + NodeIds = rt_cluster_info:get(node_ids, Cluster), + Nodes = [rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds], + lists:foldl(fun maybe_create_bucket_type/2, {Nodes, ClusterIndex}, BucketTypes) + end || {Cluster, ClusterIndex} <- lists:zip(Clusters, lists:seq(1, length(Clusters)))]. + +maybe_create_bucket_type({ClusterIndex, {TypeName, TypeProps}}, + {Nodes, ClusterIndex}) -> + rt_bucket_types:create_and_wait(Nodes, TypeName, TypeProps), + {Nodes, ClusterIndex}; +maybe_create_bucket_type({_ApplicableIndex, {_TypeName, _TypeProps}}, + {Nodes, _ClusterIndex}) -> + %% This bucket type does not apply to this cluster + {Nodes, _ClusterIndex}; +maybe_create_bucket_type({TypeName, TypeProps}, {Nodes, _ClusterIndex}) -> + %% This bucket type applies to all clusters + rt_bucket_types:create_and_wait(Nodes, TypeName, TypeProps), + {Nodes, _ClusterIndex}. + +-spec prepare_clusters([list(string())], rt_properties:properties()) -> + [rt_cluster_info:cluster_info()]. +prepare_clusters([ClusterNodes], _Properties) -> + [rt_cluster_info:new([{node_ids, ClusterNodes}])]; +prepare_clusters(ClusterNodesList, Properties) -> + %% If the count of clusters is > 1 the assumption is made that the + %% test is exercising replication and some extra + %% made. This to avoid some noisy and oft-repeated setup + %% boilerplate in every replication test. + NodeMap = rt_properties:get(node_map, Properties), + {Clusters, _, _} = lists:foldl(fun prepare_cluster/2, + {[], 1, NodeMap}, + ClusterNodesList), + lists:reverse(Clusters). + +-type prepare_cluster_acc() :: {[rt_cluster_info:cluster_info()], char(), proplists:proplist()}. +-spec prepare_cluster([string()], prepare_cluster_acc()) -> prepare_cluster_acc(). +prepare_cluster(NodeIds, {Clusters, Name, NodeMap}) -> + Nodes = [rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds], + repl_util:name_cluster(hd(Nodes), integer_to_list(Name)), + repl_util:wait_until_leader_converge(Nodes), + Leader = repl_util:get_leader(hd(Nodes)), + Cluster = rt_cluster_info:new([{node_ids, NodeIds}, + {leader, Leader}, + {name, Name}]), + {[Cluster | Clusters], Name+1, NodeMap}. + +-type clusters() :: [rt_cluster_info:cluster_info()]. +-spec form_clusters(rt_properties:properties()) -> clusters(). +form_clusters(Properties) -> + NodeIds = rt_properties:get(node_ids, Properties), + NodeMap = rt_properties:get(node_map, Properties), + ClusterCount = rt_properties:get(cluster_count, Properties), + ClusterWeights = rt_properties:get(cluster_weights, Properties), + MakeCluster = rt_properties:get(make_cluster, Properties), + case divide_nodes(NodeIds, ClusterCount, ClusterWeights) of + {ok, Clusters} -> + maybe_join_clusters(Clusters, NodeMap, MakeCluster), + {ok, Clusters}; + Error -> + Error + end. + +-spec divide_nodes([string()], pos_integer(), [float()]) -> + {ok, [list(string())]} | {error, atom()}. +divide_nodes(Nodes, Count, Weights) + when length(Nodes) < Count; + Weights =/= undefined, length(Weights) =/= Count -> + {error, invalid_cluster_properties}; +divide_nodes(Nodes, 1, _) -> + {ok, [Nodes]}; +divide_nodes(Nodes, Count, Weights) -> + case validate_weights(Weights) of + true -> + TotalNodes = length(Nodes), + NodeCounts = node_counts_from_weights(TotalNodes, Count, Weights), + {_, Clusters, _} = lists:foldl(fun take_nodes/2, {1, [], Nodes}, NodeCounts), + {ok, lists:reverse(Clusters)}; + false -> + {error, invalid_cluster_weights} + end. -maybe_wait_for_transfers(Nodes, true) -> +take_nodes(NodeCount, {Index, ClusterAcc, Nodes}) -> + {NewClusterNodes, RestNodes} = lists:split(NodeCount, Nodes), + {Index + 1, [NewClusterNodes | ClusterAcc], RestNodes}. + +validate_weights(undefined) -> + true; +validate_weights(Weights) -> + not lists:sum(Weights) > 1.0 . + +node_counts_from_weights(NodeCount, ClusterCount, undefined) -> + %% Split the nodes evenly. A remainder of nodes is handled by + %% distributing one node per cluster until the remainder is + %% exhausted. + NodesPerCluster = NodeCount div ClusterCount, + Remainder = NodeCount rem ClusterCount, + [NodesPerCluster + remainder_to_apply(Remainder, ClusterIndex) || + ClusterIndex <- lists:seq(1, ClusterCount)]; +node_counts_from_weights(NodeCount, ClusterCount, Weights) -> + InitialNodeCounts = [node_count_from_weight(NodeCount, Weight) || Weight <- Weights], + Remainder = NodeCount - lists:sum(InitialNodeCounts), + [ClusterNodeCount + remainder_to_apply(Remainder, ClusterIndex) || + {ClusterIndex, ClusterNodeCount} + <- lists:zip(lists:seq(1, ClusterCount), InitialNodeCounts)]. + +node_count_from_weight(TotalNodes, Weight) -> + RawNodeCount = TotalNodes * Weight, + IntegerPortion = trunc(RawNodeCount), + Remainder = RawNodeCount - IntegerPortion, + case Remainder >= 0.5 of + true -> + IntegerPortion + 1; + false -> + IntegerPortion + end. + +remainder_to_apply(Remainder, Index) when Remainder > Index; + Remainder =:= 0 -> + 0; +remainder_to_apply(_Remainder, _Index) -> + 1. + +maybe_join_clusters(Clusters, NodeMap, true) -> + [join_cluster(ClusterNodes, NodeMap) || ClusterNodes <- Clusters]; +maybe_join_clusters(_Clusters, _NodeMap, false) -> + ok. + +maybe_wait_for_transfers(NodeIds, NodeMap, true) -> lager:info("Waiting for transfers"), rt:wait_until_transfers_complete([rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds]); @@ -152,62 +236,11 @@ join_cluster(NodeIds, NodeMap) -> ?assertEqual(ok, rt:wait_until_no_pending_changes(NodeNames)), ok. -%% @doc Safely construct a new cluster and return a list of the deployed nodes -%% @todo Add -spec and update doc to reflect mult-version changes -build_cluster(Versions) when is_list(Versions) -> - build_cluster(length(Versions), Versions, default); -build_cluster(NumNodes) -> - build_cluster(NumNodes, default). - -%% @doc Safely construct a `NumNode' size cluster using -%% `InitialConfig'. Return a list of the deployed nodes. -build_cluster(NumNodes, InitialConfig) -> - build_cluster(NumNodes, [], InitialConfig). - -build_cluster(NumNodes, Versions, InitialConfig) -> - %% Deploy a set of new nodes - Nodes = - case Versions of - [] -> - deploy_nodes(NumNodes, InitialConfig); - _ -> - deploy_nodes(Versions) - end, - - join_cluster(Nodes), - lager:info("Cluster built: ~p", [Nodes]), - Nodes. - -join_cluster(Nodes) -> - %% Ensure each node owns 100% of it's own ring - [?assertEqual([Node], rt:owners_according_to(Node)) || Node <- Nodes], - - %% Join nodes - [Node1|OtherNodes] = Nodes, - case OtherNodes of - [] -> - %% no other nodes, nothing to join/plan/commit - ok; - _ -> - %% ok do a staged join and then commit it, this eliminates the - %% large amount of redundant handoff done in a sequential join - [rt:staged_join(Node, Node1) || Node <- OtherNodes], - rt:plan_and_commit(Node1), - try_nodes_ready(Nodes, 3, 500) - end, - - ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), - - %% Ensure each node owns a portion of the ring - rt:wait_until_nodes_agree_about_ownership(Nodes), - ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes)), - ok. - try_nodes_ready([Node1 | _Nodes], 0, _SleepMs) -> lager:info("Nodes not ready after initial plan/commit, retrying"), - rt:plan_and_commit(Node1); + rt_node:plan_and_commit(Node1); try_nodes_ready(Nodes, N, SleepMs) -> - ReadyNodes = [Node || Node <- Nodes, rt:is_ready(Node) =:= true], + ReadyNodes = [Node || Node <- Nodes, rt_node:is_ready(Node) =:= true], case ReadyNodes of Nodes -> ok; @@ -218,7 +251,7 @@ try_nodes_ready(Nodes, N, SleepMs) -> %% @doc Stop nodes and wipe out their data directories clean_cluster(Nodes) when is_list(Nodes) -> - [rt:stop_and_wait(Node) || Node <- Nodes], + [rt_node:stop_and_wait(Node) || Node <- Nodes], clean_data_dir(Nodes). clean_data_dir(Nodes) -> @@ -227,24 +260,19 @@ clean_data_dir(Nodes) -> clean_data_dir(Nodes, SubDir) when not is_list(Nodes) -> clean_data_dir([Nodes], SubDir); clean_data_dir(Nodes, SubDir) when is_list(Nodes) -> - ?HARNESS:clean_data_dir(Nodes, SubDir). + rt_harness:clean_data_dir(Nodes, SubDir). %% @doc Shutdown every node, this is for after a test run is complete. teardown() -> %% stop all connected nodes, 'cause it'll be faster that %%lager:info("RPC stopping these nodes ~p", [nodes()]), - %%[ rt:stop(Node) || Node <- nodes()], + %%[ rt_node:stop(Node) || Node <- nodes()], %% Then do the more exhaustive harness thing, in case something was up %% but not connected. - ?HARNESS:teardown(). + rt_harness:teardown(). versions() -> - ?HARNESS:versions(). - -config() -> - [{riak_core, [{handoff_concurrency, 11}]}, - {riak_search, [{enabled, true}]}, - {riak_pipe, [{worker_limit, 200}]}]. + rt_harness:versions(). augment_config(Section, Property, Config) -> UpdSectionConfig = update_section(Section, @@ -255,4 +283,4 @@ augment_config(Section, Property, Config) -> update_section(Section, Property, false) -> {Section, [Property]}; update_section(Section, Property, {Section, SectionConfig}) -> - {Section, [Property | SectionConfig]}. + {Section, [Property | SectionConfig]}. \ No newline at end of file From 50f000b388fcbd16f54cb02bbdb5c94d9a0cc235 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Fri, 13 Feb 2015 18:24:25 -0500 Subject: [PATCH 060/157] Additional rebase cleanup --- src/rt.erl | 5 +---- tests/overload.erl | 1 - tests/replication/repl_bucket_types.erl | 14 ++++++++++++ tests/verify_2i_aae.erl | 29 ++++++++++++++++++------- 4 files changed, 36 insertions(+), 13 deletions(-) diff --git a/src/rt.erl b/src/rt.erl index 85a48e9a5..9a64b4a2a 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -1044,9 +1044,6 @@ claimant_according_to(Node) -> BadRpc end. -%%%=================================================================== -======= - %%%=================================================================== %%% Cluster Utility Functions %%%=================================================================== @@ -1154,8 +1151,8 @@ teardown() -> versions() -> ?HARNESS:versions(). + %%%=================================================================== ->>>>>>> WIP: Support execution of old and new style tests %%% Basic Read/Write Functions %%%=================================================================== diff --git a/tests/overload.erl b/tests/overload.erl index c1d27cbc9..70d99cc91 100644 --- a/tests/overload.erl +++ b/tests/overload.erl @@ -1,4 +1,3 @@ -%% ------------------------------------------------------------------- %% %% Copyright (c) 2013 Basho Technologies, Inc. %% diff --git a/tests/replication/repl_bucket_types.erl b/tests/replication/repl_bucket_types.erl index c14640994..50a0885c4 100644 --- a/tests/replication/repl_bucket_types.erl +++ b/tests/replication/repl_bucket_types.erl @@ -13,12 +13,15 @@ -define(ENSURE_READ_INTERVAL, 1000). %% Replication Bucket Types test +<<<<<<< HEAD %% setup(Type) -> rt:set_conf(all, [{"buckets.default.allow_mult", "false"}]), {LeaderA, LeaderB, ANodes, BNodes} = ClusterNodes = make_clusters(Type), +======= +>>>>>>> Additional rebase cleanup PBA = rt:pbc(LeaderA), PBB = rt:pbc(LeaderB), @@ -51,6 +54,7 @@ setup(Type) -> connect_clusters(LeaderA, LeaderB), {ClusterNodes, Types, PBA, PBB}. +<<<<<<< HEAD cleanup({ClusterNodes, _Types, PBA, PBB}, CleanCluster) -> riakc_pb_socket:stop(PBA), riakc_pb_socket:stop(PBB), @@ -64,6 +68,8 @@ cleanup({ClusterNodes, _Types, PBA, PBB}, CleanCluster) -> %% @doc riak_test entry point confirm() -> +======= +>>>>>>> Additional rebase cleanup %% Test two clusters of the current version SetupData = setup(current), realtime_test(SetupData), @@ -346,11 +352,19 @@ cluster_conf() -> ]} ]. +<<<<<<< HEAD deploy_nodes(NumNodes, current) -> rt:deploy_nodes(NumNodes, cluster_conf(), [riak_kv, riak_repl]); deploy_nodes(_, mixed) -> Conf = cluster_conf(), rt:deploy_nodes([{current, Conf}, {previous, Conf}], [riak_kv, riak_repl]). +======= +%% deploy_nodes(NumNodes, current) -> +%% rt_cluster:deploy_nodes(NumNodes, cluster_conf()); +%% deploy_nodes(_, mixed) -> +%% Conf = cluster_conf(), +%% rt_cluster:deploy_nodes([{current, Conf}, {previous, Conf}]). +>>>>>>> Additional rebase cleanup %% @doc Create two clusters of 1 node each and connect them for replication: %% Cluster "A" -> cluster "B" diff --git a/tests/verify_2i_aae.erl b/tests/verify_2i_aae.erl index 55e2bde5c..54866de2b 100644 --- a/tests/verify_2i_aae.erl +++ b/tests/verify_2i_aae.erl @@ -33,13 +33,26 @@ -define(SCAN_BATCH_SIZE, 100). -define(N_VAL, 3). -confirm() -> - [Node1] = rt_cluster:build_cluster(1, - [{riak_kv, - [{anti_entropy, {off, []}}, - {anti_entropy_build_limit, {100, 500}}, - {anti_entropy_concurrency, 100}, - {anti_entropy_tick, 200}]}]), +-export([properties/0, + confirm/1]). + +properties() -> + Config = [{riak_kv, + [{anti_entropy, {off, []}}, + {anti_entropy_build_limit, {100, 500}}, + {anti_entropy_concurrency, 100}, + {anti_entropy_tick, 200}]}] ++ rt_properties:default_config(), + rt_properties:new([{node_count, 1}, + {make_cluster, false}, + {config, Config}]). + +-spec confirm(rt_properties:properties()) -> pass | fail. +confirm(Properties) -> + NodeIds = rt_properties:get(node_ids, Properties), + NodeMap = rt_properties:get(node_map, Properties), + Nodes = [rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds], + Node1 = hd(Nodes), + rt_intercept:load_code(Node1), rt_intercept:add(Node1, {riak_object, @@ -228,4 +241,4 @@ assert_range_query(Pid, Bucket, Expected0, Index, StartValue, EndValue) -> end, Expected = lists:sort(Expected0), ?assertEqual({Bucket, Expected}, {Bucket, Actual}), - lager:info("Yay! ~b (actual) == ~b (expected)", [length(Actual), length(Expected)]). + lager:info("Yay! ~b (actual) == ~b (expected)", [length(Actual), length(Expected)]). \ No newline at end of file From 3313111dd732a8538111a81aec1043716872c816 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Tue, 17 Feb 2015 12:49:02 -0500 Subject: [PATCH 061/157] - Fixes a rebase error in repl_bucket_types --- tests/replication/repl_bucket_types.erl | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/tests/replication/repl_bucket_types.erl b/tests/replication/repl_bucket_types.erl index 50a0885c4..c14640994 100644 --- a/tests/replication/repl_bucket_types.erl +++ b/tests/replication/repl_bucket_types.erl @@ -13,15 +13,12 @@ -define(ENSURE_READ_INTERVAL, 1000). %% Replication Bucket Types test -<<<<<<< HEAD %% setup(Type) -> rt:set_conf(all, [{"buckets.default.allow_mult", "false"}]), {LeaderA, LeaderB, ANodes, BNodes} = ClusterNodes = make_clusters(Type), -======= ->>>>>>> Additional rebase cleanup PBA = rt:pbc(LeaderA), PBB = rt:pbc(LeaderB), @@ -54,7 +51,6 @@ setup(Type) -> connect_clusters(LeaderA, LeaderB), {ClusterNodes, Types, PBA, PBB}. -<<<<<<< HEAD cleanup({ClusterNodes, _Types, PBA, PBB}, CleanCluster) -> riakc_pb_socket:stop(PBA), riakc_pb_socket:stop(PBB), @@ -68,8 +64,6 @@ cleanup({ClusterNodes, _Types, PBA, PBB}, CleanCluster) -> %% @doc riak_test entry point confirm() -> -======= ->>>>>>> Additional rebase cleanup %% Test two clusters of the current version SetupData = setup(current), realtime_test(SetupData), @@ -352,19 +346,11 @@ cluster_conf() -> ]} ]. -<<<<<<< HEAD deploy_nodes(NumNodes, current) -> rt:deploy_nodes(NumNodes, cluster_conf(), [riak_kv, riak_repl]); deploy_nodes(_, mixed) -> Conf = cluster_conf(), rt:deploy_nodes([{current, Conf}, {previous, Conf}], [riak_kv, riak_repl]). -======= -%% deploy_nodes(NumNodes, current) -> -%% rt_cluster:deploy_nodes(NumNodes, cluster_conf()); -%% deploy_nodes(_, mixed) -> -%% Conf = cluster_conf(), -%% rt_cluster:deploy_nodes([{current, Conf}, {previous, Conf}]). ->>>>>>> Additional rebase cleanup %% @doc Create two clusters of 1 node each and connect them for replication: %% Cluster "A" -> cluster "B" From 2ddb1b860313f029159e0ea68fec48144670388d Mon Sep 17 00:00:00 2001 From: John Burwell Date: Tue, 17 Feb 2015 16:57:57 -0500 Subject: [PATCH 062/157] Modifies node_manager to deploy zero nodes for old style tests an advance the state of the riak_test_runner FSM --- src/node_manager.erl | 2 ++ src/riak_test_runner.erl | 17 ++++++++--------- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/src/node_manager.erl b/src/node_manager.erl index d4d2bc2ee..6e7df3ec7 100644 --- a/src/node_manager.erl +++ b/src/node_manager.erl @@ -199,6 +199,8 @@ version_available_fun(Count, VersionMap) -> end end. +deploy([], _NodeMap, _Version, _Config, []) -> + []; deploy(Nodes, NodeMap, Version, Config, Services) -> rt_harness_util:deploy_nodes(Nodes, NodeMap, Version, Config, Services). diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index d4816eef2..b007a4472 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -165,23 +165,22 @@ setup(timeout, State=#state{test_type=TestType, {0, UName} = rt:cmd("uname -a"), lager:info("Test Runner: ~s", [UName]), - NodeIds = rt_properties:get(node_ids, Properties), - Services = rt_properties:get(required_services, Properties), {StartVersion, OtherVersions} = test_versions(Properties), Config = rt_backend:set(Backend, rt_properties:get(config, Properties)), case TestType of new -> - node_manager:deploy_nodes(NodeIds, - StartVersion, - Config, - Services, - notify_fun(self())), - lager:info("Waiting for deploy nodes response at ~p", [self()]); + NodeIds = rt_properties:get(node_ids, Properties), + Services = rt_properties:get(required_services, Properties); old -> - lager:warn("Test ~p has not been ported to the new framework.", [TestModule]) + NodeIds = [], + Services = [], + lager:warning("Test ~p has not been ported to the new framework.", [TestModule]) end, + node_manager:deploy_nodes(NodeIds, StartVersion, Config, Services, notify_fun(self())), + lager:info("Waiting for deploy nodes response at ~p", [self()]), + %% Set the initial value for `current_version' in the properties record {ok, UpdProperties} = rt_properties:set(current_version, StartVersion, Properties), From bdc9f4424db5e315ef272d371504ba7f68fdc354 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Wed, 18 Feb 2015 14:10:43 -0500 Subject: [PATCH 063/157] Ports more rt functions and fixes the verify_build_cluster test case --- src/rt.erl | 306 ++++++--------------------------- tests/verify_build_cluster.erl | 5 +- tests/verify_down.erl | 20 +-- 3 files changed, 67 insertions(+), 264 deletions(-) diff --git a/src/rt.erl b/src/rt.erl index 9a64b4a2a..f9e0fdb2d 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -291,7 +291,7 @@ deploy_nodes(Versions, Services) -> Nodes. deploy_nodes(NumNodes, InitialConfig, Services) when is_integer(NumNodes) -> - NodeConfig = [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)], + NodeConfig = [{head, InitialConfig} || _ <- lists:seq(1,NumNodes)], deploy_nodes(NodeConfig, Services). deploy_nodes(NodeIds, NodeMap, Version, Config, Services) -> @@ -320,7 +320,7 @@ allocate_nodes(NumNodes) -> [AllocatedNodeIds, AllocatedNodeMap, VersionMap]. version_to_config(Config) when is_tuple(Config)-> Config; -version_to_config(Version) -> {Version, default}. +version_to_config(Version) -> {Version, head}. deploy_clusters(Settings) -> ClusterConfigs = [case Setting of @@ -345,7 +345,9 @@ build_clusters(Settings) -> %% @doc Start the specified Riak node start(Node) -> - ?HARNESS:start(Node). + %% TODO Determine the best way to work with versions. + %% For now, we are using the default version ... -jsb + rt_harness:start(Node, head). %% @doc Start the specified Riak `Node' and wait for it to be pingable start_and_wait(Node) -> @@ -357,10 +359,9 @@ async_start(Node) -> %% @doc Stop the specified Riak `Node'. stop(Node) -> - lager:info("Stopping riak on ~p", [Node]), - timer:sleep(10000), %% I know, I know! - ?HARNESS:stop(Node). - %%rpc:call(Node, init, stop, []). + %% TODO Determine the best way to work with versions. + %% For now, we are using the default version ... -jsb + rt_node:stop(Node, head). %% @doc Stop the specified Riak `Node' and wait until it is not pingable stop_and_wait(Node) -> @@ -393,137 +394,78 @@ slow_upgrade(Node, NewVersion, Nodes) -> %% @doc Have `Node' send a join request to `PNode' join(Node, PNode) -> - R = rpc:call(Node, riak_core, join, [PNode]), - lager:info("[join] ~p to (~p): ~p", [Node, PNode, R]), - ?assertEqual(ok, R), - ok. + rt_node:join(Node, PNode). %% @doc Have `Node' send a join request to `PNode' staged_join(Node, PNode) -> - R = rpc:call(Node, riak_core, staged_join, [PNode]), - lager:info("[join] ~p to (~p): ~p", [Node, PNode, R]), - ?assertEqual(ok, R), - ok. + rt_node:staged_join(Node, PNode). plan_and_commit(Node) -> - timer:sleep(500), - lager:info("planning and commiting cluster join"), - case rpc:call(Node, riak_core_claimant, plan, []) of - {error, ring_not_ready} -> - lager:info("plan: ring not ready"), - timer:sleep(100), - plan_and_commit(Node); - {ok, _, _} -> - lager:info("plan: done"), - do_commit(Node) - end. + rt_node:plan_and_comment(Node). do_commit(Node) -> - case rpc:call(Node, riak_core_claimant, commit, []) of - {error, plan_changed} -> - lager:info("commit: plan changed"), - timer:sleep(100), - maybe_wait_for_changes(Node), - plan_and_commit(Node); - {error, ring_not_ready} -> - lager:info("commit: ring not ready"), - timer:sleep(100), - maybe_wait_for_changes(Node), - do_commit(Node); - {error,nothing_planned} -> - %% Assume plan actually committed somehow - ok; - ok -> - ok - end. + rt_node:do_commit(Node). maybe_wait_for_changes(Node) -> - Ring = get_ring(Node), - Changes = riak_core_ring:pending_changes(Ring), - Joining = riak_core_ring:members(Ring, [joining]), - lager:info("maybe_wait_for_changes, changes: ~p joining: ~p", - [Changes, Joining]), - if Changes =:= [] -> - ok; - Joining =/= [] -> - ok; - true -> - ok = wait_until_no_pending_changes([Node]) - end. + rt2:maybe_wait_for_changes(Node). %% @doc Have the `Node' leave the cluster leave(Node) -> - R = rpc:call(Node, riak_core, leave, []), - lager:info("[leave] ~p: ~p", [Node, R]), - ?assertEqual(ok, R), - ok. + rt_node:leave(Node). %% @doc Have `Node' remove `OtherNode' from the cluster remove(Node, OtherNode) -> - ?assertEqual(ok, - rpc:call(Node, riak_kv_console, remove, [[atom_to_list(OtherNode)]])). + rt_node:remove(Node, OtherNode). %% @doc Have `Node' mark `OtherNode' as down down(Node, OtherNode) -> - rpc:call(Node, riak_kv_console, down, [[atom_to_list(OtherNode)]]). + rt_node:down(Node, OtherNode). %% @doc partition the `P1' from `P2' nodes %% note: the nodes remained connected to riak_test@local, %% which is how `heal/1' can still work. partition(P1, P2) -> - OldCookie = rpc:call(hd(P1), erlang, get_cookie, []), - NewCookie = list_to_atom(lists:reverse(atom_to_list(OldCookie))), - [true = rpc:call(N, erlang, set_cookie, [N, NewCookie]) || N <- P1], - [[true = rpc:call(N, erlang, disconnect_node, [P2N]) || N <- P1] || P2N <- P2], - wait_until_partitioned(P1, P2), - {NewCookie, OldCookie, P1, P2}. + rt_node:partition(P1, P2). %% @doc heal the partition created by call to `partition/2' %% `OldCookie' is the original shared cookie -heal({_NewCookie, OldCookie, P1, P2}) -> - Cluster = P1 ++ P2, - % set OldCookie on P1 Nodes - [true = rpc:call(N, erlang, set_cookie, [N, OldCookie]) || N <- P1], - wait_until_connected(Cluster), - {_GN, []} = rpc:sbcast(Cluster, riak_core_node_watcher, broadcast), - ok. +heal({NewCookie, OldCookie, P1, P2}) -> + rt_node:heal({NewCookie, OldCookie, P1, P2}). %% @doc Spawn `Cmd' on the machine running the test harness spawn_cmd(Cmd) -> - ?HARNESS:spawn_cmd(Cmd). + rt2:spawn_cmd(Cmd). %% @doc Spawn `Cmd' on the machine running the test harness spawn_cmd(Cmd, Opts) -> - ?HARNESS:spawn_cmd(Cmd, Opts). + rt2:spawn_cmd(Cmd, Opts). %% @doc Wait for a command spawned by `spawn_cmd', returning %% the exit status and result wait_for_cmd(CmdHandle) -> - ?HARNESS:wait_for_cmd(CmdHandle). + rt2:wait_for_cmd(CmdHandle). %% @doc Spawn `Cmd' on the machine running the test harness, returning %% the exit status and result cmd(Cmd) -> - ?HARNESS:cmd(Cmd). + rt2:cmd(Cmd). %% @doc Spawn `Cmd' on the machine running the test harness, returning %% the exit status and result cmd(Cmd, Opts) -> - ?HARNESS:cmd(Cmd, Opts). + rt2:cmd(Cmd, Opts). %% @doc pretty much the same as os:cmd/1 but it will stream the output to lager. %% If you're running a long running command, it will dump the output %% once per second, as to not create the impression that nothing is happening. -spec stream_cmd(string()) -> {integer(), string()}. stream_cmd(Cmd) -> - Port = open_port({spawn, binary_to_list(iolist_to_binary(Cmd))}, [stream, stderr_to_stdout, exit_status]), - stream_cmd_loop(Port, "", "", now()). + rt2:stream_cmd(Cmd). %% @doc same as rt:stream_cmd/1, but with options, like open_port/2 -spec stream_cmd(string(), string()) -> {integer(), string()}. stream_cmd(Cmd, Opts) -> - Port = open_port({spawn, binary_to_list(iolist_to_binary(Cmd))}, [stream, stderr_to_stdout, exit_status] ++ Opts), - stream_cmd_loop(Port, "", "", now()). + rt2:stream_cmd(Cmd, Opts). stream_cmd_loop(Port, Buffer, NewLineBuffer, Time={_MegaSecs, Secs, _MicroSecs}) -> receive @@ -555,19 +497,8 @@ stream_cmd_loop(Port, Buffer, NewLineBuffer, Time={_MegaSecs, Secs, _MicroSecs}) %%%=================================================================== %%% Remote code management %%%=================================================================== -load_modules_on_nodes([], Nodes) - when is_list(Nodes) -> - ok; -load_modules_on_nodes([Module | MoreModules], Nodes) - when is_list(Nodes) -> - case code:get_object_code(Module) of - {Module, Bin, File} -> - {_, []} = rpc:multicall(Nodes, code, load_binary, [Module, File, Bin]); - error -> - error(lists:flatten(io_lib:format("unable to get_object_code(~s)", [Module]))) - end, - load_modules_on_nodes(MoreModules, Nodes). - +load_modules_on_nodes(Modules, Nodes) -> + rt2:load_modules_on_nodes(Modules, Nodes). %%%=================================================================== %%% Status / Wait Functions @@ -575,27 +506,14 @@ load_modules_on_nodes([Module | MoreModules], Nodes) %% @doc Is the `Node' up according to net_adm:ping is_pingable(Node) -> - net_adm:ping(Node) =:= pong. + rt_node:is_pingable(Node). -is_mixed_cluster(Nodes) when is_list(Nodes) -> - %% If the nodes are bad, we don't care what version they are - {Versions, _BadNodes} = rpc:multicall(Nodes, init, script_id, [], rt_config:get(rt_max_wait_time)), - length(lists:usort(Versions)) > 1; -is_mixed_cluster(Node) -> - Nodes = rpc:call(Node, erlang, nodes, []), - is_mixed_cluster(Nodes). +is_mixed_cluster(Nodes) -> + rt2:is_mixed_cluster(Nodes). %% @private is_ready(Node) -> - case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of - {ok, Ring} -> - case lists:member(Node, riak_core_ring:ready_members(Ring)) of - true -> true; - false -> {not_ready, Node} - end; - Other -> - Other - end. + rt_node:is_ready(Node). %% @private is_ring_ready(Node) -> @@ -612,10 +530,7 @@ is_ring_ready(Node) -> %% provided `rt_max_wait_time' and `rt_retry_delay' parameters in %% specified `riak_test' config file. wait_until(Fun) when is_function(Fun) -> - MaxTime = rt_config:get(rt_max_wait_time), - Delay = rt_config:get(rt_retry_delay), - Retry = MaxTime div Delay, - wait_until(Fun, Retry, Delay). + rt2:wait_until(Fun). %% @doc Convenience wrapper for wait_until for the myriad functions that %% take a node as single argument. @@ -626,96 +541,43 @@ wait_until(Node, Fun) when is_atom(Node), is_function(Fun) -> %% milliseconds between retries. This is our eventual consistency bread %% and butter wait_until(Fun, Retry, Delay) when Retry > 0 -> - Res = Fun(), - case Res of - true -> - ok; - _ when Retry == 1 -> - {fail, Res}; - _ -> - timer:sleep(Delay), - wait_until(Fun, Retry-1, Delay) - end. + rt2:wait_until(Fun, Retry, Delay). %% @doc Wait until the specified node is considered ready by `riak_core'. %% As of Riak 1.0, a node is ready if it is in the `valid' or `leaving' %% states. A ready node is guaranteed to have current preflist/ownership %% information. wait_until_ready(Node) -> - lager:info("Wait until ~p ready", [Node]), - ?assertEqual(ok, wait_until(Node, fun is_ready/1)), - ok. + rt2:wait_until_ready(Node). %% @doc Wait until status can be read from riak_kv_console wait_until_status_ready(Node) -> - lager:info("Wait until status ready in ~p", [Node]), - ?assertEqual(ok, wait_until(Node, - fun(_) -> - case rpc:call(Node, riak_kv_console, status, [[]]) of - ok -> - true; - Res -> - Res - end - end)). + rt2:wait_until_status_ready(Node). %% @doc Given a list of nodes, wait until all nodes believe there are no %% on-going or pending ownership transfers. -spec wait_until_no_pending_changes([node()]) -> ok | fail. wait_until_no_pending_changes(Nodes) -> - lager:info("Wait until no pending changes on ~p", [Nodes]), - F = fun() -> - rpc:multicall(Nodes, riak_core_vnode_manager, force_handoffs, []), - {Rings, BadNodes} = rpc:multicall(Nodes, riak_core_ring_manager, get_raw_ring, []), - Changes = [ riak_core_ring:pending_changes(Ring) =:= [] || {ok, Ring} <- Rings ], - BadNodes =:= [] andalso length(Changes) =:= length(Nodes) andalso lists:all(fun(T) -> T end, Changes) - end, - ?assertEqual(ok, wait_until(F)), - ok. + rt2:wait_until_no_pending_changes(Nodes). %% @doc Waits until no transfers are in-flight or pending, checked by %% riak_core_status:transfers(). -spec wait_until_transfers_complete([node()]) -> ok | fail. -wait_until_transfers_complete([Node0|_]) -> - lager:info("Wait until transfers complete ~p", [Node0]), - F = fun(Node) -> - {DownNodes, Transfers} = rpc:call(Node, riak_core_status, transfers, []), - DownNodes =:= [] andalso Transfers =:= [] - end, - ?assertEqual(ok, wait_until(Node0, F)), - ok. +wait_until_transfers_complete(Nodes) -> + rt2:wait_until_transfers_complete(Nodes). wait_for_service(Node, Services) when is_list(Services) -> - F = fun(N) -> - case rpc:call(N, riak_core_node_watcher, services, [N]) of - {badrpc, Error} -> - {badrpc, Error}; - CurrServices when is_list(CurrServices) -> - lists:all(fun(Service) -> lists:member(Service, CurrServices) end, Services); - Res -> - Res - end - end, - ?assertEqual(ok, wait_until(Node, F)), - ok; + rt2:wait_for_service(Node, Services); wait_for_service(Node, Service) -> wait_for_service(Node, [Service]). wait_for_cluster_service(Nodes, Service) -> - lager:info("Wait for cluster service ~p in ~p", [Service, Nodes]), - F = fun(N) -> - UpNodes = rpc:call(N, riak_core_node_watcher, nodes, [Service]), - (Nodes -- UpNodes) == [] - end, - [?assertEqual(ok, wait_until(Node, F)) || Node <- Nodes], - ok. + rt2:wait_for_cluster_service(Nodes, Service). %% @doc Given a list of nodes, wait until all nodes are considered ready. %% See {@link wait_until_ready/1} for definition of ready. wait_until_nodes_ready(Nodes) -> - lager:info("Wait until nodes are ready : ~p", [Nodes]), - [?assertEqual(ok, wait_until(Node, fun is_ready/1)) || Node <- Nodes], - ok. + rt_node:wait_until_nodes_ready(Nodes). %% @doc Wait until all nodes in the list `Nodes' believe each other to be %% members of the cluster. @@ -725,26 +587,12 @@ wait_until_all_members(Nodes) -> %% @doc Wait until all nodes in the list `Nodes' believes all nodes in the %% list `Members' are members of the cluster. wait_until_all_members(Nodes, ExpectedMembers) -> - lager:info("Wait until all members ~p ~p", [Nodes, ExpectedMembers]), - S1 = ordsets:from_list(ExpectedMembers), - F = fun(Node) -> - case members_according_to(Node) of - {badrpc, _} -> - false; - ReportedMembers -> - S2 = ordsets:from_list(ReportedMembers), - ordsets:is_subset(S1, S2) - end - end, - [?assertEqual(ok, wait_until(Node, F)) || Node <- Nodes], - ok. + rt2:wait_until_all_members(Nodes, ExpectedMembers). %% @doc Given a list of nodes, wait until all nodes believe the ring has %% converged (ie. `riak_core_ring:is_ready' returns `true'). wait_until_ring_converged(Nodes) -> - lager:info("Wait until ring converged on ~p", [Nodes]), - [?assertEqual(ok, wait_until(Node, fun is_ring_ready/1)) || Node <- Nodes], - ok. + rt2:wait_until_ring_converged(Nodes). wait_until_legacy_ringready(Node) -> lager:info("Wait until legacy ring ready on ~p", [Node]), @@ -770,29 +618,11 @@ wait_until_connected(Nodes) -> %% @doc Wait until the specified node is pingable wait_until_pingable(Node) -> - lager:info("Wait until ~p is pingable", [Node]), - F = fun(N) -> - net_adm:ping(N) =:= pong - end, - ?assertEqual(ok, wait_until(Node, F)), - ok. + rt2:wait_until_pingable(Node). %% @doc Wait until the specified node is no longer pingable wait_until_unpingable(Node) -> - lager:info("Wait until ~p is not pingable", [Node]), - _OSPidToKill = rpc:call(Node, os, getpid, []), - F = fun() -> net_adm:ping(Node) =:= pang end, - %% riak stop will kill -9 after 5 mins, so we try to wait at least that - %% amount of time. - Delay = rt_config:get(rt_retry_delay), - Retry = lists:max([360000, rt_config:get(rt_max_wait_time)]) div Delay, - case wait_until(F, Retry, Delay) of - ok -> ok; - _ -> - lager:error("Timed out waiting for node ~p to shutdown", [Node]), - ?assert(node_shutdown_timed_out) - end. - + rt2:wait_until_unpingable(Node). % Waits until a certain registered name pops up on the remote node. wait_until_registered(Node, Name) -> @@ -840,27 +670,17 @@ brutal_kill(Node) -> rpc:cast(Node, os, cmd, [io_lib:format("kill -15 ~s", [OSPidToKill])]), ok. -capability(Node, all) -> - rpc:call(Node, riak_core_capability, all, []); capability(Node, Capability) -> - rpc:call(Node, riak_core_capability, get, [Capability]). + rt2:capability(Node, Capability). capability(Node, Capability, Default) -> - rpc:call(Node, riak_core_capability, get, [Capability, Default]). + rt2:capability(Node, Capability, Default). wait_until_capability(Node, Capability, Value) -> - rt:wait_until(Node, - fun(_) -> - cap_equal(Value, capability(Node, Capability)) - end). + rt2:wait_until_capability(Node, Capability, Value). wait_until_capability(Node, Capability, Value, Default) -> - rt:wait_until(Node, - fun(_) -> - Cap = capability(Node, Capability, Default), - io:format("capability is ~p ~p",[Node, Cap]), - cap_equal(Value, Cap) - end). + rt2:wait_until_capability(Node, Capability, Value, Default). cap_equal(Val, Cap) when is_list(Cap) -> lists:sort(Cap) == lists:sort(Val); @@ -876,9 +696,7 @@ wait_until_owners_according_to(Node, Nodes) -> ok. wait_until_nodes_agree_about_ownership(Nodes) -> - lager:info("Wait until nodes agree about ownership ~p", [Nodes]), - Results = [ wait_until_owners_according_to(Node, Nodes) || Node <- Nodes ], - ?assert(lists:all(fun(X) -> ok =:= X end, Results)). + rt_node:wait_until_nodes_agree_about_ownership(Nodes). %% AAE support wait_until_aae_trees_built(Nodes) -> @@ -923,13 +741,7 @@ all_aae_trees_built(Node, Partitions) -> end. get_aae_tree_info(Node) -> - case rpc:call(Node, riak_kv_entropy_info, compute_tree_info, []) of - {badrpc, _} -> - {error, {badrpc, Node}}; - Info -> - lager:debug("Entropy table on node ~p : ~p", [Node, Info]), - {ok, Info} - end. + rt_aae:get_aae_tree_info(Node). all_trees_have_build_times(Info) -> not lists:keymember(undefined, 2, Info). @@ -965,11 +777,7 @@ index_built_fun(Node) -> %% @doc Ensure that the specified node is a singleton node/cluster -- a node %% that owns 100% of the ring. check_singleton_node(Node) -> - lager:info("Check ~p is a singleton", [Node]), - {ok, Ring} = rpc:call(Node, riak_core_ring_manager, get_raw_ring, []), - Owners = lists:usort([Owner || {_Idx, Owner} <- riak_core_ring:all_owners(Ring)]), - ?assertEqual([Node], Owners), - ok. + rt_ring:check_singleton_node(Node). % @doc Get list of partitions owned by node (primary). partitions_for_node(Node) -> @@ -1025,13 +833,7 @@ nearest_ringsize(Count, Power) -> %% @doc Return the cluster status of `Member' according to the ring %% retrieved from `Node'. status_of_according_to(Member, Node) -> - case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of - {ok, Ring} -> - Status = riak_core_ring:member_status(Ring, Member), - Status; - {badrpc, _}=BadRpc -> - BadRpc - end. + rt_ring:status_of_according_to(Member, Node). %% @doc Return a list of nodes that own partitions according to the ring %% retrieved from the specified node. diff --git a/tests/verify_build_cluster.erl b/tests/verify_build_cluster.erl index da72b420c..d11bcff74 100644 --- a/tests/verify_build_cluster.erl +++ b/tests/verify_build_cluster.erl @@ -19,7 +19,7 @@ %% ------------------------------------------------------------------- -module(verify_build_cluster). -export([properties/0, - confirm/2]). + confirm/1]). -include("rt.hrl"). -include_lib("eunit/include/eunit.hrl"). @@ -33,7 +33,8 @@ properties() -> {make_cluster, false}, {start_version, previous}]). -confirm(Properties, _MD) -> +-spec confirm(rt_properties:properties()) -> pass | fail. +confirm(Properties) -> [Node1, Node2, Node3, Node4] = Nodes = rt_properties:get(nodes, Properties), lager:info("Loading some data up in this cluster."), diff --git a/tests/verify_down.erl b/tests/verify_down.erl index 33ca503ea..a2c21f7a2 100644 --- a/tests/verify_down.erl +++ b/tests/verify_down.erl @@ -23,24 +23,24 @@ -include_lib("eunit/include/eunit.hrl"). confirm() -> - Nodes = rt_cluster:deploy_nodes(3), + Nodes = rt:deploy_nodes(3), [Node1, Node2, Node3] = Nodes, %% Join node2 to node1 and wait for cluster convergence lager:info("Join ~p to ~p", [Node2, Node1]), - rt_node:join(Node2, Node1), - ?assertEqual(ok, rt_node:wait_until_nodes_ready([Node1, Node2])), + rt:join(Node2, Node1), + ?assertEqual(ok, rt:wait_until_nodes_ready([Node1, Node2])), ?assertEqual(ok, rt:wait_until_no_pending_changes([Node1, Node2])), %% Shutdown node2 lager:info("Stopping ~p", [Node2]), - rt_node:stop(Node2), + rt:stop(Node2), ?assertEqual(ok, rt:wait_until_unpingable(Node2)), Remaining = Nodes -- [Node2], %% Join node3 to node1 lager:info("Join ~p to ~p", [Node3, Node1]), - rt_node:join(Node3, Node1), + rt:join(Node3, Node1), ?assertEqual(ok, rt:wait_until_all_members(Remaining, [Node3])), %% Ensure node3 remains in the joining state @@ -49,7 +49,7 @@ confirm() -> %% Mark node2 as down and wait for ring convergence lager:info("Mark ~p as down", [Node2]), - rt_node:down(Node1, Node2), + rt:down(Node1, Node2), ?assertEqual(ok, rt:wait_until_ring_converged(Remaining)), [?assertEqual(down, rt:status_of_according_to(Node2, Node)) || Node <- Remaining], @@ -58,11 +58,11 @@ confirm() -> %% Restart node2 and wait for ring convergence lager:info("Restart ~p and wait for ring convergence", [Node2]), - rt_node:start(Node2), - ?assertEqual(ok, rt_node:wait_until_nodes_ready([Node2])), + rt:start(Node2), + ?assertEqual(ok, rt:wait_until_nodes_ready([Node2])), ?assertEqual(ok, rt:wait_until_ring_converged(Nodes)), %% Verify that all three nodes are ready lager:info("Ensure all nodes are ready"), - ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes)), - pass. + ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), + pass. \ No newline at end of file From 1e2ffb61bf802a948bcdd31a128c2a67b0c62aa2 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Wed, 18 Feb 2015 19:15:58 -0500 Subject: [PATCH 064/157] Redirects function calls in the rt module to use their analogs in the restructured modules --- src/rt.erl | 760 +++++++----------------------------------------- src/rt_ring.erl | 1 + 2 files changed, 104 insertions(+), 657 deletions(-) diff --git a/src/rt.erl b/src/rt.erl index f9e0fdb2d..91972f04e 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -183,8 +183,6 @@ str(String, Substr) -> rt2:str(String, Substr). -spec set_conf(atom(), [{string(), string()}]) -> ok. -set_conf(all, NameValuePairs) -> - ?HARNESS:set_conf(all, NameValuePairs); set_conf(Node, NameValuePairs) -> stop(Node), ?assertEqual(ok, rt:wait_until_unpingable(Node)), @@ -213,15 +211,8 @@ update_app_config(Node, Config) -> %% @doc Helper that returns first successful application get_env result, %% used when different versions of Riak use different app vars for %% the same setting. -rpc_get_env(_, []) -> - undefined; -rpc_get_env(Node, [{App,Var}|Others]) -> - case rpc:call(Node, application, get_env, [App, Var]) of - {ok, Value} -> - {ok, Value}; - _ -> - rpc_get_env(Node, Others) - end. +rpc_get_env(Node, AppVars) -> + rt2:rpc_get_env(Node, AppVars). -type interface() :: {http, tuple()} | {pb, tuple()}. -type interfaces() :: [interface()]. @@ -233,38 +224,15 @@ connection_info(Node) -> -spec get_pb_conn_info(node()) -> [{inet:ip_address(), pos_integer()}]. get_pb_conn_info(Node) -> - case rpc_get_env(Node, [{riak_api, pb}, - {riak_api, pb_ip}, - {riak_kv, pb_ip}]) of - {ok, [{NewIP, NewPort}|_]} -> - {ok, [{NewIP, NewPort}]}; - {ok, PB_IP} -> - {ok, PB_Port} = rpc_get_env(Node, [{riak_api, pb_port}, - {riak_kv, pb_port}]), - {ok, [{PB_IP, PB_Port}]}; - _ -> - undefined - end. + rt_pb:get_pb_conn_info(Node). -spec get_http_conn_info(node()) -> [{inet:ip_address(), pos_integer()}]. get_http_conn_info(Node) -> - case rpc_get_env(Node, [{riak_api, http}, - {riak_core, http}]) of - {ok, [{IP, Port}|_]} -> - {ok, [{IP, Port}]}; - _ -> - undefined - end. + rt_http:get_http_conn_info(Node). -spec get_https_conn_info(node()) -> [{inet:ip_address(), pos_integer()}]. get_https_conn_info(Node) -> - case rpc_get_env(Node, [{riak_api, https}, - {riak_core, https}]) of - {ok, [{IP, Port}|_]} -> - {ok, [{IP, Port}]}; - _ -> - undefined - end. + rt_http:get_https_conn_info(Node). %% @doc Deploy a set of freshly installed Riak nodes, returning a list of the %% nodes deployed. @@ -347,7 +315,7 @@ build_clusters(Settings) -> start(Node) -> %% TODO Determine the best way to work with versions. %% For now, we are using the default version ... -jsb - rt_harness:start(Node, head). + rt_node:start(Node, head). %% @doc Start the specified Riak `Node' and wait for it to be pingable start_and_wait(Node) -> @@ -370,15 +338,18 @@ stop_and_wait(Node) -> %% @doc Upgrade a Riak `Node' to the specified `NewVersion'. upgrade(Node, NewVersion) -> + %% GAP: The new API does not provide an analog to this function. -jsb ?HARNESS:upgrade(Node, NewVersion). %% @doc Upgrade a Riak `Node' to the specified `NewVersion' and update %% the config based on entries in `Config'. upgrade(Node, NewVersion, Config) -> - ?HARNESS:upgrade(Node, NewVersion, Config). + %% TODO Determine the best way to implement the current version specification. -jsb + rt_node:upgrade(Node, head, NewVersion, Config). %% @doc Upgrade a Riak node to a specific version using the alternate %% leave/upgrade/rejoin approach +%% GAP: rt_node does not current provide slow_upgrade(Node, NewVersion, Nodes) -> lager:info("Perform leave/upgrade/join upgrade on ~p", [Node]), lager:info("Leaving ~p", [Node]), @@ -515,15 +486,6 @@ is_mixed_cluster(Nodes) -> is_ready(Node) -> rt_node:is_ready(Node). -%% @private -is_ring_ready(Node) -> - case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of - {ok, Ring} -> - riak_core_ring:ring_ready(Ring); - _ -> - false - end. - %% @doc Utility function used to construct test predicates. Retries the %% function `Fun' until it returns `true', or until the maximum %% number of retries is reached. The retry limit is based on the @@ -595,26 +557,11 @@ wait_until_ring_converged(Nodes) -> rt2:wait_until_ring_converged(Nodes). wait_until_legacy_ringready(Node) -> - lager:info("Wait until legacy ring ready on ~p", [Node]), - rt:wait_until(Node, - fun(_) -> - case rpc:call(Node, riak_kv_status, ringready, []) of - {ok, _Nodes} -> - true; - Res -> - Res - end - end). + rt2:wait_until_legacy_ringready(Node). %% @doc wait until each node in Nodes is disterl connected to each. wait_until_connected(Nodes) -> - lager:info("Wait until connected ~p", [Nodes]), - F = fun(Node) -> - Connected = rpc:call(Node, erlang, nodes, []), - lists:sort(Nodes) == lists:sort([Node]++Connected)--[node()] - end, - [?assertEqual(ok, wait_until(Node, F)) || Node <- Nodes], - ok. + rt2:wait_until_connected(Nodes). %% @doc Wait until the specified node is pingable wait_until_pingable(Node) -> @@ -626,49 +573,18 @@ wait_until_unpingable(Node) -> % Waits until a certain registered name pops up on the remote node. wait_until_registered(Node, Name) -> - lager:info("Wait until ~p is up on ~p", [Name, Node]), - - F = fun() -> - Registered = rpc:call(Node, erlang, registered, []), - lists:member(Name, Registered) - end, - case wait_until(F) of - ok -> - ok; - _ -> - lager:info("The server with the name ~p on ~p is not coming up.", - [Name, Node]), - ?assert(registered_name_timed_out) - end. - + rt2:wait_until_registered(Node, Name). %% Waits until the cluster actually detects that it is partitioned. wait_until_partitioned(P1, P2) -> - lager:info("Waiting until partition acknowledged: ~p ~p", [P1, P2]), - [ begin - lager:info("Waiting for ~p to be partitioned from ~p", [Node, P2]), - wait_until(fun() -> is_partitioned(Node, P2) end) - end || Node <- P1 ], - [ begin - lager:info("Waiting for ~p to be partitioned from ~p", [Node, P1]), - wait_until(fun() -> is_partitioned(Node, P1) end) - end || Node <- P2 ]. + rt2:wait_until_partitioned(P1, P2). is_partitioned(Node, Peers) -> - AvailableNodes = rpc:call(Node, riak_core_node_watcher, nodes, [riak_kv]), - lists:all(fun(Peer) -> not lists:member(Peer, AvailableNodes) end, Peers). + rt2:is_partitioned(Node, Peers). % when you just can't wait brutal_kill(Node) -> - rt_cover:maybe_stop_on_node(Node), - lager:info("Killing node ~p", [Node]), - OSPidToKill = rpc:call(Node, os, getpid, []), - %% try a normal kill first, but set a timer to - %% kill -9 after 5 seconds just in case - rpc:cast(Node, timer, apply_after, - [5000, os, cmd, [io_lib:format("kill -9 ~s", [OSPidToKill])]]), - rpc:cast(Node, os, cmd, [io_lib:format("kill -15 ~s", [OSPidToKill])]), - ok. + rt_node:brutal_kill(Node). capability(Node, Capability) -> rt2:capability(Node, Capability). @@ -682,93 +598,18 @@ wait_until_capability(Node, Capability, Value) -> wait_until_capability(Node, Capability, Value, Default) -> rt2:wait_until_capability(Node, Capability, Value, Default). -cap_equal(Val, Cap) when is_list(Cap) -> - lists:sort(Cap) == lists:sort(Val); cap_equal(Val, Cap) -> - Val == Cap. + rt2:cap_equal(Val, Cap). wait_until_owners_according_to(Node, Nodes) -> - SortedNodes = lists:usort(Nodes), - F = fun(N) -> - owners_according_to(N) =:= SortedNodes - end, - ?assertEqual(ok, wait_until(Node, F)), - ok. + rt_node:wait_until_owners_according_to(Node, Nodes). wait_until_nodes_agree_about_ownership(Nodes) -> rt_node:wait_until_nodes_agree_about_ownership(Nodes). %% AAE support wait_until_aae_trees_built(Nodes) -> - lager:info("Wait until AAE builds all partition trees across ~p", [Nodes]), - BuiltFun = fun() -> lists:foldl(aae_tree_built_fun(), true, Nodes) end, - ?assertEqual(ok, wait_until(BuiltFun)), - ok. - -aae_tree_built_fun() -> - fun(Node, _AllBuilt = true) -> - case get_aae_tree_info(Node) of - {ok, TreeInfos} -> - case all_trees_have_build_times(TreeInfos) of - true -> - Partitions = [I || {I, _} <- TreeInfos], - all_aae_trees_built(Node, Partitions); - false -> - some_trees_not_built - end; - Err -> - Err - end; - (_Node, Err) -> - Err - end. - -% It is unlikely but possible to get a tree built time from compute_tree_info -% but an attempt to use the tree returns not_built. This is because the build -% process has finished, but the lock on the tree won't be released until it -% dies and the manager detects it. Yes, this is super freaking paranoid. -all_aae_trees_built(Node, Partitions) -> - %% Notice that the process locking is spawned by the - %% pmap. That's important! as it should die eventually - %% so the lock is released and the test can lock the tree. - IndexBuilts = rt:pmap(index_built_fun(Node), Partitions), - BadOnes = [R || R <- IndexBuilts, R /= true], - case BadOnes of - [] -> - true; - _ -> - BadOnes - end. - -get_aae_tree_info(Node) -> - rt_aae:get_aae_tree_info(Node). - -all_trees_have_build_times(Info) -> - not lists:keymember(undefined, 2, Info). - -index_built_fun(Node) -> - fun(Idx) -> - case rpc:call(Node, riak_kv_vnode, - hashtree_pid, [Idx]) of - {ok, TreePid} -> - case rpc:call(Node, riak_kv_index_hashtree, - get_lock, [TreePid, for_riak_test]) of - {badrpc, _} -> - {error, {badrpc, Node}}; - TreeLocked when TreeLocked == ok; - TreeLocked == already_locked -> - true; - Err -> - % Either not_built or some unhandled result, - % in which case update this case please! - {error, {index_not_built, Node, Idx, Err}} - end; - {error, _}=Err -> - Err; - {badrpc, _} -> - {error, {badrpc, Node}} - end - end. + rt_aae:wait_until_app_trees_built(Nodes). %%%=================================================================== %%% Ring Functions @@ -781,54 +622,33 @@ check_singleton_node(Node) -> % @doc Get list of partitions owned by node (primary). partitions_for_node(Node) -> - Ring = get_ring(Node), - [Idx || {Idx, Owner} <- riak_core_ring:all_owners(Ring), Owner == Node]. + rt_ring:partitions_for_node(Node). %% @doc Get the raw ring for `Node'. get_ring(Node) -> - {ok, Ring} = rpc:call(Node, riak_core_ring_manager, get_raw_ring, []), - Ring. + rt_ring:get_ring(Node). assert_nodes_agree_about_ownership(Nodes) -> - ?assertEqual(ok, wait_until_ring_converged(Nodes)), - ?assertEqual(ok, wait_until_all_members(Nodes)), - [ ?assertEqual({Node, Nodes}, {Node, owners_according_to(Node)}) || Node <- Nodes]. + rt_ring:assert_nodes_agree_about_ownership(Nodes). %% @doc Return a list of nodes that own partitions according to the ring %% retrieved from the specified node. owners_according_to(Node) -> - case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of - {ok, Ring} -> - Owners = [Owner || {_Idx, Owner} <- riak_core_ring:all_owners(Ring)], - lists:usort(Owners); - {badrpc, _}=BadRpc -> - BadRpc - end. + rt_ring:owners_according_to(Node). %% @doc Return a list of cluster members according to the ring retrieved from %% the specified node. members_according_to(Node) -> - case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of - {ok, Ring} -> - Members = riak_core_ring:all_members(Ring), - Members; - {badrpc, _}=BadRpc -> - BadRpc - end. + rt_ring:members_according_to(Node). %% @doc Return an appropriate ringsize for the node count passed %% in. 24 is the number of cores on the bigger intel machines, but this %% may be too large for the single-chip machines. nearest_ringsize(Count) -> - nearest_ringsize(Count * 24, 2). + rt_ring:nearest_ringsize(Count). nearest_ringsize(Count, Power) -> - case Count < trunc(Power * 0.9) of - true -> - Power; - false -> - nearest_ringsize(Count, Power * 2) - end. + rt_ring:nearest_ringsize(Count, Power). %% @doc Return the cluster status of `Member' according to the ring %% retrieved from `Node'. @@ -838,13 +658,7 @@ status_of_according_to(Member, Node) -> %% @doc Return a list of nodes that own partitions according to the ring %% retrieved from the specified node. claimant_according_to(Node) -> - case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of - {ok, Ring} -> - Claimant = riak_core_ring:claimant(Ring), - Claimant; - {badrpc, _}=BadRpc -> - BadRpc - end. + rt_ring:claimant_according_to(Node). %%%=================================================================== %%% Cluster Utility Functions @@ -916,56 +730,40 @@ product(Node) -> true -> unknown end. -try_nodes_ready([Node1 | _Nodes], 0, _SleepMs) -> - lager:info("Nodes not ready after initial plan/commit, retrying"), - plan_and_commit(Node1); try_nodes_ready(Nodes, N, SleepMs) -> - ReadyNodes = [Node || Node <- Nodes, is_ready(Node) =:= true], - case ReadyNodes of - Nodes -> - ok; - _ -> - timer:sleep(SleepMs), - try_nodes_ready(Nodes, N-1, SleepMs) - end. + rt_cluster:try_nodes_ready(Nodes, N, SleepMs). %% @doc Stop nodes and wipe out their data directories clean_cluster(Nodes) when is_list(Nodes) -> - [stop_and_wait(Node) || Node <- Nodes], - clean_data_dir(Nodes). + rt_cluster:clean_cluster(Nodes). clean_data_dir(Nodes) -> - clean_data_dir(Nodes, ""). + %% TODO Determine the best approach to specifying the version. -jsb + rt_cluster:clean_data_dir(Nodes, head). -clean_data_dir(Nodes, SubDir) when not is_list(Nodes) -> - clean_data_dir([Nodes], SubDir); -clean_data_dir(Nodes, SubDir) when is_list(Nodes) -> - ?HARNESS:clean_data_dir(Nodes, SubDir). +clean_data_dir(Nodes, SubDir) -> + %% TODO Determine the best approach to specifying the version. -jsb + rt_cluster:clean_data_dir(Nodes, head, SubDir). %% @doc Shutdown every node, this is for after a test run is complete. teardown() -> - %% stop all connected nodes, 'cause it'll be faster that - %%lager:info("RPC stopping these nodes ~p", [nodes()]), - %%[ rt:stop(Node) || Node <- nodes()], - %% Then do the more exhaustive harness thing, in case something was up - %% but not connected. - ?HARNESS:teardown(). + rt_cluster:teardown(). versions() -> - ?HARNESS:versions(). + rt_cluster:versions(). %%%=================================================================== %%% Basic Read/Write Functions %%%=================================================================== systest_write(Node, Size) -> - systest_write(Node, Size, 2). + rt_systest:write(Node, Size). systest_write(Node, Size, W) -> - systest_write(Node, 1, Size, <<"systest">>, W). + rt_systest:write(Node, Size, W). systest_write(Node, Start, End, Bucket, W) -> - systest_write(Node, Start, End, Bucket, W, <<>>). + rt_systest:write(Node, Start, End, Bucket, W). %% @doc Write (End-Start)+1 objects to Node. Objects keys will be %% `Start', `Start+1' ... `End', each encoded as a 32-bit binary @@ -975,37 +773,20 @@ systest_write(Node, Start, End, Bucket, W) -> %% encountered. If all writes were successful, return value is an %% empty list. Each error has the form `{N :: integer(), Error :: term()}', %% where N is the unencoded key of the object that failed to store. -systest_write(Node, Start, End, Bucket, W, CommonValBin) - when is_binary(CommonValBin) -> - rt:wait_for_service(Node, riak_kv), - {ok, C} = riak:client_connect(Node), - F = fun(N, Acc) -> - Obj = riak_object:new(Bucket, <>, - <>), - try C:put(Obj, W) of - ok -> - Acc; - Other -> - [{N, Other} | Acc] - catch - What:Why -> - [{N, {What, Why}} | Acc] - end - end, - lists:foldl(F, [], lists:seq(Start, End)). +systest_write(Node, Start, End, Bucket, W, CommonValBin) -> + rt_systest:write(Node, Start, End, Bucket, W, CommonValBin). systest_read(Node, Size) -> - systest_read(Node, Size, 2). + rt_systest:read(Node, Size, 2). systest_read(Node, Size, R) -> - systest_read(Node, 1, Size, <<"systest">>, R). + rt_systest:read(Node, 1, Size, <<"systest">>, R). systest_read(Node, Start, End, Bucket, R) -> - systest_read(Node, Start, End, Bucket, R, <<>>). + rt_systest:read(Node, Start, End, Bucket, R, <<>>). -systest_read(Node, Start, End, Bucket, R, CommonValBin) - when is_binary(CommonValBin) -> - systest_read(Node, Start, End, Bucket, R, CommonValBin, false). +systest_read(Node, Start, End, Bucket, R, CommonValBin) -> + rt_systest:read(Node, Start, End, Bucket, R, CommonValBin, false). %% Read and verify the values of objects written with %% `systest_write'. The `SquashSiblings' parameter exists to @@ -1017,76 +798,8 @@ systest_read(Node, Start, End, Bucket, R, CommonValBin) %% writes, but also performs them locally or when a put coordinator %% fails to send an acknowledgment within the timeout window and %% another put request is issued. -systest_read(Node, Start, End, Bucket, R, CommonValBin, SquashSiblings) - when is_binary(CommonValBin) -> - rt:wait_for_service(Node, riak_kv), - {ok, C} = riak:client_connect(Node), - lists:foldl(systest_read_fold_fun(C, Bucket, R, CommonValBin, SquashSiblings), - [], - lists:seq(Start, End)). - -systest_read_fold_fun(C, Bucket, R, CommonValBin, SquashSiblings) -> - fun(N, Acc) -> - GetRes = C:get(Bucket, <>, R), - Val = object_value(GetRes, SquashSiblings), - update_acc(value_matches(Val, N, CommonValBin), Val, N, Acc) - end. - -object_value({error, _}=Error, _) -> - Error; -object_value({ok, Obj}, SquashSiblings) -> - object_value(riak_object:value_count(Obj), Obj, SquashSiblings). - -object_value(1, Obj, _SquashSiblings) -> - riak_object:get_value(Obj); -object_value(_ValueCount, Obj, false) -> - riak_object:get_value(Obj); -object_value(_ValueCount, Obj, true) -> - lager:debug("Siblings detected for ~p:~p", [riak_object:bucket(Obj), riak_object:key(Obj)]), - Contents = riak_object:get_contents(Obj), - case lists:foldl(fun sibling_compare/2, {true, undefined}, Contents) of - {true, {_, _, _, Value}} -> - lager:debug("Siblings determined to be a single value"), - Value; - {false, _} -> - {error, siblings} - end. - -sibling_compare({MetaData, Value}, {true, undefined}) -> - Dot = case dict:find(<<"dot">>, MetaData) of - {ok, DotVal} -> - DotVal; - error -> - {error, no_dot} - end, - VTag = dict:fetch(<<"X-Riak-VTag">>, MetaData), - LastMod = dict:fetch(<<"X-Riak-Last-Modified">>, MetaData), - {true, {element(2, Dot), VTag, LastMod, Value}}; -sibling_compare(_, {false, _}=InvalidMatch) -> - InvalidMatch; -sibling_compare({MetaData, Value}, {true, PreviousElements}) -> - Dot = case dict:find(<<"dot">>, MetaData) of - {ok, DotVal} -> - DotVal; - error -> - {error, no_dot} - end, - VTag = dict:fetch(<<"X-Riak-VTag">>, MetaData), - LastMod = dict:fetch(<<"X-Riak-Last-Modified">>, MetaData), - ComparisonElements = {element(2, Dot), VTag, LastMod, Value}, - {ComparisonElements =:= PreviousElements, ComparisonElements}. - -value_matches(<>, N, CommonValBin) -> - true; -value_matches(_WrongVal, _N, _CommonValBin) -> - false. - -update_acc(true, _, _, Acc) -> - Acc; -update_acc(false, {error, _}=Val, N, Acc) -> - [{N, Val} | Acc]; -update_acc(false, Val, N, Acc) -> - [{N, {wrong_val, Val}} | Acc]. +systest_read(Node, Start, End, Bucket, R, CommonValBin, SquashSiblings) -> + rt_systest:read(Node, Start, End, Bucket, R, CommonValBin, SquashSiblings). cap_subset(Val, Cap) when is_list(Cap) -> sets:is_subset(sets:from_list(Val), sets:from_list(Cap)). @@ -1374,75 +1087,28 @@ end. % @doc Reads a single replica of a value. This issues a get command directly % to the vnode handling the Nth primary partition of the object's preflist. get_replica(Node, Bucket, Key, I, N) -> - BKey = {Bucket, Key}, - Chash = rpc:call(Node, riak_core_util, chash_key, [BKey]), - Pl = rpc:call(Node, riak_core_apl, get_primary_apl, [Chash, N, riak_kv]), - {{Partition, PNode}, primary} = lists:nth(I, Pl), - Ref = Reqid = make_ref(), - Sender = {raw, Ref, self()}, - rpc:call(PNode, riak_kv_vnode, get, - [{Partition, PNode}, BKey, Ref, Sender]), - receive - {Ref, {r, Result, _, Reqid}} -> - Result; - {Ref, Reply} -> - Reply - after - 60000 -> - lager:error("Replica ~p get for ~p/~p timed out", - [I, Bucket, Key]), - ?assert(false) - end. + rt2:get_replica(Node, Bucket, Key, I, N). %%%=================================================================== %% @doc PBC-based version of {@link systest_write/1} pbc_systest_write(Node, Size) -> - pbc_systest_write(Node, Size, 2). + rt_pb:pbc_systest_write(Node, Size). pbc_systest_write(Node, Size, W) -> - pbc_systest_write(Node, 1, Size, <<"systest">>, W). + rt_pb:pbc_systest_write(Node, Size, W). pbc_systest_write(Node, Start, End, Bucket, W) -> - rt:wait_for_service(Node, riak_kv), - Pid = pbc(Node), - F = fun(N, Acc) -> - Obj = riakc_obj:new(Bucket, <>, <>), - try riakc_pb_socket:put(Pid, Obj, W) of - ok -> - Acc; - Other -> - [{N, Other} | Acc] - catch - What:Why -> - [{N, {What, Why}} | Acc] - end - end, - lists:foldl(F, [], lists:seq(Start, End)). + rt_pb:pbc_systest_write(Node, Start, End, Bucket, W). pbc_systest_read(Node, Size) -> - pbc_systest_read(Node, Size, 2). + rt_pb:pbc_systest_read(Node, Size). pbc_systest_read(Node, Size, R) -> - pbc_systest_read(Node, 1, Size, <<"systest">>, R). + rt_pb:pbc_systest_read(Node, Size, R). pbc_systest_read(Node, Start, End, Bucket, R) -> - rt:wait_for_service(Node, riak_kv), - Pid = pbc(Node), - F = fun(N, Acc) -> - case riakc_pb_socket:get(Pid, Bucket, <>, R) of - {ok, Obj} -> - case riakc_obj:get_value(Obj) of - <> -> - Acc; - WrongVal -> - [{N, {wrong_val, WrongVal}} | Acc] - end; - Other -> - [{N, Other} | Acc] - end - end, - lists:foldl(F, [], lists:seq(Start, End)). + rt_pb:pbc_systest_read(Node, Start, End, Bucket, R). %%%=================================================================== %%% PBC & HTTPC Functions @@ -1453,6 +1119,7 @@ pbc_systest_read(Node, Start, End, Bucket, R) -> pbc(Node) -> pbc(Node, [{auto_reconnect, true}]). +%% GAP: rt_pb does not provide an analog to this function. -jsb -spec pbc(node(), proplists:proplist()) -> pid(). pbc(Node, Options) -> rt:wait_for_service(Node, riak_kv), @@ -1468,8 +1135,7 @@ pbc_read(Pid, Bucket, Key) -> -spec pbc_read(pid(), binary(), binary(), [any()]) -> binary(). pbc_read(Pid, Bucket, Key, Options) -> - {ok, Value} = riakc_pb_socket:get(Pid, Bucket, Key, Options), - Value. + rt_pb:pbc_read(Pid, Bucket, Key, Options). -spec pbc_read_check(pid(), binary(), binary(), [any()]) -> boolean(). pbc_read_check(Pid, Bucket, Key, Allowed) -> @@ -1477,102 +1143,62 @@ pbc_read_check(Pid, Bucket, Key, Allowed) -> -spec pbc_read_check(pid(), binary(), binary(), [any()], [any()]) -> boolean(). pbc_read_check(Pid, Bucket, Key, Allowed, Options) -> - case riakc_pb_socket:get(Pid, Bucket, Key, Options) of - {ok, _} -> - true = lists:member(ok, Allowed); - Other -> - lists:member(Other, Allowed) orelse throw({failed, Other, Allowed}) - end. + rt_pb:pbc_read_check(Pid, Bucket, Key, Allowed, Options). %% @doc does a write via the erlang protobuf client -spec pbc_write(pid(), binary(), binary(), binary()) -> atom(). pbc_write(Pid, Bucket, Key, Value) -> - Object = riakc_obj:new(Bucket, Key, Value), - riakc_pb_socket:put(Pid, Object). + rt_pb:pbc_write(Pid, Bucket, Key, Value). %% @doc does a write via the erlang protobuf client plus content-type -spec pbc_write(pid(), binary(), binary(), binary(), list()) -> atom(). pbc_write(Pid, Bucket, Key, Value, CT) -> - Object = riakc_obj:new(Bucket, Key, Value, CT), - riakc_pb_socket:put(Pid, Object). + rt_pb:pbc_write(Pid, Bucket, Key, Value, CT). %% @doc sets a bucket property/properties via the erlang protobuf client -spec pbc_set_bucket_prop(pid(), binary(), [proplists:property()]) -> atom(). pbc_set_bucket_prop(Pid, Bucket, PropList) -> - riakc_pb_socket:set_bucket(Pid, Bucket, PropList). + rt_pb:pbc_set_bucket_prop(Pid, Bucket, PropList). %% @doc Puts the contents of the given file into the given bucket using the %% filename as a key and assuming a plain text content type. pbc_put_file(Pid, Bucket, Key, Filename) -> - {ok, Contents} = file:read_file(Filename), - riakc_pb_socket:put(Pid, riakc_obj:new(Bucket, Key, Contents, "text/plain")). + rt_pb:pbc_put_file(Pid, Bucket, Key, Filename). %% @doc Puts all files in the given directory into the given bucket using the %% filename as a key and assuming a plain text content type. pbc_put_dir(Pid, Bucket, Dir) -> - lager:info("Putting files from dir ~p into bucket ~p", [Dir, Bucket]), - {ok, Files} = file:list_dir(Dir), - [pbc_put_file(Pid, Bucket, list_to_binary(F), filename:join([Dir, F])) - || F <- Files]. + rt_pb:pbc_put_dir(Pid, Bucket, Dir). %% @doc True if the given keys have been really, really deleted. %% Useful when you care about the keys not being there. Delete simply writes %% tombstones under the given keys, so those are still seen by key folding %% operations. pbc_really_deleted(Pid, Bucket, Keys) -> - StillThere = - fun(K) -> - Res = riakc_pb_socket:get(Pid, Bucket, K, - [{r, 1}, - {notfound_ok, false}, - {basic_quorum, false}, - deletedvclock]), - case Res of - {error, notfound} -> - false; - _ -> - %% Tombstone still around - true - end - end, - [] == lists:filter(StillThere, Keys). + rt_pb:pbc_really_deleted(Pid, Bucket, Keys). %% @doc Returns HTTPS URL information for a list of Nodes -https_url(Nodes) when is_list(Nodes) -> - [begin - {Host, Port} = orddict:fetch(https, Connections), - lists:flatten(io_lib:format("https://~s:~b", [Host, Port])) - end || {_Node, Connections} <- connection_info(Nodes)]; -https_url(Node) -> - hd(https_url([Node])). +https_url(Nodes) -> + rt_http:https_url(Nodes). %% @doc Returns HTTP URL information for a list of Nodes -http_url(Nodes) when is_list(Nodes) -> - [begin - {Host, Port} = orddict:fetch(http, Connections), - lists:flatten(io_lib:format("http://~s:~b", [Host, Port])) - end || {_Node, Connections} <- connection_info(Nodes)]; -http_url(Node) -> - hd(http_url([Node])). +http_url(Nodes) -> + rt_http:http_url(Nodes). %% @doc get me an http client. -spec httpc(node()) -> term(). httpc(Node) -> - rt:wait_for_service(Node, riak_kv), - {ok, [{IP, Port}]} = get_http_conn_info(Node), - rhc:create(IP, Port, "riak", []). + rt_http:httpc(Node). %% @doc does a read via the http erlang client. -spec httpc_read(term(), binary(), binary()) -> binary(). httpc_read(C, Bucket, Key) -> - {_, Value} = rhc:get(C, Bucket, Key), - Value. + rt_http:httpc_read(C, Bucket, Key). %% @doc does a write via the http erlang client. -spec httpc_write(term(), binary(), binary(), binary()) -> atom(). httpc_write(C, Bucket, Key, Value) -> - Object = riakc_obj:new(Bucket, Key, Value), - rhc:put(C, Object). + rt_http:httpc_write(C, Bucket, Key, Value). %%%=================================================================== %%% Command Line Functions @@ -1586,20 +1212,18 @@ admin(Node, Args) -> %% The third parameter is a list of options. Valid options are: %% * `return_exit_code' - Return the exit code along with the command output admin(Node, Args, Options) -> - ?HARNESS:admin(Node, Args, Options). + rt_cmd_line:admin(Node, Args, Options). %% @doc Call 'bin/riak' command on `Node' with arguments `Args' riak(Node, Args) -> - ?HARNESS:riak(Node, Args). - + rt_cmd_line:riak(Node, Args). %% @doc Call 'bin/riak-repl' command on `Node' with arguments `Args' riak_repl(Node, Args) -> - ?HARNESS:riak_repl(Node, Args). + rt_cmd_line:riak_repl(Node, Args). search_cmd(Node, Args) -> - {ok, Cwd} = file:get_cwd(), - rpc:call(Node, riak_search_cmd, command, [[Cwd | Args]]). + rt_cmd_line:search_cmd(Node, Args). %% @doc Runs `riak attach' on a specific node, and tests for the expected behavoir. %% Here's an example: ``` @@ -1615,17 +1239,17 @@ search_cmd(Node, Args) -> %% expect will process based on the output following the sent data. %% attach(Node, Expected) -> - ?HARNESS:attach(Node, Expected). + rt_cmd_line:attach(Node, Expected). %% @doc Runs 'riak attach-direct' on a specific node %% @see rt:attach/2 attach_direct(Node, Expected) -> - ?HARNESS:attach_direct(Node, Expected). + rt_cmd_line:attach_direct(Node, Expected). %% @doc Runs `riak console' on a specific node %% @see rt:attach/2 console(Node, Expected) -> - ?HARNESS:console(Node, Expected). + rt_cmd_line:console(Node, Expected). %%%=================================================================== %%% Search @@ -1634,9 +1258,8 @@ console(Node, Expected) -> %% doc Enable the search KV hook for the given `Bucket'. Any `Node' %% in the cluster may be used as the change is propagated via the %% Ring. -enable_search_hook(Node, Bucket) when is_binary(Bucket) -> - lager:info("Installing search hook for bucket ~p", [Bucket]), - ?assertEqual(ok, rpc:call(Node, riak_search_kv_hook, install, [Bucket])). +enable_search_hook(Node, Bucket) -> + rt2:enable_search_hook(Node, Bucket). %%%=================================================================== %%% Test harness setup, configuration, and internal utilities @@ -1711,15 +1334,15 @@ get_backend(AppConfigProplist) -> %% or something like that, it's the version you're upgrading to. -spec get_version() -> binary(). get_version() -> - ?HARNESS:get_version(). + rt2:get_version(). %% @doc outputs some useful information about nodes that are up whats_up() -> - ?HARNESS:whats_up(). + rt2:whats_up(). -spec get_ip(node()) -> string(). get_ip(Node) -> - ?HARNESS:get_ip(Node). + rt2:get_ip(Node). %% @doc Log a message to the console of the specified test nodes. %% Messages are prefixed by the string "---riak_test--- " @@ -1731,81 +1354,24 @@ log_to_nodes(Nodes, Fmt) -> %% Messages are prefixed by the string "---riak_test--- " %% Uses lager:info/2 'LFmt' and 'LArgs' semantics log_to_nodes(Nodes0, LFmt, LArgs) -> - %% This logs to a node's info level, but if riak_test is running - %% at debug level, we want to know when we send this and what - %% we're saying - Nodes = lists:flatten(Nodes0), - lager:debug("log_to_nodes: " ++ LFmt, LArgs), - Module = lager, - Function = log, - Meta = [], - Args = case LArgs of - [] -> [info, Meta, "---riak_test--- " ++ LFmt]; - _ -> [info, Meta, "---riak_test--- " ++ LFmt, LArgs] - end, - [rpc:call(Node, Module, Function, Args) || Node <- lists:flatten(Nodes)]. - -%% @private utility function + rt2:log_to_nodes(Nodes0, LFmt, LArgs). + pmap(F, L) -> - Parent = self(), - lists:foldl( - fun(X, N) -> - spawn_link(fun() -> - Parent ! {pmap, N, F(X)} - end), - N+1 - end, 0, L), - L2 = [receive {pmap, N, R} -> {N,R} end || _ <- L], - {_, L3} = lists:unzip(lists:keysort(1, L2)), - L3. + rt2:pmap(F, L). -%% @private setup_harness(Test, Args) -> - ?HARNESS:setup_harness(Test, Args). + rt2:setup_harness(Test, Args). %% @doc Downloads any extant log files from the harness's running %% nodes. get_node_logs() -> - ?HARNESS:get_node_logs(). + rt2:get_node_logs(). check_ibrowse() -> - try sys:get_status(ibrowse) of - {status, _Pid, {module, gen_server} ,_} -> ok - catch - Throws -> - lager:error("ibrowse error ~p", [Throws]), - lager:error("Restarting ibrowse"), - application:stop(ibrowse), - application:start(ibrowse) - end. + rt2:check_ibrowse(). -post_result(TestResult, #rt_webhook{url=URL, headers=HookHeaders, name=Name}) -> - lager:info("Posting result to ~s ~s", [Name, URL]), - try ibrowse:send_req(URL, - [{"Content-Type", "application/json"}], - post, - mochijson2:encode(TestResult), - [{content_type, "application/json"}] ++ HookHeaders, - 300000) of %% 5 minute timeout - - {ok, RC=[$2|_], Headers, _Body} -> - {ok, RC, Headers}; - {ok, ResponseCode, Headers, Body} -> - lager:info("Test Result did not generate the expected 2XX HTTP response code."), - lager:debug("Post"), - lager:debug("Response Code: ~p", [ResponseCode]), - lager:debug("Headers: ~p", [Headers]), - lager:debug("Body: ~p", [Body]), - error; - X -> - lager:warning("Some error POSTing test result: ~p", [X]), - error - catch - Class:Reason -> - lager:error("Error reporting to ~s. ~p:~p", [Name, Class, Reason]), - lager:error("Payload: ~p", [TestResult]), - error - end. +post_result(TestResult, Webhook) -> + rt2:post_result(TestResult, Webhook). %%%=================================================================== %%% Bucket Types Functions @@ -1813,156 +1379,36 @@ post_result(TestResult, #rt_webhook{url=URL, headers=HookHeaders, name=Name}) -> %% @doc create and immediately activate a bucket type create_and_activate_bucket_type(Node, Type, Props) -> - ok = rpc:call(Node, riak_core_bucket_type, create, [Type, Props]), - wait_until_bucket_type_status(Type, ready, Node), - ok = rpc:call(Node, riak_core_bucket_type, activate, [Type]), - wait_until_bucket_type_status(Type, active, Node). - -wait_until_bucket_type_status(Type, ExpectedStatus, Nodes) when is_list(Nodes) -> - [wait_until_bucket_type_status(Type, ExpectedStatus, Node) || Node <- Nodes]; -wait_until_bucket_type_status(Type, ExpectedStatus, Node) -> - F = fun() -> - ActualStatus = rpc:call(Node, riak_core_bucket_type, status, [Type]), - ExpectedStatus =:= ActualStatus - end, - ?assertEqual(ok, rt:wait_until(F)). + rt_bucket_type:create_and_activate_bucket_type(Node, Type, Props). --spec bucket_type_visible([atom()], binary()|{binary(), binary()}) -> boolean(). -bucket_type_visible(Nodes, Type) -> - MaxTime = rt_config:get(rt_max_wait_time), - IsVisible = fun erlang:is_list/1, - {Res, NodesDown} = rpc:multicall(Nodes, riak_core_bucket_type, get, [Type], MaxTime), - NodesDown == [] andalso lists:all(IsVisible, Res). +wait_until_bucket_type_status(Type, ExpectedStatus, Nodes) -> + rt_bucket_type:wait_until_bucket_type_status(Type, ExpectedStatus, Nodes). wait_until_bucket_type_visible(Nodes, Type) -> - F = fun() -> bucket_type_visible(Nodes, Type) end, - ?assertEqual(ok, rt:wait_until(F)). - --spec see_bucket_props([atom()], binary()|{binary(), binary()}, - proplists:proplist()) -> boolean(). -see_bucket_props(Nodes, Bucket, ExpectProps) -> - MaxTime = rt_config:get(rt_max_wait_time), - IsBad = fun({badrpc, _}) -> true; - ({error, _}) -> true; - (Res) when is_list(Res) -> false - end, - HasProps = fun(ResProps) -> - lists:all(fun(P) -> lists:member(P, ResProps) end, - ExpectProps) - end, - case rpc:multicall(Nodes, riak_core_bucket, get_bucket, [Bucket], MaxTime) of - {Res, []} -> - % No nodes down, check no errors - case lists:any(IsBad, Res) of - true -> - false; - false -> - lists:all(HasProps, Res) - end; - {_, _NodesDown} -> - false - end. + rt_bucket_types:wait_until_bucket_type_visble(Nodes, Type). wait_until_bucket_props(Nodes, Bucket, Props) -> - F = fun() -> - see_bucket_props(Nodes, Bucket, Props) - end, - ?assertEqual(ok, rt:wait_until(F)). + rt_bucket_types:wait_until_bucket_props(Nodes, Bucket, Props). %% @doc Set up in memory log capture to check contents in a test. -setup_log_capture(Nodes) when is_list(Nodes) -> - rt:load_modules_on_nodes([riak_test_lager_backend], Nodes), - [?assertEqual({Node, ok}, - {Node, - rpc:call(Node, - gen_event, - add_handler, - [lager_event, - riak_test_lager_backend, - [info, false]])}) || Node <- Nodes], - [?assertEqual({Node, ok}, - {Node, - rpc:call(Node, - lager, - set_loglevel, - [riak_test_lager_backend, - info])}) || Node <- Nodes]; -setup_log_capture(Node) when not is_list(Node) -> - setup_log_capture([Node]). - +setup_log_capture(Nodes) -> + rt2:setup_log_capture(Nodes). expect_in_log(Node, Pattern) -> - CheckLogFun = fun() -> - Logs = rpc:call(Node, riak_test_lager_backend, get_logs, []), - lager:info("looking for pattern ~s in logs for ~p", - [Pattern, Node]), - case re:run(Logs, Pattern, []) of - {match, _} -> - lager:info("Found match"), - true; - nomatch -> - lager:info("No match"), - false - end - end, - case rt:wait_until(CheckLogFun) of - ok -> - true; - _ -> - false - end. + rt2:expect_in_log(Node, Pattern). %% @doc Wait for Riak Control to start on a single node. %% %% Non-optimal check, because we're blocking for the gen_server to start %% to ensure that the routes have been added by the supervisor. %% -wait_for_control(_Vsn, Node) when is_atom(Node) -> - lager:info("Waiting for riak_control to start on node ~p.", [Node]), - - %% Wait for the gen_server. - rt:wait_until(Node, fun(N) -> - case rpc:call(N, - riak_control_session, - get_version, - []) of - {ok, _} -> - true; - Error -> - lager:info("Error was ~p.", [Error]), - false - end - end), - - lager:info("Waiting for routes to be added to supervisor..."), - - %% Wait for routes to be added by supervisor. - rt:wait_until(Node, fun(N) -> - case rpc:call(N, - webmachine_router, - get_routes, - []) of - {badrpc, Error} -> - lager:info("Error was ~p.", [Error]), - false; - Routes -> - case is_control_gui_route_loaded(Routes) of - false -> - false; - _ -> - true - end - end - end). - -%% @doc Is the riak_control GUI route loaded? -is_control_gui_route_loaded(Routes) -> - lists:keymember(admin_gui, 2, Routes) orelse lists:keymember(riak_control_wm_gui, 2, Routes). +wait_for_control(Vsn, Node) -> + rt2:wait_for_control(Vsn, Node). %% @doc Wait for Riak Control to start on a series of nodes. -wait_for_control(VersionedNodes) when is_list(VersionedNodes) -> - [wait_for_control(Vsn, Node) || {Vsn, Node} <- VersionedNodes]. +wait_for_control(VersionedNodes) -> + rt2:wait_for_control(VersionedNodes). -ifdef(TEST). diff --git a/src/rt_ring.erl b/src/rt_ring.erl index 25b70eaa4..56d16d5f2 100644 --- a/src/rt_ring.erl +++ b/src/rt_ring.erl @@ -27,6 +27,7 @@ get_ring/1, members_according_to/1, nearest_ringsize/1, + nearest_ringsize/2, owners_according_to/1, partitions_for_node/1, status_of_according_to/2]). From e8985e0e4c984a6981244436595ca00617da24ee Mon Sep 17 00:00:00 2001 From: John Burwell Date: Wed, 18 Feb 2015 22:52:02 -0500 Subject: [PATCH 065/157] Wraps the rt:set_conf, rt:set_advanced_conf, and rt:update_app_config functions --- src/rt.erl | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/src/rt.erl b/src/rt.erl index 91972f04e..264eab54b 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -186,26 +186,26 @@ str(String, Substr) -> set_conf(Node, NameValuePairs) -> stop(Node), ?assertEqual(ok, rt:wait_until_unpingable(Node)), - ?HARNESS:set_conf(Node, NameValuePairs), + rt_config:set_conf(Node, NameValuePairs), start(Node). -spec set_advanced_conf(atom(), [{string(), string()}]) -> ok. set_advanced_conf(all, NameValuePairs) -> - ?HARNESS:set_advanced_conf(all, NameValuePairs); + rt_config:set_advanced_conf(all, NameValuePairs); set_advanced_conf(Node, NameValuePairs) -> stop(Node), ?assertEqual(ok, rt:wait_until_unpingable(Node)), - ?HARNESS:set_advanced_conf(Node, NameValuePairs), + rt_config:set_advanced_conf(Node, NameValuePairs), start(Node). %% @doc Rewrite the given node's app.config file, overriding the varialbes %% in the existing app.config with those in `Config'. update_app_config(all, Config) -> - ?HARNESS:update_app_config(all, Config); + rt_config:update_app_config(all, Config); update_app_config(Node, Config) -> stop(Node), ?assertEqual(ok, rt:wait_until_unpingable(Node)), - ?HARNESS:update_app_config(Node, Config), + rt_config:update_app_config(Node, Config), start(Node). %% @doc Helper that returns first successful application get_env result, @@ -313,8 +313,7 @@ build_clusters(Settings) -> %% @doc Start the specified Riak node start(Node) -> - %% TODO Determine the best way to work with versions. - %% For now, we are using the default version ... -jsb + %% TODO Determine the best way to implement the current version specification. -jsb rt_node:start(Node, head). %% @doc Start the specified Riak `Node' and wait for it to be pingable @@ -323,12 +322,12 @@ start_and_wait(Node) -> ?assertEqual(ok, wait_until_pingable(Node)). async_start(Node) -> - spawn(fun() -> start(Node) end). + %% TODO Determine the best way to implement the current version specification. -jsb + rt_node:async_start(Node, head). %% @doc Stop the specified Riak `Node'. stop(Node) -> - %% TODO Determine the best way to work with versions. - %% For now, we are using the default version ... -jsb + %% TODO Determine the best way to implement the current version specification. -jsb rt_node:stop(Node, head). %% @doc Stop the specified Riak `Node' and wait until it is not pingable From 93863aa5950382da0f30c0e88177bc58c0905867 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Thu, 19 Feb 2015 13:04:33 -0500 Subject: [PATCH 066/157] Refinements to the wrapper to pass the verify_down test case * Modifies rtdev:start and rtdev:stop to extract the node id and name from the passed atom * Fixes the formatting of the rt2.erl module * Adds version information to the log message of the rt_node:stop function --- src/rt2.erl | 563 ++++++++++++++++++++++++------------------------ src/rt_node.erl | 4 +- src/rtdev.erl | 60 +++++- 3 files changed, 335 insertions(+), 292 deletions(-) diff --git a/src/rt2.erl b/src/rt2.erl index 1761dc3ea..560dbc6ac 100644 --- a/src/rt2.erl +++ b/src/rt2.erl @@ -83,29 +83,27 @@ whats_up/0 ]). --define(HARNESS, (rt_config:get(rt_harness))). - priv_dir() -> -LocalPrivDir = "./priv", -%% XXX for some reason, codew:priv_dir returns riak_test/riak_test/priv, -%% which is wrong, so fix it. -DepPrivDir = re:replace(code:priv_dir(riak_test), "riak_test(/riak_test)*", -"riak_test", [{return, list}]), -PrivDir = case {filelib:is_dir(LocalPrivDir), filelib:is_dir(DepPrivDir)} of -{true, _} -> - lager:debug("Local ./priv detected, using that..."), - %% we want an absolute path! - filename:absname(LocalPrivDir); -{false, true} -> - lager:debug("riak_test dependency priv_dir detected, using that..."), - DepPrivDir; -_ -> - ?assertEqual({true, bad_priv_dir}, {false, bad_priv_dir}) -end, - -lager:info("priv dir: ~p -> ~p", [code:priv_dir(riak_test), PrivDir]), -?assert(filelib:is_dir(PrivDir)), -PrivDir. + LocalPrivDir = "./priv", + %% XXX for some reason, codew:priv_dir returns riak_test/riak_test/priv, + %% which is wrong, so fix it. + DepPrivDir = re:replace(code:priv_dir(riak_test), "riak_test(/riak_test)*", + "riak_test", [{return, list}]), + PrivDir = case {filelib:is_dir(LocalPrivDir), filelib:is_dir(DepPrivDir)} of + {true, _} -> + lager:debug("Local ./priv detected, using that..."), + %% we want an absolute path! + filename:absname(LocalPrivDir); + {false, true} -> + lager:debug("riak_test dependency priv_dir detected, using that..."), + DepPrivDir; + _ -> + ?assertEqual({true, bad_priv_dir}, {false, bad_priv_dir}) + end, + + lager:info("priv dir: ~p -> ~p", [code:priv_dir(riak_test), PrivDir]), + ?assert(filelib:is_dir(PrivDir)), + PrivDir. %% @doc gets riak deps from the appropriate harness -spec get_deps() -> list(). @@ -114,23 +112,23 @@ get_deps() -> rt_harness:get_deps(). %% @doc if String contains Substr, return true. -spec str(string(), string()) -> boolean(). str(String, Substr) -> -case string:str(String, Substr) of -0 -> false; -_ -> true -end. + case string:str(String, Substr) of + 0 -> false; + _ -> true + end. %% @doc Helper that returns first successful application get_env result, %% used when different versions of Riak use different app vars for %% the same setting. rpc_get_env(_, []) -> -undefined; + undefined; rpc_get_env(Node, [{App,Var}|Others]) -> -case rpc:call(Node, application, get_env, [App, Var]) of -{ok, Value} -> - {ok, Value}; -_ -> - rpc_get_env(Node, Others) -end. + case rpc:call(Node, application, get_env, [App, Var]) of + {ok, Value} -> + {ok, Value}; + _ -> + rpc_get_env(Node, Others) + end. -type interface() :: {http, tuple()} | {pb, tuple()}. -type interfaces() :: [interface()]. @@ -191,14 +189,14 @@ cmd(Cmd, Opts) -> %% once per second, as to not create the impression that nothing is happening. -spec stream_cmd(string()) -> {integer(), string()}. stream_cmd(Cmd) -> -Port = open_port({spawn, binary_to_list(iolist_to_binary(Cmd))}, [stream, stderr_to_stdout, exit_status]), -stream_cmd_loop(Port, "", "", now()). + Port = open_port({spawn, binary_to_list(iolist_to_binary(Cmd))}, [stream, stderr_to_stdout, exit_status]), + stream_cmd_loop(Port, "", "", now()). %% @doc same as rt:stream_cmd/1, but with options, like open_port/2 -spec stream_cmd(string(), string()) -> {integer(), string()}. stream_cmd(Cmd, Opts) -> -Port = open_port({spawn, binary_to_list(iolist_to_binary(Cmd))}, [stream, stderr_to_stdout, exit_status] ++ Opts), -stream_cmd_loop(Port, "", "", now()). + Port = open_port({spawn, binary_to_list(iolist_to_binary(Cmd))}, [stream, stderr_to_stdout, exit_status] ++ Opts), + stream_cmd_loop(Port, "", "", now()). stream_cmd_loop(Port, Buffer, NewLineBuffer, Time={_MegaSecs, Secs, _MicroSecs}) -> receive @@ -230,18 +228,16 @@ stream_cmd_loop(Port, Buffer, NewLineBuffer, Time={_MegaSecs, Secs, _MicroSecs}) %%%=================================================================== %%% Remote code management %%%=================================================================== -load_modules_on_nodes([], Nodes) -when is_list(Nodes) -> -ok; -load_modules_on_nodes([Module | MoreModules], Nodes) -when is_list(Nodes) -> -case code:get_object_code(Module) of -{Module, Bin, File} -> - {_, []} = rpc:multicall(Nodes, code, load_binary, [Module, File, Bin]); -error -> - error(lists:flatten(io_lib:format("unable to get_object_code(~s)", [Module]))) -end, -load_modules_on_nodes(MoreModules, Nodes). +load_modules_on_nodes([], Nodes) when is_list(Nodes) -> + ok; +load_modules_on_nodes([Module | MoreModules], Nodes) when is_list(Nodes) -> + case code:get_object_code(Module) of + {Module, Bin, File} -> + {_, []} = rpc:multicall(Nodes, code, load_binary, [Module, File, Bin]); + error -> + error(lists:flatten(io_lib:format("unable to get_object_code(~s)", [Module]))) + end, + load_modules_on_nodes(MoreModules, Nodes). %%%=================================================================== @@ -249,21 +245,21 @@ load_modules_on_nodes(MoreModules, Nodes). %%%=================================================================== is_mixed_cluster(Nodes) when is_list(Nodes) -> -%% If the nodes are bad, we don't care what version they are -{Versions, _BadNodes} = rpc:multicall(Nodes, init, script_id, [], rt_config:get(rt_max_wait_time)), -length(lists:usort(Versions)) > 1; -is_mixed_cluster(Node) -> -Nodes = rpc:call(Node, erlang, nodes, []), -is_mixed_cluster(Nodes). + %% If the nodes are bad, we don't care what version they are + {Versions, _BadNodes} = rpc:multicall(Nodes, init, script_id, [], rt_config:get(rt_max_wait_time)), + length(lists:usort(Versions)) > 1; + is_mixed_cluster(Node) -> + Nodes = rpc:call(Node, erlang, nodes, []), + is_mixed_cluster(Nodes). %% @private is_ring_ready(Node) -> -case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of -{ok, Ring} -> - riak_core_ring:ring_ready(Ring); -_ -> - false -end. + case rpc:call(Node, riak_core_ring_manager, get_raw_ring, []) of + {ok, Ring} -> + riak_core_ring:ring_ready(Ring); + _ -> + false + end. -type products() :: riak | riak_ee | riak_cs | unknown. @@ -287,30 +283,30 @@ product(Node) -> %% provided `rt_max_wait_time' and `rt_retry_delay' parameters in %% specified `riak_test' config file. wait_until(Fun) when is_function(Fun) -> -MaxTime = rt_config:get(rt_max_wait_time), -Delay = rt_config:get(rt_retry_delay), -Retry = MaxTime div Delay, -wait_until(Fun, Retry, Delay). + MaxTime = rt_config:get(rt_max_wait_time), + Delay = rt_config:get(rt_retry_delay), + Retry = MaxTime div Delay, + wait_until(Fun, Retry, Delay). %% @doc Convenience wrapper for wait_until for the myriad functions that %% take a node as single argument. wait_until(Node, Fun) when is_atom(Node), is_function(Fun) -> -wait_until(fun() -> Fun(Node) end). + wait_until(fun() -> Fun(Node) end). %% @doc Retry `Fun' until it returns `Retry' times, waiting `Delay' %% milliseconds between retries. This is our eventual consistency bread %% and butter wait_until(Fun, Retry, Delay) when Retry > 0 -> -Res = Fun(), -case Res of -true -> - ok; -_ when Retry == 1 -> - {fail, Res}; -_ -> - timer:sleep(Delay), - wait_until(Fun, Retry-1, Delay) -end. + Res = Fun(), + case Res of + true -> + ok; + _ when Retry == 1 -> + {fail, Res}; + _ -> + timer:sleep(Delay), + wait_until(Fun, Retry-1, Delay) + end. %% @doc Wait until the specified node is considered ready by `riak_core'. %% As of Riak 1.0, a node is ready if it is in the `valid' or `leaving' @@ -323,72 +319,72 @@ wait_until_ready(Node) -> %% @doc Wait until status can be read from riak_kv_console wait_until_status_ready(Node) -> -lager:info("Wait until status ready in ~p", [Node]), -?assertEqual(ok, wait_until(Node, - fun(_) -> - case rpc:call(Node, riak_kv_console, status, [[]]) of - ok -> - true; - Res -> - Res - end - end)). + lager:info("Wait until status ready in ~p", [Node]), + ?assertEqual(ok, wait_until(Node, + fun(_) -> + case rpc:call(Node, riak_kv_console, status, [[]]) of + ok -> + true; + Res -> + Res + end + end)). %% @doc Given a list of nodes, wait until all nodes believe there are no %% on-going or pending ownership transfers. -spec wait_until_no_pending_changes([node()]) -> ok | fail. wait_until_no_pending_changes(Nodes) -> -lager:info("Wait until no pending changes on ~p", [Nodes]), -F = fun() -> - rpc:multicall(Nodes, riak_core_vnode_manager, force_handoffs, []), - {Rings, BadNodes} = rpc:multicall(Nodes, riak_core_ring_manager, get_raw_ring, []), - Changes = [ riak_core_ring:pending_changes(Ring) =:= [] || {ok, Ring} <- Rings ], - BadNodes =:= [] andalso length(Changes) =:= length(Nodes) andalso lists:all(fun(T) -> T end, Changes) -end, -?assertEqual(ok, wait_until(F)), -ok. + lager:info("Wait until no pending changes on ~p", [Nodes]), + F = fun() -> + rpc:multicall(Nodes, riak_core_vnode_manager, force_handoffs, []), + {Rings, BadNodes} = rpc:multicall(Nodes, riak_core_ring_manager, get_raw_ring, []), + Changes = [ riak_core_ring:pending_changes(Ring) =:= [] || {ok, Ring} <- Rings ], + BadNodes =:= [] andalso length(Changes) =:= length(Nodes) andalso lists:all(fun(T) -> T end, Changes) + end, + ?assertEqual(ok, wait_until(F)), + ok. %% @doc Waits until no transfers are in-flight or pending, checked by %% riak_core_status:transfers(). -spec wait_until_transfers_complete([node()]) -> ok | fail. wait_until_transfers_complete([Node0|_]) -> -lager:info("Wait until transfers complete ~p", [Node0]), -F = fun(Node) -> - {DownNodes, Transfers} = rpc:call(Node, riak_core_status, transfers, []), - DownNodes =:= [] andalso Transfers =:= [] -end, -?assertEqual(ok, wait_until(Node0, F)), -ok. + lager:info("Wait until transfers complete ~p", [Node0]), + F = fun(Node) -> + {DownNodes, Transfers} = rpc:call(Node, riak_core_status, transfers, []), + DownNodes =:= [] andalso Transfers =:= [] + end, + ?assertEqual(ok, wait_until(Node0, F)), + ok. wait_for_service(Node, Services) when is_list(Services) -> -F = fun(N) -> - case rpc:call(N, riak_core_node_watcher, services, [N]) of - {badrpc, Error} -> - {badrpc, Error}; - CurrServices when is_list(CurrServices) -> - lists:all(fun(Service) -> lists:member(Service, CurrServices) end, Services); - Res -> - Res - end -end, -?assertEqual(ok, wait_until(Node, F)), -ok; + F = fun(N) -> + case rpc:call(N, riak_core_node_watcher, services, [N]) of + {badrpc, Error} -> + {badrpc, Error}; + CurrServices when is_list(CurrServices) -> + lists:all(fun(Service) -> lists:member(Service, CurrServices) end, Services); + Res -> + Res + end + end, + ?assertEqual(ok, wait_until(Node, F)), + ok; wait_for_service(Node, Service) -> -wait_for_service(Node, [Service]). + wait_for_service(Node, [Service]). wait_for_cluster_service(Nodes, Service) -> -lager:info("Wait for cluster service ~p in ~p", [Service, Nodes]), -F = fun(N) -> - UpNodes = rpc:call(N, riak_core_node_watcher, nodes, [Service]), - (Nodes -- UpNodes) == [] -end, -[?assertEqual(ok, wait_until(Node, F)) || Node <- Nodes], -ok. + lager:info("Wait for cluster service ~p in ~p", [Service, Nodes]), + F = fun(N) -> + UpNodes = rpc:call(N, riak_core_node_watcher, nodes, [Service]), + (Nodes -- UpNodes) == [] + end, + [?assertEqual(ok, wait_until(Node, F)) || Node <- Nodes], + ok. %% @doc Wait until all nodes in the list `Nodes' believe each other to be %% members of the cluster. wait_until_all_members(Nodes) -> -wait_until_all_members(Nodes, Nodes). + wait_until_all_members(Nodes, Nodes). %% @doc Wait until all nodes in the list `Nodes' believes all nodes in the %% list `Members' are members of the cluster. @@ -410,141 +406,140 @@ wait_until_all_members(Nodes, ExpectedMembers) -> %% @doc Given a list of nodes, wait until all nodes believe the ring has %% converged (ie. `riak_core_ring:is_ready' returns `true'). wait_until_ring_converged(Nodes) -> -lager:info("Wait until ring converged on ~p", [Nodes]), -[?assertEqual(ok, wait_until(Node, fun is_ring_ready/1)) || Node <- Nodes], -ok. + lager:info("Wait until ring converged on ~p", [Nodes]), + [?assertEqual(ok, wait_until(Node, fun is_ring_ready/1)) || Node <- Nodes], + ok. wait_until_legacy_ringready(Node) -> -lager:info("Wait until legacy ring ready on ~p", [Node]), -rt:wait_until(Node, - fun(_) -> - case rpc:call(Node, riak_kv_status, ringready, []) of - {ok, _Nodes} -> - true; - Res -> - Res - end - end). + lager:info("Wait until legacy ring ready on ~p", [Node]), + rt:wait_until(Node, + fun(_) -> + case rpc:call(Node, riak_kv_status, ringready, []) of + {ok, _Nodes} -> + true; + Res -> + Res + end + end). %% @doc wait until each node in Nodes is disterl connected to each. wait_until_connected(Nodes) -> -lager:info("Wait until connected ~p", [Nodes]), -F = fun(Node) -> - Connected = rpc:call(Node, erlang, nodes, []), - lists:sort(Nodes) == lists:sort([Node]++Connected)--[node()] -end, -[?assertEqual(ok, wait_until(Node, F)) || Node <- Nodes], -ok. + lager:info("Wait until connected ~p", [Nodes]), + F = fun(Node) -> + Connected = rpc:call(Node, erlang, nodes, []), + lists:sort(Nodes) == lists:sort([Node]++Connected)--[node()] + end, + [?assertEqual(ok, wait_until(Node, F)) || Node <- Nodes], + ok. %% @doc Wait until the specified node is pingable wait_until_pingable(Node) -> -lager:info("Wait until ~p is pingable", [Node]), -F = fun(N) -> - net_adm:ping(N) =:= pong -end, -?assertEqual(ok, wait_until(Node, F)), -ok. + lager:info("Wait until ~p is pingable", [Node]), + F = fun(N) -> + net_adm:ping(N) =:= pong + end, + ?assertEqual(ok, wait_until(Node, F)), + ok. %% @doc Wait until the specified node is no longer pingable wait_until_unpingable(Node) -> -lager:info("Wait until ~p is not pingable", [Node]), -_OSPidToKill = rpc:call(Node, os, getpid, []), -F = fun() -> net_adm:ping(Node) =:= pang end, -%% riak stop will kill -9 after 5 mins, so we try to wait at least that -%% amount of time. -Delay = rt_config:get(rt_retry_delay), -Retry = lists:max([360000, rt_config:get(rt_max_wait_time)]) div Delay, -case wait_until(F, Retry, Delay) of -ok -> ok; -_ -> - lager:error("Timed out waiting for node ~p to shutdown", [Node]), - ?assert(node_shutdown_timed_out) -end. - + _OSPidToKill = rpc:call(Node, os, getpid, []), + F = fun() -> net_adm:ping(Node) =:= pang end, + %% riak stop will kill -9 after 5 mins, so we try to wait at least that + %% amount of time. + Delay = rt_config:get(rt_retry_delay), + Retry = lists:max([360000, rt_config:get(rt_max_wait_time)]) div Delay, + lager:info("Wait until ~p is not pingable for ~p seconds with a retry of ~p", + [Node, Delay, Retry]), + case wait_until(F, Retry, Delay) of + ok -> ok; + _ -> + lager:error("Timed out waiting for node ~p to shutdown", [Node]), + ?assert(node_shutdown_timed_out) + end. % Waits until a certain registered name pops up on the remote node. wait_until_registered(Node, Name) -> -lager:info("Wait until ~p is up on ~p", [Name, Node]), - -F = fun() -> - Registered = rpc:call(Node, erlang, registered, []), - lists:member(Name, Registered) -end, -case wait_until(F) of -ok -> - ok; -_ -> - lager:info("The server with the name ~p on ~p is not coming up.", - [Name, Node]), - ?assert(registered_name_timed_out) -end. + lager:info("Wait until ~p is up on ~p", [Name, Node]), + F = fun() -> + Registered = rpc:call(Node, erlang, registered, []), + lists:member(Name, Registered) + end, + case wait_until(F) of + ok -> + ok; + _ -> + lager:info("The server with the name ~p on ~p is not coming up.", + [Name, Node]), + ?assert(registered_name_timed_out) + end. %% Waits until the cluster actually detects that it is partitioned. wait_until_partitioned(P1, P2) -> -lager:info("Waiting until partition acknowledged: ~p ~p", [P1, P2]), -[ begin - lager:info("Waiting for ~p to be partitioned from ~p", [Node, P2]), - wait_until(fun() -> is_partitioned(Node, P2) end) -end || Node <- P1 ], -[ begin - lager:info("Waiting for ~p to be partitioned from ~p", [Node, P1]), - wait_until(fun() -> is_partitioned(Node, P1) end) -end || Node <- P2 ]. + lager:info("Waiting until partition acknowledged: ~p ~p", [P1, P2]), + [ begin + lager:info("Waiting for ~p to be partitioned from ~p", [Node, P2]), + wait_until(fun() -> is_partitioned(Node, P2) end) + end || Node <- P1 ], + [ begin + lager:info("Waiting for ~p to be partitioned from ~p", [Node, P1]), + wait_until(fun() -> is_partitioned(Node, P1) end) + end || Node <- P2 ]. is_partitioned(Node, Peers) -> -AvailableNodes = rpc:call(Node, riak_core_node_watcher, nodes, [riak_kv]), -lists:all(fun(Peer) -> not lists:member(Peer, AvailableNodes) end, Peers). + AvailableNodes = rpc:call(Node, riak_core_node_watcher, nodes, [riak_kv]), + lists:all(fun(Peer) -> not lists:member(Peer, AvailableNodes) end, Peers). capability(Node, all) -> -rpc:call(Node, riak_core_capability, all, []); + rpc:call(Node, riak_core_capability, all, []); capability(Node, Capability) -> -rpc:call(Node, riak_core_capability, get, [Capability]). + rpc:call(Node, riak_core_capability, get, [Capability]). capability(Node, Capability, Default) -> -rpc:call(Node, riak_core_capability, get, [Capability, Default]). + rpc:call(Node, riak_core_capability, get, [Capability, Default]). wait_until_capability(Node, Capability, Value) -> -rt:wait_until(Node, - fun(_) -> - cap_equal(Value, capability(Node, Capability)) - end). + rt:wait_until(Node, + fun(_) -> + cap_equal(Value, capability(Node, Capability)) + end). wait_until_capability(Node, Capability, Value, Default) -> -rt:wait_until(Node, - fun(_) -> - Cap = capability(Node, Capability, Default), - io:format("capability is ~p ~p",[Node, Cap]), - cap_equal(Value, Cap) - end). + rt:wait_until(Node, + fun(_) -> + Cap = capability(Node, Capability, Default), + io:format("capability is ~p ~p",[Node, Cap]), + cap_equal(Value, Cap) + end). cap_equal(Val, Cap) when is_list(Cap) -> -lists:sort(Cap) == lists:sort(Val); + lists:sort(Cap) == lists:sort(Val); cap_equal(Val, Cap) -> -Val == Cap. + Val == Cap. % @doc Reads a single replica of a value. This issues a get command directly % to the vnode handling the Nth primary partition of the object's preflist. get_replica(Node, Bucket, Key, I, N) -> -BKey = {Bucket, Key}, -Chash = rpc:call(Node, riak_core_util, chash_key, [BKey]), -Pl = rpc:call(Node, riak_core_apl, get_primary_apl, [Chash, N, riak_kv]), -{{Partition, PNode}, primary} = lists:nth(I, Pl), -Ref = Reqid = make_ref(), -Sender = {raw, Ref, self()}, -rpc:call(PNode, riak_kv_vnode, get, - [{Partition, PNode}, BKey, Ref, Sender]), -receive -{Ref, {r, Result, _, Reqid}} -> - Result; -{Ref, Reply} -> - Reply -after -60000 -> - lager:error("Replica ~p get for ~p/~p timed out", - [I, Bucket, Key]), - ?assert(false) -end. + BKey = {Bucket, Key}, + Chash = rpc:call(Node, riak_core_util, chash_key, [BKey]), + Pl = rpc:call(Node, riak_core_apl, get_primary_apl, [Chash, N, riak_kv]), + {{Partition, PNode}, primary} = lists:nth(I, Pl), + Ref = Reqid = make_ref(), + Sender = {raw, Ref, self()}, + rpc:call(PNode, riak_kv_vnode, get, + [{Partition, PNode}, BKey, Ref, Sender]), + receive + {Ref, {r, Result, _, Reqid}} -> + Result; + {Ref, Reply} -> + Reply + after + 60000 -> + lager:error("Replica ~p get for ~p/~p timed out", + [I, Bucket, Key]), + ?assert(false) + end. %%%=================================================================== %%% Search @@ -554,8 +549,8 @@ end. %% in the cluster may be used as the change is propagated via the %% Ring. enable_search_hook(Node, Bucket) when is_binary(Bucket) -> -lager:info("Installing search hook for bucket ~p", [Bucket]), -?assertEqual(ok, rpc:call(Node, riak_search_kv_hook, install, [Bucket])). + lager:info("Installing search hook for bucket ~p", [Bucket]), + ?assertEqual(ok, rpc:call(Node, riak_search_kv_hook, install, [Bucket])). %% @doc Gets the current version under test. In the case of an upgrade test %% or something like that, it's the version you're upgrading to. @@ -575,39 +570,39 @@ get_ip(Node) -> %% Messages are prefixed by the string "---riak_test--- " %% Uses lager:info/1 'Fmt' semantics log_to_nodes(Nodes, Fmt) -> -log_to_nodes(Nodes, Fmt, []). + log_to_nodes(Nodes, Fmt, []). %% @doc Log a message to the console of the specified test nodes. %% Messages are prefixed by the string "---riak_test--- " %% Uses lager:info/2 'LFmt' and 'LArgs' semantics log_to_nodes(Nodes0, LFmt, LArgs) -> -%% This logs to a node's info level, but if riak_test is running -%% at debug level, we want to know when we send this and what -%% we're saying -Nodes = lists:flatten(Nodes0), -lager:debug("log_to_nodes: " ++ LFmt, LArgs), -Module = lager, -Function = log, -Meta = [], -Args = case LArgs of - [] -> [info, Meta, "---riak_test--- " ++ LFmt]; - _ -> [info, Meta, "---riak_test--- " ++ LFmt, LArgs] - end, -[rpc:call(Node, Module, Function, Args) || Node <- lists:flatten(Nodes)]. + %% This logs to a node's info level, but if riak_test is running + %% at debug level, we want to know when we send this and what + %% we're saying + Nodes = lists:flatten(Nodes0), + lager:debug("log_to_nodes: " ++ LFmt, LArgs), + Module = lager, + Function = log, + Meta = [], + Args = case LArgs of + [] -> [info, Meta, "---riak_test--- " ++ LFmt]; + _ -> [info, Meta, "---riak_test--- " ++ LFmt, LArgs] + end, + [rpc:call(Node, Module, Function, Args) || Node <- lists:flatten(Nodes)]. %% @private utility function pmap(F, L) -> -Parent = self(), -lists:foldl( -fun(X, N) -> - spawn_link(fun() -> - Parent ! {pmap, N, F(X)} - end), - N+1 -end, 0, L), -L2 = [receive {pmap, N, R} -> {N,R} end || _ <- L], -{_, L3} = lists:unzip(lists:keysort(1, L2)), -L3. + Parent = self(), + lists:foldl( + fun(X, N) -> + spawn_link(fun() -> + Parent ! {pmap, N, F(X)} + end), + N+1 + end, 0, L), + L2 = [receive {pmap, N, R} -> {N,R} end || _ <- L], + {_, L3} = lists:unzip(lists:keysort(1, L2)), + L3. %% @private setup_harness(Test, Args) -> @@ -619,43 +614,43 @@ get_node_logs() -> rt_harness:get_node_logs(). check_ibrowse() -> -try sys:get_status(ibrowse) of -{status, _Pid, {module, gen_server} ,_} -> ok -catch -Throws -> - lager:error("ibrowse error ~p", [Throws]), - lager:error("Restarting ibrowse"), - application:stop(ibrowse), - application:start(ibrowse) -end. + try sys:get_status(ibrowse) of + {status, _Pid, {module, gen_server} ,_} -> ok + catch + Throws -> + lager:error("ibrowse error ~p", [Throws]), + lager:error("Restarting ibrowse"), + application:stop(ibrowse), + application:start(ibrowse) + end. post_result(TestResult, #rt_webhook{url=URL, headers=HookHeaders, name=Name}) -> -lager:info("Posting result to ~s ~s", [Name, URL]), -try ibrowse:send_req(URL, - [{"Content-Type", "application/json"}], - post, - mochijson2:encode(TestResult), - [{content_type, "application/json"}] ++ HookHeaders, - 300000) of %% 5 minute timeout - -{ok, RC=[$2|_], Headers, _Body} -> - {ok, RC, Headers}; -{ok, ResponseCode, Headers, Body} -> - lager:info("Test Result did not generate the expected 2XX HTTP response code."), - lager:debug("Post"), - lager:debug("Response Code: ~p", [ResponseCode]), - lager:debug("Headers: ~p", [Headers]), - lager:debug("Body: ~p", [Body]), - error; -X -> - lager:warning("Some error POSTing test result: ~p", [X]), - error -catch -Class:Reason -> - lager:error("Error reporting to ~s. ~p:~p", [Name, Class, Reason]), - lager:error("Payload: ~p", [TestResult]), - error -end. + lager:info("Posting result to ~s ~s", [Name, URL]), + try ibrowse:send_req(URL, + [{"Content-Type", "application/json"}], + post, + mochijson2:encode(TestResult), + [{content_type, "application/json"}] ++ HookHeaders, + 300000) of %% 5 minute timeout + + {ok, RC=[$2|_], Headers, _Body} -> + {ok, RC, Headers}; + {ok, ResponseCode, Headers, Body} -> + lager:info("Test Result did not generate the expected 2XX HTTP response code."), + lager:debug("Post"), + lager:debug("Response Code: ~p", [ResponseCode]), + lager:debug("Headers: ~p", [Headers]), + lager:debug("Body: ~p", [Body]), + error; + X -> + lager:warning("Some error POSTing test result: ~p", [X]), + error + catch + Class:Reason -> + lager:error("Error reporting to ~s. ~p:~p", [Name, Class, Reason]), + lager:error("Payload: ~p", [TestResult]), + error + end. %%%=================================================================== %%% Bucket Types Functions diff --git a/src/rt_node.erl b/src/rt_node.erl index 22a1268f9..88796627c 100644 --- a/src/rt_node.erl +++ b/src/rt_node.erl @@ -77,10 +77,8 @@ async_start(Node, Version) -> %% @doc Stop the specified Riak `Node'. stop(Node, Version) -> - lager:info("Stopping riak on ~p", [Node]), - %% timer:sleep(10000), %% I know, I know! + lager:info("Stopping riak version ~p on ~p", [Version, Node]), rt_harness:stop(Node, Version). - %%rpc:call(Node, init, stop, []). %% @doc Stop the specified Riak `Node' and wait until it is not pingable -spec stop_and_wait(string(), node(), string()) -> ok. diff --git a/src/rtdev.erl b/src/rtdev.erl index 7355a9e10..498588b8f 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -660,22 +660,24 @@ stop_nodes(Path, Nodes) -> ok. stop(Node, Version) -> - case rpc:call(?DEV(Node), os, getpid, []) of + {NodeId, NodeName} = extract_node_id_and_name(Node), + lager:debug("Stopping node ~p using node name ~p", [NodeId, NodeName]), + case rpc:call(NodeName, os, getpid, []) of {badrpc, nodedown} -> ok; RiakPid -> %% rt_cover:maybe_stop_on_node(Node), - run_riak(Node, Version, "stop"), + run_riak(NodeId, Version, "stop"), F = fun(_N) -> os:cmd("kill -0 " ++ RiakPid) =/= [] end, - ?assertEqual(ok, rt:wait_until(?DEV(Node), F)), + ?assertEqual(ok, rt:wait_until(NodeName, F)), ok end. start(Node, Version) -> - %% N = node_id(Node), - run_riak(Node, Version, "start"), + {NodeId, _} = extract_node_id_and_name(Node), + run_riak(NodeId, Version, "start"), ok. attach(Node, Expected) -> @@ -888,3 +890,51 @@ get_node_logs() -> {ok, Port} = file:open(Filename, [read, binary]), {lists:nthtail(RootLen, Filename), Port} end || Filename <- filelib:wildcard(Root ++ "/*/dev/dev*/log/*") ]. + +-type node_tuple() :: {list(), atom()}. + +-spec extract_node_id_and_name(atom() | string()) -> node_tuple(). +extract_node_id_and_name(Node) when is_atom(Node) -> + NodeStr = atom_to_list(Node), + extract_node_id_and_name(NodeStr, contains(NodeStr, $@)); +extract_node_id_and_name(Node) when is_list(Node) -> + extract_node_id_and_name(Node, contains(Node, $@)); +extract_node_id_and_name(_Node) -> + erlang:error(unsupported_node_type). + +-spec extract_node_id_and_name(list(), boolean()) -> node_tuple(). +extract_node_id_and_name(Node, true) -> + [NodeId, _] = re:split(Node, "@"), + {binary_to_list(NodeId), list_to_atom(Node)}; +extract_node_id_and_name(Node, false) -> + {Node, ?DEV(Node)}. + +-spec contains(list(), char()) -> boolean. +contains(Str, Char) -> + maybe_contains(string:chr(Str, Char)). + +-spec maybe_contains(integer()) -> boolean. +maybe_contains(Pos) when Pos > 0 -> + true; +maybe_contains(_) -> + false. + +-ifdef(TEST). + +extract_node_id_and_name_test() -> + Expected = {"dev2", 'dev2@127.0.0.1'}, + ?assertEqual(Expected, extract_node_id_and_name('dev2@127.0.0.1')), + ?assertEqual(Expected, extract_node_id_and_name("dev2@127.0.0.1")), + ?assertEqual(Expected, extract_node_id_and_name("dev2")). + +maybe_contains_test() -> + ?assertEqual(true, maybe_contains(1)), + ?assertEqual(true, maybe_contains(10)), + ?assertEqual(false, maybe_contains(0)). + + +contains_test() -> + ?assertEqual(true, contains("dev2@127.0.0.1", $@)), + ?assertEqual(false, contains("dev2", $@)). + +-endif. From cadd8da3835c8c60a9b7c03c84fb38b731182350 Mon Sep 17 00:00:00 2001 From: vagrant Date: Sat, 21 Feb 2015 20:59:38 +0000 Subject: [PATCH 067/157] Update rtdev:deploy_nodes/1 - Added rtdev:devrel_node_name to build proper names - Updated Path names for r_t 2.0 --- src/rt.erl | 4 +- src/rt2.erl | 6 +- src/rtdev.erl | 153 ++++++++++++++++++++++++++------------------------ 3 files changed, 87 insertions(+), 76 deletions(-) diff --git a/src/rt.erl b/src/rt.erl index 264eab54b..900cfff4c 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -1,6 +1,6 @@ %% ------------------------------------------------------------------- %% -%% Copyright (c) 2013-2014 Basho Technologies, Inc. +%% Copyright (c) 2013-2015 Basho Technologies, Inc. %% %% This file is provided to you under the Apache License, %% Version 2.0 (the "License"); you may not use this file @@ -371,7 +371,7 @@ staged_join(Node, PNode) -> rt_node:staged_join(Node, PNode). plan_and_commit(Node) -> - rt_node:plan_and_comment(Node). + rt_node:plan_and_commit(Node). do_commit(Node) -> rt_node:do_commit(Node). diff --git a/src/rt2.erl b/src/rt2.erl index 560dbc6ac..48c021268 100644 --- a/src/rt2.erl +++ b/src/rt2.erl @@ -1,6 +1,6 @@ %% ------------------------------------------------------------------- %% -%% Copyright (c) 2013-2014 Basho Technologies, Inc. +%% Copyright (c) 2013-2015 Basho Technologies, Inc. %% %% This file is provided to you under the Apache License, %% Version 2.0 (the "License"); you may not use this file @@ -590,7 +590,9 @@ log_to_nodes(Nodes0, LFmt, LArgs) -> end, [rpc:call(Node, Module, Function, Args) || Node <- lists:flatten(Nodes)]. -%% @private utility function +%% @doc Parallel Map: Runs function F for each item in list L, then +%% returns the list of results +-spec pmap(F :: fun(), L :: list()) -> list(). pmap(F, L) -> Parent = self(), lists:foldl( diff --git a/src/rtdev.erl b/src/rtdev.erl index 498588b8f..5b4f3a0a6 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -57,16 +57,23 @@ -define(PATH, (rt_config:get(root_path))). -define(SCRATCH_DIR, (rt_config:get(rt_scratch_dir))). +%% @doc Convert a node number into a devrel node name +-spec devrel_node_name(N :: integer()) -> atom(). +devrel_node_name(N) when is_integer(N) -> + list_to_atom(lists:concat(["dev", N, "@127.0.0.1"])). + get_deps() -> DefaultVersionPath = filename:join(?PATH, rt_config:get(default_version)), lists:flatten(io_lib:format("~s/dev1/lib", [DefaultVersionPath])). +%% @doc Create a command-line command +-spec riakcmd(string(), integer(), string()) -> string(). riakcmd(Path, N, Cmd) -> ExecName = rt_config:get(exec_name, "riak"), - io_lib:format("~s/~s/bin/~s ~s", [Path, N, ExecName, Cmd]). + io_lib:format("~s/dev~B/bin/~s ~s", [Path, N, ExecName, Cmd]). riakreplcmd(Path, N, Cmd) -> - io_lib:format("~s/~s/bin/riak-repl ~s", [Path, N, Cmd]). + io_lib:format("~s/dev~B/bin/riak-repl ~s", [Path, N, Cmd]). gitcmd(Path, Cmd) -> io_lib:format("git --git-dir=\"~s/.git\" --work-tree=\"~s/\" ~s", @@ -81,13 +88,15 @@ riak_admin_cmd(Path, N, Args) -> end, Args), ArgStr = string:join(Quoted, " "), ExecName = rt_config:get(exec_name, "riak"), - io_lib:format("~s/~s/bin/~s-admin ~s", [Path, N, ExecName, ArgStr]). + io_lib:format("~s/dev~B/bin/~s-admin ~s", [Path, N, ExecName, ArgStr]). run_git(Path, Cmd) -> lager:info("Running: ~s", [gitcmd(Path, Cmd)]), {0, Out} = cmd(gitcmd(Path, Cmd)), Out. +%% @doc Run a riak command line command, returning its result +-spec run_riak(Node :: string(), Version :: string(), string()) -> string(). run_riak(Node, Version, "start") -> VersionPath = filename:join(?PATH, Version), RiakCmd = riakcmd(VersionPath, Node, "start"), @@ -96,9 +105,9 @@ run_riak(Node, Version, "start") -> %% rt_cover:maybe_start_on_node(?DEV(Node), Version), %% Intercepts may load code on top of the cover compiled %% modules. We'll just get no coverage info then. - case rt_intercept:are_intercepts_loaded(?DEV(Node)) of + case rt_intercept:are_intercepts_loaded(devrel_node_name(Node)) of false -> - ok = rt_intercept:load_intercepts([?DEV(Node)]); + ok = rt_intercept:load_intercepts([devrel_node_name(Node)]); true -> ok end, @@ -194,18 +203,17 @@ setup_harness() -> rm_dir(filename:join(?SCRATCH_DIR, "gc")), rt_harness_util:setup_harness(VersionMap, NodeIds, NodeMap). +%% @doc Tack the version onto the end of the root path by looking +%% up the root in the configuation +-spec relpath(Vsn :: string()) -> string(). relpath(Vsn) -> Path = ?PATH, relpath(Vsn, Path). -relpath(Vsn, Paths=[{_,_}|_]) -> - orddict:fetch(Vsn, orddict:from_list(Paths)); -relpath(current, Path) -> - Path; -relpath(root, Path) -> - Path; -relpath(_, _) -> - throw("Version requested but only one path provided"). +%% @doc Tack the version onto the end of the root path +-spec relpath(Vsn :: string(), Path :: string()) -> string(). +relpath(Vsn, Path) -> + lists:concat([Path, "/", Vsn]). %% upgrade(Node, CurrentVersion, NewVersion) -> %% upgrade(Node, CurrentVersion, NewVersion, same). @@ -220,7 +228,7 @@ upgrade(Node, CurrentVersion, NewVersion, Config) -> io_lib:format("cp -p -P -R \"~s\" \"~s\"", [filename:join(CurrentPath, "data"), NewPath]), - %% io_lib:format("rm -rf ~s/dev/dev~b/data/*", + %% io_lib:format("rm -rf ~s/dev~b/data/*", %% [CurrePath, N]), io_lib:format("cp -p -P -R \"~s\" \"~s\"", [filename:join(CurrentPath, "etc"), @@ -280,7 +288,7 @@ make_advanced_confs(DevPath) -> lager:error("Failed generating advanced.conf ~p is not a directory.", [DevPath]), []; true -> - Wildcard = io_lib:format("~s/dev/dev*/etc", [DevPath]), + Wildcard = io_lib:format("~s/dev*/etc", [DevPath]), ConfDirs = filelib:wildcard(Wildcard), [ begin @@ -294,12 +302,12 @@ make_advanced_confs(DevPath) -> get_riak_conf(Node) -> N = node_id(Node), Path = relpath(node_version(N)), - io_lib:format("~s/dev/dev~b/etc/riak.conf", [Path, N]). + io_lib:format("~s/dev~b/etc/riak.conf", [Path, N]). get_advanced_riak_conf(Node) -> N = node_id(Node), Path = relpath(node_version(N)), - io_lib:format("~s/dev/dev~b/etc/advanced.config", [Path, N]). + io_lib:format("~s/dev~b/etc/advanced.config", [Path, N]). append_to_conf_file(File, NameValuePairs) -> Settings = lists:flatten( @@ -309,7 +317,7 @@ append_to_conf_file(File, NameValuePairs) -> all_the_files(DevPath, File) -> case filelib:is_dir(DevPath) of true -> - Wildcard = io_lib:format("~s/dev/dev*/~s", [DevPath, File]), + Wildcard = io_lib:format("~s/dev*/~s", [DevPath, File]), filelib:wildcard(Wildcard); _ -> lager:debug("~s is not a directory.", [DevPath]), @@ -320,7 +328,7 @@ all_the_app_configs(DevPath) -> AppConfigs = all_the_files(DevPath, "etc/app.config"), case length(AppConfigs) =:= 0 of true -> - AdvConfigs = filelib:wildcard(DevPath ++ "/dev/dev*/etc"), + AdvConfigs = filelib:wildcard(DevPath ++ "/dev*/etc"), [ filename:join(AC, "advanced.config") || AC <- AdvConfigs]; _ -> AppConfigs @@ -408,7 +416,7 @@ get_backend(AppConfig) -> case filename:pathtype(Files) of absolute -> File; relative -> - io_lib:format("~s/dev/dev~s/~s", [Path, N, tl(hd(Files))]) + io_lib:format("~s/dev~s/~s", [Path, N, tl(hd(Files))]) end end end, @@ -424,7 +432,7 @@ get_backend(AppConfig) -> node_path(Node) -> N = node_id(Node), Path = relpath(node_version(N)), - lists:flatten(io_lib:format("~s/dev/dev~b", [Path, N])). + lists:flatten(io_lib:format("~s/dev~b", [Path, N])). get_ip(_Node) -> %% localhost 4 lyfe @@ -519,57 +527,57 @@ configure_nodes(Nodes, Configs) -> lists:zip(Nodes, Configs)). deploy_nodes(NodeConfig) -> - Path = relpath(root), - lager:info("Riak path: ~p", [Path]), - NumNodes = length(NodeConfig), - %% TODO: The starting index should not be fixed to 1 - NodesN = lists:seq(1, NumNodes), - Nodes = [?DEV(N) || N <- NodesN], - NodeMap = orddict:from_list(lists:zip(Nodes, NodesN)), - {Versions, Configs} = lists:unzip(NodeConfig), - VersionMap = lists:zip(NodesN, Versions), - - %% TODO The new node deployment doesn't appear to perform this check ... -jsb - %% Check that you have the right versions available - %%[ check_node(Version) || Version <- VersionMap ], - rt_config:set(rt_nodes, NodeMap), - rt_config:set(rt_versions, VersionMap), - - create_dirs(Nodes), - - %% Set initial config - add_default_node_config(Nodes), - rt:pmap(fun({_, default}) -> - ok; - ({Node, {cuttlefish, Config}}) -> - set_conf(Node, Config); - ({Node, Config}) -> - update_app_config(Node, Config) - end, - lists:zip(Nodes, Configs)), - - %% create snmp dirs, for EE - create_dirs(Nodes), - - %% Start nodes - %%[run_riak(N, relpath(node_version(N)), "start") || N <- Nodes], - rt:pmap(fun(N) -> run_riak(N, relpath(node_version(N)), "start") end, NodesN), - - %% Ensure nodes started - [ok = rt:wait_until_pingable(N) || N <- Nodes], - - %% %% Enable debug logging - %% [rpc:call(N, lager, set_loglevel, [lager_console_backend, debug]) || N <- Nodes], - - %% We have to make sure that riak_core_ring_manager is running before we can go on. - [ok = rt:wait_until_registered(N, riak_core_ring_manager) || N <- Nodes], - - %% Ensure nodes are singleton clusters - [ok = rt_ring:check_singleton_node(?DEV(N)) || {N, Version} <- VersionMap, - Version /= "0.14.2"], - - lager:info("Deployed nodes: ~p", [Nodes]), - Nodes. + Path = relpath(root), + lager:info("Riak path: ~p", [Path]), + NumNodes = length(NodeConfig), + %% TODO: The starting index should not be fixed to 1 + NodesN = lists:seq(1, NumNodes), + Nodes = [devrel_node_name(N) || N <- NodesN], + NodeMap = orddict:from_list(lists:zip(Nodes, NodesN)), + {Versions, Configs} = lists:unzip(NodeConfig), + VersionMap = lists:zip(NodesN, Versions), + + %% TODO The new node deployment doesn't appear to perform this check ... -jsb + %% Check that you have the right versions available + %%[ check_node(Version) || Version <- VersionMap ], + rt_config:set(rt_nodes, NodeMap), + rt_config:set(rt_versions, VersionMap), + + create_dirs(Nodes), + + %% Set initial config + add_default_node_config(Nodes), + rt:pmap(fun({_, default}) -> + ok; + ({Node, {cuttlefish, Config}}) -> + set_conf(Node, Config); + ({Node, Config}) -> + update_app_config(Node, Config) + end, + lists:zip(Nodes, Configs)), + + %% create snmp dirs, for EE + create_dirs(Nodes), + + %% Start nodes + %%[run_riak(N, relpath(node_version(N)), "start") || N <- NodesN], + rt:pmap(fun(N) -> run_riak(N, relpath(node_version(N)), "start") end, NodesN), + + %% Ensure nodes started + [ok = rt:wait_until_pingable(N) || N <- Nodes], + + %% %% Enable debug logging + %% [rpc:call(N, lager, set_loglevel, [lager_console_backend, debug]) || N <- Nodes], + + %% We have to make sure that riak_core_ring_manager is running before we can go on. + [ok = rt:wait_until_registered(N, riak_core_ring_manager) || N <- Nodes], + + %% Ensure nodes are singleton clusters + [ok = rt_ring:check_singleton_node(devrel_node_name(N)) || {N, Version} <- VersionMap, + Version /= "0.14.2"], + + lager:info("Deployed nodes: ~p", [Nodes]), + Nodes. gen_stop_fun(Path, Timeout) -> fun(Node) -> @@ -801,6 +809,7 @@ node_id(Node) -> NodeMap = rt_config:get(rt_nodes), orddict:fetch(Node, NodeMap). +%% @doc Return the node version from rt_versions node_version(N) -> VersionMap = rt_config:get(rt_versions), orddict:fetch(N, VersionMap). From 10d1b94d0e44dde1c37c6c176220205f26902ec6 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Sat, 21 Feb 2015 23:56:28 +0000 Subject: [PATCH 068/157] Adjust VersionMap to key off dev[0-9] and path names appropriately --- src/rt.erl | 2 +- src/rt_harness_util.erl | 2 +- src/rtdev.erl | 82 ++++++++++++++++++++--------------------- 3 files changed, 42 insertions(+), 44 deletions(-) diff --git a/src/rt.erl b/src/rt.erl index 900cfff4c..9964027a6 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -252,7 +252,7 @@ deploy_nodes(Versions, Services) -> NodeConfig = [ version_to_config(Version) || Version <- Versions ], lager:debug("Starting nodes config ~p using versions ~p", [NodeConfig, Versions]), - Nodes = ?HARNESS:deploy_nodes(NodeConfig), + Nodes = rt_harness:deploy_nodes(NodeConfig), lager:info("Waiting for services ~p to start on ~p.", [Services, Nodes]), [ ok = wait_for_service(Node, Service) || Node <- Nodes, Service <- Services ], diff --git a/src/rt_harness_util.erl b/src/rt_harness_util.erl index bdb4a33c8..4e8305c64 100644 --- a/src/rt_harness_util.erl +++ b/src/rt_harness_util.erl @@ -98,7 +98,7 @@ deploy_nodes(NodeIds, NodeMap, Version, Config, Services) -> %% Start nodes RunRiakFun = fun(Node) -> - rt_harness:run_riak(Node, Version, "start") + rt_harness:run_riak(Node, Version, "start") end, rt:pmap(RunRiakFun, NodeIds), diff --git a/src/rtdev.erl b/src/rtdev.erl index 5b4f3a0a6..eb2230565 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -67,18 +67,22 @@ get_deps() -> lists:flatten(io_lib:format("~s/dev1/lib", [DefaultVersionPath])). %% @doc Create a command-line command --spec riakcmd(string(), integer(), string()) -> string(). +-spec riakcmd(Path :: string(), N :: string(), Cmd :: string()) -> string(). riakcmd(Path, N, Cmd) -> ExecName = rt_config:get(exec_name, "riak"), - io_lib:format("~s/dev~B/bin/~s ~s", [Path, N, ExecName, Cmd]). + io_lib:format("~s/~s/bin/~s ~s", [Path, N, ExecName, Cmd]). +%% @doc Create a command-line command for repl +-spec riakreplcmd(Path :: string(), N :: string(), Cmd :: string()) -> string(). riakreplcmd(Path, N, Cmd) -> - io_lib:format("~s/dev~B/bin/riak-repl ~s", [Path, N, Cmd]). + io_lib:format("~s/~s/bin/riak-repl ~s", [Path, N, Cmd]). gitcmd(Path, Cmd) -> io_lib:format("git --git-dir=\"~s/.git\" --work-tree=\"~s/\" ~s", [Path, Path, Cmd]). +%% @doc Create a command-line command for riak-admin +-spec riak_admin_cmd(Path :: string(), N :: string(), Args :: string()) -> string(). riak_admin_cmd(Path, N, Args) -> Quoted = lists:map(fun(Arg) when is_list(Arg) -> @@ -88,7 +92,7 @@ riak_admin_cmd(Path, N, Args) -> end, Args), ArgStr = string:join(Quoted, " "), ExecName = rt_config:get(exec_name, "riak"), - io_lib:format("~s/dev~B/bin/~s-admin ~s", [Path, N, ExecName, ArgStr]). + io_lib:format("~s/~s/bin/~s-admin ~s", [Path, N, ExecName, ArgStr]). run_git(Path, Cmd) -> lager:info("Running: ~s", [gitcmd(Path, Cmd)]), @@ -105,9 +109,9 @@ run_riak(Node, Version, "start") -> %% rt_cover:maybe_start_on_node(?DEV(Node), Version), %% Intercepts may load code on top of the cover compiled %% modules. We'll just get no coverage info then. - case rt_intercept:are_intercepts_loaded(devrel_node_name(Node)) of + case rt_intercept:are_intercepts_loaded(?DEV(Node)) of false -> - ok = rt_intercept:load_intercepts([devrel_node_name(Node)]); + ok = rt_intercept:load_intercepts([?DEV(Node)]); true -> ok end, @@ -300,14 +304,12 @@ make_advanced_confs(DevPath) -> end. get_riak_conf(Node) -> - N = node_id(Node), - Path = relpath(node_version(N)), - io_lib:format("~s/dev~b/etc/riak.conf", [Path, N]). + Path = relpath(node_version(Node)), + io_lib:format("~s/~s/etc/riak.conf", [Path, Node]). get_advanced_riak_conf(Node) -> - N = node_id(Node), - Path = relpath(node_version(N)), - io_lib:format("~s/dev~b/etc/advanced.config", [Path, N]). + Path = relpath(node_version(Node)), + io_lib:format("~s/~s/etc/advanced.config", [Path, Node]). append_to_conf_file(File, NameValuePairs) -> Settings = lists:flatten( @@ -430,9 +432,8 @@ get_backend(AppConfig) -> end. node_path(Node) -> - N = node_id(Node), - Path = relpath(node_version(N)), - lists:flatten(io_lib:format("~s/dev~b", [Path, N])). + Path = relpath(node_version(Node)), + lists:flatten(io_lib:format("~s/~s", [Path, Node])). get_ip(_Node) -> %% localhost 4 lyfe @@ -532,10 +533,11 @@ deploy_nodes(NodeConfig) -> NumNodes = length(NodeConfig), %% TODO: The starting index should not be fixed to 1 NodesN = lists:seq(1, NumNodes), - Nodes = [devrel_node_name(N) || N <- NodesN], - NodeMap = orddict:from_list(lists:zip(Nodes, NodesN)), + FullNodes = [devrel_node_name(N) || N <- NodesN], + DevNodes = [list_to_atom(lists:concat(["dev", N])) || N <- NodesN], + NodeMap = orddict:from_list(lists:zip(FullNodes, NodesN)), {Versions, Configs} = lists:unzip(NodeConfig), - VersionMap = lists:zip(NodesN, Versions), + VersionMap = lists:zip(DevNodes, Versions), %% TODO The new node deployment doesn't appear to perform this check ... -jsb %% Check that you have the right versions available @@ -543,10 +545,10 @@ deploy_nodes(NodeConfig) -> rt_config:set(rt_nodes, NodeMap), rt_config:set(rt_versions, VersionMap), - create_dirs(Nodes), + create_dirs(DevNodes), %% Set initial config - add_default_node_config(Nodes), + add_default_node_config(FullNodes), rt:pmap(fun({_, default}) -> ok; ({Node, {cuttlefish, Config}}) -> @@ -554,30 +556,30 @@ deploy_nodes(NodeConfig) -> ({Node, Config}) -> update_app_config(Node, Config) end, - lists:zip(Nodes, Configs)), + lists:zip(FullNodes, Configs)), %% create snmp dirs, for EE - create_dirs(Nodes), + create_dirs(DevNodes), %% Start nodes %%[run_riak(N, relpath(node_version(N)), "start") || N <- NodesN], - rt:pmap(fun(N) -> run_riak(N, relpath(node_version(N)), "start") end, NodesN), + rt:pmap(fun(Node) -> run_riak(Node, relpath(node_version(Node)), "start") end, DevNodes), %% Ensure nodes started - [ok = rt:wait_until_pingable(N) || N <- Nodes], + [ok = rt:wait_until_pingable(N) || N <- FullNodes], %% %% Enable debug logging %% [rpc:call(N, lager, set_loglevel, [lager_console_backend, debug]) || N <- Nodes], %% We have to make sure that riak_core_ring_manager is running before we can go on. - [ok = rt:wait_until_registered(N, riak_core_ring_manager) || N <- Nodes], + [ok = rt:wait_until_registered(N, riak_core_ring_manager) || N <- FullNodes], %% Ensure nodes are singleton clusters - [ok = rt_ring:check_singleton_node(devrel_node_name(N)) || {N, Version} <- VersionMap, + [ok = rt_ring:check_singleton_node(?DEV(Node)) || {Node, Version} <- VersionMap, Version /= "0.14.2"], - lager:info("Deployed nodes: ~p", [Nodes]), - Nodes. + lager:info("Deployed nodes: ~p", [FullNodes]), + FullNodes. gen_stop_fun(Path, Timeout) -> fun(Node) -> @@ -698,9 +700,8 @@ console(Node, Expected) -> interactive(Node, "console", Expected). interactive(Node, Command, Exp) -> - N = node_id(Node), - Path = relpath(node_version(N)), - Cmd = riakcmd(Path, N, Command), + Path = relpath(node_version(Node)), + Cmd = riakcmd(Path, Node, Command), lager:debug("Opening a port for riak ~s.", [Command]), lager:debug("Calling open_port with cmd ~s", [binary_to_list(iolist_to_binary(Cmd))]), P = open_port({spawn, binary_to_list(iolist_to_binary(Cmd))}, @@ -773,9 +774,8 @@ interactive_loop(Port, Expected) -> end. admin(Node, Args, Options) -> - N = node_id(Node), - Path = relpath(node_version(N)), - Cmd = riak_admin_cmd(Path, N, Args), + Path = relpath(node_version(Node)), + Cmd = riak_admin_cmd(Path, Node, Args), lager:info("Running: ~s", [Cmd]), Result = execute_admin_cmd(Cmd, Options), lager:info("~p", [Result]), @@ -791,17 +791,15 @@ execute_admin_cmd(Cmd, Options) -> end. riak(Node, Args) -> - N = node_id(Node), - Path = relpath(node_version(N)), - Result = run_riak(N, Path, Args), + Path = relpath(node_version(Node)), + Result = run_riak(Node, Path, Args), lager:info("~s", [Result]), {ok, Result}. riak_repl(Node, Args) -> - N = node_id(Node), - Path = relpath(node_version(N)), - Result = run_riak_repl(N, Path, Args), + Path = relpath(node_version(Node)), + Result = run_riak_repl(Node, Path, Args), lager:info("~s", [Result]), {ok, Result}. @@ -810,9 +808,9 @@ node_id(Node) -> orddict:fetch(Node, NodeMap). %% @doc Return the node version from rt_versions -node_version(N) -> +node_version(Node) -> VersionMap = rt_config:get(rt_versions), - orddict:fetch(N, VersionMap). + orddict:fetch(Node, VersionMap). spawn_cmd(Cmd) -> spawn_cmd(Cmd, []). From fb5854bac4384f0823fe25de05a87f31ff7dd43e Mon Sep 17 00:00:00 2001 From: John Burwell Date: Tue, 24 Feb 2015 13:52:13 -0500 Subject: [PATCH 069/157] Adds additional debug logging to rtdev and pulls the master version of a variety of test cases --- src/rtdev.erl | 7 +- tests/basic_command_line.erl | 53 ++++----- tests/ensemble_interleave.erl | 16 +-- tests/ensemble_util.erl | 12 +- tests/gh_riak_core_155.erl | 10 +- tests/gh_riak_kv_765.erl | 12 +- tests/jmx_verify.erl | 30 ++--- tests/loaded_upgrade.erl | 14 +-- tests/overload.erl | 1 + tests/partition_repair.erl | 12 +- tests/pb_security.erl | 22 ++-- tests/replication/repl_fs_stat_caching.erl | 8 +- tests/replication/repl_rt_cascading_rtq.erl | 10 +- tests/replication/repl_util.erl | 22 ++-- tests/replication/replication2.erl | 18 +-- .../replication2_console_tests.erl | 5 +- tests/replication/replication2_pg.erl | 110 +++++++++--------- .../replication_object_reformat.erl | 14 +-- tests/riak_control.erl | 4 +- tests/riak_control_authentication.erl | 8 +- tests/verify_2i_aae.erl | 50 +++----- tests/verify_aae.erl | 12 +- tests/verify_bitcask_tombstone2_upgrade.erl | 8 +- tests/verify_build_cluster.erl | 62 +++++----- tests/verify_capabilities.erl | 17 ++- tests/verify_dt_converge.erl | 15 ++- tests/verify_handoff.erl | 29 ++--- tests/verify_handoff_mixed.erl | 10 +- tests/verify_leave.erl | 4 +- tests/verify_membackend.erl | 40 +++---- tests/verify_object_limits.erl | 6 +- tests/verify_riak_stats.erl | 12 +- tests/verify_search.erl | 4 +- tests/verify_staged_clustering.erl | 30 ++--- tests/yz_crdt.erl | 2 +- 35 files changed, 340 insertions(+), 349 deletions(-) diff --git a/src/rtdev.erl b/src/rtdev.erl index eb2230565..27b0e2a96 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -432,14 +432,17 @@ get_backend(AppConfig) -> end. node_path(Node) -> - Path = relpath(node_version(Node)), - lists:flatten(io_lib:format("~s/~s", [Path, Node])). + N = node_id(Node), + lager:debug("Node ~p node id ~p", [Node, N]), + Path = relpath(node_version(N)), + lists:flatten(io_lib:format("~s/dev/dev~b", [Path, N])). get_ip(_Node) -> %% localhost 4 lyfe "127.0.0.1". create_dirs(Nodes) -> + lager:debug("Nodes ~p", [Nodes]), Snmp = [node_path(Node) ++ "/data/snmp/agent/db" || Node <- Nodes], [?assertCmd("mkdir -p " ++ Dir) || Dir <- Snmp]. diff --git a/tests/basic_command_line.erl b/tests/basic_command_line.erl index b752ec1dd..217cc624f 100644 --- a/tests/basic_command_line.erl +++ b/tests/basic_command_line.erl @@ -19,18 +19,17 @@ %% ------------------------------------------------------------------- -module(basic_command_line). -include_lib("eunit/include/eunit.hrl"). --include("rt.hrl"). --export([properties/0, confirm/2]). +-behavior(riak_test). +-compile(export_all). +-export([confirm/0]). -properties() -> - rt_properties:new([{node_count, 1}]). +confirm() -> -confirm(Properties, _MD) -> - Nodes = rt_properties:get(nodes, Properties), - Node = hd(Nodes), %% Deploy a node to test against lager:info("Deploy node to test command line"), + [Node] = rt:deploy_nodes(1), + ?assertEqual(ok, rt:wait_until_nodes_ready([Node])), %% Verify node-up behavior ping_up_test(Node), @@ -54,7 +53,7 @@ confirm(Properties, _MD) -> console_up_test(Node) -> lager:info("Node is already up, `riak console` should fail"), - {ok, ConsoleFail} = rt_cmd_line:riak(Node, ["console"]), + {ok, ConsoleFail} = rt:riak(Node, ["console"]), ?assert(rt:str(ConsoleFail, "Node is already running")), ok. @@ -63,7 +62,7 @@ console_test(Node) -> lager:info("Testing riak console on ~s", [Node]), %% Stop node, to test console working - rt_cmd_line:console(Node, [{expect, "\(abort with ^G\)"}, + rt:console(Node, [{expect, "\(abort with ^G\)"}, {send, "riak_core_ring_manager:get_my_ring()."}, {expect, "dict,"}, {send, "q()."}, @@ -74,7 +73,7 @@ console_test(Node) -> start_up_test(Node) -> %% Try starting again and check you get the node is already running message lager:info("Testing riak start now will return 'already running'"), - {ok, StartOut} = rt_cmd_line:riak(Node, ["start"]), + {ok, StartOut} = rt:riak(Node, ["start"]), ?assert(rt:str(StartOut, "Node is already running!")), ok. @@ -83,17 +82,18 @@ start_test(Node) -> %% Test starting with /bin/riak start lager:info("Testing riak start works on ~s", [Node]), - {ok, StartPass} = rt_cmd_line:riak(Node, ["start"]), - ?assertMatch(StartPass, ""), - rt_node:stop_and_wait(Node), + {ok, StartPass} = rt:riak(Node, ["start"]), + lager:info("StartPass: ~p", [StartPass]), + ?assert(StartPass =:= "" orelse string:str(StartPass, "WARNING") =/= 0), + rt:stop_and_wait(Node), ok. stop_test(Node) -> - ?assert(rt_node:is_pingable(Node)), + ?assert(rt:is_pingable(Node)), - {ok, "ok\n"} = rt_cmd_line:riak(Node, "stop"), + {ok, "ok\n"} = rt:riak(Node, "stop"), - ?assertNot(rt_node:is_pingable(Node)), + ?assertNot(rt:is_pingable(Node)), ok. ping_up_test(Node) -> @@ -102,29 +102,29 @@ ping_up_test(Node) -> lager:info("Testing riak ping on ~s", [Node]), %% ping / pong - %% rt_node:start_and_wait(Node), + %% rt:start_and_wait(Node), lager:info("Node up, should ping"), - {ok, PongOut} = rt_cmd_line:riak(Node, ["ping"]), + {ok, PongOut} = rt:riak(Node, ["ping"]), ?assert(rt:str(PongOut, "pong")), ok. ping_down_test(Node) -> %% ping / pang lager:info("Node down, should pang"), - {ok, PangOut} = rt_cmd_line:riak(Node, ["ping"]), + {ok, PangOut} = rt:riak(Node, ["ping"]), ?assert(rt:str(PangOut, "not responding to pings")), ok. attach_down_test(Node) -> lager:info("Testing riak attach while down"), - {ok, AttachOut} = rt_cmd_line:riak(Node, ["attach"]), + {ok, AttachOut} = rt:riak(Node, ["attach"]), ?assert(rt:str(AttachOut, "Node is not running!")), ok. attach_direct_up_test(Node) -> lager:info("Testing riak attach-direct"), - rt_cmd_line:attach_direct(Node, [{expect, "\(^D to exit\)"}, + rt:attach_direct(Node, [{expect, "\(^D to exit\)"}, {send, "riak_core_ring_manager:get_my_ring()."}, {expect, "dict,"}, {send, [4]}]), %% 4 = Ctrl + D @@ -132,7 +132,7 @@ attach_direct_up_test(Node) -> attach_direct_down_test(Node) -> lager:info("Testing riak attach-direct while down"), - {ok, AttachOut} = rt_cmd_line:riak(Node, ["attach-direct"]), + {ok, AttachOut} = rt:riak(Node, ["attach-direct"]), ?assert(rt:str(AttachOut, "Node is not running!")), ok. @@ -149,19 +149,20 @@ status_up_test(Node) -> status_down_test(Node) -> lager:info("Test riak-admin status while down"), - {ok, StatusOut} = rt_cmd_line:admin(Node, ["status"]), + {ok, {ExitCode, StatusOut}} = rt:admin(Node, ["status"], [return_exit_code]), + ?assertEqual(1, ExitCode), ?assert(rt:str(StatusOut, "Node is not running!")), ok. getpid_up_test(Node) -> lager:info("Test riak getpid on ~s", [Node]), - {ok, PidOut} = rt_cmd_line:riak(Node, ["getpid"]), + {ok, PidOut} = rt:riak(Node, ["getpid"]), ?assertNot(rt:str(PidOut, "")), ?assert(rt:str(PidOut, rpc:call(Node, os, getpid, []))), ok. getpid_down_test(Node) -> lager:info("Test riak getpid fails on ~s", [Node]), - {ok, PidOut} = rt_cmd_line:riak(Node, ["getpid"]), + {ok, PidOut} = rt:riak(Node, ["getpid"]), ?assert(rt:str(PidOut, "Node is not running!")), - ok. + ok. \ No newline at end of file diff --git a/tests/ensemble_interleave.erl b/tests/ensemble_interleave.erl index 2db3da566..36c79a2c2 100644 --- a/tests/ensemble_interleave.erl +++ b/tests/ensemble_interleave.erl @@ -46,7 +46,7 @@ confirm() -> vnode_util:load(Nodes), lager:info("Creating/activating 'strong' bucket type"), - rt_bucket_types:create_and_activate_bucket_type(Node, <<"strong">>, + rt:create_and_activate_bucket_type(Node, <<"strong">>, [{consistent, true}, {n_val, NVal}]), ensemble_util:wait_until_stable(Node, NVal), Bucket = {<<"strong">>, <<"test">>}, @@ -65,19 +65,19 @@ confirm() -> [KillFirst,KillSecond|Suspend] = All -- PartitionedVN, io:format("PL: ~p~n", [PL]), - PBC = rt_pb:pbc(Node), + PBC = rt:pbc(Node), Options = [{timeout, 500}], rpc:multicall(Nodes, riak_kv_entropy_manager, set_mode, [manual]), - Part = rt_node:partition(Nodes -- Partitioned, Partitioned), + Part = rt:partition(Nodes -- Partitioned, Partitioned), ensemble_util:wait_until_stable(Node, Quorum), lager:info("Writing ~p consistent keys", [1000]), - [ok = rt_pb:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], + [ok = rt:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], lager:info("Read keys to verify they exist"), - [rt_pb:pbc_read(PBC, Bucket, Key, Options) || Key <- Keys], - rt_node:heal(Part), + [rt:pbc_read(PBC, Bucket, Key, Options) || Key <- Keys], + rt:heal(Part), [begin lager:info("Suspending vnode: ~p", [VIdx]), @@ -97,5 +97,5 @@ confirm() -> lager:info("Re-reading keys to verify they exist"), Expect = [ok, {error, timeout}, {error, <<"timeout">>}, {error, <<"failed">>}], - [rt_pb:pbc_read_check(PBC, Bucket, Key, Expect, Options) || Key <- Keys], - pass. + [rt:pbc_read_check(PBC, Bucket, Key, Expect, Options) || Key <- Keys], + pass. \ No newline at end of file diff --git a/tests/ensemble_util.erl b/tests/ensemble_util.erl index f206df9de..14348ed53 100644 --- a/tests/ensemble_util.erl +++ b/tests/ensemble_util.erl @@ -26,16 +26,16 @@ -include_lib("eunit/include/eunit.hrl"). build_cluster(Num, Config, NVal) -> - Nodes = rt_cluster:deploy_nodes(Num, Config), + Nodes = rt:deploy_nodes(Num, Config), Node = hd(Nodes), - rt_cluster:join_cluster(Nodes), + rt:join_cluster(Nodes), ensemble_util:wait_until_cluster(Nodes), ensemble_util:wait_for_membership(Node), ensemble_util:wait_until_stable(Node, NVal), Nodes. build_cluster_without_quorum(Num, Config) -> - Nodes = rt_cluster:deploy_nodes(Num, Config), + Nodes = rt:deploy_nodes(Num, Config), SetupLogCaptureFun = fun(Node) -> rt:setup_log_capture(Node) end, @@ -43,7 +43,7 @@ build_cluster_without_quorum(Num, Config) -> Node = hd(Nodes), ok = rpc:call(Node, riak_ensemble_manager, enable, []), _ = rpc:call(Node, riak_core_ring_manager, force_update, []), - rt_cluster:join_cluster(Nodes), + rt:join_cluster(Nodes), ensemble_util:wait_until_cluster(Nodes), ensemble_util:wait_for_membership(Node), Nodes. @@ -69,7 +69,7 @@ config_aae(true) -> {anti_entropy_tick, 100}, {anti_entropy, {on, []}}, {anti_entropy_timeout, 5000}, - {storage_backend, riak_kv_memory_backend}]}; + {storage_backend, riak_kv_memory_backend}]}; config_aae(false) -> {riak_kv, [{anti_entropy, {off, []}}]}. @@ -157,4 +157,4 @@ wait_for_membership(Node) -> end, ?assertEqual(ok, rt:wait_until(F)), lager:info("....ownership matches"), - ok. + ok. \ No newline at end of file diff --git a/tests/gh_riak_core_155.erl b/tests/gh_riak_core_155.erl index 522e088f2..2abb856bd 100644 --- a/tests/gh_riak_core_155.erl +++ b/tests/gh_riak_core_155.erl @@ -24,7 +24,7 @@ -include_lib("eunit/include/eunit.hrl"). confirm() -> - [Node] = rt_cluster:build_cluster(1), + [Node] = rt:build_cluster(1), %% Generate a valid preflist for our get requests rpc:call(Node, riak_core, wait_for_service, [riak_kv]), @@ -34,12 +34,12 @@ confirm() -> lager:info("Adding delayed start to app.config"), NewConfig = [{riak_core, [{delayed_start, 1000}]}], - rt_config:update_app_config(Node, NewConfig), + rt:update_app_config(Node, NewConfig), %% Restart node, add intercept that delay proxy startup, and issue gets. %% Gets will come in before proxies started, and should trigger crash. - rt_node:stop_and_wait(Node), - rt_node:async_start(Node), + rt:stop_and_wait(Node), + rt:async_start(Node), rt:wait_until_pingable(Node), rt_intercept:load_intercepts([Node]), rt_intercept:add(Node, {riak_core_vnode_proxy_sup, @@ -66,4 +66,4 @@ perform_gets2(0, _, _, _) -> ok; perform_gets2(Count, Node, PL, BKey) -> rpc:call(Node, riak_kv_vnode, get, [PL, BKey, make_ref()], 1000), - perform_gets(Count - 1, Node, PL, BKey). + perform_gets(Count - 1, Node, PL, BKey). \ No newline at end of file diff --git a/tests/gh_riak_kv_765.erl b/tests/gh_riak_kv_765.erl index f43a45802..877a28288 100644 --- a/tests/gh_riak_kv_765.erl +++ b/tests/gh_riak_kv_765.erl @@ -41,7 +41,7 @@ confirm() -> check_empty_build() -> Config = [{riak_core, [{vnode_management_timer, 1000}, {ring_creation_size, 4}]}], - Nodes = rt_cluster:build_cluster(1, Config), + Nodes = rt:build_cluster(1, Config), Node = hd(Nodes), timer:sleep(2000), Self = self(), @@ -56,7 +56,7 @@ check_empty_build() -> lager:info("Failed. Empty AAE trees were not built instantly"), fail end, - rt_cluster:clean_cluster(Nodes), + rt:clean_cluster(Nodes), Result. check_throttle_and_expiration() -> @@ -66,12 +66,12 @@ check_throttle_and_expiration() -> {anti_entropy, {off, []}}]}, {riak_core, [{vnode_management_timer, 1000}, {ring_creation_size, 4}]}], - Nodes = rt_cluster:build_cluster(1, Config), + Nodes = rt:build_cluster(1, Config), Node = hd(Nodes), timer:sleep(2000), lager:info("Write 1000 keys"), - rt_systest:write(Node, 1000), + rt:systest_write(Node, 1000), enable_aae(Node), time_build(Node), Duration1 = rebuild(Node, 30000, 1000), @@ -88,7 +88,7 @@ check_throttle_and_expiration() -> time_build(Node) -> T0 = erlang:now(), - rt_aae:wait_until_aae_trees_built([Node]), + rt:wait_until_aae_trees_built([Node]), Duration = timer:now_diff(erlang:now(), T0), lager:info("Build took ~b us", [Duration]), Duration. @@ -120,4 +120,4 @@ expire_tree(Node, Partition) -> {0, Built} = lists:keyfind(0, 1, Info), Built > Now end), - ok. + ok. \ No newline at end of file diff --git a/tests/jmx_verify.erl b/tests/jmx_verify.erl index f9cc8ca70..b4255bf8e 100644 --- a/tests/jmx_verify.erl +++ b/tests/jmx_verify.erl @@ -37,9 +37,9 @@ confirm() -> JMXPort = 41111, Config = [{riak_jmx, [{enabled, true}, {port, JMXPort}]}], - Nodes = rt_cluster:deploy_nodes(1, Config), + Nodes = rt:deploy_nodes(1, Config), [Node1] = Nodes, - ?assertEqual(ok, rt_node:wait_until_nodes_ready([Node1])), + ?assertEqual(ok, rt:wait_until_nodes_ready([Node1])), [{http, {IP, _Port}}|_] = rt:connection_info(Node1), @@ -63,9 +63,9 @@ confirm() -> lager:info("perform 5 x PUT and a GET to increment the stats"), lager:info("as the stat system only does calcs for > 5 readings"), - C = rt_http:httpc(Node1), - [rt_http:httpc_write(C, <<"systest">>, <>, <<"12345">>) || X <- lists:seq(1, 5)], - [rt_http:httpc_read(C, <<"systest">>, <>) || X <- lists:seq(1, 5)], + C = rt:httpc(Node1), + [rt:httpc_write(C, <<"systest">>, <>, <<"12345">>) || X <- lists:seq(1, 5)], + [rt:httpc_read(C, <<"systest">>, <>) || X <- lists:seq(1, 5)], JMX2 = jmx_dump(JMXDumpCmd), %% make sure the stats that were supposed to increment did @@ -91,17 +91,17 @@ confirm() -> <<"node_put_fsm_time_100">>]), lager:info("Make PBC Connection"), - Pid = rt_pb:pbc(Node1), + Pid = rt:pbc(Node1), JMX3 = jmx_dump(JMXDumpCmd), - rt_systest:write(Node1, 1), + rt:systest_write(Node1, 1), %% make sure the stats that were supposed to increment did verify_inc(JMX2, JMX3, [{<<"pbc_connects_total">>, 1}, {<<"pbc_connects">>, 1}, {<<"pbc_active">>, 1}]), lager:info("Force Read Repair"), - rt_pb:pbc_write(Pid, <<"testbucket">>, <<"1">>, <<"blah!">>), + rt:pbc_write(Pid, <<"testbucket">>, <<"1">>, <<"blah!">>), rt:pbc_set_bucket_prop(Pid, <<"testbucket">>, [{n_val, 4}]), JMX4 = jmx_dump(JMXDumpCmd), @@ -109,7 +109,7 @@ confirm() -> verify_inc(JMX3, JMX4, [{<<"read_repairs_total">>, 0}, {<<"read_repairs">>, 0}]), - _Value = rt_pb:pbc_read(Pid, <<"testbucket">>, <<"1">>), + _Value = rt:pbc_read(Pid, <<"testbucket">>, <<"1">>), %%Stats5 = get_stats(Node1), JMX5 = jmx_dump(JMXDumpCmd), @@ -120,7 +120,7 @@ confirm() -> test_supervision() -> JMXPort = 41111, Config = [{riak_jmx, [{enabled, true}, {port, JMXPort}]}], - [Node|[]] = rt_cluster:deploy_nodes(1, Config), + [Node|[]] = rt:deploy_nodes(1, Config), timer:sleep(20000), case net_adm:ping(Node) of pang -> @@ -146,7 +146,7 @@ test_supervision() -> lager:info("It can fail, it can fail 10 times"), rt:wait_until(retry_check_fun(Node)), - rt_node:stop(Node), + rt:stop(Node), ok_ok. retry_check_fun(Node) -> @@ -172,9 +172,9 @@ test_application_stop() -> lager:info("Testing application:stop()"), JMXPort = 41111, Config = [{riak_jmx, [{enabled, true}, {port, JMXPort}]}], - Nodes = rt_cluster:deploy_nodes(1, Config), + Nodes = rt:deploy_nodes(1, Config), [Node] = Nodes, - ?assertEqual(ok, rt_node:wait_until_nodes_ready([Node])), + ?assertEqual(ok, rt:wait_until_nodes_ready([Node])), %% Let's make sure the java process is alive! lager:info("checking for riak_jmx.jar running."), @@ -205,7 +205,7 @@ test_application_stop() -> ?assertEqual(nomatch, re:run(rpc:call(Node, os, cmd, ["ps -Af"]), "riak_jmx.jar", [])), - rt_node:stop(Node). + rt:stop(Node). verify_inc(Prev, Props, Keys) -> [begin @@ -234,4 +234,4 @@ jmx_dump(Cmd) -> lager:info("Dumping JMX stats using command ~s", [Cmd]), Output = string:strip(os:cmd(Cmd), both, $\n), JSONOutput = mochijson2:decode(Output), - [ {Key, Value} || {struct, [{Key, Value}]} <- JSONOutput]. + [ {Key, Value} || {struct, [{Key, Value}]} <- JSONOutput]. \ No newline at end of file diff --git a/tests/loaded_upgrade.erl b/tests/loaded_upgrade.erl index 5feab5801..948bddb8b 100644 --- a/tests/loaded_upgrade.erl +++ b/tests/loaded_upgrade.erl @@ -43,7 +43,7 @@ confirm() -> Config = [{riak_search, [{enabled, true}]}, {riak_pipe, [{worker_limit, 200}]}], NumNodes = 4, Vsns = [{OldVsn, Config} || _ <- lists:seq(1,NumNodes)], - Nodes = rt_cluster:build_cluster(Vsns), + Nodes = rt:build_cluster(Vsns), seed_cluster(Nodes), @@ -116,8 +116,8 @@ seed_cluster(Nodes=[Node1|_]) -> %% For List Keys lager:info("Writing 100 keys to ~p", [Node1]), - rt_systest:write(Node1, 100, 3), - ?assertEqual([], rt_systest:read(Node1, 100, 1)), + rt:systest_write(Node1, 100, 3), + ?assertEqual([], rt:systest_read(Node1, 100, 1)), seed(Node1, 0, 100, fun(Key) -> Bin = iolist_to_binary(io_lib:format("~p", [Key])), @@ -145,7 +145,7 @@ bucket(mapred) -> <<"bryanitbs">>; bucket(search) -> <<"scotts_spam">>. seed_search(Node) -> - Pid = rt_pb:pbc(Node), + Pid = rt:pbc(Node), SpamDir = rt_config:get(spam_dir), Files = case SpamDir of undefined -> undefined; @@ -157,7 +157,7 @@ seed_search(Node) -> seed_search(_Pid, []) -> ok; seed_search(Pid, [File|Files]) -> Key = list_to_binary(filename:basename(File)), - rt_pb:pbc_put_file(Pid, bucket(search), Key, File), + rt:pbc_put_file(Pid, bucket(search), Key, File), seed_search(Pid, Files). kv_seed(Node) -> @@ -201,7 +201,7 @@ mr_seed(Node) -> seed(Node, 0, 9999, ValFun). seed(Node, Start, End, ValFun) -> - PBC = rt_pb:pbc(Node), + PBC = rt:pbc(Node), [ begin Obj = ValFun(Key), @@ -234,4 +234,4 @@ node_monitor_loop(Node, Sup, TestProc) -> Other -> lager:warn("Node monitor for ~p got unknown message ~p", [Node, Other]), node_monitor_loop(Node, Sup, TestProc) - end. + end. \ No newline at end of file diff --git a/tests/overload.erl b/tests/overload.erl index 70d99cc91..c1d27cbc9 100644 --- a/tests/overload.erl +++ b/tests/overload.erl @@ -1,3 +1,4 @@ +%% ------------------------------------------------------------------- %% %% Copyright (c) 2013 Basho Technologies, Inc. %% diff --git a/tests/partition_repair.erl b/tests/partition_repair.erl index 3d21f4c84..d295c1a15 100644 --- a/tests/partition_repair.erl +++ b/tests/partition_repair.erl @@ -72,7 +72,7 @@ confirm() -> %% [{"./log/console.log",debug,10485760,"$D0",5}]}]}]} ], - Nodes = rt_cluster:build_cluster(NumNodes, Conf), + Nodes = rt:build_cluster(NumNodes, Conf), case NVal of undefined -> @@ -86,15 +86,15 @@ confirm() -> rt:enable_search_hook(hd(Nodes), Bucket), lager:info("Insert Scott's spam emails"), - Pbc = rt_pb:pbc(hd(Nodes)), - rt_pb:pbc_put_dir(Pbc, Bucket, SpamDir), + Pbc = rt:pbc(hd(Nodes)), + rt:pbc_put_dir(Pbc, Bucket, SpamDir), lager:info("Stash ITFs for each partition"), %% @todo Should riak_test guarantee that the scratch pad is clean instead? ?assertCmd("rm -rf " ++ base_stash_path()), %% need to load the module so riak can see the fold fun rt:load_modules_on_nodes([?MODULE], Nodes), - Ring = rt_ring:get_ring(hd(Nodes)), + Ring = rt:get_ring(hd(Nodes)), Owners = riak_core_ring:all_owners(Ring), [stash_data(riak_search, Owner) || Owner <- Owners], @@ -120,7 +120,7 @@ kill_repair_verify({Partition, Node}, DataSuffix, Service) -> %% kill the partition data Path = DataSuffix ++ "/" ++ integer_to_list(Partition), lager:info("Killing data for ~p on ~p at ~s", [Partition, Node, Path]), - rt_cluster:clean_data_dir([Node], Path), + rt:clean_data_dir([Node], Path), %% force restart of vnode since some data is kept in memory lager:info("Restarting ~p vnode for ~p on ~p", [Service, Partition, Node]), @@ -334,4 +334,4 @@ set_search_schema_nval(Bucket, NVal) -> ?assertCmd(GetSchema), ?assertCmd(ModifyNVal), ?assertCmd(SetSchema), - ?assertCmd(ClearCache). + ?assertCmd(ClearCache). \ No newline at end of file diff --git a/tests/pb_security.erl b/tests/pb_security.erl index ff3da2b5b..95e8502f6 100644 --- a/tests/pb_security.erl +++ b/tests/pb_security.erl @@ -72,7 +72,7 @@ confirm() -> _ -> true end, - Nodes = rt_cluster:build_cluster(4, Conf), + Nodes = rt:build_cluster(4, Conf), Node = hd(Nodes), %% enable security on the cluster ok = rpc:call(Node, riak_core_console, security_enable, [[]]), @@ -547,9 +547,9 @@ confirm() -> %%%%%%%%%%%% %% create a new type - rt_bucket_types:create_and_activate_bucket_type(Node, <<"mytype">>, [{n_val, 3}]), - rt_bucket_types:wait_until_bucket_type_status(<<"mytype">>, active, Nodes), - rt_bucket_types:wait_until_bucket_type_visible(Nodes, <<"mytype">>), + rt:create_and_activate_bucket_type(Node, <<"mytype">>, [{n_val, 3}]), + rt:wait_until_bucket_type_status(<<"mytype">>, active, Nodes), + rt:wait_until_bucket_type_visible(Nodes, <<"mytype">>), lager:info("Checking that get on a new bucket type is disallowed"), ?assertMatch({error, <<"Permission", _/binary>>}, riakc_pb_socket:get(PB, @@ -624,9 +624,9 @@ confirm() -> lager:info("Creating another bucket type"), %% create a new type - rt_bucket_types:create_and_activate_bucket_type(Node, <<"mytype2">>, [{allow_mult, true}]), - rt_bucket_types:wait_until_bucket_type_status(<<"mytype2">>, active, Nodes), - rt_bucket_types:wait_until_bucket_type_visible(Nodes, <<"mytype2">>), + rt:create_and_activate_bucket_type(Node, <<"mytype2">>, [{allow_mult, true}]), + rt:wait_until_bucket_type_status(<<"mytype2">>, active, Nodes), + rt:wait_until_bucket_type_visible(Nodes, <<"mytype2">>), lager:info("Checking that get on the new type is disallowed"), ?assertMatch({error, <<"Permission", _/binary>>}, riakc_pb_socket:get(PB, @@ -775,9 +775,9 @@ crdt_tests([Node|_]=Nodes, PB) -> {<<"sets">>, set, riakc_set:to_op(riakc_set:add_element(<<"foo">>, riakc_set:new()))}, {<<"maps">>, map, riakc_map:to_op(riakc_map:update({<<"bar">>, counter}, fun(In) -> riakc_counter:increment(In) end, riakc_map:new()))}], [ begin - rt_bucket_types:create_and_activate_bucket_type(Node, BType, [{allow_mult, true}, {datatype, DType}]), - rt_bucket_types:wait_until_bucket_type_status(BType, active, Nodes), - rt_bucket_types:wait_until_bucket_type_visible(Nodes, BType) + rt:create_and_activate_bucket_type(Node, BType, [{allow_mult, true}, {datatype, DType}]), + rt:wait_until_bucket_type_status(BType, active, Nodes), + rt:wait_until_bucket_type_visible(Nodes, BType) end || {BType, DType, _Op} <- Types ], lager:info("Checking that CRDT fetch is denied"), @@ -816,4 +816,4 @@ reduce_set_union(List, A) -> mapred_modfun_input(Pipe, _Args, _Timeout) -> riak_pipe:queue_work(Pipe, {{<<"hello">>, <<"world">>}, {struct, []}}), - riak_pipe:eoi(Pipe). + riak_pipe:eoi(Pipe). \ No newline at end of file diff --git a/tests/replication/repl_fs_stat_caching.erl b/tests/replication/repl_fs_stat_caching.erl index ce03cd7bd..117fafb28 100644 --- a/tests/replication/repl_fs_stat_caching.erl +++ b/tests/replication/repl_fs_stat_caching.erl @@ -25,7 +25,7 @@ confirm() -> % status. {ok, Suspended} = suspend_an_fs_source(SrcCluster), lager:info("Suspended: ~p", [Suspended]), - {ok, Status} = rt_cmd_line:riak_repl(SrcLead, "status"), + {ok, Status} = rt:riak_repl(SrcLead, "status"), FailLine = "RPC to '" ++ atom_to_list(SrcLead) ++ "' failed: timeout\n", ?assertNotEqual(FailLine, Status), @@ -34,11 +34,11 @@ confirm() -> pass. setup() -> - rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), + rt:set_conf(all, [{"buckets.default.allow_mult", "false"}]), NodeCount = rt_config:get(num_nodes, 6), lager:info("Deploy ~p nodes", [NodeCount]), - Nodes = rt_cluster:deploy_nodes(NodeCount, cluster_conf()), + Nodes = rt:deploy_nodes(NodeCount, cluster_conf(), [riak_kv, riak_repl]), SplitSize = NodeCount div 2, {SourceNodes, SinkNodes} = lists:split(SplitSize, Nodes), @@ -106,4 +106,4 @@ maybe_suspend_an_fs_source(Node, [{_Remote, Pid} | Tail]) -> maybe_suspend_an_fs_source(Node, Tail); true -> Pid - end. + end. \ No newline at end of file diff --git a/tests/replication/repl_rt_cascading_rtq.erl b/tests/replication/repl_rt_cascading_rtq.erl index 79b7fa353..94b485826 100644 --- a/tests/replication/repl_rt_cascading_rtq.erl +++ b/tests/replication/repl_rt_cascading_rtq.erl @@ -6,7 +6,7 @@ -define(TEST_BUCKET, <<"rt-cascading-rtq-systest-a">>). setup() -> - rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), + rt:set_conf(all, [{"buckets.default.allow_mult", "false"}]), {SourceLeader, SinkLeaderA, SinkLeaderB, _, _, _} = ClusterNodes = make_clusters(), @@ -153,9 +153,9 @@ cluster_conf(_CascadingWrites) -> ]. deploy_nodes(NumNodes, true) -> - rt_cluster:deploy_nodes(NumNodes, cluster_conf(always)); + rt:deploy_nodes(NumNodes, cluster_conf(always), [riak_kv, riak_repl]); deploy_nodes(NumNodes, false) -> - rt_cluster:deploy_nodes(NumNodes, cluster_conf(never)). + rt:deploy_nodes(NumNodes, cluster_conf(never), [riak_kv, riak_repl]). %% @doc Turn on Realtime replication on the cluster lead by LeaderA. %% The clusters must already have been named and connected. @@ -181,5 +181,5 @@ write_to_cluster(Node, Start, End) -> %% of errors. read_from_cluster(Node, Start, End, Errors) -> lager:info("Reading ~p keys from node ~p.", [End - Start, Node]), - Res2 = rt_systest:read(Node, Start, End, ?TEST_BUCKET, 1), - ?assertEqual(Errors, length(Res2)). + Res2 = rt:systest_read(Node, Start, End, ?TEST_BUCKET, 1), + ?assertEqual(Errors, length(Res2)). \ No newline at end of file diff --git a/tests/replication/repl_util.erl b/tests/replication/repl_util.erl index 3fec1246b..b792a0a54 100644 --- a/tests/replication/repl_util.erl +++ b/tests/replication/repl_util.erl @@ -55,9 +55,9 @@ make_cluster(Nodes) -> [First|Rest] = Nodes, - ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes)), + ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), [rt:wait_for_service(N, riak_kv) || N <- Nodes], - [rt_node:join(Node, First) || Node <- Rest], + [rt:join(Node, First) || Node <- Rest], ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes)). name_cluster(Node, Name) -> @@ -197,10 +197,10 @@ wait_until_fullsync_stopped(SourceLeader) -> wait_for_reads(Node, Start, End, Bucket, R) -> rt:wait_until(Node, fun(_) -> - Reads = rt_systest:read(Node, Start, End, Bucket, R, <<>>, true), + Reads = rt:systest_read(Node, Start, End, Bucket, R, <<>>, true), Reads == [] end), - Reads = rt_systest:read(Node, Start, End, Bucket, R, <<>>, true), + Reads = rt:systest_read(Node, Start, End, Bucket, R, <<>>, true), lager:info("Reads: ~p", [Reads]), length(Reads). @@ -456,20 +456,20 @@ stop_realtime(Node, Cluster) -> ?assertEqual(ok, Res). do_write(Node, Start, End, Bucket, W) -> - case rt_systest:write(Node, Start, End, Bucket, W) of + case rt:systest_write(Node, Start, End, Bucket, W) of [] -> []; Errors -> lager:warning("~p errors while writing: ~p", [length(Errors), Errors]), timer:sleep(1000), - lists:flatten([rt_systest:write(Node, S, S, Bucket, W) || + lists:flatten([rt:systest_write(Node, S, S, Bucket, W) || {S, _Error} <- Errors]) end. %% does the node meet the version requirement? node_has_version(Node, Version) -> - NodeVersion = rt:node_version(rt:node_id(Node)), + NodeVersion = rtdev:node_version(rtdev:node_id(Node)), case NodeVersion of current -> %% current always satisfies any version check @@ -573,7 +573,7 @@ read_from_cluster(Node, Start, End, Bucket, Errors) -> %% of errors. read_from_cluster(Node, Start, End, Bucket, Errors, Quorum) -> lager:info("Reading ~p keys from node ~p.", [End - Start, Node]), - Res2 = rt_systest:read(Node, Start, End, Bucket, Quorum, <<>>, true), + Res2 = rt:systest_read(Node, Start, End, Bucket, Quorum, <<>>, true), ?assertEqual(Errors, length(Res2)). %% @doc Assert we can perform one fullsync cycle, and that the number of @@ -626,12 +626,12 @@ validate_intercepted_fullsync(InterceptTarget, NumIndicies), %% Reboot node. - rt_node:stop_and_wait(InterceptTarget), - rt_node:start_and_wait(InterceptTarget), + rt:stop_and_wait(InterceptTarget), + rt:start_and_wait(InterceptTarget), %% Wait for riak_kv and riak_repl to initialize. rt:wait_for_service(InterceptTarget, riak_kv), rt:wait_for_service(InterceptTarget, riak_repl), %% Wait until AAE trees are compueted on the rebooted node. - rt_aae:wait_until_aae_trees_built([InterceptTarget]). + rt:wait_until_aae_trees_built([InterceptTarget]). \ No newline at end of file diff --git a/tests/replication/replication2.erl b/tests/replication/replication2.erl index e3e75759d..9f2811925 100644 --- a/tests/replication/replication2.erl +++ b/tests/replication/replication2.erl @@ -148,7 +148,7 @@ real_time_replication_test([AFirst|_] = ANodes, [BFirst|_] = BNodes, Connected) %% Check that the keys we wrote initially aren't replicated yet as %% fullsync_on_connect is disabled. lager:info("Check keys written before repl was connected are not present"), - Res2 = rt_systest:read(BFirst, 1, 100, TestBucket, 2), + Res2 = rt:systest_read(BFirst, 1, 100, TestBucket, 2), ?assertEqual(100, length(Res2)), log_to_nodes(ANodes++BNodes, "Test fullsync with leader ~p", [LeaderA]), @@ -235,7 +235,7 @@ master_failover_test([AFirst|_] = ANodes, [BFirst|_] = BNodes) -> log_to_nodes(ANodes ++ BNodes, "Testing master failover: stopping ~p", [LeaderA]), lager:info("Testing master failover: stopping ~p", [LeaderA]), - rt_node:stop(LeaderA), + rt:stop(LeaderA), rt:wait_until_unpingable(LeaderA), ASecond = hd(ANodes -- [LeaderA]), repl_util:wait_until_leader(ASecond), @@ -259,7 +259,7 @@ master_failover_test([AFirst|_] = ANodes, [BFirst|_] = BNodes) -> log_to_nodes(ANodes ++ BNodes, "Testing client failover: stopping ~p", [LeaderB]), lager:info("Testing client failover: stopping ~p", [LeaderB]), - rt_node:stop(LeaderB), + rt:stop(LeaderB), rt:wait_until_unpingable(LeaderB), BSecond = hd(BNodes -- [LeaderB]), repl_util:wait_until_leader(BSecond), @@ -521,7 +521,7 @@ offline_queueing_tests([AFirst|_] = ANodes, [BFirst|_] = BNodes) -> lager:info("Stopping node ~p", [Target]), - rt_node:stop(Target), + rt:stop(Target), rt:wait_until_unpingable(Target), lager:info("Starting realtime"), @@ -555,7 +555,7 @@ pb_write_during_shutdown([AFirst|_] = ANodes, [BFirst|_] = BNodes) -> ConnInfo = proplists:get_value(Target, rt:connection_info([Target])), {IP, Port} = proplists:get_value(pb, ConnInfo), lager:info("Connecting to pb socket ~p:~p on ~p", [IP, Port, Target]), - PBSock = rt_pb:pbc(Target), + PBSock = rt:pbc(Target), %% do the stop in the background while we're writing keys spawn(fun() -> @@ -584,7 +584,7 @@ pb_write_during_shutdown([AFirst|_] = ANodes, [BFirst|_] = BNodes) -> lager:info("pb_write_during_shutdown: Ensure node ~p is down before restart", [Target]), ?assertEqual(ok, rt:wait_until_unpingable(Target)), - rt_node:start(Target), + rt:start(Target), rt:wait_until_pingable(Target), rt:wait_for_service(Target, riak_repl), ReadErrors2 = rt:systest_read(Target, 1000, 11000, TestBucket, 2), @@ -627,7 +627,7 @@ http_write_during_shutdown([AFirst|_] = ANodes, [BFirst|_] = BNodes) -> ConnInfo = proplists:get_value(Target, rt:connection_info([Target])), {IP, Port} = proplists:get_value(http, ConnInfo), lager:info("Connecting to http socket ~p:~p on ~p", [IP, Port, Target]), - C = rt_http:httpc(Target), + C = rt:httpc(Target), %% do the stop in the background while we're writing keys spawn(fun() -> @@ -658,7 +658,7 @@ http_write_during_shutdown([AFirst|_] = ANodes, [BFirst|_] = BNodes) -> lager:info("HTTP: write_during_shutdown: Ensure node ~p is down before restart", [Target]), ?assertEqual(ok, rt:wait_until_unpingable(Target)), - rt_node:start(Target), + rt:start(Target), rt:wait_until_pingable(Target), rt:wait_for_service(Target, riak_repl), ReadErrors2 = http_read(C, 12000, 22000, TestBucket, 2), @@ -767,4 +767,4 @@ collect_results(Workers, Acc) -> collect_results(lists:keydelete(Pid, 1, Workers), Res ++ Acc); {'DOWN', _, _, Pid, _Reason} -> collect_results(lists:keydelete(Pid, 1, Workers), Acc) - end. + end. \ No newline at end of file diff --git a/tests/replication/replication2_console_tests.erl b/tests/replication/replication2_console_tests.erl index 5d5c55fae..258f88bb4 100644 --- a/tests/replication/replication2_console_tests.erl +++ b/tests/replication/replication2_console_tests.erl @@ -48,7 +48,7 @@ confirm() -> %% Deploy a node to test against lager:info("Deploy node to test riak-repl command line"), - [Node] = rt_cluster:deploy_nodes(1), + [Node] = rt:deploy_nodes(1, [], [riak_kv, riak_repl]), ?assertEqual(ok, rt:wait_until_nodes_ready([Node])), rt_intercept:add(Node, {riak_repl_console, @@ -118,6 +118,5 @@ confirm() -> check_cmd(Node, Cmd) -> lager:info("Testing riak-repl ~s on ~s", [Cmd, Node]), - {ok, Out} = rt_cmd_line:riak_repl(Node, [Cmd]), + {ok, Out} = rt:riak_repl(Node, [Cmd]), ?assertEqual("pass", Out). - diff --git a/tests/replication/replication2_pg.erl b/tests/replication/replication2_pg.erl index 1198a5099..fa9175320 100644 --- a/tests/replication/replication2_pg.erl +++ b/tests/replication/replication2_pg.erl @@ -64,8 +64,8 @@ setup_repl_clusters(Conf, SSL) -> ], - rt_config:set_advanced_conf(all, Conf), - Nodes = [ANodes, BNodes, CNodes] = rt_cluster:build_clusters([2, 2, 2]), + rt:set_advanced_conf(all, Conf), + Nodes = [ANodes, BNodes, CNodes] = rt:build_clusters([2, 2, 2]), rt:wait_for_cluster_service(ANodes, riak_repl), rt:wait_for_cluster_service(BNodes, riak_repl), @@ -84,11 +84,11 @@ setup_repl_clusters(Conf, SSL) -> case SSL of true -> lager:info("Enabling SSL for this test"), - [rt_config:update_app_config(N, merge_config(SSLConfig1, Conf)) || + [rt:update_app_config(N, merge_config(SSLConfig1, Conf)) || N <- ANodes], - [rt_config:update_app_config(N, merge_config(SSLConfig2, Conf)) || + [rt:update_app_config(N, merge_config(SSLConfig2, Conf)) || N <- BNodes], - [rt_config:update_app_config(N, merge_config(SSLConfig3, Conf)) || + [rt:update_app_config(N, merge_config(SSLConfig3, Conf)) || N <- CNodes]; _ -> lager:info("SSL not enabled for this test") @@ -172,20 +172,20 @@ test_basic_pg(Mode, SSL) -> EnabledFor -> lager:info("PG enabled for cluster ~p",[EnabledFor]) end, - PidA = rt_pb:pbc(LeaderA), + PidA = rt:pbc(LeaderA), {ok,CidA}=riak_repl_pb_api:get_clusterid(PidA), lager:info("Cluster ID for A = ~p", [CidA]), {Bucket, KeyA, ValueA} = make_test_object("a"), {Bucket, KeyB, ValueB} = make_test_object("b"), - rt_pb:pbc_write(PidA, Bucket, KeyA, ValueA), - rt_pb:pbc_write(PidA, Bucket, KeyB, ValueB), + rt:pbc_write(PidA, Bucket, KeyA, ValueA), + rt:pbc_write(PidA, Bucket, KeyB, ValueB), _FirstA = hd(ANodes), FirstB = hd(BNodes), FirstC = hd(CNodes), - PidB = rt_pb:pbc(FirstB), + PidB = rt:pbc(FirstB), lager:info("Connected to cluster B"), {ok, PGResult} = riak_repl_pb_api:get(PidB,Bucket,KeyA,CidA), ?assertEqual(ValueA, riakc_obj:get_value(PGResult)), @@ -237,7 +237,7 @@ test_basic_pg(Mode, SSL) -> EnabledFor3 -> lager:info("PG enabled for cluster ~p",[EnabledFor3]) end, - PidC = rt_pb:pbc(FirstC), + PidC = rt:pbc(FirstC), Options = [{n_val, 1}, {sloppy_quorum, false}], lager:info("Test proxy get from C using options: ~p", [Options]), @@ -299,16 +299,16 @@ test_12_pg(Mode, SSL) -> end, [rt:wait_until_ring_converged(Ns) || Ns <- [ANodes, BNodes, CNodes]], - PidA = rt_pb:pbc(LeaderA), - rt_pb:pbc_write(PidA, Bucket, KeyA, ValueA), - rt_pb:pbc_write(PidA, Bucket, KeyB, ValueB), + PidA = rt:pbc(LeaderA), + rt:pbc_write(PidA, Bucket, KeyA, ValueA), + rt:pbc_write(PidA, Bucket, KeyB, ValueB), {ok,CidA}=riak_repl_pb_api:get_clusterid(PidA), lager:info("Cluster ID for A = ~p", [CidA]), LeaderB = rpc:call(FirstB, riak_repl2_leader, leader_node, []), rt:log_to_nodes([LeaderB], "Trying to use PG while it's disabled"), - PidB = rt_pb:pbc(LeaderB), + PidB = rt:pbc(LeaderB), ?assertEqual({error, notfound}, riak_repl_pb_api:get(PidB, Bucket, KeyA, CidA)), @@ -332,7 +332,7 @@ test_12_pg(Mode, SSL) -> lager:info("Trying proxy_get"), LeaderB2 = rpc:call(FirstB, riak_repl2_leader, leader_node, []), - PidB2 = rt_pb:pbc(LeaderB2), + PidB2 = rt:pbc(LeaderB2), {ok, PGResult} = riak_repl_pb_api:get(PidB2, Bucket, KeyB, CidA), lager:info("PGResult: ~p", [PGResult]), ?assertEqual(ValueB, riakc_obj:get_value(PGResult)), @@ -388,7 +388,7 @@ test_pg_proxy(SSL) -> EnabledFor -> lager:info("PG enabled for cluster ~p",[EnabledFor]) end, - PidA = rt_pb:pbc(LeaderA), + PidA = rt:pbc(LeaderA), {ok,CidA}=riak_repl_pb_api:get_clusterid(PidA), lager:info("Cluster ID for A = ~p", [CidA]), @@ -398,16 +398,16 @@ test_pg_proxy(SSL) -> {Bucket, KeyC, ValueC} = make_test_object("c"), {Bucket, KeyD, ValueD} = make_test_object("d"), - rt_pb:pbc_write(PidA, Bucket, KeyA, ValueA), - rt_pb:pbc_write(PidA, Bucket, KeyB, ValueB), - rt_pb:pbc_write(PidA, Bucket, KeyC, ValueC), - rt_pb:pbc_write(PidA, Bucket, KeyD, ValueD), + rt:pbc_write(PidA, Bucket, KeyA, ValueA), + rt:pbc_write(PidA, Bucket, KeyB, ValueB), + rt:pbc_write(PidA, Bucket, KeyC, ValueC), + rt:pbc_write(PidA, Bucket, KeyD, ValueD), %% sanity check. You know, like the 10000 tests that autoconf runs %% before it actually does any work. FirstA = hd(ANodes), FirstB = hd(BNodes), _FirstC = hd(CNodes), - PidB = rt_pb:pbc(FirstB), + PidB = rt:pbc(FirstB), lager:info("Connected to cluster B"), {ok, PGResult} = riak_repl_pb_api:get(PidB,Bucket,KeyA,CidA), ?assertEqual(ValueA, riakc_obj:get_value(PGResult)), @@ -418,17 +418,17 @@ test_pg_proxy(SSL) -> lager:info("Stopping leader on requester cluster"), PGLeaderB = rpc:call(FirstB, riak_core_cluster_mgr, get_leader, []), rt:log_to_nodes(AllNodes, "Killing leader on requester cluster"), - rt_node:stop(PGLeaderB), + rt:stop(PGLeaderB), [RunningBNode | _ ] = BNodes -- [PGLeaderB], repl_util:wait_until_leader(RunningBNode), - PidB2 = rt_pb:pbc(RunningBNode), + PidB2 = rt:pbc(RunningBNode), lager:info("Now trying proxy_get"), ?assertEqual(ok, wait_until_pg(RunningBNode, PidB2, Bucket, KeyC, CidA)), lager:info("If you got here, proxy_get worked after the pg block requesting leader was killed"), lager:info("Stopping leader on provider cluster"), PGLeaderA = rpc:call(FirstA, riak_core_cluster_mgr, get_leader, []), - rt_node:stop(PGLeaderA), + rt:stop(PGLeaderA), [RunningANode | _ ] = ANodes -- [PGLeaderA], repl_util:wait_until_leader(RunningANode), ?assertEqual(ok, wait_until_pg(RunningBNode, PidB2, Bucket, KeyD, CidA)), @@ -487,15 +487,15 @@ test_cluster_mapping(SSL) -> [rt:wait_until_ring_converged(Ns) || Ns <- [ANodes, BNodes, CNodes]], - PidA = rt_pb:pbc(LeaderA), + PidA = rt:pbc(LeaderA), {ok,CidA}=riak_repl_pb_api:get_clusterid(PidA), lager:info("Cluster ID for A = ~p", [CidA]), - PidB = rt_pb:pbc(LeaderB), + PidB = rt:pbc(LeaderB), {ok,CidB}=riak_repl_pb_api:get_clusterid(PidB), lager:info("Cluster ID for B = ~p", [CidB]), - PidC = rt_pb:pbc(LeaderC), + PidC = rt:pbc(LeaderC), {ok,CidC}=riak_repl_pb_api:get_clusterid(PidC), lager:info("Cluster ID for C = ~p", [CidC]), @@ -505,10 +505,10 @@ test_cluster_mapping(SSL) -> {Bucket, KeyC, ValueC} = make_test_object("c"), {Bucket, KeyD, ValueD} = make_test_object("d"), - rt_pb:pbc_write(PidA, Bucket, KeyA, ValueA), - rt_pb:pbc_write(PidA, Bucket, KeyB, ValueB), - rt_pb:pbc_write(PidA, Bucket, KeyC, ValueC), - rt_pb:pbc_write(PidA, Bucket, KeyD, ValueD), + rt:pbc_write(PidA, Bucket, KeyA, ValueA), + rt:pbc_write(PidA, Bucket, KeyB, ValueB), + rt:pbc_write(PidA, Bucket, KeyC, ValueC), + rt:pbc_write(PidA, Bucket, KeyD, ValueD), {ok, PGResult} = riak_repl_pb_api:get(PidA,Bucket,KeyA,CidA), @@ -530,7 +530,7 @@ test_cluster_mapping(SSL) -> % shut down cluster A lager:info("Shutting down cluster A"), - [ rt_node:stop(Node) || Node <- ANodes ], + [ rt:stop(Node) || Node <- ANodes ], [ rt:wait_until_unpingable(Node) || Node <- ANodes ], rt:wait_until_ring_converged(BNodes), @@ -605,8 +605,8 @@ test_bidirectional_pg(SSL) -> EnabledForB -> lager:info("PG enabled for cluster ~p",[EnabledForB]) end, - PidA = rt_pb:pbc(LeaderA), - PidB = rt_pb:pbc(FirstB), + PidA = rt:pbc(LeaderA), + PidB = rt:pbc(FirstB), {ok,CidA}=riak_repl_pb_api:get_clusterid(PidA), {ok,CidB}=riak_repl_pb_api:get_clusterid(PidB), @@ -617,10 +617,10 @@ test_bidirectional_pg(SSL) -> {Bucket, KeyB, ValueB} = make_test_object("b"), %% write some data to cluster A - rt_pb:pbc_write(PidA, Bucket, KeyA, ValueA), + rt:pbc_write(PidA, Bucket, KeyA, ValueA), %% write some data to cluster B - rt_pb:pbc_write(PidB, Bucket, KeyB, ValueB), + rt:pbc_write(PidB, Bucket, KeyB, ValueB), lager:info("Trying first get"), wait_until_pg(LeaderB, PidB, Bucket, KeyA, CidA), @@ -667,22 +667,22 @@ test_multiple_sink_pg(SSL) -> EnabledForC -> lager:info("PG enabled for cluster ~p",[EnabledForC]) end, - PidA = rt_pb:pbc(LeaderA), + PidA = rt:pbc(LeaderA), {ok,CidA}=riak_repl_pb_api:get_clusterid(PidA), lager:info("Cluster ID for A = ~p", [CidA]), {Bucket, KeyA, ValueA} = make_test_object("a"), {Bucket, KeyB, ValueB} = make_test_object("b"), - rt_pb:pbc_write(PidA, Bucket, KeyA, ValueA), - rt_pb:pbc_write(PidA, Bucket, KeyB, ValueB), + rt:pbc_write(PidA, Bucket, KeyA, ValueA), + rt:pbc_write(PidA, Bucket, KeyB, ValueB), _FirstA = hd(ANodes), FirstB = hd(BNodes), FirstC = hd(CNodes), - PidB = rt_pb:pbc(FirstB), - PidC = rt_pb:pbc(FirstC), + PidB = rt:pbc(FirstB), + PidC = rt:pbc(FirstC), {ok, PGResultB} = riak_repl_pb_api:get(PidB,Bucket,KeyA,CidA), ?assertEqual(ValueA, riakc_obj:get_value(PGResultB)), @@ -720,15 +720,15 @@ test_mixed_pg(SSL) -> EnabledFor -> lager:info("PG enabled for cluster ~p",[EnabledFor]) end, - PidA = rt_pb:pbc(LeaderA), + PidA = rt:pbc(LeaderA), {ok,CidA}=riak_repl_pb_api:get_clusterid(PidA), lager:info("Cluster ID for A = ~p", [CidA]), {Bucket, KeyB, ValueB} = make_test_object("b"), {Bucket, KeyC, ValueC} = make_test_object("c"), - rt_pb:pbc_write(PidA, Bucket, KeyB, ValueB), - rt_pb:pbc_write(PidA, Bucket, KeyC, ValueC), + rt:pbc_write(PidA, Bucket, KeyB, ValueB), + rt:pbc_write(PidA, Bucket, KeyC, ValueC), _FirstA = hd(ANodes), FirstB = hd(BNodes), @@ -756,8 +756,8 @@ test_mixed_pg(SSL) -> lager:info("Trying proxy_get"), LeaderC = rpc:call(FirstC, riak_repl2_leader, leader_node, []), - PidB = rt_pb:pbc(FirstB), - PidC = rt_pb:pbc(LeaderC), + PidB = rt:pbc(FirstB), + PidC = rt:pbc(LeaderC), {ok, PGResultB} = riak_repl_pb_api:get(PidB, Bucket, KeyB, CidA), lager:info("PGResultB: ~p", [PGResultB]), @@ -914,15 +914,15 @@ verify_topology_change(SourceNodes, SinkNodes) -> %% Get connections [SourceNode1, _SourceNode2] = SourceNodes, - SourceNode1Pid = rt_pb:pbc(SourceNode1), + SourceNode1Pid = rt:pbc(SourceNode1), [SinkNode1, SinkNode2] = SinkNodes, - SinkNode1Pid = rt_pb:pbc(SinkNode1), + SinkNode1Pid = rt:pbc(SinkNode1), {ok, SourceCid} = riak_repl_pb_api:get_clusterid(SourceNode1Pid), %% Write new object to source. lager:info("Writing key 'before' to the source."), {Bucket, KeyBefore, ValueBefore} = make_test_object("before"), - rt_pb:pbc_write(SourceNode1Pid, Bucket, KeyBefore, ValueBefore), + rt:pbc_write(SourceNode1Pid, Bucket, KeyBefore, ValueBefore), %% Verify proxy_get through the sink works. lager:info("Verifying key 'before' can be read through the sink."), @@ -937,12 +937,12 @@ verify_topology_change(SourceNodes, SinkNodes) -> %% Sad this takes 2.5 minutes lager:info("Removing current leader from the cluster: ~p.", [SinkLeader]), - rt_node:leave(SinkLeader), + rt:leave(SinkLeader), ?assertEqual(ok, rt:wait_until_unpingable(SinkLeader)), %% Wait for everything to restart, and rings to converge. lager:info("Starting leader node back up and waiting for repl."), - rt_node:start(SinkLeader), + rt:start(SinkLeader), rt:wait_for_service(SinkLeader, riak_repl), rt:wait_until_ring_converged(SinkNodes), @@ -971,9 +971,9 @@ verify_topology_change(SourceNodes, SinkNodes) -> lager:info("Rejoining former leader."), case SinkLeader of SinkNode1 -> - rt_node:join(SinkNode1, SinkNode2); + rt:join(SinkNode1, SinkNode2); SinkNode2 -> - rt_node:join(SinkNode2, SinkNode1) + rt:join(SinkNode2, SinkNode1) end, rt:wait_until_ring_converged(SinkNodes), @@ -993,7 +993,7 @@ verify_topology_change(SourceNodes, SinkNodes) -> lager:info("Writing key 'after' to the source."), {ok, SourceCid} = riak_repl_pb_api:get_clusterid(SourceNode1Pid), {Bucket, KeyPost, ValuePost} = make_test_object("after"), - rt_pb:pbc_write(SourceNode1Pid, Bucket, KeyPost, ValuePost), + rt:pbc_write(SourceNode1Pid, Bucket, KeyPost, ValuePost), %% Verify we can retrieve from source. lager:info("Verifying key 'after' can be read through the source."), @@ -1006,4 +1006,4 @@ verify_topology_change(SourceNodes, SinkNodes) -> wait_until_pg(SinkNode1, SinkNode1Pid, Bucket, KeyPost, SourceCid), %% We're good! - pass. + pass. \ No newline at end of file diff --git a/tests/replication/replication_object_reformat.erl b/tests/replication/replication_object_reformat.erl index 588a0700b..6ff3ec86b 100644 --- a/tests/replication/replication_object_reformat.erl +++ b/tests/replication/replication_object_reformat.erl @@ -115,7 +115,7 @@ verify_replication(AVersion, BVersion, Start, End, Realtime) -> %% Wait until the sink cluster is in a steady state before %% starting fullsync - rt_node:wait_until_nodes_ready(BNodes), + rt:wait_until_nodes_ready(BNodes), rt:wait_until_no_pending_changes(BNodes), rt:wait_until_registered(BFirst, riak_repl2_fs_node_reserver), @@ -159,14 +159,14 @@ verify_replication(AVersion, BVersion, Start, End, Realtime) -> ok end, - rt_cluster:clean_cluster(lists:flatten(Nodes)). + rt:clean_cluster(lists:flatten(Nodes)). %% @doc Configure two clusters and set up replication between them, %% return the node list of each cluster. configure_clusters(AVersion, BVersion, Realtime) -> - rt_config:set_advanced_conf(all, ?CONF(infinity)), + rt:set_advanced_conf(all, ?CONF(infinity)), - Nodes = [ANodes, BNodes] = rt_cluster:build_clusters([3, 3]), + Nodes = [ANodes, BNodes] = rt:build_clusters([3, 3]), rt:wait_for_cluster_service(ANodes, riak_repl), rt:wait_for_cluster_service(BNodes, riak_repl), @@ -176,13 +176,13 @@ configure_clusters(AVersion, BVersion, Realtime) -> lager:info("Updating app config to force ~p on source cluster.", [AVersion]), - [rt_config:update_app_config(N, [{riak_kv, + [rt:update_app_config(N, [{riak_kv, [{object_format, AVersion}]}]) || N <- ANodes], lager:info("Updating app config to force ~p on sink cluster.", [BVersion]), - [rt_config:update_app_config(N, [{riak_kv, + [rt:update_app_config(N, [{riak_kv, [{object_format, BVersion}]}]) || N <- BNodes], @@ -236,4 +236,4 @@ configure_clusters(AVersion, BVersion, Realtime) -> lager:info("Ensuring connection from cluster A to B"), repl_util:connect_cluster_by_name(LeaderA, BPort, "B"), - Nodes. + Nodes. \ No newline at end of file diff --git a/tests/riak_control.erl b/tests/riak_control.erl index 661dc093d..cfc222b52 100644 --- a/tests/riak_control.erl +++ b/tests/riak_control.erl @@ -46,7 +46,7 @@ verify_upgrade(Vsn) -> lager:info("Verify upgrade from ~p to current.", [Vsn]), lager:info("Building cluster."), - [Nodes] = rt_cluster:build_clusters([{3, Vsn, ?RC_ENABLE_CFG}]), + [Nodes] = rt:build_clusters([{3, Vsn, ?RC_ENABLE_CFG}]), lager:info("Verifying all nodes are alive."), verify_alive(Nodes), @@ -258,4 +258,4 @@ validate_capability(VersionedNodes) -> %% Validate we are in the correct state, not the incompatible state, %% which ensure the capability has negotiated correctly. - validate_nodes(Node, Nodes, VersionedNodes, <<"valid">>). + validate_nodes(Node, Nodes, VersionedNodes, <<"valid">>). \ No newline at end of file diff --git a/tests/riak_control_authentication.erl b/tests/riak_control_authentication.erl index 88b495aca..761e8332d 100644 --- a/tests/riak_control_authentication.erl +++ b/tests/riak_control_authentication.erl @@ -209,7 +209,7 @@ verify_authentication(current, ?RC_AUTH_USERLIST_CONFIG_NO_FORCE_SSL) -> %% @doc Build a one node cluster. build_singleton_cluster(Vsn, Config) -> - [Nodes] = rt_cluster:build_clusters([{1, Vsn, Config}]), + [Nodes] = rt:build_clusters([{1, Vsn, Config}]), %% Start and stop, wait for riak_kv. %% @@ -217,8 +217,8 @@ build_singleton_cluster(Vsn, Config) -> %% the supervisor starts, we need to restart to ensure settings %% take effect. Node = lists:nth(1, Nodes), - rt_node:stop_and_wait(Node), - rt_node:start_and_wait(Node), + rt:stop_and_wait(Node), + rt:start_and_wait(Node), rt:wait_for_service(Node, riak_kv), %% Wait for control to start. @@ -226,4 +226,4 @@ build_singleton_cluster(Vsn, Config) -> rt:wait_for_control(VersionedNodes), lager:info("Build ~p, nodes: ~p.", [Vsn, Nodes]), - Nodes. + Nodes. \ No newline at end of file diff --git a/tests/verify_2i_aae.erl b/tests/verify_2i_aae.erl index 54866de2b..ac682fc8d 100644 --- a/tests/verify_2i_aae.erl +++ b/tests/verify_2i_aae.erl @@ -18,14 +18,11 @@ %% %% ------------------------------------------------------------------- -module(verify_2i_aae). - -%% -behaviour(riak_test). - +-behaviour(riak_test). +-export([confirm/0]). -include_lib("eunit/include/eunit.hrl"). -include_lib("riakc/include/riakc.hrl"). --test_type(['2i']). - %% Make it multi-backend compatible. -define(BUCKETS, [<<"eleveldb1">>, <<"memory1">>]). -define(NUM_ITEMS, 1000). @@ -33,26 +30,13 @@ -define(SCAN_BATCH_SIZE, 100). -define(N_VAL, 3). --export([properties/0, - confirm/1]). - -properties() -> - Config = [{riak_kv, - [{anti_entropy, {off, []}}, - {anti_entropy_build_limit, {100, 500}}, - {anti_entropy_concurrency, 100}, - {anti_entropy_tick, 200}]}] ++ rt_properties:default_config(), - rt_properties:new([{node_count, 1}, - {make_cluster, false}, - {config, Config}]). - --spec confirm(rt_properties:properties()) -> pass | fail. -confirm(Properties) -> - NodeIds = rt_properties:get(node_ids, Properties), - NodeMap = rt_properties:get(node_map, Properties), - Nodes = [rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds], - Node1 = hd(Nodes), - +confirm() -> + [Node1] = rt:build_cluster(1, + [{riak_kv, + [{anti_entropy, {off, []}}, + {anti_entropy_build_limit, {100, 500}}, + {anti_entropy_concurrency, 100}, + {anti_entropy_tick, 200}]}]), rt_intercept:load_code(Node1), rt_intercept:add(Node1, {riak_object, @@ -60,7 +44,7 @@ confirm(Properties) -> {{diff_index_specs, 2}, skippable_diff_index_specs}]}), lager:info("Installed intercepts to corrupt index specs on node ~p", [Node1]), %%rpc:call(Node1, lager, set_loglevel, [lager_console_backend, debug]), - PBC = rt_pb:pbc(Node1), + PBC = rt:pbc(Node1), NumItems = ?NUM_ITEMS, NumDel = ?NUM_DELETES, pass = check_lost_objects(Node1, PBC, NumItems, NumDel), @@ -84,7 +68,7 @@ check_lost_objects(Node1, PBC, NumItems, NumDel) -> ok = rpc:call(Node1, application, set_env, [riak_kv, anti_entropy, {on, [debug]}]), ok = rpc:call(Node1, riak_kv_entropy_manager, enable, []), - rt_aae:wait_until_aae_trees_built([Node1]), + rt:wait_until_aae_trees_built([Node1]), lager:info("AAE trees built, now put the rest of the data"), [put_obj(PBC, Bucket, N, N+1, Index) @@ -102,8 +86,8 @@ check_lost_objects(Node1, PBC, NumItems, NumDel) -> DelRange = lists:seq(NumItems-NumDel+1, NumItems), lager:info("Deleting ~b objects without updating indexes", [NumDel]), [del_obj(PBC, Bucket, N) || N <- DelRange, Bucket <- ?BUCKETS], - DelKeys = [to_key(N) || N <- DelRange], - [rt:wait_until(fun() -> rt_pb:pbc_really_deleted(PBC, Bucket, DelKeys) end) + DelKeys = [to_key(N) || N <- DelRange], + [rt:wait_until(fun() -> rt:pbc_really_deleted(PBC, Bucket, DelKeys) end) || Bucket <- ?BUCKETS], %% Verify they are damaged lager:info("Verify change did not take, needs repair"), @@ -141,7 +125,7 @@ do_tree_rebuild(Node) -> ?assertEqual(ok, rpc:call(Node, application, set_env, [riak_kv, anti_entropy_build_limit, {100, 1000}])), - rt_aae:wait_until_aae_trees_built([Node]), + rt:wait_until_aae_trees_built([Node]), ok. %% Write objects without a 2i index. Test that running 2i repair will generate @@ -167,7 +151,7 @@ check_kill_repair(Node1) -> lager:info("Test that killing 2i repair works as desired"), spawn(fun() -> timer:sleep(1500), - rt_cmd_line:admin(Node1, ["repair-2i", "kill"]) + rt:admin(Node1, ["repair-2i", "kill"]) end), ExitStatus = run_2i_repair(Node1), case ExitStatus of @@ -184,11 +168,11 @@ check_kill_repair(Node1) -> run_2i_repair(Node1) -> lager:info("Run 2i AAE repair"), - ?assertMatch({ok, _}, rt_cmd_line:admin(Node1, ["repair-2i"])), + ?assertMatch({ok, _}, rt:admin(Node1, ["repair-2i"])), RepairPid = rpc:call(Node1, erlang, whereis, [riak_kv_2i_aae]), lager:info("Wait for repair process to finish"), Mon = monitor(process, RepairPid), - MaxWaitTime = 120000, + MaxWaitTime = rt_config:get(rt_max_wait_time), receive {'DOWN', Mon, _, _, Status} -> lager:info("Status: ~p", [Status]), diff --git a/tests/verify_aae.erl b/tests/verify_aae.erl index a40a42b8a..56a6026bd 100644 --- a/tests/verify_aae.erl +++ b/tests/verify_aae.erl @@ -64,7 +64,7 @@ -define(N_VAL, 3). confirm() -> - Nodes = rt_cluster:build_cluster(?NUM_NODES, ?CFG), + Nodes = rt:build_cluster(?NUM_NODES, ?CFG), verify_aae(Nodes), pass. @@ -144,7 +144,7 @@ write_data(Node, KVs) -> write_data(Node, KVs, []). write_data(Node, KVs, Opts) -> - PB = rt_pb:pbc(Node), + PB = rt:pbc(Node), [begin O = case riakc_pb_socket:get(PB, ?BUCKET, K) of @@ -161,7 +161,7 @@ write_data(Node, KVs, Opts) -> % @doc Verifies that the data is eventually restored to the expected set. verify_data(Node, KeyValues) -> lager:info("Verify all replicas are eventually correct"), - PB = rt_pb:pbc(Node), + PB = rt:pbc(Node), CheckFun = fun() -> Matches = [verify_replicas(Node, ?BUCKET, K, V, ?N_VAL) @@ -242,12 +242,12 @@ test_less_than_n_mods(Node, KeyValues) -> wipe_out_partition(Node, Partition) -> lager:info("Wiping out partition ~p in node ~p", [Partition, Node]), - rt_cluster:clean_data_dir(Node, dir_for_partition(Partition)), + rt:clean_data_dir(Node, dir_for_partition(Partition)), ok. wipe_out_aae_data(Node, Partition) -> lager:info("Wiping out AAE data for partition ~p in node ~p", [Partition, Node]), - rt_cluster:clean_data_dir(Node, "anti_entropy/"++integer_to_list(Partition)), + rt:clean_data_dir(Node, "anti_entropy/"++integer_to_list(Partition)), ok. base_dir_for_backend(undefined) -> @@ -301,4 +301,4 @@ max_aae_repairs(Node) when is_atom(Node) -> Info = rpc:call(Node, riak_kv_entropy_info, compute_exchange_info, []), LastCounts = [Last || {_, _, _, {Last, _, _, _}} <- Info], MaxCount = lists:max(LastCounts), - MaxCount. + MaxCount. \ No newline at end of file diff --git a/tests/verify_bitcask_tombstone2_upgrade.erl b/tests/verify_bitcask_tombstone2_upgrade.erl index fe153eb7f..fe0eba167 100644 --- a/tests/verify_bitcask_tombstone2_upgrade.erl +++ b/tests/verify_bitcask_tombstone2_upgrade.erl @@ -19,7 +19,7 @@ confirm() -> % Configure for fast merge checks Config = [{riak_kv, [{bitcask_merge_check_interval, 2000}]}, {bitcask, [{max_file_size, 100}]}], - Nodes = rt_cluster:build_cluster([{OldVsn, Config}]), + Nodes = rt:build_cluster([{OldVsn, Config}]), verify_bitcask_tombstone2_upgrade(Nodes), pass. @@ -36,14 +36,14 @@ verify_bitcask_tombstone2_upgrade(Nodes) -> lager:info("And that is that"). write_some_data([Node1 | _]) -> - rt_pb:pbc_systest_write(Node1, 10000). + rt:pbc_systest_write(Node1, 10000). list_bitcask_files(Nodes) -> [{Node, list_node_bitcask_files(Node)} || Node <- Nodes]. list_node_bitcask_files(Node) -> % Gather partitions owned, list *.bitcask.data on each. - Partitions = rt_ring:partitions_for_node(Node), + Partitions = rt:partitions_for_node(Node), {ok, DataDir} = rt:rpc_get_env(Node, [{bitcask, data_root}]), [begin IdxStr = integer_to_list(Idx), @@ -88,4 +88,4 @@ file_exists(Node, Path) -> end. all(Val, L) -> - lists:all(fun(E) -> E == Val end, L). + lists:all(fun(E) -> E == Val end, L). \ No newline at end of file diff --git a/tests/verify_build_cluster.erl b/tests/verify_build_cluster.erl index d11bcff74..eb6d0e6ec 100644 --- a/tests/verify_build_cluster.erl +++ b/tests/verify_build_cluster.erl @@ -18,72 +18,74 @@ %% %% ------------------------------------------------------------------- -module(verify_build_cluster). --export([properties/0, - confirm/1]). --include("rt.hrl"). +-behavior(riak_test). +-export([confirm/0]). -include_lib("eunit/include/eunit.hrl"). -properties() -> - UpdConfig = rt_cluster:augment_config(riak_core, - {default_bucket_props, [{allow_mult, false}]}, - rt_properties:default_config()), - rt_properties:new([{config, UpdConfig}, - {node_count, 4}, - {rolling_upgrade, true}, - {make_cluster, false}, - {start_version, previous}]). +-import(rt, [wait_until_nodes_ready/1, + wait_until_no_pending_changes/1]). --spec confirm(rt_properties:properties()) -> pass | fail. -confirm(Properties) -> - [Node1, Node2, Node3, Node4] = Nodes = rt_properties:get(nodes, Properties), +confirm() -> + %% test requires allow_mult=false b/c of rt:systest_read + rt:set_conf(all, [{"buckets.default.allow_mult", "false"}]), + %% Deploy a set of new nodes + lager:info("Deploying 4 nodes"), + %% handoff_concurrency needs to be raised to make the leave operation faster. + %% most clusters go up to 10, but this one is one louder, isn't it? + [Node1, Node2, Node3, Node4] = Nodes = rt:deploy_nodes(4, [{riak_core, [{handoff_concurrency, 11}]}]), + + %% Ensure each node owns 100% of it's own ring + lager:info("Ensure each nodes 100% of it's own ring"), + + [rt:wait_until_owners_according_to(Node, [Node]) || Node <- Nodes], lager:info("Loading some data up in this cluster."), - ?assertEqual([], rt_systest:write(Node1, 0, 1000, <<"verify_build_cluster">>, 2)), + ?assertEqual([], rt:systest_write(Node1, 0, 1000, <<"verify_build_cluster">>, 2)), lager:info("joining Node 2 to the cluster... It takes two to make a thing go right"), - rt_node:join(Node2, Node1), + rt:join(Node2, Node1), wait_and_validate([Node1, Node2]), lager:info("joining Node 3 to the cluster"), - rt_node:join(Node3, Node1), + rt:join(Node3, Node1), wait_and_validate([Node1, Node2, Node3]), lager:info("joining Node 4 to the cluster"), - rt_node:join(Node4, Node1), + rt:join(Node4, Node1), wait_and_validate(Nodes), lager:info("taking Node 1 down"), - rt_node:stop(Node1), + rt:stop(Node1), ?assertEqual(ok, rt:wait_until_unpingable(Node1)), wait_and_validate(Nodes, [Node2, Node3, Node4]), lager:info("taking Node 2 down"), - rt_node:stop(Node2), + rt:stop(Node2), ?assertEqual(ok, rt:wait_until_unpingable(Node2)), wait_and_validate(Nodes, [Node3, Node4]), lager:info("bringing Node 1 up"), - rt_node:start(Node1), + rt:start(Node1), ok = rt:wait_until_pingable(Node1), wait_and_validate(Nodes, [Node1, Node3, Node4]), lager:info("bringing Node 2 up"), - rt_node:start(Node2), + rt:start(Node2), ok = rt:wait_until_pingable(Node2), wait_and_validate(Nodes), % leave 1, 2, and 3 lager:info("leaving Node 1"), - rt_node:leave(Node1), + rt:leave(Node1), ?assertEqual(ok, rt:wait_until_unpingable(Node1)), wait_and_validate([Node2, Node3, Node4]), lager:info("leaving Node 2"), - rt_node:leave(Node2), + rt:leave(Node2), ?assertEqual(ok, rt:wait_until_unpingable(Node2)), wait_and_validate([Node3, Node4]), lager:info("leaving Node 3"), - rt_node:leave(Node3), + rt:leave(Node3), ?assertEqual(ok, rt:wait_until_unpingable(Node3)), % verify 4 @@ -94,12 +96,12 @@ confirm(Properties) -> wait_and_validate(Nodes) -> wait_and_validate(Nodes, Nodes). wait_and_validate(RingNodes, UpNodes) -> lager:info("Wait until all nodes are ready and there are no pending changes"), - ?assertEqual(ok, rt_node:wait_until_nodes_ready(UpNodes)), + ?assertEqual(ok, rt:wait_until_nodes_ready(UpNodes)), ?assertEqual(ok, rt:wait_until_all_members(UpNodes)), ?assertEqual(ok, rt:wait_until_no_pending_changes(UpNodes)), lager:info("Ensure each node owns a portion of the ring"), - [rt_node:wait_until_owners_according_to(Node, RingNodes) || Node <- UpNodes], + [rt:wait_until_owners_according_to(Node, RingNodes) || Node <- UpNodes], [rt:wait_for_service(Node, riak_kv) || Node <- UpNodes], lager:info("Verify that you got much data... (this is how we do it)"), - ?assertEqual([], rt_systest:read(hd(UpNodes), 0, 1000, <<"verify_build_cluster">>, 2)), - done. + ?assertEqual([], rt:systest_read(hd(UpNodes), 0, 1000, <<"verify_build_cluster">>, 2)), + done. \ No newline at end of file diff --git a/tests/verify_capabilities.erl b/tests/verify_capabilities.erl index 28d345fb9..8fde91133 100644 --- a/tests/verify_capabilities.erl +++ b/tests/verify_capabilities.erl @@ -31,7 +31,7 @@ confirm() -> _ -> current end, - Nodes = rt_cluster:deploy_nodes([current, previous, Legacy]), + Nodes = rt:deploy_nodes([current, previous, Legacy]), [CNode, PNode, LNode] = Nodes, lager:info("Verifying known capabilities on a Current 1-node cluster"), @@ -69,7 +69,7 @@ confirm() -> ?assertEqual(ok, rt:wait_until_capability(CNode, {riak_core, staged_joins}, true)), lager:info("Building current + ~s cluster", [Legacy]), - rt_node:join(LNode, CNode), + rt:join(LNode, CNode), ?assertEqual(ok, rt:wait_until_all_members([CNode], [CNode, LNode])), ?assertEqual(ok, rt:wait_until_legacy_ringready(CNode)), @@ -102,7 +102,7 @@ confirm() -> restart_capability_server(CNode), lager:info("Adding previous node to cluster"), - rt_node:join(PNode, LNode), + rt:join(PNode, LNode), ?assertEqual(ok, rt:wait_until_all_members([CNode], [CNode, LNode, PNode])), ?assertEqual(ok, rt:wait_until_legacy_ringready(CNode)), @@ -139,7 +139,7 @@ confirm() -> _ -> lager:info("Legacy Riak not available, skipping legacy tests"), lager:info("Adding previous node to cluster"), - rt_node:join(PNode, LNode), + rt:join(PNode, LNode), ?assertEqual(ok, rt:wait_until_all_members([CNode], [CNode, LNode, PNode])), ?assertEqual(ok, rt:wait_until_legacy_ringready(CNode)) end, @@ -213,24 +213,24 @@ confirm() -> end, lager:info("Override: (use: legacy), (prefer: proxy)"), - [rt_config:update_app_config(Node, Override(legacy, proxy)) || Node <- Nodes], + [rt:update_app_config(Node, Override(legacy, proxy)) || Node <- Nodes], lager:info("Verify vnode_routing == legacy"), assert_capability(CNode, {riak_core, vnode_routing}, legacy), lager:info("Override: (use: proxy), (prefer: legacy)"), - [rt_config:update_app_config(Node, Override(proxy, legacy)) || Node <- Nodes], + [rt:update_app_config(Node, Override(proxy, legacy)) || Node <- Nodes], lager:info("Verify vnode_routing == proxy"), assert_capability(CNode, {riak_core, vnode_routing}, proxy), lager:info("Override: (prefer: legacy)"), - [rt_config:update_app_config(Node, Override(undefined, legacy)) || Node <- Nodes], + [rt:update_app_config(Node, Override(undefined, legacy)) || Node <- Nodes], lager:info("Verify vnode_routing == legacy"), assert_capability(CNode, {riak_core, vnode_routing}, legacy), - [rt_node:stop(Node) || Node <- Nodes], + [rt:stop(Node) || Node <- Nodes], pass. assert_capability(CNode, Capability, Value) -> @@ -264,4 +264,3 @@ restart_capability_server(Node) -> end end, rt:wait_until(Node, HasNewPid). - diff --git a/tests/verify_dt_converge.erl b/tests/verify_dt_converge.erl index fc86c5efd..f98422e25 100644 --- a/tests/verify_dt_converge.erl +++ b/tests/verify_dt_converge.erl @@ -25,8 +25,7 @@ -module(verify_dt_converge). -behavior(riak_test). -compile([export_all]). --export([confirm/0, - create_bucket_types/2]). +-export([confirm/0]). -include_lib("eunit/include/eunit.hrl"). @@ -50,7 +49,7 @@ confirm() -> {riak_core, [ {ring_creation_size, 16}, {vnode_management_timer, 1000} ]}], - [N1, N2, N3, N4]=Nodes = rt_cluster:build_cluster(4, Config), + [N1, N2, N3, N4]=Nodes = rt:build_cluster(4, Config), create_bucket_types(Nodes, ?TYPES), @@ -73,7 +72,7 @@ confirm() -> lager:info("Partition cluster in two."), - PartInfo = rt_node:partition([N1, N2], [N3, N4]), + PartInfo = rt:partition([N1, N2], [N3, N4]), lager:info("Modify data on side 1"), %% Modify one side @@ -109,7 +108,7 @@ confirm() -> %% heal lager:info("Heal and check merged values"), - ok = rt_node:heal(PartInfo), + ok = rt:heal(PartInfo), ok = rt:wait_for_cluster_service(Nodes, riak_kv), %% verify all nodes agree @@ -125,13 +124,13 @@ confirm() -> create_pb_clients(Nodes) -> [begin - C = rt_pb:pbc(N), + C = rt:pbc(N), riakc_pb_socket:set_options(C, [queue_if_disconnected]), C end || N <- Nodes]. create_http_clients(Nodes) -> - [ rt_http:httpc(N) || N <- Nodes ]. + [ rt:httpc(N) || N <- Nodes ]. create_bucket_types([N1|_]=Nodes, Types) -> lager:info("Creating bucket types with datatypes: ~p", [Types]), @@ -338,4 +337,4 @@ check_value(Client, CMod, Bucket, Key, DTMod, Expected, Options) -> Error]), false end - end). + end). \ No newline at end of file diff --git a/tests/verify_handoff.erl b/tests/verify_handoff.erl index eb3e9af06..17db94137 100644 --- a/tests/verify_handoff.erl +++ b/tests/verify_handoff.erl @@ -58,11 +58,11 @@ run_test(TestMode, NTestItems, NTestNodes, Encoding) -> || N <- Nodes], lager:info("Populating root node."), - rt_systest:write(RootNode, NTestItems), + rt:systest_write(RootNode, NTestItems), %% write one object with a bucket type - rt_bucket_types:create_and_activate_bucket_type(RootNode, <<"type">>, []), + rt:create_and_activate_bucket_type(RootNode, <<"type">>, []), %% allow cluster metadata some time to propogate - rt_systest:write(RootNode, 1, 2, {<<"type">>, <<"bucket">>}, 2), + rt:systest_write(RootNode, 1, 2, {<<"type">>, <<"bucket">>}, 2), %% Test handoff on each node: lager:info("Testing handoff for cluster."), @@ -70,11 +70,11 @@ run_test(TestMode, NTestItems, NTestNodes, Encoding) -> %% Prepare for the next call to our test (we aren't polite about it, it's faster that way): lager:info("Bringing down test nodes."), - lists:foreach(fun(N) -> rt_node:brutal_kill(N) end, TestNodes), + lists:foreach(fun(N) -> rt:brutal_kill(N) end, TestNodes), %% The "root" node can't leave() since it's the only node left: lager:info("Stopping root node."), - rt_node:brutal_kill(RootNode). + rt:brutal_kill(RootNode). set_handoff_encoding(default, _) -> lager:info("Using default encoding type."), @@ -111,16 +111,16 @@ test_handoff(RootNode, NewNode, NTestItems) -> rt:wait_for_service(NewNode, riak_kv), lager:info("Joining new node with cluster."), - rt_node:join(NewNode, RootNode), - ?assertEqual(ok, rt_node:wait_until_nodes_ready([RootNode, NewNode])), + rt:join(NewNode, RootNode), + ?assertEqual(ok, rt:wait_until_nodes_ready([RootNode, NewNode])), rt:wait_until_no_pending_changes([RootNode, NewNode]), %% See if we get the same data back from the joined node that we added to the root node. %% Note: systest_read() returns /non-matching/ items, so getting nothing back is good: lager:info("Validating data after handoff:"), - Results = rt_systest:read(NewNode, NTestItems), - ?assertEqual(0, length(Results)), - Results2 = rt_systest:read(RootNode, 1, 2, {<<"type">>, <<"bucket">>}, 2), + Results = rt:systest_read(NewNode, NTestItems), + ?assertEqual(0, length(Results)), + Results2 = rt:systest_read(RootNode, 1, 2, {<<"type">>, <<"bucket">>}, 2), ?assertEqual(0, length(Results2)), lager:info("Data looks ok."). @@ -129,8 +129,11 @@ assert_using(Node, {CapabilityCategory, CapabilityName}, ExpectedCapabilityName) ExpectedCapabilityName =:= rt:capability(Node, {CapabilityCategory, CapabilityName}). %% For some testing purposes, making these limits smaller is helpful: -deploy_test_nodes(false, N) -> - rt_cluster:deploy_nodes(N); +deploy_test_nodes(false, N) -> + Config = [{riak_core, [{ring_creation_size, 8}, + {handoff_acksync_threshold, 20}, + {handoff_receive_timeout, 2000}]}], + rt:deploy_nodes(N, Config); deploy_test_nodes(true, N) -> lager:info("WARNING: Using turbo settings for testing."), Config = [{riak_core, [{forced_ownership_handoff, 8}, @@ -140,4 +143,4 @@ deploy_test_nodes(true, N) -> {handoff_acksync_threshold, 20}, {handoff_receive_timeout, 2000}, {gossip_limit, {10000000, 60000}}]}], - rt_cluster:deploy_nodes(N, Config). + rt:deploy_nodes(N, Config). \ No newline at end of file diff --git a/tests/verify_handoff_mixed.erl b/tests/verify_handoff_mixed.erl index 0af3955a5..dbf071074 100644 --- a/tests/verify_handoff_mixed.erl +++ b/tests/verify_handoff_mixed.erl @@ -58,7 +58,7 @@ confirm() -> Versions = [{current, SearchEnabled}, {UpgradeVsn, SearchEnabled}], Services = [riak_kv, riak_search, riak_pipe], - [Current, Old] = Nodes = rt_cluster:deploy_nodes(Versions, Services), + [Current, Old] = Nodes = rt:deploy_nodes(Versions, Services), prepare_vnodes(Current), @@ -67,7 +67,7 @@ confirm() -> OldFold = rt:capability(Old, ?FOLD_CAPABILITY, v1), %% now link the nodes together and wait for handoff to complete - ok = rt_node:join(Old, Current), + ok = rt:join(Old, Current), ok = rt:wait_until_all_members(Nodes), ok = rt:wait_until_ring_converged(Nodes), @@ -101,7 +101,7 @@ prepare_vnodes(Node) -> prepare_kv_vnodes(Node) -> lager:info("Preparing KV vnodes with keys 1-~b in bucket ~s", [?KV_COUNT, ?KV_BUCKET]), - C = rt_pb:pbc(Node), + C = rt:pbc(Node), lists:foreach( fun(KV) -> ok = riakc_pb_socket:put(C, riakc_obj:new(?KV_BUCKET, KV, KV)) @@ -113,7 +113,7 @@ prepare_search_vnodes(Node) -> lager:info("Peparing Search vnodes with keys 1000-~b in bucket ~s", [1000+?SEARCH_COUNT, ?SEARCH_BUCKET]), rt:enable_search_hook(Node, ?SEARCH_BUCKET), - C = rt_pb:pbc(Node), + C = rt:pbc(Node), lists:foreach( fun(KV) -> O = riakc_obj:new(?SEARCH_BUCKET, KV, KV, "text/plain"), @@ -183,4 +183,4 @@ find_line(Port, {ok, Data}) -> find_line(Port, file:read_line(Port)) end; find_line(_, _) -> - []. + []. \ No newline at end of file diff --git a/tests/verify_leave.erl b/tests/verify_leave.erl index 52690241e..9bf079f38 100644 --- a/tests/verify_leave.erl +++ b/tests/verify_leave.erl @@ -42,7 +42,7 @@ confirm() -> lager:info("Verify ~p no longer owns partitions and all nodes believe " "it is invalid", [Node2]), Remaining1 = Nodes -- [Node2], - rt_node:wait_until_nodes_agree_about_ownership(Remaining1), + rt:wait_until_nodes_agree_about_ownership(Remaining1), [?assertEqual(invalid, status_of_according_to(Node2, Node)) || Node <- Remaining1], %% Have node1 remove node3 @@ -54,6 +54,6 @@ confirm() -> lager:info("Verify ~p no longer owns partitions, and all nodes believe " "it is invalid", [Node3]), Remaining2 = Remaining1 -- [Node3], - rt_node:wait_until_nodes_agree_about_ownership(Remaining2), + rt:wait_until_nodes_agree_about_ownership(Remaining2), [?assertEqual(invalid, status_of_according_to(Node3, Node)) || Node <- Remaining2], pass. diff --git a/tests/verify_membackend.erl b/tests/verify_membackend.erl index 894d0b8cb..d5c9b904c 100644 --- a/tests/verify_membackend.erl +++ b/tests/verify_membackend.erl @@ -35,18 +35,18 @@ confirm() -> ttl(Mode) -> Conf = mkconf(ttl, Mode), - [NodeA, NodeB] = rt_cluster:deploy_nodes(2, Conf), + [NodeA, NodeB] = rt:deploy_nodes(2, Conf), ?assertEqual(ok, check_leave_and_expiry(NodeA, NodeB)), - rt_cluster:clean_cluster([NodeA]), + rt:clean_cluster([NodeA]), ok. max_memory(Mode) -> Conf = mkconf(max_memory, Mode), - [NodeA, NodeB] = rt_cluster:deploy_nodes(2, Conf), + [NodeA, NodeB] = rt:deploy_nodes(2, Conf), - rt_node:join(NodeB, NodeA), + rt:join(NodeB, NodeA), ?assertEqual(ok, check_put_delete(NodeA)), @@ -54,14 +54,14 @@ max_memory(Mode) -> ?assertEqual(ok, check_eviction(NodeA)), - rt_cluster:clean_cluster([NodeA, NodeB]), + rt:clean_cluster([NodeA, NodeB]), ok. combo(Mode) -> Conf = mkconf(combo, Mode), - [NodeA, NodeB] = rt_cluster:deploy_nodes(2, Conf), + [NodeA, NodeB] = rt:deploy_nodes(2, Conf), ?assertEqual(ok, check_leave_and_expiry(NodeA, NodeB)), @@ -75,31 +75,31 @@ combo(Mode) -> ?assertEqual(ok, check_eviction(NodeA)), - rt_cluster:clean_cluster([NodeA]), + rt:clean_cluster([NodeA]), ok. check_leave_and_expiry(NodeA, NodeB) -> - ?assertEqual([], rt_systest:write(NodeB, 1, 100, ?BUCKET, 2)), - ?assertEqual([], rt_systest:read(NodeB, 1, 100, ?BUCKET, 2)), + ?assertEqual([], rt:systest_write(NodeB, 1, 100, ?BUCKET, 2)), + ?assertEqual([], rt:systest_read(NodeB, 1, 100, ?BUCKET, 2)), - rt_node:join(NodeB, NodeA), + rt:join(NodeB, NodeA), - ?assertEqual(ok, rt_node:wait_until_nodes_ready([NodeA, NodeB])), + ?assertEqual(ok, rt:wait_until_nodes_ready([NodeA, NodeB])), rt:wait_until_no_pending_changes([NodeA, NodeB]), - rt_node:leave(NodeB), + rt:leave(NodeB), rt:wait_until_unpingable(NodeB), - ?assertEqual([], rt_systest:read(NodeA, 1, 100, ?BUCKET, 2)), + ?assertEqual([], rt:systest_read(NodeA, 1, 100, ?BUCKET, 2)), lager:info("waiting for keys to expire"), timer:sleep(timer:seconds(210)), - _ = rt_systest:read(NodeA, 1, 100, ?BUCKET, 2), + _ = rt:systest_read(NodeA, 1, 100, ?BUCKET, 2), timer:sleep(timer:seconds(5)), - Res = rt_systest:read(NodeA, 1, 100, ?BUCKET, 2), + Res = rt:systest_read(NodeA, 1, 100, ?BUCKET, 2), ?assertEqual(100, length(Res)), ok. @@ -110,9 +110,9 @@ check_eviction(Node) -> Size = 20000 * 8, Val = <<0:Size>>, - ?assertEqual([], rt_systest:write(Node, 1, 500, ?BUCKET, 2, Val)), + ?assertEqual([], rt:systest_write(Node, 1, 500, ?BUCKET, 2, Val)), - Res = length(rt_systest:read(Node, 1, 100, ?BUCKET, 2, Val)), + Res = length(rt:systest_read(Node, 1, 100, ?BUCKET, 2, Val)), %% this is a wider range than I'd like but the final outcome is %% somewhat hard to predict. Just trying to verify that some @@ -214,7 +214,7 @@ mkconf(Test, Mode) -> case Mode of regular -> %% only memory supports TTL - rt_backend:set_backend(memory), + rt:set_backend(memory), [ {riak_core, [ @@ -227,7 +227,7 @@ mkconf(Test, Mode) -> ]} ]; multi -> - rt_backend:set_backend(multi), + rt:set_backend(multi), [ {riak_core, [ {ring_creation_size, 4} @@ -289,4 +289,4 @@ get_used_space(VNode, Node) -> State = Extract(S), Mem = State#state.used_memory, lager:info("got ~p used memory", [Mem]), - Mem. + Mem. \ No newline at end of file diff --git a/tests/verify_object_limits.erl b/tests/verify_object_limits.erl index 674ab8b96..affce14f3 100644 --- a/tests/verify_object_limits.erl +++ b/tests/verify_object_limits.erl @@ -34,13 +34,13 @@ confirm() -> - [Node1] = rt_cluster:build_cluster(1, [{riak_kv, [ + [Node1] = rt:build_cluster(1, [{riak_kv, [ {ring_creation_size, 8}, {max_object_size, ?MAX_SIZE}, {warn_object_size, ?WARN_SIZE}, {max_siblings, ?MAX_SIBLINGS}, {warn_siblings, ?WARN_SIBLINGS}]}]), - C = rt_pb:pbc(Node1), + C = rt:pbc(Node1), %% Set up to grep logs to verify messages rt:setup_log_capture(Node1), @@ -127,4 +127,4 @@ verify_sibling_limits(C, Node1) -> Res = riakc_pb_socket:put(C, O), lager:info("Result when too many siblings : ~p", [Res]), ?assertMatch({error,_}, Res), - ok. + ok. \ No newline at end of file diff --git a/tests/verify_riak_stats.erl b/tests/verify_riak_stats.erl index 9f2a409f8..bbb72639d 100644 --- a/tests/verify_riak_stats.erl +++ b/tests/verify_riak_stats.erl @@ -73,8 +73,8 @@ confirm() -> lager:info("as the stat system only does calcs for > 5 readings"), C = rt_http:httpc(Node1), - [rt_http:httpc_write(C, <<"systest">>, <>, <<"12345">>) || X <- lists:seq(1, 5)], - [rt_http:httpc_read(C, <<"systest">>, <>) || X <- lists:seq(1, 5)], + [rt:httpc_write(C, <<"systest">>, <>, <<"12345">>) || X <- lists:seq(1, 5)], + [rt:httpc_read(C, <<"systest">>, <>) || X <- lists:seq(1, 5)], Stats2 = get_stats(Node1), @@ -102,10 +102,10 @@ confirm() -> lager:info("Make PBC Connection"), - Pid = rt_pb:pbc(Node1), + Pid = rt:pbc(Node1), Stats3 = get_stats(Node1), - rt_systest:write(Node1, 1), + rt:systest_write(Node1, 1), %% make sure the stats that were supposed to increment did verify_inc(Stats2, Stats3, [{<<"pbc_connects_total">>, 1}, {<<"pbc_connects">>, 1}, @@ -114,14 +114,14 @@ confirm() -> lager:info("Force Read Repair"), - rt_pb:pbc_write(Pid, <<"testbucket">>, <<"1">>, <<"blah!">>), + rt:pbc_write(Pid, <<"testbucket">>, <<"1">>, <<"blah!">>), rt:pbc_set_bucket_prop(Pid, <<"testbucket">>, [{n_val, 4}]), Stats4 = get_stats(Node1), verify_inc(Stats3, Stats4, [{<<"read_repairs_total">>, 0}, {<<"read_repairs">>, 0}]), - _Value = rt_pb:pbc_read(Pid, <<"testbucket">>, <<"1">>), + _Value = rt:pbc_read(Pid, <<"testbucket">>, <<"1">>), Stats5 = get_stats(Node1), diff --git a/tests/verify_search.erl b/tests/verify_search.erl index 0444180ab..3267adb69 100644 --- a/tests/verify_search.erl +++ b/tests/verify_search.erl @@ -31,7 +31,7 @@ confirm() -> Config = [{riak_search, [{enabled, true}]}], - [Node0 | _RestNodes] = Nodes = rt_cluster:build_cluster(3, Config), + [Node0 | _RestNodes] = Nodes = rt:build_cluster(3, Config), rt:wait_until_ring_converged(Nodes), Path = rt_config:get(rt_scratch_dir), @@ -65,4 +65,4 @@ test_dirs(BaseDir) -> %% @todo Figure out why this one is not run by run_all.sh %% It does fail in a weird way if included SubDir /= "replication_test", - filelib:is_file(filename:join([BaseDir, SubDir, "script.def"]))]. + filelib:is_file(filename:join([BaseDir, SubDir, "script.def"]))]. \ No newline at end of file diff --git a/tests/verify_staged_clustering.erl b/tests/verify_staged_clustering.erl index 7970c6d79..a61654445 100644 --- a/tests/verify_staged_clustering.erl +++ b/tests/verify_staged_clustering.erl @@ -24,7 +24,7 @@ -include_lib("eunit/include/eunit.hrl"). confirm() -> - Nodes = rt_cluster:deploy_nodes(4), + Nodes = rt:deploy_nodes(4), [Node1, Node2, Node3, Node4] = Nodes, Nodes123 = [Node1, Node2, Node3], Nodes23 = [Node2, Node3], @@ -45,9 +45,9 @@ confirm() -> commit_staged(Node1), lager:info("Ensure that ~p now own all partitions", [Nodes123]), - ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes123)), + ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes123)), ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes123)), - rt_ring:assert_nodes_agree_about_ownership(Nodes123), + rt:assert_nodes_agree_about_ownership(Nodes123), lager:info("Join ~p to the cluster", [Node4]), stage_join(Node4, Node1), @@ -62,7 +62,7 @@ confirm() -> Nodes134 = [Node1, Node3, Node4], lager:info("Ensure that ~p now own all partitions", [Nodes134]), - ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes134)), + ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes134)), ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes134)), rt:assert_nodes_agree_about_ownership(Nodes134), @@ -87,9 +87,9 @@ confirm() -> Nodes124 = [Node1, Node2, Node4], lager:info("Ensure that ~p now own all partitions", [Nodes124]), - ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes124)), + ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes124)), ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes124)), - rt_ring:assert_nodes_agree_about_ownership(Nodes124), + rt:assert_nodes_agree_about_ownership(Nodes124), lager:info("Stage leave of ~p", [Node2]), stage_leave(Node1, Node2), @@ -121,28 +121,28 @@ n(Atom) -> stage_join(Node, OtherNode) -> %% rpc:call(Node, riak_kv_console, staged_join, [[n(OtherNode)]]). - rt_cmd_line:admin(Node, ["cluster", "join", n(OtherNode)]). + rt:admin(Node, ["cluster", "join", n(OtherNode)]). stage_leave(Node, OtherNode) -> %% rpc:call(Node, riak_core_console, stage_leave, [[n(OtherNode)]]). - rt_cmd_line:admin(Node, ["cluster", "leave", n(OtherNode)]). + rt:admin(Node, ["cluster", "leave", n(OtherNode)]). stage_remove(Node, OtherNode) -> %% rpc:call(Node, riak_core_console, stage_remove, [[n(OtherNode)]]). - rt_cmd_line:admin(Node, ["cluster", "force-remove", n(OtherNode)]). + rt:admin(Node, ["cluster", "force-remove", n(OtherNode)]). stage_replace(Node, Node1, Node2) -> %% rpc:call(Node, riak_core_console, stage_replace, [[n(Node1), n(Node2)]]). - rt_cmd_line:admin(Node, ["cluster", "replace", n(Node1), n(Node2)]). + rt:admin(Node, ["cluster", "replace", n(Node1), n(Node2)]). stage_force_replace(Node, Node1, Node2) -> %% rpc:call(Node, riak_core_console, stage_force_replace, [[n(Node1), n(Node2)]]). - rt_cmd_line:admin(Node, ["cluster", "force-replace", n(Node1), n(Node2)]). + rt:admin(Node, ["cluster", "force-replace", n(Node1), n(Node2)]). print_staged(Node) -> %% rpc:call(Node, riak_core_console, print_staged, [[]]). F = fun(_) -> - {ok, StdOut} = rt_cmd_line:admin(Node, ["cluster", "plan"]), + {ok, StdOut} = rt:admin(Node, ["cluster", "plan"]), case StdOut of "Cannot" ++ _X -> false; _ -> true @@ -152,11 +152,11 @@ print_staged(Node) -> commit_staged(Node) -> %% rpc:call(Node, riak_core_console, commit_staged, [[]]). - rt_cmd_line:admin(Node, ["cluster", "commit"]). + rt:admin(Node, ["cluster", "commit"]). clear_staged(Node) -> %% rpc:call(Node, riak_core_console, clear_staged, [[]]). - rt_cmd_line:admin(Node, ["cluster", "clear"]). + rt:admin(Node, ["cluster", "clear"]). stage_join_rpc(Node, OtherNode) -> rpc:call(Node, riak_core, staged_join, [OtherNode]). @@ -180,4 +180,4 @@ commit_staged_rpc(Node) -> rpc:call(Node, riak_core_claimant, commit, []). clear_staged_rpc(Node) -> - rpc:call(Node, riak_core_claimant, clear, []). + rpc:call(Node, riak_core_claimant, clear, []). \ No newline at end of file diff --git a/tests/yz_crdt.erl b/tests/yz_crdt.erl index b79fe49d2..ff15b1143 100644 --- a/tests/yz_crdt.erl +++ b/tests/yz_crdt.erl @@ -66,4 +66,4 @@ confirm() -> %% Clean cluster. rt:clean_cluster(Nodes), - pass. + pass. \ No newline at end of file From 19d931be254854a73ae3dc755437a3c674b15225 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Wed, 25 Feb 2015 01:27:35 +0000 Subject: [PATCH 070/157] Update to support verify_membackend --- src/rt.erl | 27 +++++++----- src/rt_harness.erl | 11 ++++- src/rt_systest.erl | 1 + src/rtdev.erl | 88 +++++++++++++++++++++++++------------ tests/verify_membackend.erl | 23 +++++----- 5 files changed, 101 insertions(+), 49 deletions(-) diff --git a/src/rt.erl b/src/rt.erl index 9964027a6..580959dc9 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -68,6 +68,7 @@ get_replica/5, get_ring/1, get_version/0, + get_version/1, heal/1, http_url/1, https_url/1, @@ -242,7 +243,7 @@ deploy_nodes(Versions) when is_list(Versions) -> deploy_nodes(NumNodes) when is_integer(NumNodes) -> [NodeIds, NodeMap, _] = allocate_nodes(NumNodes), deploy_nodes(NodeIds, NodeMap, head, rt_properties:default_config(), [riak_kv]). - + %% @doc Deploy a set of freshly installed Riak nodes with the given %% `InitialConfig', returning a list of the nodes deployed. -spec deploy_nodes(NumNodes :: integer(), any()) -> [node()]. @@ -251,7 +252,6 @@ deploy_nodes(NumNodes, InitialConfig) when is_integer(NumNodes) -> deploy_nodes(Versions, Services) -> NodeConfig = [ version_to_config(Version) || Version <- Versions ], lager:debug("Starting nodes config ~p using versions ~p", [NodeConfig, Versions]), - Nodes = rt_harness:deploy_nodes(NodeConfig), lager:info("Waiting for services ~p to start on ~p.", [Services, Nodes]), [ ok = wait_for_service(Node, Service) || Node <- Nodes, @@ -267,16 +267,16 @@ deploy_nodes(NodeIds, NodeMap, Version, Config, Services) -> lists:foldl(fun({_, NodeName}, Nodes) -> [NodeName|Nodes] end, [], NodeMap). - + allocate_nodes(NumNodes) -> [AvailableNodeIds, AvailableNodeMap, VersionMap] = rt_harness:available_resources(), - lager:debug("Availabe node ids ~p, node map ~p, and version map ~p.", + lager:debug("Available node ids ~p, node map ~p, and version map ~p.", [AvailableNodeIds, AvailableNodeMap, VersionMap]), AllocatedNodeIds = lists:sublist(AvailableNodeIds, NumNodes), lager:debug("Allocated node ids ~p with version map ~p", [AllocatedNodeIds, VersionMap]), - + AllocatedNodeMap = lists:foldl( fun(NodeId, NodeMap) -> NodeName = proplists:get_value(NodeId, AvailableNodeMap), @@ -734,15 +734,16 @@ try_nodes_ready(Nodes, N, SleepMs) -> %% @doc Stop nodes and wipe out their data directories clean_cluster(Nodes) when is_list(Nodes) -> - rt_cluster:clean_cluster(Nodes). + [stop_and_wait(Node) || Node <- Nodes], + clean_data_dir(Nodes). clean_data_dir(Nodes) -> - %% TODO Determine the best approach to specifying the version. -jsb - rt_cluster:clean_data_dir(Nodes, head). + clean_data_dir(Nodes, ""). -clean_data_dir(Nodes, SubDir) -> - %% TODO Determine the best approach to specifying the version. -jsb - rt_cluster:clean_data_dir(Nodes, head, SubDir). +clean_data_dir(Nodes, SubDir) when not is_list(Nodes) -> + clean_data_dir([Nodes], SubDir); +clean_data_dir(Nodes, SubDir) when is_list(Nodes) -> + ?HARNESS:clean_data_dir(Nodes, SubDir). %% @doc Shutdown every node, this is for after a test run is complete. teardown() -> @@ -1335,6 +1336,10 @@ get_backend(AppConfigProplist) -> get_version() -> rt2:get_version(). +-spec get_version(string()) -> binary(). +get_version(Node) -> + rt_harness:get_version(Node). + %% @doc outputs some useful information about nodes that are up whats_up() -> rt2:whats_up(). diff --git a/src/rt_harness.erl b/src/rt_harness.erl index f1b7f33d6..f43ad8ded 100644 --- a/src/rt_harness.erl +++ b/src/rt_harness.erl @@ -38,6 +38,7 @@ setup/0, get_deps/0, get_version/0, + get_version/1, get_backends/0, set_backend/1, whats_up/0, @@ -53,7 +54,8 @@ update_app_config/3, teardown/0, set_conf/2, - set_advanced_conf/2]). + set_advanced_conf/2, + update_app_config/2]). start(Node, Version) -> ?HARNESS_MODULE:start(Node, Version). @@ -94,6 +96,9 @@ get_deps() -> get_version() -> ?HARNESS_MODULE:get_version(). +get_version(Node) -> + ?HARNESS_MODULE:get_version(Node). + get_backends() -> ?HARNESS_MODULE:get_backends(). @@ -141,3 +146,7 @@ set_conf(Node, NameValuePairs) -> set_advanced_conf(Node, NameValuePairs) -> ?HARNESS_MODULE:set_advanced_conf(Node, NameValuePairs). + +update_app_config(Node, Config) -> + ?HARNESS_MODULE:update_app_config(Node, Config). + diff --git a/src/rt_systest.erl b/src/rt_systest.erl index 918496bb0..bb7173495 100644 --- a/src/rt_systest.erl +++ b/src/rt_systest.erl @@ -25,6 +25,7 @@ read/3, read/5, read/6, + read/7, write/2, write/3, write/5, diff --git a/src/rtdev.erl b/src/rtdev.erl index 27b0e2a96..731ad1712 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -37,6 +37,7 @@ whats_up/0, get_ip/1, node_id/1, + node_short_name/1, node_version/1, %% admin/2, riak/2, @@ -338,15 +339,21 @@ all_the_app_configs(DevPath) -> update_app_config(all, Config) -> lager:info("rtdev:update_app_config(all, ~p)", [Config]), - [ update_app_config(DevPath, Config) || DevPath <- devpaths()]. + [ update_app_config(DevPath, Config) || DevPath <- devpaths()]; +update_app_config(Node, Config) when is_atom(Node) -> + lager:info("rtdev:update_app_config Node(~p, ~p)", [Node, Config]), + Version = node_version(Node), + update_app_config(Node, Version, Config); +update_app_config(DevPath, Config) -> + [update_app_config_file(AppConfig, Config) || AppConfig <- all_the_app_configs(DevPath)]. update_app_config(Node, Version, Config) -> VersionPath = filename:join(?PATH, Version), FileFormatString = "~s/~s/etc/~s.config", AppConfigFile = io_lib:format(FileFormatString, - [VersionPath, Node, "app"]), + [VersionPath, node_short_name(Node), "app"]), AdvConfigFile = io_lib:format(FileFormatString, - [VersionPath, Node, "advanced"]), + [VersionPath, node_short_name(Node), "advanced"]), %% If there's an app.config, do it old style %% if not, use cuttlefish's advanced.config @@ -357,6 +364,7 @@ update_app_config(Node, Version, Config) -> update_app_config_file(AdvConfigFile, Config) end. + update_app_config_file(ConfigFile, Config) -> lager:debug("rtdev:update_app_config_file(~s, ~p)", [ConfigFile, Config]), @@ -393,12 +401,12 @@ get_backend(AppConfig) -> ["app.config"| _ ] -> AppConfig; ["advanced.config" | T] -> - ["etc", [$d, $e, $v | N], "dev" | RPath] = T, + ["etc", Node | RPath] = T, Path = filename:join(lists:reverse(RPath)), %% Why chkconfig? It generates an app.config from cuttlefish %% without starting riak. - ChkConfigOutput = string:tokens(run_riak(list_to_integer(N), Path, "chkconfig"), "\n"), + ChkConfigOutput = string:tokens(run_riak(Node, Path, "chkconfig"), "\n"), ConfigFileOutputLine = lists:last(ChkConfigOutput), @@ -418,7 +426,7 @@ get_backend(AppConfig) -> case filename:pathtype(Files) of absolute -> File; relative -> - io_lib:format("~s/dev~s/~s", [Path, N, tl(hd(Files))]) + io_lib:format("~s/~s/~s", [Path, Node, tl(hd(Files))]) end end end, @@ -432,10 +440,8 @@ get_backend(AppConfig) -> end. node_path(Node) -> - N = node_id(Node), - lager:debug("Node ~p node id ~p", [Node, N]), - Path = relpath(node_version(N)), - lists:flatten(io_lib:format("~s/dev/dev~b", [Path, N])). + Path = relpath(node_version(Node)), + lists:flatten(io_lib:format("~s/~s", [Path, node_short_name(Node)])). get_ip(_Node) -> %% localhost 4 lyfe @@ -531,7 +537,7 @@ configure_nodes(Nodes, Configs) -> lists:zip(Nodes, Configs)). deploy_nodes(NodeConfig) -> - Path = relpath(root), + Path = relpath(""), lager:info("Riak path: ~p", [Path]), NumNodes = length(NodeConfig), %% TODO: The starting index should not be fixed to 1 @@ -539,34 +545,36 @@ deploy_nodes(NodeConfig) -> FullNodes = [devrel_node_name(N) || N <- NodesN], DevNodes = [list_to_atom(lists:concat(["dev", N])) || N <- NodesN], NodeMap = orddict:from_list(lists:zip(FullNodes, NodesN)), - {Versions, Configs} = lists:unzip(NodeConfig), - VersionMap = lists:zip(DevNodes, Versions), + DevNodeMap = orddict:from_list(lists:zip(FullNodes, DevNodes)), + {Versions, _} = lists:unzip(NodeConfig), + VersionMap = lists:zip(FullNodes, Versions), %% TODO The new node deployment doesn't appear to perform this check ... -jsb %% Check that you have the right versions available %%[ check_node(Version) || Version <- VersionMap ], rt_config:set(rt_nodes, NodeMap), + rt_config:set(rt_nodenames, DevNodeMap), rt_config:set(rt_versions, VersionMap), - create_dirs(DevNodes), + create_dirs(FullNodes), %% Set initial config add_default_node_config(FullNodes), - rt:pmap(fun({_, default}) -> + rt:pmap(fun({_, {_, default}}) -> ok; - ({Node, {cuttlefish, Config}}) -> + ({Node, {_, {cuttlefish, Config}}}) -> set_conf(Node, Config); - ({Node, Config}) -> - update_app_config(Node, Config) + ({Node, {Version, Config}}) -> + update_app_config(Node, Version, Config) end, - lists:zip(FullNodes, Configs)), + lists:zip(FullNodes, NodeConfig)), %% create snmp dirs, for EE - create_dirs(DevNodes), + create_dirs(FullNodes), %% Start nodes %%[run_riak(N, relpath(node_version(N)), "start") || N <- NodesN], - rt:pmap(fun(Node) -> run_riak(Node, relpath(node_version(Node)), "start") end, DevNodes), + rt:pmap(fun(Node) -> run_riak(node_short_name(Node), relpath(node_version(Node)), "start") end, FullNodes), %% Ensure nodes started [ok = rt:wait_until_pingable(N) || N <- FullNodes], @@ -578,7 +586,7 @@ deploy_nodes(NodeConfig) -> [ok = rt:wait_until_registered(N, riak_core_ring_manager) || N <- FullNodes], %% Ensure nodes are singleton clusters - [ok = rt_ring:check_singleton_node(?DEV(Node)) || {Node, Version} <- VersionMap, + [ok = rt_ring:check_singleton_node(FullNode) || {FullNode, Version} <- VersionMap, Version /= "0.14.2"], lager:info("Deployed nodes: ~p", [FullNodes]), @@ -806,11 +814,22 @@ riak_repl(Node, Args) -> lager:info("~s", [Result]), {ok, Result}. +%% @doc Find the node number from the full name +-spec node_id(atom()) -> integer(). node_id(Node) -> NodeMap = rt_config:get(rt_nodes), orddict:fetch(Node, NodeMap). -%% @doc Return the node version from rt_versions +%% @doc Find the short dev node name from the full name +-spec node_short_name(atom()) -> atom(). +node_short_name(Node) when is_list(Node) -> + Node; +node_short_name(Node) when is_atom(Node) -> + NodeMap = rt_config:get(rt_nodenames), + orddict:fetch(Node, NodeMap). + +%% @doc Return the node version from rt_versions based on full node name +-spec node_version(atom()) -> string(). node_version(Node) -> VersionMap = rt_config:get(rt_versions), orddict:fetch(Node, VersionMap). @@ -869,8 +888,15 @@ set_backend(Backend, OtherOpts) -> update_app_config(all, version_here, [{riak_kv, Opts}]), get_backends(). +%% WRONG: Seemingly always stuck on the current version get_version() -> - case file:read_file(relpath(current) ++ "/VERSION") of + case file:read_file(relpath(head) ++ "/VERSION") of + {error, enoent} -> unknown; + {ok, Version} -> Version + end. + +get_version(Node) -> + case file:read_file(filename:join([relpath(node_version(Node)),"VERSION"])) of {error, enoent} -> unknown; {ok, Version} -> Version end. @@ -887,19 +913,27 @@ whats_up() -> Up = [rpc:call(Node, os, cmd, ["pwd"]) || Node <- nodes()], [io:format(" ~s~n",[string:substr(Dir, 1, length(Dir)-1)]) || Dir <- Up]. +%% @doc Gather the devrel directories in the root_path parent directory +-spec devpaths() -> list(). devpaths() -> - lists:usort([ DevPath || {_Name, DevPath} <- proplists:delete(root, rt_config:get(rtdev_path))]). + RootDir = rt_config:get(root_path), + {ok, RawDirs} = file:list_dir(RootDir), + %% Remove any dot files in the directory (e.g. .git) + FilteredPaths = lists:filter(fun([$.|_]) -> false; (_) -> true end, RawDirs), + %% Generate fully qualified path names + DevPaths = lists:map(fun(X) -> filename:join(RootDir, X) end, FilteredPaths), + lists:usort(DevPaths). %% versions() -> %% proplists:get_keys(rt_config:get(rtdev_path)) -- [root]. get_node_logs() -> - Root = filename:absname(proplists:get_value(root, ?PATH)), + Root = filename:absname(proplists:get_value(root_path, ?PATH)), RootLen = length(Root) + 1, %% Remove the leading slash [ begin {ok, Port} = file:open(Filename, [read, binary]), {lists:nthtail(RootLen, Filename), Port} - end || Filename <- filelib:wildcard(Root ++ "/*/dev/dev*/log/*") ]. + end || Filename <- filelib:wildcard(Root ++ "/*/dev*/log/*") ]. -type node_tuple() :: {list(), atom()}. diff --git a/tests/verify_membackend.erl b/tests/verify_membackend.erl index d5c9b904c..a7636e7cc 100644 --- a/tests/verify_membackend.erl +++ b/tests/verify_membackend.erl @@ -100,7 +100,7 @@ check_leave_and_expiry(NodeA, NodeB) -> _ = rt:systest_read(NodeA, 1, 100, ?BUCKET, 2), timer:sleep(timer:seconds(5)), Res = rt:systest_read(NodeA, 1, 100, ?BUCKET, 2), - + ?assertEqual(100, length(Res)), ok. @@ -201,7 +201,10 @@ put_until_changed(Pid, Node, Key) -> end. mkconf(Test, Mode) -> - MembConfig = + RiakCore = [ + {ring_creation_size, 4} + ], + MembConfig = case Test of ttl -> [{ttl, 200}]; @@ -217,9 +220,7 @@ mkconf(Test, Mode) -> rt:set_backend(memory), [ - {riak_core, [ - {ring_creation_size, 4} - ]}, + {riak_core, RiakCore}, {riak_kv, [ {anti_entropy, {off, []}}, {delete_mode, immediate}, @@ -229,9 +230,7 @@ mkconf(Test, Mode) -> multi -> rt:set_backend(multi), [ - {riak_core, [ - {ring_creation_size, 4} - ]}, + {riak_core, RiakCore}, {riak_kv, [ {anti_entropy, {off, []}}, {delete_mode, immediate}, @@ -256,7 +255,7 @@ get_remote_vnode_pid(Node) -> get_used_space(VNode, Node) -> S = rpc:call(Node, sys, get_state, [VNode]), Mode = get(mode), - Version = rt:get_version(), + Version = rt:get_version(Node), %% lager:info("version mode ~p", [{Version, Mode}]), TwoOhReg = fun(X) -> @@ -281,6 +280,10 @@ get_used_space(VNode, Node) -> TwoOhMulti; {<<"riak_ee-2.0",_/binary>>, multi} -> TwoOhMulti; + {<<"head",_/binary>>, regular} -> + TwoOhReg; + {<<"head",_/binary>>, multi} -> + TwoOhMulti; _Else -> lager:error("didn't understand version/mode tuple ~p", [{Version, Mode}]), @@ -289,4 +292,4 @@ get_used_space(VNode, Node) -> State = Extract(S), Mem = State#state.used_memory, lager:info("got ~p used memory", [Mem]), - Mem. \ No newline at end of file + Mem. From 660b6718e56b4894536e1515060639a7aba084fc Mon Sep 17 00:00:00 2001 From: John Burwell Date: Wed, 25 Feb 2015 11:12:29 -0500 Subject: [PATCH 071/157] Adds support for admin/3 to the rt_cmd_line, rt_harness, and rtdev modules and clarifies debugging in rt:deploy_nodes/2 --- src/rt.erl | 3 ++- src/rt_cmd_line.erl | 7 +++++++ src/rt_harness.erl | 4 ++++ src/rtdev.erl | 2 ++ 4 files changed, 15 insertions(+), 1 deletion(-) diff --git a/src/rt.erl b/src/rt.erl index 580959dc9..dcc3d5fa3 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -251,7 +251,8 @@ deploy_nodes(NumNodes, InitialConfig) when is_integer(NumNodes) -> deploy_nodes(NumNodes, InitialConfig, [riak_kv]); deploy_nodes(Versions, Services) -> NodeConfig = [ version_to_config(Version) || Version <- Versions ], - lager:debug("Starting nodes config ~p using versions ~p", [NodeConfig, Versions]), + lager:debug("Starting nodes using config ~p and versions ~p", [NodeConfig, Versions]), + Nodes = rt_harness:deploy_nodes(NodeConfig), lager:info("Waiting for services ~p to start on ~p.", [Services, Nodes]), [ ok = wait_for_service(Node, Service) || Node <- Nodes, diff --git a/src/rt_cmd_line.erl b/src/rt_cmd_line.erl index f422e68fb..af592efeb 100644 --- a/src/rt_cmd_line.erl +++ b/src/rt_cmd_line.erl @@ -22,6 +22,7 @@ -include_lib("eunit/include/eunit.hrl"). -export([admin/2, + admin/3, riak/2, riak_repl/2, search_cmd/2, @@ -38,6 +39,12 @@ admin(Node, Args) -> rt_harness:admin(Node, Args). +%% @doc Call 'bin/riak-admin' command on `Node' with arguments `Args'. +%% The third parameter is a list of options. Valid options are: +%% * `return_exit_code' - Return the exit code along with the command output +admin(Node, Args, Options) -> + rt_harness:admin(Node, Args, Options). + %% @doc Call 'bin/riak' command on `Node' with arguments `Args' riak(Node, Args) -> rt_harness:riak(Node, Args). diff --git a/src/rt_harness.erl b/src/rt_harness.erl index f43ad8ded..e89105f74 100644 --- a/src/rt_harness.erl +++ b/src/rt_harness.erl @@ -46,6 +46,7 @@ node_id/1, node_version/1, admin/2, + admin/3, riak/2, run_riak/3, attach/2, @@ -120,6 +121,9 @@ node_version(N) -> admin(Node, Args) -> ?HARNESS_MODULE:admin(Node, Args). +admin(Node, Args, Options) -> + ?HARNESS_MODULE:admin(Node, Args, Options). + riak(Node, Args) -> ?HARNESS_MODULE:riak(Node, Args). diff --git a/src/rtdev.erl b/src/rtdev.erl index 731ad1712..9bfafaa33 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -556,6 +556,8 @@ deploy_nodes(NodeConfig) -> rt_config:set(rt_nodenames, DevNodeMap), rt_config:set(rt_versions, VersionMap), + lager:debug("Set rtnodes: ~p and rt_versions: ~p", [ rt_config:get(rt_nodes), rt_config:get(rt_versions) ]), + create_dirs(FullNodes), %% Set initial config From 0be645e0d514d45cee425e5891b50d41cd3d2dc6 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Wed, 25 Feb 2015 18:30:12 -0500 Subject: [PATCH 072/157] verify_staged_clustering passes using the r_t v1 API Wrapper * Sets the rt_nodes, rt_nodenames, and rt_version configuration properties in rt:allocate_nodes * Modified rtdev:admin/3 to extract the node_id from the passed node --- src/rt.erl | 17 ++++++++++++++++- src/rtdev.erl | 5 +++-- 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/src/rt.erl b/src/rt.erl index dcc3d5fa3..fddafb55c 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -277,7 +277,6 @@ allocate_nodes(NumNodes) -> AllocatedNodeIds = lists:sublist(AvailableNodeIds, NumNodes), lager:debug("Allocated node ids ~p with version map ~p", [AllocatedNodeIds, VersionMap]), - AllocatedNodeMap = lists:foldl( fun(NodeId, NodeMap) -> NodeName = proplists:get_value(NodeId, AvailableNodeMap), @@ -285,7 +284,23 @@ allocate_nodes(NumNodes) -> end, [], AllocatedNodeIds), + NodeVersionMap = orddict:fold( + fun(Version, Nodes, NodeVersionMap) -> + lists:foldl( + fun(Node, Acc) -> orddict:append(Node, Version, Acc) end, + NodeVersionMap, + Nodes) + end, + orddict:new(), + VersionMap), + lager:debug("Allocated node map ~p", [AllocatedNodeMap]), + + rt_config:set(rt_nodes, AllocatedNodeMap), + %% May be incorrect -- see rtdev:deploy_nodes ... -jsb + rt_config:set(rt_nodenames, AllocatedNodeMap), + rt_config:set(rt_versions, NodeVersionMap), + [AllocatedNodeIds, AllocatedNodeMap, VersionMap]. version_to_config(Config) when is_tuple(Config)-> Config; diff --git a/src/rtdev.erl b/src/rtdev.erl index 9bfafaa33..d0ce6917c 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -787,8 +787,9 @@ interactive_loop(Port, Expected) -> end. admin(Node, Args, Options) -> - Path = relpath(node_version(Node)), - Cmd = riak_admin_cmd(Path, Node, Args), + {NodeId, _} = extract_node_id_and_name(Node), + Path = relpath(node_version(NodeId)), + Cmd = riak_admin_cmd(Path, NodeId, Args), lager:info("Running: ~s", [Cmd]), Result = execute_admin_cmd(Cmd, Options), lager:info("~p", [Result]), From 4ab1e6751c90a3beb888cd6f31f6cbe63d508fd6 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Thu, 26 Feb 2015 11:12:51 -0500 Subject: [PATCH 073/157] Adds debug logging to rt:allocate_nodes to introspect the calculation of rt_versions, rt_nodes, and rt_nodenames configuration properties --- src/rt.erl | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/rt.erl b/src/rt.erl index fddafb55c..b553bba48 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -301,6 +301,9 @@ allocate_nodes(NumNodes) -> rt_config:set(rt_nodenames, AllocatedNodeMap), rt_config:set(rt_versions, NodeVersionMap), + lager:debug("Set rt_nodes: ~p, rt_nodenames: ~p, rt_versions: ~p", + [ rt_config:get(rt_nodes), rt_config:get(rt_nodenames), rt_config:get(rt_versions) ]), + [AllocatedNodeIds, AllocatedNodeMap, VersionMap]. version_to_config(Config) when is_tuple(Config)-> Config; From 5f5252022813095db3002338c1853d3b6ffef4b6 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Thu, 26 Feb 2015 23:50:01 -0500 Subject: [PATCH 074/157] Modified rt_harness_util to use the rt2 rather than the rt module --- src/rt_harness_util.erl | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/rt_harness_util.erl b/src/rt_harness_util.erl index 4e8305c64..03a9d04b8 100644 --- a/src/rt_harness_util.erl +++ b/src/rt_harness_util.erl @@ -93,18 +93,18 @@ deploy_nodes(NodeIds, NodeMap, Version, Config, Services) -> fun(Node) -> rt_harness:update_app_config(Node, Version, Config) end, - rt:pmap(ConfigUpdateFun, NodeIds), + rt2:pmap(ConfigUpdateFun, NodeIds), %% Start nodes RunRiakFun = fun(Node) -> rt_harness:run_riak(Node, Version, "start") end, - rt:pmap(RunRiakFun, NodeIds), + rt2:pmap(RunRiakFun, NodeIds), %% Ensure nodes started lager:debug("Wait until pingable: ~p", [NodeIds]), - [ok = rt:wait_until_pingable(rt_node:node_name(NodeId, NodeMap)) + [ok = rt2:wait_until_pingable(rt_node:node_name(NodeId, NodeMap)) || NodeId <- NodeIds], %% TODO Rubbish! Fix this. @@ -114,7 +114,7 @@ deploy_nodes(NodeIds, NodeMap, Version, Config, Services) -> %% We have to make sure that riak_core_ring_manager is running %% before we can go on. - [ok = rt:wait_until_registered(rt_node:node_name(NodeId, NodeMap), + [ok = rt2:wait_until_registered(rt_node:node_name(NodeId, NodeMap), riak_core_ring_manager) || NodeId <- NodeIds], @@ -129,7 +129,7 @@ deploy_nodes(NodeIds, NodeMap, Version, Config, Services) -> %% Wait for services to start lager:debug("Waiting for services ~p to start on ~p.", [Services, NodeIds]), - [ ok = rt:wait_for_service(rt_node:node_name(NodeId, NodeMap), Service) + [ ok = rt2:wait_for_service(rt_node:node_name(NodeId, NodeMap), Service) || NodeId <- NodeIds, Service <- Services ], @@ -170,7 +170,7 @@ interactive_loop(Port, Expected) -> [] -> [{done, "done"}|[]]; E -> E end, - case {Type, rt:str(X, Text)} of + case {Type, rt2:str(X, Text)} of {expect, true} -> RemainingExpect; {expect, false} -> @@ -225,7 +225,7 @@ spawn_cmd(Cmd, Opts) -> Port. wait_for_cmd(Port) -> - rt:wait_until(node(), + rt2:wait_until(node(), fun(_) -> receive {Port, Msg={exit_status, _}} -> From 050ba3fdff008be35c114dd6c3f74c93b90c2d0d Mon Sep 17 00:00:00 2001 From: John Burwell Date: Thu, 26 Feb 2015 23:51:58 -0500 Subject: [PATCH 075/157] Modifies rtdev-all to use make deps for non-tagged builds --- bin/rtdev-all.sh | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/bin/rtdev-all.sh b/bin/rtdev-all.sh index 24bef9440..c932fd027 100755 --- a/bin/rtdev-all.sh +++ b/bin/rtdev-all.sh @@ -149,7 +149,14 @@ build() echo " - Building stagedevrel in $SRCDIR (this could take a while)" cd $SRCDIR - $RUN make locked-deps + # For non-tagged builds (i.e. head), use make deps. Otherwise, use + # make locked-deps for tagged builds ... + if [ "$SRCDIR" == "head" ]; then + make deps + else + $RUN make locked-deps + fi + $RUN make all stagedevrel RES=$? if [ "$RES" -ne 0 ]; then From 40730908bf7328a7e339aba6838c31ca0d651598 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Thu, 26 Feb 2015 23:52:46 -0500 Subject: [PATCH 076/157] The verify_staged_clustering test case passes using the r_t v1 API wrapper * Modifies rt:allocate_nodes to set rt_versions based on the allocated nodes rather the available nodes -- properly mapping node ids to versions -- fixing the operation of the rtdev:admin function when multiple versions are deployed. --- src/rt.erl | 31 +++++++++++++++---------------- 1 file changed, 15 insertions(+), 16 deletions(-) diff --git a/src/rt.erl b/src/rt.erl index b553bba48..28b6de39a 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -241,7 +241,7 @@ get_https_conn_info(Node) -> deploy_nodes(Versions) when is_list(Versions) -> deploy_nodes(Versions, [riak_kv]); deploy_nodes(NumNodes) when is_integer(NumNodes) -> - [NodeIds, NodeMap, _] = allocate_nodes(NumNodes), + [NodeIds, NodeMap, _] = allocate_nodes(NumNodes, head), deploy_nodes(NodeIds, NodeMap, head, rt_properties:default_config(), [riak_kv]). %% @doc Deploy a set of freshly installed Riak nodes with the given @@ -269,13 +269,16 @@ deploy_nodes(NodeIds, NodeMap, Version, Config, Services) -> [], NodeMap). -allocate_nodes(NumNodes) -> - [AvailableNodeIds, AvailableNodeMap, VersionMap] = rt_harness:available_resources(), - lager:debug("Available node ids ~p, node map ~p, and version map ~p.", - [AvailableNodeIds, AvailableNodeMap, VersionMap]), +allocate_nodes(NumNodes, Version) when is_atom(Version) -> + allocate_nodes(NumNodes, atom_to_list(Version)); +allocate_nodes(NumNodes, Version) -> + [_, AvailableNodeMap, VersionMap] = rt_harness:available_resources(), + lager:debug("Available node map ~p and version map ~p.", [AvailableNodeMap, VersionMap]), + AvailableNodeIds = proplists:get_value(Version, VersionMap), + lager:debug("Availabe node ids ~p for version ~p", [AvailableNodeIds, Version]), AllocatedNodeIds = lists:sublist(AvailableNodeIds, NumNodes), - lager:debug("Allocated node ids ~p with version map ~p", [AllocatedNodeIds, VersionMap]), + lager:debug("Allocated node ids ~p", [AllocatedNodeIds]), AllocatedNodeMap = lists:foldl( fun(NodeId, NodeMap) -> @@ -284,15 +287,10 @@ allocate_nodes(NumNodes) -> end, [], AllocatedNodeIds), - NodeVersionMap = orddict:fold( - fun(Version, Nodes, NodeVersionMap) -> - lists:foldl( - fun(Node, Acc) -> orddict:append(Node, Version, Acc) end, - NodeVersionMap, - Nodes) - end, + NodeVersionMap = lists:foldl( + fun(NodeId, Acc) -> orddict:append(NodeId, Version, Acc) end, orddict:new(), - VersionMap), + AllocatedNodeIds), lager:debug("Allocated node map ~p", [AllocatedNodeMap]), @@ -301,8 +299,9 @@ allocate_nodes(NumNodes) -> rt_config:set(rt_nodenames, AllocatedNodeMap), rt_config:set(rt_versions, NodeVersionMap), - lager:debug("Set rt_nodes: ~p, rt_nodenames: ~p, rt_versions: ~p", - [ rt_config:get(rt_nodes), rt_config:get(rt_nodenames), rt_config:get(rt_versions) ]), + lager:debug("Set rt_nodes: ~p", [ rt_config:get(rt_nodes) ]), + lager:debug("Set rt_nodenames: ~p", [ rt_config:get(rt_nodenames) ]), + lager:debug("Set rt_versions: ~p", [ rt_config:get(rt_versions) ]), [AllocatedNodeIds, AllocatedNodeMap, VersionMap]. From 3ef12cbc47e6dd043a5d62699ff32536d21e748a Mon Sep 17 00:00:00 2001 From: John Burwell Date: Fri, 27 Feb 2015 15:10:25 -0500 Subject: [PATCH 077/157] verify_riak_stats passes using the r_t v1 API wrapper * Fixes the types and format of the rt_nodes, rt_nodenames, and rt_versions to match those expected by rtdev in the rt:allocate_nodes function * Adds the rtdev:node_short_name_to_name/1 function to convert a node short name to node name using the ?DEV macro * Adds rtdev:path and rtdev:node_version implementation that accept node numbers and convert to node names using the rtdev:node_short_name_to_name function * Modifies the verify_riak_stats module to expect the xmerl_version stat key. This change should likely be merged by to master. --- src/rt.erl | 29 +++++++++++--------- src/rtdev.erl | 35 +++++++++++++++--------- tests/verify_riak_stats.erl | 53 +++++++++++++++++++------------------ 3 files changed, 66 insertions(+), 51 deletions(-) diff --git a/src/rt.erl b/src/rt.erl index 28b6de39a..d15b7a3e1 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -280,24 +280,29 @@ allocate_nodes(NumNodes, Version) -> AllocatedNodeIds = lists:sublist(AvailableNodeIds, NumNodes), lager:debug("Allocated node ids ~p", [AllocatedNodeIds]), - AllocatedNodeMap = lists:foldl( - fun(NodeId, NodeMap) -> + [AllocatedNodeMap, NodeNames, _] = lists:foldl( + fun(NodeId, [AllocatedNodeMapAcc, NodeNamesAcc, Number]) -> NodeName = proplists:get_value(NodeId, AvailableNodeMap), - [{NodeId, NodeName}|NodeMap] + [[{NodeId, NodeName}|AllocatedNodeMapAcc], + orddict:append(NodeId, Number, NodeNamesAcc), + Number + 1] end, - [], + [[], orddict:new(), 1], AllocatedNodeIds), - NodeVersionMap = lists:foldl( - fun(NodeId, Acc) -> orddict:append(NodeId, Version, Acc) end, - orddict:new(), - AllocatedNodeIds), + lager:debug("AllocatedNodeMap: ~p", [AllocatedNodeMap]), + [Nodes, Versions] = lists:foldl( + fun({NodeId, NodeName}, [NodesAcc, VersionsAcc]) -> + [orddict:append(NodeName, NodeId, NodesAcc), + orddict:append(NodeName, Version, VersionsAcc)] + end, + [orddict:new(), orddict:new()], + AllocatedNodeMap), lager:debug("Allocated node map ~p", [AllocatedNodeMap]), - rt_config:set(rt_nodes, AllocatedNodeMap), - %% May be incorrect -- see rtdev:deploy_nodes ... -jsb - rt_config:set(rt_nodenames, AllocatedNodeMap), - rt_config:set(rt_versions, NodeVersionMap), + rt_config:set(rt_nodes, Nodes), + rt_config:set(rt_nodenames, NodeNames), + rt_config:set(rt_versions, Versions), lager:debug("Set rt_nodes: ~p", [ rt_config:get(rt_nodes) ]), lager:debug("Set rt_nodenames: ~p", [ rt_config:get(rt_nodenames) ]), diff --git a/src/rtdev.erl b/src/rtdev.erl index d0ce6917c..59689b976 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -83,7 +83,9 @@ gitcmd(Path, Cmd) -> [Path, Path, Cmd]). %% @doc Create a command-line command for riak-admin --spec riak_admin_cmd(Path :: string(), N :: string(), Args :: string()) -> string(). +-spec riak_admin_cmd(Path :: string(), N :: integer() | string(), Args :: string()) -> string(). +riak_admin_cmd(Path, N, Args) when is_integer(N) -> + riak_admin_cmd(Path, node_short_name_to_name(N), Args); riak_admin_cmd(Path, N, Args) -> Quoted = lists:map(fun(Arg) when is_list(Arg) -> @@ -93,7 +95,8 @@ riak_admin_cmd(Path, N, Args) -> end, Args), ArgStr = string:join(Quoted, " "), ExecName = rt_config:get(exec_name, "riak"), - io_lib:format("~s/~s/bin/~s-admin ~s", [Path, N, ExecName, ArgStr]). + {NodeId, _} = extract_node_id_and_name(N), + io_lib:format("~s/~s/bin/~s-admin ~s", [Path, NodeId, ExecName, ArgStr]). run_git(Path, Cmd) -> lager:info("Running: ~s", [gitcmd(Path, Cmd)]), @@ -104,15 +107,16 @@ run_git(Path, Cmd) -> -spec run_riak(Node :: string(), Version :: string(), string()) -> string(). run_riak(Node, Version, "start") -> VersionPath = filename:join(?PATH, Version), - RiakCmd = riakcmd(VersionPath, Node, "start"), + {NodeId, NodeName} = extract_node_id_and_name(Node), + RiakCmd = riakcmd(VersionPath, NodeId, "start"), lager:info("Running: ~s", [RiakCmd]), CmdRes = os:cmd(RiakCmd), %% rt_cover:maybe_start_on_node(?DEV(Node), Version), %% Intercepts may load code on top of the cover compiled %% modules. We'll just get no coverage info then. - case rt_intercept:are_intercepts_loaded(?DEV(Node)) of + case rt_intercept:are_intercepts_loaded(NodeName) of false -> - ok = rt_intercept:load_intercepts([?DEV(Node)]); + ok = rt_intercept:load_intercepts([NodeName]); true -> ok end, @@ -195,9 +199,7 @@ so_fresh_so_clean(VersionMap) -> available_resources() -> VersionMap = [{Version, harness_node_ids(Version)} || Version <- versions()], NodeIds = harness_node_ids(rt_config:get(default_version, "head")), - lager:debug("Available Node IDs: ~p", [NodeIds]), NodeMap = lists:zip(NodeIds, harness_nodes(NodeIds)), - lager:debug("Available Node Map: ~p", [NodeMap]), [NodeIds, NodeMap, VersionMap]. @@ -699,8 +701,7 @@ stop(Node, Version) -> end. start(Node, Version) -> - {NodeId, _} = extract_node_id_and_name(Node), - run_riak(NodeId, Version, "start"), + run_riak(Node, Version, "start"), ok. attach(Node, Expected) -> @@ -787,9 +788,8 @@ interactive_loop(Port, Expected) -> end. admin(Node, Args, Options) -> - {NodeId, _} = extract_node_id_and_name(Node), - Path = relpath(node_version(NodeId)), - Cmd = riak_admin_cmd(Path, NodeId, Args), + Path = relpath(node_version(Node)), + Cmd = riak_admin_cmd(Path, Node, Args), lager:info("Running: ~s", [Cmd]), Result = execute_admin_cmd(Cmd, Options), lager:info("~p", [Result]), @@ -832,7 +832,9 @@ node_short_name(Node) when is_atom(Node) -> orddict:fetch(Node, NodeMap). %% @doc Return the node version from rt_versions based on full node name --spec node_version(atom()) -> string(). +-spec node_version(atom() | integer()) -> string(). +node_version(Node) when is_integer(Node) -> + node_version(node_short_name_to_name(Node)); node_version(Node) -> VersionMap = rt_config:get(rt_versions), orddict:fetch(Node, VersionMap). @@ -966,6 +968,10 @@ maybe_contains(Pos) when Pos > 0 -> maybe_contains(_) -> false. +-spec node_short_name_to_name(integer()) -> atom(). +node_short_name_to_name(N) -> + ?DEV("dev" ++ integer_to_list(N)). + -ifdef(TEST). extract_node_id_and_name_test() -> @@ -984,4 +990,7 @@ contains_test() -> ?assertEqual(true, contains("dev2@127.0.0.1", $@)), ?assertEqual(false, contains("dev2", $@)). +node_short_name_to_name_test() -> + ?assertEqual('dev1@127.0.0.1', node_short_name_to_name(1)). + -endif. diff --git a/tests/verify_riak_stats.erl b/tests/verify_riak_stats.erl index bbb72639d..e7f5c1727 100644 --- a/tests/verify_riak_stats.erl +++ b/tests/verify_riak_stats.erl @@ -35,10 +35,10 @@ %% You should have curl installed locally to do this. confirm() -> - Nodes = rt:deploy_nodes(1), + Nodes = rt:deploy_nodes(1, ?CONF), [Node1] = Nodes, verify_dt_converge:create_bucket_types(Nodes, ?TYPES), - ?assertEqual(ok, rt_node:wait_until_nodes_ready([Node1])), + ?assertEqual(ok, rt:wait_until_nodes_ready([Node1])), Stats1 = get_stats(Node1), lager:info("Verifying that all expected stats keys are present from the HTTP endpoint"), @@ -69,13 +69,14 @@ confirm() -> <<"memory_code">>, <<"memory_ets">>]), + lager:info("perform 5 x PUT and a GET to increment the stats"), lager:info("as the stat system only does calcs for > 5 readings"), - - C = rt_http:httpc(Node1), + + C = rt:httpc(Node1), [rt:httpc_write(C, <<"systest">>, <>, <<"12345">>) || X <- lists:seq(1, 5)], [rt:httpc_read(C, <<"systest">>, <>) || X <- lists:seq(1, 5)], - + Stats2 = get_stats(Node1), %% make sure the stats that were supposed to increment did @@ -103,6 +104,7 @@ confirm() -> lager:info("Make PBC Connection"), Pid = rt:pbc(Node1), + Stats3 = get_stats(Node1), rt:systest_write(Node1, 1), @@ -167,32 +169,32 @@ get_console_stats(Node) -> %% Temporary workaround: use os:cmd/1 when in 'rtdev' (needs some cheats %% in order to find the right path etc.) try - Stats = - case rt_config:get(rt_harness) of - rtdev -> - N = rtdev:node_id(Node), - Path = rtdev:relpath(rtdev:node_version(N)), - Cmd = rtdev:riak_admin_cmd(Path, N, ["status"]), - lager:info("Cmd = ~p~n", [Cmd]), - os:cmd(Cmd); - _ -> - rt:admin(Node, "status") - end, - [S || {_,_} = S <- - [list_to_tuple(re:split(L, " : ", [])) - || L <- tl(tl(string:tokens(Stats, "\n")))]] + Stats = + case rt_config:get(rt_harness) of + rtdev -> + N = rtdev:node_id(Node), + Path = rtdev:relpath(rtdev:node_version(N)), + Cmd = rtdev:riak_admin_cmd(Path, N, ["status"]), + lager:info("Cmd = ~p~n", [Cmd]), + os:cmd(Cmd); + _ -> + rt:admin(Node, "status") + end, + [S || {_,_} = S <- + [list_to_tuple(re:split(L, " : ", [])) + || L <- tl(tl(string:tokens(Stats, "\n")))]] catch - error:Reason -> - lager:info("riak-admin status ERROR: ~p~n~p~n", - [Reason, erlang:get_stacktrace()]), - [] + error:Reason -> + lager:info("riak-admin status ERROR: ~p~n~p~n", + [Reason, erlang:get_stacktrace()]), + [] end. compare_http_and_console_stats(Stats1, Stats2) -> OnlyInHttp = [S || {K,_} = S <- Stats1, - not lists:keymember(K, 1, Stats2)], + not lists:keymember(K, 1, Stats2)], OnlyInAdmin = [S || {K,_} = S <- Stats2, - not lists:keymember(K, 1, Stats1)], + not lists:keymember(K, 1, Stats1)], maybe_log_stats_keys(OnlyInHttp, "Keys missing from riak-admin"), maybe_log_stats_keys(OnlyInAdmin, "Keys missing from HTTP"), ?assertEqual([], OnlyInHttp), @@ -205,7 +207,6 @@ verify_stats_keys_complete(Node, Stats) -> MissingStatsKeys = diff_lists(ActualKeys, ExpectedKeys), AdditionalStatsKeys = diff_lists(ExpectedKeys, ActualKeys), maybe_log_stats_keys(MissingStatsKeys, "missing stats keys"), - lager:debug("Additional stats ~p", [AdditionalStatsKeys]), maybe_log_stats_keys(AdditionalStatsKeys, "additional stats"), ?assertEqual({[],[]}, {MissingStatsKeys, AdditionalStatsKeys}), ok. From 7bc3d0cbd23358d3758b34ec6c37dbc43a78f925 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Wed, 4 Mar 2015 03:37:35 +0000 Subject: [PATCH 078/157] Rename rt_max_wait_time to rt_max_receive_wait_time and document config settings --- INSTALL | 5 +-- README.md | 63 +++++++++++++++++++++++++++++++--- doc/overview.edoc | 5 ++- examples/riak_test.config.perf | 9 +++-- src/riak_test.app.src | 3 +- src/riak_test_runner.erl | 2 +- src/rt.erl | 4 +-- src/rt2.erl | 10 +++--- src/rt_bucket_types.erl | 4 +-- src/rt_config.erl | 8 ++++- src/rt_cs_dev.erl | 2 +- src/rt_harness_util.erl | 2 +- src/rt_local.erl | 2 +- src/rtdev.erl | 2 +- 14 files changed, 93 insertions(+), 28 deletions(-) diff --git a/INSTALL b/INSTALL index 142b5e950..b985bda49 100644 --- a/INSTALL +++ b/INSTALL @@ -25,7 +25,8 @@ {rt_deps, ["/usr/local/basho/evan/riak2.0/deps"]}, %% should be really long to allow full bitcasks to %% come up - {rt_max_wait_time, 600000000000000}, + {rt_max_receive_wait_time, 600000000000000}, + {test_timeout, 600000000000000}, {basho_bench, "/usr/local/basho/evan/basho_bench/"}, {basho_bench_statedir, "/tmp/bb_seqstate/"}, {rt_retry_delay, 500}, @@ -39,6 +40,6 @@ at least right now. this will problably change as the configuration stuff gets sorted out. some of these may not be necessary. - rt_max_wait_time is (could maybe be set to infinity? maybe by the + rt_max_receive_wait_time is (could maybe be set to infinity? maybe by the harness?), perf_* and basho_bench* are also critical. rt_deps should maybe be dynamic? diff --git a/README.md b/README.md index f5d46c606..3ea879b4e 100644 --- a/README.md +++ b/README.md @@ -175,7 +175,8 @@ to tell riak_test about them. The method of choice is to create a {giddyup_host, "localhost:5000"}, {giddyup_user, "user"}, {giddyup_password, "password"}, - {rt_max_wait_time, 600000}, + {test_timeout, 1800000}, + {rt_max_receive_wait_time, 600000}, {rt_retry_delay, 1000}, {rt_harness, rtdev}, {rt_scratch_dir, "/tmp/riak_test_scratch"}, @@ -193,7 +194,7 @@ to tell riak_test about them. The method of choice is to create a The `default` section of the config file will be overridden by the config name you specify. For example, running the command below will use an -`rt_retry_delay` of 500 and an `rt_max_wait_time` of 180000. If your +`rt_retry_delay` of 500 and an `rt_max_receive_wait_time` of 180000. If your defaults contain every option you need, you can run riak_test without the `-c` argument. @@ -202,8 +203,62 @@ the `-c` argument. your config file before running riak_test. Some configuration parameters: - -#### rt_default_config + +#### basho_bench +Path to local installation of `basho_bench`; used by performance testing. + +#### giddyup_host +This is a hostname and port number for communicating with Basho's internal build +reporting tool, Giddyup. + +#### giddyup_password +String used as a password when communicating with Giddyup. + +#### giddyup_user +String used to identify the user when communicating with Giddyup. + +#### platform +String identifying the current testing platform when reporting to Giddyup. +Current values include `centos-5-64`, `centos-6-64`, `fedora-17-64`, +`freebsd-9-64`, `osx-64`, `solaris-10u9-64`, `ubuntu-1004-64`, `ubuntu-1204-64` + +#### spam_dir +Name of a `tar` file containing ancientSPAM e-mail used as a data load +for a few tests. + +#### rt_harness +Which testing harness should be used. Current valid values include +- `rtdev` - Local devrel +- `rtssh` - Remote host +- `rtperf` - Performance testing + +#### rt_max_receive_wait_time +Number of milliseconds allowed for a `receive` operation to complete. + +#### rt_scratch_dir +Path to scratch directory used by `riak_test`. It's used for downloading +external files and for cleaning out the `data` directory between runs. + +#### rt_retry_delay +Number of milliseconds between attempts to send a message. + +#### test_timeout +Number of milliseconds allowed for a single test to run. + +#### rtdev +This is the devrel configuration section. + +##### rt_project +The name of the current project name; used in reporting to Giddyup. + +##### root_path +Path to the top of the installed devrel instances. + +##### default_version +If specific versions of Riak are not specified, this one is tested. +The default value is `head` which is typically the head of `develop`. + +##### rt_default_config Default configuration parameters that will be used for nodes deployed by riak_test. Tests can override these. diff --git a/doc/overview.edoc b/doc/overview.edoc index f40863b1b..f0d4b9796 100644 --- a/doc/overview.edoc +++ b/doc/overview.edoc @@ -145,7 +145,10 @@ Erlang term based config files. The references `rtdev.config' file above: {rt_deps, ["/Users/jtuple/basho/riak/deps"]}. %% Maximum time in milliseconds for wait_until to wait -{rt_max_wait_time, 180000}. +{rt_max_receive_wait_time, 180000}. + +%% Maximum time in milliseconds allowed for a single test +{test_timeout, 1800000}. %% Delay between each retry in wait_until {rt_retry_delay, 500}. diff --git a/examples/riak_test.config.perf b/examples/riak_test.config.perf index f3b75f64f..e24cdd526 100644 --- a/examples/riak_test.config.perf +++ b/examples/riak_test.config.perf @@ -2,7 +2,8 @@ {rt_deps, ["/mnt/riak_ee/deps"]}, %% should be really long to allow full bitcasks to %% come up - {rt_max_wait_time, 600000000000000}, + {rt_max_receive_wait_time, 600000000000000}, + {test_timeout, 600000000000000}, {basho_bench, "/mnt/basho_bench"}, {basho_bench_escript, "/usr/local/erlang-r16b02/bin/escript"}, {basho_bench_statedir, "/tmp/bb_seqstate/"}, @@ -11,8 +12,6 @@ {load_intercepts, false}, {perf_builds, "/mnt/perf_builds"}, {perf_loadgens, ["bench101.aws"]}, - {rtdev_path, [{root, "/mnt/rt/riak_ee"}, - {current, "/mnt/rt/riak_ee/riak-ee-2.0.0rc1"}, - {previous, "/mnt/rt/riak_ee/riak-ee-1.4.8"}, - {legacy, "/mnt/rt/riak_ee/riak-ee-1.3.4"}]} + {root_path, "/mnt/rt/riak"}, + {default_version, head} ]}. diff --git a/src/riak_test.app.src b/src/riak_test.app.src index 86c5cc8d3..d1cb3bcc2 100644 --- a/src/riak_test.app.src +++ b/src/riak_test.app.src @@ -11,7 +11,8 @@ {env, [ {platform, undefined}, {rt_scratch_dir, "/tmp/riak_test_scratch"}, - {rt_max_wait_time, 180000}, + {rt_max_receive_wait_time, 600000}, + {test_timeout, 1800000}, {rt_retry_delay, 500}, {rt_harness, rtdev}, {java, [{fat_be_url, "http://riak-java-client.s3.amazonaws.com/riak-client-1.4.2-jar-with-dependencies-and-tests.jar"}, diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index b007a4472..f9e5ac07f 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -101,7 +101,7 @@ init([TestModule, Backend, Properties]) -> {project, Project}], {ok, UpdProperties} = rt_properties:set(metadata, MetaData, Properties), - TestTimeout = rt_config:get(test_timeout, rt_config:get(rt_max_wait_time)), + TestTimeout = rt_config:get(test_timeout, rt_config:get(rt_max_receive_wait_time)), SetupModFun = function_name(setup, TestModule, 1, rt_cluster), {ConfirmMod, _} = ConfirmModFun = function_name(confirm, TestModule), lager:debug("Confirm function -- ~p:~p", [ConfirmMod, ConfirmModFun]), diff --git a/src/rt.erl b/src/rt.erl index d15b7a3e1..383de3832 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -483,7 +483,7 @@ stream_cmd_loop(Port, Buffer, NewLineBuffer, Time={_MegaSecs, Secs, _MicroSecs}) {Port, {exit_status, Status}} -> catch port_close(Port), {Status, Buffer} - after rt:config(rt_max_wait_time) -> + after rt:config(rt_max_receive_wait_time) -> {-1, Buffer} end. @@ -511,7 +511,7 @@ is_ready(Node) -> %% @doc Utility function used to construct test predicates. Retries the %% function `Fun' until it returns `true', or until the maximum %% number of retries is reached. The retry limit is based on the -%% provided `rt_max_wait_time' and `rt_retry_delay' parameters in +%% provided `rt_max_receive_wait_time' and `rt_retry_delay' parameters in %% specified `riak_test' config file. wait_until(Fun) when is_function(Fun) -> rt2:wait_until(Fun). diff --git a/src/rt2.erl b/src/rt2.erl index 48c021268..0014133dd 100644 --- a/src/rt2.erl +++ b/src/rt2.erl @@ -221,7 +221,7 @@ stream_cmd_loop(Port, Buffer, NewLineBuffer, Time={_MegaSecs, Secs, _MicroSecs}) {Port, {exit_status, Status}} -> catch port_close(Port), {Status, Buffer} - after rt_config:get(rt_max_wait_time) -> + after rt_config:get(rt_max_receive_wait_time) -> {-1, Buffer} end. @@ -246,7 +246,7 @@ load_modules_on_nodes([Module | MoreModules], Nodes) when is_list(Nodes) -> is_mixed_cluster(Nodes) when is_list(Nodes) -> %% If the nodes are bad, we don't care what version they are - {Versions, _BadNodes} = rpc:multicall(Nodes, init, script_id, [], rt_config:get(rt_max_wait_time)), + {Versions, _BadNodes} = rpc:multicall(Nodes, init, script_id, [], rt_config:get(rt_max_receive_wait_time)), length(lists:usort(Versions)) > 1; is_mixed_cluster(Node) -> Nodes = rpc:call(Node, erlang, nodes, []), @@ -280,10 +280,10 @@ product(Node) -> %% @doc Utility function used to construct test predicates. Retries the %% function `Fun' until it returns `true', or until the maximum %% number of retries is reached. The retry limit is based on the -%% provided `rt_max_wait_time' and `rt_retry_delay' parameters in +%% provided `rt_max_receive_wait_time' and `rt_retry_delay' parameters in %% specified `riak_test' config file. wait_until(Fun) when is_function(Fun) -> - MaxTime = rt_config:get(rt_max_wait_time), + MaxTime = rt_config:get(rt_max_receive_wait_time), Delay = rt_config:get(rt_retry_delay), Retry = MaxTime div Delay, wait_until(Fun, Retry, Delay). @@ -448,7 +448,7 @@ wait_until_unpingable(Node) -> %% riak stop will kill -9 after 5 mins, so we try to wait at least that %% amount of time. Delay = rt_config:get(rt_retry_delay), - Retry = lists:max([360000, rt_config:get(rt_max_wait_time)]) div Delay, + Retry = lists:max([360000, rt_config:get(rt_max_receive_wait_time)]) div Delay, lager:info("Wait until ~p is not pingable for ~p seconds with a retry of ~p", [Node, Delay, Retry]), case wait_until(F, Retry, Delay) of diff --git a/src/rt_bucket_types.erl b/src/rt_bucket_types.erl index b417f80f6..3a7466281 100644 --- a/src/rt_bucket_types.erl +++ b/src/rt_bucket_types.erl @@ -53,7 +53,7 @@ wait_until_bucket_type_status(Type, ExpectedStatus, Node) -> -spec bucket_type_visible([atom()], binary()|{binary(), binary()}) -> boolean(). bucket_type_visible(Nodes, Type) -> - MaxTime = rt_config:get(rt_max_wait_time), + MaxTime = rt_config:get(rt_max_receive_wait_time), IsVisible = fun erlang:is_list/1, {Res, NodesDown} = rpc:multicall(Nodes, riak_core_bucket_type, get, [Type], MaxTime), NodesDown == [] andalso lists:all(IsVisible, Res). @@ -65,7 +65,7 @@ wait_until_bucket_type_visible(Nodes, Type) -> -spec see_bucket_props([atom()], binary()|{binary(), binary()}, proplists:proplist()) -> boolean(). see_bucket_props(Nodes, Bucket, ExpectProps) -> - MaxTime = rt_config:get(rt_max_wait_time), + MaxTime = rt_config:get(rt_max_receive_wait_time), IsBad = fun({badrpc, _}) -> true; ({error, _}) -> true; (Res) when is_list(Res) -> false diff --git a/src/rt_config.erl b/src/rt_config.erl index 39ef3788e..df48faf44 100644 --- a/src/rt_config.erl +++ b/src/rt_config.erl @@ -88,7 +88,13 @@ get(Key) -> lager:warning("Missing configuration key: ~p", [Key]), erlang:error("Missing configuration key", [Key]); Value -> - Value + case Key of + rt_max_wait_time -> + lager:info("Obsolete key rt_max_wait_time used. Please use rt_max_receive_wait_time instead."), + Value; + _ -> + Value + end end. get(Key, Default) -> diff --git a/src/rt_cs_dev.erl b/src/rt_cs_dev.erl index f28bfedd6..04071620b 100644 --- a/src/rt_cs_dev.erl +++ b/src/rt_cs_dev.erl @@ -403,7 +403,7 @@ interactive_loop(Port, Expected) -> %% We've met every expectation. Yay! If not, it means we've exited before %% something expected happened. ?assertEqual([], Expected) - after rt_config:get(rt_max_wait_time) -> + after rt_config:get(rt_max_receive_wait_time) -> %% interactive_loop is going to wait until it matches expected behavior %% If it doesn't, the test should fail; however, without a timeout it %% will just hang forever in search of expected behavior. See also: Parenting diff --git a/src/rt_harness_util.erl b/src/rt_harness_util.erl index 03a9d04b8..e86c8aaa7 100644 --- a/src/rt_harness_util.erl +++ b/src/rt_harness_util.erl @@ -204,7 +204,7 @@ interactive_loop(Port, Expected) -> %% We've met every expectation. Yay! If not, it means we've exited before %% something expected happened. ?assertEqual([], Expected) - after rt_config:get(rt_max_wait_time) -> + after rt_config:get(rt_max_receive_wait_time) -> %% interactive_loop is going to wait until it matches expected behavior %% If it doesn't, the test should fail; however, without a timeout it %% will just hang forever in search of expected behavior. See also: Parenting diff --git a/src/rt_local.erl b/src/rt_local.erl index fd4517e17..f2db1022f 100644 --- a/src/rt_local.erl +++ b/src/rt_local.erl @@ -115,6 +115,6 @@ stream_cmd_loop(Port, Buffer, NewLineBuffer, Time={_MegaSecs, Secs, _MicroSecs}) {Port, {exit_status, Status}} -> catch port_close(Port), {Status, Buffer} - after rt_config:get(rt_max_wait_time) -> + after rt_config:get(rt_max_receive_wait_time) -> {-1, Buffer} end. diff --git a/src/rtdev.erl b/src/rtdev.erl index 59689b976..bbec93135 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -780,7 +780,7 @@ interactive_loop(Port, Expected) -> %% We've met every expectation. Yay! If not, it means we've exited before %% something expected happened. ?assertEqual([], Expected) - after rt_config:get(rt_max_wait_time) -> + after rt_config:get(rt_max_receive_wait_time) -> %% interactive_loop is going to wait until it matches expected behavior %% If it doesn't, the test should fail; however, without a timeout it %% will just hang forever in search of expected behavior. See also: Parenting From b4689eb4d1bffefea9b1aa313954c395450b91dc Mon Sep 17 00:00:00 2001 From: John Burwell Date: Fri, 27 Feb 2015 16:32:55 -0500 Subject: [PATCH 079/157] - Removes riak from the repo -- it should be ignored --- riak | 1 - 1 file changed, 1 deletion(-) delete mode 120000 riak diff --git a/riak b/riak deleted file mode 120000 index a5208ef76..000000000 --- a/riak +++ /dev/null @@ -1 +0,0 @@ -riak-2.0 \ No newline at end of file From fe1c7bf7a3e0d4aba557320c26742de794aeb6c3 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Wed, 4 Mar 2015 11:54:13 -0500 Subject: [PATCH 080/157] Refines deprecation logging for the rt_max_wait_time parameter and adds unit tests --- src/rt_config.erl | 51 +++++++++++++++++++++++++++++++++++++---------- 1 file changed, 41 insertions(+), 10 deletions(-) diff --git a/src/rt_config.erl b/src/rt_config.erl index df48faf44..78da37b2e 100644 --- a/src/rt_config.erl +++ b/src/rt_config.erl @@ -35,6 +35,8 @@ ]). -define(HARNESS, (rt_config:get(rt_harness))). +-define(CONFIG_NAMESPACE, riak_test). +-define(RECEIVE_WAIT_TIME_KEY, rt_max_recieve_wait_time). %% @doc Get the value of an OS Environment variable. The arity 1 version of %% this function will fail the test if it is undefined. @@ -82,23 +84,25 @@ load_dot_config(ConfigName, ConfigFile) -> set(Key, Value) -> ok = application:set_env(riak_test, Key, Value). + +-spec get(rt_max_wait_time | atom()) -> any(). +get(rt_max_wait_time) -> + lager:info("rt_max_wait_time is deprecated. Please use rt_max_receive_wait_time instead."), + rt_config:get(?RECEIVE_WAIT_TIME_KEY); get(Key) -> - case kvc:path(Key, application:get_all_env(riak_test)) of + case kvc:path(Key, application:get_all_env(?CONFIG_NAMESPACE)) of [] -> lager:warning("Missing configuration key: ~p", [Key]), erlang:error("Missing configuration key", [Key]); - Value -> - case Key of - rt_max_wait_time -> - lager:info("Obsolete key rt_max_wait_time used. Please use rt_max_receive_wait_time instead."), - Value; - _ -> - Value - end + Value -> Value end. +-spec get(rt_max_wait_time | atom(), any()) -> any(). +get(rt_max_wait_time, Default) -> + lager:info("rt_max_wait_time is deprecated. Please use rt_max_receive_wait_time instead."), + get(?RECEIVE_WAIT_TIME_KEY, Default); get(Key, Default) -> - case kvc:path(Key, application:get_all_env(riak_test)) of + case kvc:path(Key, application:get_all_env(?CONFIG_NAMESPACE)) of [] -> Default; Value -> Value end. @@ -165,3 +169,30 @@ update_app_config(Node, Config) -> to_upper(S) -> lists:map(fun char_to_upper/1, S). char_to_upper(C) when C >= $a, C =< $z -> C bxor $\s; char_to_upper(C) -> C. + +-ifdef(TEST). + +clear(Key) -> + application:unset_env(?CONFIG_NAMESPACE, Key). + +get_rt_max_wait_time_test() -> + clear(?RECEIVE_WAIT_TIME_KEY), + + ExpectedWaitTime = 10987, + ok = set(?RECEIVE_WAIT_TIME_KEY, ExpectedWaitTime), + ?assertEqual(ExpectedWaitTime, rt_config:get(?RECEIVE_WAIT_TIME_KEY)), + ?assertEqual(ExpectedWaitTime, rt_config:get(rt_max_wait_time)). + +get_rt_max_wait_time_default_test() -> + clear(?RECEIVE_WAIT_TIME_KEY), + + DefaultWaitTime = 20564, + ?assertEqual(DefaultWaitTime, get(?RECEIVE_WAIT_TIME_KEY, DefaultWaitTime)), + ?assertEqual(DefaultWaitTime, get(rt_max_wait_time, DefaultWaitTime)), + + ExpectedWaitTime = 30421, + ok = set(?RECEIVE_WAIT_TIME_KEY, ExpectedWaitTime), + ?assertEqual(ExpectedWaitTime, get(?RECEIVE_WAIT_TIME_KEY, DefaultWaitTime)), + ?assertEqual(ExpectedWaitTime, get(rt_max_wait_time, DefaultWaitTime)). + +-endif. From 7da4084cf20d65dd503f3180e395b2a3b75ab8be Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Thu, 5 Mar 2015 00:09:58 +0000 Subject: [PATCH 081/157] Add back support for riak_test_runner:metadata/0 for unported tests --- src/riak_test_runner.erl | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index f9e5ac07f..51a81f7a2 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -44,6 +44,7 @@ handle_event/3, handle_sync_event/4, handle_info/3, + metadata/0, terminate/3, code_change/4]). @@ -85,6 +86,13 @@ send_event(Pid, Msg) -> stop() -> gen_fsm:sync_send_all_state_event(?MODULE, stop, infinity). +-spec(metadata() -> [{atom(), term()}]). +%% @doc fetches test metadata from spawned test process +%% TODO: Remove when test ports are over +metadata() -> + FSMPid = get(test_runner_fsm), + gen_fsm:sync_send_all_state_event(FSMPid, metadata_event, infinity). + %%%=================================================================== %%% gen_fsm callbacks %%%=================================================================== @@ -132,6 +140,10 @@ handle_event(_Event, StateName, State) -> %% the same regardless of the current state. -spec handle_sync_event(term(), term(), atom(), #state{}) -> {reply, term(), atom(), #state{}}. +handle_sync_event(metadata_event, _From, _StateName, State) -> + Properties = State#state.properties, + MetaData = rt_properties:get(metadata, Properties), + {reply, MetaData, ok, State}; handle_sync_event(_Event, _From, _StateName, _State) -> {reply, ok, ok, _State}. @@ -324,7 +336,9 @@ test_fun(old, _Properties, ConfirmMod, ConfirmFun, NotifyPid) -> -spec test_fun(function(), pid()) -> function(). test_fun(ConfirmFun, NotifyPid) -> fun() -> - %% Exceptions and their handling sucks, but eunit throws + %% Store the FSM Pid for use in unported tests + put(test_runner_fsm, NotifyPid), + %% Exceptions and their handling sucks, but eunit throws %% errors `erlang:error' so here we are try ConfirmFun() of TestResult -> From 929f53a78610f5f7d15ca102e969a188711d884d Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Thu, 5 Mar 2015 14:11:07 -0700 Subject: [PATCH 082/157] Fixed typo in rt_max_receive_wait_time --- src/rt_config.erl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/rt_config.erl b/src/rt_config.erl index 78da37b2e..3c9e76767 100644 --- a/src/rt_config.erl +++ b/src/rt_config.erl @@ -36,7 +36,7 @@ -define(HARNESS, (rt_config:get(rt_harness))). -define(CONFIG_NAMESPACE, riak_test). --define(RECEIVE_WAIT_TIME_KEY, rt_max_recieve_wait_time). +-define(RECEIVE_WAIT_TIME_KEY, rt_max_receive_wait_time). %% @doc Get the value of an OS Environment variable. The arity 1 version of %% this function will fail the test if it is undefined. From 9be62bfbe2491e2f2ca73d739c8229601215290e Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Thu, 5 Mar 2015 14:48:15 -0700 Subject: [PATCH 083/157] Fix result() returned by handle_sync_event to remain in current state instead of moving to state "ok" --- src/riak_test_runner.erl | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index 51a81f7a2..90ac32d2f 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -140,12 +140,12 @@ handle_event(_Event, StateName, State) -> %% the same regardless of the current state. -spec handle_sync_event(term(), term(), atom(), #state{}) -> {reply, term(), atom(), #state{}}. -handle_sync_event(metadata_event, _From, _StateName, State) -> +handle_sync_event(metadata_event, _From, StateName, State) -> Properties = State#state.properties, MetaData = rt_properties:get(metadata, Properties), - {reply, MetaData, ok, State}; -handle_sync_event(_Event, _From, _StateName, _State) -> - {reply, ok, ok, _State}. + {reply, MetaData, StateName, State}; +handle_sync_event(_Event, _From, StateName, _State) -> + {reply, ok, StateName, _State}. handle_info(_Info, StateName, State) -> {next_state, StateName, State}. From 765283e540c35a0dff70fa94b1e751df204f7f2c Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Thu, 5 Mar 2015 15:57:06 -0700 Subject: [PATCH 084/157] Add test_summary_format_time for human-readable Duration output --- src/riak_test_escript.erl | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index bfe125b1b..586e414ed 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -479,6 +479,16 @@ parse_webhook(Props) -> undefined end. +test_summary_format_time(Milliseconds) -> + Mills = trunc(((Milliseconds / 1000000) - (Milliseconds div 1000000)) * 1000000), + TotalSecs = (Milliseconds - Mills) div 1000000, + TotalMins = TotalSecs div 60, + Hours = TotalSecs div 3600, + Secs = TotalSecs - (TotalMins * 60), + Mins = TotalMins - (Hours * 60), + list_to_binary(io_lib:format("~ph ~pm ~p.~ps", [Hours, Mins, Secs, Mills])). + + test_summary_fun({Test, pass, _}, {{Pass, _Fail, _Skipped}, Width}) -> TestNameLength = length(atom_to_list(Test)), UpdWidth = @@ -514,9 +524,9 @@ format_test_row({Test, Result, Duration}, _Width) -> TestString = atom_to_list(Test), case Result of {Status, Reason} -> - [TestString, Status, Reason, Duration]; + [TestString, Status, Reason, test_summary_format_time(Duration)]; pass -> - [TestString, "pass", "N/A", Duration] + [TestString, "pass", "N/A", test_summary_format_time(Duration)] end. print_summary(TestResults, _CoverResult, Verbose) -> From 9e0f2f76129e9597acba3d94ec9bb066bdff7654 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Fri, 6 Mar 2015 11:25:00 -0700 Subject: [PATCH 085/157] Fixed typos in rt_bucket_type function names --- src/rt.erl | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/rt.erl b/src/rt.erl index 383de3832..7c1962771 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -1406,13 +1406,13 @@ post_result(TestResult, Webhook) -> %% @doc create and immediately activate a bucket type create_and_activate_bucket_type(Node, Type, Props) -> - rt_bucket_type:create_and_activate_bucket_type(Node, Type, Props). + rt_bucket_types:create_and_activate_bucket_type(Node, Type, Props). wait_until_bucket_type_status(Type, ExpectedStatus, Nodes) -> - rt_bucket_type:wait_until_bucket_type_status(Type, ExpectedStatus, Nodes). + rt_bucket_types:wait_until_bucket_type_status(Type, ExpectedStatus, Nodes). wait_until_bucket_type_visible(Nodes, Type) -> - rt_bucket_types:wait_until_bucket_type_visble(Nodes, Type). + rt_bucket_types:wait_until_bucket_type_visible(Nodes, Type). wait_until_bucket_props(Nodes, Bucket, Props) -> rt_bucket_types:wait_until_bucket_props(Nodes, Bucket, Props). From f3402cd7a112c60e196c744f2ef56b147826cd95 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Fri, 6 Mar 2015 11:45:28 -0700 Subject: [PATCH 086/157] Update the proper settings of `spam_dir` --- README.md | 2 +- examples/riak_test.config | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 3ea879b4e..bd2cb6ade 100644 --- a/README.md +++ b/README.md @@ -181,7 +181,7 @@ to tell riak_test about them. The method of choice is to create a {rt_harness, rtdev}, {rt_scratch_dir, "/tmp/riak_test_scratch"}, {basho_bench, "/home/you/basho/basho_bench"}, - {spam_dir, "/home/you/basho/riak_test/search-corpus/spam.0"}, + {spam_dir, "/home/you/basho/riak_test/search-corpus"}, {platform, "osx-64"} ]}. diff --git a/examples/riak_test.config b/examples/riak_test.config index d5d950894..a1c6e9144 100644 --- a/examples/riak_test.config +++ b/examples/riak_test.config @@ -58,7 +58,7 @@ %% The path to a corpus of spam emails to be used when testing %% Riak Search. This is typically expanded from the tarball %% included in riak_test. - {spam_dir, "/Users/dparfitt/riak_test/search-corpus/spam.0"}, + {spam_dir, "/Users/dparfitt/riak_test/search-corpus"}, %% The number of workers-per-node to spawn when executing the %% `loaded_upgrade' test. If unspecified, this will default to From 138e21d5e6a43e49a83f575abb31d704ba64c68d Mon Sep 17 00:00:00 2001 From: John Burwell Date: Thu, 5 Mar 2015 08:22:39 -0500 Subject: [PATCH 087/157] Implements an option (rt_execution_mode) to specify that r_t should run test serially rather than in parallel * Modifies riak_test_executor read the rt_execution_mode configuration parameter and execute tests serially * Modifies riak_test_runner to replicate state across state transitions * Adds the regression_test_wrapper.sh script to execute the all test cases expected to work with the v1 API wrapper --- regression_test_wrapper.sh | 13 +++++++ src/riak_test_executor.erl | 70 ++++++++++++++++++++++++++++---------- src/riak_test_runner.erl | 54 +++++++++++++++++++++++------ 3 files changed, 109 insertions(+), 28 deletions(-) create mode 100755 regression_test_wrapper.sh diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh new file mode 100755 index 000000000..2eff804ef --- /dev/null +++ b/regression_test_wrapper.sh @@ -0,0 +1,13 @@ +#!/bin/bash + +# Bail out on error ... +set -e + +if [ -z $1 ]; then + echo "An r_t configuration is required as the first parameter" + exit 1 +fi + +TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_membackend,verify_leave" + +./riak_test -c $1 -t $TEST_CASES diff --git a/src/riak_test_executor.erl b/src/riak_test_executor.erl index a612b8a95..b4989b8dc 100644 --- a/src/riak_test_executor.erl +++ b/src/riak_test_executor.erl @@ -23,6 +23,7 @@ terminate/3, code_change/4]). +-type execution_mode() :: serial | parallel. -record(state, {pending_tests :: [atom()], running_tests=[] :: [atom()], waiting_tests=[] :: [atom()], @@ -32,7 +33,8 @@ test_properties :: [proplists:proplist()], runner_pids=[] :: [pid()], log_dir :: string(), - report_info :: string()}). + report_info :: string(), + execution_mode :: execution_mode()}). %%%=================================================================== %%% API @@ -57,12 +59,16 @@ stop() -> %%%=================================================================== init([Tests, Backend, LogDir, ReportInfo, UpgradeList, NotifyPid]) -> + %% TODO Change the default when parallel execution support is implemented -jsb + ExecutionMode = rt_config:get(rt_execution_mode, serial), + lager:notice("Starting the Riak Test executor in ~p execution mode", [ExecutionMode]), State = #state{pending_tests=Tests, backend=Backend, log_dir=LogDir, report_info=ReportInfo, upgrade_list=UpgradeList, - notify_pid=NotifyPid}, + notify_pid=NotifyPid, + execution_mode=ExecutionMode}, {ok, gather_properties, State, 0}. %% @doc there are no all-state events for this fsm @@ -114,59 +120,70 @@ request_nodes({test_complete, Test, Pid, Results, Duration}, State) -> #state{pending_tests=Pending, waiting_tests=Waiting, running_tests=Running, - runner_pids=Pids} = State, + runner_pids=Pids, + execution_mode=ExecutionMode}= State, %% Report results report_results(Test, Results, Duration, State), UpdState = State#state{running_tests=lists:delete(Test, Running), runner_pids=lists:delete(Pid, Pids), pending_tests=Pending++Waiting, - waiting_tests=[]}, + waiting_tests=[], + execution_mode=ExecutionMode}, {next_state, request_nodes, UpdState}; request_nodes(_Event, _State) -> {next_state, request_nodes, _State}. launch_test(insufficient_versions_available, State) -> - #state{pending_tests=[HeadPending | RestPending]} = State, + #state{pending_tests=[HeadPending | RestPending], + execution_mode=ExecutionMode} = State, report_results(HeadPending, {skipped, insufficient_versions}, 0, State), - UpdState = State#state{pending_tests=RestPending}, + UpdState = State#state{pending_tests=RestPending, + execution_mode=ExecutionMode}, launch_test_transition(UpdState); launch_test(not_enough_nodes, State) -> %% Move head of pending to waiting and try next test if there is %% one left in pending. #state{pending_tests=[HeadPending | RestPending], - waiting_tests=Waiting} = State, + waiting_tests=Waiting, + execution_mode=ExecutionMode} = State, UpdState = State#state{pending_tests=RestPending, - waiting_tests=[HeadPending | Waiting]}, + waiting_tests=[HeadPending | Waiting], + execution_mode=ExecutionMode}, launch_test_transition(UpdState); launch_test({nodes, Nodes, NodeMap}, State) -> %% Spawn a test runner for the head of pending. If pending is now %% empty transition to `wait_for_completion'; otherwise, %% transition to `request_nodes'. #state{pending_tests=[NextTest | RestPending], + execution_mode=ExecutionMode, backend=Backend, test_properties=PropertiesList, runner_pids=Pids, running_tests=Running} = State, + lager:debug("Executing test ~p in mode ~p", [NextTest, ExecutionMode]), {NextTest, TestProps} = lists:keyfind(NextTest, 1, PropertiesList), UpdTestProps = rt_properties:set([{node_map, NodeMap}, {node_ids, Nodes}], TestProps), - lager:debug("Spawning test runner to execute ~p", [NextTest]), - Pid = spawn_link(riak_test_runner, start, [NextTest, Backend, UpdTestProps]), + {RunnerPids, RunningTests} = run_test(ExecutionMode, NextTest, Backend, UpdTestProps, Pids, Running), UpdState = State#state{pending_tests=RestPending, - runner_pids=[Pid | Pids], - running_tests=[NextTest | Running]}, + execution_mode=ExecutionMode, + runner_pids=RunnerPids, + running_tests=RunningTests}, + launch_test_transition(UpdState); launch_test({test_complete, Test, Pid, Results, Duration}, State) -> #state{pending_tests=Pending, waiting_tests=Waiting, running_tests=Running, - runner_pids=Pids} = State, + runner_pids=Pids, + execution_mode=ExecutionMode} = State, %% Report results report_results(Test, Results, Duration, State), UpdState = State#state{running_tests=lists:delete(Test, Running), runner_pids=lists:delete(Pid, Pids), pending_tests=Pending++Waiting, - waiting_tests=[]}, + waiting_tests=[], + execution_mode=ExecutionMode}, {next_state, launch_test, UpdState}; launch_test(Event, State) -> lager:error("Unknown event ~p with state ~p.", [Event, State]), @@ -191,16 +208,19 @@ maybe_reserve_nodes(false, NextTest, VersionsToTest, _TestProps) -> ok. wait_for_completion({test_complete, Test, Pid, Results, Duration}, State) -> + lager:debug("Test ~p complete", [Test]), #state{pending_tests=Pending, waiting_tests=Waiting, running_tests=Running, - runner_pids=Pids} = State, + runner_pids=Pids, + execution_mode=ExecutionMode} = State, %% Report results report_results(Test, Results, Duration, State), UpdState = State#state{running_tests=lists:delete(Test, Running), runner_pids=lists:delete(Pid, Pids), pending_tests=Pending++Waiting, - waiting_tests=[]}, + waiting_tests=[], + execution_mode=ExecutionMode}, wait_for_completion_transition(UpdState); wait_for_completion(_Event, _State) -> ok. @@ -225,7 +245,7 @@ wait_for_completion(_Event, _From, _State) -> report_results(Test, Results, Duration, #state{notify_pid=NotifyPid}) -> NotifyPid ! {self(), {test_result, {Test, Results, Duration}}}, - ok. + ok. report_done(#state{notify_pid=NotifyPid}) -> NotifyPid ! {self(), done}, @@ -239,11 +259,16 @@ wait_for_completion_transition(State=#state{pending_tests=[]}) -> wait_for_completion_transition(State) -> {next_state, request_nodes, State, 0}. -launch_test_transition(State=#state{pending_tests=[]}) -> +launch_test_transition(State=#state{pending_tests=PendingTests, + execution_mode=ExecutionMode}) when PendingTests == [] orelse ExecutionMode == serial -> + lager:debug("Waiting for complete: execution mode ~p with pending tests ~p", [ExecutionMode, PendingTests]), {next_state, wait_for_completion, State}; launch_test_transition(State) -> {next_state, request_nodes, State, 0}. +%%launch_test_transition(State) -> +%% {next_state, wait_for_completion, State}. + reservation_notify_fun() -> fun(X) -> ?MODULE:send_event(X) @@ -288,3 +313,12 @@ override_props(State) -> UpgradeList -> [{upgrade_path, UpgradeList}] end. + +-spec run_test(parallel | serial, atom(), atom(), proplists:proplist(), [pid()], [atom()]) -> {[pid()], [atom()]}. +run_test(parallel, Test, Backend, Properties, RunningPids, RunningTests) -> + Pid = spawn_link(riak_test_runner, start, [Test, Backend, Properties]), + {[Pid | RunningPids], [Test | RunningTests]}; +run_test(serial, Test, Backend, Properties, RunningPids, RunningTests) -> + riak_test_runner:start(Test, Backend, Properties), + {RunningPids, RunningTests}. + diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index 90ac32d2f..2260f05c3 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -160,12 +160,12 @@ code_change(_OldVsn, StateName, State, _Extra) -> %% Asynchronous call handling functions for each FSM state setup(timeout, State=#state{backend_check=false}) -> - notify_executor({skipped, invalid_backend}, State), cleanup(State), + notify_executor({skipped, invalid_backend}, State), {stop, normal, State}; setup(timeout, State=#state{prereq_check=false}) -> - notify_executor({fail, prereq_check_failed}, State), cleanup(State), + notify_executor({fail, prereq_check_failed}, State), {stop, normal, State}; setup(timeout, State=#state{test_type=TestType, test_module=TestModule, @@ -237,30 +237,45 @@ maybe_execute_test({ok, Properties}, _TestModule, TestType, ConfirmModFun, Start ConfirmModFun, self())), State#state{execution_pid=Pid, + test_type=TestType, properties=Properties, start_time=StartTime}; -maybe_execute_test(Error, TestModule, _TestType, _ConfirmModFun, StartTime, State) -> +maybe_execute_test(Error, TestModule, TestType, _ConfirmModFun, StartTime, State) -> lager:error("Setup of test ~p failed due to ~p", [TestModule, Error]), ?MODULE:send_event(self(), test_result({fail, test_setup_failed})), - State#state{start_time=StartTime}. + State#state{test_type=TestType, + start_time=StartTime}. -wait_for_completion(timeout, State) -> +wait_for_completion(timeout, State=#state{test_module=TestModule, + test_type=TestType, + group_leader=GroupLeader}) -> %% Test timed out - UpdState = State#state{end_time=os:timestamp()}, - notify_executor(timeout, UpdState), + UpdState = State#state{test_module=TestModule, + test_type=TestType, + group_leader=GroupLeader, + end_time=os:timestamp()}, cleanup(UpdState), + notify_executor(timeout, UpdState), {stop, normal, UpdState}; -wait_for_completion({test_result, Result}, State=#state{remaining_versions=[]}) -> +wait_for_completion({test_result, Result}, State=#state{test_module=TestModule, + test_type=TestType, + group_leader=GroupLeader, + remaining_versions=[]}) -> %% TODO: Format results for aggregate test runs if needed. For %% upgrade tests with failure return which versions had failure %% along with reasons. - UpdState = State#state{end_time=os:timestamp()}, - notify_executor(Result, UpdState), + UpdState = State#state{test_module=TestModule, + test_type=TestType, + group_leader=GroupLeader, + end_time=os:timestamp()}, cleanup(UpdState), + notify_executor(Result, UpdState), {stop, normal, UpdState}; wait_for_completion({test_result, Result}, State) -> #state{backend=Backend, + test_module=TestModule, + test_type=TestType, test_results=TestResults, current_version=CurrentVersion, remaining_versions=[NextVersion | RestVersions], @@ -273,6 +288,8 @@ wait_for_completion({test_result, Result}, State) -> Config, notify_fun(self())), UpdState = State#state{test_results=[Result | TestResults], + test_module=TestModule, + test_type=TestType, current_version=NextVersion, remaining_versions=RestVersions}, {next_state, wait_for_upgrade, UpdState}; @@ -297,6 +314,7 @@ wait_for_upgrade(nodes_upgraded, State) -> ConfirmModFun, self())), UpdState = State#state{execution_pid=Pid, + test_type=TestType, properties=UpdProperties}, {next_state, wait_for_completion, UpdState, TestTimeout}; wait_for_upgrade(_Event, _State) -> @@ -444,7 +462,22 @@ notify_fun(Pid) -> end. cleanup(#state{group_leader=OldGroupLeader, + test_module=TestModule, + test_type=TestType}) when TestType == old -> + lager:debug("Cleaning up old style test ~p", [TestModule]), + %% Reset the state of the nodes ... + rt_harness:setup(), + + %% Reset the global variables + rt_config:set(rt_nodes, []), + rt_config:set(rt_nodenames, []), + rt_config:set(rt_versions, []), + + riak_test_group_leader:tidy_up(OldGroupLeader); +cleanup(#state{test_module=TestModule, + group_leader=OldGroupLeader, properties=Properties}) -> + lager:debug("Cleaning up new style test ~p", [TestModule]), node_manager:return_nodes(rt_properties:get(node_ids, Properties)), riak_test_group_leader:tidy_up(OldGroupLeader). @@ -496,3 +529,4 @@ now_diff(End, Start) -> %% Simple function to hide the details of the message wrapping test_result(Result) -> {test_result, Result}. + From 9766d8cff1872abdb8609eb143866d28f81707fa Mon Sep 17 00:00:00 2001 From: John Burwell Date: Fri, 6 Mar 2015 14:28:40 -0500 Subject: [PATCH 088/157] Adds the partition_repair test case to the regression test script --- regression_test_wrapper.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index 2eff804ef..8643135c4 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -8,6 +8,6 @@ if [ -z $1 ]; then exit 1 fi -TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_membackend,verify_leave" +TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_membackend,verify_leave,partition_repair" ./riak_test -c $1 -t $TEST_CASES From 961c47b9448ff451d48c1b1045c4512f7cb0f078 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Sun, 8 Mar 2015 22:19:54 -0600 Subject: [PATCH 089/157] Add back missing version of rt:set_conf to run verify_build_cluster --- regression_test_wrapper.sh | 2 +- src/rt.erl | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index 8643135c4..9462b9010 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -8,6 +8,6 @@ if [ -z $1 ]; then exit 1 fi -TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_membackend,verify_leave,partition_repair" +TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_membackend,verify_leave,partition_repair,verify_build_cluster" ./riak_test -c $1 -t $TEST_CASES diff --git a/src/rt.erl b/src/rt.erl index 7c1962771..ad22c8820 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -184,6 +184,8 @@ str(String, Substr) -> rt2:str(String, Substr). -spec set_conf(atom(), [{string(), string()}]) -> ok. +set_conf(all, NameValuePairs) -> + rt_harness:set_conf(all, NameValuePairs); set_conf(Node, NameValuePairs) -> stop(Node), ?assertEqual(ok, rt:wait_until_unpingable(Node)), From e4b4d2b1109e4005e609582bc656b93f0ef71bb3 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Mon, 9 Mar 2015 21:53:20 -0600 Subject: [PATCH 090/157] Update CS devrel paths --- src/rt_cs_dev.erl | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/rt_cs_dev.erl b/src/rt_cs_dev.erl index 04071620b..5361545fe 100644 --- a/src/rt_cs_dev.erl +++ b/src/rt_cs_dev.erl @@ -64,7 +64,7 @@ -define(SRC_PATHS, (rt_config:get(src_paths))). get_deps() -> - lists:flatten(io_lib:format("~s/dev/dev1/lib", [relpath(current)])). + lists:flatten(io_lib:format("~s/dev1/lib", [relpath(rt_config:get(default_version, "head"))])). deploy_clusters(ClusterConfig) -> rt_harness_util:deploy_clusters(ClusterConfig). @@ -136,11 +136,11 @@ upgrade(Node, NewVersion) -> NewPath = relpath(NewVersion), Commands = [ - io_lib:format("cp -p -P -R \"~s/dev/dev~b/data\" \"~s/dev/dev~b\"", + io_lib:format("cp -p -P -R \"~s/dev~b/data\" \"~s/dev~b\"", [OldPath, N, NewPath, N]), - io_lib:format("rm -rf ~s/dev/dev~b/data/*", + io_lib:format("rm -rf ~s/dev~b/data/*", [OldPath, N]), - io_lib:format("cp -p -P -R \"~s/dev/dev~b/etc\" \"~s/dev/dev~b\"", + io_lib:format("cp -p -P -R \"~s/dev~b/etc\" \"~s/dev~b\"", [OldPath, N, NewPath, N]) ], [ begin @@ -157,7 +157,7 @@ all_the_app_configs(DevPath) -> lager:error("The dev path is ~p", [DevPath]), case filelib:is_dir(DevPath) of true -> - Devs = filelib:wildcard(DevPath ++ "/dev/dev*"), + Devs = filelib:wildcard(DevPath ++ "/dev*"), [ Dev ++ "/etc/app.config" || Dev <- Devs]; _ -> lager:debug("~s is not a directory.", [DevPath]), @@ -170,7 +170,7 @@ update_app_config(all, Config) -> update_app_config(Node, Config) when is_atom(Node) -> N = node_id(Node), Path = relpath(node_version(N)), - FileFormatString = "~s/dev/dev~b/etc/~s.config", + FileFormatString = "~s/dev~b/etc/~s.config", AppConfigFile = io_lib:format(FileFormatString, [Path, N, "app"]), AdvConfigFile = io_lib:format(FileFormatString, [Path, N, "advanced"]), @@ -215,7 +215,7 @@ get_backends() -> node_path(Node) -> N = node_id(Node), Path = relpath(node_version(N)), - lists:flatten(io_lib:format("~s/dev/dev~b", [Path, N])). + lists:flatten(io_lib:format("~s/dev~b", [Path, N])). create_dirs(Nodes) -> Snmp = [node_path(Node) ++ "/data/snmp/agent/db" || Node <- Nodes], @@ -528,7 +528,7 @@ get_node_logs(Base) -> [ begin {ok, Port} = file:open(Filename, [read, binary]), {lists:nthtail(RootLen, Filename), Port} - end || Filename <- filelib:wildcard(Root ++ "/*/dev/dev*/log/*") ]. + end || Filename <- filelib:wildcard(Root ++ "/*/dev*/log/*") ]. set_advanced_conf(Node, NameValuePairs) -> rt_harness_util:set_advanced_conf(Node, NameValuePairs). From 3579657b91d443cf82811d6c89a0908df17da3ad Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Tue, 10 Mar 2015 21:36:21 -0600 Subject: [PATCH 091/157] Support riak_control_authentication - Pull default_version instead of "head" version - Add in legacy_version and previous_version for upgrade support - Use node_version/1 instead of hard-coding to head --- README.md | 14 ++++++++++++-- regression_test_wrapper.sh | 2 +- src/riak_test_executor.erl | 18 ++++++++++++++---- src/rt.erl | 24 ++++++++++++----------- src/rt_config.erl | 39 +++++++++++++++++++++++++++++++++++++- src/rt_cover.erl | 4 ++-- src/rt_properties.erl | 2 +- src/rtdev.erl | 36 +++++++++++++++++------------------ src/rtssh.erl | 2 +- 9 files changed, 100 insertions(+), 41 deletions(-) diff --git a/README.md b/README.md index bd2cb6ade..ab58a119d 100644 --- a/README.md +++ b/README.md @@ -149,7 +149,7 @@ using the following process: 1. `make stagedevrel` for the Riak release being tested (current default is 2.0, overidden with the `-v` flag). When the `-c` option is specified, `make devclean` will be executed before rebuilding. - 1. Execute `rtdev-current.sh` for the Riak release being tested + 1. Execute `rtdev-install.sh` for the Riak release being tested 1. Rebuild the current riak_test branch. When the `-c` option is specified, 'make clean' will be executed before rebuilding. @@ -188,7 +188,9 @@ to tell riak_test about them. The method of choice is to create a {rtdev, [ {rt_project, "riak"}, {root_path, "/home/you/rt/riak"}, - {default_version, head} + {default_version, head}, + {previous_version, "1.4.12"}, + {legacy_version, "1.3.4"} ]}. ``` @@ -258,6 +260,14 @@ Path to the top of the installed devrel instances. If specific versions of Riak are not specified, this one is tested. The default value is `head` which is typically the head of `develop`. +##### previous_version +Previous version of Riak EE, if not specified defaults to `1.4.12`. +Will be removed after tests have been ported. + +##### legacy_version +Ancient version of Riak EE, if not specified defaults to `1.3.4`. +Will be removed after tests have been ported. + ##### rt_default_config Default configuration parameters that will be used for nodes deployed by riak_test. Tests can override these. diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index 9462b9010..eb691a13f 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -8,6 +8,6 @@ if [ -z $1 ]; then exit 1 fi -TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_membackend,verify_leave,partition_repair,verify_build_cluster" +TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_membackend,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication" ./riak_test -c $1 -t $TEST_CASES diff --git a/src/riak_test_executor.erl b/src/riak_test_executor.erl index b4989b8dc..e2a1768b2 100644 --- a/src/riak_test_executor.erl +++ b/src/riak_test_executor.erl @@ -134,6 +134,7 @@ request_nodes(_Event, _State) -> {next_state, request_nodes, _State}. launch_test(insufficient_versions_available, State) -> + lager:debug("riak_test_executor:launch_test insufficient_versions_available"), #state{pending_tests=[HeadPending | RestPending], execution_mode=ExecutionMode} = State, report_results(HeadPending, {skipped, insufficient_versions}, 0, State), @@ -143,6 +144,7 @@ launch_test(insufficient_versions_available, State) -> launch_test(not_enough_nodes, State) -> %% Move head of pending to waiting and try next test if there is %% one left in pending. + lager:debug("riak_test_executor:launch_test not_enough_nodes"), #state{pending_tests=[HeadPending | RestPending], waiting_tests=Waiting, execution_mode=ExecutionMode} = State, @@ -261,7 +263,7 @@ wait_for_completion_transition(State) -> launch_test_transition(State=#state{pending_tests=PendingTests, execution_mode=ExecutionMode}) when PendingTests == [] orelse ExecutionMode == serial -> - lager:debug("Waiting for complete: execution mode ~p with pending tests ~p", [ExecutionMode, PendingTests]), + lager:debug("Waiting for completion: execution mode ~p with pending tests ~p", [ExecutionMode, PendingTests]), {next_state, wait_for_completion, State}; launch_test_transition(State) -> {next_state, request_nodes, State, 0}. @@ -297,12 +299,20 @@ versions_to_test(Properties) -> versions_to_test(Properties, true) -> case rt_properties:get(upgrade_path, Properties) of undefined -> - [rt_properties:get(start_version, Properties)]; + versions_to_test(Properties, false); UpgradePath -> - UpgradePath + [versions_to_test_to_string(Upgrade) || Upgrade <- UpgradePath] end; versions_to_test(Properties, false) -> - [rt_properties:get(start_version, Properties)]. + InitialVersion = rt_properties:get(start_version, Properties), + [versions_to_test_to_string(InitialVersion)]. +%% Can be in .riak_test.config as either a string or atom, but let's +%% agree upon strings for consistency +-spec versions_to_test_to_string(string()|atom()) -> string(). +versions_to_test_to_string(Version) when is_atom(Version) -> + atom_to_list(Version); +versions_to_test_to_string(Version) when is_list(Version) -> + Version. %% Function to abstract away the details of what properties %% can be overridden on the command line. diff --git a/src/rt.erl b/src/rt.erl index ad22c8820..9439877ef 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -243,8 +243,8 @@ get_https_conn_info(Node) -> deploy_nodes(Versions) when is_list(Versions) -> deploy_nodes(Versions, [riak_kv]); deploy_nodes(NumNodes) when is_integer(NumNodes) -> - [NodeIds, NodeMap, _] = allocate_nodes(NumNodes, head), - deploy_nodes(NodeIds, NodeMap, head, rt_properties:default_config(), [riak_kv]). + [NodeIds, NodeMap, _] = allocate_nodes(NumNodes, rt_config:get(default_version, "head")), + deploy_nodes(NodeIds, NodeMap, rt_config:get(default_version, "head"), rt_properties:default_config(), [riak_kv]). %% @doc Deploy a set of freshly installed Riak nodes with the given %% `InitialConfig', returning a list of the nodes deployed. @@ -262,7 +262,7 @@ deploy_nodes(Versions, Services) -> Nodes. deploy_nodes(NumNodes, InitialConfig, Services) when is_integer(NumNodes) -> - NodeConfig = [{head, InitialConfig} || _ <- lists:seq(1,NumNodes)], + NodeConfig = [{rt_config:get(default_version, "head"), InitialConfig} || _ <- lists:seq(1,NumNodes)], deploy_nodes(NodeConfig, Services). deploy_nodes(NodeIds, NodeMap, Version, Config, Services) -> @@ -313,22 +313,24 @@ allocate_nodes(NumNodes, Version) -> [AllocatedNodeIds, AllocatedNodeMap, VersionMap]. version_to_config(Config) when is_tuple(Config)-> Config; -version_to_config(Version) -> {Version, head}. +version_to_config(Version) -> {Version, rt_config:get(default_version, "head")}. deploy_clusters(Settings) -> ClusterConfigs = [case Setting of Configs when is_list(Configs) -> + lager:info("deploy_cluster Configs"), Configs; NumNodes when is_integer(NumNodes) -> - [{current, default} || _ <- lists:seq(1, NumNodes)]; + [{rt_config:get(default_version, "head"), default} || _ <- lists:seq(1, NumNodes)]; {NumNodes, InitialConfig} when is_integer(NumNodes) -> - [{current, InitialConfig} || _ <- lists:seq(1,NumNodes)]; + [{rt_config:get(default_version, "head"), InitialConfig} || _ <- lists:seq(1,NumNodes)]; {NumNodes, Vsn, InitialConfig} when is_integer(NumNodes) -> - [{Vsn, InitialConfig} || _ <- lists:seq(1,NumNodes)] + [{rt_config:version_to_path(Vsn), InitialConfig} || _ <- lists:seq(1,NumNodes)] end || Setting <- Settings], ?HARNESS:deploy_clusters(ClusterConfigs). build_clusters(Settings) -> + lager:debug("build_clusters ~p", [Settings]), Clusters = deploy_clusters(Settings), [begin join_cluster(Nodes), @@ -339,7 +341,7 @@ build_clusters(Settings) -> %% @doc Start the specified Riak node start(Node) -> %% TODO Determine the best way to implement the current version specification. -jsb - rt_node:start(Node, head). + rt_node:start(Node, rt_harness:node_version(Node)). %% @doc Start the specified Riak `Node' and wait for it to be pingable start_and_wait(Node) -> @@ -348,12 +350,12 @@ start_and_wait(Node) -> async_start(Node) -> %% TODO Determine the best way to implement the current version specification. -jsb - rt_node:async_start(Node, head). + rt_node:async_start(Node, rt_harness:node_version(Node)). %% @doc Stop the specified Riak `Node'. stop(Node) -> %% TODO Determine the best way to implement the current version specification. -jsb - rt_node:stop(Node, head). + rt_node:stop(Node, rt_harness:node_version(Node)). %% @doc Stop the specified Riak `Node' and wait until it is not pingable stop_and_wait(Node) -> @@ -369,7 +371,7 @@ upgrade(Node, NewVersion) -> %% the config based on entries in `Config'. upgrade(Node, NewVersion, Config) -> %% TODO Determine the best way to implement the current version specification. -jsb - rt_node:upgrade(Node, head, NewVersion, Config). + rt_node:upgrade(Node, rt_harness:node_version(Node), NewVersion, Config). %% @doc Upgrade a Riak node to a specific version using the alternate %% leave/upgrade/rejoin approach diff --git a/src/rt_config.erl b/src/rt_config.erl index 3c9e76767..11be8316a 100644 --- a/src/rt_config.erl +++ b/src/rt_config.erl @@ -31,12 +31,19 @@ set/2, set_conf/2, set_advanced_conf/2, - update_app_config/2 + update_app_config/2, + version_to_path/1 ]). -define(HARNESS, (rt_config:get(rt_harness))). -define(CONFIG_NAMESPACE, riak_test). -define(RECEIVE_WAIT_TIME_KEY, rt_max_receive_wait_time). +-define(DEFAULT_VERSION_KEY, default_version). +-define(PREVIOUS_VERSION_KEY, previous_version). +-define(LEGACY_VERSION_KEY, legacy_version). +-define(DEFAULT_VERSION, head). +-define(PREVIOUS_VERSION, "1.4.12"). +-define(LEGACY_VERSION, "1.3.4"). %% @doc Get the value of an OS Environment variable. The arity 1 version of %% this function will fail the test if it is undefined. @@ -170,6 +177,23 @@ to_upper(S) -> lists:map(fun char_to_upper/1, S). char_to_upper(C) when C >= $a, C =< $z -> C bxor $\s; char_to_upper(C) -> C. +%% TODO: Remove after conversion +%% @doc Look up the version by name from the config file +-spec version_to_path(atom()) -> string(). +version_to_path(Version) -> + NewVersion = case Version of + current -> default_version; + legacy -> legacy_version; + previous -> previous_version; + _ -> Version + end, + Path = rt_config:get(NewVersion, Version), + %% If path is stored as an atom, convert to a string + case is_atom(Path) of + true -> atom_to_list(Path); + _ -> Path + end. + -ifdef(TEST). clear(Key) -> @@ -195,4 +219,17 @@ get_rt_max_wait_time_default_test() -> ?assertEqual(ExpectedWaitTime, get(?RECEIVE_WAIT_TIME_KEY, DefaultWaitTime)), ?assertEqual(ExpectedWaitTime, get(rt_max_wait_time, DefaultWaitTime)). +get_version_path_test() -> + clear(?DEFAULT_VERSION_KEY), + clear(?PREVIOUS_VERSION_KEY), + clear(?LEGACY_VERSION_KEY), + + set(?DEFAULT_VERSION_KEY, atom_to_list(?DEFAULT_VERSION)), + set(?PREVIOUS_VERSION_KEY, ?PREVIOUS_VERSION), + set(?LEGACY_VERSION_KEY, ?LEGACY_VERSION), + + ?assertEqual(version_to_path(?DEFAULT_VERSION_KEY), ?DEFAULT_VERSION), + ?assertEqual(version_to_path(?PREVIOUS_VERSION_KEY), ?PREVIOUS_VERSION), + ?assertEqual(version_to_path(?LEGACY_VERSION_KEY), ?LEGACY_VERSION). + -endif. diff --git a/src/rt_cover.erl b/src/rt_cover.erl index b158b5035..dc7ef9545 100644 --- a/src/rt_cover.erl +++ b/src/rt_cover.erl @@ -176,8 +176,8 @@ find_app_modules(CoverApps) -> %% so only current will do. maybe_start_on_node(Node, Version) -> IsCurrent = case Version of - current -> true; - {current, _} -> true; + head -> true; + {head, _} -> true; _ -> false end, ShouldStart = IsCurrent andalso diff --git a/src/rt_properties.erl b/src/rt_properties.erl index 4ac8e17fb..946c80e2a 100644 --- a/src/rt_properties.erl +++ b/src/rt_properties.erl @@ -38,7 +38,7 @@ node_count=3 :: non_neg_integer(), metadata=[] :: proplists:proplist(), rolling_upgrade=false :: boolean(), - start_version="head" :: string(), + start_version=rt_config:get(default_version, "head") :: string(), current_version :: string(), upgrade_path :: [string()], wait_for_transfers=false :: boolean(), diff --git a/src/rtdev.erl b/src/rtdev.erl index bbec93135..df117d086 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -508,23 +508,23 @@ add_default_node_config(Nodes) -> throw({invalid_config, {rt_default_config, BadValue}}) end. -deploy_clusters(_ClusterConfigs) -> - ok. -%% NumNodes = rt_config:get(num_nodes, 6), -%% RequestedNodes = lists:flatten(ClusterConfigs), - -%% case length(RequestedNodes) > NumNodes of -%% true -> -%% erlang:error("Requested more nodes than available"); -%% false -> -%% Nodes = deploy_nodes(RequestedNodes), -%% {DeployedClusters, _} = lists:foldl( -%% fun(Cluster, {Clusters, RemNodes}) -> -%% {A, B} = lists:split(length(Cluster), RemNodes), -%% {Clusters ++ [A], B} -%% end, {[], Nodes}, ClusterConfigs), -%% DeployedClusters -%% end. +deploy_clusters(ClusterConfigs) -> + NumNodes = rt_config:get(num_nodes, 6), + RequestedNodes = lists:flatten(ClusterConfigs), + lager:info("RequestedNodes ~p~n", [RequestedNodes]), + + case length(RequestedNodes) > NumNodes of + true -> + erlang:error("Requested more nodes than available"); + false -> + Nodes = deploy_nodes(RequestedNodes), + {DeployedClusters, _} = lists:foldl( + fun(Cluster, {Clusters, RemNodes}) -> + {A, B} = lists:split(length(Cluster), RemNodes), + {Clusters ++ [A], B} + end, {[], Nodes}, ClusterConfigs), + DeployedClusters + end. configure_nodes(Nodes, Configs) -> %% Set initial config @@ -895,7 +895,7 @@ set_backend(Backend, OtherOpts) -> %% WRONG: Seemingly always stuck on the current version get_version() -> - case file:read_file(relpath(head) ++ "/VERSION") of + case file:read_file(relpath(rt_config:get(default_version, "head")) ++ "/VERSION") of {error, enoent} -> unknown; {ok, Version} -> Version end. diff --git a/src/rtssh.erl b/src/rtssh.erl index 03917e913..0e93b6f32 100644 --- a/src/rtssh.erl +++ b/src/rtssh.erl @@ -681,7 +681,7 @@ scp_from(Host, RemotePath, Path) -> -define(PATH, (rt_config:get(rtdev_path))). dev_path(Path, N) -> - format("~s/dev/dev~b", [Path, N]). + format("~s/dev~b", [Path, N]). dev_bin_path(Path, N) -> dev_path(Path, N) ++ "/bin". From 1631b9e0cd5bb46bb2841f3f9885fa3bb593a3f8 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Wed, 11 Mar 2015 11:47:44 -0400 Subject: [PATCH 092/157] Re-implements prompting to leave nodes running on failure and verify_jmx and basic_command_line test cases pass * Adds always_fail_test, jmx_verify, and basic_command_line to the list of tests executed. With this change, a successful run should have precisely one failure for always_fail_test. * Fixes a quirky parsing behavior in rtdev:extract_node_id_and name that resulted in nested lists when used via verify_riak_stats * Modifies rt:build_clusters to use rt_properties:default_config * Fixes unit test failes in riak_test_lager_backend * Modifies the regression test script to execute riak_test with verbose output --- regression_test_wrapper.sh | 4 +- src/node_manager.erl | 22 ++++++++--- src/riak_test_escript.erl | 33 +++++++--------- src/riak_test_lager_backend.erl | 8 ++-- src/riak_test_runner.erl | 30 +++++++++++++-- src/rt.erl | 20 ++++------ src/rtdev.erl | 67 ++++++++++++++++++++------------- 7 files changed, 111 insertions(+), 73 deletions(-) diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index eb691a13f..c738e90bc 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -8,6 +8,6 @@ if [ -z $1 ]; then exit 1 fi -TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_membackend,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication" +TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_membackend,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify" -./riak_test -c $1 -t $TEST_CASES +./riak_test -v -c $1 -t $TEST_CASES diff --git a/src/node_manager.erl b/src/node_manager.erl index 6e7df3ec7..59e3e21b9 100644 --- a/src/node_manager.erl +++ b/src/node_manager.erl @@ -113,12 +113,11 @@ handle_cast(_Msg, State) -> handle_info(_Info, State) -> {noreply, State}. -terminate(_Reason, State) -> +terminate(_Reason, _State) -> %% Stop and reset all deployed nodes - stop_and_clean(State#state.nodes_deployed, - State#state.node_map, - State#state.deployed_versions, - true), + %%stop(State#state.nodes_deployed, + %% State#state.node_map, + %% State#state.deployed_versions), ok. code_change(_OldVsn, State, _Extra) -> @@ -128,6 +127,19 @@ code_change(_OldVsn, State, _Extra) -> %%% Internal functions %%%=================================================================== +% stop(NodeIds, NodeMap, DeployedVersions) -> +% [begin +% case version_deployed(NodeId, DeployedVersions) of +% undefined -> +% ok; +% Version -> +% rt_node:stop_and_wait(NodeId, +% rt_node:node_name(NodeId, NodeMap), +% Version) +% end +% end || NodeId <- NodeIds]. + + stop_and_clean(NodeIds, NodeMap, DeployedVersions, Wait) -> [begin case version_deployed(NodeId, DeployedVersions) of diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index 586e414ed..53e7b989d 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -115,14 +115,11 @@ finalize(TestResults, Args) -> %% R <- TestResults], %% CoverDir = rt_config:get(cover_output, "coverage"), %% Coverage = rt_cover:maybe_write_coverage(all, CoverDir), - - node_manager:stop(), Verbose = proplists:is_defined(verbose, Args), report_results(TestResults, Verbose), - - %% Teardown = not proplists:get_value(keep, Args, false), - %% maybe_teardown(Teardown, TestResults, Coverage, Verbose), + Teardown = not proplists:get_value(keep, Args, false), + maybe_teardown(Teardown, TestResults), ok. %% Option Name, Short Code, Long Code, Argument Spec, Help Message @@ -296,20 +293,18 @@ set_lager_env(OutputDir, ConsoleLevel, FileLevel) -> {level, FileLevel}]}], application:set_env(lager, handlers, HandlerConfig). -%% maybe_teardown(false, TestResults, Coverage, Verbose) -> -%% print_summary(TestResults, Coverage, Verbose), -%% lager:info("Keeping cluster running as requested"); -%% maybe_teardown(true, TestResults, Coverage, Verbose) -> -%% case {length(TestResults), proplists:get_value(status, hd(TestResults))} of -%% {1, fail} -> -%% print_summary(TestResults, Coverage, Verbose), -%% so_kill_riak_maybe(); -%% _ -> -%% lager:info("Multiple tests run or no failure"), -%% rt_cluster:teardown(), -%% print_summary(TestResults, Coverage, Verbose) -%% end, -%% ok. +maybe_teardown(false, _TestResults) -> + lager:info("Keeping cluster running as requested"); +maybe_teardown(Keep, TestResults) when is_list(TestResults) andalso + erlang:length(TestResults) == 1 -> + maybe_teardown(Keep, hd(TestResults)); +maybe_teardown(true, {_, {fail, _}, _}) -> + so_kill_riak_maybe(), + ok; +maybe_teardown(true, _TestResults) -> + lager:info("Multiple tests run or no failure"), + rt_cluster:teardown(), + ok. -spec comma_tokenizer(string(), [string()]) -> [string()]. comma_tokenizer(S, Acc) -> diff --git a/src/riak_test_lager_backend.erl b/src/riak_test_lager_backend.erl index b1644bef8..a925861b0 100644 --- a/src/riak_test_lager_backend.erl +++ b/src/riak_test_lager_backend.erl @@ -183,11 +183,9 @@ log_test_() -> lager:info("Here's a message"), lager:debug("Here's another message"), {ok, Logs} = gen_event:delete_handler(lager_event, riak_test_lager_backend, []), - ?assertEqual(3, length(Logs)), - - ?assertMatch([_, "[debug]", "Lager installed handler riak_test_lager_backend into lager_event"], re:split(lists:nth(1, Logs), " ", [{return, list}, {parts, 3}])), - ?assertMatch([_, "[info]", "Here's a message"], re:split(lists:nth(2, Logs), " ", [{return, list}, {parts, 3}])), - ?assertMatch([_, "[debug]", "Here's another message"], re:split(lists:nth(3, Logs), " ", [{return, list}, {parts, 3}])) + ?assertEqual(2, length(Logs)), + ?assertMatch([_, "[info]", "Here's a message\r\n"], re:split(lists:nth(1, Logs), " ", [{return, list}, {parts, 3}])), + ?assertMatch([_, "[debug]", "Here's another message\r\n"], re:split(lists:nth(2, Logs), " ", [{return, list}, {parts, 3}])) end } diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index 2260f05c3..2c0dea778 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -67,7 +67,10 @@ prereq_check :: atom(), current_version :: string(), remaining_versions :: [string()], - test_results :: [term()]}). + test_results :: [term()], + stop_on_fail :: boolean()}). + +-deprecated([{metadata,0,next_major_release}]). %%%=================================================================== %%% API @@ -88,7 +91,6 @@ stop() -> -spec(metadata() -> [{atom(), term()}]). %% @doc fetches test metadata from spawned test process -%% TODO: Remove when test ports are over metadata() -> FSMPid = get(test_runner_fsm), gen_fsm:sync_send_all_state_event(FSMPid, metadata_event, infinity). @@ -120,6 +122,7 @@ init([TestModule, Backend, Properties]) -> BackendCheck = check_backend(Backend, rt_properties:get(valid_backends, Properties)), PreReqCheck = check_prereqs(ConfirmMod), + StopOnFail = rt_config:get(rt_stop_on_fail, true), State = #state{test_module=TestModule, test_type=TestType, properties=UpdProperties, @@ -129,7 +132,8 @@ init([TestModule, Backend, Properties]) -> confirm_modfun=ConfirmModFun, backend_check=BackendCheck, prereq_check=PreReqCheck, - group_leader=group_leader()}, + group_leader=group_leader(), + stop_on_fail=StopOnFail}, {ok, setup, State, 0}. %% @doc there are no all-state events for this fsm @@ -213,7 +217,7 @@ execute({nodes_deployed, _}, State) -> test_timeout=TestTimeout} = State, lager:notice("Running ~s", [TestModule]), lager:notice("Properties: ~p", [Properties]), - + StartTime = os:timestamp(), %% Perform test setup which includes clustering of the nodes if %% required by the test properties. The cluster information is placed @@ -258,6 +262,24 @@ wait_for_completion(timeout, State=#state{test_module=TestModule, cleanup(UpdState), notify_executor(timeout, UpdState), {stop, normal, UpdState}; +wait_for_completion({test_result, {fail, Reason}}, State=#state{test_module=TestModule, + test_type=TestType, + group_leader=GroupLeader, + stop_on_fail=StopOnFail, + remaining_versions=[]}) -> + Result = {fail, Reason}, + UpdState = State#state{test_module=TestModule, + test_type=TestType, + test_results=Result, + group_leader=GroupLeader, + stop_on_fail=StopOnFail, + end_time=os:timestamp()}, + case StopOnFail of + true -> ok; + false -> cleanup(UpdState) + end, + notify_executor(Result, UpdState), + {stop, normal, UpdState}; wait_for_completion({test_result, Result}, State=#state{test_module=TestModule, test_type=TestType, group_leader=GroupLeader, diff --git a/src/rt.erl b/src/rt.erl index 9439877ef..3605f0eca 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -187,19 +187,13 @@ str(String, Substr) -> set_conf(all, NameValuePairs) -> rt_harness:set_conf(all, NameValuePairs); set_conf(Node, NameValuePairs) -> - stop(Node), - ?assertEqual(ok, rt:wait_until_unpingable(Node)), - rt_config:set_conf(Node, NameValuePairs), - start(Node). + rt_config:set_conf(Node, NameValuePairs). -spec set_advanced_conf(atom(), [{string(), string()}]) -> ok. set_advanced_conf(all, NameValuePairs) -> rt_config:set_advanced_conf(all, NameValuePairs); set_advanced_conf(Node, NameValuePairs) -> - stop(Node), - ?assertEqual(ok, rt:wait_until_unpingable(Node)), - rt_config:set_advanced_conf(Node, NameValuePairs), - start(Node). + rt_config:set_advanced_conf(Node, NameValuePairs). %% @doc Rewrite the given node's app.config file, overriding the varialbes %% in the existing app.config with those in `Config'. @@ -262,8 +256,10 @@ deploy_nodes(Versions, Services) -> Nodes. deploy_nodes(NumNodes, InitialConfig, Services) when is_integer(NumNodes) -> - NodeConfig = [{rt_config:get(default_version, "head"), InitialConfig} || _ <- lists:seq(1,NumNodes)], - deploy_nodes(NodeConfig, Services). + Version = rt_config:get(default_version, "head"), + [NodeIds, NodeMap, _] = allocate_nodes(NumNodes, Version), + + deploy_nodes(NodeIds, NodeMap, Version, InitialConfig, Services). deploy_nodes(NodeIds, NodeMap, Version, Config, Services) -> _ = rt_harness_util:deploy_nodes(NodeIds, NodeMap, Version, Config, Services), @@ -693,9 +689,9 @@ claimant_according_to(Node) -> %% @doc Safely construct a new cluster and return a list of the deployed nodes %% @todo Add -spec and update doc to reflect mult-version changes build_cluster(Versions) when is_list(Versions) -> - build_cluster(length(Versions), Versions, default); + build_cluster(length(Versions), Versions, rt_properties:default_config()); build_cluster(NumNodes) -> - build_cluster(NumNodes, default). + build_cluster(NumNodes, rt_properties:default_config()). %% @doc Safely construct a `NumNode' size cluster using %% `InitialConfig'. Return a list of the deployed nodes. diff --git a/src/rtdev.erl b/src/rtdev.erl index df117d086..4b76c491b 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -164,19 +164,7 @@ harness_nodes(NodeIds) -> [list_to_atom(NodeId ++ "@127.0.0.1") || NodeId <- NodeIds]. so_fresh_so_clean(VersionMap) -> - %% make sure we stop any cover processes on any nodes otherwise, - %% if the next test boots a legacy node we'll end up with cover - %% incompatabilities and crash the cover server - %% rt_cover:maybe_stop_on_nodes(), - %% Path = relpath(root), - %% Stop all discoverable nodes, not just nodes we'll be using for - %% this test. - StopAllFun = - fun({Version, VersionNodes}) -> - VersionPath = filename:join([?PATH, Version]), - stop_nodes(VersionPath, VersionNodes) - end, - rt:pmap(StopAllFun, VersionMap), + ok = stop_all(VersionMap), %% Reset nodes to base state lager:info("Resetting nodes to fresh state"), @@ -196,13 +184,32 @@ so_fresh_so_clean(VersionMap) -> end, VersionMap), ok. +stop_all() -> + [_, _, VersionMap] = available_resources(), + stop_all(VersionMap). + +stop_all(VersionMap) -> + %% make sure we stop any cover processes on any nodes otherwise, + %% if the next test boots a legacy node we'll end up with cover + %% incompatabilities and crash the cover server + %% rt_cover:maybe_stop_on_nodes(), + %% Path = relpath(root), + %% Stop all discoverable nodes, not just nodes we'll be using for + %% this test. + StopAllFun = + fun({Version, VersionNodes}) -> + VersionPath = filename:join([?PATH, Version]), + stop_nodes(VersionPath, VersionNodes) + end, + rt:pmap(StopAllFun, VersionMap), + ok. + available_resources() -> VersionMap = [{Version, harness_node_ids(Version)} || Version <- versions()], NodeIds = harness_node_ids(rt_config:get(default_version, "head")), NodeMap = lists:zip(NodeIds, harness_nodes(NodeIds)), [NodeIds, NodeMap, VersionMap]. - setup_harness() -> %% Get node names and populate node map [NodeIds, NodeMap, VersionMap] = available_resources(), @@ -376,6 +383,7 @@ update_app_config_file(ConfigFile, Config) -> {error, enoent} -> [] end, + lager:debug("Config: ~p", [Config]), MergeA = orddict:from_list(Config), MergeB = orddict:from_list(BaseConfig), NewConfig = @@ -386,6 +394,7 @@ update_app_config_file(ConfigFile, Config) -> ValA end, MergeC, MergeD) end, MergeA, MergeB), + lager:debug("Writing ~p to ~p", [NewConfig, ConfigFile]), NewConfigOut = io_lib:format("~p.", [NewConfig]), ?assertEqual(ok, file:write_file(ConfigFile, NewConfigOut)), ok. @@ -442,8 +451,9 @@ get_backend(AppConfig) -> end. node_path(Node) -> - Path = relpath(node_version(Node)), - lists:flatten(io_lib:format("~s/~s", [Path, node_short_name(Node)])). + {NodeId, NodeName} = extract_node_id_and_name(Node), + Path = relpath(node_version(NodeName)), + lists:flatten(io_lib:format("~s/~s", [Path, node_short_name(NodeId)])). get_ip(_Node) -> %% localhost 4 lyfe @@ -456,6 +466,7 @@ create_dirs(Nodes) -> clean_data_dir(Nodes, SubDir) when is_list(Nodes) -> DataDirs = [node_path(Node) ++ "/data/" ++ SubDir || Node <- Nodes], + lager:debug("Cleaning data directories ~p", [DataDirs]), lists:foreach(fun rm_dir/1, DataDirs). %% Blocking to delete files is not the best use of time. Generally it @@ -714,8 +725,9 @@ console(Node, Expected) -> interactive(Node, "console", Expected). interactive(Node, Command, Exp) -> - Path = relpath(node_version(Node)), - Cmd = riakcmd(Path, Node, Command), + {NodeId, NodeName} = extract_node_id_and_name(Node), + Path = relpath(node_version(NodeName)), + Cmd = riakcmd(Path, NodeId, Command), lager:debug("Opening a port for riak ~s.", [Command]), lager:debug("Calling open_port with cmd ~s", [binary_to_list(iolist_to_binary(Cmd))]), P = open_port({spawn, binary_to_list(iolist_to_binary(Cmd))}, @@ -805,8 +817,9 @@ execute_admin_cmd(Cmd, Options) -> end. riak(Node, Args) -> - Path = relpath(node_version(Node)), - Result = run_riak(Node, Path, Args), + {NodeId, NodeName} = extract_node_id_and_name(Node), + Path = relpath(node_version(NodeName)), + Result = run_riak(NodeId, Path, Args), lager:info("~s", [Result]), {ok, Result}. @@ -824,7 +837,7 @@ node_id(Node) -> orddict:fetch(Node, NodeMap). %% @doc Find the short dev node name from the full name --spec node_short_name(atom()) -> atom(). +-spec node_short_name(atom() | list()) -> atom(). node_short_name(Node) when is_list(Node) -> Node; node_short_name(Node) when is_atom(Node) -> @@ -832,12 +845,13 @@ node_short_name(Node) when is_atom(Node) -> orddict:fetch(Node, NodeMap). %% @doc Return the node version from rt_versions based on full node name --spec node_version(atom() | integer()) -> string(). +-spec node_version(atom() | integer() | list()) -> string(). node_version(Node) when is_integer(Node) -> node_version(node_short_name_to_name(Node)); node_version(Node) -> + {_, NodeName} = extract_node_id_and_name(Node), VersionMap = rt_config:get(rt_versions), - orddict:fetch(Node, VersionMap). + orddict:fetch(NodeName, VersionMap). spawn_cmd(Cmd) -> spawn_cmd(Cmd, []). @@ -910,7 +924,7 @@ teardown() -> %% rt_cover:maybe_stop_on_nodes(), %% Stop all discoverable nodes, not just nodes we'll be using for this test. %% rt:pmap(fun(X) -> stop_all(X ++ "/dev") end, devpaths()). - ok. + stop_all(). whats_up() -> io:format("Here's what's running...~n"), @@ -945,7 +959,7 @@ get_node_logs() -> -spec extract_node_id_and_name(atom() | string()) -> node_tuple(). extract_node_id_and_name(Node) when is_atom(Node) -> NodeStr = atom_to_list(Node), - extract_node_id_and_name(NodeStr, contains(NodeStr, $@)); + extract_node_id_and_name(NodeStr); extract_node_id_and_name(Node) when is_list(Node) -> extract_node_id_and_name(Node, contains(Node, $@)); extract_node_id_and_name(_Node) -> @@ -956,7 +970,7 @@ extract_node_id_and_name(Node, true) -> [NodeId, _] = re:split(Node, "@"), {binary_to_list(NodeId), list_to_atom(Node)}; extract_node_id_and_name(Node, false) -> - {Node, ?DEV(Node)}. + {Node, ?DEV(lists:flatten(Node))}. -spec contains(list(), char()) -> boolean. contains(Str, Char) -> @@ -978,6 +992,7 @@ extract_node_id_and_name_test() -> Expected = {"dev2", 'dev2@127.0.0.1'}, ?assertEqual(Expected, extract_node_id_and_name('dev2@127.0.0.1')), ?assertEqual(Expected, extract_node_id_and_name("dev2@127.0.0.1")), + ?assertEqual(Expected, extract_node_id_and_name('dev2')), ?assertEqual(Expected, extract_node_id_and_name("dev2")). maybe_contains_test() -> From d68b88adda4334aeeb53f5a7a22a961101ccc752 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Thu, 12 Mar 2015 15:13:17 -0400 Subject: [PATCH 093/157] - Modifies rtdev-install to create directories in the form of - to be compatible with rtdev:get_versions/1 - Removes the xmerl_version stat from verify_riak_stats to pass on 2.0.5 - Maps various rt module functions to call into rt2 rather rt --- bin/rtdev-install.sh | 59 +++++++++++++++++++++++++------------------- src/rt.erl | 4 +-- 2 files changed, 35 insertions(+), 28 deletions(-) diff --git a/bin/rtdev-install.sh b/bin/rtdev-install.sh index 4b1ededb8..291a1b881 100755 --- a/bin/rtdev-install.sh +++ b/bin/rtdev-install.sh @@ -31,19 +31,21 @@ if [ -z "${RT_VERSION+xxx}" ] || ([ -z "$RT_VERSION" ] && [ "${RT_VERSION+xxx}" RT_VERSION=`cat $cwd/dependency_manifest.git | awk '/^-/ { print $NF }'` else echo "Making $(pwd) a tagged release:" - TAGS=`git describe --tags` + #TAGS=`git describe --tags` CURRENT=`git rev-parse HEAD` HEAD=`git show-ref | grep HEAD | cut -f1 -d" "` - if [ -n "`echo ${TAGS} | grep riak_ee`" ]; then + #if [ -n "`echo ${TAGS} | grep riak_ee`" ]; then # For riak_ee - RT_VERSION=`echo ${TAGS} | awk '{sub(/riak_ee-/,"",$0);print}'` - else + # RT_VERSION=`echo ${TAGS} | awk '{sub(/riak_ee-/,"",$0);print}'` + #else # For open source riak - RT_VERSION=`echo ${TAGS} | awk '{sub(/riak-/,"",$0);print}'` - fi + # RT_VERSION=`echo ${TAGS} | awk '{sub(/riak-/,"",$0);print}'` + #fi # If we are on the tip, call the version "head" if [ "${CURRENT}" == "${HEAD}" ]; then RT_VERSION="head" + else + RT_VERSION="$(git describe --tags)" fi fi fi @@ -62,11 +64,13 @@ if [ -d ".git" ]; then git reset HEAD --hard > /dev/null 2>&1 git clean -fd > /dev/null 2>&1 fi -echo " - Removing and recreating $RT_DEST_DIR/$RT_VERSION" -rm -rf $RT_DEST_DIR/$RT_VERSION -mkdir $RT_DEST_DIR/$RT_VERSION + +RT_VERSION_DIR=$RT_DEST_DIR/$RT_VERSION +echo " - Removing and recreating $RT_VERSION_DIR" +rm -rf $RT_VERSION_DIR +mkdir $RT_VERSION_DIR cd $cwd -echo " - Copying devrel to $RT_DEST_DIR/$RT_VERSION" +echo " - Copying devrel to $RT_VERSION_DIR" if [ ! -d "dev" ]; then echo "You need to run \"make devrel\" or \"make stagedevrel\" first" exit 1 @@ -75,21 +79,24 @@ cd dev # Clone the existing dev directory into RT_DEST_DIR for i in `ls`; do cp -p -P -R $i $RT_DEST_DIR/$RT_VERSION/; done -echo " - Writing $RT_DEST_DIR/$RT_VERSION/VERSION" -echo -n $RT_VERSION > $RT_DEST_DIR/$RT_VERSION/VERSION -cd $RT_DEST_DIR -if [ -d ".git" ]; then - echo " - Reinitializing git state" - git add --ignore-removal -f . - git commit -a -m "riak_test init" --amend > /dev/null 2>&1 -else - git init - ## Some versions of git and/or OS require these fields - git config user.name "Riak Test" - git config user.email "dev@basho.com" + VERSION_FILE=$RT_VERSION_DIR/VERSION + echo " - Writing $VERSION_FILE" + echo -n $RT_VERSION > $VERSION_FILE - git add --ignore-removal . - git commit -a -m "riak_test init" > /dev/null - echo " - Successfully completed initial git commit of $RT_DEST_DIR" -fi + cd $RT_DEST_DIR + if [ -d ".git" ]; then + echo " - Reinitializing git state" + git add --ignore-removal -f . + git commit -a -m "riak_test init" --amend > /dev/null 2>&1 + else + git init + + ## Some versions of git and/or OS require these fields + git config user.name "Riak Test" + git config user.email "dev@basho.com" + + git add --ignore-removal . + git commit -a -m "riak_test init" > /dev/null + echo " - Successfully completed initial git commit of $RT_DEST_DIR" + fi diff --git a/src/rt.erl b/src/rt.erl index 3605f0eca..5b297c693 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -553,7 +553,7 @@ wait_until_transfers_complete(Nodes) -> wait_for_service(Node, Services) when is_list(Services) -> rt2:wait_for_service(Node, Services); wait_for_service(Node, Service) -> - wait_for_service(Node, [Service]). + rt2:wait_for_service(Node, [Service]). wait_for_cluster_service(Nodes, Service) -> rt2:wait_for_cluster_service(Nodes, Service). @@ -566,7 +566,7 @@ wait_until_nodes_ready(Nodes) -> %% @doc Wait until all nodes in the list `Nodes' believe each other to be %% members of the cluster. wait_until_all_members(Nodes) -> - wait_until_all_members(Nodes, Nodes). + rt2:wait_until_all_members(Nodes, Nodes). %% @doc Wait until all nodes in the list `Nodes' believes all nodes in the %% list `Members' are members of the cluster. From b7b4013ecbe517db7e1b77d00578ab487917520f Mon Sep 17 00:00:00 2001 From: John Burwell Date: Thu, 12 Mar 2015 17:39:08 -0400 Subject: [PATCH 094/157] verify_aae test cases using the v1 API wrapper. Adds the test case to the regression test script. --- regression_test_wrapper.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index c738e90bc..5d651bf35 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -8,6 +8,6 @@ if [ -z $1 ]; then exit 1 fi -TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_membackend,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify" +TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_membackend,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae" ./riak_test -v -c $1 -t $TEST_CASES From f56993bddd8ca10fa67640072e14e3e9aeba86a1 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Thu, 12 Mar 2015 21:28:30 -0400 Subject: [PATCH 095/157] ensemble_bysantine, ensemble_interleave, verify_claimant, and verify_object_limits pass using the v1 wrapper and have been added to the regression test script --- regression_test_wrapper.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index 5d651bf35..a90b629b6 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -8,6 +8,6 @@ if [ -z $1 ]; then exit 1 fi -TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_membackend,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae" +TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_membackend,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantine" ./riak_test -v -c $1 -t $TEST_CASES From 904778d8b7c466abdcecd4f5288b9b09f3911d6c Mon Sep 17 00:00:00 2001 From: John Burwell Date: Thu, 12 Mar 2015 23:09:08 -0400 Subject: [PATCH 096/157] Test case gh_riak_core_155 passes using the v1 wrapper * Modifies rt:update_app_config/2 to call rt_config:update_app_config/2 * Modifies rtdev:update_app_config/2 to split the node id from the passed node name to determine the node number * Adds the gh_riak_core_155 test case to the regression test script --- regression_test_wrapper.sh | 2 +- src/rt.erl | 5 +---- src/rtdev.erl | 5 +++-- 3 files changed, 5 insertions(+), 7 deletions(-) diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index a90b629b6..667e58bdd 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -8,6 +8,6 @@ if [ -z $1 ]; then exit 1 fi -TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_membackend,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantine" +TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_membackend,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantinei,gh_riak_core_155" ./riak_test -v -c $1 -t $TEST_CASES diff --git a/src/rt.erl b/src/rt.erl index 5b297c693..93c1a6b74 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -200,10 +200,7 @@ set_advanced_conf(Node, NameValuePairs) -> update_app_config(all, Config) -> rt_config:update_app_config(all, Config); update_app_config(Node, Config) -> - stop(Node), - ?assertEqual(ok, rt:wait_until_unpingable(Node)), - rt_config:update_app_config(Node, Config), - start(Node). + rt_config:update_app_config(Node, Config). %% @doc Helper that returns first successful application get_env result, %% used when different versions of Riak use different app vars for diff --git a/src/rtdev.erl b/src/rtdev.erl index 4b76c491b..bf78ea770 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -359,10 +359,11 @@ update_app_config(DevPath, Config) -> update_app_config(Node, Version, Config) -> VersionPath = filename:join(?PATH, Version), FileFormatString = "~s/~s/etc/~s.config", + {NodeId, _} = extract_node_id_and_name(Node), AppConfigFile = io_lib:format(FileFormatString, - [VersionPath, node_short_name(Node), "app"]), + [VersionPath, node_short_name(NodeId), "app"]), AdvConfigFile = io_lib:format(FileFormatString, - [VersionPath, node_short_name(Node), "advanced"]), + [VersionPath, node_short_name(NodeId), "advanced"]), %% If there's an app.config, do it old style %% if not, use cuttlefish's advanced.config From f725fb47063cf92e493e674ea98d7b227d5ac301 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Fri, 13 Mar 2015 21:28:56 -0600 Subject: [PATCH 097/157] Update pb_security to use code:get_object_code/1 to work with new framework --- regression_test_wrapper.sh | 2 +- tests/pb_security.erl | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index 667e58bdd..62cf5a2f1 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -8,6 +8,6 @@ if [ -z $1 ]; then exit 1 fi -TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_membackend,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantinei,gh_riak_core_155" +TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_membackend,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantinei,gh_riak_core_155,pb_security" ./riak_test -v -c $1 -t $TEST_CASES diff --git a/tests/pb_security.erl b/tests/pb_security.erl index 95e8502f6..3b06d1d7a 100644 --- a/tests/pb_security.erl +++ b/tests/pb_security.erl @@ -816,4 +816,4 @@ reduce_set_union(List, A) -> mapred_modfun_input(Pipe, _Args, _Timeout) -> riak_pipe:queue_work(Pipe, {{<<"hello">>, <<"world">>}, {struct, []}}), - riak_pipe:eoi(Pipe). \ No newline at end of file + riak_pipe:eoi(Pipe). From 8f894799b94c3f5f4a4d96e0c45dca9744aaedf2 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Fri, 13 Mar 2015 21:30:21 -0600 Subject: [PATCH 098/157] Add debugging statements to test results to help debug in case of framework failure --- src/riak_test_runner.erl | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index 2c0dea778..41d9a50a9 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -268,6 +268,7 @@ wait_for_completion({test_result, {fail, Reason}}, State=#state{test_module=Test stop_on_fail=StopOnFail, remaining_versions=[]}) -> Result = {fail, Reason}, + lager:debug("Test Result ~p = {fail, ~p}", [TestModule, Reason]), UpdState = State#state{test_module=TestModule, test_type=TestType, test_results=Result, @@ -284,6 +285,7 @@ wait_for_completion({test_result, Result}, State=#state{test_module=TestModule, test_type=TestType, group_leader=GroupLeader, remaining_versions=[]}) -> + lager:debug("Test Result ~p = {~p}", [TestModule, Result]), %% TODO: Format results for aggregate test runs if needed. For %% upgrade tests with failure return which versions had failure %% along with reasons. @@ -302,6 +304,7 @@ wait_for_completion({test_result, Result}, State) -> current_version=CurrentVersion, remaining_versions=[NextVersion | RestVersions], properties=Properties} = State, + lager:debug("Test Result ~p = {~p}", [TestModule, Result]), Config = rt_backend:set(Backend, rt_properties:get(config, Properties)), NodeIds = rt_properties:get(node_ids, Properties), node_manager:upgrade_nodes(NodeIds, From ff1acf2e334847c94c9082a2b744a87228a3f0ef Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Fri, 13 Mar 2015 21:37:48 -0600 Subject: [PATCH 099/157] Update versioning - Added rt_config:get_default_version/0, et al - Require rt_project to get included in config file - Require default_version to be defined - Prefix "head" with project name --- README.md | 8 +++---- bin/rtdev-all.sh | 30 +++++++++++++++++-------- bin/rtdev-install.sh | 33 ++++++++++++++++++---------- bin/rtdev-migrate.sh | 2 +- src/rt.erl | 12 +++++----- src/rt_config.erl | 51 +++++++++++++++++++++++++++++++++---------- src/rt_cs_dev.erl | 2 +- src/rt_properties.erl | 2 +- src/rtdev.erl | 6 ++--- 9 files changed, 99 insertions(+), 47 deletions(-) diff --git a/README.md b/README.md index ab58a119d..8363d43d3 100644 --- a/README.md +++ b/README.md @@ -14,7 +14,7 @@ contents of `$HOME/rt/riak` might look something like this: ``` $ ls $HOME/rt/riak -head riak-1.3.2 riak-1.4.12 +riak-head riak-1.3.2 riak-1.4.12 ``` Inside each of these directories is a series `dev[0-9]+` directories, typically @@ -34,10 +34,10 @@ There is one folder in `$HOME/rt/riak` that does not come from `~/test-releases`: `head`. The `head` folder can refer to any version of Riak, but is typically used for something like the `master` branch, a feature branch, or a release candidate. -The `$HOME/rt/riak/head` dev release gets populated from a devrel of Riak +The `$HOME/rt/riak/riak-head` dev release gets populated from a stagedevrel of Riak that can come from anywhere, but is usually your 'normal' git checkout of Riak. The `bin/rtdev-install.sh` can be run from within that folder -to copy `dev/` into `$HOME/rt/riak/head`. +to copy `dev/` into `$HOME/rt/riak/riak-head`. Once you have everything set up (again, instructions for this are below), you'll want to run and write tests. This repository also holds code for @@ -72,7 +72,7 @@ Essentially these packages need to be on your system for a successful build: This script is for a complete installation. It performs all of the setup steps including installing the current "master" branch from -Github into "head". The releases will be built in your current working +Github into "riak-head". The releases will be built in your current working directory, so create an empty one in a place you'd like to store these builds for posterity, so that you don't have to rebuild them if your installation path (`$HOME/rt/riak` by the way this script installs it) gets into diff --git a/bin/rtdev-all.sh b/bin/rtdev-all.sh index c932fd027..bbb1eb654 100755 --- a/bin/rtdev-all.sh +++ b/bin/rtdev-all.sh @@ -40,7 +40,7 @@ # Current version of Erlang (for "head" version) : ${CURRENT_OTP:=$R16B02} # Label of the "current" version -: ${VERSION:="head"} +: ${DEFAULT_VERSION:="riak-head"} # By default the Open Source version of Riak will be used, but for internal # testing you can override this variable to use `riak_ee` instead : ${RT_USE_EE:=""} @@ -52,6 +52,7 @@ popd > /dev/null GITURL_RIAK="git://github.com/basho/riak" GITURL_RIAK_EE="git@github.com:basho/riak_ee" +# Determine if Erlang has already been built checkbuild() { ERLROOT=$1 @@ -76,21 +77,28 @@ checkbuild() fi } +# Build and install Erlangs kerl() { RELEASE=$1 BUILDNAME=$2 + export CFLAGS="-g -O2" + export LDFLAGS="-g" if [ -n "`uname -r | grep el6`" ]; then - export CFLAGS="-DOPENSSL_NO_EC=1" + export CFLAGS="-g -DOPENSSL_NO_EC=1" fi if [ "`uname`" == "Darwin" ]; then - export CFLAGS="-O0" - export KERL_CONFIGURE_OPTIONS="-disable-hipe --enable-smp-support --enable-threads --enable-kernel-poll --without-odbc --enable-darwin-64bit" + export CFLAGS="-g -O0" + export KERL_CONFIGURE_OPTIONS="--disable-hipe --enable-smp-support --enable-threads --enable-kernel-poll --without-odbc --enable-darwin-64bit" + else + export KERL_CONFIGURE_OPTIONS="--disable-hipe --enable-smp-support --enable-threads --without-odbc --enable-m64-build" fi echo " - Building Erlang $RELEASE (this could take a while)" # Use the patched version of Erlang for R16B02 builds - if [ "$RELEASE" == "R16B02" ]; then + if [ "$RELEASE" == "R15B01" ]; then + ./kerl build git git://github.com/basho/otp.git basho_OTP_R15B01p $BUILDNAME + elif [ "$RELEASE" == "R16B02" ]; then ./kerl build git git://github.com/basho/otp.git r16 $BUILDNAME else ./kerl build $RELEASE $BUILDNAME @@ -110,6 +118,7 @@ kerl() fi } +# Build stagedevrels for testing build() { SRCDIR=$1 @@ -151,10 +160,10 @@ build() # For non-tagged builds (i.e. head), use make deps. Otherwise, use # make locked-deps for tagged builds ... - if [ "$SRCDIR" == "head" ]; then - make deps + if [ -n "`echo ${SRCDIR} | grep head`" ]; then + make deps else - $RUN make locked-deps + $RUN make locked-deps fi $RUN make all stagedevrel @@ -242,11 +251,14 @@ echo if [ -z "$RT_USE_EE" ]; then build "riak-1.3.2" $R15B01 build "riak-1.4.12" $R15B01 + if [ "${DEFAULT_VERSION" == "riak-head" ]; then + DEFAULT_VERSION = "riak_ee-head" + fi else build "riak_ee-1.3.4" $R15B01 build "riak_ee-1.4.12" $R15B01 fi -build "$VERSION" $R16B02 +build "$DEFAUlT_VERSION" $R16B02 echo echo "= Build complete! ===============================================" diff --git a/bin/rtdev-install.sh b/bin/rtdev-install.sh index 291a1b881..704c3e9fa 100755 --- a/bin/rtdev-install.sh +++ b/bin/rtdev-install.sh @@ -28,28 +28,34 @@ echo -n " - Determining version: " if [ -z "${RT_VERSION+xxx}" ] || ([ -z "$RT_VERSION" ] && [ "${RT_VERSION+xxx}" = "xxx" ]); then if [ -f $cwd/dependency_manifest.git ]; then # For packaged distributions - RT_VERSION=`cat $cwd/dependency_manifest.git | awk '/^-/ { print $NF }'` + VERSION=`cat $cwd/dependency_manifest.git | awk '/^-/ { print $NF }'` else echo "Making $(pwd) a tagged release:" - #TAGS=`git describe --tags` + TAGS=`git describe --tags` CURRENT=`git rev-parse HEAD` HEAD=`git show-ref | grep HEAD | cut -f1 -d" "` - #if [ -n "`echo ${TAGS} | grep riak_ee`" ]; then + PRODUCT="" + if [ -n "`echo ${TAGS} | grep riak_ee`" ]; then # For riak_ee - # RT_VERSION=`echo ${TAGS} | awk '{sub(/riak_ee-/,"",$0);print}'` - #else + PRODUCT="riak_ee" + VERSION=`echo ${TAGS} | awk '{sub(/riak_ee-/,"",$0);print}'` + elif [ -n "`echo ${TAGS} | grep riak_cs`" ]; then + # For riak_cs + PRODUCT="riak_cs" + VERSION=`echo ${TAGS} | awk '{sub(/riak_cs-/,"",$0);print}'` + else # For open source riak - # RT_VERSION=`echo ${TAGS} | awk '{sub(/riak-/,"",$0);print}'` - #fi + PRODUCT="riak" + RT_VERSION=`echo ${TAGS} | awk '{sub(/riak-/,"",$0);print}'` + fi # If we are on the tip, call the version "head" if [ "${CURRENT}" == "${HEAD}" ]; then - RT_VERSION="head" - else - RT_VERSION="$(git describe --tags)" + VERSION="head" fi fi + RT_VERSION="${PRODUCT}-${VERSION}" fi -echo $RT_VERSION +echo "Version: ${RT_VERSION}" # Create the RT_DEST_DIR directory if it does not yet exist if [ ! -d $RT_DEST_DIR ]; then @@ -91,6 +97,11 @@ for i in `ls`; do cp -p -P -R $i $RT_DEST_DIR/$RT_VERSION/; done git commit -a -m "riak_test init" --amend > /dev/null 2>&1 else git init + cat > .gitignore < deploy_nodes(Versions) when is_list(Versions) -> deploy_nodes(Versions, [riak_kv]); deploy_nodes(NumNodes) when is_integer(NumNodes) -> - [NodeIds, NodeMap, _] = allocate_nodes(NumNodes, rt_config:get(default_version, "head")), - deploy_nodes(NodeIds, NodeMap, rt_config:get(default_version, "head"), rt_properties:default_config(), [riak_kv]). + [NodeIds, NodeMap, _] = allocate_nodes(NumNodes, rt_config:get_default_version()), + deploy_nodes(NodeIds, NodeMap, rt_config:get_default_version(), rt_properties:default_config(), [riak_kv]). %% @doc Deploy a set of freshly installed Riak nodes with the given %% `InitialConfig', returning a list of the nodes deployed. @@ -253,7 +253,7 @@ deploy_nodes(Versions, Services) -> Nodes. deploy_nodes(NumNodes, InitialConfig, Services) when is_integer(NumNodes) -> - Version = rt_config:get(default_version, "head"), + Version = rt_config:get_default_version(), [NodeIds, NodeMap, _] = allocate_nodes(NumNodes, Version), deploy_nodes(NodeIds, NodeMap, Version, InitialConfig, Services). @@ -306,7 +306,7 @@ allocate_nodes(NumNodes, Version) -> [AllocatedNodeIds, AllocatedNodeMap, VersionMap]. version_to_config(Config) when is_tuple(Config)-> Config; -version_to_config(Version) -> {Version, rt_config:get(default_version, "head")}. +version_to_config(Version) -> {Version, rt_config:get_default_version()}. deploy_clusters(Settings) -> ClusterConfigs = [case Setting of @@ -314,9 +314,9 @@ deploy_clusters(Settings) -> lager:info("deploy_cluster Configs"), Configs; NumNodes when is_integer(NumNodes) -> - [{rt_config:get(default_version, "head"), default} || _ <- lists:seq(1, NumNodes)]; + [{rt_config:get_default_version(), default} || _ <- lists:seq(1, NumNodes)]; {NumNodes, InitialConfig} when is_integer(NumNodes) -> - [{rt_config:get(default_version, "head"), InitialConfig} || _ <- lists:seq(1,NumNodes)]; + [{rt_config:get_default_version(), InitialConfig} || _ <- lists:seq(1,NumNodes)]; {NumNodes, Vsn, InitialConfig} when is_integer(NumNodes) -> [{rt_config:version_to_path(Vsn), InitialConfig} || _ <- lists:seq(1,NumNodes)] end || Setting <- Settings], diff --git a/src/rt_config.erl b/src/rt_config.erl index 11be8316a..cb96c870d 100644 --- a/src/rt_config.erl +++ b/src/rt_config.erl @@ -25,6 +25,9 @@ get/2, config_or_os_env/1, config_or_os_env/2, + get_default_version/0, + get_previous_version/0, + get_legacy_version/0, get_os_env/1, get_os_env/2, load/2, @@ -41,6 +44,7 @@ -define(DEFAULT_VERSION_KEY, default_version). -define(PREVIOUS_VERSION_KEY, previous_version). -define(LEGACY_VERSION_KEY, legacy_version). +-define(PROJECT_KEY, rt_project). -define(DEFAULT_VERSION, head). -define(PREVIOUS_VERSION, "1.4.12"). -define(LEGACY_VERSION, "1.3.4"). @@ -114,6 +118,27 @@ get(Key, Default) -> Value -> Value end. +%% @doc Return the default version +-spec get_default_version() -> string(). +get_default_version() -> + get_version(?DEFAULT_VERSION_KEY). + +%% @doc Return the default version +-spec get_previous_version() -> string(). +get_previous_version() -> + get_version(?PREVIOUS_VERSION_KEY). + +%% @doc Return the default version +-spec get_legacy_version() -> string(). +get_legacy_version() -> + get_version(?LEGACY_VERSION_KEY). + +%% @doc Prepends the project onto the default version +%% e.g. "riak_ee-3.0.1" or "riak-head" +-spec get_version(atom()) -> string(). +get_version(Vsn) -> + convert_to_string(rt_config:get(?PROJECT_KEY)) ++ "-" ++ convert_to_string(rt_config:get(Vsn)). + -spec config_or_os_env(atom()) -> term(). config_or_os_env(Config) -> OSEnvVar = to_upper(atom_to_list(Config)), @@ -181,19 +206,23 @@ char_to_upper(C) -> C. %% @doc Look up the version by name from the config file -spec version_to_path(atom()) -> string(). version_to_path(Version) -> - NewVersion = case Version of - current -> default_version; - legacy -> legacy_version; - previous -> previous_version; - _ -> Version - end, - Path = rt_config:get(NewVersion, Version), - %% If path is stored as an atom, convert to a string - case is_atom(Path) of - true -> atom_to_list(Path); - _ -> Path + case Version of + default_version -> rt_config:get_default_version(); + legacy_version -> rt_config:get_legacy_version(); + previous_version -> rt_config:get_previous_version(); + current -> rt_config:get_default_version(); + legacy -> rt_config:get_legacy_version(); + previous -> rt_config:get_previous_version(); + _ -> rt_config:get(Version) end. +%% @doc: Convert an atom to a string if it is not already +-spec convert_to_string(string() | atom()) -> string(). +convert_to_string(Val) when is_atom(Val) -> + atom_to_list(Val); +convert_to_string(Val) when is_list(Val) -> + Val. + -ifdef(TEST). clear(Key) -> diff --git a/src/rt_cs_dev.erl b/src/rt_cs_dev.erl index 5361545fe..97e590a65 100644 --- a/src/rt_cs_dev.erl +++ b/src/rt_cs_dev.erl @@ -64,7 +64,7 @@ -define(SRC_PATHS, (rt_config:get(src_paths))). get_deps() -> - lists:flatten(io_lib:format("~s/dev1/lib", [relpath(rt_config:get(default_version, "head"))])). + lists:flatten(io_lib:format("~s/dev1/lib", [relpath(rt_config:get_default_version())])). deploy_clusters(ClusterConfig) -> rt_harness_util:deploy_clusters(ClusterConfig). diff --git a/src/rt_properties.erl b/src/rt_properties.erl index 946c80e2a..f99e70568 100644 --- a/src/rt_properties.erl +++ b/src/rt_properties.erl @@ -38,7 +38,7 @@ node_count=3 :: non_neg_integer(), metadata=[] :: proplists:proplist(), rolling_upgrade=false :: boolean(), - start_version=rt_config:get(default_version, "head") :: string(), + start_version=rt_config:get_default_version() :: string(), current_version :: string(), upgrade_path :: [string()], wait_for_transfers=false :: boolean(), diff --git a/src/rtdev.erl b/src/rtdev.erl index bf78ea770..538617017 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -64,7 +64,7 @@ devrel_node_name(N) when is_integer(N) -> list_to_atom(lists:concat(["dev", N, "@127.0.0.1"])). get_deps() -> - DefaultVersionPath = filename:join(?PATH, rt_config:get(default_version)), + DefaultVersionPath = filename:join(?PATH, rt_config:get_default_version()), lists:flatten(io_lib:format("~s/dev1/lib", [DefaultVersionPath])). %% @doc Create a command-line command @@ -206,7 +206,7 @@ stop_all(VersionMap) -> available_resources() -> VersionMap = [{Version, harness_node_ids(Version)} || Version <- versions()], - NodeIds = harness_node_ids(rt_config:get(default_version, "head")), + NodeIds = harness_node_ids(rt_config:get_default_version()), NodeMap = lists:zip(NodeIds, harness_nodes(NodeIds)), [NodeIds, NodeMap, VersionMap]. @@ -910,7 +910,7 @@ set_backend(Backend, OtherOpts) -> %% WRONG: Seemingly always stuck on the current version get_version() -> - case file:read_file(relpath(rt_config:get(default_version, "head")) ++ "/VERSION") of + case file:read_file(relpath(rt_config:get_default_version()) ++ "/VERSION") of {error, enoent} -> unknown; {ok, Version} -> Version end. From 979386aad3f70a058bcf1eb48f04f1d48b93f13a Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Tue, 17 Mar 2015 21:31:20 -0600 Subject: [PATCH 100/157] Update version configuration and add version validation --- src/riak_test_escript.erl | 5 ++- src/riak_test_executor.erl | 11 +---- src/rt.erl | 5 ++- src/rt_config.erl | 83 +++++++++++++++++++++++++++++--------- src/rt_harness.erl | 5 ++- src/rtdev.erl | 18 ++++++++- src/rtssh.erl | 23 ++++++++++- 7 files changed, 113 insertions(+), 37 deletions(-) diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index 53e7b989d..dc51a7e65 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -135,9 +135,9 @@ cli_options() -> {verbose, $v, "verbose", undefined, "verbose output"}, {outdir, $o, "outdir", string, "output directory"}, {backend, $b, "backend", atom, "backend to test [memory | bitcask | eleveldb]"}, - {upgrade_path, $u, "upgrade-path", atom, "comma-separated list representing an upgrade path (e.g. 1.2.1,1.3.4,1.4.10,2.0.0)"}, + {upgrade_path, $u, "upgrade-path", atom, "comma-separated list representing an upgrade path (e.g. riak-1.3.4,riak_ee-1.4.12,riak_ee-2.0.0)"}, {keep, undefined, "keep", boolean, "do not teardown cluster"}, - {report, $r, "report", string, "you're reporting an official test run, provide platform info (e.g. ubuntu-1204-64)\nUse 'config' if you want to pull from ~/.riak_test.config"}, + {report, $r, "report", string, "you're reporting an official test run, provide platform info (e.g. ubuntu-1404-64)\nUse 'config' if you want to pull from ~/.riak_test.config"}, {file, $F, "file", string, "use the specified file instead of ~/.riak_test.config"} ]. @@ -192,6 +192,7 @@ report(ParsedArgs) -> parse_args(Args) -> help_or_parse_args(getopt:parse(cli_options(), Args)). +%% @doc Print help string if it's specified, otherwise parse the arguments help_or_parse_args({ok, {[], _}}) -> print_help(); help_or_parse_args({ok, {ParsedArgs, HarnessArgs}}) -> diff --git a/src/riak_test_executor.erl b/src/riak_test_executor.erl index e2a1768b2..e52012dff 100644 --- a/src/riak_test_executor.erl +++ b/src/riak_test_executor.erl @@ -301,18 +301,11 @@ versions_to_test(Properties, true) -> undefined -> versions_to_test(Properties, false); UpgradePath -> - [versions_to_test_to_string(Upgrade) || Upgrade <- UpgradePath] + [rt_config:convert_to_string(Upgrade) || Upgrade <- UpgradePath] end; versions_to_test(Properties, false) -> InitialVersion = rt_properties:get(start_version, Properties), - [versions_to_test_to_string(InitialVersion)]. -%% Can be in .riak_test.config as either a string or atom, but let's -%% agree upon strings for consistency --spec versions_to_test_to_string(string()|atom()) -> string(). -versions_to_test_to_string(Version) when is_atom(Version) -> - atom_to_list(Version); -versions_to_test_to_string(Version) when is_list(Version) -> - Version. + [rt_config:convert_to_string(InitialVersion)]. %% Function to abstract away the details of what properties %% can be overridden on the command line. diff --git a/src/rt.erl b/src/rt.erl index f043bf8e6..cedfa0a03 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -318,7 +318,7 @@ deploy_clusters(Settings) -> {NumNodes, InitialConfig} when is_integer(NumNodes) -> [{rt_config:get_default_version(), InitialConfig} || _ <- lists:seq(1,NumNodes)]; {NumNodes, Vsn, InitialConfig} when is_integer(NumNodes) -> - [{rt_config:version_to_path(Vsn), InitialConfig} || _ <- lists:seq(1,NumNodes)] + [{rt_config:version_to_tag(Vsn), InitialConfig} || _ <- lists:seq(1,NumNodes)] end || Setting <- Settings], ?HARNESS:deploy_clusters(ClusterConfigs). @@ -686,7 +686,8 @@ claimant_according_to(Node) -> %% @doc Safely construct a new cluster and return a list of the deployed nodes %% @todo Add -spec and update doc to reflect mult-version changes build_cluster(Versions) when is_list(Versions) -> - build_cluster(length(Versions), Versions, rt_properties:default_config()); + UpdatedVersions = [{rt_config:version_to_tag(Vsn), Cfg} || {Vsn, Cfg} <- Versions], + build_cluster(length(Versions), UpdatedVersions, rt_properties:default_config()); build_cluster(NumNodes) -> build_cluster(NumNodes, rt_properties:default_config()). diff --git a/src/rt_config.erl b/src/rt_config.erl index cb96c870d..277b3b442 100644 --- a/src/rt_config.erl +++ b/src/rt_config.erl @@ -25,27 +25,30 @@ get/2, config_or_os_env/1, config_or_os_env/2, + convert_to_string/1, get_default_version/0, get_previous_version/0, get_legacy_version/0, get_os_env/1, get_os_env/2, + get_upgrade_path/1, load/2, set/2, set_conf/2, set_advanced_conf/2, update_app_config/2, - version_to_path/1 + version_to_tag/1 ]). -define(HARNESS, (rt_config:get(rt_harness))). -define(CONFIG_NAMESPACE, riak_test). -define(RECEIVE_WAIT_TIME_KEY, rt_max_receive_wait_time). --define(DEFAULT_VERSION_KEY, default_version). --define(PREVIOUS_VERSION_KEY, previous_version). --define(LEGACY_VERSION_KEY, legacy_version). --define(PROJECT_KEY, rt_project). +-define(VERSION_KEY, versions). +-define(DEFAULT_VERSION_KEY, default). +-define(PREVIOUS_VERSION_KEY, previous). +-define(LEGACY_VERSION_KEY, legacy). -define(DEFAULT_VERSION, head). +-define(UPGRADE_KEY, upgrade_paths). -define(PREVIOUS_VERSION, "1.4.12"). -define(LEGACY_VERSION, "1.3.4"). @@ -87,10 +90,15 @@ load_dot_config(ConfigName, ConfigFile) -> %% Now, overlay the specific project Config = proplists:get_value(list_to_atom(ConfigName), Terms), [set(Key, Value) || {Key, Value} <- Config], - ok; + %% Validate all versions and upgrade paths + Versions=rt_config:get(?VERSION_KEY), + Upgrades=rt_config:get(?UPGRADE_KEY), + RealVersions = [get_version(Name) || {Name, Vsn} <- Versions, is_tuple(Vsn)], + RealUpgrades = lists:merge([get_upgrade_path(Name) || {Name, Upg} <- Upgrades, Upg =/= []]), + rt_harness:validate_config(lists:usort(RealVersions ++ RealUpgrades)); {error, Reason} -> erlang:error("Failed to parse config file", [ConfigFile, Reason]) - end. + end. set(Key, Value) -> ok = application:set_env(riak_test, Key, Value). @@ -135,9 +143,46 @@ get_legacy_version() -> %% @doc Prepends the project onto the default version %% e.g. "riak_ee-3.0.1" or "riak-head" --spec get_version(atom()) -> string(). +-spec get_version(term()) -> string() | not_found. get_version(Vsn) -> - convert_to_string(rt_config:get(?PROJECT_KEY)) ++ "-" ++ convert_to_string(rt_config:get(Vsn)). + Versions = rt_config:get(?VERSION_KEY), + resolve_version(Vsn, Versions). + +%% @doc Map logical name of version into a pathname string +-spec resolve_version(term(), [{term(), term()}]) -> string() | no_return(). +resolve_version(Vsn, Versions) -> + case find_atom_or_string(Vsn, Versions) of + undefined -> + erlang:error("Could not find version", [Vsn]); + {Product, Tag} -> + convert_to_string(Product) ++ "-" ++ convert_to_string(Tag); + Version -> + resolve_version(Version, Versions) + end. + +%% @doc Look up values by both atom and by string +find_atom_or_string(Key, Table) -> + case {Key, proplists:get_value(Key, Table)} of + {_, undefined} when is_atom(Key) -> + proplists:get_value(atom_to_list(Key), Table); + {_, undefined} when is_list(Key) -> + proplists:get_value(list_to_atom(Key), Table); + {Key, Value} -> + Value + end. + +%% @doc Look up a named upgrade path and return the resolved list of versions +-spec get_upgrade_path(term()) -> list() | not_found. +get_upgrade_path(Upg) -> + Upgrades = rt_config:get(?UPGRADE_KEY), + case proplists:get_value(Upg, Upgrades) of + undefined -> + erlang:error("Could not find upgrade path version", [Upg]); + UpgradePath when is_list(UpgradePath) -> + [get_version(Vsn) || Vsn <- UpgradePath]; + _ -> + erlang:error("Upgrade path has an invalid definition", [Upg]) + end. -spec config_or_os_env(atom()) -> term(). config_or_os_env(Config) -> @@ -204,20 +249,18 @@ char_to_upper(C) -> C. %% TODO: Remove after conversion %% @doc Look up the version by name from the config file --spec version_to_path(atom()) -> string(). -version_to_path(Version) -> +-spec version_to_tag(atom()) -> string(). +version_to_tag(Version) -> case Version of - default_version -> rt_config:get_default_version(); - legacy_version -> rt_config:get_legacy_version(); - previous_version -> rt_config:get_previous_version(); + default -> rt_config:get_default_version(); current -> rt_config:get_default_version(); legacy -> rt_config:get_legacy_version(); previous -> rt_config:get_previous_version(); - _ -> rt_config:get(Version) + _ -> rt_config:get_version(Version) end. %% @doc: Convert an atom to a string if it is not already --spec convert_to_string(string() | atom()) -> string(). +-spec convert_to_string(string()|atom()) -> string(). convert_to_string(Val) when is_atom(Val) -> atom_to_list(Val); convert_to_string(Val) when is_list(Val) -> @@ -253,12 +296,12 @@ get_version_path_test() -> clear(?PREVIOUS_VERSION_KEY), clear(?LEGACY_VERSION_KEY), - set(?DEFAULT_VERSION_KEY, atom_to_list(?DEFAULT_VERSION)), + set(?DEFAULT_VERSION_KEY, ?DEFAULT_VERSION), set(?PREVIOUS_VERSION_KEY, ?PREVIOUS_VERSION), set(?LEGACY_VERSION_KEY, ?LEGACY_VERSION), - ?assertEqual(version_to_path(?DEFAULT_VERSION_KEY), ?DEFAULT_VERSION), - ?assertEqual(version_to_path(?PREVIOUS_VERSION_KEY), ?PREVIOUS_VERSION), - ?assertEqual(version_to_path(?LEGACY_VERSION_KEY), ?LEGACY_VERSION). + ?assertEqual(version_to_tag(?DEFAULT_VERSION_KEY), ?DEFAULT_VERSION), + ?assertEqual(version_to_tag(?PREVIOUS_VERSION_KEY), ?PREVIOUS_VERSION), + ?assertEqual(version_to_tag(?LEGACY_VERSION_KEY), ?LEGACY_VERSION). -endif. diff --git a/src/rt_harness.erl b/src/rt_harness.erl index e89105f74..644abf20f 100644 --- a/src/rt_harness.erl +++ b/src/rt_harness.erl @@ -56,7 +56,8 @@ teardown/0, set_conf/2, set_advanced_conf/2, - update_app_config/2]). + update_app_config/2, + validate_config/1]). start(Node, Version) -> ?HARNESS_MODULE:start(Node, Version). @@ -154,3 +155,5 @@ set_advanced_conf(Node, NameValuePairs) -> update_app_config(Node, Config) -> ?HARNESS_MODULE:update_app_config(Node, Config). +validate_config(Versions) -> + ?HARNESS_MODULE:validate_config(Versions). diff --git a/src/rtdev.erl b/src/rtdev.erl index 538617017..2692d171f 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -48,7 +48,8 @@ teardown/0, set_conf/2, set_advanced_conf/2, - rm_dir/1]). + rm_dir/1, + validate_config/1]). -compile(export_all). -include_lib("eunit/include/eunit.hrl"). @@ -987,6 +988,21 @@ maybe_contains(_) -> node_short_name_to_name(N) -> ?DEV("dev" ++ integer_to_list(N)). +%% @doc Check to make sure that all versions specified in the config file actually exist +-spec validate_config([term()]) -> ok | no_return(). +validate_config(Versions) -> + Root = rt_config:get(root_path), + Validate = fun(Vsn) -> + {Result, _} = file:read_file_info(filename:join([Root, Vsn, "dev1/bin/riak"])), + case Result of + ok -> ok; + _ -> + erlang:error("Could not find specified devrel version", [Vsn]) + end + end, + [Validate(Vsn) || Vsn <- Versions], + ok. + -ifdef(TEST). extract_node_id_and_name_test() -> diff --git a/src/rtssh.erl b/src/rtssh.erl index 0e93b6f32..cbc3db1f4 100644 --- a/src/rtssh.erl +++ b/src/rtssh.erl @@ -26,7 +26,8 @@ update_app_config/2, teardown/0, set_conf/2, - set_advanced_conf/2]). + set_advanced_conf/2, + validate_config/1]). -compile(export_all). -include_lib("eunit/include/eunit.hrl"). @@ -678,7 +679,7 @@ scp_from(Host, RemotePath, Path) -> %%% Riak devrel path utilities %%%=================================================================== --define(PATH, (rt_config:get(rtdev_path))). +-define(PATH, (rt_config:get(root_path))). dev_path(Path, N) -> format("~s/dev~b", [Path, N]). @@ -831,6 +832,24 @@ stop_all(Host, DevPath) -> teardown() -> stop_all(rt_config:get(rt_hostnames)). +%% @doc Check to make sure that all versions specified in the config file actually exist +-spec validate_config([term()]) -> ok | no_return(). +validate_config(Versions) -> + Hosts = load_hosts(), + Root = rt_config:get(root_path), + Validate = fun(Host, Vsn) -> + Cmd = "ls " ++ filename:join([Root, Vsn, "dev1/bin/riak"]), + Result = wait_for_cmd(spawn_ssh_cmd(atom_to_list(Host), Cmd, [], true)), + io:format("Result = ~p~n", [Result]), + case Result of + {0, _} -> ok; + _ -> + erlang:error("Could not find specified devrel version", [Host, Vsn]) + end + end, + [[Validate(Host, Vsn) || Vsn <- Versions] || Host <- Hosts], + ok. + %%%=================================================================== %%% Utilities %%%=================================================================== From 224965a35417914d058c1ab52bb82df0fc540323 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Wed, 18 Mar 2015 10:40:24 -0600 Subject: [PATCH 101/157] Add in verify_search test to regression suite --- regression_test_wrapper.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index 62cf5a2f1..8a5a304a5 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -8,6 +8,6 @@ if [ -z $1 ]; then exit 1 fi -TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_membackend,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantinei,gh_riak_core_155,pb_security" +TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_membackend,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantinei,gh_riak_core_155,pb_security,verify_search" ./riak_test -v -c $1 -t $TEST_CASES From 238cc46283e1c59e6f234c6d5746d53de1962fd7 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Wed, 18 Mar 2015 22:34:20 -0400 Subject: [PATCH 102/157] verify_2i_aae passes the regression test suite. verify_membackend no longer passes due to configuration race to set the backend. This test will be ported to the new framework pre-merge to eliminate this race condition. * Modifies regression_test_wrapper.sh script to run test against all supported backends and capture results into results/regression/yyddmmHHMMss directory. The results of the last test run are symlinked to results/regression/current. * Modifies riak_test_executor to set a configuration variable, rt_backend, with the backend passed on the command line via -b or --backend. * Modifies rt.erl to set the backend to the value of rt_config:get(rt_backend) when deploying nodes when the passed configuration does not specify a backend * Fixes rt_backend:set to set the backend to riak_kv_memory_backend rather than riak_kv_eleveldb_backend when memory is specified --- regression_test_wrapper.sh | 25 +++++++++++++++++++++++-- src/riak_test_executor.erl | 5 +++++ src/rt.erl | 38 ++++++++++++++++++++++++++++++++++++-- src/rt_backend.erl | 2 +- 4 files changed, 65 insertions(+), 5 deletions(-) diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index 8a5a304a5..0edb8738b 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -8,6 +8,27 @@ if [ -z $1 ]; then exit 1 fi -TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_membackend,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantinei,gh_riak_core_155,pb_security,verify_search" +ALL_BACKEND_TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantine,gh_riak_core_155,pb_security,verify_search" -./riak_test -v -c $1 -t $TEST_CASES +BITCASK_BACKEND_TEST_CASES="$ALL_BACKEND_TEST_CASES" +ELEVELDB_BACKEND_TEST_CASES="$ALL_BACKEND_TEST_CASES,verify_2i_aae" +MEMORY_BACKEND_TEST_CASES="$ALL_BACKEND_TEST_CASES,verify_2i_aae" + +ROOT_RESULTS_DIR="results/regression" +RESULTS_DIR="$ROOT_RESULTS_DIR/`date +"%y%m%d%H%M%s"`" +mkdir -p $RESULTS_DIR + +RESULTS_SYMLINK=$ROOT_RESULTS_DIR/current +rm -f $RESULTS_SYMLINK +ln -s $RESULTS_DIR $RESULTS_SYMLINK + +echo "Running bitcask regression tests using the following test cases: $BITCASK_BACKEND_TEST_CASES" +./riak_test -v -c $1 -t $BITCASK_BACKEND_TEST_CASES &> $RESULTS_DIR/bitcask_results.log + +echo "Running leveldb regression tests using the following test cases: $ELEVELDB_BACKEND_TEST_CASES" +./riak_test -v -c $1 -t $ELEVELDB_BACKEND_TEST_CASES -b eleveldb &> $RESULTS_DIR/leveldb_results.log + +echo "Running memory regression tests using the following test cases: $MEMORY_BACKEND_TEST_CASES" +./riak_test -v -c $1 -t $MEMORY_BACKEND_TEST_CASES -b memory &> $RESULTS_DIR/memory_results.log + +echo "Results of the test run written to $RESULTS_DIR" diff --git a/src/riak_test_executor.erl b/src/riak_test_executor.erl index e52012dff..479b9e56e 100644 --- a/src/riak_test_executor.erl +++ b/src/riak_test_executor.erl @@ -61,6 +61,11 @@ stop() -> init([Tests, Backend, LogDir, ReportInfo, UpgradeList, NotifyPid]) -> %% TODO Change the default when parallel execution support is implemented -jsb ExecutionMode = rt_config:get(rt_execution_mode, serial), + + %% TODO: Remove after all tests ported 2.0 -- workaround to support + %% backend command line argument fo v1 cluster provisioning -jsb + rt_config:set(rt_backend, Backend), + lager:notice("Starting the Riak Test executor in ~p execution mode", [ExecutionMode]), State = #state{pending_tests=Tests, backend=Backend, diff --git a/src/rt.erl b/src/rt.erl index cedfa0a03..1ac3f063e 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -259,11 +259,26 @@ deploy_nodes(NumNodes, InitialConfig, Services) when is_integer(NumNodes) -> deploy_nodes(NodeIds, NodeMap, Version, InitialConfig, Services). deploy_nodes(NodeIds, NodeMap, Version, Config, Services) -> - _ = rt_harness_util:deploy_nodes(NodeIds, NodeMap, Version, Config, Services), + BackendConfig = maybe_set_backend(rt_config:get(rt_backend), Config), + _ = rt_harness_util:deploy_nodes(NodeIds, NodeMap, Version, BackendConfig, Services), lists:foldl(fun({_, NodeName}, Nodes) -> [NodeName|Nodes] end, [], NodeMap). +maybe_set_backend(Backend, Config) -> + maybe_set_backend(lists:keyfind(riak_kv, 1, Config), Backend, Config). + +maybe_set_backend(false, riak_kv_eleveldb_backend, Config) -> + maybe_set_backend(false, eleveldb, Config); +maybe_set_backend(false, riak_kv_memory_backend, Config) -> + maybe_set_backend(false, memory, Config); +maybe_set_backend(false, Backend, Config) -> + rt_backend:set(Backend, Config); +maybe_set_backend({storage_backend, _}=_KVSection, _Backend, Config) -> + Config; +maybe_set_backend({riak_kv, KVSection}, Backend, Config) -> + maybe_set_backend(lists:keyfind(storage_backend, 1, KVSection), Backend, Config). + allocate_nodes(NumNodes, Version) when is_atom(Version) -> allocate_nodes(NumNodes, atom_to_list(Version)); allocate_nodes(NumNodes, Version) -> @@ -628,7 +643,7 @@ wait_until_nodes_agree_about_ownership(Nodes) -> %% AAE support wait_until_aae_trees_built(Nodes) -> - rt_aae:wait_until_app_trees_built(Nodes). + rt_aae:wait_until_aae_trees_built(Nodes). %%%=================================================================== %%% Ring Functions @@ -1437,6 +1452,25 @@ wait_for_control(VersionedNodes) -> -ifdef(TEST). +verify_backend(InputConfig, ExpectedBackend) -> + verify_backend(InputConfig, ExpectedBackend, ExpectedBackend). + +verify_backend(InputConfig, Backend, ExpectedBackend) -> + ?_test(begin + ActualConfig = maybe_set_backend(Backend, InputConfig), + io:format("ActualConfig: ~p", [ActualConfig]), + [{riak_kv, [{storage_backend, ActualBackend}]}] = ActualConfig, + ?assertEqual(ExpectedBackend, ActualBackend) + end). + +maybe_set_backend_test_() -> + {foreach, + fun() -> ok end, + [verify_backend([], riak_kv_bitcask_backend), + verify_backend([], riak_kv_eleveldb_backend), + verify_backend([], riak_kv_memory_backend), + verify_backend([{riak_kv, [{storage_backend, riak_kv_bitcask_backend}]}], riak_kv_eleveldb_backend, riak_kv_bitcask_backend)]}. + verify_product(Applications, ExpectedApplication) -> ?_test(begin meck:new(rpc, [unstick]), diff --git a/src/rt_backend.erl b/src/rt_backend.erl index 496afd446..d31175223 100644 --- a/src/rt_backend.erl +++ b/src/rt_backend.erl @@ -44,7 +44,7 @@ set(eleveldb, Config) -> lists:keyfind(riak_kv, 1, Config)), lists:keystore(riak_kv, 1, Config, {riak_kv, UpdKVSection}); set(memory, Config) -> - UpdKVSection = replace_backend(riak_kv_eleveldb_backend, + UpdKVSection = replace_backend(riak_kv_memory_backend, lists:keyfind(riak_kv, 1, Config)), lists:keystore(riak_kv, 1, Config, {riak_kv, UpdKVSection}); set(multi, Config) -> From a7e7d0ee6124508fce2fc3c325c3fc842518e85c Mon Sep 17 00:00:00 2001 From: John Burwell Date: Thu, 19 Mar 2015 14:17:24 -0400 Subject: [PATCH 103/157] Fixes the creation of the current results symlink and various syntax errors in the rtdev-all script --- bin/rtdev-all.sh | 17 +++++++++-------- regression_test_wrapper.sh | 7 ++++--- 2 files changed, 13 insertions(+), 11 deletions(-) diff --git a/bin/rtdev-all.sh b/bin/rtdev-all.sh index bbb1eb654..3368c304b 100755 --- a/bin/rtdev-all.sh +++ b/bin/rtdev-all.sh @@ -249,16 +249,17 @@ echo echo if [ -z "$RT_USE_EE" ]; then - build "riak-1.3.2" $R15B01 - build "riak-1.4.12" $R15B01 - if [ "${DEFAULT_VERSION" == "riak-head" ]; then - DEFAULT_VERSION = "riak_ee-head" - fi + build "riak-1.3.2" $R15B01 + build "riak-1.4.12" $R15B01 else - build "riak_ee-1.3.4" $R15B01 - build "riak_ee-1.4.12" $R15B01 + build "riak_ee-1.3.4" $R15B01 + build "riak_ee-1.4.12" $R15B01 + if [ "${DEFAULT_VERSION}" == "riak-head" ]; then + DEFAULT_VERSION="riak_ee-head" + fi + echo "Default version: $DEFAULT_VERSION" fi -build "$DEFAUlT_VERSION" $R16B02 +build $DEFAULT_VERSION $R16B02 echo echo "= Build complete! ===============================================" diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index 0edb8738b..c696c6861 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -1,7 +1,7 @@ #!/bin/bash # Bail out on error ... -set -e +#set -e if [ -z $1 ]; then echo "An r_t configuration is required as the first parameter" @@ -15,12 +15,13 @@ ELEVELDB_BACKEND_TEST_CASES="$ALL_BACKEND_TEST_CASES,verify_2i_aae" MEMORY_BACKEND_TEST_CASES="$ALL_BACKEND_TEST_CASES,verify_2i_aae" ROOT_RESULTS_DIR="results/regression" -RESULTS_DIR="$ROOT_RESULTS_DIR/`date +"%y%m%d%H%M%s"`" +RESULTS=`date +"%y%m%d%H%M%s"` +RESULTS_DIR="$ROOT_RESULTS_DIR/$RESULTS" mkdir -p $RESULTS_DIR RESULTS_SYMLINK=$ROOT_RESULTS_DIR/current rm -f $RESULTS_SYMLINK -ln -s $RESULTS_DIR $RESULTS_SYMLINK +ln -s $RESULTS $RESULTS_SYMLINK echo "Running bitcask regression tests using the following test cases: $BITCASK_BACKEND_TEST_CASES" ./riak_test -v -c $1 -t $BITCASK_BACKEND_TEST_CASES &> $RESULTS_DIR/bitcask_results.log From 3c90be1973ddca58e11852b80d08e5025edcada2 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Thu, 19 Mar 2015 15:13:18 -0400 Subject: [PATCH 104/157] WIP: Initial draft of a Riak cluster FSM and associated API to replace the cluster functions in the rt, rt2, and rt_cluster modules --- src/rt_riak_cluster.erl | 323 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 323 insertions(+) create mode 100644 src/rt_riak_cluster.erl diff --git a/src/rt_riak_cluster.erl b/src/rt_riak_cluster.erl new file mode 100644 index 000000000..cce6db34b --- /dev/null +++ b/src/rt_riak_cluster.erl @@ -0,0 +1,323 @@ +%%%------------------------------------------------------------------- +%%% @author John Burwell <> +%%% @copyright (C) 2015, John Burwell +%%% @doc +%%% +%%% @end +%%% Created : 19 Mar 2015 by John Burwell <> +%%%------------------------------------------------------------------- +-module(rt_riak_cluster). + +-behaviour(gen_fsm). + +%% API +-export([activate_bucket_type/2, + decommission/1, + clean/1, + create_and_activate_bucket_type/3, + create_bucket_type/3, + is_mixed/1, + join/2, + leave/2, + nodes/1, + partition/3, + provision/4, + staged_join/2, + start/1, + stop/1, + wait_until_all_members/1, + wait_until_connected/1, + wait_until_legacy_ring_ready/1, + wait_until_no_pending_changes/1, + wait_until_nodes_agree_about_ownership/1, + wait_until_ring_converged/1, + upgrade/2]). + +%% gen_fsm callbacks +-export([init/1, state_name/2, state_name/3, handle_event/3, + handle_sync_event/4, handle_info/3, terminate/3, code_change/4]). + +-define(SERVER, ?MODULE). + +-record(state, {name :: cluster_name(), + bucket_types :: [atom()], %% should this be a string? + indexes :: [atom()], %% should this be a string? + nodes :: [node()], + version :: string() + }). + +%%%=================================================================== +%%% API +%%%=================================================================== +-type cluster_name() :: atom(). +-type cluster_id() :: pid(). +-type error() :: {error(), term()}. +%% -type partition() :: [node()]. +-type result() :: ok | error(). + +%% TODO Move to the rt_version module +%% TODO Split out the product type +-type version() :: {atom(), string()}. + +-spec activate_bucket_type(cluster_id(), atom()) -> result(). +activate_bucket_type(Cluster, BucketType) -> + lager:error("activate_bucket_type(~p, ~p, ~p) not implemented", [Cluster, BucketType]), + ok. + +-spec decommission(cluster_id()) -> result(). +decommission(Cluster) -> + lager:error("decomission(~p) not implemented", [Cluster]), + ok. + +-spec clean(cluster_id()) -> result(). +clean(Cluster) -> + lager:error("clean(~p) not implemented", [Cluster]), + ok. + +-spec create_and_activate_bucket_type(cluster_id(), atom(), proplists:proplist()) -> result(). +create_and_activate_bucket_type(Cluster, BucketType, Parameters) -> + ok = create_bucket_type(Cluster, BucketType, Parameters), + activate_bucket_type(Cluster, BucketType). + +-spec create_bucket_type(cluster_id(), atom(), proplists:proplist()) -> result(). +create_bucket_type(Cluster, BucketType, Parameters) -> + lager:error("create_bucket_type(~p, ~p, ~p) not implemented", [Cluster, BucketType, Parameters]), + ok. + +-spec is_mixed(cluster_id()) -> boolean(). +is_mixed(Cluster) -> + lager:error("is_mixed(~p) not implemented", [Cluster]), + ok. + +-spec join(cluster_id(), node()) -> result(). +join(Cluster, Node) -> + lager:error("join(~p, ~p) not implemented.", [Cluster, Node]), + ok. + +-spec leave(cluster_id(), node()) -> result(). +leave(Cluster, Node) -> + lager:error("leave(~p, ~p) not implemented.", [Cluster, Node]), + ok. + +-spec nodes(cluster_id()) -> [node()]. +nodes(Cluster) -> + lager:error("nodes(~p) not implemented", [Cluster]), + ok. + +%% -spec partition(cluster_id(), partition(), partition()) -> [atom(), atom(), partition(), partition()] | error(). +partition(Cluster, P1, P2) -> + lager:error("partition(~p, ~p, ~p) not implemented.", [Cluster, P1, P2]), + ok. + +-spec provision(cluster_name(), [node()], proplists:proplist(), proplists:proplist()) -> {cluster_name(), cluster_id()} | error(). +provision(Name, Nodes, Conf, AdvancedConfig) -> + lager:error("provision(~p, ~p, ~p, ~p) not implemented.", [Name, Nodes, Conf, AdvancedConfig]), + ok. + +-spec staged_join(cluster_id(), node()) -> result(). +staged_join(Cluster, Node) -> + lager:error("stagad_join(~p, ~p) not implemented.", [Cluster, Node]), + ok. + +-spec start(cluster_id()) -> result(). +start(Cluster) -> + lager:error("start(~p) not implemented", [Cluster]), + ok. + +-spec stop(cluster_id()) -> result(). +stop(Cluster) -> + lager:error("stop(~p) not implemented", [Cluster]), + ok. + +-spec wait_until_all_members([node()]) -> result(). +wait_until_all_members(Nodes) -> + lager:error("wait_until_all_members(~p) not implemented", [Nodes]), + ok. + +-spec wait_until_connected(cluster_id()) -> result(). +wait_until_connected(Cluster) -> + lager:error("wait_until_connected(~p) not implemented", [Cluster]), + ok. + +-spec wait_until_legacy_ring_ready(cluster_id()) -> result(). +wait_until_legacy_ring_ready(Cluster) -> + lager:error("wait_until_legacy_ring_ready(~p) not implemented", [Cluster]), + ok. + +-spec wait_until_no_pending_changes(cluster_id()) -> result(). +wait_until_no_pending_changes(Cluster) -> + lager:error("wait_until_no_pending_changes(~p) not implemented", [Cluster]), + ok. + +-spec wait_until_nodes_agree_about_ownership(cluster_id()) -> result(). +wait_until_nodes_agree_about_ownership(Cluster) -> + lager:error("wait_until_nodes_agree_about_ownership(~p) not implemented", [Cluster]), + ok. + +-spec wait_until_ring_converged(cluster_id()) -> result(). +wait_until_ring_converged(Cluster) -> + lager:error("wait_until_ring_converged(~p) not implemented", [Cluster]), + ok. + +-spec upgrade(cluster_id(), version()) -> result(). +upgrade(Cluster, Version) -> + lager:error("upgrade(~p, ~p) not implemented.", [Cluster, Version]), + ok. + +%% index creation ... + +%% security: users/acls, etc + +%%-------------------------------------------------------------------- +%% @doc +%% Creates a gen_fsm process which calls Module:init/1 to +%% initialize. To ensure a synchronized start-up procedure, this +%% function does not return until Module:init/1 has returned. +%% +%% @spec start_link() -> {ok, Pid} | ignore | {error, Error} +%% @end +%%-------------------------------------------------------------------- +%%start_link() -> +%% gen_fsm:start_link({local, ?SERVER}, ?MODULE, [], []). + +%%%=================================================================== +%%% gen_fsm callbacks +%%%=================================================================== + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Whenever a gen_fsm is started using gen_fsm:start/[3,4] or +%% gen_fsm:start_link/[3,4], this function is called by the new +%% process to initialize. +%% +%% @spec init(Args) -> {ok, StateName, State} | +%% {ok, StateName, State, Timeout} | +%% ignore | +%% {stop, StopReason} +%% @end +%%-------------------------------------------------------------------- +init([]) -> + {ok, state_name, #state{}}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% There should be one instance of this function for each possible +%% state name. Whenever a gen_fsm receives an event sent using +%% gen_fsm:send_event/2, the instance of this function with the same +%% name as the current state name StateName is called to handle +%% the event. It is also called if a timeout occurs. +%% +%% @spec state_name(Event, State) -> +%% {next_state, NextStateName, NextState} | +%% {next_state, NextStateName, NextState, Timeout} | +%% {stop, Reason, NewState} +%% @end +%%-------------------------------------------------------------------- +state_name(_Event, State) -> + {next_state, state_name, State}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% There should be one instance of this function for each possible +%% state name. Whenever a gen_fsm receives an event sent using +%% gen_fsm:sync_send_event/[2,3], the instance of this function with +%% the same name as the current state name StateName is called to +%% handle the event. +%% +%% @spec state_name(Event, From, State) -> +%% {next_state, NextStateName, NextState} | +%% {next_state, NextStateName, NextState, Timeout} | +%% {reply, Reply, NextStateName, NextState} | +%% {reply, Reply, NextStateName, NextState, Timeout} | +%% {stop, Reason, NewState} | +%% {stop, Reason, Reply, NewState} +%% @end +%%-------------------------------------------------------------------- +state_name(_Event, _From, State) -> + Reply = ok, + {reply, Reply, state_name, State}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Whenever a gen_fsm receives an event sent using +%% gen_fsm:send_all_state_event/2, this function is called to handle +%% the event. +%% +%% @spec handle_event(Event, StateName, State) -> +%% {next_state, NextStateName, NextState} | +%% {next_state, NextStateName, NextState, Timeout} | +%% {stop, Reason, NewState} +%% @end +%%-------------------------------------------------------------------- +handle_event(_Event, StateName, State) -> + {next_state, StateName, State}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Whenever a gen_fsm receives an event sent using +%% gen_fsm:sync_send_all_state_event/[2,3], this function is called +%% to handle the event. +%% +%% @spec handle_sync_event(Event, From, StateName, State) -> +%% {next_state, NextStateName, NextState} | +%% {next_state, NextStateName, NextState, Timeout} | +%% {reply, Reply, NextStateName, NextState} | +%% {reply, Reply, NextStateName, NextState, Timeout} | +%% {stop, Reason, NewState} | +%% {stop, Reason, Reply, NewState} +%% @end +%%-------------------------------------------------------------------- +handle_sync_event(_Event, _From, StateName, State) -> + Reply = ok, + {reply, Reply, StateName, State}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% This function is called by a gen_fsm when it receives any +%% message other than a synchronous or asynchronous event +%% (or a system message). +%% +%% @spec handle_info(Info,StateName,State)-> +%% {next_state, NextStateName, NextState} | +%% {next_state, NextStateName, NextState, Timeout} | +%% {stop, Reason, NewState} +%% @end +%%-------------------------------------------------------------------- +handle_info(_Info, StateName, State) -> + {next_state, StateName, State}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% This function is called by a gen_fsm when it is about to +%% terminate. It should be the opposite of Module:init/1 and do any +%% necessary cleaning up. When it returns, the gen_fsm terminates with +%% Reason. The return value is ignored. +%% +%% @spec terminate(Reason, StateName, State) -> void() +%% @end +%%-------------------------------------------------------------------- +terminate(_Reason, _StateName, _State) -> + ok. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Convert process state when code is changed +%% +%% @spec code_change(OldVsn, StateName, State, Extra) -> +%% {ok, StateName, NewState} +%% @end +%%-------------------------------------------------------------------- +code_change(_OldVsn, StateName, State, _Extra) -> + {ok, StateName, State}. + +%%%=================================================================== +%%% Internal functions +%%%=================================================================== From c4882dcc5f4acb74ac50685982e2718c0eea44a1 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Thu, 19 Mar 2015 19:26:08 -0400 Subject: [PATCH 105/157] verify_handoff now passes using the v1 API wrapper * Fixes rt:setup_harness to properly call rt_harness:setup rather than rt2:setup_harness which is private --- regression_test_wrapper.sh | 2 +- src/rt.erl | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index c696c6861..de938dcb7 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -8,7 +8,7 @@ if [ -z $1 ]; then exit 1 fi -ALL_BACKEND_TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantine,gh_riak_core_155,pb_security,verify_search" +ALL_BACKEND_TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantine,gh_riak_core_155,pb_security,verify_search,verify_handoff" BITCASK_BACKEND_TEST_CASES="$ALL_BACKEND_TEST_CASES" ELEVELDB_BACKEND_TEST_CASES="$ALL_BACKEND_TEST_CASES,verify_2i_aae" diff --git a/src/rt.erl b/src/rt.erl index 1ac3f063e..62404fdd7 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -1399,8 +1399,8 @@ log_to_nodes(Nodes0, LFmt, LArgs) -> pmap(F, L) -> rt2:pmap(F, L). -setup_harness(Test, Args) -> - rt2:setup_harness(Test, Args). +setup_harness(_Test, _Args) -> + rt_harness:setup(). %% @doc Downloads any extant log files from the harness's running %% nodes. From 9cc0f6febf625067f54fd72586957c51ebe05f0e Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Fri, 20 Mar 2015 17:15:14 -0600 Subject: [PATCH 106/157] Fix overload test for rebase --- tests/overload.erl | 31 ++++++++++++------------------- 1 file changed, 12 insertions(+), 19 deletions(-) diff --git a/tests/overload.erl b/tests/overload.erl index c1d27cbc9..1e473f73b 100644 --- a/tests/overload.erl +++ b/tests/overload.erl @@ -72,24 +72,17 @@ setup() -> {vnode_overload_threshold, undefined}]}, {riak_kv, [{fsm_limit, undefined}, {storage_backend, riak_kv_memory_backend}, - {anti_entropy, {off, []}}]}], - Nodes = rt_cluster:build_cluster(2, Config), - [_Node1, Node2] = Nodes, - - Ring = rt_ring:get_ring(Node2), - Hash = riak_core_util:chash_std_keyfun({?BUCKET, ?KEY}), - PL = lists:sublist(riak_core_ring:preflist(Hash, Ring), 3), - Victim = hd([Idx || {Idx, Node} <- PL, - Node =:= Node2]), - RO = riak_object:new(?BUCKET, ?KEY, <<"test">>), - - %% TODO commented out to get merge completed .. must be reconciled -- jsb - %% ok = test_no_overload_protection(Nodes, BKV), - ok = test_vnode_protection(Nodes, Victim, RO), - ok = test_fsm_protection(Nodes, Victim, RO), - pass. - -test_no_overload_protection(Nodes, Victim, RO) -> + {anti_entropy_build_limit, {100, 1000}}, + {anti_entropy_concurrency, 100}, + {anti_entropy_tick, 100}, + {anti_entropy, {on, []}}, + {anti_entropy_timeout, 5000}]}, + {riak_api, [{pb_backlog, 1024}]}], + ensemble_util:build_cluster(5, Config, 5). + +test_no_overload_protection(_Nodes, _BKV, true) -> + ok; +test_no_overload_protection(Nodes, BKV, ConsistentType) -> lager:info("Testing with no overload protection"), ProcFun = build_predicate_gte(test_no_overload_protection, ?NUM_REQUESTS, "ProcFun", "Procs"), @@ -505,4 +498,4 @@ build_predicate_lt(Test, Metric, Label, ValueLabel) -> lager:info("in test ~p ~p, ~p:~p, expected overload, Metric:<~p", [Test, Label, ValueLabel, X, Metric]), X < Metric - end. \ No newline at end of file + end. From 5b79d58f168eeed02075b5179766eec1af69386b Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Mon, 23 Mar 2015 15:13:34 -0600 Subject: [PATCH 107/157] Add verify_capabilities to regression suite --- regression_test_wrapper.sh | 2 +- src/rt.erl | 12 +++++++----- src/rt_cluster.erl | 1 + src/rt_harness.erl | 4 ++++ src/rtdev.erl | 29 +++++++++++++++++++++-------- 5 files changed, 34 insertions(+), 14 deletions(-) diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index de938dcb7..1c41c9030 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -8,7 +8,7 @@ if [ -z $1 ]; then exit 1 fi -ALL_BACKEND_TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantine,gh_riak_core_155,pb_security,verify_search,verify_handoff" +ALL_BACKEND_TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantine,gh_riak_core_155,pb_security,verify_search,verify_handoff,verify_capabilities" BITCASK_BACKEND_TEST_CASES="$ALL_BACKEND_TEST_CASES" ELEVELDB_BACKEND_TEST_CASES="$ALL_BACKEND_TEST_CASES,verify_2i_aae" diff --git a/src/rt.erl b/src/rt.erl index 62404fdd7..75d2027b5 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -243,8 +243,9 @@ deploy_nodes(NumNodes) when is_integer(NumNodes) -> deploy_nodes(NumNodes, InitialConfig) when is_integer(NumNodes) -> deploy_nodes(NumNodes, InitialConfig, [riak_kv]); deploy_nodes(Versions, Services) -> - NodeConfig = [ version_to_config(Version) || Version <- Versions ], - lager:debug("Starting nodes using config ~p and versions ~p", [NodeConfig, Versions]), + MappedVersions = [rt_config:version_to_tag(Vsn) || Vsn <- Versions], + NodeConfig = [ {Version, rt_properties:default_config()} || Version <- MappedVersions ], + lager:debug("Starting nodes using config ~p and versions ~p", [NodeConfig, MappedVersions]), Nodes = rt_harness:deploy_nodes(NodeConfig), lager:info("Waiting for services ~p to start on ~p.", [Services, Nodes]), @@ -321,7 +322,7 @@ allocate_nodes(NumNodes, Version) -> [AllocatedNodeIds, AllocatedNodeMap, VersionMap]. version_to_config(Config) when is_tuple(Config)-> Config; -version_to_config(Version) -> {Version, rt_config:get_default_version()}. +version_to_config(Version) -> {Version, default}. deploy_clusters(Settings) -> ClusterConfigs = [case Setting of @@ -373,7 +374,7 @@ stop_and_wait(Node) -> %% @doc Upgrade a Riak `Node' to the specified `NewVersion'. upgrade(Node, NewVersion) -> %% GAP: The new API does not provide an analog to this function. -jsb - ?HARNESS:upgrade(Node, NewVersion). + rt_harness:upgrade(Node, NewVersion). %% @doc Upgrade a Riak `Node' to the specified `NewVersion' and update %% the config based on entries in `Config'. @@ -785,8 +786,9 @@ clean_data_dir(Nodes, SubDir) when is_list(Nodes) -> teardown() -> rt_cluster:teardown(). +%% TODO: Only used in verify_capabalities. Probably should be refactored. versions() -> - rt_cluster:versions(). + proplists:get_keys(rt_config:get(versions)). %%%=================================================================== %%% Basic Read/Write Functions diff --git a/src/rt_cluster.erl b/src/rt_cluster.erl index a0e98f3fd..39b7b8d9f 100644 --- a/src/rt_cluster.erl +++ b/src/rt_cluster.erl @@ -271,6 +271,7 @@ teardown() -> %% but not connected. rt_harness:teardown(). +%% TODO: Determine if this is used outside of verify_capabilities versions() -> rt_harness:versions(). diff --git a/src/rt_harness.erl b/src/rt_harness.erl index 644abf20f..0923c6da0 100644 --- a/src/rt_harness.erl +++ b/src/rt_harness.erl @@ -57,6 +57,7 @@ set_conf/2, set_advanced_conf/2, update_app_config/2, + upgrade/2, validate_config/1]). start(Node, Version) -> @@ -155,5 +156,8 @@ set_advanced_conf(Node, NameValuePairs) -> update_app_config(Node, Config) -> ?HARNESS_MODULE:update_app_config(Node, Config). +upgrade(Node, NewVersion) -> + ?HARNESS_MODULE:upgrade(Node, NewVersion). + validate_config(Versions) -> ?HARNESS_MODULE:validate_config(Versions). diff --git a/src/rtdev.erl b/src/rtdev.erl index 2692d171f..0fe2c5b0f 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -230,6 +230,13 @@ relpath(Vsn) -> relpath(Vsn, Path) -> lists:concat([Path, "/", Vsn]). +%% TODO: Need to replace without the node_version/1 map +upgrade(Node, NewVersion) -> + N = node_id(Node), + CurrentVersion = node_version(N), + UpgradedVersion = rt_config:version_to_tag(NewVersion), + upgrade(Node, CurrentVersion, UpgradedVersion, same). + %% upgrade(Node, CurrentVersion, NewVersion) -> %% upgrade(Node, CurrentVersion, NewVersion, same). @@ -237,14 +244,14 @@ upgrade(Node, CurrentVersion, NewVersion, Config) -> lager:info("Upgrading ~p : ~p -> ~p", [Node, CurrentVersion, NewVersion]), stop(Node, CurrentVersion), rt:wait_until_unpingable(Node), - CurrentPath = filename:join([?PATH, CurrentVersion, Node]), - NewPath = filename:join([?PATH, NewVersion, Node]), + CurrentPath = filename:join([?PATH, CurrentVersion, node_short_name(Node)]), + NewPath = filename:join([?PATH, NewVersion, node_short_name(Node)]), Commands = [ io_lib:format("cp -p -P -R \"~s\" \"~s\"", [filename:join(CurrentPath, "data"), NewPath]), - %% io_lib:format("rm -rf ~s/dev~b/data/*", - %% [CurrePath, N]), + %% io_lib:format("rm -rf ~s*", + %% [filename:join([CurrentPath, "data", "*"])]), io_lib:format("cp -p -P -R \"~s\" \"~s\"", [filename:join(CurrentPath, "etc"), NewPath]) @@ -253,10 +260,11 @@ upgrade(Node, CurrentVersion, NewVersion, Config) -> lager:debug("Running: ~s", [Cmd]), os:cmd(Cmd) end || Cmd <- Commands], - clean_data_dir(Node, CurrentVersion, ""), + clean_data_dir(node_short_name(Node), CurrentVersion, ""), - %% VersionMap = orddict:store(N, NewVersion, rt_config:get(rt_versions)), - %% rt_config:set(rt_versions, VersionMap), + %% TODO: This actually is required by old framework + VersionMap = orddict:store(Node, NewVersion, rt_config:get(rt_versions)), + rt_config:set(rt_versions, VersionMap), case Config of same -> ok; @@ -841,7 +849,12 @@ node_id(Node) -> %% @doc Find the short dev node name from the full name -spec node_short_name(atom() | list()) -> atom(). node_short_name(Node) when is_list(Node) -> - Node; + case lists:member($@, Node) of + true -> + node_short_name(list_to_atom(Node)); + _ -> + Node + end; node_short_name(Node) when is_atom(Node) -> NodeMap = rt_config:get(rt_nodenames), orddict:fetch(Node, NodeMap). From 7684412cc7f2ff98a8ce96e8a59c037dfafb887e Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Tue, 24 Mar 2015 15:29:25 -0600 Subject: [PATCH 108/157] Add support for loaded_upgrade test - Look up versions based on name as well as alias - Automatically unzip the SPAM samples in search-corpus - Run tests by default in bitcask; not on all 3 backends --- Makefile | 10 ++++++++-- README.md | 5 ++++- examples/riak_test.config | 8 ++++++-- regression_test_wrapper.sh | 6 +++--- src/rt.erl | 12 ++++++++---- src/rt_config.erl | 11 +++++++---- 6 files changed, 36 insertions(+), 16 deletions(-) diff --git a/Makefile b/Makefile index 21545e640..d08a44f5b 100644 --- a/Makefile +++ b/Makefile @@ -1,6 +1,6 @@ .PHONY: deps -all: deps compile +all: deps compile testcases ./rebar skip_deps=true escriptize SMOKE_TEST=1 ./rebar skip_deps=true escriptize @@ -13,7 +13,7 @@ docsclean: compile: deps ./rebar compile -clean: +clean: clean_testcases @./rebar clean distclean: clean @@ -23,6 +23,12 @@ quickbuild: ./rebar skip_deps=true compile ./rebar escriptize +testcases: + @(cd search-corpus; tar fx spam.0.1.tar.gz) + +clean_testcases: + @rm -rf search-corpus/spam.0/ + ################## # Dialyzer targets ################## diff --git a/README.md b/README.md index 8363d43d3..8fc596038 100644 --- a/README.md +++ b/README.md @@ -182,7 +182,10 @@ to tell riak_test about them. The method of choice is to create a {rt_scratch_dir, "/tmp/riak_test_scratch"}, {basho_bench, "/home/you/basho/basho_bench"}, {spam_dir, "/home/you/basho/riak_test/search-corpus"}, - {platform, "osx-64"} + {platform, "osx-64"}, + {load_workers, 3}, + {offset, 2}, + {workers, 5} ]}. {rtdev, [ diff --git a/examples/riak_test.config b/examples/riak_test.config index a1c6e9144..82a245761 100644 --- a/examples/riak_test.config +++ b/examples/riak_test.config @@ -64,12 +64,16 @@ %% `loaded_upgrade' test. If unspecified, this will default to %% `10'. For older/slower machines, use a lower number to avoid %% unexpected node crashes. - {load_workers, 10}, + {load_workers, 3}, %% lager_level defaults to info, which is should mean %% "relevant test output". debug level output is for helping %% test writers. - {lager_level, info} + {lager_level, info}, + + %% Number of processes to run during testing + {workers, 5}, + {offset, 2} ]}. %% =============================================================== diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index 1c41c9030..caa999c58 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -10,9 +10,9 @@ fi ALL_BACKEND_TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantine,gh_riak_core_155,pb_security,verify_search,verify_handoff,verify_capabilities" -BITCASK_BACKEND_TEST_CASES="$ALL_BACKEND_TEST_CASES" -ELEVELDB_BACKEND_TEST_CASES="$ALL_BACKEND_TEST_CASES,verify_2i_aae" -MEMORY_BACKEND_TEST_CASES="$ALL_BACKEND_TEST_CASES,verify_2i_aae" +BITCASK_BACKEND_TEST_CASES="$ALL_BACKEND_TEST_CASES,loaded_upgrade" +ELEVELDB_BACKEND_TEST_CASES="verify_2i_aae,loaded_upgrade" +MEMORY_BACKEND_TEST_CASES="verify_2i_aae" ROOT_RESULTS_DIR="results/regression" RESULTS=`date +"%y%m%d%H%M%s"` diff --git a/src/rt.erl b/src/rt.erl index 75d2027b5..b2ca344d6 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -243,11 +243,10 @@ deploy_nodes(NumNodes) when is_integer(NumNodes) -> deploy_nodes(NumNodes, InitialConfig) when is_integer(NumNodes) -> deploy_nodes(NumNodes, InitialConfig, [riak_kv]); deploy_nodes(Versions, Services) -> - MappedVersions = [rt_config:version_to_tag(Vsn) || Vsn <- Versions], - NodeConfig = [ {Version, rt_properties:default_config()} || Version <- MappedVersions ], - lager:debug("Starting nodes using config ~p and versions ~p", [NodeConfig, MappedVersions]), + MappedVersions = [map_version_and_config(Vsn) || Vsn <- Versions], + lager:debug("Starting nodes using config and versions ~p", [MappedVersions]), - Nodes = rt_harness:deploy_nodes(NodeConfig), + Nodes = rt_harness:deploy_nodes(MappedVersions), lager:info("Waiting for services ~p to start on ~p.", [Services, Nodes]), [ ok = wait_for_service(Node, Service) || Node <- Nodes, Service <- Services ], @@ -266,6 +265,11 @@ deploy_nodes(NodeIds, NodeMap, Version, Config, Services) -> [], NodeMap). +map_version_and_config({Vsn, Cfg}) -> + {rt_config:version_to_tag(Vsn), Cfg}; +map_version_and_config(Vsn) -> + {rt_config:version_to_tag(Vsn), rt_properties:default_config()}. + maybe_set_backend(Backend, Config) -> maybe_set_backend(lists:keyfind(riak_kv, 1, Config), Backend, Config). diff --git a/src/rt_config.erl b/src/rt_config.erl index 277b3b442..7f5e71108 100644 --- a/src/rt_config.erl +++ b/src/rt_config.erl @@ -32,6 +32,7 @@ get_os_env/1, get_os_env/2, get_upgrade_path/1, + get_version/1, load/2, set/2, set_conf/2, @@ -141,12 +142,14 @@ get_previous_version() -> get_legacy_version() -> get_version(?LEGACY_VERSION_KEY). -%% @doc Prepends the project onto the default version -%% e.g. "riak_ee-3.0.1" or "riak-head" +%% @doc Looks up the version in the list of aliases (e.g. riak_ee-latest-2.0) +%% and actual version names (riak_ee-2.0.5) and return the actual +%% name - string, e.g. "riak_ee-1.4.12" -spec get_version(term()) -> string() | not_found. get_version(Vsn) -> - Versions = rt_config:get(?VERSION_KEY), - resolve_version(Vsn, Versions). + Aliases = rt_config:get(?VERSION_KEY), + RealNames = [{convert_to_string(Product) ++ "-" ++ convert_to_string(Tag), {Product, Tag}} || {_, {Product, Tag}} <- Aliases], + resolve_version(Vsn, lists:append(Aliases, RealNames)). %% @doc Map logical name of version into a pathname string -spec resolve_version(term(), [{term(), term()}]) -> string() | no_return(). From 74d2e9b0c07fd474ee9f64077fa2d5a792bdd5de Mon Sep 17 00:00:00 2001 From: John Burwell Date: Fri, 20 Mar 2015 19:46:46 -0400 Subject: [PATCH 109/157] replication2, overload, and repl_fs_stat_caching pass using the v1 API wrapper * Adds the riak_repl function back to rt_harness * Modifies rtdev:riak_repl to properly determine bin path * Adds replication2, overload, and repl_fs_stat_caching to the regression test script --- regression_test_wrapper.sh | 2 +- src/rt_harness.erl | 4 ++++ src/rtdev.erl | 5 +++-- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index caa999c58..090384240 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -8,7 +8,7 @@ if [ -z $1 ]; then exit 1 fi -ALL_BACKEND_TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantine,gh_riak_core_155,pb_security,verify_search,verify_handoff,verify_capabilities" +ALL_BACKEND_TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantine,gh_riak_core_155,pb_security,verify_search,verify_handoff,verify_capabilities,replication/repl_fs_stat_caching,replication/replication2,overload" BITCASK_BACKEND_TEST_CASES="$ALL_BACKEND_TEST_CASES,loaded_upgrade" ELEVELDB_BACKEND_TEST_CASES="verify_2i_aae,loaded_upgrade" diff --git a/src/rt_harness.erl b/src/rt_harness.erl index 0923c6da0..0001a3fc5 100644 --- a/src/rt_harness.erl +++ b/src/rt_harness.erl @@ -48,6 +48,7 @@ admin/2, admin/3, riak/2, + riak_repl/2, run_riak/3, attach/2, attach_direct/2, @@ -129,6 +130,9 @@ admin(Node, Args, Options) -> riak(Node, Args) -> ?HARNESS_MODULE:riak(Node, Args). +riak_repl(Node, Args) -> + ?HARNESS_MODULE:riak_repl(Node, Args). + run_riak(Node, Version, Command) -> ?HARNESS_MODULE:run_riak(Node, Version, Command). diff --git a/src/rtdev.erl b/src/rtdev.erl index 0fe2c5b0f..1af22116d 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -835,8 +835,9 @@ riak(Node, Args) -> riak_repl(Node, Args) -> - Path = relpath(node_version(Node)), - Result = run_riak_repl(Node, Path, Args), + {NodeId, NodeName} = extract_node_id_and_name(Node), + Path = relpath(node_version(NodeName)), + Result = run_riak_repl(NodeId, Path, Args), lager:info("~s", [Result]), {ok, Result}. From f9b9ab72b13c6a6b18cdbb7aed86ba7a3c0eb4ea Mon Sep 17 00:00:00 2001 From: John Burwell Date: Fri, 20 Mar 2015 19:50:10 -0400 Subject: [PATCH 110/157] - v2 API Definition: Initial work on the node API --- src/rt_riak_cluster.erl | 40 ++--- src/rt_riak_node.erl | 382 ++++++++++++++++++++++++++++++++++++++++ src/rt_util.erl | 7 + 3 files changed, 408 insertions(+), 21 deletions(-) create mode 100644 src/rt_riak_node.erl create mode 100644 src/rt_util.erl diff --git a/src/rt_riak_cluster.erl b/src/rt_riak_cluster.erl index cce6db34b..1c3c514ca 100644 --- a/src/rt_riak_cluster.erl +++ b/src/rt_riak_cluster.erl @@ -51,35 +51,33 @@ %%%=================================================================== -type cluster_name() :: atom(). -type cluster_id() :: pid(). --type error() :: {error(), term()}. %% -type partition() :: [node()]. --type result() :: ok | error(). %% TODO Move to the rt_version module %% TODO Split out the product type -type version() :: {atom(), string()}. --spec activate_bucket_type(cluster_id(), atom()) -> result(). +-spec activate_bucket_type(cluster_id(), atom()) -> rt_util:result(). activate_bucket_type(Cluster, BucketType) -> lager:error("activate_bucket_type(~p, ~p, ~p) not implemented", [Cluster, BucketType]), ok. --spec decommission(cluster_id()) -> result(). +-spec decommission(cluster_id()) -> rt_util:result(). decommission(Cluster) -> lager:error("decomission(~p) not implemented", [Cluster]), ok. --spec clean(cluster_id()) -> result(). +-spec clean(cluster_id()) -> rt_util:result(). clean(Cluster) -> lager:error("clean(~p) not implemented", [Cluster]), ok. --spec create_and_activate_bucket_type(cluster_id(), atom(), proplists:proplist()) -> result(). +-spec create_and_activate_bucket_type(cluster_id(), atom(), proplists:proplist()) -> rt_util:result(). create_and_activate_bucket_type(Cluster, BucketType, Parameters) -> ok = create_bucket_type(Cluster, BucketType, Parameters), activate_bucket_type(Cluster, BucketType). --spec create_bucket_type(cluster_id(), atom(), proplists:proplist()) -> result(). +-spec create_bucket_type(cluster_id(), atom(), proplists:proplist()) -> rt_util:result(). create_bucket_type(Cluster, BucketType, Parameters) -> lager:error("create_bucket_type(~p, ~p, ~p) not implemented", [Cluster, BucketType, Parameters]), ok. @@ -89,12 +87,12 @@ is_mixed(Cluster) -> lager:error("is_mixed(~p) not implemented", [Cluster]), ok. --spec join(cluster_id(), node()) -> result(). +-spec join(cluster_id(), node()) -> rt_util:result(). join(Cluster, Node) -> lager:error("join(~p, ~p) not implemented.", [Cluster, Node]), ok. --spec leave(cluster_id(), node()) -> result(). +-spec leave(cluster_id(), node()) -> rt_util:result(). leave(Cluster, Node) -> lager:error("leave(~p, ~p) not implemented.", [Cluster, Node]), ok. @@ -104,62 +102,62 @@ nodes(Cluster) -> lager:error("nodes(~p) not implemented", [Cluster]), ok. -%% -spec partition(cluster_id(), partition(), partition()) -> [atom(), atom(), partition(), partition()] | error(). +%% -spec partition(cluster_id(), partition(), partition()) -> [atom(), atom(), partition(), partition()] | rt_util:error(). partition(Cluster, P1, P2) -> lager:error("partition(~p, ~p, ~p) not implemented.", [Cluster, P1, P2]), ok. --spec provision(cluster_name(), [node()], proplists:proplist(), proplists:proplist()) -> {cluster_name(), cluster_id()} | error(). +-spec provision(cluster_name(), [node()], proplists:proplist(), proplists:proplist()) -> {cluster_name(), cluster_id()} | rt_util:error(). provision(Name, Nodes, Conf, AdvancedConfig) -> lager:error("provision(~p, ~p, ~p, ~p) not implemented.", [Name, Nodes, Conf, AdvancedConfig]), ok. --spec staged_join(cluster_id(), node()) -> result(). +-spec staged_join(cluster_id(), node()) -> rt_util:result(). staged_join(Cluster, Node) -> lager:error("stagad_join(~p, ~p) not implemented.", [Cluster, Node]), ok. --spec start(cluster_id()) -> result(). +-spec start(cluster_id()) -> rt_util:result(). start(Cluster) -> lager:error("start(~p) not implemented", [Cluster]), ok. --spec stop(cluster_id()) -> result(). +-spec stop(cluster_id()) -> rt_util:result(). stop(Cluster) -> lager:error("stop(~p) not implemented", [Cluster]), ok. --spec wait_until_all_members([node()]) -> result(). +-spec wait_until_all_members([node()]) -> rt_util:result(). wait_until_all_members(Nodes) -> lager:error("wait_until_all_members(~p) not implemented", [Nodes]), ok. --spec wait_until_connected(cluster_id()) -> result(). +-spec wait_until_connected(cluster_id()) -> rt_util:result(). wait_until_connected(Cluster) -> lager:error("wait_until_connected(~p) not implemented", [Cluster]), ok. --spec wait_until_legacy_ring_ready(cluster_id()) -> result(). +-spec wait_until_legacy_ring_ready(cluster_id()) -> rt_util:result(). wait_until_legacy_ring_ready(Cluster) -> lager:error("wait_until_legacy_ring_ready(~p) not implemented", [Cluster]), ok. --spec wait_until_no_pending_changes(cluster_id()) -> result(). +-spec wait_until_no_pending_changes(cluster_id()) -> rt_util:result(). wait_until_no_pending_changes(Cluster) -> lager:error("wait_until_no_pending_changes(~p) not implemented", [Cluster]), ok. --spec wait_until_nodes_agree_about_ownership(cluster_id()) -> result(). +-spec wait_until_nodes_agree_about_ownership(cluster_id()) -> rt_util:result(). wait_until_nodes_agree_about_ownership(Cluster) -> lager:error("wait_until_nodes_agree_about_ownership(~p) not implemented", [Cluster]), ok. --spec wait_until_ring_converged(cluster_id()) -> result(). +-spec wait_until_ring_converged(cluster_id()) -> rt_util:result(). wait_until_ring_converged(Cluster) -> lager:error("wait_until_ring_converged(~p) not implemented", [Cluster]), ok. --spec upgrade(cluster_id(), version()) -> result(). +-spec upgrade(cluster_id(), version()) -> rt_util:result(). upgrade(Cluster, Version) -> lager:error("upgrade(~p, ~p) not implemented.", [Cluster, Version]), ok. diff --git a/src/rt_riak_node.erl b/src/rt_riak_node.erl new file mode 100644 index 000000000..4b525b506 --- /dev/null +++ b/src/rt_riak_node.erl @@ -0,0 +1,382 @@ +%%%------------------------------------------------------------------- +%%% @author John Burwell <> +%%% @copyright (C) 2015, John Burwell +%%% @doc +%%% +%%% @end +%%% Created : 19 Mar 2015 by John Burwell <> +%%%------------------------------------------------------------------- +-module(rt_riak_node). + +-behaviour(gen_fsm). + +%% API +-export([assert_singleton/1, + brutal_kill/1, + claimant_according_to/1, + commit/1, + configure/3, + get_ring/1, + is_allocated/1, + is_ready/1, + is_stopped/1, + join/2, + members_according_to/1, + owners_according_to/1, + partitions/1, + ping/1, + plan/1, + start/1, + start/2, + start_link/4, + status_of_according_to/1, + stop/1, + stop/2, + wait_for_service/2, + wait_until_pingable/1, + wait_until_registered/2, + wait_until_unpingable/1, + version/1]). + +%% gen_fsm callbacks +-export([init/1, state_name/2, state_name/3, handle_event/3, + handle_sync_event/4, handle_info/3, ready/3, terminate/3, code_change/4]). + +-define(SERVER, ?MODULE). + +-type host() :: string(). +-type node_id() :: string(). + +-record(state, {host :: host(), + id :: node_id(), + name :: node(), + version :: string()}). + +%%%=================================================================== +%%% API +%%%=================================================================== + + +%% @doc Ensure that the specified node is a singleton node/cluster -- a node +%% that owns 100% of the ring. +-spec assert_singleton(node()) -> boolean(). +assert_singleton(Node) -> + gen_fsm:sync_send_event(Node, check_singleton). + +-spec brutal_kill(node()) -> rt_util:result(). +brutal_kill(Node) -> + gen_fsm:sync_send_all_state_event(Node, brutal_kill). + +%% @doc Return a list of nodes that own partitions according to the ring +%% retrieved from the specified node. +claimant_according_to(Node) -> + gen_fsm:sync_send_event(Node, claimant_according_to). + +-spec commit(node()) -> rt_util:result(). +commit(Node) -> + gen_fsm:sync_send_event(Node, commit). + +%% @doc Modifies the riak.conf and advanced.config +-spec configure(node(), proplists:proplist(), proplists:proplist()) -> rt_util:result(). +configure(Node, Conf, AdvancedConfig) -> + gen_fsm:sync_send_event(Node, {configure, Conf, AdvancedConfig}). + +%% @doc Get the raw ring for `Node'. +-spec get_ring(node()) -> term(). +get_ring(Node) -> + gen_fsm:sync_send_event(Node, get_ring). + +-spec is_allocated(node()) -> boolean. +is_allocated(Node) -> + gen_fsm:sync_send_event(Node, is_allocated). + +-spec is_ready(node()) -> boolean(). +is_ready(Node) -> + gen_fsm:sync_send_event(Node, is_ready). + +-spec is_stopped(node()) -> boolean(). +is_stopped(Node) -> + gen_fsm:sync_send_event(Node, is_stopped). + +-spec join(node(), node()) -> rt_util:result(). +join(Node, ToNode) -> + gen_fsm:sync_send_event(Node, {join, ToNode}). + +%% @doc Return a list of cluster members according to the ring retrieved from +%% the specified node. +-spec members_according_to(node()) -> [term()] | rt_util:error(). +members_according_to(Node) -> + gen_fsm:sync_send_event(Node, members_according_to). + +%% @doc Return a list of nodes that own partitions according to the ring +%% retrieved from the specified node. +-spec owners_according_to(node()) -> [term()]. +owners_according_to(Node) -> + gen_fsm:sync_send_event(Node, owners_according_to). + +%% @doc Get list of partitions owned by node (primary). +-spec partitions(node()) -> [term()]. +partitions(Node) -> + lager:error("partitions(~p) is not implemented.", [Node]), + []. + +-spec ping(node()) -> boolean(). +ping(Node) -> + gen_fsm:sync_send_event(Node, ping). + +-spec plan(node()) -> rt_util:result(). +plan(Node) -> + gen_fsm:sync_send_event(Node, plan). + +-spec start(node()) -> rt_util:result(). +start(Node) -> + start(Node, true). + +-spec start(node(), boolean()) -> rt_util:result(). +start(Node, Wait) -> + gen_fsm:sync_send_event(Node, {start, Wait}). + +%% @doc Starts a gen_fsm process to configure, start, and +%% manage a Riak node on the `Host' identified by `NodeId' +%% and `NodeName' using Riak `Version' ({product, release}) +-spec start_link(host(), node_id(), node(), string()) -> {ok, pid()} | ignore | rt_util:error(). +start_link(Host, NodeId, NodeName, Version) -> + Args = [Host, NodeId, NodeName, Version], + gen_fsm:start_link(NodeName, ?MODULE, Args, []). + +%% @doc Return the cluster status of `Member' according to the ring +%% retrieved from `Node'. +-spec status_of_according_to(node()) -> [term()] | rt_util:error(). +status_of_according_to(Node) -> + gen_fsm:sync_send_event(Node, status_of_according_to). + + +-spec stop(node()) -> rt_util:result(). +stop(Node) -> + stop(Node, true). + +-spec stop(node(), boolean()) -> rt_util:result(). +stop(Node, Wait) -> + gen_fsm:sync_send_event(Node, {stop, Wait}). + +-spec wait_for_service(node(), [string()]) -> rt_util:result(). +wait_for_service(Node, Services) -> + gen_fsm:sync_send_event(Node, {wait_for_services, Services}). + +-spec wait_until_pingable(node()) -> rt_util:result(). +wait_until_pingable(Node) -> + gen_fsm:sync_send_event(Node, wait_until_pingable). + +-spec wait_until_registered(node(), atom()) -> rt_util:result(). +wait_until_registered(Node, Name) -> + gen_fsm:sync_send_event(Node, {wait_until_registered, Name}). + +-spec wait_until_unpingable(node()) -> rt_util:result(). +wait_until_unpingable(Node) -> + gen_fsm:sync_send_event(Node, wait_until_unpingable). + +version(Node) -> + gen_fsm:sync_send_event(Node, version). + +%%%=================================================================== +%%% gen_fsm callbacks +%%%=================================================================== + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Whenever a gen_fsm is started using gen_fsm:start/[3,4] or +%% gen_fsm:start_link/[3,4], this function is called by the new +%% process to initialize. +%% +%% @spec init(Args) -> {ok, StateName, State} | +%% {ok, StateName, State, Timeout} | +%% ignore | +%% {stop, StopReason} +%% @end +%%-------------------------------------------------------------------- +init([Host, NodeId, NodeName, Version]) -> + State = #state{host=Host, + id=NodeId, + name=NodeName, + version=Version}, + {ok, allocated, State}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% There should be one instance of this function for each possible +%% state name. Whenever a gen_fsm receives an event sent using +%% gen_fsm:send_event/2, the instance of this function with the same +%% name as the current state name StateName is called to handle +%% the event. It is also called if a timeout occurs. +%% +%% @spec state_name(Event, State) -> +%% {next_state, NextStateName, NextState} | +%% {next_state, NextStateName, NextState, Timeout} | +%% {stop, Reason, NewState} +%% @end +%%-------------------------------------------------------------------- +state_name(_Event, State) -> + {next_state, state_name, State}. + +ready(get_ring, _From, #state{name=NodeName}=State) -> + {ok, Ring} = maybe_get_ring(NodeName), + {reply, Ring, ready, State}; +ready(members_according_to, _From, #state{name=NodeName}=State) -> + Members = maybe_members_according_to(NodeName), + {reply, Members, ready, State}; +ready(partitions, _From, #state{name=NodeName}=State) -> + Partitions = maybe_partitions(NodeName), + {reply, Partitions, ready, State}; +ready(owners_according_to, _From, #state{name=NodeName}=State) -> + Owners = maybe_owners_according_to(NodeName), + {reply, Owners, ready, State}. + +-spec maybe_get_ring(node()) -> rt_rpc_result(). +maybe_get_ring(NodeName) -> + maybe_rpc_call(NodeName, riak_core_ring_manager, get_raw_ring, []). + +-spec maybe_partitions(node()) -> rt_rpc_result(). +maybe_partitions(NodeName) -> + maybe_partitions(NodeName, maybe_get_ring(NodeName)). + +-spec maybe_partitions(node(), rt_rpc_result()) -> [term()] | rt_util:error(). +maybe_partitions(NodeName, {ok, Ring}) -> + [Idx || {Idx, Owner} <- riak_core_ring:all_owners(Ring), Owner == NodeName]; +maybe_partitions(_NodeName, {error, Reason}) -> + {error, Reason}. + +-spec maybe_members_according_to(node() | rt_rpc_result()) -> [term()] | rt_util:error(). +maybe_members_according_to({ok, Ring}) -> + riak_core_ring:all_members(Ring); +maybe_members_according_to({error, Reason}) -> + {error, Reason}; +maybe_members_according_to(NodeName) -> + maybe_members_according_to(maybe_get_ring(NodeName)). + +-spec maybe_owners_according_to(node() | rt_rpc_result()) -> [term()] | rt_util:error(). +maybe_owners_according_to({ok, Ring}) -> + Owners = [Owner || {_Idx, Owner} <- riak_core_ring:all_owners(Ring)], + lists:usort(Owners); +maybe_owners_according_to({error, Reason}) -> + {error,Reason}; +maybe_owners_according_to(NodeName) -> + maybe_owners_according_to(maybe_get_ring(NodeName)). + +%% TODO Move to rt_util ?? +-type erl_rpc_result() :: {ok, term()} | {badrpc, term()}. +-type rt_rpc_result() :: {ok, term()} | rt_util:error(). +-spec maybe_rpc_call(node(), module(), function(), [term()]) -> erl_rpc_result(). +maybe_rpc_call(NodeName, Module, Function, Args) -> + maybe_rpc_call(rpc:call(NodeName, Module, Function, Args)). + +-spec maybe_rpc_call(erl_rpc_result()) -> rt_rpc_result(). +maybe_rpc_call({badrpc, _}) -> + {error, badrpc}; +maybe_rpc_call(Result) -> + Result. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% There should be one instance of this function for each possible +%% state name. Whenever a gen_fsm receives an event sent using +%% gen_fsm:sync_send_event/[2,3], the instance of this function with +%% the same name as the current state name StateName is called to +%% handle the event. +%% +%% @spec state_name(Event, From, State) -> +%% {next_state, NextStateName, NextState} | +%% {next_state, NextStateName, NextState, Timeout} | +%% {reply, Reply, NextStateName, NextState} | +%% {reply, Reply, NextStateName, NextState, Timeout} | +%% {stop, Reason, NewState} | +%% {stop, Reason, Reply, NewState} +%% @end +%%-------------------------------------------------------------------- +state_name(_Event, _From, State) -> + Reply = ok, + {reply, Reply, state_name, State}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Whenever a gen_fsm receives an event sent using +%% gen_fsm:send_all_state_event/2, this function is called to handle +%% the event. +%% +%% @spec handle_event(Event, StateName, State) -> +%% {next_state, NextStateName, NextState} | +%% {next_state, NextStateName, NextState, Timeout} | +%% {stop, Reason, NewState} +%% @end +%%-------------------------------------------------------------------- +handle_event(_Event, StateName, State) -> + {next_state, StateName, State}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Whenever a gen_fsm receives an event sent using +%% gen_fsm:sync_send_all_state_event/[2,3], this function is called +%% to handle the event. +%% +%% @spec handle_sync_event(Event, From, StateName, State) -> +%% {next_state, NextStateName, NextState} | +%% {next_state, NextStateName, NextState, Timeout} | +%% {reply, Reply, NextStateName, NextState} | +%% {reply, Reply, NextStateName, NextState, Timeout} | +%% {stop, Reason, NewState} | +%% {stop, Reason, Reply, NewState} +%% @end +%%-------------------------------------------------------------------- +handle_sync_event(_Event, _From, StateName, State) -> + Reply = ok, + {reply, Reply, StateName, State}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% This function is called by a gen_fsm when it receives any +%% message other than a synchronous or asynchronous event +%% (or a system message). +%% +%% @spec handle_info(Info,StateName,State)-> +%% {next_state, NextStateName, NextState} | +%% {next_state, NextStateName, NextState, Timeout} | +%% {stop, Reason, NewState} +%% @end +%%-------------------------------------------------------------------- +handle_info(_Info, StateName, State) -> + {next_state, StateName, State}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% This function is called by a gen_fsm when it is about to +%% terminate. It should be the opposite of Module:init/1 and do any +%% necessary cleaning up. When it returns, the gen_fsm terminates with +%% Reason. The return value is ignored. +%% +%% @spec terminate(Reason, StateName, State) -> void() +%% @end +%%-------------------------------------------------------------------- +terminate(_Reason, _StateName, _State) -> + ok. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Convert process state when code is changed +%% +%% @spec code_change(OldVsn, StateName, State, Extra) -> +%% {ok, StateName, NewState} +%% @end +%%-------------------------------------------------------------------- +code_change(_OldVsn, StateName, State, _Extra) -> + {ok, StateName, State}. + +%%%=================================================================== +%%% Internal functions +%%%=================================================================== diff --git a/src/rt_util.erl b/src/rt_util.erl new file mode 100644 index 000000000..961887825 --- /dev/null +++ b/src/rt_util.erl @@ -0,0 +1,7 @@ +-module(rt_util). + +-type error() :: {error(), term()}. +-type result() :: ok | error(). + +-export_type([error/0, + result/0]). From 627921967493bf88ee5a5ae302c08a8659a806c7 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Tue, 24 Mar 2015 12:36:52 -0400 Subject: [PATCH 111/157] - The repl_fs_stat_caching and replication2 test cases work with the v1 API wrapper --- regression_test_wrapper.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index 090384240..0fc6e3f7a 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -8,7 +8,7 @@ if [ -z $1 ]; then exit 1 fi -ALL_BACKEND_TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantine,gh_riak_core_155,pb_security,verify_search,verify_handoff,verify_capabilities,replication/repl_fs_stat_caching,replication/replication2,overload" +ALL_BACKEND_TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantine,gh_riak_core_155,pb_security,verify_search,verify_handoff,verify_capabilities,replication/repl_fs_stat_caching,replication/replication2" BITCASK_BACKEND_TEST_CASES="$ALL_BACKEND_TEST_CASES,loaded_upgrade" ELEVELDB_BACKEND_TEST_CASES="verify_2i_aae,loaded_upgrade" From 566d0d44486fddc18bdb01993a2b8169d27983bf Mon Sep 17 00:00:00 2001 From: John Burwell Date: Wed, 25 Mar 2015 16:54:58 -0400 Subject: [PATCH 112/157] Fixes continue on fail feature to ensure that test runs actually stop when it is false * Adds the --continue command line parameter to specify that test execution should continue when one or more test cases fail * Adds the continue_on_fail configuration parameter --- regression_test_wrapper.sh | 8 +++++--- src/riak_test_escript.erl | 35 +++++++++++++++++++---------------- src/riak_test_executor.erl | 36 ++++++++++++++++++++++-------------- src/riak_test_runner.erl | 30 ++++++++++++++++-------------- 4 files changed, 62 insertions(+), 47 deletions(-) diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index 0fc6e3f7a..711707e09 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -23,13 +23,15 @@ RESULTS_SYMLINK=$ROOT_RESULTS_DIR/current rm -f $RESULTS_SYMLINK ln -s $RESULTS $RESULTS_SYMLINK +RT_OPS="-v --continue -c $1" + echo "Running bitcask regression tests using the following test cases: $BITCASK_BACKEND_TEST_CASES" -./riak_test -v -c $1 -t $BITCASK_BACKEND_TEST_CASES &> $RESULTS_DIR/bitcask_results.log +./riak_test $RT_OPTS -t $BITCASK_BACKEND_TEST_CASES &> $RESULTS_DIR/bitcask_results.log echo "Running leveldb regression tests using the following test cases: $ELEVELDB_BACKEND_TEST_CASES" -./riak_test -v -c $1 -t $ELEVELDB_BACKEND_TEST_CASES -b eleveldb &> $RESULTS_DIR/leveldb_results.log +./riak_test $RT_OPTS -t $ELEVELDB_BACKEND_TEST_CASES -b eleveldb &> $RESULTS_DIR/leveldb_results.log echo "Running memory regression tests using the following test cases: $MEMORY_BACKEND_TEST_CASES" -./riak_test -v -c $1 -t $MEMORY_BACKEND_TEST_CASES -b memory &> $RESULTS_DIR/memory_results.log +./riak_test $RT_OPTS -t $MEMORY_BACKEND_TEST_CASES -b memory &> $RESULTS_DIR/memory_results.log echo "Results of the test run written to $RESULTS_DIR" diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index dc51a7e65..6ebb94bf2 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -121,24 +121,24 @@ finalize(TestResults, Args) -> Teardown = not proplists:get_value(keep, Args, false), maybe_teardown(Teardown, TestResults), ok. - %% Option Name, Short Code, Long Code, Argument Spec, Help Message cli_options() -> [ - {help, $h, "help", undefined, "Print this usage page"}, - {config, $c, "conf", string, "specifies the project configuration"}, - {tests, $t, "tests", string, "specifies which tests to run"}, - {suites, $s, "suites", string, "which suites to run"}, - {groups, $g, "groups", string, "specifiy a list of test groups to run"}, - {dir, $d, "dir", string, "run all tests in the specified directory"}, - {skip, $x, "skip", string, "list of tests to skip in a directory"}, - {verbose, $v, "verbose", undefined, "verbose output"}, - {outdir, $o, "outdir", string, "output directory"}, - {backend, $b, "backend", atom, "backend to test [memory | bitcask | eleveldb]"}, - {upgrade_path, $u, "upgrade-path", atom, "comma-separated list representing an upgrade path (e.g. riak-1.3.4,riak_ee-1.4.12,riak_ee-2.0.0)"}, - {keep, undefined, "keep", boolean, "do not teardown cluster"}, - {report, $r, "report", string, "you're reporting an official test run, provide platform info (e.g. ubuntu-1404-64)\nUse 'config' if you want to pull from ~/.riak_test.config"}, - {file, $F, "file", string, "use the specified file instead of ~/.riak_test.config"} + {help, $h, "help", undefined, "Print this usage page"}, + {config, $c, "conf", string, "specifies the project configuration"}, + {tests, $t, "tests", string, "specifies which tests to run"}, + {suites, $s, "suites", string, "which suites to run"}, + {groups, $g, "groups", string, "specifiy a list of test groups to run"}, + {dir, $d, "dir", string, "run all tests in the specified directory"}, + {skip, $x, "skip", string, "list of tests to skip in a directory"}, + {verbose, $v, "verbose", undefined, "verbose output"}, + {outdir, $o, "outdir", string, "output directory"}, + {backend, $b, "backend", atom, "backend to test [memory | bitcask | eleveldb]"}, + {upgrade_path, $u, "upgrade-path", atom, "comma-separated list representing an upgrade path (e.g. riak-1.3.4,riak_ee-1.4.12,riak_ee-2.0.0)"}, + {keep, undefined, "keep", boolean, "do not teardown cluster"}, + {continue_on_fail,undefined, "continue", boolean, "continues executing tests on failure"}, + {report, $r, "report", string, "you're reporting an official test run, provide platform info (e.g. ubuntu-1404-64)\nUse 'config' if you want to pull from ~/.riak_test.config"}, + {file, $F, "file", string, "use the specified file instead of ~/.riak_test.config"} ]. print_help() -> @@ -209,6 +209,9 @@ help_or_parse_tests(ParsedArgs, HarnessArgs, false) -> %% test metadata load_initial_config(ParsedArgs), + %% TODO Likely need to evaluate loading all parsed args into the config .. + rt_config:set(continue_on_fail, proplists:get_value(continue_on_fail, ParsedArgs, false)), + TestData = compose_test_data(ParsedArgs), {Tests, NonTests} = which_tests_to_run(report(ParsedArgs), TestData), Offset = rt_config:get(offset, undefined), @@ -578,7 +581,7 @@ backend_list(Backends) when is_list(Backends) -> load_tests_in_dir(Dir, Groups, SkipTests) -> case filelib:is_dir(Dir) of true -> - code:add_path(Dir), + lists:sort( lists:foldl(load_tests_folder(Groups, SkipTests), [], diff --git a/src/riak_test_executor.erl b/src/riak_test_executor.erl index 479b9e56e..a628b19a3 100644 --- a/src/riak_test_executor.erl +++ b/src/riak_test_executor.erl @@ -34,7 +34,8 @@ runner_pids=[] :: [pid()], log_dir :: string(), report_info :: string(), - execution_mode :: execution_mode()}). + execution_mode :: execution_mode(), + continue_on_fail :: boolean()}). %%%=================================================================== %%% API @@ -66,6 +67,8 @@ init([Tests, Backend, LogDir, ReportInfo, UpgradeList, NotifyPid]) -> %% backend command line argument fo v1 cluster provisioning -jsb rt_config:set(rt_backend, Backend), + ContinueOnFail = rt_config:get(continue_on_fail), + lager:notice("Starting the Riak Test executor in ~p execution mode", [ExecutionMode]), State = #state{pending_tests=Tests, backend=Backend, @@ -73,7 +76,8 @@ init([Tests, Backend, LogDir, ReportInfo, UpgradeList, NotifyPid]) -> report_info=ReportInfo, upgrade_list=UpgradeList, notify_pid=NotifyPid, - execution_mode=ExecutionMode}, + execution_mode=ExecutionMode, + continue_on_fail=ContinueOnFail}, {ok, gather_properties, State, 0}. %% @doc there are no all-state events for this fsm @@ -107,7 +111,7 @@ code_change(_OldVsn, StateName, State, _Extra) -> %% properties record. gather_properties(timeout, State) -> OverrideProps = override_props(State), - Properties = test_properties(State#state.pending_tests, OverrideProps), +Properties = test_properties(State#state.pending_tests, OverrideProps), {next_state, request_nodes, State#state{test_properties=Properties}, 0}; gather_properties(_Event, _State) -> {next_state, gather_properties, _State}. @@ -166,12 +170,14 @@ launch_test({nodes, Nodes, NodeMap}, State) -> backend=Backend, test_properties=PropertiesList, runner_pids=Pids, - running_tests=Running} = State, + running_tests=Running, + continue_on_fail=ContinueOnFail} = State, lager:debug("Executing test ~p in mode ~p", [NextTest, ExecutionMode]), {NextTest, TestProps} = lists:keyfind(NextTest, 1, PropertiesList), UpdTestProps = rt_properties:set([{node_map, NodeMap}, {node_ids, Nodes}], TestProps), - {RunnerPids, RunningTests} = run_test(ExecutionMode, NextTest, Backend, UpdTestProps, Pids, Running), + {RunnerPids, RunningTests} = run_test(ExecutionMode, NextTest, Backend, UpdTestProps, + Pids, Running, ContinueOnFail), UpdState = State#state{pending_tests=RestPending, execution_mode=ExecutionMode, runner_pids=RunnerPids, @@ -228,7 +234,7 @@ wait_for_completion({test_complete, Test, Pid, Results, Duration}, State) -> pending_tests=Pending++Waiting, waiting_tests=[], execution_mode=ExecutionMode}, - wait_for_completion_transition(UpdState); + wait_for_completion_transition(Results, UpdState); wait_for_completion(_Event, _State) -> ok. @@ -258,12 +264,14 @@ report_done(#state{notify_pid=NotifyPid}) -> NotifyPid ! {self(), done}, ok. -wait_for_completion_transition(State=#state{pending_tests=[], +wait_for_completion_transition({_Status, _Reason}, State=#state{continue_on_fail=ContinueOnFail}) when ContinueOnFail == false -> + {stop, normal, State}; +wait_for_completion_transition(_Result, State=#state{pending_tests=[], running_tests=[]}) -> {stop, normal, State}; -wait_for_completion_transition(State=#state{pending_tests=[]}) -> +wait_for_completion_transition(_Result, State=#state{pending_tests=[]}) -> {next_state, wait_for_completion, State}; -wait_for_completion_transition(State) -> +wait_for_completion_transition(_Result, State) -> {next_state, request_nodes, State, 0}. launch_test_transition(State=#state{pending_tests=PendingTests, @@ -322,11 +330,11 @@ override_props(State) -> [{upgrade_path, UpgradeList}] end. --spec run_test(parallel | serial, atom(), atom(), proplists:proplist(), [pid()], [atom()]) -> {[pid()], [atom()]}. -run_test(parallel, Test, Backend, Properties, RunningPids, RunningTests) -> - Pid = spawn_link(riak_test_runner, start, [Test, Backend, Properties]), +-spec run_test(parallel | serial, atom(), atom(), proplists:proplist(), [pid()], [atom()], boolean()) -> {[pid()], [atom()]}. +run_test(parallel, Test, Backend, Properties, RunningPids, RunningTests, ContinueOnFail) -> + Pid = spawn_link(riak_test_runner, start, [Test, Backend, Properties, ContinueOnFail]), {[Pid | RunningPids], [Test | RunningTests]}; -run_test(serial, Test, Backend, Properties, RunningPids, RunningTests) -> - riak_test_runner:start(Test, Backend, Properties), +run_test(serial, Test, Backend, Properties, RunningPids, RunningTests, ContinueOnFail) -> + riak_test_runner:start(Test, Backend, Properties, ContinueOnFail), {RunningPids, RunningTests}. diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index 41d9a50a9..105e13fe9 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -24,7 +24,7 @@ -behavior(gen_fsm). %% API --export([start/3, +-export([start/4, send_event/2, stop/0]). @@ -68,7 +68,7 @@ current_version :: string(), remaining_versions :: [string()], test_results :: [term()], - stop_on_fail :: boolean()}). + continue_on_fail :: boolean()}). -deprecated([{metadata,0,next_major_release}]). @@ -77,8 +77,8 @@ %%%=================================================================== %% @doc Start the test executor -start(TestModule, Backend, Properties) -> - Args = [TestModule, Backend, Properties], +start(TestModule, Backend, Properties, ContinueOnFail) -> + Args = [TestModule, Backend, Properties, ContinueOnFail], gen_fsm:start_link(?MODULE, Args, []). send_event(Pid, Msg) -> @@ -101,8 +101,8 @@ metadata() -> %% @doc Read the storage schedule and go to idle. %% compose_test_datum(Version, Project, undefined, undefined) -> -init([TestModule, Backend, Properties]) -> - lager:debug("Started riak_test_runnner with pid ~p", [self()]), +init([TestModule, Backend, Properties, ContinueOnFail]) -> + lager:debug("Started riak_test_runnner with pid ~p (continue on fail: ~p)", [self(), ContinueOnFail]), Project = list_to_binary(rt_config:get(rt_project, "undefined")), MetaData = [{id, -1}, {platform, <<"local">>}, @@ -122,7 +122,6 @@ init([TestModule, Backend, Properties]) -> BackendCheck = check_backend(Backend, rt_properties:get(valid_backends, Properties)), PreReqCheck = check_prereqs(ConfirmMod), - StopOnFail = rt_config:get(rt_stop_on_fail, true), State = #state{test_module=TestModule, test_type=TestType, properties=UpdProperties, @@ -133,7 +132,7 @@ init([TestModule, Backend, Properties]) -> backend_check=BackendCheck, prereq_check=PreReqCheck, group_leader=group_leader(), - stop_on_fail=StopOnFail}, + continue_on_fail=ContinueOnFail}, {ok, setup, State, 0}. %% @doc there are no all-state events for this fsm @@ -265,7 +264,7 @@ wait_for_completion(timeout, State=#state{test_module=TestModule, wait_for_completion({test_result, {fail, Reason}}, State=#state{test_module=TestModule, test_type=TestType, group_leader=GroupLeader, - stop_on_fail=StopOnFail, + continue_on_fail=ContinueOnFail, remaining_versions=[]}) -> Result = {fail, Reason}, lager:debug("Test Result ~p = {fail, ~p}", [TestModule, Reason]), @@ -273,12 +272,10 @@ wait_for_completion({test_result, {fail, Reason}}, State=#state{test_module=Test test_type=TestType, test_results=Result, group_leader=GroupLeader, - stop_on_fail=StopOnFail, + continue_on_fail=ContinueOnFail, end_time=os:timestamp()}, - case StopOnFail of - true -> ok; - false -> cleanup(UpdState) - end, + lager:debug("ContinueOnFail: ~p", [ContinueOnFail]), + maybe_cleanup(ContinueOnFail, UpdState), notify_executor(Result, UpdState), {stop, normal, UpdState}; wait_for_completion({test_result, Result}, State=#state{test_module=TestModule, @@ -486,6 +483,11 @@ notify_fun(Pid) -> ?MODULE:send_event(Pid, X) end. +maybe_cleanup(true, State) -> + cleanup(State); +maybe_cleanup(false, _State) -> + ok. + cleanup(#state{group_leader=OldGroupLeader, test_module=TestModule, test_type=TestType}) when TestType == old -> From 0311cf13e2a404485c8eef631a0afd89a3dd3757 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Thu, 26 Mar 2015 13:13:03 -0400 Subject: [PATCH 113/157] Fixes a syntax error in the regression test script that causes riak_test to be called without the necessary parameters --- regression_test_wrapper.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index 711707e09..7471d1251 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -23,7 +23,7 @@ RESULTS_SYMLINK=$ROOT_RESULTS_DIR/current rm -f $RESULTS_SYMLINK ln -s $RESULTS $RESULTS_SYMLINK -RT_OPS="-v --continue -c $1" +RT_OPTS="-v --continue -c $1" echo "Running bitcask regression tests using the following test cases: $BITCASK_BACKEND_TEST_CASES" ./riak_test $RT_OPTS -t $BITCASK_BACKEND_TEST_CASES &> $RESULTS_DIR/bitcask_results.log From d5c5f9fe067ba3527438069dbb0a19f03bcd69cb Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Thu, 26 Mar 2015 14:31:31 -0600 Subject: [PATCH 114/157] Add support for verify_handoff_mixed --- regression_test_wrapper.sh | 2 +- src/rt_harness.erl | 4 ++++ src/rtdev.erl | 2 +- 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index 7471d1251..314f466c7 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -8,7 +8,7 @@ if [ -z $1 ]; then exit 1 fi -ALL_BACKEND_TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantine,gh_riak_core_155,pb_security,verify_search,verify_handoff,verify_capabilities,replication/repl_fs_stat_caching,replication/replication2" +ALL_BACKEND_TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantine,gh_riak_core_155,pb_security,verify_search,verify_handoff,verify_capabilities,replication/repl_fs_stat_caching,replication/replication2,verify_handoff_mixed" BITCASK_BACKEND_TEST_CASES="$ALL_BACKEND_TEST_CASES,loaded_upgrade" ELEVELDB_BACKEND_TEST_CASES="verify_2i_aae,loaded_upgrade" diff --git a/src/rt_harness.erl b/src/rt_harness.erl index 0001a3fc5..a2b61ad0f 100644 --- a/src/rt_harness.erl +++ b/src/rt_harness.erl @@ -37,6 +37,7 @@ cmd/2, setup/0, get_deps/0, + get_node_logs/0, get_version/0, get_version/1, get_backends/0, @@ -106,6 +107,9 @@ get_version(Node) -> get_backends() -> ?HARNESS_MODULE:get_backends(). +get_node_logs() -> + ?HARNESS_MODULE:get_node_logs(). + set_backend(Backend) -> ?HARNESS_MODULE:set_backend(Backend). diff --git a/src/rtdev.erl b/src/rtdev.erl index 1af22116d..a902dc0ec 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -963,7 +963,7 @@ devpaths() -> %% proplists:get_keys(rt_config:get(rtdev_path)) -- [root]. get_node_logs() -> - Root = filename:absname(proplists:get_value(root_path, ?PATH)), + Root = filename:absname(?PATH), RootLen = length(Root) + 1, %% Remove the leading slash [ begin {ok, Port} = file:open(Filename, [read, binary]), From 0bacc4e40b73e4e9dfa4e5cfc7bdf7834efa4323 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Thu, 26 Mar 2015 14:56:17 -0600 Subject: [PATCH 115/157] Add verify_bitcask_tombstone2_upgrade to regression test suite --- regression_test_wrapper.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index 314f466c7..014f1c1a9 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -8,7 +8,7 @@ if [ -z $1 ]; then exit 1 fi -ALL_BACKEND_TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantine,gh_riak_core_155,pb_security,verify_search,verify_handoff,verify_capabilities,replication/repl_fs_stat_caching,replication/replication2,verify_handoff_mixed" +ALL_BACKEND_TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantine,gh_riak_core_155,pb_security,verify_search,verify_handoff,verify_capabilities,replication/repl_fs_stat_caching,replication/replication2,verify_handoff_mixed,verify_bitcask_tombstone2_upgrade" BITCASK_BACKEND_TEST_CASES="$ALL_BACKEND_TEST_CASES,loaded_upgrade" ELEVELDB_BACKEND_TEST_CASES="verify_2i_aae,loaded_upgrade" From 1f4a8a91a4bb578447f7f546c4621340eb37cbd0 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Thu, 26 Mar 2015 15:18:39 -0600 Subject: [PATCH 116/157] Add replication_object_reformat to regression test list --- regression_test_wrapper.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index 014f1c1a9..82017f1ff 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -8,7 +8,7 @@ if [ -z $1 ]; then exit 1 fi -ALL_BACKEND_TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantine,gh_riak_core_155,pb_security,verify_search,verify_handoff,verify_capabilities,replication/repl_fs_stat_caching,replication/replication2,verify_handoff_mixed,verify_bitcask_tombstone2_upgrade" +ALL_BACKEND_TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantine,gh_riak_core_155,pb_security,verify_search,verify_handoff,verify_capabilities,replication/repl_fs_stat_caching,replication/replication2,verify_handoff_mixed,verify_bitcask_tombstone2_upgrade,replication_object_reformat" BITCASK_BACKEND_TEST_CASES="$ALL_BACKEND_TEST_CASES,loaded_upgrade" ELEVELDB_BACKEND_TEST_CASES="verify_2i_aae,loaded_upgrade" From f05fc207738dbe646966887f429ab63a6f4de745 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Fri, 27 Mar 2015 01:41:33 -0400 Subject: [PATCH 117/157] Wraps rt:pbc/1 to call rt_pb:pbc/1 and fixes riak_test_escript to only override the continue_to_fail setting when --continue is specified on the command line. The previous behavior overwrote the setting if it was specified in riak_test.config. --- src/riak_test_escript.erl | 12 ++++++++++-- src/rt.erl | 39 +-------------------------------------- 2 files changed, 11 insertions(+), 40 deletions(-) diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index 6ebb94bf2..88e30538e 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -209,8 +209,7 @@ help_or_parse_tests(ParsedArgs, HarnessArgs, false) -> %% test metadata load_initial_config(ParsedArgs), - %% TODO Likely need to evaluate loading all parsed args into the config .. - rt_config:set(continue_on_fail, proplists:get_value(continue_on_fail, ParsedArgs, false)), + maybe_override_setting(continue_on_fail, true, ParsedArgs), TestData = compose_test_data(ParsedArgs), {Tests, NonTests} = which_tests_to_run(report(ParsedArgs), TestData), @@ -218,6 +217,15 @@ help_or_parse_tests(ParsedArgs, HarnessArgs, false) -> Workers = rt_config:get(workers, undefined), shuffle_tests(ParsedArgs, HarnessArgs, Tests, NonTests, Offset, Workers). +maybe_override_setting(Argument, Value, Arguments) -> + maybe_override_setting(proplists:is_defined(Argument, Arguments), Argument, + Value, Arguments). + +maybe_override_setting(true, Argument, Value, Arguments) -> + rt_config:set(Argument, proplists:get_value(Argument, Arguments, Value)); +maybe_override_setting(false, _Argument, _Value, _Arguments) -> + ok. + load_initial_config(ParsedArgs) -> %% Loads application defaults application:load(riak_test), diff --git a/src/rt.erl b/src/rt.erl index b2ca344d6..2203900ac 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -88,7 +88,6 @@ partition/2, partitions_for_node/1, pbc/1, - pbc/2, pbc_read/3, pbc_read/4, pbc_read_check/4, @@ -477,33 +476,6 @@ stream_cmd(Cmd) -> stream_cmd(Cmd, Opts) -> rt2:stream_cmd(Cmd, Opts). -stream_cmd_loop(Port, Buffer, NewLineBuffer, Time={_MegaSecs, Secs, _MicroSecs}) -> - receive - {Port, {data, Data}} -> - {_, Now, _} = now(), - NewNewLineBuffer = case Now > Secs of - true -> - lager:info(NewLineBuffer), - ""; - _ -> - NewLineBuffer - end, - case rt:str(Data, "\n") of - true -> - lager:info(NewNewLineBuffer), - Tokens = string:tokens(Data, "\n"), - [ lager:info(Token) || Token <- Tokens ], - stream_cmd_loop(Port, Buffer ++ NewNewLineBuffer ++ Data, "", Time); - _ -> - stream_cmd_loop(Port, Buffer, NewNewLineBuffer ++ Data, now()) - end; - {Port, {exit_status, Status}} -> - catch port_close(Port), - {Status, Buffer} - after rt:config(rt_max_receive_wait_time) -> - {-1, Buffer} - end. - %%%=================================================================== %%% Remote code management %%%=================================================================== @@ -1159,16 +1131,7 @@ pbc_systest_read(Node, Start, End, Bucket, R) -> %% @doc get me a protobuf client process and hold the mayo! -spec pbc(node()) -> pid(). pbc(Node) -> - pbc(Node, [{auto_reconnect, true}]). - -%% GAP: rt_pb does not provide an analog to this function. -jsb --spec pbc(node(), proplists:proplist()) -> pid(). -pbc(Node, Options) -> - rt:wait_for_service(Node, riak_kv), - ConnInfo = proplists:get_value(Node, connection_info([Node])), - {IP, PBPort} = proplists:get_value(pb, ConnInfo), - {ok, Pid} = riakc_pb_socket:start_link(IP, PBPort, Options), - Pid. + rt_pb:pbc(Node). %% @doc does a read via the erlang protobuf client -spec pbc_read(pid(), binary(), binary()) -> binary(). From 94de39a5b2a02312f405ea558dc05f1eaf23d3f7 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Fri, 27 Mar 2015 01:59:17 -0400 Subject: [PATCH 118/157] Wraps rt:pbc/2 to call rt_pb:pbc/2 and adds implementation of rt_pb:pbc/2 --- src/rt.erl | 5 +++++ src/rt_pb.erl | 14 +++++++++----- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/src/rt.erl b/src/rt.erl index 2203900ac..db9f89ae0 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -88,6 +88,7 @@ partition/2, partitions_for_node/1, pbc/1, + pbc/2, pbc_read/3, pbc_read/4, pbc_read_check/4, @@ -1133,6 +1134,10 @@ pbc_systest_read(Node, Start, End, Bucket, R) -> pbc(Node) -> rt_pb:pbc(Node). +-spec pbc(node(), proplists:proplist()) -> pid(). +pbc(Node, Options) -> + rt_pb:pbc(Node, Options). + %% @doc does a read via the erlang protobuf client -spec pbc_read(pid(), binary(), binary()) -> binary(). pbc_read(Pid, Bucket, Key) -> diff --git a/src/rt_pb.erl b/src/rt_pb.erl index 3b720e6a6..97d066e0e 100644 --- a/src/rt_pb.erl +++ b/src/rt_pb.erl @@ -22,6 +22,7 @@ -include_lib("eunit/include/eunit.hrl"). -export([pbc/1, + pbc/2, stop/1, pbc_read/3, pbc_read/4, @@ -46,11 +47,14 @@ %% @doc get me a protobuf client process and hold the mayo! -spec pbc(node()) -> pid(). pbc(Node) -> - %% rt:wait_for_service(Node, riak_kv), - %% ConnInfo = proplists:get_value(Node, rt:connection_info([Node])), - %% {IP, PBPortz} = proplists:get_value(pb, ConnInfo), - {ok, [{IP, PBPort}]} = get_pb_conn_info(Node), - {ok, Pid} = riakc_pb_socket:start_link(IP, PBPort, [{auto_reconnect, true}]), + pbc(Node, [{auto_reconnect, true}]). + +-spec pbc(node(), proplists:proplist()) -> pid(). +pbc(Node, Options) -> + rt2:wait_for_service(Node, riak_kv), + ConnInfo = proplists:get_value(Node, get_pb_conn_info([Node])), + {IP, PBPort} = proplists:get_value(pb, ConnInfo), + {ok, Pid} = riakc_pb_socket:start_link(IP, PBPort, Options), Pid. stop(Pid) -> From e9215d16879e41021f2a61da73f573435ea3db1c Mon Sep 17 00:00:00 2001 From: John Burwell Date: Fri, 27 Mar 2015 15:27:46 -0400 Subject: [PATCH 119/157] Fixes the implementation of rt_pb:pbc/2 to properly match the results of get_pb_conn_info fixing a variety of test failures --- src/rt2.erl | 10 +++++----- src/rt_pb.erl | 9 +++++---- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/src/rt2.erl b/src/rt2.erl index 0014133dd..8f8220622 100644 --- a/src/rt2.erl +++ b/src/rt2.erl @@ -124,10 +124,10 @@ rpc_get_env(_, []) -> undefined; rpc_get_env(Node, [{App,Var}|Others]) -> case rpc:call(Node, application, get_env, [App, Var]) of - {ok, Value} -> - {ok, Value}; - _ -> - rpc_get_env(Node, Others) + {ok, Value} -> + {ok, Value}; + _ -> + rpc_get_env(Node, Others) end. -type interface() :: {http, tuple()} | {pb, tuple()}. @@ -145,7 +145,7 @@ connection_info(Node) when is_atom(Node) -> [{http, {HTTP_IP, HTTP_Port}}, {https, {HTTPS_IP, HTTPS_Port}}, {pb, {PB_IP, PB_Port}}] end; connection_info(Nodes) when is_list(Nodes) -> -[ {Node, connection_info(Node)} || Node <- Nodes]. + [ {Node, connection_info(Node)} || Node <- Nodes]. maybe_wait_for_changes(Node) -> Ring = rt_ring:get_ring(Node), diff --git a/src/rt_pb.erl b/src/rt_pb.erl index 97d066e0e..75c893297 100644 --- a/src/rt_pb.erl +++ b/src/rt_pb.erl @@ -52,8 +52,8 @@ pbc(Node) -> -spec pbc(node(), proplists:proplist()) -> pid(). pbc(Node, Options) -> rt2:wait_for_service(Node, riak_kv), - ConnInfo = proplists:get_value(Node, get_pb_conn_info([Node])), - {IP, PBPort} = proplists:get_value(pb, ConnInfo), + ConnInfo = get_pb_conn_info(Node), + {ok, [{IP, PBPort}]} = ConnInfo, {ok, Pid} = riakc_pb_socket:start_link(IP, PBPort, Options), Pid. @@ -186,13 +186,14 @@ pbc_systest_read(Node, Start, End, Bucket, R) -> -spec get_pb_conn_info(node()) -> [{inet:ip_address(), pos_integer()}]. get_pb_conn_info(Node) -> - case rt:rpc_get_env(Node, [{riak_api, pb}, + lager:debug("Querying connection pb connection information for node ~p", [Node]), + case rt2:rpc_get_env(Node, [{riak_api, pb}, {riak_api, pb_ip}, {riak_kv, pb_ip}]) of {ok, [{NewIP, NewPort}|_]} -> {ok, [{NewIP, NewPort}]}; {ok, PB_IP} -> - {ok, PB_Port} = rt:rpc_get_env(Node, [{riak_api, pb_port}, + {ok, PB_Port} = rt2:rpc_get_env(Node, [{riak_api, pb_port}, {riak_kv, pb_port}]), {ok, [{PB_IP, PB_Port}]}; _ -> From 145b2576686ff485e70b097512f2969256a750bd Mon Sep 17 00:00:00 2001 From: John Burwell Date: Fri, 27 Mar 2015 15:29:15 -0400 Subject: [PATCH 120/157] WIP: Additional refinements to the v2 API specification --- src/rt_properties.erl | 29 +++++++++++ src/rt_riak_cluster.erl | 105 ++++++++++++++++++-------------------- src/rt_riak_node.erl | 109 ++++++++++++++++++++++++++++++++++------ 3 files changed, 172 insertions(+), 71 deletions(-) diff --git a/src/rt_properties.erl b/src/rt_properties.erl index f99e70568..3ab343d9e 100644 --- a/src/rt_properties.erl +++ b/src/rt_properties.erl @@ -24,6 +24,13 @@ -include("rt.hrl"). +%%-record(rt_cluster_topology_v1, { +%% name :: atom(), +%% connected_to :: [atom()], +%% override_properties :: proplists:proplist() +%% }). +%%-type topology() :: #rt_cluster_topology_v1{}. + %% A quick note on the distinction between `node_ids' and %% `node_map'. `node_ids' are short identifers (e.g. dev1) and the %% `node_map' maps each node_id to a full erlang node names. Both are @@ -52,6 +59,26 @@ config=default_config() :: term(), external_properties :: term() % arbitrary properties for 3rd party use }). +%%-record(rt_properties_v2, { +%% description :: string(), +%% supported_products :: [atom()], %% TODO Use the product type when exported ... +%% minimum_version :: string(), %% TODO Use the version types when exported ... +%% maximum_version :: string(), %% TODO Use the version types when exported ... +%% supported_backends=all :: [atom()], +%% node_count :: non_neg_integer(), +%% wait_for_transfers=false :: boolean(), +%% bucket_types=[] :: bucket_types(), +%% indexes=[] :: [index()], +%% ring_size=auto :: [atom() | non_neg_integer()], +%% enable_strong_consistency=false :: boolean(), +%% enable_yokozuna=false :: boolean(), +%% enable_jmx=false :: boolean(), +%% enable_snmp=false :: boolean(), +%% config=default_config() :: term(), +%% required_services=[riak_kv] :: [atom()] + %% TODO What elements are needed to configure security? +%% }). + -type properties() :: #rt_properties_v1{}. %% Specify the bucket_types field for the properties record. The list @@ -61,8 +88,10 @@ %% with the given index. The former form is applied to all clusters. -type bucket_type() :: {binary(), proplists:proplist()}. -type bucket_types() :: [bucket_type() | {pos_integer(), bucket_type()}]. +%%-type index() :: {binary(), binary(), binary()}. -export_type([properties/0, +%% index/0, bucket_types/0]). -define(RT_PROPERTIES, #rt_properties_v1). diff --git a/src/rt_riak_cluster.erl b/src/rt_riak_cluster.erl index 1c3c514ca..1f28c0833 100644 --- a/src/rt_riak_cluster.erl +++ b/src/rt_riak_cluster.erl @@ -12,7 +12,6 @@ %% API -export([activate_bucket_type/2, - decommission/1, clean/1, create_and_activate_bucket_type/3, create_bucket_type/3, @@ -21,7 +20,6 @@ leave/2, nodes/1, partition/3, - provision/4, staged_join/2, start/1, stop/1, @@ -31,10 +29,11 @@ wait_until_no_pending_changes/1, wait_until_nodes_agree_about_ownership/1, wait_until_ring_converged/1, + version/1, upgrade/2]). %% gen_fsm callbacks --export([init/1, state_name/2, state_name/3, handle_event/3, +-export([init/1, state_name/2, state_name/3, handle_event/3, start_link/3, handle_sync_event/4, handle_info/3, terminate/3, code_change/4]). -define(SERVER, ?MODULE). @@ -43,7 +42,7 @@ bucket_types :: [atom()], %% should this be a string? indexes :: [atom()], %% should this be a string? nodes :: [node()], - version :: string() + version :: version() }). %%%=================================================================== @@ -54,113 +53,105 @@ %% -type partition() :: [node()]. %% TODO Move to the rt_version module -%% TODO Split out the product type --type version() :: {atom(), string()}. +-type product_type() :: riak | riak_ee | riak_cs. +-type version() :: {product_type(), string()}. -spec activate_bucket_type(cluster_id(), atom()) -> rt_util:result(). activate_bucket_type(Cluster, BucketType) -> - lager:error("activate_bucket_type(~p, ~p, ~p) not implemented", [Cluster, BucketType]), - ok. - --spec decommission(cluster_id()) -> rt_util:result(). -decommission(Cluster) -> - lager:error("decomission(~p) not implemented", [Cluster]), - ok. + gen_fsm:sync_send_event(Cluster, {activate_bucket_type, BucketType}). +%% @doc Stops cluster, `Cluster', removes all data, and restarts it -spec clean(cluster_id()) -> rt_util:result(). clean(Cluster) -> - lager:error("clean(~p) not implemented", [Cluster]), - ok. + gen_fsm:sync_send_event(Cluster, clean). +%% @doc Creates and activates a bucket type, `BucketType', on cluster, `Cluster', using properties, `Properties' -spec create_and_activate_bucket_type(cluster_id(), atom(), proplists:proplist()) -> rt_util:result(). -create_and_activate_bucket_type(Cluster, BucketType, Parameters) -> - ok = create_bucket_type(Cluster, BucketType, Parameters), +create_and_activate_bucket_type(Cluster, BucketType, Properties) -> + ok = create_bucket_type(Cluster, BucketType, Properties), activate_bucket_type(Cluster, BucketType). +%% @doc Creates a bucket type, `BucketType', on cluster, `Cluster', with properties, `Properties' -spec create_bucket_type(cluster_id(), atom(), proplists:proplist()) -> rt_util:result(). -create_bucket_type(Cluster, BucketType, Parameters) -> - lager:error("create_bucket_type(~p, ~p, ~p) not implemented", [Cluster, BucketType, Parameters]), - ok. +create_bucket_type(Cluster, BucketType, Properties) -> + gen_fsm:create_bucket_type(Cluster, {create_bucket_type, BucketType, Properties}). -spec is_mixed(cluster_id()) -> boolean(). is_mixed(Cluster) -> - lager:error("is_mixed(~p) not implemented", [Cluster]), - ok. + gen_fsm:sync_send_event(Cluster, is_mixed). +%% @doc Joins a node, `Node', to the cluster, `Cluster' -spec join(cluster_id(), node()) -> rt_util:result(). join(Cluster, Node) -> - lager:error("join(~p, ~p) not implemented.", [Cluster, Node]), - ok. + gen_fsm:sync_send_event(Cluster, {join, Node}). + -spec leave(cluster_id(), node()) -> rt_util:result(). leave(Cluster, Node) -> - lager:error("leave(~p, ~p) not implemented.", [Cluster, Node]), - ok. + gen_fsm:sync_send_event(Cluster, {leave, Node}). +%% @doc Returns the list of nodes in the cluster -spec nodes(cluster_id()) -> [node()]. nodes(Cluster) -> - lager:error("nodes(~p) not implemented", [Cluster]), - ok. + gen_fsm:sync_send_event(Cluster, nodes). %% -spec partition(cluster_id(), partition(), partition()) -> [atom(), atom(), partition(), partition()] | rt_util:error(). -partition(Cluster, P1, P2) -> - lager:error("partition(~p, ~p, ~p) not implemented.", [Cluster, P1, P2]), - ok. +partition(Cluster, P1, P2) -> + gen_fsm:sync_send_event(Cluster, {partition, P1, P2}). --spec provision(cluster_name(), [node()], proplists:proplist(), proplists:proplist()) -> {cluster_name(), cluster_id()} | rt_util:error(). -provision(Name, Nodes, Conf, AdvancedConfig) -> - lager:error("provision(~p, ~p, ~p, ~p) not implemented.", [Name, Nodes, Conf, AdvancedConfig]), +%% @doc Starts the FSM initializing the state with the passed `Name', `Nodes', and `Config' +-spec start_link(cluster_name(), [node()], proplists:proplist()) -> {cluster_name(), cluster_id()} | rt_util:error(). +start_link(Name, Nodes, Config) -> + lager:error("provision(~p, ~p, ~p) not implemented.", [Name, Nodes, Config]), ok. -spec staged_join(cluster_id(), node()) -> rt_util:result(). staged_join(Cluster, Node) -> - lager:error("stagad_join(~p, ~p) not implemented.", [Cluster, Node]), - ok. + gen_fsm:sync_send_event(Cluster, {staged_join, Node}). +%% @doc Starts each node in the cluster -spec start(cluster_id()) -> rt_util:result(). start(Cluster) -> - lager:error("start(~p) not implemented", [Cluster]), - ok. + gen_fsm:sync_send_event(Cluster, start). +%% @doc Stops each node in the cluster -spec stop(cluster_id()) -> rt_util:result(). stop(Cluster) -> - lager:error("stop(~p) not implemented", [Cluster]), - ok. + gen_fsm:sync_send_event(Cluster, stop). --spec wait_until_all_members([node()]) -> rt_util:result(). -wait_until_all_members(Nodes) -> - lager:error("wait_until_all_members(~p) not implemented", [Nodes]), - ok. +-spec wait_until_all_members(cluster_id()) -> rt_util:result(). +wait_until_all_members(Cluster) -> + gen_fsm:sync_send_event(Cluster, wait_unit_all_members). -spec wait_until_connected(cluster_id()) -> rt_util:result(). wait_until_connected(Cluster) -> - lager:error("wait_until_connected(~p) not implemented", [Cluster]), - ok. + gen_fsm:sync_send_event(Cluster, wait_until_connected). -spec wait_until_legacy_ring_ready(cluster_id()) -> rt_util:result(). wait_until_legacy_ring_ready(Cluster) -> - lager:error("wait_until_legacy_ring_ready(~p) not implemented", [Cluster]), - ok. + gen_fsm:sync_send_event(Cluster, wait_until_legacy_ring). -spec wait_until_no_pending_changes(cluster_id()) -> rt_util:result(). wait_until_no_pending_changes(Cluster) -> - lager:error("wait_until_no_pending_changes(~p) not implemented", [Cluster]), - ok. + gen_fsm:sync_send_event(Cluster, wait_until_no_pending_changes). -spec wait_until_nodes_agree_about_ownership(cluster_id()) -> rt_util:result(). wait_until_nodes_agree_about_ownership(Cluster) -> - lager:error("wait_until_nodes_agree_about_ownership(~p) not implemented", [Cluster]), - ok. + gen_fsm:sync_send_event(Cluster, wait_until_nodes_agree_about_ownership). -spec wait_until_ring_converged(cluster_id()) -> rt_util:result(). wait_until_ring_converged(Cluster) -> - lager:error("wait_until_ring_converged(~p) not implemented", [Cluster]), - ok. + gen_fsm:sync_send_event(Cluster, wait_until_ring_converged). +%% @doc Returns the version of the cluster, `Cluster' +-spec version(cluster_id()) -> version(). +version(Cluster) -> + gen_fsm:sync_send_event(Cluster, version). + +%% @doc Performs a rolling upgrade of the cluster, `Cluster', to version, `NewVersion'. -spec upgrade(cluster_id(), version()) -> rt_util:result(). -upgrade(Cluster, Version) -> - lager:error("upgrade(~p, ~p) not implemented.", [Cluster, Version]), - ok. +upgrade(Cluster, NewVersion) -> + gen_fsm:sync_send_event(Cluster, {upgrade, NewVersion}). %% index creation ... diff --git a/src/rt_riak_node.erl b/src/rt_riak_node.erl index 4b525b506..2463d2270 100644 --- a/src/rt_riak_node.erl +++ b/src/rt_riak_node.erl @@ -10,12 +10,20 @@ -behaviour(gen_fsm). +-include_lib("eunit/include/eunit.hrl"). + %% API --export([assert_singleton/1, +-export([admin/2, + admin/3, + assert_singleton/1, + attach/2, + attach_direct/2, brutal_kill/1, claimant_according_to/1, + clean_data_dir/1, + clean_data_dir/2, commit/1, - configure/3, + console/2, get_ring/1, is_allocated/1, is_ready/1, @@ -26,12 +34,16 @@ partitions/1, ping/1, plan/1, + riak/2, + riak_repl/2, + search_cmd/2, start/1, start/2, - start_link/4, + start_link/5, status_of_according_to/1, stop/1, stop/2, + upgrade/2, wait_for_service/2, wait_until_pingable/1, wait_until_registered/2, @@ -47,21 +59,63 @@ -type host() :: string(). -type node_id() :: string(). --record(state, {host :: host(), +-record(configuration, {one :: proplists:proplist(), + two :: proplists:proplist()}). + +-record(state, {config :: #configuration{}, + host :: host(), id :: node_id(), + install_type :: module(), name :: node(), + transport :: module(), version :: string()}). %%%=================================================================== %%% API %%%=================================================================== +%% @doc Call 'bin/riak-admin' command on `Node' with arguments `Args' +-spec admin(node(), [term()]) -> {ok, term()} | rt_util:error(). +admin(Node, Args) -> + admin(Node, Args, []). + +%% @doc Call 'bin/riak-admin' command on `Node' with arguments `Args'. +%% The third parameter is a list of options. Valid options are: +%% * `return_exit_code' - Return the exit code along with the command output +-spec admin(node(), [term()], [term()]) -> {ok, term()} | rt_util:error(). +admin(Node, Args, Options) -> + gen_fsm:sync_send_event(Node, {admin, Node, Args, Options}). %% @doc Ensure that the specified node is a singleton node/cluster -- a node %% that owns 100% of the ring. -spec assert_singleton(node()) -> boolean(). assert_singleton(Node) -> - gen_fsm:sync_send_event(Node, check_singleton). + SingletonFlag = gen_fsm:sync_send_event(Node, check_singleton), + ?assert(SingletonFlag), + SingletonFlag. + +%% @doc Runs `riak attach' on a specific node, and tests for the expected behavoir. +%% Here's an example: ``` +%% rt_cmd_line:attach(Node, [{expect, "erlang.pipe.1 \(^D to exit\)"}, +%% {send, "riak_core_ring_manager:get_my_ring()."}, +%% {expect, "dict,"}, +%% {send, [4]}]), %% 4 = Ctrl + D''' +%% `{expect, String}' scans the output for the existance of the String. +%% These tuples are processed in order. +%% +%% `{send, String}' sends the string to the console. +%% Once a send is encountered, the buffer is discarded, and the next +%% expect will process based on the output following the sent data. +%% +-spec attach(node(), {expect, list()} | {send, list()}) -> {ok, term()} | rt_util:error(). +attach(Node, Expected) -> + gen_fsm:sync_send_event(Node, {attach, Expected}). + +%% @doc Runs 'riak attach-direct' on a specific node +%% @see rt_riak_node:attach/2 +-spec attach_direct(node(), {expect, list()} | {send, list()}) -> {ok, term()} | rt_util:error(). +attach_direct(Node, Expected) -> + gen_fsm:sync_send_event(Node, {attach_direct, Expected}). -spec brutal_kill(node()) -> rt_util:result(). brutal_kill(Node) -> @@ -72,15 +126,24 @@ brutal_kill(Node) -> claimant_according_to(Node) -> gen_fsm:sync_send_event(Node, claimant_according_to). +-spec clean_data_dir(node()) -> rt_util:result(). +clean_data_dir(Node) -> + gen_fsm:sync_send_event(Node, clean_data_dir). + +-spec clean_data_dir(node(), list()) -> rt_util:result(). +clean_data_dir(Node, SubDir) -> + gen_fsm:sync_send_event(Node, {clean_data_dir, SubDir}). + +%% @doc Runs `riak console' on a specific node +%% @see rt_riak_node:attach/2 +-spec console(node(), {expect, list()} | {send, list()}) -> {ok, term()} | rt_util:error(). +console(Node, Expected) -> + geb_fsm:sync_send_event(Node, {console, Expected}). + -spec commit(node()) -> rt_util:result(). commit(Node) -> gen_fsm:sync_send_event(Node, commit). -%% @doc Modifies the riak.conf and advanced.config --spec configure(node(), proplists:proplist(), proplists:proplist()) -> rt_util:result(). -configure(Node, Conf, AdvancedConfig) -> - gen_fsm:sync_send_event(Node, {configure, Conf, AdvancedConfig}). - %% @doc Get the raw ring for `Node'. -spec get_ring(node()) -> term(). get_ring(Node) -> @@ -128,6 +191,20 @@ ping(Node) -> plan(Node) -> gen_fsm:sync_send_event(Node, plan). +%% @doc Call 'bin/riak' command on `Node' with arguments `Args' +-spec riak(node(), [term()]) -> {ok, term()} | rt_util:error(). +riak(Node, Args) -> + gen_fsm:sync_send_event(Node, {riak, Args}). + +%% @doc Call 'bin/riak' command on `Node' with arguments `Args' +-spec riak_repl(node(), [term()]) -> {ok, term()} | rt_util:error(). +riak_repl(Node, Args) -> + gen_fsm:sync_send_event(Node, {riak_repl, Args}). + +-spec search_cmd(node(), [term()]) -> {ok, term()} | rt_util:error(). +search_cmd(Node, Args) -> + gen_fsm:sync_send_event(Node, {search_cmd, Args}). + -spec start(node()) -> rt_util:result(). start(Node) -> start(Node, true). @@ -139,9 +216,9 @@ start(Node, Wait) -> %% @doc Starts a gen_fsm process to configure, start, and %% manage a Riak node on the `Host' identified by `NodeId' %% and `NodeName' using Riak `Version' ({product, release}) --spec start_link(host(), node_id(), node(), string()) -> {ok, pid()} | ignore | rt_util:error(). -start_link(Host, NodeId, NodeName, Version) -> - Args = [Host, NodeId, NodeName, Version], +-spec start_link(host(), node_id(), node(), #configuration{}, string()) -> {ok, pid()} | ignore | rt_util:error(). +start_link(Host, NodeId, NodeName, Config, Version) -> + Args = [Host, NodeId, NodeName, Config, Version], gen_fsm:start_link(NodeName, ?MODULE, Args, []). %% @doc Return the cluster status of `Member' according to the ring @@ -159,6 +236,9 @@ stop(Node) -> stop(Node, Wait) -> gen_fsm:sync_send_event(Node, {stop, Wait}). +upgrade(Node, NewVersion) -> + gen_fsm:sync_send_event(Node, {upgrade, NewVersion}). + -spec wait_for_service(node(), [string()]) -> rt_util:result(). wait_for_service(Node, Services) -> gen_fsm:sync_send_event(Node, {wait_for_services, Services}). @@ -195,10 +275,11 @@ version(Node) -> %% {stop, StopReason} %% @end %%-------------------------------------------------------------------- -init([Host, NodeId, NodeName, Version]) -> +init([Host, NodeId, NodeName, Config, Version]) -> State = #state{host=Host, id=NodeId, name=NodeName, + config=Config, version=Version}, {ok, allocated, State}. From 957fc93c63944fbd3626bda9e96e6340f59cff4a Mon Sep 17 00:00:00 2001 From: John Burwell Date: Sun, 29 Mar 2015 23:53:41 -0400 Subject: [PATCH 121/157] Modifies rt_config:get/1 to default the continue_on_fail flag when no value has been specified via the command line or configuration file --- src/rt_config.erl | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/src/rt_config.erl b/src/rt_config.erl index 7f5e71108..77e048d0f 100644 --- a/src/rt_config.erl +++ b/src/rt_config.erl @@ -52,6 +52,8 @@ -define(UPGRADE_KEY, upgrade_paths). -define(PREVIOUS_VERSION, "1.4.12"). -define(LEGACY_VERSION, "1.3.4"). +-define(CONTINUE_ON_FAIL_KEY, continue_on_fail). +-define(DEFAULT_CONTINUE_ON_FAIL, false). %% @doc Get the value of an OS Environment variable. The arity 1 version of %% this function will fail the test if it is undefined. @@ -109,6 +111,8 @@ set(Key, Value) -> get(rt_max_wait_time) -> lager:info("rt_max_wait_time is deprecated. Please use rt_max_receive_wait_time instead."), rt_config:get(?RECEIVE_WAIT_TIME_KEY); +get(?CONTINUE_ON_FAIL_KEY) -> + get(?CONTINUE_ON_FAIL_KEY, ?DEFAULT_CONTINUE_ON_FAIL); get(Key) -> case kvc:path(Key, application:get_all_env(?CONFIG_NAMESPACE)) of [] -> @@ -307,4 +311,15 @@ get_version_path_test() -> ?assertEqual(version_to_tag(?PREVIOUS_VERSION_KEY), ?PREVIOUS_VERSION), ?assertEqual(version_to_tag(?LEGACY_VERSION_KEY), ?LEGACY_VERSION). +get_continue_on_fail_test() -> + clear(?CONTINUE_ON_FAIL_KEY), + ?assertEqual(?DEFAULT_CONTINUE_ON_FAIL, rt_config:get(?CONTINUE_ON_FAIL_KEY)), + + set(?CONTINUE_ON_FAIL_KEY, false), + ?assertEqual(false, rt_config:get(?CONTINUE_ON_FAIL_KEY)), + + clear(?CONTINUE_ON_FAIL_KEY), + set(?CONTINUE_ON_FAIL_KEY, true), + ?assertEqual(true, rt_config:get(?CONTINUE_ON_FAIL_KEY)). + -endif. From 41d6940159aa0ea60fff0906bfc7be2c3f6dd56b Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Mon, 30 Mar 2015 15:24:12 -0600 Subject: [PATCH 122/157] Roll back regression test to work with new testing infrastructure --- regression_test_wrapper.sh | 3 ++- tests/replication/replication.erl | 38 ++++++++++++++++--------------- 2 files changed, 22 insertions(+), 19 deletions(-) diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index 82017f1ff..82eaf4d3f 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -8,7 +8,8 @@ if [ -z $1 ]; then exit 1 fi -ALL_BACKEND_TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantine,gh_riak_core_155,pb_security,verify_search,verify_handoff,verify_capabilities,replication/repl_fs_stat_caching,replication/replication2,verify_handoff_mixed,verify_bitcask_tombstone2_upgrade,replication_object_reformat" +REPL_TEST_CASES="replication,replication_object_reformat,replication2,repl_fs_stat_caching" +ALL_BACKEND_TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantine,gh_riak_core_155,pb_security,verify_search,verify_handoff,verify_capabilities,verify_handoff_mixed,verify_bitcask_tombstone2_upgrade,${REPL_TEST_CASES}" BITCASK_BACKEND_TEST_CASES="$ALL_BACKEND_TEST_CASES,loaded_upgrade" ELEVELDB_BACKEND_TEST_CASES="verify_2i_aae,loaded_upgrade" diff --git a/tests/replication/replication.erl b/tests/replication/replication.erl index 7bcf7b8a4..c2463265c 100644 --- a/tests/replication/replication.erl +++ b/tests/replication/replication.erl @@ -16,8 +16,10 @@ confirm() -> {diff_batch_size, 10} ]} ], - rt_config:set_advanced_conf(all, Conf), - [ANodes, BNodes] = rt_cluster:build_clusters([3, 3]), + rt:set_advanced_conf(all, Conf), + [ANodes, BNodes] = rt:build_clusters([3, 3]), + rt:wait_for_cluster_service(ANodes, riak_repl), + rt:wait_for_cluster_service(BNodes, riak_repl), replication(ANodes, BNodes, false), pass. @@ -106,7 +108,7 @@ replication([AFirst|_] = ANodes, [BFirst|_] = BNodes, Connected) -> %% check that the keys we wrote initially aren't replicated yet, because %% we've disabled fullsync_on_connect lager:info("Check keys written before repl was connected are not present"), - Res2 = rt_systest:read(BFirst, 1, 100, TestBucket, 2), + Res2 = rt:systest_read(BFirst, 1, 100, TestBucket, 2), ?assertEqual(100, length(Res2)), start_and_wait_until_fullsync_complete(LeaderA), @@ -143,7 +145,7 @@ replication([AFirst|_] = ANodes, [BFirst|_] = BNodes, Connected) -> rt:log_to_nodes(AllNodes, "Testing master failover: stopping ~p", [LeaderA]), lager:info("Testing master failover: stopping ~p", [LeaderA]), - rt_node:stop(LeaderA), + rt:stop(LeaderA), rt:wait_until_unpingable(LeaderA), wait_until_leader(ASecond), @@ -166,7 +168,7 @@ replication([AFirst|_] = ANodes, [BFirst|_] = BNodes, Connected) -> LeaderB = rpc:call(BFirst, riak_repl_leader, leader_node, []), lager:info("Testing client failover: stopping ~p", [LeaderB]), - rt_node:stop(LeaderB), + rt:stop(LeaderB), rt:wait_until_unpingable(LeaderB), BSecond = hd(BNodes -- [LeaderB]), wait_until_leader(BSecond), @@ -197,7 +199,7 @@ replication([AFirst|_] = ANodes, [BFirst|_] = BNodes, Connected) -> %% lager:info("Restarting down node ~p", [LeaderA]), - rt_node:start(LeaderA), + rt:start(LeaderA), rt:wait_until_pingable(LeaderA), rt:wait_until_no_pending_changes(ANodes), wait_until_leader_converge(ANodes), @@ -265,7 +267,7 @@ replication([AFirst|_] = ANodes, [BFirst|_] = BNodes, Connected) -> end, lager:info("Restarting down node ~p", [LeaderB]), - rt_node:start(LeaderB), + rt:start(LeaderB), rt:wait_until_pingable(LeaderB), case nodes_all_have_version(ANodes, "1.1.0") of @@ -306,19 +308,19 @@ replication([AFirst|_] = ANodes, [BFirst|_] = BNodes, Connected) -> FullsyncOnly, 2)), lager:info("Check the fullsync only bucket didn't replicate the writes"), - Res6 = rt_systest:read(BSecond, 1, 100, FullsyncOnly, 2), + Res6 = rt:systest_read(BSecond, 1, 100, FullsyncOnly, 2), ?assertEqual(100, length(Res6)), lager:info("Check the realtime only bucket that was written to offline " "isn't replicated"), - Res7 = rt_systest:read(BSecond, 1, 100, RealtimeOnly, 2), + Res7 = rt:systest_read(BSecond, 1, 100, RealtimeOnly, 2), ?assertEqual(100, length(Res7)); _ -> timer:sleep(1000) end, lager:info("Check the {repl, false} bucket didn't replicate"), - Res8 = rt_systest:read(BSecond, 1, 100, NoRepl, 2), + Res8 = rt:systest_read(BSecond, 1, 100, NoRepl, 2), ?assertEqual(100, length(Res8)), %% do a fullsync, make sure that fullsync_only is replicated, but @@ -332,7 +334,7 @@ replication([AFirst|_] = ANodes, [BFirst|_] = BNodes, Connected) -> FullsyncOnly, 2)), lager:info("Check realtime only bucket didn't replicate"), - Res10 = rt_systest:read(BSecond, 1, 100, RealtimeOnly, 2), + Res10 = rt:systest_read(BSecond, 1, 100, RealtimeOnly, 2), ?assertEqual(100, length(Res10)), @@ -347,14 +349,14 @@ replication([AFirst|_] = ANodes, [BFirst|_] = BNodes, Connected) -> RealtimeOnly, 2)), lager:info("Check the older keys in the realtime bucket did not replicate"), - Res12 = rt_systest:read(BSecond, 1, 100, RealtimeOnly, 2), + Res12 = rt:systest_read(BSecond, 1, 100, RealtimeOnly, 2), ?assertEqual(100, length(Res12)); _ -> ok end, lager:info("Check {repl, false} bucket didn't replicate"), - Res13 = rt_systest:read(BSecond, 1, 100, NoRepl, 2), + Res13 = rt:systest_read(BSecond, 1, 100, NoRepl, 2), ?assertEqual(100, length(Res13)); _ -> ok @@ -517,7 +519,7 @@ start_and_wait_until_fullsync_complete(Node, Retries) -> lager:info("waiting for fullsync count to be ~p", [Count]), lager:info("Starting fullsync on ~p (~p)", [Node, - rt:node_version(rt:node_id(Node))]), + rtdev:node_version(rtdev:node_id(Node))]), rpc:call(Node, riak_repl_console, start_fullsync, [[]]), %% sleep because of the old bug where stats will crash if you call it too @@ -685,20 +687,20 @@ wait_until_no_connection(Node) -> wait_for_reads(Node, Start, End, Bucket, R) -> rt:wait_until(Node, fun(_) -> - rt_systest:read(Node, Start, End, Bucket, R) == [] + rt:systest_read(Node, Start, End, Bucket, R) == [] end), - Reads = rt_systest:read(Node, Start, End, Bucket, R), + Reads = rt:systest_read(Node, Start, End, Bucket, R), lager:info("Reads: ~p", [Reads]), length(Reads). do_write(Node, Start, End, Bucket, W) -> - case rt_systest:write(Node, Start, End, Bucket, W) of + case rt:systest_write(Node, Start, End, Bucket, W) of [] -> []; Errors -> lager:warning("~p errors while writing: ~p", [length(Errors), Errors]), timer:sleep(1000), - lists:flatten([rt_systest:write(Node, S, S, Bucket, W) || + lists:flatten([rt:systest_write(Node, S, S, Bucket, W) || {S, _Error} <- Errors]) end. From c957e6ce72ea76ef5dd7b54bdb86bc1de8dc8bd7 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Mon, 30 Mar 2015 16:47:02 -0400 Subject: [PATCH 123/157] verifymembackend passes the using v1 wrapper * Modifies riak_test_runner to set the backend using rt:set_backend/1 before each old style test is executed * Removes backend configuration from the rt module wrapper to work around the race condition between the framework and test cases setting the backend manually --- regression_test_wrapper.sh | 2 +- src/riak_test_runner.erl | 3 ++- src/rt.erl | 38 +++----------------------------------- 3 files changed, 6 insertions(+), 37 deletions(-) diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index 82eaf4d3f..c3e0cc1bc 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -13,7 +13,7 @@ ALL_BACKEND_TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_st BITCASK_BACKEND_TEST_CASES="$ALL_BACKEND_TEST_CASES,loaded_upgrade" ELEVELDB_BACKEND_TEST_CASES="verify_2i_aae,loaded_upgrade" -MEMORY_BACKEND_TEST_CASES="verify_2i_aae" +MEMORY_BACKEND_TEST_CASES="verify_2i_aae,verify_membackend" ROOT_RESULTS_DIR="results/regression" RESULTS=`date +"%y%m%d%H%M%s"` diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index 105e13fe9..22593dad0 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -181,13 +181,14 @@ setup(timeout, State=#state{test_type=TestType, lager:info("Test Runner: ~s", [UName]), {StartVersion, OtherVersions} = test_versions(Properties), - Config = rt_backend:set(Backend, rt_properties:get(config, Properties)), case TestType of new -> + Config = rt_backend:set(Backend, rt_properties:get(config, Properties)), NodeIds = rt_properties:get(node_ids, Properties), Services = rt_properties:get(required_services, Properties); old -> + Config = rt:set_backend(Backend), NodeIds = [], Services = [], lager:warning("Test ~p has not been ported to the new framework.", [TestModule]) diff --git a/src/rt.erl b/src/rt.erl index db9f89ae0..6a1623376 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -259,8 +259,7 @@ deploy_nodes(NumNodes, InitialConfig, Services) when is_integer(NumNodes) -> deploy_nodes(NodeIds, NodeMap, Version, InitialConfig, Services). deploy_nodes(NodeIds, NodeMap, Version, Config, Services) -> - BackendConfig = maybe_set_backend(rt_config:get(rt_backend), Config), - _ = rt_harness_util:deploy_nodes(NodeIds, NodeMap, Version, BackendConfig, Services), + _ = rt_harness_util:deploy_nodes(NodeIds, NodeMap, Version, Config, Services), lists:foldl(fun({_, NodeName}, Nodes) -> [NodeName|Nodes] end, [], NodeMap). @@ -270,20 +269,6 @@ map_version_and_config({Vsn, Cfg}) -> map_version_and_config(Vsn) -> {rt_config:version_to_tag(Vsn), rt_properties:default_config()}. -maybe_set_backend(Backend, Config) -> - maybe_set_backend(lists:keyfind(riak_kv, 1, Config), Backend, Config). - -maybe_set_backend(false, riak_kv_eleveldb_backend, Config) -> - maybe_set_backend(false, eleveldb, Config); -maybe_set_backend(false, riak_kv_memory_backend, Config) -> - maybe_set_backend(false, memory, Config); -maybe_set_backend(false, Backend, Config) -> - rt_backend:set(Backend, Config); -maybe_set_backend({storage_backend, _}=_KVSection, _Backend, Config) -> - Config; -maybe_set_backend({riak_kv, KVSection}, Backend, Config) -> - maybe_set_backend(lists:keyfind(storage_backend, 1, KVSection), Backend, Config). - allocate_nodes(NumNodes, Version) when is_atom(Version) -> allocate_nodes(NumNodes, atom_to_list(Version)); allocate_nodes(NumNodes, Version) -> @@ -1289,6 +1274,8 @@ set_backend(Backend) -> -spec set_backend(atom(), [{atom(), term()}]) -> atom()|[atom()]. set_backend(bitcask, _) -> set_backend(riak_kv_bitcask_backend); +set_backend(leveldb, Extras) -> + set_backend(eleveldb, Extras); set_backend(eleveldb, _) -> set_backend(riak_kv_eleveldb_backend); set_backend(memory, _) -> @@ -1426,25 +1413,6 @@ wait_for_control(VersionedNodes) -> -ifdef(TEST). -verify_backend(InputConfig, ExpectedBackend) -> - verify_backend(InputConfig, ExpectedBackend, ExpectedBackend). - -verify_backend(InputConfig, Backend, ExpectedBackend) -> - ?_test(begin - ActualConfig = maybe_set_backend(Backend, InputConfig), - io:format("ActualConfig: ~p", [ActualConfig]), - [{riak_kv, [{storage_backend, ActualBackend}]}] = ActualConfig, - ?assertEqual(ExpectedBackend, ActualBackend) - end). - -maybe_set_backend_test_() -> - {foreach, - fun() -> ok end, - [verify_backend([], riak_kv_bitcask_backend), - verify_backend([], riak_kv_eleveldb_backend), - verify_backend([], riak_kv_memory_backend), - verify_backend([{riak_kv, [{storage_backend, riak_kv_bitcask_backend}]}], riak_kv_eleveldb_backend, riak_kv_bitcask_backend)]}. - verify_product(Applications, ExpectedApplication) -> ?_test(begin meck:new(rpc, [unstick]), From 88d3c40202f008480aa94713a31a161bc75af38c Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Tue, 31 Mar 2015 15:09:29 -0600 Subject: [PATCH 124/157] Revert verify_membackend back to master version --- tests/verify_membackend.erl | 21 +++++++++------------ 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/tests/verify_membackend.erl b/tests/verify_membackend.erl index a7636e7cc..536ef1f56 100644 --- a/tests/verify_membackend.erl +++ b/tests/verify_membackend.erl @@ -100,7 +100,7 @@ check_leave_and_expiry(NodeA, NodeB) -> _ = rt:systest_read(NodeA, 1, 100, ?BUCKET, 2), timer:sleep(timer:seconds(5)), Res = rt:systest_read(NodeA, 1, 100, ?BUCKET, 2), - + ?assertEqual(100, length(Res)), ok. @@ -201,10 +201,7 @@ put_until_changed(Pid, Node, Key) -> end. mkconf(Test, Mode) -> - RiakCore = [ - {ring_creation_size, 4} - ], - MembConfig = + MembConfig = case Test of ttl -> [{ttl, 200}]; @@ -220,7 +217,9 @@ mkconf(Test, Mode) -> rt:set_backend(memory), [ - {riak_core, RiakCore}, + {riak_core, [ + {ring_creation_size, 4} + ]}, {riak_kv, [ {anti_entropy, {off, []}}, {delete_mode, immediate}, @@ -230,7 +229,9 @@ mkconf(Test, Mode) -> multi -> rt:set_backend(multi), [ - {riak_core, RiakCore}, + {riak_core, [ + {ring_creation_size, 4} + ]}, {riak_kv, [ {anti_entropy, {off, []}}, {delete_mode, immediate}, @@ -255,7 +256,7 @@ get_remote_vnode_pid(Node) -> get_used_space(VNode, Node) -> S = rpc:call(Node, sys, get_state, [VNode]), Mode = get(mode), - Version = rt:get_version(Node), + Version = rt:get_version(), %% lager:info("version mode ~p", [{Version, Mode}]), TwoOhReg = fun(X) -> @@ -280,10 +281,6 @@ get_used_space(VNode, Node) -> TwoOhMulti; {<<"riak_ee-2.0",_/binary>>, multi} -> TwoOhMulti; - {<<"head",_/binary>>, regular} -> - TwoOhReg; - {<<"head",_/binary>>, multi} -> - TwoOhMulti; _Else -> lager:error("didn't understand version/mode tuple ~p", [{Version, Mode}]), From c5fe86c78fd1dff270e5a96a4b6e5d53943f2295 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Thu, 2 Apr 2015 08:13:59 -0600 Subject: [PATCH 125/157] Finally get replication2_pg passing in the regression suite --- regression_test_wrapper.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index c3e0cc1bc..74a354313 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -8,7 +8,7 @@ if [ -z $1 ]; then exit 1 fi -REPL_TEST_CASES="replication,replication_object_reformat,replication2,repl_fs_stat_caching" +REPL_TEST_CASES="replication,replication_object_reformat,replication2,repl_fs_stat_caching,replication2_pg:test_12_pg_mode_repl12,replication2_pg:test_12_pg_mode_repl12_ssl,replication2_pg:test_12_pg_mode_repl_mixed,replication2_pg:test_12_pg_mode_repl_mixed_ssl,replication2_pg:test_basic_pg_mode_mixed,replication2_pg:test_basic_pg_mode_mixed_ssl,replication2_pg:test_basic_pg_mode_repl13,replication2_pg:test_bidirectional_pg,replication2_pg:test_bidirectional_pg_ssl,replication2_pg:test_mixed_pg,replication2_pg:test_mixed_pg_ssl,replication2_pg:test_multiple_sink_pg,replication2_pg:test_multiple_sink_pg_ssl,replication2_pg:test_pg_proxy,replication2_pg:test_pg_proxy_ssl" ALL_BACKEND_TEST_CASES="always_pass_test,verify_riak_stats,verify_down,verify_staged_clustering,verify_leave,partition_repair,verify_build_cluster,riak_control_authentication,always_fail_test,basic_command_line,jmx_verify,verify_aae,verify_claimant,verify_object_limits,ensemble_interleave,ensemble_byzantine,gh_riak_core_155,pb_security,verify_search,verify_handoff,verify_capabilities,verify_handoff_mixed,verify_bitcask_tombstone2_upgrade,${REPL_TEST_CASES}" BITCASK_BACKEND_TEST_CASES="$ALL_BACKEND_TEST_CASES,loaded_upgrade" From 867228e0eb43046c4f3b916fb58ba8687850a276 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Thu, 2 Apr 2015 11:56:30 -0400 Subject: [PATCH 126/157] Continued work on implementing the new r_t v2 API --- src/rt_riak_node.erl | 138 +++++++++++++++++++++++++++---------------- 1 file changed, 87 insertions(+), 51 deletions(-) diff --git a/src/rt_riak_node.erl b/src/rt_riak_node.erl index 2463d2270..9ac955956 100644 --- a/src/rt_riak_node.erl +++ b/src/rt_riak_node.erl @@ -51,8 +51,8 @@ version/1]). %% gen_fsm callbacks --export([init/1, state_name/2, state_name/3, handle_event/3, - handle_sync_event/4, handle_info/3, ready/3, terminate/3, code_change/4]). +-export([init/1, handle_event/3, stopped/3, handle_sync_event/4, + handle_info/3, ready/3, terminate/3, code_change/4]). -define(SERVER, ?MODULE). @@ -62,11 +62,23 @@ -record(configuration, {one :: proplists:proplist(), two :: proplists:proplist()}). +-type path() :: string(). +-record(directory_overlay, {bin_dir :: path(), + conf_dir :: path(), + data_dir :: path(), + home_dir :: path(), + lib_dir :: path(), + log_dir :: path()}). + + +-type command() :: string(). -record(state, {config :: #configuration{}, host :: host(), id :: node_id(), - install_type :: module(), + directory_overlay :: #directory_overlay{}, name :: node(), + start_command :: command(), + stop_command :: command(), transport :: module(), version :: string()}). @@ -180,12 +192,11 @@ owners_according_to(Node) -> %% @doc Get list of partitions owned by node (primary). -spec partitions(node()) -> [term()]. partitions(Node) -> - lager:error("partitions(~p) is not implemented.", [Node]), - []. + gen_fsm:sync_send_event(Node, partitions). -spec ping(node()) -> boolean(). ping(Node) -> - gen_fsm:sync_send_event(Node, ping). + gen_fsm:sync_send_all_state_event(Node, ping). -spec plan(node()) -> rt_util:result(). plan(Node) -> @@ -227,7 +238,6 @@ start_link(Host, NodeId, NodeName, Config, Version) -> status_of_according_to(Node) -> gen_fsm:sync_send_event(Node, status_of_according_to). - -spec stop(node()) -> rt_util:result(). stop(Node) -> stop(Node, true). @@ -256,7 +266,7 @@ wait_until_unpingable(Node) -> gen_fsm:sync_send_event(Node, wait_until_unpingable). version(Node) -> - gen_fsm:sync_send_event(Node, version). + gen_fsm:sync_send_all_state_event(Node, version). %%%=================================================================== %%% gen_fsm callbacks @@ -275,44 +285,60 @@ version(Node) -> %% {stop, StopReason} %% @end %%-------------------------------------------------------------------- -init([Host, NodeId, NodeName, Config, Version]) -> +init([Host, NodeType, NodeId, NodeName, Config, Version]) -> + DirOverlay = create_directory_overlay(NodeType, Version), State = #state{host=Host, id=NodeId, name=NodeName, config=Config, + directory_overlay=DirOverlay, + start_command=start_command(NodeType, DirOverlay), + stop_command=stop_command(NodeType, DirOverlay), version=Version}, {ok, allocated, State}. -%%-------------------------------------------------------------------- -%% @private -%% @doc -%% There should be one instance of this function for each possible -%% state name. Whenever a gen_fsm receives an event sent using -%% gen_fsm:send_event/2, the instance of this function with the same -%% name as the current state name StateName is called to handle -%% the event. It is also called if a timeout occurs. -%% -%% @spec state_name(Event, State) -> -%% {next_state, NextStateName, NextState} | -%% {next_state, NextStateName, NextState, Timeout} | -%% {stop, Reason, NewState} -%% @end -%%-------------------------------------------------------------------- -state_name(_Event, State) -> - {next_state, state_name, State}. +stopped(start, _From, State) -> + transition_stopped_to_started(State); +stopped(stop, _From, State=#state{name=NodeName}) -> + lager:debug("Stop called on an already stopped node ~p", [NodeName]), + {reply, ok, stopped, State}; +stopped(_Event, _From, State) -> + %% The state of the node is not harmed. Therefore, we leave the FSM running + %% in the stopped state, but refuse to execute the command ... + {reply, {error, invalid_state}, stopped, State}. + + +transition_stopped_to_started(State=#state{start_command=StartCommand, transport=Transport}) -> + transition_stopped_to_started(Transport:exec(StartCommand), State). + +transition_stopped_to_started(ok, State) -> + {reply, ok, started, State}; +transition_stopped_to_started(Error={error, _}, State) -> + {stop, Error, State}. + + +ready({admin, Args, Options}, _From, State=#state{directory_overlay=DirOverlay, transport=Transport}) -> + Result = Transport:exec(riak_admin_path(DirOverlay), Args, Options), + {reply, Result, ready, State}; ready(get_ring, _From, #state{name=NodeName}=State) -> {ok, Ring} = maybe_get_ring(NodeName), {reply, Ring, ready, State}; ready(members_according_to, _From, #state{name=NodeName}=State) -> Members = maybe_members_according_to(NodeName), {reply, Members, ready, State}; +ready(owners_according_to, _From, #state{name=NodeName}=State) -> + Owners = maybe_owners_according_to(NodeName), + {reply, Owners, ready, State}; ready(partitions, _From, #state{name=NodeName}=State) -> Partitions = maybe_partitions(NodeName), {reply, Partitions, ready, State}; -ready(owners_according_to, _From, #state{name=NodeName}=State) -> - Owners = maybe_owners_according_to(NodeName), - {reply, Owners, ready, State}. +ready({riak, Args}, _From, State=#state{directory_overlay=DirOverlay, transport=Transport}) -> + Result = Transport:exec(riak_path(DirOverlay), Args), + {reply, Result, ready, State}; +ready({riak_repl, Args}, _From, State=#state{directory_overlay=DirOverlay, transport=Transport}) -> + Result = Transport:exec(riak_repl_path(DirOverlay), Args), + {reply, Result, ready, State}. -spec maybe_get_ring(node()) -> rt_rpc_result(). maybe_get_ring(NodeName) -> @@ -358,28 +384,6 @@ maybe_rpc_call({badrpc, _}) -> maybe_rpc_call(Result) -> Result. -%%-------------------------------------------------------------------- -%% @private -%% @doc -%% There should be one instance of this function for each possible -%% state name. Whenever a gen_fsm receives an event sent using -%% gen_fsm:sync_send_event/[2,3], the instance of this function with -%% the same name as the current state name StateName is called to -%% handle the event. -%% -%% @spec state_name(Event, From, State) -> -%% {next_state, NextStateName, NextState} | -%% {next_state, NextStateName, NextState, Timeout} | -%% {reply, Reply, NextStateName, NextState} | -%% {reply, Reply, NextStateName, NextState, Timeout} | -%% {stop, Reason, NewState} | -%% {stop, Reason, Reply, NewState} -%% @end -%%-------------------------------------------------------------------- -state_name(_Event, _From, State) -> - Reply = ok, - {reply, Reply, state_name, State}. - %%-------------------------------------------------------------------- %% @private %% @doc @@ -461,3 +465,35 @@ code_change(_OldVsn, StateName, State, _Extra) -> %%%=================================================================== %%% Internal functions %%%=================================================================== + +%% TODO Convert to the version() type when it is exported .. +-spec create_directory_overlay({atom(), path()}, {string(), string()}) -> #directory_overlay{}. +create_directory_overlay([devrel, RootPath], {Product, Version}) -> + HomeDir = filename:join([RootPath], Product ++ "-" ++ Version), + #directory_overlay{bin_dir=filename:join([HomeDir, "bin"]), + conf_dir=filename:join([HomeDir, "etc"]), + data_dir=filename:join([HomeDir, "data"]), + home_dir=HomeDir, + lib_dir=filename:join([HomeDir, "lib"]), + log_dir=filename:join([HomeDir, "log"])}. + +-spec start_command({devrel, string()}, #directory_overlay{}) -> command(). +start_command({devrel, _}, DirOverlay) -> + riak_path(DirOverlay) ++ " " ++ "start". + +-spec stop_command({devrel, string()}, #directory_overlay{}) -> command(). +stop_command({devrel, _}, DirOverlay) -> + riak_path(DirOverlay) ++ " " ++ "stop". + +-spec riak_path(#directory_overlay{}) -> path(). +riak_path(#directory_overlay{bin_dir=BinDir}) -> + filename:join([BinDir, "riak"]). + +-spec riak_admin_path(#directory_overlay{}) -> path(). +riak_admin_path(#directory_overlay{bin_dir=BinDir}) -> + filename:join([BinDir, "riak-admin"]). + +-spec riak_repl_path(#directory_overlay{}) -> path(). +riak_repl_path(#directory_overlay{bin_dir=BinDir}) -> + filename:join([BinDir, "riak-repl"]). + From 06045605aa01f16c18197f3525fca2e57f024e6e Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Fri, 10 Apr 2015 14:16:57 -0600 Subject: [PATCH 127/157] Initial introduction of rt_test_plan record to wrap tests --- src/giddyup.erl | 2 +- src/riak_test_escript.erl | 98 +++++----- src/riak_test_executor.erl | 69 ++++--- src/riak_test_runner.erl | 38 ++-- src/rt_bucket_types.erl | 10 ++ src/rt_config.erl | 8 + src/rt_planner.erl | 359 +++++++++++++++++++++++++++++++++++++ src/rt_properties2.erl | 234 ++++++++++++++++++++++++ src/rt_test_plan.erl | 199 ++++++++++++++++++++ src/smoke_test_escript.erl | 2 +- 10 files changed, 915 insertions(+), 104 deletions(-) create mode 100644 src/rt_planner.erl create mode 100644 src/rt_properties2.erl create mode 100644 src/rt_test_plan.erl diff --git a/src/giddyup.erl b/src/giddyup.erl index 1902d0f8a..42580d2f0 100644 --- a/src/giddyup.erl +++ b/src/giddyup.erl @@ -58,7 +58,7 @@ get_schema(Platform) -> get_schema(Platform, Retries) -> Host = rt_config:get(giddyup_host), - Project = rt_config:get(rt_project), + Project = rt_config:get(giddyup_project), Version = rt:get_version(), URL = lists:flatten(io_lib:format("http://~s/projects/~s?platform=~s&version=~s", [Host, Project, Platform, Version])), lager:info("giddyup url: ~s", [URL]), diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index 88e30538e..f19f8d778 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -44,38 +44,37 @@ main(Args) -> finalize(Results, ParsedArgs). prepare(ParsedArgs, Tests, NonTests) -> - lager:notice("Tests to run: ~p~n", [Tests]), + lager:notice("Test to run: ~p", [[rt_test_plan:get_module(Test) || Test <- Tests]]), case NonTests of [] -> ok; _ -> - lager:notice("These modules are not runnable tests: ~p~n", - [[NTMod || {NTMod, _} <- NonTests]]) + lager:notice("Test not to run: ~p", [[rt_test_plan:get_module(Test) || Test <- NonTests]]) end, ok = erlang_setup(ParsedArgs), test_setup(). -execute(Tests, ParsedArgs, _HarnessArgs) -> +execute(TestPlans, ParsedArgs, _HarnessArgs) -> OutDir = proplists:get_value(outdir, ParsedArgs), Report = report(ParsedArgs), UpgradeList = upgrade_list( proplists:get_value(upgrade_path, ParsedArgs)), - Backend = proplists:get_value(backend, ParsedArgs, bitcask), - {ok, Executor} = riak_test_executor:start_link(Tests, - Backend, + {ok, Executor} = riak_test_executor:start_link(TestPlans, OutDir, Report, UpgradeList, self()), - wait_for_results(Executor, [], length(Tests), 0). + wait_for_results(Executor, [], length(TestPlans), 0). report_results(Results, Verbose) -> %% TODO: Good place to also do giddyup reporting and provide a %% place for extending to any other reporting sources that might %% be useful. - print_summary(Results, undefined, Verbose), + + ModuleResults = [{rt_test_plan:get_module(TestPlan), PassFail, Duration} || {TestPlan, PassFail, Duration} <- Results], + print_summary(ModuleResults, undefined, Verbose), ok. %% TestResults = run_tests(Tests, Outdir, Report, HarnessArgs), @@ -121,6 +120,7 @@ finalize(TestResults, Args) -> Teardown = not proplists:get_value(keep, Args, false), maybe_teardown(Teardown, TestResults), ok. + %% Option Name, Short Code, Long Code, Argument Spec, Help Message cli_options() -> [ @@ -184,7 +184,7 @@ report(ParsedArgs) -> undefined -> undefined; "config" -> - rt_config:get(platform, undefined); + rt_config:get(giddyup_platform, undefined); R -> R end. @@ -212,7 +212,8 @@ help_or_parse_tests(ParsedArgs, HarnessArgs, false) -> maybe_override_setting(continue_on_fail, true, ParsedArgs), TestData = compose_test_data(ParsedArgs), - {Tests, NonTests} = which_tests_to_run(report(ParsedArgs), TestData), + Backend = proplists:get_value(backend, ParsedArgs, bitcask), + {Tests, NonTests} = wrap_test_in_test_plan(report(ParsedArgs), Backend, TestData), Offset = rt_config:get(offset, undefined), Workers = rt_config:get(workers, undefined), shuffle_tests(ParsedArgs, HarnessArgs, Tests, NonTests, Offset, Workers). @@ -235,6 +236,7 @@ load_initial_config(ParsedArgs) -> proplists:get_value(file, ParsedArgs)). shuffle_tests(_, _, [], _, _, _) -> + io:format("No tests are scheduled to run~n"), lager:error("No tests are scheduled to run~n"), halt(1); shuffle_tests(ParsedArgs, HarnessArgs, Tests, NonTests, undefined, _) -> @@ -364,49 +366,41 @@ extract_test_names(Test, {CodePaths, TestNames}) -> {[filename:dirname(Test) | CodePaths], [list_to_atom(filename:rootname(filename:basename(Test))) | TestNames]}. -which_tests_to_run(undefined, CommandLineTests) -> - lists:partition(fun is_runnable_test/1, CommandLineTests); -which_tests_to_run(Platform, []) -> - giddyup:get_suite(Platform); -which_tests_to_run(Platform, CommandLineTests) -> - Suite = filter_zip_suite(Platform, CommandLineTests), - lists:partition(fun is_runnable_test/1, - lists:foldr(fun filter_merge_tests/2, [], Suite)). - -filter_zip_suite(Platform, CommandLineTests) -> - [ {SModule, SMeta, CMeta} || {SModule, SMeta} <- giddyup:get_suite(Platform), - {CModule, CMeta} <- CommandLineTests, - SModule =:= CModule]. - -filter_merge_tests({Module, SMeta, CMeta}, Tests) -> - case filter_merge_meta(SMeta, CMeta, [backend, upgrade_version]) of - false -> - Tests; - Meta -> - [{Module, Meta}|Tests] - end. - -filter_merge_meta(SMeta, _CMeta, []) -> - SMeta; -filter_merge_meta(SMeta, CMeta, [Field|Rest]) -> - case {kvc:value(Field, SMeta, undefined), kvc:value(Field, CMeta, undefined)} of - {X, X} -> - filter_merge_meta(SMeta, CMeta, Rest); - {_, undefined} -> - filter_merge_meta(SMeta, CMeta, Rest); - {undefined, X} -> - filter_merge_meta(lists:keystore(Field, 1, SMeta, {Field, X}), CMeta, Rest); +%% @doc Determine which tests to run based on command-line argument +%% If the platform is defined, consult GiddyUp, otherwise just shovel +%% the whole thing into the Planner +-spec(load_up_test_planner(string() | undefined, string(), list()) -> list()). +load_up_test_planner(undefined, Backend, CommandLineTests) -> + [rt_planner:add_test_plan(Name, undefined, Backend, undefined, undefined) || Name <- CommandLineTests]; +%% GiddyUp Flavor +load_up_test_planner(Platform, Backend, CommandLineTests) -> + rt_planner:load_from_giddyup(Platform, Backend, CommandLineTests). + +%% @doc Push all of the test into the Planner for now and wrap them in an `rt_test_plan' +%% TODO: Let the Planner do the work, not the riak_test_executor +-spec(wrap_test_in_test_plan(string(), string(), [atom()]) -> {list(), list()}). +wrap_test_in_test_plan(Platform, Backend, CommandLineTests) -> + %% ibrowse neededfor GiddyUp + load_and_start(ibrowse), + {ok, _Pid} = rt_planner:start_link(), + load_up_test_planner(Platform, Backend, CommandLineTests), + TestPlans = [rt_planner:fetch_test_plan() || _ <- lists:seq(1, rt_planner:number_of_plans())], + NonRunnableTestPlans = [rt_planner:fetch_test_non_runnable_plan() || _ <- lists:seq(1, rt_planner:number_of_non_runable_plans())], + rt_planner:stop(), + {TestPlans, NonRunnableTestPlans}. + +%% @doc Pull all jobs from the Planner +%% Better than using rt_planner:number_of_plans/0 +-spec(fetch_all_test_plans(list()) -> list()). +fetch_all_test_plans(Acc) -> + Plan = rt_planner:fetch_test_plan(), + case Plan of + empty -> + Acc; _ -> - false + fetch_all_test_plans([Plan|Acc]) end. -%% Check for api compatibility -is_runnable_test(TestModule) -> - {Mod, Fun} = riak_test_runner:function_name(confirm, TestModule), - code:ensure_loaded(Mod), - erlang:function_exported(Mod, Fun, 0) orelse - erlang:function_exported(Mod, Fun, 1). - get_group_tests(Tests, Groups) -> lists:filter(fun(Test) -> Mod = list_to_atom(Test), @@ -546,6 +540,8 @@ print_summary(TestResults, _CoverResult, Verbose) -> true -> Rows = [format_test_row(Result, Width) || Result <- TestResults], + %% TODO: Remove once clique table is fixed + [lager:debug("ROW ~p", [Row]) || Row <- Rows], Table = clique_table:autosize_create_table(?HEADER, Rows), io:format("~ts~n", [Table]); false -> diff --git a/src/riak_test_executor.erl b/src/riak_test_executor.erl index a628b19a3..b80562316 100644 --- a/src/riak_test_executor.erl +++ b/src/riak_test_executor.erl @@ -3,7 +3,7 @@ -behavior(gen_fsm). %% API --export([start_link/6, +-export([start_link/5, send_event/1, stop/0]). @@ -28,7 +28,6 @@ running_tests=[] :: [atom()], waiting_tests=[] :: [atom()], notify_pid :: pid(), - backend :: atom(), upgrade_list :: [string()], test_properties :: [proplists:proplist()], runner_pids=[] :: [pid()], @@ -42,9 +41,9 @@ %%%=================================================================== %% @doc Start the test executor --spec start_link(atom(), atom(), string(), string(), [string()], pid()) -> {ok, pid()} | ignore | {error, term()}. -start_link(Tests, Backend, LogDir, ReportInfo, UpgradeList, NotifyPid) -> - Args = [Tests, Backend, LogDir, ReportInfo, UpgradeList, NotifyPid], +-spec start_link(atom(), string(), string(), [string()], pid()) -> {ok, pid()} | ignore | {error, term()}. +start_link(Tests, LogDir, ReportInfo, UpgradeList, NotifyPid) -> + Args = [Tests, LogDir, ReportInfo, UpgradeList, NotifyPid], gen_fsm:start_link({local, ?MODULE}, ?MODULE, Args, []). send_event(Msg) -> @@ -59,19 +58,14 @@ stop() -> %%% gen_fsm callbacks %%%=================================================================== -init([Tests, Backend, LogDir, ReportInfo, UpgradeList, NotifyPid]) -> +init([Tests, LogDir, ReportInfo, UpgradeList, NotifyPid]) -> %% TODO Change the default when parallel execution support is implemented -jsb ExecutionMode = rt_config:get(rt_execution_mode, serial), - %% TODO: Remove after all tests ported 2.0 -- workaround to support - %% backend command line argument fo v1 cluster provisioning -jsb - rt_config:set(rt_backend, Backend), - ContinueOnFail = rt_config:get(continue_on_fail), lager:notice("Starting the Riak Test executor in ~p execution mode", [ExecutionMode]), State = #state{pending_tests=Tests, - backend=Backend, log_dir=LogDir, report_info=ReportInfo, upgrade_list=UpgradeList, @@ -165,18 +159,18 @@ launch_test({nodes, Nodes, NodeMap}, State) -> %% Spawn a test runner for the head of pending. If pending is now %% empty transition to `wait_for_completion'; otherwise, %% transition to `request_nodes'. - #state{pending_tests=[NextTest | RestPending], + #state{pending_tests=[NextTestPlan | RestPending], execution_mode=ExecutionMode, - backend=Backend, test_properties=PropertiesList, runner_pids=Pids, running_tests=Running, continue_on_fail=ContinueOnFail} = State, - lager:debug("Executing test ~p in mode ~p", [NextTest, ExecutionMode]), - {NextTest, TestProps} = lists:keyfind(NextTest, 1, PropertiesList), + NextTestModule = rt_test_plan:get_module(NextTestPlan), + lager:debug("Executing test ~p in mode ~p", [NextTestModule, ExecutionMode]), + {NextTestPlan, TestProps} = lists:keyfind(NextTestPlan, 1, PropertiesList), UpdTestProps = rt_properties:set([{node_map, NodeMap}, {node_ids, Nodes}], TestProps), - {RunnerPids, RunningTests} = run_test(ExecutionMode, NextTest, Backend, UpdTestProps, + {RunnerPids, RunningTests} = run_test(ExecutionMode, NextTestPlan, UpdTestProps, Pids, Running, ContinueOnFail), UpdState = State#state{pending_tests=RestPending, execution_mode=ExecutionMode, @@ -184,15 +178,15 @@ launch_test({nodes, Nodes, NodeMap}, State) -> running_tests=RunningTests}, launch_test_transition(UpdState); -launch_test({test_complete, Test, Pid, Results, Duration}, State) -> +launch_test({test_complete, TestPlan, Pid, Results, Duration}, State) -> #state{pending_tests=Pending, waiting_tests=Waiting, running_tests=Running, runner_pids=Pids, execution_mode=ExecutionMode} = State, %% Report results - report_results(Test, Results, Duration, State), - UpdState = State#state{running_tests=lists:delete(Test, Running), + report_results(TestPlan, Results, Duration, State), + UpdState = State#state{running_tests=lists:delete(TestPlan, Running), runner_pids=lists:delete(Pid, Pids), pending_tests=Pending++Waiting, waiting_tests=[], @@ -202,26 +196,26 @@ launch_test(Event, State) -> lager:error("Unknown event ~p with state ~p.", [Event, State]), ok. -maybe_reserve_nodes(NextTest, TestProps) -> +maybe_reserve_nodes(NextTestPlan, TestProps) -> VersionsToTest = versions_to_test(TestProps), - maybe_reserve_nodes(erlang:function_exported(NextTest, confirm, 1), - NextTest, VersionsToTest, TestProps). + maybe_reserve_nodes(erlang:function_exported(rt_test_plan:get_module(NextTestPlan), confirm, 1), + NextTestPlan, VersionsToTest, TestProps). maybe_reserve_nodes(true, NextTest, VersionsToTest, TestProps) -> NodeCount = rt_properties:get(node_count, TestProps), %% Send async request to node manager - lager:notice("Requesting ~p nodes for the next test, ~p", [NodeCount, NextTest]), + lager:notice("Requesting ~p nodes for the next test, ~p", [NodeCount, rt_test_plan:get_module(NextTest)]), node_manager:reserve_nodes(NodeCount, VersionsToTest, reservation_notify_fun()); maybe_reserve_nodes(false, NextTest, VersionsToTest, _TestProps) -> - lager:warning("~p is an old style test that requires conversion.", [NextTest]), + lager:warning("~p is an old style test that requires conversion.", [rt_test_plan:get_module(NextTest)]), node_manager:reserve_nodes(0, VersionsToTest, reservation_notify_fun()), ok. wait_for_completion({test_complete, Test, Pid, Results, Duration}, State) -> - lager:debug("Test ~p complete", [Test]), + lager:debug("Test ~p complete", [rt_test_plan:get_module(Test)]), #state{pending_tests=Pending, waiting_tests=Waiting, running_tests=Running, @@ -256,8 +250,8 @@ wait_for_completion(_Event, _From, _State) -> %%% Internal functions %%%=================================================================== -report_results(Test, Results, Duration, #state{notify_pid=NotifyPid}) -> - NotifyPid ! {self(), {test_result, {Test, Results, Duration}}}, +report_results(TestPlan, Results, Duration, #state{notify_pid=NotifyPid}) -> + NotifyPid ! {self(), {test_result, {TestPlan, Results, Duration}}}, ok. report_done(#state{notify_pid=NotifyPid}) -> @@ -276,7 +270,8 @@ wait_for_completion_transition(_Result, State) -> launch_test_transition(State=#state{pending_tests=PendingTests, execution_mode=ExecutionMode}) when PendingTests == [] orelse ExecutionMode == serial -> - lager:debug("Waiting for completion: execution mode ~p with pending tests ~p", [ExecutionMode, PendingTests]), + PendingModules = [rt_test_plan:get_module(Test) || Test <- PendingTests], + lager:debug("Waiting for completion: execution mode ~p with pending tests ~p", [ExecutionMode, PendingModules]), {next_state, wait_for_completion, State}; launch_test_transition(State) -> {next_state, request_nodes, State, 0}. @@ -293,13 +288,13 @@ test_properties(Tests, OverriddenProps) -> lists:foldl(test_property_fun(OverriddenProps), [], Tests). test_property_fun(OverrideProps) -> - fun(TestModule, Acc) -> + fun(TestPlan, Acc) -> {PropsMod, PropsFun} = riak_test_runner:function_name(properties, - TestModule, + rt_test_plan:get_module(TestPlan), 0, rt_cluster), Properties = rt_properties:set(OverrideProps, PropsMod:PropsFun()), - [{TestModule, Properties} | Acc] + [{TestPlan, Properties} | Acc] end. versions_to_test(Properties) -> @@ -330,11 +325,11 @@ override_props(State) -> [{upgrade_path, UpgradeList}] end. --spec run_test(parallel | serial, atom(), atom(), proplists:proplist(), [pid()], [atom()], boolean()) -> {[pid()], [atom()]}. -run_test(parallel, Test, Backend, Properties, RunningPids, RunningTests, ContinueOnFail) -> - Pid = spawn_link(riak_test_runner, start, [Test, Backend, Properties, ContinueOnFail]), - {[Pid | RunningPids], [Test | RunningTests]}; -run_test(serial, Test, Backend, Properties, RunningPids, RunningTests, ContinueOnFail) -> - riak_test_runner:start(Test, Backend, Properties, ContinueOnFail), +-spec run_test(parallel | serial, atom(), proplists:proplist(), [pid()], [atom()], boolean()) -> {[pid()], [atom()]}. +run_test(parallel, TestPlan, Properties, RunningPids, RunningTests, ContinueOnFail) -> + Pid = spawn_link(riak_test_runner, start, [TestPlan, Properties, ContinueOnFail]), + {[Pid | RunningPids], [TestPlan | RunningTests]}; +run_test(serial, TestPlan, Properties, RunningPids, RunningTests, ContinueOnFail) -> + riak_test_runner:start(TestPlan, Properties, ContinueOnFail), {RunningPids, RunningTests}. diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index 22593dad0..dc0db8b89 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -24,7 +24,7 @@ -behavior(gen_fsm). %% API --export([start/4, +-export([start/3, send_event/2, stop/0]). @@ -52,7 +52,8 @@ -include_lib("eunit/include/eunit.hrl"). -type test_type() :: {new | old}. --record(state, {test_module :: atom(), +-record(state, {test_plan :: rt_test_plan:test_plan(), + test_module :: atom(), test_type :: test_type(), properties :: proplists:proplist(), backend :: atom(), @@ -77,8 +78,8 @@ %%%=================================================================== %% @doc Start the test executor -start(TestModule, Backend, Properties, ContinueOnFail) -> - Args = [TestModule, Backend, Properties, ContinueOnFail], +start(TestPlan, Properties, ContinueOnFail) -> + Args = [TestPlan, Properties, ContinueOnFail], gen_fsm:start_link(?MODULE, Args, []). send_event(Pid, Msg) -> @@ -101,14 +102,22 @@ metadata() -> %% @doc Read the storage schedule and go to idle. %% compose_test_datum(Version, Project, undefined, undefined) -> -init([TestModule, Backend, Properties, ContinueOnFail]) -> +init([TestPlan, Properties, ContinueOnFail]) -> lager:debug("Started riak_test_runnner with pid ~p (continue on fail: ~p)", [self(), ContinueOnFail]), Project = list_to_binary(rt_config:get(rt_project, "undefined")), + Backend = rt_test_plan:get(backend, TestPlan), + TestModule = rt_test_plan:get_module(TestPlan), MetaData = [{id, -1}, {platform, <<"local">>}, {version, rt:get_version()}, {backend, Backend}, {project, Project}], + + %% TODO: Remove after all tests ported 2.0 -- workaround to support + %% backend command line argument fo v1 cluster provisioning -jsb + rt_config:set(rt_backend, Backend), + lager:info("Using backend ~p", [Backend]), + {ok, UpdProperties} = rt_properties:set(metadata, MetaData, Properties), TestTimeout = rt_config:get(test_timeout, rt_config:get(rt_max_receive_wait_time)), @@ -122,7 +131,8 @@ init([TestModule, Backend, Properties, ContinueOnFail]) -> BackendCheck = check_backend(Backend, rt_properties:get(valid_backends, Properties)), PreReqCheck = check_prereqs(ConfirmMod), - State = #state{test_module=TestModule, + State = #state{test_plan=TestPlan, + test_module=TestModule, test_type=TestType, properties=UpdProperties, backend=Backend, @@ -509,29 +519,29 @@ cleanup(#state{test_module=TestModule, node_manager:return_nodes(rt_properties:get(node_ids, Properties)), riak_test_group_leader:tidy_up(OldGroupLeader). -notify_executor(timeout, #state{test_module=Test, +notify_executor(timeout, #state{test_plan=TestPlan, start_time=Start, end_time=End}) -> Duration = timer:now_diff(End, Start), - Notification = {test_complete, Test, self(), {fail, timeout}, Duration}, + Notification = {test_complete, TestPlan, self(), {fail, timeout}, Duration}, riak_test_executor:send_event(Notification); -notify_executor(fail, #state{test_module=Test, +notify_executor(fail, #state{test_plan=TestPlan, start_time=Start, end_time=End}) -> Duration = timer:now_diff(End, Start), - Notification = {test_complete, Test, self(), {fail, unknown}, Duration}, + Notification = {test_complete, TestPlan, self(), {fail, unknown}, Duration}, riak_test_executor:send_event(Notification); -notify_executor(pass, #state{test_module=Test, +notify_executor(pass, #state{test_plan=TestPlan, start_time=Start, end_time=End}) -> Duration = timer:now_diff(End, Start), - Notification = {test_complete, Test, self(), pass, Duration}, + Notification = {test_complete, TestPlan, self(), pass, Duration}, riak_test_executor:send_event(Notification); -notify_executor(FailResult, #state{test_module=Test, +notify_executor(FailResult, #state{test_plan=TestPlan, start_time=Start, end_time=End}) -> Duration = now_diff(End, Start), - Notification = {test_complete, Test, self(), FailResult, Duration}, + Notification = {test_complete, TestPlan, self(), FailResult, Duration}, riak_test_executor:send_event(Notification). test_versions(Properties) -> diff --git a/src/rt_bucket_types.erl b/src/rt_bucket_types.erl index 3a7466281..93b6ec878 100644 --- a/src/rt_bucket_types.erl +++ b/src/rt_bucket_types.erl @@ -29,6 +29,16 @@ -include("rt.hrl"). +%% Specify the bucket_types field for the properties record. The list +%% of bucket types may have two forms, a bucket_type or a pair +%% consisting of an integer and a bucket_type. The latter form +%% indicates that a bucket_type should only be applied to the cluster +%% with the given index. The former form is applied to all clusters. +-type bucket_type() :: {binary(), proplists:proplist()}. +-type bucket_types() :: [bucket_type() | {pos_integer(), bucket_type()}]. + +-export_type([bucket_types/0]). + -spec create_and_wait([node()], binary(), proplists:proplist()) -> ok. create_and_wait(Nodes, Type, Properties) -> create_and_activate_bucket_type(hd(Nodes), Type, Properties), diff --git a/src/rt_config.erl b/src/rt_config.erl index 77e048d0f..04f92d26a 100644 --- a/src/rt_config.erl +++ b/src/rt_config.erl @@ -44,6 +44,8 @@ -define(HARNESS, (rt_config:get(rt_harness))). -define(CONFIG_NAMESPACE, riak_test). -define(RECEIVE_WAIT_TIME_KEY, rt_max_receive_wait_time). +-define(GIDDYUP_PLATFORM_KEY, giddyup_platform). +-define(GIDDYUP_PROJECT_KEY, giddyup_project). -define(VERSION_KEY, versions). -define(DEFAULT_VERSION_KEY, default). -define(PREVIOUS_VERSION_KEY, previous). @@ -111,6 +113,12 @@ set(Key, Value) -> get(rt_max_wait_time) -> lager:info("rt_max_wait_time is deprecated. Please use rt_max_receive_wait_time instead."), rt_config:get(?RECEIVE_WAIT_TIME_KEY); +get(platform) -> + lager:info("platform is deprecated. Please use giddyup_platform instead."), + rt_config:get(?GIDDYUP_PLATFORM_KEY); +get(rt_project) -> + lager:info("rt_project is deprecated. Please use giddyup_project instead."), + rt_config:get(?GIDDYUP_PROJECT_KEY); get(?CONTINUE_ON_FAIL_KEY) -> get(?CONTINUE_ON_FAIL_KEY, ?DEFAULT_CONTINUE_ON_FAIL); get(Key) -> diff --git a/src/rt_planner.erl b/src/rt_planner.erl new file mode 100644 index 000000000..fb375ea52 --- /dev/null +++ b/src/rt_planner.erl @@ -0,0 +1,359 @@ +%%------------------------------------------------------------------- +%% +%% Copyright (c) 2015 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. +%% +%% ------------------------------------------------------------------- +%% @author Brett Hazen +%% @copyright (C) 2015, Basho Technologies +%% @doc +%% Module to manage the list of pending test plans and hand off work +%% to the appropriate test scheduler. +%% @end +%% Created : 30. Mar 2015 10:25 AM +%%------------------------------------------------------------------- +-module(rt_planner). +-author("Brett Hazen"). + +-behaviour(gen_server). + +-ifdef(TEST). +-include_lib("eunit/include/eunit.hrl"). +-endif. + +%% API +-export([start_link/0, + load_from_giddyup/3, + add_test_plan/5, + fetch_test_plan/0, + fetch_test_non_runnable_plan/0, + number_of_plans/0, + number_of_non_runable_plans/0, + stop/0]). + +%% gen_server callbacks +-export([init/1, + handle_call/3, + handle_cast/2, + handle_info/2, + terminate/2, + code_change/3]). + +-define(SERVER, ?MODULE). + +-record(state, { + %% Tests which are deemed to be runable + runnable_test_plans :: queue(), + %% Tests which are deemed not to be runable + non_runnable_test_plans :: queue() +}). + +%%%=================================================================== +%%% API +%%%=================================================================== + +%%-------------------------------------------------------------------- +%% @doc +%% Starts the server +%% +%% @end +%%-------------------------------------------------------------------- +-spec(start_link() -> + {ok, Pid :: pid()} | ignore | {error, Reason :: term()}). +start_link() -> + gen_server:start_link({local, ?SERVER}, ?MODULE, [], []). + +%%-------------------------------------------------------------------- +%% @doc +%% Reads the list of test plans from GiddyUp and queues them up +%% +%% @end +%%-------------------------------------------------------------------- +-spec(load_from_giddyup(string(), string() | undefined, list()) -> ok). +load_from_giddyup(Platform, Backend, CommandLineTests) -> + gen_server:call(?MODULE, {load_from_giddyup, Platform, Backend, CommandLineTests}). + +%%-------------------------------------------------------------------- +%% @doc +%% Queue up a new test plan +%% +%% @end +%%-------------------------------------------------------------------- +-spec(add_test_plan(string(), string(), rt_properties2:storage_backend(), rt_properties2:product_version(), rt_properties2:properties()) -> ok). +add_test_plan(Module, Platform, Backend, Version, Properties) -> + gen_server:call(?MODULE, {add_test_plan, Module, Platform, Backend, Version, Properties}). + +%%-------------------------------------------------------------------- +%% @doc +%% Fetch a test plan off the queue +%% +%% @end +%%-------------------------------------------------------------------- +-spec(fetch_test_plan() -> rt_test_plan:test_plan() | empty). +fetch_test_plan() -> + gen_server:call(?MODULE, fetch_test_plan). + +%%-------------------------------------------------------------------- +%% @doc +%% Fetch a test plan off the queue +%% +%% @end +%%-------------------------------------------------------------------- +-spec(fetch_test_non_runnable_plan() -> rt_test_plan:test_plan() | empty). +fetch_test_non_runnable_plan() -> + gen_server:call(?MODULE, fetch_test_non_runnable_plan). + +%%-------------------------------------------------------------------- +%% @doc +%% Return the number of runable test plans in the queue +%% +%% @end +%%-------------------------------------------------------------------- +-spec(number_of_plans() -> rt_test_plan:test_plan() | empty). +number_of_plans() -> + gen_server:call(?MODULE, number_of_plans). + +%%-------------------------------------------------------------------- +%% @doc +%% Return the number of non-runable test plans in the queue +%% +%% @end +%%-------------------------------------------------------------------- +-spec(number_of_non_runable_plans() -> rt_test_plan:test_plan() | empty). +number_of_non_runable_plans() -> + gen_server:call(?MODULE, number_of_non_runable_plans). + +%%-------------------------------------------------------------------- +%% @doc +%% Stops the server +%% +%% @end +%%-------------------------------------------------------------------- +-spec stop() -> ok. +stop() -> + gen_server:call(?MODULE, stop). + +%%%=================================================================== +%%% gen_server callbacks +%%%=================================================================== + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Initializes the server +%% +%% @spec init(Args) -> {ok, State} | +%% {ok, State, Timeout} | +%% ignore | +%% {stop, Reason} +%% @end +%%-------------------------------------------------------------------- +-spec(init(Args :: term()) -> + {ok, State :: #state{}} | {ok, State :: #state{}, timeout() | hibernate} | + {stop, Reason :: term()} | ignore). +init([]) -> + {ok, #state{runnable_test_plans=queue:new(), + non_runnable_test_plans=queue:new()}}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Handling call messages +%% +%% @end +%%-------------------------------------------------------------------- +-spec(handle_call(Request :: term(), From :: {pid(), Tag :: term()}, + State :: #state{}) -> + {reply, Reply :: term(), NewState :: #state{}} | + {reply, Reply :: term(), NewState :: #state{}, timeout() | hibernate} | + {noreply, NewState :: #state{}} | + {noreply, NewState :: #state{}, timeout() | hibernate} | + {stop, Reason :: term(), Reply :: term(), NewState :: #state{}} | + {stop, Reason :: term(), NewState :: #state{}}). +%% Run only those GiddyUp tests which are specified on the command line +%% If none are specified, run everything +handle_call({load_from_giddyup, Platform, _Backend, CommandLineTests}, _From, State) -> + AllGiddyupTests = giddyup:get_suite(Platform), + FilteredTests = case CommandLineTests of + [] -> + [test_plan_from_giddyup(Test) || Test <- AllGiddyupTests]; + _ -> + [test_plan_from_giddyup({GName, GMetaData}) || {GName, GMetaData} <- AllGiddyupTests, + CName <- CommandLineTests, + GName =:= CName] + end, + State1 = lists:foldl(fun sort_and_queue/2, State, FilteredTests), + {reply, ok, State1}; +%% Add a single test plan to the queue +handle_call({add_test_plan, Module, Platform, Backend, _Version, _Properties}, _From, State) -> + TestPlan = rt_test_plan:new([{module, Module}, {platform, Platform}, {backend, Backend}]), + {reply, ok, sort_and_queue(TestPlan, State)}; +handle_call(fetch_test_plan, _From, State) -> + Q = State#state.runnable_test_plans, + {Item, Q1} = queue:out(Q), + Result = case Item of + {value, Value} -> Value; + Empty -> Empty + end, + {reply, Result, State#state{runnable_test_plans=Q1}}; +handle_call(fetch_test_non_runnable_plan, _From, State) -> + Q = State#state.non_runnable_test_plans, + {Item, Q1} = queue:out(Q), + Result = case Item of + {value, Value} -> Value; + Empty -> Empty + end, + {reply, Result, State#state{non_runnable_test_plans=Q1}}; +handle_call(number_of_plans, _From, State) -> + Q = State#state.runnable_test_plans, + {reply, queue:len(Q), State}; +handle_call(number_of_non_runable_plans, _From, State) -> + Q = State#state.non_runnable_test_plans, + {reply, queue:len(Q), State}; +handle_call(_Request, _From, State) -> + {reply, ok, State}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Handling cast messages +%% +%% @end +%%-------------------------------------------------------------------- +-spec(handle_cast(Request :: term(), State :: #state{}) -> + {noreply, NewState :: #state{}} | + {noreply, NewState :: #state{}, timeout() | hibernate} | + {stop, Reason :: term(), NewState :: #state{}}). +handle_cast(_Request, State) -> + {noreply, State}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Handling all non call/cast messages +%% +%% @spec handle_info(Info, State) -> {noreply, State} | +%% {noreply, State, Timeout} | +%% {stop, Reason, State} +%% @end +%%-------------------------------------------------------------------- +-spec(handle_info(Info :: timeout() | term(), State :: #state{}) -> + {noreply, NewState :: #state{}} | + {noreply, NewState :: #state{}, timeout() | hibernate} | + {stop, Reason :: term(), NewState :: #state{}}). +handle_info(_Info, State) -> + {noreply, State}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% This function is called by a gen_server when it is about to +%% terminate. It should be the opposite of Module:init/1 and do any +%% necessary cleaning up. When it returns, the gen_server terminates +%% with Reason. The return value is ignored. +%% +%% @spec terminate(Reason, State) -> void() +%% @end +%%-------------------------------------------------------------------- +-spec(terminate(Reason :: (normal | shutdown | {shutdown, term()} | term()), + State :: #state{}) -> term()). +terminate(_Reason, _State) -> + ok. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Convert process state when code is changed +%% +%% @spec code_change(OldVsn, State, Extra) -> {ok, NewState} +%% @end +%%-------------------------------------------------------------------- +-spec(code_change(OldVsn :: term() | {down, term()}, State :: #state{}, + Extra :: term()) -> + {ok, NewState :: #state{}} | {error, Reason :: term()}). +code_change(_OldVsn, State, _Extra) -> + {ok, State}. + +%%%=================================================================== +%%% Internal functions +%%%=================================================================== + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Translate GiddyUp Output into an `rt_test_plan' record +%% +%% @end +%%-------------------------------------------------------------------- + +-spec(set_giddyup_field(atom(), {proplists:proplist(), rt_test_plan:test_plan()}) -> rt_test_plan:test_plan()). +set_giddyup_field(Field, {MetaData, TestPlan}) -> + {ok, TestPlan1} = case proplists:is_defined(Field, MetaData) of + true -> + rt_test_plan:set(Field, proplists:get_value(Field, MetaData), TestPlan); + _ -> + {ok, TestPlan} + end, + {MetaData, TestPlan1}. + +-spec(test_plan_from_giddyup({atom(), term()}) -> rt_test_plan:test_plan()). +test_plan_from_giddyup({Name, MetaData}) -> + Plan0 = rt_test_plan:new([{module, Name}]), + GiddyUpFields = [id, backend, platform, project], + {_, Plan1} = lists:foldl(fun set_giddyup_field/2, {MetaData, Plan0}, GiddyUpFields), + %% Special treatment for the upgrade path + {ok, Plan2} = case proplists:is_defined(upgrade_version, MetaData) of + true -> + rt_test_plan:set(upgrade_path, [rt_config:get_version(proplists:get_value(upgrade_version, MetaData)), rt_config:get_default_version()], Plan1); + _ -> + {ok, Plan1} + end, + Plan2. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Add a specific test into the proper list +%% +%% @end +%%-------------------------------------------------------------------- +sort_and_queue(TestPlan, State) -> + QR = State#state.runnable_test_plans, + QNR = State#state.non_runnable_test_plans, + {QR2, QNR2} = case is_runnable_test_plan(TestPlan) of + true -> + {queue:in(TestPlan, QR), QNR}; + _ -> + {QR, queue:in(TestPlan, QNR)} + end, + State#state{runnable_test_plans=QR2, + non_runnable_test_plans=QNR2}. + +%% Check for api compatibility +is_runnable_test_plan(TestPlan) -> + TestModule = rt_test_plan:get_module(TestPlan), + {Mod, Fun} = riak_test_runner:function_name(confirm, TestModule), + + code:ensure_loaded(Mod), + erlang:function_exported(Mod, Fun, 0) orelse + erlang:function_exported(Mod, Fun, 1). + +-ifdef(TEST). +set_giddyup_field_test() -> + S = {#state{runnable_test_plans =queue:new()},[]}, + T = test_plan_from_giddyup({test, [{id, 5},{backend,riak_kv_eleveldb_backend}, {platform, "os-x"}, {version, "2.0.5"},{project,<<"riak_ee">>}]}, S), + ?assertEqual(T, {#state{runnable_test_plans = {[{rt_test_plan_v1,5,test,<<"riak_ee">>,"os-x",riak_kv_eleveldb_backend,[],undefined}], []}},[]}). +-endif. diff --git a/src/rt_properties2.erl b/src/rt_properties2.erl new file mode 100644 index 000000000..ac8610cd1 --- /dev/null +++ b/src/rt_properties2.erl @@ -0,0 +1,234 @@ +-module(rt_properties2). +%% ------------------------------------------------------------------- +%% +%% Copyright (c) 2015 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. +%% +%% ------------------------------------------------------------------- + +%% @doc Implements a set of functions for accessing and manipulating +%% an `rt_properties2' record. + +-include("rt.hrl"). + +-record(rt_cluster_topology_v1, { + name :: atom(), + connected_to :: [] | [atom()], + nodes :: [product_version()] +}). +-define(RT_CLUSTER_TOPOLOGY, #rt_cluster_topology_v1). + +-record(rt_properties_v2, { + description :: string(), + supported_products :: [atom()], + minimum_version :: string(), + maximum_version :: string(), + supported_backends=all :: [storage_backend()], + wait_for_transfers=false :: boolean(), + bucket_types=[] :: rt_bucket_types:bucket_types(), + indexes=[] :: [index()], + ring_size=auto :: [atom() | non_neg_integer()], + features :: feature_flag(), + required_services=[riak_kv] :: [atom()], + cluster_topology=default_topology(3) :: [topology()], + default_version=rt_config:get_default_version() :: product_version(), + upgrade_path :: [product_version()], + config=default_config() :: term() +}). + +-type properties() :: #rt_properties_v2{}. +-type topology() :: #rt_cluster_topology_v1{}. +-type feature_flag() :: strong_consistency | yokozuna | jmx | snmp | security. +-type product_version() :: string() | atom(). +-type storage_backend() :: all | riak_kv_bitcask_backend | riak_kv_eleveldb_backend | riak_kv_memory_backend | riak_kv_multi_backend. +-type index() :: {binary(), binary(), binary()}. + +-export_type([properties/0, + feature_flag/0, + product_version/0, + storage_backend/0, + index/0]). + + +-define(RT_PROPERTIES, #rt_properties_v2). +-define(RECORD_FIELDS, record_info(fields, rt_properties_v2)). + +-export([new/0, + new/1, + get/2, + set/2, + set/3, + default_topology/1, + default_config/0]). + +%% @doc Create a new properties record with all fields initialized to +%% the default values. +-spec new() -> properties(). +new() -> + ?RT_PROPERTIES{}. + +%% @doc Create a new properties record with the fields initialized to +%% non-default value. Each field to be initialized should be +%% specified as an entry in a property list (i.e. a list of +%% pairs). Invalid property fields are ignored by this function. +-spec new(proplists:proplist()) -> properties(). +new(PropertyDefaults) -> + {Properties, _} = + lists:foldl(fun set_property/2, {?RT_PROPERTIES{}, []}, PropertyDefaults), + Properties. + +%% @doc Get the value of a property from a properties record. An error +%% is returned if `Properties' is not a valid `rt_properties2' record +%% or if the property requested is not a valid property. +-spec get(atom(), properties()) -> term() | {error, atom()}. +get(Property, Properties) -> + get(Property, Properties, validate_request(Property, Properties)). + +%% @doc Set the value for a property in a properties record. An error +%% is returned if `Properties' is not a valid `rt_properties2' record +%% or if any of the properties to be set are not a valid property. In +%% the case that invalid properties are specified the error returned +%% contains a list of erroneous properties. +-spec set([{atom(), term()}], properties()) -> properties() | {error, atom()}. +set(PropertyList, Properties) when is_list(PropertyList) -> + set_properties(PropertyList, Properties, validate_record(Properties)). + +%% @doc Set the value for a property in a properties record. An error +%% is returned if `Properties' is not a valid `rt_properties2' record +%% or if the property to be set is not a valid property. +-spec set(atom(), term(), properties()) -> {ok, properties()} | {error, atom()}. +set(Property, Value, Properties) -> + set_property(Property, Value, Properties, validate_request(Property, Properties)). + + +-spec get(atom(), properties(), ok | {error, atom()}) -> + term() | {error, atom()}. +get(Property, Properties, ok) -> + element(field_index(Property), Properties); +get(_Property, _Properties, {error, _}=Error) -> + Error. + +%% This function is used by `new/1' to set properties at record +%% creation time and by `set/2' to set multiple properties at once. +%% Node properties record validation is done by this function. It is +%% strictly used as a fold function which is the reason for the odd +%% structure of the input parameters. It accumulates any invalid +%% properties that are encountered and the caller may use that +%% information or ignore it. +-spec set_property({atom(), term()}, {properties(), [atom()]}) -> + {properties(), [atom()]}. +set_property({Property, Value}, {Properties, Invalid}) -> + case is_valid_property(Property) of + true -> + {setelement(field_index(Property), Properties, Value), Invalid}; + false -> + {Properties, [Property | Invalid]} + end. + +-spec set_property(atom(), term(), properties(), ok | {error, atom()}) -> + {ok, properties()} | {error, atom()}. +set_property(Property, Value, Properties, ok) -> + {ok, setelement(field_index(Property), Properties, Value)}; +set_property(_Property, _Value, _Properties, {error, _}=Error) -> + Error. + +-spec set_properties([{atom(), term()}], + properties(), + ok | {error, {atom(), [atom()]}}) -> + {properties(), [atom()]}. +set_properties(PropertyList, Properties, ok) -> + case lists:foldl(fun set_property/2, {Properties, []}, PropertyList) of + {UpdProperties, []} -> + UpdProperties; + {_, InvalidProperties} -> + {error, {invalid_properties, InvalidProperties}} + end; +set_properties(_, _, {error, _}=Error) -> + Error. + +-spec validate_request(atom(), term()) -> ok | {error, atom()}. +validate_request(Property, Properties) -> + validate_property(Property, validate_record(Properties)). + +-spec validate_record(term()) -> ok | {error, invalid_properties}. +validate_record(Record) -> + case is_valid_record(Record) of + true -> + ok; + false -> + {error, invalid_properties} + end. + +-spec validate_property(atom(), ok | {error, atom()}) -> ok | {error, invalid_property}. +validate_property(Property, ok) -> + case is_valid_property(Property) of + true -> + ok; + false -> + {error, invalid_property} + end; +validate_property(_Property, {error, _}=Error) -> + Error. + +-spec default_config() -> [term()]. +default_config() -> + [{riak_core, [{handoff_concurrency, 11}]}, + {riak_search, [{enabled, true}]}, + {riak_pipe, [{worker_limit, 200}]}]. + +%% @doc Create a single default cluster topology with default node versions +-spec default_topology(pos_integer()) -> [topology()]. +default_topology(N) -> + ?RT_CLUSTER_TOPOLOGY{name=cluster1, connected_to=[], nodes=[rt_config:get_default_version() || lists:seq(1,N)]}. + +-spec is_valid_record(term()) -> boolean(). +is_valid_record(Record) -> + is_record(Record, rt_properties_v2). + +-spec is_valid_property(atom()) -> boolean(). +is_valid_property(Property) -> + Fields = ?RECORD_FIELDS, + lists:member(Property, Fields). + +-spec field_index(atom()) -> non_neg_integer(). +field_index(description) -> + ?RT_PROPERTIES.description; +field_index(supported_products) -> + ?RT_PROPERTIES.supported_products; +field_index(minimum_version) -> + ?RT_PROPERTIES.minimum_version; +field_index(supported_backends) -> + ?RT_PROPERTIES.supported_backends; +field_index(wait_for_transfers) -> + ?RT_PROPERTIES.wait_for_transfers; +field_index(bucket_types) -> + ?RT_PROPERTIES.bucket_types; +field_index(indexes) -> + ?RT_PROPERTIES.indexes; +field_index(upgrade_path) -> + ?RT_PROPERTIES.upgrade_path; +field_index(ring_size) -> + ?RT_PROPERTIES.ring_size; +field_index(features) -> + ?RT_PROPERTIES.features; +field_index(required_services) -> + ?RT_PROPERTIES.required_services; +field_index(cluster_topology) -> + ?RT_PROPERTIES.cluster_topology; +field_index(default_version) -> + ?RT_PROPERTIES.default_version; +field_index(config) -> + ?RT_PROPERTIES.config. diff --git a/src/rt_test_plan.erl b/src/rt_test_plan.erl new file mode 100644 index 000000000..90374be0e --- /dev/null +++ b/src/rt_test_plan.erl @@ -0,0 +1,199 @@ +%%------------------------------------------------------------------- +%% +%% Copyright (c) 2015 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. +%% +%% ------------------------------------------------------------------- +%% @author Brett Hazen +%% @copyright (C) 2015, Basho Technologies +%% @doc +%% +%% @end +%% Created : 30. Mar 2015 3:25 PM +%%------------------------------------------------------------------- +-module(rt_test_plan). +-author("Brett Hazen"). + +-include("rt.hrl"). + +%% API +-export([new/0, + new/1, + get/2, + get_module/1, + set/2, + set/3]). + +-record(rt_test_plan_v1, { + id=-1 :: integer(), + module :: atom(), + project :: atom() | binary(), + platform :: string(), + backend=riak_kv_bitcask_backend :: rt_properties2:storage_backend(), + upgrade_path=[] :: [rt_properties2:product_version()], + properties :: rt_properties2:properties() +}). + +-type test_plan() :: #rt_test_plan_v1{}. + +-export_type([test_plan/0]). + +-define(RT_TEST_PLAN, #rt_test_plan_v1). +-define(RECORD_FIELDS, record_info(fields, rt_test_plan_v1)). + +%% Internal + +%% @doc Create a new test plan record with all fields initialized to +%% the default values. +-spec new() -> test_plan(). +new() -> + ?RT_TEST_PLAN{}. + +%% @doc Create a new test plan record with the fields initialized to +%% non-default value. Each field to be initialized should be +%% specified as an entry in a property list (i.e. a list of +%% pairs). Invalid field fields are ignored by this function. +-spec new(proplists:proplist()) -> test_plan(). +new(FieldDefaults) -> + {Fields, _} = + lists:foldl(fun set_field/2, {?RT_TEST_PLAN{}, []}, FieldDefaults), + Fields. + +%% @doc Get the value of a field from a test plan record. An error +%% is returned if `TestPlan' is not a valid `rt_test_plan' record +%% or if the field requested is not a valid field. +-spec get(atom(), test_plan()) -> term() | {error, atom()}. +get(Field, TestPlan) -> + get(Field, TestPlan, validate_request(Field, TestPlan)). + +%% @doc Get the value of the test name from a test plan record. An error +%% is returned if `TestPlan' is not a valid `rt_test_plan' record +%% or if the field requested is not a valid field. +-spec get_module(test_plan()) -> term() | {error, atom()}. +get_module(TestPlan) -> + get(module, TestPlan, validate_request(module, TestPlan)). + +%% @doc Set the value for a field in a test plan record. An error +%% is returned if `TestPlan' is not a valid `rt_test_plan' record +%% or if any of the fields to be set are not a valid field. In +%% the case that invalid fields are specified the error returned +%% contains a list of erroneous fields. +-spec set([{atom(), term()}], test_plan()) -> test_plan() | {error, atom()}. +set(FieldList, TestPlan) when is_list(FieldList) -> + set_fields(FieldList, TestPlan, validate_record(TestPlan)). + +%% @doc Set the value for a field in a test plan record. An error +%% is returned if `TestPlan' is not a valid `rt_test_plan' record +%% or if the field to be set is not a valid field. +-spec set(atom(), term(), test_plan()) -> {ok, test_plan()} | {error, atom()}. +set(Field, Value, TestPlan) -> + set_field(Field, Value, TestPlan, validate_request(Field, TestPlan)). + + +-spec get(atom(), test_plan(), ok | {error, atom()}) -> + term() | {error, atom()}. +get(Field, Fields, ok) -> + element(field_index(Field), Fields); +get(_Field, _Fields, {error, _}=Error) -> + Error. + +%% This function is used by `new/1' to set fields at record +%% creation time and by `set/2' to set multiple fields at once. +%% Test plan record validation is done by this function. It is +%% strictly used as a fold function which is the reason for the odd +%% structure of the input parameters. It accumulates any invalid +%% properties that are encountered and the caller may use that +%% information or ignore it. +-spec set_field({atom(), term()}, {test_plan(), [atom()]}) -> + {test_plan(), [atom()]}. +set_field({Field, Value}, {TestPlan, Invalid}) -> + case is_valid_field(Field) of + true -> + {setelement(field_index(Field), TestPlan, Value), Invalid}; + false -> + {TestPlan, [Field | Invalid]} + end. + +-spec set_field(atom(), term(), test_plan(), ok | {error, atom()}) -> + {ok, test_plan()} | {error, atom()}. +set_field(Field, Value, TestPlan, ok) -> + {ok, setelement(field_index(Field), TestPlan, Value)}; +set_field(_Field, _Value, _Fields, {error, _}=Error) -> + Error. + +-spec set_fields([{atom(), term()}], + test_plan(), + ok | {error, {atom(), [atom()]}}) -> + {test_plan(), [atom()]}. +set_fields(FieldList, TestPlan, ok) -> + case lists:foldl(fun set_field/2, {TestPlan, []}, FieldList) of + {UpdFields, []} -> + UpdFields; + {_, InvalidFields} -> + {error, {invalid_properties, InvalidFields}} + end; +set_fields(_, _, {error, _}=Error) -> + Error. + +-spec validate_request(atom(), test_plan()) -> ok | {error, atom()}. +validate_request(Field, TestPlan) -> + validate_field(Field, validate_record(TestPlan)). + +-spec validate_record(test_plan()) -> ok | {error, invalid_properties}. +validate_record(Record) -> + case is_valid_record(Record) of + true -> + ok; + false -> + {error, invalid_properties} + end. + +-spec validate_field(atom(), ok | {error, atom()}) -> ok | {error, invalid_field}. +validate_field(Field, ok) -> + case is_valid_field(Field) of + true -> + ok; + false -> + {error, invalid_field} + end; +validate_field(_Field, {error, _}=Error) -> + Error. + +-spec is_valid_record(term()) -> boolean(). +is_valid_record(Record) -> + is_record(Record, rt_test_plan_v1). + +-spec is_valid_field(atom()) -> boolean(). +is_valid_field(Field) -> + Fields = ?RECORD_FIELDS, + lists:member(Field, Fields). + +-spec field_index(atom()) -> non_neg_integer(). +field_index(id) -> + ?RT_TEST_PLAN.id; +field_index(module) -> + ?RT_TEST_PLAN.module; +field_index(project) -> + ?RT_TEST_PLAN.project; +field_index(platform) -> + ?RT_TEST_PLAN.platform; +field_index(backend) -> + ?RT_TEST_PLAN.backend; +field_index(upgrade_path) -> + ?RT_TEST_PLAN.upgrade_path; +field_index(properties) -> + ?RT_TEST_PLAN.properties. + diff --git a/src/smoke_test_escript.erl b/src/smoke_test_escript.erl index 62a646da3..2d2d2e0cf 100755 --- a/src/smoke_test_escript.erl +++ b/src/smoke_test_escript.erl @@ -60,7 +60,7 @@ main(Args) -> end, rt_config:set(rt_harness, ?MODULE), lager:debug("ParsedArgs ~p", [Parsed]), - Suites = giddyup:get_suite(rt_config:get(platform)), + Suites = giddyup:get_suite(rt_config:get(giddyup_platform)), Jobs = case lists:keyfind(jobs, 1, Parsed) of false -> 1; From f7e61bf75d89f3746a1875052ac6b9469fc0fd99 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Fri, 10 Apr 2015 20:27:42 -0600 Subject: [PATCH 128/157] Get GiddyUp product and version from the default version instead of config --- src/giddyup.erl | 7 ++++--- src/rt_config.erl | 18 ++++++++++++++---- src/rt_planner.erl | 10 +++++----- 3 files changed, 23 insertions(+), 12 deletions(-) diff --git a/src/giddyup.erl b/src/giddyup.erl index 42580d2f0..8e3710c92 100644 --- a/src/giddyup.erl +++ b/src/giddyup.erl @@ -27,6 +27,7 @@ get_suite(Platform) -> Schema = get_schema(Platform), Name = kvc:path('project.name', Schema), + Version = rt_config:get_default_version_number(), lager:info("Retrieved Project: ~s", [Name]), Tests = kvc:path('project.tests', Schema), TestProps = @@ -39,7 +40,7 @@ get_suite(Platform) -> X -> binary_to_atom(X, utf8) end}, {platform, list_to_binary(Platform)}, - {version, rt:get_version()}, + {version, Version}, {project, Name} ] ++ case kvc:path('tags.upgrade_version', Test) of @@ -58,8 +59,8 @@ get_schema(Platform) -> get_schema(Platform, Retries) -> Host = rt_config:get(giddyup_host), - Project = rt_config:get(giddyup_project), - Version = rt:get_version(), + Project = rt_config:get_default_version_product(), + Version = rt_config:get_default_version_number(), URL = lists:flatten(io_lib:format("http://~s/projects/~s?platform=~s&version=~s", [Host, Project, Platform, Version])), lager:info("giddyup url: ~s", [URL]), diff --git a/src/rt_config.erl b/src/rt_config.erl index 04f92d26a..075fa9308 100644 --- a/src/rt_config.erl +++ b/src/rt_config.erl @@ -27,6 +27,8 @@ config_or_os_env/2, convert_to_string/1, get_default_version/0, + get_default_version_product/0, + get_default_version_number/0, get_previous_version/0, get_legacy_version/0, get_os_env/1, @@ -45,7 +47,6 @@ -define(CONFIG_NAMESPACE, riak_test). -define(RECEIVE_WAIT_TIME_KEY, rt_max_receive_wait_time). -define(GIDDYUP_PLATFORM_KEY, giddyup_platform). --define(GIDDYUP_PROJECT_KEY, giddyup_project). -define(VERSION_KEY, versions). -define(DEFAULT_VERSION_KEY, default). -define(PREVIOUS_VERSION_KEY, previous). @@ -116,9 +117,6 @@ get(rt_max_wait_time) -> get(platform) -> lager:info("platform is deprecated. Please use giddyup_platform instead."), rt_config:get(?GIDDYUP_PLATFORM_KEY); -get(rt_project) -> - lager:info("rt_project is deprecated. Please use giddyup_project instead."), - rt_config:get(?GIDDYUP_PROJECT_KEY); get(?CONTINUE_ON_FAIL_KEY) -> get(?CONTINUE_ON_FAIL_KEY, ?DEFAULT_CONTINUE_ON_FAIL); get(Key) -> @@ -144,6 +142,18 @@ get(Key, Default) -> get_default_version() -> get_version(?DEFAULT_VERSION_KEY). +%% @doc Return the default product from the version +-spec get_default_version_product() -> string(). +get_default_version_product() -> + DefaultVersion = get_version(?DEFAULT_VERSION_KEY), + string:sub_word(DefaultVersion, 1, $-). + +%% @doc Return the default version number +-spec get_default_version_number() -> string(). +get_default_version_number() -> + DefaultVersion = get_version(?DEFAULT_VERSION_KEY), + string:sub_word(DefaultVersion, 2, $-). + %% @doc Return the default version -spec get_previous_version() -> string(). get_previous_version() -> diff --git a/src/rt_planner.erl b/src/rt_planner.erl index fb375ea52..41488bdd8 100644 --- a/src/rt_planner.erl +++ b/src/rt_planner.erl @@ -46,11 +46,11 @@ %% gen_server callbacks -export([init/1, - handle_call/3, - handle_cast/2, - handle_info/2, - terminate/2, - code_change/3]). + handle_call/3, + handle_cast/2, + handle_info/2, + terminate/2, + code_change/3]). -define(SERVER, ?MODULE). From 4186a466a8b851b24f27cefd98512612ed295531 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Wed, 15 Apr 2015 21:39:09 -0600 Subject: [PATCH 129/157] Finally get GiddyUp working - Stuff all output into the regression results directory; including logs and configs - Move start of lager to very top of escript - Move results printing into rt_reporter - Reporting now must be completed before cleaning up nodes - Test result reporting moved from executor into test runner - Node logs are copied to the outdir directory locally or uploaded to GiddyUp - Add the name of test to the rt_test_plan --- regression_test_wrapper.sh | 6 +- src/giddyup.erl | 30 +++- src/riak_test_escript.erl | 344 +++++++++++++------------------------ src/riak_test_executor.erl | 73 ++++---- src/riak_test_runner.erl | 84 ++++----- src/rt.erl | 12 +- src/rt2.erl | 34 +--- src/rt_harness.erl | 6 +- src/rt_planner.erl | 2 +- src/rt_reporter.erl | 337 ++++++++++++++++++++++++++++++++++++ src/rt_test_plan.erl | 17 +- src/rtdev.erl | 30 +++- 12 files changed, 618 insertions(+), 357 deletions(-) create mode 100644 src/rt_reporter.erl diff --git a/regression_test_wrapper.sh b/regression_test_wrapper.sh index 74a354313..224ccad04 100755 --- a/regression_test_wrapper.sh +++ b/regression_test_wrapper.sh @@ -27,12 +27,12 @@ ln -s $RESULTS $RESULTS_SYMLINK RT_OPTS="-v --continue -c $1" echo "Running bitcask regression tests using the following test cases: $BITCASK_BACKEND_TEST_CASES" -./riak_test $RT_OPTS -t $BITCASK_BACKEND_TEST_CASES &> $RESULTS_DIR/bitcask_results.log +./riak_test $RT_OPTS -t $BITCASK_BACKEND_TEST_CASES -o $RESULTS_DIR &> $RESULTS_DIR/bitcask_results.log echo "Running leveldb regression tests using the following test cases: $ELEVELDB_BACKEND_TEST_CASES" -./riak_test $RT_OPTS -t $ELEVELDB_BACKEND_TEST_CASES -b eleveldb &> $RESULTS_DIR/leveldb_results.log +./riak_test $RT_OPTS -t $ELEVELDB_BACKEND_TEST_CASES -b eleveldb -o $RESULTS_DIR &> $RESULTS_DIR/leveldb_results.log echo "Running memory regression tests using the following test cases: $MEMORY_BACKEND_TEST_CASES" -./riak_test $RT_OPTS -t $MEMORY_BACKEND_TEST_CASES -b memory &> $RESULTS_DIR/memory_results.log +./riak_test $RT_OPTS -t $MEMORY_BACKEND_TEST_CASES -b memory -o $RESULTS_DIR &> $RESULTS_DIR/memory_results.log echo "Results of the test run written to $RESULTS_DIR" diff --git a/src/giddyup.erl b/src/giddyup.erl index 8e3710c92..5a5f212a7 100644 --- a/src/giddyup.erl +++ b/src/giddyup.erl @@ -83,7 +83,7 @@ post_result(TestResult) -> URL = "http://" ++ Host ++ "/test_results", lager:info("giddyup url: ~s", [URL]), rt:check_ibrowse(), - case rt:post_result(TestResult, #rt_webhook{name="GiddyUp", url=URL, headers=[basic_auth()]}) of + case post_result(TestResult, #rt_webhook{name="GiddyUp", url=URL, headers=[basic_auth()]}) of {ok, RC, Headers} -> {_, Location} = lists:keyfind("Location", 1, Headers), lager:info("Test Result successfully POSTed to GiddyUp! ResponseCode: ~s, URL: ~s", [RC, Location]), @@ -92,6 +92,34 @@ post_result(TestResult) -> error end. +-spec(post_result(proplists:proplist(), term()) -> tuple()|error). +post_result(TestResult, #rt_webhook{url=URL, headers=HookHeaders, name=Name}) -> + try ibrowse:send_req(URL, + [{"Content-Type", "application/json"}], + post, + mochijson2:encode(TestResult), + [{content_type, "application/json"}] ++ HookHeaders, + 300000) of %% 5 minute timeout + + {ok, RC=[$2|_], Headers, _Body} -> + {ok, RC, Headers}; + {ok, ResponseCode, Headers, Body} -> + lager:info("Test Result did not generate the expected 2XX HTTP response code."), + lager:debug("Post"), + lager:debug("Response Code: ~p", [ResponseCode]), + lager:debug("Headers: ~p", [Headers]), + lager:debug("Body: ~p", [Body]), + error; + X -> + lager:warning("Some error POSTing test result: ~p", [X]), + error + catch + Class:Reason -> + lager:error("Error reporting to ~s. ~p:~p", [Name, Class, Reason]), + lager:error("Payload: ~p", [TestResult]), + error + end. + post_artifact(TRURL, {FName, Body}) -> %% First compute the path of where to post the artifact URL = artifact_url(TRURL, FName), diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index f19f8d778..907bb8805 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -26,23 +26,127 @@ -export([main/1]). -export([add_deps/1]). --define(HEADER, [<<"Test">>, <<"Result">>, <<"Reason">>, <<"Test Duration">>]). - main(Args) -> %% TODO Should we use clique? -jsb %% Parse command line arguments ... - {ParsedArgs, HarnessArgs, Tests, NonTests} = parse_args(Args), + {ParsedArgs, _NonOptionArgs} = parse_args(Args), + load_initial_config(ParsedArgs), %% Configure logging ... OutDir = proplists:get_value(outdir, ParsedArgs, "log"), ensure_dir(OutDir), - lager_setup(OutDir), - + + {Tests, NonTests} = generate_test_lists(ParsedArgs), + ok = prepare(ParsedArgs, Tests, NonTests), - Results = execute(Tests, ParsedArgs, HarnessArgs), + Results = execute(Tests, OutDir, ParsedArgs), finalize(Results, ParsedArgs). +% @doc Validate the command-line options +parse_args(Args) -> + validate_args(getopt:parse(cli_options(), Args)). + +validate_args({ok, {[], _}}) -> + print_help(); +validate_args({ok, {ParsedArgs, NonOptionArgs}}) -> + case proplists:is_defined(help, ParsedArgs) of + true -> + print_help(); + _ -> + {ParsedArgs, NonOptionArgs} + end; +validate_args(_) -> + print_help(). + +%% Option Name, Short Code, Long Code, Argument Spec, Help Message +cli_options() -> +[ + {help, $h, "help", undefined, "Print this usage page"}, + {config, $c, "conf", string, "specifies the project configuration"}, + {tests, $t, "tests", string, "specifies which tests to run"}, + {suites, $s, "suites", string, "which suites to run"}, + {groups, $g, "groups", string, "specifiy a list of test groups to run"}, + {dir, $d, "dir", string, "run all tests in the specified directory"}, + {skip, $x, "skip", string, "list of tests to skip in a directory"}, + {verbose, $v, "verbose", undefined, "verbose output"}, + {outdir, $o, "outdir", string, "output directory"}, + {backend, $b, "backend", atom, "backend to test [memory | bitcask | eleveldb]"}, + {upgrade_path, $u, "upgrade-path", atom, "comma-separated list representing an upgrade path (e.g. riak-1.3.4,riak_ee-1.4.12,riak_ee-2.0.0)"}, + {keep, undefined, "keep", boolean, "do not teardown cluster"}, + {continue_on_fail, $n, "continue", boolean, "continues executing tests on failure"}, + {report, $r, "report", string, "you're reporting an official test run, provide platform info (e.g. ubuntu-1404-64)\nUse 'config' if you want to pull from ~/.riak_test.config"}, + {file, $F, "file", string, "use the specified file instead of ~/.riak_test.config"} +]. + +print_help() -> + getopt:usage(cli_options(), escript:script_name()), + halt(0). + +report_platform(ParsedArgs) -> + case proplists:get_value(report, ParsedArgs, undefined) of + undefined -> + undefined; + "config" -> + rt_config:get(giddyup_platform, undefined); + R -> + R + end. + +%% @doc Print help string if it's specified, otherwise parse the arguments +generate_test_lists(ParsedArgs) -> + %% Have to load the `riak_test' config prior to assembling the + %% test metadata + + TestData = compose_test_data(ParsedArgs), + Backend = proplists:get_value(backend, ParsedArgs, bitcask), + {Tests, NonTests} = wrap_test_in_test_plan(report_platform(ParsedArgs), Backend, TestData), + Offset = rt_config:get(offset, undefined), + Workers = rt_config:get(workers, undefined), + shuffle_tests(Tests, NonTests, Offset, Workers). + +%% @doc Set values in the configuration with values specified on the command line +maybe_override_setting(Argument, Value, Arguments) -> + maybe_override_setting(proplists:is_defined(Argument, Arguments), Argument, + Value, Arguments). + +maybe_override_setting(true, Argument, Value, Arguments) -> + rt_config:set(Argument, proplists:get_value(Argument, Arguments, Value)); +maybe_override_setting(false, _Argument, _Value, _Arguments) -> + ok. + +load_initial_config(ParsedArgs) -> + %% Loads application defaults + application:load(riak_test), + + %% Loads from ~/.riak_test.config + rt_config:load(proplists:get_value(config, ParsedArgs), + proplists:get_value(file, ParsedArgs)), + + %% Override any command-line settings in config + maybe_override_setting(continue_on_fail, true, ParsedArgs). + +%% @doc Shuffle the order in which tests are scheduled +shuffle_tests([], _, _, _) -> + lager:error("No tests are scheduled to run~n"), + halt(1); +shuffle_tests(Tests, NonTests, undefined, _) -> + {Tests, NonTests}; +shuffle_tests(Tests, NonTests, _, undefined) -> + {Tests, NonTests}; +shuffle_tests(Tests, NonTests, Offset, Workers) -> + TestCount = length(Tests), + %% Avoid dividing by zero, computers hate that + Denominator = case Workers rem (TestCount+1) of + 0 -> 1; + D -> D + end, + ActualOffset = ((TestCount div Denominator) * Offset) rem (TestCount+1), + {TestA, TestB} = lists:split(ActualOffset, Tests), + lager:info("Offsetting ~b tests by ~b (~b workers, ~b offset)", + [TestCount, ActualOffset, Workers, Offset]), + {TestB ++ TestA, NonTests}. + prepare(ParsedArgs, Tests, NonTests) -> lager:notice("Test to run: ~p", [[rt_test_plan:get_module(Test) || Test <- Tests]]), case NonTests of @@ -54,40 +158,28 @@ prepare(ParsedArgs, Tests, NonTests) -> ok = erlang_setup(ParsedArgs), test_setup(). -execute(TestPlans, ParsedArgs, _HarnessArgs) -> - OutDir = proplists:get_value(outdir, ParsedArgs), - Report = report(ParsedArgs), +execute(TestPlans, OutDir, ParsedArgs) -> UpgradeList = upgrade_list( proplists:get_value(upgrade_path, ParsedArgs)), {ok, Executor} = riak_test_executor:start_link(TestPlans, OutDir, - Report, + report_platform(ParsedArgs), UpgradeList, self()), wait_for_results(Executor, [], length(TestPlans), 0). - -report_results(Results, Verbose) -> - %% TODO: Good place to also do giddyup reporting and provide a - %% place for extending to any other reporting sources that might - %% be useful. - - ModuleResults = [{rt_test_plan:get_module(TestPlan), PassFail, Duration} || {TestPlan, PassFail, Duration} <- Results], - print_summary(ModuleResults, undefined, Verbose), - ok. - - %% TestResults = run_tests(Tests, Outdir, Report, HarnessArgs), - %% lists:filter(fun results_filter/1, TestResults). +%% TestResults = run_tests(Tests, Outdir, Report, HarnessArgs), +%% lists:filter(fun results_filter/1, TestResults). %% run_test(Test, Outdir, TestMetaData, Report, HarnessArgs, NumTests) -> %% rt_cover:maybe_start(Test), %% SingleTestResult = riak_test_runner:run(Test, Outdir, TestMetaData, HarnessArgs), - %% case NumTests of - %% 1 -> keep_them_up; - %% _ -> rt_cluster:teardown() - %% end, +%% case NumTests of +%% 1 -> keep_them_up; +%% _ -> rt_cluster:teardown() +%% end, %% TODO: Do this in the test runner %% CoverDir = rt_config:get(cover_output, "coverage"), @@ -99,9 +191,9 @@ report_results(Results, Verbose) -> %% TODO: Use `TestCount' and `Completed' to display progress output wait_for_results(Executor, TestResults, TestCount, Completed) -> receive - {Executor, {test_result, Result}} -> + {_Executor, {test_result, Result}} -> wait_for_results(Executor, [Result | TestResults], TestCount, Completed+1); - {Executor, done} -> + {_Executor, done} -> rt_cover:stop(), TestResults; _ -> @@ -114,37 +206,12 @@ finalize(TestResults, Args) -> %% R <- TestResults], %% CoverDir = rt_config:get(cover_output, "coverage"), %% Coverage = rt_cover:maybe_write_coverage(all, CoverDir), - Verbose = proplists:is_defined(verbose, Args), - report_results(TestResults, Verbose), + %% Verbose = proplists:is_defined(verbose, Args), Teardown = not proplists:get_value(keep, Args, false), maybe_teardown(Teardown, TestResults), ok. -%% Option Name, Short Code, Long Code, Argument Spec, Help Message -cli_options() -> -[ - {help, $h, "help", undefined, "Print this usage page"}, - {config, $c, "conf", string, "specifies the project configuration"}, - {tests, $t, "tests", string, "specifies which tests to run"}, - {suites, $s, "suites", string, "which suites to run"}, - {groups, $g, "groups", string, "specifiy a list of test groups to run"}, - {dir, $d, "dir", string, "run all tests in the specified directory"}, - {skip, $x, "skip", string, "list of tests to skip in a directory"}, - {verbose, $v, "verbose", undefined, "verbose output"}, - {outdir, $o, "outdir", string, "output directory"}, - {backend, $b, "backend", atom, "backend to test [memory | bitcask | eleveldb]"}, - {upgrade_path, $u, "upgrade-path", atom, "comma-separated list representing an upgrade path (e.g. riak-1.3.4,riak_ee-1.4.12,riak_ee-2.0.0)"}, - {keep, undefined, "keep", boolean, "do not teardown cluster"}, - {continue_on_fail,undefined, "continue", boolean, "continues executing tests on failure"}, - {report, $r, "report", string, "you're reporting an official test run, provide platform info (e.g. ubuntu-1404-64)\nUse 'config' if you want to pull from ~/.riak_test.config"}, - {file, $F, "file", string, "use the specified file instead of ~/.riak_test.config"} -]. - -print_help() -> - getopt:usage(cli_options(), escript:script_name()), - halt(0). - add_deps(Path) -> lager:debug("Adding dep path ~p", [Path]), case file:list_dir(Path) of @@ -179,83 +246,6 @@ upgrade_list(undefined) -> upgrade_list(Path) -> string:tokens(Path, ","). -report(ParsedArgs) -> - case proplists:get_value(report, ParsedArgs, undefined) of - undefined -> - undefined; - "config" -> - rt_config:get(giddyup_platform, undefined); - R -> - R - end. - -parse_args(Args) -> - help_or_parse_args(getopt:parse(cli_options(), Args)). - -%% @doc Print help string if it's specified, otherwise parse the arguments -help_or_parse_args({ok, {[], _}}) -> - print_help(); -help_or_parse_args({ok, {ParsedArgs, HarnessArgs}}) -> - help_or_parse_tests(ParsedArgs, - HarnessArgs, - lists:member(help, ParsedArgs)); -help_or_parse_args(_) -> - print_help(). - -help_or_parse_tests(_, _, true) -> - print_help(); -help_or_parse_tests(ParsedArgs, HarnessArgs, false) -> - %% Have to load the `riak_test' config prior to assembling the - %% test metadata - load_initial_config(ParsedArgs), - - maybe_override_setting(continue_on_fail, true, ParsedArgs), - - TestData = compose_test_data(ParsedArgs), - Backend = proplists:get_value(backend, ParsedArgs, bitcask), - {Tests, NonTests} = wrap_test_in_test_plan(report(ParsedArgs), Backend, TestData), - Offset = rt_config:get(offset, undefined), - Workers = rt_config:get(workers, undefined), - shuffle_tests(ParsedArgs, HarnessArgs, Tests, NonTests, Offset, Workers). - -maybe_override_setting(Argument, Value, Arguments) -> - maybe_override_setting(proplists:is_defined(Argument, Arguments), Argument, - Value, Arguments). - -maybe_override_setting(true, Argument, Value, Arguments) -> - rt_config:set(Argument, proplists:get_value(Argument, Arguments, Value)); -maybe_override_setting(false, _Argument, _Value, _Arguments) -> - ok. - -load_initial_config(ParsedArgs) -> - %% Loads application defaults - application:load(riak_test), - - %% Loads from ~/.riak_test.config - rt_config:load(proplists:get_value(config, ParsedArgs), - proplists:get_value(file, ParsedArgs)). - -shuffle_tests(_, _, [], _, _, _) -> - io:format("No tests are scheduled to run~n"), - lager:error("No tests are scheduled to run~n"), - halt(1); -shuffle_tests(ParsedArgs, HarnessArgs, Tests, NonTests, undefined, _) -> - {ParsedArgs, HarnessArgs, Tests, NonTests}; -shuffle_tests(ParsedArgs, HarnessArgs, Tests, NonTests, _, undefined) -> - {ParsedArgs, HarnessArgs, Tests, NonTests}; -shuffle_tests(ParsedArgs, HarnessArgs, Tests, NonTests, Offset, Workers) -> - TestCount = length(Tests), - %% Avoid dividing by zero, computers hate that - Denominator = case Workers rem (TestCount+1) of - 0 -> 1; - D -> D - end, - ActualOffset = ((TestCount div Denominator) * Offset) rem (TestCount+1), - {TestA, TestB} = lists:split(ActualOffset, Tests), - lager:info("Offsetting ~b tests by ~b (~b workers, ~b offset)", - [TestCount, ActualOffset, Workers, Offset]), - {ParsedArgs, HarnessArgs, TestB ++ TestA, NonTests}. - erlang_setup(_ParsedArgs) -> register(riak_test, self()), maybe_add_code_path("./ebin"), @@ -480,98 +470,6 @@ parse_webhook(Props) -> undefined end. -test_summary_format_time(Milliseconds) -> - Mills = trunc(((Milliseconds / 1000000) - (Milliseconds div 1000000)) * 1000000), - TotalSecs = (Milliseconds - Mills) div 1000000, - TotalMins = TotalSecs div 60, - Hours = TotalSecs div 3600, - Secs = TotalSecs - (TotalMins * 60), - Mins = TotalMins - (Hours * 60), - list_to_binary(io_lib:format("~ph ~pm ~p.~ps", [Hours, Mins, Secs, Mills])). - - -test_summary_fun({Test, pass, _}, {{Pass, _Fail, _Skipped}, Width}) -> - TestNameLength = length(atom_to_list(Test)), - UpdWidth = - case TestNameLength > Width of - true -> - TestNameLength; - false -> - Width - end, - {{Pass+1, _Fail, _Skipped}, UpdWidth}; -test_summary_fun({Test, {fail, _}, _}, {{_Pass, Fail, _Skipped}, Width}) -> - TestNameLength = length(atom_to_list(Test)), - UpdWidth = - case TestNameLength > Width of - true -> - TestNameLength; - false -> - Width - end, - {{_Pass, Fail+1, _Skipped}, UpdWidth}; -test_summary_fun({Test, {skipped, _}, _}, {{_Pass, _Fail, Skipped}, Width}) -> - TestNameLength = length(atom_to_list(Test)), - UpdWidth = - case TestNameLength > Width of - true -> - TestNameLength; - false -> - Width - end, - {{_Pass, _Fail, Skipped+1}, UpdWidth}. - -format_test_row({Test, Result, Duration}, _Width) -> - TestString = atom_to_list(Test), - case Result of - {Status, Reason} -> - [TestString, Status, Reason, test_summary_format_time(Duration)]; - pass -> - [TestString, "pass", "N/A", test_summary_format_time(Duration)] - end. - -print_summary(TestResults, _CoverResult, Verbose) -> - %% TODO Log vs console output ... -jsb - io:format("~nTest Results:~n~n"), - - {StatusCounts, Width} = lists:foldl(fun test_summary_fun/2, {{0,0,0}, 0}, TestResults), - - case Verbose of - true -> - Rows = - [format_test_row(Result, Width) || Result <- TestResults], - %% TODO: Remove once clique table is fixed - [lager:debug("ROW ~p", [Row]) || Row <- Rows], - Table = clique_table:autosize_create_table(?HEADER, Rows), - io:format("~ts~n", [Table]); - false -> - ok - end, - - {PassCount, FailCount, SkippedCount} = StatusCounts, - io:format("---------------------------------------------~n"), - io:format("~w Tests Failed~n", [FailCount]), - io:format("~w Tests Skipped~n", [SkippedCount]), - io:format("~w Tests Passed~n", [PassCount]), - Percentage = case PassCount == 0 andalso FailCount == 0 of - true -> 0; - false -> (PassCount / (PassCount + FailCount + SkippedCount)) * 100 - end, - io:format("That's ~w% for those keeping score~n", [Percentage]), - - %% case CoverResult of - %% cover_disabled -> - %% ok; - %% {Coverage, AppCov} -> - %% io:format("Coverage : ~.1f%~n", [Coverage]), - %% [io:format(" ~s : ~.1f%~n", [App, Cov]) - %% || {App, Cov, _} <- AppCov] - %% end, - ok. - -test_name_width(Results) -> - lists:max([ length(X) || [X | _T] <- Results ]). - backend_list(Backend) when is_atom(Backend) -> atom_to_list(Backend); backend_list(Backends) when is_list(Backends) -> diff --git a/src/riak_test_executor.erl b/src/riak_test_executor.erl index b80562316..514d11224 100644 --- a/src/riak_test_executor.erl +++ b/src/riak_test_executor.erl @@ -24,26 +24,25 @@ code_change/4]). -type execution_mode() :: serial | parallel. --record(state, {pending_tests :: [atom()], - running_tests=[] :: [atom()], - waiting_tests=[] :: [atom()], - notify_pid :: pid(), +-record(state, {pending_tests :: [rt_test_plan:test_plan()], + running_tests=[] :: [rt_test_plan:test_plan()], + waiting_tests=[] :: [rt_test_plan:test_plan()], upgrade_list :: [string()], test_properties :: [proplists:proplist()], runner_pids=[] :: [pid()], log_dir :: string(), - report_info :: string(), execution_mode :: execution_mode(), - continue_on_fail :: boolean()}). + continue_on_fail :: boolean(), + reporter_pid :: pid()}). %%%=================================================================== %%% API %%%=================================================================== %% @doc Start the test executor --spec start_link(atom(), string(), string(), [string()], pid()) -> {ok, pid()} | ignore | {error, term()}. -start_link(Tests, LogDir, ReportInfo, UpgradeList, NotifyPid) -> - Args = [Tests, LogDir, ReportInfo, UpgradeList, NotifyPid], +-spec start_link([rt_test_plan:test_plan()], string(), string()|giddyup, [string()], pid()) -> {ok, pid()} | ignore | {error, term()}. +start_link(Tests, LogDir, Platform, UpgradeList, NotifyPid) -> + Args = [Tests, LogDir, Platform, UpgradeList, NotifyPid], gen_fsm:start_link({local, ?MODULE}, ?MODULE, Args, []). send_event(Msg) -> @@ -58,20 +57,25 @@ stop() -> %%% gen_fsm callbacks %%%=================================================================== -init([Tests, LogDir, ReportInfo, UpgradeList, NotifyPid]) -> +init([Tests, LogDir, Platform, UpgradeList, NotifyPid]) -> %% TODO Change the default when parallel execution support is implemented -jsb ExecutionMode = rt_config:get(rt_execution_mode, serial), ContinueOnFail = rt_config:get(continue_on_fail), + LogLocation = case Platform of + undefined -> LogDir; + _ -> giddyup + end, + {ok, Reporter} = rt_reporter:start_link(LogLocation, NotifyPid), + lager:notice("Starting the Riak Test executor in ~p execution mode", [ExecutionMode]), State = #state{pending_tests=Tests, log_dir=LogDir, - report_info=ReportInfo, upgrade_list=UpgradeList, - notify_pid=NotifyPid, execution_mode=ExecutionMode, - continue_on_fail=ContinueOnFail}, + continue_on_fail=ContinueOnFail, + reporter_pid=Reporter}, {ok, gather_properties, State, 0}. %% @doc there are no all-state events for this fsm @@ -88,8 +92,9 @@ handle_sync_event(_Event, _From, _StateName, _State) -> handle_info(_Info, StateName, State) -> {next_state, StateName, State}. -terminate(normal, _StateName, State) -> - report_done(State), +terminate(normal, _StateName, _State) -> + rt_reporter:send_result(done), + rt_reporter:stop(), ok; terminate(_Reason, _StateName, _State) -> ok. @@ -119,14 +124,12 @@ request_nodes(timeout, State) -> ok = maybe_reserve_nodes(NextTest, TestProps), {next_state, launch_test, State}; -request_nodes({test_complete, Test, Pid, Results, Duration}, State) -> +request_nodes({test_complete, Test, Pid, _Results}, State) -> #state{pending_tests=Pending, waiting_tests=Waiting, running_tests=Running, runner_pids=Pids, execution_mode=ExecutionMode}= State, - %% Report results - report_results(Test, Results, Duration, State), UpdState = State#state{running_tests=lists:delete(Test, Running), runner_pids=lists:delete(Pid, Pids), pending_tests=Pending++Waiting, @@ -140,7 +143,7 @@ launch_test(insufficient_versions_available, State) -> lager:debug("riak_test_executor:launch_test insufficient_versions_available"), #state{pending_tests=[HeadPending | RestPending], execution_mode=ExecutionMode} = State, - report_results(HeadPending, {skipped, insufficient_versions}, 0, State), + rt_reporter:send_result({test_result, {HeadPending, {skipped, insufficient_versions}, 0}}), UpdState = State#state{pending_tests=RestPending, execution_mode=ExecutionMode}, launch_test_transition(UpdState); @@ -151,6 +154,7 @@ launch_test(not_enough_nodes, State) -> #state{pending_tests=[HeadPending | RestPending], waiting_tests=Waiting, execution_mode=ExecutionMode} = State, + rt_reporter:send_result({test_result, {HeadPending, {skipped, not_enough_nodes}, 0}}), UpdState = State#state{pending_tests=RestPending, waiting_tests=[HeadPending | Waiting], execution_mode=ExecutionMode}, @@ -164,28 +168,27 @@ launch_test({nodes, Nodes, NodeMap}, State) -> test_properties=PropertiesList, runner_pids=Pids, running_tests=Running, - continue_on_fail=ContinueOnFail} = State, + continue_on_fail=ContinueOnFail, + reporter_pid=ReporterPid} = State, NextTestModule = rt_test_plan:get_module(NextTestPlan), lager:debug("Executing test ~p in mode ~p", [NextTestModule, ExecutionMode]), {NextTestPlan, TestProps} = lists:keyfind(NextTestPlan, 1, PropertiesList), UpdTestProps = rt_properties:set([{node_map, NodeMap}, {node_ids, Nodes}], TestProps), {RunnerPids, RunningTests} = run_test(ExecutionMode, NextTestPlan, UpdTestProps, - Pids, Running, ContinueOnFail), + Pids, Running, ContinueOnFail, ReporterPid), UpdState = State#state{pending_tests=RestPending, execution_mode=ExecutionMode, runner_pids=RunnerPids, running_tests=RunningTests}, launch_test_transition(UpdState); -launch_test({test_complete, TestPlan, Pid, Results, Duration}, State) -> +launch_test({test_complete, TestPlan, Pid, _Results}, State) -> #state{pending_tests=Pending, waiting_tests=Waiting, running_tests=Running, runner_pids=Pids, execution_mode=ExecutionMode} = State, - %% Report results - report_results(TestPlan, Results, Duration, State), UpdState = State#state{running_tests=lists:delete(TestPlan, Running), runner_pids=lists:delete(Pid, Pids), pending_tests=Pending++Waiting, @@ -214,15 +217,13 @@ maybe_reserve_nodes(false, NextTest, VersionsToTest, _TestProps) -> node_manager:reserve_nodes(0, VersionsToTest, reservation_notify_fun()), ok. -wait_for_completion({test_complete, Test, Pid, Results, Duration}, State) -> +wait_for_completion({test_complete, Test, Pid, Results}, State) -> lager:debug("Test ~p complete", [rt_test_plan:get_module(Test)]), #state{pending_tests=Pending, waiting_tests=Waiting, running_tests=Running, runner_pids=Pids, execution_mode=ExecutionMode} = State, - %% Report results - report_results(Test, Results, Duration, State), UpdState = State#state{running_tests=lists:delete(Test, Running), runner_pids=lists:delete(Pid, Pids), pending_tests=Pending++Waiting, @@ -250,14 +251,6 @@ wait_for_completion(_Event, _From, _State) -> %%% Internal functions %%%=================================================================== -report_results(TestPlan, Results, Duration, #state{notify_pid=NotifyPid}) -> - NotifyPid ! {self(), {test_result, {TestPlan, Results, Duration}}}, - ok. - -report_done(#state{notify_pid=NotifyPid}) -> - NotifyPid ! {self(), done}, - ok. - wait_for_completion_transition({_Status, _Reason}, State=#state{continue_on_fail=ContinueOnFail}) when ContinueOnFail == false -> {stop, normal, State}; wait_for_completion_transition(_Result, State=#state{pending_tests=[], @@ -325,11 +318,11 @@ override_props(State) -> [{upgrade_path, UpgradeList}] end. --spec run_test(parallel | serial, atom(), proplists:proplist(), [pid()], [atom()], boolean()) -> {[pid()], [atom()]}. -run_test(parallel, TestPlan, Properties, RunningPids, RunningTests, ContinueOnFail) -> - Pid = spawn_link(riak_test_runner, start, [TestPlan, Properties, ContinueOnFail]), +-spec run_test(parallel | serial, atom(), proplists:proplist(), [pid()], [rt_test_plan:test_plan()], boolean(), pid()) -> {[pid()], [atom()]}. +run_test(parallel, TestPlan, Properties, RunningPids, RunningTests, ContinueOnFail, ReporterPid) -> + Pid = spawn_link(riak_test_runner, start, [TestPlan, Properties, ContinueOnFail, ReporterPid]), {[Pid | RunningPids], [TestPlan | RunningTests]}; -run_test(serial, TestPlan, Properties, RunningPids, RunningTests, ContinueOnFail) -> - riak_test_runner:start(TestPlan, Properties, ContinueOnFail), +run_test(serial, TestPlan, Properties, RunningPids, RunningTests, ContinueOnFail, ReporterPid) -> + riak_test_runner:start(TestPlan, Properties, ContinueOnFail, ReporterPid), {RunningPids, RunningTests}. diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index dc0db8b89..8d476fea7 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -24,7 +24,7 @@ -behavior(gen_fsm). %% API --export([start/3, +-export([start/4, send_event/2, stop/0]). @@ -69,7 +69,8 @@ current_version :: string(), remaining_versions :: [string()], test_results :: [term()], - continue_on_fail :: boolean()}). + continue_on_fail :: boolean(), + reporter_pid :: pid()}). -deprecated([{metadata,0,next_major_release}]). @@ -77,9 +78,9 @@ %%% API %%%=================================================================== -%% @doc Start the test executor -start(TestPlan, Properties, ContinueOnFail) -> - Args = [TestPlan, Properties, ContinueOnFail], +%% @doc Start the test runner +start(TestPlan, Properties, ContinueOnFail, ReporterPid) -> + Args = [TestPlan, Properties, ContinueOnFail, ReporterPid], gen_fsm:start_link(?MODULE, Args, []). send_event(Pid, Msg) -> @@ -102,7 +103,7 @@ metadata() -> %% @doc Read the storage schedule and go to idle. %% compose_test_datum(Version, Project, undefined, undefined) -> -init([TestPlan, Properties, ContinueOnFail]) -> +init([TestPlan, Properties, ContinueOnFail, ReporterPid]) -> lager:debug("Started riak_test_runnner with pid ~p (continue on fail: ~p)", [self(), ContinueOnFail]), Project = list_to_binary(rt_config:get(rt_project, "undefined")), Backend = rt_test_plan:get(backend, TestPlan), @@ -142,7 +143,8 @@ init([TestPlan, Properties, ContinueOnFail]) -> backend_check=BackendCheck, prereq_check=PreReqCheck, group_leader=group_leader(), - continue_on_fail=ContinueOnFail}, + continue_on_fail=ContinueOnFail, + reporter_pid=ReporterPid}, {ok, setup, State, 0}. %% @doc there are no all-state events for this fsm @@ -173,12 +175,10 @@ code_change(_OldVsn, StateName, State, _Extra) -> %% Asynchronous call handling functions for each FSM state setup(timeout, State=#state{backend_check=false}) -> - cleanup(State), - notify_executor({skipped, invalid_backend}, State), + report_cleanup_and_notify({skipped, invalid_backend}, State), {stop, normal, State}; setup(timeout, State=#state{prereq_check=false}) -> - cleanup(State), - notify_executor({fail, prereq_check_failed}, State), + report_cleanup_and_notify({fail, prereq_check_failed}, State), {stop, normal, State}; setup(timeout, State=#state{test_type=TestType, test_module=TestModule, @@ -269,8 +269,7 @@ wait_for_completion(timeout, State=#state{test_module=TestModule, test_type=TestType, group_leader=GroupLeader, end_time=os:timestamp()}, - cleanup(UpdState), - notify_executor(timeout, UpdState), + report_cleanup_and_notify(timeout, UpdState), {stop, normal, UpdState}; wait_for_completion({test_result, {fail, Reason}}, State=#state{test_module=TestModule, test_type=TestType, @@ -286,8 +285,7 @@ wait_for_completion({test_result, {fail, Reason}}, State=#state{test_module=Test continue_on_fail=ContinueOnFail, end_time=os:timestamp()}, lager:debug("ContinueOnFail: ~p", [ContinueOnFail]), - maybe_cleanup(ContinueOnFail, UpdState), - notify_executor(Result, UpdState), + report_cleanup_and_notify(Result, ContinueOnFail, UpdState), {stop, normal, UpdState}; wait_for_completion({test_result, Result}, State=#state{test_module=TestModule, test_type=TestType, @@ -301,8 +299,7 @@ wait_for_completion({test_result, Result}, State=#state{test_module=TestModule, test_type=TestType, group_leader=GroupLeader, end_time=os:timestamp()}, - cleanup(UpdState), - notify_executor(Result, UpdState), + report_cleanup_and_notify(Result, UpdState), {stop, normal, UpdState}; wait_for_completion({test_result, Result}, State) -> #state{backend=Backend, @@ -494,6 +491,26 @@ notify_fun(Pid) -> ?MODULE:send_event(Pid, X) end. +%% @doc Send the results report, cleanup the nodes and +%% Notify the executor that we are done with the test run +%% @end +report_cleanup_and_notify(Result, State) -> + report_cleanup_and_notify(Result, true, State). + +%% @doc Send the results report, cleanup the nodes (optionally) and +%% Notify the executor that we are done with the test run +%% @end +-spec(report_cleanup_and_notify(tuple(), boolean(), term()) -> ok). +report_cleanup_and_notify(Result, CleanUp, State=#state{test_plan=TestPlan, + start_time=Start, + end_time=End}) -> + Duration = now_diff(End, Start), + ResultMessage = test_result_message(Result), + rt_reporter:send_result(test_result({TestPlan, ResultMessage, Duration})), + maybe_cleanup(CleanUp, State), + Notification = {test_complete, TestPlan, self(), ResultMessage}, + riak_test_executor:send_event(Notification). + maybe_cleanup(true, State) -> cleanup(State); maybe_cleanup(false, _State) -> @@ -519,30 +536,15 @@ cleanup(#state{test_module=TestModule, node_manager:return_nodes(rt_properties:get(node_ids, Properties)), riak_test_group_leader:tidy_up(OldGroupLeader). -notify_executor(timeout, #state{test_plan=TestPlan, - start_time=Start, - end_time=End}) -> - Duration = timer:now_diff(End, Start), - Notification = {test_complete, TestPlan, self(), {fail, timeout}, Duration}, - riak_test_executor:send_event(Notification); -notify_executor(fail, #state{test_plan=TestPlan, - start_time=Start, - end_time=End}) -> - Duration = timer:now_diff(End, Start), - Notification = {test_complete, TestPlan, self(), {fail, unknown}, Duration}, - riak_test_executor:send_event(Notification); -notify_executor(pass, #state{test_plan=TestPlan, - start_time=Start, - end_time=End}) -> - Duration = timer:now_diff(End, Start), - Notification = {test_complete, TestPlan, self(), pass, Duration}, - riak_test_executor:send_event(Notification); -notify_executor(FailResult, #state{test_plan=TestPlan, - start_time=Start, - end_time=End}) -> - Duration = now_diff(End, Start), - Notification = {test_complete, TestPlan, self(), FailResult, Duration}, - riak_test_executor:send_event(Notification). +% @doc Convert test result into report message +test_result_message(timeout) -> + {fail, timeout}; +test_result_message(fail) -> + {fail, unknown}; +test_result_message(pass) -> + pass; +test_result_message(FailResult) -> + FailResult. test_versions(Properties) -> StartVersion = rt_properties:get(start_version, Properties), diff --git a/src/rt.erl b/src/rt.erl index 6a1623376..3dc1c92d0 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -64,7 +64,7 @@ expect_in_log/2, get_deps/0, get_ip/1, - get_node_logs/0, + get_node_logs/1, get_replica/5, get_ring/1, get_version/0, @@ -99,7 +99,6 @@ pbc_put_file/4, pbc_really_deleted/3, pmap/2, - post_result/2, product/1, priv_dir/0, remove/2, @@ -1363,10 +1362,11 @@ pmap(F, L) -> setup_harness(_Test, _Args) -> rt_harness:setup(). -%% @doc Downloads any extant log files from the harness's running -%% nodes. -get_node_logs() -> - rt2:get_node_logs(). +%% @doc Copy all of the nodes' log files to a local dir or +%% open a port to each file to upload to GiddyUp +-spec(get_node_logs(string() | giddyup) -> list()). +get_node_logs(DestDir) -> + rt2:get_node_logs(DestDir). check_ibrowse() -> rt2:check_ibrowse(). diff --git a/src/rt2.erl b/src/rt2.erl index 8f8220622..7d9a32f05 100644 --- a/src/rt2.erl +++ b/src/rt2.erl @@ -39,7 +39,7 @@ expect_in_log/2, get_deps/0, get_ip/1, - get_node_logs/0, + get_node_logs/1, get_replica/5, get_version/0, is_mixed_cluster/1, @@ -47,7 +47,6 @@ log_to_nodes/2, log_to_nodes/3, pmap/2, - post_result/2, priv_dir/0, product/1, rpc_get_env/2, @@ -612,8 +611,8 @@ setup_harness(Test, Args) -> %% @doc Downloads any extant log files from the harness's running %% nodes. -get_node_logs() -> - rt_harness:get_node_logs(). +get_node_logs(DestDir) -> + rt_harness:get_node_logs(DestDir). check_ibrowse() -> try sys:get_status(ibrowse) of @@ -626,33 +625,6 @@ check_ibrowse() -> application:start(ibrowse) end. -post_result(TestResult, #rt_webhook{url=URL, headers=HookHeaders, name=Name}) -> - lager:info("Posting result to ~s ~s", [Name, URL]), - try ibrowse:send_req(URL, - [{"Content-Type", "application/json"}], - post, - mochijson2:encode(TestResult), - [{content_type, "application/json"}] ++ HookHeaders, - 300000) of %% 5 minute timeout - - {ok, RC=[$2|_], Headers, _Body} -> - {ok, RC, Headers}; - {ok, ResponseCode, Headers, Body} -> - lager:info("Test Result did not generate the expected 2XX HTTP response code."), - lager:debug("Post"), - lager:debug("Response Code: ~p", [ResponseCode]), - lager:debug("Headers: ~p", [Headers]), - lager:debug("Body: ~p", [Body]), - error; - X -> - lager:warning("Some error POSTing test result: ~p", [X]), - error - catch - Class:Reason -> - lager:error("Error reporting to ~s. ~p:~p", [Name, Class, Reason]), - lager:error("Payload: ~p", [TestResult]), - error - end. %%%=================================================================== %%% Bucket Types Functions diff --git a/src/rt_harness.erl b/src/rt_harness.erl index a2b61ad0f..9bf3a7817 100644 --- a/src/rt_harness.erl +++ b/src/rt_harness.erl @@ -37,7 +37,7 @@ cmd/2, setup/0, get_deps/0, - get_node_logs/0, + get_node_logs/1, get_version/0, get_version/1, get_backends/0, @@ -107,8 +107,8 @@ get_version(Node) -> get_backends() -> ?HARNESS_MODULE:get_backends(). -get_node_logs() -> - ?HARNESS_MODULE:get_node_logs(). +get_node_logs(DestDir) -> + ?HARNESS_MODULE:get_node_logs(DestDir). set_backend(Backend) -> ?HARNESS_MODULE:set_backend(Backend). diff --git a/src/rt_planner.erl b/src/rt_planner.erl index 41488bdd8..5ee746a06 100644 --- a/src/rt_planner.erl +++ b/src/rt_planner.erl @@ -144,7 +144,7 @@ number_of_non_runable_plans() -> %%-------------------------------------------------------------------- -spec stop() -> ok. stop() -> - gen_server:call(?MODULE, stop). + gen_server:call(?MODULE, stop, infinity). %%%=================================================================== %%% gen_server callbacks diff --git a/src/rt_reporter.erl b/src/rt_reporter.erl new file mode 100644 index 000000000..74136fe0f --- /dev/null +++ b/src/rt_reporter.erl @@ -0,0 +1,337 @@ +%%------------------------------------------------------------------- +%% +%% Copyright (c) 2015 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. +%% +%% ------------------------------------------------------------------- +%% @author Brett Hazen +%% @copyright (C) 2015, Basho Technologies +%% @doc +%% +%% @end +%% Created : 31. Mar 2015 10:25 AM +%%------------------------------------------------------------------- +-module(rt_reporter). +-author("Brett Hazen"). + +-behaviour(gen_server). + +-define(HEADER, [<<"Test">>, <<"Result">>, <<"Reason">>, <<"Test Duration">>]). + +%% API +-export([start_link/2, + stop/0, + send_result/1]). + +%% gen_server callbacks +-export([init/1, + handle_call/3, + handle_cast/2, + handle_info/2, + terminate/2, + code_change/3]). + +-define(SERVER, ?MODULE). + +-record(state, { + % Collection of log files + % Running summary of test results: {test, pass/fail, duration} + summary :: list(), + log_dir :: string() | giddyup, + % PID of escript used to update results + notify_pid :: pid() +}). + +%%%=================================================================== +%%% API +%%%=================================================================== + +%%-------------------------------------------------------------------- +%% @doc +%% Starts the server +%% +%% @end +%%-------------------------------------------------------------------- +-spec(start_link(string() | giddyup, pid()) -> + {ok, Pid :: pid()} | ignore | {error, Reason :: term()}). +start_link(LogDir, NotifyPid) -> + gen_server:start_link({local, ?SERVER}, ?MODULE, [LogDir, NotifyPid], []). + +%%-------------------------------------------------------------------- +%% @doc +%% Stops the server +%% +%% @end +%%-------------------------------------------------------------------- +-spec stop() -> ok. +stop() -> + gen_server:call(?MODULE, stop, infinity). + + +%% @doc Send an asychronous message to the reporter +%% -spec send_cast(term()) -> ok. +%% send_cast(Msg) -> +%% gen_server:cast(?MODULE, Msg). + +%% @doc Send a sychronous message to the reporter +%% -spec send_call(term()) -> ok. +%% send_call(Msg) -> +%% gen_server:call(?MODULE, Msg). + +%% @doc Send the test result to the reporter +-spec send_result(term()) -> ok. +send_result(Msg) -> + %% TODO: Determine proper timeout + gen_server:call(?MODULE, Msg, 30000). + + +%%%=================================================================== +%%% gen_server callbacks +%%%=================================================================== + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Initializes the server +%% +%% @spec init(Args) -> {ok, State} | +%% {ok, State, Timeout} | +%% ignore | +%% {stop, Reason} +%% @end +%%-------------------------------------------------------------------- +-spec(init(Args :: term()) -> + {ok, State :: #state{}} | {ok, State :: #state{}, timeout() | hibernate} | + {stop, Reason :: term()} | ignore). +init([LogDir, NotifyPid]) -> + {ok, #state{summary=[], + log_dir=LogDir, + notify_pid=NotifyPid}}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Handling call messages +%% +%% @end +%%-------------------------------------------------------------------- +-spec(handle_call(Request :: term(), From :: {pid(), Tag :: term()}, + State :: #state{}) -> + {reply, Reply :: term(), NewState :: #state{}} | + {reply, Reply :: term(), NewState :: #state{}, timeout() | hibernate} | + {noreply, NewState :: #state{}} | + {noreply, NewState :: #state{}, timeout() | hibernate} | + {stop, Reason :: term(), Reply :: term(), NewState :: #state{}} | + {stop, Reason :: term(), NewState :: #state{}}). +handle_call({test_result, Result}, From, State) -> + lager:debug("Sending test_result to ~p from ~p", [State#state.notify_pid, From]), + Results = State#state.summary, + State#state.notify_pid ! {From, {test_result, Result}}, + report_and_gather_logs(State#state.log_dir, Result), + {reply, ok, State#state{summary=[Result|Results]}}; +handle_call(done, From, State) -> + lager:debug("Sending done to ~p from ~p", [State#state.notify_pid, From]), + State#state.notify_pid ! {From, done}, + print_summary(State#state.summary, undefined, true), + {reply, ok, State}; +handle_call(_Request, _From, State) -> + {reply, ok, State}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Handling cast messages +%% +%% @end +%%-------------------------------------------------------------------- +-spec(handle_cast(Request :: term(), State :: #state{}) -> + {noreply, NewState :: #state{}} | + {noreply, NewState :: #state{}, timeout() | hibernate} | + {stop, Reason :: term(), NewState :: #state{}}). +handle_cast(_Request, State) -> + {noreply, State}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Handling all non call/cast messages +%% +%% @spec handle_info(Info, State) -> {noreply, State} | +%% {noreply, State, Timeout} | +%% {stop, Reason, State} +%% @end +%%-------------------------------------------------------------------- +-spec(handle_info(Info :: timeout() | term(), State :: #state{}) -> + {noreply, NewState :: #state{}} | + {noreply, NewState :: #state{}, timeout() | hibernate} | + {stop, Reason :: term(), NewState :: #state{}}). +handle_info(_Info, State) -> + {noreply, State}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% This function is called by a gen_server when it is about to +%% terminate. It should be the opposite of Module:init/1 and do any +%% necessary cleaning up. When it returns, the gen_server terminates +%% with Reason. The return value is ignored. +%% +%% @spec terminate(Reason, State) -> void() +%% @end +%%-------------------------------------------------------------------- +-spec(terminate(Reason :: (normal | shutdown | {shutdown, term()} | term()), + State :: #state{}) -> term()). +terminate(_Reason, _State) -> + ok. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Convert process state when code is changed +%% +%% @spec code_change(OldVsn, State, Extra) -> {ok, NewState} +%% @end +%%-------------------------------------------------------------------- +-spec(code_change(OldVsn :: term() | {down, term()}, State :: #state{}, + Extra :: term()) -> + {ok, NewState :: #state{}} | {error, Reason :: term()}). +code_change(_OldVsn, State, _Extra) -> + {ok, State}. + +%%%=================================================================== +%%% Internal functions +%%%=================================================================== + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Dump the summary of all of the log runs to the console +%% +%% @spec print_summary(TestResults, _CoverResult, Verbose) -> ok +%% @end +%%-------------------------------------------------------------------- +-spec(print_summary(list(), term(), boolean()) -> ok). +print_summary(TestResults, _CoverResult, Verbose) -> + %% TODO Log vs console output ... -jsb + lager:notice("Test Results:"), + + {StatusCounts, RowList} = lists:foldl(fun test_summary_fun/2, {{0,0,0}, []}, TestResults), + Rows = lists:reverse(RowList), + + case Verbose of + true -> + %% TODO: Remove once clique table is fixed + [lager:debug("ROW ~p", [Row]) || Row <- Rows], + Table = clique_table:autosize_create_table(?HEADER, Rows), + lager:notice("~ts", [Table]); + false -> + ok + end, + + {PassCount, FailCount, SkippedCount} = StatusCounts, + lager:notice("---------------------------------------------"), + lager:notice("~w Tests Failed", [FailCount]), + lager:notice("~w Tests Skipped", [SkippedCount]), + lager:notice("~w Tests Passed", [PassCount]), + Percentage = case PassCount == 0 andalso FailCount == 0 of + true -> 0; + false -> (PassCount / (PassCount + FailCount + SkippedCount)) * 100 + end, + lager:notice("That's ~w% for those keeping score", [Percentage]), + + %% case CoverResult of + %% cover_disabled -> + %% ok; + %% {Coverage, AppCov} -> + %% io:format("Coverage : ~.1f%~n", [Coverage]), + %% [io:format(" ~s : ~.1f%~n", [App, Cov]) + %% || {App, Cov, _} <- AppCov] + %% end, + ok. + +%% @doc Convert Milliseconds into human-readable string +-spec(test_summary_format_time(integer()) -> string()). +test_summary_format_time(Milliseconds) -> + Mills = trunc(((Milliseconds / 1000000) - (Milliseconds div 1000000)) * 1000000), + TotalSecs = (Milliseconds - Mills) div 1000000, + TotalMins = TotalSecs div 60, + Hours = TotalSecs div 3600, + Secs = TotalSecs - (TotalMins * 60), + Mins = TotalMins - (Hours * 60), + list_to_binary(io_lib:format("~ph ~pm ~p.~ps", [Hours, Mins, Secs, Mills])). + +%% @doc Count the number of passed, failed and skipped tests +test_summary_fun(Result = {_, pass, _}, {{Pass, _Fail, _Skipped}, Rows}) -> + FormattedRow = format_test_row(Result), + {{Pass+1, _Fail, _Skipped}, [FormattedRow|Rows]}; +test_summary_fun(Result = {_, {fail, _}, _}, {{_Pass, Fail, _Skipped}, Rows}) -> + FormattedRow = format_test_row(Result), + {{_Pass, Fail+1, _Skipped}, [FormattedRow|Rows]}; +test_summary_fun(Result = {_, {skipped, _}, _}, {{_Pass, _Fail, Skipped}, Rows}) -> + FormattedRow = format_test_row(Result), + {{_Pass, _Fail, Skipped+1}, [FormattedRow|Rows]}. + +%% @doc Format a row for clique +format_test_row({TestPlan, Result, Duration}) -> + TestModule = rt_test_plan:get_module(TestPlan), + TestString = atom_to_list(TestModule), + case Result of + {Status, Reason} -> + [TestString, Status, Reason, test_summary_format_time(Duration)]; + pass -> + [TestString, "pass", "N/A", test_summary_format_time(Duration)] + end. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Gather all of the log files from the nodes and either upload to +%% GiddyUp or copy them to the directory of your choice. Also upload +%% latest test result, if necessary. +%% +%% @spec report_and_gather_logs(Directory) -> ok +%% @end +%%-------------------------------------------------------------------- +%% -spec(report_and_gather_logs(giddyup|string(), term()) -> ok). +report_and_gather_logs(giddyup, TestResult) -> + {TestPlan, Reason, _Duration} = TestResult, + Status = case Reason of + pass -> pass; + _ -> fail + end, + GiddyupResult = [ + {test, rt_test_plan:get_module(TestPlan)}, + {status, Status}, + {backend, rt_test_plan:get(backend, TestPlan)}, + {id, rt_test_plan:get(id, TestPlan)}, + {platform, rt_test_plan:get(platform, TestPlan)}, + {version, rt_test_plan:get(version, TestPlan)}, + {project, rt_test_plan:get(project, TestPlan)} + ], + case giddyup:post_result(GiddyupResult) of + error -> woops; + {ok, Base} -> + [giddyup:post_artifact(Base, File) || File <- rt:get_node_logs(giddyup)] + end; +report_and_gather_logs(LogDir, {TestPlan, _, _}) -> + SubDir = filename:join([LogDir, rt_test_plan:get_name(TestPlan)]), + rt:get_node_logs(SubDir). +%% +%% RetList = [{test, TestModule}, {status, Status}, {log, Log}, {backend, Backend} | proplists:delete(backend, TestMetaData)], +%% case Status of +%% fail -> RetList ++ [{reason, iolist_to_binary(io_lib:format("~p", [Reason]))}]; +%% _ -> RetList +%% end. diff --git a/src/rt_test_plan.erl b/src/rt_test_plan.erl index 90374be0e..b48da96c6 100644 --- a/src/rt_test_plan.erl +++ b/src/rt_test_plan.erl @@ -34,15 +34,17 @@ new/1, get/2, get_module/1, + get_name/1, set/2, set/3]). -record(rt_test_plan_v1, { id=-1 :: integer(), module :: atom(), - project :: atom() | binary(), + project=rt_config:get_default_version_product() :: atom() | binary(), platform :: string(), - backend=riak_kv_bitcask_backend :: rt_properties2:storage_backend(), + version=rt_config:get_default_version_number() :: string(), + backend=bitcask :: atom(), upgrade_path=[] :: [rt_properties2:product_version()], properties :: rt_properties2:properties() }). @@ -86,6 +88,15 @@ get(Field, TestPlan) -> get_module(TestPlan) -> get(module, TestPlan, validate_request(module, TestPlan)). +% @doc Get the value of the name and backend from a test plan record. An error +%% is returned if `TestPlan' is not a valid `rt_test_plan' record +%% or if the field requested is not a valid field. +-spec get_name(test_plan()) -> term() | {error, atom()}. +get_name(TestPlan) -> + Module = get(module, TestPlan, validate_request(module, TestPlan)), + Backend = get(backend, TestPlan, validate_request(backend, TestPlan)), + atom_to_list(Module) ++ "-" ++ atom_to_list(Backend). + %% @doc Set the value for a field in a test plan record. An error %% is returned if `TestPlan' is not a valid `rt_test_plan' record %% or if any of the fields to be set are not a valid field. In @@ -190,6 +201,8 @@ field_index(project) -> ?RT_TEST_PLAN.project; field_index(platform) -> ?RT_TEST_PLAN.platform; +field_index(version) -> + ?RT_TEST_PLAN.version; field_index(backend) -> ?RT_TEST_PLAN.backend; field_index(upgrade_path) -> diff --git a/src/rtdev.erl b/src/rtdev.erl index a902dc0ec..a00bf8102 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -49,7 +49,8 @@ set_conf/2, set_advanced_conf/2, rm_dir/1, - validate_config/1]). + validate_config/1, + get_node_logs/1]). -compile(export_all). -include_lib("eunit/include/eunit.hrl"). @@ -962,13 +963,30 @@ devpaths() -> %% versions() -> %% proplists:get_keys(rt_config:get(rtdev_path)) -- [root]. -get_node_logs() -> +% @doc Get the list of log files and config files and pass them back +get_node_logs(DestDir) -> Root = filename:absname(?PATH), RootLen = length(Root) + 1, %% Remove the leading slash - [ begin - {ok, Port} = file:open(Filename, [read, binary]), - {lists:nthtail(RootLen, Filename), Port} - end || Filename <- filelib:wildcard(Root ++ "/*/dev*/log/*") ]. + Fun = get_node_log_fun(DestDir, RootLen), + [ Fun(Filename) || Filename <- filelib:wildcard(Root ++ "/*/dev*/log/*") ++ + filelib:wildcard(Root ++ "/*/dev*/etc/*.conf*") ]. + +% @doc Either open a port for uploading each file to GiddyUp or +% Copy each file to a local directory +get_node_log_fun(giddyup, RootLen) -> + fun(Filename) -> + {ok, Port} = file:open(Filename, [read, binary]), + {lists:nthtail(RootLen, Filename), Port} + end; +get_node_log_fun(DestDir, RootLen) -> + DestRoot = filename:absname(DestDir), + lager:debug("Copying log files to ~p", [DestRoot]), + fun(Filename) -> + Target = filename:join([DestRoot, lists:nthtail(RootLen, Filename)]), + ok = filelib:ensure_dir(Target), + {ok, _BytesWritten} = file:copy(Filename, Target), + Filename + end. -type node_tuple() :: {list(), atom()}. From 62f03fd795ebb1be09f86b0d206bfb1acbaa125b Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Thu, 16 Apr 2015 22:02:56 +0000 Subject: [PATCH 130/157] Re-sync all tests from master --- tests/basic_command_line.erl | 2 +- tests/bucket_props_roundtrip.erl | 17 +- tests/bucket_types.erl | 148 +++++++++--- tests/cluster_meta_basic.erl | 16 +- tests/cluster_meta_rmr.erl | 8 +- tests/cuttlefish_configuration.erl | 2 +- tests/ensemble_basic.erl | 2 - tests/ensemble_basic2.erl | 4 +- tests/ensemble_basic3.erl | 16 +- tests/ensemble_basic4.erl | 12 +- tests/ensemble_interleave.erl | 2 +- tests/ensemble_remove_node.erl | 6 +- tests/ensemble_remove_node2.erl | 16 +- tests/ensemble_sync.erl | 14 +- tests/ensemble_util.erl | 4 +- tests/ensemble_vnode_crash.erl | 12 +- tests/gh_riak_core_154.erl | 18 +- tests/gh_riak_core_155.erl | 2 +- tests/gh_riak_core_176.erl | 20 +- tests/gh_riak_kv_765.erl | 2 +- tests/http_bucket_types.erl | 40 +--- tests/http_security.erl | 12 +- tests/jmx_verify.erl | 2 +- tests/loaded_upgrade.erl | 2 +- tests/loaded_upgrade_worker_sup.erl | 4 +- tests/mapred_basic_compat.erl | 10 +- tests/mapred_buffer_prereduce.erl | 2 +- tests/mapred_dead_pipe.erl | 2 +- tests/mapred_http_errors.erl | 2 +- tests/mapred_javascript.erl | 4 +- tests/mapred_notfound_failover.erl | 6 +- tests/mapred_search_switch.erl | 10 +- tests/mapred_verify_rt.erl | 2 +- tests/partition_repair.erl | 2 +- tests/pb_cipher_suites.erl | 2 +- tests/pipe_verify_basics.erl | 4 +- tests/pipe_verify_exceptions.erl | 4 +- tests/pipe_verify_handoff.erl | 6 +- tests/pipe_verify_handoff_blocking.erl | 6 +- .../pipe_verify_restart_input_forwarding.erl | 4 +- tests/pipe_verify_sink_types.erl | 4 +- tests/post_generate_key.erl | 2 +- tests/pr_pw.erl | 10 +- tests/replication/repl_aae_fullsync.erl | 44 ++-- tests/replication/repl_aae_fullsync_util.erl | 6 +- tests/replication/repl_bucket_types.erl | 2 +- tests/replication/repl_cancel_fullsync.erl | 8 +- .../repl_consistent_object_filter.erl | 14 +- tests/replication/repl_fs_bench.erl | 16 +- tests/replication/repl_fs_stat_caching.erl | 2 +- tests/replication/repl_location_failures.erl | 10 +- tests/replication/repl_rt_cascading_rtq.erl | 2 +- tests/replication/repl_rt_heartbeat.erl | 2 +- tests/replication/repl_rt_overload.erl | 4 +- tests/replication/repl_rt_pending.erl | 2 +- tests/replication/repl_util.erl | 2 +- tests/replication/replication2.erl | 4 +- .../replication/replication2_connections.erl | 20 +- .../replication2_console_tests.erl | 1 + tests/replication/replication2_fsschedule.erl | 6 +- tests/replication/replication2_pg.erl | 2 +- .../replication2_rt_sink_connection.erl | 4 +- tests/replication/replication2_ssl.erl | 6 +- tests/replication/replication2_upgrade.erl | 4 +- .../replication_object_reformat.erl | 2 +- tests/replication/replication_ssl.erl | 14 +- tests/replication/replication_stats.erl | 8 +- tests/replication/replication_upgrade.erl | 4 +- tests/riak_admin_console_tests.erl | 6 +- tests/riak_control.erl | 2 +- tests/riak_control_authentication.erl | 2 +- tests/riak_rex.erl | 4 +- tests/riaknostic_rt.erl | 12 +- tests/rolling_capabilities.erl | 2 +- tests/rt_basic_test.erl | 4 +- tests/secondary_index_tests.erl | 218 ++++++++---------- tests/sibling_explosion.erl | 4 +- tests/test_cluster.erl | 4 +- tests/verify_2i_aae.erl | 2 +- tests/verify_2i_limit.erl | 8 +- tests/verify_2i_mixed_cluster.erl | 12 +- tests/verify_2i_returnterms.erl | 4 +- tests/verify_2i_stream.erl | 4 +- tests/verify_2i_timeout.erl | 4 +- tests/verify_aae.erl | 2 +- tests/verify_api_timeouts.erl | 18 +- tests/verify_asis_put.erl | 10 +- tests/verify_backup_restore.erl | 24 +- tests/verify_basic_upgrade.erl | 10 +- tests/verify_bitcask_tombstone2_upgrade.erl | 2 +- tests/verify_build_cluster.erl | 2 +- tests/verify_busy_dist_port.erl | 2 +- tests/verify_capabilities.erl | 1 + tests/verify_commit_hooks.erl | 20 +- tests/verify_conditional_postcommit.erl | 10 +- tests/verify_corruption_filtering.erl | 4 +- tests/verify_counter_capability.erl | 4 +- tests/verify_counter_converge.erl | 6 +- tests/verify_counter_repl.erl | 4 +- tests/verify_crdt_capability.erl | 2 +- tests/verify_cs_bucket.erl | 4 +- tests/verify_down.erl | 2 +- tests/verify_dt_context.erl | 10 +- tests/verify_dt_converge.erl | 2 +- tests/verify_dt_upgrade.erl | 10 +- tests/verify_dvv_repl.erl | 10 +- tests/verify_dynamic_ring.erl | 28 +-- tests/verify_handoff.erl | 2 +- tests/verify_handoff_mixed.erl | 2 +- tests/verify_kv_health_check.erl | 4 +- tests/verify_link_walk_urls.erl | 2 +- tests/verify_listkeys.erl | 43 ++-- tests/verify_listkeys_eqcfsm.erl | 12 +- tests/verify_mr_prereduce_node_down.erl | 8 +- tests/verify_no_writes_on_read.erl | 2 +- tests/verify_object_limits.erl | 2 +- tests/verify_reset_bucket_props.erl | 4 +- tests/verify_riak_lager.erl | 6 +- tests/verify_riak_object_reformat.erl | 10 +- tests/verify_riak_stats.erl | 40 ++-- tests/verify_search.erl | 2 +- tests/verify_secondary_index_reformat.erl | 16 +- tests/verify_snmp.erl | 8 +- tests/verify_staged_clustering.erl | 2 +- tests/verify_tick_change.erl | 8 +- tests/verify_vclock.erl | 14 +- tests/verify_vclock_encoding_upgrade.erl | 2 +- tests/yz_crdt.erl | 2 +- tests/yz_ensemble.erl | 10 +- 129 files changed, 687 insertions(+), 647 deletions(-) diff --git a/tests/basic_command_line.erl b/tests/basic_command_line.erl index 217cc624f..958a5455e 100644 --- a/tests/basic_command_line.erl +++ b/tests/basic_command_line.erl @@ -165,4 +165,4 @@ getpid_down_test(Node) -> lager:info("Test riak getpid fails on ~s", [Node]), {ok, PidOut} = rt:riak(Node, ["getpid"]), ?assert(rt:str(PidOut, "Node is not running!")), - ok. \ No newline at end of file + ok. diff --git a/tests/bucket_props_roundtrip.erl b/tests/bucket_props_roundtrip.erl index a2d7876c9..ef2fe5c84 100644 --- a/tests/bucket_props_roundtrip.erl +++ b/tests/bucket_props_roundtrip.erl @@ -18,10 +18,9 @@ %% %% ------------------------------------------------------------------- -module(bucket_props_roundtrip). --export([properties/0, confirm/2]). - +-behaviour(riak_test). +-export([confirm/0]). -include_lib("eunit/include/eunit.hrl"). --include("rt.hrl"). -define(BUCKET, <<"pbc_props_verify">>). -define(COMMIT_HOOK, {struct, [{<<"mod">>, <<"foo">>}, {<<"fun">>, <<"bar">>}]}). @@ -53,11 +52,9 @@ {young_vclock, 0, 20} ]). -properties() -> - rt_properties:new([{node_count, 1}]). - -confirm(Properties, _MD) -> - [Node] = rt_properties:get(nodes, Properties), +confirm() -> + [Node] = Nodes = rt:build_cluster(1), + ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), [ check_prop_set_and_get(Node, Prop, FirstVal, SecondVal) || {Prop, FirstVal, SecondVal} <- ?PROPS ], @@ -66,8 +63,8 @@ confirm(Properties, _MD) -> check_prop_set_and_get(Node, Prop, One, Two) -> lager:info("-------- Testing roundtrip for property '~p' ---------", [Prop]), - HTTP = rt_http:httpc(Node), - PBC = rt_pb:pbc(Node), + HTTP = rt:httpc(Node), + PBC = rt:pbc(Node), lager:info("HTTP set = ~p", [One]), http_set_property(HTTP, Prop, One), lager:info("PBC get should == ~p", [One]), diff --git a/tests/bucket_types.erl b/tests/bucket_types.erl index 715f2cc07..1715cee82 100644 --- a/tests/bucket_types.erl +++ b/tests/bucket_types.erl @@ -1,44 +1,26 @@ -module(bucket_types). -%% -behaviour(riak_test). - --export([properties/0, - confirm/1, - mapred_modfun/3, - mapred_modfun_type/3]). +-behavior(riak_test). +-export([confirm/0, mapred_modfun/3, mapred_modfun_type/3]). -include_lib("eunit/include/eunit.hrl"). --include("rt.hrl"). - --test_type([bucket_types]). - -properties() -> - CustomConfig = rt_cluster:augment_config(riak_core, - {default_bucket_props, [{n_val, 2}]}, - rt_properties:default_config()), - rt_properties:new([{node_count, 4}, - {config, CustomConfig}]). - --spec confirm(rt_properties:properties()) -> pass | fail. -confirm(Properties) -> - NodeIds = rt_properties:get(node_ids, Properties), - NodeMap = rt_properties:get(node_map, Properties), - Nodes = [rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds], - Node = hd(Nodes), +confirm() -> application:start(inets), + lager:info("Deploy some nodes"), + Nodes = rt:build_cluster(4, [], [ + {riak_core, [{default_bucket_props, + [{n_val, 2}]}]}]), + Node = hd(Nodes), - RMD = rt_properties:get(metadata, Properties), + RMD = riak_test_runner:metadata(), HaveIndexes = case proplists:get_value(backend, RMD) of undefined -> false; %% default is da 'cask bitcask -> false; _ -> true end, - {ok, [{"127.0.0.1", Port}]} = rpc:call(Node, application, get_env, - [riak_api, pb]), - - {ok, PB} = riakc_pb_socket:start_link("127.0.0.1", Port, []), + PB = rt:pbc(Node), lager:info("default type get/put test"), %% write explicitly to the default type @@ -99,9 +81,13 @@ confirm(Properties) -> %% read from the default bucket explicitly {error, notfound} = riakc_pb_socket:get(PB, {<<"default">>, <<"bucket">>}, <<"key">>), - timer:sleep(5000), %% wait for delete_mode 3s to expire + ?assertEqual(ok, rt:wait_until(fun() -> + rt:pbc_really_deleted(PB, + <<"bucket">>, + [<<"key">>]) + end)), - %% now there shoyld be no buckets or keys to be listed... + %% now there should be no buckets or keys to be listed... %% %% list keys ?assertEqual({ok, []}, riakc_pb_socket:list_keys(PB, <<"bucket">>)), @@ -114,13 +100,20 @@ confirm(Properties) -> lager:info("custom type get/put test"), Type = <<"mytype">>, - rt_bucket_types:create_and_activate_bucket_type(Node, Type, [{n_val, 3}]), - rt_bucket_types:wait_until_bucket_type_status(Type, active, Nodes), + TypeProps = [{n_val, 3}], + lager:info("Create bucket type ~p, wait for propagation", [Type]), + rt:create_and_activate_bucket_type(Node, Type, TypeProps), + rt:wait_until_bucket_type_status(Type, active, Nodes), + rt:wait_until_bucket_props(Nodes, {Type, <<"bucket">>}, TypeProps), lager:info("doing put"), riakc_pb_socket:put(PB, riakc_obj:new({Type, <<"bucket">>}, <<"key">>, <<"newestvalue">>)), + lager:info("custom type list_keys test"), + ?assertEqual({ok, []}, riakc_pb_socket:list_keys(PB, <<"bucket">>)), + ?assertEqual({ok, [<<"key">>]}, riakc_pb_socket:list_keys(PB, {Type, + <<"bucket">>})), lager:info("doing get"), {ok, O5} = riakc_pb_socket:get(PB, {Type, <<"bucket">>}, <<"key">>), @@ -130,15 +123,42 @@ confirm(Properties) -> %% this type is NOT aliased to the default buckey {error, notfound} = riakc_pb_socket:get(PB, <<"bucket">>, <<"key">>), - lager:info("custom type list_keys test"), - ?assertEqual({ok, []}, riakc_pb_socket:list_keys(PB, <<"bucket">>)), - ?assertEqual({ok, [<<"key">>]}, riakc_pb_socket:list_keys(PB, {Type, - <<"bucket">>})), lager:info("custom type list_buckets test"), %% list buckets ?assertEqual({ok, []}, riakc_pb_socket:list_buckets(PB)), ?assertEqual({ok, [<<"bucket">>]}, riakc_pb_socket:list_buckets(PB, Type)), + %%% Beginning of UTF-8 test + + lager:info("UTF-8 type get/put test"), + %% こんにちは - konnichiwa (Japanese) + UnicodeType = unicode:characters_to_binary([12371,12435,12395,12385,12399], utf8), + %% سلام - Salam (Arabic) + UnicodeBucket = unicode:characters_to_binary([1587,1604,1575,1605], utf8), + lager:info("Create bucket type, wait for propagation"), + rt:create_and_activate_bucket_type(Node, UnicodeType, TypeProps), + rt:wait_until_bucket_type_status(UnicodeType, active, Nodes), + rt:wait_until_bucket_props(Nodes, {UnicodeType, UnicodeBucket}, TypeProps), + + lager:info("doing put"), + riakc_pb_socket:put(PB, riakc_obj:new({UnicodeType, UnicodeBucket}, + <<"key">>, <<"yetanothervalue">>)), + + lager:info("custom type list_keys test"), + ?assertEqual({ok, [<<"key">>]}, riakc_pb_socket:list_keys(PB, + {UnicodeType, + UnicodeBucket})), + lager:info("doing get"), + {ok, O6} = riakc_pb_socket:get(PB, {UnicodeType, UnicodeBucket}, <<"key">>), + + ?assertEqual(<<"yetanothervalue">>, riakc_obj:get_value(O6)), + + lager:info("custom type list_buckets test"), + %% list buckets + ?assertEqual({ok, [UnicodeBucket]}, riakc_pb_socket:list_buckets(PB, UnicodeType)), + + %%% End of UTF-8 test + lager:info("bucket properties tests"), riakc_pb_socket:set_bucket(PB, {<<"default">>, <<"mybucket">>}, [{n_val, 5}]), @@ -160,12 +180,38 @@ confirm(Properties) -> <<"mybucket">>}), ?assertEqual(5, proplists:get_value(n_val, BProps3)), + %% Check our unicode brethren + riakc_pb_socket:set_bucket(PB, {UnicodeType, UnicodeBucket}, + [{n_val, 4}]), + {ok, UBProps1} = riakc_pb_socket:get_bucket(PB, {UnicodeType, + UnicodeBucket}), + ?assertEqual(4, proplists:get_value(n_val, UBProps1)), + riakc_pb_socket:reset_bucket(PB, {Type, <<"mybucket">>}), {ok, BProps4} = riakc_pb_socket:get_bucket(PB, {Type, <<"mybucket">>}), ?assertEqual(3, proplists:get_value(n_val, BProps4)), + riakc_pb_socket:reset_bucket(PB, {UnicodeType, UnicodeBucket}), + + {ok, UBProps2} = riakc_pb_socket:get_bucket(PB, {UnicodeType, + UnicodeBucket}), + + ?assertEqual(3, proplists:get_value(n_val, UBProps2)), + + {error, NTGR} = riakc_pb_socket:get_bucket(PB, {<<"nonexistent">>, <<"mybucket">>}), + + lager:info("GOT ERROR ~s", [NTGR]), + + ?assertMatch(<<"No bucket-type named 'nonexistent'", _/binary>>, NTGR), + + {error, NTSR} = riakc_pb_socket:set_bucket(PB, {<<"nonexistent">>, <<"mybucket">>}, [{n_val, 3}]), + + lager:info("GOT ERROR ~s", [NTSR]), + + ?assertMatch(<<"No bucket-type named 'nonexistent'", _/binary>>, NTSR), + lager:info("bucket type properties test"), riakc_pb_socket:set_bucket_type(PB, Type, @@ -186,6 +232,27 @@ confirm(Properties) -> ?assertEqual(3, proplists:get_value(n_val, BProps7)), + %% Repeat type checks for unicode type/bucket + + riakc_pb_socket:set_bucket_type(PB, UnicodeType, + [{n_val, 5}]), + + {ok, UBProps3} = riakc_pb_socket:get_bucket_type(PB, UnicodeType), + + ?assertEqual(5, proplists:get_value(n_val, UBProps3)), + + %% check that the bucket inherits from its type + {ok, UBProps4} = riakc_pb_socket:get_bucket(PB, {UnicodeType, + UnicodeBucket}), + + ?assertEqual(5, proplists:get_value(n_val, UBProps4)), + + riakc_pb_socket:set_bucket_type(PB, UnicodeType, [{n_val, 3}]), + + {ok, UBProps5} = riakc_pb_socket:get_bucket_type(PB, UnicodeType), + + ?assertEqual(3, proplists:get_value(n_val, UBProps5)), + %% make sure a regular bucket under the default type reflects app.config {ok, BProps8} = riakc_pb_socket:get_bucket(PB, {<<"default">>, <<"mybucket">>}), @@ -204,8 +271,8 @@ confirm(Properties) -> %% make sure a newly created type is not affected either %% create a new type Type2 = <<"mynewtype">>, - rt_bucket_types:create_and_activate_bucket_type(Node, Type2, []), - rt_bucket_types:wait_until_bucket_type_status(Type2, active, Nodes), + rt:create_and_activate_bucket_type(Node, Type2, []), + rt:wait_until_bucket_type_status(Type2, active, Nodes), {ok, BProps11} = riakc_pb_socket:get_bucket_type(PB, Type2), @@ -296,6 +363,11 @@ confirm(Properties) -> {<<"baz">>, <<"4">>, <<"a">>, 4}, {<<"bam">>, <<"5">>, <<"a">>, 3}]], + ?assertEqual({ok, [{0, [<<"2">>]}]}, + riakc_pb_socket:mapred(PB, {{Type, <<"MRbucket">>}, + [[<<"starts_with">>, <<"f">>]]}, + [{map, {modfun, riak_kv_mapreduce, map_object_value}, none, true}])), + ?assertEqual({ok, [{1, [14]}]}, riakc_pb_socket:mapred_bucket(PB, {Type, <<"MRbucket">>}, [{map, {jsfun, <<"Riak.mapValuesJson">>}, undefined, false}, diff --git a/tests/cluster_meta_basic.erl b/tests/cluster_meta_basic.erl index f4012a0b4..b659ec107 100644 --- a/tests/cluster_meta_basic.erl +++ b/tests/cluster_meta_basic.erl @@ -18,11 +18,10 @@ %% %% ------------------------------------------------------------------- -module(cluster_meta_basic). --export([properties/0, confirm/2, object_count/2]). - +-behavior(riak_test). +-export([confirm/0, object_count/2]). -include_lib("eunit/include/eunit.hrl"). --include("rt.hrl"). -define(PREFIX1, {a, b}). -define(PREFIX2, {fold, prefix}). -define(KEY1, key1). @@ -30,11 +29,8 @@ -define(VAL1, val1). -define(VAL2, val2). -properties() -> - rt_properties:new([{node_count, 5}]). - -confirm(Properties, _MD) -> - Nodes = rt_properties:get(nodes, Properties), +confirm() -> + Nodes = rt:build_cluster(5), ok = test_fold_full_prefix(Nodes), ok = test_metadata_conflicts(Nodes), ok = test_writes_after_partial_cluster_failure(Nodes), @@ -54,13 +50,13 @@ test_writes_after_partial_cluster_failure([N1 | _]=Nodes) -> StopNodes = eager_peers(N1, N1), AliveNodes = Nodes -- StopNodes, lager:info("stopping nodes: ~p remaining nodes: ~p", [StopNodes, AliveNodes]), - [rt_node:stop(N) || N <- StopNodes], + [rt:stop(N) || N <- StopNodes], metadata_put(N1, ?PREFIX1, ?KEY1, ?VAL2), wait_until_metadata_value(AliveNodes, ?PREFIX1, ?KEY1, ?VAL2), lager:info("bring stopped nodes back up: ~p", [StopNodes]), - [rt_node:start(N) || N <- StopNodes], + [rt:start(N) || N <- StopNodes], wait_until_metadata_value(Nodes, ?PREFIX1, ?KEY1, ?VAL2), ok. diff --git a/tests/cluster_meta_rmr.erl b/tests/cluster_meta_rmr.erl index 5e39fdc1a..bc5fcab11 100644 --- a/tests/cluster_meta_rmr.erl +++ b/tests/cluster_meta_rmr.erl @@ -24,7 +24,7 @@ -define(CM_PREFIX, {test, cm}). confirm() -> - rt_config:set_conf(all, [{"ring_size", "128"}]), + rt:set_conf(all, [{"ring_size", "128"}]), Seed = erlang:now(), lager:info("SEED: ~p", [Seed]), random:seed(Seed), @@ -59,10 +59,10 @@ run(NumNodes, NumRounds, StableRounds) -> exit(Pid, kill), %% start all the down nodes so we can clean them :( [rt:start(Node) || Node <- DownNodes], - rt_cluster:clean_cluster(AllNodes). + rt:clean_cluster(AllNodes). setup_nodes(NumNodes) -> - Nodes = rt_cluster:build_cluster(NumNodes), + Nodes = rt:build_cluster(NumNodes), [begin ok = rpc:call(Node, application, set_env, [riak_core, broadcast_exchange_timer, 4294967295]), ok = rpc:call(Node, application, set_env, [riak_core, gossip_limit, {10000000, 4294967295}]), @@ -108,7 +108,7 @@ run_rounds(Round, StableRound, SendFun, ConsistentFun, [SenderNode | _]=UpNodes, fail_node(Round, OtherNodes) -> Failed = lists:nth(random:uniform(length(OtherNodes)), OtherNodes), lager:info("round: ~p (unstable): shutting down ~p", [Round, Failed]), - rt_node:stop(Failed), + rt:stop(Failed), {Failed, lists:delete(Failed, OtherNodes)}. calc_stuff(AllNodes, NumNodes, NumRounds) -> diff --git a/tests/cuttlefish_configuration.erl b/tests/cuttlefish_configuration.erl index ea10009bf..4447bfca4 100644 --- a/tests/cuttlefish_configuration.erl +++ b/tests/cuttlefish_configuration.erl @@ -12,7 +12,7 @@ confirm() -> {"leveldb.sync_on_write", "on"} ], - [Node] = rt_cluster:deploy_nodes(1, {cuttlefish, CuttlefishConf}), + [Node] = rt:deploy_nodes(1, {cuttlefish, CuttlefishConf}), {ok, RingSize} = rt:rpc_get_env(Node, [{riak_core, ring_creation_size}]), ?assertEqual(8, RingSize), diff --git a/tests/ensemble_basic.erl b/tests/ensemble_basic.erl index 1f2d71e04..5e2b308d2 100644 --- a/tests/ensemble_basic.erl +++ b/tests/ensemble_basic.erl @@ -22,8 +22,6 @@ -export([confirm/0]). -include_lib("eunit/include/eunit.hrl"). --test_type([ensemble]). - confirm() -> NumNodes = 5, NVal = 5, diff --git a/tests/ensemble_basic2.erl b/tests/ensemble_basic2.erl index 77ec73c9b..dee9e815f 100644 --- a/tests/ensemble_basic2.erl +++ b/tests/ensemble_basic2.erl @@ -42,8 +42,8 @@ confirm() -> timer:sleep(Delay), riak_kv_vnode_orig:init_orig(Args) end}}]}), - rt_node:stop_and_wait(Node), - rt_node:start(Node), + rt:stop_and_wait(Node), + rt:start(Node), lager:info("Polling peers while riak_kv starts. We should see none"), UpNoPeersFun = fun() -> diff --git a/tests/ensemble_basic3.erl b/tests/ensemble_basic3.erl index 2fd7e5ba7..50878e797 100644 --- a/tests/ensemble_basic3.erl +++ b/tests/ensemble_basic3.erl @@ -37,7 +37,7 @@ confirm() -> ensemble_util:wait_until_stable(Node, NVal), lager:info("Creating/activating 'strong' bucket type"), - rt_bucket_types:create_and_activate_bucket_type(Node, <<"strong">>, + rt:create_and_activate_bucket_type(Node, <<"strong">>, [{consistent, true}, {n_val, NVal}]), ensemble_util:wait_until_stable(Node, NVal), Bucket = {<<"strong">>, <<"test">>}, @@ -55,20 +55,20 @@ confirm() -> Partitioned = [VNode || {_, VNode} <- PartitionedVN], MajorityVN = All -- PartitionedVN, - PBC = rt_pb:pbc(Node), + PBC = rt:pbc(Node), lager:info("Partitioning quorum minority: ~p", [Partitioned]), - Part = rt_node:partition(Nodes -- Partitioned, Partitioned), + Part = rt:partition(Nodes -- Partitioned, Partitioned), ensemble_util:wait_until_stable(Node, Quorum), lager:info("Writing ~p consistent keys", [1000]), - [ok = rt_pb:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], + [ok = rt:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], lager:info("Read keys to verify they exist"), - [rt_pb:pbc_read(PBC, Bucket, Key) || Key <- Keys], + [rt:pbc_read(PBC, Bucket, Key) || Key <- Keys], lager:info("Healing partition"), - rt_node:heal(Part), + rt:heal(Part), lager:info("Suspending majority vnodes"), L = [begin @@ -83,7 +83,7 @@ confirm() -> vnode_util:resume_vnode(Pid), ensemble_util:wait_until_stable(Node, Quorum), lager:info("Re-reading keys"), - [rt_pb:pbc_read(PBC, Bucket, Key) || Key <- Keys], + [rt:pbc_read(PBC, Bucket, Key) || Key <- Keys], lager:info("Suspending vnode: ~p", [VIdx]), Pid2 = vnode_util:suspend_vnode(VNode, VIdx), orddict:store(VN, Pid2, Suspended) @@ -93,5 +93,5 @@ confirm() -> [vnode_util:resume_vnode(Pid) || {_, Pid} <- L3], ensemble_util:wait_until_stable(Node, NVal), lager:info("Re-reading keys"), - [rt_pb:pbc_read(PBC, Bucket, Key) || Key <- Keys], + [rt:pbc_read(PBC, Bucket, Key) || Key <- Keys], pass. diff --git a/tests/ensemble_basic4.erl b/tests/ensemble_basic4.erl index 9c1f0abf3..494e21c55 100644 --- a/tests/ensemble_basic4.erl +++ b/tests/ensemble_basic4.erl @@ -32,7 +32,7 @@ confirm() -> Node = hd(Nodes), lager:info("Creating/activating 'strong' bucket type"), - rt_bucket_types:create_and_activate_bucket_type(Node, <<"strong">>, + rt:create_and_activate_bucket_type(Node, <<"strong">>, [{consistent, true}, {n_val, NVal}]), ensemble_util:wait_until_stable(Node, NVal), Bucket = {<<"strong">>, <<"test">>}, @@ -48,20 +48,20 @@ confirm() -> PartitionedVN = lists:sublist(Other, Minority), Partitioned = [VNode || {_, VNode} <- PartitionedVN], - PBC = rt_pb:pbc(Node), + PBC = rt:pbc(Node), lager:info("Partitioning quorum minority: ~p", [Partitioned]), - Part = rt_node:partition(Nodes -- Partitioned, Partitioned), + Part = rt:partition(Nodes -- Partitioned, Partitioned), rpc:multicall(Nodes, riak_kv_entropy_manager, set_mode, [manual]), ensemble_util:wait_until_stable(Node, Quorum), lager:info("Writing ~p consistent keys", [1000]), - [ok = rt_pb:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], + [ok = rt:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], lager:info("Read keys to verify they exist"), - [rt_pb:pbc_read(PBC, Bucket, Key) || Key <- Keys], + [rt:pbc_read(PBC, Bucket, Key) || Key <- Keys], lager:info("Healing partition"), - rt_node:heal(Part), + rt:heal(Part), pass. diff --git a/tests/ensemble_interleave.erl b/tests/ensemble_interleave.erl index 36c79a2c2..5e34659f6 100644 --- a/tests/ensemble_interleave.erl +++ b/tests/ensemble_interleave.erl @@ -98,4 +98,4 @@ confirm() -> lager:info("Re-reading keys to verify they exist"), Expect = [ok, {error, timeout}, {error, <<"timeout">>}, {error, <<"failed">>}], [rt:pbc_read_check(PBC, Bucket, Key, Expect, Options) || Key <- Keys], - pass. \ No newline at end of file + pass. diff --git a/tests/ensemble_remove_node.erl b/tests/ensemble_remove_node.erl index 51f92d13a..2d9e8e9e0 100644 --- a/tests/ensemble_remove_node.erl +++ b/tests/ensemble_remove_node.erl @@ -54,12 +54,12 @@ confirm() -> end}}]}), lager:info("Removing Nodes 2 and 3 from the cluster"), - rt_node:leave(Node2), + rt:leave(Node2), ok = ensemble_util:wait_until_stable(Node, NVal), - rt_node:leave(Node3), + rt:leave(Node3), ok = ensemble_util:wait_until_stable(Node, NVal), Remaining = Nodes -- [Node2, Node3], - rt_node:wait_until_nodes_agree_about_ownership(Remaining), + rt:wait_until_nodes_agree_about_ownership(Remaining), ok = rt:wait_until_unpingable(Node2), ok = rt:wait_until_unpingable(Node3), lager:info("Read value from the root ensemble"), diff --git a/tests/ensemble_remove_node2.erl b/tests/ensemble_remove_node2.erl index ee82e716e..3efaa7ace 100644 --- a/tests/ensemble_remove_node2.erl +++ b/tests/ensemble_remove_node2.erl @@ -39,14 +39,14 @@ confirm() -> {ok, _} = riak_ensemble_client:kget(Node, root, testerooni, 1000), lager:info("Creating/activating 'strong' bucket type"), - rt_bucket_types:create_and_activate_bucket_type(Node, <<"strong">>, + rt:create_and_activate_bucket_type(Node, <<"strong">>, [{consistent, true}, {n_val, NVal}]), ensemble_util:wait_until_stable(Node, NVal), Bucket = {<<"strong">>, <<"test">>}, Key = <<"testkey">>, - PBC = rt_pb:pbc(Node), - ok = rt_pb:pbc_write(PBC, Bucket, Key, testval), - Val1 = rt_pb:pbc_read(PBC, Bucket, Key), + PBC = rt:pbc(Node), + ok = rt:pbc_write(PBC, Bucket, Key, testval), + Val1 = rt:pbc_read(PBC, Bucket, Key), ?assertEqual(element(1, Val1), riakc_obj), %% Don't allow node deletions in riak_ensemble. This should prevent the @@ -61,12 +61,12 @@ confirm() -> end}}]}), lager:info("Removing Nodes 2 and 3 from the cluster"), - rt_node:leave(Node2), + rt:leave(Node2), ok = ensemble_util:wait_until_stable(Node, NVal), - rt_node:leave(Node3), + rt:leave(Node3), ok = ensemble_util:wait_until_stable(Node, NVal), Remaining = Nodes -- [Node2, Node3], - rt_node:wait_until_nodes_agree_about_ownership(Remaining), + rt:wait_until_nodes_agree_about_ownership(Remaining), %% TODO: How do we wait indefinitely for nodes to never exit here? A 30s %% sleep? @@ -80,7 +80,7 @@ confirm() -> %% We should still be able to read from k/v ensembles, but the nodes should %% never exit lager:info("Reading From SC Bucket"), - Val2 = rt_pb:pbc_read(PBC, Bucket, Key), + Val2 = rt:pbc_read(PBC, Bucket, Key), ?assertEqual(element(1, Val2), riakc_obj), ok = ensemble_util:wait_until_stable(Node, NVal), diff --git a/tests/ensemble_sync.erl b/tests/ensemble_sync.erl index 849c6a0aa..db1d19695 100644 --- a/tests/ensemble_sync.erl +++ b/tests/ensemble_sync.erl @@ -33,7 +33,7 @@ confirm() -> vnode_util:load(Nodes), lager:info("Creating/activating 'strong' bucket type"), - rt_bucket_types:create_and_activate_bucket_type(Node, <<"strong">>, + rt:create_and_activate_bucket_type(Node, <<"strong">>, [{consistent, true}, {n_val, NVal}]), ensemble_util:wait_until_stable(Node, NVal), @@ -97,7 +97,7 @@ run_scenario(Nodes, NVal, {NumKill, NumSuspend, NumValid, _, Name, Expect}) -> {AfterVN, _} = lists:split(NumValid, Valid3), io:format("PL: ~p~n", [PL]), - PBC = rt_pb:pbc(Node), + PBC = rt:pbc(Node), Options = [{timeout, 2000}], rpc:multicall(Nodes, riak_kv_entropy_manager, set_mode, [manual]), @@ -107,11 +107,11 @@ run_scenario(Nodes, NVal, {NumKill, NumSuspend, NumValid, _, Name, Expect}) -> %% Write data while minority is partitioned lager:info("Writing ~p consistent keys", [1000]), - [ok = rt_pb:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], + [ok = rt:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], lager:info("Read keys to verify they exist"), - [rt_pb:pbc_read(PBC, Bucket, Key, Options) || Key <- Keys], - rt_node:heal(Part), + [rt:pbc_read(PBC, Bucket, Key, Options) || Key <- Keys], + rt:heal(Part), %% Suspend desired number of valid vnodes S1 = [vnode_util:suspend_vnode(VNode, VIdx) || {VIdx, VNode} <- SuspendVN], @@ -133,7 +133,7 @@ run_scenario(Nodes, NVal, {NumKill, NumSuspend, NumValid, _, Name, Expect}) -> ensemble_util:wait_until_stable(Node, Quorum), lager:info("Checking that key results match scenario"), - [rt_pb:pbc_read_check(PBC, Bucket, Key, Expect, Options) || Key <- Keys], + [rt:pbc_read_check(PBC, Bucket, Key, Expect, Options) || Key <- Keys], lager:info("Re-enabling AAE"), rpc:multicall(Nodes, riak_kv_entropy_manager, enable, []), @@ -150,7 +150,7 @@ run_scenario(Nodes, NVal, {NumKill, NumSuspend, NumValid, _, Name, Expect}) -> ok; false -> lager:info("Re-reading keys to verify they exist"), - [rt_pb:pbc_read(PBC, Bucket, Key, Options) || Key <- Keys] + [rt:pbc_read(PBC, Bucket, Key, Options) || Key <- Keys] end, lager:info("Scenario passed"), diff --git a/tests/ensemble_util.erl b/tests/ensemble_util.erl index 14348ed53..d6f79145b 100644 --- a/tests/ensemble_util.erl +++ b/tests/ensemble_util.erl @@ -69,7 +69,7 @@ config_aae(true) -> {anti_entropy_tick, 100}, {anti_entropy, {on, []}}, {anti_entropy_timeout, 5000}, - {storage_backend, riak_kv_memory_backend}]}; + {storage_backend, riak_kv_memory_backend}]}; config_aae(false) -> {riak_kv, [{anti_entropy, {off, []}}]}. @@ -157,4 +157,4 @@ wait_for_membership(Node) -> end, ?assertEqual(ok, rt:wait_until(F)), lager:info("....ownership matches"), - ok. \ No newline at end of file + ok. diff --git a/tests/ensemble_vnode_crash.erl b/tests/ensemble_vnode_crash.erl index 35e666ebf..59fbd56e8 100644 --- a/tests/ensemble_vnode_crash.erl +++ b/tests/ensemble_vnode_crash.erl @@ -35,7 +35,7 @@ confirm() -> ensemble_util:wait_until_stable(Node, NVal), lager:info("Creating/activating 'strong' bucket type"), - rt_bucket_types:create_and_activate_bucket_type(Node, <<"strong">>, + rt:create_and_activate_bucket_type(Node, <<"strong">>, [{consistent, true}, {n_val, NVal}]), ensemble_util:wait_until_stable(Node, NVal), Bucket = {<<"strong">>, <<"test">>}, @@ -46,13 +46,13 @@ confirm() -> PL = rpc:call(Node, riak_core_apl, get_primary_apl, [DocIdx, NVal, riak_kv]), {{Key1Idx, Key1Node}, _} = hd(PL), - PBC = rt_pb:pbc(Node), + PBC = rt:pbc(Node), lager:info("Writing ~p consistent keys", [1000]), - [ok = rt_pb:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], + [ok = rt:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], lager:info("Read keys to verify they exist"), - [rt_pb:pbc_read(PBC, Bucket, Key) || Key <- Keys], + [rt:pbc_read(PBC, Bucket, Key) || Key <- Keys], %% Setting up intercept to ensure that %% riak_kv_ensemble_backend:handle_down/4 gets called when a vnode or vnode @@ -79,7 +79,7 @@ confirm() -> lager:info("Wait for stable ensembles"), ensemble_util:wait_until_stable(Node, NVal), lager:info("Re-reading keys"), - [rt_pb:pbc_read(PBC, Bucket, Key) || Key <- Keys], + [rt:pbc_read(PBC, Bucket, Key) || Key <- Keys], lager:info("Killing Vnode Proxy for Key1"), Proxy = rpc:call(Key1Node, riak_core_vnode_proxy, reg_name, [riak_kv_vnode, @@ -95,7 +95,7 @@ confirm() -> lager:info("Wait for stable ensembles"), ensemble_util:wait_until_stable(Node, NVal), lager:info("Re-reading keys"), - [rt_pb:pbc_read(PBC, Bucket, Key) || Key <- Keys], + [rt:pbc_read(PBC, Bucket, Key) || Key <- Keys], pass. diff --git a/tests/gh_riak_core_154.erl b/tests/gh_riak_core_154.erl index 5b9a50454..ff722a483 100644 --- a/tests/gh_riak_core_154.erl +++ b/tests/gh_riak_core_154.erl @@ -28,30 +28,30 @@ confirm() -> %% Increase handoff concurrency on nodes NewConfig = [{riak_core, [{handoff_concurrency, 1024}]}], - Nodes = rt_cluster:build_cluster(2, NewConfig), + Nodes = rt:build_cluster(2, NewConfig), ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), [Node1, Node2] = Nodes, lager:info("Write data while ~p is offline", [Node2]), - rt_node:stop(Node2), + rt:stop(Node2), rt:wait_until_unpingable(Node2), - ?assertEqual([], rt_systest:write(Node1, 1000, 3)), + ?assertEqual([], rt:systest_write(Node1, 1000, 3)), lager:info("Verify that ~p is missing data", [Node2]), - rt_node:start(Node2), - rt_node:stop(Node1), + rt:start(Node2), + rt:stop(Node1), rt:wait_until_unpingable(Node1), ?assertMatch([{_,{error,notfound}}|_], - rt_systest:read(Node2, 1000, 3)), + rt:systest_read(Node2, 1000, 3)), lager:info("Restart ~p and wait for handoff to occur", [Node1]), - rt_node:start(Node1), + rt:start(Node1), rt:wait_for_service(Node1, riak_kv), rt:wait_until_transfers_complete([Node1]), lager:info("Verify that ~p has all data", [Node2]), - rt_node:stop(Node1), - ?assertEqual([], rt_systest:read(Node2, 1000, 3)), + rt:stop(Node1), + ?assertEqual([], rt:systest_read(Node2, 1000, 3)), lager:info("gh_riak_core_154: passed"), pass. diff --git a/tests/gh_riak_core_155.erl b/tests/gh_riak_core_155.erl index 2abb856bd..6a6e7da82 100644 --- a/tests/gh_riak_core_155.erl +++ b/tests/gh_riak_core_155.erl @@ -66,4 +66,4 @@ perform_gets2(0, _, _, _) -> ok; perform_gets2(Count, Node, PL, BKey) -> rpc:call(Node, riak_kv_vnode, get, [PL, BKey, make_ref()], 1000), - perform_gets(Count - 1, Node, PL, BKey). \ No newline at end of file + perform_gets(Count - 1, Node, PL, BKey). diff --git a/tests/gh_riak_core_176.erl b/tests/gh_riak_core_176.erl index 9860a656e..00821a615 100644 --- a/tests/gh_riak_core_176.erl +++ b/tests/gh_riak_core_176.erl @@ -23,7 +23,7 @@ -include_lib("eunit/include/eunit.hrl"). confirm() -> - Nodes = rt_cluster:deploy_nodes(3), + Nodes = rt:deploy_nodes(3), [Node1, Node2, Node3] = Nodes, Nodes12 = [Node1, Node2], Nodes123 = Nodes, @@ -47,31 +47,31 @@ confirm() -> lager:info("Change ~p handoff_ip from ~p to ~p", [Node2, NodeIP, AlternateIP]), NewConfig = [{riak_core, [{handoff_ip, AlternateIP}]}], - rt_config:update_app_config(Node2, NewConfig), + rt:update_app_config(Node2, NewConfig), rt:wait_for_service(Node2, riak_kv), lager:info("Write data to the cluster"), - rt_systest:write(Node1, 100), + rt:systest_write(Node1, 100), lager:info("Join ~p to the cluster and wait for handoff to finish", [Node2]), - rt_node:join(Node2, Node1), - ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes12)), + rt:join(Node2, Node1), + ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes12)), ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes12)), - rt_node:wait_until_nodes_agree_about_ownership(Nodes12), + rt:wait_until_nodes_agree_about_ownership(Nodes12), %% Check 0.0.0.0 address works lager:info("Change ~p handoff_ip to \"0.0.0.0\"", [Node3]), - rt_config:update_app_config(Node3, + rt:update_app_config(Node3, [{riak_core, [{handoff_ip, "0.0.0.0"}]}]), lager:info("Join ~p to the cluster and wait for handoff to finish", [Node3]), rt:wait_for_service(Node3, riak_kv), - rt_node:join(Node3, Node1), - ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes123)), + rt:join(Node3, Node1), + ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes123)), ?assertEqual(ok, rt:wait_until_no_pending_changes(Nodes123)), - rt_node:wait_until_nodes_agree_about_ownership(Nodes123), + rt:wait_until_nodes_agree_about_ownership(Nodes123), lager:info("Test gh_riak_core_176 passed"), pass. diff --git a/tests/gh_riak_kv_765.erl b/tests/gh_riak_kv_765.erl index 877a28288..a8f070cdd 100644 --- a/tests/gh_riak_kv_765.erl +++ b/tests/gh_riak_kv_765.erl @@ -120,4 +120,4 @@ expire_tree(Node, Partition) -> {0, Built} = lists:keyfind(0, 1, Info), Built > Now end), - ok. \ No newline at end of file + ok. diff --git a/tests/http_bucket_types.erl b/tests/http_bucket_types.erl index 8b2c55672..bc56b9d11 100644 --- a/tests/http_bucket_types.erl +++ b/tests/http_bucket_types.erl @@ -1,47 +1,29 @@ -module(http_bucket_types). --export([properties/0, - confirm/1, - mapred_modfun/3, - mapred_modfun_type/3]). +-behavior(riak_test). +-export([confirm/0, mapred_modfun/3, mapred_modfun_type/3]). -include_lib("eunit/include/eunit.hrl"). -include_lib("riakc/include/riakc.hrl"). --include("rt.hrl"). - --define(WAIT(E), ?assertEqual(ok, rt:wait_until(fun() -> - (E) end))). --test_type([bucket_types, http]). - -properties() -> - CustomConfig = rt_cluster:augment_config(riak_core, - {default_bucket_props, [{n_val, 2}]}, - rt_properties:default_config()), - rt_properties:new([{node_count, 1}, - {config, CustomConfig}]). - --spec confirm(rt_properties:properties()) -> pass | fail. -confirm(Properties) -> - NodeIds = rt_properties:get(node_ids, Properties), - NodeMap = rt_properties:get(node_map, Properties), - Nodes = [rt_node:node_name(NodeId, NodeMap) || NodeId <- NodeIds], - Node = hd(Nodes), +-define(WAIT(E), ?assertEqual(ok, rt:wait_until(fun() -> (E) end))). + +confirm() -> application:start(ibrowse), lager:info("Deploy some nodes"), - Nodes = rt_cluster:build_cluster(4, [], [ + Nodes = rt:build_cluster(4, [], [ {riak_core, [{default_bucket_props, [{n_val, 2}]}]}]), Node = hd(Nodes), - RMD = rt_properties:get(metadata, Properties), + RMD = riak_test_runner:metadata(), HaveIndexes = case proplists:get_value(backend, RMD) of undefined -> false; %% default is da 'cask bitcask -> false; _ -> true end, - RHC = rt_http:httpc(Node), + RHC = rt:httpc(Node), lager:info("default type get/put test"), %% write explicitly to the default type ok = rhc:put(RHC, riakc_obj:new({<<"default">>, <<"bucket">>}, @@ -119,7 +101,7 @@ confirm(Properties) -> lager:info("custom type get/put test"), %% create a new type - ok = rt_bucket_types:create_and_activate_bucket_type(Node, <<"mytype">>, [{n_val,3}]), + ok = rt:create_and_activate_bucket_type(Node, <<"mytype">>, [{n_val,3}]), %% allow cluster metadata some time to propogate timer:sleep(1000), @@ -154,7 +136,7 @@ confirm(Properties) -> UCBBin = {UnicodeTypeBin, UnicodeBucketBin}, - ok = rt_bucket_types:create_and_activate_bucket_type(Node, UnicodeTypeBin, [{n_val,3}]), + ok = rt:create_and_activate_bucket_type(Node, UnicodeTypeBin, [{n_val,3}]), lager:info("doing put"), ok = rhc:put(RHC, riakc_obj:new(UCBBin, @@ -243,7 +225,7 @@ confirm(Properties) -> %% make sure a newly created type is not affected either %% create a new type - ok = rt_bucket_types:create_and_activate_bucket_type(Node, <<"mynewtype">>, []), + ok = rt:create_and_activate_bucket_type(Node, <<"mynewtype">>, []), %% allow cluster metadata some time to propogate timer:sleep(1000), diff --git a/tests/http_security.erl b/tests/http_security.erl index 6134cdd64..8bf09ddb6 100644 --- a/tests/http_security.erl +++ b/tests/http_security.erl @@ -43,7 +43,7 @@ confirm() -> {enabled, true} ]} ], - Nodes = rt_cluster:build_cluster(4, Conf), + Nodes = rt:build_cluster(4, Conf), Node = hd(Nodes), %% enable security on the cluster ok = rpc:call(Node, riak_core_console, security_enable, [[]]), @@ -246,7 +246,7 @@ confirm() -> "default", "from", Username]]), %% list keys with bucket type - rt_bucket_types:create_and_activate_bucket_type(Node, <<"list-keys-test">>, []), + rt:create_and_activate_bucket_type(Node, <<"list-keys-test">>, []), lager:info("Checking that list keys on a bucket-type is disallowed"), ?assertMatch({error, {"403", _}}, rhc:list_keys(C7, {<<"list-keys-test">>, <<"hello">>})), @@ -520,7 +520,7 @@ confirm() -> enable_ssl(Node) -> [{http, {IP, Port}}|_] = rt:connection_info(Node), - rt_config:update_app_config(Node, [{riak_api, [{https, [{IP, + rt:update_app_config(Node, [{riak_api, [{https, [{IP, Port+1000}]}]}]), rt:wait_until_pingable(Node), rt:wait_for_service(Node, riak_kv). @@ -542,9 +542,9 @@ crdt_tests([Node|_]=Nodes, RHC) -> Types = [{<<"counters">>, counter, riakc_counter:to_op(riakc_counter:increment(5, riakc_counter:new()))}, {<<"sets">>, set, riakc_set:to_op(riakc_set:add_element(<<"foo">>, riakc_set:new()))}], [ begin - rt_bucket_types:create_and_activate_bucket_type(Node, BType, [{allow_mult, true}, {datatype, DType}]), - rt_bucket_types:wait_until_bucket_type_status(BType, active, Nodes), - rt_bucket_types:wait_until_bucket_type_visible(Nodes, BType) + rt:create_and_activate_bucket_type(Node, BType, [{allow_mult, true}, {datatype, DType}]), + rt:wait_until_bucket_type_status(BType, active, Nodes), + rt:wait_until_bucket_type_visible(Nodes, BType) end || {BType, DType, _Op} <- Types ], lager:info("Checking that CRDT fetch is denied"), diff --git a/tests/jmx_verify.erl b/tests/jmx_verify.erl index b4255bf8e..731295623 100644 --- a/tests/jmx_verify.erl +++ b/tests/jmx_verify.erl @@ -234,4 +234,4 @@ jmx_dump(Cmd) -> lager:info("Dumping JMX stats using command ~s", [Cmd]), Output = string:strip(os:cmd(Cmd), both, $\n), JSONOutput = mochijson2:decode(Output), - [ {Key, Value} || {struct, [{Key, Value}]} <- JSONOutput]. \ No newline at end of file + [ {Key, Value} || {struct, [{Key, Value}]} <- JSONOutput]. diff --git a/tests/loaded_upgrade.erl b/tests/loaded_upgrade.erl index 948bddb8b..9dd09fbdc 100644 --- a/tests/loaded_upgrade.erl +++ b/tests/loaded_upgrade.erl @@ -234,4 +234,4 @@ node_monitor_loop(Node, Sup, TestProc) -> Other -> lager:warn("Node monitor for ~p got unknown message ~p", [Node, Other]), node_monitor_loop(Node, Sup, TestProc) - end. \ No newline at end of file + end. diff --git a/tests/loaded_upgrade_worker_sup.erl b/tests/loaded_upgrade_worker_sup.erl index c1d1c5c26..3e62c440c 100644 --- a/tests/loaded_upgrade_worker_sup.erl +++ b/tests/loaded_upgrade_worker_sup.erl @@ -237,12 +237,12 @@ assert_equal(Expected, Actual) -> Actual == Expected. pb_pid_recycler(undefined, Node) -> - rt_pb:pbc(Node); + rt:pbc(Node); pb_pid_recycler(Pid, Node) -> case riakc_pb_socket:is_connected(Pid) of true -> Pid; _ -> riakc_pb_socket:stop(Pid), - rt_pb:pbc(Node) + rt:pbc(Node) end. diff --git a/tests/mapred_basic_compat.erl b/tests/mapred_basic_compat.erl index 35d6cba56..0aa0f202f 100644 --- a/tests/mapred_basic_compat.erl +++ b/tests/mapred_basic_compat.erl @@ -41,13 +41,13 @@ -define(BUCKET_TYPE, <<"mytype">>). confirm() -> - Nodes = rt_cluster:build_cluster(3), + Nodes = rt:build_cluster(3), [Node1|_] = Nodes, %% create a new type - rt_bucket_types:create_and_activate_bucket_type(Node1, ?BUCKET_TYPE, [{n_val, 3}]), - rt_bucket_types:wait_until_bucket_type_status(?BUCKET_TYPE, active, Nodes), - rt_bucket_types:wait_until_bucket_type_visible(Nodes, ?BUCKET_TYPE), + rt:create_and_activate_bucket_type(Node1, ?BUCKET_TYPE, [{n_val, 3}]), + rt:wait_until_bucket_type_status(?BUCKET_TYPE, active, Nodes), + rt:wait_until_bucket_type_visible(Nodes, ?BUCKET_TYPE), load_test_data(Nodes), rt:load_modules_on_nodes([?MODULE], Nodes), @@ -89,7 +89,7 @@ load_test_data([Node|_]) -> [{<<"link 1">>, [{?LINK_BUCKET, <<"nokey-1">>}]}, {<<"link 2">>, [{?LINK_BUCKET, <<"nokey-2">>}]}]), - C = rt_pb:pbc(Node), + C = rt:pbc(Node), ok = riakc_pb_socket:put(C, riakc_obj:update_metadata(Obj, MD)), %% Some bucket type entries {mytype,foonum}/bar{1..10} diff --git a/tests/mapred_buffer_prereduce.erl b/tests/mapred_buffer_prereduce.erl index d6b6cbac7..9356dbc34 100644 --- a/tests/mapred_buffer_prereduce.erl +++ b/tests/mapred_buffer_prereduce.erl @@ -35,7 +35,7 @@ -define(NUM_INTS, 1000). confirm() -> - Nodes = rt_cluster:build_cluster(3), + Nodes = rt:build_cluster(3), load_test_data(Nodes), diff --git a/tests/mapred_dead_pipe.erl b/tests/mapred_dead_pipe.erl index 53abb7f4d..b60728d46 100644 --- a/tests/mapred_dead_pipe.erl +++ b/tests/mapred_dead_pipe.erl @@ -44,7 +44,7 @@ }">>). confirm() -> - Nodes = rt_cluster:build_cluster(3), + Nodes = rt:build_cluster(3), %% to pick up fake_builder/1 rt:load_modules_on_nodes([?MODULE], Nodes), diff --git a/tests/mapred_http_errors.erl b/tests/mapred_http_errors.erl index aa07c2f4d..5d31f8995 100644 --- a/tests/mapred_http_errors.erl +++ b/tests/mapred_http_errors.erl @@ -36,7 +36,7 @@ map_never_notfound(Object, _, _) when Object /= {error, notfound} -> [ok]. confirm() -> - Nodes = rt_cluster:build_cluster(1), + Nodes = rt:build_cluster(1), rt:load_modules_on_nodes([?MODULE], Nodes), diff --git a/tests/mapred_javascript.erl b/tests/mapred_javascript.erl index 301c494f0..5f8d8d502 100644 --- a/tests/mapred_javascript.erl +++ b/tests/mapred_javascript.erl @@ -43,7 +43,7 @@ }">>). confirm() -> - Nodes = rt_cluster:build_cluster(3), + Nodes = rt:build_cluster(3), load_test_data(Nodes), @@ -67,7 +67,7 @@ load_test_data([Node|_]) -> Map = riakc_obj:new(?JS_BUCKET, <<"map">>, ?MAP_JS, "text/plain"), Red = riakc_obj:new(?JS_BUCKET, <<"reduce">>, ?REDUCE_JS, "text/plain"), - C = rt_pb:pbc(Node), + C = rt:pbc(Node), ok = riakc_pb_socket:put(C, Map), ok = riakc_pb_socket:put(C, Red), riakc_pb_socket:stop(C). diff --git a/tests/mapred_notfound_failover.erl b/tests/mapred_notfound_failover.erl index b0e8fad9f..8c8e2e424 100644 --- a/tests/mapred_notfound_failover.erl +++ b/tests/mapred_notfound_failover.erl @@ -39,9 +39,9 @@ confirm() -> %% we need the volatility of memory, so we can cause a replica %% notfound by killing a vnode - rt_backend:set_backend(memory), + rt:set_backend(memory), - Nodes = rt_cluster:build_cluster(3), + Nodes = rt:build_cluster(3), %% for our custom reduce phase rt:load_modules_on_nodes([?MODULE], Nodes), @@ -78,7 +78,7 @@ replica_notfound(Node, {HashMod, HashFun}, MissingBucket, MissingKey, MissingValue) -> %% create a value for the "missing" key Obj = riakc_obj:new(MissingBucket, MissingKey, MissingValue), - C = rt_pb:pbc(Node), + C = rt:pbc(Node), ok = riakc_pb_socket:put(C, Obj, [{w, 3}]), riakc_pb_socket:stop(C), %% and now kill the first replica; this will make the vnode local diff --git a/tests/mapred_search_switch.erl b/tests/mapred_search_switch.erl index f0c6a89d0..85434e20f 100644 --- a/tests/mapred_search_switch.erl +++ b/tests/mapred_search_switch.erl @@ -67,9 +67,9 @@ setup_test_env() -> %% must enable both RS and YZ at startup to get test data indexed; %% nothing extra would be tested by using multiple nodes, so just %% deploy one to make the test run faster - Nodes = rt_cluster:deploy_nodes(1, [{riak_search, [{enabled, true}]}, + Nodes = rt:deploy_nodes(1, [{riak_search, [{enabled, true}]}, {yokozuna, [{enabled, true}]}]), - ok = rt_node:wait_until_nodes_ready(Nodes), + ok = rt:wait_until_nodes_ready(Nodes), ok = rt:wait_for_cluster_service(Nodes, riak_search), ok = rt:wait_for_cluster_service(Nodes, yokozuna), @@ -162,7 +162,7 @@ generate_test_data(System) -> %% setup riak_search hook setup_rs_bucket([Node|_], Bucket) -> lager:info("Setting up riak_search hook"), - C = rt_http:httpc(Node), + C = rt:httpc(Node), ok = rhc:set_bucket(C, Bucket, [{search, true}]). %% setup yokozuna hook/index - bucket name == index name @@ -205,7 +205,7 @@ iburl(Node, Path) -> %% value, and each of which has a unique term in its value load_test_data([Node|_], Bucket, KeyAndUniques, Common) -> lager:info("Loading test data"), - C = rt_http:httpc(Node), + C = rt:httpc(Node), [ begin Value = list_to_binary([Common, " ", Unique]), ok = rhc:put(C, riakc_obj:new(Bucket, Key, Value, "text/plain")) @@ -282,7 +282,7 @@ got_error(_) -> false. run_bucket_mr([Node|_], Bucket, Common) -> - C = rt_pb:pbc(Node), + C = rt:pbc(Node), riakc_pb_socket:mapred( C, %% TODO: check {search, Bucket, Common, Filter} diff --git a/tests/mapred_verify_rt.erl b/tests/mapred_verify_rt.erl index b7f4e9b98..c09a63e5b 100644 --- a/tests/mapred_verify_rt.erl +++ b/tests/mapred_verify_rt.erl @@ -29,7 +29,7 @@ confirm() -> lager:info("Build ~b node cluster", [?NODE_COUNT]), - Nodes = rt_cluster:build_cluster(?NODE_COUNT), + Nodes = rt:build_cluster(?NODE_COUNT), %% @todo longer term fix is probably one or more of: diff --git a/tests/partition_repair.erl b/tests/partition_repair.erl index d295c1a15..62ba5806c 100644 --- a/tests/partition_repair.erl +++ b/tests/partition_repair.erl @@ -334,4 +334,4 @@ set_search_schema_nval(Bucket, NVal) -> ?assertCmd(GetSchema), ?assertCmd(ModifyNVal), ?assertCmd(SetSchema), - ?assertCmd(ClearCache). \ No newline at end of file + ?assertCmd(ClearCache). diff --git a/tests/pb_cipher_suites.erl b/tests/pb_cipher_suites.erl index afbe93f8f..5618c89e2 100644 --- a/tests/pb_cipher_suites.erl +++ b/tests/pb_cipher_suites.erl @@ -49,7 +49,7 @@ confirm() -> ]} ], - Nodes = rt_cluster:build_cluster(4, Conf), + Nodes = rt:build_cluster(4, Conf), Node = hd(Nodes), %% enable security on the cluster ok = rpc:call(Node, riak_core_console, security_enable, [[]]), diff --git a/tests/pipe_verify_basics.erl b/tests/pipe_verify_basics.erl index 27b20cf2c..878c2cede 100644 --- a/tests/pipe_verify_basics.erl +++ b/tests/pipe_verify_basics.erl @@ -40,7 +40,9 @@ confirm() -> lager:info("Build ~b node cluster", [?NODE_COUNT]), - Nodes = rt_cluster:build_cluster(?NODE_COUNT), + Nodes = rt:build_cluster(?NODE_COUNT), + + [rt:wait_for_service(Node, riak_pipe) || Node <- Nodes], [rt:wait_for_service(Node, riak_pipe) || Node <- Nodes], diff --git a/tests/pipe_verify_exceptions.erl b/tests/pipe_verify_exceptions.erl index 9dcd21e19..accf5565e 100644 --- a/tests/pipe_verify_exceptions.erl +++ b/tests/pipe_verify_exceptions.erl @@ -45,7 +45,9 @@ %% @doc riak_test callback confirm() -> lager:info("Build ~b node cluster", [?NODE_COUNT]), - Nodes = rt_cluster:build_cluster(?NODE_COUNT), + Nodes = rt:build_cluster(?NODE_COUNT), + + [rt:wait_for_service(Node, riak_pipe) || Node <- Nodes], [rt:wait_for_service(Node, riak_pipe) || Node <- Nodes], diff --git a/tests/pipe_verify_handoff.erl b/tests/pipe_verify_handoff.erl index 61cd440d1..ac578e5a8 100644 --- a/tests/pipe_verify_handoff.erl +++ b/tests/pipe_verify_handoff.erl @@ -62,7 +62,7 @@ confirm() -> lager:info("Start ~b nodes", [?NODE_COUNT]), NodeDefs = lists:duplicate(?NODE_COUNT, {current, default}), Services = [riak_pipe], - [Primary,Secondary] = Nodes = rt_cluster:deploy_nodes(NodeDefs, Services), + [Primary,Secondary] = Nodes = rt:deploy_nodes(NodeDefs, Services), %% Ensure each node owns 100% of it's own ring [?assertEqual([Node], rt:owners_according_to(Node)) || Node <- Nodes], @@ -107,9 +107,9 @@ confirm() -> lager:info("Join Secondary to Primary"), %% Give slave a chance to start and master to notice it. - rt_node:join(Secondary, Primary), + rt:join(Secondary, Primary), rt:wait_until_no_pending_changes(Nodes), - rt_node:wait_until_nodes_agree_about_ownership(Nodes), + rt:wait_until_nodes_agree_about_ownership(Nodes), lager:info("Unpause workers"), Runner ! go, diff --git a/tests/pipe_verify_handoff_blocking.erl b/tests/pipe_verify_handoff_blocking.erl index 139eede1a..4c48bdabc 100644 --- a/tests/pipe_verify_handoff_blocking.erl +++ b/tests/pipe_verify_handoff_blocking.erl @@ -82,7 +82,7 @@ confirm() -> lager:info("Start ~b nodes", [?NODE_COUNT]), NodeDefs = lists:duplicate(?NODE_COUNT, {current, default}), Services = [riak_pipe], - [Primary,Secondary] = Nodes = rt_cluster:deploy_nodes(NodeDefs, Services), + [Primary,Secondary] = Nodes = rt:deploy_nodes(NodeDefs, Services), %% Ensure each node owns 100% of it's own ring [?assertEqual([Node], rt:owners_according_to(Node)) || Node <- Nodes], @@ -108,9 +108,9 @@ confirm() -> lager:info("Join Secondary to Primary"), %% Give slave a chance to start and master to notice it. - rt_node:join(Secondary, Primary), + rt:join(Secondary, Primary), rt:wait_until_no_pending_changes(Nodes), - rt_node:wait_until_nodes_agree_about_ownership(Nodes), + rt:wait_until_nodes_agree_about_ownership(Nodes), lager:info("Unpause workers"), Runner ! go, diff --git a/tests/pipe_verify_restart_input_forwarding.erl b/tests/pipe_verify_restart_input_forwarding.erl index 392495a1c..945c680a0 100644 --- a/tests/pipe_verify_restart_input_forwarding.erl +++ b/tests/pipe_verify_restart_input_forwarding.erl @@ -52,7 +52,9 @@ %% @doc riak_test callback confirm() -> lager:info("Build ~b node cluster", [?NODE_COUNT]), - Nodes = rt_cluster:build_cluster(?NODE_COUNT), + Nodes = rt:build_cluster(?NODE_COUNT), + + [rt:wait_for_service(Node, riak_pipe) || Node <- Nodes], [rt:wait_for_service(Node, riak_pipe) || Node <- Nodes], diff --git a/tests/pipe_verify_sink_types.erl b/tests/pipe_verify_sink_types.erl index d3ebd7916..ec883d2aa 100644 --- a/tests/pipe_verify_sink_types.erl +++ b/tests/pipe_verify_sink_types.erl @@ -39,7 +39,9 @@ %% @doc riak_test callback confirm() -> lager:info("Build ~b node cluster", [?NODE_COUNT]), - Nodes = rt_cluster:build_cluster(?NODE_COUNT), + Nodes = rt:build_cluster(?NODE_COUNT), + + [rt:wait_for_service(Node, riak_pipe) || Node <- Nodes], [rt:wait_for_service(Node, riak_pipe) || Node <- Nodes], diff --git a/tests/post_generate_key.erl b/tests/post_generate_key.erl index de77942b9..b8197caa9 100644 --- a/tests/post_generate_key.erl +++ b/tests/post_generate_key.erl @@ -25,7 +25,7 @@ -include_lib("eunit/include/eunit.hrl"). confirm() -> - Nodes = rt_cluster:build_cluster(1), + Nodes = rt:build_cluster(1), ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), [Base|_] = rt:http_url(Nodes), diff --git a/tests/pr_pw.erl b/tests/pr_pw.erl index 5e1e5c802..15b5a59a7 100644 --- a/tests/pr_pw.erl +++ b/tests/pr_pw.erl @@ -8,7 +8,7 @@ confirm() -> application:start(inets), lager:info("Deploy some nodes"), - Nodes = rt_cluster:build_cluster(4), + Nodes = rt:build_cluster(4), %% calculate the preflist for foo/bar {ok, Ring} = rpc:call(hd(Nodes), riak_core_ring_manager, get_my_ring, []), @@ -68,7 +68,7 @@ confirm() -> ?assertMatch({ok, _}, C:get(<<"foo">>, <<"bar">>, [{pr, quorum}])), - rt_node:stop_and_wait(Node), + rt:stop_and_wait(Node), %% there's now a fallback in the preflist, so PR/PW won't be satisfied %% anymore @@ -98,7 +98,7 @@ confirm() -> ?assertEqual({error, timeout}, C:put(Obj, [{pw, quorum}])), %% restart the node - rt_node:start_and_wait(Node), + rt:start_and_wait(Node), rt:wait_for_service(Node, riak_kv), %% we can make quorum again @@ -110,8 +110,8 @@ confirm() -> ?assertEqual({error, timeout}, C:put(Obj, [{pw, all}])), %% reboot the node - rt_node:stop_and_wait(Node2), - rt_node:start_and_wait(Node2), + rt:stop_and_wait(Node2), + rt:start_and_wait(Node2), rt:wait_for_service(Node2, riak_kv), %% everything is happy again diff --git a/tests/replication/repl_aae_fullsync.erl b/tests/replication/repl_aae_fullsync.erl index f4623aad9..a04e86012 100644 --- a/tests/replication/repl_aae_fullsync.erl +++ b/tests/replication/repl_aae_fullsync.erl @@ -90,8 +90,8 @@ simple_test() -> read_from_cluster(BFirst, 1, ?NUM_KEYS, ?NUM_KEYS), %% Wait for trees to compute. - rt_aae:wait_until_aae_trees_built(ANodes), - rt_aae:wait_until_aae_trees_built(BNodes), + rt:wait_until_aae_trees_built(ANodes), + rt:wait_until_aae_trees_built(BNodes), lager:info("Test fullsync from cluster A leader ~p to cluster B", [LeaderA]), @@ -113,7 +113,7 @@ simple_test() -> %% intercepts are removed. validate_completed_fullsync(LeaderA, BFirst, "B", 1, ?NUM_KEYS), - rt_cluster:clean_cluster(Nodes), + rt:clean_cluster(Nodes), pass. @@ -185,9 +185,9 @@ dual_test() -> rt:wait_until_ring_converged(ANodes), %% Wait for trees to compute. - rt_aae:wait_until_aae_trees_built(ANodes), - rt_aae:wait_until_aae_trees_built(BNodes), - rt_aae:wait_until_aae_trees_built(CNodes), + rt:wait_until_aae_trees_built(ANodes), + rt:wait_until_aae_trees_built(BNodes), + rt:wait_until_aae_trees_built(CNodes), %% Flush AAE trees to disk. perform_sacrifice(AFirst), @@ -277,7 +277,7 @@ bidirectional_test() -> perform_sacrifice(AFirst), %% Wait for trees to compute. - rt_aae:wait_until_aae_trees_built(ANodes), + rt:wait_until_aae_trees_built(ANodes), %% Verify A replicated to B. validate_completed_fullsync(LeaderA, BFirst, "B", 1, ?NUM_KEYS), @@ -290,13 +290,13 @@ bidirectional_test() -> perform_sacrifice(BFirst), %% Wait for trees to compute. - rt_aae:wait_until_aae_trees_built(BNodes), + rt:wait_until_aae_trees_built(BNodes), %% Verify B replicated to A. validate_completed_fullsync(LeaderB, AFirst, "A", ?NUM_KEYS + 1, ?NUM_KEYS + ?NUM_KEYS), %% Clean. - rt_cluster:clean_cluster(Nodes), + rt:clean_cluster(Nodes), pass. @@ -339,8 +339,8 @@ difference_test() -> connect_cluster(LeaderA, BPort, "B"), %% Get PBC connections. - APBC = rt_pb:pbc(LeaderA), - BPBC = rt_pb:pbc(LeaderB), + APBC = rt:pbc(LeaderA), + BPBC = rt:pbc(LeaderB), %% Write key. ok = riakc_pb_socket:put(APBC, @@ -349,8 +349,8 @@ difference_test() -> [{timeout, 4000}]), %% Wait for trees to compute. - rt_aae:wait_until_aae_trees_built(ANodes), - rt_aae:wait_until_aae_trees_built(BNodes), + rt:wait_until_aae_trees_built(ANodes), + rt:wait_until_aae_trees_built(BNodes), lager:info("Test fullsync from cluster A leader ~p to cluster B", [LeaderA]), @@ -388,7 +388,7 @@ difference_test() -> [{timeout, 4000}]), ?assertEqual([<<"baz">>, <<"baz2">>], lists:sort(riakc_obj:get_values(O2))), - rt_cluster:clean_cluster(Nodes), + rt:clean_cluster(Nodes), pass. @@ -435,8 +435,8 @@ deadlock_test() -> [ok = rt_intercept:add(Target, Intercept) || Target <- ANodes], %% Wait for trees to compute. - rt_aae:wait_until_aae_trees_built(ANodes), - rt_aae:wait_until_aae_trees_built(BNodes), + rt:wait_until_aae_trees_built(ANodes), + rt:wait_until_aae_trees_built(BNodes), lager:info("Test fullsync from cluster A leader ~p to cluster B", [LeaderA]), @@ -457,7 +457,7 @@ deadlock_test() -> lager:info("Status result: ~p", [Result]), ?assertNotEqual({badrpc, timeout}, Result), - rt_cluster:clean_cluster(Nodes), + rt:clean_cluster(Nodes), pass. @@ -516,7 +516,7 @@ validate_intercepted_fullsync(InterceptTarget, NumIndicies = length(rpc:call(InterceptTarget, riak_core_ring, my_indices, - [rt_ring:get_ring(InterceptTarget)])), + [rt:get_ring(InterceptTarget)])), lager:info("~p owns ~p indices", [InterceptTarget, NumIndicies]), @@ -566,15 +566,15 @@ validate_intercepted_fullsync(InterceptTarget, NumIndicies), %% Reboot node. - rt_node:stop_and_wait(InterceptTarget), - rt_node:start_and_wait(InterceptTarget), + rt:stop_and_wait(InterceptTarget), + rt:start_and_wait(InterceptTarget), %% Wait for riak_kv and riak_repl to initialize. rt:wait_for_service(InterceptTarget, riak_kv), rt:wait_for_service(InterceptTarget, riak_repl), %% Wait until AAE trees are compueted on the rebooted node. - rt_aae:wait_until_aae_trees_built([InterceptTarget]). + rt:wait_until_aae_trees_built([InterceptTarget]). %% @doc Given a node, find the port that the cluster manager is %% listening on. @@ -607,5 +607,5 @@ write_to_cluster(Node, Start, End) -> %% of errors. read_from_cluster(Node, Start, End, Errors) -> lager:info("Reading ~p keys from node ~p.", [End - Start, Node]), - Res2 = rt_systest:read(Node, Start, End, ?TEST_BUCKET, 1), + Res2 = rt:systest_read(Node, Start, End, ?TEST_BUCKET, 1), ?assertEqual(Errors, length(Res2)). diff --git a/tests/replication/repl_aae_fullsync_util.erl b/tests/replication/repl_aae_fullsync_util.erl index 3d47ca573..33714c77e 100644 --- a/tests/replication/repl_aae_fullsync_util.erl +++ b/tests/replication/repl_aae_fullsync_util.erl @@ -71,10 +71,10 @@ prepare_cluster_data(TestBucket, NumKeysAOnly, _NumKeysBoth, [AFirst|_] = ANodes %% check that the keys we wrote initially aren't replicated yet, because %% we've disabled fullsync_on_connect lager:info("Check keys written before repl was connected are not present"), - Res2 = rt_systest:read(BFirst, 1, NumKeysAOnly, TestBucket, 1, <<>>, true), + Res2 = rt:systest_read(BFirst, 1, NumKeysAOnly, TestBucket, 1, <<>>, true), ?assertEqual(NumKeysAOnly, length(Res2)), %% wait for the AAE trees to be built so that we don't get a not_built error - rt_aae:wait_until_aae_trees_built(ANodes), - rt_aae:wait_until_aae_trees_built(BNodes), + rt:wait_until_aae_trees_built(ANodes), + rt:wait_until_aae_trees_built(BNodes), ok. diff --git a/tests/replication/repl_bucket_types.erl b/tests/replication/repl_bucket_types.erl index c14640994..b993a592d 100644 --- a/tests/replication/repl_bucket_types.erl +++ b/tests/replication/repl_bucket_types.erl @@ -465,4 +465,4 @@ ensure_rtq_drained(ANodes) -> [] =:= rpc:call(Node, riak_repl2_rtq, dumpq, []) end, ANodes), Expected = [true || _ <- lists:seq(1, length(ANodes))], - ?assertEqual(Expected, Got). \ No newline at end of file + ?assertEqual(Expected, Got). diff --git a/tests/replication/repl_cancel_fullsync.erl b/tests/replication/repl_cancel_fullsync.erl index 2763c2469..03c69b9ef 100644 --- a/tests/replication/repl_cancel_fullsync.erl +++ b/tests/replication/repl_cancel_fullsync.erl @@ -33,9 +33,9 @@ %% @doc Ensure we can cancel a fullsync and restart it. confirm() -> - rt_config:set_advanced_conf(all, ?CONF(5)), + rt:set_advanced_conf(all, ?CONF(5)), - Nodes = [ANodes, BNodes] = rt_cluster:build_clusters([3, 3]), + Nodes = [ANodes, BNodes] = rt:build_clusters([3, 3]), rt:wait_for_cluster_service(ANodes, riak_repl), rt:wait_for_cluster_service(BNodes, riak_repl), @@ -141,7 +141,7 @@ confirm() -> lager:info("Fullsync Complete"), rt:log_to_nodes(Nodes, "Test completed."), - rt_cluster:clean_cluster(ANodes), - rt_cluster:clean_cluster(BNodes), + rt:clean_cluster(ANodes), + rt:clean_cluster(BNodes), pass. diff --git a/tests/replication/repl_consistent_object_filter.erl b/tests/replication/repl_consistent_object_filter.erl index 213b459c8..a75b534cc 100644 --- a/tests/replication/repl_consistent_object_filter.erl +++ b/tests/replication/repl_consistent_object_filter.erl @@ -27,18 +27,18 @@ confirm() -> BucketType = <<"consistent_type">>, %% Create consistent bucket type on cluster A - rt_bucket_types:create_and_activate_bucket_type(LeaderA, + rt:create_and_activate_bucket_type(LeaderA, BucketType, [{consistent, true}, {n_val, 5}]), - rt_bucket_types:wait_until_bucket_type_status(BucketType, active, ANodes), - rt_bucket_types:wait_until_bucket_type_visible(ANodes, BucketType), + rt:wait_until_bucket_type_status(BucketType, active, ANodes), + rt:wait_until_bucket_type_visible(ANodes, BucketType), %% Create consistent bucket type on cluster B - rt_bucket_types:create_and_activate_bucket_type(LeaderB, + rt:create_and_activate_bucket_type(LeaderB, BucketType, [{consistent, true}, {n_val, 5}]), - rt_bucket_types:wait_until_bucket_type_status(BucketType, active, BNodes), - rt_bucket_types:wait_until_bucket_type_visible(BNodes, BucketType), + rt:wait_until_bucket_type_status(BucketType, active, BNodes), + rt:wait_until_bucket_type_visible(BNodes, BucketType), connect_clusters(LeaderA, LeaderB), @@ -103,7 +103,7 @@ make_clusters() -> ]} ], - Nodes = rt_cluster:deploy_nodes(NumNodes, Conf), + Nodes = rt:deploy_nodes(NumNodes, Conf, [riak_kv, riak_repl]), {ANodes, BNodes} = lists:split(ClusterASize, Nodes), lager:info("ANodes: ~p", [ANodes]), lager:info("BNodes: ~p", [BNodes]), diff --git a/tests/replication/repl_fs_bench.erl b/tests/replication/repl_fs_bench.erl index c2b0dfc5d..657a25aa6 100644 --- a/tests/replication/repl_fs_bench.erl +++ b/tests/replication/repl_fs_bench.erl @@ -60,9 +60,9 @@ confirm() -> %% @doc Perform a fullsync, with given latency injected via intercept %% and return times for each fullsync time. fullsync_test(Strategy, Latency) -> - rt_config:set_advanced_conf(all, ?CONF(Strategy)), + rt:set_advanced_conf(all, ?CONF(Strategy)), - [ANodes, BNodes] = rt_cluster:build_clusters([3, 3]), + [ANodes, BNodes] = rt:build_clusters([3, 3]), rt:wait_for_cluster_service(ANodes, riak_repl), rt:wait_for_cluster_service(BNodes, riak_repl), @@ -125,32 +125,32 @@ fullsync_test(Strategy, Latency) -> ?assertEqual(ok, repl_util:wait_for_connection(LeaderA, "B")), %% Perform fullsync of an empty cluster. - rt_aae:wait_until_aae_trees_built(ANodes ++ BNodes), + rt:wait_until_aae_trees_built(ANodes ++ BNodes), {EmptyTime, _} = timer:tc(repl_util, start_and_wait_until_fullsync_complete, [LeaderA]), %% Write keys and perform fullsync. repl_util:write_to_cluster(AFirst, 0, ?FULL_NUM_KEYS, ?TEST_BUCKET), - rt_aae:wait_until_aae_trees_built(ANodes ++ BNodes), + rt:wait_until_aae_trees_built(ANodes ++ BNodes), {FullTime, _} = timer:tc(repl_util, start_and_wait_until_fullsync_complete, [LeaderA]), %% Rewrite first 10% keys and perform fullsync. repl_util:write_to_cluster(AFirst, 0, ?DIFF_NUM_KEYS, ?TEST_BUCKET), - rt_aae:wait_until_aae_trees_built(ANodes ++ BNodes), + rt:wait_until_aae_trees_built(ANodes ++ BNodes), {DiffTime, _} = timer:tc(repl_util, start_and_wait_until_fullsync_complete, [LeaderA]), %% Write no keys, and perform the fullsync. - rt_aae:wait_until_aae_trees_built(ANodes ++ BNodes), + rt:wait_until_aae_trees_built(ANodes ++ BNodes), {NoneTime, _} = timer:tc(repl_util, start_and_wait_until_fullsync_complete, [LeaderA]), - rt_cluster:clean_cluster(ANodes), - rt_cluster:clean_cluster(BNodes), + rt:clean_cluster(ANodes), + rt:clean_cluster(BNodes), {EmptyTime, FullTime, DiffTime, NoneTime}. diff --git a/tests/replication/repl_fs_stat_caching.erl b/tests/replication/repl_fs_stat_caching.erl index 117fafb28..6f187ef49 100644 --- a/tests/replication/repl_fs_stat_caching.erl +++ b/tests/replication/repl_fs_stat_caching.erl @@ -106,4 +106,4 @@ maybe_suspend_an_fs_source(Node, [{_Remote, Pid} | Tail]) -> maybe_suspend_an_fs_source(Node, Tail); true -> Pid - end. \ No newline at end of file + end. diff --git a/tests/replication/repl_location_failures.erl b/tests/replication/repl_location_failures.erl index 9793c0fe6..cb74985f5 100644 --- a/tests/replication/repl_location_failures.erl +++ b/tests/replication/repl_location_failures.erl @@ -34,9 +34,9 @@ ]). confirm() -> - rt_config:set_advanced_conf(all, ?CONF(5)), + rt:set_advanced_conf(all, ?CONF(5)), - [ANodes, BNodes] = rt_cluster:build_clusters([3, 3]), + [ANodes, BNodes] = rt:build_clusters([3, 3]), rt:wait_for_cluster_service(ANodes, riak_repl), rt:wait_for_cluster_service(BNodes, riak_repl), @@ -84,7 +84,7 @@ confirm() -> BIndicies = length(rpc:call(LeaderB, riak_core_ring, my_indices, - [rt_ring:get_ring(LeaderB)])), + [rt:get_ring(LeaderB)])), lager:warning("BIndicies: ~p", [BIndicies]), @@ -102,7 +102,7 @@ confirm() -> repl_util:validate_completed_fullsync(LeaderA, BFirst, "B", 1, ?NUM_KEYS, ?TEST_BUCKET), - rt_cluster:clean_cluster(ANodes), - rt_cluster:clean_cluster(BNodes), + rt:clean_cluster(ANodes), + rt:clean_cluster(BNodes), pass. diff --git a/tests/replication/repl_rt_cascading_rtq.erl b/tests/replication/repl_rt_cascading_rtq.erl index 94b485826..8d31446f6 100644 --- a/tests/replication/repl_rt_cascading_rtq.erl +++ b/tests/replication/repl_rt_cascading_rtq.erl @@ -182,4 +182,4 @@ write_to_cluster(Node, Start, End) -> read_from_cluster(Node, Start, End, Errors) -> lager:info("Reading ~p keys from node ~p.", [End - Start, Node]), Res2 = rt:systest_read(Node, Start, End, ?TEST_BUCKET, 1), - ?assertEqual(Errors, length(Res2)). \ No newline at end of file + ?assertEqual(Errors, length(Res2)). diff --git a/tests/replication/repl_rt_heartbeat.erl b/tests/replication/repl_rt_heartbeat.erl index 8cf4a3614..7306404a1 100644 --- a/tests/replication/repl_rt_heartbeat.erl +++ b/tests/replication/repl_rt_heartbeat.erl @@ -174,7 +174,7 @@ make_connected_clusters() -> ]} ], - Nodes = rt_cluster:deploy_nodes(NumNodes, Conf), + Nodes = rt:deploy_nodes(NumNodes, Conf, [riak_kv, riak_repl]), {ANodes, BNodes} = lists:split(ClusterASize, Nodes), lager:info("ANodes: ~p", [ANodes]), diff --git a/tests/replication/repl_rt_overload.erl b/tests/replication/repl_rt_overload.erl index 6b13cbd8f..1440d5b1e 100644 --- a/tests/replication/repl_rt_overload.erl +++ b/tests/replication/repl_rt_overload.erl @@ -89,7 +89,7 @@ verify_overload_writes(LeaderA, LeaderB) -> ?assertEqual([], repl_util:do_write(LeaderA, First, Last, TestBucket, 2)), lager:info("Reading ~p keys from ~p", [Last-First+1, LeaderB]), - NumReads = rt_systest:read(LeaderB, First, Last, TestBucket, 2), + NumReads = rt:systest_read(LeaderB, First, Last, TestBucket, 2), lager:info("systest_read saw ~p errors", [length(NumReads)]), @@ -128,7 +128,7 @@ make_connected_clusters() -> ]} ], - Nodes = rt_cluster:deploy_nodes(NumNodes, Conf), + Nodes = rt:deploy_nodes(NumNodes, Conf, [riak_kv, riak_repl]), {ANodes, BNodes} = lists:split(ClusterASize, Nodes), lager:info("ANodes: ~p", [ANodes]), lager:info("BNodes: ~p", [BNodes]), diff --git a/tests/replication/repl_rt_pending.erl b/tests/replication/repl_rt_pending.erl index e393e979a..24754389a 100644 --- a/tests/replication/repl_rt_pending.erl +++ b/tests/replication/repl_rt_pending.erl @@ -174,7 +174,7 @@ make_connected_clusters() -> ]} ], - Nodes = rt_cluster:deploy_nodes(NumNodes, Conf), + Nodes = rt:deploy_nodes(NumNodes, Conf, [riak_kv, riak_repl]), {ANodes, BNodes} = lists:split(ClusterASize, Nodes), lager:info("ANodes: ~p", [ANodes]), diff --git a/tests/replication/repl_util.erl b/tests/replication/repl_util.erl index b792a0a54..a7be23f35 100644 --- a/tests/replication/repl_util.erl +++ b/tests/replication/repl_util.erl @@ -634,4 +634,4 @@ validate_intercepted_fullsync(InterceptTarget, rt:wait_for_service(InterceptTarget, riak_repl), %% Wait until AAE trees are compueted on the rebooted node. - rt:wait_until_aae_trees_built([InterceptTarget]). \ No newline at end of file + rt:wait_until_aae_trees_built([InterceptTarget]). diff --git a/tests/replication/replication2.erl b/tests/replication/replication2.erl index 9f2811925..cc9b07e70 100644 --- a/tests/replication/replication2.erl +++ b/tests/replication/replication2.erl @@ -376,7 +376,7 @@ network_partition_test([AFirst|_] = ANodes, [BFirst|_] = BNodes) -> %% Verify that the original real time keys did not replicate %% Verify that the No replication bucket didn't replicate. bucket_sync_test([AFirst|_] = ANodes, [BFirst|_] = BNodes) -> - + TestHash = list_to_binary([io_lib:format("~2.16.0b", [X]) || <> <= erlang:md5(term_to_binary(os:timestamp()))]), FullsyncOnly = <>, RealtimeOnly = <>, @@ -767,4 +767,4 @@ collect_results(Workers, Acc) -> collect_results(lists:keydelete(Pid, 1, Workers), Res ++ Acc); {'DOWN', _, _, Pid, _Reason} -> collect_results(lists:keydelete(Pid, 1, Workers), Acc) - end. \ No newline at end of file + end. diff --git a/tests/replication/replication2_connections.erl b/tests/replication/replication2_connections.erl index 760375b30..e9419d435 100644 --- a/tests/replication/replication2_connections.erl +++ b/tests/replication/replication2_connections.erl @@ -45,9 +45,9 @@ simple_test() -> {rt_heartbeat_timeout, ?HB_TIMEOUT} ]}], - rt_config:set_advanced_conf(all, Conf), + rt:set_advanced_conf(all, Conf), - [ANodes, BNodes] = rt_cluster:build_clusters([3, 3]), + [ANodes, BNodes] = rt:build_clusters([3, 3]), rt:wait_for_cluster_service(ANodes, riak_repl), rt:wait_for_cluster_service(BNodes, riak_repl), @@ -114,9 +114,9 @@ disconnect_test() -> {rt_heartbeat_timeout, ?HB_TIMEOUT} ]}], - rt_config:set_advanced_conf(all, Conf), + rt:set_advanced_conf(all, Conf), - [ANodes, BNodes] = rt_cluster:build_clusters([3, 3]), + [ANodes, BNodes] = rt:build_clusters([3, 3]), lager:info("ANodes: ~p", [ANodes]), lager:info("BNodes: ~p", [BNodes]), @@ -159,8 +159,8 @@ disconnect_test() -> lager:info("Verifying disconnect from B to A."), [verify_disconnect(Node, "A") || Node <- BNodes], - rt_cluster:clean_cluster(ANodes), - rt_cluster:clean_cluster(BNodes), + rt:clean_cluster(ANodes), + rt:clean_cluster(BNodes), pass. @@ -182,9 +182,9 @@ error_cleanup_test() -> {cm_cancellation_interval, 5 * 1000} ]}], - rt_config:set_advanced_conf(all, Conf), + rt:set_advanced_conf(all, Conf), - [ANodes, BNodes] = rt_cluster:build_clusters([3, 3]), + [ANodes, BNodes] = rt:build_clusters([3, 3]), lager:info("ANodes: ~p", [ANodes]), lager:info("BNodes: ~p", [BNodes]), @@ -236,8 +236,8 @@ error_cleanup_test() -> lager:info("Verifying connection from A to B"), [verify_connectivity(Node, "B") || Node <- ANodes], - rt_cluster:clean_cluster(ANodes), - rt_cluster:clean_cluster(BNodes), + rt:clean_cluster(ANodes), + rt:clean_cluster(BNodes), pass. diff --git a/tests/replication/replication2_console_tests.erl b/tests/replication/replication2_console_tests.erl index 258f88bb4..068aa7cd0 100644 --- a/tests/replication/replication2_console_tests.erl +++ b/tests/replication/replication2_console_tests.erl @@ -120,3 +120,4 @@ check_cmd(Node, Cmd) -> lager:info("Testing riak-repl ~s on ~s", [Cmd, Node]), {ok, Out} = rt:riak_repl(Node, [Cmd]), ?assertEqual("pass", Out). + diff --git a/tests/replication/replication2_fsschedule.erl b/tests/replication/replication2_fsschedule.erl index e49d55b72..9f455aaec 100644 --- a/tests/replication/replication2_fsschedule.erl +++ b/tests/replication/replication2_fsschedule.erl @@ -94,7 +94,7 @@ test_multiple_schedules() -> lager:info("Waiting for fullsyncs"), wait_until_fullsyncs(LeaderA, "B", 5), wait_until_fullsyncs(LeaderA, "C", 5), - rt_cluster:clean_cluster(AllNodes), + rt:clean_cluster(AllNodes), pass. test_single_schedule() -> @@ -113,7 +113,7 @@ test_single_schedule() -> lager:info("Waiting for fullsyncs"), wait_until_fullsyncs(LeaderA, "B", 10), wait_until_fullsyncs(LeaderA, "C", 10), - rt_cluster:clean_cluster(AllNodes), + rt:clean_cluster(AllNodes), pass. test_mixed_12_13() -> @@ -147,7 +147,7 @@ test_mixed_12_13() -> wait_until_fullsyncs(LeaderA, "B", 3), wait_until_fullsyncs(LeaderA, "C", 3), wait_until_12_fs_complete(LeaderA, 9), - rt_cluster:clean_cluster(AllNodes), + rt:clean_cluster(AllNodes), pass. diff --git a/tests/replication/replication2_pg.erl b/tests/replication/replication2_pg.erl index fa9175320..6c9345875 100644 --- a/tests/replication/replication2_pg.erl +++ b/tests/replication/replication2_pg.erl @@ -1006,4 +1006,4 @@ verify_topology_change(SourceNodes, SinkNodes) -> wait_until_pg(SinkNode1, SinkNode1Pid, Bucket, KeyPost, SourceCid), %% We're good! - pass. \ No newline at end of file + pass. diff --git a/tests/replication/replication2_rt_sink_connection.erl b/tests/replication/replication2_rt_sink_connection.erl index 03b024e0a..4c38a7d82 100644 --- a/tests/replication/replication2_rt_sink_connection.erl +++ b/tests/replication/replication2_rt_sink_connection.erl @@ -43,7 +43,7 @@ confirm() -> ]} ], - Nodes = rt_cluster:deploy_nodes(NumNodes, Conf), + Nodes = rt:deploy_nodes(NumNodes, Conf, [riak_kv, riak_repl]), {ANodes, Rest} = lists:split(2, Nodes), {BNodes, CNodes} = lists:split(2, Rest), @@ -88,7 +88,7 @@ confirm() -> enable_rt(AFirst, ANodes), lager:info("Adding 4th node to the A cluster"), - rt_node:join(CNode, AFirst), + rt:join(CNode, AFirst), [verify_connectivity(Node) || Node <- ANodes], diff --git a/tests/replication/replication2_ssl.erl b/tests/replication/replication2_ssl.erl index 89d18c4c2..c080b424e 100644 --- a/tests/replication/replication2_ssl.erl +++ b/tests/replication/replication2_ssl.erl @@ -59,7 +59,7 @@ confirm() -> %% test requires allow_mult=false - rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), + rt:set_conf(all, [{"buckets.default.allow_mult", "false"}]), NumNodes = rt_config:get(num_nodes, 6), ClusterASize = rt_config:get(cluster_a_size, (NumNodes div 2)), @@ -369,9 +369,9 @@ test_connection(Left, Right, false) -> test_connection({Node1, Config1}, {Node2, Config2}) -> repl_util:disconnect_cluster(Node1, "B"), repl_util:wait_for_disconnect(Node1, "B"), - rt_config:update_app_config(Node2, Config2), + rt:update_app_config(Node2, Config2), rt:wait_until_pingable(Node2), - rt_config:update_app_config(Node1, Config1), + rt:update_app_config(Node1, Config1), rt:wait_until_pingable(Node1), rt:wait_for_service(Node1, [riak_kv, riak_repl]), rt:wait_for_service(Node2, [riak_kv, riak_repl]), diff --git a/tests/replication/replication2_upgrade.erl b/tests/replication/replication2_upgrade.erl index 6f1ac3067..fea58b752 100644 --- a/tests/replication/replication2_upgrade.erl +++ b/tests/replication/replication2_upgrade.erl @@ -32,7 +32,7 @@ confirm() -> NodeConfig = [{FromVersion, Conf} || _ <- lists:seq(1, NumNodes)], - Nodes = rt_cluster:deploy_nodes(NodeConfig), + Nodes = rt:deploy_nodes(NodeConfig, [riak_kv, riak_repl]), NodeUpgrades = case UpgradeOrder of "forwards" -> @@ -74,7 +74,7 @@ confirm() -> ok = lists:foreach(fun(Node) -> lager:info("Upgrade node: ~p", [Node]), rt:log_to_nodes(Nodes, "Upgrade node: ~p", [Node]), - rt:upgrade(Node, current), + rtdev:upgrade(Node, current), %% The upgrade did a wait for pingable rt:wait_for_service(Node, [riak_kv, riak_pipe, riak_repl]), [rt:wait_until_ring_converged(N) || N <- [ANodes, BNodes]], diff --git a/tests/replication/replication_object_reformat.erl b/tests/replication/replication_object_reformat.erl index 6ff3ec86b..cb1cf6851 100644 --- a/tests/replication/replication_object_reformat.erl +++ b/tests/replication/replication_object_reformat.erl @@ -236,4 +236,4 @@ configure_clusters(AVersion, BVersion, Realtime) -> lager:info("Ensuring connection from cluster A to B"), repl_util:connect_cluster_by_name(LeaderA, BPort, "B"), - Nodes. \ No newline at end of file + Nodes. diff --git a/tests/replication/replication_ssl.erl b/tests/replication/replication_ssl.erl index 800c706a9..b04fa3eaa 100644 --- a/tests/replication/replication_ssl.erl +++ b/tests/replication/replication_ssl.erl @@ -6,7 +6,7 @@ confirm() -> %% test requires allow_mult=false - rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), + rt:set_conf(all, [{"buckets.default.allow_mult", "false"}]), NumNodes = rt_config:get(num_nodes, 6), ClusterASize = rt_config:get(cluster_a_size, 3), @@ -163,7 +163,7 @@ confirm() -> lager:info("===testing basic connectivity"), - [Node1, Node2] = rt_cluster:deploy_nodes(2, BaseConf), + [Node1, Node2] = rt:deploy_nodes(2, BaseConf, [riak_kv, riak_repl]), Listeners = replication:add_listeners([Node1]), replication:verify_listeners(Listeners), @@ -225,17 +225,17 @@ confirm() -> lager:info("Re-deploying 6 nodes"), - Nodes = rt_cluster:deploy_nodes(6, BaseConf), + Nodes = rt:deploy_nodes(6, BaseConf, [riak_kv, riak_repl]), [rt:wait_until_pingable(N) || N <- Nodes], {ANodes, BNodes} = lists:split(ClusterASize, Nodes), lager:info("Reconfiguring nodes with SSL options"), - [rt_config:update_app_config(N, merge_config(SSLConfig5, BaseConf)) || N <- + [rt:update_app_config(N, merge_config(SSLConfig5, BaseConf)) || N <- ANodes], - [rt_config:update_app_config(N, merge_config(SSLConfig6, BaseConf)) || N <- + [rt:update_app_config(N, merge_config(SSLConfig6, BaseConf)) || N <- BNodes], [rt:wait_until_pingable(N) || N <- Nodes], @@ -254,9 +254,9 @@ merge_config(Mixin, Base) -> lists:ukeymerge(1, lists:keysort(1, Mixin), lists:keysort(1, Base)). test_connection({Node1, Config1}, {Node2, Config2}) -> - rt_config:update_app_config(Node1, Config1), + rt:update_app_config(Node1, Config1), rt:wait_until_pingable(Node1), - rt_config:update_app_config(Node2, Config2), + rt:update_app_config(Node2, Config2), rt:wait_until_pingable(Node2), rt:wait_for_service(Node1, [riak_kv, riak_repl]), rt:wait_for_service(Node2, [riak_kv, riak_repl]), diff --git a/tests/replication/replication_stats.erl b/tests/replication/replication_stats.erl index 1adcbd87a..cce909651 100644 --- a/tests/replication/replication_stats.erl +++ b/tests/replication/replication_stats.erl @@ -39,9 +39,9 @@ confirm() -> fullsync_enabled_and_started(). fullsync_enabled_and_started() -> - rt_config:set_advanced_conf(all, ?CONF), + rt:set_advanced_conf(all, ?CONF), - [ANodes, BNodes] = rt_cluster:build_clusters([3, 3]), + [ANodes, BNodes] = rt:build_clusters([3, 3]), rt:wait_for_cluster_service(ANodes, riak_repl), rt:wait_for_cluster_service(BNodes, riak_repl), @@ -102,7 +102,7 @@ fullsync_enabled_and_started() -> fail end, - rt_cluster:clean_cluster(ANodes), - rt_cluster:clean_cluster(BNodes), + rt:clean_cluster(ANodes), + rt:clean_cluster(BNodes), Result. diff --git a/tests/replication/replication_upgrade.erl b/tests/replication/replication_upgrade.erl index e4bb26169..86d82adb9 100644 --- a/tests/replication/replication_upgrade.erl +++ b/tests/replication/replication_upgrade.erl @@ -25,7 +25,7 @@ confirm() -> NodeConfig = [{FromVersion, Conf} || _ <- lists:seq(1, NumNodes)], - Nodes = rt_cluster:deploy_nodes(NodeConfig), + Nodes = rt:deploy_nodes(NodeConfig, [riak_kv, riak_repl]), NodeUpgrades = case UpgradeOrder of "forwards" -> @@ -66,7 +66,7 @@ confirm() -> ok = lists:foreach(fun(Node) -> lager:info("Upgrade node: ~p", [Node]), rt:log_to_nodes(Nodes, "Upgrade node: ~p", [Node]), - rt:upgrade(Node, current), + rtdev:upgrade(Node, current), rt:wait_until_pingable(Node), rt:wait_for_service(Node, [riak_kv, riak_pipe, riak_repl]), [rt:wait_until_ring_converged(N) || N <- [ANodes, BNodes]], diff --git a/tests/riak_admin_console_tests.erl b/tests/riak_admin_console_tests.erl index 95ad3b421..8084b8424 100644 --- a/tests/riak_admin_console_tests.erl +++ b/tests/riak_admin_console_tests.erl @@ -154,7 +154,7 @@ riak_admin_tests(Node) -> confirm() -> %% Deploy a node to test against lager:info("Deploy node to test riak command line"), - [Node] = rt_cluster:deploy_nodes(1), + [Node] = rt:deploy_nodes(1), ?assertEqual(ok, rt:wait_until_nodes_ready([Node])), rt_intercept:add(Node, {riak_core_console, @@ -233,7 +233,7 @@ confirm() -> check_admin_cmd(Node, Cmd) -> S = string:tokens(Cmd, " "), lager:info("Testing riak-admin ~s on ~s", [Cmd, Node]), - {ok, Out} = rt_cmd_line:admin(Node, S), + {ok, Out} = rt:admin(Node, S), ?assertEqual("pass", Out). %% Recently we've started calling riak_core_console twice from the @@ -242,5 +242,5 @@ check_admin_cmd(Node, Cmd) -> check_admin_cmd_2x(Node, Cmd) -> S = string:tokens(Cmd, " "), lager:info("Testing riak-admin ~s on ~s", [Cmd, Node]), - {ok, Out} = rt_cmd_line:admin(Node, S), + {ok, Out} = rt:admin(Node, S), ?assertEqual("passpass", Out). diff --git a/tests/riak_control.erl b/tests/riak_control.erl index cfc222b52..8ad4878b7 100644 --- a/tests/riak_control.erl +++ b/tests/riak_control.erl @@ -258,4 +258,4 @@ validate_capability(VersionedNodes) -> %% Validate we are in the correct state, not the incompatible state, %% which ensure the capability has negotiated correctly. - validate_nodes(Node, Nodes, VersionedNodes, <<"valid">>). \ No newline at end of file + validate_nodes(Node, Nodes, VersionedNodes, <<"valid">>). diff --git a/tests/riak_control_authentication.erl b/tests/riak_control_authentication.erl index 761e8332d..a8bbb7f8f 100644 --- a/tests/riak_control_authentication.erl +++ b/tests/riak_control_authentication.erl @@ -226,4 +226,4 @@ build_singleton_cluster(Vsn, Config) -> rt:wait_for_control(VersionedNodes), lager:info("Build ~p, nodes: ~p.", [Vsn, Nodes]), - Nodes. \ No newline at end of file + Nodes. diff --git a/tests/riak_rex.erl b/tests/riak_rex.erl index ed0052e0b..30ace1353 100644 --- a/tests/riak_rex.erl +++ b/tests/riak_rex.erl @@ -33,10 +33,10 @@ rex_test(Node) -> deploy_node(NumNodes, current) -> - rt_cluster:deploy_nodes(NumNodes, conf()); + rt:deploy_nodes(NumNodes, conf()); deploy_node(_, mixed) -> Conf = conf(), - rt_cluster:deploy_nodes([{current, Conf}, {previous, Conf}]). + rt:deploy_nodes([{current, Conf}, {previous, Conf}]). deploy_node(Type) -> NumNodes = rt_config:get(num_nodes, 1), diff --git a/tests/riaknostic_rt.erl b/tests/riaknostic_rt.erl index 9193b3bf2..85ed822a5 100644 --- a/tests/riaknostic_rt.erl +++ b/tests/riaknostic_rt.erl @@ -29,7 +29,7 @@ confirm() -> %% Build a small cluster - [Node1, _Node2] = rt_cluster:build_cluster(2, []), + [Node1, _Node2] = rt:build_cluster(2, []), ?assertEqual(ok, rt:wait_until_nodes_ready([Node1])), %% Install riaknostic for Riak versions below 1.3.0 @@ -47,7 +47,7 @@ confirm() -> riaknostic_bootstrap(Node) -> lager:info("Check if riaknostic is installed"), - {ok, RiaknosticOut1} = rt_cmd_line:admin(Node, ["diag"]), + {ok, RiaknosticOut1} = rt:admin(Node, ["diag"]), riaknostic_install((rt:str(RiaknosticOut1, "is not present!")), Node). %% riaknostic is already installed, move along @@ -69,7 +69,7 @@ riaknostic_install(true, Node) -> check_riaknostic_execute(Node) -> %% Execute lager:info("** Check Riaknostic executes"), - {ok, RiaknosticOut} = rt_cmd_line:admin(Node, ["diag"]), + {ok, RiaknosticOut} = rt:admin(Node, ["diag"]), ?assertNot(rt:str(RiaknosticOut, "is not present!")), ?assertNot(rt:str(RiaknosticOut, "[debug]")), ok. @@ -78,7 +78,7 @@ check_riaknostic_execute(Node) -> check_riaknostic_usage(Node) -> %% Check usage message lager:info("** Run Riaknostic usage message"), - {ok, RiaknosticOut} = rt_cmd_line:admin(Node, ["diag", "--help"]), + {ok, RiaknosticOut} = rt:admin(Node, ["diag", "--help"]), ?assert(rt:str(RiaknosticOut, "Usage: riak-admin")), ok. @@ -86,7 +86,7 @@ check_riaknostic_usage(Node) -> check_riaknostic_command_list(Node) -> %% Check commands list lager:info("** Run Riaknostic commands list message"), - {ok, RiaknosticOut} = rt_cmd_line:admin(Node, ["diag", "--list"]), + {ok, RiaknosticOut} = rt:admin(Node, ["diag", "--list"]), ?assert(rt:str(RiaknosticOut, "Available diagnostic checks")), ?assert(rt:str(RiaknosticOut, " disk ")), ?assert(rt:str(RiaknosticOut, " dumps ")), @@ -102,7 +102,7 @@ check_riaknostic_command_list(Node) -> check_riaknostic_log_levels(Node) -> %% Check log levels lager:info("** Run Riaknostic with a different log level"), - {ok, RiaknosticOut} = rt_cmd_line:admin(Node, ["diag", "--level", "debug"]), + {ok, RiaknosticOut} = rt:admin(Node, ["diag", "--level", "debug"]), ?assert(rt:str(RiaknosticOut, "[debug]")), ok. diff --git a/tests/rolling_capabilities.erl b/tests/rolling_capabilities.erl index e87a9fe72..5b05788b4 100644 --- a/tests/rolling_capabilities.erl +++ b/tests/rolling_capabilities.erl @@ -55,7 +55,7 @@ confirm() -> end, lager:info("Deploying Riak ~p cluster", [OldVsn]), - Nodes = rt_cluster:build_cluster([OldVsn || _ <- lists:seq(1,Count)]), + Nodes = rt:build_cluster([OldVsn || _ <- lists:seq(1,Count)]), lists:foldl(fun(Node, Upgraded) -> rt:upgrade(Node, current), Upgraded2 = Upgraded ++ [Node], diff --git a/tests/rt_basic_test.erl b/tests/rt_basic_test.erl index ed91736e0..91881c57f 100644 --- a/tests/rt_basic_test.erl +++ b/tests/rt_basic_test.erl @@ -23,7 +23,7 @@ confirm() -> lager:info("Deploy some nodes"), - Nodes = rt_cluster:deploy_nodes(2), + Nodes = rt:deploy_nodes(2), lager:info("Stop the nodes"), - [rt_node:stop(Node) || Node <- Nodes], + [rt:stop(Node) || Node <- Nodes], pass. diff --git a/tests/secondary_index_tests.erl b/tests/secondary_index_tests.erl index 78d25a87d..aa4618623 100644 --- a/tests/secondary_index_tests.erl +++ b/tests/secondary_index_tests.erl @@ -1,4 +1,3 @@ --module(secondary_index_tests). %% ------------------------------------------------------------------- %% %% Copyright (c) 2012 Basho Technologies, Inc. @@ -18,131 +17,115 @@ %% under the License. %% %% ------------------------------------------------------------------- - -%% -behaviour(riak_test). - --export([properties/0, - confirm/1]). --export([put_an_object/3, put_an_object/5, int_to_key/1, - stream_pb/3, stream_pb/4, pb_query/4, http_query/3, - http_query/4, http_stream/4, int_to_field1_bin/1, url/2, +-module(secondary_index_tests). +-behavior(riak_test). +-export([confirm/0]). +-export([put_an_object/2, put_an_object/4, int_to_key/1, + stream_pb/2, stream_pb/3, pb_query/3, http_query/2, + http_query/3, http_stream/3, int_to_field1_bin/1, url/2, assertExactQuery/5, assertRangeQuery/7]). - --include("rt.hrl"). -include_lib("eunit/include/eunit.hrl"). -include_lib("riakc/include/riakc.hrl"). --test_type(['2i']). - +-define(BUCKET, <<"2ibucket">>). -define(KEYS(A), [int_to_key(A)]). -define(KEYS(A,B), [int_to_key(N) || N <- lists:seq(A,B)]). -define(KEYS(A,B,C), [int_to_key(N) || N <- lists:seq(A,B), C]). -define(KEYS(A,B,G1,G2), [int_to_key(N) || N <- lists:seq(A,B), G1, G2]). --define(DEVS(N), lists:concat([N, "@127.0.0.1"])). --define(DEV(N), list_to_atom(?DEVS(N))). - -properties() -> - rt_properties:new([{node_count, 3}, - {wait_for_transfers, true}, - {valid_backends, [eleveldb, memory]}, - {config, config()}]). - -config() -> - [{riak_kv, [{secondary_index_sort_default, false}]}, - {riak_core, [{handoff_concurrency, 11}]}]. - --spec confirm(rt_properties:properties()) -> pass | fail. -confirm(Properties) -> - NodeMap= rt_properties:get(node_map, Properties), - Nodes = [rt_node:node_name(NodeId, NodeMap) - || NodeId <- rt_properties:get(node_ids, Properties)], - Bucket = druuid:v4_str(), - lager:info("Bucket: ~p", [Bucket]), - PBC = rt_pb:pbc(hd(Nodes)), - HTTPC = rt_http:httpc(hd(Nodes)), +confirm() -> + Nodes = rt:build_cluster(3), + ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), + + %% First test with sorting non-paginated results off by default + SetResult = rpc:multicall(Nodes, application, set_env, + [riak_kv, secondary_index_sort_default, false]), + AOK = [ok || _ <- lists:seq(1, length(Nodes))], + ?assertMatch({AOK, []}, SetResult), + + PBC = rt:pbc(hd(Nodes)), + HTTPC = rt:httpc(hd(Nodes)), Clients = [{pb, PBC}, {http, HTTPC}], - - [put_an_object(PBC, Bucket, N) || N <- lists:seq(0, 20)], - + + [put_an_object(PBC, N) || N <- lists:seq(0, 20)], + K = fun int_to_key/1, - assertExactQuery(Clients, Bucket, ?KEYS(5), <<"field1_bin">>, <<"val5">>), - assertExactQuery(Clients, Bucket, ?KEYS(5), <<"field2_int">>, 5), - assertExactQuery(Clients, Bucket, ?KEYS(5, 9), <<"field3_int">>, 5), - assertRangeQuery(Clients, Bucket, ?KEYS(10, 18), <<"field1_bin">>, <<"val10">>, <<"val18">>), - assertRangeQuery(Clients, Bucket, ?KEYS(12), <<"field1_bin">>, <<"val10">>, <<"val18">>, <<"v...2">>), - assertRangeQuery(Clients, Bucket, ?KEYS(10, 19), <<"field2_int">>, 10, 19), - assertRangeQuery(Clients, Bucket, ?KEYS(10, 17), <<"$key">>, <<"obj10">>, <<"obj17">>), - assertRangeQuery(Clients, Bucket, ?KEYS(12), <<"$key">>, <<"obj10">>, <<"obj17">>, <<"ob..2">>), + assertExactQuery(Clients, ?KEYS(5), <<"field1_bin">>, <<"val5">>), + assertExactQuery(Clients, ?KEYS(5), <<"field2_int">>, 5), + assertExactQuery(Clients, ?KEYS(5, 9), <<"field3_int">>, 5), + assertRangeQuery(Clients, ?KEYS(10, 18), <<"field1_bin">>, <<"val10">>, <<"val18">>), + assertRangeQuery(Clients, ?KEYS(12), <<"field1_bin">>, <<"val10">>, <<"val18">>, <<"v...2">>), + assertRangeQuery(Clients, ?KEYS(10, 19), <<"field2_int">>, 10, 19), + assertRangeQuery(Clients, ?KEYS(10, 17), <<"$key">>, <<"obj10">>, <<"obj17">>), + assertRangeQuery(Clients, ?KEYS(12), <<"$key">>, <<"obj10">>, <<"obj17">>, <<"ob..2">>), lager:info("Delete an object, verify deletion..."), ToDel = [<<"obj05">>, <<"obj11">>], - [?assertMatch(ok, riakc_pb_socket:delete(PBC, Bucket, KD)) || KD <- ToDel], + [?assertMatch(ok, riakc_pb_socket:delete(PBC, ?BUCKET, KD)) || KD <- ToDel], lager:info("Make sure the tombstone is reaped..."), - ?assertMatch(ok, rt:wait_until(fun() -> rt_pb:pbc_really_deleted(PBC, Bucket, ToDel) end)), - - assertExactQuery(Clients, Bucket, [], <<"field1_bin">>, <<"val5">>), - assertExactQuery(Clients, Bucket, [], <<"field2_int">>, 5), - assertExactQuery(Clients, Bucket, ?KEYS(6, 9), <<"field3_int">>, 5), - assertRangeQuery(Clients, Bucket, ?KEYS(10, 18, N /= 11), <<"field1_bin">>, <<"val10">>, <<"val18">>), - assertRangeQuery(Clients, Bucket, ?KEYS(10), <<"field1_bin">>, <<"val10">>, <<"val18">>, <<"10$">>), - assertRangeQuery(Clients, Bucket, ?KEYS(10, 19, N /= 11), <<"field2_int">>, 10, 19), - assertRangeQuery(Clients, Bucket, ?KEYS(10, 17, N /= 11), <<"$key">>, <<"obj10">>, <<"obj17">>), - assertRangeQuery(Clients, Bucket, ?KEYS(12), <<"$key">>, <<"obj10">>, <<"obj17">>, <<"2">>), + ?assertMatch(ok, rt:wait_until(fun() -> rt:pbc_really_deleted(PBC, ?BUCKET, ToDel) end)), + + assertExactQuery(Clients, [], <<"field1_bin">>, <<"val5">>), + assertExactQuery(Clients, [], <<"field2_int">>, 5), + assertExactQuery(Clients, ?KEYS(6, 9), <<"field3_int">>, 5), + assertRangeQuery(Clients, ?KEYS(10, 18, N /= 11), <<"field1_bin">>, <<"val10">>, <<"val18">>), + assertRangeQuery(Clients, ?KEYS(10), <<"field1_bin">>, <<"val10">>, <<"val18">>, <<"10$">>), + assertRangeQuery(Clients, ?KEYS(10, 19, N /= 11), <<"field2_int">>, 10, 19), + assertRangeQuery(Clients, ?KEYS(10, 17, N /= 11), <<"$key">>, <<"obj10">>, <<"obj17">>), + assertRangeQuery(Clients, ?KEYS(12), <<"$key">>, <<"obj10">>, <<"obj17">>, <<"2">>), %% Verify the $key index, and riak_kv#367 regression - assertRangeQuery(Clients, Bucket, ?KEYS(6), <<"$key">>, <<"obj06">>, <<"obj06">>), - assertRangeQuery(Clients, Bucket, ?KEYS(6,7), <<"$key">>, <<"obj06">>, <<"obj07">>), + assertRangeQuery(Clients, ?KEYS(6), <<"$key">>, <<"obj06">>, <<"obj06">>), + assertRangeQuery(Clients, ?KEYS(6,7), <<"$key">>, <<"obj06">>, <<"obj07">>), %% Exercise sort set to true by default SetResult2 = rpc:multicall(Nodes, application, set_env, [riak_kv, secondary_index_sort_default, true]), - ?assertMatch({_, []}, SetResult2), - - assertExactQuery(Clients, Bucket, ?KEYS(15, 19), + ?assertMatch({AOK, []}, SetResult2), + + assertExactQuery(Clients, ?KEYS(15, 19), <<"field3_int">>, 15, {undefined, true}), %% Keys ordered by val index term, since 2i order is {term, key} KsVal = [A || {_, A} <- lists:sort([{int_to_field1_bin(N), K(N)} || N <- lists:seq(0, 20), N /= 11, N /= 5])], - assertRangeQuery(Clients, Bucket, KsVal, + assertRangeQuery(Clients, KsVal, <<"field1_bin">>, <<"val0">>, <<"val9">>, undefined, {undefined, true}), - assertRangeQuery(Clients, Bucket, ?KEYS(0, 20, N /= 11, N /= 5), + assertRangeQuery(Clients, ?KEYS(0, 20, N /= 11, N /= 5), <<"field2_int">>, 0, 20, undefined, {undefined, true}), - assertRangeQuery(Clients, Bucket, ?KEYS(0, 20, N /= 11, N /= 5), + assertRangeQuery(Clients, ?KEYS(0, 20, N /= 11, N /= 5), <<"$key">>, <<"obj00">>, <<"obj20">>, undefined, {undefined, true}), %% Verify bignum sort order in sext -- eleveldb only (riak_kv#499) TestIdxVal = 1362400142028, - put_an_object(PBC, Bucket, TestIdxVal), + put_an_object(PBC, TestIdxVal), assertRangeQuery(Clients, - Bucket, [<<"obj1362400142028">>], <<"field2_int">>, 1000000000000, TestIdxVal), - rt_pb:stop(PBC), + pass. -assertExactQuery(Clients, Bucket, Expected, Index, Value) -> - assertExactQuery(Clients, Bucket, Expected, Index, Value, {false, false}), - assertExactQuery(Clients, Bucket, Expected, Index, Value, {true, true}). +assertExactQuery(Clients, Expected, Index, Value) -> + assertExactQuery(Clients, Expected, Index, Value, {false, false}), + assertExactQuery(Clients, Expected, Index, Value, {true, true}). -assertExactQuery(Clients, Bucket, Expected, Index, Value, Sorted) when is_list(Clients) -> - [assertExactQuery(C, Bucket, Expected, Index, Value, Sorted) || C <- Clients]; -assertExactQuery({ClientType, Client}, Bucket, Expected, Index, Value, - {Sort, ExpectSorted}) -> +assertExactQuery(Clients, Expected, Index, Value, Sorted) when is_list(Clients) -> + [assertExactQuery(C, Expected, Index, Value, Sorted) || C <- Clients]; +assertExactQuery({ClientType, Client}, Expected, Index, Value, + {Sort, ExpectSorted}) -> lager:info("Searching Index ~p for ~p, sort: ~p ~p with client ~p", [Index, Value, Sort, ExpectSorted, ClientType]), {ok, ?INDEX_RESULTS{keys=Results}} = case ClientType of pb -> - riakc_pb_socket:get_index_eq(Client, Bucket, Index, Value, + riakc_pb_socket:get_index_eq(Client, ?BUCKET, Index, Value, [{pagination_sort, Sort} || Sort /= undefined]); http -> - rhc:get_index(Client, Bucket, Index, Value, [{pagination_sort, Sort}]) + rhc:get_index(Client, ?BUCKET, Index, Value, [{pagination_sort, Sort}]) end, - + ActualKeys = case ExpectSorted of true -> Results; _ -> lists:sort(Results) @@ -150,28 +133,28 @@ assertExactQuery({ClientType, Client}, Bucket, Expected, Index, Value, lager:info("Expected: ~p", [Expected]), lager:info("Actual : ~p", [Results]), lager:info("Sorted : ~p", [ActualKeys]), - ?assertEqual(Expected, ActualKeys). - -assertRangeQuery(Clients, Bucket, Expected, Index, StartValue, EndValue) -> - assertRangeQuery(Clients, Bucket, Expected, Index, StartValue, EndValue, undefined). - -assertRangeQuery(Clients, Bucket, Expected, Index, StartValue, EndValue, Re) -> - assertRangeQuery(Clients, Bucket, Expected, Index, StartValue, EndValue, Re, {false, false}), - assertRangeQuery(Clients, Bucket, Expected, Index, StartValue, EndValue, Re, {true, true}). - -assertRangeQuery(Clients, Bucket, Expected, Index, StartValue, EndValue, Re, Sort) when is_list(Clients) -> - [assertRangeQuery(C, Bucket, Expected, Index, StartValue, EndValue, Re, Sort) || C <- Clients]; -assertRangeQuery({ClientType, Client}, Bucket, Expected, Index, StartValue, EndValue, Re, + ?assertEqual(Expected, ActualKeys). + +assertRangeQuery(Clients, Expected, Index, StartValue, EndValue) -> + assertRangeQuery(Clients, Expected, Index, StartValue, EndValue, undefined). + +assertRangeQuery(Clients, Expected, Index, StartValue, EndValue, Re) -> + assertRangeQuery(Clients, Expected, Index, StartValue, EndValue, Re, {false, false}), + assertRangeQuery(Clients, Expected, Index, StartValue, EndValue, Re, {true, true}). + +assertRangeQuery(Clients, Expected, Index, StartValue, EndValue, Re, Sort) when is_list(Clients) -> + [assertRangeQuery(C, Expected, Index, StartValue, EndValue, Re, Sort) || C <- Clients]; +assertRangeQuery({ClientType, Client}, Expected, Index, StartValue, EndValue, Re, {Sort, ExpectSorted}) -> lager:info("Searching Index ~p for ~p-~p re:~p, sort: ~p, ~p with ~p client", [Index, StartValue, EndValue, Re, Sort, ExpectSorted, ClientType]), {ok, ?INDEX_RESULTS{keys=Results}} = case ClientType of pb -> - riakc_pb_socket:get_index_range(Client, Bucket, Index, StartValue, EndValue, + riakc_pb_socket:get_index_range(Client, ?BUCKET, Index, StartValue, EndValue, [{term_regex, Re} || Re /= undefined] ++ [{pagination_sort, Sort} || Sort /= undefined]); http -> - rhc:get_index(Client, Bucket, Index, {StartValue, EndValue}, + rhc:get_index(Client, ?BUCKET, Index, {StartValue, EndValue}, [{term_regex, Re} || Re /= undefined] ++ [{pagination_sort, Sort}]) end, @@ -185,7 +168,7 @@ assertRangeQuery({ClientType, Client}, Bucket, Expected, Index, StartValue, EndV ?assertEqual(Expected, ActualKeys). %% general 2i utility -put_an_object(Pid, Bucket, N) -> +put_an_object(Pid, N) -> Key = int_to_key(N), Data = io_lib:format("data~p", [N]), BinIndex = int_to_field1_bin(N), @@ -194,17 +177,17 @@ put_an_object(Pid, Bucket, N) -> % every 5 items indexed together {"field3_int", N - (N rem 5)} ], - put_an_object(Pid, Bucket, Key, Data, Indexes). + put_an_object(Pid, Key, Data, Indexes). -put_an_object(Pid, Bucket, Key, Data, Indexes) when is_list(Indexes) -> +put_an_object(Pid, Key, Data, Indexes) when is_list(Indexes) -> lager:info("Putting object ~p", [Key]), MetaData = dict:from_list([{<<"index">>, Indexes}]), - Robj0 = riakc_obj:new(Bucket, Key), + Robj0 = riakc_obj:new(?BUCKET, Key), Robj1 = riakc_obj:update_value(Robj0, Data), Robj2 = riakc_obj:update_metadata(Robj1, MetaData), riakc_pb_socket:put(Pid, Robj2); -put_an_object(Pid, Bucket, Key, IntIndex, BinIndex) when is_integer(IntIndex), is_binary(BinIndex) -> - put_an_object(Pid, Bucket, Key, Key, [{"field1_bin", BinIndex},{"field2_int", IntIndex}]). +put_an_object(Pid, Key, IntIndex, BinIndex) when is_integer(IntIndex), is_binary(BinIndex) -> + put_an_object(Pid, Key, Key, [{"field1_bin", BinIndex},{"field2_int", IntIndex}]). int_to_key(N) -> case N < 100 of @@ -217,12 +200,12 @@ int_to_key(N) -> int_to_field1_bin(N) -> list_to_binary(io_lib:format("val~p", [N])). -stream_pb(Pid, Bucket, Q) -> - pb_query(Pid, Bucket, Q, [stream]), +stream_pb(Pid, Q) -> + pb_query(Pid, Q, [stream]), stream_loop(). -stream_pb(Pid, Bucket, Q, Opts) -> - pb_query(Pid, Bucket, Q, [stream|Opts]), +stream_pb(Pid, Q, Opts) -> + pb_query(Pid, Q, [stream|Opts]), stream_loop(). stream_loop() -> @@ -247,29 +230,29 @@ stream_loop(Acc) -> stream_loop(Acc) end. -pb_query(Pid, Bucket, {Field, Val}, Opts) -> - riakc_pb_socket:get_index_eq(Pid, Bucket, Field, Val, Opts); -pb_query(Pid, Bucket, {Field, Start, End}, Opts) -> - riakc_pb_socket:get_index_range(Pid, Bucket, Field, Start, End, Opts). +pb_query(Pid, {Field, Val}, Opts) -> + riakc_pb_socket:get_index_eq(Pid, ?BUCKET, Field, Val, Opts); +pb_query(Pid, {Field, Start, End}, Opts) -> + riakc_pb_socket:get_index_range(Pid, ?BUCKET, Field, Start, End, Opts). -http_stream(NodePath, Bucket, Query, Opts) -> - http_query(NodePath, Bucket, Query, [{stream, true} | Opts], stream). +http_stream(NodePath, Query, Opts) -> + http_query(NodePath, Query, [{stream, true} | Opts], stream). -http_query(NodePath, Bucket, Q) -> - http_query(NodePath, Bucket, Q, []). +http_query(NodePath, Q) -> + http_query(NodePath, Q, []). -http_query(NodePath, Bucket, Query, Opts) -> - http_query(NodePath, Bucket, Query, Opts, undefined). +http_query(NodePath, Query, Opts) -> + http_query(NodePath, Query, Opts, undefined). -http_query(NodePath, Bucket, {Field, Value}, Opts, Pid) -> +http_query(NodePath, {Field, Value}, Opts, Pid) -> QString = opts_to_qstring(Opts, []), Flag = case is_integer(Value) of true -> "w"; false -> "s" end, - Url = url("~s/buckets/~s/index/~s/~"++Flag++"~s", [NodePath, Bucket, Field, Value, QString]), + Url = url("~s/buckets/~s/index/~s/~"++Flag++"~s", [NodePath, ?BUCKET, Field, Value, QString]), http_get(Url, Pid); -http_query(NodePath, Bucket, {Field, Start, End}, Opts, Pid) -> +http_query(NodePath, {Field, Start, End}, Opts, Pid) -> QString = opts_to_qstring(Opts, []), Flag = case is_integer(Start) of true -> "w"; false -> "s" end, - Url = url("~s/buckets/~s/index/~s/~"++Flag++"/~"++Flag++"~s", [NodePath, Bucket, Field, Start, End, QString]), + Url = url("~s/buckets/~s/index/~s/~"++Flag++"/~"++Flag++"~s", [NodePath, ?BUCKET, Field, Start, End, QString]), http_get(Url, Pid). url(Format, Elements) -> @@ -317,7 +300,7 @@ start_http_stream(Ref) -> Other -> lager:error("Unexpected message ~p", [Other]), {error, unknown_message} after 60000 -> - {error, timeout_local} + {error, timeout_local} end. http_stream_loop(Ref, Acc, {Boundary, BLen}=B) -> @@ -346,3 +329,4 @@ get_boundary("multipart/mixed;boundary=" ++ Boundary) -> {B, byte_size(B)}; get_boundary(_) -> undefined. + diff --git a/tests/sibling_explosion.erl b/tests/sibling_explosion.erl index 510742ab8..aee706c68 100644 --- a/tests/sibling_explosion.erl +++ b/tests/sibling_explosion.erl @@ -21,11 +21,11 @@ confirm() -> Conf = [{riak_core, [{default_bucket_props, [{allow_mult, true}, {dvv_enabled, true}]}]}], - [Node1] = rt_cluster:deploy_nodes(1, Conf), + [Node1] = rt:deploy_nodes(1, Conf), N = 100, lager:info("Put new object in ~p via PBC.", [Node1]), - PB = rt_pb:pbc(Node1), + PB = rt:pbc(Node1), A0 = riakc_obj:new(<<"b">>, <<"k">>, sets:from_list([0])), B0 = riakc_obj:new(<<"b">>, <<"k">>, sets:from_list([1])), diff --git a/tests/test_cluster.erl b/tests/test_cluster.erl index 165a0a2d7..abe34d661 100644 --- a/tests/test_cluster.erl +++ b/tests/test_cluster.erl @@ -24,6 +24,6 @@ confirm() -> Config = [{riak_search, [{enabled, true}]}], - rt_cluster:build_cluster(4, Config), + rt:build_cluster(4, Config), ?assert(false), - fail. + fail. \ No newline at end of file diff --git a/tests/verify_2i_aae.erl b/tests/verify_2i_aae.erl index ac682fc8d..1ea4c0e2d 100644 --- a/tests/verify_2i_aae.erl +++ b/tests/verify_2i_aae.erl @@ -225,4 +225,4 @@ assert_range_query(Pid, Bucket, Expected0, Index, StartValue, EndValue) -> end, Expected = lists:sort(Expected0), ?assertEqual({Bucket, Expected}, {Bucket, Actual}), - lager:info("Yay! ~b (actual) == ~b (expected)", [length(Actual), length(Expected)]). \ No newline at end of file + lager:info("Yay! ~b (actual) == ~b (expected)", [length(Actual), length(Expected)]). diff --git a/tests/verify_2i_limit.erl b/tests/verify_2i_limit.erl index 6c2acdfbf..944b371d4 100644 --- a/tests/verify_2i_limit.erl +++ b/tests/verify_2i_limit.erl @@ -31,12 +31,12 @@ confirm() -> inets:start(), - Nodes = rt_cluster:build_cluster(3), + Nodes = rt:build_cluster(3), ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), - RiakHttp = rt_http:httpc(hd(Nodes)), + RiakHttp = rt:httpc(hd(Nodes)), HttpUrl = rt:http_url(hd(Nodes)), - PBPid = rt_pb:pbc(hd(Nodes)), + PBPid = rt:pbc(hd(Nodes)), [put_an_object(PBPid, N) || N <- lists:seq(0, 100)], @@ -89,7 +89,7 @@ confirm() -> %% gh611 - equals query pagination riakc_pb_socket:delete(PBPid, ?BUCKET, <<"bob">>), - rt:wait_until(fun() -> rt_pb:pbc_really_deleted(PBPid, ?BUCKET, [<<"bob">>]) end), + rt:wait_until(fun() -> rt:pbc_really_deleted(PBPid, ?BUCKET, [<<"bob">>]) end), [put_an_object(PBPid, int_to_key(N), 1000, <<"myval">>) || N <- lists:seq(0, 100)], diff --git a/tests/verify_2i_mixed_cluster.erl b/tests/verify_2i_mixed_cluster.erl index 2552e2b50..02369948b 100644 --- a/tests/verify_2i_mixed_cluster.erl +++ b/tests/verify_2i_mixed_cluster.erl @@ -31,14 +31,14 @@ confirm() -> OldVsn = proplists:get_value(upgrade_version, TestMetaData, previous), Nodes = [CurrentNode, OldNode1, _] = - rt_cluster:build_cluster([{current, + rt:build_cluster([{current, [{riak_kv, [{anti_entropy, {off, []}}]}]}, OldVsn, OldVsn]), - ?assertEqual(ok, rt_node:wait_until_nodes_ready(Nodes)), + ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), - PBC1 = rt_pb:pbc(CurrentNode), - PBC2 = rt_pb:pbc(OldNode1), - HTTPC1 = rt_http:httpc(CurrentNode), + PBC1 = rt:pbc(CurrentNode), + PBC2 = rt:pbc(OldNode1), + HTTPC1 = rt:httpc(CurrentNode), Clients = [{pb, PBC1}, {pb, PBC2}, {http, HTTPC1}], @@ -57,7 +57,7 @@ confirm() -> ToDel = [<<"obj05">>, <<"obj11">>], [?assertMatch(ok, riakc_pb_socket:delete(PBC1, ?BUCKET, KD)) || KD <- ToDel], lager:info("Make sure the tombstone is reaped..."), - ?assertMatch(ok, rt:wait_until(fun() -> rt_pb:pbc_really_deleted(PBC1, ?BUCKET, ToDel) end)), + ?assertMatch(ok, rt:wait_until(fun() -> rt:pbc_really_deleted(PBC1, ?BUCKET, ToDel) end)), assertExactQuery(Clients, [], <<"field1_bin">>, <<"val5">>), assertExactQuery(Clients, [], <<"field2_int">>, 5), diff --git a/tests/verify_2i_returnterms.erl b/tests/verify_2i_returnterms.erl index 7548256e3..7a9f50ee4 100644 --- a/tests/verify_2i_returnterms.erl +++ b/tests/verify_2i_returnterms.erl @@ -30,11 +30,11 @@ confirm() -> inets:start(), - Nodes = rt_cluster:build_cluster(3), + Nodes = rt:build_cluster(3), ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), RiakHttp = rt:http_url(hd(Nodes)), - PBPid = rt_pb:pbc(hd(Nodes)), + PBPid = rt:pbc(hd(Nodes)), [put_an_object(PBPid, N) || N <- lists:seq(0, 100)], [put_an_object(PBPid, int_to_key(N), N, ?FOO) || N <- lists:seq(101, 200)], diff --git a/tests/verify_2i_stream.erl b/tests/verify_2i_stream.erl index 8440fc30a..32aff6939 100644 --- a/tests/verify_2i_stream.erl +++ b/tests/verify_2i_stream.erl @@ -29,11 +29,11 @@ confirm() -> inets:start(), - Nodes = rt_cluster:build_cluster(3), + Nodes = rt:build_cluster(3), ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), RiakHttp = rt:http_url(hd(Nodes)), - PBPid = rt_pb:pbc(hd(Nodes)), + PBPid = rt:pbc(hd(Nodes)), [put_an_object(PBPid, N) || N <- lists:seq(0, 100)], [put_an_object(PBPid, int_to_key(N), N, ?FOO) || N <- lists:seq(101, 200)], diff --git a/tests/verify_2i_timeout.erl b/tests/verify_2i_timeout.erl index 6f05fa955..d913b5631 100644 --- a/tests/verify_2i_timeout.erl +++ b/tests/verify_2i_timeout.erl @@ -29,10 +29,10 @@ confirm() -> inets:start(), Config = [{riak_kv, [{secondary_index_timeout, 1}]}], %% ludicrously short, should fail always - Nodes = rt_cluster:build_cluster([{current, Config}, {current, Config}, {current, Config}]), + Nodes = rt:build_cluster([{current, Config}, {current, Config}, {current, Config}]), ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), - PBPid = rt_pb:pbc(hd(Nodes)), + PBPid = rt:pbc(hd(Nodes)), Http = rt:http_url(hd(Nodes)), [put_an_object(PBPid, N) || N <- lists:seq(0, 100)], diff --git a/tests/verify_aae.erl b/tests/verify_aae.erl index 56a6026bd..1d5fa2dda 100644 --- a/tests/verify_aae.erl +++ b/tests/verify_aae.erl @@ -301,4 +301,4 @@ max_aae_repairs(Node) when is_atom(Node) -> Info = rpc:call(Node, riak_kv_entropy_info, compute_exchange_info, []), LastCounts = [Last || {_, _, _, {Last, _, _, _}} <- Info], MaxCount = lists:max(LastCounts), - MaxCount. \ No newline at end of file + MaxCount. diff --git a/tests/verify_api_timeouts.erl b/tests/verify_api_timeouts.erl index 763e3f8bd..0a6851cfd 100644 --- a/tests/verify_api_timeouts.erl +++ b/tests/verify_api_timeouts.erl @@ -9,13 +9,13 @@ confirm() -> %% test requires allow_mult=false b/c of rt:systest_read - [Node] = rt_cluster:build_cluster(1), + [Node] = rt:build_cluster(1), rt:wait_until_pingable(Node), - HC = rt_http:httpc(Node), + HC = rt:httpc(Node), lager:info("setting up initial data and loading remote code"), - rt_http:httpc_write(HC, <<"foo">>, <<"bar">>, <<"foobarbaz\n">>), - rt_http:httpc_write(HC, <<"foo">>, <<"bar2">>, <<"foobarbaz2\n">>), + rt:httpc_write(HC, <<"foo">>, <<"bar">>, <<"foobarbaz\n">>), + rt:httpc_write(HC, <<"foo">>, <<"bar2">>, <<"foobarbaz2\n">>), put_keys(Node, ?BUCKET, ?NUM_KEYS), put_buckets(Node, ?NUM_BUCKETS), @@ -72,7 +72,7 @@ confirm() -> end, - PC = rt_pb:pbc(Node), + PC = rt:pbc(Node), lager:info("testing PBC API"), @@ -126,7 +126,7 @@ confirm() -> lager:info("Checking List timeouts"), lager:info("Checking PBC"), - Pid = rt_pb:pbc(Node), + Pid = rt:pbc(Node), lager:info("Checking keys timeout"), ?assertMatch({error, <<"timeout">>}, riakc_pb_socket:list_keys(Pid, ?BUCKET, Short)), @@ -155,7 +155,7 @@ confirm() -> lager:info("Checking HTTP"), - LHC = rt_http:httpc(Node), + LHC = rt:httpc(Node), lager:info("Checking keys timeout"), ?assertMatch({error, <<"timeout">>}, rhc:list_keys(LHC, ?BUCKET, Short)), @@ -228,7 +228,7 @@ wait_for_end(ReqId) -> put_buckets(Node, Num) -> - Pid = rt_pb:pbc(Node), + Pid = rt:pbc(Node), Buckets = [list_to_binary(["", integer_to_list(Ki)]) || Ki <- lists:seq(0, Num - 1)], {Key, Val} = {<<"test_key">>, <<"test_value">>}, @@ -238,7 +238,7 @@ put_buckets(Node, Num) -> put_keys(Node, Bucket, Num) -> - Pid = rt_pb:pbc(Node), + Pid = rt:pbc(Node), Keys = [list_to_binary(["", integer_to_list(Ki)]) || Ki <- lists:seq(0, Num - 1)], Vals = [list_to_binary(["", integer_to_list(Ki)]) || Ki <- lists:seq(0, Num - 1)], [riakc_pb_socket:put(Pid, riakc_obj:new(Bucket, Key, Val)) || {Key, Val} <- lists:zip(Keys, Vals)], diff --git a/tests/verify_asis_put.erl b/tests/verify_asis_put.erl index 7e5a71df5..d34439c0f 100644 --- a/tests/verify_asis_put.erl +++ b/tests/verify_asis_put.erl @@ -4,11 +4,11 @@ confirm() -> %% 1. Deploy two nodes - [Node1, Node2] = rt_cluster:deploy_nodes(2), + [Node1, Node2] = rt:deploy_nodes(2), %% 2. With PBC lager:info("Put new object in ~p via PBC.", [Node1]), - PB1 = rt_pb:pbc(Node1), - PB2 = rt_pb:pbc(Node2), + PB1 = rt:pbc(Node1), + PB2 = rt:pbc(Node2), Obj1 = riakc_obj:new(<<"verify_asis_put">>, <<"1">>, <<"test">>, "text/plain"), %% a. put in node 1 %% b. fetch from node 1 for vclock @@ -23,8 +23,8 @@ confirm() -> %% 3. Repeat with HTTP, nodes reversed lager:info("Put new object in ~p via HTTP.", [Node2]), - HTTP1 = rt_http:httpc(Node1), - HTTP2 = rt_http:httpc(Node2), + HTTP1 = rt:httpc(Node1), + HTTP2 = rt:httpc(Node2), Obj2 = riakc_obj:new(<<"verify_asis_put">>, <<"2">>, <<"test">>, "text/plain"), %% a. put in node 2 %% b. fetch from node 2 for vclock diff --git a/tests/verify_backup_restore.erl b/tests/verify_backup_restore.erl index 95272ff59..b587160ad 100644 --- a/tests/verify_backup_restore.erl +++ b/tests/verify_backup_restore.erl @@ -39,10 +39,10 @@ confirm() -> lager:info("Building cluster of ~p nodes", [?NUM_NODES]), SpamDir = rt_config:config_or_os_env(spam_dir), Config = [{riak_search, [{enabled, true}]}], - [Node0 | _RestNodes] = Nodes = rt_cluster:build_cluster(?NUM_NODES, Config), + [Node0 | _RestNodes] = Nodes = rt:build_cluster(?NUM_NODES, Config), rt:enable_search_hook(Node0, ?SEARCH_BUCKET), rt:wait_until_ring_converged(Nodes), - PbcPid = rt_pb:pbc(Node0), + PbcPid = rt:pbc(Node0), Searches = [ {<<"ZiaSun">>, 1}, @@ -57,7 +57,7 @@ confirm() -> AllTerms = lists:foldl(ConcatBin, <<"">>, Searches), lager:info("Indexing data for search from ~p", [SpamDir]), - rt_pb:pbc_put_dir(PbcPid, ?SEARCH_BUCKET, SpamDir), + rt:pbc_put_dir(PbcPid, ?SEARCH_BUCKET, SpamDir), ExtraKey = <<"Extra1">>, riakc_pb_socket:put(PbcPid, riakc_obj:new(?SEARCH_BUCKET, @@ -82,7 +82,7 @@ confirm() -> lager:info("Backing up the data to ~p", [BackupFile]), Cookie = "riak", - rt_cmd_line:admin(Node0, ["backup", atom_to_list(Node0), Cookie, BackupFile, "all"]), + rt:admin(Node0, ["backup", atom_to_list(Node0), Cookie, BackupFile, "all"]), lager:info("Modifying data on cluster"), ModF = fun(N) -> @@ -98,7 +98,7 @@ confirm() -> {last, ?NUM_MOD+?NUM_DEL}]), lager:info("Deleting extra search doc"), riakc_pb_socket:delete(PbcPid, ?SEARCH_BUCKET, ExtraKey), - rt:wait_until(fun() -> rt_pb:pbc_really_deleted(PbcPid, + rt:wait_until(fun() -> rt:pbc_really_deleted(PbcPid, ?SEARCH_BUCKET, [ExtraKey]) end), @@ -114,7 +114,7 @@ confirm() -> verify_searches(PbcPid, Searches, 0), lager:info("Restoring from backup ~p", [BackupFile]), - rt_cmd_line:admin(Node0, ["restore", atom_to_list(Node0), Cookie, BackupFile]), + rt:admin(Node0, ["restore", atom_to_list(Node0), Cookie, BackupFile]), rt:wait_until_no_pending_changes(Nodes), %% When allow_mult=false, the mods overwrite the restored data. When @@ -135,13 +135,13 @@ confirm() -> lager:info("Wipe out entire cluster and start fresh"), riakc_pb_socket:stop(PbcPid), - rt_cluster:clean_cluster(Nodes), + rt:clean_cluster(Nodes), lager:info("Rebuilding the cluster"), - rt_cluster:build_cluster(?NUM_NODES, Config), + rt:build_cluster(?NUM_NODES, Config), rt:enable_search_hook(Node0, ?SEARCH_BUCKET), rt:wait_until_ring_converged(Nodes), rt:wait_until_no_pending_changes(Nodes), - PbcPid2 = rt_pb:pbc(Node0), + PbcPid2 = rt:pbc(Node0), lager:info("Verify no data in cluster"), [?assertEqual([], read_some(Node, [{last, ?NUM_KEYS}, @@ -150,7 +150,7 @@ confirm() -> verify_searches(PbcPid2, EmptySearches, 0), lager:info("Restoring from backup ~p again", [BackupFile]), - rt_cmd_line:admin(Node0, ["restore", atom_to_list(Node0), Cookie, BackupFile]), + rt:admin(Node0, ["restore", atom_to_list(Node0), Cookie, BackupFile]), rt:enable_search_hook(Node0, ?SEARCH_BUCKET), lager:info("Verifying data is back to original backup"), @@ -198,7 +198,7 @@ write_some(PBC, Props) -> end end, ?assertEqual([], lists:foldl(DelFun, [], Keys)), - rt:wait_until(fun() -> rt_pb:pbc_really_deleted(PBC, Bucket, Keys1) end); + rt:wait_until(fun() -> rt:pbc_really_deleted(PBC, Bucket, Keys1) end); _ -> ok end, @@ -283,7 +283,7 @@ delete_some(PBC, Props) -> end end, lists:foldl(F, [], Keys), - rt:wait_until(fun() -> rt_pb:pbc_really_deleted(PBC, Bucket, Keys) end), + rt:wait_until(fun() -> rt:pbc_really_deleted(PBC, Bucket, Keys) end), ok. verify_search_count(Pid, SearchQuery, Count) -> diff --git a/tests/verify_basic_upgrade.erl b/tests/verify_basic_upgrade.erl index 138243048..f02a7cc11 100644 --- a/tests/verify_basic_upgrade.erl +++ b/tests/verify_basic_upgrade.erl @@ -26,11 +26,11 @@ confirm() -> TestMetaData = riak_test_runner:metadata(), OldVsn = proplists:get_value(upgrade_version, TestMetaData, previous), - Nodes = [Node1|_] = rt_cluster:build_cluster([OldVsn, OldVsn, OldVsn, OldVsn]), + Nodes = [Node1|_] = rt:build_cluster([OldVsn, OldVsn, OldVsn, OldVsn]), lager:info("Writing 100 keys to ~p", [Node1]), - rt_systest:write(Node1, 100, 3), - ?assertEqual([], rt_systest:read(Node1, 100, 1)), + rt:systest_write(Node1, 100, 3), + ?assertEqual([], rt:systest_read(Node1, 100, 1)), [upgrade(Node, current) || Node <- Nodes], @@ -43,6 +43,6 @@ upgrade(Node, NewVsn) -> rt:upgrade(Node, NewVsn), rt:wait_for_service(Node, riak_kv), lager:info("Ensuring keys still exist"), - rt_systest:read(Node, 100, 1), - ?assertEqual([], rt_systest:read(Node, 100, 1)), + rt:systest_read(Node, 100, 1), + ?assertEqual([], rt:systest_read(Node, 100, 1)), ok. diff --git a/tests/verify_bitcask_tombstone2_upgrade.erl b/tests/verify_bitcask_tombstone2_upgrade.erl index fe0eba167..72543e2c8 100644 --- a/tests/verify_bitcask_tombstone2_upgrade.erl +++ b/tests/verify_bitcask_tombstone2_upgrade.erl @@ -88,4 +88,4 @@ file_exists(Node, Path) -> end. all(Val, L) -> - lists:all(fun(E) -> E == Val end, L). \ No newline at end of file + lists:all(fun(E) -> E == Val end, L). diff --git a/tests/verify_build_cluster.erl b/tests/verify_build_cluster.erl index eb6d0e6ec..e934eb434 100644 --- a/tests/verify_build_cluster.erl +++ b/tests/verify_build_cluster.erl @@ -104,4 +104,4 @@ wait_and_validate(RingNodes, UpNodes) -> [rt:wait_for_service(Node, riak_kv) || Node <- UpNodes], lager:info("Verify that you got much data... (this is how we do it)"), ?assertEqual([], rt:systest_read(hd(UpNodes), 0, 1000, <<"verify_build_cluster">>, 2)), - done. \ No newline at end of file + done. diff --git a/tests/verify_busy_dist_port.erl b/tests/verify_busy_dist_port.erl index ac29da4a6..c8828d902 100644 --- a/tests/verify_busy_dist_port.erl +++ b/tests/verify_busy_dist_port.erl @@ -47,7 +47,7 @@ -include_lib("eunit/include/eunit.hrl"). confirm() -> - [Node1, Node2] = rt_cluster:build_cluster(2), + [Node1, Node2] = rt:build_cluster(2), lager:info("deployed 2 nodes"), rt:load_modules_on_nodes([cause_bdp, verify_bdp_event_handler, diff --git a/tests/verify_capabilities.erl b/tests/verify_capabilities.erl index 8fde91133..a1ff966df 100644 --- a/tests/verify_capabilities.erl +++ b/tests/verify_capabilities.erl @@ -264,3 +264,4 @@ restart_capability_server(Node) -> end end, rt:wait_until(Node, HasNewPid). + diff --git a/tests/verify_commit_hooks.erl b/tests/verify_commit_hooks.erl index 8e754bb7d..fdc8ca40a 100644 --- a/tests/verify_commit_hooks.erl +++ b/tests/verify_commit_hooks.erl @@ -23,7 +23,7 @@ -export([confirm/0]). confirm() -> - [Node] = rt_cluster:deploy_nodes(1), + [Node] = rt:deploy_nodes(1), lager:info("Loading the hooks module into ~p", [Node]), rt:load_modules_on_nodes([hooks], [Node]), @@ -34,36 +34,36 @@ confirm() -> ?assertEqual(ok, rpc:call(Node, hooks, set_hooks, [])), lager:info("Checking precommit atom failure reason."), - HTTP = rt_http:httpc(Node), + HTTP = rt:httpc(Node), ?assertMatch({error, {ok, "500", _, _}}, - rt_http:httpc_write(HTTP, <<"failatom">>, <<"key">>, <<"value">>)), + rt:httpc_write(HTTP, <<"failatom">>, <<"key">>, <<"value">>)), lager:info("Checking Bug 1145 - string failure reason"), ?assertMatch({error, {ok, "403", _, _}}, - rt_http:httpc_write(HTTP, <<"failstr">>, <<"key">>, <<"value">>)), + rt:httpc_write(HTTP, <<"failstr">>, <<"key">>, <<"value">>)), lager:info("Checking Bug 1145 - binary failure reason"), ?assertMatch({error, {ok, "403", _, _}}, - rt_http:httpc_write(HTTP, <<"failbin">>, <<"key">>, <<"value">>)), + rt:httpc_write(HTTP, <<"failbin">>, <<"key">>, <<"value">>)), lager:info("Checking that bucket without commit hooks passes."), - ?assertEqual(ok, rt_http:httpc_write(HTTP, <<"fail">>, <<"key">>, <<"value">>)), + ?assertEqual(ok, rt:httpc_write(HTTP, <<"fail">>, <<"key">>, <<"value">>)), lager:info("Checking that bucket with passing precommit passes."), - ?assertEqual(ok, rt_http:httpc_write(HTTP, <<"failkey">>, <<"key">>, <<"value">>)), + ?assertEqual(ok, rt:httpc_write(HTTP, <<"failkey">>, <<"key">>, <<"value">>)), lager:info("Checking that bucket with failing precommit fails."), ?assertMatch({error, {ok, "403", _, _}}, - rt_http:httpc_write(HTTP, <<"failkey">>, <<"fail">>, <<"value">>)), + rt:httpc_write(HTTP, <<"failkey">>, <<"fail">>, <<"value">>)), lager:info("Checking fix for BZ1244 - riak_kv_wm_object makes call to riak_client:get/3 with invalid type for key"), %% riak_kv_wm_object:ensure_doc will return {error, not_found}, leading to 404. %% see https://github.com/basho/riak_kv/pull/237 for details of the fix. ?assertMatch({error, {ok, "404", _, _}}, - rt_http:httpc_write(HTTP, <<"bz1244bucket">>, undefined, <<"value">>)), + rt:httpc_write(HTTP, <<"bz1244bucket">>, undefined, <<"value">>)), lager:info("Checking that postcommit fires."), - ?assertMatch(ok, rt_http:httpc_write(HTTP, <<"postcommit">>, <<"key">>, <<"value">>)), + ?assertMatch(ok, rt:httpc_write(HTTP, <<"postcommit">>, <<"key">>, <<"value">>)), receive {wrote, _Bucket, _Key}=Msg -> diff --git a/tests/verify_conditional_postcommit.erl b/tests/verify_conditional_postcommit.erl index d5e4208fe..fcb2c5f0c 100644 --- a/tests/verify_conditional_postcommit.erl +++ b/tests/verify_conditional_postcommit.erl @@ -24,13 +24,13 @@ confirm() -> Config = [{riak_core, [{vnode_management_timer, 1000}, {ring_creation_size, 4}]}], - Nodes = rt_cluster:deploy_nodes(1, Config), + Nodes = rt:deploy_nodes(1, Config), Node = hd(Nodes), ok = rt:load_modules_on_nodes([?MODULE], Nodes), lager:info("Creating bucket types 'type1' and 'type2'"), - rt_bucket_types:create_and_activate_bucket_type(Node, <<"type1">>, [{magic, false}]), - rt_bucket_types:create_and_activate_bucket_type(Node, <<"type2">>, [{magic, true}]), + rt:create_and_activate_bucket_type(Node, <<"type1">>, [{magic, false}]), + rt:create_and_activate_bucket_type(Node, <<"type2">>, [{magic, true}]), lager:info("Installing conditional hook"), CondHook = {?MODULE, conditional_hook}, @@ -39,7 +39,7 @@ confirm() -> Bucket1 = {<<"type1">>, <<"test">>}, Bucket2 = {<<"type2">>, <<"test">>}, Keys = [<> || N <- lists:seq(1,1000)], - PBC = rt_pb:pbc(Node), + PBC = rt:pbc(Node), lager:info("Writing keys as 'type1' and verifying hook is not triggered"), write_keys(Node, PBC, Bucket1, Keys, false), @@ -55,7 +55,7 @@ confirm() -> write_keys(Node, PBC, Bucket, Keys, ShouldHook) -> rpc:call(Node, application, set_env, [riak_kv, hook_count, 0]), - [ok = rt_pb:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], + [ok = rt:pbc_write(PBC, Bucket, Key, Key) || Key <- Keys], {ok, Count} = rpc:call(Node, application, get_env, [riak_kv, hook_count]), case ShouldHook of true -> diff --git a/tests/verify_corruption_filtering.erl b/tests/verify_corruption_filtering.erl index df899458f..7154786d4 100644 --- a/tests/verify_corruption_filtering.erl +++ b/tests/verify_corruption_filtering.erl @@ -65,7 +65,7 @@ confirm() -> pass. get_put_mix(Node) -> - PB = rt_pb:pbc(Node), + PB = rt:pbc(Node), [begin Key = random:uniform(1000), case random:uniform(2) of @@ -90,7 +90,7 @@ get_put_mix(Node) -> || _ <- lists:seq(1, 2000)]. load_cluster(Node) -> - PB = rt_pb:pbc(Node), + PB = rt:pbc(Node), [riakc_pb_socket:put(PB, riakc_obj:new(<<"foo">>, <>, <>)) diff --git a/tests/verify_counter_capability.erl b/tests/verify_counter_capability.erl index 0d596f971..5f259c2dc 100644 --- a/tests/verify_counter_capability.erl +++ b/tests/verify_counter_capability.erl @@ -61,7 +61,7 @@ confirm() -> rt:upgrade(Legacy, previous), - PrevPB2 = rt_pb:pbc(Legacy), + PrevPB2 = rt:pbc(Legacy), ?assertEqual(ok, rt:wait_until_capability_contains(Previous, {riak_kv, crdt}, [pncounter,riak_dt_pncounter,riak_dt_orswot,riak_dt_map])), @@ -81,4 +81,4 @@ confirm() -> pass. get_clients(Node) -> - {rt_pb:pbc(Node), rt_http:httpc(Node)}. + {rt:pbc(Node), rt:httpc(Node)}. diff --git a/tests/verify_counter_converge.erl b/tests/verify_counter_converge.erl index af8c37872..2217d582b 100644 --- a/tests/verify_counter_converge.erl +++ b/tests/verify_counter_converge.erl @@ -33,7 +33,7 @@ confirm() -> Key = <<"a">>, - [N1, N2, N3, N4]=Nodes = rt_cluster:build_cluster(4), + [N1, N2, N3, N4]=Nodes = rt:build_cluster(4), [C1, C2, C3, C4]=Clients = [ rt:httpc(N) || N <- Nodes ], set_allow_mult_true(Nodes), @@ -50,7 +50,7 @@ confirm() -> lager:info("Partition cluster in two."), - PartInfo = rt_node:partition([N1, N2], [N3, N4]), + PartInfo = rt:partition([N1, N2], [N3, N4]), %% increment one side increment_counter(C1, Key, 5), @@ -68,7 +68,7 @@ confirm() -> %% heal lager:info("Heal and check merged values"), - ok = rt_node:heal(PartInfo), + ok = rt:heal(PartInfo), ok = rt:wait_for_cluster_service(Nodes, riak_kv), %% verify all nodes agree diff --git a/tests/verify_counter_repl.erl b/tests/verify_counter_repl.erl index 42fc97049..79a107b37 100644 --- a/tests/verify_counter_repl.erl +++ b/tests/verify_counter_repl.erl @@ -63,7 +63,7 @@ make_clusters() -> Conf = [{riak_repl, [{fullsync_on_connect, false}, {fullsync_interval, disabled}]}, {riak_core, [{default_bucket_props, [{allow_mult, true}]}]}], - Nodes = rt_cluster:deploy_nodes(6, Conf), + Nodes = rt:deploy_nodes(6, Conf, [riak_kv, riak_repl]), {ClusterA, ClusterB} = lists:split(3, Nodes), A = make_cluster(ClusterA, "A"), B = make_cluster(ClusterB, "B"), @@ -73,7 +73,7 @@ make_cluster(Nodes, Name) -> repl_util:make_cluster(Nodes), repl_util:name_cluster(hd(Nodes), Name), repl_util:wait_until_leader_converge(Nodes), - Clients = [ rt_http:httpc(Node) || Node <- Nodes ], + Clients = [ rt:httpc(Node) || Node <- Nodes ], lists:zip(Clients, Nodes). increment_cluster_counter(Cluster) -> diff --git a/tests/verify_crdt_capability.erl b/tests/verify_crdt_capability.erl index 27e0d9b14..9b49b5a57 100644 --- a/tests/verify_crdt_capability.erl +++ b/tests/verify_crdt_capability.erl @@ -98,4 +98,4 @@ gen_counter_op() -> riakc_counter:to_op(riakc_counter:increment(riakc_counter:new())). get_clients(Node) -> - {rt_pb:pbc(Node), rt_http:httpc(Node)}. + {rt:pbc(Node), rt:httpc(Node)}. diff --git a/tests/verify_cs_bucket.erl b/tests/verify_cs_bucket.erl index 4b517b043..5db40bbca 100644 --- a/tests/verify_cs_bucket.erl +++ b/tests/verify_cs_bucket.erl @@ -29,10 +29,10 @@ -define(FOO, <<"foo">>). confirm() -> - Nodes = rt_cluster:build_cluster(3), + Nodes = rt:build_cluster(3), ?assertEqual(ok, (rt:wait_until_nodes_ready(Nodes))), - PBPid = rt_pb:pbc(hd(Nodes)), + PBPid = rt:pbc(hd(Nodes)), [put_an_object(PBPid, N) || N <- lists:seq(0, 200)], diff --git a/tests/verify_down.erl b/tests/verify_down.erl index a2c21f7a2..973fb9f10 100644 --- a/tests/verify_down.erl +++ b/tests/verify_down.erl @@ -65,4 +65,4 @@ confirm() -> %% Verify that all three nodes are ready lager:info("Ensure all nodes are ready"), ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), - pass. \ No newline at end of file + pass. diff --git a/tests/verify_dt_context.erl b/tests/verify_dt_context.erl index b4b6e11e1..acdd74e27 100644 --- a/tests/verify_dt_context.erl +++ b/tests/verify_dt_context.erl @@ -44,7 +44,7 @@ confirm() -> {riak_core, [ {ring_creation_size, 16}, {vnode_management_timer, 1000} ]}], - [N1, N2]=Nodes = rt_cluster:build_cluster(2, Config), + [N1, N2]=Nodes = rt:build_cluster(2, Config), create_bucket_types(Nodes, ?TYPES), @@ -72,7 +72,7 @@ confirm() -> lager:info("Partition cluster in two."), - PartInfo = rt_node:partition([N1], [N2]), + PartInfo = rt:partition([N1], [N2]), lager:info("Modify data on side 1"), %% Modify one side @@ -135,7 +135,7 @@ confirm() -> %% Check both sides %% heal lager:info("Heal and check merged values"), - ok = rt_node:heal(PartInfo), + ok = rt:heal(PartInfo), ok = rt:wait_for_cluster_service(Nodes, riak_kv), %% verify all nodes agree @@ -192,14 +192,14 @@ store_map(Client, Map) -> create_pb_clients(Nodes) -> [begin - C = rt_pb:pbc(N), + C = rt:pbc(N), riakc_pb_socket:set_options(C, [queue_if_disconnected]), C end || N <- Nodes]. create_bucket_types([N1|_], Types) -> lager:info("Creating bucket types with datatypes: ~p", [Types]), - [rt_bucket_types:create_and_activate_bucket_type(N1, Name, [{datatype, Type}, {allow_mult, true}]) + [rt:create_and_activate_bucket_type(N1, Name, [{datatype, Type}, {allow_mult, true}]) || {Name, Type} <- Types ]. bucket_type_ready_fun(Name) -> diff --git a/tests/verify_dt_converge.erl b/tests/verify_dt_converge.erl index f98422e25..60c78ab36 100644 --- a/tests/verify_dt_converge.erl +++ b/tests/verify_dt_converge.erl @@ -337,4 +337,4 @@ check_value(Client, CMod, Bucket, Key, DTMod, Expected, Options) -> Error]), false end - end). \ No newline at end of file + end). diff --git a/tests/verify_dt_upgrade.erl b/tests/verify_dt_upgrade.erl index 258e211d3..c43f3fb09 100644 --- a/tests/verify_dt_upgrade.erl +++ b/tests/verify_dt_upgrade.erl @@ -31,7 +31,7 @@ confirm() -> TestMetaData = riak_test_runner:metadata(), OldVsn = proplists:get_value(upgrade_version, TestMetaData, previous), - Nodes = [Node1|_] = rt_cluster:build_cluster([OldVsn, OldVsn, OldVsn, OldVsn]), + Nodes = [Node1|_] = rt:build_cluster([OldVsn, OldVsn, OldVsn, OldVsn]), verify_counter_converge:set_allow_mult_true(Nodes, ?COUNTER_BUCKET), populate_counters(Node1), @@ -52,11 +52,11 @@ populate_counters(Node) -> rt:wait_for_service(Node, riak_kv), ?assertEqual(ok, rt:wait_until(Node, fun has_counter_capability/1)), - RHC = rt_http:httpc(Node), + RHC = rt:httpc(Node), ?assertMatch(ok, rhc:counter_incr(RHC, ?COUNTER_BUCKET, <<"httpkey">>, 2)), ?assertMatch({ok, 2}, rhc:counter_val(RHC, ?COUNTER_BUCKET, <<"httpkey">>)), - PBC = rt_pb:pbc(Node), + PBC = rt:pbc(Node), ?assertEqual(ok, riakc_pb_socket:counter_incr(PBC, ?COUNTER_BUCKET, <<"pbkey">>, 4)), ?assertEqual({ok, 4}, riakc_pb_socket:counter_val(PBC, ?COUNTER_BUCKET, <<"pbkey">>)), ok. @@ -65,10 +65,10 @@ populate_counters(Node) -> %% check that you can get via default bucket verify_counters(Node) -> lager:info("Verifying counters on ~p", [Node]), - RHC = rt_http:httpc(Node), + RHC = rt:httpc(Node), ?assertMatch({ok, 4}, rhc:counter_val(RHC, ?COUNTER_BUCKET, <<"pbkey">>)), - PBC = rt_pb:pbc(Node), + PBC = rt:pbc(Node), ?assertEqual({ok, 2}, riakc_pb_socket:counter_val(PBC, ?COUNTER_BUCKET, <<"httpkey">>)), %% Check that 1.4 counters work with bucket types diff --git a/tests/verify_dvv_repl.erl b/tests/verify_dvv_repl.erl index 06c7edd12..c7354d3a5 100644 --- a/tests/verify_dvv_repl.erl +++ b/tests/verify_dvv_repl.erl @@ -97,10 +97,10 @@ make_replicate_test_fun(From, To) -> make_clusters() -> Conf = [{riak_repl, [{fullsync_on_connect, false}, {fullsync_interval, disabled}]}, - {riak_core, [{default_bucket_props, - [{dvv_enabled, true}, - {allow_mult, true}]}]}], - Nodes = rt_cluster:deploy_nodes(6, Conf), + {riak_core, [{default_bucket_props, + [{dvv_enabled, true}, + {allow_mult, true}]}]}], + Nodes = rt:deploy_nodes(6, Conf, [riak_kv, riak_repl]), {ClusterA, ClusterB} = lists:split(3, Nodes), A = make_cluster(ClusterA, "A"), B = make_cluster(ClusterB, "B"), @@ -110,7 +110,7 @@ make_cluster(Nodes, Name) -> repl_util:make_cluster(Nodes), repl_util:name_cluster(hd(Nodes), Name), repl_util:wait_until_leader_converge(Nodes), - C = rt_pb:pbc(hd(Nodes)), + C = rt:pbc(hd(Nodes)), riakc_pb_socket:set_options(C, [queue_if_disconnected]), {C, Nodes}. diff --git a/tests/verify_dynamic_ring.erl b/tests/verify_dynamic_ring.erl index 368c95a51..afd87f267 100644 --- a/tests/verify_dynamic_ring.erl +++ b/tests/verify_dynamic_ring.erl @@ -31,17 +31,17 @@ confirm() -> %% test requires allow_mult=false b/c of rt:systest_read - rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), - rt_config:update_app_config(all, [{riak_core, + rt:set_conf(all, [{"buckets.default.allow_mult", "false"}]), + rt:update_app_config(all, [{riak_core, [{ring_creation_size, ?START_SIZE}]}]), - [ANode, AnotherNode, YetAnother, _ReplacingNode] = _AllNodes = rt_cluster:deploy_nodes(4), + [ANode, AnotherNode, YetAnother, _ReplacingNode] = _AllNodes = rt:deploy_nodes(4), NewNodes = Nodes = [ANode, AnotherNode, YetAnother], %% This assignment for `NewNodes' is commented until riak_core %% issue #570 is resolved %% NewNodes = [ANode, YetAnother, ReplacingNode], - rt_node:join(AnotherNode, ANode), - rt_node:join(YetAnother, ANode), - rt_node:wait_until_nodes_agree_about_ownership(Nodes), + rt:join(AnotherNode, ANode), + rt:join(YetAnother, ANode), + rt:wait_until_nodes_agree_about_ownership(Nodes), rt:wait_until_ring_converged(Nodes), rt:wait_until_no_pending_changes(Nodes), @@ -52,14 +52,14 @@ confirm() -> wait_until_extra_proxies_shutdown(Nodes), lager:info("writing 500 keys"), - ?assertEqual([], rt_systest:write(ANode, 1, 500, ?BUCKET, ?W)), + ?assertEqual([], rt:systest_write(ANode, 1, 500, ?BUCKET, ?W)), test_resize(?SHRUNK_SIZE, ?START_SIZE, ANode, Nodes, {501, 750}), lager:info("verifying previously written data"), - ?assertEqual([], rt_systest:read(ANode, 1, 500, ?BUCKET, ?R)), + ?assertEqual([], rt:systest_read(ANode, 1, 500, ?BUCKET, ?R)), test_resize(?START_SIZE, ?EXPANDED_SIZE, ANode, Nodes), lager:info("verifying previously written data"), - ?assertEqual([], rt_systest:read(ANode, 1, 750, ?BUCKET, ?R)), + ?assertEqual([], rt:systest_read(ANode, 1, 750, ?BUCKET, ?R)), %% This following test code for force-replace is commented until %% riak_core issue #570 is resolved. At that time the preceding 3 @@ -82,11 +82,11 @@ confirm() -> %% rt:wait_until_no_pending_changes(NewNodes), %% assert_ring_size(?EXPANDED_SIZE, NewNodes), %% lager:info("verifying written data"), - %% ?assertEqual([], rt_systest:read(ANode, 1, 750, ?BUCKET, ?R)), + %% ?assertEqual([], rt:systest_read(ANode, 1, 750, ?BUCKET, ?R)), test_resize(?EXPANDED_SIZE, ?SHRUNK_SIZE, ANode, NewNodes), lager:info("verifying written data"), - ?assertEqual([], rt_systest:read(ANode, 1, 750, ?BUCKET, ?R)), + ?assertEqual([], rt:systest_read(ANode, 1, 750, ?BUCKET, ?R)), wait_until_extra_vnodes_shutdown(NewNodes), wait_until_extra_proxies_shutdown(NewNodes), @@ -110,7 +110,7 @@ confirm() -> rt:wait_until_ring_converged(NewNodes), assert_ring_size(?SHRUNK_SIZE, NewNodes), lager:info("verifying written data"), - ?assertEqual([], rt_systest:read(ANode, 1, 750, ?BUCKET, ?R)), + ?assertEqual([], rt:systest_read(ANode, 1, 750, ?BUCKET, ?R)), pass. @@ -137,7 +137,7 @@ write_during_resize(_, Start, End) when Start =:= undefined orelse End =:= undef write_during_resize(Node, Start, End) -> Pid = self(), spawn(fun() -> - case rt_systest:write(Node, Start, End, ?BUCKET, ?W) of + case rt:systest_write(Node, Start, End, ?BUCKET, ?W) of [] -> Pid ! done_writing; Ers -> @@ -151,7 +151,7 @@ verify_write_during_resize(Node, Start, End) -> receive done_writing -> lager:info("verifying data written during operation"), - ?assertEqual([], rt_systest:read(Node, Start, End, ?BUCKET, ?R)), + ?assertEqual([], rt:systest_read(Node, Start, End, ?BUCKET, ?R)), ok; {errors_writing, Ers} -> lager:error("errors were encountered while writing during operation: ~p", [Ers]), diff --git a/tests/verify_handoff.erl b/tests/verify_handoff.erl index 17db94137..211a14167 100644 --- a/tests/verify_handoff.erl +++ b/tests/verify_handoff.erl @@ -143,4 +143,4 @@ deploy_test_nodes(true, N) -> {handoff_acksync_threshold, 20}, {handoff_receive_timeout, 2000}, {gossip_limit, {10000000, 60000}}]}], - rt:deploy_nodes(N, Config). \ No newline at end of file + rt:deploy_nodes(N, Config). diff --git a/tests/verify_handoff_mixed.erl b/tests/verify_handoff_mixed.erl index dbf071074..9439109c4 100644 --- a/tests/verify_handoff_mixed.erl +++ b/tests/verify_handoff_mixed.erl @@ -183,4 +183,4 @@ find_line(Port, {ok, Data}) -> find_line(Port, file:read_line(Port)) end; find_line(_, _) -> - []. \ No newline at end of file + []. diff --git a/tests/verify_kv_health_check.erl b/tests/verify_kv_health_check.erl index 861157bfd..ebca92d1e 100644 --- a/tests/verify_kv_health_check.erl +++ b/tests/verify_kv_health_check.erl @@ -22,7 +22,7 @@ -export([confirm/0]). confirm() -> - [Node1, Node2, _Node3] = rt_cluster:build_cluster(3), + [Node1, Node2, _Node3] = rt:build_cluster(3), %% add intercept that delays handling of vnode commands %% on a single node (the "slow" node) @@ -43,7 +43,7 @@ confirm() -> %% make DisableThreshold+5 requests and trigger the health check explicitly %% we only need to backup one vnode's msg queue on the node to fail the health check %% so we read the same key again and again - C = rt_pb:pbc(Node2), + C = rt:pbc(Node2), [riakc_pb_socket:get(C, <<"b">>, <<"k">>) || _ <- lists:seq(1,DisableThreshold+5)], ok = rpc:call(Node1, riak_core_node_watcher, check_health, [riak_kv]), diff --git a/tests/verify_link_walk_urls.erl b/tests/verify_link_walk_urls.erl index ba174b875..ff5fcb044 100644 --- a/tests/verify_link_walk_urls.erl +++ b/tests/verify_link_walk_urls.erl @@ -33,7 +33,7 @@ confirm() -> - [Node0 | _] = rt_cluster:build_cluster(?NUM_NODES), + [Node0 | _] = rt:build_cluster(?NUM_NODES), Pbc = rt:pbc(Node0), lager:info("Inserting linked graph"), diff --git a/tests/verify_listkeys.erl b/tests/verify_listkeys.erl index 8842f990a..33af44937 100644 --- a/tests/verify_listkeys.erl +++ b/tests/verify_listkeys.erl @@ -138,11 +138,11 @@ list_keys_for_undefined_bucket_type(Node, Interface, Bucket, Attempt, ShouldPass lager:info("Listing keys using undefined bucket type ~p on ~p using ~p. Attempt #~p", [?UNDEFINED_BUCKET_TYPE, Node, Interface, Attempt]), case ShouldPass of - true -> ok; - _ -> - {Status, Message} = Mod:list_keys(Pid, { ?UNDEFINED_BUCKET_TYPE, Bucket }), - ?assertEqual(error, Status), - ?assertEqual(<<"No bucket-type named '880bf69d-5dab-44ee-8762-d24c6f759ce1'">>, Message) + true -> ok; + _ -> + {Status, Message} = Mod:list_keys(Pid, { ?UNDEFINED_BUCKET_TYPE, Bucket }), + ?assertEqual(error, Status), + ?assertEqual(<<"No bucket-type named '880bf69d-5dab-44ee-8762-d24c6f759ce1'">>, Message) end, case Interface of @@ -195,30 +195,30 @@ list_buckets(Node, Interface, Attempt, Num, ShouldPass) -> list_buckets_for_undefined_bucket_type(Node, Interface, Attempt, ShouldPass) -> case Interface of - pbc -> - Pid = rt:pbc(Node), - Mod = riakc_pb_socket; - http -> - Pid = rt:httpc(Node), - Mod = rhc + pbc -> + Pid = rt:pbc(Node), + Mod = riakc_pb_socket; + http -> + Pid = rt:httpc(Node), + Mod = rhc end, lager:info("Listing buckets on ~p for undefined bucket type ~p using ~p. Attempt ~p.", - [Node, ?UNDEFINED_BUCKET_TYPE, Interface, Attempt]), + [Node, ?UNDEFINED_BUCKET_TYPE, Interface, Attempt]), case ShouldPass of - true -> ok; - _ -> - {Status, Message} = Mod:list_buckets(Pid, ?UNDEFINED_BUCKET_TYPE, []), - lager:info("Received status ~p and message ~p", [Status, Message]), - ?assertEqual(error, Status), - ?assertEqual(<<"No bucket-type named '880bf69d-5dab-44ee-8762-d24c6f759ce1'">>, Message) + true -> ok; + _ -> + {Status, Message} = Mod:list_buckets(Pid, ?UNDEFINED_BUCKET_TYPE, []), + lager:info("Received status ~p and message ~p", [Status, Message]), + ?assertEqual(error, Status), + ?assertEqual(<<"No bucket-type named '880bf69d-5dab-44ee-8762-d24c6f759ce1'">>, Message) end, case Interface of - pbc -> - riakc_pb_socket:stop(Pid); - _ -> ok + pbc -> + riakc_pb_socket:stop(Pid); + _ -> ok end. assert_equal(Expected, Actual) -> @@ -247,3 +247,4 @@ check_a_node(Node, Interface, ShouldPass) -> || Attempt <- [1,2,3] ], [list_buckets_for_undefined_bucket_type(Node, Interface, Attempt, ShouldPass) || Attempt <- [1,2,3] ]. + diff --git a/tests/verify_listkeys_eqcfsm.erl b/tests/verify_listkeys_eqcfsm.erl index d8ef9d672..d8f5f01ee 100644 --- a/tests/verify_listkeys_eqcfsm.erl +++ b/tests/verify_listkeys_eqcfsm.erl @@ -69,7 +69,7 @@ prop_test() -> [lager:info(" Command : ~p~n", [Cmd]) || Cmd <- Cmds], {H, _S, Res} = run_commands(?MODULE, Cmds, [{nodelist, Nodes}]), lager:info("======================== Ran commands"), - rt_cluster:clean_cluster(Nodes), + rt:clean_cluster(Nodes), aggregate(zip(state_names(H),command_names(Cmds)), equals(Res, ok)) end))). @@ -178,14 +178,14 @@ log_transition(S) -> %% Helpers %% ==================================================================== setup_cluster(NumNodes) -> - Nodes = rt_cluster:build_cluster(NumNodes), + Nodes = rt:build_cluster(NumNodes), ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), ?assertEqual(ok, rt:wait_until_transfers_complete(Nodes)), Node = hd(Nodes), [begin - rt_bucket_types:create_and_activate_bucket_type(Node, BucketType, [{n_val, NVal}]), - rt_bucket_types:wait_until_bucket_type_status(BucketType, active, Nodes), - rt_bucket_types:wait_until_bucket_type_visible(Nodes, BucketType) + rt:create_and_activate_bucket_type(Node, BucketType, [{n_val, NVal}]), + rt:wait_until_bucket_type_status(BucketType, active, Nodes), + rt:wait_until_bucket_type_visible(Nodes, BucketType) end || {BucketType, NVal} <- bucket_types()], Nodes. @@ -229,7 +229,7 @@ node_list(NumNodes) -> put_keys(Node, Bucket, Num) -> lager:info("*******************[CMD] Putting ~p keys into bucket ~p on node ~p", [Num, Bucket, Node]), - Pid = rt_pb:pbc(Node), + Pid = rt:pbc(Node), try Keys = [list_to_binary(["", integer_to_list(Ki)]) || Ki <- lists:seq(0, Num - 1)], Vals = [list_to_binary(["", integer_to_list(Ki)]) || Ki <- lists:seq(0, Num - 1)], diff --git a/tests/verify_mr_prereduce_node_down.erl b/tests/verify_mr_prereduce_node_down.erl index d232946fe..265dd7a29 100644 --- a/tests/verify_mr_prereduce_node_down.erl +++ b/tests/verify_mr_prereduce_node_down.erl @@ -44,19 +44,19 @@ confirm() -> NodeCount = 4, lager:info("Build ~b-node cluster", [NodeCount]), - [Primary,ToKill|_] = rt_cluster:build_cluster(NodeCount), + [Primary,ToKill|_] = rt:build_cluster(NodeCount), %% We need one node down for this test - rt_node:stop(ToKill), + rt:stop(ToKill), %% store our test data Bucket = <<"verify_mr_prereduce_node_down">>, ObjCount = 100, lager:info("Loading ~b objects of test data", [ObjCount]), - [] = rt_systest:write(Primary, 1, ObjCount, Bucket, 3), + [] = rt:systest_write(Primary, 1, ObjCount, Bucket, 3), %% run the query a bunch - C = rt_pb:pbc(Primary), + C = rt:pbc(Primary), TestCount = 100, lager:info("Running the MR query ~b times", [TestCount]), Runs = [ run_query(C, Bucket) || _ <- lists:seq(1, TestCount) ], diff --git a/tests/verify_no_writes_on_read.erl b/tests/verify_no_writes_on_read.erl index 5b48ed048..ca4e95d31 100644 --- a/tests/verify_no_writes_on_read.erl +++ b/tests/verify_no_writes_on_read.erl @@ -11,7 +11,7 @@ confirm() -> Backend = proplists:get_value(backend, riak_test_runner:metadata()), lager:info("Running with backend ~p", [Backend]), ?assertEqual(bitcask, Backend), - [Node1 | _Rest] = _Nodes = rt_cluster:build_cluster(?NUM_NODES), + [Node1 | _Rest] = _Nodes = rt:build_cluster(?NUM_NODES), PBC = rt:pbc(Node1), lager:info("Setting last write wins on bucket"), B = ?BUCKET, diff --git a/tests/verify_object_limits.erl b/tests/verify_object_limits.erl index affce14f3..fd8af35dc 100644 --- a/tests/verify_object_limits.erl +++ b/tests/verify_object_limits.erl @@ -127,4 +127,4 @@ verify_sibling_limits(C, Node1) -> Res = riakc_pb_socket:put(C, O), lager:info("Result when too many siblings : ~p", [Res]), ?assertMatch({error,_}, Res), - ok. \ No newline at end of file + ok. diff --git a/tests/verify_reset_bucket_props.erl b/tests/verify_reset_bucket_props.erl index 175ebcbdc..1c099a81c 100644 --- a/tests/verify_reset_bucket_props.erl +++ b/tests/verify_reset_bucket_props.erl @@ -29,7 +29,7 @@ confirm() -> %% we will be using two of the nodes to perform an %% update and then a reset (one on each node) of a bucket's properties. %% All nodes are checked to make sure the reset is affected on them - [Node1, Node2, Node3] = Nodes = rt_cluster:build_cluster(3), + [Node1, Node2, Node3] = Nodes = rt:build_cluster(3), DefaultProps = get_current_bucket_props(Nodes, ?BUCKET), @@ -43,7 +43,7 @@ confirm() -> update_props(DefaultProps, Node1, Nodes), - C = rt_pb:pbc(Node3), + C = rt:pbc(Node3), lager:info("Resetting bucket properties for bucket ~p on node ~p via pbc", [?BUCKET, Node3]), ok = riakc_pb_socket:reset_bucket(C, ?BUCKET), diff --git a/tests/verify_riak_lager.erl b/tests/verify_riak_lager.erl index a11b6a03e..98505538b 100644 --- a/tests/verify_riak_lager.erl +++ b/tests/verify_riak_lager.erl @@ -29,13 +29,13 @@ confirm() -> lager:info("Staring a node"), - Nodes = [Node] = rt_cluster:deploy_nodes(1), + Nodes = [Node] = rt:deploy_nodes(1), ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), lager:info("Stopping that node"), - rt_node:stop(Node), + rt:stop(Node), - rt_node:start(Node), + rt:start(Node), lager:info("Checking for log files"), {ok, LagerHandlers} = rt:rpc_get_env(Node, [{lager, handlers}]), diff --git a/tests/verify_riak_object_reformat.erl b/tests/verify_riak_object_reformat.erl index 7e7a29bd7..ca44f9668 100644 --- a/tests/verify_riak_object_reformat.erl +++ b/tests/verify_riak_object_reformat.erl @@ -31,16 +31,16 @@ -define(N, 3). confirm() -> - rt_config:update_app_config(all, [{riak_kv, [{object_format, v1}]}]), + rt:update_app_config(all, [{riak_kv, [{object_format, v1}]}]), TestMetaData = riak_test_runner:metadata(), DowngradeVsn = proplists:get_value(upgrade_version, TestMetaData, previous), - Nodes = [Node1|_] = rt_cluster:build_cluster(?N), + Nodes = [Node1|_] = rt:build_cluster(?N), [rt:wait_until_capability(N, {riak_kv, object_format}, v1, v0) || N <- Nodes], lager:info("Writing 100 keys in format v1 to ~p", [Node1]), - rt_systest:write(Node1, 100, ?N), - ?assertEqual([], rt_systest:read(Node1, 100, ?N)), + rt:systest_write(Node1, 100, ?N), + ?assertEqual([], rt:systest_read(Node1, 100, ?N)), lager:info("100 keys successfully written to ~p", [Node1]), %% TODO: introduce some handoff @@ -51,7 +51,7 @@ confirm() -> rt:upgrade(Node, DowngradeVsn), %% use upgrade to downgrade rt:wait_for_service(Node, riak_kv), lager:info("Ensuring keys still readable on ~p", [Node]), - ?assertEqual([], rt_systest:read(Node, 100, ?N)) + ?assertEqual([], rt:systest_read(Node, 100, ?N)) end || Node <- Nodes], pass. diff --git a/tests/verify_riak_stats.erl b/tests/verify_riak_stats.erl index e7f5c1727..53ae72d1e 100644 --- a/tests/verify_riak_stats.erl +++ b/tests/verify_riak_stats.erl @@ -169,32 +169,32 @@ get_console_stats(Node) -> %% Temporary workaround: use os:cmd/1 when in 'rtdev' (needs some cheats %% in order to find the right path etc.) try - Stats = - case rt_config:get(rt_harness) of - rtdev -> - N = rtdev:node_id(Node), - Path = rtdev:relpath(rtdev:node_version(N)), - Cmd = rtdev:riak_admin_cmd(Path, N, ["status"]), - lager:info("Cmd = ~p~n", [Cmd]), - os:cmd(Cmd); - _ -> - rt:admin(Node, "status") - end, - [S || {_,_} = S <- - [list_to_tuple(re:split(L, " : ", [])) - || L <- tl(tl(string:tokens(Stats, "\n")))]] + Stats = + case rt_config:get(rt_harness) of + rtdev -> + N = rtdev:node_id(Node), + Path = rtdev:relpath(rtdev:node_version(N)), + Cmd = rtdev:riak_admin_cmd(Path, N, ["status"]), + lager:info("Cmd = ~p~n", [Cmd]), + os:cmd(Cmd); + _ -> + rt:admin(Node, "status") + end, + [S || {_,_} = S <- + [list_to_tuple(re:split(L, " : ", [])) + || L <- tl(tl(string:tokens(Stats, "\n")))]] catch - error:Reason -> - lager:info("riak-admin status ERROR: ~p~n~p~n", - [Reason, erlang:get_stacktrace()]), - [] + error:Reason -> + lager:info("riak-admin status ERROR: ~p~n~p~n", + [Reason, erlang:get_stacktrace()]), + [] end. compare_http_and_console_stats(Stats1, Stats2) -> OnlyInHttp = [S || {K,_} = S <- Stats1, - not lists:keymember(K, 1, Stats2)], + not lists:keymember(K, 1, Stats2)], OnlyInAdmin = [S || {K,_} = S <- Stats2, - not lists:keymember(K, 1, Stats1)], + not lists:keymember(K, 1, Stats1)], maybe_log_stats_keys(OnlyInHttp, "Keys missing from riak-admin"), maybe_log_stats_keys(OnlyInAdmin, "Keys missing from HTTP"), ?assertEqual([], OnlyInHttp), diff --git a/tests/verify_search.erl b/tests/verify_search.erl index 3267adb69..2d98ebb32 100644 --- a/tests/verify_search.erl +++ b/tests/verify_search.erl @@ -65,4 +65,4 @@ test_dirs(BaseDir) -> %% @todo Figure out why this one is not run by run_all.sh %% It does fail in a weird way if included SubDir /= "replication_test", - filelib:is_file(filename:join([BaseDir, SubDir, "script.def"]))]. \ No newline at end of file + filelib:is_file(filename:join([BaseDir, SubDir, "script.def"]))]. diff --git a/tests/verify_secondary_index_reformat.erl b/tests/verify_secondary_index_reformat.erl index 237b57109..1defc4ec0 100644 --- a/tests/verify_secondary_index_reformat.erl +++ b/tests/verify_secondary_index_reformat.erl @@ -24,7 +24,7 @@ -include_lib("riakc/include/riakc.hrl"). confirm() -> - [Node] = rt_cluster:build_cluster([legacy]), + [Node] = rt:build_cluster([legacy]), rt:wait_until_nodes_ready([Node]), check_fixed_index_statuses(Node, undefined), @@ -37,7 +37,7 @@ confirm() -> %% write key with index that old version of sext would encode improperly (not perserving %% sort order) lager:info("writing test key"), - Client0 = rt_pb:pbc(Node), + Client0 = rt:pbc(Node), Obj0 = riakc_obj:new(TestBucket, TestKey, <<"somevalue">>), ObjMD0 = riakc_obj:get_update_metadata(Obj0), ObjMD1 = riakc_obj:set_secondary_index(ObjMD0, @@ -57,7 +57,7 @@ confirm() -> %% should rewrite 1 index (* n = 3), ignore 0 and have zero errors {3, 0, 0} = rpc:call(Node, riak_kv_util, fix_incorrect_index_entries, []), - Client1 = rt_pb:pbc(Node), + Client1 = rt:pbc(Node), Results = riakc_pb_socket:get_index(Client1, TestBucket, TestIndex, 1000000000000, TestIdxValue), @@ -70,11 +70,11 @@ confirm() -> %% write some more data (make sure flag doesn't "roll back" on restart lager:info("writing some more data"), - rt_systest:write(Node, 10, 1), + rt:systest_write(Node, 10, 1), lager:info("restarting node"), - rt_node:stop_and_wait(Node), - rt_node:start(Node), + rt:stop_and_wait(Node), + rt:start(Node), rt:wait_for_service(Node, riak_kv), check_fixed_index_statuses(Node, true), @@ -84,8 +84,8 @@ confirm() -> check_fixed_index_statuses(Node, false), - rt_node:stop_and_wait(Node), - rt_node:start(Node), + rt:stop_and_wait(Node), + rt:start(Node), rt:wait_for_service(Node, riak_kv), check_fixed_index_statuses(Node, false), diff --git a/tests/verify_snmp.erl b/tests/verify_snmp.erl index aba9b8734..639972fcf 100644 --- a/tests/verify_snmp.erl +++ b/tests/verify_snmp.erl @@ -27,8 +27,8 @@ confirm() -> %% Bring up a small cluster - %% Config = [{riak_snmp, [{polling_interval, 1000}]}], - [Node1] = rt_cluster:deploy_nodes(1), + Config = [{riak_snmp, [{polling_interval, 1000}]}], + [Node1] = rt:deploy_nodes(1, Config), ?assertEqual(ok, rt:wait_until_nodes_ready([Node1])), Keys = [{vnodeGets,<<"vnode_gets">>}, @@ -60,8 +60,8 @@ confirm() -> lager:info("Doing some reads and writes to record some stats."), - rt_systest:write(Node1, 10), - rt_systest:read(Node1, 10), + rt:systest_write(Node1, 10), + rt:systest_read(Node1, 10), lager:info("Waiting for HTTP Stats to be non-zero"), ?assertEqual(ok, diff --git a/tests/verify_staged_clustering.erl b/tests/verify_staged_clustering.erl index a61654445..f792c4800 100644 --- a/tests/verify_staged_clustering.erl +++ b/tests/verify_staged_clustering.erl @@ -180,4 +180,4 @@ commit_staged_rpc(Node) -> rpc:call(Node, riak_core_claimant, commit, []). clear_staged_rpc(Node) -> - rpc:call(Node, riak_core_claimant, clear, []). \ No newline at end of file + rpc:call(Node, riak_core_claimant, clear, []). diff --git a/tests/verify_tick_change.erl b/tests/verify_tick_change.erl index e31a956cc..3390fbac8 100644 --- a/tests/verify_tick_change.erl +++ b/tests/verify_tick_change.erl @@ -25,9 +25,9 @@ confirm() -> ClusterSize = 4, - rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), + rt:set_conf(all, [{"buckets.default.allow_mult", "false"}]), NewConfig = [], - Nodes = rt_cluster:build_cluster(ClusterSize, NewConfig), + Nodes = rt:build_cluster(ClusterSize, NewConfig), ?assertEqual(ok, rt:wait_until_nodes_ready(Nodes)), [Node1|_] = Nodes, Bucket = <<"systest">>, @@ -63,11 +63,11 @@ make_common() -> write_stuff(Nodes, Start, End, Bucket, W, Common) -> Nd = lists:nth(length(Nodes), Nodes), - [] = rt_systest:write(Nd, Start, End, Bucket, W, Common). + [] = rt:systest_write(Nd, Start, End, Bucket, W, Common). read_stuff(Nodes, Start, End, Bucket, W, Common) -> Nd = lists:nth(length(Nodes), Nodes), - [] = rt_systest:read(Nd, Start, End, Bucket, W, Common). + [] = rt:systest_read(Nd, Start, End, Bucket, W, Common). is_set_net_ticktime_done(Nodes, Time) -> case lists:usort([(catch rpc:call(Node, net_kernel, get_net_ticktime,[])) diff --git a/tests/verify_vclock.erl b/tests/verify_vclock.erl index 44ec38d9a..64e03ea1e 100644 --- a/tests/verify_vclock.erl +++ b/tests/verify_vclock.erl @@ -113,12 +113,12 @@ force_encoding(Node, EncodingMethod) -> } ], - rt_config:update_app_config(Node, OverrideData) + rt:update_app_config(Node, OverrideData) end. stopall(Nodes) -> - lists:foreach(fun(N) -> rt_node:brutal_kill(N) end, Nodes). + lists:foreach(fun(N) -> rt:brutal_kill(N) end, Nodes). make_kv(N, VSuffix) -> K = <>, @@ -134,10 +134,10 @@ our_pbc_write(Node, Size, Suffix) -> our_pbc_write(Node, 1, Size, <<"systest">>, Suffix). our_pbc_write(Node, Start, End, Bucket, VSuffix) -> - PBC = rt_pb:pbc(Node), + PBC = rt:pbc(Node), F = fun(N, Acc) -> {K, V} = make_kv(N, VSuffix), - try rt_pb:pbc_write(PBC, Bucket, K, V) of + try rt:pbc_write(PBC, Bucket, K, V) of ok -> Acc; Other -> @@ -156,7 +156,7 @@ our_pbc_read(Node, Size, Suffix) -> our_pbc_read(Node, 1, Size, <<"systest">>, Suffix). our_pbc_read(Node, Start, End, Bucket, VSuffix) -> - PBC = rt_pb:pbc(Node), + PBC = rt:pbc(Node), %% Trundle along through the list, collecting mismatches: F = fun(N, Acc) -> @@ -193,12 +193,12 @@ our_pbc_read(Node, Start, End, Bucket, VSuffix) -> %% For some testing purposes, making these limits smaller is helpful: deploy_test_nodes(false, N) -> - rt_cluster:deploy_nodes(N); + rt:deploy_nodes(N); deploy_test_nodes(true, N) -> lager:info("NOTICE: Using turbo settings for testing."), Config = [{riak_core, [{forced_ownership_handoff, 8}, {handoff_concurrency, 8}, {vnode_inactivity_timeout, 1000}, {gossip_limit, {10000000, 60000}}]}], - rt_cluster:deploy_nodes(N, Config). + rt:deploy_nodes(N, Config). diff --git a/tests/verify_vclock_encoding_upgrade.erl b/tests/verify_vclock_encoding_upgrade.erl index 110951099..e8614e83b 100644 --- a/tests/verify_vclock_encoding_upgrade.erl +++ b/tests/verify_vclock_encoding_upgrade.erl @@ -23,7 +23,7 @@ confirm() -> lager:info("Deploying previous cluster"), - [Prev, Current] = rt_cluster:build_cluster([previous, current]), + [Prev, Current] = rt:build_cluster([previous, current]), PrevClient = rt:pbc(Prev), CurrentClient = rt:pbc(Current), K = <<"key">>, diff --git a/tests/yz_crdt.erl b/tests/yz_crdt.erl index ff15b1143..b79fe49d2 100644 --- a/tests/yz_crdt.erl +++ b/tests/yz_crdt.erl @@ -66,4 +66,4 @@ confirm() -> %% Clean cluster. rt:clean_cluster(Nodes), - pass. \ No newline at end of file + pass. diff --git a/tests/yz_ensemble.erl b/tests/yz_ensemble.erl index b0e090af6..5b0361e58 100644 --- a/tests/yz_ensemble.erl +++ b/tests/yz_ensemble.erl @@ -26,7 +26,7 @@ confirm() -> Node = hd(Nodes), lager:info("Creating/activating 'strong' bucket type"), - rt_bucket_types:create_and_activate_bucket_type(Node, <<"strong">>, + rt:create_and_activate_bucket_type(Node, <<"strong">>, [{consistent, true}, {n_val, NVal}]), Bucket = {<<"strong">>, <<"test">>}, @@ -46,10 +46,10 @@ verify_ensemble_delete_support(Node, Bucket, Index) -> Keys = [<> || N <- lists:seq(1,2000), not lists:any(fun(E) -> E > 127 end,binary_to_list(<>))], - PBC = rt_pb:pbc(Node), + PBC = rt:pbc(Node), lager:info("Writing ~p keys", [length(Keys)]), - [ok = rt_pb:pbc_write(PBC, Bucket, Key, Key, "text/plain") || Key <- Keys], + [ok = rt:pbc_write(PBC, Bucket, Key, Key, "text/plain") || Key <- Keys], %% soft commit wait, then check that last key is indexed lager:info("Search for keys to verify they exist"), @@ -88,10 +88,10 @@ verify_ensemble_delete_support(Node, Bucket, Index) -> %% node when adding yokozuna and ensemble support. Waiting for yokozuna %% to load on each node allows join_cluster to complete consistently build_cluster_with_yz_support(Num, Config, NVal) -> - Nodes = rt_cluster:deploy_nodes(Num, Config), + Nodes = rt:deploy_nodes(Num, Config), [rt:wait_for_cluster_service([N], yokozuna) || N <- Nodes], Node = hd(Nodes), - rt_cluster:join_cluster(Nodes), + rt:join_cluster(Nodes), ensemble_util:wait_until_cluster(Nodes), ensemble_util:wait_for_membership(Node), ensemble_util:wait_until_stable(Node, NVal), From a57a0b92c21cbcb4e7b2b2efad2f6c79925d647e Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Sun, 19 Apr 2015 16:33:25 -0600 Subject: [PATCH 131/157] Get GiddyUp consuming riak_test.log - Have lager fire up an extra handler for each test run - Place lager output in riak_test.log and upload to GiddyUp - Remove rt.hrl and move the rt_webhook record into giddyup.erl - Remove unused GiddyUp code from escript - Let the test_runner know the name of the log directory so it knows where to put the riak_test.log - Add separate flag to rt_reporter to indicate the need to upload files to GiddyUp - Always copy log files to local directory before uploading to GiddyUp --- include/rt.hrl | 25 --------------- src/giddyup.erl | 7 ++++- src/riak_test_escript.erl | 45 +-------------------------- src/riak_test_executor.erl | 23 +++++++------- src/riak_test_runner.erl | 42 ++++++++++++++++++-------- src/rt.erl | 9 +++--- src/rt2.erl | 8 ++--- src/rt_aae.erl | 2 -- src/rt_backend.erl | 1 - src/rt_bucket_types.erl | 2 -- src/rt_cluster.erl | 2 -- src/rt_cmd_line.erl | 2 -- src/rt_harness.erl | 6 ++-- src/rt_http.erl | 1 - src/rt_node.erl | 1 - src/rt_pb.erl | 1 - src/rt_properties.erl | 2 -- src/rt_properties2.erl | 2 -- src/rt_reporter.erl | 62 ++++++++++++++++++++++---------------- src/rt_ring.erl | 1 - src/rt_systest.erl | 1 - src/rt_test_plan.erl | 2 -- src/rtdev.erl | 42 ++++++++++++++++++-------- 23 files changed, 125 insertions(+), 164 deletions(-) delete mode 100644 include/rt.hrl diff --git a/include/rt.hrl b/include/rt.hrl deleted file mode 100644 index 78de7e028..000000000 --- a/include/rt.hrl +++ /dev/null @@ -1,25 +0,0 @@ -%% ------------------------------------------------------------------- -%% -%% 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. -%% -%% ------------------------------------------------------------------- - --record(rt_webhook, { - name :: string(), - url :: string(), - headers=[] :: [{atom(), string()}] - }). diff --git a/src/giddyup.erl b/src/giddyup.erl index 5a5f212a7..c09209cff 100644 --- a/src/giddyup.erl +++ b/src/giddyup.erl @@ -21,7 +21,12 @@ -export([get_suite/1, post_result/1, post_artifact/2]). -define(STREAM_CHUNK_SIZE, 8192). --include("rt.hrl"). + +-record(rt_webhook, { + name :: string(), + url :: string(), + headers=[] :: [{atom(), string()}] +}). -spec get_suite(string()) -> [{atom(), term()}]. get_suite(Platform) -> diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index 907bb8805..c249bd2f1 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -20,7 +20,6 @@ %% @private -module(riak_test_escript). --include("rt.hrl"). %% TODO: Temporary build workaround, remove!! -compile(export_all). -export([main/1]). @@ -426,49 +425,7 @@ match_group_attributes(Attributes, Groups) -> %% TestCount) || %% {Test, TestMetaData} <- Tests], -publish_report(_SingleTestResult, _CoverFile, undefined) -> - ok; -publish_report(SingleTestResult, CoverFile, _Report) -> - {value, {log, Log}, TestResult} = lists:keytake(log, 1, SingleTestResult), - publish_artifacts(TestResult, - Log, - CoverFile, - giddyup:post_result(TestResult)). - -publish_artifacts(_TestResult, _Log, _CoverFile, error) -> - whoomp; %% there it is -publish_artifacts(TestResult, Log, CoverFile, {ok, Base}) -> - %% Now push up the artifacts, starting with the test log - giddyup:post_artifact(Base, {"riak_test.log", Log}), - [giddyup:post_artifact(Base, File) || File <- rt:get_node_logs()], - post_cover_artifact(Base, CoverFile), - ResultPlusGiddyUp = TestResult ++ [{giddyup_url, list_to_binary(Base)}], - [rt:post_result(ResultPlusGiddyUp, WebHook) || WebHook <- get_webhooks()]. - -post_cover_artifact(_Base, cover_disabled) -> - ok; -post_cover_artifact(Base, CoverFile) -> - CoverArchiveName = filename:basename(CoverFile) ++ ".gz", - CoverArchive = zlib:gzip(element(2, file:read_file(CoverFile))), - giddyup:post_artifact(Base, {CoverArchiveName, CoverArchive}). - -get_webhooks() -> - Hooks = lists:foldl(fun(E, Acc) -> [parse_webhook(E) | Acc] end, - [], - rt_config:get(webhooks, [])), - lists:filter(fun(E) -> E =/= undefined end, Hooks). - -parse_webhook(Props) -> - Url = proplists:get_value(url, Props), - case is_list(Url) of - true -> - #rt_webhook{url= Url, - name=proplists:get_value(name, Props, "Webhook"), - headers=proplists:get_value(headers, Props, [])}; - false -> - lager:error("Invalid configuration for webhook : ~p", Props), - undefined - end. + backend_list(Backend) when is_atom(Backend) -> atom_to_list(Backend); diff --git a/src/riak_test_executor.erl b/src/riak_test_executor.erl index 514d11224..12b2865ce 100644 --- a/src/riak_test_executor.erl +++ b/src/riak_test_executor.erl @@ -63,11 +63,11 @@ init([Tests, LogDir, Platform, UpgradeList, NotifyPid]) -> ContinueOnFail = rt_config:get(continue_on_fail), - LogLocation = case Platform of - undefined -> LogDir; - _ -> giddyup + UploadToGiddyUp = case Platform of + undefined -> false; + _ -> true end, - {ok, Reporter} = rt_reporter:start_link(LogLocation, NotifyPid), + {ok, Reporter} = rt_reporter:start_link(UploadToGiddyUp, LogDir, NotifyPid), lager:notice("Starting the Riak Test executor in ~p execution mode", [ExecutionMode]), State = #state{pending_tests=Tests, @@ -169,14 +169,15 @@ launch_test({nodes, Nodes, NodeMap}, State) -> runner_pids=Pids, running_tests=Running, continue_on_fail=ContinueOnFail, - reporter_pid=ReporterPid} = State, + reporter_pid=ReporterPid, + log_dir=LogDir} = State, NextTestModule = rt_test_plan:get_module(NextTestPlan), lager:debug("Executing test ~p in mode ~p", [NextTestModule, ExecutionMode]), {NextTestPlan, TestProps} = lists:keyfind(NextTestPlan, 1, PropertiesList), UpdTestProps = rt_properties:set([{node_map, NodeMap}, {node_ids, Nodes}], TestProps), {RunnerPids, RunningTests} = run_test(ExecutionMode, NextTestPlan, UpdTestProps, - Pids, Running, ContinueOnFail, ReporterPid), + Pids, Running, ContinueOnFail, ReporterPid, LogDir), UpdState = State#state{pending_tests=RestPending, execution_mode=ExecutionMode, runner_pids=RunnerPids, @@ -318,11 +319,11 @@ override_props(State) -> [{upgrade_path, UpgradeList}] end. --spec run_test(parallel | serial, atom(), proplists:proplist(), [pid()], [rt_test_plan:test_plan()], boolean(), pid()) -> {[pid()], [atom()]}. -run_test(parallel, TestPlan, Properties, RunningPids, RunningTests, ContinueOnFail, ReporterPid) -> - Pid = spawn_link(riak_test_runner, start, [TestPlan, Properties, ContinueOnFail, ReporterPid]), +-spec run_test(parallel | serial, atom(), proplists:proplist(), [pid()], [rt_test_plan:test_plan()], boolean(), pid(), string()) -> {[pid()], [atom()]}. +run_test(parallel, TestPlan, Properties, RunningPids, RunningTests, ContinueOnFail, ReporterPid, LogDir) -> + Pid = spawn_link(riak_test_runner, start, [TestPlan, Properties, ContinueOnFail, ReporterPid, LogDir]), {[Pid | RunningPids], [TestPlan | RunningTests]}; -run_test(serial, TestPlan, Properties, RunningPids, RunningTests, ContinueOnFail, ReporterPid) -> - riak_test_runner:start(TestPlan, Properties, ContinueOnFail, ReporterPid), +run_test(serial, TestPlan, Properties, RunningPids, RunningTests, ContinueOnFail, ReporterPid, LogDir) -> + riak_test_runner:start(TestPlan, Properties, ContinueOnFail, ReporterPid, LogDir), {RunningPids, RunningTests}. diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index 8d476fea7..694c81005 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -24,7 +24,7 @@ -behavior(gen_fsm). %% API --export([start/4, +-export([start/5, send_event/2, stop/0]). @@ -48,7 +48,6 @@ terminate/3, code_change/4]). --include("rt.hrl"). -include_lib("eunit/include/eunit.hrl"). -type test_type() :: {new | old}. @@ -70,7 +69,8 @@ remaining_versions :: [string()], test_results :: [term()], continue_on_fail :: boolean(), - reporter_pid :: pid()}). + log_dir :: string(), + reporter_pids :: pid()}). -deprecated([{metadata,0,next_major_release}]). @@ -79,8 +79,8 @@ %%%=================================================================== %% @doc Start the test runner -start(TestPlan, Properties, ContinueOnFail, ReporterPid) -> - Args = [TestPlan, Properties, ContinueOnFail, ReporterPid], +start(TestPlan, Properties, ContinueOnFail, ReporterPids, LogDir) -> + Args = [TestPlan, Properties, ContinueOnFail, ReporterPids, LogDir], gen_fsm:start_link(?MODULE, Args, []). send_event(Pid, Msg) -> @@ -103,7 +103,7 @@ metadata() -> %% @doc Read the storage schedule and go to idle. %% compose_test_datum(Version, Project, undefined, undefined) -> -init([TestPlan, Properties, ContinueOnFail, ReporterPid]) -> +init([TestPlan, Properties, ContinueOnFail, ReporterPid, LogDir]) -> lager:debug("Started riak_test_runnner with pid ~p (continue on fail: ~p)", [self(), ContinueOnFail]), Project = list_to_binary(rt_config:get(rt_project, "undefined")), Backend = rt_test_plan:get(backend, TestPlan), @@ -144,7 +144,8 @@ init([TestPlan, Properties, ContinueOnFail, ReporterPid]) -> prereq_check=PreReqCheck, group_leader=group_leader(), continue_on_fail=ContinueOnFail, - reporter_pid=ReporterPid}, + reporter_pids=ReporterPid, + log_dir=LogDir}, {ok, setup, State, 0}. %% @doc there are no all-state events for this fsm @@ -219,12 +220,14 @@ setup(_Event, _State) -> ok. execute({nodes_deployed, _}, State) -> - #state{test_module=TestModule, + #state{test_plan=TestPlan, + test_module=TestModule, test_type=TestType, properties=Properties, setup_modfun=SetupModFun, confirm_modfun=ConfirmModFun, - test_timeout=TestTimeout} = State, + test_timeout=TestTimeout, + log_dir=OutDir} = State, lager:notice("Running ~s", [TestModule]), lager:notice("Properties: ~p", [Properties]), @@ -232,6 +235,7 @@ execute({nodes_deployed, _}, State) -> %% Perform test setup which includes clustering of the nodes if %% required by the test properties. The cluster information is placed %% into the properties record and returned by the `setup' function. + start_lager_backend(rt_test_plan:get_name(TestPlan), OutDir), SetupResult = maybe_setup_test(TestModule, TestType, SetupModFun, Properties), UpdState = maybe_execute_test(SetupResult, TestModule, TestType, ConfirmModFun, StartTime, State), @@ -451,10 +455,22 @@ function_name(FunName, TestModule, Arity, Default) when is_atom(TestModule) -> {Default, FunName} end. -%% remove_lager_backend() -> -%% gen_event:delete_handler(lager_event, lager_file_backend, []), -%% gen_event:delete_handler(lager_event, riak_test_lager_backend, []). +start_lager_backend(TestName, Outdir) -> + LogLevel = rt_config:get(lager_level, info), + case Outdir of + undefined -> ok; + _ -> + gen_event:add_handler(lager_event, lager_file_backend, + {filename:join([Outdir, TestName, "riak_test.log"]), + LogLevel, 10485760, "$D0", 1}), + lager:set_loglevel(lager_file_backend, LogLevel) + end, + gen_event:add_handler(lager_event, riak_test_lager_backend, [LogLevel, false]), + lager:set_loglevel(riak_test_lager_backend, LogLevel). +stop_lager_backend() -> + gen_event:delete_handler(lager_event, lager_file_backend, []), + gen_event:delete_handler(lager_event, riak_test_lager_backend, []). %% A return of `fail' must be converted to a non normal exit since %% status is determined by `rec_loop'. @@ -508,6 +524,8 @@ report_cleanup_and_notify(Result, CleanUp, State=#state{test_plan=TestPlan, ResultMessage = test_result_message(Result), rt_reporter:send_result(test_result({TestPlan, ResultMessage, Duration})), maybe_cleanup(CleanUp, State), + {ok, Logs} = stop_lager_backend(), + _Log = unicode:characters_to_binary(Logs), Notification = {test_complete, TestPlan, self(), ResultMessage}, riak_test_executor:send_event(Notification). diff --git a/src/rt.erl b/src/rt.erl index 3dc1c92d0..48dcb2be1 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -25,7 +25,6 @@ %% multiple independent tests. -module(rt). -deprecated(module). --include("rt.hrl"). -include_lib("eunit/include/eunit.hrl"). -compile(export_all). @@ -64,7 +63,7 @@ expect_in_log/2, get_deps/0, get_ip/1, - get_node_logs/1, + get_node_logs/3, get_replica/5, get_ring/1, get_version/0, @@ -1364,9 +1363,9 @@ setup_harness(_Test, _Args) -> %% @doc Copy all of the nodes' log files to a local dir or %% open a port to each file to upload to GiddyUp --spec(get_node_logs(string() | giddyup) -> list()). -get_node_logs(DestDir) -> - rt2:get_node_logs(DestDir). +-spec(get_node_logs(boolean(), string(), string()) -> list()). +get_node_logs(UploadToGiddyUp, LogFile, DestDir) -> + rt2:get_node_logs(UploadToGiddyUp, LogFile, DestDir). check_ibrowse() -> rt2:check_ibrowse(). diff --git a/src/rt2.erl b/src/rt2.erl index 7d9a32f05..60a086ed9 100644 --- a/src/rt2.erl +++ b/src/rt2.erl @@ -24,7 +24,6 @@ %% Please extend this module with new functions that prove useful between %% multiple independent tests. -module(rt2). --include("rt.hrl"). -include_lib("eunit/include/eunit.hrl"). -compile(export_all). @@ -39,7 +38,7 @@ expect_in_log/2, get_deps/0, get_ip/1, - get_node_logs/1, + get_node_logs/3, get_replica/5, get_version/0, is_mixed_cluster/1, @@ -611,8 +610,8 @@ setup_harness(Test, Args) -> %% @doc Downloads any extant log files from the harness's running %% nodes. -get_node_logs(DestDir) -> - rt_harness:get_node_logs(DestDir). +get_node_logs(UploadToGiddyUp, LogFile, DestDir) -> + rt_harness:get_node_logs(UploadToGiddyUp, LogFile, DestDir). check_ibrowse() -> try sys:get_status(ibrowse) of @@ -630,6 +629,7 @@ check_ibrowse() -> %%% Bucket Types Functions %%%=================================================================== +%% TODO: Determine if this can leverage riak_test_runner:start_lager_backend/2 %% @doc Set up in memory log capture to check contents in a test. setup_log_capture(Nodes) when is_list(Nodes) -> rt:load_modules_on_nodes([riak_test_lager_backend], Nodes), diff --git a/src/rt_aae.erl b/src/rt_aae.erl index 841a2f15d..170221103 100644 --- a/src/rt_aae.erl +++ b/src/rt_aae.erl @@ -23,8 +23,6 @@ -export([wait_until_aae_trees_built/1]). --include("rt.hrl"). - wait_until_aae_trees_built(Nodes) -> lager:info("Wait until AAE builds all partition trees across ~p", [Nodes]), BuiltFun = fun() -> lists:foldl(aae_tree_built_fun(), true, Nodes) end, diff --git a/src/rt_backend.erl b/src/rt_backend.erl index d31175223..949f26781 100644 --- a/src/rt_backend.erl +++ b/src/rt_backend.erl @@ -18,7 +18,6 @@ %% %% ------------------------------------------------------------------- -module(rt_backend). --include("rt.hrl"). -include_lib("eunit/include/eunit.hrl"). -compile(export_all). diff --git a/src/rt_bucket_types.erl b/src/rt_bucket_types.erl index 93b6ec878..9d07f8d1d 100644 --- a/src/rt_bucket_types.erl +++ b/src/rt_bucket_types.erl @@ -27,8 +27,6 @@ wait_until_bucket_type_status/3, wait_until_bucket_props/3]). --include("rt.hrl"). - %% Specify the bucket_types field for the properties record. The list %% of bucket types may have two forms, a bucket_type or a pair %% consisting of an integer and a bucket_type. The latter form diff --git a/src/rt_cluster.erl b/src/rt_cluster.erl index 39b7b8d9f..f521ad837 100644 --- a/src/rt_cluster.erl +++ b/src/rt_cluster.erl @@ -34,8 +34,6 @@ -export([maybe_wait_for_transfers/3]). --include("rt.hrl"). - %% @doc Default properties used if a riak_test module does not specify %% a custom properties function. -spec properties() -> rt_properties:properties(). diff --git a/src/rt_cmd_line.erl b/src/rt_cmd_line.erl index af592efeb..ec83c3ff9 100644 --- a/src/rt_cmd_line.erl +++ b/src/rt_cmd_line.erl @@ -31,8 +31,6 @@ console/2 ]). --include("rt.hrl"). - -define(HARNESS, (rt_config:get(rt_harness))). %% @doc Call 'bin/riak-admin' command on `Node' with arguments `Args' diff --git a/src/rt_harness.erl b/src/rt_harness.erl index 9bf3a7817..99f8636a8 100644 --- a/src/rt_harness.erl +++ b/src/rt_harness.erl @@ -37,7 +37,7 @@ cmd/2, setup/0, get_deps/0, - get_node_logs/1, + get_node_logs/3, get_version/0, get_version/1, get_backends/0, @@ -107,8 +107,8 @@ get_version(Node) -> get_backends() -> ?HARNESS_MODULE:get_backends(). -get_node_logs(DestDir) -> - ?HARNESS_MODULE:get_node_logs(DestDir). +get_node_logs(UploadToGiddyUp, LogFile, DestDir) -> + ?HARNESS_MODULE:get_node_logs(UploadToGiddyUp, LogFile, DestDir). set_backend(Backend) -> ?HARNESS_MODULE:set_backend(Backend). diff --git a/src/rt_http.erl b/src/rt_http.erl index 1941e38aa..7135de51b 100644 --- a/src/rt_http.erl +++ b/src/rt_http.erl @@ -12,7 +12,6 @@ %% %% ------------------------------------------------------------------- -module(rt_http). --include("rt.hrl"). -include_lib("eunit/include/eunit.hrl"). -export([http_url/1, diff --git a/src/rt_node.erl b/src/rt_node.erl index 88796627c..4cb68821b 100644 --- a/src/rt_node.erl +++ b/src/rt_node.erl @@ -18,7 +18,6 @@ %% %% ------------------------------------------------------------------- -module(rt_node). --include("rt.hrl"). -include_lib("eunit/include/eunit.hrl"). -export([start/2, diff --git a/src/rt_pb.erl b/src/rt_pb.erl index 75c893297..21cf14b00 100644 --- a/src/rt_pb.erl +++ b/src/rt_pb.erl @@ -18,7 +18,6 @@ %% %% ------------------------------------------------------------------- -module(rt_pb). --include("rt.hrl"). -include_lib("eunit/include/eunit.hrl"). -export([pbc/1, diff --git a/src/rt_properties.erl b/src/rt_properties.erl index 3ab343d9e..7b45c3a58 100644 --- a/src/rt_properties.erl +++ b/src/rt_properties.erl @@ -22,8 +22,6 @@ %% @doc Implements a set of functions for accessing and manipulating %% an `rt_properties' record. --include("rt.hrl"). - %%-record(rt_cluster_topology_v1, { %% name :: atom(), %% connected_to :: [atom()], diff --git a/src/rt_properties2.erl b/src/rt_properties2.erl index ac8610cd1..ee86add48 100644 --- a/src/rt_properties2.erl +++ b/src/rt_properties2.erl @@ -22,8 +22,6 @@ %% @doc Implements a set of functions for accessing and manipulating %% an `rt_properties2' record. --include("rt.hrl"). - -record(rt_cluster_topology_v1, { name :: atom(), connected_to :: [] | [atom()], diff --git a/src/rt_reporter.erl b/src/rt_reporter.erl index 74136fe0f..34a315048 100644 --- a/src/rt_reporter.erl +++ b/src/rt_reporter.erl @@ -32,7 +32,7 @@ -define(HEADER, [<<"Test">>, <<"Result">>, <<"Reason">>, <<"Test Duration">>]). %% API --export([start_link/2, +-export([start_link/3, stop/0, send_result/1]). @@ -50,8 +50,10 @@ % Collection of log files % Running summary of test results: {test, pass/fail, duration} summary :: list(), - log_dir :: string() | giddyup, - % PID of escript used to update results + log_dir :: string(), + %% True if results should be uploaded to GiddyUp + giddyup :: boolean(), + %% PID of escript used to update results notify_pid :: pid() }). @@ -65,10 +67,10 @@ %% %% @end %%-------------------------------------------------------------------- --spec(start_link(string() | giddyup, pid()) -> +-spec(start_link(boolean(), string(), pid()) -> {ok, Pid :: pid()} | ignore | {error, Reason :: term()}). -start_link(LogDir, NotifyPid) -> - gen_server:start_link({local, ?SERVER}, ?MODULE, [LogDir, NotifyPid], []). +start_link(UploadToGiddyUp, LogDir, NotifyPid) -> + gen_server:start_link({local, ?SERVER}, ?MODULE, [UploadToGiddyUp, LogDir, NotifyPid], []). %%-------------------------------------------------------------------- %% @doc @@ -116,9 +118,10 @@ send_result(Msg) -> -spec(init(Args :: term()) -> {ok, State :: #state{}} | {ok, State :: #state{}, timeout() | hibernate} | {stop, Reason :: term()} | ignore). -init([LogDir, NotifyPid]) -> +init([UploadToGiddyUp, LogDir, NotifyPid]) -> {ok, #state{summary=[], log_dir=LogDir, + giddyup=UploadToGiddyUp, notify_pid=NotifyPid}}. %%-------------------------------------------------------------------- @@ -137,13 +140,11 @@ init([LogDir, NotifyPid]) -> {stop, Reason :: term(), Reply :: term(), NewState :: #state{}} | {stop, Reason :: term(), NewState :: #state{}}). handle_call({test_result, Result}, From, State) -> - lager:debug("Sending test_result to ~p from ~p", [State#state.notify_pid, From]), Results = State#state.summary, State#state.notify_pid ! {From, {test_result, Result}}, - report_and_gather_logs(State#state.log_dir, Result), + report_and_gather_logs(State#state.giddyup, State#state.log_dir, Result), {reply, ok, State#state{summary=[Result|Results]}}; handle_call(done, From, State) -> - lager:debug("Sending done to ~p from ~p", [State#state.notify_pid, From]), State#state.notify_pid ! {From, done}, print_summary(State#state.summary, undefined, true), {reply, ok, State}; @@ -295,18 +296,8 @@ format_test_row({TestPlan, Result, Duration}) -> [TestString, "pass", "N/A", test_summary_format_time(Duration)] end. -%%-------------------------------------------------------------------- -%% @private -%% @doc -%% Gather all of the log files from the nodes and either upload to -%% GiddyUp or copy them to the directory of your choice. Also upload -%% latest test result, if necessary. -%% -%% @spec report_and_gather_logs(Directory) -> ok -%% @end -%%-------------------------------------------------------------------- -%% -spec(report_and_gather_logs(giddyup|string(), term()) -> ok). -report_and_gather_logs(giddyup, TestResult) -> +-spec(report_to_giddyup(term(), list()) -> list). +report_to_giddyup(TestResult, Logs) -> {TestPlan, Reason, _Duration} = TestResult, Status = case Reason of pass -> pass; @@ -324,11 +315,30 @@ report_and_gather_logs(giddyup, TestResult) -> case giddyup:post_result(GiddyupResult) of error -> woops; {ok, Base} -> - [giddyup:post_artifact(Base, File) || File <- rt:get_node_logs(giddyup)] - end; -report_and_gather_logs(LogDir, {TestPlan, _, _}) -> + [giddyup:post_artifact(Base, File) || File <- Logs] + end. +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Gather all of the log files from the nodes and either upload to +%% GiddyUp or copy them to the directory of your choice. Also upload +%% latest test result, if necessary. +%% +%% @spec report_and_gather_logs(Directory) -> ok +%% @end +%%-------------------------------------------------------------------- +-spec(report_and_gather_logs(boolean(), string(), term()) -> ok). +report_and_gather_logs(UploadToGiddyUp, LogDir, TestResult = {TestPlan, _, _}) -> SubDir = filename:join([LogDir, rt_test_plan:get_name(TestPlan)]), - rt:get_node_logs(SubDir). + LogFile = filename:join([SubDir, "riak_test.log"]), + Logs = rt:get_node_logs(UploadToGiddyUp, LogFile, SubDir), + case UploadToGiddyUp of + true -> + report_to_giddyup(TestResult, Logs); + _ -> + Logs + end. + %% %% RetList = [{test, TestModule}, {status, Status}, {log, Log}, {backend, Backend} | proplists:delete(backend, TestMetaData)], %% case Status of diff --git a/src/rt_ring.erl b/src/rt_ring.erl index 56d16d5f2..1ce488cb3 100644 --- a/src/rt_ring.erl +++ b/src/rt_ring.erl @@ -18,7 +18,6 @@ %% %% ------------------------------------------------------------------- -module(rt_ring). --include("rt.hrl"). -include_lib("eunit/include/eunit.hrl"). -export([assert_nodes_agree_about_ownership/1, diff --git a/src/rt_systest.erl b/src/rt_systest.erl index bb7173495..1b524c027 100644 --- a/src/rt_systest.erl +++ b/src/rt_systest.erl @@ -18,7 +18,6 @@ %% %% ------------------------------------------------------------------- -module(rt_systest). --include("rt.hrl"). -include_lib("eunit/include/eunit.hrl"). -export([read/2, diff --git a/src/rt_test_plan.erl b/src/rt_test_plan.erl index b48da96c6..55e74d887 100644 --- a/src/rt_test_plan.erl +++ b/src/rt_test_plan.erl @@ -27,8 +27,6 @@ -module(rt_test_plan). -author("Brett Hazen"). --include("rt.hrl"). - %% API -export([new/0, new/1, diff --git a/src/rtdev.erl b/src/rtdev.erl index a00bf8102..38c43570d 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -50,7 +50,7 @@ set_advanced_conf/2, rm_dir/1, validate_config/1, - get_node_logs/1]). + get_node_logs/3]). -compile(export_all). -include_lib("eunit/include/eunit.hrl"). @@ -964,28 +964,44 @@ devpaths() -> %% proplists:get_keys(rt_config:get(rtdev_path)) -- [root]. % @doc Get the list of log files and config files and pass them back -get_node_logs(DestDir) -> +-spec(get_node_logs(boolean(), string(), string()) -> list()). +get_node_logs(UploadToGiddyUp, LogFile, DestDir) -> Root = filename:absname(?PATH), RootLen = length(Root) + 1, %% Remove the leading slash - Fun = get_node_log_fun(DestDir, RootLen), - [ Fun(Filename) || Filename <- filelib:wildcard(Root ++ "/*/dev*/log/*") ++ - filelib:wildcard(Root ++ "/*/dev*/etc/*.conf*") ]. + Fun = get_node_log_fun(UploadToGiddyUp, DestDir, RootLen), + NodeLogs = [ Fun(Filename) || Filename <- filelib:wildcard(Root ++ "/*/dev*/log/*") ++ + filelib:wildcard(Root ++ "/*/dev*/etc/*.conf*") ], + %% Trim the Lager file path slightly differently + LagerFile = filename:absname(LogFile), + LagerLen = length(filename:dirname(LagerFile)) + 1, + LagerFun = get_node_log_fun(UploadToGiddyUp, DestDir, LagerLen), + LagerLog = LagerFun(LagerFile), + lists:append([LagerLog], NodeLogs). % @doc Either open a port for uploading each file to GiddyUp or % Copy each file to a local directory -get_node_log_fun(giddyup, RootLen) -> - fun(Filename) -> - {ok, Port} = file:open(Filename, [read, binary]), - {lists:nthtail(RootLen, Filename), Port} - end; -get_node_log_fun(DestDir, RootLen) -> +-spec(get_node_log_fun(boolean(), string(), integer()) -> fun()). +get_node_log_fun(UploadToGiddyUp, DestDir, RootLen) -> DestRoot = filename:absname(DestDir), lager:debug("Copying log files to ~p", [DestRoot]), fun(Filename) -> Target = filename:join([DestRoot, lists:nthtail(RootLen, Filename)]), ok = filelib:ensure_dir(Target), - {ok, _BytesWritten} = file:copy(Filename, Target), - Filename + %% Copy the file only if it's a new location + case Target of + Filename -> ok; + _ -> + lager:debug("Copying ~p to ~p", [Filename, Target]), + {ok, _BytesWritten} = file:copy(Filename, Target) + end, + %% Open a port if this is to be uploaded to GiddyUp + case UploadToGiddyUp of + true -> + {ok, Port} = file:open(Target, [read, binary]), + {lists:nthtail(RootLen, Filename), Port}; + _ -> + Target + end end. -type node_tuple() :: {list(), atom()}. From 33772f3847da2b743dda0d38f01003d883aa27a0 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Wed, 22 Apr 2015 11:15:50 -0600 Subject: [PATCH 132/157] Convert giddyup module into a gen_server - Always copy log files to local directory - Removed unused command-line options in escript - Get rid of extraneous commented-out code - Move as much GiddyUp-specific logic into `giddyup' - Correctly upload config files - Upload files asynchronously to scheduler can reclaim used nodes --- src/giddyup.erl | 455 ++++++++++++++++++++++++++++++++------ src/riak_test_escript.erl | 104 ++++----- src/rt.erl | 11 +- src/rt2.erl | 6 +- src/rt_config.erl | 2 +- src/rt_harness.erl | 6 +- src/rt_planner.erl | 83 +++---- src/rt_reporter.erl | 23 +- src/rtdev.erl | 26 +-- 9 files changed, 475 insertions(+), 241 deletions(-) diff --git a/src/giddyup.erl b/src/giddyup.erl index c09209cff..b77bf28ff 100644 --- a/src/giddyup.erl +++ b/src/giddyup.erl @@ -1,6 +1,6 @@ -%% ------------------------------------------------------------------- +%%------------------------------------------------------------------- %% -%% Copyright (c) 2013 Basho Technologies, Inc. +%% Copyright (c) 2015 Basho Technologies, Inc. %% %% This file is provided to you under the Apache License, %% Version 2.0 (the "License"); you may not use this file @@ -17,9 +17,25 @@ %% under the License. %% %% ------------------------------------------------------------------- +%% @author Brett Hazen +%% @copyright (C) 2015, Basho Technologies +%% @doc +%% Communicate with the GiddyUp web service. Pulls jobs to do and +%% report results back up to GiddyUp. +%% @end +%% Created : 20. Apr 2015 10:39 AM +%%------------------------------------------------------------------- -module(giddyup). +-author("Brett Hazen"). + +-behaviour(gen_server). + +%% API +-export([start_link/7, + get_test_plans/0, + post_result/2, + post_artifact/3]). --export([get_suite/1, post_result/1, post_artifact/2]). -define(STREAM_CHUNK_SIZE, 8192). -record(rt_webhook, { @@ -28,45 +44,265 @@ headers=[] :: [{atom(), string()}] }). --spec get_suite(string()) -> [{atom(), term()}]. -get_suite(Platform) -> - Schema = get_schema(Platform), - Name = kvc:path('project.name', Schema), - Version = rt_config:get_default_version_number(), - lager:info("Retrieved Project: ~s", [Name]), +%% gen_server callbacks +-export([init/1, + stop/0, + handle_call/3, + handle_cast/2, + handle_info/2, + terminate/2, + code_change/3]). + +-define(SERVER, ?MODULE). + +-record(state, { + platform :: string(), + default_product :: string(), + default_version_number :: string(), + default_version :: string(), + giddyup_host :: string(), + giddyup_user :: string(), + giddyup_password :: string(), + %% A dictionary of Base URLs to store artifacts + artifact_base :: dict() +}). + +%%%=================================================================== +%%% API +%%%=================================================================== + +%%-------------------------------------------------------------------- +%% @doc +%% Starts the server +%% +%% @end +%%-------------------------------------------------------------------- +-spec(start_link(string(), string(), string(), string(), string(), string(), string()) -> + {ok, Pid :: pid()} | ignore | {error, Reason :: term()}). +start_link(Platform, Product, VersionNumber, Version, GiddyUpHost, GiddyUpUser, GiddyUpPassword) -> + gen_server:start_link({local, ?SERVER}, ?MODULE, [Platform, Product, VersionNumber, Version, GiddyUpHost, GiddyUpUser, GiddyUpPassword], []). + +%%-------------------------------------------------------------------- +%% @doc +%% Stops the server +%% +%% @end +%%-------------------------------------------------------------------- +-spec stop() -> ok. +stop() -> + gen_server:call(?MODULE, stop, infinity). + +%%-------------------------------------------------------------------- +%% @doc +%% Get the entire test suite for the specified platform +%% +%% @end +%%-------------------------------------------------------------------- +get_test_plans() -> + %% TODO: Is this a good timeout? + gen_server:call(?MODULE, get_test_plans, 60000). + +%%-------------------------------------------------------------------- +%% @doc +%% Send a test result back up to GiddyUp +%% @end +%%-------------------------------------------------------------------- + +-spec post_result(rt_test_plan:test_plan(), pass | {fail, string()}) -> ok. +post_result(TestPlan, TestResult) -> + gen_server:cast(?MODULE, {post_result, TestPlan, TestResult}). + +%%-------------------------------------------------------------------- +%% @doc +%% Send a test log file back up to GiddyUp +%% @end +%%-------------------------------------------------------------------- + +-spec(post_artifact(string(), string(), string()) -> ok | error). +post_artifact(TestPlan, Label, Filename) -> + gen_server:cast(?MODULE, {post_artifact, TestPlan, Label, Filename}). + +%%%=================================================================== +%%% gen_server callbacks +%%%=================================================================== + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Initializes the GiddyUp server +%% +%% @spec init(Args) -> {ok, State} | +%% {ok, State, Timeout} | +%% ignore | +%% {stop, Reason} +%% @end +%%-------------------------------------------------------------------- +-spec(init(Args :: term()) -> + {ok, State :: #state{}} | {ok, State :: #state{}, timeout() | hibernate} | + {stop, Reason :: term()} | ignore). +init([Platform, Product, VersionNumber, Version, GiddyUpHost, GiddyUpUser, GiddyUpPassword]) -> + load_and_start(ibrowse), + {ok, #state{platform=Platform, + default_version=Version, + default_version_number=VersionNumber, + default_product=Product, + giddyup_host=GiddyUpHost, + giddyup_user=GiddyUpUser, + giddyup_password=GiddyUpPassword, + artifact_base = dict:new()}}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Handling call messages +%% +%% @end +%%-------------------------------------------------------------------- +-spec(handle_call(Request :: term(), From :: {pid(), Tag :: term()}, + State :: #state{}) -> + {reply, Reply :: term(), NewState :: #state{}} | + {reply, Reply :: term(), NewState :: #state{}, timeout() | hibernate} | + {noreply, NewState :: #state{}} | + {noreply, NewState :: #state{}, timeout() | hibernate} | + {stop, Reason :: term(), Reply :: term(), NewState :: #state{}} | + {stop, Reason :: term(), NewState :: #state{}}). +handle_call(get_test_plans, _From, State) -> + TestPlans = fetch_all_test_plans(State#state.platform, State#state.default_product, State#state.default_version_number, State#state.default_version, State#state.giddyup_host), + {reply, TestPlans, State}; +handle_call(_Request, _From, State) -> + {reply, ok, State}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Handling cast messages +%% +%% @end +%%-------------------------------------------------------------------- +-spec(handle_cast(Request :: term(), State :: #state{}) -> + {noreply, NewState :: #state{}} | + {noreply, NewState :: #state{}, timeout() | hibernate} | + {stop, Reason :: term(), NewState :: #state{}}). +handle_cast({post_result, TestPlan, TestResult}, State) -> + {ok, Location} = post_result(TestPlan, TestResult, State#state.giddyup_host, State#state.giddyup_user, State#state.giddyup_password), + Dictionary = State#state.artifact_base, + %% Store the Base URL in a dictionary keyed off the test name + {noreply, State#state{artifact_base=dict:store(rt_test_plan:get_name(TestPlan), Location, Dictionary)}}; +handle_cast({post_artifact, TestPlan, Label, Filename}, State) -> + BaseURL = dict:fetch(rt_test_plan:get_name(TestPlan), State#state.artifact_base), + post_artifact(BaseURL, Label, Filename, State#state.giddyup_user, State#state.giddyup_password), + {noreply, State}; +handle_cast(_Request, State) -> + {noreply, State}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Handling all non call/cast messages +%% +%% @spec handle_info(Info, State) -> {noreply, State} | +%% {noreply, State, Timeout} | +%% {stop, Reason, State} +%% @end +%%-------------------------------------------------------------------- +-spec(handle_info(Info :: timeout() | term(), State :: #state{}) -> + {noreply, NewState :: #state{}} | + {noreply, NewState :: #state{}, timeout() | hibernate} | + {stop, Reason :: term(), NewState :: #state{}}). +handle_info(_Info, State) -> + {noreply, State}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% This function is called by a gen_server when it is about to +%% terminate. It should be the opposite of Module:init/1 and do any +%% necessary cleaning up. When it returns, the gen_server terminates +%% with Reason. The return value is ignored. +%% +%% @spec terminate(Reason, State) -> void() +%% @end +%%-------------------------------------------------------------------- +-spec(terminate(Reason :: (normal | shutdown | {shutdown, term()} | term()), + State :: #state{}) -> term()). +terminate(_Reason, _State) -> + ok. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Convert process state when code is changed +%% +%% @spec code_change(OldVsn, State, Extra) -> {ok, NewState} +%% @end +%%-------------------------------------------------------------------- +-spec(code_change(OldVsn :: term() | {down, term()}, State :: #state{}, + Extra :: term()) -> + {ok, NewState :: #state{}} | {error, Reason :: term()}). +code_change(_OldVsn, State, _Extra) -> + {ok, State}. + +%%%=================================================================== +%%% Internal functions +%%%=================================================================== + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Get the entire test suite from Giddyup in an `rt_test_plan' list +%% @end +%%-------------------------------------------------------------------- + +-spec fetch_all_test_plans(string(), string(), string(), string(), string()) -> [rt_test_plan:test_plan()]. +fetch_all_test_plans(Platform, Product, VersionNumber, DefaultVersion, Host) -> + %% Make sure ibrowse is up and running + rt:check_ibrowse(), + Schema = get_schema(Platform, Product, VersionNumber, Host), + Project = kvc:path('project.name', Schema), + lager:info("Retrieved Project: ~s", [Project]), Tests = kvc:path('project.tests', Schema), TestProps = fun(Test) -> - [ - {id, kvc:path(id, Test)}, - {backend, - case kvc:path('tags.backend', Test) of - [] -> undefined; - X -> binary_to_atom(X, utf8) - end}, - {platform, list_to_binary(Platform)}, - {version, Version}, - {project, Name} - ] ++ - case kvc:path('tags.upgrade_version', Test) of - [] -> []; - UpgradeVsn -> [{upgrade_version, binary_to_atom(UpgradeVsn, utf8)}] - end ++ - case kvc:path('tags.multi_config', Test) of - [] -> []; - MultiConfig -> [{multi_config, binary_to_atom(MultiConfig, utf8)}] - end + Id = kvc:path(id, Test), + Module = binary_to_atom(kvc:path(name, Test), utf8), + Plan0 = rt_test_plan:new([{id, Id}, {module, Module}, {project, Project}, {platform, Platform}, {version, VersionNumber}]), + Plan1 = case kvc:path('tags.backend', Test) of + [] -> Plan0; + X -> rt_test_plan:set(backend, binary_to_atom(X, utf8), Plan0) + end, + Plan2 = case kvc:path('tags.upgrade_version', Test) of + [] -> Plan1; + UpgradeVsn -> + UpgradeVersion = case UpgradeVsn of + <<"legacy">> -> rt_config:get_legacy_version(); + <<"previous">> -> rt_config:get_previous_version(); + _ -> rt_config:get_version(binary_to_list(UpgradeVsn)) + end, + rt_test_plan:set(upgrade_path, [UpgradeVersion, DefaultVersion], Plan1) + end, + %% TODO: Remove? No tests currently use this multi_config setting + %% Plan3 = case kvc:path('tags.multi_config', Test) of + %% [] -> Plan1; + %% MultiConfig -> rt_test_plan:set(multi_config, binary_to_atom(MultiConfig, utf8), Plan2) + %%end, + Plan2 end, - [ { binary_to_atom(kvc:path(name, Test), utf8), TestProps(Test) } || Test <- Tests]. + [ TestProps(Test) || Test <- Tests]. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Get the GiddyUp Schema in JSON format (decoded to Erlang terms) +%% Retry up to 3 times. +%% @end +%%-------------------------------------------------------------------- -get_schema(Platform) -> - get_schema(Platform, 3). +-spec(get_schema(atom(), string(), string(), string()) -> term()). +get_schema(Platform, Product, VersionNumber, Host) -> + get_schema(Platform, Product, VersionNumber, Host, 3). -get_schema(Platform, Retries) -> - Host = rt_config:get(giddyup_host), - Project = rt_config:get_default_version_product(), - Version = rt_config:get_default_version_number(), - URL = lists:flatten(io_lib:format("http://~s/projects/~s?platform=~s&version=~s", [Host, Project, Platform, Version])), +get_schema(Platform, Product, VersionNumber, Host, Retries) -> + URL = lists:flatten(io_lib:format("http://~s/projects/~s?platform=~s&version=~s", [Host, Product, Platform, VersionNumber])), lager:info("giddyup url: ~s", [URL]), rt:check_ibrowse(), @@ -79,16 +315,23 @@ get_schema(Platform, Retries) -> lager:warning("GiddyUp GET failed: ~p", [Error]), lager:warning("GiddyUp trying ~p more times", [Retries]), timer:sleep(60000), - get_schema(Platform, Retries - 1) + get_schema(Platform, Product, VersionNumber, Host, Retries - 1) end. --spec post_result([{atom(), term()}]) -> {ok, string()} | error. -post_result(TestResult) -> - Host = rt_config:get(giddyup_host), +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Send a test result back up to GiddyUp +%% @end +%%-------------------------------------------------------------------- + +-spec post_result(rt_test_plan:test_plan(), pass | {fail, string()}, string(), string(), string()) -> {ok, string()} | error. +post_result(TestPlan, TestResult, Host, User, Password) -> URL = "http://" ++ Host ++ "/test_results", lager:info("giddyup url: ~s", [URL]), rt:check_ibrowse(), - case post_result(TestResult, #rt_webhook{name="GiddyUp", url=URL, headers=[basic_auth()]}) of + BasicAuth = {basic_auth, {User, Password}}, + case post_result(TestPlan, TestResult, #rt_webhook{name="GiddyUp", url=URL, headers=[BasicAuth]}) of {ok, RC, Headers} -> {_, Location} = lists:keyfind("Location", 1, Headers), lager:info("Test Result successfully POSTed to GiddyUp! ResponseCode: ~s, URL: ~s", [RC, Location]), @@ -97,14 +340,27 @@ post_result(TestResult) -> error end. --spec(post_result(proplists:proplist(), term()) -> tuple()|error). -post_result(TestResult, #rt_webhook{url=URL, headers=HookHeaders, name=Name}) -> +-spec(post_result(rt_test_plan:test_plan(), pass | {fail, string()}, term()) -> {ok, integer(), [string()]} | error). +post_result(TestPlan, TestResult, #rt_webhook{url=URL, headers=HookHeaders, name=Name}) -> + Status = case TestResult of + pass -> pass; + _ -> fail + end, + GiddyupResult = [ + {test, rt_test_plan:get_module(TestPlan)}, + {status, Status}, + {backend, rt_test_plan:get(backend, TestPlan)}, + {id, rt_test_plan:get(id, TestPlan)}, + {platform, rt_test_plan:get(platform, TestPlan)}, + {version, rt_test_plan:get(version, TestPlan)}, + {project, rt_test_plan:get(project, TestPlan)} + ], try ibrowse:send_req(URL, - [{"Content-Type", "application/json"}], - post, - mochijson2:encode(TestResult), - [{content_type, "application/json"}] ++ HookHeaders, - 300000) of %% 5 minute timeout + [{"Content-Type", "application/json"}], + post, + mochijson2:encode(GiddyupResult), + [{content_type, "application/json"}] ++ HookHeaders, + 300000) of %% 5 minute timeout {ok, RC=[$2|_], Headers, _Body} -> {ok, RC, Headers}; @@ -121,50 +377,64 @@ post_result(TestResult, #rt_webhook{url=URL, headers=HookHeaders, name=Name}) -> catch Class:Reason -> lager:error("Error reporting to ~s. ~p:~p", [Name, Class, Reason]), - lager:error("Payload: ~p", [TestResult]), + lager:error("Payload: ~p", [GiddyupResult]), error end. -post_artifact(TRURL, {FName, Body}) -> +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Send a test log file back up to GiddyUp +%% @end +%%-------------------------------------------------------------------- + +-spec(post_artifact(string(), string(), string(), string(), string()) -> ok | error). +post_artifact(BaseURL, Label, Filename, User, Password) -> %% First compute the path of where to post the artifact - URL = artifact_url(TRURL, FName), + URL = artifact_url(BaseURL, Label), + {ok, Body} = file:open(Filename, [read, binary]), ReqBody = make_req_body(Body), - CType = guess_ctype(FName), + CType = guess_ctype(Label), + BasicAuth = {basic_auth, {User, Password}}, + %% Send request try ibrowse:send_req(URL, [{"Content-Type", CType}], - post, - ReqBody, - [{content_type, CType}, basic_auth()], - 300000) of + post, + ReqBody, + [{content_type, CType}, BasicAuth], + 300000) of {ok, [$2|_], Headers, _Body} -> {_, Location} = lists:keyfind("Location", 1, Headers), - lager:info("Successfully uploaded test artifact ~s to GiddyUp! URL: ~s", [FName, Location]), + lager:info("Successfully uploaded test artifact ~s to GiddyUp! URL: ~s", [Label, Location]), ok; {ok, RC, Headers, Body} -> - lager:info("Test artifact ~s failed to upload!", [FName]), + lager:info("Test artifact ~s failed to upload!", [Label]), lager:debug("Status: ~p~nHeaders: ~p~nBody: ~s~n", [RC, Headers, Body]), error; X -> lager:error("Error uploading ~s to giddyup. ~p~n" - "URL: ~p~nRequest Body: ~p~nContent Type: ~p~n", - [FName, X, URL, ReqBody, CType]), + "URL: ~p~nRequest Body: ~p~nContent Type: ~p~n", + [Label, X, URL, ReqBody, CType]), error catch Throws -> lager:error("Error uploading ~s to giddyup. ~p~n" - "URL: ~p~nRequest Body: ~p~nContent Type: ~p~n", - [FName, Throws, URL, ReqBody, CType]) + "URL: ~p~nRequest Body: ~p~nContent Type: ~p~n", + [Label, Throws, URL, ReqBody, CType]) end. -basic_auth() -> - {basic_auth, {rt_config:get(giddyup_user), rt_config:get(giddyup_password)}}. - +%%-------------------------------------------------------------------- +%% @private +%% @doc %% Given a URI parsed by http_uri, reconstitute it. -generate({_Scheme, _UserInfo, _Host, _Port, _Path, _Query}=URI) -> - generate(URI, http_uri:scheme_defaults()). +%% @end +%%-------------------------------------------------------------------- + +generate_uri({_Scheme, _UserInfo, _Host, _Port, _Path, _Query}=URI) -> + generate_uri(URI, http_uri:scheme_defaults()). -generate({Scheme, UserInfo, Host, Port, Path, Query}, SchemeDefaults) -> +generate_uri({Scheme, UserInfo, Host, Port, Path, Query}, SchemeDefaults) -> {Scheme, DefaultPort} = lists:keyfind(Scheme, 1, SchemeDefaults), lists:flatten([ [ atom_to_list(Scheme), "://" ], @@ -174,26 +444,45 @@ generate({Scheme, UserInfo, Host, Port, Path, Query}, SchemeDefaults) -> Path, Query ]). +%%-------------------------------------------------------------------- +%% @private +%% @doc %% Given the test result URL, constructs the appropriate URL for the artifact. -artifact_url(TRURL, FName) -> - {ok, {Scheme, UserInfo, Host, Port, Path, Query}} = http_uri:parse(TRURL), +%% @end +%%-------------------------------------------------------------------- + +artifact_url(BaseURL, FName) -> + {ok, {Scheme, UserInfo, Host, Port, Path, Query}} = http_uri:parse(BaseURL), ArtifactPath = filename:join([Path, "artifacts", FName]), - generate({Scheme, UserInfo, Host, Port, ArtifactPath, Query}). + generate_uri({Scheme, UserInfo, Host, Port, ArtifactPath, Query}). +%%-------------------------------------------------------------------- +%% @private +%% @doc %% ibrowse support streaming request bodies, so in the case where we %% have a Port/File to read from, we should stream it. +%% @end +%%-------------------------------------------------------------------- + make_req_body(Body) when is_port(Body); is_pid(Body) -> read_fully(Body); -make_req_body(Body) when is_list(Body); - is_binary(Body) -> +make_req_body(Body) when is_list(Body); is_binary(Body) -> Body. +%%-------------------------------------------------------------------- +%% @private +%% @doc %% Read the file/port fully until eof. This is a workaround for the %% fact that ibrowse doesn't seem to send file streams correctly, or %% giddyup dislikes them. (shrug) +%% @end +%%-------------------------------------------------------------------- + +-spec(read_fully(string() | port()) -> binary()). read_fully(File) -> read_fully(File, <<>>). +-spec(read_fully(string() | port(), binary()) -> binary()). read_fully(File, Data0) -> case file:read(File, ?STREAM_CHUNK_SIZE) of {ok, Data} -> @@ -202,10 +491,19 @@ read_fully(File, Data0) -> Data0 end. +%%-------------------------------------------------------------------- +%% @private +%% @doc %% Guesses the MIME type of the file being uploaded. +%% @end +%%-------------------------------------------------------------------- + +-spec(guess_ctype(string()) -> string()). guess_ctype(FName) -> case string:tokens(filename:basename(FName), ".") of [_, "log"|_] -> "text/plain"; %% console.log, erlang.log.5, etc + [_, "conf"|_] -> "text/plain"; %% riak.conf + [_, "config"|_] -> "text/plain"; %% advanced.config, etc ["erl_crash", "dump"] -> "text/plain"; %% An erl_crash.dump file [_, Else] -> case mochiweb_mime:from_extension(Else) of @@ -214,3 +512,14 @@ guess_ctype(FName) -> end; _ -> "binary/octet-stream" end. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Verify that an application is running +%% @end +%%-------------------------------------------------------------------- + +load_and_start(Application) -> + application:load(Application), + application:start(Application). \ No newline at end of file diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index c249bd2f1..1dfdbdbd4 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -36,11 +36,19 @@ main(Args) -> ensure_dir(OutDir), lager_setup(OutDir), - {Tests, NonTests} = generate_test_lists(ParsedArgs), + %% Do we use GiddyUp for this run? + Platform = report_platform(ParsedArgs), + UseGiddyUp = case Platform of + undefined -> false; + _ -> true + end, + start_giddyup(Platform), + {Tests, NonTests} = generate_test_lists(UseGiddyUp, ParsedArgs), ok = prepare(ParsedArgs, Tests, NonTests), Results = execute(Tests, OutDir, ParsedArgs), - finalize(Results, ParsedArgs). + finalize(Results, ParsedArgs), + stop_giddyup(UseGiddyUp). % @doc Validate the command-line options parse_args(Args) -> @@ -64,14 +72,11 @@ cli_options() -> {help, $h, "help", undefined, "Print this usage page"}, {config, $c, "conf", string, "specifies the project configuration"}, {tests, $t, "tests", string, "specifies which tests to run"}, - {suites, $s, "suites", string, "which suites to run"}, - {groups, $g, "groups", string, "specifiy a list of test groups to run"}, {dir, $d, "dir", string, "run all tests in the specified directory"}, {skip, $x, "skip", string, "list of tests to skip in a directory"}, {verbose, $v, "verbose", undefined, "verbose output"}, {outdir, $o, "outdir", string, "output directory"}, {backend, $b, "backend", atom, "backend to test [memory | bitcask | eleveldb]"}, - {upgrade_path, $u, "upgrade-path", atom, "comma-separated list representing an upgrade path (e.g. riak-1.3.4,riak_ee-1.4.12,riak_ee-2.0.0)"}, {keep, undefined, "keep", boolean, "do not teardown cluster"}, {continue_on_fail, $n, "continue", boolean, "continues executing tests on failure"}, {report, $r, "report", string, "you're reporting an official test run, provide platform info (e.g. ubuntu-1404-64)\nUse 'config' if you want to pull from ~/.riak_test.config"}, @@ -93,13 +98,13 @@ report_platform(ParsedArgs) -> end. %% @doc Print help string if it's specified, otherwise parse the arguments -generate_test_lists(ParsedArgs) -> +generate_test_lists(UseGiddyUp, ParsedArgs) -> %% Have to load the `riak_test' config prior to assembling the %% test metadata TestData = compose_test_data(ParsedArgs), - Backend = proplists:get_value(backend, ParsedArgs, bitcask), - {Tests, NonTests} = wrap_test_in_test_plan(report_platform(ParsedArgs), Backend, TestData), + Backends = [proplists:get_value(backend, ParsedArgs, bitcask)], + {Tests, NonTests} = wrap_test_in_test_plan(UseGiddyUp, Backends, TestData), Offset = rt_config:get(offset, undefined), Workers = rt_config:get(workers, undefined), shuffle_tests(Tests, NonTests, Offset, Workers). @@ -168,24 +173,6 @@ execute(TestPlans, OutDir, ParsedArgs) -> self()), wait_for_results(Executor, [], length(TestPlans), 0). -%% TestResults = run_tests(Tests, Outdir, Report, HarnessArgs), -%% lists:filter(fun results_filter/1, TestResults). - -%% run_test(Test, Outdir, TestMetaData, Report, HarnessArgs, NumTests) -> -%% rt_cover:maybe_start(Test), -%% SingleTestResult = riak_test_runner:run(Test, Outdir, TestMetaData, HarnessArgs), - -%% case NumTests of -%% 1 -> keep_them_up; -%% _ -> rt_cluster:teardown() -%% end, - -%% TODO: Do this in the test runner -%% CoverDir = rt_config:get(cover_output, "coverage"), -%% CoverFile = rt_cover:maybe_export_coverage(Test, CoverDir, erlang:phash2(TestMetaData)), -%% publish_report(SingleTestResult, CoverFile, Report), - -%% [{coverdata, CoverFile} | SingleTestResult]. %% TODO: Use `TestCount' and `Completed' to display progress output wait_for_results(Executor, TestResults, TestCount, Completed) -> @@ -249,9 +236,6 @@ erlang_setup(_ParsedArgs) -> register(riak_test, self()), maybe_add_code_path("./ebin"), - %% ibrowse - load_and_start(ibrowse), - %% Sets up extra paths earlier so that tests can be loadable %% without needing the -d flag. code:add_paths(rt_config:get(test_paths, [])), @@ -274,10 +258,6 @@ maybe_add_code_path(Path, true) -> maybe_add_code_path(_, false) -> meh. -load_and_start(Application) -> - application:load(Application), - application:start(Application). - ensure_dir(undefined) -> ok; ensure_dir(Dir) -> @@ -358,21 +338,18 @@ extract_test_names(Test, {CodePaths, TestNames}) -> %% @doc Determine which tests to run based on command-line argument %% If the platform is defined, consult GiddyUp, otherwise just shovel %% the whole thing into the Planner --spec(load_up_test_planner(string() | undefined, string(), list()) -> list()). -load_up_test_planner(undefined, Backend, CommandLineTests) -> - [rt_planner:add_test_plan(Name, undefined, Backend, undefined, undefined) || Name <- CommandLineTests]; -%% GiddyUp Flavor -load_up_test_planner(Platform, Backend, CommandLineTests) -> - rt_planner:load_from_giddyup(Platform, Backend, CommandLineTests). +-spec(load_up_test_planner(boolean(), [string()], list()) -> list()). +load_up_test_planner(true, Backends, CommandLineTests) -> + rt_planner:load_from_giddyup(Backends, CommandLineTests); +load_up_test_planner(_, Backends, CommandLineTests) -> + [rt_planner:add_test_plan(Name, undefined, Backends, undefined, undefined) || Name <- CommandLineTests]. %% @doc Push all of the test into the Planner for now and wrap them in an `rt_test_plan' %% TODO: Let the Planner do the work, not the riak_test_executor --spec(wrap_test_in_test_plan(string(), string(), [atom()]) -> {list(), list()}). -wrap_test_in_test_plan(Platform, Backend, CommandLineTests) -> - %% ibrowse neededfor GiddyUp - load_and_start(ibrowse), +-spec(wrap_test_in_test_plan(boolean(), [string()], [atom()]) -> {list(), list()}). +wrap_test_in_test_plan(UseGiddyUp, Backends, CommandLineTests) -> {ok, _Pid} = rt_planner:start_link(), - load_up_test_planner(Platform, Backend, CommandLineTests), + load_up_test_planner(UseGiddyUp, Backends, CommandLineTests), TestPlans = [rt_planner:fetch_test_plan() || _ <- lists:seq(1, rt_planner:number_of_plans())], NonRunnableTestPlans = [rt_planner:fetch_test_non_runnable_plan() || _ <- lists:seq(1, rt_planner:number_of_non_runable_plans())], rt_planner:stop(), @@ -407,26 +384,6 @@ match_group_attributes(Attributes, Groups) -> || Group <- Groups, TestType <- TestTypes ]) end. -%% run_tests(Tests, Outdir, Report, HarnessArgs) -> - %% Need properties for tests prior to getting here Need server to - %% manage the aquisition of nodes and to handle comparison of test - %% `node_count' property with resources available. Also handle - %% notification of test completion. Hmm, maybe test execution - %% should be handled by a `gen_fsm' at this point to distinguish - %% the case when there are tests left to be tried with available - %% resources versus all have been tried or resources are - %% exhausted. - -%% [run_test(Test, -%% Outdir, -%% TestMetaData, -%% Report, -%% HarnessArgs, -%% TestCount) || -%% {Test, TestMetaData} <- Tests], - - - backend_list(Backend) when is_atom(Backend) -> atom_to_list(Backend); backend_list(Backends) when is_list(Backends) -> @@ -493,3 +450,22 @@ so_kill_riak_maybe() -> io:format("Leaving Riak Up... "), rt:whats_up() end. + +%% @doc Start the GiddyUp reporting service if the report is defined +start_giddyup(undefined) -> + ok; +start_giddyup(Platform) -> + {ok, _Pid} = giddyup:start_link(Platform, + rt_config:get_default_version_product(), + rt_config:get_default_version_number(), + rt_config:get_default_version(), + rt_config:get(giddyup_host), + rt_config:get(giddyup_user), + rt_config:get(giddyup_password)). + +%% @doc Stop the GiddyUp reporting service if the report is defined +stop_giddyup(true) -> + giddyup:stop(); +stop_giddyup(_) -> + ok. + diff --git a/src/rt.erl b/src/rt.erl index 48dcb2be1..d813c2a5f 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -63,7 +63,7 @@ expect_in_log/2, get_deps/0, get_ip/1, - get_node_logs/3, + get_node_logs/2, get_replica/5, get_ring/1, get_version/0, @@ -1361,11 +1361,10 @@ pmap(F, L) -> setup_harness(_Test, _Args) -> rt_harness:setup(). -%% @doc Copy all of the nodes' log files to a local dir or -%% open a port to each file to upload to GiddyUp --spec(get_node_logs(boolean(), string(), string()) -> list()). -get_node_logs(UploadToGiddyUp, LogFile, DestDir) -> - rt2:get_node_logs(UploadToGiddyUp, LogFile, DestDir). +%% @doc Copy all of the nodes' log files to a local dir +-spec(get_node_logs(string(), string()) -> list()). +get_node_logs(LogFile, DestDir) -> + rt2:get_node_logs(LogFile, DestDir). check_ibrowse() -> rt2:check_ibrowse(). diff --git a/src/rt2.erl b/src/rt2.erl index 60a086ed9..12839117a 100644 --- a/src/rt2.erl +++ b/src/rt2.erl @@ -38,7 +38,7 @@ expect_in_log/2, get_deps/0, get_ip/1, - get_node_logs/3, + get_node_logs/2, get_replica/5, get_version/0, is_mixed_cluster/1, @@ -610,8 +610,8 @@ setup_harness(Test, Args) -> %% @doc Downloads any extant log files from the harness's running %% nodes. -get_node_logs(UploadToGiddyUp, LogFile, DestDir) -> - rt_harness:get_node_logs(UploadToGiddyUp, LogFile, DestDir). +get_node_logs(LogFile, DestDir) -> + rt_harness:get_node_logs(LogFile, DestDir). check_ibrowse() -> try sys:get_status(ibrowse) of diff --git a/src/rt_config.erl b/src/rt_config.erl index 075fa9308..793e00996 100644 --- a/src/rt_config.erl +++ b/src/rt_config.erl @@ -178,7 +178,7 @@ get_version(Vsn) -> resolve_version(Vsn, Versions) -> case find_atom_or_string(Vsn, Versions) of undefined -> - erlang:error("Could not find version", [Vsn]); + erlang:error("Could not find Riak version", [Vsn]); {Product, Tag} -> convert_to_string(Product) ++ "-" ++ convert_to_string(Tag); Version -> diff --git a/src/rt_harness.erl b/src/rt_harness.erl index 99f8636a8..0db02049a 100644 --- a/src/rt_harness.erl +++ b/src/rt_harness.erl @@ -37,7 +37,7 @@ cmd/2, setup/0, get_deps/0, - get_node_logs/3, + get_node_logs/2, get_version/0, get_version/1, get_backends/0, @@ -107,8 +107,8 @@ get_version(Node) -> get_backends() -> ?HARNESS_MODULE:get_backends(). -get_node_logs(UploadToGiddyUp, LogFile, DestDir) -> - ?HARNESS_MODULE:get_node_logs(UploadToGiddyUp, LogFile, DestDir). +get_node_logs(LogFile, DestDir) -> + ?HARNESS_MODULE:get_node_logs(LogFile, DestDir). set_backend(Backend) -> ?HARNESS_MODULE:set_backend(Backend). diff --git a/src/rt_planner.erl b/src/rt_planner.erl index 5ee746a06..9a6114ea9 100644 --- a/src/rt_planner.erl +++ b/src/rt_planner.erl @@ -36,7 +36,7 @@ %% API -export([start_link/0, - load_from_giddyup/3, + load_from_giddyup/2, add_test_plan/5, fetch_test_plan/0, fetch_test_non_runnable_plan/0, @@ -82,9 +82,9 @@ start_link() -> %% %% @end %%-------------------------------------------------------------------- --spec(load_from_giddyup(string(), string() | undefined, list()) -> ok). -load_from_giddyup(Platform, Backend, CommandLineTests) -> - gen_server:call(?MODULE, {load_from_giddyup, Platform, Backend, CommandLineTests}). +-spec(load_from_giddyup([string()] | undefined, list()) -> ok). +load_from_giddyup(Backends, CommandLineTests) -> + gen_server:call(?MODULE, {load_from_giddyup, Backends, CommandLineTests}). %%-------------------------------------------------------------------- %% @doc @@ -92,9 +92,9 @@ load_from_giddyup(Platform, Backend, CommandLineTests) -> %% %% @end %%-------------------------------------------------------------------- --spec(add_test_plan(string(), string(), rt_properties2:storage_backend(), rt_properties2:product_version(), rt_properties2:properties()) -> ok). -add_test_plan(Module, Platform, Backend, Version, Properties) -> - gen_server:call(?MODULE, {add_test_plan, Module, Platform, Backend, Version, Properties}). +-spec(add_test_plan(string(), string(), [string()], rt_properties2:product_version(), rt_properties2:properties()) -> ok). +add_test_plan(Module, Platform, Backends, Version, Properties) -> + gen_server:call(?MODULE, {add_test_plan, Module, Platform, Backends, Version, Properties}). %%-------------------------------------------------------------------- %% @doc @@ -185,22 +185,33 @@ init([]) -> {stop, Reason :: term(), NewState :: #state{}}). %% Run only those GiddyUp tests which are specified on the command line %% If none are specified, run everything -handle_call({load_from_giddyup, Platform, _Backend, CommandLineTests}, _From, State) -> - AllGiddyupTests = giddyup:get_suite(Platform), - FilteredTests = case CommandLineTests of +handle_call({load_from_giddyup, Backends, CommandLineTests}, _From, State) -> + AllGiddyupTests = giddyup:get_test_plans(), + FilteredNames = case CommandLineTests of [] -> - [test_plan_from_giddyup(Test) || Test <- AllGiddyupTests]; + AllGiddyupTests; _ -> - [test_plan_from_giddyup({GName, GMetaData}) || {GName, GMetaData} <- AllGiddyupTests, - CName <- CommandLineTests, - GName =:= CName] + [TestPlan || TestPlan <- AllGiddyupTests, + CName <- CommandLineTests, + rt_test_plan:get_module(TestPlan) =:= CName] + end, + FilteredTests = case Backends of + undefined -> + FilteredNames; + _ -> + [TestPlan || TestPlan <- FilteredNames, + lists:member(rt_test_plan:get(backend, TestPlan), Backends)] end, State1 = lists:foldl(fun sort_and_queue/2, State, FilteredTests), {reply, ok, State1}; %% Add a single test plan to the queue -handle_call({add_test_plan, Module, Platform, Backend, _Version, _Properties}, _From, State) -> - TestPlan = rt_test_plan:new([{module, Module}, {platform, Platform}, {backend, Backend}]), - {reply, ok, sort_and_queue(TestPlan, State)}; +handle_call({add_test_plan, Module, Platform, Backends, _Version, _Properties}, _From, State) -> + State1 = lists:foldl(fun(Backend, AccState) -> + TestPlan = rt_test_plan:new([{module, Module}, {platform, Platform}, {backend, Backend}]), + sort_and_queue(TestPlan, AccState) + end, + State, Backends), + {reply, ok, State1}; handle_call(fetch_test_plan, _From, State) -> Q = State#state.runnable_test_plans, {Item, Q1} = queue:out(Q), @@ -291,37 +302,6 @@ code_change(_OldVsn, State, _Extra) -> %%% Internal functions %%%=================================================================== -%%-------------------------------------------------------------------- -%% @private -%% @doc -%% Translate GiddyUp Output into an `rt_test_plan' record -%% -%% @end -%%-------------------------------------------------------------------- - --spec(set_giddyup_field(atom(), {proplists:proplist(), rt_test_plan:test_plan()}) -> rt_test_plan:test_plan()). -set_giddyup_field(Field, {MetaData, TestPlan}) -> - {ok, TestPlan1} = case proplists:is_defined(Field, MetaData) of - true -> - rt_test_plan:set(Field, proplists:get_value(Field, MetaData), TestPlan); - _ -> - {ok, TestPlan} - end, - {MetaData, TestPlan1}. - --spec(test_plan_from_giddyup({atom(), term()}) -> rt_test_plan:test_plan()). -test_plan_from_giddyup({Name, MetaData}) -> - Plan0 = rt_test_plan:new([{module, Name}]), - GiddyUpFields = [id, backend, platform, project], - {_, Plan1} = lists:foldl(fun set_giddyup_field/2, {MetaData, Plan0}, GiddyUpFields), - %% Special treatment for the upgrade path - {ok, Plan2} = case proplists:is_defined(upgrade_version, MetaData) of - true -> - rt_test_plan:set(upgrade_path, [rt_config:get_version(proplists:get_value(upgrade_version, MetaData)), rt_config:get_default_version()], Plan1); - _ -> - {ok, Plan1} - end, - Plan2. %%-------------------------------------------------------------------- %% @private @@ -350,10 +330,3 @@ is_runnable_test_plan(TestPlan) -> code:ensure_loaded(Mod), erlang:function_exported(Mod, Fun, 0) orelse erlang:function_exported(Mod, Fun, 1). - --ifdef(TEST). -set_giddyup_field_test() -> - S = {#state{runnable_test_plans =queue:new()},[]}, - T = test_plan_from_giddyup({test, [{id, 5},{backend,riak_kv_eleveldb_backend}, {platform, "os-x"}, {version, "2.0.5"},{project,<<"riak_ee">>}]}, S), - ?assertEqual(T, {#state{runnable_test_plans = {[{rt_test_plan_v1,5,test,<<"riak_ee">>,"os-x",riak_kv_eleveldb_backend,[],undefined}], []}},[]}). --endif. diff --git a/src/rt_reporter.erl b/src/rt_reporter.erl index 34a315048..575b891d9 100644 --- a/src/rt_reporter.erl +++ b/src/rt_reporter.erl @@ -299,24 +299,9 @@ format_test_row({TestPlan, Result, Duration}) -> -spec(report_to_giddyup(term(), list()) -> list). report_to_giddyup(TestResult, Logs) -> {TestPlan, Reason, _Duration} = TestResult, - Status = case Reason of - pass -> pass; - _ -> fail - end, - GiddyupResult = [ - {test, rt_test_plan:get_module(TestPlan)}, - {status, Status}, - {backend, rt_test_plan:get(backend, TestPlan)}, - {id, rt_test_plan:get(id, TestPlan)}, - {platform, rt_test_plan:get(platform, TestPlan)}, - {version, rt_test_plan:get(version, TestPlan)}, - {project, rt_test_plan:get(project, TestPlan)} - ], - case giddyup:post_result(GiddyupResult) of - error -> woops; - {ok, Base} -> - [giddyup:post_artifact(Base, File) || File <- Logs] - end. + giddyup:post_result(TestPlan, Reason), + [giddyup:post_artifact(TestPlan, Label, Filename) || {Label, Filename} <- Logs]. + %%-------------------------------------------------------------------- %% @private %% @doc @@ -331,7 +316,7 @@ report_to_giddyup(TestResult, Logs) -> report_and_gather_logs(UploadToGiddyUp, LogDir, TestResult = {TestPlan, _, _}) -> SubDir = filename:join([LogDir, rt_test_plan:get_name(TestPlan)]), LogFile = filename:join([SubDir, "riak_test.log"]), - Logs = rt:get_node_logs(UploadToGiddyUp, LogFile, SubDir), + Logs = rt:get_node_logs(LogFile, SubDir), case UploadToGiddyUp of true -> report_to_giddyup(TestResult, Logs); diff --git a/src/rtdev.erl b/src/rtdev.erl index 38c43570d..d673973d4 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -50,7 +50,7 @@ set_advanced_conf/2, rm_dir/1, validate_config/1, - get_node_logs/3]). + get_node_logs/2]). -compile(export_all). -include_lib("eunit/include/eunit.hrl"). @@ -964,24 +964,23 @@ devpaths() -> %% proplists:get_keys(rt_config:get(rtdev_path)) -- [root]. % @doc Get the list of log files and config files and pass them back --spec(get_node_logs(boolean(), string(), string()) -> list()). -get_node_logs(UploadToGiddyUp, LogFile, DestDir) -> +-spec(get_node_logs(string(), string()) -> list()). +get_node_logs(LogFile, DestDir) -> Root = filename:absname(?PATH), RootLen = length(Root) + 1, %% Remove the leading slash - Fun = get_node_log_fun(UploadToGiddyUp, DestDir, RootLen), + Fun = get_node_log_fun(DestDir, RootLen), NodeLogs = [ Fun(Filename) || Filename <- filelib:wildcard(Root ++ "/*/dev*/log/*") ++ filelib:wildcard(Root ++ "/*/dev*/etc/*.conf*") ], %% Trim the Lager file path slightly differently LagerFile = filename:absname(LogFile), LagerLen = length(filename:dirname(LagerFile)) + 1, - LagerFun = get_node_log_fun(UploadToGiddyUp, DestDir, LagerLen), + LagerFun = get_node_log_fun(DestDir, LagerLen), LagerLog = LagerFun(LagerFile), lists:append([LagerLog], NodeLogs). -% @doc Either open a port for uploading each file to GiddyUp or -% Copy each file to a local directory --spec(get_node_log_fun(boolean(), string(), integer()) -> fun()). -get_node_log_fun(UploadToGiddyUp, DestDir, RootLen) -> +% @doc Copy each file to a local directory +-spec(get_node_log_fun(string(), integer()) -> fun()). +get_node_log_fun(DestDir, RootLen) -> DestRoot = filename:absname(DestDir), lager:debug("Copying log files to ~p", [DestRoot]), fun(Filename) -> @@ -994,14 +993,7 @@ get_node_log_fun(UploadToGiddyUp, DestDir, RootLen) -> lager:debug("Copying ~p to ~p", [Filename, Target]), {ok, _BytesWritten} = file:copy(Filename, Target) end, - %% Open a port if this is to be uploaded to GiddyUp - case UploadToGiddyUp of - true -> - {ok, Port} = file:open(Target, [read, binary]), - {lists:nthtail(RootLen, Filename), Port}; - _ -> - Target - end + {lists:nthtail(RootLen, Filename), Target} end. -type node_tuple() :: {list(), atom()}. From a529f17e8d93597cdefb0e758b40f53ccf9445f9 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Wed, 22 Apr 2015 12:50:10 -0600 Subject: [PATCH 133/157] Fix creation of test plans from GiddyUp --- src/giddyup.erl | 11 ++++++----- src/rt_planner.erl | 35 ++++++++++++++++++++++++----------- src/rt_test_plan.erl | 4 ++-- 3 files changed, 32 insertions(+), 18 deletions(-) diff --git a/src/giddyup.erl b/src/giddyup.erl index b77bf28ff..39b04c925 100644 --- a/src/giddyup.erl +++ b/src/giddyup.erl @@ -266,12 +266,12 @@ fetch_all_test_plans(Platform, Product, VersionNumber, DefaultVersion, Host) -> Id = kvc:path(id, Test), Module = binary_to_atom(kvc:path(name, Test), utf8), Plan0 = rt_test_plan:new([{id, Id}, {module, Module}, {project, Project}, {platform, Platform}, {version, VersionNumber}]), - Plan1 = case kvc:path('tags.backend', Test) of - [] -> Plan0; - X -> rt_test_plan:set(backend, binary_to_atom(X, utf8), Plan0) + {ok, Plan1} = case kvc:path('tags.backend', Test) of + [] -> {ok, Plan0}; + Backend -> rt_test_plan:set(backend, binary_to_atom(Backend, utf8), Plan0) end, - Plan2 = case kvc:path('tags.upgrade_version', Test) of - [] -> Plan1; + {ok, Plan2} = case kvc:path('tags.upgrade_version', Test) of + [] -> {ok, Plan1}; UpgradeVsn -> UpgradeVersion = case UpgradeVsn of <<"legacy">> -> rt_config:get_legacy_version(); @@ -285,6 +285,7 @@ fetch_all_test_plans(Platform, Product, VersionNumber, DefaultVersion, Host) -> %% [] -> Plan1; %% MultiConfig -> rt_test_plan:set(multi_config, binary_to_atom(MultiConfig, utf8), Plan2) %%end, + lager:debug("Giddyup Module ~p using TestPlan ~p", [Module, Plan2]), Plan2 end, [ TestProps(Test) || Test <- Tests]. diff --git a/src/rt_planner.erl b/src/rt_planner.erl index 9a6114ea9..4cb62620d 100644 --- a/src/rt_planner.erl +++ b/src/rt_planner.erl @@ -187,23 +187,26 @@ init([]) -> %% If none are specified, run everything handle_call({load_from_giddyup, Backends, CommandLineTests}, _From, State) -> AllGiddyupTests = giddyup:get_test_plans(), - FilteredNames = case CommandLineTests of + {Included, Excluded} = case CommandLineTests of [] -> - AllGiddyupTests; + {AllGiddyupTests, []}; _ -> - [TestPlan || TestPlan <- AllGiddyupTests, - CName <- CommandLineTests, - rt_test_plan:get_module(TestPlan) =:= CName] + Inc = [TestPlan || TestPlan <- AllGiddyupTests, + CName <- CommandLineTests, + rt_test_plan:get_module(TestPlan) =:= CName], + {Inc, lists:filter(fun(Elem) -> not lists:member(Elem, Inc) end, AllGiddyupTests)} end, - FilteredTests = case Backends of + {Included1, Excluded1} = case Backends of undefined -> - FilteredNames; + {Included, Excluded}; _ -> - [TestPlan || TestPlan <- FilteredNames, - lists:member(rt_test_plan:get(backend, TestPlan), Backends)] + Inc1 = [TestPlan || TestPlan <- Included, + lists:member(rt_test_plan:get(backend, TestPlan), Backends)], + {Inc1, lists:filter(fun(Elem) -> not lists:member(Elem, Inc1) end, AllGiddyupTests)} end, - State1 = lists:foldl(fun sort_and_queue/2, State, FilteredTests), - {reply, ok, State1}; + State1 = lists:foldl(fun sort_and_queue/2, State, Included1), + State2 = lists:foldl(fun exclude_test_plan/2, State1, Excluded1), + {reply, ok, State2}; %% Add a single test plan to the queue handle_call({add_test_plan, Module, Platform, Backends, _Version, _Properties}, _From, State) -> State1 = lists:foldl(fun(Backend, AccState) -> @@ -330,3 +333,13 @@ is_runnable_test_plan(TestPlan) -> code:ensure_loaded(Mod), erlang:function_exported(Mod, Fun, 0) orelse erlang:function_exported(Mod, Fun, 1). + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Add a unused test to the list of non_runnable_test_plans +%% @end +%%-------------------------------------------------------------------- +exclude_test_plan(TestPlan, State) -> + QNR = queue:in(TestPlan, State#state.non_runnable_test_plans), + State#state{non_runnable_test_plans=QNR}. \ No newline at end of file diff --git a/src/rt_test_plan.erl b/src/rt_test_plan.erl index 55e74d887..a9f7abd9c 100644 --- a/src/rt_test_plan.erl +++ b/src/rt_test_plan.erl @@ -161,13 +161,13 @@ set_fields(_, _, {error, _}=Error) -> validate_request(Field, TestPlan) -> validate_field(Field, validate_record(TestPlan)). --spec validate_record(test_plan()) -> ok | {error, invalid_properties}. +-spec validate_record(test_plan()) -> ok | {error, invalid_test_plan}. validate_record(Record) -> case is_valid_record(Record) of true -> ok; false -> - {error, invalid_properties} + {error, invalid_test_plan} end. -spec validate_field(atom(), ok | {error, atom()}) -> ok | {error, invalid_field}. From d50506c9b155897498f7e63626a00dcff6fae080 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Wed, 22 Apr 2015 13:26:23 -0600 Subject: [PATCH 134/157] Report full test names when showing which tests to run or not run --- src/riak_test_escript.erl | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index 1dfdbdbd4..77e1461c7 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -152,12 +152,12 @@ shuffle_tests(Tests, NonTests, Offset, Workers) -> {TestB ++ TestA, NonTests}. prepare(ParsedArgs, Tests, NonTests) -> - lager:notice("Test to run: ~p", [[rt_test_plan:get_module(Test) || Test <- Tests]]), + [lager:notice("Test to run: ~p", [rt_test_plan:get_name(Test)]) || Test <- Tests], case NonTests of [] -> ok; _ -> - lager:notice("Test not to run: ~p", [[rt_test_plan:get_module(Test) || Test <- NonTests]]) + [lager:notice("Test not to run: ~p", [rt_test_plan:get_name(Test)]) || Test <- NonTests] end, ok = erlang_setup(ParsedArgs), test_setup(). From c55fd462a7de7137ba7fab65d5eb1fc04ac4b9d0 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Wed, 22 Apr 2015 14:49:55 -0600 Subject: [PATCH 135/157] Sync up botched rebase of rt.erl --- src/rt.erl | 287 +---------------------------------------------------- 1 file changed, 2 insertions(+), 285 deletions(-) diff --git a/src/rt.erl b/src/rt.erl index d813c2a5f..d867c429d 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -799,289 +799,6 @@ systest_read(Node, Start, End, Bucket, R, CommonValBin) -> systest_read(Node, Start, End, Bucket, R, CommonValBin, SquashSiblings) -> rt_systest:read(Node, Start, End, Bucket, R, CommonValBin, SquashSiblings). -cap_subset(Val, Cap) when is_list(Cap) -> - sets:is_subset(sets:from_list(Val), sets:from_list(Cap)). - -wait_until_owners_according_to(Node, Nodes) -> - SortedNodes = lists:usort(Nodes), - F = fun(N) -> - rt_ring:owners_according_to(N) =:= SortedNodes - end, - ?assertEqual(ok, wait_until(Node, F)), - ok. - -wait_until_nodes_agree_about_ownership(Nodes) -> -lager:info("Wait until nodes agree about ownership ~p", [Nodes]), -Results = [ wait_until_owners_according_to(Node, Nodes) || Node <- Nodes ], -?assert(lists:all(fun(X) -> ok =:= X end, Results)). - -%%%=================================================================== -%%% Ring Functions -%%%=================================================================== - - -%%%=================================================================== -%%% Cluster Utility Functions -%%%=================================================================== - -%% @doc Safely construct a new cluster and return a list of the deployed nodes -%% @todo Add -spec and update doc to reflect mult-version changes -build_cluster(Versions) when is_list(Versions) -> -build_cluster(length(Versions), Versions, default); -build_cluster(NumNodes) -> -build_cluster(NumNodes, default). - -%% @doc Safely construct a `NumNode' size cluster using -%% `InitialConfig'. Return a list of the deployed nodes. -build_cluster(NumNodes, InitialConfig) -> -build_cluster(NumNodes, [], InitialConfig). - -build_cluster(NumNodes, Versions, InitialConfig) -> -%% Deploy a set of new nodes -Nodes = -case Versions of - [] -> - deploy_nodes(NumNodes, InitialConfig); - _ -> - deploy_nodes(Versions) -end, - -join_cluster(Nodes), -lager:info("Cluster built: ~p", [Nodes]), -Nodes. - -join_cluster(Nodes) -> - %% Ensure each node owns 100% of it's own ring - [?assertEqual([Node], owners_according_to(Node)) || Node <- Nodes], - - %% Potential fix for BTA-116 and other similar "join before nodes ready" issues. - %% TODO: Investigate if there is an actual race in Riak relating to cluster joins. - [ok = wait_for_service(Node, riak_kv) || Node <- Nodes], - - %% Join nodes - [Node1|OtherNodes] = Nodes, - case OtherNodes of - [] -> - %% no other nodes, nothing to join/plan/commit - ok; - _ -> - %% ok do a staged join and then commit it, this eliminates the - %% large amount of redundant handoff done in a sequential join - [staged_join(Node, Node1) || Node <- OtherNodes], - plan_and_commit(Node1), - try_nodes_ready(Nodes, 3, 500) - end, - -?assertEqual(ok, wait_until_nodes_ready(Nodes)), - -%% Ensure each node owns a portion of the ring -wait_until_nodes_agree_about_ownership(Nodes), -?assertEqual(ok, wait_until_no_pending_changes(Nodes)), -ok. - --type products() :: riak | riak_ee | riak_cs | unknown. - --spec product(node()) -> products(). -product(Node) -> - Applications = rpc:call(Node, application, which_applications, []), - - HasRiakCS = proplists:is_defined(riak_cs, Applications), - HasRiakEE = proplists:is_defined(riak_repl, Applications), - HasRiak = proplists:is_defined(riak_kv, Applications), - if HasRiakCS -> riak_cs; - HasRiakEE -> riak_ee; - HasRiak -> riak; - true -> unknown - end. - -try_nodes_ready([Node1 | _Nodes], 0, _SleepMs) -> -lager:info("Nodes not ready after initial plan/commit, retrying"), -plan_and_commit(Node1); -try_nodes_ready(Nodes, N, SleepMs) -> -ReadyNodes = [Node || Node <- Nodes, is_ready(Node) =:= true], -case ReadyNodes of -Nodes -> - ok; -_ -> - timer:sleep(SleepMs), - try_nodes_ready(Nodes, N-1, SleepMs) -end. - -%% @doc Stop nodes and wipe out their data directories -clean_cluster(Nodes) when is_list(Nodes) -> -[stop_and_wait(Node) || Node <- Nodes], -clean_data_dir(Nodes). - -clean_data_dir(Nodes) -> -clean_data_dir(Nodes, ""). - -clean_data_dir(Nodes, SubDir) when not is_list(Nodes) -> -clean_data_dir([Nodes], SubDir); -clean_data_dir(Nodes, SubDir) when is_list(Nodes) -> -?HARNESS:clean_data_dir(Nodes, SubDir). - -%% @doc Shutdown every node, this is for after a test run is complete. -teardown() -> -%% stop all connected nodes, 'cause it'll be faster that -%%lager:info("RPC stopping these nodes ~p", [nodes()]), -%%[ rt:stop(Node) || Node <- nodes()], -%% Then do the more exhaustive harness thing, in case something was up -%% but not connected. -?HARNESS:teardown(). - -versions() -> -?HARNESS:versions(). -%%%=================================================================== -%%% Basic Read/Write Functions -%%%=================================================================== - -systest_write(Node, Size) -> -systest_write(Node, Size, 2). - -systest_write(Node, Size, W) -> -systest_write(Node, 1, Size, <<"systest">>, W). - -systest_write(Node, Start, End, Bucket, W) -> -systest_write(Node, Start, End, Bucket, W, <<>>). - -%% @doc Write (End-Start)+1 objects to Node. Objects keys will be -%% `Start', `Start+1' ... `End', each encoded as a 32-bit binary -%% (`<>'). Object values are the same as their keys. -%% -%% The return value of this function is a list of errors -%% encountered. If all writes were successful, return value is an -%% empty list. Each error has the form `{N :: integer(), Error :: term()}', -%% where N is the unencoded key of the object that failed to store. -systest_write(Node, Start, End, Bucket, W, CommonValBin) -when is_binary(CommonValBin) -> -rt:wait_for_service(Node, riak_kv), -{ok, C} = riak:client_connect(Node), -F = fun(N, Acc) -> - Obj = riak_object:new(Bucket, <>, - <>), - try C:put(Obj, W) of - ok -> - Acc; - Other -> - [{N, Other} | Acc] - catch - What:Why -> - [{N, {What, Why}} | Acc] - end -end, -lists:foldl(F, [], lists:seq(Start, End)). - -systest_read(Node, Size) -> -systest_read(Node, Size, 2). - -systest_read(Node, Size, R) -> -systest_read(Node, 1, Size, <<"systest">>, R). - -systest_read(Node, Start, End, Bucket, R) -> -systest_read(Node, Start, End, Bucket, R, <<>>). - -systest_read(Node, Start, End, Bucket, R, CommonValBin) -when is_binary(CommonValBin) -> -systest_read(Node, Start, End, Bucket, R, CommonValBin, false). - -%% Read and verify the values of objects written with -%% `systest_write'. The `SquashSiblings' parameter exists to -%% optionally allow handling of siblings whose value and metadata are -%% identical except for the dot. This goal is to facilitate testing -%% with DVV enabled because siblings can be created internally by Riak -%% in cases where testing with DVV disabled would not. Such cases -%% include writes that happen during handoff when a vnode forwards -%% writes, but also performs them locally or when a put coordinator -%% fails to send an acknowledgment within the timeout window and -%% another put request is issued. -systest_read(Node, Start, End, Bucket, R, CommonValBin, SquashSiblings) -when is_binary(CommonValBin) -> -rt:wait_for_service(Node, riak_kv), -{ok, C} = riak:client_connect(Node), -lists:foldl(systest_read_fold_fun(C, Bucket, R, CommonValBin, SquashSiblings), - [], - lists:seq(Start, End)). - -systest_read_fold_fun(C, Bucket, R, CommonValBin, SquashSiblings) -> -fun(N, Acc) -> - GetRes = C:get(Bucket, <>, R), - Val = object_value(GetRes, SquashSiblings), - update_acc(value_matches(Val, N, CommonValBin), Val, N, Acc) -end. - -object_value({error, _}=Error, _) -> -Error; -object_value({ok, Obj}, SquashSiblings) -> -object_value(riak_object:value_count(Obj), Obj, SquashSiblings). - -object_value(1, Obj, _SquashSiblings) -> -riak_object:get_value(Obj); -object_value(_ValueCount, Obj, false) -> -riak_object:get_value(Obj); -object_value(_ValueCount, Obj, true) -> -lager:debug("Siblings detected for ~p:~p", [riak_object:bucket(Obj), riak_object:key(Obj)]), -Contents = riak_object:get_contents(Obj), -case lists:foldl(fun sibling_compare/2, {true, undefined}, Contents) of -{true, {_, _, _, Value}} -> - lager:debug("Siblings determined to be a single value"), - Value; -{false, _} -> - {error, siblings} -end. - -sibling_compare({MetaData, Value}, {true, undefined}) -> -Dot = case dict:find(<<"dot">>, MetaData) of - {ok, DotVal} -> - DotVal; - error -> - {error, no_dot} - end, -VTag = dict:fetch(<<"X-Riak-VTag">>, MetaData), -LastMod = dict:fetch(<<"X-Riak-Last-Modified">>, MetaData), -{true, {element(2, Dot), VTag, LastMod, Value}}; -sibling_compare(_, {false, _}=InvalidMatch) -> -InvalidMatch; -sibling_compare({MetaData, Value}, {true, PreviousElements}) -> -Dot = case dict:find(<<"dot">>, MetaData) of - {ok, DotVal} -> - DotVal; - error -> - {error, no_dot} - end, -VTag = dict:fetch(<<"X-Riak-VTag">>, MetaData), -LastMod = dict:fetch(<<"X-Riak-Last-Modified">>, MetaData), -ComparisonElements = {element(2, Dot), VTag, LastMod, Value}, -{ComparisonElements =:= PreviousElements, ComparisonElements}. - -value_matches(<>, N, CommonValBin) -> -true; -value_matches(_WrongVal, _N, _CommonValBin) -> -false. - -update_acc(true, _, _, Acc) -> -Acc; -update_acc(false, {error, _}=Val, N, Acc) -> -[{N, Val} | Acc]; -update_acc(false, Val, N, Acc) -> -[{N, {wrong_val, Val}} | Acc]. - -verify_systest_value(N, Acc, CommonValBin, Obj) -> -Values = riak_object:get_values(Obj), -Res = [begin - case V of - <> -> - ok; - _WrongVal -> - wrong_val - end - end || V <- Values], -case lists:any(fun(X) -> X =:= ok end, Res) of -true -> - Acc; -false -> - [{N, {wrong_val, hd(Values)}} | Acc] -end. - % @doc Reads a single replica of a value. This issues a get command directly % to the vnode handling the Nth primary partition of the object's preflist. get_replica(Node, Bucket, Key, I, N) -> @@ -1414,7 +1131,7 @@ wait_for_control(VersionedNodes) -> verify_product(Applications, ExpectedApplication) -> ?_test(begin meck:new(rpc, [unstick]), - meck:expect(rpc, call, fun([], application, which_applications, []) -> + meck:expect(rpc, call, fun([], application, which_applications, []) -> Applications end), ?assertMatch(ExpectedApplication, product([])), meck:unload(rpc) @@ -1429,5 +1146,5 @@ product_test_() -> verify_product([riak_repl, riak_kv], riak_ee), verify_product([riak_kv], riak), verify_product([kernel], unknown)]}. - + -endif. From 4d70aaaa81b8427e38a0bfb507b0b1c6eebd54dd Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Wed, 22 Apr 2015 17:47:20 -0600 Subject: [PATCH 136/157] Use the full name of a test case (including backend) when reporting --- src/rt_reporter.erl | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/rt_reporter.erl b/src/rt_reporter.erl index 575b891d9..c107dc823 100644 --- a/src/rt_reporter.erl +++ b/src/rt_reporter.erl @@ -287,13 +287,12 @@ test_summary_fun(Result = {_, {skipped, _}, _}, {{_Pass, _Fail, Skipped}, Rows}) %% @doc Format a row for clique format_test_row({TestPlan, Result, Duration}) -> - TestModule = rt_test_plan:get_module(TestPlan), - TestString = atom_to_list(TestModule), + TestName = rt_test_plan:get_name(TestPlan), case Result of {Status, Reason} -> - [TestString, Status, Reason, test_summary_format_time(Duration)]; + [TestName, Status, Reason, test_summary_format_time(Duration)]; pass -> - [TestString, "pass", "N/A", test_summary_format_time(Duration)] + [TestName, "pass", "N/A", test_summary_format_time(Duration)] end. -spec(report_to_giddyup(term(), list()) -> list). From cc9fd329f44308e3d357e21fb515d71ade025d60 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Thu, 23 Apr 2015 14:57:05 -0600 Subject: [PATCH 137/157] Update riak_test.config Add a few notes about debugging and version info --- examples/riak_test.config | 73 +++++++++++++++++++++++++-------------- 1 file changed, 47 insertions(+), 26 deletions(-) diff --git a/examples/riak_test.config b/examples/riak_test.config index 82a245761..535e9eaca 100644 --- a/examples/riak_test.config +++ b/examples/riak_test.config @@ -22,7 +22,7 @@ %% builder. Typically this is in the format %% "NAME-VERSION-ARCHITECTURE". See GiddyUp for valid platform %% names. - {platform, "osx-64"}, + {giddyup_platform, "osx-64"}, %% riak_test includes various wait_for_X functions that will %% repeatedly test for specific conditions until they are @@ -71,6 +71,11 @@ %% test writers. {lager_level, info}, + %% lager_console_level defaults to info, which is should mean + %% "relevant test output" to the console. debug level output is for helping + %% test writers. + {lager_console_level, debug}, + %% Number of processes to run during testing {workers, 5}, {offset, 2} @@ -88,28 +93,34 @@ %% The name of the project/product, used when fetching the test %% suite and reporting. {rt_project, "riak"}, - + %% Paths to the locations of various versions of the project. This %% is only valid for the `rtdev' harness. - {rtdev_path, [ - %% This is the root of the built `rtdev' repository, - %% used for manipulating the repo with git. All - %% versions should be inside this directory. - {root, "/Users/dparfitt/rt/riak"}, - - %% The path to the `current' version, which is used - %% exclusively except during upgrade tests. - {current, "/Users/dparfitt/rt/riak/current"}, - - %% The path to the most immediately previous version - %% of the project, which is used when doing upgrade - %% tests. - {previous, "/Users/dparfitt/rt/riak/riak-1.2.1"}, - - %% The path to the version before `previous', which - %% is used when doing upgrade tests. - {legacy, "/Users/dparfitt/rt/riak/riak-1.1.4"} - ]} + {root_path, "/Users/hazen/dev/rt/riak"}, + {versions, [ + %% The path to the version before `previous', which + %% is used when doing upgrade tests. + %% Versions are tuple of product and version number + {'latest_1.3_ee', {riak, "1.3.4"}}, + %% The path to the most immediately previous version + %% of the project, which is used when doing upgrade + %% tests. + {'latest_1.4_ee', {riak, "1.4.12"}}, + %% The path to the `current' version, which is used + %% exclusively except during upgrade tests. + {'latest_2.0_ee', {riak, "2.0.5"}}, + {'previous_2.0_ee', {riak, "2.0.4"}}, + %% Backwards-compatible aliases to versions + {default, 'latest_2.0_ee'}, + {previous, 'latest_1.4_ee'}, + {legacy, 'latest_1.3_ee'} + ]}, + {upgrade_paths, [ + %% Lists of aliases to possible upgrade versions + {full, ['latest_1.3_ee', 'latest_1.4_ee', 'latest_2.0_ee']}, + {latest, ['latest_1.4_ee', 'latest_2.0_ee']}, + {minor, ['previous_2.0_ee', 'latest_2.0_ee']} + ]} ]}. %% Sample project for Riak EDS ("EE"). @@ -133,11 +144,21 @@ {repl_upgrade_order, "forwards"}, %% [See rtdev.rtdev_path above] - {rtdev_path, [{root, "/Users/dparfitt/rt/riak_ee"}, - {current, "/Users/dparfitt/rt/riak_ee/current"}, - {previous, "/Users/dparfitt/rt/riak_ee/riak_ee-1.2.1"}, - {legacy, "/Users/dparfitt/rt/riak_ee/riak_ee-1.1.4"} - ]} + {root_path, "/Users/hazen/dev/rt/riak"}, + {versions, [ + {'latest_1.3_ee', {riak_ee, "1.3.4"}}, + {'latest_1.4_ee', {riak_ee, "1.4.12"}}, + {'latest_2.0_ee', {riak_ee, "2.0.5"}}, + {'previous_2.0_ee', {riak_ee, "2.0.4"}}, + {default, 'latest_2.0_ee'}, + {previous, 'latest_1.4_ee'}, + {legacy, 'latest_1.3_ee'} + ]}, + {upgrade_paths, [ + {full, ['latest_1.3_ee', 'latest_1.4_ee', 'latest_2.0_ee']}, + {latest, ['latest_1.4_ee', 'latest_2.0_ee']}, + {minor, ['previous_2.0_ee', 'latest_2.0_ee']} + ]} ]}. %% Sample project to demonstrate use of local configs for the From 6b6afe2149417cd91c5666847267dc71c0d6a79c Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Thu, 23 Apr 2015 14:57:59 -0600 Subject: [PATCH 138/157] Add better debugging messages when no tests are specified and when giddyup_host is not set in config file --- src/riak_test_escript.erl | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index 77e1461c7..499106749 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -92,7 +92,7 @@ report_platform(ParsedArgs) -> undefined -> undefined; "config" -> - rt_config:get(giddyup_platform, undefined); + rt_config:get(giddyup_platform); R -> R end. @@ -132,7 +132,8 @@ load_initial_config(ParsedArgs) -> %% @doc Shuffle the order in which tests are scheduled shuffle_tests([], _, _, _) -> - lager:error("No tests are scheduled to run~n"), + io:format("ERROR: No tests are scheduled to run~n"), + lager:error("No tests are scheduled to run"), halt(1); shuffle_tests(Tests, NonTests, undefined, _) -> {Tests, NonTests}; From c59d537261c57b2e28e09f497ad0398390572148 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Fri, 24 Apr 2015 16:29:43 -0600 Subject: [PATCH 139/157] Update riak_test.config Add comment and example of `continue_on_fail' --- examples/riak_test.config | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/examples/riak_test.config b/examples/riak_test.config index 535e9eaca..f72b3564f 100644 --- a/examples/riak_test.config +++ b/examples/riak_test.config @@ -78,7 +78,11 @@ %% Number of processes to run during testing {workers, 5}, - {offset, 2} + {offset, 2}, + + %% Continue to run subsequent test cases instead of halting immediately. + %% False by default + {continue_on_fail, false} ]}. %% =============================================================== From 8704610f3a4544f3421c95e9f251091f2e1ea9bd Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Sun, 26 Apr 2015 21:32:03 -0600 Subject: [PATCH 140/157] Change how results table is written to lager file --- src/rt_reporter.erl | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/rt_reporter.erl b/src/rt_reporter.erl index c107dc823..54e0b10ef 100644 --- a/src/rt_reporter.erl +++ b/src/rt_reporter.erl @@ -237,7 +237,7 @@ print_summary(TestResults, _CoverResult, Verbose) -> %% TODO: Remove once clique table is fixed [lager:debug("ROW ~p", [Row]) || Row <- Rows], Table = clique_table:autosize_create_table(?HEADER, Rows), - lager:notice("~ts", [Table]); + [lager:notice(string:tokens(lists:flatten(FormattedRow), "\n")) || FormattedRow <- Table]; false -> ok end, @@ -290,7 +290,7 @@ format_test_row({TestPlan, Result, Duration}) -> TestName = rt_test_plan:get_name(TestPlan), case Result of {Status, Reason} -> - [TestName, Status, Reason, test_summary_format_time(Duration)]; + [TestName, Status, lists:flatten(Reason), test_summary_format_time(Duration)]; pass -> [TestName, "pass", "N/A", test_summary_format_time(Duration)] end. From a3671c9fec1007fc7131d2d1ff9d95b7e44812a5 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Tue, 28 Apr 2015 17:35:35 -0600 Subject: [PATCH 141/157] Fix unit tests for rt_config --- src/rt_config.erl | 28 +++++++++++++++------------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/src/rt_config.erl b/src/rt_config.erl index 793e00996..95b531632 100644 --- a/src/rt_config.erl +++ b/src/rt_config.erl @@ -52,9 +52,13 @@ -define(PREVIOUS_VERSION_KEY, previous). -define(LEGACY_VERSION_KEY, legacy). -define(DEFAULT_VERSION, head). +-define(DEFAULT_ACTUAL_VERSION, {riak_ee, "2.1.0"}). +-define(DEFAULT_ACTUAL_VERSION_TAG, "riak_ee-2.1.0"). -define(UPGRADE_KEY, upgrade_paths). --define(PREVIOUS_VERSION, "1.4.12"). --define(LEGACY_VERSION, "1.3.4"). +-define(PREVIOUS_VERSION, {riak_ee, "1.4.12"}). +-define(PREVIOUS_VERSION_TAG, "riak_ee-1.4.12"). +-define(LEGACY_VERSION, {riak_ee, "1.3.4"}). +-define(LEGACY_VERSION_TAG, "riak_ee-1.3.4"). -define(CONTINUE_ON_FAIL_KEY, continue_on_fail). -define(DEFAULT_CONTINUE_ON_FAIL, false). @@ -317,17 +321,15 @@ get_rt_max_wait_time_default_test() -> ?assertEqual(ExpectedWaitTime, get(rt_max_wait_time, DefaultWaitTime)). get_version_path_test() -> - clear(?DEFAULT_VERSION_KEY), - clear(?PREVIOUS_VERSION_KEY), - clear(?LEGACY_VERSION_KEY), - - set(?DEFAULT_VERSION_KEY, ?DEFAULT_VERSION), - set(?PREVIOUS_VERSION_KEY, ?PREVIOUS_VERSION), - set(?LEGACY_VERSION_KEY, ?LEGACY_VERSION), - - ?assertEqual(version_to_tag(?DEFAULT_VERSION_KEY), ?DEFAULT_VERSION), - ?assertEqual(version_to_tag(?PREVIOUS_VERSION_KEY), ?PREVIOUS_VERSION), - ?assertEqual(version_to_tag(?LEGACY_VERSION_KEY), ?LEGACY_VERSION). + Versions = [{?DEFAULT_VERSION_KEY, ?DEFAULT_VERSION}, + {?DEFAULT_VERSION, ?DEFAULT_ACTUAL_VERSION}, + {?PREVIOUS_VERSION_KEY, ?PREVIOUS_VERSION}, + {?LEGACY_VERSION_KEY, ?LEGACY_VERSION}], + ok = rt_config:set(?VERSION_KEY, Versions), + + ?assertEqual(version_to_tag(?DEFAULT_VERSION_KEY), ?DEFAULT_ACTUAL_VERSION_TAG), + ?assertEqual(version_to_tag(?PREVIOUS_VERSION_KEY), ?PREVIOUS_VERSION_TAG), + ?assertEqual(version_to_tag(?LEGACY_VERSION_KEY), ?LEGACY_VERSION_TAG). get_continue_on_fail_test() -> clear(?CONTINUE_ON_FAIL_KEY), From d6dca14bc4747871328565fec87b070a4fb69214 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Thu, 30 Apr 2015 09:43:00 -0600 Subject: [PATCH 142/157] Register Erlang setup before determining which tests to run so YZ tests can be run --- src/riak_test_escript.erl | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index 499106749..b760fa98c 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -35,6 +35,7 @@ main(Args) -> OutDir = proplists:get_value(outdir, ParsedArgs, "log"), ensure_dir(OutDir), lager_setup(OutDir), + ok = erlang_setup(ParsedArgs), %% Do we use GiddyUp for this run? Platform = report_platform(ParsedArgs), @@ -45,7 +46,7 @@ main(Args) -> start_giddyup(Platform), {Tests, NonTests} = generate_test_lists(UseGiddyUp, ParsedArgs), - ok = prepare(ParsedArgs, Tests, NonTests), + ok = prepare_tests(Tests, NonTests), Results = execute(Tests, OutDir, ParsedArgs), finalize(Results, ParsedArgs), stop_giddyup(UseGiddyUp). @@ -152,7 +153,7 @@ shuffle_tests(Tests, NonTests, Offset, Workers) -> [TestCount, ActualOffset, Workers, Offset]), {TestB ++ TestA, NonTests}. -prepare(ParsedArgs, Tests, NonTests) -> +prepare_tests(Tests, NonTests) -> [lager:notice("Test to run: ~p", [rt_test_plan:get_name(Test)]) || Test <- Tests], case NonTests of [] -> @@ -160,7 +161,6 @@ prepare(ParsedArgs, Tests, NonTests) -> _ -> [lager:notice("Test not to run: ~p", [rt_test_plan:get_name(Test)]) || Test <- NonTests] end, - ok = erlang_setup(ParsedArgs), test_setup(). execute(TestPlans, OutDir, ParsedArgs) -> From 92fd1e5554232b6c2921c004329c0515198837b8 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Thu, 30 Apr 2015 12:16:47 -0600 Subject: [PATCH 143/157] Addresses BTA-207 to fix cuttlefish_configuration test and rt_harness_util:deploy_nodes/5 to handle cuttlefish --- src/rt_harness_util.erl | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/src/rt_harness_util.erl b/src/rt_harness_util.erl index e86c8aaa7..e77a381e1 100644 --- a/src/rt_harness_util.erl +++ b/src/rt_harness_util.erl @@ -89,9 +89,17 @@ deploy_nodes(NodeIds, NodeMap, Version, Config, Services) -> create_dirs(Version, NodeIds), %% Set initial config + %% TODO: Cuttlefish should not need to have Nodes be atoms explicitly ConfigUpdateFun = - fun(Node) -> - rt_harness:update_app_config(Node, Version, Config) + case Config of + {cuttlefish, CuttleConfig} -> + fun(Node) -> + rt_harness:set_conf(list_to_atom(Node), CuttleConfig) + end; + _ -> + fun(Node) -> + rt_harness:update_app_config(Node, Version, Config) + end end, rt2:pmap(ConfigUpdateFun, NodeIds), From 1428fca6cba1fda5eb5dc4f2e6ed6fbdaf8c3675 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Tue, 5 May 2015 10:06:57 -0600 Subject: [PATCH 144/157] Close files after making local copies for GiddyUp --- src/giddyup.erl | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/giddyup.erl b/src/giddyup.erl index 39b04c925..809425257 100644 --- a/src/giddyup.erl +++ b/src/giddyup.erl @@ -393,8 +393,9 @@ post_result(TestPlan, TestResult, #rt_webhook{url=URL, headers=HookHeaders, name post_artifact(BaseURL, Label, Filename, User, Password) -> %% First compute the path of where to post the artifact URL = artifact_url(BaseURL, Label), - {ok, Body} = file:open(Filename, [read, binary]), - ReqBody = make_req_body(Body), + {ok, BodyIoDevice} = file:open(Filename, [read, binary]), + ReqBody = make_req_body(BodyIoDevice), + ok = file:close(BodyIoDevice), CType = guess_ctype(Label), BasicAuth = {basic_auth, {User, Password}}, From fe374eca2af4005360948dfafc523e5c97f1ea26 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Wed, 6 May 2015 11:38:43 -0600 Subject: [PATCH 145/157] Fix clique table for atoms as failure reasons --- src/rt_reporter.erl | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/src/rt_reporter.erl b/src/rt_reporter.erl index 54e0b10ef..bd5d1eba2 100644 --- a/src/rt_reporter.erl +++ b/src/rt_reporter.erl @@ -288,12 +288,15 @@ test_summary_fun(Result = {_, {skipped, _}, _}, {{_Pass, _Fail, Skipped}, Rows}) %% @doc Format a row for clique format_test_row({TestPlan, Result, Duration}) -> TestName = rt_test_plan:get_name(TestPlan), - case Result of - {Status, Reason} -> - [TestName, Status, lists:flatten(Reason), test_summary_format_time(Duration)]; + {Status, Reason} = case Result of + {FailOrSkip, Failure} when is_list(Failure) -> + {FailOrSkip, lists:flatten(Failure)}; + {FailOrSkip, Failure} -> + {FailOrSkip, lists:flatten(io_lib:format("~p", [Failure]))}; pass -> - [TestName, "pass", "N/A", test_summary_format_time(Duration)] - end. + {"pass", "N/A"} + end, + [TestName, Status, Reason, test_summary_format_time(Duration)]. -spec(report_to_giddyup(term(), list()) -> list). report_to_giddyup(TestResult, Logs) -> From 2019b2ec812d648ff155a4b892b749bd69e10fa9 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Wed, 6 May 2015 18:38:13 -0600 Subject: [PATCH 146/157] Add unit tests for results table --- src/rt_reporter.erl | 61 ++++++++++++++++++++++++++++++++++++++++----- 1 file changed, 55 insertions(+), 6 deletions(-) diff --git a/src/rt_reporter.erl b/src/rt_reporter.erl index bd5d1eba2..759377f65 100644 --- a/src/rt_reporter.erl +++ b/src/rt_reporter.erl @@ -30,6 +30,10 @@ -behaviour(gen_server). -define(HEADER, [<<"Test">>, <<"Result">>, <<"Reason">>, <<"Test Duration">>]). +-ifdef(TEST). +-include_lib("eunit/include/eunit.hrl"). +-endif. + %% API -export([start_link/3, @@ -263,16 +267,19 @@ print_summary(TestResults, _CoverResult, Verbose) -> %% end, ok. -%% @doc Convert Milliseconds into human-readable string +%% @doc Convert Microseconds into human-readable string -spec(test_summary_format_time(integer()) -> string()). -test_summary_format_time(Milliseconds) -> - Mills = trunc(((Milliseconds / 1000000) - (Milliseconds div 1000000)) * 1000000), - TotalSecs = (Milliseconds - Mills) div 1000000, +test_summary_format_time(Microseconds) -> + Micros = trunc(((Microseconds / 1000000) - (Microseconds div 1000000)) * 1000000), + TotalSecs = (Microseconds - Micros) div 1000000, TotalMins = TotalSecs div 60, Hours = TotalSecs div 3600, Secs = TotalSecs - (TotalMins * 60), Mins = TotalMins - (Hours * 60), - list_to_binary(io_lib:format("~ph ~pm ~p.~ps", [Hours, Mins, Secs, Mills])). + Decimal = lists:flatten(io_lib:format("~6..0B", [Micros])), + FirstDigit = string:left(Decimal, 1), + Fractional = string:strip(tl(Decimal), right, $0), + list_to_binary(io_lib:format("~ph ~pm ~p.~s~ss", [Hours, Mins, Secs, FirstDigit, Fractional])). %% @doc Count the number of passed, failed and skipped tests test_summary_fun(Result = {_, pass, _}, {{Pass, _Fail, _Skipped}, Rows}) -> @@ -294,7 +301,7 @@ format_test_row({TestPlan, Result, Duration}) -> {FailOrSkip, Failure} -> {FailOrSkip, lists:flatten(io_lib:format("~p", [Failure]))}; pass -> - {"pass", "N/A"} + {pass, "N/A"} end, [TestName, Status, Reason, test_summary_format_time(Duration)]. @@ -332,3 +339,45 @@ report_and_gather_logs(UploadToGiddyUp, LogDir, TestResult = {TestPlan, _, _}) - %% fail -> RetList ++ [{reason, iolist_to_binary(io_lib:format("~p", [Reason]))}]; %% _ -> RetList %% end. + +-ifdef(TEST). + +format_result_row_pass_test() -> + %% Need to prime the config with any old default version + rt_config:set(versions, [{default, {riak_ee, "1.3.4"}}]), + Plan = rt_test_plan:new([{module,test},{backend,bitcask}]), + ?assertEqual(["test-bitcask", pass, "N/A", <<"0h 0m 0.012345s">>], format_test_row({Plan, pass, 12345})). + +format_result_row_fail_atom_test() -> + %% Need to prime the config with any old default version + rt_config:set(versions, [{default, {riak_ee, "1.3.4"}}]), + Plan = rt_test_plan:new([{module,test},{backend,bitcask}]), + ?assertEqual(["test-bitcask", fail, "timeout", <<"0h 0m 0.012345s">>], format_test_row({Plan, {fail,timeout}, 12345})). + +format_result_row_fail_string_test() -> + %% Need to prime the config with any old default version + rt_config:set(versions, [{default, {riak_ee, "1.3.4"}}]), + Plan = rt_test_plan:new([{module,test},{backend,bitcask}]), + ?assertEqual(["test-bitcask", fail, "some reason", <<"0h 0m 0.012345s">>], format_test_row({Plan, {fail,"some reason"}, 12345})). + +format_result_row_fail_list_test() -> + %% Need to prime the config with any old default version + rt_config:set(versions, [{default, {riak_ee, "1.3.4"}}]), + Plan = rt_test_plan:new([{module,test},{backend,bitcask}]), + ?assertEqual(["test-bitcask", fail, "nested", <<"0h 0m 0.012345s">>], format_test_row({Plan, {fail,[[$n],[$e],[[$s]],[$t],$e,$d]}, 12345})). + +format_time_microsecond_test() -> + ?assertEqual(<<"0h 0m 0.000001s">>, test_summary_format_time(1)). + +format_time_millisecond_test() -> + ?assertEqual(<<"0h 0m 0.001s">>, test_summary_format_time(1000)). + +format_time_second_test() -> + ?assertEqual(<<"0h 0m 1.0s">>, test_summary_format_time(1000000)). + +format_time_minute_test() -> + ?assertEqual(<<"0h 1m 0.0s">>, test_summary_format_time(60000000)). + +format_time_hour_test() -> + ?assertEqual(<<"1h 0m 0.0s">>, test_summary_format_time(3600000000)). +-endif. \ No newline at end of file From a212c4046685a7641ddbc1739d1088a547a543fe Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Fri, 8 May 2015 16:30:20 -0600 Subject: [PATCH 147/157] Resync tests from master --- src/riak_test_runner.erl | 22 ++-- tests/pipe_verify_basics.erl | 2 - tests/pipe_verify_exceptions.erl | 2 - .../pipe_verify_restart_input_forwarding.erl | 2 - tests/pipe_verify_sink_types.erl | 2 - tests/replication/repl_bucket_types.erl | 1 + tests/replication/rt_cascading.erl | 90 ++++++++-------- tests/yz_core_properties_create_unload.erl | 102 ++++++++++++++---- tests/yz_crdt.erl | 90 ++++++++++++---- tests/yz_handoff.erl | 79 ++++++++------ 10 files changed, 255 insertions(+), 137 deletions(-) diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index 694c81005..3d4f665d5 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -391,17 +391,17 @@ test_fun(ConfirmFun, NotifyPid) -> %% Store the FSM Pid for use in unported tests put(test_runner_fsm, NotifyPid), %% Exceptions and their handling sucks, but eunit throws - %% errors `erlang:error' so here we are - try ConfirmFun() of - TestResult -> - ?MODULE:send_event(NotifyPid, test_result(TestResult)) - catch - Error:Reason -> - lager:error("Failed to execute confirm function ~p due to ~p with reason ~p (trace: ~p)", - [ConfirmFun, Error, Reason, erlang:get_stacktrace()]), - TestResult = format_eunit_error(Reason), - ?MODULE:send_event(NotifyPid, test_result(TestResult)) - end + %% errors `erlang:error' so here we are + try ConfirmFun() of + TestResult -> + ?MODULE:send_event(NotifyPid, test_result(TestResult)) + catch + Error:Reason -> + lager:error("Failed to execute confirm function ~p due to ~p with reason ~p (trace: ~p)", + [ConfirmFun, Error, Reason, erlang:get_stacktrace()]), + TestResult = format_eunit_error(Reason), + ?MODULE:send_event(NotifyPid, test_result(TestResult)) + end end. format_eunit_error({assertion_failed, InfoList}) -> diff --git a/tests/pipe_verify_basics.erl b/tests/pipe_verify_basics.erl index 878c2cede..67ea4f5e7 100644 --- a/tests/pipe_verify_basics.erl +++ b/tests/pipe_verify_basics.erl @@ -44,8 +44,6 @@ confirm() -> [rt:wait_for_service(Node, riak_pipe) || Node <- Nodes], - [rt:wait_for_service(Node, riak_pipe) || Node <- Nodes], - rt:load_modules_on_nodes([?MODULE], Nodes), verify_order(Nodes), diff --git a/tests/pipe_verify_exceptions.erl b/tests/pipe_verify_exceptions.erl index accf5565e..fd0b42c59 100644 --- a/tests/pipe_verify_exceptions.erl +++ b/tests/pipe_verify_exceptions.erl @@ -49,8 +49,6 @@ confirm() -> [rt:wait_for_service(Node, riak_pipe) || Node <- Nodes], - [rt:wait_for_service(Node, riak_pipe) || Node <- Nodes], - rt:load_modules_on_nodes([?MODULE, rt_pipe], Nodes), verify_xbad1(Nodes), diff --git a/tests/pipe_verify_restart_input_forwarding.erl b/tests/pipe_verify_restart_input_forwarding.erl index 945c680a0..1df9301ac 100644 --- a/tests/pipe_verify_restart_input_forwarding.erl +++ b/tests/pipe_verify_restart_input_forwarding.erl @@ -56,8 +56,6 @@ confirm() -> [rt:wait_for_service(Node, riak_pipe) || Node <- Nodes], - [rt:wait_for_service(Node, riak_pipe) || Node <- Nodes], - rt:load_modules_on_nodes([?MODULE, rt_pipe], Nodes), verify_worker_restart_failure_input_forwarding(Nodes), diff --git a/tests/pipe_verify_sink_types.erl b/tests/pipe_verify_sink_types.erl index ec883d2aa..32b42d50c 100644 --- a/tests/pipe_verify_sink_types.erl +++ b/tests/pipe_verify_sink_types.erl @@ -43,8 +43,6 @@ confirm() -> [rt:wait_for_service(Node, riak_pipe) || Node <- Nodes], - [rt:wait_for_service(Node, riak_pipe) || Node <- Nodes], - verify_raw(Nodes), verify_fsm(Nodes), verify_fsm_timeout(Nodes), diff --git a/tests/replication/repl_bucket_types.erl b/tests/replication/repl_bucket_types.erl index b993a592d..aacec149b 100644 --- a/tests/replication/repl_bucket_types.erl +++ b/tests/replication/repl_bucket_types.erl @@ -429,6 +429,7 @@ value_unchanged(Pid, Bucket, Key, Bin) -> end, timer:sleep(?ENSURE_READ_INTERVAL). + assert_bucket_not_found(Pid, Bucket, Key) -> case riakc_pb_socket:get(Pid, Bucket, Key) of {error, notfound} -> diff --git a/tests/replication/rt_cascading.erl b/tests/replication/rt_cascading.erl index 804aba5f5..8c6702ed8 100644 --- a/tests/replication/rt_cascading.erl +++ b/tests/replication/rt_cascading.erl @@ -38,7 +38,7 @@ confirm() -> %% test requires allow_mult=false b/c of rt:systest_read - rt_config:set_conf(all, [{"buckets.default.allow_mult", "false"}]), + rt:set_conf(all, [{"buckets.default.allow_mult", "false"}]), case eunit:test(?MODULE, [verbose]) of ok -> @@ -64,7 +64,7 @@ simple_test_() -> % +-----------+ +--------+ +-----+ {timeout, timeout(90), {setup, fun() -> Conf = conf(), - [BeginNode, MiddleNode, EndNode] = Nodes = rt_cluster:deploy_nodes(3, Conf), + [BeginNode, MiddleNode, EndNode] = Nodes = rt:deploy_nodes(3, Conf), repl_util:make_cluster([BeginNode]), repl_util:make_cluster([MiddleNode]), repl_util:make_cluster([EndNode]), @@ -78,7 +78,7 @@ simple_test_() -> fun(State) -> Nodes = [State#simple_state.beginning, State#simple_state.middle, State#simple_state.ending], - rt_cluster:clean_cluster(Nodes) + rt:clean_cluster(Nodes) end, fun(State) -> [ @@ -97,7 +97,7 @@ simple_test_() -> end}, {"cascade a put from beginning down to ending", timeout, timeout(25), fun() -> - BeginningClient = rt_pb:pbc(State#simple_state.beginning), + BeginningClient = rt:pbc(State#simple_state.beginning), Bin = <<"cascading realtime">>, Obj = riakc_obj:new(<<"objects">>, Bin, Bin), riakc_pb_socket:put(BeginningClient, Obj, [{w,1}]), @@ -110,7 +110,7 @@ simple_test_() -> rpc:call(State#simple_state.middle, riak_repl_console, realtime_cascades, [["never"]]), Bin = <<"disabled cascading">>, Obj = riakc_obj:new(?bucket, Bin, Bin), - Client = rt_pb:pbc(State#simple_state.beginning), + Client = rt:pbc(State#simple_state.beginning), riakc_pb_socket:put(Client, Obj, [{w,1}]), riakc_pb_socket:stop(Client), ?assertEqual(Bin, maybe_eventually_exists(State#simple_state.middle, ?bucket, Bin)), @@ -122,7 +122,7 @@ simple_test_() -> rpc:call(State#simple_state.middle, riak_repl_console, realtime_cascades, [["always"]]), Bin = <<"cascading re-enabled">>, Obj = riakc_obj:new(?bucket, Bin, Bin), - Client = rt_pb:pbc(State#simple_state.beginning), + Client = rt:pbc(State#simple_state.beginning), riakc_pb_socket:put(Client, Obj, [{w,1}]), riakc_pb_socket:stop(Client), ?assertEqual(Bin, maybe_eventually_exists(State#simple_state.middle, ?bucket, Bin)), @@ -161,7 +161,7 @@ big_circle_test_() -> % +---+ {timeout, timeout(130), {setup, fun() -> Conf = conf(), - Nodes = rt_cluster:deploy_nodes(6, Conf), + Nodes = rt:deploy_nodes(6, Conf), [repl_util:make_cluster([N]) || N <- Nodes], [repl_util:wait_until_is_leader(N) || N <- Nodes], Names = ["1", "2", "3", "4", "5", "6"], @@ -181,13 +181,13 @@ big_circle_test_() -> Nodes end, fun(Nodes) -> - rt_cluster:clean_cluster(Nodes) + rt:clean_cluster(Nodes) end, fun(Nodes) -> [ {"circle it", timeout, timeout(65), fun() -> [One | _] = Nodes, - C = rt_pb:pbc(One), + C = rt:pbc(One), Bin = <<"goober">>, Bucket = <<"objects">>, Obj = riakc_obj:new(Bucket, Bin, Bin), @@ -208,7 +208,7 @@ big_circle_test_() -> connect_rt(Node, Port, ConnectToName) end, lists:map(Connect, lists:zip(Nodes, ConnectTo)), - C = rt_pb:pbc(hd(Nodes)), + C = rt:pbc(hd(Nodes)), Bin = <<"2 way repl">>, Bucket = <<"objects">>, Obj = riakc_obj:new(Bucket, Bin, Bin), @@ -262,7 +262,7 @@ circle_test_() -> % +-------+ +-----+ {timeout, timeout(30), {setup, fun() -> Conf = conf(), - [One, Two, Three] = Nodes = rt_cluster:deploy_nodes(3, Conf), + [One, Two, Three] = Nodes = rt:deploy_nodes(3, Conf), [repl_util:make_cluster([N]) || N <- Nodes], [repl_util:wait_until_is_leader(N) || N <- Nodes], Names = ["one", "two", "three"], @@ -280,12 +280,12 @@ circle_test_() -> Nodes end, fun(Nodes) -> - rt_cluster:clean_cluster(Nodes) + rt:clean_cluster(Nodes) end, fun(Nodes) -> [ {"cascade all the way to the other end, but no further", timeout, timeout(12), fun() -> - Client = rt_pb:pbc(hd(Nodes)), + Client = rt:pbc(hd(Nodes)), Bin = <<"cascading">>, Obj = riakc_obj:new(<<"objects">>, Bin, Bin), riakc_pb_socket:put(Client, Obj, [{w,1}]), @@ -301,7 +301,7 @@ circle_test_() -> {"cascade starting at a different point", timeout, timeout(12), fun() -> [One, Two | _] = Nodes, - Client = rt_pb:pbc(Two), + Client = rt:pbc(Two), Bin = <<"start_at_two">>, Obj = riakc_obj:new(<<"objects">>, Bin, Bin), riakc_pb_socket:put(Client, Obj, [{w,1}]), @@ -333,7 +333,7 @@ pyramid_test_() -> {timeout, timeout(70), {setup, fun() -> Conf = conf(), - [Top, Left, Left2, Right, Right2] = Nodes = rt_cluster:deploy_nodes(5, Conf), + [Top, Left, Left2, Right, Right2] = Nodes = rt:deploy_nodes(5, Conf), [repl_util:make_cluster([N]) || N <- Nodes], [repl_util:wait_until_is_leader(N) || N <- Nodes], Names = ["top", "left", "left2", "right", "right2"], @@ -349,13 +349,13 @@ pyramid_test_() -> Nodes end, fun(Nodes) -> - rt_cluster:clean_cluster(Nodes) + rt:clean_cluster(Nodes) end, fun(Nodes) -> [ {"Cascade to both kids", timeout, timeout(65), fun() -> [Top | _] = Nodes, - Client = rt_pb:pbc(Top), + Client = rt:pbc(Top), Bucket = <<"objects">>, Bin = <<"pyramid_top">>, Obj = riakc_obj:new(Bucket, Bin, Bin), @@ -387,7 +387,7 @@ diamond_test_() -> % +--------+ {timeout, timeout(180), {setup, fun() -> Conf = conf(), - [Top, MidLeft, MidRight, Bottom] = Nodes = rt_cluster:deploy_nodes(4, Conf), + [Top, MidLeft, MidRight, Bottom] = Nodes = rt:deploy_nodes(4, Conf), [repl_util:make_cluster([N]) || N <- Nodes], Names = ["top", "midleft", "midright", "bottom"], [repl_util:name_cluster(Node, Name) || {Node, Name} <- lists:zip(Nodes, Names)], @@ -403,13 +403,13 @@ diamond_test_() -> Nodes end, fun(Nodes) -> - rt_cluster:clean_cluster(Nodes) + rt:clean_cluster(Nodes) end, fun(Nodes) -> [ {"unfortunate double write", timeout, timeout(135), fun() -> [Top, MidLeft, MidRight, Bottom] = Nodes, - Client = rt_pb:pbc(Top), + Client = rt:pbc(Top), Bin = <<"start_at_top">>, Obj = riakc_obj:new(<<"objects">>, Bin, Bin), riakc_pb_socket:put(Client, Obj, [{w,1}]), @@ -444,7 +444,7 @@ diamond_test_() -> [Sink] = proplists:get_value(sinks, Status, [[]]), ExpectSeq = proplists:get_value(expect_seq, Sink), - Client = rt_pb:pbc(MidRight), + Client = rt:pbc(MidRight), Bin = <<"start at midright">>, Bucket = <<"objects">>, Obj = riakc_obj:new(Bucket, Bin, Bin), @@ -479,7 +479,7 @@ circle_and_spurs_test_() -> % +-----------+ +------+ +------+ +-----------+ {timeout, timeout(170), {setup, fun() -> Conf = conf(), - [North, East, West, NorthSpur, EastSpur, WestSpur] = Nodes = rt_cluster:deploy_nodes(6, Conf), + [North, East, West, NorthSpur, EastSpur, WestSpur] = Nodes = rt:deploy_nodes(6, Conf), [repl_util:make_cluster([N]) || N <- Nodes], Names = ["north", "east", "west", "north_spur", "east_spur", "west_spur"], [repl_util:name_cluster(Node, Name) || {Node, Name} <- lists:zip(Nodes, Names)], @@ -493,13 +493,13 @@ circle_and_spurs_test_() -> Nodes end, fun(Nodes) -> - rt_cluster:clean_cluster(Nodes) + rt:clean_cluster(Nodes) end, fun(Nodes) -> [ {"start at north", timeout, timeout(55), fun() -> [North | _Rest] = Nodes, - Client = rt_pb:pbc(North), + Client = rt:pbc(North), Bin = <<"start at north">>, Bucket = <<"objects">>, Obj = riakc_obj:new(Bucket, Bin, Bin), @@ -512,7 +512,7 @@ circle_and_spurs_test_() -> {"Start at west", timeout, timeout(55), fun() -> [_North, _East, West | _Rest] = Nodes, - Client = rt_pb:pbc(West), + Client = rt:pbc(West), Bin = <<"start at west">>, Bucket = <<"objects">>, Obj = riakc_obj:new(Bucket, Bin, Bin), @@ -525,7 +525,7 @@ circle_and_spurs_test_() -> {"spurs don't replicate back", timeout, timeout(55), fun() -> [_North, _East, _West, NorthSpur | _Rest] = Nodes, - Client = rt_pb:pbc(NorthSpur), + Client = rt:pbc(NorthSpur), Bin = <<"start at north_spur">>, Bucket = <<"objects">>, Obj = riakc_obj:new(Bucket, Bin, Bin), @@ -580,7 +580,7 @@ mixed_version_clusters_test_dep() -> {timeout, 60000, {setup, fun() -> Conf = conf(), DeployConfs = [{previous, Conf} || _ <- lists:seq(1,6)], - Nodes = rt_cluster:deploy_nodes(DeployConfs), + Nodes = rt:deploy_nodes(DeployConfs), [N1, N2, N3, N4, N5, N6] = Nodes, case rpc:call(N1, application, get_key, [riak_core, vsn]) of % this is meant to test upgrading from early BNW aka @@ -611,14 +611,14 @@ mixed_version_clusters_test_dep() -> {too_old, Ns} -> Ns; _ -> MaybeNodes end, - rt_cluster:clean_cluster(Nodes) + rt:clean_cluster(Nodes) end, fun({too_old, _Nodes}) -> []; ([N1, N2, N3, N4, N5, N6] = Nodes) -> [ {"no cascading at first", timeout, timeout(35), [ {timeout, timeout(15), fun() -> - Client = rt_pb:pbc(N1), + Client = rt:pbc(N1), Bin = <<"no cascade yet">>, Obj = riakc_obj:new(?bucket, Bin, Bin), riakc_pb_socket:put(Client, Obj, [{w, 2}]), @@ -628,7 +628,7 @@ mixed_version_clusters_test_dep() -> end}, {timeout, timeout(15), fun() -> - Client = rt_pb:pbc(N2), + Client = rt:pbc(N2), Bin = <<"no cascade yet 2">>, Obj = riakc_obj:new(?bucket, Bin, Bin), riakc_pb_socket:put(Client, Obj, [{w, 2}]), @@ -672,7 +672,7 @@ mixed_version_clusters_test_dep() -> fun(_) -> [ {"node1 put", timeout, timeout(205), fun() -> - Client = rt_pb:pbc(N1), + Client = rt:pbc(N1), Bin = <<"rt after upgrade">>, Obj = riakc_obj:new(?bucket, Bin, Bin), riakc_pb_socket:put(Client, Obj, [{w, 2}]), @@ -682,7 +682,7 @@ mixed_version_clusters_test_dep() -> end}, {"node2 put", timeout, timeout(25), fun() -> - Client = rt_pb:pbc(N2), + Client = rt:pbc(N2), Bin = <<"rt after upgrade 2">>, Obj = riakc_obj:new(?bucket, Bin, Bin), riakc_pb_socket:put(Client, Obj, [{w, 2}]), @@ -735,7 +735,7 @@ Reses)]), ExistsLookup = NewHead ++ NewTail, Test = fun() -> ?debugFmt("Running test ~p", [Name]), - Client = rt_pb:pbc(Node), + Client = rt:pbc(Node), Key = <<(ToB(Node))/binary, "-write-", (ToB(N))/binary>>, Obj = riakc_obj:new(?bucket, Key, Key), riakc_pb_socket:put(Client, Obj, [{w, 2}]), @@ -793,7 +793,7 @@ new_to_old_test_dep() -> {timeout, timeout(105), {setup, fun() -> Conf = conf(), DeployConfs = [{current, Conf}, {previous, Conf}, {current, Conf}], - [New1, Old2, New3] = Nodes = rt_cluster:deploy_nodes(DeployConfs), + [New1, Old2, New3] = Nodes = rt:deploy_nodes(DeployConfs), case rpc:call(Old2, application, get_key, [riak_core, vsn]) of % this is meant to test upgrading from early BNW aka % Brave New World aka Advanced Repl aka version 3 repl to @@ -817,13 +817,13 @@ new_to_old_test_dep() -> {too_old, Ns} -> Ns; _ -> MaybeNodes end, - rt_cluster:clean_cluster(Nodes) + rt:clean_cluster(Nodes) end, fun({too_old, _}) -> []; ([New1, Old2, New3]) -> [ {"From new1 to old2", timeout, timeout(25), fun() -> - Client = rt_pb:pbc(New1), + Client = rt:pbc(New1), Bin = <<"new1 to old2">>, Obj = riakc_obj:new(?bucket, Bin, Bin), riakc_pb_socket:put(Client, Obj, [{w, 1}]), @@ -833,7 +833,7 @@ new_to_old_test_dep() -> end}, {"old2 does not cascade at all", timeout, timeout(25), fun() -> - Client = rt_pb:pbc(New1), + Client = rt:pbc(New1), Bin = <<"old2 no cascade">>, Obj = riakc_obj:new(?bucket, Bin, Bin), riakc_pb_socket:put(Client, Obj, [{w, 1}]), @@ -843,7 +843,7 @@ new_to_old_test_dep() -> end}, {"from new3 to old2", timeout, timeout(25), fun() -> - Client = rt_pb:pbc(New3), + Client = rt:pbc(New3), Bin = <<"new3 to old2">>, Obj = riakc_obj:new(?bucket, Bin, Bin), riakc_pb_socket:put(Client, Obj, [{w, 1}]), @@ -857,7 +857,7 @@ new_to_old_test_dep() -> % from an older source cluster/node. It is prevented for now by % having no easy/good way to get the name of the source cluster, % thus preventing complete information on the routed clusters. - Client = rt_pb:pbc(Old2), + Client = rt:pbc(Old2), Bin = <<"old2 to new3">>, Obj = riakc_obj:new(?bucket, Bin, Bin), riakc_pb_socket:put(Client, Obj, [{w,1}]), @@ -873,7 +873,7 @@ new_to_old_test_dep() -> ensure_ack_test_() -> {timeout, timeout(130), {setup, fun() -> Conf = conf(), - [LeaderA, LeaderB] = Nodes = rt_cluster:deploy_nodes(2, Conf), + [LeaderA, LeaderB] = Nodes = rt:deploy_nodes(2, Conf), [repl_util:make_cluster([N]) || N <- Nodes], [repl_util:wait_until_is_leader(N) || N <- Nodes], Names = ["A", "B"], @@ -888,7 +888,7 @@ ensure_ack_test_() -> [LeaderA, LeaderB] end, fun(Nodes) -> - rt_cluster:clean_cluster(Nodes) + rt:clean_cluster(Nodes) end, fun([LeaderA, LeaderB] = _Nodes) -> [ @@ -928,7 +928,7 @@ ensure_unacked_and_queue() -> ensure_unacked_and_queue_test_() -> {timeout, timeout(2300), {setup, fun() -> - Nodes = rt_cluster:deploy_nodes(6, conf()), + Nodes = rt:deploy_nodes(6, conf()), {N123, N456} = lists:split(3, Nodes), repl_util:make_cluster(N123), repl_util:make_cluster(N456), @@ -943,8 +943,8 @@ ensure_unacked_and_queue_test_() -> {N123, N456} end, maybe_skip_teardown(fun({N123, N456}) -> - rt_cluster:clean_cluster(N123), - rt_cluster:clean_cluster(N456) + rt:clean_cluster(N123), + rt:clean_cluster(N456) end), fun({N123, N456}) -> [ @@ -1132,7 +1132,7 @@ exists(Nodes, Bucket, Key) -> exists(Got, [], _Bucket, _Key) -> Got; exists({error, notfound}, [Node | Tail], Bucket, Key) -> - Pid = rt_pb:pbc(Node), + Pid = rt:pbc(Node), Got = riakc_pb_socket:get(Pid, Bucket, Key), riakc_pb_socket:stop(Pid), exists(Got, Tail, Bucket, Key); diff --git a/tests/yz_core_properties_create_unload.erl b/tests/yz_core_properties_create_unload.erl index 31bc57f3e..a53fde692 100644 --- a/tests/yz_core_properties_create_unload.erl +++ b/tests/yz_core_properties_create_unload.erl @@ -21,9 +21,20 @@ -compile(export_all). -include_lib("eunit/include/eunit.hrl"). --define(CFG, [{yokozuna, [{enabled, true}]}]). +-define(CFG, [{riak_kv, + [ + %% allow AAE to build trees and exchange rapidly + {anti_entropy_build_limit, {100, 1000}}, + {anti_entropy_concurrency, 4} + ]}, + {yokozuna, + [ + {enabled, true}, + {anti_entropy_tick, 1000} + ]}]). -define(INDEX, <<"test_idx_core">>). --define(BUCKET, <<"test_bkt_core">>). +-define(TYPE, <<"data">>). +-define(BUCKET, {?TYPE, <<"test_bkt_core">>}). -define(SEQMAX, 100). confirm() -> @@ -48,7 +59,9 @@ confirm() -> %% Create a search index and associate with a bucket lager:info("Create and set Index ~p for Bucket ~p~n", [?INDEX, ?BUCKET]), ok = riakc_pb_socket:create_search_index(Pid, ?INDEX), - ok = riakc_pb_socket:set_search_index(Pid, ?BUCKET, ?INDEX), + ok = rt:create_and_activate_bucket_type(Node, + ?TYPE, + [{search_index, ?INDEX}]), timer:sleep(1000), %% Write keys and wait for soft commit @@ -58,13 +71,10 @@ confirm() -> verify_count(Pid, KeyCount), - %% Remove core.properties from the selected subset - remove_core_props(RandNodes), + lager:info("Remove core.properties file in each index data dir"), + remove_core_props(RandNodes, ?INDEX), - wait_until(RandNodes, - fun(N) -> - rpc:call(N, yz_index, exists, [?INDEX]) - end), + check_exists(Cluster, ?INDEX), lager:info("Write one more piece of data"), ok = rt:pbc_write(Pid, ?BUCKET, <<"foo">>, <<"foo">>, "text/plain"), @@ -72,6 +82,38 @@ confirm() -> verify_count(Pid, KeyCount + 1), + lager:info("Remove index directories on each node and let them recreate/reindex"), + remove_index_dirs(RandNodes, ?INDEX), + + check_exists(Cluster, ?INDEX), + + yz_rt:expire_trees(Cluster), + yz_rt:wait_for_aae(Cluster), + + lager:info("Write second piece of data"), + ok = rt:pbc_write(Pid, ?BUCKET, <<"food">>, <<"foody">>, "text/plain"), + timer:sleep(1100), + + verify_count(Pid, KeyCount + 2), + + lager:info("Remove segment info files in each index data dir"), + remove_segment_infos(RandNodes, ?INDEX), + + lager:info("To fix, we remove index directories on each node and let them recreate/reindex"), + + remove_index_dirs(RandNodes, ?INDEX), + + check_exists(Cluster, ?INDEX), + + yz_rt:expire_trees(Cluster), + yz_rt:wait_for_aae(Cluster), + + lager:info("Write third piece of data"), + ok = rt:pbc_write(Pid, ?BUCKET, <<"baz">>, <<"bar">>, "text/plain"), + timer:sleep(1100), + + verify_count(Pid, KeyCount + 3), + riakc_pb_socket:stop(Pid), pass. @@ -88,9 +130,9 @@ verify_count(Pid, ExpectedKeyCount) -> end. %% @doc Remove core properties file on nodes. -remove_core_props(Nodes) -> - IndexDirs = [rpc:call(Node, yz_index, index_dir, [?INDEX]) || - Node <- Nodes], +remove_core_props(Nodes, IndexName) -> + IndexDirs = [rpc:call(Node, yz_index, index_dir, [IndexName]) || + Node <- Nodes], PropsFiles = [filename:join([IndexDir, "core.properties"]) || IndexDir <- IndexDirs], lager:info("Remove core.properties files: ~p, on nodes: ~p~n", @@ -98,11 +140,31 @@ remove_core_props(Nodes) -> [file:delete(PropsFile) || PropsFile <- PropsFiles], ok. -%% @doc Wrapper around `rt:wait_until' to verify `F' against multiple -%% nodes. The function `F' is passed one of the `Nodes' as -%% argument and must return a `boolean()' delcaring whether the -%% success condition has been met or not. --spec wait_until([node()], fun((node()) -> boolean())) -> ok. -wait_until(Nodes, F) -> - [?assertEqual(ok, rt:wait_until(Node, F)) || Node <- Nodes], - ok. +%% @doc Check if index/core exists in metadata, disk via yz_index:exists. +check_exists(Nodes, IndexName) -> + rt:wait_until(Nodes, + fun(N) -> + rpc:call(N, yz_index, exists, [IndexName]) + end). + +%% @doc Remove index directories, removing the index. +remove_index_dirs(Nodes, IndexName) -> + IndexDirs = [rpc:call(Node, yz_index, index_dir, [IndexName]) || + Node <- Nodes], + lager:info("Remove index dirs: ~p, on nodes: ~p~n", + [IndexDirs, Nodes]), + [rt:stop(ANode) || ANode <- Nodes], + [rt:del_dir(binary_to_list(IndexDir)) || IndexDir <- IndexDirs], + [rt:start(ANode) || ANode <- Nodes]. + +%% @doc Remove lucence segment info files to check if reindexing will occur +%% on re-creation/re-indexing. +remove_segment_infos(Nodes, IndexName) -> + IndexDirs = [rpc:call(Node, yz_index, index_dir, [IndexName]) || + Node <- Nodes], + SiPaths = [binary_to_list(filename:join([IndexDir, "data/index/*.si"])) || + IndexDir <- IndexDirs], + SiFiles = lists:append([filelib:wildcard(Path) || Path <- SiPaths]), + lager:info("Remove segment info files: ~p, on in dirs: ~p~n", + [SiFiles, IndexDirs]), + [file:delete(SiFile) || SiFile <- SiFiles]. diff --git a/tests/yz_crdt.erl b/tests/yz_crdt.erl index b79fe49d2..815f9bd98 100644 --- a/tests/yz_crdt.erl +++ b/tests/yz_crdt.erl @@ -9,21 +9,24 @@ -define(TYPE, <<"maps">>). -define(KEY, "Chris Meiklejohn"). -define(BUCKET, {?TYPE, <<"testbucket">>}). +-define(GET(K,L), proplists:get_value(K, L)). --define(CONF, [ - {riak_core, - [{ring_creation_size, 8}] - }, - {yokozuna, - [{enabled, true}] - }]). +-define(CONF, + [ + {riak_core, + [{ring_creation_size, 8}] + }, + {yokozuna, + [{enabled, true}] + }]). confirm() -> rt:set_advanced_conf(all, ?CONF), %% Configure cluster. - [Nodes] = rt:build_clusters([1]), - [Node|_] = Nodes, + Nodes = rt:build_cluster(5, ?CONF), + + Node = rt:select_random(Nodes), %% Create PB connection. Pid = rt:pbc(Node), @@ -39,26 +42,69 @@ confirm() -> {search_index, ?INDEX}]), %% Write some sample data. - Map = riakc_map:update( + + Map1 = riakc_map:update( {<<"name">>, register}, fun(R) -> - riakc_register:set( - list_to_binary(?KEY), R) + riakc_register:set(list_to_binary(?KEY), R) end, riakc_map:new()), + Map2 = riakc_map:update( + {<<"interests">>, set}, + fun(S) -> + riakc_set:add_element(<<"thing">>, S) end, + Map1), ok = riakc_pb_socket:update_type( - Pid, - ?BUCKET, - ?KEY, - riakc_map:to_op(Map)), + Pid, + ?BUCKET, + ?KEY, + riakc_map:to_op(Map2)), %% Wait for yokozuna index to trigger. timer:sleep(1000), - %% Perform a simple query. - {ok, {search_results, Results, _, _}} = riakc_pb_socket:search( + %% Perform simple queries, check for register, set fields. + {ok, {search_results, Results1a, _, _}} = riakc_pb_socket:search( Pid, ?INDEX, <<"name_register:Chris*">>), - ?assertEqual(length(Results), 1), - lager:info("~p~n", [Results]), + lager:info("Search name_register:Chris*: ~p~n", [Results1a]), + ?assertEqual(length(Results1a), 1), + ?assertEqual(?GET(<<"name_register">>, ?GET(?INDEX, Results1a)), + list_to_binary(?KEY)), + ?assertEqual(?GET(<<"interests_set">>, ?GET(?INDEX, Results1a)), + <<"thing">>), + + {ok, {search_results, Results2a, _, _}} = riakc_pb_socket:search( + Pid, ?INDEX, <<"interests_set:thing*">>), + lager:info("Search interests_set:thing*: ~p~n", [Results2a]), + ?assertEqual(length(Results2a), 1), + ?assertEqual(?GET(<<"name_register">>, ?GET(?INDEX, Results2a)), + list_to_binary(?KEY)), + ?assertEqual(?GET(<<"interests_set">>, ?GET(?INDEX, Results2a)), + <<"thing">>), + + {ok, {search_results, Results3a, _, _}} = riakc_pb_socket:search( + Pid, ?INDEX, <<"_yz_rb:testbucket">>), + lager:info("Search testbucket: ~p~n", [Results3a]), + ?assertEqual(length(Results3a), 1), + ?assertEqual(?GET(<<"name_register">>, ?GET(?INDEX, Results3a)), + list_to_binary(?KEY)), + ?assertEqual(?GET(<<"interests_set">>, ?GET(?INDEX, Results3a)), + <<"thing">>), + + %% Redo queries and check if results are equal + {ok, {search_results, Results1b, _, _}} = riakc_pb_socket:search( + Pid, ?INDEX, <<"name_register:Chris*">>), + ?assertEqual(number_of_fields(Results1a), + number_of_fields(Results1b)), + + {ok, {search_results, Results2b, _, _}} = riakc_pb_socket:search( + Pid, ?INDEX, <<"interests_set:thing*">>), + ?assertEqual(number_of_fields(Results2a), + number_of_fields(Results2b)), + + {ok, {search_results, Results3b, _, _}} = riakc_pb_socket:search( + Pid, ?INDEX, <<"_yz_rb:testbucket">>), + ?assertEqual(number_of_fields(Results3a), + number_of_fields(Results3b)), %% Stop PB connection. riakc_pb_socket:stop(Pid), @@ -67,3 +113,7 @@ confirm() -> rt:clean_cluster(Nodes), pass. + +%% @private +number_of_fields(Resp) -> + length(?GET(?INDEX, Resp)). diff --git a/tests/yz_handoff.erl b/tests/yz_handoff.erl index 79c9d489b..27a37b2fc 100644 --- a/tests/yz_handoff.erl +++ b/tests/yz_handoff.erl @@ -29,18 +29,26 @@ -define(NUMRUNSTATES, 1). -define(SEQMAX, 1000). -define(TESTCYCLE, 20). +-define(N, 3). -define(CFG, [ {riak_core, [ - {ring_creation_size, 16} + {ring_creation_size, 16}, + {n_val, ?N}, + {handoff_concurrency, 10}, + {vnode_management_timer, 1000} ]}, {riak_kv, [ + %% allow AAE to build trees and exchange rapidly + {anti_entropy_build_limit, {100, 1000}}, + {anti_entropy_concurrency, 8}, {handoff_rejected_max, infinity} ]}, {yokozuna, [ + {anti_entropy_tick, 1000}, {enabled, true} ]} ]). @@ -54,15 +62,13 @@ confirm() -> %% Setup cluster initially - Nodes = rt:build_cluster(5, ?CFG), + [Node1, Node2, _Node3, _Node4, _Node5] = Nodes = rt:build_cluster(5, ?CFG), - %% We're going to always keep Node2 in the cluster. - [Node1, Node2, _Node3, _Node4, _Node5] = Nodes, rt:wait_for_cluster_service(Nodes, yokozuna), ConnInfo = ?GET(Node2, rt:connection_info([Node2])), {Host, Port} = ?GET(http, ConnInfo), - Shards = [begin {ok, P} = node_solr_port(Node), {Node, P} end || Node <- Nodes], + Shards = [{N, node_solr_port(N)} || N <- Nodes], %% Generate keys, YZ only supports UTF-8 compatible keys Keys = [<> || N <- lists:seq(1, ?SEQMAX), @@ -71,16 +77,7 @@ confirm() -> KeyCount = length(Keys), Pid = rt:pbc(Node2), - riakc_pb_socket:set_options(Pid, [queue_if_disconnected]), - - %% Create a search index and associate with a bucket - ok = riakc_pb_socket:create_search_index(Pid, ?INDEX), - ok = riakc_pb_socket:set_search_index(Pid, ?BUCKET, ?INDEX), - timer:sleep(1000), - - %% Write keys and wait for soft commit - lager:info("Writing ~p keys", [KeyCount]), - [ok = rt:pbc_write(Pid, ?BUCKET, Key, Key, "text/plain") || Key <- Keys], + yz_rt:write_data(Pid, ?INDEX, ?BUCKET, Keys), timer:sleep(1100), %% Separate out shards for multiple runs @@ -92,16 +89,20 @@ confirm() -> SearchURL = search_url(Host, Port, ?INDEX), lager:info("Verify Replicas Count = (3 * docs/keys) count"), - verify_count(SolrURL, KeyCount * 3), + verify_count(SolrURL, (KeyCount * ?N)), States = [#trial_state{solr_url_before = SolrURL, solr_url_after = internal_solr_url(Host, SolrPort2, ?INDEX, Shards2Rest), - leave_node = Node1}], + leave_node = Node1}, + #trial_state{solr_url_before = internal_solr_url(Host, SolrPort2, ?INDEX, Shards2Rest), + solr_url_after = SolrURL, + join_node = Node1, + admin_node = Node2}], %% Run Shell Script to count/test # of replicas and leave/join %% nodes from the cluster [[begin - check_data(State, KeyCount, BucketURL, SearchURL), + check_data(Nodes, KeyCount, BucketURL, SearchURL, State), check_counts(Pid, KeyCount, BucketURL) end || State <- States] || _ <- lists:seq(1,?NUMRUNSTATES)], @@ -113,8 +114,9 @@ confirm() -> %%%=================================================================== node_solr_port(Node) -> - riak_core_util:safe_rpc(Node, application, get_env, - [yokozuna, solr_port]). + {ok, P} = riak_core_util:safe_rpc(Node, application, get_env, + [yokozuna, solr_port]), + P. internal_solr_url(Host, Port, Index) -> ?FMT("http://~s:~B/internal_solr/~s", [Host, Port, Index]). @@ -137,7 +139,8 @@ verify_count(Url, ExpectedCount) -> fun() -> {ok, "200", _, DBody} = ibrowse:send_req(Url, [], get, []), FoundCount = get_count(DBody), - lager:info("FoundCount: ~b, ExpectedCount: ~b", [FoundCount, ExpectedCount]), + lager:info("FoundCount: ~b, ExpectedCount: ~b", + [FoundCount, ExpectedCount]), ExpectedCount =:= FoundCount end, ?assertEqual(ok, rt:wait_until(AreUp)), @@ -153,43 +156,53 @@ get_keys_count(BucketURL) -> length(kvc:path([<<"keys">>], Struct)). check_counts(Pid, InitKeyCount, BucketURL) -> - PBCounts = [begin {ok, Resp} = riakc_pb_socket:search(Pid, ?INDEX, <<"*:*">>), + PBCounts = [begin {ok, Resp} = riakc_pb_socket:search( + Pid, ?INDEX, <<"*:*">>), Resp#search_results.num_found end || _ <- lists:seq(1,?TESTCYCLE)], - HTTPCounts = [begin {ok, "200", _, RBody} = ibrowse:send_req(BucketURL, [], get, []), + HTTPCounts = [begin {ok, "200", _, RBody} = ibrowse:send_req( + BucketURL, [], get, []), Struct = mochijson2:decode(RBody), length(kvc:path([<<"keys">>], Struct)) end || _ <- lists:seq(1,?TESTCYCLE)], MinPBCount = lists:min(PBCounts), MinHTTPCount = lists:min(HTTPCounts), - lager:info("Before-Node-Leave PB: ~b, After-Node-Leave PB: ~b", [InitKeyCount, MinPBCount]), + lager:info("Before-Node-Leave PB: ~b, After-Node-Leave PB: ~b", + [InitKeyCount, MinPBCount]), ?assertEqual(InitKeyCount, MinPBCount), - lager:info("Before-Node-Leave PB: ~b, After-Node-Leave HTTP: ~b", [InitKeyCount, MinHTTPCount]), + lager:info("Before-Node-Leave PB: ~b, After-Node-Leave HTTP: ~b", + [InitKeyCount, MinHTTPCount]), ?assertEqual(InitKeyCount, MinHTTPCount). -check_data(S, KeyCount, BucketURL, SearchURL) -> - CheckCount = KeyCount * 3, +check_data(Cluster, KeyCount, BucketURL, SearchURL, S) -> + CheckCount = KeyCount * ?N, KeysBefore = get_keys_count(BucketURL), - leave_or_join(S), + UpdatedCluster = leave_or_join(Cluster, S), + + yz_rt:wait_for_aae(UpdatedCluster), KeysAfter = get_keys_count(BucketURL), lager:info("KeysBefore: ~b, KeysAfter: ~b", [KeysBefore, KeysAfter]), ?assertEqual(KeysBefore, KeysAfter), lager:info("Verify Search Docs Count =:= key count"), + lager:info("Run Search URL: ~s", [SearchURL]), verify_count(SearchURL, KeysAfter), lager:info("Verify Replicas Count = (3 * docs/keys) count"), + lager:info("Run Search URL: ~s", [S#trial_state.solr_url_after]), verify_count(S#trial_state.solr_url_after, CheckCount). -leave_or_join(S=#trial_state{join_node=undefined}) -> +leave_or_join(Cluster, S=#trial_state{join_node=undefined}) -> Node = S#trial_state.leave_node, rt:leave(Node), - ?assertEqual(ok, rt:wait_until_unpingable(Node)); -leave_or_join(S=#trial_state{leave_node=undefined}) -> + ?assertEqual(ok, rt:wait_until_unpingable(Node)), + Cluster -- [Node]; +leave_or_join(Cluster, S=#trial_state{leave_node=undefined}) -> Node = S#trial_state.join_node, NodeAdmin = S#trial_state.admin_node, ok = rt:start_and_wait(Node), ok = rt:join(Node, NodeAdmin), - ?assertEqual(ok, rt:wait_until_nodes_ready([NodeAdmin, Node])), - ?assertEqual(ok, rt:wait_until_no_pending_changes([NodeAdmin, Node])). + ?assertEqual(ok, rt:wait_until_nodes_ready(Cluster)), + ?assertEqual(ok, rt:wait_until_no_pending_changes(Cluster)), + Cluster ++ [Node]. From d3975f679bb03005623bf13d0c9fd1953777a76a Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Tue, 12 May 2015 14:11:17 -0600 Subject: [PATCH 148/157] Put lib directory back in git because intercepts are stored there --- bin/rtdev-install.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/bin/rtdev-install.sh b/bin/rtdev-install.sh index 704c3e9fa..fb5ad5a56 100755 --- a/bin/rtdev-install.sh +++ b/bin/rtdev-install.sh @@ -100,7 +100,6 @@ for i in `ls`; do cp -p -P -R $i $RT_DEST_DIR/$RT_VERSION/; done cat > .gitignore < Date: Wed, 13 May 2015 14:40:01 -0600 Subject: [PATCH 149/157] Fix specification of backends - Default backend in rt_planner is now undefined - All lists of backends are now atoms - Command-line limits backends when using GiddyUp - Command-line now properly generates multiple tests when a list of backends is specified - "multi" backend added for verify_2i_aae-multi for GiddyUp --- src/giddyup.erl | 3 ++- src/riak_test_escript.erl | 47 +++++++++++++++++++++++++++++---------- src/rt_planner.erl | 8 ++++--- src/rt_properties2.erl | 2 +- src/rt_test_plan.erl | 2 +- src/rt_util.erl | 35 +++++++++++++++++++++++++++++ 6 files changed, 79 insertions(+), 18 deletions(-) diff --git a/src/giddyup.erl b/src/giddyup.erl index 809425257..21f8a800e 100644 --- a/src/giddyup.erl +++ b/src/giddyup.erl @@ -267,7 +267,8 @@ fetch_all_test_plans(Platform, Product, VersionNumber, DefaultVersion, Host) -> Module = binary_to_atom(kvc:path(name, Test), utf8), Plan0 = rt_test_plan:new([{id, Id}, {module, Module}, {project, Project}, {platform, Platform}, {version, VersionNumber}]), {ok, Plan1} = case kvc:path('tags.backend', Test) of - [] -> {ok, Plan0}; + %% Bitcask is the default version + [] -> rt_test_plan:set(backend, bitcask, Plan0); Backend -> rt_test_plan:set(backend, binary_to_atom(Backend, utf8), Plan0) end, {ok, Plan2} = case kvc:path('tags.upgrade_version', Test) of diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index b760fa98c..f8b97853f 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -25,6 +25,10 @@ -export([main/1]). -export([add_deps/1]). +-ifdef(TEST). +-include_lib("eunit/include/eunit.hrl"). +-endif. + main(Args) -> %% TODO Should we use clique? -jsb %% Parse command line arguments ... @@ -77,7 +81,7 @@ cli_options() -> {skip, $x, "skip", string, "list of tests to skip in a directory"}, {verbose, $v, "verbose", undefined, "verbose output"}, {outdir, $o, "outdir", string, "output directory"}, - {backend, $b, "backend", atom, "backend to test [memory | bitcask | eleveldb]"}, + {backend, $b, "backend", atom, "backend to test [memory | bitcask | eleveldb | multi]"}, {keep, undefined, "keep", boolean, "do not teardown cluster"}, {continue_on_fail, $n, "continue", boolean, "continues executing tests on failure"}, {report, $r, "report", string, "you're reporting an official test run, provide platform info (e.g. ubuntu-1404-64)\nUse 'config' if you want to pull from ~/.riak_test.config"}, @@ -104,12 +108,25 @@ generate_test_lists(UseGiddyUp, ParsedArgs) -> %% test metadata TestData = compose_test_data(ParsedArgs), - Backends = [proplists:get_value(backend, ParsedArgs, bitcask)], + CmdLineBackends = rt_util:backend_to_atom_list(proplists:get_value(backend, ParsedArgs)), + Backends = determine_backends(CmdLineBackends, UseGiddyUp), {Tests, NonTests} = wrap_test_in_test_plan(UseGiddyUp, Backends, TestData), Offset = rt_config:get(offset, undefined), Workers = rt_config:get(workers, undefined), shuffle_tests(Tests, NonTests, Offset, Workers). +%% @doc Which backends should be tested? +%% Use the command-line specified backend, otherwise default to bitcask +%% If running under GiddyUp, then default to ALL backends +%% First argument is a list of command-line backends and second is whether or not in GiddyUp mode +-spec(determine_backends(atom(), boolean()) -> list()). +determine_backends(undefined, true) -> + [memory, bitcask, eleveldb, multi]; +determine_backends(undefined, _) -> + [bitcask]; +determine_backends(Backends, _) -> + Backends. + %% @doc Set values in the configuration with values specified on the command line maybe_override_setting(Argument, Value, Arguments) -> maybe_override_setting(proplists:is_defined(Argument, Arguments), Argument, @@ -385,16 +402,6 @@ match_group_attributes(Attributes, Groups) -> || Group <- Groups, TestType <- TestTypes ]) end. -backend_list(Backend) when is_atom(Backend) -> - atom_to_list(Backend); -backend_list(Backends) when is_list(Backends) -> - FoldFun = fun(X, []) -> - atom_to_list(X); - (X, Acc) -> - Acc ++ "," ++ atom_to_list(X) - end, - lists:foldl(FoldFun, [], Backends). - load_tests_in_dir(Dir, Groups, SkipTests) -> case filelib:is_dir(Dir) of true -> @@ -470,3 +477,19 @@ stop_giddyup(true) -> stop_giddyup(_) -> ok. +-ifdef(TEST). +%% Make sure that bitcask is the default backend +default_backend_test() -> + ?assertEqual([bitcask], determine_backends(undefined, false)). + +%% Make sure that GiddyUp supports all backends +default_giddyup_backend_test() -> + ?assertEqual([bitcask, eleveldb, memory], lists:sort(determine_backends(undefined, true))). + +%% Command-line backends should always rule +cmdline_backend_test() -> + ?assertEqual([memory], determine_backends([memory], false)), + ?assertEqual([memory], determine_backends([memory], true)), + ?assertEqual([eleveldb, memory], lists:sort(determine_backends([memory, eleveldb], false))), + ?assertEqual([eleveldb, memory], lists:sort(determine_backends([memory, eleveldb], true))). +-endif. diff --git a/src/rt_planner.erl b/src/rt_planner.erl index 4cb62620d..9383483ca 100644 --- a/src/rt_planner.erl +++ b/src/rt_planner.erl @@ -92,7 +92,7 @@ load_from_giddyup(Backends, CommandLineTests) -> %% %% @end %%-------------------------------------------------------------------- --spec(add_test_plan(string(), string(), [string()], rt_properties2:product_version(), rt_properties2:properties()) -> ok). +-spec(add_test_plan(string(), string(), [atom()], rt_properties2:product_version(), rt_properties2:properties()) -> ok). add_test_plan(Module, Platform, Backends, Version, Properties) -> gen_server:call(?MODULE, {add_test_plan, Module, Platform, Backends, Version, Properties}). @@ -183,7 +183,8 @@ init([]) -> {noreply, NewState :: #state{}, timeout() | hibernate} | {stop, Reason :: term(), Reply :: term(), NewState :: #state{}} | {stop, Reason :: term(), NewState :: #state{}}). -%% Run only those GiddyUp tests which are specified on the command line +%% Run only those GiddyUp tests which are specified on the command line and are +%% included in the specified backends. %% If none are specified, run everything handle_call({load_from_giddyup, Backends, CommandLineTests}, _From, State) -> AllGiddyupTests = giddyup:get_test_plans(), @@ -207,7 +208,7 @@ handle_call({load_from_giddyup, Backends, CommandLineTests}, _From, State) -> State1 = lists:foldl(fun sort_and_queue/2, State, Included1), State2 = lists:foldl(fun exclude_test_plan/2, State1, Excluded1), {reply, ok, State2}; -%% Add a single test plan to the queue +%% Add a single test plan for each backend to the queue handle_call({add_test_plan, Module, Platform, Backends, _Version, _Properties}, _From, State) -> State1 = lists:foldl(fun(Backend, AccState) -> TestPlan = rt_test_plan:new([{module, Module}, {platform, Platform}, {backend, Backend}]), @@ -326,6 +327,7 @@ sort_and_queue(TestPlan, State) -> non_runnable_test_plans=QNR2}. %% Check for api compatibility +%% TODO: Move into "harness" or "driver" since it might be on a remote node. is_runnable_test_plan(TestPlan) -> TestModule = rt_test_plan:get_module(TestPlan), {Mod, Fun} = riak_test_runner:function_name(confirm, TestModule), diff --git a/src/rt_properties2.erl b/src/rt_properties2.erl index ee86add48..e943139f4 100644 --- a/src/rt_properties2.erl +++ b/src/rt_properties2.erl @@ -51,7 +51,7 @@ -type topology() :: #rt_cluster_topology_v1{}. -type feature_flag() :: strong_consistency | yokozuna | jmx | snmp | security. -type product_version() :: string() | atom(). --type storage_backend() :: all | riak_kv_bitcask_backend | riak_kv_eleveldb_backend | riak_kv_memory_backend | riak_kv_multi_backend. +-type storage_backend() :: all | bitcask | eleveldb | memory | multi. -type index() :: {binary(), binary(), binary()}. -export_type([properties/0, diff --git a/src/rt_test_plan.erl b/src/rt_test_plan.erl index a9f7abd9c..22c725db5 100644 --- a/src/rt_test_plan.erl +++ b/src/rt_test_plan.erl @@ -42,7 +42,7 @@ project=rt_config:get_default_version_product() :: atom() | binary(), platform :: string(), version=rt_config:get_default_version_number() :: string(), - backend=bitcask :: atom(), + backend=undefined :: atom(), upgrade_path=[] :: [rt_properties2:product_version()], properties :: rt_properties2:properties() }). diff --git a/src/rt_util.erl b/src/rt_util.erl index 961887825..6217c9876 100644 --- a/src/rt_util.erl +++ b/src/rt_util.erl @@ -5,3 +5,38 @@ -export_type([error/0, result/0]). +-export([backend_to_atom_list/1]). + +-ifdef(TEST). +-include_lib("eunit/include/eunit.hrl"). +-endif. + +%% @doc Convert string or atom to list of atoms +-spec(backend_to_atom_list(atom()|string()) -> undefined | list()). +backend_to_atom_list(undefined) -> + undefined; +backend_to_atom_list(Backends) when is_atom(Backends) -> + ListBackends = atom_to_list(Backends), + case lists:member($, , ListBackends) of + true -> + [list_to_atom(X) || X <- string:tokens(ListBackends, ", ")]; + _ -> + [Backends] + end; +backend_to_atom_list(Backends) when is_list(Backends) -> + case lists:member($, , Backends) of + true -> + [list_to_atom(X) || X <- string:tokens(Backends, ", ")]; + _ -> + [list_to_atom(Backends)] + end. + +-ifdef(TEST). +%% Properly covert backends to atoms +backend_to_atom_list_test() -> + ?assertEqual(undefined, backend_to_atom_list(undefined)), + ?assertEqual([memory], backend_to_atom_list(memory)), + ?assertEqual([memory], backend_to_atom_list("memory")), + ?assertEqual([bitcask, eleveldb, memory], lists:sort(backend_to_atom_list("memory, bitcask,eleveldb"))), + ?assertEqual([bitcask, eleveldb, memory], lists:sort(backend_to_atom_list('memory, bitcask,eleveldb'))). +-endif. \ No newline at end of file From f25fbac92a63920f729aaa7d0c61aa57b645a336 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Thu, 14 May 2015 14:51:21 -0600 Subject: [PATCH 150/157] Handle the case where versions are single atoms, not version/config tuples when building a cluster --- src/rt.erl | 8 +++++++- src/rt_reporter.erl | 2 -- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/src/rt.erl b/src/rt.erl index d867c429d..38243bed6 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -662,11 +662,17 @@ claimant_according_to(Node) -> %% @doc Safely construct a new cluster and return a list of the deployed nodes %% @todo Add -spec and update doc to reflect mult-version changes build_cluster(Versions) when is_list(Versions) -> - UpdatedVersions = [{rt_config:version_to_tag(Vsn), Cfg} || {Vsn, Cfg} <- Versions], + UpdatedVersions = [build_version_config(Vsn) || Vsn <- Versions], build_cluster(length(Versions), UpdatedVersions, rt_properties:default_config()); build_cluster(NumNodes) -> build_cluster(NumNodes, rt_properties:default_config()). +%% @doc Use the default configuration when a configuration is not specified +build_version_config(Vsn) when not is_tuple(Vsn) -> + {rt_config:version_to_tag(Vsn), rt_properties:default_config()}; +build_version_config({Vsn, Cfg}) -> + {rt_config:version_to_tag(Vsn), Cfg}. + %% @doc Safely construct a `NumNode' size cluster using %% `InitialConfig'. Return a list of the deployed nodes. build_cluster(NumNodes, InitialConfig) -> diff --git a/src/rt_reporter.erl b/src/rt_reporter.erl index 759377f65..616251c32 100644 --- a/src/rt_reporter.erl +++ b/src/rt_reporter.erl @@ -238,8 +238,6 @@ print_summary(TestResults, _CoverResult, Verbose) -> case Verbose of true -> - %% TODO: Remove once clique table is fixed - [lager:debug("ROW ~p", [Row]) || Row <- Rows], Table = clique_table:autosize_create_table(?HEADER, Rows), [lager:notice(string:tokens(lists:flatten(FormattedRow), "\n")) || FormattedRow <- Table]; false -> From 8f1ff946237e7530e6eb74a14e3c110cd8b6b4a2 Mon Sep 17 00:00:00 2001 From: John Burwell Date: Tue, 19 May 2015 10:31:58 -0400 Subject: [PATCH 151/157] Introduces the rt_riak_cluster and rt_riak_node FSMs to orchestrate Riak - rt_host defines a protocol for performing operations on a host (e.g. exec, mkdir, etc). Adds rt_local_host for working on a local host - Moves various general purpose functions to rt_util - Introduces rt_driver to define extension points for product specific test planning and scheduling activities --- rebar.config | 5 +- src/riak_test_escript.erl | 18 +- src/rt_driver.erl | 39 ++ src/rt_host.erl | 128 +++++ src/rt_local_host.erl | 375 ++++++++++++ src/rt_properties2.erl | 93 ++- src/rt_riak_cluster.erl | 69 ++- src/rt_riak_driver.erl | 103 ++++ src/rt_riak_node.erl | 1135 ++++++++++++++++++++++++++++++++----- src/rt_util.erl | 269 ++++++++- 10 files changed, 1949 insertions(+), 285 deletions(-) create mode 100644 src/rt_driver.erl create mode 100644 src/rt_host.erl create mode 100644 src/rt_local_host.erl create mode 100644 src/rt_riak_driver.erl diff --git a/rebar.config b/rebar.config index ae795dd41..1ebe176d6 100644 --- a/rebar.config +++ b/rebar.config @@ -6,7 +6,7 @@ %%{edoc_opts, [{layout, my_layout}, {file_suffix, ".xml"}, {pretty_printer, erl_pp}]}. {erl_opts, [{src_dirs, [src, intercepts, perf]}, warnings_as_errors, {parse_transform, lager_transform}]}. -{erl_first_files, ["src/rt_intercept_pt.erl"]}. +{erl_first_files, ["src/rt_driver.erl", "src/rt_intercept_pt.erl", "src/rt_host.erl"]}. {eunit_opts, [verbose]}. @@ -19,7 +19,8 @@ {riakhttpc, ".*", {git, "git://github.com/basho/riak-erlang-http-client", {branch, "master"}}}, {kvc, "1.3.0", {git, "https://github.com/etrepum/kvc", {tag, "v1.3.0"}}}, {druuid, ".*", {git, "git://github.com/kellymclaughlin/druuid.git", {tag, "0.2"}}}, - {clique, ".*", {git, "git://github.com/basho/clique", {branch, "develop"}}} + {clique, ".*", {git, "git://github.com/basho/clique", {branch, "develop"}}}, + {exec, ".*", {git, "https://github.com/saleyn/erlexec.git", "6c311527bdafc87f3491e4e60b29e7a1b0f33c6e"}} ]}. {escript_incl_apps, [goldrush, lager, getopt, riakhttpc, riakc, ibrowse, mochiweb, kvc]}. diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index f8b97853f..7b8a3a540 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -23,7 +23,6 @@ %% TODO: Temporary build workaround, remove!! -compile(export_all). -export([main/1]). --export([add_deps/1]). -ifdef(TEST). -include_lib("eunit/include/eunit.hrl"). @@ -216,17 +215,6 @@ finalize(TestResults, Args) -> maybe_teardown(Teardown, TestResults), ok. -add_deps(Path) -> - lager:debug("Adding dep path ~p", [Path]), - case file:list_dir(Path) of - {ok, Deps} -> - [code:add_path(lists:append([Path, "/", Dep, "/ebin"])) || Dep <- Deps], - ok; - {error, Reason} -> - lager:error("Failed to add dep path ~p due to ~p.", [Path, Reason]), - erlang:error(Reason) - end. - test_setup() -> %% Prepare the test harness {NodeIds, NodeMap, VersionMap} = rt_harness:setup(), @@ -259,10 +247,10 @@ erlang_setup(_ParsedArgs) -> code:add_paths(rt_config:get(test_paths, [])), %% Two hard-coded deps... - add_deps(rt:get_deps()), - add_deps("deps"), + rt_util:add_deps(rt:get_deps()), + rt_util:add_deps("deps"), - [add_deps(Dep) || Dep <- rt_config:get(rt_deps, [])], + [rt_util:add_deps(Dep) || Dep <- rt_config:get(rt_deps, [])], [] = os:cmd("epmd -daemon"), net_kernel:start([rt_config:get(rt_nodename, 'riak_test@127.0.0.1')]), erlang:set_cookie(node(), rt_config:get(rt_cookie, riak)), diff --git a/src/rt_driver.erl b/src/rt_driver.erl new file mode 100644 index 000000000..859784b4d --- /dev/null +++ b/src/rt_driver.erl @@ -0,0 +1,39 @@ +-module(rt_driver). + +-define(DRIVER_MODULE, (rt_config:get(driver))). + +-export([behaviour_info/1, + cluster_module/0, + new_configuration/0, + new_configuration/1, + get_configuration_key/2, + set_configuration_key/3]). + +-type configuration() :: term(). +-exporttype([configuration/0]). + +behaviour_info(callbacks) -> + [{new_configuration, 0}, {new_configuration, 1}, {get_configuration_key, 2}, + {set_configuration_key, 3}, {cluster_module, 0}]; +behaviour_info(_) -> + undefined. + +-spec cluster_module() -> module(). +cluster_module() -> + ?DRIVER_MODULE:cluster_module(). + +-spec new_configuration() -> configuration(). +new_configuration() -> + ?DRIVER_MODULE:new_configuration(). + +-spec new_configuration(atom()) -> configuration(). +new_configuration(Profile) -> + ?DRIVER_MODULE:new_configuration(Profile). + +-spec get_configuration_key(atom(), configuration()) -> term(). +get_configuration_key(Key, Configuration) -> + ?DRIVER_MODULE:get_configuration_key(Configuration, Key). + +-spec set_configuration_key(atom(), term(), configuration()) -> {ok, configuration()} | {error, string()}. +set_configuration_key(Configuration, Key, Value) -> + ?DRIVER_MODULE:set_configuration_key(Configuration, Key, Value). diff --git a/src/rt_host.erl b/src/rt_host.erl new file mode 100644 index 000000000..8cd97c93b --- /dev/null +++ b/src/rt_host.erl @@ -0,0 +1,128 @@ +-module(rt_host). +-export([behaviour_info/1, + command_line_from_command/1, + connect/1, + connect/2, + consult/2, + copy_dir/4, + disconnect/1, + exec/2, + ip_addr/1, + hostname/1, + kill/3, + killall/3, + make_temp_directory_cmd/1, + mkdirs/2, + mvdir/3, + rmdir/2, + rmdir_cmd/1, + mvdir_cmd/2, + temp_dir/1, + write_file/3]). + +-type command() :: {filelib:filename(), [string()]}. +-type command_result() :: {ok, string()} | rt_util:error(). +-type host_id() :: pid(). +-type host() :: {module(), host_id()}. +-type hostname() :: atom(). + +-exporttype([command/0, + command_result/0, + host_id/0, + host/0, + hostname/0]). + +behaviour_info(callbacks) -> + [{connect, 1}, {connect, 2}, {consult, 2}, {copy_dir, 4}, {disconnect, 1}, + {exec, 2}, {ip_addr, 1}, {hostname, 1}, {kill, 3}, {killall, 3}, + {mkdirs, 2}, {mvdir, 3}, {rmdir, 2}, {temp_dir, 1}, {write_file, 3}]; +behaviour_info(_) -> + undefined. + +-spec command_line_from_command(command()) -> string(). +command_line_from_command({Command, Args}) -> + string:join([Command] ++ Args, " "). + +-spec connect(hostname()) -> {ok, host()} | rt_util:error(). +connect(Hostname) -> + connect(Hostname, []). + +%% To add remote host support, add an implementation of this function that +%% handles all atoms != localhost and create a connect_remote function to +%% startup a rt_remote_host gen_server with the appropriate configuration +%% (e.g. user name, creds, ports, etc) read from the r_t hosts file ... +-spec connect(hostname(), proplists:proplist()) -> {ok, host()} | rt_util:error(). +connect(localhost, Options) -> + connect_localhost(rt_local_host:connect(localhost, Options)); +connect(_, _) -> + erlang:error("Remote hosts are not supported."). + +-spec connect_localhost({ok, host_id()} | rt_util:error()) -> {ok, host()} | rt_util:error(). +connect_localhost({ok, Pid}) -> + {ok, {rt_local_host, Pid}}; +connect_localhost(Error) -> + Error. + +-spec consult(host(), filelib:filename()) -> {ok, term()} | rt_util:error(). +consult({HostModule, HostPid}, Filename) -> + HostModule:consult(HostPid, Filename). + +-spec copy_dir(host(), filelib:dirname(), filelib:dirname(), boolean()) -> command_result(). +copy_dir({HostModule, HostPid}, FromDir, ToDir, Recursive) -> + HostModule:copy_dir(HostPid, FromDir, ToDir, Recursive). + +-spec disconnect(host()) -> ok. +disconnect({HostModule, HostPid}) -> + HostModule:disconnect(HostPid). + +-spec exec(host(), rt_host:command()) -> command_result(). +exec({HostModule, HostPid}, Command) -> + HostModule:exec(HostPid, Command). + +-spec hostname(host()) -> rt_host:hostname(). +hostname({HostModule, HostPid}) -> + HostModule:hostname(HostPid). + +-spec ip_addr(host()) -> string(). +ip_addr({HostModule, HostPid}) -> + HostModule:ip_addr(HostPid). + +-spec kill(host(), pos_integer(), pos_integer()) -> rt_util:result(). +kill({HostModule, HostPid}, Signal, OSPid) -> + HostModule:kill(HostPid, Signal, OSPid). + +-spec killall(host(), pos_integer(), string()) -> rt_util:result(). +killall({HostModule, HostPid}, Signal, Name) -> + HostModule:killall(HostPid, Signal, Name). + +-spec make_temp_directory_cmd(string()) -> command(). +make_temp_directory_cmd(Template) -> + {"/usr/bin/mktemp", ["-d", "-t", Template]}. + +-spec mkdirs(host(), filelib:dirname()) -> rt_util:result(). +mkdirs({HostModule, HostPid}, Path) -> + HostModule:mkdirs(HostPid, Path). + +-spec mvdir(host(), filelib:dirname(), filelib:dirname()) -> rt_util:result(). +mvdir({HostModule, HostPid}, FromDir, ToDir) -> + HostModule:mvdir(HostPid, FromDir, ToDir). + +-spec rmdir(host(), filelib:dirname()) -> rt_util:result(). +rmdir({HostModule, HostPid}, Dir) -> + HostModule:rmdir(HostPid, Dir). + +-spec rmdir_cmd(filelib:dirname()) -> command(). +rmdir_cmd(Dir) -> + {"/bin/rm", ["-rf", Dir]}. + +-spec mvdir_cmd(filelib:dirname(), filelib:dirname()) -> command(). +mvdir_cmd(FromDir, ToDir) -> + {"/bin/mv", [FromDir, ToDir]}. + +-spec temp_dir(host()) -> filelib:dirname(). +temp_dir({HostModule, HostPid}) -> + HostModule:temp_dir(HostPid). + +-spec write_file(host(), filelib:filename(), term()) -> rt_util:result(). +write_file({HostModule, HostPid}, Filename, Content) -> + HostModule:write_file(HostPid, Filename, Content). diff --git a/src/rt_local_host.erl b/src/rt_local_host.erl new file mode 100644 index 000000000..160c740be --- /dev/null +++ b/src/rt_local_host.erl @@ -0,0 +1,375 @@ +-module(rt_local_host). + +-behaviour(gen_server). +-behaviour(rt_host). + +%% API +-export([connect/1, + connect/2, + copy_dir/4, + consult/2, + disconnect/1, + exec/2, + hostname/1, + ip_addr/1, + killall/3, + kill/3, + mkdirs/2, + mvdir/3, + rmdir/2, + temp_dir/1, + write_file/3]). + +-define(TIMEOUT, infinity). + +%% gen_server callbacks +-export([init/1, handle_call/3, handle_cast/2, handle_info/2, + terminate/2, code_change/3]). + +-record(state, {options=[] :: proplists:proplist(), + temp_dir :: filelib:dirname()}). + +%%%=================================================================== +%%% API +%%%=================================================================== + +-spec connect(rt_host:hostname()) -> {ok, rt_host:host_id()} | rt_util:error(). +connect(Hostname) -> + connect(Hostname, []). + +-spec connect(rt_host:hostname(), proplists:proplist()) -> {ok, rt_host:host_id()} | rt_util:error(). +connect(_Hostname, Options) -> + gen_server:start_link(?MODULE, [Options], []). + +-spec consult(rt_host:host_id(), filelib:filename()) -> {ok, term()} | rt_util:error(). +consult(Pid, Filename) -> + gen_server:call(Pid, {consult, Filename}, ?TIMEOUT). + +-spec copy_dir(rt_host:host_id(), filelib:dirname(), filelib:dirname(), boolean()) -> rt_host:command_result(). +copy_dir(Pid, FromDir, ToDir, Recursive) -> + %% TODO What is the best way to handle timeouts? + gen_server:call(Pid, {copy_dir, FromDir, ToDir, Recursive}, ?TIMEOUT). + +-spec disconnect(rt_host:host_id()) -> ok. +disconnect(Pid) -> + gen_server:call(Pid, stop, ?TIMEOUT). + +-spec exec(rt_host:host_id(), rt_host:command()) -> rt_host:command_result(). +exec(Pid, Command) -> + %% TODO What is the best way to handle timeouts? -> Not the OTP timeout, bot to erlexec + gen_server:call(Pid, {exec, Command}, ?TIMEOUT). + +-spec hostname(rt_host:host_id()) -> rt_host:hostname(). +hostname(Pid) -> + gen_server:call(Pid, hostname, ?TIMEOUT). + +-spec ip_addr(rt_host:host_id()) -> string(). +ip_addr(Pid) -> + gen_server:call(Pid, ip_addr, ?TIMEOUT). + +-spec kill(rt_host:host_id(), pos_integer(), pos_integer()) -> rt_util:result(). +kill(Pid, Signal, OSPid) -> + gen_server:call(Pid, {kill, Signal, OSPid}, ?TIMEOUT). + +-spec killall(rt_host:host_id(), pos_integer(), string()) -> rt_util:result(). +killall(Pid, Signal, Name) -> + gen_server:call(Pid, {killall, Signal, Name}, ?TIMEOUT). + +-spec mvdir(rt_host:host_id(), filelib:dirname(), filelib:dirname()) -> rt_util:result(). +mvdir(Pid, FromDir, ToDir) -> + gen_server:call(Pid, {mvdir, FromDir, ToDir}, ?TIMEOUT). + +-spec mkdirs(rt_host:host_id(), filelib:dirname()) -> {ok, filelib:dirname()} | rt_result:error(). +mkdirs(Pid, Path) -> + gen_server:call(Pid, {mkdirs, Path}, ?TIMEOUT). + +-spec rmdir(rt_host:host_id(), filelib:dirname()) -> rt_result:result(). +rmdir(Pid, Dir) -> + gen_server:call(Pid, {rmdir, Dir}, ?TIMEOUT). + +-spec temp_dir(rt_host:host_id()) -> filelib:dirname(). +temp_dir(Pid) -> + gen_server:call(Pid, temp_dir, ?TIMEOUT). + +-spec write_file(rt_host:host_id(), filelib:filename(), term()) -> rt_util:result(). +write_file(Pid, Filename, Content) -> + gen_server:call(Pid, {write_file, Filename, Content}, ?TIMEOUT). + +%%%=================================================================== +%%% gen_server callbacks +%%%=================================================================== + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Initializes the server +%% +%% @spec init(Args) -> {ok, State} | +%% {ok, State, Timeout} | +%% ignore | +%% {stop, Reason} +%% @end +%%-------------------------------------------------------------------- +init([Options]) -> + maybe_init(make_temp_directory(), Options). + +-spec maybe_init({ok, filelib:dirname()} | {error, term()}, proplists:proplist()) -> {ok, #state{}} | {stop, term()}. +maybe_init({ok, TempDir}, Options) -> + State = #state{options=Options, + temp_dir=TempDir}, + lager:debug("Starting localhost gen_server with state ~p", [State]), + {ok, State}; +maybe_init({error, Reason}, _Options) -> + lager:error("Failed to start localhost gen_server -- temp directory failed to be created due to ~p", [Reason]), + {stop, Reason}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Handling call messages +%% +%% @spec handle_call(Request, From, State) -> +%% {reply, Reply, State} | +%% {reply, Reply, State, Timeout} | +%% {noreply, State} | +%% {noreply, State, Timeout} | +%% {stop, Reason, Reply, State} | +%% {stop, Reason, State} +%% @end +%%-------------------------------------------------------------------- +handle_call({copy_dir, FromDir, ToDir, Recursive}, _From, State) -> + {reply, do_copy_dir(FromDir, ToDir, Recursive), State}; +handle_call({consult, Filename}, _From, State) -> + {reply, file:consult(Filename), State}; +handle_call({exec, Command}, _From, State) -> + {reply, maybe_exec(Command), State}; +handle_call(hostname, _From, State) -> + {reply, localhost, State}; +handle_call(ip_addr, _From, State) -> + {reply, "127.0.0.1", State}; +handle_call({kill, Signal, OSPid}, _From, State) -> + lager:info("Killing process ~p with signal ~p on localhost", [OSPid, Signal]), + {reply, exec:kill(OSPid, Signal), State}; +handle_call({killall, Signal, Name}, _From, State) -> + lager:info("Killing all processes named ~p with signal ~p on localhost", [Name, Signal]), + SignalStr = lists:concat(["-", integer_to_list(Signal)]), + {reply, maybe_exec({"/usr/bin/killall", [SignalStr, Name]}), State}; +handle_call({mkdirs, Path}, _From, State) -> + %% filelib:ensure_dir requires the path to end with a / in order to + %% create a directory ... + SanitizedPath = rt_util:maybe_append_when_not_endswith(Path, "/"), + lager:debug("Creating directory ~p on localhost", [SanitizedPath]), + {reply, filelib:ensure_dir(SanitizedPath), State}; +handle_call({mvdir, FromDir, ToDir}, _From, State) -> + {reply, maybe_exec(rt_host:mvdir_cmd(FromDir, ToDir)), State}; +handle_call({rmdir, Dir}, _From, State) -> + {reply, maybe_exec(rt_host:rmdir_cmd(Dir)), State}; +handle_call(stop, _From, State) -> + {stop, normal, ok, State}; +handle_call(temp_dir, _From, State=#state{temp_dir=TempDir}) -> + {reply, TempDir, State}; +handle_call({write_file, Filename, Content}, _From, State) -> + lager:debug("Writing ~p to file ~p on localhost", [Content, Filename]), + {reply, file:write_file(Filename, Content), State}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Handling cast messages +%% +%% @spec handle_cast(Msg, State) -> {noreply, State} | +%% {noreply, State, Timeout} | +%% {stop, Reason, State} +%% @end +%%-------------------------------------------------------------------- +handle_cast(_Msg, State) -> + {noreply, State}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Handling all non call/cast messages +%% +%% @spec handle_info(Info, State) -> {noreply, State} | +%% {noreply, State, Timeout} | +%% {stop, Reason, State} +%% @end +%%-------------------------------------------------------------------- +handle_info(_Info, State) -> + {noreply, State}. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% This function is called by a gen_server when it is about to +%% terminate. It should be the opposite of Module:init/1 and do any +%% necessary cleaning up. When it returns, the gen_server terminates +%% with Reason. The return value is ignored. +%% +%% @spec terminate(Reason, State) -> void() +%% @end +%%-------------------------------------------------------------------- +terminate(_Reason, #state{temp_dir=TempDir}) -> + lager:debug("Shuttting down localhost gen_server."), + case maybe_exec(rt_host:rmdir_cmd(TempDir)) of + {ok, _} -> + lager:debug("Removed temporary directory ~p on localhost", [TempDir]), + ok; + {error, Reason} -> + lager:warning("Failed to remove temporary directory ~p due to ~p on localhost", + [TempDir, Reason]), + ok + end. + +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Convert process state when code is changed +%% +%% @spec code_change(OldVsn, State, Extra) -> {ok, NewState} +%% @end +%%-------------------------------------------------------------------- +code_change(_OldVsn, State, _Extra) -> + {ok, State}. + +%%%=================================================================== +%%% Internal functions +%%%=================================================================== +%% TODO Implement stdout and stderr redirection to lager ... +-spec maybe_exec(rt_host:command()) -> rt_host:command_result(). +maybe_exec(Command) -> + maybe_exec(Command, [sync, stdout, stderr]). + +-spec maybe_exec({ok | error, proplists:proplists()} | rt_host:command(), string() | proplists:proplist()) -> rt_host:command_result(). +maybe_exec(Result={ok, Reason}, CommandLine) -> + lager:debug("Command ~s succeeded with result ~p", [CommandLine, Result]), + Output = proplists:get_value(stdout, Reason, []), + {ok, join_binaries_to_string(Output)}; +maybe_exec(Result={error, Reason}, CommandLine) -> + lager:error("Command ~s failed with result ~p", [CommandLine, Result]), + Output = proplists:get_value(stderr, Reason, []), + {error, join_binaries_to_string(Output)}; +maybe_exec(Command, Options) -> + CommandLine = rt_host:command_line_from_command(Command), + lager:debug("Executing command ~p with options ~p", [CommandLine, Options]), + maybe_exec(exec:run(CommandLine, Options), CommandLine). + +%% TODO Consider implementing in pure Erlang to get more granular result info ... +-spec do_copy_dir(filelib:dirname(), filelib:dirname(), boolean()) -> rt_host:command_result(). +do_copy_dir(FromDir, ToDir, true) -> + lager:debug("Copying ~p to ~p recursively", [FromDir, ToDir]), + maybe_exec("cp", ["-R", FromDir, ToDir]); +do_copy_dir(FromDir, ToDir, false) -> + lager:debug("Copying ~p to ~p non-recursively", [FromDir, ToDir]), + maybe_exec("cp", [FromDir, ToDir]). + +-spec make_temp_directory() -> rt_host:command_result(). +make_temp_directory() -> + lager:debug("Creating a temporary directory on localhost"), + maybe_exec(rt_host:make_temp_directory_cmd("riak_test")). + +-spec join_binaries_to_string([binary()]) -> string(). +join_binaries_to_string(Bins) -> + Strings = lists:foldr(fun(Element, List) -> + [binary:bin_to_list(Element)|List] + end, [], Bins), + Result = string:join(Strings, "\n"), + + %% Only strip off the trailing newline ... + string:strip(Result, right, $\n). + +-ifdef(TEST). +-include_lib("eunit/include/eunit.hrl"). + +setup() -> + ok = application:ensure_started(exec), + {Result, Pid} = connect(localhost), + ?assertEqual(ok, Result), + ?assertEqual(true, is_pid(Pid)), + ?assertEqual(true, filelib:is_dir(temp_dir(Pid))), + Pid. + +teardown(Pid) -> + TempDir = temp_dir(Pid), + disconnect(Pid), + ?assertEqual(false, filelib:is_dir(TempDir)). + +verify_connect({Result, Pid}) -> + ?_test(begin + ?assertEqual(ok, Result), + ?assertEqual(true, is_pid(Pid)), + ok = disconnect(Pid) + end). + +connect_test_() -> + {foreach, + fun() -> ok = application:ensure_started(exec) end, + [fun() -> verify_connect(connect(localhost)) end, + fun() -> verify_connect(connect(localhost, [])) end]}. + +check_exec_args(Pid) -> + ?_test(begin + verify_exec_results(exec(Pid, {"echo", ["test"]}), "test") + end). + +check_exec_no_args(Pid) -> + ?_test(begin + verify_exec_results(exec(Pid, {"echo", []}), "") + end). + +verify_exec_results({Result, Output}, Expected) -> + ?assertEqual(ok, Result), + ?assertEqual(Expected, Output). + +exec_success_test_() -> + {foreach, + fun setup/0, + fun teardown/1, + [fun check_exec_no_args/1, + fun check_exec_args/1]}. + +exec_failure_test() -> + Pid = setup(), + {Result, _} = exec(Pid, {"asdfasdf", []}), + ?assertEqual(error, Result), + teardown(Pid). + +check_kill(Pid, Signal) -> + ?_test(begin + {ok, _, OSPid} = exec:run("while true; do sleep 1; done", + [{success_exit_code, Signal}]), + Result = kill(Pid, Signal, OSPid), + ?assertEqual(ok, Result) + end). + +kill_test_() -> + {foreach, + fun setup/0, + fun teardown/1, + [fun(Pid) -> check_kill(Pid, 9) end, + fun(Pid) -> check_kill(Pid, 15) end]}. + +mkdirs_test() -> + Pid = setup(), + TestDir = filename:join([temp_dir(Pid), "test123", "foo", "bar"]), + Result = mkdirs(Pid, TestDir), + ?assertEqual(ok, Result), + ?assertEqual(true, filelib:is_dir(TestDir)), + teardown(Pid). + +hostname_test() -> + Pid = setup(), + ?assertEqual(localhost, hostname(Pid)), + teardown(Pid). + +ip_addr_test() -> + Pid = setup(), + ?assertEqual("127.0.0.1", ip_addr(Pid)), + teardown(Pid). + +join_binaries_to_string_test() -> + Bins = [<<"foo">>, <<"bar">>, <<"zoo">>], + Actual = join_binaries_to_string(Bins), + ?assertEqual("foo\nbar\nzoo", Actual). + +-endif. diff --git a/src/rt_properties2.erl b/src/rt_properties2.erl index e943139f4..b7b52a62f 100644 --- a/src/rt_properties2.erl +++ b/src/rt_properties2.erl @@ -22,55 +22,51 @@ %% @doc Implements a set of functions for accessing and manipulating %% an `rt_properties2' record. +-type cluster_name() :: atom(). -record(rt_cluster_topology_v1, { - name :: atom(), - connected_to :: [] | [atom()], - nodes :: [product_version()] + name :: cluster_name(), + %% TODO: Account for full sync/real time connection types, ssl, repl protocol version + connected_to :: [] | [cluster_name()], + nodes :: pos_integer() | [rt_util:version_selector()] }). --define(RT_CLUSTER_TOPOLOGY, #rt_cluster_topology_v1). --record(rt_properties_v2, { + +-record(rt_properties_v2, { description :: string(), - supported_products :: [atom()], - minimum_version :: string(), - maximum_version :: string(), - supported_backends=all :: [storage_backend()], - wait_for_transfers=false :: boolean(), - bucket_types=[] :: rt_bucket_types:bucket_types(), - indexes=[] :: [index()], - ring_size=auto :: [atom() | non_neg_integer()], - features :: feature_flag(), - required_services=[riak_kv] :: [atom()], + supported_products=riak :: [rt_util:products()], + minimum_version=any :: any | rt_util:release(), + maximum_version=any :: any | rt_util:release(), cluster_topology=default_topology(3) :: [topology()], - default_version=rt_config:get_default_version() :: product_version(), - upgrade_path :: [product_version()], - config=default_config() :: term() + groups=[] :: [atom()], + driver_configuration=rt_driver:new_configuration() :: rt_driver:configuration(), + %% TODO Do we need these two properties since the versions are specified in the topology + %% and default will be used when is specified and the upgrade pathes are specified in + %% the configuration ... + default_version=rt_config:get_default_version() :: rt_util:version(), + upgrade_path :: [rt_util:version_selector()] }). +%% What if we moved the Riak specific bits to an rt_riak_driver module +%% and provided some additional callback functions for product +%% specific extension points ... + -type properties() :: #rt_properties_v2{}. -type topology() :: #rt_cluster_topology_v1{}. --type feature_flag() :: strong_consistency | yokozuna | jmx | snmp | security. --type product_version() :: string() | atom(). --type storage_backend() :: all | bitcask | eleveldb | memory | multi. --type index() :: {binary(), binary(), binary()}. - -export_type([properties/0, - feature_flag/0, - product_version/0, - storage_backend/0, - index/0]). - + topology/0]). +-define(RT_CLUSTER_TOPOLOGY, #rt_cluster_topology_v1). -define(RT_PROPERTIES, #rt_properties_v2). -define(RECORD_FIELDS, record_info(fields, rt_properties_v2)). -export([new/0, new/1, get/2, + get_configuration_key/2, set/2, set/3, - default_topology/1, - default_config/0]). + set_configuration_key/3, + default_topology/1]). %% @doc Create a new properties record with all fields initialized to %% the default values. @@ -111,7 +107,6 @@ set(PropertyList, Properties) when is_list(PropertyList) -> set(Property, Value, Properties) -> set_property(Property, Value, Properties, validate_request(Property, Properties)). - -spec get(atom(), properties(), ok | {error, atom()}) -> term() | {error, atom()}. get(Property, Properties, ok) -> @@ -157,6 +152,16 @@ set_properties(PropertyList, Properties, ok) -> set_properties(_, _, {error, _}=Error) -> Error. +-spec get_configuration_key(atom(), properties()) -> term() | {error, string()}. +get_configuration_key(Key, Properties) -> + DriverConfiguration = get(driver_configuration, Properties), + rt_driver:get_configuration_key(DriverConfiguration, Key). + +-spec set_configuration_key(atom(), term(), properties()) -> {ok, term()} | {error, string()}. +set_configuration_key(Key, Value, Properties) -> + DriverConfiguration = get(driver_configuration, Properties), + rt_driver:set_configuration_key(Key, Value, DriverConfiguration). + -spec validate_request(atom(), term()) -> ok | {error, atom()}. validate_request(Property, Properties) -> validate_property(Property, validate_record(Properties)). @@ -181,12 +186,6 @@ validate_property(Property, ok) -> validate_property(_Property, {error, _}=Error) -> Error. --spec default_config() -> [term()]. -default_config() -> - [{riak_core, [{handoff_concurrency, 11}]}, - {riak_search, [{enabled, true}]}, - {riak_pipe, [{worker_limit, 200}]}]. - %% @doc Create a single default cluster topology with default node versions -spec default_topology(pos_integer()) -> [topology()]. default_topology(N) -> @@ -206,27 +205,9 @@ field_index(description) -> ?RT_PROPERTIES.description; field_index(supported_products) -> ?RT_PROPERTIES.supported_products; -field_index(minimum_version) -> - ?RT_PROPERTIES.minimum_version; -field_index(supported_backends) -> - ?RT_PROPERTIES.supported_backends; -field_index(wait_for_transfers) -> - ?RT_PROPERTIES.wait_for_transfers; -field_index(bucket_types) -> - ?RT_PROPERTIES.bucket_types; -field_index(indexes) -> - ?RT_PROPERTIES.indexes; field_index(upgrade_path) -> ?RT_PROPERTIES.upgrade_path; -field_index(ring_size) -> - ?RT_PROPERTIES.ring_size; -field_index(features) -> - ?RT_PROPERTIES.features; -field_index(required_services) -> - ?RT_PROPERTIES.required_services; field_index(cluster_topology) -> ?RT_PROPERTIES.cluster_topology; field_index(default_version) -> - ?RT_PROPERTIES.default_version; -field_index(config) -> - ?RT_PROPERTIES.config. + ?RT_PROPERTIES.default_version. diff --git a/src/rt_riak_cluster.erl b/src/rt_riak_cluster.erl index 1f28c0833..8ac87f240 100644 --- a/src/rt_riak_cluster.erl +++ b/src/rt_riak_cluster.erl @@ -18,10 +18,11 @@ is_mixed/1, join/2, leave/2, + name/1, nodes/1, partition/3, - staged_join/2, start/1, + start_link/2, stop/1, wait_until_all_members/1, wait_until_connected/1, @@ -33,17 +34,23 @@ upgrade/2]). %% gen_fsm callbacks --export([init/1, state_name/2, state_name/3, handle_event/3, start_link/3, +-export([init/1, state_name/2, state_name/3, handle_event/3, handle_sync_event/4, handle_info/3, terminate/3, code_change/4]). -define(SERVER, ?MODULE). +-type node_attribute() :: {name, atom()} | {version, rt_util:version()} | + {hostname, rt_host:hostname()} | {config, term()} | + {type, proplists:proplist()}. +-type node_definition() :: [node_attribute()]. + -record(state, {name :: cluster_name(), - bucket_types :: [atom()], %% should this be a string? + bucket_types :: [term()], %% should this be a string? indexes :: [atom()], %% should this be a string? nodes :: [node()], - version :: version() - }). + version :: rt_util:version(), + config :: term(), + is_mixed=false :: boolean()}). %%%=================================================================== %%% API @@ -52,10 +59,6 @@ -type cluster_id() :: pid(). %% -type partition() :: [node()]. -%% TODO Move to the rt_version module --type product_type() :: riak | riak_ee | riak_cs. --type version() :: {product_type(), string()}. - -spec activate_bucket_type(cluster_id(), atom()) -> rt_util:result(). activate_bucket_type(Cluster, BucketType) -> gen_fsm:sync_send_event(Cluster, {activate_bucket_type, BucketType}). @@ -78,37 +81,30 @@ create_bucket_type(Cluster, BucketType, Properties) -> -spec is_mixed(cluster_id()) -> boolean(). is_mixed(Cluster) -> - gen_fsm:sync_send_event(Cluster, is_mixed). + gen_fsm:sync_send_all_state_event(Cluster, is_mixed). %% @doc Joins a node, `Node', to the cluster, `Cluster' -spec join(cluster_id(), node()) -> rt_util:result(). join(Cluster, Node) -> gen_fsm:sync_send_event(Cluster, {join, Node}). - -spec leave(cluster_id(), node()) -> rt_util:result(). leave(Cluster, Node) -> gen_fsm:sync_send_event(Cluster, {leave, Node}). +-spec name(cluster_id()) -> atom(). +name(Cluster) -> + gen_fsm:sync_send_all_state_event(Cluster, name). + %% @doc Returns the list of nodes in the cluster -spec nodes(cluster_id()) -> [node()]. nodes(Cluster) -> - gen_fsm:sync_send_event(Cluster, nodes). + gen_fsm:sync_send_all_state_event(Cluster, nodes). %% -spec partition(cluster_id(), partition(), partition()) -> [atom(), atom(), partition(), partition()] | rt_util:error(). partition(Cluster, P1, P2) -> gen_fsm:sync_send_event(Cluster, {partition, P1, P2}). -%% @doc Starts the FSM initializing the state with the passed `Name', `Nodes', and `Config' --spec start_link(cluster_name(), [node()], proplists:proplist()) -> {cluster_name(), cluster_id()} | rt_util:error(). -start_link(Name, Nodes, Config) -> - lager:error("provision(~p, ~p, ~p) not implemented.", [Name, Nodes, Config]), - ok. - --spec staged_join(cluster_id(), node()) -> rt_util:result(). -staged_join(Cluster, Node) -> - gen_fsm:sync_send_event(Cluster, {staged_join, Node}). - %% @doc Starts each node in the cluster -spec start(cluster_id()) -> rt_util:result(). start(Cluster) -> @@ -144,12 +140,12 @@ wait_until_ring_converged(Cluster) -> gen_fsm:sync_send_event(Cluster, wait_until_ring_converged). %% @doc Returns the version of the cluster, `Cluster' --spec version(cluster_id()) -> version(). +-spec version(cluster_id()) -> rt_util:version(). version(Cluster) -> - gen_fsm:sync_send_event(Cluster, version). + gen_fsm:sync_send_all_state_event(Cluster, version). %% @doc Performs a rolling upgrade of the cluster, `Cluster', to version, `NewVersion'. --spec upgrade(cluster_id(), version()) -> rt_util:result(). +-spec upgrade(cluster_id(), rt_util:version()) -> rt_util:result(). upgrade(Cluster, NewVersion) -> gen_fsm:sync_send_event(Cluster, {upgrade, NewVersion}). @@ -166,8 +162,10 @@ upgrade(Cluster, NewVersion) -> %% @spec start_link() -> {ok, Pid} | ignore | {error, Error} %% @end %%-------------------------------------------------------------------- -%%start_link() -> -%% gen_fsm:start_link({local, ?SERVER}, ?MODULE, [], []). +%%-spec start_link([cluster_name(), [{node(), rt_util:version()}], term()]) -> {ok, pid()} | ignore | rt_util:error(). +-spec start_link(cluster_name(), [node_definition()]) -> {ok, pid()} | ignore | rt_util:error(). +start_link(Name, NodeDefinitions) -> + gen_fsm:start_link(?MODULE, [Name, NodeDefinitions], []). %%%=================================================================== %%% gen_fsm callbacks @@ -186,8 +184,21 @@ upgrade(Cluster, NewVersion) -> %% {stop, StopReason} %% @end %%-------------------------------------------------------------------- -init([]) -> - {ok, state_name, #state{}}. +init([Name, NodeDefinitions]) -> + %% TODO Calculate the mixed flag + Nodes = lists:foldl(fun(NodeDefinition, Nodes) -> + Hostname = proplists:get_value(hostname, NodeDefinition), + Type = proplists:get_value(type, NodeDefinition), + Config = proplists:get_value(config, NodeDefinition), + Version = proplists:get_value(version, NodeDefinition), + + {ok, NodeName} = rt_riak_node:start_link(Hostname, Type, + Config, Version), + + lists:apped(NodeName, Nodes) + end, [], NodeDefinitions), + + {ok, allocated, #state{name=Name, nodes=Nodes}}. %%-------------------------------------------------------------------- %% @private diff --git a/src/rt_riak_driver.erl b/src/rt_riak_driver.erl new file mode 100644 index 000000000..2da9cbd3f --- /dev/null +++ b/src/rt_riak_driver.erl @@ -0,0 +1,103 @@ +-module(rt_riak_driver). + +-behavior(rt_driver). + +-export([cluster_module/0, + new_configuration/0, + new_configuration/1, + get_configuration_key/2, + set_configuration_key/3]). + +-record(riak_configuration_v1, { + %% TODO Add support for backend configuration options + supported_backends=all :: [storage_backend()], + wait_for_transfers=false :: boolean(), + bucket_types=[] :: rt_bucket_types:bucket_types(), + indexes=[] :: [index()], + ring_size=auto :: [atom() | pos_integer()], + features=[] :: [feature_flag()], + mr_modules=[] :: [module()], + %% TODO Can rt_riak_node properly calculate the list of required services? + required_services=[riak_kv] :: [atom()], + node_config=default_node_config() :: proplists:proplist() + %% TODO What do we need to configure security? +}). +-define(CONFIGURATION_RECORD, #riak_configuration_v1). + +-type riak_configuration() :: ?CONFIGURATION_RECORD{}. + +-type feature() :: aae | strong_consistency | yokozuna | jmx | snmp | security. +-type feature_flag() :: { feature(), boolean() }. +-type storage_backend() :: bitcask | leveldb | memory | multi_backend. +-type index() :: {binary(), binary(), binary()}. + +-exporttype([feature/0, + feature_flag/0, + storage_backend/0, + index/0]). + +-spec cluster_module() -> module(). +cluster_module() -> + rt_riak_cluster. + +-spec default_node_config() -> [term()]. +default_node_config() -> + [{riak_core, [{handoff_concurrency, 11}]}, + {riak_search, [{enabled, true}]}, + {riak_pipe, [{worker_limit, 200}]}]. + +-spec new_configuration() -> riak_configuration(). +new_configuration() -> + ?CONFIGURATION_RECORD{}. + +-spec new_configuration(atom()) -> riak_configuration(). +new_configuration(default) -> + ?CONFIGURATION_RECORD{}. + +-spec get_configuration_key(atom(), riak_configuration()) -> term() | {error, string()}. +get_configuration_key(Key, Configuration) when is_record(Configuration, riak_configuration_v1) -> + maybe_get_configuration_key(field_index(Key), Configuration); +get_configuration_key(Key, Configuration) -> + {error, io_lib:format("~w is not a riak_configuration_v1 record from which to retrieve ~w", [Configuration, Key])}. + +-spec maybe_get_configuration_key(pos_integer() | {error, string()}, riak_configuration()) -> term() | {error, string()}. +maybe_get_configuration_key(Error={error, _}, _Configuration) -> + Error; +maybe_get_configuration_key(FieldIndex, Configuration) -> + element(FieldIndex, Configuration). + +-spec set_configuration_key(atom(), term(), riak_configuration()) -> {ok, riak_configuration()} | {error, string()}. +set_configuration_key(Key, Value, Configuration) when is_record(Configuration, riak_configuration_v1) -> + maybe_set_configuration_key(field_index(Key), Configuration, Value); +set_configuration_key(Configuration, Key, Value) -> + {error, io_lib:format("~w is not a riak_configuration_v1 record from which to set ~w to ~w", [Configuration, Key, Value])}. + +-spec maybe_set_configuration_key({error, string()} | pos_integer, riak_configuration, term()) -> + {ok, riak_configuration()} | {error, string()}. +maybe_set_configuration_key(Error={error, _}, _Configuration, _Value) -> + Error; +maybe_set_configuration_key(FieldIndex, Configuration, Value) -> + {ok, setelement(FieldIndex, Configuration, Value)}. + +-spec field_index(atom()) -> pos_integer | {error, string()}. +field_index(supported_backends) -> + ?CONFIGURATION_RECORD.supported_backends; +field_index(wait_for_transfers) -> + ?CONFIGURATION_RECORD.supported_backends; +field_index(bucket_types) -> + ?CONFIGURATION_RECORD.bucket_types; +field_index(indexes) -> + ?CONFIGURATION_RECORD.indexes; +field_index(ring_sizes) -> + ?CONFIGURATION_RECORD.ring_size; +field_index(features) -> + ?CONFIGURATION_RECORD.features; +field_index(mr_modules) -> + ?CONFIGURATION_RECORD.mr_modules; +field_index(required_services) -> + ?CONFIGURATION_RECORD.required_services; +field_index(node_config) -> + ?CONFIGURATION_RECORD.node_config; +field_index(Unknown) -> + {error, "Unknown Riak configuration field: " ++ Unknown}. + diff --git a/src/rt_riak_node.erl b/src/rt_riak_node.erl index 9ac955956..5c0039fc7 100644 --- a/src/rt_riak_node.erl +++ b/src/rt_riak_node.erl @@ -12,31 +12,45 @@ -include_lib("eunit/include/eunit.hrl"). +%% TODO Document the following topics: +%% - State model and defintions +%% - Error handling: what is passed back to the client to handle +%% vs. what stops the FSM +%% - + + %% API -export([admin/2, admin/3, - assert_singleton/1, attach/2, attach_direct/2, brutal_kill/1, + check_singleton/1, claimant_according_to/1, clean_data_dir/1, clean_data_dir/2, commit/1, console/2, + cookie/1, + copy_logs/2, get_ring/1, - is_allocated/1, + ip/1, is_ready/1, is_stopped/1, + host/1, join/2, + maybe_wait_for_changes/1, members_according_to/1, owners_according_to/1, partitions/1, ping/1, plan/1, + release/1, riak/2, riak_repl/2, search_cmd/2, + set_cookie/2, + staged_join/2, start/1, start/2, start_link/5, @@ -51,42 +65,47 @@ version/1]). %% gen_fsm callbacks --export([init/1, handle_event/3, stopped/3, handle_sync_event/4, - handle_info/3, ready/3, terminate/3, code_change/4]). +-export([init/1, handle_event/3, stopped/3, stopped/2, + handle_sync_event/4, handle_info/3, invalid/3, + ready/2, ready/3, started/3, terminate/3, + code_change/4]). --define(SERVER, ?MODULE). +%% internal exports +-export([do_check_singleton/3, do_wait_until_pingable/2, + do_wait_until_registered/3, do_wait_for_service/3, + do_unpingable/1, get_os_pid/2, start_riak_daemon/3]). --type host() :: string(). --type node_id() :: string(). +-define(SERVER, ?MODULE). +-define(TIMEOUT, infinity). --record(configuration, {one :: proplists:proplist(), - two :: proplists:proplist()}). +-type node_id() :: pos_integer(). +-type node_type() :: {devrel, filelib:dirname()}. --type path() :: string(). --record(directory_overlay, {bin_dir :: path(), - conf_dir :: path(), - data_dir :: path(), - home_dir :: path(), - lib_dir :: path(), - log_dir :: path()}). - +%% TODO Document the purpose of the directory_overlay +-record(directory_overlay, {bin_dir :: filelib:dirname(), + conf_dir :: filelib:dirname(), + data_dir :: filelib:dirname(), + home_dir :: filelib:dirname(), + lib_dir :: filelib:dirname(), + log_dir :: filelib:dirname()}). --type command() :: string(). --record(state, {config :: #configuration{}, - host :: host(), +-record(state, {host :: rt_host:host(), id :: node_id(), directory_overlay :: #directory_overlay{}, name :: node(), - start_command :: command(), - stop_command :: command(), - transport :: module(), - version :: string()}). + os_pid=0 :: pos_integer(), + required_services=[riak_kv] :: [atom()], + start_command :: rt_host:command(), + stop_command :: rt_host:command(), + version :: rt_util:version()}). %%%=================================================================== %%% API %%%=================================================================== %% @doc Call 'bin/riak-admin' command on `Node' with arguments `Args' +%% +%% @since 1.1.0 -spec admin(node(), [term()]) -> {ok, term()} | rt_util:error(). admin(Node, Args) -> admin(Node, Args, []). @@ -94,17 +113,11 @@ admin(Node, Args) -> %% @doc Call 'bin/riak-admin' command on `Node' with arguments `Args'. %% The third parameter is a list of options. Valid options are: %% * `return_exit_code' - Return the exit code along with the command output +%% +%% @since 1.1.0 -spec admin(node(), [term()], [term()]) -> {ok, term()} | rt_util:error(). admin(Node, Args, Options) -> - gen_fsm:sync_send_event(Node, {admin, Node, Args, Options}). - -%% @doc Ensure that the specified node is a singleton node/cluster -- a node -%% that owns 100% of the ring. --spec assert_singleton(node()) -> boolean(). -assert_singleton(Node) -> - SingletonFlag = gen_fsm:sync_send_event(Node, check_singleton), - ?assert(SingletonFlag), - SingletonFlag. + gen_fsm:sync_send_event(Node, {admin, Node, Args, Options}, ?TIMEOUT). %% @doc Runs `riak attach' on a specific node, and tests for the expected behavoir. %% Here's an example: ``` @@ -119,154 +132,241 @@ assert_singleton(Node) -> %% Once a send is encountered, the buffer is discarded, and the next %% expect will process based on the output following the sent data. %% +%% @since 1.1.0 -spec attach(node(), {expect, list()} | {send, list()}) -> {ok, term()} | rt_util:error(). attach(Node, Expected) -> - gen_fsm:sync_send_event(Node, {attach, Expected}). + gen_fsm:sync_send_event(Node, {attach, Expected}, ?TIMEOUT). %% @doc Runs 'riak attach-direct' on a specific node %% @see rt_riak_node:attach/2 +%% +%% @since 1.1.0 -spec attach_direct(node(), {expect, list()} | {send, list()}) -> {ok, term()} | rt_util:error(). attach_direct(Node, Expected) -> - gen_fsm:sync_send_event(Node, {attach_direct, Expected}). + gen_fsm:sync_send_event(Node, {attach_direct, Expected}, ?TIMEOUT). -spec brutal_kill(node()) -> rt_util:result(). brutal_kill(Node) -> - gen_fsm:sync_send_all_state_event(Node, brutal_kill). + gen_fsm:sync_send_all_state_event(Node, brutal_kill, ?TIMEOUT). + +%% @doc Ensure that the specified node is a singleton node/cluster -- a node +%% that owns 100% of the ring. +%% +%% @since 1.1.0 +-spec check_singleton(node()) -> boolean(). +check_singleton(Node) -> + gen_fsm:sync_send_event(Node, check_singleton, ?TIMEOUT). %% @doc Return a list of nodes that own partitions according to the ring %% retrieved from the specified node. +%% +%% @since 1.1.0 claimant_according_to(Node) -> - gen_fsm:sync_send_event(Node, claimant_according_to). + gen_fsm:sync_send_event(Node, claimant_according_to, ?TIMEOUT). -spec clean_data_dir(node()) -> rt_util:result(). clean_data_dir(Node) -> - gen_fsm:sync_send_event(Node, clean_data_dir). + gen_fsm:sync_send_event(Node, clean_data_dir, ?TIMEOUT). -spec clean_data_dir(node(), list()) -> rt_util:result(). clean_data_dir(Node, SubDir) -> - gen_fsm:sync_send_event(Node, {clean_data_dir, SubDir}). + gen_fsm:sync_send_event(Node, {clean_data_dir, SubDir}, ?TIMEOUT). %% @doc Runs `riak console' on a specific node %% @see rt_riak_node:attach/2 +%% +%% @since 1.1.0 -spec console(node(), {expect, list()} | {send, list()}) -> {ok, term()} | rt_util:error(). console(Node, Expected) -> - geb_fsm:sync_send_event(Node, {console, Expected}). + geb_fsm:sync_send_event(Node, {console, Expected}, ?TIMEOUT). -spec commit(node()) -> rt_util:result(). commit(Node) -> - gen_fsm:sync_send_event(Node, commit). + gen_fsm:sync_send_event(Node, commit, ?TIMEOUT). + +-spec cookie(node()) -> atom() | rt_util:error(). +cookie(Node) -> + gen_fsm:sync_send_event(Node, cookie, ?TIMEOUT). + +%% @doc Copy all logs from the passed node, `Node', to the directory, `ToDir' +%% +%% @since 1.1.0 +-spec copy_logs(node(), string()) -> rt_util:result(). +copy_logs(Node, ToDir) -> + gen_fsm:sync_send_all_state_event(Node, {copy_logs, ToDir}, ?TIMEOUT). %% @doc Get the raw ring for `Node'. --spec get_ring(node()) -> term(). +%% +%% @since 1.1.0 +-spec get_ring(node()) -> {ok, term()} | rt_util:error(). get_ring(Node) -> - gen_fsm:sync_send_event(Node, get_ring). + gen_fsm:sync_send_event(Node, get_ring, ?TIMEOUT). --spec is_allocated(node()) -> boolean. -is_allocated(Node) -> - gen_fsm:sync_send_event(Node, is_allocated). +-spec ip(node() | string()) -> string(). +ip(Node) -> + gen_fsm:sync_send_all_state_event(Node, ip, ?TIMEOUT). -spec is_ready(node()) -> boolean(). is_ready(Node) -> - gen_fsm:sync_send_event(Node, is_ready). + gen_fsm:sync_send_all_state_event(Node, is_ready, ?TIMEOUT). -spec is_stopped(node()) -> boolean(). is_stopped(Node) -> - gen_fsm:sync_send_event(Node, is_stopped). + gen_fsm:sync_send_all_state_event(Node, is_stopped, ?TIMEOUT). + +-spec host(node()) -> rt_host:host(). +host(Node) -> + gen_fsm:sync_send_all_state_event(Node, host, ?TIMEOUT). -spec join(node(), node()) -> rt_util:result(). join(Node, ToNode) -> - gen_fsm:sync_send_event(Node, {join, ToNode}). + gen_fsm:sync_send_event(Node, {join, ToNode}, ?TIMEOUT). + +-spec maybe_wait_for_changes(node()) -> rt_util:result(). +maybe_wait_for_changes(Node) -> + gen_fsm:sync_send_event(Node, maybe_wait_for_changes, ?TIMEOUT). %% @doc Return a list of cluster members according to the ring retrieved from %% the specified node. +%% +%% @since 1.1.0 -spec members_according_to(node()) -> [term()] | rt_util:error(). members_according_to(Node) -> - gen_fsm:sync_send_event(Node, members_according_to). + gen_fsm:sync_send_event(Node, members_according_to, ?TIMEOUT). %% @doc Return a list of nodes that own partitions according to the ring %% retrieved from the specified node. +%% +%% @since 1.1.0 -spec owners_according_to(node()) -> [term()]. owners_according_to(Node) -> - gen_fsm:sync_send_event(Node, owners_according_to). + gen_fsm:sync_send_event(Node, owners_according_to, ?TIMEOUT). %% @doc Get list of partitions owned by node (primary). +%% +%% @since 1.1.0 -spec partitions(node()) -> [term()]. partitions(Node) -> - gen_fsm:sync_send_event(Node, partitions). + gen_fsm:sync_send_event(Node, partitions, ?TIMEOUT). -spec ping(node()) -> boolean(). ping(Node) -> - gen_fsm:sync_send_all_state_event(Node, ping). + gen_fsm:sync_send_all_state_event(Node, ping, ?TIMEOUT). -spec plan(node()) -> rt_util:result(). plan(Node) -> - gen_fsm:sync_send_event(Node, plan). + gen_fsm:sync_send_event(Node, plan, ?TIMEOUT). + +%% @doc Releases the `Node' for use by another cluster. This function ensures +%% that the node is stopped before returning it for use by another cluster. +%% +%% @since 1.1.0 +-spec release(node()) -> rt_util:result(). +release(Node) -> + case whereis(Node) of + undefined -> + ok; + _ -> + gen_fsm:sync_send_all_state_event(Node, release, ?TIMEOUT) + end. %% @doc Call 'bin/riak' command on `Node' with arguments `Args' +%% +%% @since 1.1.0 -spec riak(node(), [term()]) -> {ok, term()} | rt_util:error(). riak(Node, Args) -> - gen_fsm:sync_send_event(Node, {riak, Args}). + gen_fsm:sync_send_all_state_event(Node, {riak, Args}, ?TIMEOUT). %% @doc Call 'bin/riak' command on `Node' with arguments `Args' +%% +%% @since 1.1.0 -spec riak_repl(node(), [term()]) -> {ok, term()} | rt_util:error(). riak_repl(Node, Args) -> - gen_fsm:sync_send_event(Node, {riak_repl, Args}). + gen_fsm:sync_send_event(Node, {riak_repl, Args}, ?TIMEOUT). -spec search_cmd(node(), [term()]) -> {ok, term()} | rt_util:error(). search_cmd(Node, Args) -> - gen_fsm:sync_send_event(Node, {search_cmd, Args}). + gen_fsm:sync_send_event(Node, {search_cmd, Args}, ?TIMEOUT). + +-spec set_cookie(node(), atom()) -> rt_util:result(). +set_cookie(Node, NewCookie) -> + gen_fsm:sync_send_event(Node, {set_cookie, NewCookie}, ?TIMEOUT). +-spec staged_join(node(), node()) -> rt_util:result(). +staged_join(Node, ToNode) -> + gen_fsm:sync_send_event(Node, {staged_join, ToNode}, ?TIMEOUT). + +%% TODO Document the behavior of start including ready checks -spec start(node()) -> rt_util:result(). start(Node) -> start(Node, true). -spec start(node(), boolean()) -> rt_util:result(). -start(Node, Wait) -> - gen_fsm:sync_send_event(Node, {start, Wait}). +start(Node, true) -> + gen_fsm:sync_send_event(Node, start, ?TIMEOUT); +start(Node, false) -> + gen_fsm:send_event(Node, start). %% @doc Starts a gen_fsm process to configure, start, and %% manage a Riak node on the `Host' identified by `NodeId' %% and `NodeName' using Riak `Version' ({product, release}) --spec start_link(host(), node_id(), node(), #configuration{}, string()) -> {ok, pid()} | ignore | rt_util:error(). -start_link(Host, NodeId, NodeName, Config, Version) -> - Args = [Host, NodeId, NodeName, Config, Version], - gen_fsm:start_link(NodeName, ?MODULE, Args, []). +-spec start_link(rt_host:hostname(), node_type(), node_id(), proplists:proplist(), rt_util:version()) -> + {ok, node()} | ignore | rt_util:error(). +start_link(Hostname, NodeType, NodeId, Config, Version) -> + %% TODO Re-implement node naming when 1.x and 2.x configuration is propely implemented + %% NodeName = list_to_atom(string:join([dev(NodeId), atom_to_list(Hostname)], "@")), + NodeName = list_to_atom(string:join([dev(NodeId), "127.0.0.1"], "@")), + Args = [Hostname, NodeType, NodeId, NodeName, Config, Version], + case gen_fsm:start({local, NodeName}, ?MODULE, Args, []) of + {ok, _} -> + {ok, NodeName}; + Error={error, _} -> + Error + end. %% @doc Return the cluster status of `Member' according to the ring %% retrieved from `Node'. +%% +%% @since 1.1.0 -spec status_of_according_to(node()) -> [term()] | rt_util:error(). status_of_according_to(Node) -> - gen_fsm:sync_send_event(Node, status_of_according_to). + gen_fsm:sync_send_event(Node, status_of_according_to, ?TIMEOUT). -spec stop(node()) -> rt_util:result(). stop(Node) -> stop(Node, true). -spec stop(node(), boolean()) -> rt_util:result(). -stop(Node, Wait) -> - gen_fsm:sync_send_event(Node, {stop, Wait}). +stop(Node, true) -> + gen_fsm:sync_send_event(Node, stop, ?TIMEOUT); +stop(Node, false) -> + gen_fsm:send_event(Node, stop). +-spec upgrade(node(), rt_util:version()) -> rt_util:result(). upgrade(Node, NewVersion) -> - gen_fsm:sync_send_event(Node, {upgrade, NewVersion}). + gen_fsm:sync_send_event(Node, {upgrade, NewVersion}, ?TIMEOUT). --spec wait_for_service(node(), [string()]) -> rt_util:result(). -wait_for_service(Node, Services) -> - gen_fsm:sync_send_event(Node, {wait_for_services, Services}). +-spec wait_for_service(node(), atom() | [atom()]) -> rt_util:wait_result(). +wait_for_service(Node, Services) when is_list(Services) -> + gen_fsm:sync_send_event(Node, {wait_for_services, Services}, ?TIMEOUT); +wait_for_service(Node, Service) -> + wait_for_service(Node, [Service]). --spec wait_until_pingable(node()) -> rt_util:result(). +-spec wait_until_pingable(node()) -> rt_util:wait_result(). wait_until_pingable(Node) -> - gen_fsm:sync_send_event(Node, wait_until_pingable). + gen_fsm:sync_send_event(Node, wait_until_pingable, ?TIMEOUT). --spec wait_until_registered(node(), atom()) -> rt_util:result(). +-spec wait_until_registered(node(), atom()) -> rt_util:wait_result(). wait_until_registered(Node, Name) -> - gen_fsm:sync_send_event(Node, {wait_until_registered, Name}). + gen_fsm:sync_send_event(Node, {wait_until_registered, Name}, ?TIMEOUT). --spec wait_until_unpingable(node()) -> rt_util:result(). +-spec wait_until_unpingable(node()) -> rt_util:wait_result(). wait_until_unpingable(Node) -> - gen_fsm:sync_send_event(Node, wait_until_unpingable). + gen_fsm:sync_send_all_state_event(Node, wait_until_unpingable, ?TIMEOUT). +-spec version(node()) -> rt_util:version(). version(Node) -> - gen_fsm:sync_send_all_state_event(Node, version). + gen_fsm:sync_send_all_state_event(Node, version, ?TIMEOUT). %%%=================================================================== %%% gen_fsm callbacks @@ -285,76 +385,187 @@ version(Node) -> %% {stop, StopReason} %% @end %%-------------------------------------------------------------------- -init([Host, NodeType, NodeId, NodeName, Config, Version]) -> - DirOverlay = create_directory_overlay(NodeType, Version), - State = #state{host=Host, - id=NodeId, - name=NodeName, - config=Config, - directory_overlay=DirOverlay, - start_command=start_command(NodeType, DirOverlay), - stop_command=stop_command(NodeType, DirOverlay), - version=Version}, - {ok, allocated, State}. - - -stopped(start, _From, State) -> - transition_stopped_to_started(State); -stopped(stop, _From, State=#state{name=NodeName}) -> - lager:debug("Stop called on an already stopped node ~p", [NodeName]), +init([Hostname, NodeType, NodeId, NodeName, ConfigOverrides, Version]) -> + DirOverlay = create_directory_overlay(NodeType, Version, NodeId), + case rt_host:connect(Hostname) of + {ok, Host} -> + DirCreateResult = create_snmp_dirs(Host, DirOverlay#directory_overlay.data_dir), + ConfigureResult = maybe_configure_node(DirCreateResult, Version, NodeName, Host, + DirOverlay, ConfigOverrides), + maybe_transition_to_stopped(ConfigureResult, Version, NodeId, NodeName, Host, + DirOverlay, NodeType); + Error -> + maybe_transition_to_stopped(Error, Version, NodeId, NodeName, none, DirOverlay, + NodeType) + end. + +-spec maybe_configure_node(rt_util:result(), rt_util:version(), node(), rt_host:host(), #directory_overlay{}, proplists:proplist()) -> rt_util:result(). +maybe_configure_node(ok, Version, NodeName, Host, DirOverlay, ConfigOverrides) -> + ExistingConfig = case load_configuration(Version, Host, DirOverlay) of + {ok, [Term]} -> + Term; + {error, LoadError} -> + lager:warning("Unable to load existing configuration for node ~p due to ~p. Defaulting to an empty configuration.", + [NodeName, LoadError]), + [] + end, + %% TODO account for properly assigning the node name ... + %% TODO handle backend ... + Config = rt_util:merge_configs(ExistingConfig, ConfigOverrides), + save_configuration(Version, Host, Config, DirOverlay); +maybe_configure_node(Error={error, _}, _Version, _NodeName, _Host, _DirOverlay, _ConfigOverrides) -> + Error. + +-spec maybe_transition_to_stopped(rt_util:result(), rt_util:version(), node_id(), node(), rt_host:host(), + #directory_overlay{}, atom()) -> {ok, stopped, #state{}} | {stop, term()}. +maybe_transition_to_stopped(ok, Version, NodeId, NodeName, Host, DirOverlay, NodeType) -> + State=#state{host=Host, + id=NodeId, + name=NodeName, + directory_overlay=DirOverlay, + start_command=start_command(NodeType, DirOverlay), + stop_command=stop_command(NodeType, DirOverlay), + version=Version}, + {ok, stopped, State}; +maybe_transition_to_stopped({error, Reason}, _Version, _NodeId, _NodeName, _Host, _DirOverlay, _NodeType) -> + {stop, Reason}. + +invalid(Event, _From, State) -> + lager:error("Attempt to perform ~p operation (state: ~p)", [Event, State]), + {reply, {error, node_invalid}, invalid, State}. + +stopped(clean_data_dir, _From, State=#state{directory_overlay=DirOverlay, host=Host}) -> + {reply, do_clean_data_dir(Host, DirOverlay), stopped, State}; +stopped({clean_data_dir, SubDir}, _From, State=#state{directory_overlay=DirOverlay, host=Host}) -> + {reply, do_clean_data_dir(Host, DirOverlay, SubDir), stopped, State}; +stopped(start, _From, State=#state{host=Host, name=Node}) -> + lager:info("Starting node synchronously ~p on ~p", [Node, Host]), + {Result, {NextState, UpdState}} = do_start_and_update_state(State), + {reply, Result, NextState, UpdState}; +stopped(stop, _From, State=#state{host=Host, name=Node}) -> + lager:warning("Stop called on an already stopped node ~p on ~p", [Node, Host]), {reply, ok, stopped, State}; -stopped(_Event, _From, State) -> +stopped(Event, _From, State=#state{host=Host, name=Node}) -> %% The state of the node is not harmed. Therefore, we leave the FSM running %% in the stopped state, but refuse to execute the command ... - {reply, {error, invalid_state}, stopped, State}. - - -transition_stopped_to_started(State=#state{start_command=StartCommand, transport=Transport}) -> - transition_stopped_to_started(Transport:exec(StartCommand), State). - -transition_stopped_to_started(ok, State) -> - {reply, ok, started, State}; -transition_stopped_to_started(Error={error, _}, State) -> - {stop, Error, State}. - - -ready({admin, Args, Options}, _From, State=#state{directory_overlay=DirOverlay, transport=Transport}) -> - Result = Transport:exec(riak_admin_path(DirOverlay), Args, Options), - {reply, Result, ready, State}; + lager:error("Invalid operation ~p when node ~p on ~p is in stopped state", [Event, Node, Host]), + {reply, {error, invalid_stopped_event}, stopped, State}. + +started(start, _From, State=#state{host=Host, name=Node, required_services=RequiredServices, + version=Version}) -> + case do_wait_until_ready(Host, Node, RequiredServices, Version) of + true -> {reply, ok, ready, State}; + false -> {error, node_not_ready, started, State} + end; +started(Event, _From, State=#state{host=Host, name=Node}) -> + lager:error("Invalid operation ~p when node ~p on ~p is in started state", [Event, Node, Host]), + {reply, {error, invalid_started_event}, started, State}. + +ready({admin, Args, Options}, _From, State=#state{host=Host, directory_overlay=DirOverlay}) -> + {reply, do_admin(Args, Options, DirOverlay, Host), ready, State}; +ready({attach, Expected}, _From, State=#state{directory_overlay=DirOverlay}) -> + {reply, do_attach(DirOverlay, Expected), ready, State}; +ready({attach_direct, Expected}, _From, State=#state{directory_overlay=DirOverlay}) -> + {reply, do_attach_direct(DirOverlay, Expected), ready, State}; +ready(check_singleton, _From, State=#state{host=Host, name=Node, version=Version}) -> + %% TODO consider adding the started state back + transition_to_state_and_reply(do_check_singleton(Host, Node, Version), ready, stopped, State); +ready(commit, _From, State=#state{host=Host, name=Node}) -> + {reply, do_commit(Host, Node), ready, State}; +ready({console, Expected}, _From, State=#state{directory_overlay=DirOverlay}) -> + {reply, do_console(DirOverlay, Expected), ready, State}; +ready(cookie, _From, State=#state{name=Node}) -> + {reply, rt_util:maybe_rpc_call(Node, erlang, get_cookie, []), ready, State}; +%% TODO Determine whether or not it makes sense to support get_ring in the started +%% state ... ready(get_ring, _From, #state{name=NodeName}=State) -> - {ok, Ring} = maybe_get_ring(NodeName), - {reply, Ring, ready, State}; + Result = maybe_get_ring(NodeName), + {reply, Result, ready, State}; +ready({join, ToNode}, _From, State=#state{host=Host, name=Node}) -> + {reply, do_join(Host, Node, ToNode), ready, State}; +ready(maybe_wait_for_changes, _From, State=#state{host=Host, name=Node}) -> + {reply, do_maybe_wait_for_changes(Host, Node), ready, State}; ready(members_according_to, _From, #state{name=NodeName}=State) -> Members = maybe_members_according_to(NodeName), {reply, Members, ready, State}; ready(owners_according_to, _From, #state{name=NodeName}=State) -> Owners = maybe_owners_according_to(NodeName), {reply, Owners, ready, State}; -ready(partitions, _From, #state{name=NodeName}=State) -> - Partitions = maybe_partitions(NodeName), +ready(partitions, _From, State=#state{name=Node}) -> + Partitions = maybe_partitions(Node), {reply, Partitions, ready, State}; -ready({riak, Args}, _From, State=#state{directory_overlay=DirOverlay, transport=Transport}) -> - Result = Transport:exec(riak_path(DirOverlay), Args), +ready(plan, _From, State=#state{host=Host, name=Node}) -> + {reply, do_plan(Host, Node), ready, State}; +ready({riak_repl, Args}, _From, State=#state{host=Host, directory_overlay=DirOverlay}) -> + Result = rt_host:exec(Host, riak_repl_path(DirOverlay), Args), {reply, Result, ready, State}; -ready({riak_repl, Args}, _From, State=#state{directory_overlay=DirOverlay, transport=Transport}) -> - Result = Transport:exec(riak_repl_path(DirOverlay), Args), - {reply, Result, ready, State}. - --spec maybe_get_ring(node()) -> rt_rpc_result(). +ready({set_cookie, NewCookie}, _From, State=#state{name=Node}) -> + {reply, rt_util:maybe_rpc_call(Node, erlang, set_cookie, [Node, NewCookie]), ready, State}; +ready({staged_join, ToNode}, _From, State=#state{host=Host, name=Node}) -> + {reply, do_staged_join(Host, Node, ToNode), ready, State}; +ready(stop, _From, State) -> + {Result, UpdState} = do_stop_and_update_state(State), + %% TODO need an invalid state ... + transition_to_state_and_reply(Result, stopped, ready, UpdState); +ready({wait_for_service, Services}, _From, State=#state{host=Host, name=Node}) -> + %% TODO consider adding back the started state + transition_to_state_and_reply(do_wait_for_service(Host, Node, Services), ready, stopped, State); +ready(wait_until_pingable, _From, State=#state{host=Host, name=Node}) -> + %% TODO consider adding back the started state + transition_to_state_and_reply(do_wait_until_pingable(Host, Node), ready, stopped, State); +ready({wait_until_registered, Name}, _From, State=#state{host=Host, name=Node}) -> + transition_to_state_and_reply(do_wait_until_registered(Host, Node, Name), ready, stopped, State); +ready(Event, _From, State=#state{host=Host, name=Node}) -> + %% The state of the node is not harmed. Therefore, we leave the FSM running + %% in the stopped state, but refuse to execute the command ... + lager:error("Invalid operation ~p when node ~p on ~p is in ready state", [Event, Node, Host]), + {reply, {error, invalid_ready_event}, ready, State}. + +transition_to_state_and_reply(Result={error, _}, _SuccessState, FailedState, State) -> + {reply, Result, FailedState, State}; +transition_to_state_and_reply(Result, SuccessState, _FailedState, State) -> + {reply, Result, SuccessState, State}. + + +transition_to_state({error, _}, _SuccessState, FailedState, State) -> + {next_state, FailedState, State}; +transition_to_state(ok, SuccessState, _FailedState, State) -> + {next_state, SuccessState, State}. + +stopped(start, State=#state{host=Host, name=Node}) -> + lager:info("Starting node asynchronously ~p on ~p", [Node, Host]), + {_, {NextState, UpdState}} = do_start_and_update_state(State), + {next_state, NextState, UpdState}; +stopped(stop, State=#state{host=Host, name=Node}) -> + lager:warning("Stop called on a stopped node ~p on ~p", [Node, Host]), + {next_state, stopped, State}; +stopped(_Event, State) -> + {next_state, stopped, State}. + +ready(stop, State) -> + {Result, UpdState} = do_stop_and_update_state(State), + transition_to_state(Result, stopped, ready, UpdState); +ready(start, State=#state{host=Host, name=Node}) -> + lager:warning("Start called on ready node ~p on ~p", [Node, Host]), + {next_state, ready, State}; +ready(_Event, State) -> + {next_state, ready, State}. + +-spec maybe_get_ring(node()) -> rt_util:rt_rpc_result(). maybe_get_ring(NodeName) -> - maybe_rpc_call(NodeName, riak_core_ring_manager, get_raw_ring, []). + rt_util:maybe_rpc_call(NodeName, riak_core_ring_manager, get_raw_ring, []). --spec maybe_partitions(node()) -> rt_rpc_result(). +-spec maybe_partitions(node()) -> rt_util:rt_rpc_result(). maybe_partitions(NodeName) -> maybe_partitions(NodeName, maybe_get_ring(NodeName)). --spec maybe_partitions(node(), rt_rpc_result()) -> [term()] | rt_util:error(). +-spec maybe_partitions(node(), rt_util:rt_rpc_result()) -> [term()] | rt_util:error(). maybe_partitions(NodeName, {ok, Ring}) -> [Idx || {Idx, Owner} <- riak_core_ring:all_owners(Ring), Owner == NodeName]; maybe_partitions(_NodeName, {error, Reason}) -> {error, Reason}. --spec maybe_members_according_to(node() | rt_rpc_result()) -> [term()] | rt_util:error(). +-spec maybe_members_according_to(node() | rt_util:tt_rpc_result()) -> [term()] | rt_util:error(). maybe_members_according_to({ok, Ring}) -> riak_core_ring:all_members(Ring); maybe_members_according_to({error, Reason}) -> @@ -362,7 +573,7 @@ maybe_members_according_to({error, Reason}) -> maybe_members_according_to(NodeName) -> maybe_members_according_to(maybe_get_ring(NodeName)). --spec maybe_owners_according_to(node() | rt_rpc_result()) -> [term()] | rt_util:error(). +-spec maybe_owners_according_to(node() | rt_util:rt_rpc_result()) -> [term()] | rt_util:error(). maybe_owners_according_to({ok, Ring}) -> Owners = [Owner || {_Idx, Owner} <- riak_core_ring:all_owners(Ring)], lists:usort(Owners); @@ -371,19 +582,6 @@ maybe_owners_according_to({error, Reason}) -> maybe_owners_according_to(NodeName) -> maybe_owners_according_to(maybe_get_ring(NodeName)). -%% TODO Move to rt_util ?? --type erl_rpc_result() :: {ok, term()} | {badrpc, term()}. --type rt_rpc_result() :: {ok, term()} | rt_util:error(). --spec maybe_rpc_call(node(), module(), function(), [term()]) -> erl_rpc_result(). -maybe_rpc_call(NodeName, Module, Function, Args) -> - maybe_rpc_call(rpc:call(NodeName, Module, Function, Args)). - --spec maybe_rpc_call(erl_rpc_result()) -> rt_rpc_result(). -maybe_rpc_call({badrpc, _}) -> - {error, badrpc}; -maybe_rpc_call(Result) -> - Result. - %%-------------------------------------------------------------------- %% @private %% @doc @@ -416,9 +614,44 @@ handle_event(_Event, StateName, State) -> %% {stop, Reason, Reply, NewState} %% @end %%-------------------------------------------------------------------- -handle_sync_event(_Event, _From, StateName, State) -> - Reply = ok, - {reply, Reply, StateName, State}. +%% handle_sync_event(_Event, _From, StateName, State) -> +%% Reply = ok, +%% {reply, Reply, StateName, State}. +handle_sync_event(brutal_kill, _From, _StateName, + State=#state{host=Host, name=Node, os_pid=OSPid}) -> + case do_brutal_kill(Node, Host, OSPid) of + ok -> UpdState=State#state{os_pid=0}, + {reply, ok, stopped, UpdState}; + Error={error, _} -> {stop, Error, State} + end; +handle_sync_event({copy_logs, ToDir}, _From, StateName, + State=#state{host=Host, directory_overlay=DirOverlay}) -> + Result = rt_host:copy_dir(Host, DirOverlay#directory_overlay.log_dir, ToDir), + transition_to_state_and_reply(Result, StateName, StateName, State); +handle_sync_event(host, _From, StateName, State=#state{host=Host}) -> + {reply, Host, StateName, State}; +handle_sync_event(ip, _From, StateName, State=#state{host=Host}) -> + {reply, rt_host:ip_addr(Host), StateName, State}; +handle_sync_event(is_ready, _From, ready, State) -> + {reply, true, ready, State}; +handle_sync_event(is_ready, _From, StateName, State) -> + {reply, false, StateName, State}; +handle_sync_event(is_stopped, _From, stopped, State) -> + {reply, true, stopped, State}; +handle_sync_event(is_stopped, _From, StateName, State) -> + {reply, false, StateName, State}; +handle_sync_event(ping, _From, StateName, State=#state{name=Node}) -> + {reply, do_ping(Node), StateName, State}; +handle_sync_event({riak, Args}, _From, StateName, + State=#state{host=Host, directory_overlay=DirOverlay}) -> + {reply, rt_host:exec(Host, riak_path(DirOverlay), Args), StateName, State}; +handle_sync_event(release, _From, _StateName, State=#state{host=Host, name=Node}) -> + lager:info("Releasing node ~p on ~p", [Node, Host]), + {stop, normal, ok, State}; +handle_sync_event(version, _From, StateName, State=#state{version=Version}) -> + {reply, Version, StateName, State}; +handle_sync_event(wait_until_unpingable, _From, StateName, State=#state{host=Host, name=Node}) -> + {reply, do_wait_until_unpingable(Host, Node), StateName, State}. %%-------------------------------------------------------------------- %% @private @@ -447,7 +680,10 @@ handle_info(_Info, StateName, State) -> %% @spec terminate(Reason, StateName, State) -> void() %% @end %%-------------------------------------------------------------------- -terminate(_Reason, _StateName, _State) -> +terminate(_Reason, _StateName, + #state{host=Host=Host, name=Node, os_pid=OSPid}) -> + _ = do_brutal_kill(Node, Host, OSPid), + rt_host:disconnect(Host), ok. %%-------------------------------------------------------------------- @@ -466,10 +702,16 @@ code_change(_OldVsn, StateName, State, _Extra) -> %%% Internal functions %%%=================================================================== -%% TODO Convert to the version() type when it is exported .. --spec create_directory_overlay({atom(), path()}, {string(), string()}) -> #directory_overlay{}. -create_directory_overlay([devrel, RootPath], {Product, Version}) -> - HomeDir = filename:join([RootPath], Product ++ "-" ++ Version), +-spec create_snmp_dirs(rt_host:host(), filelib:dirname()) -> rt_util:result(). +create_snmp_dirs(Host, DataDir) -> + SnmpDir = filename:join([DataDir, "snmp", "agent", "db"]), + lager:debug("Creating SNMP data directory ~p on localhost", [SnmpDir]), + rt_host:mkdirs(Host, SnmpDir). + +-spec create_directory_overlay({atom(), filelib:dirname()}, rt_util:version(), node()) -> #directory_overlay{}. +create_directory_overlay({devrel, RootPath}, Version, NodeId) -> + HomeDir = filename:join([rt_util:base_dir_for_version(RootPath, Version), + dev(NodeId)]), #directory_overlay{bin_dir=filename:join([HomeDir, "bin"]), conf_dir=filename:join([HomeDir, "etc"]), data_dir=filename:join([HomeDir, "data"]), @@ -477,23 +719,608 @@ create_directory_overlay([devrel, RootPath], {Product, Version}) -> lib_dir=filename:join([HomeDir, "lib"]), log_dir=filename:join([HomeDir, "log"])}. --spec start_command({devrel, string()}, #directory_overlay{}) -> command(). +-spec dev(pos_integer()) -> string(). +dev(NodeId) -> + lists:concat(["dev", integer_to_list(NodeId)]). + +-spec do_admin([term()], [term()], #directory_overlay{}, rt_host:host()) -> {ok, term()} | rt_util:error(). +do_admin(Args, Options, DirOverlay, Host) -> + rt_host:exec(Host, riak_admin_path(DirOverlay), Args, Options). + +-spec do_attach(#directory_overlay{}, term()) -> term(). +do_attach(DirOverlay, Expected) -> + interactive(DirOverlay, "attach", Expected). + +-spec do_attach_direct(#directory_overlay{}, term()) -> term(). +do_attach_direct(DirOverlay, Expected) -> + interactive(DirOverlay, "attach-direct", Expected). + +-spec do_brutal_kill(node(), rt_util:host(), pos_integer()) -> rt_util:result(). +do_brutal_kill(_Node, _Host, 0) -> + ok; +do_brutal_kill(Node, Host, OSPid) -> + lager:info("Killing node ~p running as PID ~s", [Node, OSPid]), + %% try a normal kill first, but set a timer to + %% kill -9 after 5 seconds just in case + timer:apply_after(5000, rt_host, kill, [Host, 9, OSPid]), + rt_host:kill(Host, 15, OSPid). + +%% @private +-spec do_check_singleton(rt_host:host(), node(), rt_util:version()) -> boolean() | rt_util:error(). +do_check_singleton(Host, Node, {_, Release}) when Release =/= "0.14.2" -> + HostName = rt_host:hostname(Host), + lager:info("Check that node ~p on ~p is a singleton", [Node, HostName]), + Result = case maybe_get_ring(Node) of + {ok, Ring} -> + Owners = lists:usort([Owner || {_Idx, Owner} <- riak_core_ring:all_owners(Ring)]), + [Node] =:= Owners; + Error -> + Error + end, + do_check_singleton(Result); +do_check_singleton(_Host, _Node, _Version) -> + true. + +do_check_singleton(true) -> + true; +do_check_singleton(false) -> + {error, node_not_singleton}; +do_check_singleton(Error) -> + Error. + +-spec do_clean_data_dir(rt_host:host(), #directory_overlay{}) -> rt_util:result(). +do_clean_data_dir(Host, DirOverlay) -> + do_clean_data_dir(Host, DirOverlay, ""). + +-spec do_clean_data_dir(rt_host:host(), #directory_overlay{}, filelib:dirname()) -> rt_util:result(). +do_clean_data_dir(Host, #directory_overlay{data_dir=DataDir}, SubDir) -> + TmpDir = rt_host:temp_dir(Host), + FromDir = filename:join([DataDir, SubDir]), + ToDir = filename:join([TmpDir, "child"]), + HostName = rt_host:hostname(Host), + + lager:info("Cleaning data directory ~p on ~p", [FromDir, HostName]), + rt_util:maybe_call_funs([ + [rt_host, mkdirs, [Host, ToDir]], + [rt_host, mvdir, [Host, FromDir, ToDir]], + [rt_host, rmdir, [Host, ToDir]], + [rt_host, mkdirs, [Host, ToDir]] + ]). + +-spec do_commit(rt_util:host(), node()) -> rt_util:result(). +do_commit(Host, Node) -> + lager:info("Commit cluster plan using node ~p on host ~p", + [Node, Host]), + case rt_util:maybe_rpc_call(Node, riak_core_claimant, commit, []) of + {error, plan_changed} -> + lager:info("commit: plan changed"), + timer:sleep(100), + ok = do_maybe_wait_for_changes(Host, Node), + ok = do_plan(Host, Node), + ok = do_commit(Host, Node); + {error, ring_not_ready} -> + lager:info("commit: ring not ready"), + timer:sleep(100), + ok = do_maybe_wait_for_changes(Host, Node), + ok = do_commit(Host, Node); + {error,nothing_planned} -> + %% Assume plan actually committed somehow + ok; + ok -> + ok + end. + +-spec do_console(#directory_overlay{}, term()) -> term(). +do_console(DirOverlay, Expected) -> + interactive(DirOverlay, "console", Expected). + +-spec do_join(rt_util:host(), node(), node()) -> rt_util:result(). +do_join(Host, Node, ToNode) -> + Result = rt_util:maybe_rpc_call(Node, riak_core, join, [ToNode]), + lager:info("[join] ~p on ~p to ~p: ~p", [Node, Host, ToNode, Result]), + Result. + +-spec do_staged_join(rt_util:host(), node(), node()) -> rt_util:result(). +do_staged_join(Host, Node, ToNode) -> + Result = rt_util:maybe_rpc_call(Node, riak_core, staged_join, [ToNode]), + lager:info("[staged join] ~p on ~p to (~p): ~p", [Node, Host, ToNode, Result]), + Result. + +-spec do_maybe_wait_for_changes(rt_host:host(), node()) -> ok. +do_maybe_wait_for_changes(Host, Node) -> + Ring = rt_ring:get_ring(Node), + Changes = riak_core_ring:pending_changes(Ring), + Joining = riak_core_ring:members(Ring, [joining]), + lager:info("maybe_wait_for_changes node ~p on ~p, changes: ~p joining: ~p ", + [Node, Host, Changes, Joining]), + if Changes =:= [] -> + ok; + Joining =/= [] -> + ok; + true -> + ok = rt_util:wait_until_no_pending_changes(Host, [Node]) + end. + +-spec do_plan(rt_host:host(), node()) -> rt_util:result(). +do_plan(Host, Node) -> + Result = rt_util:maybe_rpc_call(Node, riak_core_claimant, plan, []), + lager:info("Planned cluster using node ~p on ~p with result ~p", [Node, Host, Result]), + case Result of + {ok, _, _} -> ok; + Error -> Error + end. + +-spec do_ping(node()) -> boolean(). +do_ping(Node) -> + net_adm:ping(Node) =:= pong. + + +-spec do_start(rt_util:host(), node(), rt_host:command()) -> pos_integer | rt_util:error(). +do_start(Host, Node, StartCommand) -> + OSPid = rt_util:maybe_call_funs([ + [?MODULE, start_riak_daemon, [Host, Node, StartCommand]], + [?MODULE, get_os_pid, [Host, Node]] + ]), + lager:info("Started node ~p on ~p with OS pid ~p", [Node, Host ,OSPid]), + OSPid. + +-spec do_wait_until_ready(rt_host:host(), node(), [atom()], rt_util:version()) -> boolean. +do_wait_until_ready(Host, Node, RequiredServices, Version) -> + lager:debug("Checking pingable, registered, singleton, and services for node ~p", + [Node]), + Result = rt_util:maybe_call_funs([ + [?MODULE, do_wait_until_pingable, [Host, Node]], + [?MODULE, do_wait_until_registered, [Host, Node, riak_core_ring_manager]], + [?MODULE, do_check_singleton, [Host, Node, Version]], + [?MODULE, do_wait_for_service, [Host, Node, RequiredServices]] + ]), + case Result of + ok -> true; + Error -> lager:warning("Unable to determine that node ~p on ~p is ready due to ~p", + [Node, Host, Error]), + false + end. + +-spec do_start_and_update_state(#state{}) -> {ok | rt_util:error(), {invalid | started | ready, #state{}}}. +do_start_and_update_state(State=#state{host=Host, name=Node, required_services=RequiredServices, + start_command=StartCommand, version=Version}) -> + + OSPid = do_start(Host, Node, StartCommand), + + case OSPid of + Error={error, _} -> {Error, {invalid, State}}; + _ -> UpdState = State#state{os_pid=OSPid}, + Result = do_wait_until_ready(Host, Node, RequiredServices, Version), + case Result of + true -> {ok, {ready, UpdState}}; + false -> {{error, node_not_ready}, {started, UpdState}} + end + end. + + +-spec do_stop_and_update_state(rt_host:host()) -> {ok | {error, term()}, #state{}}. +do_stop_and_update_state(State=#state{host=Host, stop_command=StopCommand}) -> + case do_stop(Host, StopCommand) of + {ok, _} -> {ok, State#state{os_pid=0}}; + {error, Reason} -> {{error, Reason}, State} + end. + +-spec do_stop(rt_host:host(), string()) -> rt_util:result(). +do_stop(Host, StopCommand) -> + rt_host:exec(Host, StopCommand). + +%% @private +-spec do_wait_for_service(rt_host:host(), node(), [atom()]) -> rt_util:wait_result(). +do_wait_for_service(Host, Node, Services) -> + HostName = rt_host:hostname(Host), + lager:info("Waiting for services ~p on node ~p on ~p", [Services, Node, HostName]), + F = fun(N) -> + case rt_util:maybe_rpc_call(N, riak_core_node_watcher, services, [N]) of + Error={error, _} -> + lager:error("Request for the list of services on node ~p on ~p failed due to ~p", + [Node, HostName, Error]), + Error; + CurrServices when is_list(CurrServices) -> + lager:debug("Found services ~p on ~p on ~p", [CurrServices, N, HostName]), + lists:all(fun(Service) -> lists:member(Service, CurrServices) end, Services); + Res -> + Res + end + end, + rt_util:wait_until(Node, F). + +%% @private +-spec do_wait_until_pingable(rt_host:host(), node()) -> rt_util:wait_result(). +do_wait_until_pingable(Host, Node) -> + HostName = rt_host:hostname(Host), + lager:info("Waiting for node ~p on ~p to become pingable", [Node, HostName]), + rt_util:wait_until(Node, fun do_ping/1). + +%% @private +-spec do_wait_until_registered(rt_host:host(), node(), atom()) -> rt_util:wait_result(). +do_wait_until_registered(Host, Node, Name) -> + HostName = rt_host:hostname(Host), + lager:info("Waiting for node ~p on ~p to become registered", [Node, HostName]), + F = fun() -> + Registered = rpc:call(Node, erlang, registered, []), + lists:member(Name, Registered) + end, + case rt_util:wait_until(F) of + ok -> + lager:info("Node ~p on ~p is registered", [Node, HostName]), + ok; + _ -> + lager:error("The server ~p on node ~p on ~p is not coming up.", + [Name, Node, HostName]), + ?assert(registered_name_timed_out) + end. + +-spec do_wait_until_unpingable(rt_host:host(), node()) -> rt_util:result(). +do_wait_until_unpingable(Host, Node) -> + do_wait_until_unpingable(Host, Node, rt_config:get(rt_max_receive_wait_time)). + +-spec do_wait_until_unpingable(rt_host:host(), node(), pos_integer()) -> rt_util:result(). +do_wait_until_unpingable(Host, Node, WaitTime) -> + Delay = rt_config:get(rt_retry_delay), + Retry = WaitTime div Delay, + lager:info("Wait until ~p on ~p is not pingable for ~p seconds with a retry of ~p", + [Node, Host, Delay, Retry]), + %% TODO Move to stop ... + case rt_util:wait_until(fun() -> do_unpingable(Node) end, Retry, Delay) of + ok -> ok; + _ -> + lager:error("Timed out waiting for node ~p on ~p to shutdown", [Node, Host]), + {error, node_shutdown_timed_out} + end. + + +%% @private +-spec do_unpingable(node()) -> boolean(). +do_unpingable(Node) -> + net_adm:ping(Node) =:= pang. + +%% @private +-spec get_os_pid(rt_host:host(), node()) -> pos_integer(). +get_os_pid(Host, Node) -> + HostName = rt_host:hostname(Host), + lager:info("Retrieving the OS pid for node ~p on ~p", [Node, HostName]), + case rt_util:maybe_rpc_call(Node, os, getpid, []) of + Error={error, _} -> + Error; + OSPid -> + list_to_integer(OSPid) + end. + +%% TODO Consider how to capture errors to provide a more meaningul status. Using assertions +%% feels wrong ... +-spec interactive(#directory_overlay{}, string(), term()) -> term(). +interactive(DirOverlay, Command, Expected) -> + Cmd = riak_path(DirOverlay), + lager:info("Opening a port for riak ~s.", [Command]), + lager:debug("Calling open_port with cmd ~s", [binary_to_list(iolist_to_binary(Cmd))]), + Port = open_port({spawn, binary_to_list(iolist_to_binary(Cmd))}, + [stream, use_stdio, exit_status, binary, stderr_to_stdout]), + interactive_loop(Port, Expected). + +interactive_loop(Port, Expected) -> + receive + {Port, {data, Data}} -> + %% We've gotten some data, so the port isn't done executing + %% Let's break it up by newline and display it. + Tokens = string:tokens(binary_to_list(Data), "\n"), + [lager:debug("~s", [Text]) || Text <- Tokens], + + %% Now we're going to take hd(Expected) which is either {expect, X} + %% or {send, X}. If it's {expect, X}, we foldl through the Tokenized + %% data looking for a partial match via rt:str/2. If we find one, + %% we pop hd off the stack and continue iterating through the list + %% with the next hd until we run out of input. Once hd is a tuple + %% {send, X}, we send that test to the port. The assumption is that + %% once we send data, anything else we still have in the buffer is + %% meaningless, so we skip it. That's what that {sent, sent} thing + %% is about. If there were a way to abort mid-foldl, I'd have done + %% that. {sent, _} -> is just a pass through to get out of the fold. + + NewExpected = lists:foldl(fun(X, Expect) -> + [{Type, Text}|RemainingExpect] = case Expect of + [] -> [{done, "done"}|[]]; + E -> E + end, + case {Type, rt2:str(X, Text)} of + {expect, true} -> + RemainingExpect; + {expect, false} -> + [{Type, Text}|RemainingExpect]; + {send, _} -> + port_command(Port, list_to_binary(Text ++ "\n")), + [{sent, "sent"}|RemainingExpect]; + {sent, _} -> + Expect; + {done, _} -> + [] + end + end, Expected, Tokens), + %% Now that the fold is over, we should remove {sent, sent} if it's there. + %% The fold might have ended not matching anything or not sending anything + %% so it's possible we don't have to remove {sent, sent}. This will be passed + %% to interactive_loop's next iteration. + NewerExpected = case NewExpected of + [{sent, "sent"}|E] -> E; + E -> E + end, + %% If NewerExpected is empty, we've met all expected criteria and in order to boot + %% Otherwise, loop. + case NewerExpected of + [] -> ?assert(true); + _ -> interactive_loop(Port, NewerExpected) + end; + {Port, {exit_status,_}} -> + %% This port has exited. Maybe the last thing we did was {send, [4]} which + %% as Ctrl-D would have exited the console. If Expected is empty, then + %% We've met every expectation. Yay! If not, it means we've exited before + %% something expected happened. + ?assertEqual([], Expected) + after rt_config:get(rt_max_receive_wait_time) -> + %% interactive_loop is going to wait until it matches expected behavior + %% If it doesn't, the test should fail; however, without a timeout it + %% will just hang forever in search of expected behavior. See also: Parenting + ?assertEqual([], Expected) + end. + + +-spec start_command({devrel, string()}, #directory_overlay{}) -> rt_host:command(). start_command({devrel, _}, DirOverlay) -> - riak_path(DirOverlay) ++ " " ++ "start". + {riak_path(DirOverlay), ["start"]}. + +-spec start_riak_daemon(rt_host:host(), node(), rt_host:command()) -> rt_util:result(). +start_riak_daemon(Host, Node, StartCommand) -> + HostName = rt_host:hostname(Host), + lager:notice("Starting riak node ~p on ~p", [Node, HostName]), + rt_host:exec(Host, StartCommand). --spec stop_command({devrel, string()}, #directory_overlay{}) -> command(). +-spec stop_command({devrel, string()}, #directory_overlay{}) -> rt_host:command(). stop_command({devrel, _}, DirOverlay) -> - riak_path(DirOverlay) ++ " " ++ "stop". + {riak_path(DirOverlay), ["stop"]}. --spec riak_path(#directory_overlay{}) -> path(). +-spec riak_path(#directory_overlay{}) -> filelib:filename(). riak_path(#directory_overlay{bin_dir=BinDir}) -> filename:join([BinDir, "riak"]). --spec riak_admin_path(#directory_overlay{}) -> path(). +-spec riak_admin_path(#directory_overlay{}) -> filelib:filename(). riak_admin_path(#directory_overlay{bin_dir=BinDir}) -> filename:join([BinDir, "riak-admin"]). --spec riak_repl_path(#directory_overlay{}) -> path(). +-spec riak_repl_path(#directory_overlay{}) -> filelib:filename(). riak_repl_path(#directory_overlay{bin_dir=BinDir}) -> filename:join([BinDir, "riak-repl"]). +-spec load_configuration(rt_util:version(), rt_host:host(), #directory_overlay{}) -> + {ok, proplists:proplist()} | rt_util:error(). +load_configuration(Version, Host, #directory_overlay{conf_dir=ConfDir}) -> + rt_host:consult(Host, config_file_path(rt_util:major_release(Version), ConfDir)). + +-spec save_configuration(rt_util:release(), rt_host:host(), term(), #directory_overlay{}) -> rt_util:result(). +save_configuration(Version, Host, Config, #directory_overlay{conf_dir=ConfDir}) -> + MajorRelease = rt_util:major_release(Version), + rt_host:write_file(Host, config_file_path(MajorRelease, ConfDir), rt_util:term_serialized_form(Config)). + +-spec config_file_path(rt_util:release(), filelib:dirname()) -> filelib:filename() | rt_util:error(). +config_file_path(MajorRelease, ConfDir) -> + filename:join(ConfDir, config_file_name(MajorRelease)). + +-spec config_file_name(rt_util:release()) -> string(). +config_file_name(1) -> + "app.config"; +config_file_name(2) -> + "advanced.config"; +config_file_name(Release) -> + erlang:error(io_lib:format("Configuration of release ~p is not supported", [Release])). + +-ifdef(TEST). + +-define(TEST_ROOT_PATH, filename:join([os:getenv("HOME"), "rt", "riak"])). +-define(TEST_VERSION, {riak_ee, "2.0.5"}). + +bootstrap() -> + rt_util:setup_test_env(). + +init_node(HostName, NodeId) -> + {Result, Node} = start_link(HostName, {devrel, ?TEST_ROOT_PATH}, NodeId, [], ?TEST_VERSION), + ?assertEqual(ok, clean_data_dir(Node)), + ?debugFmt("Initialized node id ~p (~p) on ~p as ~p with result ~p", + [NodeId, ?TEST_VERSION, HostName, Node, Result]), + ?assertEqual(ok, Result), + ?assertEqual(true, is_stopped(Node)), + ?assertEqual(false, is_ready(Node)), + + Node. + +setup() -> + true = bootstrap(), + init_node(localhost, 1). + +multi_node_setup(NumNodes) -> + true = bootstrap(), + [ init_node(localhost, NodeId) || NodeId <- lists:seq(1, NumNodes) ]. + +teardown(Nodes) when is_list(Nodes) -> + lists:map(fun(Node) -> release(Node) end, Nodes); +teardown(Node) -> + ?debugFmt("Releasing node ~p", [Node]), + release(Node). + +dev_test_() -> + ?_assertEqual("dev1", dev(1)). + +ip_test_() -> + {timeout, 300, + {foreach, + fun setup/0, + fun teardown/1, + [fun(Node) -> {"get_ip", ?_assertEqual("127.0.0.1", ip(Node))} end]}}. + +verify_async_start(Node) -> + Result = start(Node, false), + ?debugFmt("Started node asynchronously ~p with result ~p", [Node, Result]), + + ?assertEqual(ok, Result), + Result. + +verify_sync_start(Node) -> + Result = start(Node), + ?debugFmt("Started node synchronously ~p with result ~p", [Node, Result]), + + ?assertEqual(ok, Result), + ?assertEqual(false, is_stopped(Node)), + ?assertEqual(true, is_ready(Node)), + Result. + +verify_sync_stop(Node) -> + Result = stop(Node), + ?debugFmt("Stopped node ~p with result ~p", [Node, Result]), + + ?assertEqual(ok, Result), + ?assertEqual(true, is_stopped(Node)), + ?assertEqual(false, is_ready(Node)), + Result. + +async_start_test_() -> + {foreach, + fun setup/0, + fun teardown/1, + [fun(Node) -> + {timeout, 600000, + ?_test(begin + ok = verify_async_start(Node), + ?assertEqual(false, is_stopped(Node)), + ?debugFmt("Node ~p asyncronously started .. checking is_ready", [Node]), + ?assertEqual(ok, rt_util:wait_until(Node, fun(Node1) -> is_ready(Node1) end)) + end)} + end]}. + +cookie_test_() -> + {foreach, + fun setup/0, + fun teardown/1, + [fun(Node) -> + {timeout, 600000, + ?_test(begin + ok = verify_sync_start(Node), + ?assertEqual(riak, cookie(Node)) + end)} + end]}. + +load_configuration_test_() -> + {foreach, + fun () -> + application:ensure_started(exec), + {Result, Host} = rt_host:connect(localhost), + ?assertEqual(ok, Result), + Host + end, + fun(Host) -> rt_host:disconnect(Host) end, + [fun(Host) -> + ?_test(begin + DirOverlay = create_directory_overlay({devrel, ?TEST_ROOT_PATH}, ?TEST_VERSION, 1), + {Result, Config} = load_configuration(?TEST_VERSION, Host, DirOverlay), + + ?assertEqual(ok, Result), + ?assertEqual(true, is_list(Config)) + end) + end]}. + +plan_commit_join_test_() -> + {foreach, + fun() -> multi_node_setup(3) end, + fun teardown/1, + [fun(Nodes) -> + {timeout, 600000, + ?_test(begin + %% Start all of the nodes ... + StartResults = rt_util:pmap(fun(Node) -> rt_riak_node:start(Node) end, Nodes), + lists:map(fun(Result) -> ?assertEqual(ok, Result) end, StartResults), + + %% Split the list into a leader and followers ... + [Leader|Followers] = Nodes, + + %% Join the followers to the leader ... + JoinResults = rt_util:pmap(fun(Follower) -> rt_riak_node:join(Follower, Leader) end, Followers), + lists:map(fun(Result) -> ?assertEqual(ok, Result) end, JoinResults), + ?debugFmt("Joined nodes ~p to ~p with result ~p", [Followers, Leader, JoinResults]), + ?assertEqual([ok, ok], JoinResults), + + %% Plan the cluster on the leader ... + PlanResult = plan(Leader), + ?debugFmt("Plan result: ~p", [PlanResult]), + ?assertEqual(ok, PlanResult), + + %% Commit the cluster changes on the leader ... + CommitResult = commit(Leader), + ?assertEqual(ok, CommitResult) + end)} + end]}. + + +sync_start_stop_test_() -> + {foreach, + fun setup/0, + fun teardown/1, + [fun(Node) -> + {timeout, 600000, + ?_test(begin + ok = verify_sync_start(Node), + ok = verify_sync_stop(Node) + end)} + end]}. + +wait_until_pingable_test_() -> + {foreach, + fun setup/0, + fun teardown/1, + [fun(Node) -> + {timeout, 600000, + ?_test(begin + ok = verify_sync_start(Node), + ?assertEqual(ok, wait_until_pingable(Node)) + end)} + end, + fun(Node) -> + {timeout, 600000, + ?_test(begin + ok = verify_async_start(Node), + ?assertEqual(ok, wait_until_pingable(Node)) + end)} + end]}. + +wait_until_unpingable_test_() -> + {foreach, + fun setup/0, + fun teardown/1, + [fun(Node) -> + {timeout, 600000, + ?_test(begin + ok = verify_sync_start(Node), + Result = stop(Node), + ?assertEqual(ok, Result), + WaitResult = wait_until_unpingable(Node), + ?assertEqual(ok, WaitResult) + end)} + end, + fun(Node) -> + {timeout, 600000, + ?_test(begin + ok = verify_sync_start(Node), + Result = stop(Node, false), + ?assertEqual(ok, Result), + WaitResult = wait_until_unpingable(Node), + ?assertEqual(ok, WaitResult) + end)} + end]}. + +version_test_() -> + {foreach, + fun setup/0, + fun teardown/1, + [fun(Node) -> {"get_version", ?_assertEqual(?TEST_VERSION, version(Node))} end]}. + +-endif. diff --git a/src/rt_util.erl b/src/rt_util.erl index 6217c9876..a8b213934 100644 --- a/src/rt_util.erl +++ b/src/rt_util.erl @@ -1,42 +1,253 @@ -module(rt_util). +-include_lib("eunit/include/eunit.hrl"). -type error() :: {error(), term()}. -type result() :: ok | error(). +-type wait_result() ::ok | {fail, term()}. + +-type erl_rpc_result() :: {ok, term()} | {badrpc, term()}. +-type rt_rpc_result() :: {ok, term()} | rt_util:error(). + +-type release() :: string(). +-type products() :: riak | riak_ee | riak_cs | riak_cs_ee. +-type version() :: {products(), release()}. +-type version_selector() :: atom() | version(). -export_type([error/0, - result/0]). --export([backend_to_atom_list/1]). + products/0, + release/0, + result/0, + rt_rpc_result/0, + version/0, + version_selector/0, + wait_result/0]). + +-export([add_deps/1, + base_dir_for_version/2, + ip_addr_to_string/1, + maybe_append_when_not_endswith/2, + maybe_call_funs/1, + maybe_rpc_call/4, + major_release/1, + merge_configs/2, + pmap/2, + parse_release/1, + term_serialized_form/1, + version_to_string/1, + wait_until/1, + wait_until/2, + wait_until/3, + wait_until_no_pending_changes/2]). -ifdef(TEST). --include_lib("eunit/include/eunit.hrl"). +-export([setup_test_env/0, + test_success_fun/0]). -endif. -%% @doc Convert string or atom to list of atoms --spec(backend_to_atom_list(atom()|string()) -> undefined | list()). -backend_to_atom_list(undefined) -> - undefined; -backend_to_atom_list(Backends) when is_atom(Backends) -> - ListBackends = atom_to_list(Backends), - case lists:member($, , ListBackends) of - true -> - [list_to_atom(X) || X <- string:tokens(ListBackends, ", ")]; - _ -> - [Backends] - end; -backend_to_atom_list(Backends) when is_list(Backends) -> - case lists:member($, , Backends) of - true -> - [list_to_atom(X) || X <- string:tokens(Backends, ", ")]; - _ -> - [list_to_atom(Backends)] +-spec add_deps(filelib:dirname()) -> ok. +add_deps(Path) -> + lager:debug("Adding dep path ~p", [Path]), + case file:list_dir(Path) of + {ok, Deps} -> + [code:add_path(lists:append([Path, "/", Dep, "/ebin"])) || Dep <- Deps], + ok; + {error, Reason} -> + lager:error("Failed to add dep path ~p due to ~p.", [Path, Reason]), + erlang:error(Reason) + end. + +-spec base_dir_for_version(filelib:dirname(), version()) -> filelib:dirname(). +base_dir_for_version(RootPath, Version) -> + filename:join(RootPath, version_to_string(Version)). + +-spec ip_addr_to_string({pos_integer(), pos_integer(), pos_integer(), pos_integer()}) -> string(). +ip_addr_to_string(IP) -> + string:join([integer_to_list(X) || X <- tuple_to_list(IP)], "."). + +-spec maybe_append_when_not_endswith(string(), string()) -> string(). +maybe_append_when_not_endswith(String, Suffix) -> + maybe_append_when_not_endswith(lists:suffix(Suffix, String), String, Suffix). + +-spec maybe_append_when_not_endswith(boolean(), string(), string()) -> string(). +maybe_append_when_not_endswith(true, String, _Suffix) -> + String; +maybe_append_when_not_endswith(false, String, Suffix) -> + String ++ Suffix. + +-spec maybe_rpc_call(node(), module(), function(), [term()]) -> erl_rpc_result(). +maybe_rpc_call(NodeName, Module, Function, Args) -> + maybe_rpc_call(rpc:call(NodeName, Module, Function, Args)). + +%% -spec maybe_call_funs([module(), function(), [term()]]) -> term(). +maybe_call_funs(CallSpecs) -> + lists:foldl(fun([Module, Function, Args], PrevResult) -> + maybe_call_fun(PrevResult, Module, Function, Args) + end, ok, CallSpecs). + +-spec maybe_call_fun(ok | {ok, term()} | rt_util:error(), module(), function(), [term()]) -> term(). +maybe_call_fun(ok, Module, Function, Args) -> + erlang:apply(Module, Function, Args); +maybe_call_fun({ok, _}, Module, Function, Args) -> + erlang:apply(Module, Function, Args); +maybe_call_fun(true, Module, Function, Args) -> + erlang:apply(Module, Function, Args); +maybe_call_fun(Error, Module, Function, Args) -> + lager:debug("~p:~p(~p) not called due error ~p", [Module, Function, Args, Error]), + Error. + +-spec maybe_rpc_call(erl_rpc_result()) -> rt_rpc_result(). +maybe_rpc_call({badrpc, _}) -> + {error, badrpc}; +maybe_rpc_call(Result) -> + Result. + +-spec merge_configs(proplists:proplist() | tuple(), proplists:proplist() | tuple()) -> orddict:orddict() | tuple(). +merge_configs(PropList, ThatPropList) when is_list(PropList) and is_list(ThatPropList) -> + MergeA = orddict:from_list(PropList), + MergeB = orddict:from_list(ThatPropList), + orddict:merge(fun(_, VarsA, VarsB) -> + merge_configs(VarsA, VarsB) + end, MergeA, MergeB); +merge_configs(_, Value) -> + Value. + +-spec major_release(version() | release()) -> pos_integer(). +major_release({_, Release}) -> + major_release(Release); +major_release(Release) -> + {Major, _, _} = parse_release(Release), + Major. + +%% @doc Parallel Map: Runs function F for each item in list L, then +%% returns the list of results +-spec pmap(F :: fun(), L :: list()) -> list(). +pmap(F, L) -> + Parent = self(), + lists:foldl( + fun(X, N) -> + spawn_link(fun() -> + Parent ! {pmap, N, F(X)} + end), + N+1 + end, 0, L), + L2 = [receive {pmap, N, R} -> {N,R} end || _ <- L], + {_, L3} = lists:unzip(lists:keysort(1, L2)), + L3. + +-spec parse_release(version() | release()) -> { pos_integer(), pos_integer(), pos_integer() }. +parse_release({_, Release}) -> + parse_release(Release); +parse_release(Release) -> + list_to_tuple([list_to_integer(Token) || Token <- string:tokens(Release, ".")]). + +-spec term_serialized_form(term()) -> string(). +term_serialized_form(Term) -> + io_lib:format("~p.", [Term]). + +-spec version_to_string(version()) -> string(). +version_to_string({Product, Release}) -> + string:join([atom_to_list(Product), Release], "-"). + +%% @doc Utility function used to construct test predicates. Retries the +%% function `Fun' until it returns `true', or until the maximum +%% number of retries is reached. The retry limit is based on the +%% provided `rt_max_receive_wait_time' and `rt_retry_delay' parameters in +%% specified `riak_test' config file. +%% +%% @since 1.1.0 +-spec wait_until(function()) -> wait_result(). +wait_until(Fun) when is_function(Fun) -> + MaxTime = rt_config:get(rt_max_receive_wait_time), + Delay = rt_config:get(rt_retry_delay), + Retry = MaxTime div Delay, + wait_until(Fun, Retry, Delay). + +%% @doc Convenience wrapper for wait_until for the myriad functions that +%% take a node as single argument. +%% +%% @since 1.1.0 +-spec wait_until(node(), function()) -> wait_result(). +wait_until(Node, Fun) when is_atom(Node), is_function(Fun) -> + wait_until(fun() -> Fun(Node) end). + +%% @doc Retry `Fun' until it returns `Retry' times, waiting `Delay' +%% milliseconds between retries. This is our eventual consistency bread +%% and butter +%% +%% @since 1.1.0 +-spec wait_until(function(), pos_integer(), pos_integer()) -> wait_result(). +wait_until(Fun, Retry, Delay) when Retry > 0 -> + Res = Fun(), + case Res of + true -> + ok; + _ when Retry == 1 -> + {fail, Res}; + _ -> + timer:sleep(Delay), + wait_until(Fun, Retry-1, Delay) end. -ifdef(TEST). -%% Properly covert backends to atoms -backend_to_atom_list_test() -> - ?assertEqual(undefined, backend_to_atom_list(undefined)), - ?assertEqual([memory], backend_to_atom_list(memory)), - ?assertEqual([memory], backend_to_atom_list("memory")), - ?assertEqual([bitcask, eleveldb, memory], lists:sort(backend_to_atom_list("memory, bitcask,eleveldb"))), - ?assertEqual([bitcask, eleveldb, memory], lists:sort(backend_to_atom_list('memory, bitcask,eleveldb'))). --endif. \ No newline at end of file + +-spec wait_until_no_pending_changes(rt_host:host(), [node()]) -> ok | fail. +wait_until_no_pending_changes(Host, Nodes) -> + lager:info("Wait until no pending changes for nodes ~p on ~p", [Nodes, Host]), + F = fun() -> + rpc:multicall(Nodes, riak_core_vnode_manager, force_handoffs, []), + {Rings, BadNodes} = rpc:multicall(Nodes, riak_core_ring_manager, get_raw_ring, []), + Changes = [ riak_core_ring:pending_changes(Ring) =:= [] || {ok, Ring} <- Rings ], + BadNodes =:= [] andalso length(Changes) =:= length(Nodes) andalso lists:all(fun(T) -> T end, Changes) + end, + wait_until(F). + +test_success_fun() -> + ok. + +setup_test_env() -> + application:ensure_started(exec), + rt_config:set(rt_max_receive_wait_time, 600000), + rt_config:set(rt_retry_delay, 1000), + + %% TODO Consider loading up the riak_test.config to get this information + add_deps(filename:join([os:getenv("HOME"), "rt", ".riak-builds", "riak_ee-head", "deps"])), + + {ok, _} = exec:run("epmd -daemon", [sync, stdout, stderr]), + net_kernel:start(['riak_test@127.0.0.1']), + erlang:set_cookie(node(), riak). + +base_dir_for_version_test_() -> + [?_assertEqual("foo/riak_ee-2.0.5", base_dir_for_version("foo", {riak_ee, "2.0.5"}))]. + +%% TODO Refactor into an EQC model ... +ip_addr_to_string_test_() -> + [?_assertEqual("127.0.0.1", ip_addr_to_string({127, 0, 0, 1}))]. + +maybe_append_when_not_endswith_test_() -> + [?_assertEqual("foobar", maybe_append_when_not_endswith("foobar", "bar")), + ?_assertEqual("foobar", maybe_append_when_not_endswith("foo", "bar"))]. + +maybe_call_fun_test_() -> + [?_assertEqual(ok, maybe_call_fun(ok, ?MODULE, test_success_fun, [])), + ?_assertEqual({error, test_failure}, maybe_call_fun({error, test_failure}, ?MODULE, test_success_fun, [])) ]. + +%% TODO Refactor into an EQC model ... +major_version_test_() -> + [?_assertEqual(1, major_release("1.3.4")), + ?_assertEqual(1, major_release({riak_ee, "1.3.4"}))]. + +%% TODO Refactor into an EQC model ... +merge_configs_test_() -> + [?_assertEqual([{a,1},{b,2},{c,3},{d,4}], merge_configs([{a,1},{b,2}],[{c,3},{d,4}])), + ?_assertEqual([{a,1},{b,3},{c,3},{d,4}], merge_configs([{a,1},{b,2}],[{b,3},{c,3},{d,4}])), + ?_assertEqual([{a, [{b,3}, {c,5}, {d,6}]}, {e,7}], merge_configs([{a, [{b,2}, {c,5}]}], [{a, [{b,3}, {d,6}]}, {e,7}]))]. + +%% TODO Refactor into an EQC model ... +parse_release_test_() -> + [?_assertEqual({1, 3, 4}, parse_release("1.3.4")), + ?_assertEqual({1, 3, 4}, parse_release({riak_ee, "1.3.4"}))]. + +version_to_string_test_() -> + [?_assertEqual("riak_ee-2.0.5", version_to_string({riak_ee, "2.0.5"}))]. + +-endif. From 2f11dd30a21d52c6b2fe02dfe247bceb4e76d232 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Wed, 20 May 2015 09:45:39 -0600 Subject: [PATCH 152/157] Add back in support for version upgrade path for BTA-204 - Required to get legacy upgrade tests to pass (e.g. BTA-231, BTA-232) - Upgrade path name is in rt_test_plan currently for execution - Fix get_node_logs/0 for verify_handoff_mixed (BTA-75), yz_rs_migration_test and yz_solr_start_timeout - Fix rtdev:node_id/0 for kv679_dataloss (BTA-218) and verify_riak_stats (BTA-174) --- examples/riak_test.config | 38 ++++++++++++++++++++------------------ src/giddyup.erl | 13 ++++--------- src/riak_test_escript.erl | 35 ++++++++++++++++------------------- src/riak_test_executor.erl | 34 ++++++++++++++++++---------------- src/riak_test_runner.erl | 10 +++++++++- src/rt.erl | 7 +++++++ src/rt2.erl | 3 +++ src/rt_harness.erl | 4 ++++ src/rt_planner.erl | 30 +++++++++++++++++++++++------- src/rt_properties.erl | 2 +- src/rt_test_plan.erl | 13 +++++++++---- src/rt_util.erl | 30 ++++++++++++++++++++++++++++++ src/rtdev.erl | 24 ++++++++++++++++++++++-- 13 files changed, 166 insertions(+), 77 deletions(-) diff --git a/examples/riak_test.config b/examples/riak_test.config index f72b3564f..cfa7f82ae 100644 --- a/examples/riak_test.config +++ b/examples/riak_test.config @@ -105,25 +105,26 @@ %% The path to the version before `previous', which %% is used when doing upgrade tests. %% Versions are tuple of product and version number - {'latest_1.3_ee', {riak, "1.3.4"}}, + {'latest_1.4_ee', {riak, "1.4.12"}}, %% The path to the most immediately previous version %% of the project, which is used when doing upgrade %% tests. - {'latest_1.4_ee', {riak, "1.4.12"}}, + {'latest_2.0_ee', {riak, "2.0.5"}}, %% The path to the `current' version, which is used %% exclusively except during upgrade tests. - {'latest_2.0_ee', {riak, "2.0.5"}}, - {'previous_2.0_ee', {riak, "2.0.4"}}, + {'latest_2.1_ee', {riak, "2.1.1"}}, + {'previous_2.1_ee', {riak, "2.1.0"}}, %% Backwards-compatible aliases to versions - {default, 'latest_2.0_ee'}, - {previous, 'latest_1.4_ee'}, - {legacy, 'latest_1.3_ee'} + {default, 'latest_2.1_ee'}, + {previous, 'latest_2.0_ee'}, + {legacy, 'latest_1.4_ee'} ]}, {upgrade_paths, [ %% Lists of aliases to possible upgrade versions - {full, ['latest_1.3_ee', 'latest_1.4_ee', 'latest_2.0_ee']}, - {latest, ['latest_1.4_ee', 'latest_2.0_ee']}, - {minor, ['previous_2.0_ee', 'latest_2.0_ee']} + {full, ['latest_1.4_ee', 'latest_2.0_ee', 'latest_2.1_ee']}, + {legacy, ['latest_1.4_ee', 'latest_2.1_ee']}, + {previous, ['latest_2.0_ee', 'latest_2.1_ee']}, + {minor, ['previous_2.1_ee', 'latest_2.1_ee']} ]} ]}. @@ -150,18 +151,19 @@ %% [See rtdev.rtdev_path above] {root_path, "/Users/hazen/dev/rt/riak"}, {versions, [ - {'latest_1.3_ee', {riak_ee, "1.3.4"}}, {'latest_1.4_ee', {riak_ee, "1.4.12"}}, {'latest_2.0_ee', {riak_ee, "2.0.5"}}, - {'previous_2.0_ee', {riak_ee, "2.0.4"}}, - {default, 'latest_2.0_ee'}, - {previous, 'latest_1.4_ee'}, - {legacy, 'latest_1.3_ee'} + {'latest_2.1_ee', {riak_ee, "2.1.1"}}, + {'previous_2.1_ee', {riak_ee, "2.1.0"}}, + {default, 'latest_2.1_ee'}, + {previous, 'latest_2.0_ee'}, + {legacy, 'latest_1.4_ee'} ]}, {upgrade_paths, [ - {full, ['latest_1.3_ee', 'latest_1.4_ee', 'latest_2.0_ee']}, - {latest, ['latest_1.4_ee', 'latest_2.0_ee']}, - {minor, ['previous_2.0_ee', 'latest_2.0_ee']} + {full, ['latest_1.4_ee', 'latest_2.0_ee', 'latest_2.1_ee']}, + {previous, ['latest_2.0_ee', 'latest_2.1_ee']}, + {legacy, ['latest_1.4_ee', 'latest_2.1_ee']}, + {minor, ['previous_2.1_ee', 'latest_2.1_ee']} ]} ]}. diff --git a/src/giddyup.erl b/src/giddyup.erl index 21f8a800e..ec8ee4273 100644 --- a/src/giddyup.erl +++ b/src/giddyup.erl @@ -167,7 +167,7 @@ init([Platform, Product, VersionNumber, Version, GiddyUpHost, GiddyUpUser, Giddy {stop, Reason :: term(), Reply :: term(), NewState :: #state{}} | {stop, Reason :: term(), NewState :: #state{}}). handle_call(get_test_plans, _From, State) -> - TestPlans = fetch_all_test_plans(State#state.platform, State#state.default_product, State#state.default_version_number, State#state.default_version, State#state.giddyup_host), + TestPlans = fetch_all_test_plans(State#state.platform, State#state.default_product, State#state.default_version_number, State#state.giddyup_host), {reply, TestPlans, State}; handle_call(_Request, _From, State) -> {reply, ok, State}. @@ -253,8 +253,8 @@ code_change(_OldVsn, State, _Extra) -> %% @end %%-------------------------------------------------------------------- --spec fetch_all_test_plans(string(), string(), string(), string(), string()) -> [rt_test_plan:test_plan()]. -fetch_all_test_plans(Platform, Product, VersionNumber, DefaultVersion, Host) -> +-spec fetch_all_test_plans(string(), string(), string(), string()) -> [rt_test_plan:test_plan()]. +fetch_all_test_plans(Platform, Product, VersionNumber, Host) -> %% Make sure ibrowse is up and running rt:check_ibrowse(), Schema = get_schema(Platform, Product, VersionNumber, Host), @@ -274,12 +274,7 @@ fetch_all_test_plans(Platform, Product, VersionNumber, DefaultVersion, Host) -> {ok, Plan2} = case kvc:path('tags.upgrade_version', Test) of [] -> {ok, Plan1}; UpgradeVsn -> - UpgradeVersion = case UpgradeVsn of - <<"legacy">> -> rt_config:get_legacy_version(); - <<"previous">> -> rt_config:get_previous_version(); - _ -> rt_config:get_version(binary_to_list(UpgradeVsn)) - end, - rt_test_plan:set(upgrade_path, [UpgradeVersion, DefaultVersion], Plan1) + rt_test_plan:set(upgrade_path, binary_to_atom(UpgradeVsn, utf8), Plan1) end, %% TODO: Remove? No tests currently use this multi_config setting %% Plan3 = case kvc:path('tags.multi_config', Test) of diff --git a/src/riak_test_escript.erl b/src/riak_test_escript.erl index 7b8a3a540..182a6d6fa 100644 --- a/src/riak_test_escript.erl +++ b/src/riak_test_escript.erl @@ -81,6 +81,7 @@ cli_options() -> {verbose, $v, "verbose", undefined, "verbose output"}, {outdir, $o, "outdir", string, "output directory"}, {backend, $b, "backend", atom, "backend to test [memory | bitcask | eleveldb | multi]"}, + {upgrade_path, $u, "upgrade", atom, "which user-defined upgrade path to use [ e.g. previous | legacy ]"}, {keep, undefined, "keep", boolean, "do not teardown cluster"}, {continue_on_fail, $n, "continue", boolean, "continues executing tests on failure"}, {report, $r, "report", string, "you're reporting an official test run, provide platform info (e.g. ubuntu-1404-64)\nUse 'config' if you want to pull from ~/.riak_test.config"}, @@ -107,9 +108,10 @@ generate_test_lists(UseGiddyUp, ParsedArgs) -> %% test metadata TestData = compose_test_data(ParsedArgs), - CmdLineBackends = rt_util:backend_to_atom_list(proplists:get_value(backend, ParsedArgs)), + CmdLineBackends = rt_util:convert_to_atom_list(proplists:get_value(backend, ParsedArgs)), Backends = determine_backends(CmdLineBackends, UseGiddyUp), - {Tests, NonTests} = wrap_test_in_test_plan(UseGiddyUp, Backends, TestData), + UpgradeList = rt_util:convert_to_atom_list(proplists:get_value(upgrade_path, ParsedArgs)), + {Tests, NonTests} = wrap_test_in_test_plan(UseGiddyUp, Backends, UpgradeList, TestData), Offset = rt_config:get(offset, undefined), Workers = rt_config:get(workers, undefined), shuffle_tests(Tests, NonTests, Offset, Workers). @@ -180,13 +182,14 @@ prepare_tests(Tests, NonTests) -> test_setup(). execute(TestPlans, OutDir, ParsedArgs) -> - UpgradeList = upgrade_list( - proplists:get_value(upgrade_path, ParsedArgs)), + %% TODO: Clean up upgrade paths. Living in test plans at the moment. + %% UpgradeList = upgrade_list( + %% proplists:get_value(upgrade_path, ParsedArgs)), {ok, Executor} = riak_test_executor:start_link(TestPlans, OutDir, report_platform(ParsedArgs), - UpgradeList, + undefined, self()), wait_for_results(Executor, [], length(TestPlans), 0). @@ -232,12 +235,6 @@ test_setup() -> end, ok. --spec upgrade_list(undefined | string()) -> undefined | [string()]. -upgrade_list(undefined) -> - undefined; -upgrade_list(Path) -> - string:tokens(Path, ","). - erlang_setup(_ParsedArgs) -> register(riak_test, self()), maybe_add_code_path("./ebin"), @@ -344,18 +341,18 @@ extract_test_names(Test, {CodePaths, TestNames}) -> %% @doc Determine which tests to run based on command-line argument %% If the platform is defined, consult GiddyUp, otherwise just shovel %% the whole thing into the Planner --spec(load_up_test_planner(boolean(), [string()], list()) -> list()). -load_up_test_planner(true, Backends, CommandLineTests) -> +-spec(load_up_test_planner(boolean(), [string()], undefined | [string()], list()) -> list()). +load_up_test_planner(true, Backends, _UpgradePaths, CommandLineTests) -> rt_planner:load_from_giddyup(Backends, CommandLineTests); -load_up_test_planner(_, Backends, CommandLineTests) -> - [rt_planner:add_test_plan(Name, undefined, Backends, undefined, undefined) || Name <- CommandLineTests]. +load_up_test_planner(_, Backends, UpgradePaths, CommandLineTests) -> + [rt_planner:add_test_plan(Name, undefined, Backends, UpgradePaths, undefined) || Name <- CommandLineTests]. %% @doc Push all of the test into the Planner for now and wrap them in an `rt_test_plan' %% TODO: Let the Planner do the work, not the riak_test_executor --spec(wrap_test_in_test_plan(boolean(), [string()], [atom()]) -> {list(), list()}). -wrap_test_in_test_plan(UseGiddyUp, Backends, CommandLineTests) -> +-spec(wrap_test_in_test_plan(boolean(), [atom()], undefined | [atom()], [atom()]) -> {list(), list()}). +wrap_test_in_test_plan(UseGiddyUp, Backends, UpgradeList, CommandLineTests) -> {ok, _Pid} = rt_planner:start_link(), - load_up_test_planner(UseGiddyUp, Backends, CommandLineTests), + load_up_test_planner(UseGiddyUp, Backends, UpgradeList, CommandLineTests), TestPlans = [rt_planner:fetch_test_plan() || _ <- lists:seq(1, rt_planner:number_of_plans())], NonRunnableTestPlans = [rt_planner:fetch_test_non_runnable_plan() || _ <- lists:seq(1, rt_planner:number_of_non_runable_plans())], rt_planner:stop(), @@ -472,7 +469,7 @@ default_backend_test() -> %% Make sure that GiddyUp supports all backends default_giddyup_backend_test() -> - ?assertEqual([bitcask, eleveldb, memory], lists:sort(determine_backends(undefined, true))). + ?assertEqual([bitcask, eleveldb, memory, multi], lists:sort(determine_backends(undefined, true))). %% Command-line backends should always rule cmdline_backend_test() -> diff --git a/src/riak_test_executor.erl b/src/riak_test_executor.erl index 12b2865ce..51e0484bf 100644 --- a/src/riak_test_executor.erl +++ b/src/riak_test_executor.erl @@ -201,7 +201,9 @@ launch_test(Event, State) -> ok. maybe_reserve_nodes(NextTestPlan, TestProps) -> - VersionsToTest = versions_to_test(TestProps), + %% TODO: Clean up upgrade resolution. Go either with executor or test plan. + %% VersionsToTest = versions_to_test(TestProps), + VersionsToTest = [rt_config:convert_to_string(rt_test_plan:get(version, NextTestPlan))], maybe_reserve_nodes(erlang:function_exported(rt_test_plan:get_module(NextTestPlan), confirm, 1), NextTestPlan, VersionsToTest, TestProps). @@ -209,12 +211,12 @@ maybe_reserve_nodes(true, NextTest, VersionsToTest, TestProps) -> NodeCount = rt_properties:get(node_count, TestProps), %% Send async request to node manager - lager:notice("Requesting ~p nodes for the next test, ~p", [NodeCount, rt_test_plan:get_module(NextTest)]), + lager:notice("Requesting ~p nodes for the next test, ~p", [NodeCount, rt_test_plan:get_name(NextTest)]), node_manager:reserve_nodes(NodeCount, VersionsToTest, reservation_notify_fun()); maybe_reserve_nodes(false, NextTest, VersionsToTest, _TestProps) -> - lager:warning("~p is an old style test that requires conversion.", [rt_test_plan:get_module(NextTest)]), + lager:warning("~p is an old style test that requires conversion.", [rt_test_plan:get_name(NextTest)]), node_manager:reserve_nodes(0, VersionsToTest, reservation_notify_fun()), ok. @@ -291,23 +293,23 @@ test_property_fun(OverrideProps) -> [{TestPlan, Properties} | Acc] end. -versions_to_test(Properties) -> - versions_to_test(Properties, rt_properties:get(rolling_upgrade, Properties)). - %% An `upgrade_path' specified on the command line overrides the test %% property setting. If the `rolling_upgrade' property is is `false' %% then the `start_version' property of the test is the only version %% tested. -versions_to_test(Properties, true) -> - case rt_properties:get(upgrade_path, Properties) of - undefined -> - versions_to_test(Properties, false); - UpgradePath -> - [rt_config:convert_to_string(Upgrade) || Upgrade <- UpgradePath] - end; -versions_to_test(Properties, false) -> - InitialVersion = rt_properties:get(start_version, Properties), - [rt_config:convert_to_string(InitialVersion)]. +%% versions_to_test(Properties) -> +%% versions_to_test(Properties, rt_properties:get(rolling_upgrade, Properties)). +%% +%% versions_to_test(Properties, true) -> +%% case rt_properties:get(upgrade_path, Properties) of +%% undefined -> +%% versions_to_test(Properties, false); +%% UpgradePath -> +%% [rt_config:convert_to_string(Upgrade) || Upgrade <- UpgradePath] +%% end; +%% versions_to_test(Properties, false) -> +%% InitialVersion = rt_properties:get(start_version, Properties), +%% [rt_config:convert_to_string(InitialVersion)]. %% Function to abstract away the details of what properties %% can be overridden on the command line. diff --git a/src/riak_test_runner.erl b/src/riak_test_runner.erl index 3d4f665d5..8ba63a866 100644 --- a/src/riak_test_runner.erl +++ b/src/riak_test_runner.erl @@ -108,11 +108,19 @@ init([TestPlan, Properties, ContinueOnFail, ReporterPid, LogDir]) -> Project = list_to_binary(rt_config:get(rt_project, "undefined")), Backend = rt_test_plan:get(backend, TestPlan), TestModule = rt_test_plan:get_module(TestPlan), - MetaData = [{id, -1}, + %% Populate metadata for backwards-compatiblity + MetaData0 = [{id, -1}, {platform, <<"local">>}, {version, rt:get_version()}, {backend, Backend}, {project, Project}], + %% Upgrade Version is legacy for 'previous' or 'legacy' to 'current' upgrade + %% For old tests, keep those labels. Converted tests could use an arbitrary name. + UpgradePath = rt_test_plan:get(upgrade_path, TestPlan), + MetaData = case UpgradePath of + undefined -> MetaData0; + _ -> MetaData0 ++ [{upgrade_version, UpgradePath}] + end, %% TODO: Remove after all tests ported 2.0 -- workaround to support %% backend command line argument fo v1 cluster provisioning -jsb diff --git a/src/rt.erl b/src/rt.erl index 38243bed6..ef395aadd 100644 --- a/src/rt.erl +++ b/src/rt.erl @@ -63,6 +63,7 @@ expect_in_log/2, get_deps/0, get_ip/1, + get_node_logs/0, get_node_logs/2, get_replica/5, get_ring/1, @@ -1089,6 +1090,12 @@ setup_harness(_Test, _Args) -> get_node_logs(LogFile, DestDir) -> rt2:get_node_logs(LogFile, DestDir). +%% @doc Open all of the nodes' log files to a port +%% OBSOLETE +-spec(get_node_logs() -> list()). +get_node_logs() -> + rt2:get_node_logs(). + check_ibrowse() -> rt2:check_ibrowse(). diff --git a/src/rt2.erl b/src/rt2.erl index 12839117a..81a01f8b4 100644 --- a/src/rt2.erl +++ b/src/rt2.erl @@ -613,6 +613,9 @@ setup_harness(Test, Args) -> get_node_logs(LogFile, DestDir) -> rt_harness:get_node_logs(LogFile, DestDir). +get_node_logs() -> + rt_harness:get_node_logs(). + check_ibrowse() -> try sys:get_status(ibrowse) of {status, _Pid, {module, gen_server} ,_} -> ok diff --git a/src/rt_harness.erl b/src/rt_harness.erl index 0db02049a..d0d79bc0f 100644 --- a/src/rt_harness.erl +++ b/src/rt_harness.erl @@ -37,6 +37,7 @@ cmd/2, setup/0, get_deps/0, + get_node_logs/0, get_node_logs/2, get_version/0, get_version/1, @@ -107,6 +108,9 @@ get_version(Node) -> get_backends() -> ?HARNESS_MODULE:get_backends(). +get_node_logs() -> + ?HARNESS_MODULE:get_node_logs(). + get_node_logs(LogFile, DestDir) -> ?HARNESS_MODULE:get_node_logs(LogFile, DestDir). diff --git a/src/rt_planner.erl b/src/rt_planner.erl index 9383483ca..14bd2c5be 100644 --- a/src/rt_planner.erl +++ b/src/rt_planner.erl @@ -92,9 +92,9 @@ load_from_giddyup(Backends, CommandLineTests) -> %% %% @end %%-------------------------------------------------------------------- --spec(add_test_plan(string(), string(), [atom()], rt_properties2:product_version(), rt_properties2:properties()) -> ok). -add_test_plan(Module, Platform, Backends, Version, Properties) -> - gen_server:call(?MODULE, {add_test_plan, Module, Platform, Backends, Version, Properties}). +-spec(add_test_plan(string(), string(), [atom()], [rt_properties2:product_version()], rt_properties2:properties()) -> ok). +add_test_plan(Module, Platform, Backends, UpgradePaths, Version) -> + gen_server:call(?MODULE, {add_test_plan, Module, Platform, Backends, UpgradePaths, Version}). %%-------------------------------------------------------------------- %% @doc @@ -209,11 +209,20 @@ handle_call({load_from_giddyup, Backends, CommandLineTests}, _From, State) -> State2 = lists:foldl(fun exclude_test_plan/2, State1, Excluded1), {reply, ok, State2}; %% Add a single test plan for each backend to the queue -handle_call({add_test_plan, Module, Platform, Backends, _Version, _Properties}, _From, State) -> +handle_call({add_test_plan, Module, Platform, Backends, UpgradePaths, _Version}, _From, State) -> State1 = lists:foldl(fun(Backend, AccState) -> - TestPlan = rt_test_plan:new([{module, Module}, {platform, Platform}, {backend, Backend}]), - sort_and_queue(TestPlan, AccState) - end, + case UpgradePaths of + undefined -> + TestPlan = rt_test_plan:new([{module, Module}, {platform, Platform}, {backend, Backend}]), + sort_and_queue(TestPlan, AccState); + UpgradePaths -> + lists:foldl(fun(UpgradePath, AccState1) -> + TestPlan = rt_test_plan:new([{module, Module}, {platform, Platform}, {backend, Backend}, {upgrade_path, UpgradePath}]), + sort_and_queue(TestPlan, AccState1) + end, + AccState, UpgradePaths) + end + end, State, Backends), {reply, ok, State1}; handle_call(fetch_test_plan, _From, State) -> @@ -306,6 +315,13 @@ code_change(_OldVsn, State, _Extra) -> %%% Internal functions %%%=================================================================== +%%-------------------------------------------------------------------- +%% @private +%% @doc +%% Add a single test plan +%% +%% @end +%%-------------------------------------------------------------------- %%-------------------------------------------------------------------- %% @private diff --git a/src/rt_properties.erl b/src/rt_properties.erl index 7b45c3a58..8b1fb66d6 100644 --- a/src/rt_properties.erl +++ b/src/rt_properties.erl @@ -45,7 +45,7 @@ rolling_upgrade=false :: boolean(), start_version=rt_config:get_default_version() :: string(), current_version :: string(), - upgrade_path :: [string()], + upgrade_path=undefined :: [atom()], wait_for_transfers=false :: boolean(), valid_backends=all :: all | [atom()], make_cluster=true :: boolean(), diff --git a/src/rt_test_plan.erl b/src/rt_test_plan.erl index 22c725db5..9679f9899 100644 --- a/src/rt_test_plan.erl +++ b/src/rt_test_plan.erl @@ -41,9 +41,9 @@ module :: atom(), project=rt_config:get_default_version_product() :: atom() | binary(), platform :: string(), - version=rt_config:get_default_version_number() :: string(), + version=rt_config:get_default_version() :: string(), backend=undefined :: atom(), - upgrade_path=[] :: [rt_properties2:product_version()], + upgrade_path=undefined :: [rt_properties2:product_version()], properties :: rt_properties2:properties() }). @@ -86,14 +86,19 @@ get(Field, TestPlan) -> get_module(TestPlan) -> get(module, TestPlan, validate_request(module, TestPlan)). -% @doc Get the value of the name and backend from a test plan record. An error +%% @doc Get the value of the name, backend and upgrade from a test plan record. An error %% is returned if `TestPlan' is not a valid `rt_test_plan' record %% or if the field requested is not a valid field. -spec get_name(test_plan()) -> term() | {error, atom()}. get_name(TestPlan) -> Module = get(module, TestPlan, validate_request(module, TestPlan)), Backend = get(backend, TestPlan, validate_request(backend, TestPlan)), - atom_to_list(Module) ++ "-" ++ atom_to_list(Backend). + ModBackend = atom_to_list(Module) ++ "-" ++ atom_to_list(Backend), + Upgrade = get(upgrade_path, TestPlan, validate_request(upgrade_path, TestPlan)), + case Upgrade of + undefined -> ModBackend; + _ -> ModBackend ++ "-" ++ rt_config:convert_to_string(Upgrade) + end. %% @doc Set the value for a field in a test plan record. An error %% is returned if `TestPlan' is not a valid `rt_test_plan' record diff --git a/src/rt_util.erl b/src/rt_util.erl index a8b213934..521915c2f 100644 --- a/src/rt_util.erl +++ b/src/rt_util.erl @@ -23,6 +23,7 @@ wait_result/0]). -export([add_deps/1, + convert_to_atom_list/1, base_dir_for_version/2, ip_addr_to_string/1, maybe_append_when_not_endswith/2, @@ -40,10 +41,31 @@ wait_until_no_pending_changes/2]). -ifdef(TEST). +-include_lib("eunit/include/eunit.hrl"). -export([setup_test_env/0, test_success_fun/0]). -endif. +%% @doc Convert string or atom to list of atoms +-spec(convert_to_atom_list(atom()|string()) -> undefined | list()). +convert_to_atom_list(undefined) -> + undefined; +convert_to_atom_list(Values) when is_atom(Values) -> + ListOfValues = atom_to_list(Values), + case lists:member($, , ListOfValues) of + true -> + [list_to_atom(X) || X <- string:tokens(ListOfValues, ", ")]; + _ -> + [Values] + end; +convert_to_atom_list(Values) when is_list(Values) -> + case lists:member($, , Values) of + true -> + [list_to_atom(X) || X <- string:tokens(Values, ", ")]; + _ -> + [list_to_atom(Values)] + end. + -spec add_deps(filelib:dirname()) -> ok. add_deps(Path) -> lager:debug("Adding dep path ~p", [Path]), @@ -190,6 +212,14 @@ wait_until(Fun, Retry, Delay) when Retry > 0 -> -ifdef(TEST). +%% Properly covert backends to atoms +convert_to_atom_list_test() -> + ?assertEqual(undefined, convert_to_atom_list(undefined)), + ?assertEqual([memory], convert_to_atom_list(memory)), + ?assertEqual([memory], convert_to_atom_list("memory")), + ?assertEqual([bitcask, eleveldb, memory], lists:sort(convert_to_atom_list("memory, bitcask,eleveldb"))), + ?assertEqual([bitcask, eleveldb, memory], lists:sort(convert_to_atom_list('memory, bitcask,eleveldb'))). + -spec wait_until_no_pending_changes(rt_host:host(), [node()]) -> ok | fail. wait_until_no_pending_changes(Host, Nodes) -> lager:info("Wait until no pending changes for nodes ~p on ~p", [Nodes, Host]), diff --git a/src/rtdev.erl b/src/rtdev.erl index d673973d4..9c0735f40 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -50,7 +50,8 @@ set_advanced_conf/2, rm_dir/1, validate_config/1, - get_node_logs/2]). + get_node_logs/2, + get_node_logs/0]). -compile(export_all). -include_lib("eunit/include/eunit.hrl"). @@ -843,10 +844,16 @@ riak_repl(Node, Args) -> {ok, Result}. %% @doc Find the node number from the full name +%% Certain tests are storing short names (dev1) and some are storing integers -spec node_id(atom()) -> integer(). node_id(Node) -> NodeMap = rt_config:get(rt_nodes), - orddict:fetch(Node, NodeMap). + NodeNumber = orddict:fetch(Node, NodeMap), + case is_integer(NodeNumber) of + true -> NodeNumber; + _ -> + list_to_integer(string:right(lists:flatten(NodeNumber), 1)) + end. %% @doc Find the short dev node name from the full name -spec node_short_name(atom() | list()) -> atom(). @@ -996,6 +1003,19 @@ get_node_log_fun(DestDir, RootLen) -> {lists:nthtail(RootLen, Filename), Target} end. +%% @doc Open all of the nodes' log files to a list of {filename, port} +%% OBSOLETE +-spec(get_node_logs() -> list()). +get_node_logs() -> + Root = filename:absname(?PATH), + lager:debug("ROOT ~p", [Root]), + RootLen = length(Root) + 1, %% Remove the leading slash + [ begin + {ok, Port} = file:open(Filename, [read, binary]), + lager:debug("Opening ~p", [lists:nthtail(RootLen, Filename)]), + {lists:nthtail(RootLen, Filename), Port} + end || Filename <- filelib:wildcard(Root ++ "/*/dev*/log/*") ]. + -type node_tuple() :: {list(), atom()}. -spec extract_node_id_and_name(atom() | string()) -> node_tuple(). From 25c80f9ae7906de9ae8299f02807286a066c491b Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Wed, 20 May 2015 15:17:30 -0600 Subject: [PATCH 153/157] Update README.md --- README.md | 80 +++++++++++++++++++++++++++++++++++++++---------------- 1 file changed, 57 insertions(+), 23 deletions(-) diff --git a/README.md b/README.md index 8fc596038..e1c13c588 100644 --- a/README.md +++ b/README.md @@ -14,7 +14,7 @@ contents of `$HOME/rt/riak` might look something like this: ``` $ ls $HOME/rt/riak -riak-head riak-1.3.2 riak-1.4.12 +riak-head riak-1.4.12 riak-2.0.5 riak-2.1.1 ``` Inside each of these directories is a series `dev[0-9]+` directories, typically @@ -81,7 +81,8 @@ a bad state. This script is going to do the following: 1. Download the source for the past three major Riak versions (e.g. - 1.3.2, 1.4.12 and master) + 1.4.12, 2.0.5 and 2.1.1) +1. Any additional, test-specific versions required to run all tests (e.g. `kv679_mixed` needs 2.0.2 and 2.0.4) 1. Build the proper version of Erlang that release was built with, using **kerl** (which it will also download) 1. Build those releases of Riak. @@ -112,7 +113,7 @@ building for a little while. To use `riak_ee` instead of `riak` set [`$RT_USE_EE`](https://github.com/basho/riak_test/blob/master/bin/rtdev-all.sh#L46) to any non-empty string. -**Note**: There is a bug in 1.3.x `leveldb` which does not properly resolve +**Historical Note**: There is a bug in 1.3.x `leveldb` which does not properly resolve the location of `pthread.h` when building on Macintosh OS X 10.9, aka Mavericks, and 10.10 (Yosemite). This has been fixed in subsequent releases, but for now a fix is to manually add `#include ` to the top of @@ -137,7 +138,7 @@ so all your Riak builds are in one place. `rtdev-migrate.sh` will convert existing devrels installed in `$RT_DEST_DIR` from the legacy format to the new format. It also will reset the local -Git repo. It is only necessary to run this script once. +Git repo. It is only necessary to run this script once. WORK IN PROGRESS. ### reset-current-env.sh @@ -175,26 +176,41 @@ to tell riak_test about them. The method of choice is to create a {giddyup_host, "localhost:5000"}, {giddyup_user, "user"}, {giddyup_password, "password"}, + {giddyup_platform, "osx-64"}, {test_timeout, 1800000}, {rt_max_receive_wait_time, 600000}, {rt_retry_delay, 1000}, {rt_harness, rtdev}, {rt_scratch_dir, "/tmp/riak_test_scratch"}, {basho_bench, "/home/you/basho/basho_bench"}, - {spam_dir, "/home/you/basho/riak_test/search-corpus"}, - {platform, "osx-64"}, + {spam_dir, "/home/you/basho/riak_test/search-corpus/spam.0"}, {load_workers, 3}, + {test_paths, ["/home/you/basho/riak_test/ebin"]}, + {lager_console_level, debug}, + {lager_level, debug}, + {conn_fail_time, 60000}, {offset, 2}, {workers, 5} ]}. {rtdev, [ - {rt_project, "riak"}, - {root_path, "/home/you/rt/riak"}, - {default_version, head}, - {previous_version, "1.4.12"}, - {legacy_version, "1.3.4"} -]}. + {yz_dir, ["/home/you/basho/riak_ee/deps/yokozuna"]}, + {test_paths, ["/home/you/basho/riak_ee/yokozuna/riak_test/ebin"]}, + {root_path, "/home/you/basho/rt/riak"}, + {versions, [ + {'latest_1.4_ee', {riak_ee, "1.4.12"}}, + {'latest_2.0_ee', {riak_ee, "2.0.5"}}, + {'latest_2.1_ee', {riak_ee, "2.1.1"}}, + {default, 'latest_2.1_ee'}, + {previous, 'latest_2.0_ee'}, + {legacy, 'latest_1.4_ee'} + ]}, + {upgrade_paths, [ + {full, ['latest_1.4_ee', 'latest_2.0_ee', 'latest_2.1_ee']}, + {previous, ['latest_2.0_ee', 'latest_2.1_ee']} + {legacy, ['latest_1.4_ee', 'latest_2.1_ee']} + ]} +]} ``` The `default` section of the config file will be overridden by the config @@ -222,13 +238,13 @@ String used as a password when communicating with Giddyup. #### giddyup_user String used to identify the user when communicating with Giddyup. -#### platform +#### giddyup_platform String identifying the current testing platform when reporting to Giddyup. Current values include `centos-5-64`, `centos-6-64`, `fedora-17-64`, `freebsd-9-64`, `osx-64`, `solaris-10u9-64`, `ubuntu-1004-64`, `ubuntu-1204-64` #### spam_dir -Name of a `tar` file containing ancientSPAM e-mail used as a data load +Name of a `tar` file containing ancient SPAM e-mail used as a data load for a few tests. #### rt_harness @@ -253,34 +269,52 @@ Number of milliseconds allowed for a single test to run. #### rtdev This is the devrel configuration section. -##### rt_project -The name of the current project name; used in reporting to Giddyup. - ##### root_path Path to the top of the installed devrel instances. -##### default_version +##### default If specific versions of Riak are not specified, this one is tested. The default value is `head` which is typically the head of `develop`. -##### previous_version -Previous version of Riak EE, if not specified defaults to `1.4.12`. +##### previous +Previous version of Riak EE, if not specified defaults to `2.0.5`. Will be removed after tests have been ported. -##### legacy_version -Ancient version of Riak EE, if not specified defaults to `1.3.4`. +##### legacy +Ancient version of Riak EE, if not specified defaults to `1.4.12`. Will be removed after tests have been ported. ##### rt_default_config Default configuration parameters that will be used for nodes deployed by riak_test. Tests can override these. - ```erlang {rtdev, [ { rt_default_config, [ {riak_core, [ {ring_creation_size, 16} ]} ] } ]} ``` + +##### upgrade_paths +Definitions of lists of versions for programatic upgrades. The names `legacy` and `previous` support tests which have not yet been updated to the current framework. + +##### test_paths +List of directorys to search for tests. Currently used to help support Yokozuna tests. + +##### yz_dir +Directory to root of the built Yokozuna tree. This will go away once Yokozuna tests join the main herd under `riak`. + +##### lager_console_level and lager_level +This are [lager](https://github.com/basho/lager)-specific settings. By default they are set to `info`. + +##### load_workers +Number of concurrent processes used to load the system in the `overload` test. + +##### conn_fail_time +A magic value to override the default timeout in `replication2_ssl`. + +##### offset/workers +Values used to reproducably shuffle the order in which tests are executed. + #### Coverage You can generate a coverage report for a test run through [Erlang Cover](http://www.erlang.org/doc/apps/tools/cover_chapter.html). Coverage information for all **current** code run on any Riak node started by any of the tests in the run will be output as HTML in the coverage directory. From e76c192830968ea90be2c0a6ea3b16b4ce074903 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Wed, 20 May 2015 15:19:20 -0600 Subject: [PATCH 154/157] Update README.md --- README.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index e1c13c588..293444c18 100644 --- a/README.md +++ b/README.md @@ -307,7 +307,7 @@ Directory to root of the built Yokozuna tree. This will go away once Yokozuna t This are [lager](https://github.com/basho/lager)-specific settings. By default they are set to `info`. ##### load_workers -Number of concurrent processes used to load the system in the `overload` test. +Number of concurrent processes used to load the system in the `loaded_upgrade` test. ##### conn_fail_time A magic value to override the default timeout in `replication2_ssl`. @@ -360,6 +360,7 @@ This is an example test result JSON message posted to a webhook: "giddyup_url": "http://giddyup.basho.com/test_results/53" } ``` Notice that the giddyup URL is not the page for the test result, but a resource from which you can GET information about the test in JSON. + ### Running riak_test for the first time Run a test! After running `make` from the root of your `riak_test` From 524833bfc17e146c75e8301920d412f7dcc8452c Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Tue, 26 May 2015 11:18:45 -0600 Subject: [PATCH 155/157] Fix kv679_dataloss for new mixed framework - Changed hard-coded devrel path in test - Update relpath/1 to resolve new version paths --- src/rtdev.erl | 3 ++- tests/kv679_dataloss.erl | 1 - 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/rtdev.erl b/src/rtdev.erl index 9c0735f40..acfaea952 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -225,7 +225,8 @@ setup_harness() -> -spec relpath(Vsn :: string()) -> string(). relpath(Vsn) -> Path = ?PATH, - relpath(Vsn, Path). + ActualVersion = rt_config:version_to_tag(Vsn), + relpath(ActualVersion, Path). %% @doc Tack the version onto the end of the root path -spec relpath(Vsn :: string(), Path :: string()) -> string(). diff --git a/tests/kv679_dataloss.erl b/tests/kv679_dataloss.erl index 500314bac..5973a062a 100644 --- a/tests/kv679_dataloss.erl +++ b/tests/kv679_dataloss.erl @@ -117,7 +117,6 @@ delete_datadir({{Idx, Node}, Type}) -> DataRoot = rpc:call(Node, app_helper, get_env, [BackendName, data_root]), %% get datadir from Idx Path = filename:join([rtdev:relpath(current), - "dev", "dev"++ integer_to_list(rtdev:node_id(Node)), DataRoot, integer_to_list(Idx)]), From f53e63037106be5aa593c7b29560ee48041718c1 Mon Sep 17 00:00:00 2001 From: Brett Hazen Date: Tue, 26 May 2015 12:02:25 -0600 Subject: [PATCH 156/157] Fix deploy_nodes/1 versions for riak667_safe --- src/rtdev.erl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/rtdev.erl b/src/rtdev.erl index acfaea952..7f5822ca5 100644 --- a/src/rtdev.erl +++ b/src/rtdev.erl @@ -563,7 +563,7 @@ configure_nodes(Nodes, Configs) -> lists:zip(Nodes, Configs)). deploy_nodes(NodeConfig) -> - Path = relpath(""), + Path = rt_config:get(root_path), lager:info("Riak path: ~p", [Path]), NumNodes = length(NodeConfig), %% TODO: The starting index should not be fixed to 1 From b446979925ed5bbdebe6a4a09c30278b958a9b2a Mon Sep 17 00:00:00 2001 From: John Burwell Date: Fri, 5 Jun 2015 17:05:54 -0400 Subject: [PATCH 157/157] - WIP: Breaking change that is untested for node defintions to support cluster construction from rt_riak_cluster. --- src/rt_riak_node.erl | 102 +++++++++++++++++++++++++++++++++++++++---- 1 file changed, 94 insertions(+), 8 deletions(-) diff --git a/src/rt_riak_node.erl b/src/rt_riak_node.erl index 5c0039fc7..5b98c6a6d 100644 --- a/src/rt_riak_node.erl +++ b/src/rt_riak_node.erl @@ -35,7 +35,9 @@ copy_logs/2, get_ring/1, ip/1, + is_invalid/1, is_ready/1, + is_started/1, is_stopped/1, host/1, join/2, @@ -78,10 +80,22 @@ -define(SERVER, ?MODULE). -define(TIMEOUT, infinity). --type node_id() :: pos_integer(). --type node_type() :: {devrel, filelib:dirname()}. - -%% TODO Document the purpose of the directory_overlay +%% @doc The directory overlay describes the layout of the various +%% directories on the node. While the paths are absolute, they +%% are agnostic of a local/remote installation. The values in +%% this structure correspond to the directory locations in the +%% Riak 2.x+ riak.conf file. Use of this structure allows paths +%% to commands and files to be calculated in an installation/transport +%% neutral manner. +%% +%% For devrel-based installations, the layout will be calculated +%% relative to the root_path provided in the node_type. +%% +%% When support for package-based installations is implemented, +%% the directories will correspond to those used by the package +%% to deploy Riak on the host OS. +%% +%% @since 1.1.0 -record(directory_overlay, {bin_dir :: filelib:dirname(), conf_dir :: filelib:dirname(), data_dir :: filelib:dirname(), @@ -89,6 +103,35 @@ lib_dir :: filelib:dirname(), log_dir :: filelib:dirname()}). +%% @doc Defines the following metadata elements required to initialize +%% and control a Riak devrel node: +%% +%% * id: The devrel atom +%% * root_path: The absolute path to the root directory to +%% the available version/node sets +%% * node_id: The number of the devrel node to be managed by +%% the FSM process (e.g. 1). This number is +%% used to form the base path of node as +%% //dev +%% +%% @since 1.1.0 +-type devrel_node_type() :: [{root_path, filelib:dirname()} | + {id, devrel} | + {node_id, pos_integer()}]. + +%% @doc This record bundles the pieces required to provision a node +%% and attach orchestration to it. +%% +%% @since 1.1.0 +-record(definition, {config=[] :: proplists:proplist(), + hostname=localhost :: rt_host:hostname(), + name :: node(), + type :: devrel_node_type(), + version :: rt_util:version()}). + +-type(node_definition() :: #definition{}). +-exporttype(node_definition/0). + -record(state, {host :: rt_host:host(), id :: node_id(), directory_overlay :: #directory_overlay{}, @@ -121,7 +164,7 @@ admin(Node, Args, Options) -> %% @doc Runs `riak attach' on a specific node, and tests for the expected behavoir. %% Here's an example: ``` -%% rt_cmd_line:attach(Node, [{expect, "erlang.pipe.1 \(^D to exit\)"}, +%% rt_riak_node:attach(Node, [{expect, "erlang.pipe.1 \(^D to exit\)"}, %% {send, "riak_core_ring_manager:get_my_ring()."}, %% {expect, "dict,"}, %% {send, [4]}]), %% 4 = Ctrl + D''' @@ -131,6 +174,7 @@ admin(Node, Args, Options) -> %% `{send, String}' sends the string to the console. %% Once a send is encountered, the buffer is discarded, and the next %% expect will process based on the output following the sent data. +%% ``` %% %% @since 1.1.0 -spec attach(node(), {expect, list()} | {send, list()}) -> {ok, term()} | rt_util:error(). @@ -145,6 +189,11 @@ attach(Node, Expected) -> attach_direct(Node, Expected) -> gen_fsm:sync_send_event(Node, {attach_direct, Expected}, ?TIMEOUT). +%% @doc Kills any Riak processes running on the passed `Node', and resets the +%% the state of the FSM to `stopped'. Therefore, this function is the means +%% to reset/resync the state of a Riak node FSM with a running Riak node. +%% +%% @since 1.1.0 -spec brutal_kill(node()) -> rt_util:result(). brutal_kill(Node) -> gen_fsm:sync_send_all_state_event(Node, brutal_kill, ?TIMEOUT). @@ -172,7 +221,7 @@ clean_data_dir(Node) -> clean_data_dir(Node, SubDir) -> gen_fsm:sync_send_event(Node, {clean_data_dir, SubDir}, ?TIMEOUT). -%% @doc Runs `riak console' on a specific node +%% @doc Runs `riak console' on a the passed `Node' %% @see rt_riak_node:attach/2 %% %% @since 1.1.0 @@ -180,10 +229,16 @@ clean_data_dir(Node, SubDir) -> console(Node, Expected) -> geb_fsm:sync_send_event(Node, {console, Expected}, ?TIMEOUT). +%% @doc Commits changes to a cluster using the passed `Node' +%% +%% @since 1.1.0 -spec commit(node()) -> rt_util:result(). commit(Node) -> gen_fsm:sync_send_event(Node, commit, ?TIMEOUT). +%% @doc Retrieves the Erlang cookie current of the passed `Node' +%% +%% @since 1.1.0 -spec cookie(node()) -> atom() | rt_util:error(). cookie(Node) -> gen_fsm:sync_send_event(Node, cookie, ?TIMEOUT). @@ -202,14 +257,33 @@ copy_logs(Node, ToDir) -> get_ring(Node) -> gen_fsm:sync_send_event(Node, get_ring, ?TIMEOUT). +%% @doc Returns the IP address of the passed `Node' +%% +%% @since 1.1.0 -spec ip(node() | string()) -> string(). ip(Node) -> gen_fsm:sync_send_all_state_event(Node, ip, ?TIMEOUT). +is_invalid(Node) -> + gen_fsm:sync_send_all_state_event(Node, is_invalid, ?TIMEOUT). + +%% @doc Returns `true' if the passed node, `Node', is ready to +%% accept requests. If the node is not ready or stopped, +%5 this function returns `false'. +%% +%% @since 1.1.0 -spec is_ready(node()) -> boolean(). is_ready(Node) -> gen_fsm:sync_send_all_state_event(Node, is_ready, ?TIMEOUT). +is_started(Node) -> + gen_fsm:sync_send_all_state_event(Node, is_started, ?TIMEOUT). + +%% @doc Returns `true' if the passed node, `Node', is not running. +%% If the node is started, ready, or invalid, this function +%% returns `false'. +%% +%% @since 1.1.0 -spec is_stopped(node()) -> boolean(). is_stopped(Node) -> gen_fsm:sync_send_all_state_event(Node, is_stopped, ?TIMEOUT). @@ -310,9 +384,9 @@ start(Node, false) -> %% @doc Starts a gen_fsm process to configure, start, and %% manage a Riak node on the `Host' identified by `NodeId' %% and `NodeName' using Riak `Version' ({product, release}) --spec start_link(rt_host:hostname(), node_type(), node_id(), proplists:proplist(), rt_util:version()) -> +-spec start_link(rt_host:hostname(), node_definition(), node_id(), proplists:proplist(), rt_util:version()) -> {ok, node()} | ignore | rt_util:error(). -start_link(Hostname, NodeType, NodeId, Config, Version) -> +start_link(Hostname, NodeDefinition, NodeId, Config, Version) -> %% TODO Re-implement node naming when 1.x and 2.x configuration is propely implemented %% NodeName = list_to_atom(string:join([dev(NodeId), atom_to_list(Hostname)], "@")), NodeName = list_to_atom(string:join([dev(NodeId), "127.0.0.1"], "@")), @@ -632,10 +706,18 @@ handle_sync_event(host, _From, StateName, State=#state{host=Host}) -> {reply, Host, StateName, State}; handle_sync_event(ip, _From, StateName, State=#state{host=Host}) -> {reply, rt_host:ip_addr(Host), StateName, State}; +handle_sync_event(is_invalid, _From, invalid, State) -> + {reply, true, invalid, State}; +handle_sync_event(is_invalid, _From, StateName, State) -> + {reply, false, StateName, State}; handle_sync_event(is_ready, _From, ready, State) -> {reply, true, ready, State}; handle_sync_event(is_ready, _From, StateName, State) -> {reply, false, StateName, State}; +handle_sync_event(is_started, _From, started, State) -> + {reply, true, started, State}; +handle_sync_event(is_started, _From, StateName, State) -> + {reply, false, StateName, State}; handle_sync_event(is_stopped, _From, stopped, State) -> {reply, true, stopped, State}; handle_sync_event(is_stopped, _From, StateName, State) -> @@ -1171,7 +1253,9 @@ verify_sync_start(Node) -> ?debugFmt("Started node synchronously ~p with result ~p", [Node, Result]), ?assertEqual(ok, Result), + ?assertEqual(false, is_invalid(Node)), ?assertEqual(false, is_stopped(Node)), + ?assertEqual(false, is_started(Node)), ?assertEqual(true, is_ready(Node)), Result. @@ -1182,6 +1266,8 @@ verify_sync_stop(Node) -> ?assertEqual(ok, Result), ?assertEqual(true, is_stopped(Node)), ?assertEqual(false, is_ready(Node)), + ?assertEqual(false, is_invalid(Node)), + ?assertEqual(false, is_started(Node)), Result. async_start_test_() ->