Merge branch '3096-fix-config-listener-accumulation'
diff --git a/src/config.erl b/src/config.erl
index 5c9ac6f..c0d0446 100644
--- a/src/config.erl
+++ b/src/config.erl
@@ -183,7 +183,8 @@
 
 
 listen_for_changes(CallbackModule, InitialState) ->
-    gen_server:call(?MODULE, {listen_for_changes, CallbackModule, InitialState}).
+    config_listener_mon:subscribe(CallbackModule, InitialState).
+
 
 init(IniFiles) ->
     ets:new(?MODULE, [named_table, set, protected, {read_concurrency, true}]),
@@ -258,26 +259,14 @@
                 ets:delete(?MODULE, K)
         end
     end, nil, ?MODULE),
-    {reply, ok, Config};
-handle_call({listen_for_changes, CallbackModule, InitialState},
-        {Subscriber, _}, Config) ->
-    Reply = config_listener:start(CallbackModule, {Subscriber, InitialState}),
-    {reply, Reply, Config}.
+    {reply, ok, Config}.
 
 handle_cast(stop, State) ->
     {stop, normal, State};
 handle_cast(_Msg, State) ->
     {noreply, State}.
 
-handle_info({gen_event_EXIT, {config_listener, Module}, shutdown}, State)  ->
-    couch_log:notice("config_listener(~p) stopped with reason: shutdown~n", [Module]),
-    {noreply, State};
-handle_info({gen_event_EXIT, {config_listener, Module}, normal}, State)  ->
-    couch_log:info("config_listener(~p) stopped with reason: shutdown~n", [Module]),
-    {noreply, State};
-handle_info({gen_event_EXIT, {config_listener, Module}, Reason}, State) ->
-    couch_log:error("config_listener(~p) stopped with reason: ~p~n", [Module, Reason]),
-    {noreply, State};
+
 handle_info(Info, State) ->
     couch_log:error("config:handle_info Info: ~p~n", [Info]),
     {noreply, State}.
diff --git a/src/config_listener_mon.erl b/src/config_listener_mon.erl
new file mode 100644
index 0000000..70c2707
--- /dev/null
+++ b/src/config_listener_mon.erl
@@ -0,0 +1,84 @@
+% 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(config_listener_mon).
+-behaviour(gen_server).
+-vsn(1).
+
+
+-export([
+    subscribe/2
+]).
+
+
+-export([
+    init/1,
+    terminate/2,
+    handle_call/3,
+    handle_cast/2,
+    handle_info/2,
+    code_change/3
+]).
+
+
+-record(st, {
+    pid,
+    ref
+}).
+
+
+subscribe(Module, InitSt) ->
+    proc_lib:start(?MODULE, init, [{self(), Module, InitSt}]).
+
+
+init({Pid, Mod, InitSt}) ->
+    Ref = erlang:monitor(process, Pid),
+    case config_listener:start(Mod, {Mod, Pid}, {Pid, InitSt}) of
+        ok ->
+            proc_lib:init_ack(ok),
+            gen_server:enter_loop(?MODULE, [], #st{pid = Pid, ref = Ref});
+        Else ->
+            proc_lib:init_ack(Else)
+    end.
+
+
+terminate(_Reason, _St) ->
+    ok.
+
+
+handle_call(_Message, _From, St) ->
+    {reply, ignored, St}.
+
+
+handle_cast(_Message, St) ->
+    {noreply, St}.
+
+
+handle_info({'DOWN', Ref, _, _, _}, #st{ref = Ref} = St) ->
+    {stop, normal, St};
+
+handle_info({gen_event_EXIT, {config_listener, Module}, Reason}, St)  ->
+    Level = case Reason of
+        normal -> debug;
+        shutdown -> debug;
+        _ -> error
+    end,
+    Fmt = "config_listener(~p) for ~p stopped with reason: ~r~n",
+    couch_log:Level(Fmt, [Module, St#st.pid, Reason]),
+    {stop, shutdown, St};
+
+handle_info(_, St) ->
+    {noreply, St}.
+
+
+code_change(_OldVsn, St, _Extra) ->
+    {ok, St}.
diff --git a/test/config_tests.erl b/test/config_tests.erl
index 802e0e3..311a805 100644
--- a/test/config_tests.erl
+++ b/test/config_tests.erl
@@ -11,27 +11,36 @@
 % the License.
 
 -module(config_tests).
-
 -behaviour(config_listener).
 
--export([handle_config_change/5, handle_config_terminate/3]).
 
 -include_lib("couch/include/couch_eunit.hrl").
 -include_lib("couch/include/couch_db.hrl").
 
--define(SHORT_TIMEOUT, 100).
--define(TIMEOUT, 1000).
+
+-export([
+    handle_config_change/5,
+    handle_config_terminate/3
+]).
+
+
+-define(TIMEOUT, 4000).
 
 -define(CONFIG_FIXTURESDIR,
         filename:join([?BUILDDIR(), "src", "config", "test", "fixtures"])).
+
 -define(CONFIG_FIXTURE_1,
         filename:join([?CONFIG_FIXTURESDIR, "config_tests_1.ini"])).
+
 -define(CONFIG_FIXTURE_2,
         filename:join([?CONFIG_FIXTURESDIR, "config_tests_2.ini"])).
+
 -define(CONFIG_DEFAULT_D,
         filename:join([?CONFIG_FIXTURESDIR, "default.d"])).
+
 -define(CONFIG_LOCAL_D,
         filename:join([?CONFIG_FIXTURESDIR, "local.d"])).
+
 -define(CONFIG_FIXTURE_TEMP,
     begin
         FileName = filename:join([?TEMPDIR, "config_temp.ini"]),
@@ -44,417 +53,523 @@
 -define(DEPS, [couch_stats, couch_log, config]).
 
 
+-define(T(F), {erlang:fun_to_list(F), F}).
+-define(FEXT(F), fun(_, _) -> F() end).
+
+
+
 setup() ->
     setup(?CONFIG_CHAIN).
+
 setup({temporary, Chain}) ->
     setup(Chain);
+
 setup({persistent, Chain}) ->
-    setup(lists:append(Chain, [?CONFIG_FIXTURE_TEMP]));
+    setup(Chain ++ [?CONFIG_FIXTURE_TEMP]);
+
 setup(Chain) ->
     ok = application:set_env(config, ini_files, Chain),
     test_util:start_applications(?DEPS).
 
+
 setup_empty() ->
     setup([]).
 
+
 setup_config_listener() ->
     setup(),
-    {ok, Pid} = spawn_listener(),
-    config:listen_for_changes(?MODULE, {Pid, self(), []}),
-    {Pid, self()}.
+    spawn_config_listener().
 
-teardown({Pid, _}) ->
-    stop_listener(Pid),
-    [application:stop(App) || App <- ?DEPS];
+
+teardown(Pid) when is_pid(Pid) ->
+    catch exit(Pid, kill),
+    teardown(undefined);
+
 teardown(_) ->
     [application:stop(App) || App <- ?DEPS].
 
-teardown(_, _) ->
-    [application:stop(App) || App <- ?DEPS].
 
-handle_config_change("remove_handler", _Key, _Value, _Persist, _State) ->
+teardown(_, _) ->
+    teardown(undefined).
+
+
+handle_config_change("remove_handler", _Key, _Value, _Persist, {Pid, _State}) ->
     remove_handler;
-handle_config_change("update_state", Key, _Value, _Persist, {Listener, Subscriber, Items}) ->
-    NewState = {Listener, Subscriber, [Key|Items]},
-    ok = reply(NewState, NewState),
-    {ok, NewState};
-handle_config_change(Section, Key, Value, Persist, State) ->
-    ok = reply({{Section, Key, Value, Persist}, State}, State),
-    {ok, State}.
-handle_config_terminate(Self, Reason, State) ->
-    ok = reply({stop, Self, Reason, State}, State),
+
+handle_config_change("update_state", Key, Value, Persist, {Pid, State}) ->
+    Pid ! {config_msg, {{"update_state", Key, Value, Persist}, State}},
+    {ok, {Pid, Key}};
+
+handle_config_change("throw_error", _Key, _Value, _Persist, {_Pid, _State}) ->
+    throw(this_is_an_error);
+
+handle_config_change(Section, Key, Value, Persist, {Pid, State}) ->
+    Pid ! {config_msg, {{Section, Key, Value, Persist}, State}},
+    {ok, {Pid, State}}.
+
+
+handle_config_terminate(Self, Reason, {Pid, State}) ->
+    Pid ! {config_msg, {Self, Reason, State}},
     ok.
 
-reply(Reply, {Listener, _, _}) ->
-    call_sync(Listener, {set, Reply}).
 
-wait_reply(Listener) ->
-    call_sync(Listener, get).
-
-config_test_() ->
-    {
-        "CouchDB config tests",
-        [
-            config_get_tests(),
-            config_set_tests(),
-            config_del_tests(),
-            config_override_tests(),
-            config_persistent_changes_tests(),
-            config_no_files_tests(),
-            config_listener_behaviour_tests()
-        ]
-    }.
-
-config_get_tests() ->
+config_get_test_() ->
     {
         "Config get tests",
         {
             foreach,
-            fun setup/0, fun teardown/1,
+            fun setup/0,
+            fun teardown/1,
             [
-                should_load_all_configs(),
-                should_locate_daemons_section(),
-                should_locate_mrview_handler(),
-                should_return_undefined_atom_on_missed_section(),
-                should_return_undefined_atom_on_missed_option(),
-                should_return_custom_default_value_on_missed_option(),
-                should_only_return_default_on_missed_option(),
-                should_fail_to_get_binary_value(),
-                should_return_any_supported_default()
+                fun should_load_all_configs/0,
+                fun should_locate_daemons_section/0,
+                fun should_locate_mrview_handler/0,
+                fun should_return_undefined_atom_on_missed_section/0,
+                fun should_return_undefined_atom_on_missed_option/0,
+                fun should_return_custom_default_value_on_missed_option/0,
+                fun should_only_return_default_on_missed_option/0,
+                fun should_fail_to_get_binary_value/0,
+                fun should_return_any_supported_default/0
             ]
         }
     }.
 
-config_set_tests() ->
+
+config_set_test_() ->
     {
         "Config set tests",
         {
             foreach,
-            fun setup/0, fun teardown/1,
+            fun setup/0,
+            fun teardown/1,
             [
-                should_update_option(),
-                should_create_new_section(),
-                should_fail_to_set_binary_value()
+                fun should_update_option/0,
+                fun should_create_new_section/0,
+                fun should_fail_to_set_binary_value/0
             ]
         }
     }.
 
-config_del_tests() ->
+
+config_del_test_() ->
     {
         "Config deletion tests",
         {
             foreach,
-            fun setup/0, fun teardown/1,
+            fun setup/0,
+            fun teardown/1,
             [
-                should_return_undefined_atom_after_option_deletion(),
-                should_be_ok_on_deleting_unknown_options()
+                fun should_return_undefined_atom_after_option_deletion/0,
+                fun should_be_ok_on_deleting_unknown_options/0
             ]
         }
     }.
 
-config_override_tests() ->
+
+config_override_test_() ->
     {
         "Configs overide tests",
         {
             foreachx,
-            fun setup/1, fun teardown/2,
+            fun setup/1,
+            fun teardown/2,
             [
                 {{temporary, [?CONFIG_DEFAULT]},
-                 fun should_ensure_in_defaults/2},
+                        fun should_ensure_in_defaults/2},
                 {{temporary, [?CONFIG_DEFAULT, ?CONFIG_FIXTURE_1]},
-                 fun should_override_options/2},
-                {{temporary, [?CONFIG_DEFAULT, ?CONFIG_FIXTURE_1]},
-                 fun should_override_options/2},
-                {{temporary, [?CONFIG_DEFAULT, ?CONFIG_DEFAULT_D]},
-                 fun(_, _) -> ?_assertEqual("11", config:get("couchdb", "max_dbs_open")) end},
-                {{temporary, [?CONFIG_DEFAULT, ?CONFIG_LOCAL_D]},
-                 fun(_, _) -> ?_assertEqual("12", config:get("couchdb", "max_dbs_open")) end},
-                {{temporary, [?CONFIG_DEFAULT, ?CONFIG_DEFAULT_D, ?CONFIG_LOCAL_D]},
-                 fun(_, _) -> ?_assertEqual("12", config:get("couchdb", "max_dbs_open")) end},
+                        fun should_override_options/2},
                 {{temporary, [?CONFIG_DEFAULT, ?CONFIG_FIXTURE_2]},
-                 fun should_create_new_sections_on_override/2},
+                        fun should_create_new_sections_on_override/2},
                 {{temporary, [?CONFIG_DEFAULT, ?CONFIG_FIXTURE_1,
-                              ?CONFIG_FIXTURE_2]},
-                 fun should_win_last_in_chain/2}
+                                ?CONFIG_FIXTURE_2]},
+                        fun should_win_last_in_chain/2},
+                {{temporary, [?CONFIG_DEFAULT, ?CONFIG_DEFAULT_D]},
+                        fun should_read_default_d/2},
+                {{temporary, [?CONFIG_DEFAULT, ?CONFIG_LOCAL_D]},
+                        fun should_read_local_d/2},
+                {{temporary, [?CONFIG_DEFAULT, ?CONFIG_DEFAULT_D,
+                                ?CONFIG_LOCAL_D]},
+                        fun should_read_default_and_local_d/2}
             ]
         }
     }.
 
-config_persistent_changes_tests() ->
+
+config_persistent_changes_test_() ->
     {
         "Config persistent changes",
         {
             foreachx,
-            fun setup/1, fun teardown/2,
+            fun setup/1,
+            fun teardown/2,
             [
                 {{persistent, [?CONFIG_DEFAULT]},
-                 fun should_write_changes/2},
+                        fun should_write_changes/2},
                 {{temporary, [?CONFIG_DEFAULT]},
-                 fun should_ensure_that_default_wasnt_modified/2},
+                        fun should_ensure_default_wasnt_modified/2},
                 {{temporary, [?CONFIG_FIXTURE_TEMP]},
-                 fun should_ensure_that_written_to_last_config_in_chain/2}
+                        fun should_ensure_written_to_last_config_in_chain/2}
             ]
         }
     }.
 
-config_no_files_tests() ->
+
+config_no_files_test_() ->
     {
         "Test config with no files",
         {
             foreach,
-            fun setup_empty/0, fun teardown/1,
+            fun setup_empty/0,
+            fun teardown/1,
             [
-                should_ensure_that_no_ini_files_loaded(),
-                should_create_non_persistent_option(),
-                should_create_persistent_option()
+                fun should_ensure_that_no_ini_files_loaded/0,
+                fun should_create_non_persistent_option/0,
+                fun should_create_persistent_option/0
             ]
         }
     }.
 
-config_listener_behaviour_tests() ->
+
+config_listener_behaviour_test_() ->
     {
         "Test config_listener behaviour",
         {
             foreach,
-            fun setup_config_listener/0, fun teardown/1,
+            local,
+            fun setup_config_listener/0,
+            fun teardown/1,
             [
                 fun should_handle_value_change/1,
                 fun should_pass_correct_state_to_handle_config_change/1,
                 fun should_pass_correct_state_to_handle_config_terminate/1,
                 fun should_pass_subscriber_pid_to_handle_config_terminate/1,
                 fun should_not_call_handle_config_after_related_process_death/1,
-                fun should_remove_handler_when_requested/1
+                fun should_remove_handler_when_requested/1,
+                fun should_remove_handler_when_pid_exits/1,
+                fun should_stop_monitor_on_error/1
             ]
         }
     }.
 
+
 should_load_all_configs() ->
-    ?_assert(length(config:all()) > 0).
+    ?assert(length(config:all()) > 0).
+
 
 should_locate_daemons_section() ->
-    ?_assert(length(config:get("daemons")) > 0).
+    ?assert(length(config:get("daemons")) > 0).
+
 
 should_locate_mrview_handler() ->
-    ?_assertEqual("{couch_mrview_http, handle_view_req}",
-                  config:get("httpd_design_handlers", "_view")).
+    Expect = "{couch_mrview_http, handle_view_req}",
+    ?assertEqual(Expect, config:get("httpd_design_handlers", "_view")).
+
 
 should_return_undefined_atom_on_missed_section() ->
-    ?_assertEqual(undefined,
-                  config:get("foo", "bar")).
+    ?assertEqual(undefined, config:get("foo", "bar")).
+
 
 should_return_undefined_atom_on_missed_option() ->
-    ?_assertEqual(undefined,
-                  config:get("httpd", "foo")).
+    ?assertEqual(undefined, config:get("httpd", "foo")).
+
 
 should_return_custom_default_value_on_missed_option() ->
-    ?_assertEqual("bar",
-                  config:get("httpd", "foo", "bar")).
+    ?assertEqual("bar", config:get("httpd", "foo", "bar")).
+
 
 should_only_return_default_on_missed_option() ->
-    ?_assertEqual("0",
-                  config:get("httpd", "port", "bar")).
+    ?assertEqual("0", config:get("httpd", "port", "bar")).
+
 
 should_fail_to_get_binary_value() ->
-    ?_assertException(error, badarg,
-                  config:get(<<"foo">>, <<"bar">>, <<"baz">>)).
+    ?assertException(error, badarg, config:get(<<"a">>, <<"b">>, <<"c">>)).
+
 
 should_return_any_supported_default() ->
     Values = [undefined, "list", true, false, 0.1, 1],
-    Tests = [{lists:flatten(io_lib:format("for type(~p)", [V])), V}
-        || V <- Values],
-    [{T, ?_assertEqual(V, config:get(<<"foo">>, <<"bar">>, V))}
-        || {T, V} <- Tests].
+    lists:map(fun(V) ->
+        ?assertEqual(V, config:get(<<"foo">>, <<"bar">>, V))
+    end, Values).
+
 
 should_update_option() ->
-    ?_assertEqual("severe",
-        begin
-            ok = config:set("mock_log", "level", "severe", false),
-            config:get("mock_log", "level")
-        end).
+    ok = config:set("mock_log", "level", "severe", false),
+    ?assertEqual("severe", config:get("mock_log", "level")).
+
 
 should_create_new_section() ->
-    ?_assertEqual("bang",
-        begin
-            undefined = config:get("new_section", "bizzle"),
-            ok = config:set("new_section", "bizzle", "bang", false),
-            config:get("new_section", "bizzle")
-        end).
+    ?assertEqual(undefined, config:get("new_section", "bizzle")),
+    ?assertEqual(ok, config:set("new_section", "bizzle", "bang", false)),
+    ?assertEqual("bang", config:get("new_section", "bizzle")).
+
 
 should_fail_to_set_binary_value() ->
-    ?_assertException(error, badarg,
-        config:set(<<"foo">>, <<"bar">>, <<"baz">>, false)).
+    ?assertException(error, badarg,
+            config:set(<<"a">>, <<"b">>, <<"c">>, false)).
+
 
 should_return_undefined_atom_after_option_deletion() ->
-    ?_assertEqual(undefined,
-        begin
-            ok = config:delete("mock_log", "level", false),
-            config:get("mock_log", "level")
-        end).
+    ?assertEqual(ok, config:delete("mock_log", "level", false)),
+    ?assertEqual(undefined, config:get("mock_log", "level")).
+
 
 should_be_ok_on_deleting_unknown_options() ->
-    ?_assertEqual(ok, config:delete("zoo", "boo", false)).
+    ?assertEqual(ok, config:delete("zoo", "boo", false)).
+
 
 should_ensure_in_defaults(_, _) ->
     ?_test(begin
-        ?assertEqual("500",
-                     config:get("couchdb", "max_dbs_open")),
-        ?assertEqual("5986",
-                     config:get("httpd", "port")),
-        ?assertEqual(undefined,
-                     config:get("fizbang", "unicode"))
+        ?assertEqual("500", config:get("couchdb", "max_dbs_open")),
+        ?assertEqual("5986", config:get("httpd", "port")),
+        ?assertEqual(undefined, config:get("fizbang", "unicode"))
     end).
 
+
 should_override_options(_, _) ->
     ?_test(begin
-        ?assertEqual("10",
-                     config:get("couchdb", "max_dbs_open")),
-        ?assertEqual("4895",
-                     config:get("httpd", "port"))
+        ?assertEqual("10", config:get("couchdb", "max_dbs_open")),
+        ?assertEqual("4895", config:get("httpd", "port"))
     end).
 
+
+should_read_default_d(_, _) ->
+    ?_test(begin
+        ?assertEqual("11", config:get("couchdb", "max_dbs_open"))
+    end).
+
+
+should_read_local_d(_, _) ->
+    ?_test(begin
+        ?assertEqual("12", config:get("couchdb", "max_dbs_open"))
+    end).
+
+
+should_read_default_and_local_d(_, _) ->
+    ?_test(begin
+        ?assertEqual("12", config:get("couchdb", "max_dbs_open"))
+    end).
+
+
 should_create_new_sections_on_override(_, _) ->
     ?_test(begin
-        ?assertEqual("80",
-                     config:get("httpd", "port")),
-        ?assertEqual("normalized",
-                     config:get("fizbang", "unicode"))
+        ?assertEqual("80", config:get("httpd", "port")),
+        ?assertEqual("normalized", config:get("fizbang", "unicode"))
     end).
 
+
 should_win_last_in_chain(_, _) ->
-    ?_assertEqual("80", config:get("httpd", "port")).
+    ?_test(begin
+        ?assertEqual("80", config:get("httpd", "port"))
+    end).
+
 
 should_write_changes(_, _) ->
     ?_test(begin
-        ?assertEqual("5986",
-                     config:get("httpd", "port")),
-        ?assertEqual(ok,
-                     config:set("httpd", "port", "8080")),
-        ?assertEqual("8080",
-                     config:get("httpd", "port")),
-        ?assertEqual(ok,
-                     config:delete("httpd", "bind_address", "8080")),
-        ?assertEqual(undefined,
-                     config:get("httpd", "bind_address"))
+        ?assertEqual("5986", config:get("httpd", "port")),
+        ?assertEqual(ok, config:set("httpd", "port", "8080")),
+        ?assertEqual("8080", config:get("httpd", "port")),
+        ?assertEqual(ok, config:delete("httpd", "bind_address", "8080")),
+        ?assertEqual(undefined, config:get("httpd", "bind_address"))
     end).
 
-should_ensure_that_default_wasnt_modified(_, _) ->
+
+should_ensure_default_wasnt_modified(_, _) ->
     ?_test(begin
-        ?assertEqual("5986",
-                     config:get("httpd", "port")),
-        ?assertEqual("127.0.0.1",
-                     config:get("httpd", "bind_address"))
+        ?assertEqual("5986", config:get("httpd", "port")),
+        ?assertEqual("127.0.0.1", config:get("httpd", "bind_address"))
     end).
 
-should_ensure_that_written_to_last_config_in_chain(_, _) ->
+
+should_ensure_written_to_last_config_in_chain(_, _) ->
     ?_test(begin
-        ?assertEqual("8080",
-                     config:get("httpd", "port")),
-        ?assertEqual(undefined,
-                     config:get("httpd", "bind_address"))
+        ?assertEqual("8080", config:get("httpd", "port")),
+        ?assertEqual(undefined, config:get("httpd", "bind_address"))
     end).
 
+
 should_ensure_that_no_ini_files_loaded() ->
-    ?_assertEqual(0, length(config:all())).
+    ?assertEqual(0, length(config:all())).
+
 
 should_create_non_persistent_option() ->
-    ?_assertEqual("80",
-        begin
-            ok = config:set("httpd", "port", "80", false),
-            config:get("httpd", "port")
-        end).
+    ?_test(begin
+        ?assertEqual(ok, config:set("httpd", "port", "80", false)),
+        ?assertEqual("80", config:get("httpd", "port"))
+    end).
+
 
 should_create_persistent_option() ->
-    ?_assertEqual("127.0.0.1",
-        begin
-            ok = config:set("httpd", "bind_address", "127.0.0.1"),
-            config:get("httpd", "bind_address")
-        end).
-
-should_handle_value_change({Pid, _}) ->
     ?_test(begin
-        ok = config:set("httpd", "port", "80", false),
-        ?assertMatch({{"httpd", "port", "80", false}, _}, wait_reply(Pid))
+        ?assertEqual(ok, config:set("httpd", "bind_address", "127.0.0.1")),
+        ?assertEqual("127.0.0.1", config:get("httpd", "bind_address"))
     end).
-should_pass_correct_state_to_handle_config_change({Pid, _}) ->
-    ?_test(begin
-        ok = config:set("httpd", "port", "80", false),
-        ?assertMatch({_, {Pid, _, []}}, wait_reply(Pid)),
-        ok = config:set("update_state", "foo", "any", false),
-        ?assertMatch({Pid, _, ["foo"]}, wait_reply(Pid))
-    end).
-should_pass_correct_state_to_handle_config_terminate({Pid, _}) ->
-    ?_test(begin
-        %% prepare some state
-        ok = config:set("httpd", "port", "80", false),
-        ?assertMatch({_, {Pid, _, []}}, wait_reply(Pid)),
-        ok = config:set("update_state", "foo", "any", false),
-        ?assertMatch({Pid, _, ["foo"]}, wait_reply(Pid)),
 
-        %% remove handler
-        ok = config:set("remove_handler", "any", "any", false),
-        Reply = wait_reply(Pid),
-        ?assertMatch({stop, _, remove_handler, _}, Reply),
 
-        {stop, Subscriber, _Reason, State} = Reply,
-        ?assert(is_pid(Subscriber)),
-        ?assertMatch({Pid, Subscriber, ["foo"]}, State)
-    end).
-should_pass_subscriber_pid_to_handle_config_terminate({Pid, SubscriberPid}) ->
+should_handle_value_change(Pid) ->
     ?_test(begin
-        ok = config:set("remove_handler", "any", "any", false),
-        Reply = wait_reply(Pid),
-        ?assertMatch({stop, _, remove_handler, _}, Reply),
-
-        {stop, Subscriber, _Reason, _State} = Reply,
-        ?assertMatch(SubscriberPid, Subscriber)
+        ?assertEqual(ok, config:set("httpd", "port", "80", false)),
+        ?assertMatch({{"httpd", "port", "80", false}, _}, getmsg(Pid))
     end).
-should_not_call_handle_config_after_related_process_death({Pid, _}) ->
+
+
+should_pass_correct_state_to_handle_config_change(Pid) ->
     ?_test(begin
-        ok = config:set("remove_handler", "any", "any", false),
-        Reply = wait_reply(Pid),
-        ?assertMatch({stop, _, remove_handler, _}, Reply),
-
-        ok = config:set("httpd", "port", "80", false),
-        ?assertMatch(undefined, wait_reply(Pid))
+        ?assertEqual(ok, config:set("update_state", "foo", "any", false)),
+        ?assertMatch({_, undefined}, getmsg(Pid)),
+        ?assertEqual(ok, config:set("httpd", "port", "80", false)),
+        ?assertMatch({_, "foo"}, getmsg(Pid))
     end).
-should_remove_handler_when_requested({Pid, _}) ->
+
+
+should_pass_correct_state_to_handle_config_terminate(Pid) ->
+    ?_test(begin
+        ?assertEqual(ok, config:set("update_state", "foo", "any", false)),
+        ?assertMatch({_, undefined}, getmsg(Pid)),
+        ?assertEqual(ok, config:set("httpd", "port", "80", false)),
+        ?assertMatch({_, "foo"}, getmsg(Pid)),
+        ?assertEqual(ok, config:set("remove_handler", "any", "any", false)),
+        ?assertEqual({Pid, remove_handler, "foo"}, getmsg(Pid))
+    end).
+
+
+should_pass_subscriber_pid_to_handle_config_terminate(Pid) ->
+    ?_test(begin
+        ?assertEqual(ok, config:set("remove_handler", "any", "any", false)),
+        ?assertEqual({Pid, remove_handler, undefined}, getmsg(Pid))
+    end).
+
+
+should_not_call_handle_config_after_related_process_death(Pid) ->
+    ?_test(begin
+        ?assertEqual(ok, config:set("remove_handler", "any", "any", false)),
+        ?assertEqual({Pid, remove_handler, undefined}, getmsg(Pid)),
+        ?assertEqual(ok, config:set("httpd", "port", "80", false)),
+        Event = receive
+            {config_msg, _} -> got_msg
+            after 250 -> no_msg
+        end,
+        ?assertEqual(no_msg, Event)
+    end).
+
+
+should_remove_handler_when_requested(Pid) ->
+    ?_test(begin
+        ?assertEqual(2, n_handlers()),
+        ?assertEqual(ok, config:set("remove_handler", "any", "any", false)),
+        ?assertEqual({Pid, remove_handler, undefined}, getmsg(Pid)),
+        ?assertEqual(1, n_handlers())
+    end).
+
+
+should_remove_handler_when_pid_exits(Pid) ->
     ?_test(begin
         ?assertEqual(2, n_handlers()),
 
-        ok = config:set("remove_handler", "any", "any", false),
-        Reply = wait_reply(Pid),
-        ?assertMatch({stop, _, remove_handler, _}, Reply),
+        % Monitor the config_listener_mon process
+        {monitored_by, [Mon]} = process_info(Pid, monitored_by),
+        MonRef = erlang:monitor(process, Mon),
+
+        % Kill the process synchronously
+        PidRef = erlang:monitor(process, Pid),
+        exit(Pid, kill),
+        receive
+            {'DOWN', PidRef, _, _, _} -> ok
+        after ?TIMEOUT ->
+            erlang:error({timeout, config_listener_death})
+        end,
+
+        % Wait for the config_listener_mon process to
+        % exit to indicate the handler has been removed.
+        receive
+            {'DOWN', MonRef, _, _, normal} -> ok
+        after ?TIMEOUT ->
+            erlang:error({timeout, config_listener_mon_death})
+        end,
 
         ?assertEqual(1, n_handlers())
     end).
 
-call_sync(Listener, Msg) ->
-    Ref = make_ref(),
-    Listener ! {Ref, self(), Msg},
+
+should_stop_monitor_on_error(Pid) ->
+    ?_test(begin
+        ?assertEqual(2, n_handlers()),
+
+        % Monitor the config_listener_mon process
+        {monitored_by, [Mon]} = process_info(Pid, monitored_by),
+        MonRef = erlang:monitor(process, Mon),
+
+        % Have the process throw an error
+        ?assertEqual(ok, config:set("throw_error", "foo", "bar", false)),
+
+        % Make sure handle_config_terminate is called
+        ?assertEqual({Pid, {error, this_is_an_error}, undefined}, getmsg(Pid)),
+
+        % Wait for the config_listener_mon process to
+        % exit to indicate the handler has been removed
+        % due to an error
+        receive
+            {'DOWN', MonRef, _, _, shutdown} -> ok
+        after ?TIMEOUT ->
+            erlang:error({timeout, config_listener_mon_shutdown})
+        end,
+
+        ?assertEqual(1, n_handlers())
+    end).
+
+
+spawn_config_listener() ->
+    Self = self(),
+    Pid = erlang:spawn(fun() ->
+        ok = config:listen_for_changes(?MODULE, {self(), undefined}),
+        Self ! registered,
+        loop(undefined)
+    end),
     receive
-        {ok, Ref, Reply} -> Reply
+        registered -> ok
     after ?TIMEOUT ->
-        throw({error, {timeout, call_sync}})
-    end.
+        erlang:error({timeout, config_handler_register})
+    end,
+    Pid.
 
-spawn_listener() ->
-    {ok, spawn(fun() -> loop(undefined) end)}.
 
-stop_listener(Listener) ->
-    call_sync(Listener, stop).
-
-loop(State) ->
+loop(undefined) ->
     receive
-        {Ref, From, stop} ->
-            From ! {ok, Ref, ok},
-            ok;
-        {Ref, From, {set, Value}} ->
-            From ! {ok, Ref, ok},
-            loop(Value);
-        {Ref, From, get} ->
-            From ! {ok, Ref, State},
-            loop(undefined)
+        {config_msg, _} = Msg ->
+            loop(Msg);
+        {get_msg, _, _} = Msg ->
+            loop(Msg);
+        Msg ->
+            erlang:error({invalid_message, Msg})
+    end;
+
+loop({get_msg, From, Ref}) ->
+    receive
+        {config_msg, _} = Msg ->
+            From ! {Ref, Msg};
+        Msg ->
+            erlang:error({invalid_message, Msg})
+    end,
+    loop(undefined);
+
+loop({config_msg, _} = Msg) ->
+    receive
+        {get_msg, From, Ref} ->
+            From ! {Ref, Msg};
+        Msg ->
+            erlang:error({invalid_message, Msg})
+    end,
+    loop(undefined).
+
+
+getmsg(Pid) ->
+    Ref = erlang:make_ref(),
+    Pid ! {get_msg, self(), Ref},
+    receive
+        {Ref, {config_msg, Msg}} -> Msg
+    after ?TIMEOUT ->
+        erlang:error({timeout, config_msg})
     end.
 
+
 n_handlers() ->
     length(gen_event:which_handlers(config_event)).