summaryrefslogtreecommitdiff
path: root/src
diff options
context:
space:
mode:
Diffstat (limited to 'src')
-rw-r--r--src/couchdb/Makefile.am2
-rw-r--r--src/couchdb/couch_db.hrl1
-rw-r--r--src/couchdb/couch_httpd_stats_handlers.erl67
-rw-r--r--src/couchdb/couch_stats.hrl22
-rw-r--r--src/couchdb/couch_stats_aggregator.erl582
-rw-r--r--src/couchdb/couch_stats_collector.erl143
6 files changed, 343 insertions, 474 deletions
diff --git a/src/couchdb/Makefile.am b/src/couchdb/Makefile.am
index 97d8a3b0..34aaf2c7 100644
--- a/src/couchdb/Makefile.am
+++ b/src/couchdb/Makefile.am
@@ -109,7 +109,7 @@ source_files = \
couch_db_updater.erl \
couch_work_queue.erl
-EXTRA_DIST = $(source_files) couch_db.hrl couch_stats.hrl
+EXTRA_DIST = $(source_files) couch_db.hrl
compiled_files = \
couch.app \
diff --git a/src/couchdb/couch_db.hrl b/src/couchdb/couch_db.hrl
index 053cbaa4..bd297b2a 100644
--- a/src/couchdb/couch_db.hrl
+++ b/src/couchdb/couch_db.hrl
@@ -20,6 +20,7 @@
-define(JSON_ENCODE(V), mochijson2:encode(V)).
-define(JSON_DECODE(V), mochijson2:decode(V)).
+-define(b2a(V), list_to_atom(binary_to_list(V))).
-define(b2l(V), binary_to_list(V)).
-define(l2b(V), list_to_binary(V)).
diff --git a/src/couchdb/couch_httpd_stats_handlers.erl b/src/couchdb/couch_httpd_stats_handlers.erl
index 81ff2621..40444bf8 100644
--- a/src/couchdb/couch_httpd_stats_handlers.erl
+++ b/src/couchdb/couch_httpd_stats_handlers.erl
@@ -12,51 +12,44 @@
-module(couch_httpd_stats_handlers).
-include("couch_db.hrl").
--include("couch_stats.hrl").
-export([handle_stats_req/1]).
--import(couch_httpd,
- [send_json/2,send_json/3,send_json/4,send_method_not_allowed/2,
- start_json_response/2,send_chunk/2,end_json_response/1,
- start_chunked_response/3, send_error/4]).
-
--define(b2a(V), list_to_atom(binary_to_list(V))).
-
--record(stats_query_args, {
- range='0',
- flush=false
-}).
+-import(couch_httpd, [
+ send_json/2, send_json/3, send_json/4, send_method_not_allowed/2,
+ start_json_response/2, send_chunk/2, end_json_response/1,
+ start_chunked_response/3, send_error/4
+]).
handle_stats_req(#httpd{method='GET', path_parts=[_]}=Req) ->
- send_json(Req, couch_stats_aggregator:all());
+ flush(Req),
+ send_json(Req, couch_stats_aggregator:all(range(Req)));
-handle_stats_req(#httpd{method='GET', path_parts=[_Stats, Module, Key]}=Req) ->
- #stats_query_args{
- range=Range,
- flush=Flush
- } = parse_stats_query(Req),
+handle_stats_req(#httpd{method='GET', path_parts=[_, _Mod]}) ->
+ throw({bad_request, <<"Stat names must have exactly to parts.">>});
- case Flush of
- true ->
- couch_stats_aggregator:time_passed();
- _ -> ok
- end,
+handle_stats_req(#httpd{method='GET', path_parts=[_, Mod, Key]}=Req) ->
+ flush(Req),
+ Stats = couch_stats_aggregator:get_json({?b2a(Mod), ?b2a(Key)}, range(Req)),
+ send_json(Req, {[{Mod, {[{Key, Stats}]}}]});
- Stats = couch_stats_aggregator:get_json({?b2a(Module), ?b2a(Key)}, Range),
- Response = {[{Module, {[{Key, Stats}]}}]},
- send_json(Req, Response);
+handle_stats_req(#httpd{method='GET', path_parts=[_, _Mod, _Key | _Extra]}) ->
+ throw({bad_request, <<"Stat names must have exactly two parts.">>});
handle_stats_req(Req) ->
send_method_not_allowed(Req, "GET").
-parse_stats_query(Req) ->
- lists:foldl(fun({Key,Value}, Args) ->
- case {Key, Value} of
- {"range", Range} ->
- Args#stats_query_args{range=list_to_atom(Range)};
- {"flush", "true"} ->
- Args#stats_query_args{flush=true};
- _Else -> % unknown key value pair, ignore.
- Args
- end
- end, #stats_query_args{}, couch_httpd:qs(Req)).
+range(Req) ->
+ case proplists:get_value("range", couch_httpd:qs(Req)) of
+ undefined ->
+ 0;
+ Value ->
+ list_to_integer(Value)
+ end.
+
+flush(Req) ->
+ case proplists:get_value("flush", couch_httpd:qs(Req)) of
+ "true" ->
+ couch_stats_aggregator:collect_sample();
+ _Else ->
+ ok
+ end.
diff --git a/src/couchdb/couch_stats.hrl b/src/couchdb/couch_stats.hrl
deleted file mode 100644
index 41d7fd0a..00000000
--- a/src/couchdb/couch_stats.hrl
+++ /dev/null
@@ -1,22 +0,0 @@
-% Licensed 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(aggregates, {
- min = 0,
- max = 0,
- mean = 0.0,
- variance = 0.0,
- stddev = 0.0,
- count = 0,
- current = 0,
- description
-}).
diff --git a/src/couchdb/couch_stats_aggregator.erl b/src/couchdb/couch_stats_aggregator.erl
index 1602ccbb..6833fcbe 100644
--- a/src/couchdb/couch_stats_aggregator.erl
+++ b/src/couchdb/couch_stats_aggregator.erl
@@ -11,375 +11,283 @@
% the License.
-module(couch_stats_aggregator).
--include("couch_stats.hrl").
-
-behaviour(gen_server).
--export([init/1, handle_call/3, handle_cast/2, handle_info/2,
- terminate/2, code_change/3]).
-
--export([start/0, stop/0,
- get/1, get/2, get_json/1, get_json/2, all/0,
- time_passed/0, clear_aggregates/1]).
-
--record(state, {
- aggregates = [],
- descriptions = []
+-export([start/0, stop/0]).
+-export([all/0, all/1, get/1, get/2, get_json/1, get_json/2, collect_sample/0]).
+
+-export([init/1, terminate/2, code_change/3]).
+-export([handle_call/3, handle_cast/2, handle_info/2]).
+
+-record(aggregate, {
+ description = <<"">>,
+ seconds = 0,
+ count = 0,
+ current = null,
+ sum = null,
+ mean = null,
+ variance = null,
+ stddev = null,
+ min = null,
+ max = null,
+ samples = []
}).
--define(COLLECTOR, couch_stats_collector).
--define(QUEUE_MAX_LENGTH, 900). % maximimum number of seconds
-
-% PUBLIC API
start() ->
gen_server:start_link({local, ?MODULE}, ?MODULE, [], []).
stop() ->
- gen_server:call(?MODULE, stop).
+ gen_server:cast(?MODULE, stop).
+
+all() ->
+ ?MODULE:all(0).
+all(Time) when is_binary(Time) ->
+ ?MODULE:all(list_to_integer(binary_to_list(Time)));
+all(Time) when is_atom(Time) ->
+ ?MODULE:all(list_to_integer(atom_to_list(Time)));
+all(Time) when is_integer(Time) ->
+ Aggs = ets:match(?MODULE, {{'$1', Time}, '$2'}),
+ Stats = lists:map(fun([Key, Agg]) -> {Key, Agg} end, Aggs),
+ case Stats of
+ [] ->
+ {[]};
+ _ ->
+ Ret = lists:foldl(fun({{Mod, Key}, Agg}, Acc) ->
+ CurrKeys = case proplists:lookup(Mod, Acc) of
+ none -> [];
+ {Mod, {Keys}} -> Keys
+ end,
+ NewMod = {[{Key, to_json_term(Agg)} | CurrKeys]},
+ [{Mod, NewMod} | proplists:delete(Mod, Acc)]
+ end, [], Stats),
+ {Ret}
+ end.
get(Key) ->
- gen_server:call(?MODULE, {get, Key}).
-get(Key, Time) ->
- gen_server:call(?MODULE, {get, Key, Time}).
+ ?MODULE:get(Key, 0).
+get(Key, Time) when is_binary(Time) ->
+ ?MODULE:get(Key, list_to_integer(binary_to_list(Time)));
+get(Key, Time) when is_atom(Time) ->
+ ?MODULE:get(Key, list_to_integer(atom_to_list(Time)));
+get(Key, Time) when is_integer(Time) ->
+ case ets:lookup(?MODULE, {make_key(Key), Time}) of
+ [] -> #aggregate{seconds=Time};
+ [{_, Agg}] -> Agg
+ end.
get_json(Key) ->
- gen_server:call(?MODULE, {get_json, Key}).
+ get_json(Key, 0).
get_json(Key, Time) ->
- gen_server:call(?MODULE, {get_json, Key, Time}).
+ to_json_term(?MODULE:get(Key, Time)).
-time_passed() ->
- gen_server:call(?MODULE, time_passed).
+collect_sample() ->
+ gen_server:call(?MODULE, collect_sample).
-clear_aggregates(Time) ->
- gen_server:call(?MODULE, {clear_aggregates, Time}).
-
-all() ->
- gen_server:call(?MODULE, all).
-
-% GEN_SERVER
init(_) ->
+ % Create an aggregate entry for each {description, rate} pair.
ets:new(?MODULE, [named_table, set, protected]),
- init_timers(),
- init_descriptions(),
- {ok, #state{}}.
-
-handle_call({get, Key}, _, State) ->
- Value = get_aggregate(Key, State),
- {reply, Value, State};
-
-handle_call({get, Key, Time}, _, State) ->
- Value = get_aggregate(Key, State, Time),
- {reply, Value, State};
-
-handle_call({get_json, Key}, _, State) ->
- Value = aggregate_to_json_term(get_aggregate(Key, State)),
- {reply, Value, State};
-
-handle_call({get_json, Key, Time}, _, State) ->
- Value = aggregate_to_json_term(get_aggregate(Key, State, Time)),
- {reply, Value, State};
-
-handle_call(time_passed, _, OldState) ->
-
- % the foldls below could probably be refactored into a less code-duping form
-
- % update aggregates on incremental counters
- NextState = lists:foldl(fun(Counter, State) ->
- {Key, Value} = Counter,
- update_aggregates_loop(Key, Value, State, incremental)
- end, OldState, ?COLLECTOR:all(incremental)),
-
- % update aggregates on absolute value counters
- NewState = lists:foldl(fun(Counter, State) ->
- {Key, Value} = Counter,
- % clear the counter, we've got the important bits in State
- ?COLLECTOR:clear(Key),
- update_aggregates_loop(Key, Value, State, absolute)
- end, NextState, ?COLLECTOR:all(absolute)),
-
- {reply, ok, NewState};
-
-handle_call({clear_aggregates, Time}, _, State) ->
- {reply, ok, do_clear_aggregates(Time, State)};
-
-handle_call(all, _ , State) ->
- Results = do_get_all(State),
- {reply, Results, State};
-
-handle_call(stop, _, State) ->
- {stop, normal, stopped, State}.
+ SampleStr = couch_config:get("stats", "samples", "[0]"),
+ {ok, Samples} = couch_util:parse_term(SampleStr),
+ lists:foreach(fun({KeyStr, Value}) ->
+ {ok, Key} = couch_util:parse_term(KeyStr),
+ lists:foreach(fun(Secs) ->
+ Agg = #aggregate{
+ description=list_to_binary(Value),
+ seconds=Secs
+ },
+ ets:insert(?MODULE, {{Key, Secs}, Agg})
+ end, Samples)
+ end, couch_config:get("stats_descriptions")),
+
+ Self = self(),
+ ok = couch_config:register(
+ fun("stats", _) -> exit(Self, config_change) end
+ ),
+ ok = couch_config:register(
+ fun("stats_descriptions", _) -> exit(Self, config_change) end
+ ),
+
+ Rate = list_to_integer(couch_config:get("stats", "rate", "1000")),
+ % TODO: Add timer_start to kernel start options.
+ timer:apply_interval(Rate, ?MODULE, collect_sample, []).
+
+terminate(_Reason, TRef) ->
+ timer:cancel(TRef),
+ ok.
+
+handle_call(collect_sample, _, State) ->
+ % Gather new stats values to add.
+ Incs = lists:map(fun({Key, Value}) ->
+ {Key, {incremental, Value}}
+ end, couch_stats_collector:all(incremental)),
+ Abs = lists:map(fun({Key, Values}) ->
+ couch_stats_collector:clear(Key),
+ Values2 = case Values of
+ X when is_list(X) -> X;
+ Else -> [Else]
+ end,
+ {_, Mean} = lists:foldl(fun(Val, {Count, Curr}) ->
+ {Count+1, Curr + (Val - Curr) / (Count+1)}
+ end, {0, 0}, Values2),
+ {Key, {absolute, Mean}}
+ end, couch_stats_collector:all(absolute)),
+
+ Values = Incs ++ Abs,
+ Now = erlang:now(),
+ lists:foreach(fun({{Key, Rate}, Agg}) ->
+ NewAgg = case proplists:lookup(Key, Values) of
+ none ->
+ rem_values(Now, Agg);
+ {Key, {Type, Value}} ->
+ NewValue = new_value(Type, Value, Agg#aggregate.current),
+ Agg2 = add_value(Now, NewValue, Agg),
+ rem_values(Now, Agg2)
+ end,
+ ets:insert(?MODULE, {{Key, Rate}, NewAgg})
+ end, ets:tab2list(?MODULE)),
+ {reply, ok, State}.
+
+handle_cast(stop, State) ->
+ {stop, normal, State}.
+handle_info(_Info, State) ->
+ {noreply, State}.
-% PRIVATE API
+code_change(_OldVersion, State, _Extra) ->
+ {ok, State}.
-% Stats = [{Key, TimesProplist}]
-% TimesProplist = [{Time, Aggrgates}]
-% Aggregates = #aggregates{}
-%
-% [
-% {Key, [
-% {TimeA, #aggregates{}},
-% {TimeB, #aggregates{}},
-% {TimeC, #aggregates{}},
-% {TimeD, #aggregates{}}
-% ]
-% },
-%
-% ]
-%% clear the aggregats record for a specific Time = 60 | 300 | 900
-do_clear_aggregates(Time, #state{aggregates=Stats}) ->
- NewStats = lists:map(fun({Key, TimesProplist}) ->
- {Key, case proplists:lookup(Time, TimesProplist) of
- % do have stats for this key, if we don't, return Stat unmodified
- none ->
- TimesProplist;
- % there are stats, let's unset the Time one
- {_Time, _Stat} ->
- [{Time, #aggregates{}} | proplists:delete(Time, TimesProplist)]
- end}
- end, Stats),
- #state{aggregates=NewStats}.
+new_value(incremental, Value, null) ->
+ Value;
+new_value(incremental, Value, Current) ->
+ Value - Current;
+new_value(absolute, Value, _Current) ->
+ Value.
-get_aggregate(Key, State) ->
- %% default Time is 0, which is when CouchDB started
- get_aggregate(Key, State, '0').
-get_aggregate(Key, #state{aggregates=StatsList}, Time) ->
- Description = get_description(Key),
- Aggregates = case proplists:lookup(Key, StatsList) of
- % if we don't have any data here, return an empty record
- none -> #aggregates{description=Description};
- {Key, Stats} ->
- case proplists:lookup(Time, Stats) of
- none -> #aggregates{description=Description}; % empty record again
- {Time, Stat} -> Stat#aggregates{description=Description}
- end
+add_value(Time, Value, #aggregate{count=Count}=Agg) when Count < 1 ->
+ Samples = case Agg#aggregate.seconds of
+ 0 -> [];
+ _ -> [{Time, Value}]
end,
- Aggregates.
-
-get_description(Key) ->
- case ets:lookup(?MODULE, Key) of
- [] -> <<"No description yet.">>;
- [{_Key, Description}] -> Description
+ Agg#aggregate{
+ count=1,
+ current=Value,
+ sum=Value,
+ mean=Value,
+ variance=0.0,
+ stddev=null,
+ min=Value,
+ max=Value,
+ samples=Samples
+ };
+add_value(Time, Value, Agg) ->
+ #aggregate{
+ count=Count,
+ current=Current,
+ sum=Sum,
+ mean=Mean,
+ variance=Variance,
+ samples=Samples
+ } = Agg,
+
+ NewCount = Count + 1,
+ NewMean = Mean + (Value - Mean) / NewCount,
+ NewVariance = Variance + (Value - Mean) * (Value - NewMean),
+ StdDev = case NewCount > 1 of
+ false -> null;
+ _ -> math:sqrt(NewVariance / (NewCount - 1))
+ end,
+ Agg2 = Agg#aggregate{
+ count=NewCount,
+ current=Current + Value,
+ sum=Sum + Value,
+ mean=NewMean,
+ variance=NewVariance,
+ stddev=StdDev,
+ min=lists:min([Agg#aggregate.min, Value]),
+ max=lists:max([Agg#aggregate.max, Value])
+ },
+ case Agg2#aggregate.seconds of
+ 0 -> Agg2;
+ _ -> Agg2#aggregate{samples=[{Time, Value} | Samples]}
end.
-%% updates all aggregates for Key
-update_aggregates_loop(Key, Values, State, CounterType) ->
- #state{aggregates=AllStats} = State,
- % if we don't have any aggregates yet, put a list of empty atoms in
- % so we can loop over them in update_aggregates().
- % [{{httpd,requests},
- % [{'0',{aggregates,1,1,1,0,0,1,1}},
- % {'60',{aggregates,1,1,1,0,0,1,1}},
- % {'300',{aggregates,1,1,1,0,0,1,1}},
- % {'900',{aggregates,1,1,1,0,0,1,1}}]}]
- [{_Key, StatsList}] = case proplists:lookup(Key, AllStats) of
- none -> [{Key, [
- {'0', empty},
- {'60', empty},
- {'300', empty},
- {'900', empty}
- ]}];
- AllStatsMatch ->
- [AllStatsMatch]
+rem_values(Time, Agg) ->
+ Seconds = Agg#aggregate.seconds,
+ Samples = Agg#aggregate.samples,
+ Pred = fun({When, _Value}) ->
+ timer:now_diff(Time, When) =< (Seconds * 1000000)
end,
-
- % if we get called with a single value, wrap in in a list
- ValuesList = case is_list(Values) of
- false -> [Values];
- _True -> Values
+ {Keep, Remove} = lists:splitwith(Pred, Samples),
+ Agg2 = lists:foldl(fun({_, Value}, Acc) ->
+ rem_value(Value, Acc)
+ end, Agg, Remove),
+ Agg2#aggregate{samples=Keep}.
+
+rem_value(_Value, #aggregate{count=Count}=Agg) when Count =< 1 ->
+ #aggregate{seconds=Agg#aggregate.seconds};
+rem_value(Value, Agg) ->
+ #aggregate{
+ count=Count,
+ sum=Sum,
+ mean=Mean,
+ variance=Variance
+ } = Agg,
+
+ OldMean = (Mean * Count - Value) / (Count - 1),
+ OldVariance = Variance - (Value - OldMean) * (Value - Mean),
+ OldCount = Count - 1,
+ StdDev = case OldCount > 1 of
+ false -> null;
+ _ -> math:sqrt(clamp_value(OldVariance / (OldCount - 1)))
end,
-
- % loop over all Time's
- NewStats = lists:map(fun({Time, Stats}) ->
- % loop over all values for Key
- lists:foldl(fun(Value, Stat) ->
- {Time, update_aggregates(Value, Stat, CounterType)}
- end, Stats, ValuesList)
- end, StatsList),
-
- % put the newly calculated aggregates into State and delete the previous
- % entry
- #state{
- aggregates=[{Key, NewStats} | proplists:delete(Key, AllStats)]
+ Agg#aggregate{
+ count=OldCount,
+ sum=Sum-Value,
+ mean=clamp_value(OldMean),
+ variance=clamp_value(OldVariance),
+ stddev=StdDev
}.
-% does the actual updating of the aggregate record
-update_aggregates(Value, Stat, CounterType) ->
- case Stat of
- % the first time this is called, we don't have to calculate anything
- % we just populate the record with Value
- empty -> #aggregates{
- min=Value,
- max=Value,
- mean=Value,
- variance=0,
- stddev=0,
- count=1,
- current=Value
- };
- % this sure could look nicer -- any ideas?
- StatsRecord ->
- #aggregates{
- min=Min,
- max=Max,
- mean=Mean,
- variance=Variance,
- count=Count,
- current=Current
- } = StatsRecord,
-
- % incremental counters need to keep track of the last update's value
- NewValue = case CounterType of
- incremental -> Value - Current;
- absolute -> Value
- end,
- % Knuth, The Art of Computer Programming, vol. 2, p. 232.
- NewCount = Count + 1,
- NewMean = Mean + (NewValue - Mean) / NewCount, % NewCount is never 0.
- NewVariance = Variance + (NewValue - Mean) * (NewValue - NewMean),
- #aggregates{
- min=lists:min([NewValue, Min]),
- max=lists:max([NewValue, Max]),
- mean=NewMean,
- variance=NewVariance,
- stddev=math:sqrt(NewVariance / NewCount),
- count=NewCount,
- current=Value
- }
- end.
-
-
-aggregate_to_json_term(#aggregates{min=Min,max=Max,mean=Mean,stddev=Stddev,count=Count,current=Current,description=Description}) ->
+to_json_term(Agg) ->
+ {Min, Max} = case Agg#aggregate.seconds > 0 of
+ false ->
+ {Agg#aggregate.min, Agg#aggregate.max};
+ _ ->
+ case length(Agg#aggregate.samples) > 0 of
+ true ->
+ Extract = fun({_Time, Value}) -> Value end,
+ Samples = lists:map(Extract, Agg#aggregate.samples),
+ {lists:min(Samples), lists:max(Samples)};
+ _ ->
+ {null, null}
+ end
+ end,
{[
- {current, Current},
- {count, Count},
- {mean, Mean},
+ {description, Agg#aggregate.description},
+ {current, round_value(Agg#aggregate.sum)},
+ {sum, round_value(Agg#aggregate.sum)},
+ {mean, round_value(Agg#aggregate.mean)},
+ {stddev, round_value(Agg#aggregate.stddev)},
{min, Min},
- {max, Max},
- {stddev, Stddev},
- {description, Description}
+ {max, Max}
]}.
-get_stats(Key, State) ->
- aggregate_to_json_term(get_aggregate(Key, State)).
-
-% convert ets2list() list into JSON-erlang-terms.
-% Thanks to Paul Davis
-do_get_all(#state{aggregates=Stats}=State) ->
- case Stats of
- [] -> {[]};
- _ ->
- [{LastMod, LastVals} | LastRestMods] = lists:foldl(fun({{Module, Key}, _Count}, AccIn) ->
- case AccIn of
- [] ->
- [{Module, [{Key, get_stats({Module, Key}, State)}]}];
- [{Module, PrevVals} | RestMods] ->
- [{Module, [{Key, get_stats({Module, Key}, State)} | PrevVals]} | RestMods];
- [{OtherMod, ModVals} | RestMods] ->
- [{Module, [{Key, get_stats({Module, Key}, State)}]}, {OtherMod, {lists:reverse(ModVals)}} | RestMods]
- end
- end, [], lists:sort(Stats)),
- {[{LastMod, {lists:sort(LastVals)}} | LastRestMods]}
- end.
-
-
-init_descriptions() ->
-
- % ets is probably overkill here, but I didn't manage to keep the
- % descriptions in the gen_server state. Which means there is probably
- % a bug in one of the handle_call() functions most likely the one that
- % handles the time_passed message. But don't tell anyone, the math is
- % correct :) -- Jan
-
-
- % Style guide for descriptions: Start with a lowercase letter & do not add
- % a trailing full-stop / period.
-
- % please keep this in alphabetical order
- ets:insert(?MODULE, {{couchdb, database_writes}, <<"number of times a database was changed">>}),
- ets:insert(?MODULE, {{couchdb, database_reads}, <<"number of times a document was read from a database">>}),
- ets:insert(?MODULE, {{couchdb, open_databases}, <<"number of open databases">>}),
- ets:insert(?MODULE, {{couchdb, open_os_files}, <<"number of file descriptors CouchDB has open">>}),
- ets:insert(?MODULE, {{couchdb, request_time}, <<"length of a request inside CouchDB without MochiWeb">>}),
-
- ets:insert(?MODULE, {{httpd, bulk_requests}, <<"number of bulk requests">>}),
- ets:insert(?MODULE, {{httpd, requests}, <<"number of HTTP requests">>}),
- ets:insert(?MODULE, {{httpd, temporary_view_reads}, <<"number of temporary view reads">>}),
- ets:insert(?MODULE, {{httpd, view_reads}, <<"number of view reads">>}),
- ets:insert(?MODULE, {{httpd, clients_requesting_changes}, <<"Number of clients currently requesting continuous _changes">>}),
-
- ets:insert(?MODULE, {{httpd_request_methods, 'COPY'}, <<"number of HTTP COPY requests">>}),
- ets:insert(?MODULE, {{httpd_request_methods, 'DELETE'}, <<"number of HTTP DELETE requests">>}),
- ets:insert(?MODULE, {{httpd_request_methods, 'GET'}, <<"number of HTTP GET requests">>}),
- ets:insert(?MODULE, {{httpd_request_methods, 'HEAD'}, <<"number of HTTP HEAD requests">>}),
- ets:insert(?MODULE, {{httpd_request_methods, 'MOVE'}, <<"number of HTTP MOVE requests">>}),
- ets:insert(?MODULE, {{httpd_request_methods, 'POST'}, <<"number of HTTP POST requests">>}),
- ets:insert(?MODULE, {{httpd_request_methods, 'PUT'}, <<"number of HTTP PUT requests">>}),
-
- ets:insert(?MODULE, {{httpd_status_codes, '200'}, <<"number of HTTP 200 OK responses">>}),
- ets:insert(?MODULE, {{httpd_status_codes, '201'}, <<"number of HTTP 201 Created responses">>}),
- ets:insert(?MODULE, {{httpd_status_codes, '202'}, <<"number of HTTP 202 Accepted responses">>}),
- ets:insert(?MODULE, {{httpd_status_codes, '301'}, <<"number of HTTP 301 Moved Permanently responses">>}),
- ets:insert(?MODULE, {{httpd_status_codes, '304'}, <<"number of HTTP 304 Not Modified responses">>}),
- ets:insert(?MODULE, {{httpd_status_codes, '400'}, <<"number of HTTP 400 Bad Request responses">>}),
- ets:insert(?MODULE, {{httpd_status_codes, '401'}, <<"number of HTTP 401 Unauthorized responses">>}),
- ets:insert(?MODULE, {{httpd_status_codes, '403'}, <<"number of HTTP 403 Forbidden responses">>}),
- ets:insert(?MODULE, {{httpd_status_codes, '404'}, <<"number of HTTP 404 Not Found responses">>}),
- ets:insert(?MODULE, {{httpd_status_codes, '405'}, <<"number of HTTP 405 Method Not Allowed responses">>}),
- ets:insert(?MODULE, {{httpd_status_codes, '409'}, <<"number of HTTP 409 Conflict responses">>}),
- ets:insert(?MODULE, {{httpd_status_codes, '412'}, <<"number of HTTP 412 Precondition Failed responses">>}),
- ets:insert(?MODULE, {{httpd_status_codes, '500'}, <<"number of HTTP 500 Internal Server Error responses">>}).
- % please keep this in alphabetical order
-
-
-% Timer
-
-init_timers() ->
-
- % OTP docs on timer: http://erlang.org/doc/man/timer.html
- % start() -> ok
- % Starts the timer server. Normally, the server does not need to be
- % started explicitly. It is started dynamically if it is needed. This is
- % useful during development, but in a target system the server should be
- % started explicitly. Use configuration parameters for kernel for this.
- %
- % TODO: Add timer_start to kernel start options.
-
-
- % start timers every second, minute, five minutes and fifteen minutes
- % in the rare event of a timer death, couch_stats_aggregator will die,
- % too and restarted by the supervision tree, all stats (for the last
- % fifteen minutes) are gone.
-
- {ok, _} = timer:apply_interval(1000, ?MODULE, time_passed, []),
- {ok, _} = timer:apply_interval(60000, ?MODULE, clear_aggregates, ['60']),
- {ok, _} = timer:apply_interval(300000, ?MODULE, clear_aggregates, ['300']),
- {ok, _} = timer:apply_interval(900000, ?MODULE, clear_aggregates, ['900']).
-
-
-% Unused gen_server behaviour API functions that we need to declare.
-
-%% @doc Unused
-handle_cast(foo, State) ->
- {noreply, State}.
-
-handle_info(_Info, State) ->
- {noreply, State}.
-
-%% @doc Unused
-terminate(_Reason, _State) -> ok.
-
-%% @doc Unused
-code_change(_OldVersion, State, _Extra) -> {ok, State}.
-
-
-%% Tests
-
--ifdef(TEST).
-% Internal API unit tests go here
-
-
--endif.
+make_key({Mod, Val}) when is_integer(Val) ->
+ {Mod, list_to_atom(integer_to_list(Val))};
+make_key(Key) ->
+ Key.
+
+round_value(Val) when not is_number(Val) ->
+ Val;
+round_value(Val) when Val == 0 ->
+ Val;
+round_value(Val) ->
+ erlang:round(Val * 1000.0) / 1000.0.
+
+clamp_value(Val) when Val > 0.00000000000001 ->
+ Val;
+clamp_value(_) ->
+ 0.0.
diff --git a/src/couchdb/couch_stats_collector.erl b/src/couchdb/couch_stats_collector.erl
index 327a37c1..59d62a6e 100644
--- a/src/couchdb/couch_stats_collector.erl
+++ b/src/couchdb/couch_stats_collector.erl
@@ -18,23 +18,15 @@
-behaviour(gen_server).
--export([init/1, handle_call/3, handle_cast/2, handle_info/2,
- terminate/2, code_change/3]).
+-export([start/0, stop/0]).
+-export([all/0, all/1, get/1, increment/1, decrement/1, record/2, clear/1]).
+-export([track_process_count/1, track_process_count/2]).
+-export([init/1, terminate/2, code_change/3]).
+-export([handle_call/3, handle_cast/2, handle_info/2]).
--export([start/0, stop/0, get/1,
- increment/1, decrement/1,
- track_process_count/1, track_process_count/2,
- record/2, clear/1,
- all/0, all/1]).
-
--record(state, {}).
-
--define(ABSOLUTE_VALUE_COUNTER_TABLE, abs_table).
--define(HIT_COUNTER_TABLE, hit_table).
-
-
-% PUBLIC API
+-define(HIT_TABLE, stats_hit_table).
+-define(ABS_TABLE, stats_abs_table).
start() ->
gen_server:start_link({local, ?MODULE}, ?MODULE, [], []).
@@ -42,106 +34,103 @@ start() ->
stop() ->
gen_server:call(?MODULE, stop).
+all() ->
+ ets:tab2list(?HIT_TABLE) ++ abs_to_list().
+
+all(Type) ->
+ case Type of
+ incremental -> ets:tab2list(?HIT_TABLE);
+ absolute -> abs_to_list()
+ end.
+
get(Key) ->
- case ets:lookup(?HIT_COUNTER_TABLE, Key) of
+ case ets:lookup(?HIT_TABLE, Key) of
[] ->
- case ets:lookup(?ABSOLUTE_VALUE_COUNTER_TABLE, Key) of
+ case ets:lookup(?ABS_TABLE, Key) of
[] ->
- 0;
- Result2 -> extract_value_from_ets_result(Key, Result2)
+ nil;
+ AbsVals ->
+ lists:map(fun({_, Value}) -> Value end, AbsVals)
end;
- [{_,Result1}] -> Result1
+ [{_, Counter}] ->
+ Counter
end.
-increment({Module, Key}) when is_integer(Key) ->
- increment({Module, list_to_atom(integer_to_list(Key))});
increment(Key) ->
- case catch ets:update_counter(?HIT_COUNTER_TABLE, Key, 1) of
+ Key2 = make_key(Key),
+ case catch ets:update_counter(?HIT_TABLE, Key2, 1) of
{'EXIT', {badarg, _}} ->
- true = ets:insert(?HIT_COUNTER_TABLE, {Key, 1}),
+ true = ets:insert(?HIT_TABLE, {Key2, 1}),
ok;
- _ -> ok
+ _ ->
+ ok
end.
decrement(Key) ->
- case catch ets:update_counter(?HIT_COUNTER_TABLE, Key, -1) of
+ Key2 = make_key(Key),
+ case catch ets:update_counter(?HIT_TABLE, Key2, -1) of
{'EXIT', {badarg, _}} ->
- true = ets:insert(?HIT_COUNTER_TABLE, {Key, -1}),
+ true = ets:insert(?HIT_TABLE, {Key2, -1}),
ok;
_ -> ok
end.
record(Key, Value) ->
- ets:insert(?ABSOLUTE_VALUE_COUNTER_TABLE, {Key, Value}).
+ true = ets:insert(?ABS_TABLE, {make_key(Key), Value}).
clear(Key) ->
- true = ets:delete(?ABSOLUTE_VALUE_COUNTER_TABLE, Key).
-
-all() ->
- lists:append(ets:tab2list(?HIT_COUNTER_TABLE),
- ets:tab2list(?ABSOLUTE_VALUE_COUNTER_TABLE)).
-
-all(Type) ->
- case Type of
- incremental -> ets:tab2list(?HIT_COUNTER_TABLE);
- absolute -> ets:tab2list(?ABSOLUTE_VALUE_COUNTER_TABLE)
- end.
+ true = ets:delete(?ABS_TABLE, make_key(Key)).
track_process_count(Stat) ->
track_process_count(self(), Stat).
track_process_count(Pid, Stat) ->
+ MonitorFun = fun() ->
+ Ref = erlang:monitor(process, Pid),
+ receive {'DOWN', Ref, _, _, _} -> ok end,
+ couch_stats_collector:decrement(Stat)
+ end,
case (catch couch_stats_collector:increment(Stat)) of
- ok ->
- spawn(
- fun() ->
- erlang:monitor(process, Pid),
- receive {'DOWN', _, _, _, _} -> ok end,
- couch_stats_collector:decrement(Stat)
- end);
- _ -> ok
+ ok -> spawn(MonitorFun);
+ _ -> ok
end.
-% GEN_SERVER
-
-
init(_) ->
- ets:new(?HIT_COUNTER_TABLE, [named_table, set, public]),
- ets:new(?ABSOLUTE_VALUE_COUNTER_TABLE, [named_table, duplicate_bag, public]),
- {ok, #state{}}.
+ ets:new(?HIT_TABLE, [named_table, set, public]),
+ ets:new(?ABS_TABLE, [named_table, duplicate_bag, public]),
+ {ok, nil}.
+terminate(_Reason, _State) ->
+ ok.
handle_call(stop, _, State) ->
{stop, normal, stopped, State}.
-
-% PRIVATE API
-
-extract_value_from_ets_result(_Key, Result) ->
- lists:map(fun({_, Value}) -> Value end, Result).
-
-
-% Unused gen_server behaviour API functions that we need to declare.
-
-%% @doc Unused
handle_cast(foo, State) ->
{noreply, State}.
handle_info(_Info, State) ->
{noreply, State}.
-%% @doc Unused
-terminate(_Reason, _State) -> ok.
-
-%% @doc Unused
-code_change(_OldVersion, State, _Extra) -> {ok, State}.
-
-
-%% Tests
-
--ifdef(TEST).
-% Internal API unit tests go here
-
-
--endif.
+code_change(_OldVersion, State, _Extra) ->
+ {ok, State}.
+
+
+make_key({Module, Key}) when is_integer(Key) ->
+ {Module, list_to_atom(integer_to_list(Key))};
+make_key(Key) ->
+ Key.
+
+abs_to_list() ->
+ SortedKVs = lists:sort(ets:tab2list(?ABS_TABLE)),
+ lists:foldl(fun({Key, Val}, Acc) ->
+ case Acc of
+ [] ->
+ [{Key, [Val]}];
+ [{Key, Prev} | Rest] ->
+ [{Key, [Val | Prev]} | Rest];
+ Others ->
+ [{Key, [Val]} | Others]
+ end
+ end, [], SortedKVs). \ No newline at end of file