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/08 20:39:25 UTC

[1/6] couch-index commit: updated refs/heads/8409-view-cache to 5a9f05c

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


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/4f184e48
Tree: http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/tree/4f184e48
Diff: http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/diff/4f184e48

Branch: refs/heads/8409-view-cache
Commit: 4f184e4886910e37ff67be2011e2ff90f9b3fc07
Parents: 53555fd
Author: Benjamin Bastian <be...@gmail.com>
Authored: Fri Jan 20 17:27:29 2017 -0800
Committer: Benjamin Bastian <be...@gmail.com>
Committed: Thu Mar 2 10:07:08 2017 -0800

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


http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/blob/4f184e48/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/4f184e48/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..4cc4bc7
--- /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/4f184e48/src/couch_index_server.erl
----------------------------------------------------------------------
diff --git a/src/couch_index_server.erl b/src/couch_index_server.erl
index 4e86f5e..0350492 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, [public, set, named_table]),
+    ets:new(?BY_IDLE, [public, set, 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,122 @@ 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#st.sys_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 ->
             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 +283,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 +294,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 +308,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 +343,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 +352,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.


[4/6] couch-index commit: updated refs/heads/8409-view-cache to 5a9f05c

Posted by bb...@apache.org.
squash


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/ada99aa0
Tree: http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/tree/ada99aa0
Diff: http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/diff/ada99aa0

Branch: refs/heads/8409-view-cache
Commit: ada99aa0956d7d5908fe8d94a6e6d6c524563266
Parents: 78a65a3
Author: Benjamin Bastian <be...@gmail.com>
Authored: Thu Mar 2 19:12:00 2017 -0800
Committer: Benjamin Bastian <be...@gmail.com>
Committed: Thu Mar 2 19:12:00 2017 -0800

----------------------------------------------------------------------
 src/couch_index_server.erl | 25 ++++++++++++++++++-------
 1 file changed, 18 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/blob/ada99aa0/src/couch_index_server.erl
----------------------------------------------------------------------
diff --git a/src/couch_index_server.erl b/src/couch_index_server.erl
index 21028a1..f191700 100644
--- a/src/couch_index_server.erl
+++ b/src/couch_index_server.erl
@@ -34,11 +34,13 @@
 
 -define(RELISTEN_DELAY, 5000).
 -define(MAX_INDEXES_OPEN, 500).
+-define(CLOSE_INTERVAL, 5000).
 
 -record(st, {
     root_dir,
     soft_max_indexes,
-    open
+    open,
+    timer
 }).
 
 start_link() ->
@@ -130,9 +132,10 @@ init([]) ->
     couch_event:link_listener(?MODULE, handle_db_event, nil, [all_dbs]),
     RootDir = couch_index_util:root_dir(),
     couch_file:init_delete_dir(RootDir),
+    {ok, Timer} = timer:send_interval(?CLOSE_INTERVAL, close_idle),
     MaxIndexes = list_to_integer(
         config:get("couchdb", "soft_max_indexes_open", integer_to_list(?MAX_INDEXES_OPEN))),
-    {ok, #st{root_dir=RootDir, soft_max_indexes=MaxIndexes, open=0}}.
+    {ok, #st{root_dir=RootDir, soft_max_indexes=MaxIndexes, open=0, timer=Timer}}.
 
 
 terminate(_Reason, _State) ->
@@ -158,19 +161,22 @@ maybe_close_idle(State) ->
 close_idle(State, '$end_of_table') ->
     State;
 
-close_idle(State, Name) ->
-    true = ets:delete(?BY_IDLE, Name),
-    case ets:lookup(?BY_SIG, Name) of
+close_idle(State, {DbName, Sig}) ->
+    true = ets:delete(?BY_IDLE, {DbName, Sig}),
+    case ets:lookup(?BY_SIG, {DbName, Sig}) of
         [{_, {Pid, _Monitor}}] ->
             couch_index:stop(Pid),
+            [{DbName, {DDocId, Sig}}] =
+                ets:match_object(?BY_DB, {DbName, {'$1', Sig}}),
+            rem_from_ets(DbName, Sig, DDocId, Pid),
             case closed(State) of
                 NewState when NewState#st.open > State#st.soft_max_indexes ->
-                    close_idle(NewState, ets:next(?BY_IDLE, Name));
+                    close_idle(NewState, ets:next(?BY_IDLE, {DbName, Sig}));
                 NewState ->
                     NewState
             end;
         [] ->
-            close_idle(State, ets:next(?BY_IDLE, Name))
+            close_idle(State, ets:next(?BY_IDLE, {DbName, Sig}))
     end.
 
 
@@ -238,6 +244,11 @@ handle_info({'EXIT', Pid, Reason}, Server) ->
         _Else ->
             {noreply, Server}
     end;
+handle_info(close_idle, State) ->
+    {ok, NewState} = maybe_close_idle(State),
+    {ok, cancel} = timer:cancel(State#st.timer),
+    {ok, Timer} = timer:send_interval(?CLOSE_INTERVAL, close_idle),
+    {noreply, NewState#st{timer=Timer}};
 handle_info(restart_config_listener, State) ->
     ok = config:listen_for_changes(?MODULE, couch_index_util:root_dir()),
     {noreply, State};


[3/6] couch-index commit: updated refs/heads/8409-view-cache to 5a9f05c

Posted by bb...@apache.org.
Make index cache only have a soft limit


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/78a65a3f
Tree: http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/tree/78a65a3f
Diff: http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/diff/78a65a3f

Branch: refs/heads/8409-view-cache
Commit: 78a65a3f5eefd5669b096fcbc4806cd9029299fb
Parents: 19a3a49
Author: Benjamin Bastian <be...@gmail.com>
Authored: Thu Mar 2 14:23:24 2017 -0800
Committer: Benjamin Bastian <be...@gmail.com>
Committed: Thu Mar 2 14:23:24 2017 -0800

----------------------------------------------------------------------
 src/couch_index_monitor.erl        |  20 ++---
 src/couch_index_server.erl         | 133 +++++++++++++-------------------
 test/couch_index_monitor_tests.erl |   5 +-
 3 files changed, 61 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/blob/78a65a3f/src/couch_index_monitor.erl
----------------------------------------------------------------------
diff --git a/src/couch_index_monitor.erl b/src/couch_index_monitor.erl
index 4cc4bc7..fc2fef1 100644
--- a/src/couch_index_monitor.erl
+++ b/src/couch_index_monitor.erl
@@ -14,7 +14,7 @@
 
 
 -export([
-    spawn_link/2,
+    spawn_link/1,
     close/1,
     set_pid/2,
 
@@ -24,7 +24,7 @@
 ]).
 
 -export([
-    init/2
+    init/1
 ]).
 
 
@@ -34,15 +34,14 @@
 -record(st, {
     name,
     type,
-    is_sys,
     ref,
     client_refs,
     closing
 }).
 
 
-spawn_link(Name, IsSys) ->
-    erlang:spawn_link(?MODULE, init, [Name, IsSys]).
+spawn_link(Name) ->
+    erlang:spawn_link(?MODULE, init, [Name]).
 
 
 close(Monitor) ->
@@ -67,11 +66,11 @@ notify(Monitor, {Client, _}) when is_pid(Client) ->
     notify(Monitor, Client).
 
 
-cancel(Name, {Client, Monitor, IsSys})
+cancel(Name, {Client, Monitor})
         when Client == self(), is_pid(Monitor) ->
     Monitor ! {cancel, self()},
     case (catch ets:update_counter(?BY_COUNTERS, Name, -1)) of
-        0 when not IsSys ->
+        0 ->
             true = ets:insert(?BY_IDLE, {Name}),
             ok;
         _ ->
@@ -79,11 +78,10 @@ cancel(Name, {Client, Monitor, IsSys})
     end.
 
 
-init(Name, IsSys) ->
+init(Name) ->
     {ok, CRefs} = khash:new(),
     loop(#st{
         name = Name,
-        is_sys = IsSys,
         ref = undefined,
         client_refs = CRefs,
         closing = false
@@ -153,10 +151,6 @@ handle_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});

http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/blob/78a65a3f/src/couch_index_server.erl
----------------------------------------------------------------------
diff --git a/src/couch_index_server.erl b/src/couch_index_server.erl
index 0350492..21028a1 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_from_state/3]).
+-export([start_link/0, validate/2, get_index/4, get_index/3, get_index/2]).
 
 -export([init/1, terminate/2, code_change/3]).
 -export([handle_call/3, handle_cast/2, handle_info/2]).
@@ -37,9 +37,8 @@
 
 -record(st, {
     root_dir,
-    max_indexes,
-    open,
-    sys_open
+    soft_max_indexes,
+    open
 }).
 
 start_link() ->
@@ -99,24 +98,20 @@ 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),
-    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;
+    {ok, Pid} = get_index(Module, InitState),
+    {ok, Pid, FunResp};
 get_index(Module, Db, DDoc, _Fun) ->
     {ok, InitState} = Module:init(Db, DDoc),
-    get_index_from_state(Module, InitState, couch_db:is_system_db(Db)).
+    get_index(Module, InitState).
 
 
-get_index_from_state(Module, IdxState, SysOwned) ->
+get_index(Module, IdxState) ->
     DbName = Module:get(db_name, IdxState),
     Sig = Module:get(signature, IdxState),
-    Args = {Module, IdxState, DbName, Sig, SysOwned},
+    Args = {Module, IdxState, DbName, Sig},
     case incref({DbName, Sig}) of
         ok ->
-            [{_, {Pid, Monitor, _SysOwned}}] = ets:lookup(?BY_SIG, {DbName, Sig}),
+            [{_, {Pid, Monitor}}] = ets:lookup(?BY_SIG, {DbName, Sig}),
             ok = couch_index_monitor:notify(Monitor),
             {ok, Pid};
         _ ->
@@ -136,8 +131,8 @@ init([]) ->
     RootDir = couch_index_util:root_dir(),
     couch_file:init_delete_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}}.
+        config:get("couchdb", "soft_max_indexes_open", integer_to_list(?MAX_INDEXES_OPEN))),
+    {ok, #st{root_dir=RootDir, soft_max_indexes=MaxIndexes, open=0}}.
 
 
 terminate(_Reason, _State) ->
@@ -145,77 +140,60 @@ 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 ->
+-spec maybe_close_idle(#st{}) -> {ok, #st{}}.
+maybe_close_idle(#st{open=Open, soft_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))
+        {ok, close_idle(State, ets:first(?BY_IDLE))}
     after
         ets:safe_fixtable(?BY_IDLE, false)
     end.
 
 
 -spec close_idle(#st{}, term()) -> #st{}.
-close_idle(_State, '$end_of_table') ->
-    erlang:error(all_active);
+close_idle(State, '$end_of_table') ->
+    State;
 
 close_idle(State, Name) ->
+    true = ets:delete(?BY_IDLE, Name),
     case ets:lookup(?BY_SIG, Name) of
-        [{_, {Pid, _Monitor, SysOwned}}] ->
-            true = ets:delete(?BY_IDLE, Name),
+        [{_, {Pid, _Monitor}}] ->
             couch_index:stop(Pid),
-            closed(State, SysOwned);
+            case closed(State) of
+                NewState when NewState#st.open > State#st.soft_max_indexes ->
+                    close_idle(NewState, ets:next(?BY_IDLE, Name));
+                NewState ->
+                    NewState
+            end;
         [] ->
-            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) ->
+handle_call({get_index, {_Mod, _IdxState, DbName, Sig}=Args}, From, State) ->
     case ets:lookup(?BY_SIG, {DbName, Sig}) of
         [] ->
-            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}}),
+            {ok, NewState} = maybe_close_idle(State),
+            spawn_link(fun() -> new_index(Args) end),
+            Monitor = couch_index_monitor:spawn_link({DbName, Sig}),
+            ets:insert(?BY_SIG, {{DbName, Sig}, {[From], Monitor}}),
+            {noreply, NewState};
+        [{_, {Waiters, Monitor}}] when is_list(Waiters) ->
+            ets:insert(?BY_SIG, {{DbName, Sig}, {[From | Waiters], Monitor}}),
             {noreply, State};
-        [{_, {Pid, Monitor, _SysOwned}}] when is_pid(Pid) ->
+        [{_, {Pid, Monitor}}] 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, Monitor, SysOwned}}] = ets:lookup(?BY_SIG, {DbName, Sig}),
+    [{_, {Waiters, Monitor}}] = ets:lookup(?BY_SIG, {DbName, Sig}),
     link(Pid),
-    ets:insert(?BY_SIG, {{DbName, Sig}, {Pid, Monitor, SysOwned}}),
+    ets:insert(?BY_SIG, {{DbName, Sig}, {Pid, Monitor}}),
     ets:insert(?BY_PID, {Pid, {DbName, Sig}}),
     ets:insert(?BY_COUNTERS, {{DbName, Sig}, 0}),
     ets:insert(?BY_DB, {DbName, {DDocId, Sig}}),
@@ -225,7 +203,7 @@ handle_call({async_open, {DbName, DDocId, Sig}, {ok, Pid}}, _From, State) ->
         ok = couch_index_monitor:notify(Monitor, Client),
         gen_server:reply(From, {ok, Pid})
     end, Waiters),
-    {reply, ok, opened(State, SysOwned)};
+    {reply, ok, opened(State)};
 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],
@@ -235,12 +213,12 @@ handle_call({async_error, {DbName, _DDocId, Sig}, Error}, {FromPid, _}, State) -
     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({set_soft_max_indexes_open, Max}, _From, State) ->
+    {reply, ok, State#st{soft_max_indexes=Max}};
 handle_call({reset_indexes, DbName}, _From, State) ->
     {reply, ok, reset_indexes(DbName, State)};
 handle_call(open_index_count, _From, State) ->
-    {reply, {State#st.open, State#st.sys_open}, State};
+    {reply, State#st.open, State};
 handle_call(get_server, _From, State) ->
     {reply, State, State}.
 
@@ -251,11 +229,10 @@ handle_cast({reset_indexes, DbName}, State) ->
 handle_info({'EXIT', Pid, Reason}, Server) ->
     case ets:lookup(?BY_PID, Pid) of
         [{_, {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),
-            {noreply, closed(Server, SysOwned)};
+            {noreply, closed(Server)};
         [] when Reason /= normal ->
             exit(Reason);
         _Else ->
@@ -283,8 +260,8 @@ 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("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("couchdb", "soft_max_indexes_open", Max, _, _) when is_list(Max) ->
+    {ok, gen_server:call(?MODULE, {set_soft_max_indexes_open, list_to_integer(Max)})};
 handle_config_change(_, _, _, _, _) ->
     {ok, nil}.
 
@@ -295,7 +272,7 @@ handle_config_terminate(_Server, _Reason, _State) ->
     {ok, couch_index_util:root_dir()}.
 
 
-new_index({Mod, IdxState, DbName, Sig, _SysOwned}) ->
+new_index({Mod, IdxState, DbName, Sig}) ->
     DDocId = Mod:get(idx_name, IdxState),
     case couch_index:start_link({Mod, IdxState}) of
         {ok, Pid} ->
@@ -312,13 +289,13 @@ 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}}, StateAcc) ->
-        [{_, {Pid, Monitor, SysOwned}}] = ets:lookup(?BY_SIG, {DbName, Sig}),
+        [{_, {Pid, Monitor}}] = 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),
-        closed(StateAcc, SysOwned)
+        closed(StateAcc)
     end,
     NewState = lists:foldl(Fun, State, ets:lookup(?BY_DB, DbName)),
     Path = couch_index_util:index_dir("", DbName),
@@ -343,7 +320,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, _Monitor, _SysOwned}}] ->
+            [{_, {IndexPid, _Monitor}}] ->
                 (catch gen_server:cast(IndexPid, ddoc_updated));
             [] ->
                 ok
@@ -354,20 +331,14 @@ 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 opened(#st{}) -> #st{}.
+opened(State) ->
+    State#st{open=State#st.open + 1}.
 
 
--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.
+-spec closed(#st{}) -> #st{}.
+closed(State) ->
+    State#st{open=State#st.open - 1}.
 
 
 incref(Name) ->

http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/blob/78a65a3f/test/couch_index_monitor_tests.erl
----------------------------------------------------------------------
diff --git a/test/couch_index_monitor_tests.erl b/test/couch_index_monitor_tests.erl
index d4fd2c9..9e0d8e0 100644
--- a/test/couch_index_monitor_tests.erl
+++ b/test/couch_index_monitor_tests.erl
@@ -60,10 +60,9 @@ test_basic({Db, DDoc}) ->
         {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}),
+        [{_, {Pid, Monitor}}] = 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})),
@@ -73,7 +72,7 @@ test_basic({Db, DDoc}) ->
         ?assertEqual(2, get_count({DbName, Sig})),
         wait_down(Acq1),
         ?assertEqual(1, get_count({DbName, Sig})),
-        couch_index_monitor:cancel({DbName, Sig}, {self(), Monitor, SysOwned}),
+        couch_index_monitor:cancel({DbName, Sig}, {self(), Monitor}),
         ?assertEqual(0, get_count({DbName, Sig})),
         ?assertEqual(1, length(ets:lookup(?BY_IDLE, {DbName, Sig}))),
         ok


[5/6] couch-index commit: updated refs/heads/8409-view-cache to 5a9f05c

Posted by bb...@apache.org.
Add test for index cache soft limit


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/0ad8d9d6
Tree: http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/tree/0ad8d9d6
Diff: http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/diff/0ad8d9d6

Branch: refs/heads/8409-view-cache
Commit: 0ad8d9d6657c52058d0de35a729a3a27bb689d33
Parents: ada99aa
Author: Benjamin Bastian <be...@gmail.com>
Authored: Thu Mar 2 19:12:24 2017 -0800
Committer: Benjamin Bastian <be...@gmail.com>
Committed: Thu Mar 2 19:12:24 2017 -0800

----------------------------------------------------------------------
 test/couch_index_monitor_tests.erl | 77 +++++++++++++++++++++++++--------
 1 file changed, 59 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/blob/0ad8d9d6/test/couch_index_monitor_tests.erl
----------------------------------------------------------------------
diff --git a/test/couch_index_monitor_tests.erl b/test/couch_index_monitor_tests.erl
index 9e0d8e0..44d1bd3 100644
--- a/test/couch_index_monitor_tests.erl
+++ b/test/couch_index_monitor_tests.erl
@@ -16,24 +16,25 @@
 -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);}">>}]}
-    }]}
-}]}).
+ddoc_gen(I) ->
+    IBin = integer_to_binary(I),
+    #doc{
+        id = <<"_design/mydb_", IBin/binary>>,
+        body = {[
+            {<<"views">>, {[
+                {<<"test_view_", IBin/binary>>, {[
+                    {<<"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}.
+    ok.
 
 
-teardown({Db, _DDoc}) ->
-    couch_db:close(Db),
+teardown(_) ->
     ok.
 
 
@@ -47,18 +48,21 @@ monitoring_test_() ->
                 foreach,
                 fun setup/0, fun teardown/1,
                 [
-                    fun test_basic/1
+                    fun test_basic/1,
+                    fun test_soft_max/1
                 ]
             }
         }
     }.
 
-
-test_basic({Db, DDoc}) ->
+test_basic(_) ->
     ?_test(begin
-        {ok, Pid} = couch_index_server:get_index(couch_mrview_index, Db#db.name, DDoc#doc.id),
+        DbName = ?tempdb(),
+        {ok, Db} = couch_db:create(DbName, [?ADMIN_CTX]),
+        DDoc = ddoc_gen(1),
+        {ok, _} = couch_db:update_doc(Db, DDoc, [?ADMIN_CTX]),
+        {ok, Pid} = couch_index_server:get_index(couch_mrview_index, DbName, 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}}] = ets:lookup(?BY_SIG, {DbName, Sig}),
         ?assert(is_pid(Pid)),
@@ -75,6 +79,41 @@ test_basic({Db, DDoc}) ->
         couch_index_monitor:cancel({DbName, Sig}, {self(), Monitor}),
         ?assertEqual(0, get_count({DbName, Sig})),
         ?assertEqual(1, length(ets:lookup(?BY_IDLE, {DbName, Sig}))),
+        couch_db:close(Db),
+        couch_server:delete(DbName, [?ADMIN_CTX]),
+        ok
+    end).
+
+
+test_soft_max(_) ->
+    ?_test(begin
+        DbName = ?tempdb(),
+        {ok, Db} = couch_db:create(DbName, [?ADMIN_CTX]),
+        lists:foreach(fun(I) ->
+            {ok, _} = couch_db:update_doc(Db, ddoc_gen(I), [?ADMIN_CTX])
+        end, lists:seq(1, 10)),
+        config:set("couchdb", "soft_max_indexes_open", "5"),
+        ?assertEqual(0, length(ets:tab2list(?BY_SIG))),
+        ?assertEqual(0, gen_server:call(couch_index_server, open_index_count)),
+        Acqs = lists:map(fun(I) ->
+            DDoc = ddoc_gen(I),
+            Acq = spawn_monitor_acquirer(DbName, DDoc#doc.id),
+            ?assertEqual(I, length(ets:tab2list(?BY_SIG))),
+            ?assertEqual(I, gen_server:call(couch_index_server, open_index_count)),
+            Acq
+        end, lists:seq(1, 10)),
+        {First5, Last5} = lists:split(5, Acqs),
+        lists:foldl(fun(Acq, Acc) ->
+            wait_down(Acq),
+            couch_index_server ! close_idle,
+            sys:get_status(couch_index_server), % wait until close_idle processed
+            ?assertEqual(10-Acc, gen_server:call(couch_index_server, open_index_count)),
+            Acc+1
+        end, 1, Last5),
+        lists:foreach(fun(Acq) ->
+            wait_down(Acq),
+            ?assertEqual(5, gen_server:call(couch_index_server, open_index_count))
+        end, First5),
         ok
     end).
 
@@ -82,6 +121,8 @@ test_basic({Db, DDoc}) ->
 wait_down({Pid, Ref}) ->
     Pid ! close,
     receive {'DOWN', Ref, process, Pid, normal} ->
+        % this is a hack to wait for couch_index_server to process DOWN message
+        sys:get_status(couch_index_server),
         ok
     end.
 


[2/6] couch-index commit: updated refs/heads/8409-view-cache to 5a9f05c

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/19a3a497
Tree: http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/tree/19a3a497
Diff: http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/diff/19a3a497

Branch: refs/heads/8409-view-cache
Commit: 19a3a4977761bebf510c24cb4b680b2167f7f6ec
Parents: 4f184e4
Author: Benjamin Bastian <be...@gmail.com>
Authored: Wed Mar 1 16:26:50 2017 -0800
Committer: Benjamin Bastian <be...@gmail.com>
Committed: Thu Mar 2 10:07:12 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/19a3a497/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.


[6/6] couch-index commit: updated refs/heads/8409-view-cache to 5a9f05c

Posted by bb...@apache.org.
[squash] change couch_index_monitor to be gen_server


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/5a9f05c9
Tree: http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/tree/5a9f05c9
Diff: http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/diff/5a9f05c9

Branch: refs/heads/8409-view-cache
Commit: 5a9f05c9f6ad8e8240cf4a4c453e89db5abcfcd7
Parents: 0ad8d9d
Author: Benjamin Bastian <be...@gmail.com>
Authored: Wed Mar 8 12:39:18 2017 -0800
Committer: Benjamin Bastian <be...@gmail.com>
Committed: Wed Mar 8 12:39:18 2017 -0800

----------------------------------------------------------------------
 src/couch_index_monitor.erl | 82 ++++++++++++++++++++--------------------
 src/couch_index_server.erl  |  2 +-
 2 files changed, 43 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/couchdb-couch-index/blob/5a9f05c9/src/couch_index_monitor.erl
----------------------------------------------------------------------
diff --git a/src/couch_index_monitor.erl b/src/couch_index_monitor.erl
index fc2fef1..a881c7b 100644
--- a/src/couch_index_monitor.erl
+++ b/src/couch_index_monitor.erl
@@ -12,9 +12,11 @@
 
 -module(couch_index_monitor).
 
+-behaviour(gen_server).
+
 
 -export([
-    spawn_link/1,
+    start_link/1,
     close/1,
     set_pid/2,
 
@@ -24,7 +26,12 @@
 ]).
 
 -export([
-    init/1
+    init/1,
+    handle_call/3,
+    handle_cast/2,
+    handle_info/2,
+    code_change/3,
+    terminate/2
 ]).
 
 
@@ -40,18 +47,16 @@
 }).
 
 
-spawn_link(Name) ->
-    erlang:spawn_link(?MODULE, init, [Name]).
+start_link(Name) ->
+    gen_server:start_link(?MODULE, [Name], []).
 
 
 close(Monitor) ->
-    Monitor ! exit,
-    ok.
+    gen_server:cast(Monitor, exit).
 
 
 set_pid(Monitor, Pid) ->
-    Monitor ! {set_pid, Pid},
-    ok.
+    gen_server:cast(Monitor, {set_pid, Pid}).
 
 
 notify(Monitor) ->
@@ -59,8 +64,7 @@ notify(Monitor) ->
 
 
 notify(Monitor, Client) when is_pid(Client) ->
-    Monitor ! {notify, Client},
-    ok;
+    gen_server:cast(Monitor, {notify, Client});
 
 notify(Monitor, {Client, _}) when is_pid(Client) ->
     notify(Monitor, Client).
@@ -68,7 +72,7 @@ notify(Monitor, {Client, _}) when is_pid(Client) ->
 
 cancel(Name, {Client, Monitor})
         when Client == self(), is_pid(Monitor) ->
-    Monitor ! {cancel, self()},
+    gen_server:cast(Monitor, {cancel, self()}),
     case (catch ets:update_counter(?BY_COUNTERS, Name, -1)) of
         0 ->
             true = ets:insert(?BY_IDLE, {Name}),
@@ -78,28 +82,32 @@ cancel(Name, {Client, Monitor})
     end.
 
 
-init(Name) ->
+init([Name]) ->
     {ok, CRefs} = khash:new(),
-    loop(#st{
+    {ok, #st{
         name = Name,
         ref = undefined,
         client_refs = CRefs,
         closing = false
-    }).
+    }}.
+
+
+handle_call(Msg, From, St) ->
+    {stop, {unknown_call, Msg, From}, St}.
 
 
-handle_info(exit, St) ->
-    {stop, normal, St};
+handle_cast(exit, St) ->
+    {stop, shutdown, St};
 
-handle_info({set_pid, Pid}, #st{ref = undefined} = St) ->
+handle_cast({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) ->
+handle_cast({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_cast({set_pid, Pid}, St#st{ref = undefined});
 
-handle_info({notify, Client}, St) when is_pid(Client) ->
+handle_cast({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});
@@ -118,7 +126,7 @@ handle_info({notify, Client}, St) when is_pid(Client) ->
     end,
     {noreply, St};
 
-handle_info({cancel, Client}, St) when is_pid(Client) ->
+handle_cast({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]),
@@ -129,8 +137,12 @@ handle_info({cancel, Client}, St) when is_pid(Client) ->
     end,
     {noreply, St};
 
+handle_cast(Msg, St) ->
+    {stop, {unknown_cast, Msg}, St}.
+
+
 handle_info({'DOWN', Ref, process, _, _}, #st{ref = Ref} = St) ->
-    {stop, normal, St};
+    {stop, shutdown, St};
 
 handle_info({'DOWN', _Ref, process, Pid, _Reason}, St) ->
     #st{name=Name} = St,
@@ -149,6 +161,14 @@ handle_info(Msg, St) ->
     {stop, {bad_info, Msg}, St}.
 
 
+code_change(_OldVsn, St, _Extra) ->
+    {ok, St}.
+
+
+terminate(_Reason, _St) ->
+    ok.
+
+
 maybe_set_idle(St) ->
     case khash:size(St#st.client_refs) of
         0 ->
@@ -158,21 +178,3 @@ maybe_set_idle(St) ->
             % 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/5a9f05c9/src/couch_index_server.erl
----------------------------------------------------------------------
diff --git a/src/couch_index_server.erl b/src/couch_index_server.erl
index f191700..981eb96 100644
--- a/src/couch_index_server.erl
+++ b/src/couch_index_server.erl
@@ -185,7 +185,7 @@ handle_call({get_index, {_Mod, _IdxState, DbName, Sig}=Args}, From, State) ->
         [] ->
             {ok, NewState} = maybe_close_idle(State),
             spawn_link(fun() -> new_index(Args) end),
-            Monitor = couch_index_monitor:spawn_link({DbName, Sig}),
+            {ok, Monitor} = couch_index_monitor:start_link({DbName, Sig}),
             ets:insert(?BY_SIG, {{DbName, Sig}, {[From], Monitor}}),
             {noreply, NewState};
         [{_, {Waiters, Monitor}}] when is_list(Waiters) ->