summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorRussell Branca <chewbranca@gmail.com>2017-10-10 17:54:14 -0700
committerJoan Touzet <wohali@users.noreply.github.com>2017-10-10 20:54:14 -0400
commitd71ce9f8083a0c2cc5f81c353946f19589a5cba4 (patch)
tree5b639ca6d18ee28aca94b78d655aad7e97d7a458
parenta0e08857dbfc4a5d941b7ec2e90457df1c87c38e (diff)
downloadcouchdb-d71ce9f8083a0c2cc5f81c353946f19589a5cba4.tar.gz
Make couch_peruser a proper Erlang app (#756)
* Make couch_peruser a proper Erlang app * Start and stop couch_peruser in the test suite * feat: mango test runner: do not rely on timeout for CouchDB start alone On slow build nodes, 10 seconds might not be enough of a wait. * Ensure a user creation is handlined on one node only This patch makes use of the mechanism that ensures that replications are only run on one node. When the cluster has nodes added/removed all changes listeners are restarted. * track cluster state in gen_server state and get notfied from mem3 directly * move couch_replication_clustering:owner/3 to mem3.erl * remove reliance on couch_replicator_clustering, handle cluster state internally * make sure peruser listeners are only initialised once per node * add type specs * fix tests * simplify couch_persuer.app definition * add registered modules * remove leftover code from olde notification system * s/clusterState/state/ && s/state/changes_state/ * s,init/0,init_state/0, * move function declaration around for internal consistency * whitespace * update README * document ini entries * unlink changes listeners before exiting them so we survive * fix state call * fix style * fix state * whitespace and more state fixes * 80 cols Closes #749
-rw-r--r--rel/overlay/etc/default.ini5
-rw-r--r--src/couch_peruser/README.md19
-rw-r--r--src/couch_peruser/src/couch_peruser.app.src6
-rw-r--r--src/couch_peruser/src/couch_peruser.erl312
-rw-r--r--src/couch_peruser/src/couch_peruser_app.erl26
-rw-r--r--src/couch_peruser/src/couch_peruser_sup.erl29
-rw-r--r--src/couch_peruser/test/couch_peruser_test.erl19
-rw-r--r--src/couch_replicator/src/couch_replicator.erl2
-rw-r--r--src/couch_replicator/src/couch_replicator_clustering.erl10
-rw-r--r--src/mem3/src/mem3.erl8
-rwxr-xr-xtest/build/test-run-couch-for-mango.sh13
11 files changed, 353 insertions, 96 deletions
diff --git a/rel/overlay/etc/default.ini b/rel/overlay/etc/default.ini
index 122853542..4e61deb60 100644
--- a/rel/overlay/etc/default.ini
+++ b/rel/overlay/etc/default.ini
@@ -88,6 +88,10 @@ enable = false
; If set to true and a user is deleted, the respective database gets
; deleted as well.
delete_dbs = false
+; Wait this many seconds after startup before attaching changes listeners
+; cluster_start_period = 5
+; Re-check cluster state at least every cluster_quiet_period seconds
+; cluster_quiet_period = 60
[httpd]
port = {{backend_port}}
@@ -254,7 +258,6 @@ uuids={couch_uuids, start, []}
auth_cache={couch_auth_cache, start_link, []}
os_daemons={couch_os_daemons, start_link, []}
compaction_daemon={couch_compaction_daemon, start_link, []}
-couch_peruser={couch_peruser, start_link, []}
[mango]
; Set to true to disable the "index all fields" text index, which can lead
diff --git a/src/couch_peruser/README.md b/src/couch_peruser/README.md
index 70f8348f5..64a05188c 100644
--- a/src/couch_peruser/README.md
+++ b/src/couch_peruser/README.md
@@ -1,6 +1,6 @@
# couch_peruser [![Build Status](https://travis-ci.org/apache/couchdb-peruser.svg?branch=master)](https://travis-ci.org/apache/couchdb-peruser)
-couch_peruser is a CouchDB daemon that ensures that a private per-user
+couch_peruser is a CouchDB application that ensures that a private per-user
database exists for each document in _users. These databases are
writable only by the corresponding user. Databases are in the form:
@@ -15,3 +15,20 @@ correctly implement in just about any language, especially JavaScript
and Erlang. Other encodings would be possible, but would require
additional client and server-side code to support that encoding. This
is the simplest scheme that is obviously correct.
+
+## Implementation Notes
+
+The module itself is a `gen_server` and it implements the `mem3_cluster`
+behaviour.
+
+In a CouchDB cluster, the module runs on each node in the cluster. On startup,
+it launches a changes listener for each shard of the `authentication_db`
+(`_users`).
+
+In a cluster, when a change notification comes in (after a user doc has been
+created/updated/deleted), each node independently calculates if it should
+handle the notification based on the current list of active nodes in the
+cluster. This ensures that we avoid trying to update the internal `_dbs`
+concurrently and causing conflicts. It also ensures that at least one node
+does handle a notification. The mechanism that handles this does survive
+cluster reconfigurations transparently.
diff --git a/src/couch_peruser/src/couch_peruser.app.src b/src/couch_peruser/src/couch_peruser.app.src
index fb6d45bf1..6cfaf4421 100644
--- a/src/couch_peruser/src/couch_peruser.app.src
+++ b/src/couch_peruser/src/couch_peruser.app.src
@@ -13,6 +13,8 @@
{application, couch_peruser, [
{description, "couch_peruser - maintains per-user databases in CouchDB"},
{vsn, git},
- {registered, []},
- {applications, [kernel, stdlib, config, couch, fabric]}
+ {registered, [couch_peruser, couch_peruser_sup]},
+ {applications, [kernel, stdlib, config, couch, fabric, mem3]},
+ {mod, {couch_peruser_app, []}},
+ {env, []}
]}.
diff --git a/src/couch_peruser/src/couch_peruser.erl b/src/couch_peruser/src/couch_peruser.erl
index 63ef084ce..0c769324a 100644
--- a/src/couch_peruser/src/couch_peruser.erl
+++ b/src/couch_peruser/src/couch_peruser.erl
@@ -12,62 +12,116 @@
-module(couch_peruser).
-behaviour(gen_server).
+-behaviour(mem3_cluster).
-include_lib("couch/include/couch_db.hrl").
-include_lib("mem3/include/mem3.hrl").
--define(USERDB_PREFIX, "userdb-").
-
% gen_server callbacks
-export([start_link/0, init/1, handle_call/3, handle_cast/2, handle_info/2,
terminate/2, code_change/3]).
-export([init_changes_handler/1, changes_handler/3]).
--record(state, {parent, db_name, delete_dbs, changes_pid, changes_ref}).
--record(clusterState, {parent, db_name, delete_dbs, states}).
+% mem3_cluster callbacks
+-export([
+ cluster_stable/1,
+ cluster_unstable/1
+]).
+
+-record(changes_state, {
+ parent :: pid(),
+ db_name :: binary(),
+ delete_dbs :: boolean(),
+ changes_pid :: pid(),
+ changes_ref :: reference()
+}).
+
+-record(state, {
+ parent :: pid(),
+ db_name :: binary(),
+ delete_dbs :: boolean(),
+ states :: list(),
+ mem3_cluster_pid :: pid(),
+ cluster_stable :: boolean()
+}).
+-define(USERDB_PREFIX, "userdb-").
-define(RELISTEN_DELAY, 5000).
+-define(DEFAULT_QUIET_PERIOD, 60). % seconds
+-define(DEFAULT_START_PERIOD, 5). % seconds
-
+%%
+%% Please leave in the commented-out couch_log:debug calls, thanks! — Jan
+%%
+-spec start_link() -> {ok, pid()} | ignore | {error, term()}.
start_link() ->
- gen_server:start_link(?MODULE, [], []).
+ gen_server:start_link({local, ?MODULE}, ?MODULE, [], []).
-init() ->
+-spec init_state() -> #state{}.
+init_state() ->
+ couch_log:debug("peruser: starting on node ~p in pid ~p", [node(), self()]),
case config:get_boolean("couch_peruser", "enable", false) of
false ->
- #clusterState{};
+ couch_log:debug("peruser: disabled on node ~p", [node()]),
+ #state{};
true ->
+ couch_log:debug("peruser: enabled on node ~p", [node()]),
DbName = ?l2b(config:get(
"couch_httpd_auth", "authentication_db", "_users")),
DeleteDbs = config:get_boolean("couch_peruser", "delete_dbs", false),
- ClusterState = #clusterState{
+ % set up cluster-stable listener
+ Period = abs(config:get_integer("couch_peruser", "cluster_quiet_period",
+ ?DEFAULT_QUIET_PERIOD)),
+ StartPeriod = abs(config:get_integer("couch_peruser",
+ "cluster_start_period", ?DEFAULT_START_PERIOD)),
+
+ {ok, Mem3Cluster} = mem3_cluster:start_link(?MODULE, self(), StartPeriod,
+ Period),
+
+ #state{
parent = self(),
db_name = DbName,
- delete_dbs = DeleteDbs
- },
- try
- States = lists:map(fun (A) ->
- S = #state{parent = ClusterState#clusterState.parent,
- db_name = A#shard.name,
- delete_dbs = DeleteDbs},
- {Pid, Ref} = spawn_opt(
- ?MODULE, init_changes_handler, [S], [link, monitor]),
- S#state{changes_pid=Pid, changes_ref=Ref}
- end, mem3:local_shards(DbName)),
-
- ClusterState#clusterState{states = States}
- catch error:database_does_not_exist ->
- couch_log:warning("couch_peruser can't proceed as underlying database (~s) is missing, disables itself.", [DbName]),
- config:set("couch_peruser", "enable", "false", lists:concat([binary_to_list(DbName), " is missing"]))
- end
+ delete_dbs = DeleteDbs,
+ mem3_cluster_pid = Mem3Cluster,
+ cluster_stable = false
+ }
end.
-init_changes_handler(#state{db_name=DbName} = State) ->
+
+-spec start_listening(State :: #state{}) -> #state{} | ok.
+start_listening(#state{states=ChangesStates}=State)
+ when length(ChangesStates) > 0 ->
+ % couch_log:debug("peruser: start_listening() already run on node ~p in pid ~p", [node(), self()]),
+ State;
+start_listening(#state{db_name=DbName, delete_dbs=DeleteDbs} = State) ->
+ % couch_log:debug("peruser: start_listening() on node ~p", [node()]),
+ try
+ States = lists:map(fun (A) ->
+ S = #changes_state{
+ parent = State#state.parent,
+ db_name = A#shard.name,
+ delete_dbs = DeleteDbs
+ },
+ {Pid, Ref} = spawn_opt(
+ ?MODULE, init_changes_handler, [S], [link, monitor]),
+ S#changes_state{changes_pid=Pid, changes_ref=Ref}
+ end, mem3:local_shards(DbName)),
+ % couch_log:debug("peruser: start_listening() States ~p", [States]),
+
+ State#state{states = States, cluster_stable = true}
+ catch error:database_does_not_exist ->
+ couch_log:warning("couch_peruser can't proceed as underlying database (~s) is missing, disables itself.", [DbName]),
+ config:set("couch_peruser", "enable", "false", lists:concat([binary_to_list(DbName), " is missing"]))
+ end.
+
+-spec init_changes_handler(ChangesState :: #changes_state{}) -> ok.
+init_changes_handler(#changes_state{db_name=DbName} = ChangesState) ->
+ % couch_log:debug("peruser: init_changes_handler() on DbName ~p", [DbName]),
try
{ok, Db} = couch_db:open_int(DbName, [?ADMIN_CTX, sys_db]),
- FunAcc = {fun ?MODULE:changes_handler/3, State},
+ FunAcc = {fun ?MODULE:changes_handler/3, ChangesState},
(couch_changes:handle_db_changes(
#changes_args{feed="continuous", timeout=infinity},
{json_req, null},
@@ -76,31 +130,78 @@ init_changes_handler(#state{db_name=DbName} = State) ->
ok
end.
-changes_handler({change, {Doc}, _Prepend}, _ResType, State=#state{}) ->
+-type db_change() :: {atom(), tuple(), binary()}.
+-spec changes_handler(
+ Change :: db_change(),
+ ResultType :: any(),
+ ChangesState :: #changes_state{}) -> #changes_state{}.
+changes_handler(
+ {change, {Doc}, _Prepend},
+ _ResType,
+ ChangesState=#changes_state{db_name=DbName}) ->
+ % couch_log:debug("peruser: changes_handler() on DbName/Doc ~p/~p", [DbName, Doc]),
+
case couch_util:get_value(<<"id">>, Doc) of
- <<"org.couchdb.user:",User/binary>> ->
- case couch_util:get_value(<<"deleted">>, Doc, false) of
- false ->
- UserDb = ensure_user_db(User),
- ok = ensure_security(User, UserDb, fun add_user/3),
- State;
+ <<"org.couchdb.user:",User/binary>> = DocId ->
+ case should_handle_doc(DbName, DocId) of
true ->
- case State#state.delete_dbs of
- true ->
- _UserDb = delete_user_db(User),
- State;
+ case couch_util:get_value(<<"deleted">>, Doc, false) of
false ->
- UserDb = user_db_name(User),
- ok = ensure_security(User, UserDb, fun remove_user/3),
- State
- end
+ UserDb = ensure_user_db(User),
+ ok = ensure_security(User, UserDb, fun add_user/3),
+ ChangesState;
+ true ->
+ case ChangesState#changes_state.delete_dbs of
+ true ->
+ _UserDb = delete_user_db(User),
+ ChangesState;
+ false ->
+ UserDb = user_db_name(User),
+ ok = ensure_security(User, UserDb, fun remove_user/3),
+ ChangesState
+ end
+ end;
+ false ->
+ ChangesState
end;
_ ->
- State
+ ChangesState
end;
-changes_handler(_Event, _ResType, State) ->
- State.
+changes_handler(_Event, _ResType, ChangesState) ->
+ ChangesState.
+
+-spec should_handle_doc(ShardName :: binary(), DocId::binary()) -> boolean().
+should_handle_doc(ShardName, DocId) ->
+ case is_stable() of
+ false ->
+ % when the cluster is unstable, we have already stopped all Listeners
+ % the next stable event will restart all listeners and pick up this
+ % doc change
+ couch_log:debug("peruser: skipping, cluster unstable ~s/~s",
+ [ShardName, DocId]),
+ false;
+ true ->
+ should_handle_doc_int(ShardName, DocId)
+ end.
+-spec should_handle_doc_int(
+ ShardName :: binary(),
+ DocId :: binary()) -> boolean().
+should_handle_doc_int(ShardName, DocId) ->
+ DbName = mem3:dbname(ShardName),
+ Live = [erlang:node() | erlang:nodes()],
+ Shards = mem3:shards(DbName, DocId),
+ Nodes = [N || #shard{node=N} <- Shards, lists:member(N, Live)],
+ case mem3:owner(DbName, DocId, Nodes) of
+ ThisNode when ThisNode =:= node() ->
+ couch_log:debug("peruser: handling ~s/~s", [DbName, DocId]),
+ true; % do the database action
+ _OtherNode ->
+ couch_log:debug("peruser: skipping ~s/~s", [DbName, DocId]),
+ false
+ end.
+
+-spec delete_user_db(User :: binary()) -> binary().
delete_user_db(User) ->
UserDb = user_db_name(User),
try
@@ -113,18 +214,24 @@ delete_user_db(User) ->
end,
UserDb.
+-spec ensure_user_db(User :: binary()) -> binary().
ensure_user_db(User) ->
UserDb = user_db_name(User),
try
{ok, _DbInfo} = fabric:get_db_info(UserDb)
catch error:database_does_not_exist ->
case fabric:create_db(UserDb, [?ADMIN_CTX]) of
+ {error, file_exists} -> ok;
ok -> ok;
accepted -> ok
end
end,
UserDb.
+-spec add_user(
+ User :: binary(),
+ Properties :: tuple(),
+ Acc :: tuple()) -> tuple().
add_user(User, Prop, {Modified, SecProps}) ->
{PropValue} = couch_util:get_value(Prop, SecProps, {[]}),
Names = couch_util:get_value(<<"names">>, PropValue, []),
@@ -141,6 +248,10 @@ add_user(User, Prop, {Modified, SecProps}) ->
{<<"names">>, [User | Names]})}})}
end.
+-spec remove_user(
+ User :: binary(),
+ Properties :: tuple(),
+ Acc :: tuple()) -> tuple().
remove_user(User, Prop, {Modified, SecProps}) ->
{PropValue} = couch_util:get_value(Prop, SecProps, {[]}),
Names = couch_util:get_value(<<"names">>, PropValue, []),
@@ -157,58 +268,108 @@ remove_user(User, Prop, {Modified, SecProps}) ->
{<<"names">>, lists:delete(User, Names)})}})}
end.
+-spec ensure_security(
+ User :: binary(),
+ UserDb :: binary(),
+ TransformFun :: fun()) -> ok.
ensure_security(User, UserDb, TransformFun) ->
- {ok, Shards} = fabric:get_all_security(UserDb, [?ADMIN_CTX]),
- {_ShardInfo, {SecProps}} = hd(Shards),
- % assert that shards have the same security object
- true = lists:all(fun ({_, {SecProps1}}) ->
- SecProps =:= SecProps1
- end, Shards),
- case lists:foldl(
- fun (Prop, SAcc) -> TransformFun(User, Prop, SAcc) end,
- {false, SecProps},
- [<<"admins">>, <<"members">>]) of
- {false, _} ->
- ok;
- {true, SecProps1} ->
- ok = fabric:set_security(UserDb, {SecProps1}, [?ADMIN_CTX])
+ case fabric:get_all_security(UserDb, [?ADMIN_CTX]) of
+ {error, no_majority} ->
+ % TODO: make sure this is still true: single node, ignore
+ ok;
+ {ok, Shards} ->
+ {_ShardInfo, {SecProps}} = hd(Shards),
+ % assert that shards have the same security object
+ true = lists:all(fun ({_, {SecProps1}}) ->
+ SecProps =:= SecProps1
+ end, Shards),
+ case lists:foldl(
+ fun (Prop, SAcc) -> TransformFun(User, Prop, SAcc) end,
+ {false, SecProps},
+ [<<"admins">>, <<"members">>]) of
+ {false, _} ->
+ ok;
+ {true, SecProps1} ->
+ ok = fabric:set_security(UserDb, {SecProps1}, [?ADMIN_CTX])
+ end
end.
+-spec user_db_name(User :: binary()) -> binary().
user_db_name(User) ->
HexUser = list_to_binary(
[string:to_lower(integer_to_list(X, 16)) || <<X>> <= User]),
<<?USERDB_PREFIX,HexUser/binary>>.
+-spec exit_changes(State :: #state{}) -> ok.
+exit_changes(State) ->
+ lists:foreach(fun (ChangesState) ->
+ demonitor(ChangesState#changes_state.changes_ref, [flush]),
+ unlink(ChangesState#changes_state.changes_pid),
+ exit(ChangesState#changes_state.changes_pid, kill)
+ end, State#state.states).
-%% gen_server callbacks
+-spec is_stable() -> true | false.
+is_stable() ->
+ gen_server:call(?MODULE, is_stable).
+
+-spec subscribe_for_changes() -> ok.
+subscribe_for_changes() ->
+ config:subscribe_for_changes([
+ {"couch_httpd_auth", "authentication_db"},
+ "couch_peruser"
+ ]).
+
+% Mem3 cluster callbacks
+
+% TODO: find out what type Server is
+-spec cluster_unstable(Server :: any()) -> any().
+cluster_unstable(Server) ->
+ gen_server:cast(Server, cluster_unstable),
+ Server.
+% TODO: find out what type Server is
+-spec cluster_stable(Server :: any()) -> any().
+cluster_stable(Server) ->
+ gen_server:cast(Server, cluster_stable),
+ Server.
+
+%% gen_server callbacks
+-spec init(Options :: list()) -> {ok, #state{}}.
init([]) ->
ok = subscribe_for_changes(),
- {ok, init()}.
+ {ok, init_state()}.
+handle_call(is_stable, _From, #state{cluster_stable = IsStable} = State) ->
+ {reply, IsStable, State};
handle_call(_Msg, _From, State) ->
{reply, error, State}.
-handle_cast(update_config, ClusterState) when ClusterState#clusterState.states =/= undefined ->
- lists:foreach(fun (State) ->
- demonitor(State#state.changes_ref, [flush]),
- exit(State#state.changes_pid, kill)
- end, ClusterState#clusterState.states),
-
- {noreply, init()};
+handle_cast(update_config, State) when State#state.states =/= undefined ->
+ exit_changes(State),
+ {noreply, init_state()};
handle_cast(update_config, _) ->
- {noreply, init()};
+ {noreply, init_state()};
handle_cast(stop, State) ->
{stop, normal, State};
+handle_cast(cluster_unstable, State) when State#state.states =/= undefined ->
+ exit_changes(State),
+ {noreply, init_state()};
+handle_cast(cluster_unstable, _) ->
+ {noreply, init_state()};
+handle_cast(cluster_stable, State) ->
+ {noreply, start_listening(State)};
handle_cast(_Msg, State) ->
{noreply, State}.
-handle_info({'DOWN', Ref, _, _, _Reason}, #state{changes_ref=Ref} = State) ->
+handle_info({'DOWN', _Ref, _, _, _Reason}, State) ->
{stop, normal, State};
handle_info({config_change, "couch_peruser", _, _, _}, State) ->
handle_cast(update_config, State);
-handle_info({config_change, "couch_httpd_auth", "authentication_db", _, _}, State) ->
+handle_info({
+ config_change,
+ "couch_httpd_auth",
+ "authentication_db", _, _}, State) ->
handle_cast(update_config, State);
handle_info({gen_event_EXIT, _Handler, _Reason}, State) ->
erlang:send_after(?RELISTEN_DELAY, self(), restart_config_listener),
@@ -222,13 +383,6 @@ handle_info(restart_config_listener, State) ->
handle_info(_Msg, State) ->
{noreply, State}.
-subscribe_for_changes() ->
- config:subscribe_for_changes([
- {"couch_httpd_auth", "authentication_db"},
- "couch_peruser"
- ]).
-
-
terminate(_Reason, _State) ->
%% Everything should be linked or monitored, let nature
%% take its course.
diff --git a/src/couch_peruser/src/couch_peruser_app.erl b/src/couch_peruser/src/couch_peruser_app.erl
new file mode 100644
index 000000000..770c08237
--- /dev/null
+++ b/src/couch_peruser/src/couch_peruser_app.erl
@@ -0,0 +1,26 @@
+% 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_peruser_app).
+
+-behaviour(application).
+
+-export([start/2, stop/1]).
+
+
+start(_Type, _StartArgs) ->
+ couch_peruser_sup:start_link().
+
+
+stop(_State) ->
+ ok.
+
diff --git a/src/couch_peruser/src/couch_peruser_sup.erl b/src/couch_peruser/src/couch_peruser_sup.erl
new file mode 100644
index 000000000..b89a36324
--- /dev/null
+++ b/src/couch_peruser/src/couch_peruser_sup.erl
@@ -0,0 +1,29 @@
+% 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_peruser_sup).
+
+-behaviour(supervisor).
+
+-export([start_link/0, init/1]).
+
+%% Helper macro for declaring children of supervisor
+-define(CHILD(I, Type), {I, {I, start_link, []}, permanent, 5000, Type, [I]}).
+
+
+start_link() ->
+ supervisor:start_link({local, ?MODULE}, ?MODULE, []).
+
+
+init([]) ->
+ {ok, { {one_for_one, 5, 10}, [?CHILD(couch_peruser, worker)]}}.
+
diff --git a/src/couch_peruser/test/couch_peruser_test.erl b/src/couch_peruser/test/couch_peruser_test.erl
index c6fde034e..726b2dbee 100644
--- a/src/couch_peruser/test/couch_peruser_test.erl
+++ b/src/couch_peruser/test/couch_peruser_test.erl
@@ -20,12 +20,14 @@
setup_all() ->
TestCtx = test_util:start_couch([chttpd]),
+ ok = application:start(couch_peruser),
Hashed = couch_passwords:hash_admin_password(?ADMIN_PASSWORD),
ok = config:set("admins", ?ADMIN_USERNAME, ?b2l(Hashed), _Persist=false),
TestCtx.
teardown_all(TestCtx) ->
config:delete("admins", ?ADMIN_USERNAME),
+ ok = application:stop(couch_peruser),
test_util:stop_couch(TestCtx).
setup() ->
@@ -33,13 +35,22 @@ setup() ->
do_request(put, get_base_url() ++ "/" ++ ?b2l(TestAuthDb)),
do_request(put, get_cluster_base_url() ++ "/" ++ ?b2l(TestAuthDb)),
set_config("couch_httpd_auth", "authentication_db", ?b2l(TestAuthDb)),
+ set_config("couch_peruser", "cluster_quiet_period", "1"),
+ set_config("couch_peruser", "cluster_start_period", "1"),
set_config("couch_peruser", "enable", "true"),
+ set_config("cluster", "n", "1"),
+ set_config("log", "level", "debug"),
+ timer:sleep(6000),
TestAuthDb.
teardown(TestAuthDb) ->
set_config("couch_peruser", "enable", "false"),
set_config("couch_peruser", "delete_dbs", "false"),
set_config("couch_httpd_auth", "authentication_db", "_users"),
+ set_config("couch_peruser", "cluster_quiet_period", "60"),
+ set_config("couch_peruser", "cluster_start_period", "5"),
+ set_config("cluster", "n", "3"),
+ set_config("log", "level", "info"),
do_request(delete, get_cluster_base_url() ++ "/" ++ ?b2l(TestAuthDb)),
do_request(delete, get_base_url() ++ "/" ++ ?b2l(TestAuthDb)),
lists:foreach(fun (DbName) ->
@@ -151,8 +162,10 @@ should_delete_user_db(TestAuthDb) ->
UserDbName = <<"userdb-626172">>,
set_config("couch_peruser", "delete_dbs", "true"),
create_user(TestAuthDb, User),
+ timer:sleep(2000),
?assert(lists:member(UserDbName, all_dbs())),
delete_user(TestAuthDb, User),
+ timer:sleep(2000),
?_assert(not lists:member(UserDbName, all_dbs())).
should_reflect_config_changes(TestAuthDb) ->
@@ -160,20 +173,26 @@ should_reflect_config_changes(TestAuthDb) ->
UserDbName = <<"userdb-62617a">>,
set_config("couch_peruser", "delete_dbs", "true"),
create_user(TestAuthDb, User),
+ timer:sleep(2000),
?assert(lists:member(UserDbName, all_dbs())),
delete_user(TestAuthDb, User),
+ timer:sleep(2000),
?assert(not lists:member(UserDbName, all_dbs())),
create_user(TestAuthDb, User),
+ timer:sleep(2000),
?assert(lists:member(UserDbName, all_dbs())),
set_config("couch_peruser", "delete_dbs", "false"),
delete_user(TestAuthDb, User),
+ timer:sleep(2000),
?assert(lists:member(UserDbName, all_dbs())),
create_user(TestAuthDb, User),
set_config("couch_peruser", "delete_dbs", "true"),
delete_user(TestAuthDb, User),
+ timer:sleep(2000),
?assert(not lists:member(UserDbName, all_dbs())),
set_config("couch_peruser", "enable", "false"),
create_user(TestAuthDb, User),
+ timer:sleep(2000),
?_assert(not lists:member(UserDbName, all_dbs())).
should_add_user_to_db_admins(TestAuthDb) ->
diff --git a/src/couch_replicator/src/couch_replicator.erl b/src/couch_replicator/src/couch_replicator.erl
index c67b37d19..8b7cd5cb1 100644
--- a/src/couch_replicator/src/couch_replicator.erl
+++ b/src/couch_replicator/src/couch_replicator.erl
@@ -184,7 +184,7 @@ active_doc(DbName, DocId) ->
Live = [node() | nodes()],
Nodes = lists:usort([N || #shard{node=N} <- Shards,
lists:member(N, Live)]),
- Owner = couch_replicator_clustering:owner(DbName, DocId, Nodes),
+ Owner = mem3:owner(DbName, DocId, Nodes),
case active_doc_rpc(DbName, DocId, [Owner]) of
{ok, DocInfo} ->
{ok, DocInfo};
diff --git a/src/couch_replicator/src/couch_replicator_clustering.erl b/src/couch_replicator/src/couch_replicator_clustering.erl
index ed01465d5..3d5229b9f 100644
--- a/src/couch_replicator/src/couch_replicator_clustering.erl
+++ b/src/couch_replicator/src/couch_replicator_clustering.erl
@@ -45,7 +45,6 @@
-export([
owner/2,
- owner/3,
is_stable/0,
link_cluster_event_listener/3
]).
@@ -96,13 +95,6 @@ owner(_DbName, _DocId) ->
node().
-% Direct calculation of node membership. This is the algorithm part. It
-% doesn't read the shard map, just picks owner based on a hash.
--spec owner(binary(), binary(), [node()]) -> node().
-owner(DbName, DocId, Nodes) ->
- hd(mem3_util:rotate_list({DbName, DocId}, lists:usort(Nodes))).
-
-
-spec is_stable() -> true | false.
is_stable() ->
gen_server:call(?MODULE, is_stable).
@@ -200,4 +192,4 @@ owner_int(ShardName, DocId) ->
Live = [node() | nodes()],
Shards = mem3:shards(DbName, DocId),
Nodes = [N || #shard{node=N} <- Shards, lists:member(N, Live)],
- owner(DbName, DocId, Nodes).
+ mem3:owner(DbName, DocId, Nodes).
diff --git a/src/mem3/src/mem3.erl b/src/mem3/src/mem3.erl
index e2cbb2ec6..047154af8 100644
--- a/src/mem3/src/mem3.erl
+++ b/src/mem3/src/mem3.erl
@@ -19,7 +19,7 @@
-export([compare_nodelists/0, compare_shards/1]).
-export([quorum/1, group_by_proximity/1]).
-export([live_shards/2]).
--export([belongs/2]).
+-export([belongs/2, owner/3]).
-export([get_placement/1]).
%% For mem3 use only.
@@ -311,6 +311,12 @@ name(#shard{name=Name}) ->
name(#ordered_shard{name=Name}) ->
Name.
+% Direct calculation of node membership. This is the algorithm part. It
+% doesn't read the shard map, just picks owner based on a hash.
+-spec owner(binary(), binary(), [node()]) -> node().
+owner(DbName, DocId, Nodes) ->
+ hd(mem3_util:rotate_list({DbName, DocId}, lists:usort(Nodes))).
+
-ifdef(TEST).
diff --git a/test/build/test-run-couch-for-mango.sh b/test/build/test-run-couch-for-mango.sh
index 6034a794c..0597a8fca 100755
--- a/test/build/test-run-couch-for-mango.sh
+++ b/test/build/test-run-couch-for-mango.sh
@@ -13,8 +13,17 @@
./dev/run -n 1 --admin=testuser:testpass &
export SERVER_PID=$!
-sleep 10
-curl http://dev:15984
+
+COUCH_STARTED=-1
+while ( [ $COUCH_STARTED -ne 0 ] ); do
+ curl -s http://127.0.0.1:15984
+ COUCH_STARTED=$?
+ if [ $COUCH_STARTED -ne 0 ]; then
+ # do not wait another 5 seconds if couch started now
+ sleep 5
+ fi
+done
+
cd src/mango/
nosetests