You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@couchdb.apache.org by bb...@apache.org on 2017/03/02 00:26:58 UTC

[1/2] couch-index commit: updated refs/heads/8409-view-cache to 886314c

Repository: couchdb-couch-index
Updated Branches:
  refs/heads/8409-view-cache [created] 886314c7e


Add view index eviction cache

Prior to these changes, CouchDB would open all view indexes at the same
time. This would cause resource utilization issues if a user would
iterate over many indices. This changes solves the problem by adding a
cache which allows admins to set a maximum number of concurrently open
indices.

The view index cache works by reference counting procs which have opened
indices. When a process opens an index, it tries to increment a counter
in an ets table. The increment operation acts as a per-index lock. If
the lock is acquired, the opener makes a request to open the index. When
the index is opened, a corresponding ref tracking process will be
spawned. This process will monitor the opener process as well as any
subsequent process which opens the index. When all opener procs close
the index or exit, the index will be marked as idle and therefore a
candidate to be closed. If the maximum number of indexes are open and a
new index is opened, a random idle index will be closed. If all indexes
are active, an error is returned.


Project: http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/repo
Commit: http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/commit/ebca40a9
Tree: http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/tree/ebca40a9
Diff: http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/diff/ebca40a9

Branch: refs/heads/8409-view-cache
Commit: ebca40a9e042a326d434b4b5aef0a1ff3c9dda4c
Parents: 53555fd
Author: Benjamin Bastian <be...@gmail.com>
Authored: Fri Jan 20 17:27:29 2017 -0800
Committer: Benjamin Bastian <be...@gmail.com>
Committed: Tue Feb 28 09:06:52 2017 -0800

----------------------------------------------------------------------
 include/couch_index.hrl     |   5 +
 src/couch_index_monitor.erl | 184 ++++++++++++++++++++++++++++++
 src/couch_index_server.erl  | 234 +++++++++++++++++++++++++++++----------
 3 files changed, 366 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/blob/ebca40a9/include/couch_index.hrl
----------------------------------------------------------------------
diff --git a/include/couch_index.hrl b/include/couch_index.hrl
new file mode 100644
index 0000000..03d4dc3
--- /dev/null
+++ b/include/couch_index.hrl
@@ -0,0 +1,5 @@
+-define(BY_SIG, couchdb_indexes_by_sig).
+-define(BY_COUNTERS, couchdb_indexes_by_counters).
+-define(BY_PID, couchdb_indexes_by_pid).
+-define(BY_IDLE, couchdb_indexes_by_idle).
+-define(BY_DB, couchdb_indexes_by_db).

http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/blob/ebca40a9/src/couch_index_monitor.erl
----------------------------------------------------------------------
diff --git a/src/couch_index_monitor.erl b/src/couch_index_monitor.erl
new file mode 100644
index 0000000..eebdc20
--- /dev/null
+++ b/src/couch_index_monitor.erl
@@ -0,0 +1,184 @@
+% 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.
+
+-module(couch_index_monitor).
+
+
+-export([
+    spawn_link/2,
+    close/1,
+    set_pid/2,
+
+    notify/1,
+    notify/2,
+    cancel/2
+]).
+
+-export([
+    init/2
+]).
+
+
+-include("couch_index.hrl").
+
+
+-record(st, {
+    name,
+    type,
+    is_sys,
+    ref,
+    client_refs,
+    closing
+}).
+
+
+spawn_link(Name, IsSys) ->
+    erlang:spawn_link(?MODULE, init, [Name, IsSys]).
+
+
+close(Monitor) ->
+    Monitor ! exit,
+    ok.
+
+
+set_pid(Monitor, Pid) ->
+    Monitor ! {set_pid, Pid},
+    ok.
+
+
+notify(Monitor) ->
+    notify(Monitor, self()).
+
+
+notify(Monitor, Client) when is_pid(Client) ->
+    Monitor ! {notify, Client},
+    ok;
+
+notify(Monitor, {Client, _}) when is_pid(Client) ->
+    notify(Monitor, Client).
+
+
+cancel(Name, {Client, Monitor, IsSys})
+        when Client == self(), is_pid(Monitor) ->
+    Monitor ! {cancel, self()},
+    case (catch ets:update_counter(?BY_COUNTERS, Name, -1)) of
+        0 when not IsSys ->
+            true = ets:insert(?BY_IDLE, Name),
+            ok;
+        _ ->
+            ok
+    end.
+
+
+init(Name, IsSys) ->
+    {ok, CRefs} = khash:new(),
+    loop(#st{
+        name = Name,
+        is_sys = IsSys,
+        ref = undefined,
+        client_refs = CRefs,
+        closing = false
+    }).
+
+
+handle_info(exit, St) ->
+    {stop, normal, St};
+
+handle_info({set_pid, Pid}, #st{ref = undefined} = St) ->
+    Ref = erlang:monitor(process, Pid),
+    {noreply, St#st{ref = Ref}};
+
+handle_info({set_pid, Pid}, #st{ref = Ref} = St) when is_reference(Ref) ->
+    erlang:demonitor(Ref, [flush]),
+    handle_info({set_pid, Pid}, St#st{ref = undefined});
+
+handle_info({notify, Client}, St) when is_pid(Client) ->
+    case khash:get(St#st.client_refs, Client) of
+        {Ref, Count} when is_reference(Ref), is_integer(Count), Count > 0 ->
+            khash:put(St#st.client_refs, Client, {Ref, Count + 1});
+        undefined ->
+            Ref = erlang:monitor(process, Client),
+            case khash:size(St#st.client_refs) of
+                0 ->
+                    % Our first monitor after being idle
+                    khash:put(St#st.client_refs, Client, {Ref, 1}),
+                    true = ets:delete(?BY_IDLE, St#st.name);
+                N when is_integer(N), N > 0 ->
+                    % Still not idle
+                    khash:put(St#st.client_refs, Client, {Ref, 1}),
+                    ok
+            end
+    end,
+    {noreply, St};
+
+handle_info({cancel, Client}, St) when is_pid(Client) ->
+    case khash:get(St#st.client_refs, Client) of
+        {Ref, 1} when is_reference(Ref) ->
+            erlang:demonitor(Ref, [flush]),
+            khash:del(St#st.client_refs, Client),
+            maybe_set_idle(St);
+        {Ref, Count} when is_reference(Ref), is_integer(Count), Count > 1 ->
+            khash:put(St#st.client_refs, Client, {Ref, Count - 1})
+    end,
+    {noreply, St};
+
+handle_info({'DOWN', Ref, process, _, _}, #st{ref = Ref} = St) ->
+    {stop, normal, St};
+
+handle_info({'DOWN', _Ref, process, Pid, _Reason}, St) ->
+    #st{name=Name} = St,
+    case khash:get(St#st.client_refs, Pid) of
+        {Ref, N} when is_reference(Ref), is_integer(N), N > 0 ->
+            ets:update_counter(?BY_COUNTERS, Name, -N),
+            khash:del(St#st.client_refs, Pid),
+            maybe_set_idle(St);
+        undefined ->
+            % Ignore unknown processes
+            ok
+    end,
+    {noreply, St};
+
+handle_info(Msg, St) ->
+    {stop, {bad_info, Msg}, St}.
+
+
+maybe_set_idle(St) ->
+    case khash:size(St#st.client_refs) of
+        0 when St#st.is_sys ->
+            % System dbs don't go idle so they're
+            % never a candidate to get closed
+            ok;
+        0 ->
+            % We're now idle
+            ets:insert(?BY_IDLE, {St#st.name});
+        N when is_integer(N), N > 0 ->
+            % We have other clients
+            ok
+    end.
+
+
+loop(St) ->
+    receive
+        Other ->
+            do_handle_info(Other, St)
+    end.
+
+
+do_handle_info(Msg, St) ->
+    try handle_info(Msg, St) of
+        {noreply, NewSt} ->
+            loop(NewSt);
+        {stop, Reason, _NewSt} ->
+            exit(Reason)
+    catch T:R ->
+        exit({T, R})
+    end.

http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/blob/ebca40a9/src/couch_index_server.erl
----------------------------------------------------------------------
diff --git a/src/couch_index_server.erl b/src/couch_index_server.erl
index 4e86f5e..fdd9835 100644
--- a/src/couch_index_server.erl
+++ b/src/couch_index_server.erl
@@ -16,7 +16,7 @@
 
 -vsn(2).
 
--export([start_link/0, validate/2, get_index/4, get_index/3, get_index/2]).
+-export([start_link/0, validate/2, get_index/4, get_index/3, get_index_from_state/3]).
 
 -export([init/1, terminate/2, code_change/3]).
 -export([handle_call/3, handle_cast/2, handle_info/2]).
@@ -28,14 +28,19 @@
     handle_db_event/3
 ]).
 
+-include("couch_index.hrl").
+
 -include_lib("couch/include/couch_db.hrl").
 
--define(BY_SIG, couchdb_indexes_by_sig).
--define(BY_PID, couchdb_indexes_by_pid).
--define(BY_DB, couchdb_indexes_by_db).
 -define(RELISTEN_DELAY, 5000).
+-define(MAX_INDEXES_OPEN, 500).
 
--record(st, {root_dir}).
+-record(st, {
+    root_dir,
+    max_indexes,
+    open,
+    sys_open
+}).
 
 start_link() ->
     gen_server:start_link({local, ?MODULE}, ?MODULE, [], []).
@@ -94,21 +99,27 @@ get_index(Module, Db, DDoc, Fun) when is_binary(DDoc) ->
 get_index(Module, Db, DDoc, Fun) when is_function(Fun, 1) ->
     {ok, InitState} = Module:init(Db, DDoc),
     {ok, FunResp} = Fun(InitState),
-    {ok, Pid} = get_index(Module, InitState),
-    {ok, Pid, FunResp};
+    case get_index_from_state(Module, InitState, couch_db:is_system_db(Db)) of
+        {ok, Pid} ->
+            {ok, Pid, FunResp};
+        {error, all_active} ->
+            {error, all_active}
+    end;
 get_index(Module, Db, DDoc, _Fun) ->
     {ok, InitState} = Module:init(Db, DDoc),
-    get_index(Module, InitState).
+    get_index_from_state(Module, InitState, couch_db:is_system_db(Db)).
 
 
-get_index(Module, IdxState) ->
+get_index_from_state(Module, IdxState, SysOwned) ->
     DbName = Module:get(db_name, IdxState),
     Sig = Module:get(signature, IdxState),
-    case ets:lookup(?BY_SIG, {DbName, Sig}) of
-        [{_, Pid}] when is_pid(Pid) ->
+    Args = {Module, IdxState, DbName, Sig, SysOwned},
+    case incref({DbName, Sig}) of
+        ok ->
+            [{_, {Pid, Monitor, _SysOwned}}] = ets:lookup(?BY_SIG, {DbName, Sig}),
+            ok = couch_index_monitor:notify(Monitor),
             {ok, Pid};
         _ ->
-            Args = {Module, IdxState, DbName, Sig},
             gen_server:call(?MODULE, {get_index, Args}, infinity)
     end.
 
@@ -116,13 +127,17 @@ get_index(Module, IdxState) ->
 init([]) ->
     process_flag(trap_exit, true),
     ok = config:listen_for_changes(?MODULE, couch_index_util:root_dir()),
-    ets:new(?BY_SIG, [protected, set, named_table]),
-    ets:new(?BY_PID, [private, set, named_table]),
-    ets:new(?BY_DB, [protected, bag, named_table]),
+    ets:new(?BY_DB, [public, bag, named_table]),
+    ets:new(?BY_SIG, [public, set, named_table]),
+    ets:new(?BY_PID, [public, set, named_table]),
+    ets:new(?BY_COUNTERS, [set, public, named_table]),
+    ets:new(?BY_IDLE, [set, public, named_table]),
     couch_event:link_listener(?MODULE, handle_db_event, nil, [all_dbs]),
     RootDir = couch_index_util:root_dir(),
     couch_file:init_delete_dir(RootDir),
-    {ok, #st{root_dir=RootDir}}.
+    MaxIndexes = list_to_integer(
+        config:get("couchdb", "max_indexes_open", integer_to_list(?MAX_INDEXES_OPEN))),
+    {ok, #st{root_dir=RootDir, max_indexes=MaxIndexes, open=0, sys_open=0}}.
 
 
 terminate(_Reason, _State) ->
@@ -130,51 +145,123 @@ terminate(_Reason, _State) ->
     lists:map(fun couch_util:shutdown_sync/1, Pids),
     ok.
 
+make_room(State, false) ->
+    case maybe_close_idle(State) of
+        {ok, NewState} ->
+            {ok, NewState};
+        Other ->
+            Other
+    end;
+make_room(State, true) ->
+    {ok, State}.
+
+-spec maybe_close_idle(#st{}) -> {ok, #st{}} | {error, all_active}.
+maybe_close_idle(#st{open=Open, max_indexes=Max}=State) when Open < Max ->
+    {ok, State};
+
+maybe_close_idle(State) ->
+    try
+        {ok, close_idle(State)}
+    catch error:all_active ->
+        {error, all_active}
+    end.
+
+-spec close_idle(#st{}) -> #st{}.
+close_idle(State) ->
+    ets:safe_fixtable(?BY_IDLE, true),
+    try
+        close_idle(State, ets:first(?BY_IDLE))
+    after
+        ets:safe_fixtable(?BY_IDLE, false)
+    end.
+
 
-handle_call({get_index, {_Mod, _IdxState, DbName, Sig}=Args}, From, State) ->
+-spec close_idle(#st{}, term()) -> #st{}.
+close_idle(_State, '$end_of_table') ->
+    erlang:error(all_active);
+
+close_idle(State, Name) ->
+    case ets:lookup(?BY_SIG, Name) of
+        [{_, {Pid, _Monitor, SysOwned}}] ->
+            true = ets:delete(?BY_IDLE, Name),
+            couch_index:stop(Pid),
+            closed(State, SysOwned);
+        [] ->
+            true = ets:delete(?BY_IDLE, Name),
+            close_idle(State, ets:next(?BY_IDLE, Name))
+    end.
+
+
+handle_call({get_index, {_Mod, _IdxState, DbName, Sig, SysOwned}=Args}, From, State) ->
     case ets:lookup(?BY_SIG, {DbName, Sig}) of
         [] ->
-            spawn_link(fun() -> new_index(Args) end),
-            ets:insert(?BY_SIG, {{DbName, Sig}, [From]}),
-            {noreply, State};
-        [{_, Waiters}] when is_list(Waiters) ->
-            ets:insert(?BY_SIG, {{DbName, Sig}, [From | Waiters]}),
+            case make_room(State, SysOwned) of
+                {ok, NewState} ->
+                    spawn_link(fun() -> new_index(Args) end),
+                    Monitor = couch_index_monitor:spawn_link({DbName, Sig}, SysOwned),
+                    ets:insert(?BY_SIG, {{DbName, Sig}, {[From], Monitor, SysOwned}}),
+                    {noreply, NewState};
+                {error, all_active} ->
+                    {reply, {error, all_active}, State}
+            end;
+        [{_, {Waiters, Monitor, SysOwned}}] when is_list(Waiters) ->
+            ets:insert(?BY_SIG, {{DbName, Sig}, {[From | Waiters], Monitor, SysOwned}}),
             {noreply, State};
-        [{_, Pid}] when is_pid(Pid) ->
+        [{_, {Pid, Monitor, _SysOwned}}] when is_pid(Pid) ->
+            ok = incref({DbName, Sig}),
+            ok = couch_index_monitor:notify(Monitor, From),
             {reply, {ok, Pid}, State}
     end;
 handle_call({async_open, {DbName, DDocId, Sig}, {ok, Pid}}, _From, State) ->
-    [{_, Waiters}] = ets:lookup(?BY_SIG, {DbName, Sig}),
-    [gen_server:reply(From, {ok, Pid}) || From <- Waiters],
+    [{_, {Waiters, Monitor, SysOwned}}] = ets:lookup(?BY_SIG, {DbName, Sig}),
     link(Pid),
-    add_to_ets(DbName, Sig, DDocId, Pid),
-    {reply, ok, State};
-handle_call({async_error, {DbName, _DDocId, Sig}, Error}, _From, State) ->
-    [{_, Waiters}] = ets:lookup(?BY_SIG, {DbName, Sig}),
+    ets:insert(?BY_SIG, {{DbName, Sig}, {Pid, Monitor, SysOwned}}),
+    ets:insert(?BY_PID, {Pid, {DbName, Sig}}),
+    ets:insert(?BY_COUNTERS, {{DbName, Sig}, 0}),
+    ets:insert(?BY_DB, {DbName, {DDocId, Sig}}),
+    lists:foreach(fun(From) ->
+        {Client, _} = From,
+        ok = incref({DbName, Sig}),
+        ok = couch_index_monitor:notify(Monitor, Client),
+        gen_server:reply(From, {ok, Pid})
+    end, Waiters),
+    {reply, ok, opened(State, SysOwned)};
+handle_call({async_error, {DbName, _DDocId, Sig}, Error}, {FromPid, _}, State) ->
+    [{_, {Waiters, Monitor, _SO}}] = ets:lookup(?BY_SIG, {DbName, Sig}),
     [gen_server:reply(From, Error) || From <- Waiters],
-    ets:delete(?BY_SIG, {DbName, Sig}),
+    true = ets:delete(?BY_COUNTERS, {DbName, Sig}),
+    true = ets:delete(?BY_SIG, {DbName, Sig}),
+    true = ets:delete(?BY_PID, FromPid),
+    true = ets:delete(?BY_IDLE, {DbName, Sig}),
+    ok = couch_index_monitor:close(Monitor),
     {reply, ok, State};
+handle_call({set_max_indexes_open, Max}, _From, State) ->
+    {reply, ok, State#st{max_indexes=Max}};
 handle_call({reset_indexes, DbName}, _From, State) ->
-    reset_indexes(DbName, State#st.root_dir),
-    {reply, ok, State}.
+    {reply, ok, reset_indexes(DbName, State)};
+handle_call(open_index_count, _From, State) ->
+    {reply, State#st.open, State};
+handle_call(get_server, _From, State) ->
+    {reply, State, State}.
 
 
 handle_cast({reset_indexes, DbName}, State) ->
-    reset_indexes(DbName, State#st.root_dir),
-    {noreply, State}.
+    {noreply, reset_indexes(DbName, State)}.
 
 handle_info({'EXIT', Pid, Reason}, Server) ->
     case ets:lookup(?BY_PID, Pid) of
-        [{Pid, {DbName, Sig}}] ->
+        [{_, {DbName, Sig}}] ->
+            [{_, {_W, _M, SysOwned}}] = ets:lookup(?BY_SIG, {DbName, Sig}),
             [{DbName, {DDocId, Sig}}] =
                 ets:match_object(?BY_DB, {DbName, {'$1', Sig}}),
-            rem_from_ets(DbName, Sig, DDocId, Pid);
+            rem_from_ets(DbName, Sig, DDocId, Pid),
+            {noreply, closed(Server, SysOwned)};
         [] when Reason /= normal ->
+            couch_log:error("Looked up: ~p ~p ~p", [?MODULE, pid, Pid]),
             exit(Reason);
         _Else ->
-            ok
-    end,
-    {noreply, Server};
+            {noreply, Server}
+    end;
 handle_info(restart_config_listener, State) ->
     ok = config:listen_for_changes(?MODULE, couch_index_util:root_dir()),
     {noreply, State};
@@ -197,8 +284,10 @@ handle_config_change("couchdb", "index_dir", _, _, _) ->
 handle_config_change("couchdb", "view_index_dir", _, _, _) ->
     exit(whereis(couch_index_server), config_change),
     remove_handler;
-handle_config_change(_, _, _, _, RootDir) ->
-    {ok, RootDir}.
+handle_config_change("couchdb", "max_indexes_open", Max, _, _) when is_list(Max) ->
+    {ok, gen_server:call(?MODULE, {set_max_indexes_open, list_to_integer(Max)})};
+handle_config_change(_, _, _, _, _) ->
+    {ok, nil}.
 
 handle_config_terminate(_, stop, _) ->
     ok;
@@ -206,7 +295,8 @@ handle_config_terminate(_Server, _Reason, _State) ->
     erlang:send_after(?RELISTEN_DELAY, whereis(?MODULE), restart_config_listener),
     {ok, couch_index_util:root_dir()}.
 
-new_index({Mod, IdxState, DbName, Sig}) ->
+
+new_index({Mod, IdxState, DbName, Sig, _SysOwned}) ->
     DDocId = Mod:get(idx_name, IdxState),
     case couch_index:start_link({Mod, IdxState}) of
         {ok, Pid} ->
@@ -219,29 +309,29 @@ new_index({Mod, IdxState, DbName, Sig}) ->
     end.
 
 
-reset_indexes(DbName, Root) ->
+reset_indexes(DbName, State) ->
+    #st{root_dir=Root} = State,
     % shutdown all the updaters and clear the files, the db got changed
-    Fun = fun({_, {DDocId, Sig}}) ->
-        [{_, Pid}] = ets:lookup(?BY_SIG, {DbName, Sig}),
+    Fun = fun({_, {DDocId, Sig}}, StateAcc) ->
+        [{_, {Pid, Monitor, SysOwned}}] = ets:lookup(?BY_SIG, {DbName, Sig}),
+        couch_index_monitor:close(Monitor),
         MRef = erlang:monitor(process, Pid),
         gen_server:cast(Pid, delete),
         receive {'DOWN', MRef, _, _, _} -> ok end,
-        rem_from_ets(DbName, Sig, DDocId, Pid)
+        rem_from_ets(DbName, Sig, DDocId, Pid),
+        closed(StateAcc, SysOwned)
     end,
-    lists:foreach(Fun, ets:lookup(?BY_DB, DbName)),
+    NewState = lists:foldl(Fun, State, ets:lookup(?BY_DB, DbName)),
     Path = couch_index_util:index_dir("", DbName),
-    couch_file:nuke_dir(Root, Path).
-
-
-add_to_ets(DbName, Sig, DDocId, Pid) ->
-    ets:insert(?BY_SIG, {{DbName, Sig}, Pid}),
-    ets:insert(?BY_PID, {Pid, {DbName, Sig}}),
-    ets:insert(?BY_DB, {DbName, {DDocId, Sig}}).
+    couch_file:nuke_dir(Root, Path),
+    NewState.
 
 
 rem_from_ets(DbName, Sig, DDocId, Pid) ->
-    ets:delete(?BY_SIG, {DbName, Sig}),
-    ets:delete(?BY_PID, Pid),
+    true = ets:delete(?BY_COUNTERS, {DbName, Sig}),
+    true = ets:delete(?BY_SIG, {DbName, Sig}),
+    true = ets:delete(?BY_PID, Pid),
+    true = ets:delete(?BY_IDLE, {DbName, Sig}),
     ets:delete_object(?BY_DB, {DbName, {DDocId, Sig}}).
 
 
@@ -254,7 +344,7 @@ handle_db_event(DbName, deleted, St) ->
 handle_db_event(DbName, {ddoc_updated, DDocId}, St) ->
     lists:foreach(fun({_DbName, {_DDocId, Sig}}) ->
         case ets:lookup(?BY_SIG, {DbName, Sig}) of
-            [{_, IndexPid}] ->
+            [{_, {IndexPid, _Monitor, _SysOwned}}] ->
                 (catch gen_server:cast(IndexPid, ddoc_updated));
             [] ->
                 ok
@@ -263,3 +353,33 @@ handle_db_event(DbName, {ddoc_updated, DDocId}, St) ->
     {ok, St};
 handle_db_event(_DbName, _Event, St) ->
     {ok, St}.
+
+
+-spec opened(#st{}, boolean()) -> #st{}.
+opened(State, IsSysOwned) ->
+    case IsSysOwned of
+        true -> State#st{sys_open=State#st.sys_open + 1};
+        false -> State#st{open=State#st.open + 1}
+    end.
+
+
+-spec closed(#st{}, boolean()) -> #st{}.
+closed(State, IsSysOwned) ->
+    case IsSysOwned of
+        true -> State#st{sys_open=State#st.sys_open - 1};
+        false -> State#st{open=State#st.open - 1}
+    end.
+
+
+incref(Name) ->
+    case (catch ets:update_counter(?BY_COUNTERS, Name, 1)) of
+        1 ->
+            ets:delete(?BY_IDLE, Name),
+            ok;
+        N when is_integer(N), N > 0 ->
+            ok;
+        N when is_integer(N) ->
+            {invalid_refcount, N};
+        {'EXIT', {badarg, _}} ->
+            missing_counter
+	end.


[2/2] couch-index commit: updated refs/heads/8409-view-cache to 886314c

Posted by bb...@apache.org.
Add basic test for index cache monitor


Project: http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/repo
Commit: http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/commit/886314c7
Tree: http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/tree/886314c7
Diff: http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/diff/886314c7

Branch: refs/heads/8409-view-cache
Commit: 886314c7ecbc5f7c0a1a3adaf3f79ca6ec641eb9
Parents: ebca40a
Author: Benjamin Bastian <be...@gmail.com>
Authored: Wed Mar 1 16:26:50 2017 -0800
Committer: Benjamin Bastian <be...@gmail.com>
Committed: Wed Mar 1 16:26:50 2017 -0800

----------------------------------------------------------------------
 test/couch_index_monitor_tests.erl | 107 ++++++++++++++++++++++++++++++++
 1 file changed, 107 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/blob/886314c7/test/couch_index_monitor_tests.erl
----------------------------------------------------------------------
diff --git a/test/couch_index_monitor_tests.erl b/test/couch_index_monitor_tests.erl
new file mode 100644
index 0000000..d4fd2c9
--- /dev/null
+++ b/test/couch_index_monitor_tests.erl
@@ -0,0 +1,107 @@
+% 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.
+
+-module(couch_index_monitor_tests).
+
+-include("couch_index.hrl").
+-include_lib("couch/include/couch_eunit.hrl").
+-include_lib("couch/include/couch_db.hrl").
+
+-define(DDOC, {[{
+    <<"views">>, {[
+        {<<"test_view">>, {[{<<"map">>, <<"function(doc) {emit(null, 1);}">>}]}
+    }]}
+}]}).
+
+
+setup() ->
+    DbName = ?tempdb(),
+    {ok, Db} = couch_db:create(DbName, [?ADMIN_CTX]),
+    DDocName = <<"_design/", DbName/binary>>,
+    DDoc = #doc{id = DDocName, body = ?DDOC},
+    {ok, _} = couch_db:update_doc(Db, DDoc, [?ADMIN_CTX]),
+    {Db, DDoc}.
+
+
+teardown({Db, _DDoc}) ->
+    couch_db:close(Db),
+    ok.
+
+
+monitoring_test_() ->
+    {
+        "Test index monitoring",
+        {
+            setup,
+            fun() -> test_util:start_couch([]) end, fun test_util:stop_couch/1,
+            {
+                foreach,
+                fun setup/0, fun teardown/1,
+                [
+                    fun test_basic/1
+                ]
+            }
+        }
+    }.
+
+
+test_basic({Db, DDoc}) ->
+    ?_test(begin
+        {ok, Pid} = couch_index_server:get_index(couch_mrview_index, Db#db.name, DDoc#doc.id),
+        {ok, IdxState} = couch_mrview_index:init(Db, DDoc),
+        DbName = couch_mrview_index:get(db_name, IdxState),
+        Sig = couch_mrview_index:get(signature, IdxState),
+        [{_, {Pid, Monitor, SysOwned}}] = ets:lookup(?BY_SIG, {DbName, Sig}),
+        ?assert(is_pid(Pid)),
+        ?assert(is_pid(Monitor)),
+        ?assertNot(SysOwned),
+        ?assertEqual(1, get_count({DbName, Sig})),
+        Acq1 = spawn_monitor_acquirer(DbName, DDoc#doc.id),
+        ?assertEqual(2, get_count({DbName, Sig})),
+        Acq2 = spawn_monitor_acquirer(DbName, DDoc#doc.id),
+        ?assertEqual(3, get_count({DbName, Sig})),
+        wait_down(Acq2),
+        ?assertEqual(2, get_count({DbName, Sig})),
+        wait_down(Acq1),
+        ?assertEqual(1, get_count({DbName, Sig})),
+        couch_index_monitor:cancel({DbName, Sig}, {self(), Monitor, SysOwned}),
+        ?assertEqual(0, get_count({DbName, Sig})),
+        ?assertEqual(1, length(ets:lookup(?BY_IDLE, {DbName, Sig}))),
+        ok
+    end).
+
+
+wait_down({Pid, Ref}) ->
+    Pid ! close,
+    receive {'DOWN', Ref, process, Pid, normal} ->
+        ok
+    end.
+
+
+spawn_monitor_acquirer(DbName, DDocID) ->
+    {Pid, Ref} = spawn_monitor(fun() ->
+        receive {acquire, From} ->
+            {ok, _} = couch_index_server:get_index(couch_mrview_index, DbName, DDocID),
+            From ! acquired
+        end,
+        receive close -> ok end
+    end),
+    Pid ! {acquire, self()},
+    receive acquired -> ok end,
+    {Pid, Ref}.
+
+
+get_count(NameSig) ->
+    case ets:lookup(?BY_COUNTERS, NameSig) of
+        [] -> 0;
+        [{_, Count}] -> Count
+    end.