Verify the list of HTTP stats keys is complete

* Checks the list of stats keys returned from the HTTP endpoint
    is complete -- delineating between riak and riak_ee.  The test will
    fail if the list returned from the HTTP endpoint does not exactly match
    the expected list.  This behavior acts as a forcing function to ensure
    that the expected list is properly maintained as stats are added and
    removed.
    * Modifies reset-current-env to properly clean dependencies when a
    full clean is requested and remove the current directory in the
    target test instance.
    * Adds logging to verify_riak_stats to explain the addition steps
    being performed
    * Adds rt:product/1 to determine whether a node is running riak,
    riak_ee, or riak_cs
    * Adds tools.mk support and eunit scaffolding to rebar.config
    * Modifies reset-current-env.sh to remove the current directory in
    the target test instance
This commit is contained in:
John Burwell 2014-12-19 18:04:21 -05:00
parent f98b3aed87
commit a37832d7f9
8 changed files with 653 additions and 35 deletions

4
.gitignore vendored
View File

@ -18,3 +18,7 @@ tags
riak-*
*.png
.rebar/
.local_dialyzer_plt
dialyzer_unhandled_warnings
dialyzer_warnings

View File

@ -12,9 +12,6 @@ all: deps compile
deps:
./rebar get-deps
docs:
./rebar skip_deps=true doc
docsclean:
@rm -rf doc/*.png doc/*.html doc/*.css edoc-info
@ -35,26 +32,9 @@ quickbuild:
# Dialyzer targets
##################
# public targets
dialyzer: compile $(PLT)
dialyzer -Wno_return -Wunderspecs -Wunmatched_returns --plt $(PLT) ebin deps/*/ebin | \
egrep -v -f ./dialyzer.ignore-warnings
# Legacy target left for compatibility with any existing automation
# scripts ...
clean_plt:
@echo
@echo "Are you sure? It takes about 1/2 hour to re-build."
@echo Deleting $(PLT) in 5 seconds.
@echo
sleep 5
rm $(PLT)
cleanplt
# internal targets
# build plt file. assumes 'compile' was already run, e.g. from 'dialyzer' target
$(PLT):
@echo
@echo "Building dialyzer's plt file. This can take 1/2 hour."
@echo " Because it wasn't here:" $(PLT)
@echo " Consider using R15B03 or later for 100x faster build time!"
@echo
@sleep 1
dialyzer --build_plt --output_plt $(PLT) --apps $(APPS)
include tools.mk

View File

@ -56,16 +56,18 @@ 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"
cd $RIAK_HOME
rm -rf current
# Clean out previous devrel build ...
# Clean out deps for when a full clean is requested ...
if [ "$FULL_CLEAN" = true ] ; then
echo "Cleaning Riak in $RIAK_HOME ..."
make devclean
else
echo "Removing previous stagedevrel instance from $RIAK_HOME and rebuilding ..."
rm -rf dev
make distclean
fi
echo "Removing previous stagedevrel instance from $RIAK_HOME and rebuilding ..."
make devclean
# Rebuild Riak ...
echo "Building Riak stagedevrel with $NUM_NODES nodes in $RIAK_HOME ..."
make stagedevrel DEVNODES=$NUM_NODES
@ -76,7 +78,7 @@ cd $RT_HOME
if [ "$FULL_CLEAN" = true ] ; then
echo "Cleaning riak_test in $RT_HOME ..."
make clean
make distclean
fi
echo "Rebuilding riak_test in $RT_HOME ..."

View File

@ -8,6 +8,8 @@
warnings_as_errors, {parse_transform, lager_transform}]}.
{erl_first_files, ["src/rt_intercept_pt.erl"]}.
{eunit_opts, [verbose]}.
{deps, [
{lager, ".*", {git, "git://github.com/basho/lager", {tag, "2.0.3"}}},
{getopt, ".*", {git, "git://github.com/jcomellas/getopt", {tag, "v0.4"}}},

1
riak Symbolic link
View File

@ -0,0 +1 @@
riak-2.0

View File

@ -98,6 +98,7 @@
pbc_really_deleted/3,
pmap/2,
post_result/2,
product/1,
priv_dir/0,
remove/2,
riak/2,
@ -1099,6 +1100,21 @@ join_cluster(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);
@ -1862,3 +1878,26 @@ is_control_gui_route_loaded(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].
-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.

View File

@ -28,17 +28,28 @@
-define(TYPES, [{?CTYPE, counter},
{?STYPE, set},
{?MTYPE, map}]).
-define(CONF, [
{yokozuna,
[{enabled, true}]
}]).
%% 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:wait_until_nodes_ready([Node1])),
Stats1 = get_stats(Node1),
lager:info("Verifying that all expected stats keys are present from the HTTP endpoint"),
ok = verify_stats_keys_complete(Node1, Stats1),
AdminStats1 = get_console_stats(Node1),
compare_http_and_console_stats(Stats1, AdminStats1),
lager:info("Verifying that the stats keys in riak-admin status and HTTP match"),
ok = compare_http_and_console_stats(Stats1, AdminStats1),
%% make sure a set of stats have valid values
lager:info("Verifying that the system and ring stats have valid values"),
verify_nz(Stats1,[<<"cpu_nprocs">>,
<<"mem_total">>,
<<"mem_allocated">>,
@ -58,6 +69,7 @@ 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"),
@ -143,7 +155,10 @@ verify_nz(Props, Keys) ->
get_stats(Node) ->
timer:sleep(10000),
StatString = os:cmd(io_lib:format("curl -s -S ~s/stats", [rt:http_url(Node)])),
lager:info("Retrieving stats from node ~s", [Node]),
StatsCommand = io_lib:format("curl -s -S ~s/stats", [rt:http_url(Node)]),
lager:debug("Retrieving stats using command ~s", [StatsCommand]),
StatString = os:cmd(StatsCommand),
{struct, Stats} = mochijson2:decode(StatString),
%%lager:debug(StatString),
Stats.
@ -180,12 +195,37 @@ compare_http_and_console_stats(Stats1, Stats2) ->
not lists:keymember(K, 1, Stats2)],
OnlyInAdmin = [S || {K,_} = S <- Stats2,
not lists:keymember(K, 1, Stats1)],
lager:info("OnlyInHttp = ~p~n"
"OnlyInAdmin = ~p~n", [OnlyInHttp, OnlyInAdmin]),
maybe_log_stats_keys(OnlyInHttp, "Keys missing from riak-admin"),
maybe_log_stats_keys(OnlyInAdmin, "Keys missing from HTTP"),
?assertEqual([], OnlyInHttp),
?assertEqual([], OnlyInAdmin),
ok.
verify_stats_keys_complete(Node, Stats) ->
ActualKeys = proplists:get_keys(Stats),
ExpectedKeys = all_stats(Node),
MissingStatsKeys = diff_lists(ActualKeys, ExpectedKeys),
AdditionalStatsKeys = diff_lists(ExpectedKeys, ActualKeys),
maybe_log_stats_keys(MissingStatsKeys, "missing stats keys"),
maybe_log_stats_keys(AdditionalStatsKeys, "additional stats"),
?assert(length(MissingStatsKeys) == 0),
?assert(length(AdditionalStatsKeys) == 0),
ok.
diff_lists(List, ThatList) ->
lists:filter(fun(Element) -> not lists:member(Element, List) end, ThatList).
-spec maybe_log_stats_keys([binary()], string()) -> ok.
maybe_log_stats_keys(StatsKeys, _Description) when length(StatsKeys) == 0 ->
ok;
maybe_log_stats_keys(StatsKeys, Description) when StatsKeys ->
lager:info("~s: ~s", [Description, pretty_print_stats_keys(StatsKeys)]).
-spec pretty_print_stats_keys([binary()]) -> string().
pretty_print_stats_keys(StatsKeys) ->
ConvertedStatsKeys = lists:map(fun(StatsKey) -> binary_to_list(StatsKey) end, StatsKeys),
string:join(ConvertedStatsKeys, ", ").
datatype_stats() ->
%% Merge stats are excluded because we likely never merge disjoint
%% copies on a single node after a single write each.
@ -358,3 +398,404 @@ get_and_update(Pid, map) ->
_ = [ riakc_pb_socket:fetch_type(Pid, {?MTYPE, <<"pb">>}, <<I>>)
|| I <- lists:seq(1, 10) ].
all_stats(Node) ->
common_stats() ++ product_stats(rt:product(Node)).
common_stats() ->
[
<<"asn1_version">>,
<<"basho_stats_version">>,
<<"bitcask_version">>,
<<"clique_version">>,
<<"cluster_info_version">>,
<<"compiler_version">>,
<<"connected_nodes">>,
<<"consistent_get_objsize_100">>,
<<"consistent_get_objsize_95">>,
<<"consistent_get_objsize_99">>,
<<"consistent_get_objsize_mean">>,
<<"consistent_get_objsize_median">>,
<<"consistent_get_time_100">>,
<<"consistent_get_time_95">>,
<<"consistent_get_time_99">>,
<<"consistent_get_time_mean">>,
<<"consistent_get_time_median">>,
<<"consistent_gets">>,
<<"consistent_gets_total">>,
<<"consistent_put_objsize_100">>,
<<"consistent_put_objsize_95">>,
<<"consistent_put_objsize_99">>,
<<"consistent_put_objsize_mean">>,
<<"consistent_put_objsize_median">>,
<<"consistent_put_time_100">>,
<<"consistent_put_time_95">>,
<<"consistent_put_time_99">>,
<<"consistent_put_time_mean">>,
<<"consistent_put_time_median">>,
<<"consistent_puts">>,
<<"consistent_puts_total">>,
<<"converge_delay_last">>,
<<"converge_delay_max">>,
<<"converge_delay_mean">>,
<<"converge_delay_min">>,
<<"coord_redirs_total">>,
<<"counter_actor_counts_100">>,
<<"counter_actor_counts_95">>,
<<"counter_actor_counts_99">>,
<<"counter_actor_counts_mean">>,
<<"counter_actor_counts_median">>,
<<"cpu_avg1">>,
<<"cpu_avg15">>,
<<"cpu_avg5">>,
<<"cpu_nprocs">>,
<<"crypto_version">>,
<<"disk">>,
<<"dropped_vnode_requests_total">>,
<<"eleveldb_version">>,
<<"erlang_js_version">>,
<<"erlydtl_version">>,
<<"executing_mappers">>,
<<"exometer_core_version">>,
<<"goldrush_version">>,
<<"gossip_received">>,
<<"handoff_timeouts">>,
<<"ibrowse_version">>,
<<"ignored_gossip_total">>,
<<"index_fsm_active">>,
<<"index_fsm_create">>,
<<"index_fsm_create_error">>,
<<"inets_version">>,
<<"kernel_version">>,
<<"lager_version">>,
<<"late_put_fsm_coordinator_ack">>,
<<"leveldb_read_block_error">>,
<<"list_fsm_active">>,
<<"list_fsm_create">>,
<<"list_fsm_create_error">>,
<<"list_fsm_create_error_total">>,
<<"list_fsm_create_total">>,
<<"map_actor_counts_100">>,
<<"map_actor_counts_95">>,
<<"map_actor_counts_99">>,
<<"map_actor_counts_mean">>,
<<"map_actor_counts_median">>,
<<"mem_allocated">>,
<<"mem_total">>,
<<"memory_atom">>,
<<"memory_atom_used">>,
<<"memory_binary">>,
<<"memory_code">>,
<<"memory_ets">>,
<<"memory_processes">>,
<<"memory_processes_used">>,
<<"memory_system">>,
<<"memory_total">>,
<<"merge_index_version">>,
<<"mochiweb_version">>,
<<"node_get_fsm_active">>,
<<"node_get_fsm_active_60s">>,
<<"node_get_fsm_counter_objsize_100">>,
<<"node_get_fsm_counter_objsize_95">>,
<<"node_get_fsm_counter_objsize_99">>,
<<"node_get_fsm_counter_objsize_mean">>,
<<"node_get_fsm_counter_objsize_median">>,
<<"node_get_fsm_counter_siblings_100">>,
<<"node_get_fsm_counter_siblings_95">>,
<<"node_get_fsm_counter_siblings_99">>,
<<"node_get_fsm_counter_siblings_mean">>,
<<"node_get_fsm_counter_siblings_median">>,
<<"node_get_fsm_counter_time_100">>,
<<"node_get_fsm_counter_time_95">>,
<<"node_get_fsm_counter_time_99">>,
<<"node_get_fsm_counter_time_mean">>,
<<"node_get_fsm_counter_time_median">>,
<<"node_get_fsm_errors">>,
<<"node_get_fsm_errors_total">>,
<<"node_get_fsm_in_rate">>,
<<"node_get_fsm_map_objsize_100">>,
<<"node_get_fsm_map_objsize_95">>,
<<"node_get_fsm_map_objsize_99">>,
<<"node_get_fsm_map_objsize_mean">>,
<<"node_get_fsm_map_objsize_median">>,
<<"node_get_fsm_map_siblings_100">>,
<<"node_get_fsm_map_siblings_95">>,
<<"node_get_fsm_map_siblings_99">>,
<<"node_get_fsm_map_siblings_mean">>,
<<"node_get_fsm_map_siblings_median">>,
<<"node_get_fsm_map_time_100">>,
<<"node_get_fsm_map_time_95">>,
<<"node_get_fsm_map_time_99">>,
<<"node_get_fsm_map_time_mean">>,
<<"node_get_fsm_map_time_median">>,
<<"node_get_fsm_objsize_100">>,
<<"node_get_fsm_objsize_95">>,
<<"node_get_fsm_objsize_99">>,
<<"node_get_fsm_objsize_mean">>,
<<"node_get_fsm_objsize_median">>,
<<"node_get_fsm_out_rate">>,
<<"node_get_fsm_rejected">>,
<<"node_get_fsm_rejected_60s">>,
<<"node_get_fsm_rejected_total">>,
<<"node_get_fsm_set_objsize_100">>,
<<"node_get_fsm_set_objsize_95">>,
<<"node_get_fsm_set_objsize_99">>,
<<"node_get_fsm_set_objsize_mean">>,
<<"node_get_fsm_set_objsize_median">>,
<<"node_get_fsm_set_siblings_100">>,
<<"node_get_fsm_set_siblings_95">>,
<<"node_get_fsm_set_siblings_99">>,
<<"node_get_fsm_set_siblings_mean">>,
<<"node_get_fsm_set_siblings_median">>,
<<"node_get_fsm_set_time_100">>,
<<"node_get_fsm_set_time_95">>,
<<"node_get_fsm_set_time_99">>,
<<"node_get_fsm_set_time_mean">>,
<<"node_get_fsm_set_time_median">>,
<<"node_get_fsm_siblings_100">>,
<<"node_get_fsm_siblings_95">>,
<<"node_get_fsm_siblings_99">>,
<<"node_get_fsm_siblings_mean">>,
<<"node_get_fsm_siblings_median">>,
<<"node_get_fsm_time_100">>,
<<"node_get_fsm_time_95">>,
<<"node_get_fsm_time_99">>,
<<"node_get_fsm_time_mean">>,
<<"node_get_fsm_time_median">>,
<<"node_gets">>,
<<"node_gets_counter">>,
<<"node_gets_counter_total">>,
<<"node_gets_map">>,
<<"node_gets_map_total">>,
<<"node_gets_set">>,
<<"node_gets_set_total">>,
<<"node_gets_total">>,
<<"node_put_fsm_active">>,
<<"node_put_fsm_active_60s">>,
<<"node_put_fsm_counter_time_100">>,
<<"node_put_fsm_counter_time_95">>,
<<"node_put_fsm_counter_time_99">>,
<<"node_put_fsm_counter_time_mean">>,
<<"node_put_fsm_counter_time_median">>,
<<"node_put_fsm_in_rate">>,
<<"node_put_fsm_map_time_100">>,
<<"node_put_fsm_map_time_95">>,
<<"node_put_fsm_map_time_99">>,
<<"node_put_fsm_map_time_mean">>,
<<"node_put_fsm_map_time_median">>,
<<"node_put_fsm_out_rate">>,
<<"node_put_fsm_rejected">>,
<<"node_put_fsm_rejected_60s">>,
<<"node_put_fsm_rejected_total">>,
<<"node_put_fsm_set_time_100">>,
<<"node_put_fsm_set_time_95">>,
<<"node_put_fsm_set_time_99">>,
<<"node_put_fsm_set_time_mean">>,
<<"node_put_fsm_set_time_median">>,
<<"node_put_fsm_time_100">>,
<<"node_put_fsm_time_95">>,
<<"node_put_fsm_time_99">>,
<<"node_put_fsm_time_mean">>,
<<"node_put_fsm_time_median">>,
<<"node_puts">>,
<<"node_puts_counter">>,
<<"node_puts_counter_total">>,
<<"node_puts_map">>,
<<"node_puts_map_total">>,
<<"node_puts_set">>,
<<"node_puts_set_total">>,
<<"node_puts_total">>,
<<"nodename">>,
<<"object_counter_merge">>,
<<"object_counter_merge_time_100">>,
<<"object_counter_merge_time_95">>,
<<"object_counter_merge_time_99">>,
<<"object_counter_merge_time_mean">>,
<<"object_counter_merge_time_median">>,
<<"object_counter_merge_total">>,
<<"object_map_merge">>,
<<"object_map_merge_time_100">>,
<<"object_map_merge_time_95">>,
<<"object_map_merge_time_99">>,
<<"object_map_merge_time_mean">>,
<<"object_map_merge_time_median">>,
<<"object_map_merge_total">>,
<<"object_merge">>,
<<"object_merge_time_100">>,
<<"object_merge_time_95">>,
<<"object_merge_time_99">>,
<<"object_merge_time_mean">>,
<<"object_merge_time_median">>,
<<"object_merge_total">>,
<<"object_set_merge">>,
<<"object_set_merge_time_100">>,
<<"object_set_merge_time_95">>,
<<"object_set_merge_time_99">>,
<<"object_set_merge_time_mean">>,
<<"object_set_merge_time_median">>,
<<"object_set_merge_total">>,
<<"os_mon_version">>,
<<"pbc_active">>,
<<"pbc_connects">>,
<<"pbc_connects_total">>,
<<"pbkdf2_version">>,
<<"pipeline_active">>,
<<"pipeline_create_count">>,
<<"pipeline_create_error_count">>,
<<"pipeline_create_error_one">>,
<<"pipeline_create_one">>,
<<"poolboy_version">>,
<<"postcommit_fail">>,
<<"precommit_fail">>,
<<"protobuffs_version">>,
<<"public_key_version">>,
<<"read_repairs">>,
<<"read_repairs_counter">>,
<<"read_repairs_counter_total">>,
<<"read_repairs_fallback_notfound_one">>,
<<"read_repairs_fallback_notfound_count">>,
<<"read_repairs_fallback_outofdate_one">>,
<<"read_repairs_fallback_outofdate_count">>,
<<"read_repairs_map">>,
<<"read_repairs_map_total">>,
<<"read_repairs_primary_notfound_count">>,
<<"read_repairs_primary_notfound_one">>,
<<"read_repairs_primary_outofdate_one">>,
<<"read_repairs_primary_outofdate_count">>,
<<"read_repairs_set">>,
<<"read_repairs_set_total">>,
<<"read_repairs_total">>,
<<"rebalance_delay_last">>,
<<"rebalance_delay_max">>,
<<"rebalance_delay_mean">>,
<<"rebalance_delay_min">>,
<<"rejected_handoffs">>,
<<"riak_api_version">>,
<<"riak_auth_mods_version">>,
<<"riak_control_version">>,
<<"riak_core_version">>,
<<"riak_dt_version">>,
<<"riak_kv_version">>,
<<"riak_kv_vnodeq_max">>,
<<"riak_kv_vnodeq_mean">>,
<<"riak_kv_vnodeq_median">>,
<<"riak_kv_vnodeq_min">>,
<<"riak_kv_vnodeq_total">>,
<<"riak_kv_vnodes_running">>,
<<"riak_pb_version">>,
<<"riak_pipe_version">>,
<<"riak_pipe_vnodeq_max">>,
<<"riak_pipe_vnodeq_mean">>,
<<"riak_pipe_vnodeq_median">>,
<<"riak_pipe_vnodeq_min">>,
<<"riak_pipe_vnodeq_total">>,
<<"riak_pipe_vnodes_running">>,
<<"riak_search_version">>,
<<"riak_sysmon_version">>,
<<"ring_creation_size">>,
<<"ring_members">>,
<<"ring_num_partitions">>,
<<"ring_ownership">>,
<<"rings_reconciled">>,
<<"rings_reconciled_total">>,
<<"runtime_tools_version">>,
<<"sasl_version">>,
<<"search_index_fail_count">>,
<<"search_index_fail_one">>,
<<"search_index_latency_95">>,
<<"search_index_latency_99">>,
<<"search_index_latency_999">>,
<<"search_index_latency_max">>,
<<"search_index_latency_median">>,
<<"search_index_latency_min">>,
<<"search_index_throughput_count">>,
<<"search_index_throughtput_one">>,
<<"search_query_fail_count">>,
<<"search_query_fail_one">>,
<<"search_query_latency_95">>,
<<"search_query_latency_99">>,
<<"search_query_latency_999">>,
<<"search_query_latency_max">>,
<<"search_query_latency_median">>,
<<"search_query_latency_min">>,
<<"search_query_throughput_count">>,
<<"search_query_throughput_one">>,
<<"set_actor_counts_100">>,
<<"set_actor_counts_95">>,
<<"set_actor_counts_99">>,
<<"set_actor_counts_mean">>,
<<"set_actor_counts_median">>,
<<"sidejob_version">>,
<<"skipped_read_repairs">>,
<<"skipped_read_repairs_total">>,
<<"ssl_version">>,
<<"stdlib_version">>,
<<"storage_backend">>,
<<"syntax_tools_version">>,
<<"sys_driver_version">>,
<<"sys_global_heaps_size">>,
<<"sys_heap_type">>,
<<"sys_logical_processors">>,
<<"sys_monitor_count">>,
<<"sys_otp_release">>,
<<"sys_port_count">>,
<<"sys_process_count">>,
<<"sys_smp_support">>,
<<"sys_system_architecture">>,
<<"sys_system_version">>,
<<"sys_thread_pool_size">>,
<<"sys_threads_enabled">>,
<<"sys_wordsize">>,
<<"vnode_counter_update">>,
<<"vnode_counter_update_time_100">>,
<<"vnode_counter_update_time_95">>,
<<"vnode_counter_update_time_99">>,
<<"vnode_counter_update_time_mean">>,
<<"vnode_counter_update_time_median">>,
<<"vnode_counter_update_total">>,
<<"vnode_gets">>,
<<"vnode_gets_total">>,
<<"vnode_index_deletes">>,
<<"vnode_index_deletes_postings">>,
<<"vnode_index_deletes_postings_total">>,
<<"vnode_index_deletes_total">>,
<<"vnode_index_reads">>,
<<"vnode_index_reads_total">>,
<<"vnode_index_refreshes">>,
<<"vnode_index_refreshes_total">>,
<<"vnode_index_writes">>,
<<"vnode_index_writes_postings">>,
<<"vnode_index_writes_postings_total">>,
<<"vnode_index_writes_total">>,
<<"vnode_map_update">>,
<<"vnode_map_update_time_100">>,
<<"vnode_map_update_time_95">>,
<<"vnode_map_update_time_99">>,
<<"vnode_map_update_time_mean">>,
<<"vnode_map_update_time_median">>,
<<"vnode_map_update_total">>,
<<"vnode_puts">>,
<<"vnode_puts_total">>,
<<"vnode_set_update">>,
<<"vnode_set_update_time_100">>,
<<"vnode_set_update_time_95">>,
<<"vnode_set_update_time_99">>,
<<"vnode_set_update_time_mean">>,
<<"vnode_set_update_time_median">>,
<<"vnode_set_update_total">>,
<<"webmachine_version">>,
<<"yokozuna_version">>
].
product_stats(riak_ee) ->
[
<<"riak_jmx_version">>,
<<"ebloom_version">>,
<<"riak_snmp_version">>,
<<"riak_repl_version">>,
<<"snmp_version">>,
<<"ranch_version">>,
<<"mnesia_version">>
];
product_stats(riak) ->
[].

149
tools.mk Normal file
View File

@ -0,0 +1,149 @@
# -------------------------------------------------------------------
#
# 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.
#
# -------------------------------------------------------------------
# -------------------------------------------------------------------
# NOTE: This file is is from https://github.com/basho/tools.mk.
# It should not be edited in a project. It should simply be updated
# wholesale when a new version of tools.mk is released.
# -------------------------------------------------------------------
REBAR ?= ./rebar
REVISION ?= $(shell git rev-parse --short HEAD)
PROJECT ?= $(shell basename `find src -name "*.app.src"` .app.src)
.PHONY: compile-no-deps test docs xref dialyzer-run dialyzer-quick dialyzer \
cleanplt upload-docs
compile-no-deps:
${REBAR} compile skip_deps=true
test: compile
${REBAR} eunit skip_deps=true
upload-docs: docs
@if [ -z "${BUCKET}" -o -z "${PROJECT}" -o -z "${REVISION}" ]; then \
echo "Set BUCKET, PROJECT, and REVISION env vars to upload docs"; \
exit 1; fi
@cd doc; s3cmd put -P * "s3://${BUCKET}/${PROJECT}/${REVISION}/" > /dev/null
@echo "Docs built at: http://${BUCKET}.s3-website-us-east-1.amazonaws.com/${PROJECT}/${REVISION}"
docs:
${REBAR} doc skip_deps=true
xref: compile
${REBAR} xref skip_deps=true
PLT ?= $(HOME)/.combo_dialyzer_plt
LOCAL_PLT = .local_dialyzer_plt
DIALYZER_FLAGS ?= -Wunmatched_returns
${PLT}: compile
@if [ -f $(PLT) ]; then \
dialyzer --check_plt --plt $(PLT) --apps $(DIALYZER_APPS) && \
dialyzer --add_to_plt --plt $(PLT) --output_plt $(PLT) --apps $(DIALYZER_APPS) ; test $$? -ne 1; \
else \
dialyzer --build_plt --output_plt $(PLT) --apps $(DIALYZER_APPS); test $$? -ne 1; \
fi
${LOCAL_PLT}: compile
@if [ -d deps ]; then \
if [ -f $(LOCAL_PLT) ]; then \
dialyzer --check_plt --plt $(LOCAL_PLT) deps/*/ebin && \
dialyzer --add_to_plt --plt $(LOCAL_PLT) --output_plt $(LOCAL_PLT) deps/*/ebin ; test $$? -ne 1; \
else \
dialyzer --build_plt --output_plt $(LOCAL_PLT) deps/*/ebin ; test $$? -ne 1; \
fi \
fi
dialyzer-run:
@echo "==> $(shell basename $(shell pwd)) (dialyzer)"
# The bulk of the code below deals with the dialyzer.ignore-warnings file
# which contains strings to ignore if output by dialyzer.
# Typically the strings include line numbers. Using them exactly is hard
# to maintain as the code changes. This approach instead ignores the line
# numbers, but takes into account the number of times a string is listed
# for a given file. So if one string is listed once, for example, and it
# appears twice in the warnings, the user is alerted. It is possible but
# unlikely that this approach could mask a warning if one ignored warning
# is removed and two warnings of the same kind appear in the file, for
# example. But it is a trade-off that seems worth it.
# Details of the cryptic commands:
# - Remove line numbers from dialyzer.ignore-warnings
# - Pre-pend duplicate count to each warning with sort | uniq -c
# - Remove annoying white space around duplicate count
# - Save in dialyer.ignore-warnings.tmp
# - Do the same to dialyzer_warnings
# - Remove matches from dialyzer.ignore-warnings.tmp from output
# - Remove duplicate count
# - Escape regex special chars to use lines as regex patterns
# - Add pattern to match any line number (file.erl:\d+:)
# - Anchor to match the entire line (^entire line$)
# - Save in dialyzer_unhandled_warnings
# - Output matches for those patterns found in the original warnings
@if [ -f $(LOCAL_PLT) ]; then \
PLTS="$(PLT) $(LOCAL_PLT)"; \
else \
PLTS=$(PLT); \
fi; \
if [ -f dialyzer.ignore-warnings ]; then \
if [ $$(grep -cvE '[^[:space:]]' dialyzer.ignore-warnings) -ne 0 ]; then \
echo "ERROR: dialyzer.ignore-warnings contains a blank/empty line, this will match all messages!"; \
exit 1; \
fi; \
dialyzer $(DIALYZER_FLAGS) --plts $${PLTS} -c ebin > dialyzer_warnings ; \
cat dialyzer.ignore-warnings \
| sed -E 's/^([^:]+:)[^:]+:/\1/' \
| sort \
| uniq -c \
| sed -E '/.*\.erl: /!s/^[[:space:]]*[0-9]+[[:space:]]*//' \
> dialyzer.ignore-warnings.tmp ; \
egrep -v "^[[:space:]]*(done|Checking|Proceeding|Compiling)" dialyzer_warnings \
| sed -E 's/^([^:]+:)[^:]+:/\1/' \
| sort \
| uniq -c \
| sed -E '/.*\.erl: /!s/^[[:space:]]*[0-9]+[[:space:]]*//' \
| grep -F -f dialyzer.ignore-warnings.tmp -v \
| sed -E 's/^[[:space:]]*[0-9]+[[:space:]]*//' \
| sed -E 's/([]\^:+?|()*.$${}\[])/\\\1/g' \
| sed -E 's/(\\\.erl\\\:)/\1\\d+:/g' \
| sed -E 's/^(.*)$$/^[[:space:]]*\1$$/g' \
> dialyzer_unhandled_warnings ; \
rm dialyzer.ignore-warnings.tmp; \
if [ $$(cat dialyzer_unhandled_warnings | wc -l) -gt 0 ]; then \
egrep -f dialyzer_unhandled_warnings dialyzer_warnings ; \
found_warnings=1; \
fi; \
[ "$$found_warnings" != 1 ] ; \
else \
dialyzer $(DIALYZER_FLAGS) --plts $${PLTS} -c ebin; \
fi
dialyzer-quick: compile-no-deps dialyzer-run
dialyzer: ${PLT} ${LOCAL_PLT} dialyzer-run
cleanplt:
@echo
@echo "Are you sure? It takes several minutes to re-build."
@echo Deleting $(PLT) and $(LOCAL_PLT) in 5 seconds.
@echo
sleep 5
rm $(PLT)
rm $(LOCAL_PLT)