Merge branch 'master' into 63012-scheduler
diff --git a/.travis.yml b/.travis.yml
index ed8f466..e1f7e85 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -16,9 +16,7 @@
 language: erlang
 
 otp_release:
-  - 18.2
-  - 18.1
-  - 18.0
+  - 18.3
   - 17.5
   - R16B03-1
 
@@ -35,6 +33,7 @@
 before_script:
   - cd couchdb
   - ./configure --disable-docs --disable-fauxton
+  - rm -rf ./src/couch_replicator/*
   - cp -r ../!(couchdb) ./src/couch_replicator
   - make
 
@@ -42,3 +41,6 @@
   - make eunit apps=couch_replicator skip_deps=couch_epi,couch_log
 
 cache: apt
+
+notifications:
+  email: false
diff --git a/priv/stats_descriptions.cfg b/priv/stats_descriptions.cfg
index 2564f92..c009cc9 100644
--- a/priv/stats_descriptions.cfg
+++ b/priv/stats_descriptions.cfg
@@ -54,3 +54,111 @@
     {type, counter},
     {desc, <<"number of replicator workers started">>}
 ]}.
+{[couch_replicator, cluster_is_stable], [
+    {type, gauge},
+    {desc, <<"1 if cluster is stable, 0 if unstable">>}
+]}.
+{[couch_replicator, db_scans], [
+    {type, counter},
+    {desc, <<"number of time replicator db scans have been started">>}
+]}.
+{[couch_replicator, docs, dbs_created], [
+    {type, counter},
+    {desc, <<"number of db shard creations seen by replicator doc processor">>}
+]}.
+{[couch_replicator, docs, dbs_deleted], [
+    {type, counter},
+    {desc, <<"number of db shard deletions seen by replicator doc processor">>}
+]}.
+{[couch_replicator, docs, dbs_found], [
+    {type, counter},
+    {desc, <<"number of db shard found by replicator doc processor">>}
+]}.
+{[couch_replicator, docs, db_changes], [
+    {type, counter},
+    {desc, <<"number of db changes processed by replicator doc processor">>}
+]}.
+{[couch_replicator, docs, failed_state_updates], [
+    {type, counter},
+    {desc, <<"number of 'failed' state document updates">>}
+]}.
+{[couch_replicator, docs, completed_state_updates], [
+    {type, counter},
+    {desc, <<"number of 'completed' state document updates">>}
+]}.
+{[couch_replicator, jobs, adds], [
+    {type, counter},
+    {desc, <<"number of jobs added to replicator scheduler">>}
+]}.
+{[couch_replicator, jobs, duplicate_adds], [
+    {type, counter},
+    {desc, <<"number of duplicate jobs added to replicator scheduler">>}
+]}.
+{[couch_replicator, jobs, removes], [
+    {type, counter},
+    {desc, <<"number of jobs removed from replicator scheduler">>}
+]}.
+{[couch_replicator, jobs, starts], [
+    {type, counter},
+    {desc, <<"number of jobs started by replicator scheduler">>}
+]}.
+{[couch_replicator, jobs, stops], [
+    {type, counter},
+    {desc, <<"number of jobs stopped to replicator scheduler">>}
+]}.
+{[couch_replicator, jobs, crashes], [
+    {type, counter},
+    {desc, <<"number of job crashed noticed by replicator scheduler">>}
+]}.
+{[couch_replicator, jobs, running], [
+    {type, gauge},
+    {desc, <<"replicator scheduler running jobs">>}
+]}.
+{[couch_replicator, jobs, pending], [
+    {type, gauge},
+    {desc, <<"replicator scheduler pending jobs">>}
+]}.
+{[couch_replicator, jobs, crashed], [
+    {type, gauge},
+    {desc, <<"replicator scheduler crashed jobs">>}
+]}.
+{[couch_replicator, jobs, total], [
+    {type, gauge},
+    {desc, <<"total number of replicator scheduler jobs">>}
+]}.
+{[couch_replicator, jobs, avg_running], [
+    {type, gauge},
+    {desc, <<"average of length of time current jobs have been running">>}
+]}.
+{[couch_replicator, jobs, avg_pending], [
+    {type, gauge},
+    {desc, <<"average length of time spent waiting to run">>}
+]}.
+{[couch_replicator, jobs, avg_crashed], [
+    {type, gauge},
+    {desc, <<"average of length of time since last crash">>}
+]}.
+{[couch_replicator, connection, acquires], [
+    {type, counter},
+    {desc, <<"number of times connections are shared">>}
+]}.
+{[couch_replicator, connection, creates], [
+    {type, counter},
+    {desc, <<"number of connections created">>}
+]}.
+{[couch_replicator, connection, relinquishes], [
+    {type, counter},
+    {desc, <<"number of times ownership of a connection is relinquished">>}
+]}.
+{[couch_replicator, connection, owner_crashes], [
+    {type, counter},
+    {desc, <<"number of times a connection owner crashes while owning at least one connection">>}
+]}.
+{[couch_replicator, connection, worker_crashes], [
+    {type, counter},
+    {desc, <<"number of times a worker unexpectedly terminates">>}
+]}.
+{[couch_replicator, connection, closes], [
+    {type, counter},
+    {desc, <<"number of times a worker is gracefully shut down">>}
+]}.
diff --git a/src/couch_multidb_changes.erl b/src/couch_multidb_changes.erl
new file mode 100644
index 0000000..9c0b7cf
--- /dev/null
+++ b/src/couch_multidb_changes.erl
@@ -0,0 +1,819 @@
+% 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_multidb_changes).
+-behaviour(gen_server).
+
+-export([start_link/4]).
+
+-export([init/1, handle_call/3, handle_info/2, handle_cast/2]).
+-export([code_change/3, terminate/2]).
+
+-export([changes_reader/3, changes_reader_cb/3]).
+
+-include_lib("couch/include/couch_db.hrl").
+-include_lib("mem3/include/mem3.hrl").
+
+-define(CTX, {user_ctx, #user_ctx{roles=[<<"_admin">>, <<"_replicator">>]}}).
+
+-define(AVG_DELAY_MSEC, 100).
+-define(MAX_DELAY_MSEC, 60000).
+
+-record(state, {
+    tid :: ets:tid(),
+    mod :: atom(),
+    ctx :: term(),
+    suffix :: binary(),
+    event_server :: reference(),
+    scanner :: nil | pid(),
+    pids :: [{binary(),pid()}],
+    skip_ddocs :: boolean()
+}).
+
+
+
+% Behavior API
+
+% For each db shard with a matching suffix, report created,
+% deleted, found (discovered) and change events.
+
+-callback db_created(DbName :: binary(), Context :: term()) ->
+    Context :: term().
+
+-callback db_deleted(DbName :: binary(), Context :: term()) ->
+    Context :: term().
+
+-callback db_found(DbName :: binary(), Context :: term()) ->
+    Context :: term().
+
+-callback db_change(DbName :: binary(), Change :: term(), Context :: term()) ->
+    Context :: term().
+
+
+
+% External API
+
+
+% Opts list can contain:
+%  - `skip_ddocs` : Skip design docs
+
+-spec start_link(binary(), module(), term(), list()) ->
+    {ok, pid()} | ignore | {error, term()}.
+start_link(DbSuffix, Module, Context, Opts) when
+    is_binary(DbSuffix), is_atom(Module), is_list(Opts) ->
+    gen_server:start_link(?MODULE, [DbSuffix, Module, Context, Opts], []).
+
+
+% gen_server callbacks
+
+init([DbSuffix, Module, Context, Opts]) ->
+    process_flag(trap_exit, true),
+    Server = self(),
+    {ok, #state{
+        tid = ets:new(?MODULE, [set, protected]),
+        mod = Module,
+        ctx = Context,
+        suffix = DbSuffix,
+        event_server = register_with_event_server(Server),
+        scanner = spawn_link(fun() -> scan_all_dbs(Server, DbSuffix) end),
+        pids = [],
+        skip_ddocs = proplists:is_defined(skip_ddocs, Opts)
+    }}.
+
+
+terminate(_Reason, _State) ->
+    ok.
+
+
+handle_call({change, DbName, Change}, _From,
+    #state{skip_ddocs=SkipDDocs, mod=Mod, ctx=Ctx} = State) ->
+    case {SkipDDocs, is_design_doc(Change)} of
+        {true, true} ->
+            {reply, ok, State};
+        {_, _} ->
+            {reply, ok, State#state{ctx=Mod:db_change(DbName, Change, Ctx)}}
+    end;
+
+handle_call({checkpoint, DbName, EndSeq}, _From, #state{tid=Ets} = State) ->
+    case ets:lookup(Ets, DbName) of
+        [] ->
+            true = ets:insert(Ets, {DbName, EndSeq, false});
+        [{DbName, _OldSeq, Rescan}] ->
+            true = ets:insert(Ets, {DbName, EndSeq, Rescan})
+    end,
+    {reply, ok, State}.
+
+
+handle_cast({resume_scan, DbName}, State) ->
+    {noreply, resume_scan(DbName, State)}.
+
+
+handle_info({'$couch_event', DbName, Event}, #state{suffix = Suf} = State) ->
+    case Suf =:= couch_db:dbname_suffix(DbName) of
+        true ->
+            {noreply, db_callback(Event, DbName, State)};
+        _ ->
+            {noreply, State}
+    end;
+
+handle_info({'DOWN', Ref, _, _, Info}, #state{event_server = Ref} = State) ->
+    {stop, {couch_event_server_died, Info}, State};
+
+handle_info({'EXIT', From, normal}, #state{scanner = From} = State) ->
+    {noreply, State#state{scanner=nil}};
+
+handle_info({'EXIT', From, Reason}, #state{scanner = From} = State) ->
+    {stop, {scanner_died, Reason}, State};
+
+handle_info({'EXIT', From, Reason}, #state{pids = Pids} = State) ->
+    couch_log:info("~p change feed exited ~p",[State#state.suffix, From]),
+    case lists:keytake(From, 2, Pids) of
+        {value, {DbName, From}, NewPids} ->
+            if Reason == normal -> ok; true ->
+                Fmt = "~s : Known change feed ~w died :: ~w",
+                couch_log:error(Fmt, [?MODULE, From, Reason])
+            end,
+            NewState = State#state{pids = NewPids},
+            case ets:lookup(State#state.tid, DbName) of
+                [{DbName, _EndSeq, true}] ->
+                    {noreply, resume_scan(DbName, NewState)};
+                _ ->
+                    {noreply, NewState}
+            end;
+        false when Reason == normal ->
+            {noreply, State};
+        false ->
+            Fmt = "~s(~p) : Unknown pid ~w died :: ~w",
+            couch_log:error(Fmt, [?MODULE, State#state.suffix, From, Reason]),
+            {stop, {unexpected_exit, From, Reason}, State}
+    end;
+
+handle_info(_Msg, State) ->
+    {noreply, State}.
+
+
+code_change(_OldVsn, State, _Extra) ->
+    {ok, State}.
+
+
+% Private functions
+
+-spec register_with_event_server(pid()) -> reference().
+register_with_event_server(Server) ->
+    Ref = erlang:monitor(process, couch_event_server),
+    couch_event:register_all(Server),
+    Ref.
+
+
+-spec db_callback(created | deleted | updated, binary(), #state{}) -> #state{}.
+db_callback(created, DbName, #state{mod = Mod, ctx = Ctx} = State) ->
+    State#state{ctx = Mod:db_created(DbName, Ctx)};
+
+db_callback(deleted, DbName, #state{mod = Mod, ctx = Ctx} = State) ->
+    State#state{ctx = Mod:db_deleted(DbName, Ctx)};
+
+db_callback(updated, DbName, State) ->
+    resume_scan(DbName, State);
+
+db_callback(_Other, _DbName, State) ->
+    State.
+
+
+-spec resume_scan(binary(), #state{}) -> #state{}.
+resume_scan(DbName, #state{pids=Pids, tid=Ets} = State) ->
+    case {lists:keyfind(DbName, 1, Pids), ets:lookup(Ets, DbName)} of
+        {{DbName, _}, []} ->
+            % Found existing change feed, but not entry in ETS
+            % Flag a need to rescan from begining
+            true = ets:insert(Ets, {DbName, 0, true}),
+            State;
+        {{DbName, _}, [{DbName, EndSeq, _}]} ->
+            % Found existing change feed and entry in ETS
+            % Flag a need to rescan from last ETS checkpoint
+            true = ets:insert(Ets, {DbName, EndSeq, true}),
+            State;
+        {false, []} ->
+            % No existing change feed running. No entry in ETS.
+            % This is first time seeing this db shard.
+            % Notify user with a found callback. Insert checkpoint
+            % entry in ETS to start from 0. And start a change feed.
+            true = ets:insert(Ets, {DbName, 0, false}),
+            Mod = State#state.mod,
+            Ctx = Mod:db_found(DbName, State#state.ctx),
+            Pid = start_changes_reader(DbName, 0),
+            State#state{ctx=Ctx, pids=[{DbName, Pid} | Pids]};
+        {false, [{DbName, EndSeq, _}]} ->
+            % No existing change feed running. Found existing checkpoint.
+            % Start a new change reader from last checkpoint.
+            true = ets:insert(Ets, {DbName, EndSeq, false}),
+            Pid = start_changes_reader(DbName, EndSeq),
+            State#state{pids=[{DbName, Pid} | Pids]}
+     end.
+
+
+
+start_changes_reader(DbName, Since) ->
+    spawn_link(?MODULE, changes_reader, [self(), DbName, Since]).
+
+
+changes_reader(Server, DbName, Since) ->
+    {ok, Db} = couch_db:open_int(DbName, [?CTX, sys_db]),
+    ChFun = couch_changes:handle_db_changes(
+        #changes_args{
+            include_docs = true,
+            since = Since,
+            feed = "normal",
+            timeout = infinity
+        }, {json_req, null}, Db),
+    ChFun({fun ?MODULE:changes_reader_cb/3, {Server, DbName}}).
+
+
+changes_reader_cb({change, Change, _}, _, {Server, DbName}) ->
+    ok = gen_server:call(Server, {change, DbName, Change}, infinity),
+    {Server, DbName};
+
+changes_reader_cb({stop, EndSeq}, _, {Server, DbName}) ->
+    ok = gen_server:call(Server, {checkpoint, DbName, EndSeq}, infinity),
+    {Server, DbName};
+
+changes_reader_cb(_, _, Acc) ->
+    Acc.
+
+
+scan_all_dbs(Server, DbSuffix) when is_pid(Server) ->
+    ok = scan_local_db(Server, DbSuffix),
+    {ok, Db} = mem3_util:ensure_exists(
+        config:get("mem3", "shards_db", "_dbs")),
+    ChangesFun = couch_changes:handle_changes(#changes_args{}, nil, Db, nil),
+    ChangesFun(fun({change, {Change}, _}, _) ->
+        DbName = couch_util:get_value(<<"id">>, Change),
+        case DbName of <<"_design/", _/binary>> -> ok; _Else ->
+            case couch_replicator_utils:is_deleted(Change) of
+            true ->
+                ok;
+            false ->
+                [gen_server:cast(Server, {resume_scan, ShardName})
+                    || ShardName <- filter_shards(DbName, DbSuffix)],
+                ok
+            end
+        end;
+        (_, _) -> ok
+    end),
+    couch_db:close(Db).
+
+
+filter_shards(DbName, DbSuffix) ->
+    case DbSuffix =:= couch_db:dbname_suffix(DbName) of
+    false ->
+        [];
+    true ->
+        try
+            [ShardName || #shard{name = ShardName} <- mem3:local_shards(DbName)]
+        catch
+            error:database_does_not_exist ->
+                []
+        end
+    end.
+
+
+scan_local_db(Server, DbSuffix) when is_pid(Server) ->
+    case couch_db:open_int(DbSuffix, [?CTX, sys_db, nologifmissing]) of
+        {ok, Db} ->
+            gen_server:cast(Server, {resume_scan, DbSuffix}),
+            ok = couch_db:close(Db);
+        _Error ->
+            ok
+    end.
+
+
+is_design_doc({Change}) ->
+    case lists:keyfind(<<"id">>, 1, Change) of
+        false ->
+            false;
+        {_, Id} ->
+            is_design_doc_id(Id)
+    end.
+
+
+is_design_doc_id(<<?DESIGN_DOC_PREFIX, _/binary>>) ->
+    true;
+
+is_design_doc_id(_) ->
+    false.
+
+
+
+-ifdef(TEST).
+
+-include_lib("eunit/include/eunit.hrl").
+-include_lib("couch/include/couch_eunit.hrl").
+
+-define(MOD, multidb_test_module).
+-define(SUFFIX, <<"suff">>).
+-define(DBNAME, <<"shards/40000000-5fffffff/acct/suff.0123456789">>).
+
+couch_multidb_changes_test_() ->
+    {
+        foreach,
+        fun setup/0,
+        fun teardown/1,
+        [
+            t_handle_call_change(),
+            t_handle_call_change_filter_design_docs(),
+            t_handle_call_checkpoint_new(),
+            t_handle_call_checkpoint_existing(),
+            t_handle_info_created(),
+            t_handle_info_deleted(),
+            t_handle_info_updated(),
+            t_handle_info_other_event(),
+            t_handle_info_created_other_db(),
+            t_handle_info_scanner_exit_normal(),
+            t_handle_info_scanner_crashed(),
+            t_handle_info_event_server_exited(),
+            t_handle_info_unknown_pid_exited(),
+            t_handle_info_change_feed_exited(),
+            t_handle_info_change_feed_exited_and_need_rescan(),
+            t_spawn_changes_reader(),
+            t_changes_reader_cb_change(),
+            t_changes_reader_cb_stop(),
+            t_changes_reader_cb_other(),
+            t_handle_call_resume_scan_no_chfeed_no_ets_entry(),
+            t_handle_call_resume_scan_chfeed_no_ets_entry(),
+            t_handle_call_resume_scan_chfeed_ets_entry(),
+            t_handle_call_resume_scan_no_chfeed_ets_entry(),
+            t_start_link(),
+            t_start_link_no_ddocs(),
+            t_misc_gen_server_callbacks()
+        ]
+    }.
+
+setup() ->
+    mock_logs(),
+    mock_callback_mod(),
+    meck:expect(couch_event, register_all, 1, ok),
+    meck:expect(config, get, ["mem3", "shards_db", '_'], "_dbs"),
+    meck:expect(mem3_util, ensure_exists, 1, {ok, dbs}),
+    ChangesFun = meck:val(fun(_) -> ok end),
+    meck:expect(couch_changes, handle_changes, 4, ChangesFun),
+    meck:expect(couch_db, open_int,
+        fun(?DBNAME, [?CTX, sys_db]) -> {ok, db};
+            (_, _) -> {not_found, no_db_file}
+        end),
+    meck:expect(couch_db, close, 1, ok),
+    mock_changes_reader(),
+    % create process to stand in for couch_ever_server
+    % mocking erlang:monitor doesn't, so give it real process to monitor
+    EvtPid = spawn_link(fun() -> receive looper -> ok end end),
+    true = register(couch_event_server, EvtPid),
+    EvtPid.
+
+
+teardown(EvtPid) ->
+    unlink(EvtPid),
+    exit(EvtPid, kill),
+    meck:unload().
+
+
+t_handle_call_change() ->
+    ?_test(begin
+        State = mock_state(),
+        Change = change_row(<<"blah">>),
+        handle_call_ok({change, ?DBNAME, Change}, State),
+        ?assert(meck:validate(?MOD)),
+        ?assert(meck:called(?MOD, db_change, [?DBNAME, Change, zig]))
+    end).
+
+t_handle_call_change_filter_design_docs() ->
+    ?_test(begin
+        State0 = mock_state(),
+        State = State0#state{skip_ddocs = true},
+        Change = change_row(<<"_design/blah">>),
+        handle_call_ok({change, ?DBNAME, Change}, State),
+        ?assert(meck:validate(?MOD)),
+        ?assertNot(meck:called(?MOD, db_change, [?DBNAME, Change, zig]))
+    end).
+
+t_handle_call_checkpoint_new() ->
+    ?_test(begin
+        Tid = mock_ets(),
+        State = mock_state(Tid),
+        handle_call_ok({checkpoint, ?DBNAME, 1}, State),
+        ?assertEqual([{?DBNAME, 1, false}], ets:tab2list(Tid)),
+        ets:delete(Tid)
+    end).
+
+t_handle_call_checkpoint_existing() ->
+    ?_test(begin
+        Tid = mock_ets(),
+        State = mock_state(Tid),
+        true = ets:insert(Tid, {?DBNAME, 1, true}),
+        handle_call_ok({checkpoint, ?DBNAME, 2}, State),
+        ?assertEqual([{?DBNAME, 2, true}], ets:tab2list(Tid)),
+        ets:delete(Tid)
+    end).
+
+t_handle_info_created() ->
+    ?_test(begin
+        State = mock_state(),
+        handle_info_check({'$couch_event', ?DBNAME, created}, State),
+        ?assert(meck:validate(?MOD)),
+        ?assert(meck:called(?MOD, db_created, [?DBNAME, zig]))
+    end).
+
+t_handle_info_deleted() ->
+     ?_test(begin
+        State = mock_state(),
+        handle_info_check({'$couch_event', ?DBNAME, deleted}, State),
+        ?assert(meck:validate(?MOD)),
+        ?assert(meck:called(?MOD, db_deleted, [?DBNAME, zig]))
+    end).
+
+t_handle_info_updated() ->
+     ?_test(begin
+        Tid = mock_ets(),
+        State = mock_state(Tid),
+        handle_info_check({'$couch_event', ?DBNAME, updated}, State),
+        ?assert(meck:validate(?MOD)),
+        ?assert(meck:called(?MOD, db_found, [?DBNAME, zig]))
+    end).
+
+t_handle_info_other_event() ->
+     ?_test(begin
+        State = mock_state(),
+        handle_info_check({'$couch_event', ?DBNAME, somethingelse}, State)
+    end).
+
+t_handle_info_created_other_db() ->
+     ?_test(begin
+        State = mock_state(),
+        handle_info_check({'$couch_event', <<"otherdb">>, created}, State),
+        ?assertNot(meck:called(?MOD, db_created, [?DBNAME, zig]))
+    end).
+
+t_handle_info_scanner_exit_normal() ->
+    ?_test(begin
+        Res = handle_info({'EXIT', spid, normal}, mock_state()),
+        ?assertMatch({noreply, _}, Res),
+        {noreply, RState} = Res,
+        ?assertEqual(nil, RState#state.scanner)
+    end).
+
+t_handle_info_scanner_crashed() ->
+    ?_test(begin
+        Res = handle_info({'EXIT', spid, oops}, mock_state()),
+        ?assertMatch({stop, {scanner_died, oops}, _State}, Res)
+    end).
+
+t_handle_info_event_server_exited() ->
+    ?_test(begin
+        Res = handle_info({'DOWN', esref, type, espid, reason}, mock_state()),
+        ?assertMatch({stop, {couch_event_server_died, reason},_}, Res)
+    end).
+
+t_handle_info_unknown_pid_exited() ->
+    ?_test(begin
+        State0 = mock_state(),
+        Res0 =  handle_info({'EXIT', somepid, normal}, State0),
+        ?assertMatch({noreply, State0}, Res0),
+        State1 = mock_state(),
+        Res1 = handle_info({'EXIT', somepid, oops}, State1),
+        ?assertMatch({stop, {unexpected_exit, somepid, oops}, State1}, Res1)
+    end).
+
+t_handle_info_change_feed_exited() ->
+    ?_test(begin
+        Tid0 = mock_ets(),
+        State0 = mock_state(Tid0, cpid),
+        Res0 = handle_info({'EXIT', cpid, normal}, State0),
+        ?assertMatch({noreply, _}, Res0),
+        {noreply, RState0} = Res0,
+        ?assertEqual([], RState0#state.pids),
+        ets:delete(Tid0),
+        Tid1 = mock_ets(),
+        State1 = mock_state(Tid1, cpid),
+        Res1 = handle_info({'EXIT', cpid, oops}, State1),
+        ?assertMatch({noreply, _}, Res1),
+        {noreply, RState1} = Res1,
+        ?assertEqual([], RState1#state.pids),
+        ets:delete(Tid1)
+    end).
+
+t_handle_info_change_feed_exited_and_need_rescan() ->
+    ?_test(begin
+        Tid = mock_ets(),
+        true = ets:insert(Tid, {?DBNAME, 1, true}),
+        State = mock_state(Tid, cpid),
+        Res = handle_info({'EXIT', cpid, normal}, State),
+        ?assertMatch({noreply, _}, Res),
+        {noreply, RState} = Res,
+        % rescan flag should have been reset to false
+        ?assertEqual([{?DBNAME, 1, false}], ets:tab2list(Tid)),
+        % a mock change feed process should be running
+        [{?DBNAME, Pid}] = RState#state.pids,
+        ?assert(is_pid(Pid)),
+        ChArgs = kill_mock_change_reader_and_get_its_args(Pid),
+        ?assertEqual({self(), ?DBNAME}, ChArgs),
+        ets:delete(Tid)
+    end).
+
+t_spawn_changes_reader() ->
+    ?_test(begin
+        Pid = start_changes_reader(?DBNAME, 3),
+        ?assert(erlang:is_process_alive(Pid)),
+        ChArgs = kill_mock_change_reader_and_get_its_args(Pid),
+        ?assertEqual({self(), ?DBNAME}, ChArgs),
+        ?assert(meck:validate(couch_db)),
+        ?assert(meck:validate(couch_changes)),
+        ?assert(meck:called(couch_db, open_int, [?DBNAME, [?CTX, sys_db]])),
+        ?assert(meck:called(couch_changes, handle_db_changes, [
+            #changes_args{
+                include_docs = true,
+                since = 3,
+                feed = "normal",
+                timeout = infinity
+            }, {json_req, null}, db]))
+    end).
+
+t_changes_reader_cb_change() ->
+    ?_test(begin
+        {ok, Pid} = start_link(?SUFFIX, ?MOD, zig, []),
+        Change = change_row(<<"blah">>),
+        ChArg = {change, Change, ignore},
+        {Pid, ?DBNAME} = changes_reader_cb(ChArg, chtype, {Pid, ?DBNAME}),
+        ?assert(meck:called(?MOD, db_change, [?DBNAME, Change, zig])),
+        unlink(Pid),
+        exit(Pid, kill)
+    end).
+
+t_changes_reader_cb_stop() ->
+    ?_test(begin
+        {ok, Pid} = start_link(?SUFFIX, ?MOD, zig, []),
+        ChArg = {stop, 11},
+        {Pid, ?DBNAME} = changes_reader_cb(ChArg, chtype, {Pid, ?DBNAME}),
+        % We checkpoint on stop, check if checkpointed at correct sequence
+        #state{tid = Tid} = sys:get_state(Pid),
+        ?assertEqual([{?DBNAME, 11, false}], ets:tab2list(Tid)),
+        unlink(Pid),
+        exit(Pid, kill)
+    end).
+
+t_changes_reader_cb_other() ->
+    ?_assertEqual(acc, changes_reader_cb(other, chtype, acc)).
+
+t_handle_call_resume_scan_no_chfeed_no_ets_entry() ->
+    ?_test(begin
+        Tid = mock_ets(),
+        State = mock_state(Tid),
+        RState = resume_scan(?DBNAME, State),
+        % Check if inserted checkpoint entry in ets starting at 0
+        ?assertEqual([{?DBNAME, 0, false}], ets:tab2list(Tid)),
+        % Check if called db_found callback
+        ?assert(meck:called(?MOD, db_found, [?DBNAME, zig])),
+        % Check if started a change reader
+        [{?DBNAME, Pid}] = RState#state.pids,
+        ChArgs = kill_mock_change_reader_and_get_its_args(Pid),
+        ?assertEqual({self(), ?DBNAME}, ChArgs),
+        ?assert(meck:called(couch_changes, handle_db_changes, [
+              #changes_args{
+                  include_docs = true,
+                  since = 0,
+                  feed = "normal",
+                  timeout = infinity
+              }, {json_req, null}, db])),
+        ets:delete(Tid)
+    end).
+
+t_handle_call_resume_scan_chfeed_no_ets_entry() ->
+    ?_test(begin
+        Tid = mock_ets(),
+        Pid = start_changes_reader(?DBNAME, 0),
+        State = mock_state(Tid, Pid),
+        resume_scan(?DBNAME, State),
+        % Check ets checkpoint is set to 0 and rescan = true
+        ?assertEqual([{?DBNAME, 0, true}], ets:tab2list(Tid)),
+        ets:delete(Tid),
+        kill_mock_change_reader_and_get_its_args(Pid)
+    end).
+
+t_handle_call_resume_scan_chfeed_ets_entry() ->
+    ?_test(begin
+        Tid = mock_ets(),
+        true = ets:insert(Tid, [{?DBNAME, 2, false}]),
+        Pid = start_changes_reader(?DBNAME, 1),
+        State = mock_state(Tid, Pid),
+        resume_scan(?DBNAME, State),
+        % Check ets checkpoint is set to same endseq but rescan = true
+        ?assertEqual([{?DBNAME, 2, true}], ets:tab2list(Tid)),
+        ets:delete(Tid),
+        kill_mock_change_reader_and_get_its_args(Pid)
+    end).
+
+t_handle_call_resume_scan_no_chfeed_ets_entry() ->
+    ?_test(begin
+        Tid = mock_ets(),
+        true = ets:insert(Tid, [{?DBNAME, 1, true}]),
+        State = mock_state(Tid),
+        RState = resume_scan(?DBNAME, State),
+        % Check if reset rescan to false but kept same endseq
+        ?assertEqual([{?DBNAME, 1, false}], ets:tab2list(Tid)),
+        % Check if started a change reader
+        [{?DBNAME, Pid}] = RState#state.pids,
+        ChArgs = kill_mock_change_reader_and_get_its_args(Pid),
+        ?assertEqual({self(), ?DBNAME}, ChArgs),
+        ?assert(meck:called(couch_changes, handle_db_changes, [
+            #changes_args{
+                include_docs = true,
+                since = 1,
+                feed = "normal",
+                timeout = infinity
+            }, {json_req, null}, db])),
+        ets:delete(Tid)
+    end).
+
+t_start_link() ->
+    ?_test(begin
+        {ok, Pid} = start_link(?SUFFIX, ?MOD, nil, []),
+        ?assert(is_pid(Pid)),
+        ?assertMatch(#state{
+            mod = ?MOD,
+            suffix = ?SUFFIX,
+            ctx = nil,
+            pids = [],
+            skip_ddocs = false
+        },  sys:get_state(Pid)),
+        unlink(Pid),
+        exit(Pid, kill),
+        ?assert(meck:called(couch_event, register_all, [Pid]))
+    end).
+
+t_start_link_no_ddocs() ->
+    ?_test(begin
+        {ok, Pid} = start_link(?SUFFIX, ?MOD, nil, [skip_ddocs]),
+        ?assert(is_pid(Pid)),
+        ?assertMatch(#state{
+            mod = ?MOD,
+            suffix = ?SUFFIX,
+            ctx = nil,
+            pids = [],
+            skip_ddocs = true
+        },  sys:get_state(Pid)),
+        unlink(Pid),
+        exit(Pid, kill)
+    end).
+
+t_misc_gen_server_callbacks() ->
+    ?_test(begin
+        ?assertEqual(ok, terminate(reason, state)),
+        ?assertEqual({ok, state}, code_change(old, state, extra))
+    end).
+
+
+scan_dbs_test_() ->
+{
+    foreach,
+    fun() -> test_util:start_couch([mem3, fabric]) end,
+    fun(Ctx) -> test_util:stop_couch(Ctx) end,
+    [
+        t_pass_shard(),
+        t_fail_shard(),
+        t_pass_local(),
+        t_fail_local()
+    ]
+}.
+
+
+t_pass_shard() ->
+    ?_test(begin
+        DbName0 = ?tempdb(),
+        DbSuffix = <<"_replicator">>,
+        DbName = <<DbName0/binary, "/", DbSuffix/binary>>,
+        ok = fabric:create_db(DbName, [?CTX]),
+        ?assertEqual(8, length(filter_shards(DbName, DbSuffix))),
+        fabric:delete_db(DbName, [?CTX])
+    end).
+
+
+t_fail_shard() ->
+    ?_test(begin
+        DbName = ?tempdb(),
+        ok = fabric:create_db(DbName, [?CTX]),
+        ?assertEqual([], filter_shards(DbName, <<"_replicator">>)),
+        fabric:delete_db(DbName, [?CTX])
+    end).
+
+
+t_pass_local() ->
+    ?_test(begin
+        LocalDb = ?tempdb(),
+        {ok, Db} = couch_db:create(LocalDb, [?CTX]),
+        ok = couch_db:close(Db),
+        scan_local_db(self(), LocalDb),
+        receive
+            {'$gen_cast', Msg} ->
+                ?assertEqual(Msg, {resume_scan, LocalDb})
+        after 0 ->
+                ?assert(false)
+        end
+    end).
+
+
+t_fail_local() ->
+    ?_test(begin
+        LocalDb = ?tempdb(),
+        {ok, Db} = couch_db:create(LocalDb, [?CTX]),
+        ok = couch_db:close(Db),
+        scan_local_db(self(), <<"some_other_db">>),
+        receive
+            {'$gen_cast', Msg} ->
+                ?assertNotEqual(Msg, {resume_scan, LocalDb})
+        after 0 ->
+                ?assert(true)
+        end
+    end).
+
+
+% Test helper functions
+
+mock_logs() ->
+    meck:expect(couch_log, error, 2, ok),
+    meck:expect(couch_log, notice, 2, ok),
+    meck:expect(couch_log, info, 2, ok),
+    meck:expect(couch_log, debug, 2, ok).
+
+
+mock_callback_mod() ->
+    meck:new(?MOD, [non_strict]),
+    meck:expect(?MOD, db_created, fun(_DbName, Ctx) -> Ctx end),
+    meck:expect(?MOD, db_deleted, fun(_DbName, Ctx) -> Ctx end),
+    meck:expect(?MOD, db_found, fun(_DbName, Ctx) -> Ctx end),
+    meck:expect(?MOD, db_change, fun(_DbName, _Change, Ctx) -> Ctx end).
+
+
+mock_changes_reader_loop({_CbFun, {Server, DbName}}) ->
+    receive
+        die ->
+            exit({Server, DbName})
+    end.
+
+kill_mock_change_reader_and_get_its_args(Pid) ->
+    Ref = monitor(process, Pid),
+    unlink(Pid),
+    Pid ! die,
+    receive
+        {'DOWN', Ref, _, Pid, {Server, DbName}} ->
+            {Server, DbName}
+        after 1000 ->
+            erlang:error(spawn_change_reader_timeout)
+    end.
+
+mock_changes_reader() ->
+    meck:expect(couch_changes, handle_db_changes,
+        fun(_ChArgs, _Req, db) ->
+            fun mock_changes_reader_loop/1
+        end).
+
+mock_ets() ->
+    ets:new(multidb_test_ets, [set, public]).
+
+mock_state() ->
+    #state{
+        mod = ?MOD,
+        ctx = zig,
+        suffix = ?SUFFIX,
+        event_server = esref,
+        scanner = spid,
+        pids = []}.
+
+mock_state(Ets) ->
+    State = mock_state(),
+    State#state{tid = Ets}.
+
+mock_state(Ets, Pid) ->
+    State = mock_state(Ets),
+    State#state{pids = [{?DBNAME, Pid}]}.
+
+
+change_row(Id) when is_binary(Id) ->
+    {[
+        {<<"seq">>, 1},
+        {<<"id">>, Id},
+        {<<"changes">>, [{[{<<"rev">>, <<"1-f00">>}]}]},
+        {doc, {[{<<"_id">>, Id}, {<<"_rev">>, <<"1-f00">>}]}}
+    ]}.
+
+handle_call_ok(Msg, State) ->
+    ?assertMatch({reply, ok, _}, handle_call(Msg, from, State)).
+
+handle_info_check(Msg, State) ->
+    ?assertMatch({noreply, _}, handle_info(Msg, State)).
+
+-endif.
diff --git a/src/couch_replicator.app.src b/src/couch_replicator.app.src
index 4f12195..f3d3dae 100644
--- a/src/couch_replicator.app.src
+++ b/src/couch_replicator.app.src
@@ -14,24 +14,12 @@
     {description, "CouchDB replicator"},
     {vsn, git},
     {mod, {couch_replicator_app, []}},
-    {modules, [
-        couch_replicator,
-        couch_replicator_api_wrap,
-        couch_replicator_app,
-        couch_replicator_httpc,
-        couch_replicator_httpd,
-        couch_replicator_job_sup,
-        couch_replicator_notifier,
-        couch_replicator_manager,
-        couch_replicator_httpc_pool,
-        couch_replicator_sup,
-        couch_replicator_utils,
-        couch_replicator_worker
-    ]},
     {registered, [
-        couch_replicator,
-        couch_replicator_manager,
-        couch_replicator_job_sup
+        couch_replicator_sup,
+        couch_replication,  % couch_replication_event gen_event
+        couch_replicator_clustering,
+        couch_replicator_scheduler,
+        couch_replicator_scheduler_sup
     ]},
     {applications, [
         kernel,
diff --git a/src/couch_replicator.erl b/src/couch_replicator.erl
index 7f0c7ee..f2b0d02 100644
--- a/src/couch_replicator.erl
+++ b/src/couch_replicator.erl
@@ -11,91 +11,62 @@
 % the License.
 
 -module(couch_replicator).
--behaviour(gen_server).
--vsn(1).
 
-% public API
--export([replicate/2]).
-
-% meant to be used only by the replicator database listener
--export([async_replicate/1]).
--export([cancel_replication/1]).
-
-% gen_server callbacks
--export([init/1, terminate/2, code_change/3]).
--export([handle_call/3, handle_cast/2, handle_info/2]).
--export([format_status/2]).
-
--export([details/1]).
+-export([replicate/2, ensure_rep_db_exists/0]).
+-export([stream_active_docs_info/3, stream_terminal_docs_info/4]).
+-export([replication_states/0]).
+-export([job/1, doc/3]).
 
 -include_lib("couch/include/couch_db.hrl").
--include("couch_replicator_api_wrap.hrl").
 -include("couch_replicator.hrl").
+-include("couch_replicator_api_wrap.hrl").
+-include_lib("couch_mrview/include/couch_mrview.hrl").
 
--define(LOWEST_SEQ, 0).
-
--define(DEFAULT_CHECKPOINT_INTERVAL, 30000).
+-define(REPLICATION_STATES, [
+    initializing,  % Just added to scheduler
+    error,         % Could not be turned into a replication job
+    running,       % Scheduled and running
+    pending,       % Scheduled and waiting to run
+    crashing,      % Scheduled but crashing, possibly backed off by the scheduler
+    completed,     % Non-continuous (normal) completed replication
+    failed         % Terminal failure, will not be retried anymore
+]).
 
 -import(couch_util, [
     get_value/2,
-    get_value/3,
-    to_binary/1
+    get_value/3
 ]).
 
--import(couch_replicator_utils, [
-    start_db_compaction_notifier/2,
-    stop_db_compaction_notifier/1
-]).
 
--record(rep_state, {
-    rep_details,
-    source_name,
-    target_name,
-    source,
-    target,
-    history,
-    checkpoint_history,
-    start_seq,
-    committed_seq,
-    current_through_seq,
-    seqs_in_progress = [],
-    highest_seq_done = {0, ?LOWEST_SEQ},
-    source_log,
-    target_log,
-    rep_starttime,
-    src_starttime,
-    tgt_starttime,
-    timer, % checkpoint timer
-    changes_queue,
-    changes_manager,
-    changes_reader,
-    workers,
-    stats = couch_replicator_stats:new(),
-    session_id,
-    source_db_compaction_notifier = nil,
-    target_db_compaction_notifier = nil,
-    source_monitor = nil,
-    target_monitor = nil,
-    source_seq = nil,
-    use_checkpoints = true,
-    checkpoint_interval = ?DEFAULT_CHECKPOINT_INTERVAL,
-    type = db,
-    view = nil
-}).
+-type user_doc_cb() :: fun(({[_]}, any()) -> any()).
+-type query_acc() :: {binary(), user_doc_cb(), any()}.
 
 
+-spec replicate({[_]}, #user_ctx{}) ->
+    {ok, {continuous, binary()}} |
+    {ok, {[_]}} |
+    {ok, {cancelled, binary()}} |
+    {error, any()}.
 replicate(PostBody, Ctx) ->
-    {ok, #rep{id = RepId, options = Options, user_ctx = UserCtx} = Rep} =
-        couch_replicator_utils:parse_rep_doc(PostBody, Ctx),
+    {ok, Rep0} = couch_replicator_utils:parse_rep_doc(PostBody, Ctx),
+    Rep = Rep0#rep{start_time = os:timestamp()},
+    #rep{id = RepId, options = Options, user_ctx = UserCtx} = Rep,
     case get_value(cancel, Options, false) of
     true ->
-        case get_value(id, Options, nil) of
+        CancelRepId = case get_value(id, Options, nil) of
         nil ->
-            cancel_replication(RepId);
+            RepId;
         RepId2 ->
-            cancel_replication(RepId2, UserCtx)
+            RepId2
+        end,
+        case check_authorization(CancelRepId, UserCtx) of
+        ok ->
+            cancel_replication(CancelRepId);
+        not_found ->
+            {error, not_found}
         end;
     false ->
+        check_authorization(RepId, UserCtx),
         {ok, Listener} = rep_result_listener(RepId),
         Result = do_replication_loop(Rep),
         couch_replicator_notifier:stop(Listener),
@@ -103,75 +74,34 @@
     end.
 
 
+% This is called from supervisor. Must respect supervisor protocol so
+% it returns `ignore`.
+-spec ensure_rep_db_exists() -> ignore.
+ensure_rep_db_exists() ->
+    {ok, _Db} = couch_replicator_docs:ensure_rep_db_exists(),
+    couch_log:notice("~p : created local _replicator database", [?MODULE]),
+    ignore.
+
+
+-spec do_replication_loop(#rep{}) ->
+    {ok, {continuous, binary()}} | {ok, tuple()} | {error, any()}.
 do_replication_loop(#rep{id = {BaseId, Ext} = Id, options = Options} = Rep) ->
-    case async_replicate(Rep) of
-    {ok, _Pid} ->
-        case get_value(continuous, Options, false) of
-        true ->
-            {ok, {continuous, ?l2b(BaseId ++ Ext)}};
-        false ->
-            wait_for_result(Id)
-        end;
-    Error ->
-        Error
+    case couch_replicator_scheduler:add_job(Rep) of
+    ok ->
+        ok;
+    {error, already_added} ->
+        couch_log:notice("Replication '~s' already running", [BaseId ++ Ext]),
+        ok
+    end,
+    case get_value(continuous, Options, false) of
+    true ->
+        {ok, {continuous, ?l2b(BaseId ++ Ext)}};
+    false ->
+        wait_for_result(Id)
     end.
 
 
-async_replicate(#rep{id = {BaseId, Ext}, source = Src, target = Tgt} = Rep) ->
-    RepChildId = BaseId ++ Ext,
-    Source = couch_replicator_api_wrap:db_uri(Src),
-    Target = couch_replicator_api_wrap:db_uri(Tgt),
-    Timeout = get_value(connection_timeout, Rep#rep.options),
-    ChildSpec = {
-        RepChildId,
-        {gen_server, start_link, [?MODULE, Rep, [{timeout, Timeout}]]},
-        temporary,
-        250,
-        worker,
-        [?MODULE]
-    },
-    % All these nested cases to attempt starting/restarting a replication child
-    % are ugly and not 100% race condition free. The following patch submission
-    % is a solution:
-    %
-    % http://erlang.2086793.n4.nabble.com/PATCH-supervisor-atomically-delete-child-spec-when-child-terminates-td3226098.html
-    %
-    case supervisor:start_child(couch_replicator_job_sup, ChildSpec) of
-    {ok, Pid} ->
-        couch_log:notice("starting new replication `~s` at ~p (`~s` -> `~s`)",
-            [RepChildId, Pid, Source, Target]),
-        {ok, Pid};
-    {error, already_present} ->
-        case supervisor:restart_child(couch_replicator_job_sup, RepChildId) of
-        {ok, Pid} ->
-            couch_log:notice("restarting replication `~s` at ~p (`~s` -> `~s`)",
-                [RepChildId, Pid, Source, Target]),
-            {ok, Pid};
-        {error, running} ->
-            %% this error occurs if multiple replicators are racing
-            %% each other to start and somebody else won. Just grab
-            %% the Pid by calling start_child again.
-            timer:sleep(50 + random:uniform(100)),
-            async_replicate(Rep);
-        {error, {'EXIT', {badarg,
-            [{erlang, apply, [gen_server, start_link, undefined]} | _]}}} ->
-            % Clause to deal with a change in the supervisor module introduced
-            % in R14B02. For more details consult the thread at:
-            %     http://erlang.org/pipermail/erlang-bugs/2011-March/002273.html
-            _ = supervisor:delete_child(couch_replicator_job_sup, RepChildId),
-            async_replicate(Rep);
-        {error, _} = Error ->
-            Error
-        end;
-    {error, {already_started, Pid}} ->
-        couch_log:notice("replication `~s` already running at ~p (`~s` -> `~s`)",
-            [RepChildId, Pid, Source, Target]),
-        {ok, Pid};
-    {error, {Error, _}} ->
-        {error, Error}
-    end.
-
-
+-spec rep_result_listener(rep_id()) -> {ok, pid()}.
 rep_result_listener(RepId) ->
     ReplyTo = self(),
     {ok, _Listener} = couch_replicator_notifier:start_link(
@@ -182,6 +112,8 @@
         end).
 
 
+-spec wait_for_result(rep_id()) ->
+    {ok, any()} | {error, any()}.
 wait_for_result(RepId) ->
     receive
     {finished, RepId, RepResult} ->
@@ -191,847 +123,294 @@
     end.
 
 
-cancel_replication({BaseId, Extension}) ->
-    FullRepId = BaseId ++ Extension,
-    couch_log:notice("Canceling replication `~s`...", [FullRepId]),
-    case supervisor:terminate_child(couch_replicator_job_sup, FullRepId) of
-    ok ->
-        couch_log:notice("Replication `~s` canceled.", [FullRepId]),
-        case supervisor:delete_child(couch_replicator_job_sup, FullRepId) of
-            ok ->
-                {ok, {cancelled, ?l2b(FullRepId)}};
-            {error, not_found} ->
-                {ok, {cancelled, ?l2b(FullRepId)}};
-            Error ->
-                Error
-        end;
-    Error ->
-        couch_log:error("Error canceling replication `~s`: ~p", [FullRepId, Error]),
-        Error
+-spec cancel_replication(rep_id()) ->
+    {ok, {cancelled, binary()}} | {error, not_found}.
+cancel_replication({BasedId, Extension} = RepId) ->
+    FullRepId = BasedId ++ Extension,
+    couch_log:notice("Canceling replication '~s' ...", [FullRepId]),
+    case couch_replicator_scheduler:rep_state(RepId) of
+    #rep{} ->
+        ok = couch_replicator_scheduler:remove_job(RepId),
+        couch_log:notice("Replication '~s' cancelled", [FullRepId]),
+        {ok, {cancelled, ?l2b(FullRepId)}};
+    nil ->
+        couch_log:notice("Replication '~s' not found", [FullRepId]),
+        {error, not_found}
     end.
 
-cancel_replication(RepId, #user_ctx{name = Name, roles = Roles}) ->
-    case lists:member(<<"_admin">>, Roles) of
-    true ->
-        cancel_replication(RepId);
-    false ->
-        case find_replicator(RepId) of
-        {ok, Pid} ->
-            case details(Pid) of
-            {ok, #rep{user_ctx = #user_ctx{name = Name}}} ->
-                cancel_replication(RepId);
-            {ok, _} ->
-                throw({unauthorized,
-                    <<"Can't cancel a replication triggered by another user">>});
-            Error ->
-                Error
-            end;
-        Error ->
-            Error
-        end
+
+-spec replication_states() -> [atom()].
+replication_states() ->
+    ?REPLICATION_STATES.
+
+
+-spec stream_terminal_docs_info(binary(), user_doc_cb(), any(), [atom()]) -> any().
+stream_terminal_docs_info(Db, Cb, UserAcc, States) ->
+    DDoc = <<"_replicator">>,
+    View = <<"terminal_states">>,
+    QueryCb = fun handle_replicator_doc_query/2,
+    Args = #mrargs{view_type = map, reduce = false},
+    Acc = {Db, Cb, UserAcc, States},
+    try fabric:query_view(Db, DDoc, View, QueryCb, Acc, Args) of
+    {ok, {Db, Cb, UserAcc1, States}} ->
+        UserAcc1
+    catch
+        error:database_does_not_exist ->
+            UserAcc;
+        error:{badmatch, {not_found, Reason}} ->
+            Msg = "Could not find _design/~s ~s view in replicator db ~s : ~p",
+            couch_log:error(Msg, [DDoc, View, Db, Reason]),
+            couch_replicator_docs:ensure_cluster_rep_ddoc_exists(Db),
+            timer:sleep(1000),
+            stream_terminal_docs_info(Db, Cb, UserAcc, States)
     end.
 
-find_replicator({BaseId, Ext} = _RepId) ->
-    case lists:keysearch(
-        BaseId ++ Ext, 1, supervisor:which_children(couch_replicator_job_sup)) of
-    {value, {_, Pid, _, _}} when is_pid(Pid) ->
-            {ok, Pid};
-    _ ->
+
+-spec stream_active_docs_info(user_doc_cb(), any(), [atom()]) -> any().
+stream_active_docs_info(Cb, UserAcc, States) ->
+    Nodes = lists:sort([node() | nodes()]),
+    stream_active_docs_info(Nodes, Cb, UserAcc, States).
+
+
+-spec stream_active_docs_info([node()], user_doc_cb(), any(), [atom()]) -> any().
+stream_active_docs_info([], _Cb, UserAcc, _States) ->
+    UserAcc;
+
+stream_active_docs_info([Node | Nodes], Cb, UserAcc, States) ->
+    case rpc:call(Node, couch_replicator_doc_processor, docs, [States]) of
+        {badrpc, Reason} ->
+            ErrMsg = "Could not get replicator docs from ~p. Error: ~p",
+            couch_log:error(ErrMsg, [Node, Reason]),
+            stream_active_docs_info(Nodes, Cb, UserAcc, States);
+        Results ->
+            UserAcc1 = lists:foldl(Cb, UserAcc, Results),
+            stream_active_docs_info(Nodes, Cb, UserAcc1, States)
+    end.
+
+
+-spec handle_replicator_doc_query
+    ({row, [_]} , query_acc()) -> {ok, query_acc()};
+    ({error, any()}, query_acc()) -> {error, any()};
+    ({meta, any()}, query_acc()) -> {ok,  query_acc()};
+    (complete, query_acc()) -> {ok, query_acc()}.
+handle_replicator_doc_query({row, Props}, {Db, Cb, UserAcc, States}) ->
+    DocId = couch_util:get_value(id, Props),
+    DocStateBin = couch_util:get_value(key, Props),
+    DocState = erlang:binary_to_existing_atom(DocStateBin, utf8),
+    MapValue = couch_util:get_value(value, Props),
+    {Source, Target, StartTime, StateTime, StateInfo} = case MapValue of
+        [Src, Tgt, StartT, StateT, Info] ->
+            {Src, Tgt, StartT, StateT, Info};
+        _Other ->
+            % Handle the case where the view code was upgraded but new view code
+            % wasn't updated yet (before a _scheduler/docs request was made)
+            {null, null, null, null, null}
+    end,
+    % Set the error_count to 1 if failed. This is mainly for consistency as
+    % jobs from doc_processor and scheduler will have that value set
+    ErrorCount = case DocState of failed -> 1; _ -> 0 end,
+    case filter_replicator_doc_query(DocState, States) of
+        true ->
+            EjsonInfo = {[
+                {doc_id, DocId},
+                {database, Db},
+                {id, null},
+                {source, strip_url_creds(Source)},
+                {target, strip_url_creds(Target)},
+                {state, DocState},
+                {error_count, ErrorCount},
+                {info, StateInfo},
+                {last_updated, StateTime},
+                {start_time, StartTime}
+            ]},
+            {ok, {Db, Cb, Cb(EjsonInfo, UserAcc), States}};
+        false ->
+            {ok, {Db, Cb, UserAcc, States}}
+    end;
+handle_replicator_doc_query({error, Reason}, _Acc) ->
+    {error, Reason};
+handle_replicator_doc_query({meta, _Meta}, Acc) ->
+    {ok, Acc};
+handle_replicator_doc_query(complete, Acc) ->
+    {stop, Acc}.
+
+
+-spec strip_url_creds(binary() | {[_]}) -> binary().
+strip_url_creds(Endpoint) ->
+    case couch_replicator_docs:parse_rep_db(Endpoint, [], []) of
+        #httpdb{url=Url} ->
+            iolist_to_binary(couch_util:url_strip_password(Url));
+        LocalDb when is_binary(LocalDb) ->
+            LocalDb
+    end.
+
+
+-spec filter_replicator_doc_query(atom(), [atom()]) -> boolean().
+filter_replicator_doc_query(_DocState, []) ->
+    true;
+filter_replicator_doc_query(State, States) when is_list(States) ->
+    lists:member(State, States).
+
+
+-spec job(binary()) -> {ok, {[_]}} | {error, not_found}.
+job(JobId0) when is_binary(JobId0) ->
+    JobId = couch_replicator_ids:convert(JobId0),
+    {Res, _Bad} = rpc:multicall(couch_replicator_scheduler, job, [JobId]),
+    case [JobInfo || {ok, JobInfo} <- Res] of
+        [JobInfo| _] ->
+            {ok, JobInfo};
+        [] ->
             {error, not_found}
     end.
 
-details(Pid) ->
-    case (catch gen_server:call(Pid, get_details)) of
-    {ok, Rep} ->
-        {ok, Rep};
-    {'EXIT', {noproc, {gen_server, call, _}}} ->
-        {error, not_found};
-    Error ->
-        throw(Error)
+
+-spec doc(binary(), binary(), [_]) -> {ok, {[_]}} | {error, not_found}.
+doc(RepDb, DocId, UserCtx) ->
+    {Res, _Bad} = rpc:multicall(couch_replicator_doc_processor, doc, [RepDb, DocId]),
+    case [DocInfo || {ok, DocInfo} <- Res] of
+        [DocInfo| _] ->
+            {ok, DocInfo};
+        [] ->
+            doc_from_db(RepDb, DocId, UserCtx)
     end.
 
-init(InitArgs) ->
-    {ok, InitArgs, 0}.
 
-do_init(#rep{options = Options, id = {BaseId, Ext}, user_ctx=UserCtx} = Rep) ->
-    process_flag(trap_exit, true),
-
-    random:seed(os:timestamp()),
-
-    #rep_state{
-        source = Source,
-        target = Target,
-        source_name = SourceName,
-        target_name = TargetName,
-        start_seq = {_Ts, StartSeq},
-        committed_seq = {_, CommittedSeq},
-        highest_seq_done = {_, HighestSeq},
-        checkpoint_interval = CheckpointInterval
-    } = State = init_state(Rep),
-
-    NumWorkers = get_value(worker_processes, Options),
-    BatchSize = get_value(worker_batch_size, Options),
-    {ok, ChangesQueue} = couch_work_queue:new([
-        {max_items, BatchSize * NumWorkers * 2},
-        {max_size, 100 * 1024 * NumWorkers}
-    ]),
-    % This starts the _changes reader process. It adds the changes from
-    % the source db to the ChangesQueue.
-    {ok, ChangesReader} = couch_replicator_changes_reader:start_link(
-        StartSeq, Source, ChangesQueue, Options
-    ),
-    % Changes manager - responsible for dequeing batches from the changes queue
-    % and deliver them to the worker processes.
-    ChangesManager = spawn_changes_manager(self(), ChangesQueue, BatchSize),
-    % This starts the worker processes. They ask the changes queue manager for a
-    % a batch of _changes rows to process -> check which revs are missing in the
-    % target, and for the missing ones, it copies them from the source to the target.
-    MaxConns = get_value(http_connections, Options),
-    Workers = lists:map(
-        fun(_) ->
-            couch_stats:increment_counter([couch_replicator, workers_started]),
-            {ok, Pid} = couch_replicator_worker:start_link(
-                self(), Source, Target, ChangesManager, MaxConns),
-            Pid
-        end,
-        lists:seq(1, NumWorkers)),
-
-    couch_task_status:add_task([
-        {type, replication},
-        {user, UserCtx#user_ctx.name},
-        {replication_id, ?l2b(BaseId ++ Ext)},
-        {database, Rep#rep.db_name},
-        {doc_id, Rep#rep.doc_id},
-        {source, ?l2b(SourceName)},
-        {target, ?l2b(TargetName)},
-        {continuous, get_value(continuous, Options, false)},
-        {revisions_checked, 0},
-        {missing_revisions_found, 0},
-        {docs_read, 0},
-        {docs_written, 0},
-        {changes_pending, get_pending_count(State)},
-        {doc_write_failures, 0},
-        {source_seq, HighestSeq},
-        {checkpointed_source_seq, CommittedSeq},
-        {checkpoint_interval, CheckpointInterval}
-    ]),
-    couch_task_status:set_update_frequency(1000),
-
-    % Until OTP R14B03:
-    %
-    % Restarting a temporary supervised child implies that the original arguments
-    % (#rep{} record) specified in the MFA component of the supervisor
-    % child spec will always be used whenever the child is restarted.
-    % This implies the same replication performance tunning parameters will
-    % always be used. The solution is to delete the child spec (see
-    % cancel_replication/1) and then start the replication again, but this is
-    % unfortunately not immune to race conditions.
-
-    couch_log:notice("Replication `~p` is using:~n"
-        "~c~p worker processes~n"
-        "~ca worker batch size of ~p~n"
-        "~c~p HTTP connections~n"
-        "~ca connection timeout of ~p milliseconds~n"
-        "~c~p retries per request~n"
-        "~csocket options are: ~s~s",
-        [BaseId ++ Ext, $\t, NumWorkers, $\t, BatchSize, $\t,
-            MaxConns, $\t, get_value(connection_timeout, Options),
-            $\t, get_value(retries, Options),
-            $\t, io_lib:format("~p", [get_value(socket_options, Options)]),
-            case StartSeq of
-            ?LOWEST_SEQ ->
-                "";
-            _ ->
-                io_lib:format("~n~csource start sequence ~p", [$\t, StartSeq])
-            end]),
-
-    couch_log:debug("Worker pids are: ~p", [Workers]),
-
-    couch_replicator_manager:replication_started(Rep),
-
-    {ok, State#rep_state{
-            changes_queue = ChangesQueue,
-            changes_manager = ChangesManager,
-            changes_reader = ChangesReader,
-            workers = Workers
-        }
-    }.
-
-adjust_maxconn(Src = #httpdb{http_connections = 1}, RepId) ->
-    Msg = "Adjusting minimum number of HTTP source connections to 2 for ~p",
-    couch_log:notice(Msg, [RepId]),
-    Src#httpdb{http_connections = 2};
-
-adjust_maxconn(Src, _RepId) ->
-    Src.
-
-handle_info(shutdown, St) ->
-    {stop, shutdown, St};
-
-handle_info({'DOWN', Ref, _, _, Why}, #rep_state{source_monitor = Ref} = St) ->
-    couch_log:error("Source database is down. Reason: ~p", [Why]),
-    {stop, source_db_down, St};
-
-handle_info({'DOWN', Ref, _, _, Why}, #rep_state{target_monitor = Ref} = St) ->
-    couch_log:error("Target database is down. Reason: ~p", [Why]),
-    {stop, target_db_down, St};
-
-handle_info({'EXIT', Pid, normal}, #rep_state{changes_reader=Pid} = State) ->
-    {noreply, State};
-
-handle_info({'EXIT', Pid, Reason}, #rep_state{changes_reader=Pid} = State) ->
-    couch_stats:increment_counter([couch_replicator, changes_reader_deaths]),
-    couch_log:error("ChangesReader process died with reason: ~p", [Reason]),
-    {stop, changes_reader_died, cancel_timer(State)};
-
-handle_info({'EXIT', Pid, normal}, #rep_state{changes_manager = Pid} = State) ->
-    {noreply, State};
-
-handle_info({'EXIT', Pid, Reason}, #rep_state{changes_manager = Pid} = State) ->
-    couch_stats:increment_counter([couch_replicator, changes_manager_deaths]),
-    couch_log:error("ChangesManager process died with reason: ~p", [Reason]),
-    {stop, changes_manager_died, cancel_timer(State)};
-
-handle_info({'EXIT', Pid, normal}, #rep_state{changes_queue=Pid} = State) ->
-    {noreply, State};
-
-handle_info({'EXIT', Pid, Reason}, #rep_state{changes_queue=Pid} = State) ->
-    couch_stats:increment_counter([couch_replicator, changes_queue_deaths]),
-    couch_log:error("ChangesQueue process died with reason: ~p", [Reason]),
-    {stop, changes_queue_died, cancel_timer(State)};
-
-handle_info({'EXIT', Pid, normal}, #rep_state{workers = Workers} = State) ->
-    case Workers -- [Pid] of
-    Workers ->
-        couch_log:error("unknown pid bit the dust ~p ~n",[Pid]),
-        {noreply, State#rep_state{workers = Workers}};
-        %% not clear why a stop was here before
-        %%{stop, {unknown_process_died, Pid, normal}, State};
-    [] ->
-        catch unlink(State#rep_state.changes_manager),
-        catch exit(State#rep_state.changes_manager, kill),
-        do_last_checkpoint(State);
-    Workers2 ->
-        {noreply, State#rep_state{workers = Workers2}}
-    end;
-
-handle_info({'EXIT', Pid, Reason}, #rep_state{workers = Workers} = State) ->
-    State2 = cancel_timer(State),
-    case lists:member(Pid, Workers) of
-    false ->
-        {stop, {unknown_process_died, Pid, Reason}, State2};
-    true ->
-        couch_stats:increment_counter([couch_replicator, worker_deaths]),
-        couch_log:error("Worker ~p died with reason: ~p", [Pid, Reason]),
-        {stop, {worker_died, Pid, Reason}, State2}
-    end;
-
-handle_info(timeout, InitArgs) ->
-    try do_init(InitArgs) of {ok, State} ->
-        {noreply, State}
-    catch Class:Error ->
-        Stack = erlang:get_stacktrace(),
-        {stop, shutdown, {error, Class, Error, Stack, InitArgs}}
+-spec doc_from_db(binary(), binary(), [_]) -> {ok, {[_]}} | {error, not_found}.
+doc_from_db(RepDb, DocId, UserCtx) ->
+    case fabric:open_doc(RepDb, DocId, [UserCtx, ejson_body]) of
+        {ok, Doc} ->
+            {Props} = couch_doc:to_json_obj(Doc, []),
+            Source = get_value(<<"source">>, Props),
+            Target = get_value(<<"target">>, Props),
+            State = get_value(<<"_replication_state">>, Props, null),
+            StartTime = get_value(<<"_replication_start_time">>, Props, null),
+            StateTime = get_value(<<"_replication_state_time">>, Props, null),
+            {StateInfo, ErrorCount} = case State of
+                <<"completed">> ->
+                    Info = get_value(<<"_replication_stats">>, Props, null),
+                    {Info, 0};
+                <<"failed">> ->
+                    Info = get_value(<<"_replication_state_reason">>, Props, null),
+                    {Info, 1};
+                _OtherState ->
+                    {null, 0}
+            end,
+            {ok, {[
+                {doc_id, DocId},
+                {database, RepDb},
+                {id, null},
+                {source, strip_url_creds(Source)},
+                {target, strip_url_creds(Target)},
+                {state, State},
+                {error_count, ErrorCount},
+                {info, StateInfo},
+                {start_time, StartTime},
+                {last_updated, StateTime}
+            ]}};
+         {not_found, _Reason} ->
+            {error, not_found}
     end.
 
-handle_call(get_details, _From, #rep_state{rep_details = Rep} = State) ->
-    {reply, {ok, Rep}, State};
 
-handle_call({add_stats, Stats}, From, State) ->
-    gen_server:reply(From, ok),
-    NewStats = couch_replicator_utils:sum_stats(State#rep_state.stats, Stats),
-    {noreply, State#rep_state{stats = NewStats}};
-
-handle_call({report_seq_done, Seq, StatsInc}, From,
-    #rep_state{seqs_in_progress = SeqsInProgress, highest_seq_done = HighestDone,
-        current_through_seq = ThroughSeq, stats = Stats} = State) ->
-    gen_server:reply(From, ok),
-    {NewThroughSeq0, NewSeqsInProgress} = case SeqsInProgress of
-    [] ->
-        {Seq, []};
-    [Seq | Rest] ->
-        {Seq, Rest};
-    [_ | _] ->
-        {ThroughSeq, ordsets:del_element(Seq, SeqsInProgress)}
-    end,
-    NewHighestDone = lists:max([HighestDone, Seq]),
-    NewThroughSeq = case NewSeqsInProgress of
-    [] ->
-        lists:max([NewThroughSeq0, NewHighestDone]);
-    _ ->
-        NewThroughSeq0
-    end,
-    couch_log:debug("Worker reported seq ~p, through seq was ~p, "
-        "new through seq is ~p, highest seq done was ~p, "
-        "new highest seq done is ~p~n"
-        "Seqs in progress were: ~p~nSeqs in progress are now: ~p",
-        [Seq, ThroughSeq, NewThroughSeq, HighestDone,
-            NewHighestDone, SeqsInProgress, NewSeqsInProgress]),
-    NewState = State#rep_state{
-        stats = couch_replicator_utils:sum_stats(Stats, StatsInc),
-        current_through_seq = NewThroughSeq,
-        seqs_in_progress = NewSeqsInProgress,
-        highest_seq_done = NewHighestDone
-    },
-    update_task(NewState),
-    {noreply, NewState}.
+-spec check_authorization(rep_id(), #user_ctx{}) -> ok | not_found.
+check_authorization(RepId, #user_ctx{name = Name} = Ctx) ->
+    case couch_replicator_scheduler:rep_state(RepId) of
+    #rep{user_ctx = #user_ctx{name = Name}} ->
+        ok;
+    #rep{} ->
+        couch_httpd:verify_is_server_admin(Ctx);
+    nil ->
+        not_found
+    end.
 
 
-handle_cast({db_compacted, DbName},
-    #rep_state{source = #db{name = DbName} = Source} = State) ->
-    {ok, NewSource} = couch_db:reopen(Source),
-    {noreply, State#rep_state{source = NewSource}};
+-ifdef(TEST).
 
-handle_cast({db_compacted, DbName},
-    #rep_state{target = #db{name = DbName} = Target} = State) ->
-    {ok, NewTarget} = couch_db:reopen(Target),
-    {noreply, State#rep_state{target = NewTarget}};
+-include_lib("eunit/include/eunit.hrl").
 
-handle_cast(checkpoint, State) ->
-    #rep_state{rep_details = #rep{} = Rep} = State,
-    case couch_replicator_manager:continue(Rep) of
-    {true, _} ->
-        case do_checkpoint(State) of
-        {ok, NewState} ->
-            couch_stats:increment_counter([couch_replicator, checkpoints, success]),
-            {noreply, NewState#rep_state{timer = start_timer(State)}};
-        Error ->
-            couch_stats:increment_counter([couch_replicator, checkpoints, failure]),
-            {stop, Error, State}
-        end;
-    {false, Owner} ->
-        couch_replicator_manager:replication_usurped(Rep, Owner),
-        {stop, shutdown, State}
-    end;
-
-handle_cast({report_seq, Seq},
-    #rep_state{seqs_in_progress = SeqsInProgress} = State) ->
-    NewSeqsInProgress = ordsets:add_element(Seq, SeqsInProgress),
-    {noreply, State#rep_state{seqs_in_progress = NewSeqsInProgress}}.
-
-
-code_change(_OldVsn, #rep_state{}=State, _Extra) ->
-    {ok, State}.
-
-
-headers_strip_creds([], Acc) ->
-    lists:reverse(Acc);
-headers_strip_creds([{Key, Value0} | Rest], Acc) ->
-    Value = case string:to_lower(Key) of
-    "authorization" ->
-        "****";
-    _ ->
-        Value0
-    end,
-    headers_strip_creds(Rest, [{Key, Value} | Acc]).
-
-
-httpdb_strip_creds(#httpdb{url = Url, headers = Headers} = HttpDb) ->
-    HttpDb#httpdb{
-        url = couch_util:url_strip_password(Url),
-        headers = headers_strip_creds(Headers, [])
-    };
-httpdb_strip_creds(LocalDb) ->
-    LocalDb.
-
-
-rep_strip_creds(#rep{source = Source, target = Target} = Rep) ->
-    Rep#rep{
-        source = httpdb_strip_creds(Source),
-        target = httpdb_strip_creds(Target)
+authorization_test_() ->
+    {
+        foreach,
+        fun () -> ok end,
+        fun (_) -> meck:unload() end,
+        [
+            t_admin_is_always_authorized(),
+            t_username_must_match(),
+            t_replication_not_found()
+        ]
     }.
 
 
-state_strip_creds(#rep_state{rep_details = Rep, source = Source, target = Target} = State) ->
-    % #rep_state contains the source and target at the top level and also
-    % in the nested #rep_details record
-    State#rep_state{
-        rep_details = rep_strip_creds(Rep),
-        source = httpdb_strip_creds(Source),
-        target = httpdb_strip_creds(Target)
-    }.
-
-
-terminate(normal, #rep_state{rep_details = #rep{id = RepId} = Rep,
-    checkpoint_history = CheckpointHistory} = State) ->
-    terminate_cleanup(State),
-    couch_replicator_notifier:notify({finished, RepId, CheckpointHistory}),
-    couch_replicator_manager:replication_completed(Rep, rep_stats(State));
-
-terminate(shutdown, #rep_state{rep_details = #rep{id = RepId}} = State) ->
-    % cancelled replication throught ?MODULE:cancel_replication/1
-    couch_replicator_notifier:notify({error, RepId, <<"cancelled">>}),
-    terminate_cleanup(State);
-
-terminate(shutdown, {error, Class, Error, Stack, InitArgs}) ->
-    #rep{id=RepId} = InitArgs,
-    couch_stats:increment_counter([couch_replicator, failed_starts]),
-    CleanInitArgs = rep_strip_creds(InitArgs),
-    couch_log:error("~p:~p: Replication failed to start for args ~p: ~p",
-             [Class, Error, CleanInitArgs, Stack]),
-    case Error of
-    {unauthorized, DbUri} ->
-        NotifyError = {unauthorized, <<"unauthorized to access or create database ", DbUri/binary>>};
-    {db_not_found, DbUri} ->
-        NotifyError = {db_not_found, <<"could not open ", DbUri/binary>>};
-    _ ->
-        NotifyError = Error
-    end,
-    couch_replicator_notifier:notify({error, RepId, NotifyError}),
-    couch_replicator_manager:replication_error(InitArgs, NotifyError);
-terminate(Reason, State) ->
-    #rep_state{
-        source_name = Source,
-        target_name = Target,
-        rep_details = #rep{id = {BaseId, Ext} = RepId} = Rep
-    } = State,
-    couch_log:error("Replication `~s` (`~s` -> `~s`) failed: ~s",
-        [BaseId ++ Ext, Source, Target, to_binary(Reason)]),
-    terminate_cleanup(State),
-    couch_replicator_notifier:notify({error, RepId, Reason}),
-    couch_replicator_manager:replication_error(Rep, Reason).
-
-terminate_cleanup(State) ->
-    update_task(State),
-    stop_db_compaction_notifier(State#rep_state.source_db_compaction_notifier),
-    stop_db_compaction_notifier(State#rep_state.target_db_compaction_notifier),
-    couch_replicator_api_wrap:db_close(State#rep_state.source),
-    couch_replicator_api_wrap:db_close(State#rep_state.target).
-
-
-format_status(_Opt, [_PDict, State]) ->
-    [{data, [{"State", state_strip_creds(State)}]}].
-
-
-do_last_checkpoint(#rep_state{seqs_in_progress = [],
-    highest_seq_done = {_Ts, ?LOWEST_SEQ}} = State) ->
-    {stop, normal, cancel_timer(State)};
-do_last_checkpoint(#rep_state{seqs_in_progress = [],
-    highest_seq_done = Seq} = State) ->
-    case do_checkpoint(State#rep_state{current_through_seq = Seq}) of
-    {ok, NewState} ->
-        couch_stats:increment_counter([couch_replicator, checkpoints, success]),
-        {stop, normal, cancel_timer(NewState)};
-    Error ->
-        couch_stats:increment_counter([couch_replicator, checkpoints, failure]),
-        {stop, Error, State}
-    end.
-
-
-start_timer(State) ->
-    After = State#rep_state.checkpoint_interval,
-    case timer:apply_after(After, gen_server, cast, [self(), checkpoint]) of
-    {ok, Ref} ->
-        Ref;
-    Error ->
-        couch_log:error("Replicator, error scheduling checkpoint:  ~p", [Error]),
-        nil
-    end.
-
-
-cancel_timer(#rep_state{timer = nil} = State) ->
-    State;
-cancel_timer(#rep_state{timer = Timer} = State) ->
-    {ok, cancel} = timer:cancel(Timer),
-    State#rep_state{timer = nil}.
-
-
-init_state(Rep) ->
-    #rep{
-        id = {BaseId, _Ext},
-        source = Src0, target = Tgt,
-        options = Options, user_ctx = UserCtx,
-        type = Type, view = View
-    } = Rep,
-    % Adjust minimum number of http source connections to 2 to avoid deadlock
-    Src = adjust_maxconn(Src0, BaseId),
-    {ok, Source} = couch_replicator_api_wrap:db_open(Src, [{user_ctx, UserCtx}]),
-    {ok, Target} = couch_replicator_api_wrap:db_open(Tgt, [{user_ctx, UserCtx}],
-        get_value(create_target, Options, false)),
-
-    {ok, SourceInfo} = couch_replicator_api_wrap:get_db_info(Source),
-    {ok, TargetInfo} = couch_replicator_api_wrap:get_db_info(Target),
-
-    [SourceLog, TargetLog] = find_replication_logs([Source, Target], Rep),
-
-    {StartSeq0, History} = compare_replication_logs(SourceLog, TargetLog),
-    StartSeq1 = get_value(since_seq, Options, StartSeq0),
-    StartSeq = {0, StartSeq1},
-
-    SourceSeq = get_value(<<"update_seq">>, SourceInfo, ?LOWEST_SEQ),
-
-    #doc{body={CheckpointHistory}} = SourceLog,
-    State = #rep_state{
-        rep_details = Rep,
-        source_name = couch_replicator_api_wrap:db_uri(Source),
-        target_name = couch_replicator_api_wrap:db_uri(Target),
-        source = Source,
-        target = Target,
-        history = History,
-        checkpoint_history = {[{<<"no_changes">>, true}| CheckpointHistory]},
-        start_seq = StartSeq,
-        current_through_seq = StartSeq,
-        committed_seq = StartSeq,
-        source_log = SourceLog,
-        target_log = TargetLog,
-        rep_starttime = httpd_util:rfc1123_date(),
-        src_starttime = get_value(<<"instance_start_time">>, SourceInfo),
-        tgt_starttime = get_value(<<"instance_start_time">>, TargetInfo),
-        session_id = couch_uuids:random(),
-        source_db_compaction_notifier =
-            start_db_compaction_notifier(Source, self()),
-        target_db_compaction_notifier =
-            start_db_compaction_notifier(Target, self()),
-        source_monitor = db_monitor(Source),
-        target_monitor = db_monitor(Target),
-        source_seq = SourceSeq,
-        use_checkpoints = get_value(use_checkpoints, Options, true),
-        checkpoint_interval = get_value(checkpoint_interval, Options,
-                                        ?DEFAULT_CHECKPOINT_INTERVAL),
-        type = Type,
-        view = View
-    },
-    State#rep_state{timer = start_timer(State)}.
-
-
-find_replication_logs(DbList, #rep{id = {BaseId, _}} = Rep) ->
-    LogId = ?l2b(?LOCAL_DOC_PREFIX ++ BaseId),
-    fold_replication_logs(DbList, ?REP_ID_VERSION, LogId, LogId, Rep, []).
-
-
-fold_replication_logs([], _Vsn, _LogId, _NewId, _Rep, Acc) ->
-    lists:reverse(Acc);
-
-fold_replication_logs([Db | Rest] = Dbs, Vsn, LogId, NewId, Rep, Acc) ->
-    case couch_replicator_api_wrap:open_doc(Db, LogId, [ejson_body]) of
-    {error, <<"not_found">>} when Vsn > 1 ->
-        OldRepId = couch_replicator_utils:replication_id(Rep, Vsn - 1),
-        fold_replication_logs(Dbs, Vsn - 1,
-            ?l2b(?LOCAL_DOC_PREFIX ++ OldRepId), NewId, Rep, Acc);
-    {error, <<"not_found">>} ->
-        fold_replication_logs(
-            Rest, ?REP_ID_VERSION, NewId, NewId, Rep, [#doc{id = NewId} | Acc]);
-    {ok, Doc} when LogId =:= NewId ->
-        fold_replication_logs(
-            Rest, ?REP_ID_VERSION, NewId, NewId, Rep, [Doc | Acc]);
-    {ok, Doc} ->
-        MigratedLog = #doc{id = NewId, body = Doc#doc.body},
-        fold_replication_logs(
-            Rest, ?REP_ID_VERSION, NewId, NewId, Rep, [MigratedLog | Acc])
-    end.
-
-
-spawn_changes_manager(Parent, ChangesQueue, BatchSize) ->
-    spawn_link(fun() ->
-        changes_manager_loop_open(Parent, ChangesQueue, BatchSize, 1)
+t_admin_is_always_authorized() ->
+    ?_test(begin
+        expect_rep_user_ctx(<<"someuser">>, <<"_admin">>),
+        UserCtx = #user_ctx{name = <<"adm">>, roles = [<<"_admin">>]},
+        ?assertEqual(ok, check_authorization(<<"RepId">>, UserCtx))
     end).
 
-changes_manager_loop_open(Parent, ChangesQueue, BatchSize, Ts) ->
-    receive
-    {get_changes, From} ->
-        case couch_work_queue:dequeue(ChangesQueue, BatchSize) of
-        closed ->
-            From ! {closed, self()};
-        {ok, Changes} ->
-            #doc_info{high_seq = Seq} = lists:last(Changes),
-            ReportSeq = {Ts, Seq},
-            ok = gen_server:cast(Parent, {report_seq, ReportSeq}),
-            From ! {changes, self(), Changes, ReportSeq}
-        end,
-        changes_manager_loop_open(Parent, ChangesQueue, BatchSize, Ts + 1)
-    end.
+
+t_username_must_match() ->
+     ?_test(begin
+        expect_rep_user_ctx(<<"user">>, <<"somerole">>),
+        UserCtx1 = #user_ctx{name = <<"user">>, roles = [<<"somerole">>]},
+        ?assertEqual(ok, check_authorization(<<"RepId">>, UserCtx1)),
+        UserCtx2 = #user_ctx{name = <<"other">>, roles = [<<"somerole">>]},
+        ?assertThrow({unauthorized, _}, check_authorization(<<"RepId">>, UserCtx2))
+    end).
 
 
-do_checkpoint(#rep_state{use_checkpoints=false} = State) ->
-    NewState = State#rep_state{checkpoint_history = {[{<<"use_checkpoints">>, false}]} },
-    {ok, NewState};
-do_checkpoint(#rep_state{current_through_seq=Seq, committed_seq=Seq} = State) ->
-    update_task(State),
-    {ok, State};
-do_checkpoint(State) ->
-    #rep_state{
-        source_name=SourceName,
-        target_name=TargetName,
-        source = Source,
-        target = Target,
-        history = OldHistory,
-        start_seq = {_, StartSeq},
-        current_through_seq = {_Ts, NewSeq} = NewTsSeq,
-        source_log = SourceLog,
-        target_log = TargetLog,
-        rep_starttime = ReplicationStartTime,
-        src_starttime = SrcInstanceStartTime,
-        tgt_starttime = TgtInstanceStartTime,
-        stats = Stats,
-        rep_details = #rep{options = Options},
-        session_id = SessionId
-    } = State,
-    case commit_to_both(Source, Target) of
-    {source_error, Reason} ->
-         {checkpoint_commit_failure,
-             <<"Failure on source commit: ", (to_binary(Reason))/binary>>};
-    {target_error, Reason} ->
-         {checkpoint_commit_failure,
-             <<"Failure on target commit: ", (to_binary(Reason))/binary>>};
-    {SrcInstanceStartTime, TgtInstanceStartTime} ->
-        couch_log:notice("recording a checkpoint for `~s` -> `~s` at source update_seq ~p",
-            [SourceName, TargetName, NewSeq]),
-        StartTime = ?l2b(ReplicationStartTime),
-        EndTime = ?l2b(httpd_util:rfc1123_date()),
-        NewHistoryEntry = {[
-            {<<"session_id">>, SessionId},
-            {<<"start_time">>, StartTime},
-            {<<"end_time">>, EndTime},
-            {<<"start_last_seq">>, StartSeq},
-            {<<"end_last_seq">>, NewSeq},
-            {<<"recorded_seq">>, NewSeq},
-            {<<"missing_checked">>, couch_replicator_stats:missing_checked(Stats)},
-            {<<"missing_found">>, couch_replicator_stats:missing_found(Stats)},
-            {<<"docs_read">>, couch_replicator_stats:docs_read(Stats)},
-            {<<"docs_written">>, couch_replicator_stats:docs_written(Stats)},
-            {<<"doc_write_failures">>, couch_replicator_stats:doc_write_failures(Stats)}
+t_replication_not_found() ->
+     ?_test(begin
+        meck:expect(couch_replicator_scheduler, rep_state, 1, nil),
+        UserCtx1 = #user_ctx{name = <<"user">>, roles = [<<"somerole">>]},
+        ?assertEqual(not_found, check_authorization(<<"RepId">>, UserCtx1)),
+        UserCtx2 = #user_ctx{name = <<"adm">>, roles = [<<"_admin">>]},
+        ?assertEqual(not_found, check_authorization(<<"RepId">>, UserCtx2))
+    end).
+
+
+expect_rep_user_ctx(Name, Role) ->
+    meck:expect(couch_replicator_scheduler, rep_state,
+        fun(_Id) ->
+            UserCtx = #user_ctx{name = Name, roles = [Role]},
+            #rep{user_ctx = UserCtx}
+        end).
+
+
+strip_url_creds_test_() ->
+     {
+        foreach,
+        fun () -> meck:expect(config, get, fun(_, _, Default) -> Default end) end,
+        fun (_) -> meck:unload() end,
+        [
+            t_strip_local_db_creds(),
+            t_strip_http_basic_creds(),
+            t_strip_http_props_creds()
+        ]
+    }.
+
+
+t_strip_local_db_creds() ->
+    ?_test(?assertEqual(<<"localdb">>, strip_url_creds(<<"localdb">>))).
+
+
+t_strip_http_basic_creds() ->
+    ?_test(begin
+        Url1 = <<"http://adm:pass@host/db">>,
+        ?assertEqual(<<"http://adm:*****@host/db/">>, strip_url_creds(Url1)),
+        Url2 = <<"https://adm:pass@host/db">>,
+        ?assertEqual(<<"https://adm:*****@host/db/">>, strip_url_creds(Url2))
+    end).
+
+
+t_strip_http_props_creds() ->
+    ?_test(begin
+        Props1 = {[{<<"url">>, <<"http://adm:pass@host/db">>}]},
+        ?assertEqual(<<"http://adm:*****@host/db/">>, strip_url_creds(Props1)),
+        Props2 = {[ {<<"url">>, <<"http://host/db">>},
+            {<<"headers">>, {[{<<"Authorization">>, <<"Basic pa55">>}]}}
         ]},
-        BaseHistory = [
-            {<<"session_id">>, SessionId},
-            {<<"source_last_seq">>, NewSeq},
-            {<<"replication_id_version">>, ?REP_ID_VERSION}
-        ] ++ case get_value(doc_ids, Options) of
-        undefined ->
-            [];
-        _DocIds ->
-            % backwards compatibility with the result of a replication by
-            % doc IDs in versions 0.11.x and 1.0.x
-            % TODO: deprecate (use same history format, simplify code)
-            [
-                {<<"start_time">>, StartTime},
-                {<<"end_time">>, EndTime},
-                {<<"docs_read">>, couch_replicator_stats:docs_read(Stats)},
-                {<<"docs_written">>, couch_replicator_stats:docs_written(Stats)},
-                {<<"doc_write_failures">>, couch_replicator_stats:doc_write_failures(Stats)}
-            ]
-        end,
-        % limit history to 50 entries
-        NewRepHistory = {
-            BaseHistory ++
-            [{<<"history">>, lists:sublist([NewHistoryEntry | OldHistory], 50)}]
-        },
+        ?assertEqual(<<"http://host/db/">>, strip_url_creds(Props2))
+    end).
 
-        try
-            {SrcRevPos, SrcRevId} = update_checkpoint(
-                Source, SourceLog#doc{body = NewRepHistory}, source),
-            {TgtRevPos, TgtRevId} = update_checkpoint(
-                Target, TargetLog#doc{body = NewRepHistory}, target),
-            NewState = State#rep_state{
-                checkpoint_history = NewRepHistory,
-                committed_seq = NewTsSeq,
-                source_log = SourceLog#doc{revs={SrcRevPos, [SrcRevId]}},
-                target_log = TargetLog#doc{revs={TgtRevPos, [TgtRevId]}}
-            },
-            update_task(NewState),
-            {ok, NewState}
-        catch throw:{checkpoint_commit_failure, _} = Failure ->
-            Failure
-        end;
-    {SrcInstanceStartTime, _NewTgtInstanceStartTime} ->
-        {checkpoint_commit_failure, <<"Target database out of sync. "
-            "Try to increase max_dbs_open at the target's server.">>};
-    {_NewSrcInstanceStartTime, TgtInstanceStartTime} ->
-        {checkpoint_commit_failure, <<"Source database out of sync. "
-            "Try to increase max_dbs_open at the source's server.">>};
-    {_NewSrcInstanceStartTime, _NewTgtInstanceStartTime} ->
-        {checkpoint_commit_failure, <<"Source and target databases out of "
-            "sync. Try to increase max_dbs_open at both servers.">>}
-    end.
-
-
-update_checkpoint(Db, Doc, DbType) ->
-    try
-        update_checkpoint(Db, Doc)
-    catch throw:{checkpoint_commit_failure, Reason} ->
-        throw({checkpoint_commit_failure,
-            <<"Error updating the ", (to_binary(DbType))/binary,
-                " checkpoint document: ", (to_binary(Reason))/binary>>})
-    end.
-
-update_checkpoint(Db, #doc{id = LogId, body = LogBody} = Doc) ->
-    try
-        case couch_replicator_api_wrap:update_doc(Db, Doc, [delay_commit]) of
-        {ok, PosRevId} ->
-            PosRevId;
-        {error, Reason} ->
-            throw({checkpoint_commit_failure, Reason})
-        end
-    catch throw:conflict ->
-        case (catch couch_replicator_api_wrap:open_doc(Db, LogId, [ejson_body])) of
-        {ok, #doc{body = LogBody, revs = {Pos, [RevId | _]}}} ->
-            % This means that we were able to update successfully the
-            % checkpoint doc in a previous attempt but we got a connection
-            % error (timeout for e.g.) before receiving the success response.
-            % Therefore the request was retried and we got a conflict, as the
-            % revision we sent is not the current one.
-            % We confirm this by verifying the doc body we just got is the same
-            % that we have just sent.
-            {Pos, RevId};
-        _ ->
-            throw({checkpoint_commit_failure, conflict})
-        end
-    end.
-
-
-commit_to_both(Source, Target) ->
-    % commit the src async
-    ParentPid = self(),
-    SrcCommitPid = spawn_link(
-        fun() ->
-            Result = (catch couch_replicator_api_wrap:ensure_full_commit(Source)),
-            ParentPid ! {self(), Result}
-        end),
-
-    % commit tgt sync
-    TargetResult = (catch couch_replicator_api_wrap:ensure_full_commit(Target)),
-
-    SourceResult = receive
-    {SrcCommitPid, Result} ->
-        unlink(SrcCommitPid),
-        receive {'EXIT', SrcCommitPid, _} -> ok after 0 -> ok end,
-        Result;
-    {'EXIT', SrcCommitPid, Reason} ->
-        {error, Reason}
-    end,
-    case TargetResult of
-    {ok, TargetStartTime} ->
-        case SourceResult of
-        {ok, SourceStartTime} ->
-            {SourceStartTime, TargetStartTime};
-        SourceError ->
-            {source_error, SourceError}
-        end;
-    TargetError ->
-        {target_error, TargetError}
-    end.
-
-
-compare_replication_logs(SrcDoc, TgtDoc) ->
-    #doc{body={RepRecProps}} = SrcDoc,
-    #doc{body={RepRecPropsTgt}} = TgtDoc,
-    case get_value(<<"session_id">>, RepRecProps) ==
-            get_value(<<"session_id">>, RepRecPropsTgt) of
-    true ->
-        % if the records have the same session id,
-        % then we have a valid replication history
-        OldSeqNum = get_value(<<"source_last_seq">>, RepRecProps, ?LOWEST_SEQ),
-        OldHistory = get_value(<<"history">>, RepRecProps, []),
-        {OldSeqNum, OldHistory};
-    false ->
-        SourceHistory = get_value(<<"history">>, RepRecProps, []),
-        TargetHistory = get_value(<<"history">>, RepRecPropsTgt, []),
-        couch_log:notice("Replication records differ. "
-                "Scanning histories to find a common ancestor.", []),
-        couch_log:debug("Record on source:~p~nRecord on target:~p~n",
-                [RepRecProps, RepRecPropsTgt]),
-        compare_rep_history(SourceHistory, TargetHistory)
-    end.
-
-compare_rep_history(S, T) when S =:= [] orelse T =:= [] ->
-    couch_log:notice("no common ancestry -- performing full replication", []),
-    {?LOWEST_SEQ, []};
-compare_rep_history([{S} | SourceRest], [{T} | TargetRest] = Target) ->
-    SourceId = get_value(<<"session_id">>, S),
-    case has_session_id(SourceId, Target) of
-    true ->
-        RecordSeqNum = get_value(<<"recorded_seq">>, S, ?LOWEST_SEQ),
-        couch_log:notice("found a common replication record with source_seq ~p",
-            [RecordSeqNum]),
-        {RecordSeqNum, SourceRest};
-    false ->
-        TargetId = get_value(<<"session_id">>, T),
-        case has_session_id(TargetId, SourceRest) of
-        true ->
-            RecordSeqNum = get_value(<<"recorded_seq">>, T, ?LOWEST_SEQ),
-            couch_log:notice("found a common replication record with source_seq ~p",
-                [RecordSeqNum]),
-            {RecordSeqNum, TargetRest};
-        false ->
-            compare_rep_history(SourceRest, TargetRest)
-        end
-    end.
-
-
-has_session_id(_SessionId, []) ->
-    false;
-has_session_id(SessionId, [{Props} | Rest]) ->
-    case get_value(<<"session_id">>, Props, nil) of
-    SessionId ->
-        true;
-    _Else ->
-        has_session_id(SessionId, Rest)
-    end.
-
-
-db_monitor(#db{} = Db) ->
-    couch_db:monitor(Db);
-db_monitor(_HttpDb) ->
-    nil.
-
-get_pending_count(St) ->
-    Rep = St#rep_state.rep_details,
-    Timeout = get_value(connection_timeout, Rep#rep.options),
-    TimeoutMicro = Timeout * 1000,
-    case get(pending_count_state) of
-        {LastUpdate, PendingCount} ->
-            case timer:now_diff(os:timestamp(), LastUpdate) > TimeoutMicro of
-                true ->
-                    NewPendingCount = get_pending_count_int(St),
-                    put(pending_count_state, {os:timestamp(), NewPendingCount}),
-                    NewPendingCount;
-                false ->
-                    PendingCount
-            end;
-        undefined ->
-            NewPendingCount = get_pending_count_int(St),
-            put(pending_count_state, {os:timestamp(), NewPendingCount}),
-            NewPendingCount
-    end.
-
-
-get_pending_count_int(#rep_state{source = #httpdb{} = Db0}=St) ->
-    {_, Seq} = St#rep_state.highest_seq_done,
-    Db = Db0#httpdb{retries = 3},
-    case (catch couch_replicator_api_wrap:get_pending_count(Db, Seq)) of
-    {ok, Pending} ->
-        Pending;
-    _ ->
-        null
-    end;
-get_pending_count_int(#rep_state{source = Db}=St) ->
-    {_, Seq} = St#rep_state.highest_seq_done,
-    {ok, Pending} = couch_replicator_api_wrap:get_pending_count(Db, Seq),
-    Pending.
-
-
-update_task(State) ->
-    #rep_state{
-        current_through_seq = {_, ThroughSeq},
-        highest_seq_done = {_, HighestSeq}
-    } = State,
-    couch_task_status:update(
-        rep_stats(State) ++ [
-        {source_seq, HighestSeq},
-        {through_seq, ThroughSeq}
-    ]).
-
-
-rep_stats(State) ->
-    #rep_state{
-        committed_seq = {_, CommittedSeq},
-        stats = Stats
-    } = State,
-    [
-        {revisions_checked, couch_replicator_stats:missing_checked(Stats)},
-        {missing_revisions_found, couch_replicator_stats:missing_found(Stats)},
-        {docs_read, couch_replicator_stats:docs_read(Stats)},
-        {docs_written, couch_replicator_stats:docs_written(Stats)},
-        {changes_pending, get_pending_count(State)},
-        {doc_write_failures, couch_replicator_stats:doc_write_failures(Stats)},
-        {checkpointed_source_seq, CommittedSeq}
-    ].
+-endif.
diff --git a/src/couch_replicator.hrl b/src/couch_replicator.hrl
index d3485c0..b8669e8 100644
--- a/src/couch_replicator.hrl
+++ b/src/couch_replicator.hrl
@@ -13,13 +13,30 @@
 -define(REP_ID_VERSION, 3).
 
 -record(rep, {
-    id,
-    source,
-    target,
-    options,
-    user_ctx,
-    type = db,
-    view = nil,
-    doc_id,
-    db_name = null
+    id :: rep_id() | '_',
+    source :: any() | '_',
+    target :: any() | '_',
+    options :: [_] | '_',
+    user_ctx :: any() | '_',
+    type = db :: atom() | '_',
+    view = nil :: any() | '_',
+    doc_id :: any() | '_',
+    db_name = null :: null | binary() | '_',
+    start_time :: erlang:timestamp() | '_'
+}).
+
+-type rep_id() :: {string(), string()}.
+-type db_doc_id() :: {binary(), binary() | '_'}.
+-type seconds() :: non_neg_integer().
+-type rep_start_result() ::
+    {ok, rep_id()} |
+    ignore |
+    {temporary_error, binary()} |
+    {permanent_failure, binary()}.
+
+
+-record(doc_worker_result, {
+    id :: db_doc_id(),
+    wref :: reference(),
+    result :: rep_start_result()
 }).
diff --git a/src/couch_replicator_api_wrap.erl b/src/couch_replicator_api_wrap.erl
index e5f6253..a0d08d7 100644
--- a/src/couch_replicator_api_wrap.erl
+++ b/src/couch_replicator_api_wrap.erl
@@ -38,7 +38,8 @@
     open_doc/3,
     open_doc_revs/6,
     changes_since/5,
-    db_uri/1
+    db_uri/1,
+    normalize_db/1
     ]).
 
 -import(couch_replicator_httpc, [
@@ -290,6 +291,8 @@
             throw(missing_doc);
         {'DOWN', Ref, process, Pid, {{nocatch, {missing_stub,_} = Stub}, _}} ->
             throw(Stub);
+        {'DOWN', Ref, process, Pid, {http_request_failed, _, _, max_backoff}} ->
+            exit(max_backoff);
         {'DOWN', Ref, process, Pid, request_uri_too_long} ->
             NewMaxLen = get_value(max_url_len, Options, ?MAX_URL_LEN) div 2,
             case NewMaxLen < ?MIN_URL_LEN of
@@ -517,6 +520,8 @@
                         end)
             end)
     catch
+        exit:{http_request_failed, _, _, max_backoff} ->
+            exit(max_backoff);
         exit:{http_request_failed, _, _, {error, {connection_closed,
                 mid_stream}}} ->
             throw(retry_no_limit);
@@ -985,3 +990,46 @@
         _ ->
             Default
     end.
+
+
+% Normalize an #httpdb{} or #db{} record such that it can be used for
+% comparisons. This means remove things like pids and also sort options / props.
+normalize_db(#httpdb{} = HttpDb) ->
+    #httpdb{
+        url = HttpDb#httpdb.url,
+        oauth = HttpDb#httpdb.oauth,
+        headers = lists:keysort(1, HttpDb#httpdb.headers),
+        timeout = HttpDb#httpdb.timeout,
+        ibrowse_options = lists:keysort(1, HttpDb#httpdb.ibrowse_options),
+        retries = HttpDb#httpdb.retries,
+        http_connections = HttpDb#httpdb.http_connections
+    };
+
+normalize_db(<<DbName/binary>>) ->
+    DbName.
+
+
+-ifdef(TEST).
+
+-include_lib("eunit/include/eunit.hrl").
+
+
+normalize_http_db_test() ->
+    HttpDb =  #httpdb{
+        url = "http://host/db",
+        oauth = #oauth{},
+        headers = [{"k2","v2"}, {"k1","v1"}],
+        timeout = 30000,
+        ibrowse_options = [{k2, v2}, {k1, v1}],
+        retries = 10,
+        http_connections = 20
+    },
+    Expected = HttpDb#httpdb{
+        headers = [{"k1","v1"}, {"k2","v2"}],
+        ibrowse_options = [{k1, v1}, {k2, v2}]
+    },
+    ?assertEqual(Expected, normalize_db(HttpDb)),
+    ?assertEqual(<<"local">>, normalize_db(<<"local">>)).
+
+
+-endif.
diff --git a/src/couch_replicator_api_wrap.hrl b/src/couch_replicator_api_wrap.hrl
index 24e204b..fc94054 100644
--- a/src/couch_replicator_api_wrap.hrl
+++ b/src/couch_replicator_api_wrap.hrl
@@ -25,7 +25,8 @@
     wait = 250,         % milliseconds
     httpc_pool = nil,
     http_connections,
-    backoff = 25
+    first_error_timestamp = nil,
+    proxy_url
 }).
 
 -record(oauth, {
diff --git a/src/couch_replicator_clustering.erl b/src/couch_replicator_clustering.erl
new file mode 100644
index 0000000..4a6abd7
--- /dev/null
+++ b/src/couch_replicator_clustering.erl
@@ -0,0 +1,218 @@
+% 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.
+
+
+% Maintain cluster membership and stability notifications for replications.
+% On changes to cluster membership, broadcast events to `replication` gen_event.
+% Listeners will get `{cluster, stable}` or `{cluster, unstable}` events.
+%
+% Cluster stability is defined as "there have been no nodes added or removed in
+% last `QuietPeriod` seconds". QuietPeriod value is configurable. To ensure a
+% speedier startup, during initialization there is a shorter StartupQuietPeriod in
+% effect (also configurable).
+%
+% This module is also in charge of calculating ownership of replications based on
+% where their _repicator db documents shards live.
+
+
+-module(couch_replicator_clustering).
+-behaviour(gen_server).
+-behaviour(config_listener).
+
+% public API
+-export([start_link/0, owner/2, is_stable/0]).
+-export([link_cluster_event_listener/1]).
+
+% gen_server callbacks
+-export([init/1, handle_call/3, handle_info/2, handle_cast/2,
+         code_change/3, terminate/2]).
+
+% config_listener callbacks
+-export([handle_config_change/5, handle_config_terminate/3]).
+
+-include_lib("couch/include/couch_db.hrl").
+-include_lib("mem3/include/mem3.hrl").
+
+-define(DEFAULT_QUIET_PERIOD, 60). % seconds
+-define(DEFAULT_START_PERIOD, 5). % seconds
+-define(RELISTEN_DELAY, 5000).
+
+-record(state, {
+    start_time :: erlang:timestamp(),
+    last_change :: erlang:timestamp(),
+    period = ?DEFAULT_QUIET_PERIOD :: non_neg_integer(),
+    start_period = ?DEFAULT_START_PERIOD :: non_neg_integer(),
+    timer :: timer:tref()
+}).
+
+
+-spec start_link() -> {ok, pid()} | ignore | {error, term()}.
+start_link() ->
+    gen_server:start_link({local, ?MODULE}, ?MODULE, [], []).
+
+
+% owner/2 function computes ownership for a {DbName, DocId} tuple
+% `unstable` if cluster is considered to be unstable i.e. it has changed
+% recently, or returns node() which of the owner.
+%
+-spec owner(Dbname :: binary(), DocId :: binary()) -> node() | unstable.
+owner(<<"shards/", _/binary>> = DbName, DocId) ->
+    case is_stable() of
+        false ->
+            unstable;
+        true ->
+            owner_int(DbName, DocId)
+    end;
+owner(_DbName, _DocId) ->
+    node().
+
+
+-spec is_stable() -> true | false.
+is_stable() ->
+    gen_server:call(?MODULE, is_stable).
+
+
+% Convenience function for gen_servers to subscribe to {cluster, stable} and
+% {cluster, unstable} events from couch_replicator clustering module.
+-spec link_cluster_event_listener(pid()) -> pid().
+link_cluster_event_listener(GenServer) when is_pid(GenServer) ->
+    CallbackFun =
+        fun(Event = {cluster, _}) -> gen_server:cast(GenServer, Event);
+           (_) -> ok
+        end,
+    {ok, Pid} = couch_replicator_notifier:start_link(CallbackFun),
+    Pid.
+
+
+% gen_server callbacks
+
+init([]) ->
+    net_kernel:monitor_nodes(true),
+    ok = config:listen_for_changes(?MODULE, nil),
+    Period = abs(config:get_integer("replicator", "cluster_quiet_period",
+        ?DEFAULT_QUIET_PERIOD)),
+    StartPeriod = abs(config:get_integer("replicator", "cluster_start_period",
+        ?DEFAULT_START_PERIOD)),
+    couch_log:debug("Initialized clustering gen_server ~w", [self()]),
+    couch_stats:update_gauge([couch_replicator, cluster_is_stable], 0),
+    {ok, #state{
+        start_time = os:timestamp(),
+        last_change = os:timestamp(),
+        period = Period,
+        start_period = StartPeriod,
+        timer = new_timer(StartPeriod)
+    }}.
+
+
+terminate(_Reason, _State) ->
+    ok.
+
+
+handle_call(is_stable, _From, State) ->
+    {reply, is_stable(State), State}.
+
+
+handle_cast({set_period, QuietPeriod}, State) when
+    is_integer(QuietPeriod), QuietPeriod > 0 ->
+    {noreply, State#state{period = QuietPeriod}}.
+
+
+handle_info({nodeup, Node}, State) ->
+    Timer = new_timer(interval(State)),
+    couch_replicator_notifier:notify({cluster, unstable}),
+    couch_stats:update_gauge([couch_replicator, cluster_is_stable], 0),
+    couch_log:notice("~s : nodeup ~s, cluster unstable", [?MODULE, Node]),
+    {noreply, State#state{last_change = os:timestamp(), timer = Timer}};
+
+handle_info({nodedown, Node}, State) ->
+    Timer = new_timer(interval(State)),
+    couch_replicator_notifier:notify({cluster, unstable}),
+    couch_stats:update_gauge([couch_replicator, cluster_is_stable], 0),
+    couch_log:notice("~s : nodedown ~s, cluster unstable", [?MODULE, Node]),
+    {noreply, State#state{last_change = os:timestamp(), timer = Timer}};
+
+handle_info(stability_check, State) ->
+   timer:cancel(State#state.timer),
+   case is_stable(State) of
+       true ->
+           couch_replicator_notifier:notify({cluster, stable}),
+           couch_stats:update_gauge([couch_replicator, cluster_is_stable], 1),
+           couch_log:notice("~s : publishing cluster `stable` event", [?MODULE]),
+           {noreply, State};
+       false ->
+           Timer = new_timer(interval(State)),
+           {noreply, State#state{timer = Timer}}
+   end;
+
+handle_info(restart_config_listener, State) ->
+    ok = config:listen_for_changes(?MODULE, nil),
+    {noreply, State}.
+
+
+code_change(_OldVsn, State, _Extra) ->
+    {ok, State}.
+
+
+%% Internal functions
+
+-spec new_timer(non_neg_integer()) -> timer:tref().
+new_timer(IntervalSec) ->
+    {ok, Timer} = timer:send_after(IntervalSec * 1000, stability_check),
+    Timer.
+
+
+-spec interval(#state{}) -> non_neg_integer().
+interval(#state{period = Period, start_period = Period0, start_time = T0}) ->
+    case now_diff_sec(T0) > Period of
+        true ->
+            % Normal operation
+            Period;
+        false ->
+            % During startup
+            Period0
+    end.
+
+
+-spec is_stable(#state{}) -> boolean().
+is_stable(#state{last_change = TS} = State) ->
+    now_diff_sec(TS) > interval(State).
+
+
+-spec now_diff_sec(erlang:timestamp()) -> non_neg_integer().
+now_diff_sec(Time) ->
+    case timer:now_diff(os:timestamp(), Time) of
+        USec when USec < 0 ->
+            0;
+        USec when USec >= 0 ->
+             USec / 1000000
+    end.
+
+
+handle_config_change("replicator", "cluster_quiet_period", V, _, S) ->
+    ok = gen_server:cast(?MODULE, {set_period, list_to_integer(V)}),
+    {ok, S};
+handle_config_change(_, _, _, _, S) ->
+    {ok, S}.
+
+
+handle_config_terminate(_, stop, _) -> ok;
+handle_config_terminate(_S, _R, _St) ->
+    Pid = whereis(?MODULE),
+    erlang:send_after(?RELISTEN_DELAY, Pid, restart_config_listener).
+
+
+-spec owner_int(binary(), binary()) -> node().
+owner_int(DbName, DocId) ->
+    Live = [node() | nodes()],
+    Nodes = [N || #shard{node=N} <- mem3:shards(mem3:dbname(DbName), DocId),
+                  lists:member(N, Live)],
+    hd(mem3_util:rotate_list({DbName, DocId}, lists:sort(Nodes))).
diff --git a/src/couch_replicator_connection.erl b/src/couch_replicator_connection.erl
new file mode 100644
index 0000000..e580663
--- /dev/null
+++ b/src/couch_replicator_connection.erl
@@ -0,0 +1,211 @@
+% 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_replicator_connection).
+
+-behavior(gen_server).
+-behavior(config_listener).
+
+-export([start_link/0]).
+-export([init/1, handle_call/3, handle_cast/2, handle_info/2]).
+-export([code_change/3, terminate/2]).
+
+-export([acquire/1, relinquish/1]).
+
+-export([handle_config_change/5, handle_config_terminate/3]).
+
+-define(DEFAULT_CLOSE_INTERVAL, 90000).
+-define(RELISTEN_DELAY, 5000).
+
+-record(state, {
+    close_interval,
+    timer
+}).
+
+-record(connection, {
+    worker,
+    host,
+    port,
+    mref
+}).
+
+-include_lib("ibrowse/include/ibrowse.hrl").
+
+
+start_link() ->
+    gen_server:start_link({local, ?MODULE}, ?MODULE, [], []).
+
+
+init([]) ->
+    process_flag(trap_exit, true),
+    ?MODULE = ets:new(?MODULE, [named_table, public, {keypos, #connection.worker}]),
+    ok = config:listen_for_changes(?MODULE, nil),
+    Interval = config:get_integer("replicator", "connection_close_interval", ?DEFAULT_CLOSE_INTERVAL),
+    {ok, Timer} = timer:send_after(Interval, close_idle_connections),
+    ibrowse:add_config([{inactivity_timeout, Interval}]),
+    {ok, #state{close_interval=Interval, timer=Timer}}.
+
+
+acquire(URL) when is_binary(URL) ->
+    acquire(binary_to_list(URL));
+
+acquire(URL) ->
+    case gen_server:call(?MODULE, {acquire, URL}) of
+        {ok, Worker} ->
+            link(Worker),
+            {ok, Worker};
+        {error, all_allocated} ->
+            {ok, Pid} = ibrowse:spawn_link_worker_process(URL),
+            ok = gen_server:call(?MODULE, {create, URL, Pid}),
+            {ok, Pid};
+        {error, Reason} ->
+            {error, Reason}
+    end.
+
+
+relinquish(Worker) ->
+    unlink(Worker),
+    gen_server:cast(?MODULE, {relinquish, Worker}).
+
+
+handle_call({acquire, URL}, From, State) ->
+    {Pid, _Ref} = From,
+    case ibrowse_lib:parse_url(URL) of
+        #url{host=Host, port=Port} ->
+            case ets:match_object(?MODULE, #connection{host=Host, port=Port, mref=undefined, _='_'}, 1) of
+                '$end_of_table' ->
+                    {reply, {error, all_allocated}, State};
+                {[Worker], _Cont} ->
+                    couch_stats:increment_counter([couch_replicator, connection, acquires]),
+                    ets:insert(?MODULE, Worker#connection{mref=monitor(process, Pid)}),
+                    {reply, {ok, Worker#connection.worker}, State}
+            end;
+        {error, invalid_uri} ->
+            {reply, {error, invalid_uri}, State}
+    end;
+
+handle_call({create, URL, Worker}, From, State) ->
+    {Pid, _Ref} = From,
+    case ibrowse_lib:parse_url(URL) of
+        #url{host=Host, port=Port} ->
+            link(Worker),
+            couch_stats:increment_counter([couch_replicator, connection, creates]),
+            true = ets:insert_new(
+                ?MODULE,
+                #connection{host=Host, port=Port, worker=Worker, mref=monitor(process, Pid)}
+            ),
+            {reply, ok, State}
+    end.
+
+
+handle_cast({relinquish, WorkerPid}, State) ->
+    couch_stats:increment_counter([couch_replicator, connection, relinquishes]),
+    case ets:lookup(?MODULE, WorkerPid) of
+        [Worker] ->
+            case Worker#connection.mref of
+                MRef when is_reference(MRef) -> demonitor(MRef, [flush]);
+                undefined -> ok
+            end,
+            ets:insert(?MODULE, Worker#connection{mref=undefined});
+        [] ->
+            ok
+    end,
+    {noreply, State};
+
+handle_cast({connection_close_interval, V}, State) ->
+    {ok, cancel} = timer:cancel(State#state.timer),
+    {ok, NewTimer} = timer:send_after(V, close_idle_connections),
+    ibrowse:add_config([{inactivity_timeout, V}]),
+    {noreply, State#state{close_interval=V, timer=NewTimer}}.
+
+
+% owner crashed
+handle_info({'DOWN', Ref, process, _Pid, _Reason}, State) ->
+    couch_stats:increment_counter([couch_replicator, connection, owner_crashes]),
+    ets:match_delete(?MODULE, #connection{mref=Ref, _='_'}),
+    {noreply, State};
+
+% worker crashed
+handle_info({'EXIT', Pid, Reason}, State) ->
+    couch_stats:increment_counter([couch_replicator, connection, worker_crashes]),
+    case ets:lookup(?MODULE, Pid) of
+        [] ->
+            ok;
+        [Worker] ->
+            #connection{host=Host, port=Port} = Worker,
+            maybe_log_worker_death(Host, Port, Reason),
+            case Worker#connection.mref of
+                MRef when is_reference(MRef) -> demonitor(MRef, [flush]);
+                undefined -> ok
+            end,
+            ets:delete(?MODULE, Pid)
+    end,
+    {noreply, State};
+
+handle_info(close_idle_connections, State) ->
+    #state{
+        close_interval=Interval,
+        timer=Timer
+    } = State,
+    Conns = ets:match_object(?MODULE, #connection{mref=undefined, _='_'}),
+    lists:foreach(fun(Conn) ->
+        couch_stats:increment_counter([couch_replicator, connection, closes]),
+        delete_worker(Conn)
+    end, Conns),
+    {ok, cancel} = timer:cancel(Timer),
+    {ok, NewTimer} = timer:send_after(Interval, close_idle_connections),
+    {noreply, State#state{timer=NewTimer}};
+
+handle_info(restart_config_listener, State) ->
+    ok = config:listen_for_changes(?MODULE, nil),
+    {noreply, State}.
+
+
+code_change(_OldVsn, State, _Extra) ->
+    {ok, State}.
+
+
+terminate(_Reason, _State) ->
+    ok.
+
+
+maybe_log_worker_death(_Host, _Port, normal) ->
+    ok;
+
+maybe_log_worker_death(Host, Port, Reason) ->
+    ErrMsg = "Replication connection to: ~p:~p died with reason ~p",
+    couch_log:info(ErrMsg, [Host, Port, Reason]).
+
+
+-spec delete_worker(#connection{}) -> ok.
+delete_worker(Worker) ->
+    ets:delete(?MODULE, Worker#connection.worker),
+    unlink(Worker#connection.worker),
+    spawn(fun() -> ibrowse_http_client:stop(Worker#connection.worker) end),
+    ok.
+
+
+handle_config_change("replicator", "connection_close_interval", V, _, S) ->
+    ok = gen_server:cast(?MODULE, {connection_close_interval,
+        list_to_integer(V)}),
+    {ok, S};
+
+handle_config_change(_, _, _, _, S) ->
+    {ok, S}.
+
+
+handle_config_terminate(_, stop, _) ->
+    ok;
+
+handle_config_terminate(_, _, _) ->
+    Pid = whereis(?MODULE),
+    erlang:send_after(?RELISTEN_DELAY, Pid, restart_config_listener).
diff --git a/src/couch_replicator_db_changes.erl b/src/couch_replicator_db_changes.erl
new file mode 100644
index 0000000..924c24f
--- /dev/null
+++ b/src/couch_replicator_db_changes.erl
@@ -0,0 +1,90 @@
+% 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_replicator_db_changes).
+-behaviour(gen_server).
+
+-export([start_link/0]).
+
+-export([init/1, handle_call/3, handle_info/2, handle_cast/2]).
+-export([code_change/3, terminate/2]).
+
+
+-record(state, {
+   event_listener :: pid(),
+   mdb_changes :: pid() | nil
+}).
+
+
+-spec start_link() ->
+    {ok, pid()} | ignore | {error, any()}.
+start_link() ->
+    gen_server:start_link(?MODULE, [], []).
+
+
+init([]) ->
+    EvtPid = couch_replicator_clustering:link_cluster_event_listener(self()),
+    State = #state{event_listener = EvtPid, mdb_changes = nil},
+    case couch_replicator_clustering:is_stable() of
+        true ->
+            {ok, restart_mdb_changes(State)};
+        false ->
+            {ok, State}
+    end.
+
+
+terminate(_Reason, _State) ->
+    ok.
+
+
+handle_call(_Msg, _From, State) ->
+    {reply, {error, invalid_call}, State}.
+
+
+handle_cast({cluster, unstable}, State) ->
+    {noreply, stop_mdb_changes(State)};
+
+handle_cast({cluster, stable}, State) ->
+    {noreply, restart_mdb_changes(State)}.
+
+
+handle_info(_Msg, State) ->
+    {noreply, State}.
+
+
+code_change(_OldVsn, State, _Extra) ->
+    {ok, State}.
+
+
+-spec restart_mdb_changes(#state{}) -> #state{}.
+restart_mdb_changes(#state{mdb_changes = nil} = State) ->
+    Suffix = <<"_replicator">>,
+    CallbackMod = couch_replicator_doc_processor,
+    Options = [skip_ddocs],
+    {ok, Pid} = couch_multidb_changes:start_link(Suffix, CallbackMod, nil, Options),
+    couch_stats:increment_counter([couch_replicator, db_scans]),
+    couch_log:notice("Started replicator db changes listener ~p", [Pid]),
+    State#state{mdb_changes = Pid};
+
+restart_mdb_changes(#state{mdb_changes = _Pid} = State) ->
+    restart_mdb_changes(stop_mdb_changes(State)).
+
+
+-spec stop_mdb_changes(#state{}) -> #state{}.
+stop_mdb_changes(#state{mdb_changes = nil} = State) ->
+    State;
+
+stop_mdb_changes(#state{mdb_changes = Pid} = State) ->
+    couch_log:notice("Stopping replicator db changes listener ~p", [Pid]),
+    unlink(Pid),
+    exit(Pid, kill),
+    State#state{mdb_changes = nil}.
diff --git a/src/couch_replicator_doc_processor.erl b/src/couch_replicator_doc_processor.erl
new file mode 100644
index 0000000..9c2e2b3
--- /dev/null
+++ b/src/couch_replicator_doc_processor.erl
@@ -0,0 +1,916 @@
+% 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_replicator_doc_processor).
+-behaviour(couch_multidb_changes).
+
+-export([start_link/0]).
+-export([docs/1, doc/2]).
+-export([update_docs/0]).
+-export([get_worker_ref/1]).
+
+% multidb changes callback
+-export([db_created/2, db_deleted/2, db_found/2, db_change/3]).
+
+% gen_server callbacks
+-export([init/1, handle_call/3, handle_info/2, handle_cast/2,
+         code_change/3, terminate/2]).
+
+-include_lib("couch/include/couch_db.hrl").
+-include("couch_replicator.hrl").
+
+-import(couch_replicator_utils, [
+    get_json_value/2,
+    get_json_value/3
+]).
+
+-define(DEFAULT_UPDATE_DOCS, false).
+-define(ERROR_MAX_BACKOFF_EXPONENT, 12).  % ~ 1 day on average
+-define(TS_DAY_SEC, 86400).
+
+-type filter_type() ::  nil | view | user | docids | mango.
+-type repstate() :: initializing | error | scheduled.
+
+
+-record(rdoc, {
+    id :: db_doc_id() | '_' | {any(), '_'},
+    state :: repstate() | '_',
+    rep :: #rep{} | nil | '_',
+    rid :: rep_id() | nil | '_',
+    filter :: filter_type() | '_',
+    info :: binary() | nil | '_',
+    errcnt :: non_neg_integer() | '_',
+    worker :: reference() | nil | '_',
+    last_updated :: erlang:timestamp() | '_'
+}).
+
+
+
+% couch_multidb_changes API callbacks
+
+db_created(DbName, Server) ->
+    couch_stats:increment_counter([couch_replicator, docs, dbs_created]),
+    couch_replicator_docs:ensure_rep_ddoc_exists(DbName),
+    Server.
+
+
+db_deleted(DbName, Server) ->
+    couch_stats:increment_counter([couch_replicator, docs, dbs_deleted]),
+    ok = gen_server:call(?MODULE, {clean_up_replications, DbName}, infinity),
+    Server.
+
+
+db_found(DbName, Server) ->
+    couch_stats:increment_counter([couch_replicator, docs, dbs_found]),
+    couch_replicator_docs:ensure_rep_ddoc_exists(DbName),
+    Server.
+
+
+db_change(DbName, {ChangeProps} = Change, Server) ->
+    couch_stats:increment_counter([couch_replicator, docs, db_changes]),
+    try
+        ok = process_change(DbName, Change)
+    catch
+    _Tag:Error ->
+        {RepProps} = get_json_value(doc, ChangeProps),
+        DocId = get_json_value(<<"_id">>, RepProps),
+        Timestamp = os:timestamp(),
+        couch_replicator_docs:update_failed(DbName, DocId, Error, Timestamp)
+    end,
+    Server.
+
+
+-spec get_worker_ref(db_doc_id()) -> reference() | nil.
+get_worker_ref({DbName, DocId}) when is_binary(DbName), is_binary(DocId) ->
+    case ets:lookup(?MODULE, {DbName, DocId}) of
+        [#rdoc{worker = WRef}] when is_reference(WRef) ->
+            WRef;
+        [#rdoc{worker = nil}] ->
+            nil;
+        [] ->
+            nil
+    end.
+
+
+% Private helpers for multidb changes API, these updates into the doc
+% processor gen_server
+
+process_change(DbName, {Change}) ->
+    {RepProps} = JsonRepDoc = get_json_value(doc, Change),
+    DocId = get_json_value(<<"_id">>, RepProps),
+    Owner = couch_replicator_clustering:owner(DbName, DocId),
+    Id = {DbName, DocId},
+    case {Owner, get_json_value(deleted, Change, false)} of
+    {_, true} ->
+        ok = gen_server:call(?MODULE, {removed, Id}, infinity);
+    {unstable, false} ->
+        couch_log:notice("Not starting '~s' as cluster is unstable", [DocId]);
+    {ThisNode, false} when ThisNode =:= node() ->
+        case get_json_value(<<"_replication_state">>, RepProps) of
+        undefined ->
+            ok = process_updated(Id, JsonRepDoc);
+        <<"triggered">> ->
+            maybe_remove_state_fields(DbName, DocId),
+            ok = process_updated(Id, JsonRepDoc);
+        <<"completed">> ->
+            ok = gen_server:call(?MODULE, {completed, Id}, infinity);
+        <<"error">> ->
+            % Handle replications started from older versions of replicator
+            % which wrote transient errors to replication docs
+            maybe_remove_state_fields(DbName, DocId),
+            ok = process_updated(Id, JsonRepDoc);
+        <<"failed">> ->
+            ok
+        end;
+    {Owner, false} ->
+        ok
+    end,
+    ok.
+
+
+maybe_remove_state_fields(DbName, DocId) ->
+    case update_docs() of
+        true ->
+            ok;
+        false ->
+            couch_replicator_docs:remove_state_fields(DbName, DocId)
+    end.
+
+
+process_updated({DbName, _DocId} = Id, JsonRepDoc) ->
+    % Parsing replication doc (but not calculating the id) could throw an
+    % exception which would indicate this document is malformed. This exception
+    % should propagate to db_change function and will be recorded as permanent
+    % failure in the document. User will have to delete and re-create the document
+    % to fix the problem.
+    Rep0 = couch_replicator_docs:parse_rep_doc_without_id(JsonRepDoc),
+    Rep = Rep0#rep{db_name = DbName, start_time = os:timestamp()},
+    Filter = case couch_replicator_filters:parse(Rep#rep.options) of
+    {ok, nil} ->
+        nil;
+    {ok, {user, _FName, _QP}} ->
+        user;
+    {ok, {view, _FName, _QP}} ->
+        view;
+    {ok, {docids, _DocIds}} ->
+        docids;
+    {ok, {mango, _Selector}} ->
+        mango;
+    {error, FilterError} ->
+        throw(FilterError)
+    end,
+    gen_server:call(?MODULE, {updated, Id, Rep, Filter}, infinity).
+
+
+% Doc processor gen_server API and callbacks
+
+start_link() ->
+    gen_server:start_link({local, ?MODULE}, ?MODULE, [],  []).
+
+
+init([]) ->
+    ?MODULE = ets:new(?MODULE, [ordered_set, named_table, {keypos, #rdoc.id}]),
+    couch_replicator_clustering:link_cluster_event_listener(self()),
+    {ok, nil}.
+
+
+terminate(_Reason, _State) ->
+    ok.
+
+
+handle_call({updated, Id, Rep, Filter}, _From, State) ->
+    ok = updated_doc(Id, Rep, Filter),
+    {reply, ok, State};
+
+handle_call({removed, Id}, _From, State) ->
+    ok = removed_doc(Id),
+    {reply, ok, State};
+
+handle_call({completed, Id}, _From, State) ->
+    true = ets:delete(?MODULE, Id),
+    {reply, ok, State};
+
+handle_call({clean_up_replications, DbName}, _From, State) ->
+    ok = removed_db(DbName),
+    {reply, ok, State}.
+
+handle_cast({cluster, unstable}, State) ->
+    % Ignoring unstable state transition
+    {noreply, State};
+
+handle_cast({cluster, stable}, State) ->
+    % Membership changed recheck all the replication document ownership
+    nil = ets:foldl(fun cluster_membership_foldl/2, nil, ?MODULE),
+    {noreply, State};
+
+handle_cast(Msg, State) ->
+    {stop, {error, unexpected_message, Msg}, State}.
+
+
+handle_info({'DOWN', _, _, _, #doc_worker_result{id = Id, wref = Ref,
+        result = Res}}, State) ->
+    ok = worker_returned(Ref, Id, Res),
+    {noreply, State};
+
+handle_info(_Msg, State) ->
+    {noreply, State}.
+
+
+code_change(_OldVsn, State, _Extra) ->
+    {ok, State}.
+
+
+% Doc processor gen_server private helper functions
+
+% Handle doc update -- add to ets, then start a worker to try to turn it into
+% a replication job. In most cases it will succeed quickly but for filtered
+% replicationss or if there are duplicates, it could take longer
+% (theoretically indefinitely) until a replication could be started. Before
+% adding replication job, make sure to delete all old jobs associated with
+% same document.
+-spec updated_doc(db_doc_id(), #rep{}, filter_type()) -> ok.
+updated_doc(Id, Rep, Filter) ->
+    case normalize_rep(current_rep(Id)) == normalize_rep(Rep) of
+        false ->
+            removed_doc(Id),
+            Row = #rdoc{
+                id = Id,
+                state = initializing,
+                rep = Rep,
+                rid = nil,
+                filter = Filter,
+                info = nil,
+                errcnt = 0,
+                worker = nil,
+                last_updated = os:timestamp()
+            },
+            true = ets:insert(?MODULE, Row),
+            ok = maybe_start_worker(Id);
+        true ->
+            ok
+    end.
+
+
+% Return current #rep{} record if any. If replication hasn't been submitted
+% to the scheduler yet, #rep{} record will be in the document processor's
+% ETS table, otherwise query scheduler for the #rep{} record.
+-spec current_rep({binary(), binary()}) -> #rep{} | nil.
+current_rep({DbName, DocId}) when is_binary(DbName), is_binary(DocId) ->
+    case ets:lookup(?MODULE, {DbName, DocId}) of
+        [] ->
+            nil;
+        [#rdoc{state = scheduled, rep = nil, rid = JobId}] ->
+            % When replication is scheduled, #rep{} record which can be quite
+            % large compared to other bits in #rdoc is removed in order to avoid
+            % having to keep 2 copies of it. So have to fetch it from the
+            % scheduler.
+            couch_replicator_scheduler:rep_state(JobId);
+        [#rdoc{rep = Rep}] ->
+            Rep
+    end.
+
+
+% Normalize a #rep{} record such that it doesn't contain time dependent fields
+% pids (like httpc pools), and options / props are sorted. This function would
+% used during comparisons.
+-spec normalize_rep(#rep{} | nil) -> #rep{} | nil.
+normalize_rep(nil) ->
+    nil;
+
+normalize_rep(#rep{} = Rep)->
+    #rep{
+        source = couch_replicator_api_wrap:normalize_db(Rep#rep.source),
+        target = couch_replicator_api_wrap:normalize_db(Rep#rep.target),
+        options = Rep#rep.options,  % already sorted in make_options/1
+        type = Rep#rep.type,
+        view = Rep#rep.view,
+        doc_id = Rep#rep.doc_id,
+        db_name = Rep#rep.db_name
+    }.
+
+
+-spec worker_returned(reference(), db_doc_id(), rep_start_result()) -> ok.
+worker_returned(Ref, Id, {ok, RepId}) ->
+    case ets:lookup(?MODULE, Id) of
+    [#rdoc{worker = Ref} = Row] ->
+        Row0 = Row#rdoc{
+            state = scheduled,
+            errcnt = 0,
+            worker = nil,
+            last_updated = os:timestamp()
+        },
+        NewRow = case Row0 of
+            #rdoc{rid = RepId, filter = user} ->
+                % Filtered replication id didn't change.
+                Row0;
+            #rdoc{rid = nil, filter = user} ->
+                % Calculated new replication id for a filtered replication. Make sure
+                % to schedule another check as filter code could change. Replications
+                % starts could have been failing, so also clear error count.
+                Row0#rdoc{rid = RepId};
+            #rdoc{rid = OldRepId, filter = user} ->
+                % Replication id of existing replication job with filter has changed.
+                % Remove old replication job from scheduler and schedule check to check
+                % for future changes.
+                ok = couch_replicator_scheduler:remove_job(OldRepId),
+                Msg = io_lib:format("Replication id changed: ~p -> ~p", [OldRepId, RepId]),
+                Row0#rdoc{rid = RepId, info = couch_util:to_binary(Msg)};
+            #rdoc{rid = nil} ->
+                % Calculated new replication id for non-filtered replication. Remove
+                % replication doc body, after this we won't needed any more.
+                Row0#rdoc{rep=nil, rid=RepId, info=nil}
+        end,
+        true = ets:insert(?MODULE, NewRow),
+        ok = maybe_update_doc_triggered(Row#rdoc.rep, RepId),
+        ok = maybe_start_worker(Id);
+    _ ->
+        ok  % doc could have been deleted, ignore
+    end,
+    ok;
+
+worker_returned(_Ref, _Id, ignore) ->
+    ok;
+
+worker_returned(Ref, Id, {temporary_error, Reason}) ->
+    case ets:lookup(?MODULE, Id) of
+    [#rdoc{worker = Ref, errcnt = ErrCnt} = Row] ->
+        NewRow = Row#rdoc{
+            rid = nil,
+            state = error,
+            info = Reason,
+            errcnt = ErrCnt + 1,
+            worker = nil,
+            last_updated = os:timestamp()
+        },
+        true = ets:insert(?MODULE, NewRow),
+        ok = maybe_update_doc_error(NewRow#rdoc.rep, Reason),
+        ok = maybe_start_worker(Id);
+    _ ->
+        ok  % doc could have been deleted, ignore
+    end,
+    ok;
+
+worker_returned(Ref, Id, {permanent_failure, _Reason}) ->
+    case ets:lookup(?MODULE, Id) of
+    [#rdoc{worker = Ref}] ->
+        true = ets:delete(?MODULE, Id);
+    _ ->
+        ok  % doc could have been deleted, ignore
+    end,
+    ok.
+
+-spec maybe_update_doc_error(#rep{}, any()) -> ok.
+maybe_update_doc_error(Rep, Reason) ->
+    case update_docs() of
+        true ->
+            couch_replicator_docs:update_error(Rep, Reason);
+        false ->
+            ok
+    end.
+
+-spec maybe_update_doc_triggered(#rep{}, rep_id()) -> ok.
+maybe_update_doc_triggered(Rep, RepId) ->
+    case update_docs() of
+        true ->
+            couch_replicator_docs:update_triggered(Rep, RepId);
+        false ->
+            ok
+    end.
+
+
+-spec error_backoff(non_neg_integer()) -> seconds().
+error_backoff(ErrCnt) ->
+    Exp = min(ErrCnt, ?ERROR_MAX_BACKOFF_EXPONENT),
+    random:uniform(64 bsl Exp).
+
+
+-spec filter_backoff() -> seconds().
+filter_backoff() ->
+    Total = ets:info(?MODULE, size),
+    Range = 1 + min(2 * (Total / 10), ?TS_DAY_SEC),
+    60 + random:uniform(round(Range)).
+
+
+% Document removed from db -- clear ets table and remove all scheduled jobs
+-spec removed_doc(db_doc_id()) -> ok.
+removed_doc({DbName, DocId} = Id) ->
+    ets:delete(?MODULE, Id),
+    RepIds = couch_replicator_scheduler:find_jobs_by_doc(DbName, DocId),
+    lists:foreach(fun couch_replicator_scheduler:remove_job/1, RepIds).
+
+
+% Whole db shard is gone -- remove all its ets rows and stop jobs
+-spec removed_db(binary()) -> ok.
+removed_db(DbName) ->
+    EtsPat = #rdoc{id = {DbName, '_'}, _ = '_'},
+    ets:match_delete(?MODULE, EtsPat),
+    RepIds = couch_replicator_scheduler:find_jobs_by_dbname(DbName),
+    lists:foreach(fun couch_replicator_scheduler:remove_job/1, RepIds).
+
+
+% Spawn a worker process which will attempt to calculate a replication id, then
+% start a replication. Returns a process monitor reference. The worker is
+% guaranteed to exit with rep_start_result() type only.
+-spec maybe_start_worker(db_doc_id()) -> ok.
+maybe_start_worker(Id) ->
+    case ets:lookup(?MODULE, Id) of
+    [] ->
+        ok;
+    [#rdoc{state = scheduled, filter = Filter}] when Filter =/= user ->
+        ok;
+    [#rdoc{rep = Rep} = Doc] ->
+        Wait = get_worker_wait(Doc),
+        Ref = make_ref(),
+        true = ets:insert(?MODULE, Doc#rdoc{worker = Ref}),
+        couch_replicator_doc_processor_worker:spawn_worker(Id, Rep, Wait, Ref),
+        ok
+    end.
+
+
+-spec get_worker_wait(#rdoc{}) -> seconds().
+get_worker_wait(#rdoc{state = scheduled, filter = user}) ->
+    filter_backoff();
+
+get_worker_wait(#rdoc{state = error, errcnt = ErrCnt}) ->
+    error_backoff(ErrCnt);
+
+get_worker_wait(#rdoc{state = initializing}) ->
+    0.
+
+-spec update_docs() -> boolean().
+update_docs() ->
+    config:get_boolean("replicator", "update_docs", ?DEFAULT_UPDATE_DOCS).
+
+
+% _scheduler/docs HTTP endpoint helpers
+
+-spec docs([atom()]) -> [{[_]}] | [].
+docs(States) ->
+    HealthThreshold = couch_replicator_scheduler:health_threshold(),
+    ets:foldl(fun(RDoc, Acc) ->
+        case ejson_doc(RDoc, HealthThreshold) of
+            nil ->
+                Acc;  % Could have been deleted if job just completed
+            {Props} = EJson ->
+                {state, DocState} = lists:keyfind(state, 1, Props),
+                case ejson_doc_state_filter(DocState, States) of
+                    true ->
+                        [EJson | Acc];
+                    false ->
+                        Acc
+                end
+        end
+    end, [], ?MODULE).
+
+
+-spec doc(binary(), binary()) -> {ok, {[_]}} | {error, not_found}.
+doc(Db, DocId) ->
+    HealthThreshold = couch_replicator_scheduler:health_threshold(),
+    Res = ets:foldl(fun(_RDoc, [_] = Acc) -> Acc;
+        (RDoc, []) ->
+            {Shard, RDocId} = RDoc#rdoc.id,
+            case {mem3:dbname(Shard), RDocId} of
+                {Db, DocId} ->
+                    [ejson_doc(RDoc, HealthThreshold)];
+                {_OtherDb, _OtherDocId} ->
+                    []
+            end
+    end, [], ?MODULE),
+    case Res of
+        [DocInfo] ->
+            {ok, DocInfo};
+        [] ->
+            {error, not_found}
+    end.
+
+
+-spec ejson_state_info(binary() | nil) -> binary() | null.
+ejson_state_info(nil) ->
+    null;
+ejson_state_info(Info) when is_binary(Info) ->
+    Info;
+ejson_state_info(Info) ->
+    couch_replicator_utils:rep_error_to_binary(Info).
+
+
+-spec ejson_rep_id(rep_id() | nil) -> binary() | null.
+ejson_rep_id(nil) ->
+    null;
+ejson_rep_id({BaseId, Ext}) ->
+    iolist_to_binary([BaseId, Ext]).
+
+
+-spec ejson_doc(#rdoc{}, non_neg_integer()) -> {[_]} | nil.
+ejson_doc(#rdoc{state = scheduled} = RDoc, HealthThreshold) ->
+    #rdoc{id = {DbName, DocId}, rid = RepId} = RDoc,
+    JobProps = couch_replicator_scheduler:job_summary(RepId, HealthThreshold),
+    case JobProps of
+        nil ->
+            nil;
+        [{_, _} | _] ->
+            {[
+                {doc_id, DocId},
+                {database, DbName},
+                {id, ejson_rep_id(RepId)},
+                {node, node()} | JobProps
+            ]}
+    end;
+
+ejson_doc(#rdoc{state = RepState} = RDoc, _HealthThreshold) ->
+    #rdoc{
+       id = {DbName, DocId},
+       info = StateInfo,
+       rid = RepId,
+       errcnt = ErrorCount,
+       last_updated = StateTime,
+       rep = Rep
+    } = RDoc,
+    {[
+        {doc_id, DocId},
+        {database, DbName},
+        {id, ejson_rep_id(RepId)},
+        {state, RepState},
+        {info, ejson_state_info(StateInfo)},
+        {error_count, ErrorCount},
+        {node, node()},
+        {last_updated, couch_replicator_utils:iso8601(StateTime)},
+        {start_time, couch_replicator_utils:iso8601(Rep#rep.start_time)}
+    ]}.
+
+
+
+-spec ejson_doc_state_filter(atom(), [atom()]) -> boolean().
+ejson_doc_state_filter(_DocState, []) ->
+    true;
+
+ejson_doc_state_filter(State, States) when is_list(States), is_atom(State) ->
+    lists:member(State, States).
+
+
+-spec cluster_membership_foldl(#rdoc{}, nil) -> nil.
+cluster_membership_foldl(#rdoc{id = {DbName, DocId} = Id, rid = RepId}, nil) ->
+    case couch_replicator_clustering:owner(DbName, DocId) of
+        unstable ->
+            nil;
+        ThisNode when ThisNode =:= node() ->
+            nil;
+        OtherNode ->
+            Msg = "Replication doc ~p:~p with id ~p usurped by node ~p",
+            couch_log:notice(Msg, [DbName, DocId, RepId, OtherNode]),
+            removed_doc(Id),
+            nil
+    end.
+
+
+-ifdef(TEST).
+
+-include_lib("eunit/include/eunit.hrl").
+
+-define(DB, <<"db">>).
+-define(DOC1, <<"doc1">>).
+-define(DOC2, <<"doc2">>).
+-define(R1, {"1", ""}).
+-define(R2, {"2", ""}).
+
+
+doc_processor_test_() ->
+    {
+        foreach,
+        fun setup/0,
+        fun teardown/1,
+        [
+            t_bad_change(),
+            t_regular_change(),
+            t_change_with_existing_job(),
+            t_deleted_change(),
+            t_triggered_change(),
+            t_completed_change(),
+            t_active_replication_completed(),
+            t_error_change(),
+            t_failed_change(),
+            t_change_for_different_node(),
+            t_change_when_cluster_unstable(),
+            t_ejson_docs(),
+            t_cluster_membership_foldl()
+        ]
+    }.
+
+
+% Can't parse replication doc, so should write failure state to document.
+t_bad_change() ->
+    ?_test(begin
+        ?assertEqual(acc, db_change(?DB, bad_change(), acc)),
+        ?assert(updated_doc_with_failed_state())
+    end).
+
+
+% Regular change, parse to a #rep{} and then add job.
+t_regular_change() ->
+    ?_test(begin
+        mock_existing_jobs_lookup([]),
+        ?assertEqual(ok, process_change(?DB, change())),
+        ?assert(ets:member(?MODULE, {?DB, ?DOC1})),
+        ?assert(started_worker({?DB, ?DOC1}))
+    end).
+
+
+% Regular change, parse to a #rep{} and then add job but there is already
+% a running job with same Id found.
+t_change_with_existing_job() ->
+    ?_test(begin
+        mock_existing_jobs_lookup([test_rep(?R2)]),
+        ?assertEqual(ok, process_change(?DB, change())),
+        ?assert(ets:member(?MODULE, {?DB, ?DOC1})),
+        ?assert(started_worker({?DB, ?DOC1}))
+    end).
+
+
+% Change is a deletion, and job is running, so remove job.
+t_deleted_change() ->
+    ?_test(begin
+        mock_existing_jobs_lookup([test_rep(?R2)]),
+        ?assertEqual(ok, process_change(?DB, deleted_change())),
+        ?assert(removed_job(?R2))
+    end).
+
+
+% Change is in `triggered` state. Remove legacy state and add job.
+t_triggered_change() ->
+    ?_test(begin
+        mock_existing_jobs_lookup([]),
+        ?assertEqual(ok, process_change(?DB, change(<<"triggered">>))),
+        ?assert(removed_state_fields()),
+        ?assert(ets:member(?MODULE, {?DB, ?DOC1})),
+        ?assert(started_worker({?DB, ?DOC1}))
+    end).
+
+
+% Change is in `completed` state, so skip over it.
+t_completed_change() ->
+    ?_test(begin
+        ?assertEqual(ok, process_change(?DB, change(<<"completed">>))),
+        ?assert(did_not_remove_state_fields()),
+        ?assertNot(ets:member(?MODULE, {?DB, ?DOC1})),
+        ?assert(did_not_spawn_worker())
+    end).
+
+
+% Completed change comes for what used to be an active job. In this case
+% remove entry from doc_processor's ets (because there is no linkage or
+% callback mechanism for scheduler to tell doc_processsor a replication just
+% completed).
+t_active_replication_completed() ->
+    ?_test(begin
+        mock_existing_jobs_lookup([]),
+        ?assertEqual(ok, process_change(?DB, change())),
+        ?assert(ets:member(?MODULE, {?DB, ?DOC1})),
+        ?assertEqual(ok, process_change(?DB, change(<<"completed">>))),
+        ?assert(did_not_remove_state_fields()),
+        ?assertNot(ets:member(?MODULE, {?DB, ?DOC1}))
+    end).
+
+
+% Change is in `error` state. Remove legacy state and retry
+% running the job. This state was used for transient erorrs which are not
+% written to the document anymore.
+t_error_change() ->
+    ?_test(begin
+        mock_existing_jobs_lookup([]),
+        ?assertEqual(ok, process_change(?DB, change(<<"error">>))),
+        ?assert(removed_state_fields()),
+        ?assert(ets:member(?MODULE, {?DB, ?DOC1})),
+        ?assert(started_worker({?DB, ?DOC1}))
+    end).
+
+
+% Change is in `failed` state. This is a terminal state and it will not
+% be tried again, so skip over it.
+t_failed_change() ->
+    ?_test(begin
+        ?assertEqual(ok, process_change(?DB, change(<<"failed">>))),
+        ?assert(did_not_remove_state_fields()),
+        ?assertNot(ets:member(?MODULE, {?DB, ?DOC1})),
+        ?assert(did_not_spawn_worker())
+    end).
+
+
+% Normal change, but according to cluster ownership algorithm, replication belongs to
+% a different node, so this node should skip it.
+t_change_for_different_node() ->
+   ?_test(begin
+        meck:expect(couch_replicator_clustering, owner, 2, different_node),
+        ?assertEqual(ok, process_change(?DB, change())),
+        ?assert(did_not_spawn_worker())
+   end).
+
+
+% Change handled when cluster is unstable (nodes are added or removed), so
+% job is not added. A rescan will be triggered soon and change will be evaluated again.
+t_change_when_cluster_unstable() ->
+   ?_test(begin
+       meck:expect(couch_replicator_clustering, owner, 2, unstable),
+       ?assertEqual(ok, process_change(?DB, change())),
+       ?assert(did_not_spawn_worker())
+   end).
+
+
+% Check if docs/0 function produces expected ejson after adding a job
+t_ejson_docs() ->
+    ?_test(begin
+        mock_existing_jobs_lookup([]),
+        ?assertEqual(ok, process_change(?DB, change())),
+        ?assert(ets:member(?MODULE, {?DB, ?DOC1})),
+        EJsonDocs = docs([]),
+        ?assertMatch([{[_|_]}], EJsonDocs),
+        [{DocProps}] = EJsonDocs,
+        {value, StateTime, DocProps1} = lists:keytake(last_updated, 1, DocProps),
+        ?assertMatch({last_updated, BinVal1} when is_binary(BinVal1), StateTime),
+        {value, StartTime, DocProps2} = lists:keytake(start_time, 1, DocProps1),
+        ?assertMatch({start_time, BinVal2} when is_binary(BinVal2), StartTime),
+        ExpectedProps = [
+            {database, ?DB},
+            {doc_id, ?DOC1},
+            {error_count, 0},
+            {id, null},
+            {info, null},
+            {node, node()},
+            {state, initializing}
+        ],
+        ?assertEqual(ExpectedProps, lists:usort(DocProps2))
+    end).
+
+
+% Check that when cluster membership changes records from doc processor and job
+% scheduler get removed
+t_cluster_membership_foldl() ->
+   ?_test(begin
+        mock_existing_jobs_lookup([test_rep(?R1)]),
+        ?assertEqual(ok, process_change(?DB, change())),
+        meck:expect(couch_replicator_clustering, owner, 2, different_node),
+        ?assert(ets:member(?MODULE, {?DB, ?DOC1})),
+        gen_server:cast(?MODULE, {cluster, stable}),
+        timer:sleep(100),
+        ?assertNot(ets:member(?MODULE, {?DB, ?DOC1})),
+        ?assert(removed_job(?R1))
+   end).
+
+
+normalize_rep_test_() ->
+    {
+        setup,
+        fun() -> meck:expect(config, get, fun(_, _, Default) -> Default end) end,
+        fun(_) -> meck:unload() end,
+        ?_test(begin
+            EJson1 = {[
+                {<<"source">>, <<"http://host.com/source_db">>},
+                {<<"target">>, <<"local">>},
+                {<<"doc_ids">>, [<<"a">>, <<"c">>, <<"b">>]},
+                {<<"other_field">>, <<"some_value">>}
+            ]},
+            Rep1 = couch_replicator_docs:parse_rep_doc_without_id(EJson1),
+            EJson2 = {[
+                {<<"other_field">>, <<"unrelated">>},
+                {<<"target">>, <<"local">>},
+                {<<"source">>, <<"http://host.com/source_db">>},
+                {<<"doc_ids">>, [<<"c">>, <<"a">>, <<"b">>]},
+                {<<"other_field2">>, <<"unrelated2">>}
+            ]},
+            Rep2 = couch_replicator_docs:parse_rep_doc_without_id(EJson2),
+            ?assertEqual(normalize_rep(Rep1), normalize_rep(Rep2))
+        end)
+    }.
+
+
+get_worker_ref_test_() ->
+    {
+        setup,
+        fun() -> ets:new(?MODULE, [named_table, public, {keypos, #rdoc.id}]) end,
+        fun(_) -> ets:delete(?MODULE) end,
+        ?_test(begin
+            Id = {<<"db">>, <<"doc">>},
+            ?assertEqual(nil, get_worker_ref(Id)),
+            ets:insert(?MODULE, #rdoc{id = Id, worker = nil}),
+            ?assertEqual(nil, get_worker_ref(Id)),
+            Ref = make_ref(),
+            ets:insert(?MODULE, #rdoc{id = Id, worker = Ref}),
+            ?assertEqual(Ref, get_worker_ref(Id))
+        end)
+    }.
+
+
+% Test helper functions
+
+
+setup() ->
+    meck:expect(couch_log, info, 2, ok),
+    meck:expect(couch_log, notice, 2, ok),
+    meck:expect(couch_log, warning, 2, ok),
+    meck:expect(couch_log, error, 2, ok),
+    meck:expect(config, get, fun(_, _, Default) -> Default end),
+    meck:expect(config, listen_for_changes, 2, ok),
+    meck:expect(couch_replicator_clustering, owner, 2, node()),
+    meck:expect(couch_replicator_clustering, link_cluster_event_listener, 1, ok),
+    meck:expect(couch_replicator_doc_processor_worker, spawn_worker, 4, pid),
+    meck:expect(couch_replicator_scheduler, remove_job, 1, ok),
+    meck:expect(couch_replicator_docs, remove_state_fields, 2, ok),
+    meck:expect(couch_replicator_docs, update_failed, 4, ok),
+    {ok, Pid} = start_link(),
+    Pid.
+
+
+teardown(Pid) ->
+    unlink(Pid),
+    exit(Pid, kill),
+    meck:unload().
+
+
+removed_state_fields() ->
+    meck:called(couch_replicator_docs, remove_state_fields, [?DB, ?DOC1]).
+
+
+started_worker(_Id) ->
+    1 == meck:num_calls(couch_replicator_doc_processor_worker, spawn_worker, 4).
+
+
+removed_job(Id) ->
+    meck:called(couch_replicator_scheduler, remove_job, [test_rep(Id)]).
+
+
+did_not_remove_state_fields() ->
+    0 == meck:num_calls(couch_replicator_docs, remove_state_fields, '_').
+
+
+did_not_spawn_worker() ->
+    0 == meck:num_calls(couch_replicator_doc_processor_worker, spawn_worker,
+        '_').
+
+updated_doc_with_failed_state() ->
+    1 == meck:num_calls(couch_replicator_docs, update_failed, '_').
+
+
+mock_existing_jobs_lookup(ExistingJobs) ->
+    meck:expect(couch_replicator_scheduler, find_jobs_by_doc,
+            fun(?DB, ?DOC1) -> ExistingJobs end).
+
+
+test_rep(Id) ->
+  #rep{id = Id, start_time = {0, 0, 0}}.
+
+
+change() ->
+    {[
+        {<<"id">>, ?DOC1},
+        {doc, {[
+            {<<"_id">>, ?DOC1},
+            {<<"source">>, <<"src">>},
+            {<<"target">>, <<"tgt">>}
+        ]}}
+    ]}.
+
+
+change(State) ->
+    {[
+        {<<"id">>, ?DOC1},
+        {doc, {[
+            {<<"_id">>, ?DOC1},
+            {<<"source">>, <<"src">>},
+            {<<"target">>, <<"tgt">>},
+            {<<"_replication_state">>, State}
+        ]}}
+    ]}.
+
+
+deleted_change() ->
+    {[
+        {<<"id">>, ?DOC1},
+        {<<"deleted">>, true},
+        {doc, {[
+            {<<"_id">>, ?DOC1},
+            {<<"source">>, <<"src">>},
+            {<<"target">>, <<"tgt">>}
+        ]}}
+    ]}.
+
+
+bad_change() ->
+    {[
+        {<<"id">>, ?DOC2},
+        {doc, {[
+            {<<"_id">>, ?DOC2},
+            {<<"source">>, <<"src">>}
+        ]}}
+    ]}.
+
+
+
+
+-endif.
diff --git a/src/couch_replicator_doc_processor_worker.erl b/src/couch_replicator_doc_processor_worker.erl
new file mode 100644
index 0000000..30a6988
--- /dev/null
+++ b/src/couch_replicator_doc_processor_worker.erl
@@ -0,0 +1,268 @@
+% 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_replicator_doc_processor_worker).
+
+-export([spawn_worker/4]).
+
+-include("couch_replicator.hrl").
+
+-import(couch_replicator_utils, [
+    pp_rep_id/1
+]).
+
+-define(WORKER_TIMEOUT_MSEC, 61000).
+
+% Spawn a worker which attempts to calculate replication id then add a
+% replication job to scheduler. This function create a monitor to the worker
+% a worker will then exit with the #doc_worker_result{} record within
+% ?WORKER_TIMEOUT_MSEC timeout period.A timeout is considered a `temporary_error`.
+% Result will be sent as the `Reason` in the {'DOWN',...} message.
+-spec spawn_worker(db_doc_id(), #rep{}, seconds(), reference()) -> pid().
+spawn_worker(Id, Rep, WaitSec, WRef) ->
+    {Pid, _Ref} = spawn_monitor(fun() -> worker_fun(Id, Rep, WaitSec, WRef) end),
+    Pid.
+
+
+% Private functions
+
+-spec worker_fun(db_doc_id(), #rep{}, seconds(), reference()) -> no_return().
+worker_fun(Id, Rep, WaitSec, WRef) ->
+    timer:sleep(WaitSec * 1000),
+    Fun = fun() ->
+        try maybe_start_replication(Id, Rep, WRef) of
+            Res ->
+                exit(Res)
+        catch
+            throw:{filter_fetch_error, Reason} ->
+                exit({temporary_error, Reason});
+            _Tag:Reason ->
+                exit({temporary_error, Reason})
+        end
+    end,
+    {Pid, Ref} = spawn_monitor(Fun),
+    receive
+        {'DOWN', Ref, _, Pid, Result} ->
+            exit(#doc_worker_result{id = Id, wref = WRef, result = Result})
+    after ?WORKER_TIMEOUT_MSEC ->
+        erlang:demonitor(Ref, [flush]),
+        exit(Pid, kill),
+        {DbName, DocId} = Id,
+        TimeoutSec = round(?WORKER_TIMEOUT_MSEC / 1000),
+        Msg = io_lib:format("Replication for db ~p doc ~p failed to start due "
+            "to timeout after ~B seconds", [DbName, DocId, TimeoutSec]),
+        Result = {temporary_error, couch_util:to_binary(Msg)},
+        exit(#doc_worker_result{id = Id, wref = WRef, result = Result})
+    end.
+
+
+% Try to start a replication. Used by a worker. This function should return
+% rep_start_result(), also throws {filter_fetch_error, Reason} if cannot fetch filter.
+% It can also block for an indeterminate amount of time while fetching the
+% filter.
+maybe_start_replication(Id, RepWithoutId, WRef) ->
+    Rep = couch_replicator_docs:update_rep_id(RepWithoutId),
+    case maybe_add_job_to_scheduler(Id, Rep, WRef) of
+    ignore ->
+        ignore;
+    {ok, RepId} ->
+        {ok, RepId};
+    {temporary_error, Reason} ->
+        {temporary_error, Reason};
+    {permanent_failure, Reason} ->
+        {DbName, DocId} = Id,
+        StartTime = Rep#rep.start_time,
+        couch_replicator_docs:update_failed(DbName, DocId, Reason, StartTime),
+        {permanent_failure, Reason}
+    end.
+
+
+-spec maybe_add_job_to_scheduler(db_doc_id(), #rep{}, reference()) ->
+   rep_start_result().
+maybe_add_job_to_scheduler({DbName, DocId}, Rep, WRef) ->
+    RepId = Rep#rep.id,
+    case couch_replicator_scheduler:rep_state(RepId) of
+    nil ->
+        % Before adding a job check that this worker is still the current
+        % worker. This is to handle a race condition where a worker which was
+        % sleeping and then checking a replication filter may inadvertently re-add
+        % a replication which was already deleted.
+        case couch_replicator_doc_processor:get_worker_ref({DbName, DocId}) of
+        WRef ->
+            ok = couch_replicator_scheduler:add_job(Rep),
+            {ok, RepId};
+        _NilOrOtherWRef ->
+            ignore
+        end;
+    #rep{doc_id = DocId} ->
+        {ok, RepId};
+    #rep{doc_id = null} ->
+        Msg = io_lib:format("Replication `~s` specified by document `~s`"
+            " already running as a transient replication, started via"
+            " `_replicate` API endpoint", [pp_rep_id(RepId), DocId]),
+        {temporary_error, couch_util:to_binary(Msg)};
+    #rep{db_name = OtherDb, doc_id = OtherDocId} ->
+        Msg = io_lib:format("Replication `~s` specified by document `~s`"
+            " already started, triggered by document `~s` from db `~s`",
+            [pp_rep_id(RepId), DocId, OtherDocId, mem3:dbname(OtherDb)]),
+        {permanent_failure, couch_util:to_binary(Msg)}
+    end.
+
+
+-ifdef(TEST).
+
+-include_lib("eunit/include/eunit.hrl").
+
+-define(DB, <<"db">>).
+-define(DOC1, <<"doc1">>).
+-define(R1, {"0b7831e9a41f9322a8600ccfa02245f2", ""}).
+
+
+doc_processor_worker_test_() ->
+    {
+        foreach,
+        fun setup/0,
+        fun teardown/1,
+        [
+            t_should_add_job(),
+            t_already_running_same_docid(),
+            t_already_running_transient(),
+            t_already_running_other_db_other_doc(),
+            t_spawn_worker(),
+            t_ignore_if_doc_deleted(),
+            t_ignore_if_worker_ref_does_not_match()
+        ]
+    }.
+
+
+% Replication is already running, with same doc id. Ignore change.
+t_should_add_job() ->
+   ?_test(begin
+       Id = {?DB, ?DOC1},
+       Rep = couch_replicator_docs:parse_rep_doc_without_id(change()),
+       ?assertEqual({ok, ?R1}, maybe_start_replication(Id, Rep, nil)),
+       ?assert(added_job())
+   end).
+
+
+% Replication is already running, with same doc id. Ignore change.
+t_already_running_same_docid() ->
+   ?_test(begin
+       Id = {?DB, ?DOC1},
+       mock_already_running(?DB, ?DOC1),
+       Rep = couch_replicator_docs:parse_rep_doc_without_id(change()),
+       ?assertEqual({ok, ?R1}, maybe_start_replication(Id, Rep, nil)),
+       ?assert(did_not_add_job())
+   end).
+
+
+% There is a transient replication with same replication id running. Ignore change.
+t_already_running_transient() ->
+   ?_test(begin
+       Id = {?DB, ?DOC1},
+       mock_already_running(null, null),
+       Rep = couch_replicator_docs:parse_rep_doc_without_id(change()),
+       ?assertMatch({temporary_error, _}, maybe_start_replication(Id, Rep, nil)),
+       ?assert(did_not_add_job())
+   end).
+
+
+% There is a duplicate replication potentially from a different db and doc.
+% Write permanent failure to doc.
+t_already_running_other_db_other_doc() ->
+   ?_test(begin
+       Id = {?DB, ?DOC1},
+       mock_already_running(<<"otherdb">>, <<"otherdoc">>),
+       Rep = couch_replicator_docs:parse_rep_doc_without_id(change()),
+       ?assertMatch({permanent_failure, _}, maybe_start_replication(Id, Rep, nil)),
+       ?assert(did_not_add_job()),
+       1 == meck:num_calls(couch_replicator_docs, update_failed, '_')
+   end).
+
+
+% Should spawn worker
+t_spawn_worker() ->
+   ?_test(begin
+       Id = {?DB, ?DOC1},
+       Rep = couch_replicator_docs:parse_rep_doc_without_id(change()),
+       WRef = make_ref(),
+       meck:expect(couch_replicator_doc_processor, get_worker_ref, 1, WRef),
+       Pid = spawn_worker(Id, Rep, 0, WRef),
+       Res = receive  {'DOWN', _Ref, process, Pid, Reason} -> Reason
+           after 1000 -> timeout end,
+       Expect = #doc_worker_result{id = Id, wref = WRef, result = {ok, ?R1}},
+       ?assertEqual(Expect, Res),
+       ?assert(added_job())
+   end).
+
+
+% Should not add job if by the time worker got to fetching the filter
+% and getting a replication id, replication doc was deleted
+t_ignore_if_doc_deleted() ->
+   ?_test(begin
+       Id = {?DB, ?DOC1},
+       Rep = couch_replicator_docs:parse_rep_doc_without_id(change()),
+       meck:expect(couch_replicator_doc_processor, get_worker_ref, 1, nil),
+       ?assertEqual(ignore, maybe_start_replication(Id, Rep, make_ref())),
+       ?assertNot(added_job())
+   end).
+
+
+% Should not add job if by the time worker got to fetchign the filter
+% and building a replication id, another worker was spawned.
+t_ignore_if_worker_ref_does_not_match() ->
+    ?_test(begin
+       Id = {?DB, ?DOC1},
+       Rep = couch_replicator_docs:parse_rep_doc_without_id(change()),
+       meck:expect(couch_replicator_doc_processor, get_worker_ref, 1, make_ref()),
+       ?assertEqual(ignore, maybe_start_replication(Id, Rep, make_ref())),
+       ?assertNot(added_job())
+   end).
+
+
+% Test helper functions
+
+setup() ->
+    meck:expect(couch_replicator_scheduler, add_job, 1, ok),
+    meck:expect(config, get, fun(_, _, Default) -> Default end),
+    meck:expect(couch_server, get_uuid, 0, this_is_snek),
+    meck:expect(couch_replicator_docs, update_failed, 4, ok),
+    meck:expect(couch_replicator_scheduler, rep_state, 1, nil),
+    meck:expect(couch_replicator_doc_processor, get_worker_ref, 1, nil),
+    ok.
+
+
+teardown(_) ->
+    meck:unload().
+
+
+mock_already_running(DbName, DocId) ->
+    meck:expect(couch_replicator_scheduler, rep_state,
+         fun(RepId) -> #rep{id = RepId, doc_id = DocId, db_name = DbName} end).
+
+
+added_job() ->
+    1 == meck:num_calls(couch_replicator_scheduler, add_job, '_').
+
+
+did_not_add_job() ->
+    0 == meck:num_calls(couch_replicator_scheduler, add_job, '_').
+
+
+change() ->
+    {[
+         {<<"_id">>, ?DOC1},
+         {<<"source">>, <<"src">>},
+         {<<"target">>, <<"tgt">>}
+     ]}.
+
+-endif.
diff --git a/src/couch_replicator_docs.erl b/src/couch_replicator_docs.erl
new file mode 100644
index 0000000..5bdfe92
--- /dev/null
+++ b/src/couch_replicator_docs.erl
@@ -0,0 +1,736 @@
+% 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_replicator_docs).
+
+-export([parse_rep_doc/1, parse_rep_doc/2, parse_rep_db/3]).
+-export([parse_rep_doc_without_id/1, parse_rep_doc_without_id/2]).
+-export([before_doc_update/2, after_doc_read/2]).
+-export([ensure_rep_db_exists/0, ensure_rep_ddoc_exists/1]).
+-export([ensure_cluster_rep_ddoc_exists/1]).
+-export([
+    remove_state_fields/2,
+    update_doc_completed/4,
+    update_failed/4,
+    update_rep_id/1
+]).
+-export([update_triggered/2, update_error/2]).
+
+
+-define(REP_DB_NAME, <<"_replicator">>).
+-define(REP_DESIGN_DOC, <<"_design/_replicator">>).
+
+-include_lib("couch/include/couch_db.hrl").
+-include_lib("ibrowse/include/ibrowse.hrl").
+-include_lib("mem3/include/mem3.hrl").
+-include("couch_replicator_api_wrap.hrl").
+-include("couch_replicator.hrl").
+
+-include("couch_replicator_js_functions.hrl").
+
+-import(couch_util, [
+    get_value/2,
+    get_value/3,
+    to_binary/1
+]).
+
+-import(couch_replicator_utils, [
+    get_json_value/2,
+    get_json_value/3
+]).
+
+
+-define(OWNER, <<"owner">>).
+-define(CTX, {user_ctx, #user_ctx{roles=[<<"_admin">>, <<"_replicator">>]}}).
+
+-define(replace(L, K, V), lists:keystore(K, 1, L, {K, V})).
+
+remove_state_fields(DbName, DocId) ->
+    update_rep_doc(DbName, DocId, [
+        {<<"_replication_state">>, undefined},
+        {<<"_replication_state_time">>, undefined},
+        {<<"_replication_state_reason">>, undefined},
+        {<<"_replication_start_time">>, undefined},
+        {<<"_replication_id">>, undefined},
+        {<<"_replication_stats">>, undefined}]).
+
+-spec update_doc_completed(binary(), binary(), [_], erlang:timestamp()) -> any().
+update_doc_completed(DbName, DocId, Stats, StartTime) ->
+    StartTimeBin = couch_replicator_utils:iso8601(StartTime),
+    update_rep_doc(DbName, DocId, [
+        {<<"_replication_state">>, <<"completed">>},
+        {<<"_replication_state_reason">>, undefined},
+        {<<"_replication_start_time">>,  StartTimeBin},
+        {<<"_replication_stats">>, {Stats}}]),
+    couch_stats:increment_counter([couch_replicator, docs, completed_state_updates]).
+
+
+-spec update_failed(binary(), binary(), any(), erlang:timestamp()) -> any().
+update_failed(DbName, DocId, Error, StartTime) ->
+    Reason = error_reason(Error),
+    couch_log:error("Error processing replication doc `~s` from `~s`: ~s",
+        [DocId, DbName, Reason]),
+    StartTimeBin = couch_replicator_utils:iso8601(StartTime),
+    update_rep_doc(DbName, DocId, [
+        {<<"_replication_state">>, <<"failed">>},
+        {<<"_replication_start_time">>, StartTimeBin},
+        {<<"_replication_stats">>, undefined},
+        {<<"_replication_state_reason">>, Reason}]),
+    couch_stats:increment_counter([couch_replicator, docs, failed_state_updates]).
+
+
+-spec update_triggered(#rep{}, rep_id()) -> ok.
+update_triggered(Rep, {Base, Ext}) ->
+    #rep{
+        db_name = DbName,
+        doc_id = DocId,
+        start_time = StartTime
+    } = Rep,
+    StartTimeBin = couch_replicator_utils:iso8601(StartTime),
+    update_rep_doc(DbName, DocId, [
+        {<<"_replication_state">>, <<"triggered">>},
+        {<<"_replication_state_reason">>, undefined},
+        {<<"_replication_id">>, iolist_to_binary([Base, Ext])},
+        {<<"_replication_start_time">>, StartTimeBin},
+        {<<"_replication_stats">>, undefined}]),
+    ok.
+
+
+-spec update_error(#rep{}, any()) -> ok.
+update_error(#rep{db_name = DbName, doc_id = DocId, id = RepId}, Error) ->
+    Reason = error_reason(Error),
+    BinRepId = case RepId of
+        {Base, Ext} ->
+            iolist_to_binary([Base, Ext]);
+        _Other ->
+            null
+    end,
+    update_rep_doc(DbName, DocId, [
+        {<<"_replication_state">>, <<"error">>},
+        {<<"_replication_state_reason">>, Reason},
+        {<<"_replication_stats">>, undefined},
+        {<<"_replication_id">>, BinRepId}]),
+    ok.
+
+
+-spec ensure_rep_db_exists() -> {ok, #db{}}.
+ensure_rep_db_exists() ->
+    Db = case couch_db:open_int(?REP_DB_NAME, [?CTX, sys_db, nologifmissing]) of
+        {ok, Db0} ->
+            Db0;
+        _Error ->
+            {ok, Db0} = couch_db:create(?REP_DB_NAME, [?CTX, sys_db]),
+            Db0
+    end,
+    ok = ensure_rep_ddoc_exists(?REP_DB_NAME),
+    {ok, Db}.
+
+
+-spec ensure_rep_ddoc_exists(binary()) -> ok.
+ensure_rep_ddoc_exists(RepDb) ->
+    case mem3:belongs(RepDb, ?REP_DESIGN_DOC) of
+        true ->
+            ensure_rep_ddoc_exists(RepDb, ?REP_DESIGN_DOC);
+        false ->
+            ok
+    end.
+
+
+-spec ensure_rep_ddoc_exists(binary(), binary()) -> ok.
+ensure_rep_ddoc_exists(RepDb, DDocId) ->
+    case open_rep_doc(RepDb, DDocId) of
+        {not_found, no_db_file} ->
+            %% database was deleted.
+            ok;
+        {not_found, _Reason} ->
+            DocProps = replication_design_doc_props(DDocId),
+            DDoc = couch_doc:from_json_obj({DocProps}),
+            couch_log:notice("creating replicator ddoc", []),
+            {ok, _Rev} = save_rep_doc(RepDb, DDoc);
+        {ok, Doc} ->
+            Latest = replication_design_doc_props(DDocId),
+            {Props0} = couch_doc:to_json_obj(Doc, []),
+            {value, {_, Rev}, Props} = lists:keytake(<<"_rev">>, 1, Props0),
+            case compare_ejson({Props}, {Latest}) of
+                true ->
+                    ok;
+                false ->
+                    LatestWithRev = [{<<"_rev">>, Rev} | Latest],
+                    DDoc = couch_doc:from_json_obj({LatestWithRev}),
+                    couch_log:notice("updating replicator ddoc", []),
+                    try
+                        {ok, _} = save_rep_doc(RepDb, DDoc)
+                    catch
+                        throw:conflict ->
+                            %% ignore, we'll retry next time
+                            ok
+                    end
+            end
+    end,
+    ok.
+
+
+-spec ensure_cluster_rep_ddoc_exists(binary()) -> ok.
+ensure_cluster_rep_ddoc_exists(RepDb) ->
+    DDocId = ?REP_DESIGN_DOC,
+    [#shard{name = DbShard} | _] = mem3:shards(RepDb, DDocId),
+    ensure_rep_ddoc_exists(DbShard, DDocId).
+
+
+-spec compare_ejson({[_]}, {[_]}) -> boolean().
+compare_ejson(EJson1, EJson2) ->
+    EjsonSorted1 = couch_replicator_filters:ejsort(EJson1),
+    EjsonSorted2 = couch_replicator_filters:ejsort(EJson2),
+    EjsonSorted1 == EjsonSorted2.
+
+
+-spec replication_design_doc_props(binary()) -> [_].
+replication_design_doc_props(DDocId) ->
+    TerminalViewEJson = {[
+                {<<"map">>, ?REP_DB_TERMINAL_STATE_VIEW_MAP_FUN},
+                {<<"reduce">>, <<"_count">>}
+            ]},
+    DocProps = [
+        {<<"_id">>, DDocId},
+        {<<"language">>, <<"javascript">>},
+        {<<"validate_doc_update">>, ?REP_DB_DOC_VALIDATE_FUN},
+        {<<"views">>, {[
+            {<<"terminal_states">>, TerminalViewEJson}
+        ]}}
+   ].
+
+
+% Note: parse_rep_doc can handle filtered replications. During parsing of the
+% replication doc it will make possibly remote http requests to the source
+% database. If failure or parsing of filter docs fails, parse_doc throws a
+% {filter_fetch_error, Error} excation. This exception should be considered transient
+% in respect to the contents of the document itself, since it depends on
+% netowrk availability of the source db and other factors.
+-spec parse_rep_doc({[_]}) -> #rep{}.
+parse_rep_doc(RepDoc) ->
+    {ok, Rep} = try
+        parse_rep_doc(RepDoc, rep_user_ctx(RepDoc))
+    catch
+    throw:{error, Reason} ->
+        throw({bad_rep_doc, Reason});
+    throw:{filter_fetch_error, Reason} ->
+        throw({filter_fetch_error, Reason});
+    Tag:Err ->
+        throw({bad_rep_doc, to_binary({Tag, Err})})
+    end,
+    Rep.
+
+
+-spec parse_rep_doc_without_id({[_]}) -> #rep{}.
+parse_rep_doc_without_id(RepDoc) ->
+    {ok, Rep} = try
+        parse_rep_doc_without_id(RepDoc, rep_user_ctx(RepDoc))
+    catch
+    throw:{error, Reason} ->
+        throw({bad_rep_doc, Reason});
+    Tag:Err ->
+        throw({bad_rep_doc, to_binary({Tag, Err})})
+    end,
+    Rep.
+
+
+-spec parse_rep_doc({[_]}, #user_ctx{}) -> {ok, #rep{}}.
+parse_rep_doc(Doc, UserCtx) ->
+    {ok, Rep} = parse_rep_doc_without_id(Doc, UserCtx),
+    Cancel = get_value(cancel, Rep#rep.options, false),
+    Id = get_value(id, Rep#rep.options, nil),
+    case {Cancel, Id} of
+        {true, nil} ->
+            % Cancel request with no id, must parse id out of body contents
+            {ok, update_rep_id(Rep)};
+        {true, Id} ->
+            % Cancel request with an id specified, so do not parse id from body
+            {ok, Rep};
+        {false, _Id} ->
+            % Not a cancel request, regular replication doc
+            {ok, update_rep_id(Rep)}
+    end.
+
+
+-spec parse_rep_doc_without_id({[_]}, #user_ctx{}) -> {ok, #rep{}}.
+parse_rep_doc_without_id({Props}, UserCtx) ->
+    Proxy = get_value(<<"proxy">>, Props, <<>>),
+    Opts = make_options(Props),
+    case get_value(cancel, Opts, false) andalso
+        (get_value(id, Opts, nil) =/= nil) of
+    true ->
+        {ok, #rep{options = Opts, user_ctx = UserCtx}};
+    false ->
+        Source = parse_rep_db(get_value(<<"source">>, Props), Proxy, Opts),
+        Target = parse_rep_db(get_value(<<"target">>, Props), Proxy, Opts),
+        {Type, View} = case couch_replicator_filters:view_type(Props, Opts) of
+        {error, Error} ->
+            throw({bad_request, Error});
+        Result ->
+            Result
+        end,
+        Rep = #rep{
+            source = Source,
+            target = Target,
+            options = Opts,
+            user_ctx = UserCtx,
+            type = Type,
+            view = View,
+            doc_id = get_value(<<"_id">>, Props, null)
+        },
+        % Check if can parse filter code, if not throw exception
+        case couch_replicator_filters:parse(Opts) of
+        {error, FilterError} ->
+            throw({error, FilterError});
+        {ok, _Filter} ->
+             ok
+        end,
+        {ok, Rep}
+    end.
+
+
+% Update a #rep{} record with a replication_id. Calculating the id might involve
+% fetching a filter from the source db, and so it could fail intermetently.
+% In case of a failure to fetch the filter this function will throw a
+%  `{filter_fetch_error, Reason} exception.
+update_rep_id(Rep) ->
+    RepId = couch_replicator_ids:replication_id(Rep),
+    Rep#rep{id = RepId}.
+
+
+update_rep_doc(RepDbName, RepDocId, KVs) ->
+    update_rep_doc(RepDbName, RepDocId, KVs, 1).
+
+update_rep_doc(RepDbName, RepDocId, KVs, Wait) when is_binary(RepDocId) ->
+    try
+        case open_rep_doc(RepDbName, RepDocId) of
+            {ok, LastRepDoc} ->
+                update_rep_doc(RepDbName, LastRepDoc, KVs, Wait * 2);
+            _ ->
+                ok
+        end
+    catch
+        throw:conflict ->
+            Msg = "Conflict when updating replication document `~s`. Retrying.",
+            couch_log:error(Msg, [RepDocId]),
+            ok = timer:sleep(random:uniform(erlang:min(128, Wait)) * 100),
+            update_rep_doc(RepDbName, RepDocId, KVs, Wait * 2)
+    end;
+update_rep_doc(RepDbName, #doc{body = {RepDocBody}} = RepDoc, KVs, _Try) ->
+    NewRepDocBody = lists:foldl(
+        fun({K, undefined}, Body) ->
+                lists:keydelete(K, 1, Body);
+           ({<<"_replication_state">> = K, State} = KV, Body) ->
+                case get_json_value(K, Body) of
+                State ->
+                    Body;
+                _ ->
+                    Body1 = lists:keystore(K, 1, Body, KV),
+                    Timestamp = couch_replicator_utils:iso8601(os:timestamp()),
+                    lists:keystore(
+                        <<"_replication_state_time">>, 1, Body1,
+                        {<<"_replication_state_time">>, Timestamp})
+                end;
+            ({K, _V} = KV, Body) ->
+                lists:keystore(K, 1, Body, KV)
+        end,
+        RepDocBody, KVs),
+    case NewRepDocBody of
+    RepDocBody ->
+        ok;
+    _ ->
+        % Might not succeed - when the replication doc is deleted right
+        % before this update (not an error, ignore).
+        save_rep_doc(RepDbName, RepDoc#doc{body = {NewRepDocBody}})
+    end.
+
+open_rep_doc(DbName, DocId) ->
+    case couch_db:open_int(DbName, [?CTX, sys_db]) of
+        {ok, Db} ->
+            try
+                couch_db:open_doc(Db, DocId, [ejson_body])
+            after
+                couch_db:close(Db)
+            end;
+        Else ->
+            Else
+    end.
+
+save_rep_doc(DbName, Doc) ->
+    {ok, Db} = couch_db:open_int(DbName, [?CTX, sys_db]),
+    try
+        couch_db:update_doc(Db, Doc, [])
+    after
+        couch_db:close(Db)
+    end.
+
+
+-spec rep_user_ctx({[_]}) -> #user_ctx{}.
+rep_user_ctx({RepDoc}) ->
+    case get_json_value(<<"user_ctx">>, RepDoc) of
+    undefined ->
+        #user_ctx{};
+    {UserCtx} ->
+        #user_ctx{
+            name = get_json_value(<<"name">>, UserCtx, null),
+            roles = get_json_value(<<"roles">>, UserCtx, [])
+        }
+    end.
+
+-spec parse_rep_db({[_]} | binary(), binary(), [_]) -> #httpd{} | binary().
+parse_rep_db({Props}, Proxy, Options) ->
+    ProxyParams = parse_proxy_params(Proxy),
+    ProxyURL = case ProxyParams of
+        [] -> undefined;
+        _ -> binary_to_list(Proxy)
+    end,
+    Url = maybe_add_trailing_slash(get_value(<<"url">>, Props)),
+    {AuthProps} = get_value(<<"auth">>, Props, {[]}),
+    {BinHeaders} = get_value(<<"headers">>, Props, {[]}),
+    Headers = lists:ukeysort(1, [{?b2l(K), ?b2l(V)} || {K, V} <- BinHeaders]),
+    DefaultHeaders = (#httpdb{})#httpdb.headers,
+    OAuth = case get_value(<<"oauth">>, AuthProps) of
+    undefined ->
+        nil;
+    {OauthProps} ->
+        #oauth{
+            consumer_key = ?b2l(get_value(<<"consumer_key">>, OauthProps)),
+            token = ?b2l(get_value(<<"token">>, OauthProps)),
+            token_secret = ?b2l(get_value(<<"token_secret">>, OauthProps)),
+            consumer_secret = ?b2l(get_value(<<"consumer_secret">>, OauthProps)),
+            signature_method =
+                case get_value(<<"signature_method">>, OauthProps) of
+                undefined ->        hmac_sha1;
+                <<"PLAINTEXT">> ->  plaintext;
+                <<"HMAC-SHA1">> ->  hmac_sha1;
+                <<"RSA-SHA1">> ->   rsa_sha1
+                end
+        }
+    end,
+    #httpdb{
+        url = Url,
+        oauth = OAuth,
+        headers = lists:ukeymerge(1, Headers, DefaultHeaders),
+        ibrowse_options = lists:keysort(1,
+            [{socket_options, get_value(socket_options, Options)} |
+                ProxyParams ++ ssl_params(Url)]),
+        timeout = get_value(connection_timeout, Options),
+        http_connections = get_value(http_connections, Options),
+        retries = get_value(retries, Options),
+        proxy_url = ProxyURL
+    };
+parse_rep_db(<<"http://", _/binary>> = Url, Proxy, Options) ->
+    parse_rep_db({[{<<"url">>, Url}]}, Proxy, Options);
+parse_rep_db(<<"https://", _/binary>> = Url, Proxy, Options) ->
+    parse_rep_db({[{<<"url">>, Url}]}, Proxy, Options);
+parse_rep_db(<<DbName/binary>>, _Proxy, _Options) ->
+    DbName;
+parse_rep_db(undefined, _Proxy, _Options) ->
+    throw({error, <<"Missing replicator database">>}).
+
+
+-spec maybe_add_trailing_slash(binary() | list()) -> list().
+maybe_add_trailing_slash(Url) when is_binary(Url) ->
+    maybe_add_trailing_slash(?b2l(Url));
+maybe_add_trailing_slash(Url) ->
+    case lists:last(Url) of
+    $/ ->
+        Url;
+    _ ->
+        Url ++ "/"
+    end.
+
+-spec make_options([_]) -> [_].
+make_options(Props) ->
+    Options0 = lists:ukeysort(1, convert_options(Props)),
+    Options = check_options(Options0),
+    DefWorkers = config:get("replicator", "worker_processes", "4"),
+    DefBatchSize = config:get("replicator", "worker_batch_size", "500"),
+    DefConns = config:get("replicator", "http_connections", "20"),
+    DefTimeout = config:get("replicator", "connection_timeout", "30000"),
+    DefRetries = config:get("replicator", "retries_per_request", "10"),
+    UseCheckpoints = config:get("replicator", "use_checkpoints", "true"),
+    DefCheckpointInterval = config:get("replicator", "checkpoint_interval", "30000"),
+    {ok, DefSocketOptions} = couch_util:parse_term(
+        config:get("replicator", "socket_options",
+            "[{keepalive, true}, {nodelay, false}]")),
+    lists:ukeymerge(1, Options, lists:keysort(1, [
+        {connection_timeout, list_to_integer(DefTimeout)},
+        {retries, list_to_integer(DefRetries)},
+        {http_connections, list_to_integer(DefConns)},
+        {socket_options, DefSocketOptions},
+        {worker_batch_size, list_to_integer(DefBatchSize)},
+        {worker_processes, list_to_integer(DefWorkers)},
+        {use_checkpoints, list_to_existing_atom(UseCheckpoints)},
+        {checkpoint_interval, list_to_integer(DefCheckpointInterval)}
+    ])).
+
+
+-spec convert_options([_]) -> [_].
+convert_options([])->
+    [];
+convert_options([{<<"cancel">>, V} | _R]) when not is_boolean(V)->
+    throw({bad_request, <<"parameter `cancel` must be a boolean">>});
+convert_options([{<<"cancel">>, V} | R]) ->
+    [{cancel, V} | convert_options(R)];
+convert_options([{IdOpt, V} | R]) when IdOpt =:= <<"_local_id">>;
+        IdOpt =:= <<"replication_id">>; IdOpt =:= <<"id">> ->
+    [{id, couch_replicator_ids:convert(V)} | convert_options(R)];
+convert_options([{<<"create_target">>, V} | _R]) when not is_boolean(V)->
+    throw({bad_request, <<"parameter `create_target` must be a boolean">>});
+convert_options([{<<"create_target">>, V} | R]) ->
+    [{create_target, V} | convert_options(R)];
+convert_options([{<<"continuous">>, V} | _R]) when not is_boolean(V)->
+    throw({bad_request, <<"parameter `continuous` must be a boolean">>});
+convert_options([{<<"continuous">>, V} | R]) ->
+    [{continuous, V} | convert_options(R)];
+convert_options([{<<"filter">>, V} | R]) ->
+    [{filter, V} | convert_options(R)];
+convert_options([{<<"query_params">>, V} | R]) ->
+    [{query_params, V} | convert_options(R)];
+convert_options([{<<"doc_ids">>, null} | R]) ->
+    convert_options(R);
+convert_options([{<<"doc_ids">>, V} | _R]) when not is_list(V) ->
+    throw({bad_request, <<"parameter `doc_ids` must be an array">>});
+convert_options([{<<"doc_ids">>, V} | R]) ->
+    % Ensure same behaviour as old replicator: accept a list of percent
+    % encoded doc IDs.
+    DocIds = lists:usort([?l2b(couch_httpd:unquote(Id)) || Id <- V]),
+    [{doc_ids, DocIds} | convert_options(R)];
+convert_options([{<<"selector">>, V} | _R]) when not is_tuple(V) ->
+    throw({bad_request, <<"parameter `selector` must be a JSON object">>});
+convert_options([{<<"selector">>, V} | R]) ->
+    [{selector, V} | convert_options(R)];
+convert_options([{<<"worker_processes">>, V} | R]) ->
+    [{worker_processes, couch_util:to_integer(V)} | convert_options(R)];
+convert_options([{<<"worker_batch_size">>, V} | R]) ->
+    [{worker_batch_size, couch_util:to_integer(V)} | convert_options(R)];
+convert_options([{<<"http_connections">>, V} | R]) ->
+    [{http_connections, couch_util:to_integer(V)} | convert_options(R)];
+convert_options([{<<"connection_timeout">>, V} | R]) ->
+    [{connection_timeout, couch_util:to_integer(V)} | convert_options(R)];
+convert_options([{<<"retries_per_request">>, V} | R]) ->
+    [{retries, couch_util:to_integer(V)} | convert_options(R)];
+convert_options([{<<"socket_options">>, V} | R]) ->
+    {ok, SocketOptions} = couch_util:parse_term(V),
+    [{socket_options, SocketOptions} | convert_options(R)];
+convert_options([{<<"since_seq">>, V} | R]) ->
+    [{since_seq, V} | convert_options(R)];
+convert_options([{<<"use_checkpoints">>, V} | R]) ->
+    [{use_checkpoints, V} | convert_options(R)];
+convert_options([{<<"checkpoint_interval">>, V} | R]) ->
+    [{checkpoint_interval, couch_util:to_integer(V)} | convert_options(R)];
+convert_options([_ | R]) -> % skip unknown option
+    convert_options(R).
+
+-spec check_options([_]) -> [_].
+check_options(Options) ->
+    DocIds = lists:keyfind(doc_ids, 1, Options),
+    Filter = lists:keyfind(filter, 1, Options),
+    Selector = lists:keyfind(selector, 1, Options),
+    case {DocIds, Filter, Selector} of
+        {false, false, false} -> Options;
+        {false, false, _} -> Options;
+        {false, _, false} -> Options;
+        {_, false, false} -> Options;
+        _ ->
+            throw({bad_request, "`doc_ids`, `filter`, `selector` are mutually exclusive options"})
+    end.
+
+-spec parse_proxy_params(binary() | [_]) -> [_].
+parse_proxy_params(ProxyUrl) when is_binary(ProxyUrl) ->
+    parse_proxy_params(?b2l(ProxyUrl));
+parse_proxy_params([]) ->
+    [];
+parse_proxy_params(ProxyUrl) ->
+    #url{
+        host = Host,
+        port = Port,
+        username = User,
+        password = Passwd,
+        protocol = Protocol
+    } = ibrowse_lib:parse_url(ProxyUrl),
+    [{proxy_protocol, Protocol}, {proxy_host, Host}, {proxy_port, Port}] ++
+        case is_list(User) andalso is_list(Passwd) of
+        false ->
+            [];
+        true ->
+            [{proxy_user, User}, {proxy_password, Passwd}]
+        end.
+
+-spec ssl_params([_]) -> [_].
+ssl_params(Url) ->
+    case ibrowse_lib:parse_url(Url) of
+    #url{protocol = https} ->
+        Depth = list_to_integer(
+            config:get("replicator", "ssl_certificate_max_depth", "3")
+        ),
+        VerifyCerts = config:get("replicator", "verify_ssl_certificates"),
+        CertFile = config:get("replicator", "cert_file", undefined),
+        KeyFile = config:get("replicator", "key_file", undefined),
+        Password = config:get("replicator", "password", undefined),
+        SslOpts = [{depth, Depth} | ssl_verify_options(VerifyCerts =:= "true")],
+        SslOpts1 = case CertFile /= undefined andalso KeyFile /= undefined of
+            true ->
+                case Password of
+                    undefined ->
+                        [{certfile, CertFile}, {keyfile, KeyFile}] ++ SslOpts;
+                    _ ->
+                        [{certfile, CertFile}, {keyfile, KeyFile},
+                            {password, Password}] ++ SslOpts
+                end;
+            false -> SslOpts
+        end,
+        [{is_ssl, true}, {ssl_options, SslOpts1}];
+    #url{protocol = http} ->
+        []
+    end.
+
+-spec ssl_verify_options(true | false) -> [_].
+ssl_verify_options(true) ->
+    CAFile = config:get("replicator", "ssl_trusted_certificates_file"),
+    [{verify, verify_peer}, {cacertfile, CAFile}];
+ssl_verify_options(false) ->
+    [{verify, verify_none}].
+
+
+-spec before_doc_update(#doc{}, #db{}) -> #doc{}.
+before_doc_update(#doc{id = <<?DESIGN_DOC_PREFIX, _/binary>>} = Doc, _Db) ->
+    Doc;
+before_doc_update(#doc{body = {Body}} = Doc, #db{user_ctx=UserCtx} = Db) ->
+    #user_ctx{roles = Roles, name = Name} = UserCtx,
+    case lists:member(<<"_replicator">>, Roles) of
+    true ->
+        Doc;
+    false ->
+        case couch_util:get_value(?OWNER, Body) of
+        undefined ->
+            Doc#doc{body = {?replace(Body, ?OWNER, Name)}};
+        Name ->
+            Doc;
+        Other ->
+            case (catch couch_db:check_is_admin(Db)) of
+            ok when Other =:= null ->
+                Doc#doc{body = {?replace(Body, ?OWNER, Name)}};
+            ok ->
+                Doc;
+            _ ->
+                throw({forbidden, <<"Can't update replication documents",
+                    " from other users.">>})
+            end
+        end
+    end.
+
+
+-spec after_doc_read(#doc{}, #db{}) -> #doc{}.
+after_doc_read(#doc{id = <<?DESIGN_DOC_PREFIX, _/binary>>} = Doc, _Db) ->
+    Doc;
+after_doc_read(#doc{body = {Body}} = Doc, #db{user_ctx=UserCtx} = Db) ->
+    #user_ctx{name = Name} = UserCtx,
+    case (catch couch_db:check_is_admin(Db)) of
+    ok ->
+        Doc;
+    _ ->
+        case couch_util:get_value(?OWNER, Body) of
+        Name ->
+            Doc;
+        _Other ->
+            Source = strip_credentials(couch_util:get_value(<<"source">>,
+Body)),
+            Target = strip_credentials(couch_util:get_value(<<"target">>,
+Body)),
+            NewBody0 = ?replace(Body, <<"source">>, Source),
+            NewBody = ?replace(NewBody0, <<"target">>, Target),
+            #doc{revs = {Pos, [_ | Revs]}} = Doc,
+            NewDoc = Doc#doc{body = {NewBody}, revs = {Pos - 1, Revs}},
+            NewRevId = couch_db:new_revid(NewDoc),
+            NewDoc#doc{revs = {Pos, [NewRevId | Revs]}}
+        end
+    end.
+
+
+-spec strip_credentials(undefined) -> undefined;
+    (binary()) -> binary();
+    ({[_]}) -> {[_]}.
+strip_credentials(undefined) ->
+    undefined;
+strip_credentials(Url) when is_binary(Url) ->
+    re:replace(Url,
+        "http(s)?://(?:[^:]+):[^@]+@(.*)$",
+        "http\\1://\\2",
+        [{return, binary}]);
+strip_credentials({Props}) ->
+    {lists:keydelete(<<"oauth">>, 1, Props)}.
+
+
+error_reason({shutdown, Error}) ->
+    error_reason(Error);
+error_reason({bad_rep_doc, Reason}) ->
+    to_binary(Reason);
+error_reason({error, {Error, Reason}})
+  when is_atom(Error), is_binary(Reason) ->
+    to_binary(io_lib:format("~s: ~s", [Error, Reason]));
+error_reason({error, Reason}) ->
+    to_binary(Reason);
+error_reason(Reason) ->
+    to_binary(Reason).
+
+
+
+-ifdef(TEST).
+
+-include_lib("eunit/include/eunit.hrl").
+
+check_options_pass_values_test() ->
+    ?assertEqual(check_options([]), []),
+    ?assertEqual(check_options([baz, {other,fiz}]), [baz, {other, fiz}]),
+    ?assertEqual(check_options([{doc_ids, x}]), [{doc_ids, x}]),
+    ?assertEqual(check_options([{filter, x}]), [{filter, x}]),
+    ?assertEqual(check_options([{selector, x}]), [{selector, x}]).
+
+check_options_fail_values_test() ->
+    ?assertThrow({bad_request, _},
+        check_options([{doc_ids, x}, {filter, y}])),
+    ?assertThrow({bad_request, _},
+        check_options([{doc_ids, x}, {selector, y}])),
+    ?assertThrow({bad_request, _},
+        check_options([{filter, x}, {selector, y}])),
+    ?assertThrow({bad_request, _},
+        check_options([{doc_ids, x}, {selector, y}, {filter, z}])).
+
+check_convert_options_pass_test() ->
+    ?assertEqual([], convert_options([])),
+    ?assertEqual([], convert_options([{<<"random">>, 42}])),
+    ?assertEqual([{cancel, true}],
+        convert_options([{<<"cancel">>, true}])),
+    ?assertEqual([{create_target, true}],
+        convert_options([{<<"create_target">>, true}])),
+    ?assertEqual([{continuous, true}],
+        convert_options([{<<"continuous">>, true}])),
+    ?assertEqual([{doc_ids, [<<"id">>]}],
+        convert_options([{<<"doc_ids">>, [<<"id">>]}])),
+    ?assertEqual([{selector, {key, value}}],
+        convert_options([{<<"selector">>, {key, value}}])).
+
+check_convert_options_fail_test() ->
+    ?assertThrow({bad_request, _},
+        convert_options([{<<"cancel">>, <<"true">>}])),
+    ?assertThrow({bad_request, _},
+        convert_options([{<<"create_target">>, <<"true">>}])),
+    ?assertThrow({bad_request, _},
+        convert_options([{<<"continuous">>, <<"true">>}])),
+    ?assertThrow({bad_request, _},
+        convert_options([{<<"doc_ids">>, not_a_list}])),
+    ?assertThrow({bad_request, _},
+        convert_options([{<<"selector">>, [{key, value}]}])).
+
+-endif.
diff --git a/src/couch_replicator_filters.erl b/src/couch_replicator_filters.erl
new file mode 100644
index 0000000..346b448
--- /dev/null
+++ b/src/couch_replicator_filters.erl
@@ -0,0 +1,208 @@
+% 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_replicator_filters).
+
+-export([parse/1, fetch/4, view_type/2]).
+-export([ejsort/1]).
+
+-include_lib("couch/include/couch_db.hrl").
+
+-import(couch_util, [
+    get_value/2,
+    get_value/3
+]).
+
+
+% Parse the filter from replication options proplist.
+% Return {ok, {FilterType,...}} | {error, ParseError}.
+% For `user` filter, i.e. filters specified as user code
+% in source database, this code doesn't fetch the filter
+% code, but only returns the name of the filter.
+-spec parse([_]) ->
+  {ok, nil} |
+  {ok, {view, binary(), {[_]}}} |
+  {ok, {user, {binary(), binary()}, {[_]}}} |
+  {ok, {docids, [_]}} |
+  {ok, {mango, {[_]}}} |
+  {error, binary()}.
+parse(Options) ->
+    Filter = get_value(filter, Options),
+    DocIds = get_value(doc_ids, Options),
+    Selector = get_value(selector, Options),
+    case {Filter, DocIds, Selector} of
+        {undefined, undefined, undefined} ->
+            {ok, nil};
+        {<<"_", _/binary>>, undefined, undefined} ->
+            {ok, {view, Filter, query_params(Options)}};
+        {_, undefined, undefined} ->
+            case parse_user_filter(Filter) of
+                {ok, {Doc, FilterName}} ->
+                    {ok, {user, {Doc, FilterName}, query_params(Options)}};
+                {error, Error} ->
+                    {error, Error}
+            end;
+        {undefined, _, undefined} ->
+            {ok, {docids, DocIds}};
+        {undefined, undefined, _} ->
+            {ok, {mango, ejsort(mango_selector:normalize(Selector))}};
+        _ ->
+            Err = "`selector`, `filter` and `doc_ids` are mutually exclusive",
+            {error, list_to_binary(Err)}
+    end.
+
+
+% Fetches body of filter function from source database. Guaranteed to either
+% return {ok, Body} or an {error, Reason}. Also assume this function might
+% block due to network / socket issues for an undeterminted amount of time.
+-spec fetch(binary(), binary(), binary(), #user_ctx{}) ->
+    {ok, {[_]}} | {error, binary()}.
+fetch(DDocName, FilterName, Source, UserCtx) ->
+    {Pid, Ref} = spawn_monitor(fun() ->
+        try fetch_internal(DDocName, FilterName, Source, UserCtx) of
+            Resp ->
+                exit({exit_ok, Resp})
+        catch
+            throw:{fetch_error, Reason} ->
+                exit({exit_fetch_error, Reason});
+            _OtherTag:Reason ->
+                exit({exit_other_error, Reason})
+        end
+    end),
+    receive
+        {'DOWN', Ref, process, Pid, {exit_ok, Resp}} ->
+            {ok, Resp};
+        {'DOWN', Ref, process, Pid, {exit_fetch_error, Reason}} ->
+            {error, Reason};
+        {'DOWN', Ref, process, Pid, {exit_other_error, Reason}} ->
+            {error, couch_util:to_binary(Reason)}
+    end.
+
+
+% Get replication type and view (if any) from replication document props
+-spec view_type([_], [_]) ->
+    {view, {binary(), binary()}} | {db, nil} | {error, binary()}.
+view_type(Props, Options) ->
+    case get_value(<<"filter">>, Props) of
+        <<"_view">> ->
+            {QP}  = get_value(query_params, Options, {[]}),
+            ViewParam = get_value(<<"view">>, QP),
+            case re:split(ViewParam, <<"/">>) of
+                [DName, ViewName] ->
+                    {view, {<< "_design/", DName/binary >>, ViewName}};
+                _ ->
+                    {error, <<"Invalid `view` parameter.">>}
+            end;
+        _ ->
+            {db, nil}
+    end.
+
+
+% Private functions
+
+fetch_internal(DDocName, FilterName, Source, UserCtx) ->
+    Db = case (catch couch_replicator_api_wrap:db_open(Source, [{user_ctx, UserCtx}])) of
+    {ok, Db0} ->
+        Db0;
+    DbError ->
+        DbErrorMsg = io_lib:format("Could not open source database `~s`: ~s",
+           [couch_replicator_api_wrap:db_uri(Source), couch_util:to_binary(DbError)]),
+        throw({fetch_error, iolist_to_binary(DbErrorMsg)})
+    end,
+    try
+        Body = case (catch couch_replicator_api_wrap:open_doc(
+            Db, <<"_design/", DDocName/binary>>, [ejson_body])) of
+        {ok, #doc{body = Body0}} ->
+            Body0;
+        DocError ->
+            DocErrorMsg = io_lib:format(
+                "Couldn't open document `_design/~s` from source "
+                "database `~s`: ~s", [DDocName, couch_replicator_api_wrap:db_uri(Source),
+                    couch_util:to_binary(DocError)]),
+            throw({fetch_error, iolist_to_binary(DocErrorMsg)})
+        end,
+        try
+            Code = couch_util:get_nested_json_value(
+                     Body, [<<"filters">>, FilterName]),
+            re:replace(Code, [$^, "\s*(.*?)\s*", $$], "\\1", [{return, binary}])
+         catch
+             _Tag:CodeError ->
+                 CodeErrorMsg = io_lib:format(
+                     "Couldn't parse filter code from document ~s on `~s` "
+                     " Error: ~s", [DDocName, couch_replicator_api_wrap:db_uri(Source),
+                         couch_util:to_binary(CodeError)]),
+                 throw({fetch_error, CodeErrorMsg})
+         end
+    after
+        couch_replicator_api_wrap:db_close(Db)
+    end.
+
+
+
+-spec query_params([_]) -> {[_]}.
+query_params(Options)->
+    get_value(query_params, Options, {[]}).
+
+
+
+parse_user_filter(Filter) ->
+    case re:run(Filter, "(.*?)/(.*)", [{capture, [1, 2], binary}]) of
+        {match, [DDocName0, FilterName0]} ->
+            {ok, {DDocName0, FilterName0}};
+        _ ->
+            {error, <<"Invalid filter. Must match `ddocname/filtername`.">>}
+    end.
+
+
+% Sort an EJSON object's properties to attempt
+% to generate a unique representation. This is used
+% to reduce the chance of getting different
+% replication checkpoints for the same Mango selector
+ejsort({V})->
+    ejsort_props(V, []);
+ejsort(V) when is_list(V) ->
+    ejsort_array(V, []);
+ejsort(V) ->
+    V.
+
+ejsort_props([], Acc)->
+    {lists:keysort(1, Acc)};
+ejsort_props([{K, V}| R], Acc) ->
+    ejsort_props(R, [{K, ejsort(V)} | Acc]).
+
+ejsort_array([], Acc)->
+    lists:reverse(Acc);
+ejsort_array([V | R], Acc) ->
+    ejsort_array(R, [ejsort(V) | Acc]).
+
+
+-ifdef(TEST).
+
+-include_lib("eunit/include/eunit.hrl").
+
+ejsort_basic_values_test() ->
+    ?assertEqual(ejsort(0), 0),
+    ?assertEqual(ejsort(<<"a">>), <<"a">>),
+    ?assertEqual(ejsort(true), true),
+    ?assertEqual(ejsort([]), []),
+    ?assertEqual(ejsort({[]}), {[]}).
+
+ejsort_compound_values_test() ->
+    ?assertEqual(ejsort([2, 1, 3 ,<<"a">>]), [2, 1, 3, <<"a">>]),
+    Ej1 = {[{<<"a">>, 0}, {<<"c">>, 0},  {<<"b">>, 0}]},
+    Ej1s =  {[{<<"a">>, 0}, {<<"b">>, 0}, {<<"c">>, 0}]},
+    ?assertEqual(ejsort(Ej1), Ej1s),
+    Ej2 = {[{<<"x">>, Ej1}, {<<"z">>, Ej1}, {<<"y">>, [Ej1, Ej1]}]},
+    ?assertEqual(ejsort(Ej2),
+        {[{<<"x">>, Ej1s}, {<<"y">>, [Ej1s, Ej1s]}, {<<"z">>, Ej1s}]}).
+
+-endif.
diff --git a/src/couch_replicator_httpc.erl b/src/couch_replicator_httpc.erl
index 309a230..58fb0e1 100644
--- a/src/couch_replicator_httpc.erl
+++ b/src/couch_replicator_httpc.erl
@@ -40,8 +40,18 @@
 -define(MAX_DISCARDED_MESSAGES, 16).
 
 
-setup(#httpdb{httpc_pool = nil, url = Url, http_connections = MaxConns} = Db) ->
-    {ok, Pid} = couch_replicator_httpc_pool:start_link(Url, [{max_connections, MaxConns}]),
+setup(Db) ->
+    #httpdb{
+        httpc_pool = nil,
+        url = Url,
+        http_connections = MaxConns,
+        proxy_url = ProxyURL
+    } = Db,
+    HttpcURL = case ProxyURL of
+        undefined -> Url;
+        _ when is_list(ProxyURL) -> ProxyURL
+    end,
+    {ok, Pid} = couch_replicator_httpc_pool:start_link(HttpcURL, [{max_connections, MaxConns}]),
     {ok, Db#httpdb{httpc_pool = Pid}}.
 
 
@@ -98,6 +108,7 @@
         lists:ukeymerge(1, get_value(ibrowse_options, Params, []),
             HttpDb#httpdb.ibrowse_options)
     ],
+    backoff_before_request(Worker, HttpDb, Params),
     Response = ibrowse:send_req_direct(
         Worker, Url, Headers2, Method, Body, IbrowseOptions, Timeout),
     {Worker, Response}.
@@ -139,8 +150,9 @@
 process_response({ok, Code, Headers, Body}, Worker, HttpDb, Params, Callback) ->
     case list_to_integer(Code) of
     429 ->
-        backoff(Worker, HttpDb, Params);
+        backoff(HttpDb, Params);
     Ok when (Ok >= 200 andalso Ok < 300) ; (Ok >= 400 andalso Ok < 500) ->
+        backoff_success(HttpDb, Params),
         couch_stats:increment_counter([couch_replicator, responses, success]),
         EJson = case Body of
         <<>> ->
@@ -150,6 +162,7 @@
         end,
         Callback(Ok, Headers, EJson);
     R when R =:= 301 ; R =:= 302 ; R =:= 303 ->
+        backoff_success(HttpDb, Params),
         do_redirect(Worker, R, Headers, HttpDb, Params, Callback);
     Error ->
         couch_stats:increment_counter([couch_replicator, responses, failure]),
@@ -165,9 +178,10 @@
     {ibrowse_async_headers, ReqId, Code, Headers} ->
         case list_to_integer(Code) of
         429 ->
-            backoff(Worker, HttpDb#httpdb{timeout = get_max_back_off()},
-                Params);
+            Timeout = couch_replicator_rate_limiter:max_interval(),
+            backoff(HttpDb#httpdb{timeout = Timeout}, Params);
         Ok when (Ok >= 200 andalso Ok < 300) ; (Ok >= 400 andalso Ok < 500) ->
+            backoff_success(HttpDb, Params),
             StreamDataFun = fun() ->
                 stream_data_self(HttpDb, Params, Worker, ReqId, Callback)
             end,
@@ -184,6 +198,7 @@
                     maybe_retry(Err, Worker, HttpDb, Params)
             end;
         R when R =:= 301 ; R =:= 302 ; R =:= 303 ->
+            backoff_success(HttpDb, Params),
             do_redirect(Worker, R, Headers, HttpDb, Params, Callback);
         Error ->
             couch_stats:increment_counter(
@@ -266,37 +281,48 @@
     end.
 
 
-%% For 429 errors, we perform an exponential backoff up to 2.17 hours.
-%% We use Backoff time as a timeout/failure end.
-backoff(Worker, #httpdb{backoff = Backoff} = HttpDb, Params) ->
-    MaxBackOff = get_max_back_off(),
-    MaxBackOffLog = get_back_off_log_threshold(),
-    ok = timer:sleep(random:uniform(Backoff)),
-    Backoff2 = round(Backoff*get_back_off_exp()),
-    NewBackoff = erlang:min(Backoff2, MaxBackOff),
-    NewHttpDb = HttpDb#httpdb{backoff = NewBackoff},
-    case Backoff2 of
-        W0 when W0 > MaxBackOff ->
-            report_error(Worker, HttpDb, Params, {error,
-                "Long 429-induced Retry Time Out"});
-        W1 when W1 >=  MaxBackOffLog -> % Past 8 min, we log retries
-            log_retry_error(Params, HttpDb, Backoff2, "429 Retry"),
-            throw({retry, NewHttpDb, Params});
-        _ ->
-            throw({retry, NewHttpDb, Params})
-    end.
-
-
 maybe_retry(Error, Worker, #httpdb{retries = 0} = HttpDb, Params) ->
     report_error(Worker, HttpDb, Params, {error, Error});
 
-maybe_retry(Error, _Worker, #httpdb{retries = Retries, wait = Wait} = HttpDb,
+maybe_retry(Error, Worker, #httpdb{retries = Retries, wait = Wait} = HttpDb,
     Params) ->
-    ok = timer:sleep(Wait),
-    log_retry_error(Params, HttpDb, Wait, Error),
-    Wait2 = erlang:min(Wait * 2, ?MAX_WAIT),
-    NewHttpDb = HttpDb#httpdb{retries = Retries - 1, wait = Wait2},
-    throw({retry, NewHttpDb, Params}).
+    case total_error_time_exceeded(HttpDb) of
+        true ->
+            report_error(Worker, HttpDb, Params, {error, Error});
+        false ->
+            ok = timer:sleep(Wait),
+            log_retry_error(Params, HttpDb, Wait, Error),
+            Wait2 = erlang:min(Wait * 2, ?MAX_WAIT),
+            HttpDb1 = HttpDb#httpdb{retries = Retries - 1, wait = Wait2},
+            HttpDb2 = update_first_error_timestamp(HttpDb1),
+            throw({retry, HttpDb2, Params})
+    end.
+
+
+% When retrying, check to make total time spent retrying a request is below
+% the current scheduler health threshold. The goal is to not exceed the
+% threshold, otherwise the job which keep retrying too long will still be
+% considered healthy.
+total_error_time_exceeded(#httpdb{first_error_timestamp = nil}) ->
+    false;
+
+total_error_time_exceeded(#httpdb{first_error_timestamp = ErrorTimestamp}) ->
+    HealthThresholdSec = couch_replicator_scheduler:health_threshold(),
+    % Theshold value is halved because in the calling code the next step
+    % is a doubling. Not halving here could mean sleeping too long and
+    % exceeding the health threshold.
+    ThresholdUSec = (HealthThresholdSec / 2) * 1000000,
+    timer:now_diff(os:timestamp(), ErrorTimestamp) > ThresholdUSec.
+
+
+% Remember the first time an error occurs. This value is used later to check
+% the total time spend retrying a request. Because retrying is cursive, on
+% successful result #httpdb{} record is reset back to the original value.
+update_first_error_timestamp(#httpdb{first_error_timestamp = nil} = HttpDb) ->
+    HttpDb#httpdb{first_error_timestamp = os:timestamp()};
+
+update_first_error_timestamp(HttpDb) ->
+    HttpDb.
 
 
 log_retry_error(Params, HttpDb, Wait, Error) ->
@@ -440,11 +466,32 @@
     Params2 = lists:keydelete(path, 1, lists:keydelete(qs, 1, Params)),
     {HttpDb#httpdb{url = RedirectUrl}, Params2}.
 
-get_max_back_off() ->
-    config:get_integer("replicator", "max_backoff_wait", 250 * 32768).
 
-get_back_off_log_threshold() ->
-    config:get_integer("replicator", "max_backoff_log_threshold", 512000).
+backoff_key(HttpDb, Params) ->
+    Method = get_value(method, Params, get),
+    Url = HttpDb#httpdb.url,
+    {Url, Method}.
 
-get_back_off_exp() ->
-    config:get_float("replicator", "backoff_exp", 1.5).
+
+backoff(HttpDb, Params) ->
+    Key = backoff_key(HttpDb, Params),
+    couch_replicator_rate_limiter:failure(Key),
+    throw({retry, HttpDb, Params}).
+
+
+backoff_success(HttpDb, Params) ->
+    Key = backoff_key(HttpDb, Params),
+    couch_replicator_rate_limiter:success(Key).
+
+
+backoff_before_request(Worker, HttpDb, Params) ->
+    Key = backoff_key(HttpDb, Params),
+    Limit = couch_replicator_rate_limiter:max_interval(),
+    case couch_replicator_rate_limiter:interval(Key) of
+        Sleep when Sleep >= Limit ->
+            report_error(Worker, HttpDb, Params, max_backoff);
+        Sleep when Sleep >= 1 ->
+            timer:sleep(Sleep);
+        Sleep when Sleep == 0 ->
+            ok
+    end.
diff --git a/src/couch_replicator_httpc_pool.erl b/src/couch_replicator_httpc_pool.erl
index 09e3b23..9f1f1a8 100644
--- a/src/couch_replicator_httpc_pool.erl
+++ b/src/couch_replicator_httpc_pool.erl
@@ -31,8 +31,7 @@
 -record(state, {
     url,
     limit,                  % max # of workers allowed
-    free = [],              % free workers (connections)
-    busy = [],              % busy workers (connections)
+    conns = [],
     waiting = queue:new(),  % blocked clients waiting for a worker
     callers = []            % clients who've been given a worker
 }).
@@ -70,23 +69,17 @@
         callers = Callers,
         url = Url,
         limit = Limit,
-        busy = Busy,
-        free = Free
+        conns = Conns
     } = State,
-    case length(Busy) >= Limit of
+    case length(Conns) >= Limit of
     true ->
         {noreply, State#state{waiting = queue:in(From, Waiting)}};
     false ->
-        case Free of
-        [] ->
-           {ok, Worker} = ibrowse:spawn_link_worker_process(Url),
-           Free2 = Free;
-        [Worker | Free2] ->
-           ok
-        end,
+        % If the call to acquire fails, the worker pool will crash with a
+        % badmatch.
+        {ok, Worker} = couch_replicator_connection:acquire(Url),
         NewState = State#state{
-            free = Free2,
-            busy = [Worker | Busy],
+            conns = [Worker | Conns],
             callers = monitor_client(Callers, Worker, From)
         },
         {reply, {ok, Worker}, NewState}
@@ -104,35 +97,29 @@
 handle_info({'EXIT', Pid, _Reason}, State) ->
     #state{
         url = Url,
-        busy = Busy,
-        free = Free,
+        conns = Conns,
         waiting = Waiting,
         callers = Callers
     } = State,
     NewCallers0 = demonitor_client(Callers, Pid),
-    case Free -- [Pid] of
-    Free ->
-        case Busy -- [Pid] of
-        Busy ->
+    case Conns -- [Pid] of
+        Conns ->
             {noreply, State#state{callers = NewCallers0}};
-        Busy2 ->
+        Conns2 ->
             case queue:out(Waiting) of
-            {empty, _} ->
-                {noreply, State#state{busy = Busy2, callers = NewCallers0}};
-            {{value, From}, Waiting2} ->
-                {ok, Worker} = ibrowse:spawn_link_worker_process(Url),
-                NewCallers1 = monitor_client(NewCallers0, Worker, From),
-                gen_server:reply(From, {ok, Worker}),
-                NewState = State#state{
-                    busy = [Worker | Busy2],
-                    waiting = Waiting2,
-                    callers = NewCallers1
-                },
-                {noreply, NewState}
+                {empty, _} ->
+                    {noreply, State#state{conns = Conns2, callers = NewCallers0}};
+                {{value, From}, Waiting2} ->
+                    {ok, Worker} = couch_replicator_connection:acquire(Url),
+                    NewCallers1 = monitor_client(NewCallers0, Worker, From),
+                    gen_server:reply(From, {ok, Worker}),
+                    NewState = State#state{
+                        conns = [Worker | Conns2],
+                        waiting = Waiting2,
+                        callers = NewCallers1
+                    },
+                    {noreply, NewState}
             end
-        end;
-    Free2 ->
-        {noreply, State#state{free = Free2, callers = NewCallers0}}
     end;
 
 handle_info({'DOWN', Ref, process, _, _}, #state{callers = Callers} = State) ->
@@ -147,9 +134,8 @@
     {ok, State}.
 
 
-terminate(_Reason, State) ->
-    lists:foreach(fun ibrowse_http_client:stop/1, State#state.free),
-    lists:foreach(fun ibrowse_http_client:stop/1, State#state.busy).
+terminate(_Reason, _State) ->
+    ok.
 
 monitor_client(Callers, Worker, {ClientPid, _}) ->
     [{Worker, erlang:monitor(process, ClientPid)} | Callers].
@@ -167,22 +153,20 @@
     #state{waiting = Waiting, callers = Callers} = State,
     NewCallers0 = demonitor_client(Callers, Worker),
     case is_process_alive(Worker) andalso
-        lists:member(Worker, State#state.busy) of
+        lists:member(Worker, State#state.conns) of
     true ->
-        case queue:out(Waiting) of
+        Conns = case queue:out(Waiting) of
         {empty, Waiting2} ->
             NewCallers1 = NewCallers0,
-            Busy2 = State#state.busy -- [Worker],
-            Free2 = [Worker | State#state.free];
+            couch_replicator_connection:relinquish(Worker),
+            State#state.conns -- [Worker];
         {{value, From}, Waiting2} ->
             NewCallers1 = monitor_client(NewCallers0, Worker, From),
             gen_server:reply(From, {ok, Worker}),
-            Busy2 = State#state.busy,
-            Free2 = State#state.free
+            State#state.conns
         end,
         NewState = State#state{
-           busy = Busy2,
-           free = Free2,
+           conns = Conns,
            waiting = Waiting2,
            callers = NewCallers1
         },
diff --git a/src/couch_replicator_ids.erl b/src/couch_replicator_ids.erl
new file mode 100644
index 0000000..565ed9d
--- /dev/null
+++ b/src/couch_replicator_ids.erl
@@ -0,0 +1,129 @@
+% 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_replicator_ids).
+
+-export([replication_id/1, replication_id/2, convert/1]).
+
+-include_lib("couch/include/couch_db.hrl").
+-include("couch_replicator_api_wrap.hrl").
+-include("couch_replicator.hrl").
+
+-import(couch_util, [
+    get_value/2,
+    get_value/3
+]).
+
+% replication_id/1 and replication_id/2 will attempt to fetch
+% filter code for filtered replications. If fetching or parsing
+% of the remotely fetched filter code fails they throw:
+%  {filter_fetch_error, Error} exception.
+%
+
+replication_id(#rep{options = Options} = Rep) ->
+    BaseId = replication_id(Rep, ?REP_ID_VERSION),
+    {BaseId, maybe_append_options([continuous, create_target], Options)}.
+
+% Versioned clauses for generating replication IDs.
+% If a change is made to how replications are identified,
+% please add a new clause and increase ?REP_ID_VERSION.
+
+replication_id(#rep{user_ctx = UserCtx} = Rep, 3) ->
+    UUID = couch_server:get_uuid(),
+    Src = get_rep_endpoint(UserCtx, Rep#rep.source),
+    Tgt = get_rep_endpoint(UserCtx, Rep#rep.target),
+    maybe_append_filters([UUID, Src, Tgt], Rep);
+
+replication_id(#rep{user_ctx = UserCtx} = Rep, 2) ->
+    {ok, HostName} = inet:gethostname(),
+    Port = case (catch mochiweb_socket_server:get(couch_httpd, port)) of
+    P when is_number(P) ->
+        P;
+    _ ->
+        % On restart we might be called before the couch_httpd process is
+        % started.
+        % TODO: we might be under an SSL socket server only, or both under
+        % SSL and a non-SSL socket.
+        % ... mochiweb_socket_server:get(https, port)
+        list_to_integer(config:get("httpd", "port", "5984"))
+    end,
+    Src = get_rep_endpoint(UserCtx, Rep#rep.source),
+    Tgt = get_rep_endpoint(UserCtx, Rep#rep.target),
+    maybe_append_filters([HostName, Port, Src, Tgt], Rep);
+
+replication_id(#rep{user_ctx = UserCtx} = Rep, 1) ->
+    {ok, HostName} = inet:gethostname(),
+    Src = get_rep_endpoint(UserCtx, Rep#rep.source),
+    Tgt = get_rep_endpoint(UserCtx, Rep#rep.target),
+    maybe_append_filters([HostName, Src, Tgt], Rep).
+
+
+-spec convert([_] | binary() | {string(), string()}) -> {string(), string()}.
+convert(Id) when is_list(Id) ->
+    convert(?l2b(Id));
+
+convert(Id) when is_binary(Id) ->
+    lists:splitwith(fun(Char) -> Char =/= $+ end, ?b2l(Id));
+
+convert({BaseId, Ext} = Id) when is_list(BaseId), is_list(Ext) ->
+    Id.
+
+
+% Private functions
+
+maybe_append_filters(Base,
+        #rep{source = Source, user_ctx = UserCtx, options = Options}) ->
+    Base2 = Base ++
+        case couch_replicator_filters:parse(Options) of
+        {ok, nil} ->
+            [];
+        {ok, {view, Filter, QueryParams}} ->
+            [Filter, QueryParams];
+        {ok, {user, {Doc, Filter}, QueryParams}} ->
+            case couch_replicator_filters:fetch(Doc, Filter, Source, UserCtx) of
+                {ok, Code} ->
+                    [Code, QueryParams];
+                {error, Error} ->
+                    throw({filter_fetch_error, Error})
+            end;
+        {ok, {docids, DocIds}} ->
+            [DocIds];
+        {ok, {mango, Selector}} ->
+            [Selector];
+        {error, FilterParseError} ->
+            throw({error, FilterParseError})
+        end,
+    couch_util:to_hex(couch_crypto:hash(md5, term_to_binary(Base2))).
+
+
+maybe_append_options(Options, RepOptions) ->
+    lists:foldl(fun(Option, Acc) ->
+        Acc ++
+        case get_value(Option, RepOptions, false) of
+        true ->
+            "+" ++ atom_to_list(Option);
+        false ->
+            ""
+        end
+    end, [], Options).
+
+
+get_rep_endpoint(_UserCtx, #httpdb{url=Url, headers=Headers, oauth=OAuth}) ->
+    DefaultHeaders = (#httpdb{})#httpdb.headers,
+    case OAuth of
+    nil ->
+        {remote, Url, Headers -- DefaultHeaders};
+    #oauth{} ->
+        {remote, Url, Headers -- DefaultHeaders, OAuth}
+    end;
+get_rep_endpoint(UserCtx, <<DbName/binary>>) ->
+    {local, DbName, UserCtx}.
diff --git a/src/couch_replicator_js_functions.hrl b/src/couch_replicator_js_functions.hrl
index eba1973..dbad050 100644
--- a/src/couch_replicator_js_functions.hrl
+++ b/src/couch_replicator_js_functions.hrl
@@ -53,7 +53,7 @@
         var isReplicator = (userCtx.roles.indexOf('_replicator') >= 0);
         var isAdmin = (userCtx.roles.indexOf('_admin') >= 0);
 
-        if (newDoc._replication_state === 'error') {
+        if (newDoc._replication_state === 'failed') {
             // Skip validation in case when we update the document with the
             // failed state. In this case it might be malformed. However,
             // replicator will not pay attention to failed documents so this
@@ -61,12 +61,6 @@
             return;
         }
 
-        if (oldDoc && !newDoc._deleted && !isReplicator &&
-            (oldDoc._replication_state === 'triggered')) {
-            reportError('Only the replicator can edit replication documents ' +
-                'that are in the triggered state.');
-        }
-
         if (!newDoc._deleted) {
             validateEndpoint(newDoc.source, 'source');
             validateEndpoint(newDoc.target, 'target');
@@ -176,3 +170,25 @@
         }
     }
 ">>).
+
+
+-define(REP_DB_TERMINAL_STATE_VIEW_MAP_FUN, <<"
+    function(doc) {
+        state = doc._replication_state;
+        if (state === 'failed') {
+            source = doc.source;
+            target = doc.target;
+            start_time = doc._replication_start_time;
+            last_updated = doc._replication_state_time;
+            state_reason = doc._replication_state_reason;
+            emit('failed', [source, target, start_time, last_updated, state_reason]);
+        } else if (state === 'completed') {
+            source = doc.source;
+            target = doc.target;
+            start_time = doc._replication_start_time;
+            last_updated = doc._replication_state_time;
+            stats = doc._replication_stats;
+            emit('completed', [source, target, start_time, last_updated, stats]);
+        }
+    }
+">>).
diff --git a/src/couch_replicator_manager.erl b/src/couch_replicator_manager.erl
index 85dd428..6a79ce6 100644
--- a/src/couch_replicator_manager.erl
+++ b/src/couch_replicator_manager.erl
@@ -11,1033 +11,18 @@
 % the License.
 
 -module(couch_replicator_manager).
--behaviour(gen_server).
--vsn(3).
--behaviour(config_listener).
 
-% public API
--export([replication_started/1, replication_completed/2, replication_error/2]).
--export([continue/1, replication_usurped/2]).
+% TODO: This is a temporary proxy module to external calls (outside replicator) to other 
+% replicator modules. This is done to avoid juggling multiple repos during development.
 
+% NV: TODO: These functions were moved to couch_replicator_docs
+% but it is still called from fabric_doc_update. Keep it here for now
+% later, update fabric to call couch_replicator_docs instead
 -export([before_doc_update/2, after_doc_read/2]).
 
-% gen_server callbacks
--export([start_link/0, init/1, handle_call/3, handle_info/2, handle_cast/2]).
--export([code_change/3, terminate/2]).
 
-% changes callbacks
--export([changes_reader/3, changes_reader_cb/3]).
+before_doc_update(Doc, Db) ->
+    couch_replicator_docs:before_doc_update(Doc, Db).
 
-% config_listener callback
--export([handle_config_change/5, handle_config_terminate/3]).
-
--export([handle_db_event/3]).
-
-%% exported but private
--export([start_replication/2]).
-
--include_lib("couch/include/couch_db.hrl").
--include_lib("mem3/include/mem3.hrl").
--include("couch_replicator.hrl").
--include("couch_replicator_js_functions.hrl").
-
--define(DOC_TO_REP, couch_rep_doc_id_to_rep_id).
--define(REP_TO_STATE, couch_rep_id_to_rep_state).
--define(INITIAL_WAIT, 2.5). % seconds
--define(MAX_WAIT, 600).     % seconds
--define(AVG_DELAY_MSEC, 100).
--define(MAX_DELAY_MSEC, 60000).
--define(OWNER, <<"owner">>).
--define(REPLICATOR_DB, <<"_replicator">>).
-
--define(DB_TO_SEQ, db_to_seq).
--define(CTX, {user_ctx, #user_ctx{roles=[<<"_admin">>, <<"_replicator">>]}}).
-
--define(replace(L, K, V), lists:keystore(K, 1, L, {K, V})).
-
--define(RELISTEN_DELAY, 5000).
-
--record(rep_state, {
-    rep,
-    starting,
-    retries_left,
-    max_retries,
-    wait = ?INITIAL_WAIT
-}).
-
--import(couch_util, [
-    to_binary/1
-]).
-
--record(state, {
-    event_listener = nil,
-    scan_pid = nil,
-    rep_start_pids = [],
-    max_retries,
-    live = [],
-    epoch = nil
-}).
-
-start_link() ->
-    gen_server:start_link({local, ?MODULE}, ?MODULE, [], []).
-
-
-replication_started(#rep{id = {BaseId, _} = RepId}) ->
-    case rep_state(RepId) of
-    nil ->
-        ok;
-    #rep_state{rep = #rep{db_name = DbName, doc_id = DocId}} ->
-        update_rep_doc(DbName, DocId, [
-            {<<"_replication_state">>, <<"triggered">>},
-            {<<"_replication_state_reason">>, undefined},
-            {<<"_replication_id">>, ?l2b(BaseId)},
-            {<<"_replication_stats">>, undefined}]),
-        ok = gen_server:call(?MODULE, {rep_started, RepId}, infinity),
-        couch_log:notice("Document `~s` from `~s` triggered replication `~s`",
-            [DocId, DbName, pp_rep_id(RepId)])
-    end.
-
-
-replication_completed(#rep{id = RepId}, Stats) ->
-    case rep_state(RepId) of
-    nil ->
-        ok;
-    #rep_state{rep = #rep{db_name = DbName, doc_id = DocId}} ->
-        update_rep_doc(DbName, DocId, [
-            {<<"_replication_state">>, <<"completed">>},
-            {<<"_replication_state_reason">>, undefined},
-            {<<"_replication_stats">>, {Stats}}]),
-        ok = gen_server:call(?MODULE, {rep_complete, RepId}, infinity),
-        couch_log:notice("Replication `~s` finished (triggered by document `~s`"
-            " from `~s`)", [pp_rep_id(RepId), DocId, DbName])
-    end.
-
-
-replication_usurped(#rep{id = RepId}, By) ->
-    case rep_state(RepId) of
-    nil ->
-        ok;
-    #rep_state{rep = #rep{db_name = DbName, doc_id = DocId}} ->
-        ok = gen_server:call(?MODULE, {rep_complete, RepId}, infinity),
-        couch_log:notice("Replication `~s` usurped by ~s (triggered by document"
-            " `~s` from `~s`)", [pp_rep_id(RepId), By, DocId, DbName])
-    end.
-
-
-replication_error(#rep{id = {BaseId, _} = RepId}, Error) ->
-    case rep_state(RepId) of
-    nil ->
-        ok;
-    #rep_state{rep = #rep{db_name = DbName, doc_id = DocId}} ->
-        ok = timer:sleep(jitter(ets:info(?REP_TO_STATE, size))),
-        update_rep_doc(DbName, DocId, [
-            {<<"_replication_state">>, <<"error">>},
-            {<<"_replication_state_reason">>, to_binary(error_reason(Error))},
-            {<<"_replication_id">>, ?l2b(BaseId)}]),
-        ok = gen_server:call(?MODULE, {rep_error, RepId, Error}, infinity)
-    end.
-
-continue(#rep{doc_id = null}) ->
-    {true, no_owner};
-continue(#rep{id = RepId}) ->
-    Owner = gen_server:call(?MODULE, {owner, RepId}, infinity),
-    {node() == Owner, Owner}.
-
-
-handle_config_change("replicator", "max_replication_retry_count", V, _, S) ->
-    ok = gen_server:cast(?MODULE, {set_max_retries, retries_value(V)}),
-    {ok, S};
-handle_config_change(_, _, _, _, S) ->
-    {ok, S}.
-
-handle_config_terminate(_, stop, _) ->
-    ok;
-handle_config_terminate(_Server, _Reason, _State) ->
-    erlang:send_after(?RELISTEN_DELAY, whereis(?MODULE), restart_config_listener).
-
-init(_) ->
-    process_flag(trap_exit, true),
-    net_kernel:monitor_nodes(true),
-    Live = [node() | nodes()],
-    ?DOC_TO_REP = ets:new(?DOC_TO_REP, [named_table, set, public]),
-    ?REP_TO_STATE = ets:new(?REP_TO_STATE, [named_table, set, public]),
-    ?DB_TO_SEQ = ets:new(?DB_TO_SEQ, [named_table, set, public]),
-    Server = self(),
-    ok = config:listen_for_changes(?MODULE, nil),
-    Epoch = make_ref(),
-    ScanPid = spawn_link(fun() -> scan_all_dbs(Server) end),
-    % Automatically start node local changes feed loop
-    ensure_rep_db_exists(?REPLICATOR_DB),
-    Pid = start_changes_reader(?REPLICATOR_DB, 0, Epoch),
-    {ok, #state{
-        event_listener = start_event_listener(),
-        scan_pid = ScanPid,
-        max_retries = retries_value(
-            config:get("replicator", "max_replication_retry_count", "10")),
-        rep_start_pids = [{?REPLICATOR_DB, Pid}],
-        live = Live,
-        epoch = Epoch
-    }}.
-
-handle_call({owner, RepId}, _From, State) ->
-    case rep_state(RepId) of
-    nil ->
-        {reply, nonode, State};
-    #rep_state{rep = #rep{db_name = DbName, doc_id = DocId}} ->
-        {reply, owner(DbName, DocId, State#state.live), State}
-    end;
-
-handle_call({rep_db_update, DbName, {ChangeProps} = Change}, _From, State) ->
-    NewState = try
-        process_update(State, DbName, Change)
-    catch
-    _Tag:Error ->
-        {RepProps} = get_json_value(doc, ChangeProps),
-        DocId = get_json_value(<<"_id">>, RepProps),
-        rep_db_update_error(Error, DbName, DocId),
-        State
-    end,
-    {reply, ok, NewState};
-
-
-handle_call({rep_started, RepId}, _From, State) ->
-    case rep_state(RepId) of
-    nil ->
-        ok;
-    RepState ->
-        NewRepState = RepState#rep_state{
-            starting = false,
-            retries_left = State#state.max_retries,
-            max_retries = State#state.max_retries,
-            wait = ?INITIAL_WAIT
-        },
-        true = ets:insert(?REP_TO_STATE, {RepId, NewRepState})
-    end,
-    {reply, ok, State};
-
-handle_call({rep_complete, RepId}, _From, State) ->
-    true = ets:delete(?REP_TO_STATE, RepId),
-    {reply, ok, State};
-
-handle_call({rep_error, RepId, Error}, _From, State) ->
-    {reply, ok, replication_error(State, RepId, Error)};
-
-% Match changes epoch with the current epoch in the state.
-% New epoch ref is created on a full rescan. Change feeds have to
-% be replayed from the start to determine ownership in the new
-% cluster configuration and epoch is used to match & checkpoint
-% only changes from the current cluster configuration.
-handle_call({rep_db_checkpoint, DbName, EndSeq, Epoch}, _From,
-            #state{epoch = Epoch} = State) ->
-    Entry = case ets:lookup(?DB_TO_SEQ, DbName) of
-        [] ->
-            {DbName, EndSeq, false};
-        [{DbName, _OldSeq, Rescan}] ->
-            {DbName, EndSeq, Rescan}
-    end,
-    true = ets:insert(?DB_TO_SEQ, Entry),
-    {reply, ok, State};
-
-% Ignore checkpoints from previous epoch.
-handle_call({rep_db_checkpoint, _DbName, _EndSeq, _Epoch}, _From, State) ->
-    {reply, ok, State};
-
-handle_call(Msg, From, State) ->
-    couch_log:error("Replication manager received unexpected call ~p from ~p",
-        [Msg, From]),
-    {stop, {error, {unexpected_call, Msg}}, State}.
-
-handle_cast({resume_scan, DbName}, State) ->
-    Pids = State#state.rep_start_pids,
-    NewPids = case lists:keyfind(DbName, 1, Pids) of
-        {DbName, _Pid} ->
-            Entry = case ets:lookup(?DB_TO_SEQ, DbName) of
-                [] ->
-                    {DbName, 0, true};
-                [{DbName, EndSeq, _Rescan}] ->
-                    {DbName, EndSeq, true}
-            end,
-            true = ets:insert(?DB_TO_SEQ, Entry),
-            Pids;
-        false ->
-            Since = case ets:lookup(?DB_TO_SEQ, DbName) of
-                [] -> 0;
-                [{DbName, EndSeq, _Rescan}] -> EndSeq
-            end,
-            true = ets:insert(?DB_TO_SEQ, {DbName, Since, false}),
-            ensure_rep_ddoc_exists(DbName),
-            Pid = start_changes_reader(DbName, Since, State#state.epoch),
-            couch_log:debug("Scanning ~s from update_seq ~p", [DbName, Since]),
-            [{DbName, Pid} | Pids]
-    end,
-    {noreply, State#state{rep_start_pids = NewPids}};
-
-handle_cast({set_max_retries, MaxRetries}, State) ->
-    {noreply, State#state{max_retries = MaxRetries}};
-
-handle_cast(Msg, State) ->
-    couch_log:error("Replication manager received unexpected cast ~p", [Msg]),
-    {stop, {error, {unexpected_cast, Msg}}, State}.
-
-handle_info({nodeup, Node}, State) ->
-    couch_log:notice("Rescanning replicator dbs as ~s came up.", [Node]),
-    Live = lists:usort([Node | State#state.live]),
-    {noreply, rescan(State#state{live=Live})};
-
-handle_info({nodedown, Node}, State) ->
-    couch_log:notice("Rescanning replicator dbs ~s went down.", [Node]),
-    Live = State#state.live -- [Node],
-    {noreply, rescan(State#state{live=Live})};
-
-handle_info({'EXIT', From, normal}, #state{scan_pid = From} = State) ->
-    couch_log:debug("Background scan has completed.", []),
-    {noreply, State#state{scan_pid=nil}};
-
-handle_info({'EXIT', From, Reason}, #state{scan_pid = From} = State) ->
-    couch_log:error("Background scanner died. Reason: ~p", [Reason]),
-    {stop, {scanner_died, Reason}, State};
-
-handle_info({'EXIT', From, Reason}, #state{event_listener = From} = State) ->
-    couch_log:error("Database update notifier died. Reason: ~p", [Reason]),
-    {stop, {db_update_notifier_died, Reason}, State};
-
-handle_info({'EXIT', From, Reason}, #state{rep_start_pids = Pids} = State) ->
-    case lists:keytake(From, 2, Pids) of
-        {value, {DbName, From}, NewPids} ->
-            if Reason == normal -> ok; true ->
-                Fmt = "~s : Known replication or change feed pid ~w died :: ~w",
-                couch_log:error(Fmt, [?MODULE, From, Reason])
-            end,
-            NewState = State#state{rep_start_pids = NewPids},
-            case ets:lookup(?DB_TO_SEQ, DbName) of
-                [{DbName, _EndSeq, true}] ->
-                    handle_cast({resume_scan, DbName}, NewState);
-                _ ->
-                    {noreply, NewState}
-            end;
-        false when Reason == normal ->
-            {noreply, State};
-        false ->
-            Fmt = "~s : Unknown pid ~w died :: ~w",
-            couch_log:error(Fmt, [?MODULE, From, Reason]),
-            {stop, {unexpected_exit, From, Reason}, State}
-    end;
-
-handle_info({'DOWN', _Ref, _, _, _}, State) ->
-    % From a db monitor created by a replication process. Ignore.
-    {noreply, State};
-
-handle_info(shutdown, State) ->
-    {stop, shutdown, State};
-
-handle_info(restart_config_listener, State) ->
-    ok = config:listen_for_changes(?MODULE, nil),
-    {noreply, State};
-
-handle_info(Msg, State) ->
-    couch_log:error("Replication manager received unexpected message ~p", [Msg]),
-    {stop, {unexpected_msg, Msg}, State}.
-
-
-terminate(_Reason, State) ->
-    #state{
-        scan_pid = ScanPid,
-        rep_start_pids = StartPids,
-        event_listener = Listener
-    } = State,
-    stop_all_replications(),
-    lists:foreach(
-        fun({_Tag, Pid}) ->
-            catch unlink(Pid),
-            catch exit(Pid, stop)
-        end,
-        [{scanner, ScanPid} | StartPids]),
-    true = ets:delete(?REP_TO_STATE),
-    true = ets:delete(?DOC_TO_REP),
-    true = ets:delete(?DB_TO_SEQ),
-    couch_event:stop_listener(Listener).
-
-
-code_change(1, State, _Extra) ->
-    {ok, erlang:append_element(State, [node() | nodes()])};
-code_change(_OldVsn, State, _Extra) ->
-    {ok, State}.
-
-
-start_changes_reader(DbName, Since, Epoch) ->
-    spawn_link(?MODULE, changes_reader, [{self(), Epoch}, DbName, Since]).
-
-changes_reader({Server, Epoch}, DbName, Since) ->
-    UserCtx = #user_ctx{roles = [<<"_admin">>, <<"_replicator">>]},
-    DbOpenOptions = [{user_ctx, UserCtx}, sys_db],
-    {ok, Db} = couch_db:open_int(DbName, DbOpenOptions),
-    ChangesFeedFun = couch_changes:handle_db_changes(
-        #changes_args{
-            include_docs = true,
-            since = Since,
-            feed = "normal",
-            timeout = infinity
-        },
-        {json_req, null},
-        Db
-    ),
-    ChangesFeedFun({fun ?MODULE:changes_reader_cb/3, {Server, DbName, Epoch}}).
-
-changes_reader_cb({change, Change, _}, _, {Server, DbName, Epoch}) ->
-    case has_valid_rep_id(Change) of
-        true ->
-            Msg = {rep_db_update, DbName, Change},
-            ok = gen_server:call(Server, Msg, infinity);
-        false ->
-            ok
-    end,
-    {Server, DbName, Epoch};
-changes_reader_cb({stop, EndSeq}, _, {Server, DbName, Epoch}) ->
-    Msg = {rep_db_checkpoint, DbName, EndSeq, Epoch},
-    ok = gen_server:call(Server, Msg, infinity),
-    {Server, DbName, Epoch};
-changes_reader_cb(_, _, Acc) ->
-    Acc.
-
-has_valid_rep_id({Change}) ->
-    has_valid_rep_id(get_json_value(<<"id">>, Change));
-has_valid_rep_id(<<?DESIGN_DOC_PREFIX, _Rest/binary>>) ->
-    false;
-has_valid_rep_id(_Else) ->
-    true.
-
-
-start_event_listener() ->
-    {ok, Pid} = couch_event:link_listener(
-            ?MODULE, handle_db_event, self(), [all_dbs]
-        ),
-    Pid.
-
-
-handle_db_event(DbName, created, Server) ->
-    case is_replicator_db(DbName) of
-	true ->
-	    ensure_rep_ddoc_exists(DbName);
-	_ ->
-	    ok
-    end,
-    {ok, Server};
-handle_db_event(DbName, updated, Server) ->
-    case is_replicator_db(DbName) of
-        true ->
-	    Msg = {resume_scan, DbName},
-	    ok = gen_server:cast(Server, Msg);
-        _ ->
-            ok
-    end,
-    {ok, Server};
-handle_db_event(DbName, deleted, Server) ->
-    case is_replicator_db(DbName) of
-        true ->
-            clean_up_replications(DbName);
-        _ ->
-            ok
-    end,
-    {ok, Server};
-handle_db_event(_DbName, _Event, Server) ->
-    {ok, Server}.
-
-rescan(#state{scan_pid = nil} = State) ->
-    true = ets:delete_all_objects(?DB_TO_SEQ),
-    Server = self(),
-    Epoch = make_ref(),
-    NewScanPid = spawn_link(fun() -> scan_all_dbs(Server) end),
-    State#state{scan_pid = NewScanPid, epoch = Epoch};
-rescan(#state{scan_pid = ScanPid} = State) ->
-    unlink(ScanPid),
-    exit(ScanPid, exit),
-    rescan(State#state{scan_pid = nil}).
-
-process_update(State, DbName, {Change}) ->
-    {RepProps} = JsonRepDoc = get_json_value(doc, Change),
-    DocId = get_json_value(<<"_id">>, RepProps),
-    case {owner(DbName, DocId, State#state.live), get_json_value(deleted, Change, false)} of
-    {_, true} ->
-        rep_doc_deleted(DbName, DocId),
-        State;
-    {Owner, false} when Owner /= node() ->
-        couch_log:notice("Not starting '~s' from '~s' as owner is ~s.",
-            [DocId, DbName, Owner]),
-        State;
-    {_Owner, false} ->
-        couch_log:notice("Maybe starting '~s' from '~s' as I'm the owner", [DocId, DbName]),
-        case get_json_value(<<"_replication_state">>, RepProps) of
-        undefined ->
-            maybe_start_replication(State, DbName, DocId, JsonRepDoc);
-        <<"triggered">> ->
-            maybe_start_replication(State, DbName, DocId, JsonRepDoc);
-        <<"completed">> ->
-            replication_complete(DbName, DocId),
-            State;
-        <<"error">> ->
-            case ets:lookup(?DOC_TO_REP, {DbName, DocId}) of
-            [] ->
-                maybe_start_replication(State, DbName, DocId, JsonRepDoc);
-            _ ->
-                State
-            end
-        end
-    end.
-
-owner(<<"shards/", _/binary>> = DbName, DocId, Live) ->
-    Nodes = lists:sort([N || #shard{node=N} <- mem3:shards(mem3:dbname(DbName), DocId),
-			     lists:member(N, Live)]),
-    hd(mem3_util:rotate_list({DbName, DocId}, Nodes));
-owner(_DbName, _DocId, _Live) ->
-    node().
-
-rep_db_update_error(Error, DbName, DocId) ->
-    case Error of
-    {bad_rep_doc, Reason} ->
-        ok;
-    _ ->
-        Reason = to_binary(Error)
-    end,
-    couch_log:error("Replication manager, error processing document `~s`"
-        " from `~s`: ~s", [DocId, DbName, Reason]),
-    update_rep_doc(DbName, DocId, [{<<"_replication_state">>, <<"error">>},
-                           {<<"_replication_state_reason">>, Reason}]).
-
-
-rep_user_ctx({RepDoc}) ->
-    case get_json_value(<<"user_ctx">>, RepDoc) of
-    undefined ->
-        #user_ctx{};
-    {UserCtx} ->
-        #user_ctx{
-            name = get_json_value(<<"name">>, UserCtx, null),
-            roles = get_json_value(<<"roles">>, UserCtx, [])
-        }
-    end.
-
-
-maybe_start_replication(State, DbName, DocId, RepDoc) ->
-    #rep{id = {BaseId, _} = RepId} = Rep0 = parse_rep_doc(RepDoc),
-    Rep = Rep0#rep{db_name = DbName},
-    case rep_state(RepId) of
-    nil ->
-        RepState = #rep_state{
-            rep = Rep,
-            starting = true,
-            retries_left = State#state.max_retries,
-            max_retries = State#state.max_retries
-        },
-        true = ets:insert(?REP_TO_STATE, {RepId, RepState}),
-        true = ets:insert(?DOC_TO_REP, {{DbName, DocId}, RepId}),
-        couch_log:notice("Attempting to start replication `~s` (document `~s`"
-            " from `~s`).", [pp_rep_id(RepId), DocId, DbName]),
-        StartDelaySecs = erlang:max(0,
-            config:get_integer("replicator", "start_delay", 10)),
-        StartSplaySecs = erlang:max(1,
-            config:get_integer("replicator", "start_splay", 50)),
-        DelaySecs = StartDelaySecs + random:uniform(StartSplaySecs),
-        couch_log:notice("Delaying replication `~s` start by ~p seconds.",
-            [pp_rep_id(RepId), DelaySecs]),
-        Pid = spawn_link(?MODULE, start_replication, [Rep, DelaySecs]),
-        State#state{
-            rep_start_pids = [{rep_start, Pid} | State#state.rep_start_pids]
-        };
-    #rep_state{rep = #rep{doc_id = DocId}} ->
-        State;
-    #rep_state{starting = false, rep = #rep{db_name = DbName, doc_id = OtherDocId}} ->
-        couch_log:notice("The replication specified by the document `~s` from"
-            " `~s` was already triggered by the document `~s`",
-            [DocId, DbName, OtherDocId]),
-        maybe_tag_rep_doc(DbName, DocId, RepDoc, ?l2b(BaseId)),
-        State;
-    #rep_state{starting = true, rep = #rep{db_name = DbName, doc_id = OtherDocId}} ->
-        couch_log:notice("The replication specified by the document `~s` from"
-            " `~s` is already being triggered by the document `~s`",
-            [DocId, DbName, OtherDocId]),
-        maybe_tag_rep_doc(DbName, DocId, RepDoc, ?l2b(BaseId)),
-        State
-    end.
-
-
-parse_rep_doc(RepDoc) ->
-    {ok, Rep} = try
-        couch_replicator_utils:parse_rep_doc(RepDoc, rep_user_ctx(RepDoc))
-    catch
-    throw:{error, Reason} ->
-        throw({bad_rep_doc, Reason});
-    Tag:Err ->
-        throw({bad_rep_doc, to_binary({Tag, Err})})
-    end,
-    Rep.
-
-
-maybe_tag_rep_doc(DbName, DocId, {RepProps}, RepId) ->
-    case get_json_value(<<"_replication_id">>, RepProps) of
-    RepId ->
-        ok;
-    _ ->
-        update_rep_doc(DbName, DocId, [{<<"_replication_id">>, RepId}])
-    end.
-
-start_replication(Rep, Wait) ->
-    ok = timer:sleep(Wait * 1000),
-    case (catch couch_replicator:async_replicate(Rep)) of
-    {ok, _} ->
-        ok;
-    Error ->
-        replication_error(Rep, Error)
-    end.
-
-replication_complete(DbName, DocId) ->
-    case ets:lookup(?DOC_TO_REP, {DbName, DocId}) of
-    [{{DbName, DocId}, {BaseId, Ext} = RepId}] ->
-        case rep_state(RepId) of
-        nil ->
-            % Prior to OTP R14B02, temporary child specs remain in
-            % in the supervisor after a worker finishes - remove them.
-            % We want to be able to start the same replication but with
-            % eventually different values for parameters that don't
-            % contribute to its ID calculation.
-            case erlang:system_info(otp_release) < "R14B02" of
-            true ->
-                spawn(fun() ->
-                    _ = supervisor:delete_child(couch_replicator_job_sup, BaseId ++ Ext)
-                end);
-            false ->
-                ok
-            end;
-        #rep_state{} ->
-            ok
-        end,
-        true = ets:delete(?DOC_TO_REP, {DbName, DocId});
-    _ ->
-        ok
-    end.
-
-
-rep_doc_deleted(DbName, DocId) ->
-    case ets:lookup(?DOC_TO_REP, {DbName, DocId}) of
-    [{{DbName, DocId}, RepId}] ->
-        couch_replicator:cancel_replication(RepId),
-        true = ets:delete(?REP_TO_STATE, RepId),
-        true = ets:delete(?DOC_TO_REP, {DbName, DocId}),
-        couch_log:notice("Stopped replication `~s` because replication document"
-            " `~s` from `~s` was deleted", [pp_rep_id(RepId), DocId, DbName]);
-    [] ->
-        ok
-    end.
-
-
-replication_error(State, RepId, Error) ->
-    case rep_state(RepId) of
-    nil ->
-        State;
-    RepState ->
-        maybe_retry_replication(RepState, Error, State)
-    end.
-
-maybe_retry_replication(#rep_state{retries_left = 0} = RepState, Error, State) ->
-    #rep_state{
-        rep = #rep{id = RepId, doc_id = DocId, db_name = DbName},
-        max_retries = MaxRetries
-    } = RepState,
-    couch_replicator:cancel_replication(RepId),
-    true = ets:delete(?REP_TO_STATE, RepId),
-    true = ets:delete(?DOC_TO_REP, {DbName, DocId}),
-    couch_log:error("Error in replication `~s` (triggered by document `~s` from"
-        " `~s` ): ~s~nReached maximum retry attempts (~p).", [pp_rep_id(RepId),
-        DocId, DbName, to_binary(error_reason(Error)), MaxRetries]),
-    State;
-
-maybe_retry_replication(RepState, Error, State) ->
-    #rep_state{
-        rep = #rep{id = RepId, doc_id = DocId, db_name = DbName} = Rep
-    } = RepState,
-    #rep_state{wait = Wait} = NewRepState = state_after_error(RepState),
-    true = ets:insert(?REP_TO_STATE, {RepId, NewRepState}),
-    couch_log:error("Error in replication `~s` (triggered by document `~s` from"
-        " `~s` ): ~s~nRestarting replication in ~p seconds.", [pp_rep_id(RepId),
-        DocId, DbName, to_binary(error_reason(Error)), Wait]),
-    Pid = spawn_link(?MODULE, start_replication, [Rep, Wait]),
-    State#state{
-        rep_start_pids = [{rep_start, Pid} | State#state.rep_start_pids]
-    }.
-
-
-stop_all_replications() ->
-    couch_log:notice("Stopping all ongoing replications", []),
-    ets:foldl(
-        fun({_, RepId}, _) ->
-            couch_replicator:cancel_replication(RepId)
-        end,
-        ok, ?DOC_TO_REP),
-    true = ets:delete_all_objects(?REP_TO_STATE),
-    true = ets:delete_all_objects(?DOC_TO_REP),
-    true = ets:delete_all_objects(?DB_TO_SEQ).
-
-clean_up_replications(DbName) ->
-    ets:foldl(
-        fun({{Name, DocId}, RepId}, _) when Name =:= DbName ->
-            couch_replicator:cancel_replication(RepId),
-            ets:delete(?DOC_TO_REP,{Name, DocId}),
-            ets:delete(?REP_TO_STATE, RepId);
-           ({_,_}, _) ->
-            ok
-        end,
-        ok, ?DOC_TO_REP),
-    ets:delete(?DB_TO_SEQ,DbName).
-
-
-update_rep_doc(RepDbName, RepDocId, KVs) ->
-    update_rep_doc(RepDbName, RepDocId, KVs, 1).
-
-update_rep_doc(RepDbName, RepDocId, KVs, Wait) when is_binary(RepDocId) ->
-    try
-        case open_rep_doc(RepDbName, RepDocId) of
-            {ok, LastRepDoc} ->
-                update_rep_doc(RepDbName, LastRepDoc, KVs, Wait * 2);
-            _ ->
-                ok
-        end
-    catch
-        throw:conflict ->
-            Msg = "Conflict when updating replication document `~s`. Retrying.",
-            couch_log:error(Msg, [RepDocId]),
-            ok = timer:sleep(random:uniform(erlang:min(128, Wait)) * 100),
-            update_rep_doc(RepDbName, RepDocId, KVs, Wait * 2)
-    end;
-update_rep_doc(RepDbName, #doc{body = {RepDocBody}} = RepDoc, KVs, _Try) ->
-    NewRepDocBody = lists:foldl(
-        fun({K, undefined}, Body) ->
-                lists:keydelete(K, 1, Body);
-           ({<<"_replication_state">> = K, State} = KV, Body) ->
-                case get_json_value(K, Body) of
-                State ->
-                    Body;
-                _ ->
-                    Body1 = lists:keystore(K, 1, Body, KV),
-                    lists:keystore(
-                        <<"_replication_state_time">>, 1, Body1,
-                        {<<"_replication_state_time">>, timestamp()})
-                end;
-            ({K, _V} = KV, Body) ->
-                lists:keystore(K, 1, Body, KV)
-        end,
-        RepDocBody, KVs),
-    case NewRepDocBody of
-    RepDocBody ->
-        ok;
-    _ ->
-        % Might not succeed - when the replication doc is deleted right
-        % before this update (not an error, ignore).
-        save_rep_doc(RepDbName, RepDoc#doc{body = {NewRepDocBody}})
-    end.
-
-open_rep_doc(DbName, DocId) ->
-    case couch_db:open_int(DbName, [?CTX, sys_db]) of
-        {ok, Db} ->
-            try
-                couch_db:open_doc(Db, DocId, [ejson_body])
-            after
-                couch_db:close(Db)
-            end;
-        Else ->
-            Else
-    end.
-
-save_rep_doc(DbName, Doc) ->
-    {ok, Db} = couch_db:open_int(DbName, [?CTX, sys_db]),
-    try
-        couch_db:update_doc(Db, Doc, [])
-    after
-        couch_db:close(Db)
-    end.
-
-% RFC3339 timestamps.
-% Note: doesn't include the time seconds fraction (RFC3339 says it's optional).
-timestamp() ->
-    {{Year, Month, Day}, {Hour, Min, Sec}} = calendar:now_to_local_time(now()),
-    UTime = erlang:universaltime(),
-    LocalTime = calendar:universal_time_to_local_time(UTime),
-    DiffSecs = calendar:datetime_to_gregorian_seconds(LocalTime) -
-        calendar:datetime_to_gregorian_seconds(UTime),
-    zone(DiffSecs div 3600, (DiffSecs rem 3600) div 60),
-    iolist_to_binary(
-        io_lib:format("~4..0w-~2..0w-~2..0wT~2..0w:~2..0w:~2..0w~s",
-            [Year, Month, Day, Hour, Min, Sec,
-                zone(DiffSecs div 3600, (DiffSecs rem 3600) div 60)])).
-
-zone(Hr, Min) when Hr >= 0, Min >= 0 ->
-    io_lib:format("+~2..0w:~2..0w", [Hr, Min]);
-zone(Hr, Min) ->
-    io_lib:format("-~2..0w:~2..0w", [abs(Hr), abs(Min)]).
-
-
-ensure_rep_db_exists(<<"shards/", _/binary>>=DbName) ->
-    ensure_rep_ddoc_exists(DbName),
-    ok;
-ensure_rep_db_exists(DbName) ->
-    Db = case couch_db:open_int(DbName, [?CTX, sys_db, nologifmissing]) of
-        {ok, Db0} ->
-            Db0;
-        _Error ->
-            {ok, Db0} = couch_db:create(DbName, [?CTX, sys_db]),
-            Db0
-    end,
-    ensure_rep_ddoc_exists(DbName),
-    {ok, Db}.
-
-ensure_rep_ddoc_exists(RepDb) ->
-    DDocId = <<"_design/_replicator">>,
-    case mem3:belongs(RepDb, DDocId) of
-	true ->
-	    ensure_rep_ddoc_exists(RepDb, DDocId);
-	false ->
-	    ok
-    end.
-
-ensure_rep_ddoc_exists(RepDb, DDocId) ->
-    case open_rep_doc(RepDb, DDocId) of
-        {not_found, no_db_file} ->
-            %% database was deleted.
-            ok;
-        {not_found, _Reason} ->
-            {ok, DDoc} = replication_design_doc(DDocId),
-            couch_log:notice("creating replicator ddoc", []),
-            {ok, _Rev} = save_rep_doc(RepDb, DDoc);
-        {ok, Doc} ->
-            {Props} = couch_doc:to_json_obj(Doc, []),
-            case couch_util:get_value(<<"validate_doc_update">>, Props, []) of
-                ?REP_DB_DOC_VALIDATE_FUN ->
-                    ok;
-                _ ->
-                    Props1 = lists:keyreplace(<<"validate_doc_update">>, 1, Props,
-                         {<<"validate_doc_update">>,
-                        ?REP_DB_DOC_VALIDATE_FUN}),
-                    DDoc = couch_doc:from_json_obj({Props1}),
-                    couch_log:notice("updating replicator ddoc", []),
-                    try
-                        {ok, _} = save_rep_doc(RepDb, DDoc)
-                    catch
-                        throw:conflict ->
-                            %% ignore, we'll retry next time
-                            ok
-                    end
-            end
-    end,
-    ok.
-
-replication_design_doc(DDocId) ->
-    DocProps = [
-        {<<"_id">>, DDocId},
-        {<<"language">>, <<"javascript">>},
-        {<<"validate_doc_update">>, ?REP_DB_DOC_VALIDATE_FUN}
-   ],
-   {ok, couch_doc:from_json_obj({DocProps})}.
-
-
-% pretty-print replication id
-pp_rep_id(#rep{id = RepId}) ->
-    pp_rep_id(RepId);
-pp_rep_id({Base, Extension}) ->
-    Base ++ Extension.
-
-
-rep_state(RepId) ->
-    case ets:lookup(?REP_TO_STATE, RepId) of
-    [{RepId, RepState}] ->
-        RepState;
-    [] ->
-        nil
-    end.
-
-
-error_reason({error, {Error, Reason}})
-  when is_atom(Error), is_binary(Reason) ->
-    io_lib:format("~s: ~s", [Error, Reason]);
-error_reason({error, Reason}) ->
-    Reason;
-error_reason(Reason) ->
-    Reason.
-
-
-retries_value("infinity") ->
-    infinity;
-retries_value(Value) ->
-    list_to_integer(Value).
-
-
-state_after_error(#rep_state{retries_left = Left, wait = Wait} = State) ->
-    Wait2 = erlang:min(trunc(Wait * 2), ?MAX_WAIT),
-    case Left of
-    infinity ->
-        State#rep_state{wait = Wait2};
-    _ ->
-        State#rep_state{retries_left = Left - 1, wait = Wait2}
-    end.
-
-
-before_doc_update(#doc{id = <<?DESIGN_DOC_PREFIX, _/binary>>} = Doc, _Db) ->
-    Doc;
-before_doc_update(#doc{body = {Body}} = Doc, #db{user_ctx=UserCtx} = Db) ->
-    #user_ctx{roles = Roles, name = Name} = UserCtx,
-    case lists:member(<<"_replicator">>, Roles) of
-    true ->
-        Doc;
-    false ->
-        case couch_util:get_value(?OWNER, Body) of
-        undefined ->
-            Doc#doc{body = {?replace(Body, ?OWNER, Name)}};
-        Name ->
-            Doc;
-        Other ->
-            case (catch couch_db:check_is_admin(Db)) of
-            ok when Other =:= null ->
-                Doc#doc{body = {?replace(Body, ?OWNER, Name)}};
-            ok ->
-                Doc;
-            _ ->
-                throw({forbidden, <<"Can't update replication documents",
-                    " from other users.">>})
-            end
-        end
-    end.
-
-
-after_doc_read(#doc{id = <<?DESIGN_DOC_PREFIX, _/binary>>} = Doc, _Db) ->
-    Doc;
-after_doc_read(#doc{body = {Body}} = Doc, #db{user_ctx=UserCtx} = Db) ->
-    #user_ctx{name = Name} = UserCtx,
-    case (catch couch_db:check_is_admin(Db)) of
-    ok ->
-        Doc;
-    _ ->
-        case couch_util:get_value(?OWNER, Body) of
-        Name ->
-            Doc;
-        _Other ->
-            Source = strip_credentials(couch_util:get_value(<<"source">>,
-Body)),
-            Target = strip_credentials(couch_util:get_value(<<"target">>,
-Body)),
-            NewBody0 = ?replace(Body, <<"source">>, Source),
-            NewBody = ?replace(NewBody0, <<"target">>, Target),
-            #doc{revs = {Pos, [_ | Revs]}} = Doc,
-            NewDoc = Doc#doc{body = {NewBody}, revs = {Pos - 1, Revs}},
-            NewRevId = couch_db:new_revid(NewDoc),
-            NewDoc#doc{revs = {Pos, [NewRevId | Revs]}}
-        end
-    end.
-
-
-strip_credentials(undefined) ->
-    undefined;
-strip_credentials(Url) when is_binary(Url) ->
-    re:replace(Url,
-        "http(s)?://(?:[^:]+):[^@]+@(.*)$",
-        "http\\1://\\2",
-        [{return, binary}]);
-strip_credentials({Props}) ->
-    {lists:keydelete(<<"oauth">>, 1, Props)}.
-
-scan_all_dbs(Server) when is_pid(Server) ->
-    {ok, Db} = mem3_util:ensure_exists(
-        config:get("mem3", "shards_db", "_dbs")),
-    ChangesFun = couch_changes:handle_changes(#changes_args{}, nil, Db, nil),
-    ChangesFun(fun({change, {Change}, _}, _) ->
-        DbName = couch_util:get_value(<<"id">>, Change),
-        case DbName of <<"_design/", _/binary>> -> ok; _Else ->
-            case couch_replicator_utils:is_deleted(Change) of
-            true ->
-                ok;
-            false ->
-                try
-                    [gen_server:cast(Server, {resume_scan, ShardName})
-                        || ShardName <- replicator_shards(DbName)]
-                catch error:database_does_not_exist ->
-                    ok
-                end
-            end
-        end;
-        (_, _) -> ok
-    end),
-    couch_db:close(Db).
-
-
-replicator_shards(DbName) ->
-    case is_replicator_db(DbName) of
-    false ->
-        [];
-    true ->
-        [ShardName || #shard{name = ShardName} <- mem3:local_shards(DbName)]
-    end.
-
-
-% calculate random delay proportional to the number of replications
-% on current node, in order to prevent a stampede:
-%   - when a source with multiple replication targets fails
-%   - when we restart couch_replication_manager
-jitter(N) ->
-    Range = min(2 * N * ?AVG_DELAY_MSEC, ?MAX_DELAY_MSEC),
-    random:uniform(Range).
-
-is_replicator_db(DbName) ->
-    ?REPLICATOR_DB =:= couch_db:dbname_suffix(DbName).
-
-get_json_value(Key, Props) ->
-    get_json_value(Key, Props, undefined).
-
-get_json_value(Key, Props, Default) when is_atom(Key) ->
-    Ref = make_ref(),
-    case couch_util:get_value(Key, Props, Ref) of
-        Ref ->
-            couch_util:get_value(?l2b(atom_to_list(Key)), Props, Default);
-        Else ->
-            Else
-    end;
-get_json_value(Key, Props, Default) when is_binary(Key) ->
-    Ref = make_ref(),
-    case couch_util:get_value(Key, Props, Ref) of
-        Ref ->
-            couch_util:get_value(list_to_atom(?b2l(Key)), Props, Default);
-        Else ->
-            Else
-    end.
-
-
--ifdef(TEST).
-
--include_lib("couch/include/couch_eunit.hrl").
-
-replicator_shards_test_() ->
-{
-      foreach,
-      fun() -> test_util:start_couch([mem3, fabric]) end,
-      fun(Ctx) -> test_util:stop_couch(Ctx) end,
-      [
-          t_pass_replicator_shard(),
-          t_fail_non_replicator_shard()
-     ]
-}.
-
-
-t_pass_replicator_shard() ->
-    ?_test(begin
-        DbName0 = ?tempdb(),
-        DbName = <<DbName0/binary, "/_replicator">>,
-        ok = fabric:create_db(DbName, [?CTX]),
-        ?assertEqual(8, length(replicator_shards(DbName))),
-        fabric:delete_db(DbName, [?CTX])
-    end).
-
-
-t_fail_non_replicator_shard() ->
-    ?_test(begin
-        DbName = ?tempdb(),
-        ok = fabric:create_db(DbName, [?CTX]),
-        ?assertEqual([], replicator_shards(DbName)),
-        fabric:delete_db(DbName, [?CTX])
-    end).
-
-
--endif.
+after_doc_read(Doc, Db) ->
+    couch_replicator_docs:after_doc_read(Doc, Db).
diff --git a/src/couch_replicator_rate_limiter.erl b/src/couch_replicator_rate_limiter.erl
new file mode 100644
index 0000000..f4ae8d2
--- /dev/null
+++ b/src/couch_replicator_rate_limiter.erl
@@ -0,0 +1,274 @@
+% 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.
+
+
+% This module implements rate limiting based on a variation the additive
+% increase / multiplicative decrease feedback control algorithm.
+%
+%  https://en.wikipedia.org/wiki/Additive_increase/multiplicative_decrease
+%
+% This is an adaptive algorithm which converges on available channel
+% capacity where each participant (client) doesn't a priori know the
+% capacity, and participants don't communicate or know about each other (so they
+% don't coordinate to divide the capacity among themselves).
+%
+% The algorithm referenced above estimates a rate, whereas the implemented
+% algorithm uses an interval (in milliseconds). It preserves the original
+% semantics, that is the failure part is multplicative and the success part is
+% additive. The relationship between rate and interval is: rate = 1000 / interval.
+%
+% There are two main API functions:
+%
+%   success(Key) -> IntervalInMilliseconds
+%   failure(Key) -> IntervalInMilliseconds
+%
+% Key is any term, typically something like {Method, Url}. The result from the
+% function is the current period value. Caller then might decide to sleep for
+% that amount of time before or after each request.
+
+
+-module(couch_replicator_rate_limiter).
+-behaviour(gen_server).
+
+
+% public API
+
+-export([start_link/0]).
+-export([interval/1, max_interval/0, failure/1, success/1]).
+
+
+% gen_server callbacks
+
+-export([init/1, handle_call/3, handle_info/2, handle_cast/2,
+         code_change/3, terminate/2]).
+
+
+% Types
+
+-type key() :: any().
+-type interval() :: non_neg_integer().
+-type msec() :: non_neg_integer().
+
+
+% Definitions
+
+-define(SHARDS_N, 16).
+
+% Main parameters of the algorithm. The factor is the multiplicative part and
+% base interval is the additive.
+-define(BASE_INTERVAL, 20).
+-define(BACKOFF_FACTOR, 1.2).
+
+% If estimated period exceeds a limit, it is clipped to this value. This
+% defines a practical limit of this algorithm. This is driven by real world
+% concerns such as having a connection which sleeps for too long and ends up
+% with socket timeout errors, or replication jobs which occupy a scheduler
+% slot without making any progress.
+-define(MAX_INTERVAL, 25000).
+
+% Specify when (threshold) and how much (factor) to decay the estimated period.
+% If there is a long pause between consecutive updates, the estimated period
+% would become less accurate as more time passes. In such case choose to
+% optimistically decay the estimated value. That is assume there a certain
+% rate of successful requests happened. (For reference, TCP congestion algorithm
+% also handles a variation of this in RFC 5681 under "Restarting Idle Connections"
+% section).
+-define(TIME_DECAY_FACTOR, 2).
+-define(TIME_DECAY_THRESHOLD, 1000).
+
+% Limit the rate of updates applied. This controls the rate of change of the
+% estimated value. In colloquial terms it defines how "twitchy" the algorithm
+% is. Or, another way to look at it, this is as a poor version of a low pass
+% filter. (Some alternative TCP congestion control algorithms, like Westwood+
+% use something similar to solve the ACK compression problem).
+-define(SENSITIVITY_TIME_WINDOW, 80).
+
+
+-record(state, {timer}).
+-record(rec, {id, backoff, ts}).
+
+
+-spec start_link() -> {ok, pid()} | ignore | {error, term()}.
+start_link() ->
+    gen_server:start_link({local, ?MODULE}, ?MODULE, [], []).
+
+
+-spec interval(key()) -> interval().
+interval(Key) ->
+    {Interval, _Timestamp} = interval_and_timestamp(Key),
+    Interval.
+
+
+-spec max_interval() -> interval().
+max_interval() ->
+    ?MAX_INTERVAL.
+
+
+-spec failure(key()) -> interval().
+failure(Key) ->
+    {Interval, Timestamp} = interval_and_timestamp(Key),
+    update_failure(Key, Interval, Timestamp, now_msec()).
+
+
+-spec success(key()) -> interval().
+success(Key) ->
+    {Interval, Timestamp} = interval_and_timestamp(Key),
+    update_success(Key, Interval, Timestamp, now_msec()).
+
+
+% gen_server callbacks
+
+init([]) ->
+    Opts = [named_table, public, {keypos, #rec.id}, {read_concurrency, true}],
+    [ets:new(list_to_atom(TableName), Opts) || TableName <- table_names()],
+    {ok, #state{timer = new_timer()}}.
+
+
+terminate(_Reason, _State) ->
+    ok.
+
+
+handle_call(_Msg, _From, State) ->
+    {reply, invalid, State}.
+
+
+handle_cast(_, State) ->
+    {noreply, State}.
+
+
+handle_info(cleanup, #state{timer = Timer}) ->
+    timer:cancel(Timer),
+    TIds = [list_to_existing_atom(TableName) || TableName <- table_names()],
+    [cleanup_table(TId, now_msec() - ?MAX_INTERVAL) || TId <- TIds],
+    {noreply, #state{timer = new_timer()}}.
+
+
+code_change(_OldVsn, State, _Extra) ->
+    {ok, State}.
+
+
+% Private functions
+
+-spec update_success(any(), interval(), msec(), msec()) -> interval().
+update_success(_Key, _Interval, _Timestamp = 0, _Now) ->
+    0;  % No ets entry. Keep it that way and don't insert a new one.
+
+update_success(_Key, Interval, Timestamp, Now)
+    when Now - Timestamp =< ?SENSITIVITY_TIME_WINDOW ->
+    Interval;  % Ignore too frequent updates.
+
+update_success(Key, Interval, Timestamp, Now) ->
+    DecayedInterval = time_decay(Now - Timestamp, Interval),
+    AdditiveFactor = additive_factor(DecayedInterval),
+    NewInterval = DecayedInterval - AdditiveFactor,
+    if
+        NewInterval =< 0 ->
+            ets:delete(term_to_table(Key), Key),
+            0;
+        NewInterval =< ?BASE_INTERVAL ->
+            insert(Key, ?BASE_INTERVAL, Now);
+        NewInterval > ?BASE_INTERVAL ->
+            insert(Key, NewInterval, Now)
+    end.
+
+
+-spec update_failure(any(), interval(), msec(), msec()) -> interval().
+update_failure(_Key, Interval, Timestamp, Now)
+    when Now - Timestamp =< ?SENSITIVITY_TIME_WINDOW ->
+    Interval;  % Ignore too frequent updates.
+
+update_failure(Key, Interval, _Timestamp, Now) ->
+    Interval1 = erlang:max(Interval, ?BASE_INTERVAL),
+    Interval2 = round(Interval1 * ?BACKOFF_FACTOR),
+    Interval3 = erlang:min(Interval2, ?MAX_INTERVAL),
+    insert(Key, Interval3, Now).
+
+
+-spec insert(any(), interval(), msec()) -> interval().
+insert(Key, Interval, Timestamp) ->
+    Entry = #rec{id = Key, backoff = Interval, ts = Timestamp},
+    ets:insert(term_to_table(Key), Entry),
+    Interval.
+
+
+-spec interval_and_timestamp(key()) -> {interval(), msec()}.
+interval_and_timestamp(Key) ->
+    case ets:lookup(term_to_table(Key), Key) of
+        [] ->
+            {0, 0};
+        [#rec{backoff = Interval, ts = Timestamp}] ->
+            {Interval, Timestamp}
+    end.
+
+
+-spec time_decay(msec(), interval()) -> interval().
+time_decay(Dt, Interval) when Dt > ?TIME_DECAY_THRESHOLD ->
+    DecayedInterval = Interval - ?TIME_DECAY_FACTOR * Dt,
+    erlang:max(round(DecayedInterval), 0);
+
+time_decay(_Dt, Interval) ->
+    Interval.
+
+
+% Calculate additive factor. Ideally it would be a constant but in this case
+% it is a step function to help handle larger values as they are approaching
+% the backoff limit. Large success values closer to the limit add some
+% pressure against the limit, which is useful, as at the backoff limit the
+% whole replication job is killed which can be costly in time and temporary work
+% lost by those jobs.
+-spec additive_factor(interval()) -> interval().
+additive_factor(Interval) when Interval > 10000 ->
+    ?BASE_INTERVAL * 50;
+
+ additive_factor(Interval) when Interval > 1000 ->
+    ?BASE_INTERVAL * 5;
+
+additive_factor(Interval) when Interval > 100 ->
+    ?BASE_INTERVAL * 2;
+
+additive_factor(_Interval) ->
+    ?BASE_INTERVAL.
+
+
+-spec table_name(non_neg_integer()) -> string().
+table_name(Id) when is_integer(Id), Id >= 0 andalso Id < ?SHARDS_N ->
+    atom_to_list(?MODULE) ++ "_" ++ integer_to_list(Id).
+
+
+-spec table_names() -> [string()].
+table_names() ->
+    [table_name(N) || N <- lists:seq(0, ?SHARDS_N - 1)].
+
+
+-spec term_to_table(any()) -> atom().
+term_to_table(Term) ->
+    PHash = erlang:phash2(Term),
+    list_to_existing_atom(table_name(PHash rem ?SHARDS_N)).
+
+
+-spec new_timer() -> timer:tref().
+new_timer() ->
+    {ok, Timer} = timer:send_after(?MAX_INTERVAL * 2, cleanup),
+    Timer.
+
+
+-spec now_msec() -> msec().
+now_msec() ->
+    {Mega, Sec, Micro} = os:timestamp(),
+    ((Mega * 1000000) + Sec) * 1000 + round(Micro / 1000).
+
+
+-spec cleanup_table(atom(), msec()) -> non_neg_integer().
+cleanup_table(Tid, LimitMSec) ->
+    Head = #rec{ts = '$1', _ = '_'},
+    Guard = {'<', '$1', LimitMSec},
+    ets:select_delete(Tid, [{Head, [Guard], [true]}]).
diff --git a/src/couch_replicator_scheduler.erl b/src/couch_replicator_scheduler.erl
new file mode 100644
index 0000000..a741a9f
--- /dev/null
+++ b/src/couch_replicator_scheduler.erl
@@ -0,0 +1,1380 @@
+% 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_replicator_scheduler).
+-behaviour(gen_server).
+-behaviour(config_listener).
+-vsn(1).
+
+-include("couch_replicator_scheduler.hrl").
+-include("couch_replicator.hrl").
+-include("couch_replicator_api_wrap.hrl").
+-include_lib("couch/include/couch_db.hrl").
+
+%% public api
+-export([start_link/0, add_job/1, remove_job/1, reschedule/0]).
+-export([rep_state/1, find_jobs_by_dbname/1, find_jobs_by_doc/2]).
+-export([job_summary/2, health_threshold/0]).
+-export([jobs/0, job/1]).
+
+%% gen_server callbacks
+-export([init/1, terminate/2, code_change/3]).
+-export([handle_call/3, handle_cast/2, handle_info/2]).
+-export([format_status/2]).
+
+%% config_listener callback
+-export([handle_config_change/5, handle_config_terminate/3]).
+
+%% types
+-type event_type() :: added | started | stopped | {crashed, any()}.
+-type event() :: {Type:: event_type(), When :: erlang:timestamp()}.
+-type history() :: nonempty_list(event()).
+
+%% definitions
+-define(MAX_BACKOFF_EXPONENT, 10).
+-define(BACKOFF_INTERVAL_MICROS, 30 * 1000 * 1000).
+-define(DEFAULT_HEALTH_THRESHOLD_SEC, 2 * 60).
+-define(RELISTEN_DELAY, 5000).
+
+-define(DEFAULT_MAX_JOBS, 500).
+-define(DEFAULT_MAX_CHURN, 20).
+-define(DEFAULT_MAX_HISTORY, 20).
+-define(DEFAULT_SCHEDULER_INTERVAL, 60000).
+-record(state, {interval, timer, max_jobs, max_churn, max_history}).
+-record(job, {
+          id :: job_id() | '$1' | '_',
+          rep :: #rep{} | '_',
+          pid :: undefined | pid() | '$1' | '_',
+          monitor :: undefined | reference() | '_',
+          history :: history() | '_'}).
+
+-record(stats_acc, {
+          now  :: erlang:timestamp(),
+          pending_t = 0 :: non_neg_integer(),
+          running_t = 0 :: non_neg_integer(),
+          crashed_t = 0 :: non_neg_integer(),
+          pending_n = 0 :: non_neg_integer(),
+          running_n = 0 :: non_neg_integer(),
+          crashed_n = 0 :: non_neg_integer()}).
+
+
+%% public functions
+
+-spec start_link() -> {ok, pid()} | ignore | {error, term()}.
+start_link() ->
+    gen_server:start_link({local, ?MODULE}, ?MODULE, [], []).
+
+
+-spec add_job(#rep{}) -> ok.
+add_job(#rep{} = Rep) when Rep#rep.id /= undefined ->
+    Job = #job{
+        id = Rep#rep.id,
+        rep = Rep,
+        history = [{added, os:timestamp()}]},
+    gen_server:call(?MODULE, {add_job, Job}, infinity).
+
+
+-spec remove_job(job_id()) -> ok.
+remove_job(Id) ->
+    gen_server:call(?MODULE, {remove_job, Id}, infinity).
+
+
+-spec reschedule() -> ok.
+% Trigger a manual reschedule. Used for testing and/or ops.
+reschedule() ->
+    gen_server:call(?MODULE, reschedule, infinity).
+
+
+-spec rep_state(rep_id()) -> #rep{} | nil.
+rep_state(RepId) ->
+    case (catch ets:lookup_element(?MODULE, RepId, #job.rep)) of
+        {'EXIT',{badarg, _}} ->
+            nil;
+        Rep ->
+            Rep
+    end.
+
+
+-spec job_summary(job_id(), non_neg_integer()) -> [_] | nil.
+job_summary(JobId, HealthThreshold) ->
+    case job_by_id(JobId) of
+        {ok, #job{pid = Pid, history = History, rep = Rep}} ->
+            ErrorCount = consecutive_crashes(History, HealthThreshold),
+            {State, Info} = case {Pid, ErrorCount} of
+                {undefined, 0}  ->
+                    {pending, null};
+                {undefined, ErrorCount} when ErrorCount > 0 ->
+                     [{{crashed, Error}, _When} | _] = History,
+                     ErrMsg = couch_replicator_utils:rep_error_to_binary(Error),
+                     {crashing, ErrMsg};
+                {Pid, ErrorCount} when is_pid(Pid) ->
+                     {running, null}
+            end,
+            [
+                {source, iolist_to_binary(ejson_url(Rep#rep.source))},
+                {target, iolist_to_binary(ejson_url(Rep#rep.target))},
+                {state, State},
+                {info, Info},
+                {error_count, ErrorCount},
+                {last_updated, last_updated(History)},
+                {start_time, couch_replicator_utils:iso8601(Rep#rep.start_time)},
+                {proxy, job_proxy_url(Rep#rep.source)}
+            ];
+        {error, not_found} ->
+            nil  % Job might have just completed
+    end.
+
+
+job_proxy_url(#httpdb{proxy_url = ProxyUrl}) when is_list(ProxyUrl) ->
+    list_to_binary(couch_util:url_strip_password(ProxyUrl));
+
+job_proxy_url(_Endpoint) ->
+    null.
+
+
+-spec health_threshold() -> non_neg_integer().
+health_threshold() ->
+    config:get_integer("replicator", "health_threshold",
+        ?DEFAULT_HEALTH_THRESHOLD_SEC).
+
+
+-spec find_jobs_by_dbname(binary()) -> list(#rep{}).
+find_jobs_by_dbname(DbName) ->
+    Rep = #rep{db_name = DbName, _ = '_'},
+    MatchSpec = #job{id = '$1', rep = Rep, _ = '_'},
+    [RepId || [RepId] <- ets:match(?MODULE, MatchSpec)].
+
+
+-spec find_jobs_by_doc(binary(), binary()) -> list(#rep{}).
+find_jobs_by_doc(DbName, DocId) ->
+    Rep =  #rep{db_name = DbName, doc_id = DocId, _ = '_'},
+    MatchSpec = #job{id = '$1', rep = Rep, _ = '_'},
+    [RepId || [RepId] <- ets:match(?MODULE, MatchSpec)].
+
+
+
+
+%% gen_server functions
+
+init(_) ->
+    EtsOpts = [named_table, {read_concurrency, true}, {keypos, #job.id}],
+    ?MODULE = ets:new(?MODULE, EtsOpts),
+    ok = config:listen_for_changes(?MODULE, nil),
+    Interval = config:get_integer("replicator", "interval", ?DEFAULT_SCHEDULER_INTERVAL),
+    MaxJobs = config:get_integer("replicator", "max_jobs", ?DEFAULT_MAX_JOBS),
+    MaxChurn = config:get_integer("replicator", "max_churn", ?DEFAULT_MAX_CHURN),
+    MaxHistory = config:get_integer("replicator", "max_history", ?DEFAULT_MAX_HISTORY),
+    {ok, Timer} = timer:send_after(Interval, reschedule),
+    State = #state{
+        interval = Interval,
+        max_jobs = MaxJobs,
+        max_churn = MaxChurn,
+        max_history = MaxHistory,
+        timer = Timer
+    },
+    {ok, State}.
+
+
+handle_call({add_job, Job}, _From, State) ->
+    ok = maybe_remove_job_int(Job#job.id, State),
+    true = add_job_int(Job),
+    ok = maybe_start_newly_added_job(Job, State),
+    couch_stats:increment_counter([couch_replicator, jobs, adds]),
+    TotalJobs = ets:info(?MODULE, size),
+    couch_stats:update_gauge([couch_replicator, jobs, total], TotalJobs),
+    {reply, ok, State};
+
+handle_call({remove_job, Id}, _From, State) ->
+    ok = maybe_remove_job_int(Id, State),
+    {reply, ok, State};
+
+handle_call(reschedule, _From, State) ->
+    ok = reschedule(State),
+    {reply, ok, State};
+
+handle_call(_, _From, State) ->
+    {noreply, State}.
+
+
+handle_cast({set_max_jobs, MaxJobs}, State) when is_integer(MaxJobs), MaxJobs >= 0 ->
+    couch_log:notice("~p: max_jobs set to ~B", [?MODULE, MaxJobs]),
+    {noreply, State#state{max_jobs = MaxJobs}};
+
+handle_cast({set_max_churn, MaxChurn}, State) when is_integer(MaxChurn), MaxChurn > 0 ->
+    couch_log:notice("~p: max_churn set to ~B", [?MODULE, MaxChurn]),
+    {noreply, State#state{max_churn = MaxChurn}};
+
+handle_cast({set_max_history, MaxHistory}, State) when is_integer(MaxHistory), MaxHistory > 0 ->
+    couch_log:notice("~p: max_history set to ~B", [?MODULE, MaxHistory]),
+    {noreply, State#state{max_history = MaxHistory}};
+
+handle_cast({set_interval, Interval}, State) when is_integer(Interval), Interval > 0 ->
+    couch_log:notice("~p: interval set to ~B", [?MODULE, Interval]),
+    {noreply, State#state{interval = Interval}};
+
+handle_cast(_, State) ->
+    {noreply, State}.
+
+
+handle_info(reschedule, State) ->
+    ok = reschedule(State),
+    {ok, cancel} = timer:cancel(State#state.timer),
+    {ok, Timer} = timer:send_after(State#state.interval, reschedule),
+    {noreply, State#state{timer = Timer}};
+
+handle_info({'DOWN', _Ref, process, Pid, normal}, State) ->
+    {ok, Job} = job_by_pid(Pid),
+    couch_log:notice("~p: Job ~p completed normally", [?MODULE, Job#job.id]),
+    remove_job_int(Job),
+    update_running_jobs_stats(),
+    {noreply, State};
+
+handle_info({'DOWN', _Ref, process, Pid, Reason}, State) ->
+    {ok, Job} = job_by_pid(Pid),
+    ok = handle_crashed_job(Job, Reason, State),
+    {noreply, State};
+
+handle_info(restart_config_listener, State) ->
+    ok = config:listen_for_changes(?MODULE, nil),
+    {noreply, State};
+
+handle_info(_, State) ->
+    {noreply, State}.
+
+
+code_change(_OldVsn, State, _Extra) ->
+    {ok, State}.
+
+
+terminate(_Reason, _State) ->
+    ok.
+
+
+format_status(_Opt, [_PDict, State]) ->
+    [{max_jobs, State#state.max_jobs},
+     {running_jobs, running_job_count()},
+     {pending_jobs, pending_job_count()}].
+
+
+%% config listener functions
+
+handle_config_change("replicator", "max_jobs", V, _, S) ->
+    ok = gen_server:cast(?MODULE, {set_max_jobs, list_to_integer(V)}),
+    {ok, S};
+
+handle_config_change("replicator", "max_churn", V, _, S) ->
+    ok = gen_server:cast(?MODULE, {set_max_churn, list_to_integer(V)}),
+    {ok, S};
+
+handle_config_change("replicator", "interval", V, _, S) ->
+    ok = gen_server:cast(?MODULE, {set_interval, list_to_integer(V)}),
+    {ok, S};
+
+handle_config_change("replicator", "max_history", V, _, S) ->
+    ok = gen_server:cast(?MODULE, {set_history, list_to_integer(V)}),
+    {ok, S};
+
+handle_config_change(_, _, _, _, S) ->
+    {ok, S}.
+
+
+handle_config_terminate(_, stop, _) ->
+    ok;
+
+handle_config_terminate(_, _, _) ->
+    Pid = whereis(?MODULE),
+    erlang:send_after(?RELISTEN_DELAY, Pid, restart_config_listener).
+
+
+%% private functions
+
+
+% Handle crashed jobs. Handling differs between transient and permanent jobs.
+% Transient jobs are those posted to the _replicate endpoint. They don't have a
+% db associated with them. When those jobs crash, they are not restarted. That
+% is also consistent with behavior when the node they run on, crashed and they
+% do not migrate to other nodes. Permanent jobs are those created from
+% replicator documents. Those jobs, once they pass basic validation and end up
+% in the scheduler will be retried indefinitely (with appropriate exponential
+% backoffs).
+-spec handle_crashed_job(#job{}, any(), #state{}) -> ok.
+handle_crashed_job(#job{rep = #rep{db_name = null}} = Job, Reason, _State) ->
+    Msg = "~p : Transient job ~p failed, removing. Error: ~p",
+    ErrorBinary = couch_replicator_utils:rep_error_to_binary(Reason),
+    couch_log:error(Msg, [?MODULE, Job#job.id, ErrorBinary]),
+    remove_job_int(Job),
+    update_running_jobs_stats(),
+    ok;
+
+handle_crashed_job(Job, Reason, State) ->
+    ok = update_state_crashed(Job, Reason, State),
+    case couch_replicator_doc_processor:update_docs() of
+        true ->
+            couch_replicator_docs:update_error(Job#job.rep, Reason);
+        false ->
+            ok
+    end,
+    case ets:info(?MODULE, size) < State#state.max_jobs of
+        true ->
+            % Starting pending jobs is an O(TotalJobsCount) operation. Only do
+            % it if there is a relatively small number of jobs. Otherwise
+            % scheduler could be blocked if there is a cascade of lots failing
+            % jobs in a row.
+            start_pending_jobs(State),
+            update_running_jobs_stats(),
+            ok;
+        false ->
+            ok
+    end.
+
+
+% Attempt to start a newly added job. First quickly check if total jobs
+% already exceed max jobs, then do a more expensive check which runs a
+% select (an O(n) operation) to check pending jobs specifically.
+-spec maybe_start_newly_added_job(#job{}, #state{}) -> ok.
+maybe_start_newly_added_job(Job, State) ->
+    MaxJobs = State#state.max_jobs,
+    TotalJobs = ets:info(?MODULE, size),
+    case TotalJobs < MaxJobs andalso running_job_count() < MaxJobs of
+        true ->
+            start_job_int(Job, State),
+            update_running_jobs_stats(),
+            ok;
+        false ->
+            ok
+    end.
+
+% Return up to a given number of oldest, not recently crashed jobs. Try to be
+% memory efficient and use ets:foldl to accumulate jobs.
+-spec pending_jobs(non_neg_integer()) -> [#job{}].
+pending_jobs(0) ->
+    % Handle this case as user could set max_churn to 0. If this is passed to
+    % other function clause it will crash as gb_sets:largest assumes set is not
+    % empty.
+    [];
+
+pending_jobs(Count) when is_integer(Count), Count > 0 ->
+    Set0 = gb_sets:new(),  % [{LastStart, Job},...]
+    Now = os:timestamp(),
+    Acc0 = {Set0, Now, Count, health_threshold()},
+    {Set1, _, _, _} = ets:foldl(fun pending_fold/2, Acc0, ?MODULE),
+    [Job || {_Started, Job} <- gb_sets:to_list(Set1)].
+
+
+pending_fold(Job, {Set, Now, Count, HealthThreshold}) ->
+    Set1 = case {not_recently_crashed(Job, Now, HealthThreshold),
+        gb_sets:size(Set) >= Count} of
+        {true, true} ->
+             % Job is healthy but already reached accumulated limit, so might
+             % have to replace one of the accumulated jobs
+             pending_maybe_replace(Job, Set);
+        {true, false} ->
+             % Job is healthy and we haven't reached the limit, so add job
+             % to accumulator
+             gb_sets:add_element({last_started(Job), Job}, Set);
+        {false, _} ->
+             % This jobs is not healthy (has crashed too recently), so skip it.
+             Set
+    end,
+    {Set1, Now, Count, HealthThreshold}.
+
+
+% Replace Job in the accumulator if it is older than youngest job there.
+pending_maybe_replace(Job, Set) ->
+    Started = last_started(Job),
+    {Youngest, YoungestJob} = gb_sets:largest(Set),
+    case Started < Youngest of
+        true ->
+            Set1 = gb_sets:delete({Youngest, YoungestJob}, Set),
+            gb_sets:add_element({Started, Job}, Set1);
+        false ->
+            Set
+    end.
+
+
+start_jobs(Count, State) ->
+    [start_job_int(Job, State) || Job <- pending_jobs(Count)],
+    ok.
+
+
+-spec stop_jobs(non_neg_integer(), boolean(), #state{}) -> non_neg_integer().
+stop_jobs(Count, IsContinuous, State) ->
+    Running0 = running_jobs(),
+    ContinuousPred = fun(Job) -> is_continuous(Job) =:= IsContinuous end,
+    Running1 = lists:filter(ContinuousPred, Running0),
+    Running2 = lists:sort(fun oldest_job_first/2, Running1),
+    Running3 = lists:sublist(Running2, Count),
+    length([stop_job_int(Job, State) || Job <- Running3]).
+
+
+oldest_job_first(#job{} = A, #job{} = B) ->
+    last_started(A) =< last_started(B).
+
+
+not_recently_crashed(#job{history = History}, Now, HealthThreshold) ->
+    case History of
+        [{added, _When}] ->
+            true;
+        [{stopped, _When} | _] ->
+            true;
+        _ ->
+            LatestCrashT = latest_crash_timestamp(History),
+            CrashCount = consecutive_crashes(History, HealthThreshold),
+            timer:now_diff(Now, LatestCrashT) >= backoff_micros(CrashCount)
+    end.
+
+
+% Count consecutive crashes. A crash happens when there is a `crashed` event
+% within a short period of time (configurable) after any other event. It could
+% be `crashed, started` for jobs crashing quickly after starting, `crashed,
+% crashed`, `crashed, stopped` if job repeatedly failed to start
+% being stopped. Or it could be `crashed, added` if it crashed immediately after
+% being added during start.
+%
+% The end of the consecutive crashes ends when a crashed event is seen with
+% the time delta between previous greater then the threshold.
+-spec consecutive_crashes(history(), non_neg_integer()) -> non_neg_integer().
+consecutive_crashes(History, HealthThreshold) when is_list(History) ->
+    consecutive_crashes(History, HealthThreshold, 0).
+
+
+-spec consecutive_crashes(history(), non_neg_integer(), non_neg_integer()) ->
+     non_neg_integer().
+consecutive_crashes([], _HealthThreashold, Count) ->
+    Count;
+
+consecutive_crashes([{{crashed, _}, CrashT}, {_, PrevT} = PrevEvent | Rest],
+    HealthThreshold, Count) ->
+    case timer:now_diff(CrashT, PrevT) > HealthThreshold * 1000000 of
+        true ->
+            Count;
+        false ->
+            consecutive_crashes([PrevEvent | Rest], HealthThreshold, Count + 1)
+    end;
+
+consecutive_crashes([{stopped, _}, {started, _} | _], _HealthThreshold, Count) ->
+    Count;
+
+consecutive_crashes([_ | Rest], HealthThreshold, Count) ->
+    consecutive_crashes(Rest, HealthThreshold, Count).
+
+
+-spec latest_crash_timestamp(history()) -> erlang:timestamp().
+latest_crash_timestamp([]) ->
+    {0, 0, 0};  % Used to avoid special-casing "no crash" when doing now_diff
+
+latest_crash_timestamp([{{crashed, _Reason}, When} | _]) ->
+    When;
+
+latest_crash_timestamp([_Event | Rest]) ->
+    latest_crash_timestamp(Rest).
+
+
+-spec backoff_micros(non_neg_integer()) -> non_neg_integer().
+backoff_micros(CrashCount) ->
+    BackoffExp = erlang:min(CrashCount - 1, ?MAX_BACKOFF_EXPONENT),
+    (1 bsl BackoffExp) * ?BACKOFF_INTERVAL_MICROS.
+
+
+-spec add_job_int(#job{}) -> boolean().
+add_job_int(#job{} = Job) ->
+    ets:insert_new(?MODULE, Job).
+
+
+-spec maybe_remove_job_int(job_id(), #state{}) -> ok.
+maybe_remove_job_int(JobId, State) ->
+    case job_by_id(JobId) of
+        {ok, Job} ->
+            ok = stop_job_int(Job, State),
+            true = remove_job_int(Job),
+            couch_stats:increment_counter([couch_replicator, jobs, removes]),
+            TotalJobs = ets:info(?MODULE, size),
+            couch_stats:update_gauge([couch_replicator, jobs, total], TotalJobs),
+            update_running_jobs_stats(),
+            ok;
+        {error, not_found} ->
+            ok
+    end.
+
+
+start_job_int(#job{pid = Pid}, _State) when Pid /= undefined ->
+    ok;
+
+start_job_int(#job{} = Job0, State) ->
+    Job = maybe_optimize_job_for_rate_limiting(Job0),
+    case couch_replicator_scheduler_sup:start_child(Job#job.rep) of
+        {ok, Child} ->
+            Ref = monitor(process, Child),
+            ok = update_state_started(Job, Child, Ref, State),
+            couch_log:notice("~p: Job ~p started as ~p",
+                [?MODULE, Job#job.id, Child]);
+        {error, {already_started, OtherPid}} when node(OtherPid) =:= node ->
+            Ref = monitor(process, OtherPid),
+            ok = update_state_started(Job, OtherPid, Ref, State),
+            couch_log:notice("~p: Job ~p already running as ~p. Most likely",
+                " because replicator scheduler was restarted",
+                 [?MODULE, Job#job.id, OtherPid]);
+        {error, {already_started, OtherPid}} when node(OtherPid) =/= node ->
+            CrashMsg = "Duplicate replication running on another node",
+            couch_log:notice("~p: Job ~p already running as ~p. Most likely"
+                " because a duplicate replication is running on another node",
+                [?MODULE, Job#job.id, OtherPid]),
+            ok = update_state_crashed(Job, CrashMsg, State);
+        {error, Reason} ->
+            couch_log:notice("~p: Job ~p failed to start for reason ~p",
+                [?MODULE, Job, Reason]),
+            ok = update_state_crashed(Job, Reason, State)
+    end.
+
+
+-spec stop_job_int(#job{}, #state{}) -> ok | {error, term()}.
+stop_job_int(#job{pid = undefined}, _State) ->
+    ok;
+
+stop_job_int(#job{} = Job, State) ->
+    ok = couch_replicator_scheduler_sup:terminate_child(Job#job.pid),
+    demonitor(Job#job.monitor, [flush]),
+    ok = update_state_stopped(Job, State),
+    couch_log:notice("~p: Job ~p stopped as ~p",
+        [?MODULE, Job#job.id, Job#job.pid]).
+
+
+-spec remove_job_int(#job{}) -> true.
+remove_job_int(#job{} = Job) ->
+    ets:delete(?MODULE, Job#job.id).
+
+
+-spec running_job_count() -> non_neg_integer().
+running_job_count() ->
+    ets:info(?MODULE, size) - pending_job_count().
+
+
+-spec running_jobs() -> [#job{}].
+running_jobs() ->
+    ets:select(?MODULE, [{#job{pid = '$1', _='_'}, [{is_pid, '$1'}], ['$_']}]).
+
+
+-spec pending_job_count() -> non_neg_integer().
+pending_job_count() ->
+    ets:select_count(?MODULE, [{#job{pid=undefined, _='_'}, [], [true]}]).
+
+
+
+
+
+-spec job_by_pid(pid()) -> {ok, #job{}} | {error, not_found}.
+job_by_pid(Pid) when is_pid(Pid) ->
+    case ets:match_object(?MODULE, #job{pid=Pid, _='_'}) of
+        [] ->
+            {error, not_found};
+        [#job{}=Job] ->
+            {ok, Job}
+    end.
+
+-spec job_by_id(job_id()) -> {ok, #job{}} | {error, not_found}.
+job_by_id(Id) ->
+    case ets:lookup(?MODULE, Id) of
+        [] ->
+            {error, not_found};
+        [#job{}=Job] ->
+            {ok, Job}
+    end.
+
+
+-spec update_state_stopped(#job{}, #state{}) -> ok.
+update_state_stopped(Job, State) ->
+    Job1 = reset_job_process(Job),
+    Job2 = update_history(Job1, stopped, os:timestamp(), State),
+    true = ets:insert(?MODULE, Job2),
+    couch_stats:increment_counter([couch_replicator, jobs, stops]),
+    ok.
+
+
+-spec update_state_started(#job{}, pid(), reference(), #state{}) -> ok.
+update_state_started(Job, Pid, Ref, State) ->
+    Job1 = set_job_process(Job, Pid, Ref),
+    Job2 = update_history(Job1, started, os:timestamp(), State),
+    true = ets:insert(?MODULE, Job2),
+    couch_stats:increment_counter([couch_replicator, jobs, starts]),
+    ok.
+
+
+-spec update_state_crashed(#job{}, any(), #state{}) -> ok.
+update_state_crashed(Job, Reason, State) ->
+    Job1 = reset_job_process(Job),
+    Job2 = update_history(Job1, {crashed, Reason}, os:timestamp(), State),
+    true = ets:insert(?MODULE, Job2),
+    couch_stats:increment_counter([couch_replicator, jobs, crashes]),
+    ok.
+
+
+-spec set_job_process(#job{}, pid(), reference()) -> #job{}.
+set_job_process(#job{} = Job, Pid, Ref) when is_pid(Pid), is_reference(Ref) ->
+    Job#job{pid = Pid, monitor = Ref}.
+
+
+-spec reset_job_process(#job{}) -> #job{}.
+reset_job_process(#job{} = Job) ->
+    Job#job{pid = undefined, monitor = undefined}.
+
+
+-spec reschedule(#state{}) -> ok.
+reschedule(State) ->
+    Running = running_job_count(),
+    Pending = pending_job_count(),
+    stop_excess_jobs(State, Running),
+    start_pending_jobs(State, Running, Pending),
+    rotate_jobs(State, Running, Pending),
+    update_running_jobs_stats(),
+    ok.
+
+
+-spec stop_excess_jobs(#state{}, non_neg_integer()) -> ok.
+stop_excess_jobs(State, Running) ->
+    #state{max_jobs=MaxJobs} = State,
+    StopCount = Running - MaxJobs,
+    if StopCount > 0 ->
+        Stopped = stop_jobs(StopCount, true, State),
+        OneshotLeft = StopCount - Stopped,
+        if OneshotLeft > 0 ->
+            stop_jobs(OneshotLeft, false, State),
+            ok;
+        true ->
+            ok
+        end;
+    true ->
+        ok
+    end.
+
+
+start_pending_jobs(State) ->
+    start_pending_jobs(State, running_job_count(), pending_job_count()).
+
+
+start_pending_jobs(State, Running, Pending) ->
+    #state{max_jobs=MaxJobs} = State,
+    if Running < MaxJobs, Pending > 0 ->
+        start_jobs(MaxJobs - Running, State);
+    true ->
+        ok
+    end.
+
+-spec rotate_jobs(#state{}, non_neg_integer(), non_neg_integer()) -> ok.
+rotate_jobs(State, Running, Pending) ->
+    #state{max_jobs=MaxJobs, max_churn=MaxChurn} = State,
+    if Running == MaxJobs, Pending > 0 ->
+        RotateCount = lists:min([Pending, Running, MaxChurn]),
+        StopCount = stop_jobs(RotateCount, true, State),
+        start_jobs(StopCount, State);
+    true ->
+        ok
+    end.
+
+
+-spec last_started(#job{}) -> erlang:timestamp().
+last_started(#job{} = Job) ->
+    case lists:keyfind(started, 1, Job#job.history) of
+        false ->
+            {0, 0, 0};
+        {started, When} ->
+            When
+    end.
+
+
+-spec update_history(#job{}, event_type(), erlang:timestamp(), #state{}) -> #job{}.
+update_history(Job, Type, When, State) ->
+    History0 = [{Type, When} | Job#job.history],
+    History1 = lists:sublist(History0, State#state.max_history),
+    Job#job{history = History1}.
+
+
+-spec update_running_jobs_stats() -> ok.
+update_running_jobs_stats() ->
+    Acc0 = #stats_acc{now = os:timestamp()},
+    AccR = ets:foldl(fun stats_fold/2, Acc0, ?MODULE),
+    #stats_acc{
+        pending_t = PendingSum,
+        running_t = RunningSum,
+        crashed_t = CrashedSum,
+        pending_n = PendingN,
+        running_n = RunningN,
+        crashed_n = CrashedN
+    } = AccR,
+    PendingAvg = avg(PendingSum, PendingN),
+    RunningAvg = avg(RunningSum, RunningN),
+    CrashedAvg = avg(CrashedSum, CrashedN),
+    couch_stats:update_gauge([couch_replicator, jobs, pending], PendingN),
+    couch_stats:update_gauge([couch_replicator, jobs, running], RunningN),
+    couch_stats:update_gauge([couch_replicator, jobs, crashed], CrashedN),
+    couch_stats:update_gauge([couch_replicator, jobs, avg_pending], PendingAvg),
+    couch_stats:update_gauge([couch_replicator, jobs, avg_running], RunningAvg),
+    couch_stats:update_gauge([couch_replicator, jobs, avg_crashed], CrashedAvg),
+    ok.
+
+
+-spec stats_fold(#job{}, #stats_acc{}) -> #stats_acc{}.
+stats_fold(#job{pid = undefined, history = [{added, T}]}, Acc) ->
+    #stats_acc{now = Now, pending_t = SumT, pending_n = Cnt} = Acc,
+    Dt = round(timer:now_diff(Now, T) / 1000000),
+    Acc#stats_acc{pending_t = SumT + Dt, pending_n = Cnt + 1};
+
+stats_fold(#job{pid = undefined, history = [{stopped, T} | _]}, Acc) ->
+    #stats_acc{now = Now, pending_t = SumT, pending_n = Cnt} = Acc,
+    Dt = round(timer:now_diff(Now, T) / 1000000),
+    Acc#stats_acc{pending_t = SumT + Dt, pending_n = Cnt + 1};
+
+stats_fold(#job{pid = undefined, history = [{{crashed, _}, T} | _]}, Acc) ->
+    #stats_acc{now = Now, crashed_t = SumT, crashed_n = Cnt} = Acc,
+    Dt = round(timer:now_diff(Now, T) / 1000000),
+    Acc#stats_acc{crashed_t = SumT + Dt, crashed_n = Cnt + 1};
+
+stats_fold(#job{pid = P, history = [{started, T} | _]}, Acc) when is_pid(P) ->
+    #stats_acc{now = Now, running_t = SumT, running_n = Cnt} = Acc,
+    Dt = round(timer:now_diff(Now, T) / 1000000),
+    Acc#stats_acc{running_t = SumT + Dt, running_n = Cnt + 1}.
+
+
+
+-spec avg(Sum :: non_neg_integer(), N :: non_neg_integer())  -> non_neg_integer().
+avg(_Sum, 0) ->
+    0;
+
+avg(Sum, N) when N > 0 ->
+    round(Sum / N).
+
+
+-spec ejson_url(#httpdb{} | binary()) -> binary().
+ejson_url(#httpdb{}=Httpdb) ->
+    couch_util:url_strip_password(Httpdb#httpdb.url);
+ejson_url(DbName) when is_binary(DbName) ->
+    DbName.
+
+
+-spec job_ejson(#job{}) -> {[_ | _]}.
+job_ejson(Job) ->
+    Rep = Job#job.rep,
+    Source = ejson_url(Rep#rep.source),
+    Target = ejson_url(Rep#rep.target),
+    History = lists:map(fun({Type, When}) ->
+        EventProps  = case Type of
+            {crashed, Reason} ->
+                [{type, crashed}, {reason, crash_reason_json(Reason)}];
+            Type ->
+                [{type, Type}]
+        end,
+        {[{timestamp, couch_replicator_utils:iso8601(When)} | EventProps]}
+    end, Job#job.history),
+    {BaseID, Ext} = Job#job.id,
+    Pid = case Job#job.pid of
+        undefined ->
+            null;
+        P when is_pid(P) ->
+            ?l2b(pid_to_list(P))
+    end,
+    {[
+        {id, iolist_to_binary([BaseID, Ext])},
+        {pid, Pid},
+        {source, iolist_to_binary(Source)},
+        {target, iolist_to_binary(Target)},
+        {database, Rep#rep.db_name},
+        {user, (Rep#rep.user_ctx)#user_ctx.name},
+        {doc_id, Rep#rep.doc_id},
+        {history, History},
+        {node, node()},
+        {start_time, couch_replicator_utils:iso8601(Rep#rep.start_time)}
+    ]}.
+
+
+-spec jobs() -> [[tuple()]].
+jobs() ->
+    ets:foldl(fun(Job, Acc) ->
+        [job_ejson(Job) | Acc]
+    end, [], couch_replicator_scheduler).
+
+
+-spec job(job_id()) -> {ok, {[_ | _]}} | {error, not_found}.
+job(JobId) ->
+    case job_by_id(JobId) of
+        {ok, Job} ->
+            {ok, job_ejson(Job)};
+        Error ->
+            Error
+    end.
+
+
+crash_reason_json({_CrashType, Info}) when is_binary(Info) ->
+    Info;
+crash_reason_json(Reason) when is_binary(Reason) ->
+    Reason;
+crash_reason_json(Error) ->
+    couch_replicator_utils:rep_error_to_binary(Error).
+
+
+-spec last_updated([_]) -> binary().
+last_updated([{_Type, When} | _]) ->
+    couch_replicator_utils:iso8601(When).
+
+
+-spec is_continuous(#job{}) -> boolean().
+is_continuous(#job{rep = Rep}) ->
+    couch_util:get_value(continuous, Rep#rep.options, false).
+
+
+% If job crashed last time because it was rate limited, try to
+% optimize some options to help the job make progress.
+-spec maybe_optimize_job_for_rate_limiting(#job{}) -> #job{}.
+maybe_optimize_job_for_rate_limiting(Job = #job{history =
+    [{{crashed, {shutdown, max_backoff}}, _} | _]}) ->
+    Opts = [
+        {checkpoint_interval, 5000},
+        {worker_processes, 2},
+        {worker_batch_size, 100},
+        {http_connections, 5}
+    ],
+    Rep = lists:foldl(fun optimize_int_option/2, Job#job.rep, Opts),
+    Job#job{rep = Rep};
+
+maybe_optimize_job_for_rate_limiting(Job) ->
+    Job.
+
+
+-spec optimize_int_option({atom(), any()}, #rep{}) -> #rep{}.
+optimize_int_option({Key, Val}, #rep{options = Options} = Rep) ->
+    case couch_util:get_value(Key, Options) of
+        CurVal when is_integer(CurVal), CurVal > Val ->
+            Msg = "~p replication ~p : setting ~p = ~p due to rate limiting",
+            couch_log:warning(Msg, [?MODULE, Rep#rep.id, Key, Val]),
+            Options1 = lists:keyreplace(Key, 1, Options, {Key, Val}),
+            Rep#rep{options = Options1};
+        _ ->
+            Rep
+    end.
+
+
+-ifdef(TEST).
+-include_lib("eunit/include/eunit.hrl").
+
+
+backoff_micros_test_() ->
+    BaseInterval = ?BACKOFF_INTERVAL_MICROS,
+    [?_assertEqual(R * BaseInterval, backoff_micros(N)) || {R, N} <- [
+        {1, 1}, {2, 2}, {4, 3}, {8, 4}, {16, 5}, {32, 6}, {64, 7}, {128, 8},
+        {256, 9}, {512, 10}, {1024, 11}, {1024, 12}
+    ]].
+
+
+consecutive_crashes_test_() ->
+    Threshold = ?DEFAULT_HEALTH_THRESHOLD_SEC,
+    [?_assertEqual(R, consecutive_crashes(H, Threshold)) || {R, H} <- [
+        {0, []},
+        {0, [added()]},
+        {0, [stopped()]},
+        {0, [crashed()]},
+        {1, [crashed(), added()]},
+        {1, [crashed(), crashed()]},
+        {1, [crashed(), stopped()]},
+        {3, [crashed(), crashed(), crashed(), added()]},
+        {2, [crashed(), crashed(), stopped()]},
+        {1, [crashed(), started(), added()]},
+        {2, [crashed(3), started(2), crashed(1), started(0)]},
+        {0, [stopped(3), started(2), crashed(1), started(0)]},
+        {1, [crashed(3), started(2), stopped(1), started(0)]},
+        {0, [crashed(999), started(0)]},
+        {1, [crashed(999), started(998), crashed(997), started(0)]}
+    ]].
+
+
+consecutive_crashes_non_default_threshold_test_() ->
+    [?_assertEqual(R, consecutive_crashes(H, T)) || {R, H, T} <- [
+        {0, [crashed(11), started(0)], 10},
+        {1, [crashed(10), started(0)], 10}
+    ]].
+
+
+latest_crash_timestamp_test_() ->
+    [?_assertEqual({0, R, 0}, latest_crash_timestamp(H)) || {R, H} <- [
+         {0, [added()]},
+         {1, [crashed(1)]},
+         {3, [crashed(3), started(2), crashed(1), started(0)]},
+         {1, [started(3), stopped(2), crashed(1), started(0)]}
+    ]].
+
+
+last_started_test_() ->
+    [?_assertEqual({0, R, 0}, last_started(testjob(H))) || {R, H} <- [
+         {0, [added()]},
+         {0, [crashed(1)]},
+         {1, [started(1)]},
+         {1, [added(), started(1)]},
+         {2, [started(2), started(1)]},
+         {2, [crashed(3), started(2), started(1)]}
+    ]].
+
+
+oldest_job_first_test() ->
+    J0 = testjob([crashed()]),
+    J1 = testjob([started(1)]),
+    J2 = testjob([started(2)]),
+    Sort = fun(Jobs) -> lists:sort(fun oldest_job_first/2, Jobs) end,
+    ?assertEqual([], Sort([])),
+    ?assertEqual([J1], Sort([J1])),
+    ?assertEqual([J1, J2], Sort([J2, J1])),
+    ?assertEqual([J0, J1, J2], Sort([J2, J1, J0])).
+
+
+scheduler_test_() ->
+    {
+        foreach,
+        fun setup/0,
+        fun teardown/1,
+        [
+            t_pending_jobs_simple(),
+            t_pending_jobs_skip_crashed(),
+            t_one_job_starts(),
+            t_no_jobs_start_if_max_is_0(),
+            t_one_job_starts_if_max_is_1(),
+            t_max_churn_does_not_throttle_initial_start(),
+            t_excess_oneshot_only_jobs(),
+            t_excess_continuous_only_jobs(),
+            t_excess_prefer_continuous_first(),
+            t_stop_oldest_first(),
+            t_start_oldest_first(),
+            t_dont_stop_if_nothing_pending(),
+            t_max_churn_limits_number_of_rotated_jobs(),
+            t_if_pending_less_than_running_start_all_pending(),
+            t_running_less_than_pending_swap_all_running(),
+            t_oneshot_dont_get_rotated(),
+            t_rotate_continuous_only_if_mixed(),
+            t_oneshot_dont_get_starting_priority(),
+            t_oneshot_will_hog_the_scheduler(),
+            t_if_excess_is_trimmed_rotation_doesnt_happen(),
+            t_if_transient_job_crashes_it_gets_removed(),
+            t_if_permanent_job_crashes_it_stays_in_ets()
+         ]
+    }.
+
+
+t_pending_jobs_simple() ->
+   ?_test(begin
+        Job1 = oneshot(1),
+        Job2 = oneshot(2),
+        setup_jobs([Job2, Job1]),
+        ?assertEqual([], pending_jobs(0)),
+        ?assertEqual([Job1], pending_jobs(1)),
+        ?assertEqual([Job1, Job2], pending_jobs(2)),
+        ?assertEqual([Job1, Job2], pending_jobs(3))
+    end).
+
+
+t_pending_jobs_skip_crashed() ->
+   ?_test(begin
+        Job = oneshot(1),
+        Ts = os:timestamp(),
+        History = [crashed(Ts), started(Ts) | Job#job.history],
+        Job1 = Job#job{history = History},
+        Job2 = oneshot(2),
+        Job3 = oneshot(3),
+        setup_jobs([Job2, Job1, Job3]),
+        ?assertEqual([Job2], pending_jobs(1)),
+        ?assertEqual([Job2, Job3], pending_jobs(2)),
+        ?assertEqual([Job2, Job3], pending_jobs(3))
+    end).
+
+
+t_one_job_starts() ->
+    ?_test(begin
+        setup_jobs([oneshot(1)]),
+        ?assertEqual({0, 1}, run_stop_count()),
+        reschedule(mock_state(?DEFAULT_MAX_JOBS)),
+        ?assertEqual({1, 0}, run_stop_count())
+    end).
+
+
+t_no_jobs_start_if_max_is_0() ->
+    ?_test(begin
+        setup_jobs([oneshot(1)]),
+        reschedule(mock_state(0)),
+        ?assertEqual({0, 1}, run_stop_count())
+    end).
+
+
+t_one_job_starts_if_max_is_1() ->
+    ?_test(begin
+        setup_jobs([oneshot(1), oneshot(2)]),
+        reschedule(mock_state(1)),
+        ?assertEqual({1, 1}, run_stop_count())
+    end).
+
+
+t_max_churn_does_not_throttle_initial_start() ->
+    ?_test(begin
+        setup_jobs([oneshot(1), oneshot(2)]),
+        reschedule(mock_state(?DEFAULT_MAX_JOBS, 0)),
+        ?assertEqual({2, 0}, run_stop_count())
+    end).
+
+
+t_excess_oneshot_only_jobs() ->
+    ?_test(begin
+        setup_jobs([oneshot_running(1), oneshot_running(2)]),
+        ?assertEqual({2, 0}, run_stop_count()),
+        reschedule(mock_state(1)),
+        ?assertEqual({1, 1}, run_stop_count()),
+        reschedule(mock_state(0)),
+        ?assertEqual({0, 2}, run_stop_count())
+    end).
+
+
+t_excess_continuous_only_jobs() ->
+    ?_test(begin
+        setup_jobs([continuous_running(1), continuous_running(2)]),
+        ?assertEqual({2, 0}, run_stop_count()),
+        reschedule(mock_state(1)),
+        ?assertEqual({1, 1}, run_stop_count()),
+        reschedule(mock_state(0)),
+        ?assertEqual({0, 2}, run_stop_count())
+    end).
+
+
+t_excess_prefer_continuous_first() ->
+    ?_test(begin
+        Jobs = [
+            continuous_running(1),
+            oneshot_running(2),
+            continuous_running(3)
+        ],
+        setup_jobs(Jobs),
+        ?assertEqual({3, 0}, run_stop_count()),
+        ?assertEqual({1, 0}, oneshot_run_stop_count()),
+        reschedule(mock_state(2)),
+        ?assertEqual({2, 1}, run_stop_count()),
+        ?assertEqual({1, 0}, oneshot_run_stop_count()),
+        reschedule(mock_state(1)),
+        ?assertEqual({1, 0}, oneshot_run_stop_count()),
+        reschedule(mock_state(0)),
+        ?assertEqual({0, 1}, oneshot_run_stop_count())
+    end).
+
+
+t_stop_oldest_first() ->
+    ?_test(begin
+        Jobs = [
+            continuous_running(7),
+            continuous_running(4),
+            continuous_running(5)
+        ],
+        setup_jobs(Jobs),
+        reschedule(mock_state(2)),
+        ?assertEqual({2, 1}, run_stop_count()),
+        ?assertEqual([4], jobs_stopped()),
+        reschedule(mock_state(1)),
+        ?assertEqual([7], jobs_running())
+    end).
+
+
+t_start_oldest_first() ->
+    ?_test(begin
+        setup_jobs([continuous(7), continuous(2), continuous(5)]),
+        reschedule(mock_state(1)),
+        ?assertEqual({1, 2}, run_stop_count()),
+        ?assertEqual([2], jobs_running()),
+        reschedule(mock_state(2)),
+        ?assertEqual([7], jobs_stopped())
+    end).
+
+
+t_dont_stop_if_nothing_pending() ->
+    ?_test(begin
+        setup_jobs([continuous_running(1), continuous_running(2)]),
+        reschedule(mock_state(2)),
+        ?assertEqual({2, 0}, run_stop_count())
+    end).
+
+
+t_max_churn_limits_number_of_rotated_jobs() ->
+    ?_test(begin
+        Jobs = [
+            continuous(1),
+            continuous_running(2),
+            continuous(3),
+            continuous_running(4)
+        ],
+        setup_jobs(Jobs),
+        reschedule(mock_state(2, 1)),
+        ?assertEqual([2, 3], jobs_stopped())
+    end).
+
+
+t_if_pending_less_than_running_start_all_pending() ->
+    ?_test(begin
+        Jobs = [
+            continuous(1),
+            continuous_running(2),
+            continuous(3),
+            continuous_running(4),
+            continuous_running(5)
+        ],
+        setup_jobs(Jobs),
+        reschedule(mock_state(3)),
+        ?assertEqual([1, 2, 5], jobs_running())
+    end).
+
+
+t_running_less_than_pending_swap_all_running() ->
+    ?_test(begin
+        Jobs = [
+            continuous(1),
+            continuous(2),
+            continuous(3),
+            continuous_running(4),
+            continuous_running(5)
+        ],
+        setup_jobs(Jobs),
+        reschedule(mock_state(2)),
+        ?assertEqual([3, 4, 5], jobs_stopped())
+    end).
+
+
+t_oneshot_dont_get_rotated() ->
+    ?_test(begin
+        setup_jobs([oneshot_running(1), continuous(2)]),
+        reschedule(mock_state(1)),
+        ?assertEqual([1], jobs_running())
+    end).
+
+
+t_rotate_continuous_only_if_mixed() ->
+    ?_test(begin
+        setup_jobs([continuous(1), oneshot_running(2), continuous_running(3)]),
+        reschedule(mock_state(2)),
+        ?assertEqual([1, 2], jobs_running())
+    end).
+
+
+t_oneshot_dont_get_starting_priority() ->
+    ?_test(begin
+        setup_jobs([continuous(1), oneshot(2), continuous_running(3)]),
+        reschedule(mock_state(1)),
+        ?assertEqual([1], jobs_running())
+    end).
+
+
+% This tested in other test cases, it is here to mainly make explicit a property
+% of one-shot replications -- they can starve other jobs if they "take control" of
+% all the available scheduler slots.
+t_oneshot_will_hog_the_scheduler() ->
+    ?_test(begin
+        Jobs = [
+            oneshot_running(1),
+            oneshot_running(2),
+            oneshot(3),
+            continuous(4)
+        ],
+        setup_jobs(Jobs),
+        reschedule(mock_state(2)),
+        ?assertEqual([1, 2], jobs_running())
+    end).
+
+
+t_if_excess_is_trimmed_rotation_doesnt_happen() ->
+    ?_test(begin
+        Jobs = [
+            continuous(1),
+            continuous_running(2),
+            continuous_running(3)
+        ],
+        setup_jobs(Jobs),
+        reschedule(mock_state(1)),
+        ?assertEqual([3], jobs_running())
+    end).
+
+
+t_if_transient_job_crashes_it_gets_removed() ->
+    ?_test(begin
+        Pid = mock_pid(),
+        Job =  #job{
+            id = job1,
+            pid = Pid,
+            history = [added()],
+            rep = #rep{db_name = null, options = [{continuous, true}]}
+        },
+        setup_jobs([Job]),
+        ?assertEqual(1, ets:info(?MODULE, size)),
+        State = #state{max_history = 3},
+        {noreply, State} = handle_info({'DOWN', r1, process, Pid, failed}, State),
+        ?assertEqual(0, ets:info(?MODULE, size))
+   end).
+
+
+t_if_permanent_job_crashes_it_stays_in_ets() ->
+    ?_test(begin
+        Pid = mock_pid(),
+        Job =  #job{
+            id = job1,
+            pid = Pid,
+            history = [added()],
+            rep = #rep{db_name = <<"db1">>, options = [{continuous, true}]}
+        },
+        setup_jobs([Job]),
+        ?assertEqual(1, ets:info(?MODULE, size)),
+        State = #state{max_jobs =1, max_history = 3},
+        {noreply, State} = handle_info({'DOWN', r1, process, Pid, failed}, State),
+        ?assertEqual(1, ets:info(?MODULE, size)),
+        [Job1] = ets:lookup(?MODULE, job1),
+        [Latest | _] = Job1#job.history,
+        ?assertMatch({{crashed, failed}, _}, Latest)
+   end).
+
+
+% Test helper functions
+
+setup() ->
+    catch ets:delete(?MODULE),
+    meck:expect(couch_log, notice, 2, ok),
+    meck:expect(couch_log, warning, 2, ok),
+    meck:expect(couch_log, error, 2, ok),
+    meck:expect(couch_replicator_scheduler_sup, terminate_child, 1, ok),
+    meck:expect(couch_stats, increment_counter, 1, ok),
+    meck:expect(couch_stats, update_gauge, 2, ok),
+    Pid = mock_pid(),
+    meck:expect(couch_replicator_scheduler_sup, start_child, 1, {ok, Pid}).
+
+
+teardown(_) ->
+    catch ets:delete(?MODULE),
+    meck:unload().
+
+
+setup_jobs(Jobs) when is_list(Jobs) ->
+    ?MODULE = ets:new(?MODULE, [named_table, {keypos, #job.id}]),
+    ets:insert(?MODULE, Jobs).
+
+
+all_jobs() ->
+    lists:usort(ets:tab2list(?MODULE)).
+
+
+jobs_stopped() ->
+    [Job#job.id || Job <- all_jobs(), Job#job.pid =:= undefined].
+
+
+jobs_running() ->
+    [Job#job.id || Job <- all_jobs(), Job#job.pid =/= undefined].
+
+
+run_stop_count() ->
+    {length(jobs_running()), length(jobs_stopped())}.
+
+
+oneshot_run_stop_count() ->
+    Running = [Job#job.id || Job <- all_jobs(), Job#job.pid =/= undefined,
+        not is_continuous(Job)],
+    Stopped = [Job#job.id || Job <- all_jobs(), Job#job.pid =:= undefined,
+        not is_continuous(Job)],
+    {length(Running), length(Stopped)}.
+
+
+mock_state(MaxJobs) ->
+    #state{
+        max_jobs = MaxJobs,
+        max_churn = ?DEFAULT_MAX_CHURN,
+        max_history = ?DEFAULT_MAX_HISTORY
+    }.
+
+mock_state(MaxJobs, MaxChurn) ->
+    #state{
+        max_jobs = MaxJobs,
+        max_churn = MaxChurn,
+        max_history = ?DEFAULT_MAX_HISTORY
+    }.
+
+
+continuous(Id) when is_integer(Id) ->
+    Started = Id,
+    Hist = [stopped(Started+1), started(Started), added()],
+    #job{
+        id = Id,
+        history = Hist,
+        rep = #rep{options = [{continuous, true}]}
+    }.
+
+
+continuous_running(Id) when is_integer(Id) ->
+    Started = Id,
+    Pid = mock_pid(),
+    #job{
+        id = Id,
+        history = [started(Started), added()],
+        rep = #rep{options = [{continuous, true}]},
+        pid = Pid,
+        monitor = monitor(process, Pid)
+    }.
+
+
+oneshot(Id) when is_integer(Id) ->
+    Started = Id,
+    Hist = [stopped(Started + 1), started(Started), added()],
+    #job{id = Id, history = Hist, rep = #rep{options = []}}.
+
+
+oneshot_running(Id) when is_integer(Id) ->
+    Started = Id,
+    Pid = mock_pid(),
+    #job{
+        id = Id,
+        history = [started(Started), added()],
+        rep = #rep{options = []},
+        pid = Pid,
+        monitor = monitor(process, Pid)
+    }.
+
+
+testjob(Hist) when is_list(Hist) ->
+    #job{history = Hist}.
+
+
+mock_pid() ->
+   list_to_pid("<0.999.999>").
+
+crashed() ->
+    crashed(0).
+
+
+crashed(WhenSec) when is_integer(WhenSec)->
+    {{crashed, some_reason}, {0, WhenSec, 0}};
+crashed({MSec, Sec, USec}) ->
+    {{crashed, some_reason}, {MSec, Sec, USec}}.
+
+
+started() ->
+    started(0).
+
+
+started(WhenSec) when is_integer(WhenSec)->
+    {started, {0, WhenSec, 0}};
+
+started({MSec, Sec, USec}) ->
+    {started, {MSec, Sec, USec}}.
+
+
+stopped() ->
+    stopped(0).
+
+
+stopped(WhenSec) ->
+    {stopped, {0, WhenSec, 0}}.
+
+added() ->
+    {added, {0, 0, 0}}.
+
+-endif.
+
diff --git a/src/couch_replicator_scheduler.hrl b/src/couch_replicator_scheduler.hrl
new file mode 100644
index 0000000..5203b0c
--- /dev/null
+++ b/src/couch_replicator_scheduler.hrl
@@ -0,0 +1,15 @@
+% 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.
+
+
+-type job_id() :: term().
+-type job_args() :: term().
diff --git a/src/couch_replicator_scheduler_job.erl b/src/couch_replicator_scheduler_job.erl
new file mode 100644
index 0000000..1c9faaf
--- /dev/null
+++ b/src/couch_replicator_scheduler_job.erl
@@ -0,0 +1,945 @@
+% 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_replicator_scheduler_job).
+-behaviour(gen_server).
+-vsn(1).
+
+-include_lib("couch/include/couch_db.hrl").
+-include("couch_replicator_api_wrap.hrl").
+-include("couch_replicator_scheduler.hrl").
+-include("couch_replicator.hrl").
+
+%% public api
+-export([start_link/1]).
+
+%% gen_server callbacks
+-export([init/1, terminate/2, code_change/3]).
+-export([handle_call/3, handle_cast/2, handle_info/2]).
+-export([format_status/2]).
+
+
+%% imports
+-import(couch_util, [
+    get_value/2,
+    get_value/3,
+    to_binary/1
+]).
+
+-import(couch_replicator_utils, [
+    start_db_compaction_notifier/2,
+    stop_db_compaction_notifier/1,
+    pp_rep_id/1
+]).
+
+
+%% definitions
+-define(LOWEST_SEQ, 0).
+-define(DEFAULT_CHECKPOINT_INTERVAL, 30000).
+-record(rep_state, {
+    rep_details,
+    source_name,
+    target_name,
+    source,
+    target,
+    history,
+    checkpoint_history,
+    start_seq,
+    committed_seq,
+    current_through_seq,
+    seqs_in_progress = [],
+    highest_seq_done = {0, ?LOWEST_SEQ},
+    source_log,
+    target_log,
+    rep_starttime,
+    src_starttime,
+    tgt_starttime,
+    timer, % checkpoint timer
+    changes_queue,
+    changes_manager,
+    changes_reader,
+    workers,
+    stats = couch_replicator_stats:new(),
+    session_id,
+    source_db_compaction_notifier = nil,
+    target_db_compaction_notifier = nil,
+    source_monitor = nil,
+    target_monitor = nil,
+    source_seq = nil,
+    use_checkpoints = true,
+    checkpoint_interval = ?DEFAULT_CHECKPOINT_INTERVAL,
+    type = db,
+    view = nil
+}).
+
+start_link(#rep{id = {BaseId, Ext}, source = Src, target = Tgt} = Rep) ->
+    RepChildId = BaseId ++ Ext,
+    Source = couch_replicator_api_wrap:db_uri(Src),
+    Target = couch_replicator_api_wrap:db_uri(Tgt),
+    ServerName = {global, {?MODULE, Rep#rep.id}},
+
+    case gen_server:start_link(ServerName, ?MODULE, Rep, []) of
+        {ok, Pid} ->
+            couch_log:notice("starting new replication `~s` at ~p (`~s` -> `~s`)",
+                             [RepChildId, Pid, Source, Target]),
+            {ok, Pid};
+        {error, Reason} ->
+            couch_log:warning("failed to start replication `~s` (`~s` -> `~s`)",
+                           [RepChildId, Source, Target]),
+            {error, Reason}
+    end.
+
+init(InitArgs) ->
+    {ok, InitArgs, 0}.
+
+do_init(#rep{options = Options, id = {BaseId, Ext}, user_ctx=UserCtx} = Rep) ->
+    process_flag(trap_exit, true),
+
+    #rep_state{
+        source = Source,
+        target = Target,
+        source_name = SourceName,
+        target_name = TargetName,
+        start_seq = {_Ts, StartSeq},
+        committed_seq = {_, CommittedSeq},
+        highest_seq_done = {_, HighestSeq},
+        checkpoint_interval = CheckpointInterval
+    } = State = init_state(Rep),
+
+    NumWorkers = get_value(worker_processes, Options),
+    BatchSize = get_value(worker_batch_size, Options),
+    {ok, ChangesQueue} = couch_work_queue:new([
+        {max_items, BatchSize * NumWorkers * 2},
+        {max_size, 100 * 1024 * NumWorkers}
+    ]),
+    % This starts the _changes reader process. It adds the changes from
+    % the source db to the ChangesQueue.
+    {ok, ChangesReader} = couch_replicator_changes_reader:start_link(
+        StartSeq, Source, ChangesQueue, Options
+    ),
+    % Changes manager - responsible for dequeing batches from the changes queue
+    % and deliver them to the worker processes.
+    ChangesManager = spawn_changes_manager(self(), ChangesQueue, BatchSize),
+    % This starts the worker processes. They ask the changes queue manager for a
+    % a batch of _changes rows to process -> check which revs are missing in the
+    % target, and for the missing ones, it copies them from the source to the target.
+    MaxConns = get_value(http_connections, Options),
+    Workers = lists:map(
+        fun(_) ->
+            couch_stats:increment_counter([couch_replicator, workers_started]),
+            {ok, Pid} = couch_replicator_worker:start_link(
+                self(), Source, Target, ChangesManager, MaxConns),
+            Pid
+        end,
+        lists:seq(1, NumWorkers)),
+
+    couch_task_status:add_task([
+        {type, replication},
+        {user, UserCtx#user_ctx.name},
+        {replication_id, ?l2b(BaseId ++ Ext)},
+        {database, Rep#rep.db_name},
+        {doc_id, Rep#rep.doc_id},
+        {source, ?l2b(SourceName)},
+        {target, ?l2b(TargetName)},
+        {continuous, get_value(continuous, Options, false)},
+        {revisions_checked, 0},
+        {missing_revisions_found, 0},
+        {docs_read, 0},
+        {docs_written, 0},
+        {changes_pending, get_pending_count(State)},
+        {doc_write_failures, 0},
+        {source_seq, HighestSeq},
+        {checkpointed_source_seq, CommittedSeq},
+        {checkpoint_interval, CheckpointInterval}
+    ]),
+    couch_task_status:set_update_frequency(1000),
+
+    % Until OTP R14B03:
+    %
+    % Restarting a temporary supervised child implies that the original arguments
+    % (#rep{} record) specified in the MFA component of the supervisor
+    % child spec will always be used whenever the child is restarted.
+    % This implies the same replication performance tunning parameters will
+    % always be used. The solution is to delete the child spec (see
+    % cancel_replication/1) and then start the replication again, but this is
+    % unfortunately not immune to race conditions.
+
+    couch_log:notice("Replication `~p` is using:~n"
+        "~c~p worker processes~n"
+        "~ca worker batch size of ~p~n"
+        "~c~p HTTP connections~n"
+        "~ca connection timeout of ~p milliseconds~n"
+        "~c~p retries per request~n"
+        "~csocket options are: ~s~s",
+        [BaseId ++ Ext, $\t, NumWorkers, $\t, BatchSize, $\t,
+            MaxConns, $\t, get_value(connection_timeout, Options),
+            $\t, get_value(retries, Options),
+            $\t, io_lib:format("~p", [get_value(socket_options, Options)]),
+            case StartSeq of
+            ?LOWEST_SEQ ->
+                "";
+            _ ->
+                io_lib:format("~n~csource start sequence ~p", [$\t, StartSeq])
+            end]),
+
+    couch_log:debug("Worker pids are: ~p", [Workers]),
+
+    doc_update_triggered(Rep),
+
+    {ok, State#rep_state{
+            changes_queue = ChangesQueue,
+            changes_manager = ChangesManager,
+            changes_reader = ChangesReader,
+            workers = Workers
+        }
+    }.
+
+adjust_maxconn(Src = #httpdb{http_connections = 1}, RepId) ->
+    Msg = "Adjusting minimum number of HTTP source connections to 2 for ~p",
+    couch_log:notice(Msg, [RepId]),
+    Src#httpdb{http_connections = 2};
+
+adjust_maxconn(Src, _RepId) ->
+    Src.
+
+handle_info(shutdown, St) ->
+    {stop, shutdown, St};
+
+handle_info({'DOWN', Ref, _, _, Why}, #rep_state{source_monitor = Ref} = St) ->
+    couch_log:error("Source database is down. Reason: ~p", [Why]),
+    {stop, source_db_down, St};
+
+handle_info({'DOWN', Ref, _, _, Why}, #rep_state{target_monitor = Ref} = St) ->
+    couch_log:error("Target database is down. Reason: ~p", [Why]),
+    {stop, target_db_down, St};
+
+handle_info({'EXIT', Pid, max_backoff}, State) ->
+    couch_log:error("Max backoff reached child process ~p", [Pid]),
+    {stop, {shutdown, max_backoff}, State};
+
+handle_info({'EXIT', Pid, {shutdown, max_backoff}}, State) ->
+    couch_log:error("Max backoff reached child process ~p", [Pid]),
+    {stop, {shutdown, max_backoff}, State};
+
+handle_info({'EXIT', Pid, normal}, #rep_state{changes_reader=Pid} = State) ->
+    {noreply, State};
+
+handle_info({'EXIT', Pid, Reason}, #rep_state{changes_reader=Pid} = State) ->
+    couch_stats:increment_counter([couch_replicator, changes_reader_deaths]),
+    couch_log:error("ChangesReader process died with reason: ~p", [Reason]),
+    {stop, changes_reader_died, cancel_timer(State)};
+
+handle_info({'EXIT', Pid, normal}, #rep_state{changes_manager = Pid} = State) ->
+    {noreply, State};
+
+handle_info({'EXIT', Pid, Reason}, #rep_state{changes_manager = Pid} = State) ->
+    couch_stats:increment_counter([couch_replicator, changes_manager_deaths]),
+    couch_log:error("ChangesManager process died with reason: ~p", [Reason]),
+    {stop, changes_manager_died, cancel_timer(State)};
+
+handle_info({'EXIT', Pid, normal}, #rep_state{changes_queue=Pid} = State) ->
+    {noreply, State};
+
+handle_info({'EXIT', Pid, Reason}, #rep_state{changes_queue=Pid} = State) ->
+    couch_stats:increment_counter([couch_replicator, changes_queue_deaths]),
+    couch_log:error("ChangesQueue process died with reason: ~p", [Reason]),
+    {stop, changes_queue_died, cancel_timer(State)};
+
+handle_info({'EXIT', Pid, normal}, #rep_state{workers = Workers} = State) ->
+    case Workers -- [Pid] of
+    Workers ->
+        couch_log:error("unknown pid bit the dust ~p ~n",[Pid]),
+        {noreply, State#rep_state{workers = Workers}};
+        %% not clear why a stop was here before
+        %%{stop, {unknown_process_died, Pid, normal}, State};
+    [] ->
+        catch unlink(State#rep_state.changes_manager),
+        catch exit(State#rep_state.changes_manager, kill),
+        do_last_checkpoint(State);
+    Workers2 ->
+        {noreply, State#rep_state{workers = Workers2}}
+    end;
+
+handle_info({'EXIT', Pid, Reason}, #rep_state{workers = Workers} = State) ->
+    State2 = cancel_timer(State),
+    case lists:member(Pid, Workers) of
+    false ->
+        {stop, {unknown_process_died, Pid, Reason}, State2};
+    true ->
+        couch_stats:increment_counter([couch_replicator, worker_deaths]),
+        couch_log:error("Worker ~p died with reason: ~p", [Pid, Reason]),
+        {stop, {worker_died, Pid, Reason}, State2}
+    end;
+
+handle_info(timeout, InitArgs) ->
+    try do_init(InitArgs) of {ok, State} ->
+        {noreply, State}
+    catch
+        exit:{http_request_failed, _, _, max_backoff} ->
+            {stop, {shutdown, max_backoff}, {error, InitArgs}};
+        Class:Error ->
+            ShutdownReason = {error, replication_start_error(Error)},
+            % Shutdown state is a hack as it is not really the state of the
+            % gen_server (it failed to initialize, so it doesn't have one).
+            % Shutdown state is used to pass extra info about why start failed.
+            ShutdownState = {error, Class, erlang:get_stacktrace(), InitArgs},
+            {stop, {shutdown, ShutdownReason}, ShutdownState}
+    end.
+
+handle_call(get_details, _From, #rep_state{rep_details = Rep} = State) ->
+    {reply, {ok, Rep}, State};
+
+handle_call({add_stats, Stats}, From, State) ->
+    gen_server:reply(From, ok),
+    NewStats = couch_replicator_utils:sum_stats(State#rep_state.stats, Stats),
+    {noreply, State#rep_state{stats = NewStats}};
+
+handle_call({report_seq_done, Seq, StatsInc}, From,
+    #rep_state{seqs_in_progress = SeqsInProgress, highest_seq_done = HighestDone,
+        current_through_seq = ThroughSeq, stats = Stats} = State) ->
+    gen_server:reply(From, ok),
+    {NewThroughSeq0, NewSeqsInProgress} = case SeqsInProgress of
+    [] ->
+        {Seq, []};
+    [Seq | Rest] ->
+        {Seq, Rest};
+    [_ | _] ->
+        {ThroughSeq, ordsets:del_element(Seq, SeqsInProgress)}
+    end,
+    NewHighestDone = lists:max([HighestDone, Seq]),
+    NewThroughSeq = case NewSeqsInProgress of
+    [] ->
+        lists:max([NewThroughSeq0, NewHighestDone]);
+    _ ->
+        NewThroughSeq0
+    end,
+    couch_log:debug("Worker reported seq ~p, through seq was ~p, "
+        "new through seq is ~p, highest seq done was ~p, "
+        "new highest seq done is ~p~n"
+        "Seqs in progress were: ~p~nSeqs in progress are now: ~p",
+        [Seq, ThroughSeq, NewThroughSeq, HighestDone,
+            NewHighestDone, SeqsInProgress, NewSeqsInProgress]),
+    NewState = State#rep_state{
+        stats = couch_replicator_utils:sum_stats(Stats, StatsInc),
+        current_through_seq = NewThroughSeq,
+        seqs_in_progress = NewSeqsInProgress,
+        highest_seq_done = NewHighestDone
+    },
+    update_task(NewState),
+    {noreply, NewState}.
+
+
+handle_cast({db_compacted, DbName},
+    #rep_state{source = #db{name = DbName} = Source} = State) ->
+    {ok, NewSource} = couch_db:reopen(Source),
+    {noreply, State#rep_state{source = NewSource}};
+
+handle_cast({db_compacted, DbName},
+    #rep_state{target = #db{name = DbName} = Target} = State) ->
+    {ok, NewTarget} = couch_db:reopen(Target),
+    {noreply, State#rep_state{target = NewTarget}};
+
+handle_cast(checkpoint, State) ->
+    case do_checkpoint(State) of
+    {ok, NewState} ->
+        couch_stats:increment_counter([couch_replicator, checkpoints, success]),
+        {noreply, NewState#rep_state{timer = start_timer(State)}};
+    Error ->
+        couch_stats:increment_counter([couch_replicator, checkpoints, failure]),
+        {stop, Error, State}
+    end;
+
+handle_cast({report_seq, Seq},
+    #rep_state{seqs_in_progress = SeqsInProgress} = State) ->
+    NewSeqsInProgress = ordsets:add_element(Seq, SeqsInProgress),
+    {noreply, State#rep_state{seqs_in_progress = NewSeqsInProgress}}.
+
+
+code_change(_OldVsn, #rep_state{}=State, _Extra) ->
+    {ok, State}.
+
+
+headers_strip_creds([], Acc) ->
+    lists:reverse(Acc);
+headers_strip_creds([{Key, Value0} | Rest], Acc) ->
+    Value = case string:to_lower(Key) of
+    "authorization" ->
+        "****";
+    _ ->
+        Value0
+    end,
+    headers_strip_creds(Rest, [{Key, Value} | Acc]).
+
+
+httpdb_strip_creds(#httpdb{url = Url, headers = Headers} = HttpDb) ->
+    HttpDb#httpdb{
+        url = couch_util:url_strip_password(Url),
+        headers = headers_strip_creds(Headers, [])
+    };
+httpdb_strip_creds(LocalDb) ->
+    LocalDb.
+
+
+rep_strip_creds(#rep{source = Source, target = Target} = Rep) ->
+    Rep#rep{
+        source = httpdb_strip_creds(Source),
+        target = httpdb_strip_creds(Target)
+    }.
+
+
+state_strip_creds(#rep_state{rep_details = Rep, source = Source, target = Target} = State) ->
+    % #rep_state contains the source and target at the top level and also
+    % in the nested #rep_details record
+    State#rep_state{
+        rep_details = rep_strip_creds(Rep),
+        source = httpdb_strip_creds(Source),
+        target = httpdb_strip_creds(Target)
+    }.
+
+
+terminate(normal, #rep_state{rep_details = #rep{id = RepId} = Rep,
+    checkpoint_history = CheckpointHistory} = State) ->
+    terminate_cleanup(State),
+    couch_replicator_notifier:notify({finished, RepId, CheckpointHistory}),
+    doc_update_completed(Rep, rep_stats(State));
+
+terminate(shutdown, #rep_state{rep_details = #rep{id = RepId}} = State) ->
+    % Replication stopped via _scheduler_sup:terminate_child/1, which can be
+    % occur during regular scheduler operation or when job is removed from
+    % the scheduler.
+    State1 = case do_checkpoint(State) of
+        {ok, NewState} ->
+            NewState;
+        Error ->
+            LogMsg = "~p : Failed last checkpoint. Job: ~p Error: ~p",
+            couch_log:error(LogMsg, [?MODULE, RepId, Error]),
+            State
+    end,
+    couch_replicator_notifier:notify({stopped, RepId, <<"stopped">>}),
+    terminate_cleanup(State1);
+
+terminate({shutdown, max_backoff}, {error, InitArgs}) ->
+    #rep{id = {BaseId, Ext} = RepId} = InitArgs,
+    couch_stats:increment_counter([couch_replicator, failed_starts]),
+    couch_log:warning("Replication `~s` reached max backoff ", [BaseId ++ Ext]),
+    couch_replicator_notifier:notify({error, RepId, max_backoff});
+
+terminate({shutdown, {error, Error}}, {error, Class, Stack, InitArgs}) ->
+    #rep{id=RepId} = InitArgs,
+    couch_stats:increment_counter([couch_replicator, failed_starts]),
+    CleanInitArgs = rep_strip_creds(InitArgs),
+    couch_log:error("~p:~p: Replication failed to start for args ~p: ~p",
+             [Class, Error, CleanInitArgs, Stack]),
+    couch_replicator_notifier:notify({error, RepId, Error});
+
+terminate({shutdown, max_backoff}, State) ->
+    #rep_state{
+        source_name = Source,
+        target_name = Target,
+        rep_details = #rep{id = {BaseId, Ext} = RepId}
+    } = State,
+    couch_log:error("Replication `~s` (`~s` -> `~s`) reached max backoff",
+        [BaseId ++ Ext, Source, Target]),
+    terminate_cleanup(State),
+    couch_replicator_notifier:notify({error, RepId, max_backoff});
+
+terminate(Reason, State) ->
+#rep_state{
+        source_name = Source,
+        target_name = Target,
+        rep_details = #rep{id = {BaseId, Ext} = RepId}
+    } = State,
+    couch_log:error("Replication `~s` (`~s` -> `~s`) failed: ~s",
+        [BaseId ++ Ext, Source, Target, to_binary(Reason)]),
+    terminate_cleanup(State),
+    couch_replicator_notifier:notify({error, RepId, Reason}).
+
+terminate_cleanup(State) ->
+    update_task(State),
+    stop_db_compaction_notifier(State#rep_state.source_db_compaction_notifier),
+    stop_db_compaction_notifier(State#rep_state.target_db_compaction_notifier),
+    couch_replicator_api_wrap:db_close(State#rep_state.source),
+    couch_replicator_api_wrap:db_close(State#rep_state.target).
+
+
+format_status(_Opt, [_PDict, State]) ->
+    [{data, [{"State", state_strip_creds(State)}]}].
+
+
+-spec doc_update_triggered(#rep{}) -> ok.
+doc_update_triggered(#rep{db_name = null}) ->
+    ok;
+doc_update_triggered(#rep{id = RepId, doc_id = DocId} = Rep) ->
+    case couch_replicator_doc_processor:update_docs() of
+        true ->
+            couch_replicator_docs:update_triggered(Rep, RepId);
+        false ->
+            ok
+    end,
+    couch_log:notice("Document `~s` triggered replication `~s`",
+        [DocId, pp_rep_id(RepId)]),
+    ok.
+
+-spec doc_update_completed(#rep{}, list()) -> ok.
+doc_update_completed(#rep{db_name = null}, _Stats) ->
+    ok;
+doc_update_completed(#rep{id = RepId, doc_id = DocId, db_name = DbName,
+    start_time = StartTime}, Stats) ->
+    couch_replicator_docs:update_doc_completed(DbName, DocId, Stats, StartTime),
+    couch_log:notice("Replication `~s` completed (triggered by `~s`)",
+        [pp_rep_id(RepId), DocId]),
+    ok.
+
+do_last_checkpoint(#rep_state{seqs_in_progress = [],
+    highest_seq_done = {_Ts, ?LOWEST_SEQ}} = State) ->
+    {stop, normal, cancel_timer(State)};
+do_last_checkpoint(#rep_state{seqs_in_progress = [],
+    highest_seq_done = Seq} = State) ->
+    case do_checkpoint(State#rep_state{current_through_seq = Seq}) of
+    {ok, NewState} ->
+        couch_stats:increment_counter([couch_replicator, checkpoints, success]),
+        {stop, normal, cancel_timer(NewState)};
+    Error ->
+        couch_stats:increment_counter([couch_replicator, checkpoints, failure]),
+        {stop, Error, State}
+    end.
+
+
+start_timer(State) ->
+    After = State#rep_state.checkpoint_interval,
+    case timer:apply_after(After, gen_server, cast, [self(), checkpoint]) of
+    {ok, Ref} ->
+        Ref;
+    Error ->
+        couch_log:error("Replicator, error scheduling checkpoint:  ~p", [Error]),
+        nil
+    end.
+
+
+cancel_timer(#rep_state{timer = nil} = State) ->
+    State;
+cancel_timer(#rep_state{timer = Timer} = State) ->
+    {ok, cancel} = timer:cancel(Timer),
+    State#rep_state{timer = nil}.
+
+
+init_state(Rep) ->
+    #rep{
+        id = {BaseId, _Ext},
+        source = Src0, target = Tgt,
+        options = Options, user_ctx = UserCtx,
+        type = Type, view = View,
+        start_time = StartTime
+    } = Rep,
+    % Adjust minimum number of http source connections to 2 to avoid deadlock
+    Src = adjust_maxconn(Src0, BaseId),
+    {ok, Source} = couch_replicator_api_wrap:db_open(Src, [{user_ctx, UserCtx}]),
+    {ok, Target} = couch_replicator_api_wrap:db_open(Tgt, [{user_ctx, UserCtx}],
+        get_value(create_target, Options, false)),
+
+    {ok, SourceInfo} = couch_replicator_api_wrap:get_db_info(Source),
+    {ok, TargetInfo} = couch_replicator_api_wrap:get_db_info(Target),
+
+    [SourceLog, TargetLog] = find_replication_logs([Source, Target], Rep),
+
+    {StartSeq0, History} = compare_replication_logs(SourceLog, TargetLog),
+    StartSeq1 = get_value(since_seq, Options, StartSeq0),
+    StartSeq = {0, StartSeq1},
+
+    SourceSeq = get_value(<<"update_seq">>, SourceInfo, ?LOWEST_SEQ),
+
+    #doc{body={CheckpointHistory}} = SourceLog,
+    State = #rep_state{
+        rep_details = Rep,
+        source_name = couch_replicator_api_wrap:db_uri(Source),
+        target_name = couch_replicator_api_wrap:db_uri(Target),
+        source = Source,
+        target = Target,
+        history = History,
+        checkpoint_history = {[{<<"no_changes">>, true}| CheckpointHistory]},
+        start_seq = StartSeq,
+        current_through_seq = StartSeq,
+        committed_seq = StartSeq,
+        source_log = SourceLog,
+        target_log = TargetLog,
+        rep_starttime = StartTime,
+        src_starttime = get_value(<<"instance_start_time">>, SourceInfo),
+        tgt_starttime = get_value(<<"instance_start_time">>, TargetInfo),
+        session_id = couch_uuids:random(),
+        source_db_compaction_notifier =
+            start_db_compaction_notifier(Source, self()),
+        target_db_compaction_notifier =
+            start_db_compaction_notifier(Target, self()),
+        source_monitor = db_monitor(Source),
+        target_monitor = db_monitor(Target),
+        source_seq = SourceSeq,
+        use_checkpoints = get_value(use_checkpoints, Options, true),
+        checkpoint_interval = get_value(checkpoint_interval, Options,
+                                        ?DEFAULT_CHECKPOINT_INTERVAL),
+        type = Type,
+        view = View
+    },
+    State#rep_state{timer = start_timer(State)}.
+
+
+find_replication_logs(DbList, #rep{id = {BaseId, _}} = Rep) ->
+    LogId = ?l2b(?LOCAL_DOC_PREFIX ++ BaseId),
+    fold_replication_logs(DbList, ?REP_ID_VERSION, LogId, LogId, Rep, []).
+
+
+fold_replication_logs([], _Vsn, _LogId, _NewId, _Rep, Acc) ->
+    lists:reverse(Acc);
+
+fold_replication_logs([Db | Rest] = Dbs, Vsn, LogId, NewId, Rep, Acc) ->
+    case couch_replicator_api_wrap:open_doc(Db, LogId, [ejson_body]) of
+    {error, <<"not_found">>} when Vsn > 1 ->
+        OldRepId = couch_replicator_utils:replication_id(Rep, Vsn - 1),
+        fold_replication_logs(Dbs, Vsn - 1,
+            ?l2b(?LOCAL_DOC_PREFIX ++ OldRepId), NewId, Rep, Acc);
+    {error, <<"not_found">>} ->
+        fold_replication_logs(
+            Rest, ?REP_ID_VERSION, NewId, NewId, Rep, [#doc{id = NewId} | Acc]);
+    {ok, Doc} when LogId =:= NewId ->
+        fold_replication_logs(
+            Rest, ?REP_ID_VERSION, NewId, NewId, Rep, [Doc | Acc]);
+    {ok, Doc} ->
+        MigratedLog = #doc{id = NewId, body = Doc#doc.body},
+        fold_replication_logs(
+            Rest, ?REP_ID_VERSION, NewId, NewId, Rep, [MigratedLog | Acc])
+    end.
+
+
+spawn_changes_manager(Parent, ChangesQueue, BatchSize) ->
+    spawn_link(fun() ->
+        changes_manager_loop_open(Parent, ChangesQueue, BatchSize, 1)
+    end).
+
+changes_manager_loop_open(Parent, ChangesQueue, BatchSize, Ts) ->
+    receive
+    {get_changes, From} ->
+        case couch_work_queue:dequeue(ChangesQueue, BatchSize) of
+        closed ->
+            From ! {closed, self()};
+        {ok, Changes} ->
+            #doc_info{high_seq = Seq} = lists:last(Changes),
+            ReportSeq = {Ts, Seq},
+            ok = gen_server:cast(Parent, {report_seq, ReportSeq}),
+            From ! {changes, self(), Changes, ReportSeq}
+        end,
+        changes_manager_loop_open(Parent, ChangesQueue, BatchSize, Ts + 1)
+    end.
+
+
+do_checkpoint(#rep_state{use_checkpoints=false} = State) ->
+    NewState = State#rep_state{checkpoint_history = {[{<<"use_checkpoints">>, false}]} },
+    {ok, NewState};
+do_checkpoint(#rep_state{current_through_seq=Seq, committed_seq=Seq} = State) ->
+    update_task(State),
+    {ok, State};
+do_checkpoint(State) ->
+    #rep_state{
+        source_name=SourceName,
+        target_name=TargetName,
+        source = Source,
+        target = Target,
+        history = OldHistory,
+        start_seq = {_, StartSeq},
+        current_through_seq = {_Ts, NewSeq} = NewTsSeq,
+        source_log = SourceLog,
+        target_log = TargetLog,
+        rep_starttime = ReplicationStartTime,
+        src_starttime = SrcInstanceStartTime,
+        tgt_starttime = TgtInstanceStartTime,
+        stats = Stats,
+        rep_details = #rep{options = Options},
+        session_id = SessionId
+    } = State,
+    case commit_to_both(Source, Target) of
+    {source_error, Reason} ->
+         {checkpoint_commit_failure,
+             <<"Failure on source commit: ", (to_binary(Reason))/binary>>};
+    {target_error, Reason} ->
+         {checkpoint_commit_failure,
+             <<"Failure on target commit: ", (to_binary(Reason))/binary>>};
+    {SrcInstanceStartTime, TgtInstanceStartTime} ->
+        couch_log:notice("recording a checkpoint for `~s` -> `~s` at source update_seq ~p",
+            [SourceName, TargetName, NewSeq]),
+        UniversalStartTime = calendar:now_to_universal_time(ReplicationStartTime),
+        StartTime = ?l2b(httpd_util:rfc1123_date(UniversalStartTime)),
+        EndTime = ?l2b(httpd_util:rfc1123_date()),
+        NewHistoryEntry = {[
+            {<<"session_id">>, SessionId},
+            {<<"start_time">>, StartTime},
+            {<<"end_time">>, EndTime},
+            {<<"start_last_seq">>, StartSeq},
+            {<<"end_last_seq">>, NewSeq},
+            {<<"recorded_seq">>, NewSeq},
+            {<<"missing_checked">>, couch_replicator_stats:missing_checked(Stats)},
+            {<<"missing_found">>, couch_replicator_stats:missing_found(Stats)},
+            {<<"docs_read">>, couch_replicator_stats:docs_read(Stats)},
+            {<<"docs_written">>, couch_replicator_stats:docs_written(Stats)},
+            {<<"doc_write_failures">>, couch_replicator_stats:doc_write_failures(Stats)}
+        ]},
+        BaseHistory = [
+            {<<"session_id">>, SessionId},
+            {<<"source_last_seq">>, NewSeq},
+            {<<"replication_id_version">>, ?REP_ID_VERSION}
+        ] ++ case get_value(doc_ids, Options) of
+        undefined ->
+            [];
+        _DocIds ->
+            % backwards compatibility with the result of a replication by
+            % doc IDs in versions 0.11.x and 1.0.x
+            % TODO: deprecate (use same history format, simplify code)
+            [
+                {<<"start_time">>, StartTime},
+                {<<"end_time">>, EndTime},
+                {<<"docs_read">>, couch_replicator_stats:docs_read(Stats)},
+                {<<"docs_written">>, couch_replicator_stats:docs_written(Stats)},
+                {<<"doc_write_failures">>, couch_replicator_stats:doc_write_failures(Stats)}
+            ]
+        end,
+        % limit history to 50 entries
+        NewRepHistory = {
+            BaseHistory ++
+            [{<<"history">>, lists:sublist([NewHistoryEntry | OldHistory], 50)}]
+        },
+
+        try
+            {SrcRevPos, SrcRevId} = update_checkpoint(
+                Source, SourceLog#doc{body = NewRepHistory}, source),
+            {TgtRevPos, TgtRevId} = update_checkpoint(
+                Target, TargetLog#doc{body = NewRepHistory}, target),
+            NewState = State#rep_state{
+                checkpoint_history = NewRepHistory,
+                committed_seq = NewTsSeq,
+                source_log = SourceLog#doc{revs={SrcRevPos, [SrcRevId]}},
+                target_log = TargetLog#doc{revs={TgtRevPos, [TgtRevId]}}
+            },
+            update_task(NewState),
+            {ok, NewState}
+        catch throw:{checkpoint_commit_failure, _} = Failure ->
+            Failure
+        end;
+    {SrcInstanceStartTime, _NewTgtInstanceStartTime} ->
+        {checkpoint_commit_failure, <<"Target database out of sync. "
+            "Try to increase max_dbs_open at the target's server.">>};
+    {_NewSrcInstanceStartTime, TgtInstanceStartTime} ->
+        {checkpoint_commit_failure, <<"Source database out of sync. "
+            "Try to increase max_dbs_open at the source's server.">>};
+    {_NewSrcInstanceStartTime, _NewTgtInstanceStartTime} ->
+        {checkpoint_commit_failure, <<"Source and target databases out of "
+            "sync. Try to increase max_dbs_open at both servers.">>}
+    end.
+
+
+update_checkpoint(Db, Doc, DbType) ->
+    try
+        update_checkpoint(Db, Doc)
+    catch throw:{checkpoint_commit_failure, Reason} ->
+        throw({checkpoint_commit_failure,
+            <<"Error updating the ", (to_binary(DbType))/binary,
+                " checkpoint document: ", (to_binary(Reason))/binary>>})
+    end.
+
+update_checkpoint(Db, #doc{id = LogId, body = LogBody} = Doc) ->
+    try
+        case couch_replicator_api_wrap:update_doc(Db, Doc, [delay_commit]) of
+        {ok, PosRevId} ->
+            PosRevId;
+        {error, Reason} ->
+            throw({checkpoint_commit_failure, Reason})
+        end
+    catch throw:conflict ->
+        case (catch couch_replicator_api_wrap:open_doc(Db, LogId, [ejson_body])) of
+        {ok, #doc{body = LogBody, revs = {Pos, [RevId | _]}}} ->
+            % This means that we were able to update successfully the
+            % checkpoint doc in a previous attempt but we got a connection
+            % error (timeout for e.g.) before receiving the success response.
+            % Therefore the request was retried and we got a conflict, as the
+            % revision we sent is not the current one.
+            % We confirm this by verifying the doc body we just got is the same
+            % that we have just sent.
+            {Pos, RevId};
+        _ ->
+            throw({checkpoint_commit_failure, conflict})
+        end
+    end.
+
+
+commit_to_both(Source, Target) ->
+    % commit the src async
+    ParentPid = self(),
+    SrcCommitPid = spawn_link(
+        fun() ->
+            Result = (catch couch_replicator_api_wrap:ensure_full_commit(Source)),
+            ParentPid ! {self(), Result}
+        end),
+
+    % commit tgt sync
+    TargetResult = (catch couch_replicator_api_wrap:ensure_full_commit(Target)),
+
+    SourceResult = receive
+    {SrcCommitPid, Result} ->
+        unlink(SrcCommitPid),
+        receive {'EXIT', SrcCommitPid, _} -> ok after 0 -> ok end,
+        Result;
+    {'EXIT', SrcCommitPid, Reason} ->
+        {error, Reason}
+    end,
+    case TargetResult of
+    {ok, TargetStartTime} ->
+        case SourceResult of
+        {ok, SourceStartTime} ->
+            {SourceStartTime, TargetStartTime};
+        SourceError ->
+            {source_error, SourceError}
+        end;
+    TargetError ->
+        {target_error, TargetError}
+    end.
+
+
+compare_replication_logs(SrcDoc, TgtDoc) ->
+    #doc{body={RepRecProps}} = SrcDoc,
+    #doc{body={RepRecPropsTgt}} = TgtDoc,
+    case get_value(<<"session_id">>, RepRecProps) ==
+            get_value(<<"session_id">>, RepRecPropsTgt) of
+    true ->
+        % if the records have the same session id,
+        % then we have a valid replication history
+        OldSeqNum = get_value(<<"source_last_seq">>, RepRecProps, ?LOWEST_SEQ),
+        OldHistory = get_value(<<"history">>, RepRecProps, []),
+        {OldSeqNum, OldHistory};
+    false ->
+        SourceHistory = get_value(<<"history">>, RepRecProps, []),
+        TargetHistory = get_value(<<"history">>, RepRecPropsTgt, []),
+        couch_log:notice("Replication records differ. "
+                "Scanning histories to find a common ancestor.", []),
+        couch_log:debug("Record on source:~p~nRecord on target:~p~n",
+                [RepRecProps, RepRecPropsTgt]),
+        compare_rep_history(SourceHistory, TargetHistory)
+    end.
+
+compare_rep_history(S, T) when S =:= [] orelse T =:= [] ->
+    couch_log:notice("no common ancestry -- performing full replication", []),
+    {?LOWEST_SEQ, []};
+compare_rep_history([{S} | SourceRest], [{T} | TargetRest] = Target) ->
+    SourceId = get_value(<<"session_id">>, S),
+    case has_session_id(SourceId, Target) of
+    true ->
+        RecordSeqNum = get_value(<<"recorded_seq">>, S, ?LOWEST_SEQ),
+        couch_log:notice("found a common replication record with source_seq ~p",
+            [RecordSeqNum]),
+        {RecordSeqNum, SourceRest};
+    false ->
+        TargetId = get_value(<<"session_id">>, T),
+        case has_session_id(TargetId, SourceRest) of
+        true ->
+            RecordSeqNum = get_value(<<"recorded_seq">>, T, ?LOWEST_SEQ),
+            couch_log:notice("found a common replication record with source_seq ~p",
+                [RecordSeqNum]),
+            {RecordSeqNum, TargetRest};
+        false ->
+            compare_rep_history(SourceRest, TargetRest)
+        end
+    end.
+
+
+has_session_id(_SessionId, []) ->
+    false;
+has_session_id(SessionId, [{Props} | Rest]) ->
+    case get_value(<<"session_id">>, Props, nil) of
+    SessionId ->
+        true;
+    _Else ->
+        has_session_id(SessionId, Rest)
+    end.
+
+
+db_monitor(#db{} = Db) ->
+    couch_db:monitor(Db);
+db_monitor(_HttpDb) ->
+    nil.
+
+get_pending_count(St) ->
+    Rep = St#rep_state.rep_details,
+    Timeout = get_value(connection_timeout, Rep#rep.options),
+    TimeoutMicro = Timeout * 1000,
+    case get(pending_count_state) of
+        {LastUpdate, PendingCount} ->
+            case timer:now_diff(os:timestamp(), LastUpdate) > TimeoutMicro of
+                true ->
+                    NewPendingCount = get_pending_count_int(St),
+                    put(pending_count_state, {os:timestamp(), NewPendingCount}),
+                    NewPendingCount;
+                false ->
+                    PendingCount
+            end;
+        undefined ->
+            NewPendingCount = get_pending_count_int(St),
+            put(pending_count_state, {os:timestamp(), NewPendingCount}),
+            NewPendingCount
+    end.
+
+
+get_pending_count_int(#rep_state{source = #httpdb{} = Db0}=St) ->
+    {_, Seq} = St#rep_state.highest_seq_done,
+    Db = Db0#httpdb{retries = 3},
+    case (catch couch_replicator_api_wrap:get_pending_count(Db, Seq)) of
+    {ok, Pending} ->
+        Pending;
+    _ ->
+        null
+    end;
+get_pending_count_int(#rep_state{source = Db}=St) ->
+    {_, Seq} = St#rep_state.highest_seq_done,
+    {ok, Pending} = couch_replicator_api_wrap:get_pending_count(Db, Seq),
+    Pending.
+
+
+update_task(State) ->
+    #rep_state{
+        current_through_seq = {_, ThroughSeq},
+        highest_seq_done = {_, HighestSeq}
+    } = State,
+    couch_task_status:update(
+        rep_stats(State) ++ [
+        {source_seq, HighestSeq},
+        {through_seq, ThroughSeq}
+    ]).
+
+
+rep_stats(State) ->
+    #rep_state{
+        committed_seq = {_, CommittedSeq},
+        stats = Stats
+    } = State,
+    [
+        {revisions_checked, couch_replicator_stats:missing_checked(Stats)},
+        {missing_revisions_found, couch_replicator_stats:missing_found(Stats)},
+        {docs_read, couch_replicator_stats:docs_read(Stats)},
+        {docs_written, couch_replicator_stats:docs_written(Stats)},
+        {changes_pending, get_pending_count(State)},
+        {doc_write_failures, couch_replicator_stats:doc_write_failures(Stats)},
+        {checkpointed_source_seq, CommittedSeq}
+    ].
+
+
+replication_start_error({unauthorized, DbUri}) ->
+    {unauthorized, <<"unauthorized to access or create database ", DbUri/binary>>};
+
+replication_start_error({db_not_found, DbUri}) ->
+    {db_not_found, <<"could not open ", DbUri/binary>>};
+
+replication_start_error(Error) ->
+    Error.
diff --git a/src/couch_replicator_scheduler_sup.erl b/src/couch_replicator_scheduler_sup.erl
new file mode 100644
index 0000000..a2cf9bd
--- /dev/null
+++ b/src/couch_replicator_scheduler_sup.erl
@@ -0,0 +1,54 @@
+% 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_replicator_scheduler_sup).
+-behaviour(supervisor).
+-vsn(1).
+
+%% includes
+-include("couch_replicator.hrl").
+
+%% public api
+-export([start_link/0, start_child/1, terminate_child/1]).
+
+%% supervisor api
+-export([init/1]).
+
+%% public functions
+
+start_link() ->
+    supervisor:start_link({local, ?MODULE}, ?MODULE, []).
+
+
+start_child(#rep{} = Rep) ->
+    supervisor:start_child(?MODULE, [Rep]).
+
+
+terminate_child(Pid) ->
+    supervisor:terminate_child(?MODULE, Pid).
+
+%% supervisor functions
+
+init(_Args) ->
+    Start = {couch_replicator_scheduler_job, start_link, []},
+    Restart = temporary, % A crashed job is not entitled to immediate restart.
+    Shutdown = 5000,
+    Type = worker,
+    Modules = [couch_replicator_scheduler_job],
+
+    RestartStrategy = simple_one_for_one,
+    MaxR = 10,
+    MaxT = 3,
+
+    ChildSpec =
+        {undefined, Start, Restart, Shutdown, Type, Modules},
+    {ok, {{RestartStrategy, MaxR, MaxT}, [ChildSpec]}}.
diff --git a/src/couch_replicator_sup.erl b/src/couch_replicator_sup.erl
index 57ad63b..b50c0a1 100644
--- a/src/couch_replicator_sup.erl
+++ b/src/couch_replicator_sup.erl
@@ -26,18 +26,56 @@
             brutal_kill,
             worker,
             dynamic},
-        {couch_replicator_manager,
-            {couch_replicator_manager, start_link, []},
+       {couch_replicator_clustering,
+            {couch_replicator_clustering, start_link, []},
             permanent,
             brutal_kill,
             worker,
-            [couch_replicator_manager]},
-        {couch_replicator_job_sup,
-            {couch_replicator_job_sup, start_link, []},
+            [couch_replicator_clustering]},
+       {couch_replicator_connection,
+            {couch_replicator_connection, start_link, []},
+            permanent,
+            brutal_kill,
+            worker,
+            [couch_replicator_connection]},
+       {couch_replicator_rate_limiter,
+            {couch_replicator_rate_limiter, start_link, []},
+            permanent,
+            brutal_kill,
+            worker,
+            [couch_replicator_rate_limiter]},
+        {couch_replicator_scheduler_sup,
+            {couch_replicator_scheduler_sup, start_link, []},
             permanent,
             infinity,
             supervisor,
-            [couch_replicator_job_sup]}
+            [couch_replicator_scheduler_sup]},
+        {couch_replicator_scheduler,
+            {couch_replicator_scheduler, start_link, []},
+            permanent,
+            brutal_kill,
+            worker,
+            [couch_replicator_scheduler]},
+        {couch_replicator_doc_processor,
+            {couch_replicator_doc_processor, start_link, []},
+            permanent,
+            brutal_kill,
+            worker,
+            [couch_replicator_doc_processor]},
+        {couch_replicator,
+            % This is a simple function call which does not create a process
+            % but returns `ignore`. It is used to make sure each node
+            % a local `_replicator` database.
+            {couch_replicator, ensure_rep_db_exists, []},
+            transient,
+            brutal_kill,
+            worker,
+            [couch_replicator]},
+        {couch_replicator_db_changes,
+            {couch_replicator_db_changes, start_link, []},
+            permanent,
+            brutal_kill,
+            worker,
+            [couch_multidb_changes]}
     ],
-    {ok, {{one_for_one,10,1}, Children}}.
-
+    {ok, {{rest_for_one,10,1}, Children}}.
diff --git a/src/couch_replicator_utils.erl b/src/couch_replicator_utils.erl
index e96d52a..3fd98a3 100644
--- a/src/couch_replicator_utils.erl
+++ b/src/couch_replicator_utils.erl
@@ -17,12 +17,14 @@
 -export([start_db_compaction_notifier/2, stop_db_compaction_notifier/1]).
 -export([replication_id/2]).
 -export([sum_stats/2, is_deleted/1]).
+-export([rep_error_to_binary/1]).
+-export([get_json_value/2, get_json_value/3]).
+-export([pp_rep_id/1]).
+-export([iso8601/1]).
 
 -export([handle_db_event/3]).
 
 -include_lib("couch/include/couch_db.hrl").
--include_lib("ibrowse/include/ibrowse.hrl").
--include("couch_replicator_api_wrap.hrl").
 -include("couch_replicator.hrl").
 
 -import(couch_util, [
@@ -31,385 +33,7 @@
 ]).
 
 
-parse_rep_doc({Props}, UserCtx) ->
-    ProxyParams = parse_proxy_params(get_value(<<"proxy">>, Props, <<>>)),
-    Options = make_options(Props),
-    case get_value(cancel, Options, false) andalso
-        (get_value(id, Options, nil) =/= nil) of
-    true ->
-        {ok, #rep{options = Options, user_ctx = UserCtx}};
-    false ->
-        Source = parse_rep_db(get_value(<<"source">>, Props),
-                              ProxyParams, Options),
-        Target = parse_rep_db(get_value(<<"target">>, Props),
-                              ProxyParams, Options),
 
-
-        {RepType, View} = case get_value(<<"filter">>, Props) of
-                <<"_view">> ->
-                    {QP}  = get_value(query_params, Options, {[]}),
-                    ViewParam = get_value(<<"view">>, QP),
-                    View1 = case re:split(ViewParam, <<"/">>) of
-                        [DName, ViewName] ->
-                            {<< "_design/", DName/binary >>, ViewName};
-                        _ ->
-                            throw({bad_request, "Invalid `view` parameter."})
-                    end,
-                    {view, View1};
-                _ ->
-                    {db, nil}
-            end,
-
-        Rep = #rep{
-            source = Source,
-            target = Target,
-            options = Options,
-            user_ctx = UserCtx,
-            type = RepType,
-            view = View,
-            doc_id = get_value(<<"_id">>, Props, null)
-        },
-        {ok, Rep#rep{id = replication_id(Rep)}}
-    end.
-
-
-replication_id(#rep{options = Options} = Rep) ->
-    BaseId = replication_id(Rep, ?REP_ID_VERSION),
-    {BaseId, maybe_append_options([continuous, create_target], Options)}.
-
-
-% Versioned clauses for generating replication IDs.
-% If a change is made to how replications are identified,
-% please add a new clause and increase ?REP_ID_VERSION.
-
-replication_id(#rep{user_ctx = UserCtx} = Rep, 3) ->
-    UUID = couch_server:get_uuid(),
-    Src = get_rep_endpoint(UserCtx, Rep#rep.source),
-    Tgt = get_rep_endpoint(UserCtx, Rep#rep.target),
-    maybe_append_filters([UUID, Src, Tgt], Rep);
-
-replication_id(#rep{user_ctx = UserCtx} = Rep, 2) ->
-    {ok, HostName} = inet:gethostname(),
-    Port = case (catch mochiweb_socket_server:get(couch_httpd, port)) of
-    P when is_number(P) ->
-        P;
-    _ ->
-        % On restart we might be called before the couch_httpd process is
-        % started.
-        % TODO: we might be under an SSL socket server only, or both under
-        % SSL and a non-SSL socket.
-        % ... mochiweb_socket_server:get(https, port)
-        list_to_integer(config:get("httpd", "port", "5984"))
-    end,
-    Src = get_rep_endpoint(UserCtx, Rep#rep.source),
-    Tgt = get_rep_endpoint(UserCtx, Rep#rep.target),
-    maybe_append_filters([HostName, Port, Src, Tgt], Rep);
-
-replication_id(#rep{user_ctx = UserCtx} = Rep, 1) ->
-    {ok, HostName} = inet:gethostname(),
-    Src = get_rep_endpoint(UserCtx, Rep#rep.source),
-    Tgt = get_rep_endpoint(UserCtx, Rep#rep.target),
-    maybe_append_filters([HostName, Src, Tgt], Rep).
-
-
-maybe_append_filters(Base,
-        #rep{source = Source, user_ctx = UserCtx, options = Options}) ->
-    Filter = get_value(filter, Options),
-    DocIds = get_value(doc_ids, Options),
-    Selector = get_value(selector, Options),
-    Base2 = Base ++
-        case {Filter, DocIds, Selector} of
-        {undefined, undefined, undefined} ->
-            [];
-        {<<"_", _/binary>>, undefined, undefined} ->
-            [Filter, get_value(query_params, Options, {[]})];
-        {_, undefined, undefined} ->
-            [filter_code(Filter, Source, UserCtx),
-                get_value(query_params, Options, {[]})];
-        {undefined, _, undefined} ->
-            [DocIds];
-        {undefined, undefined, _} ->
-            [ejsort(mango_selector:normalize(Selector))];
-        _ ->
-            throw({error, <<"`selector`, `filter` and `doc_ids` fields are mutually exclusive">>})
-        end,
-    couch_util:to_hex(couch_crypto:hash(md5, term_to_binary(Base2))).
-
-
-filter_code(Filter, Source, UserCtx) ->
-    {DDocName, FilterName} =
-    case re:run(Filter, "(.*?)/(.*)", [{capture, [1, 2], binary}]) of
-    {match, [DDocName0, FilterName0]} ->
-        {DDocName0, FilterName0};
-    _ ->
-        throw({error, <<"Invalid filter. Must match `ddocname/filtername`.">>})
-    end,
-    Db = case (catch couch_replicator_api_wrap:db_open(Source, [{user_ctx, UserCtx}])) of
-    {ok, Db0} ->
-        Db0;
-    DbError ->
-        DbErrorMsg = io_lib:format("Could not open source database `~s`: ~s",
-           [couch_replicator_api_wrap:db_uri(Source), couch_util:to_binary(DbError)]),
-        throw({error, iolist_to_binary(DbErrorMsg)})
-    end,
-    try
-        Body = case (catch couch_replicator_api_wrap:open_doc(
-            Db, <<"_design/", DDocName/binary>>, [ejson_body])) of
-        {ok, #doc{body = Body0}} ->
-            Body0;
-        DocError ->
-            DocErrorMsg = io_lib:format(
-                "Couldn't open document `_design/~s` from source "
-                "database `~s`: ~s", [DDocName, couch_replicator_api_wrap:db_uri(Source),
-                    couch_util:to_binary(DocError)]),
-            throw({error, iolist_to_binary(DocErrorMsg)})
-        end,
-        Code = couch_util:get_nested_json_value(
-            Body, [<<"filters">>, FilterName]),
-        re:replace(Code, [$^, "\s*(.*?)\s*", $$], "\\1", [{return, binary}])
-    after
-        couch_replicator_api_wrap:db_close(Db)
-    end.
-
-
-maybe_append_options(Options, RepOptions) ->
-    lists:foldl(fun(Option, Acc) ->
-        Acc ++
-        case get_value(Option, RepOptions, false) of
-        true ->
-            "+" ++ atom_to_list(Option);
-        false ->
-            ""
-        end
-    end, [], Options).
-
-
-get_rep_endpoint(_UserCtx, #httpdb{url=Url, headers=Headers, oauth=OAuth}) ->
-    DefaultHeaders = (#httpdb{})#httpdb.headers,
-    case OAuth of
-    nil ->
-        {remote, Url, Headers -- DefaultHeaders};
-    #oauth{} ->
-        {remote, Url, Headers -- DefaultHeaders, OAuth}
-    end;
-get_rep_endpoint(UserCtx, <<DbName/binary>>) ->
-    {local, DbName, UserCtx}.
-
-
-parse_rep_db({Props}, ProxyParams, Options) ->
-    Url = maybe_add_trailing_slash(get_value(<<"url">>, Props)),
-    {AuthProps} = get_value(<<"auth">>, Props, {[]}),
-    {BinHeaders} = get_value(<<"headers">>, Props, {[]}),
-    Headers = lists:ukeysort(1, [{?b2l(K), ?b2l(V)} || {K, V} <- BinHeaders]),
-    DefaultHeaders = (#httpdb{})#httpdb.headers,
-    OAuth = case get_value(<<"oauth">>, AuthProps) of
-    undefined ->
-        nil;
-    {OauthProps} ->
-        #oauth{
-            consumer_key = ?b2l(get_value(<<"consumer_key">>, OauthProps)),
-            token = ?b2l(get_value(<<"token">>, OauthProps)),
-            token_secret = ?b2l(get_value(<<"token_secret">>, OauthProps)),
-            consumer_secret = ?b2l(get_value(<<"consumer_secret">>, OauthProps)),
-            signature_method =
-                case get_value(<<"signature_method">>, OauthProps) of
-                undefined ->        hmac_sha1;
-                <<"PLAINTEXT">> ->  plaintext;
-                <<"HMAC-SHA1">> ->  hmac_sha1;
-                <<"RSA-SHA1">> ->   rsa_sha1
-                end
-        }
-    end,
-    #httpdb{
-        url = Url,
-        oauth = OAuth,
-        headers = lists:ukeymerge(1, Headers, DefaultHeaders),
-        ibrowse_options = lists:keysort(1,
-            [{socket_options, get_value(socket_options, Options)} |
-                ProxyParams ++ ssl_params(Url)]),
-        timeout = get_value(connection_timeout, Options),
-        http_connections = get_value(http_connections, Options),
-        retries = get_value(retries, Options)
-    };
-parse_rep_db(<<"http://", _/binary>> = Url, ProxyParams, Options) ->
-    parse_rep_db({[{<<"url">>, Url}]}, ProxyParams, Options);
-parse_rep_db(<<"https://", _/binary>> = Url, ProxyParams, Options) ->
-    parse_rep_db({[{<<"url">>, Url}]}, ProxyParams, Options);
-parse_rep_db(<<DbName/binary>>, _ProxyParams, _Options) ->
-    DbName.
-
-
-maybe_add_trailing_slash(Url) when is_binary(Url) ->
-    maybe_add_trailing_slash(?b2l(Url));
-maybe_add_trailing_slash(Url) ->
-    case lists:last(Url) of
-    $/ ->
-        Url;
-    _ ->
-        Url ++ "/"
-    end.
-
-
-make_options(Props) ->
-    Options0 = lists:ukeysort(1, convert_options(Props)),
-    Options = check_options(Options0),
-    DefWorkers = config:get("replicator", "worker_processes", "4"),
-    DefBatchSize = config:get("replicator", "worker_batch_size", "500"),
-    DefConns = config:get("replicator", "http_connections", "20"),
-    DefTimeout = config:get("replicator", "connection_timeout", "30000"),
-    DefRetries = config:get("replicator", "retries_per_request", "10"),
-    UseCheckpoints = config:get("replicator", "use_checkpoints", "true"),
-    DefCheckpointInterval = config:get("replicator", "checkpoint_interval", "30000"),
-    {ok, DefSocketOptions} = couch_util:parse_term(
-        config:get("replicator", "socket_options",
-            "[{keepalive, true}, {nodelay, false}]")),
-    lists:ukeymerge(1, Options, lists:keysort(1, [
-        {connection_timeout, list_to_integer(DefTimeout)},
-        {retries, list_to_integer(DefRetries)},
-        {http_connections, list_to_integer(DefConns)},
-        {socket_options, DefSocketOptions},
-        {worker_batch_size, list_to_integer(DefBatchSize)},
-        {worker_processes, list_to_integer(DefWorkers)},
-        {use_checkpoints, list_to_existing_atom(UseCheckpoints)},
-        {checkpoint_interval, list_to_integer(DefCheckpointInterval)}
-    ])).
-
-
-convert_options([])->
-    [];
-convert_options([{<<"cancel">>, V} | _R]) when not is_boolean(V)->
-    throw({bad_request, <<"parameter `cancel` must be a boolean">>});
-convert_options([{<<"cancel">>, V} | R]) ->
-    [{cancel, V} | convert_options(R)];
-convert_options([{IdOpt, V} | R]) when IdOpt =:= <<"_local_id">>;
-        IdOpt =:= <<"replication_id">>; IdOpt =:= <<"id">> ->
-    Id = lists:splitwith(fun(X) -> X =/= $+ end, ?b2l(V)),
-    [{id, Id} | convert_options(R)];
-convert_options([{<<"create_target">>, V} | _R]) when not is_boolean(V)->
-    throw({bad_request, <<"parameter `create_target` must be a boolean">>});
-convert_options([{<<"create_target">>, V} | R]) ->
-    [{create_target, V} | convert_options(R)];
-convert_options([{<<"continuous">>, V} | _R]) when not is_boolean(V)->
-    throw({bad_request, <<"parameter `continuous` must be a boolean">>});
-convert_options([{<<"continuous">>, V} | R]) ->
-    [{continuous, V} | convert_options(R)];
-convert_options([{<<"filter">>, V} | R]) ->
-    [{filter, V} | convert_options(R)];
-convert_options([{<<"query_params">>, V} | R]) ->
-    [{query_params, V} | convert_options(R)];
-convert_options([{<<"doc_ids">>, null} | R]) ->
-    convert_options(R);
-convert_options([{<<"doc_ids">>, V} | _R]) when not is_list(V) ->
-    throw({bad_request, <<"parameter `doc_ids` must be an array">>});
-convert_options([{<<"doc_ids">>, V} | R]) ->
-    % Ensure same behaviour as old replicator: accept a list of percent
-    % encoded doc IDs.
-    DocIds = [?l2b(couch_httpd:unquote(Id)) || Id <- V],
-    [{doc_ids, DocIds} | convert_options(R)];
-convert_options([{<<"selector">>, V} | _R]) when not is_tuple(V) ->
-    throw({bad_request, <<"parameter `selector` must be a JSON object">>});
-convert_options([{<<"selector">>, V} | R]) ->
-    [{selector, V} | convert_options(R)];
-convert_options([{<<"worker_processes">>, V} | R]) ->
-    [{worker_processes, couch_util:to_integer(V)} | convert_options(R)];
-convert_options([{<<"worker_batch_size">>, V} | R]) ->
-    [{worker_batch_size, couch_util:to_integer(V)} | convert_options(R)];
-convert_options([{<<"http_connections">>, V} | R]) ->
-    [{http_connections, couch_util:to_integer(V)} | convert_options(R)];
-convert_options([{<<"connection_timeout">>, V} | R]) ->
-    [{connection_timeout, couch_util:to_integer(V)} | convert_options(R)];
-convert_options([{<<"retries_per_request">>, V} | R]) ->
-    [{retries, couch_util:to_integer(V)} | convert_options(R)];
-convert_options([{<<"socket_options">>, V} | R]) ->
-    {ok, SocketOptions} = couch_util:parse_term(V),
-    [{socket_options, SocketOptions} | convert_options(R)];
-convert_options([{<<"since_seq">>, V} | R]) ->
-    [{since_seq, V} | convert_options(R)];
-convert_options([{<<"use_checkpoints">>, V} | R]) ->
-    [{use_checkpoints, V} | convert_options(R)];
-convert_options([{<<"checkpoint_interval">>, V} | R]) ->
-    [{checkpoint_interval, couch_util:to_integer(V)} | convert_options(R)];
-convert_options([_ | R]) -> % skip unknown option
-    convert_options(R).
-
-check_options(Options) ->
-    DocIds = lists:keyfind(doc_ids, 1, Options),
-    Filter = lists:keyfind(filter, 1, Options),
-    Selector = lists:keyfind(selector, 1, Options),
-    case {DocIds, Filter, Selector} of
-        {false, false, false} -> Options;
-        {false, false, _} -> Options;
-        {false, _, false} -> Options;
-        {_, false, false} -> Options;
-        _ ->
-            throw({bad_request, "`doc_ids`, `filter`, `selector` are mutually exclusive options"})
-    end.
-
-
-parse_proxy_params(ProxyUrl) when is_binary(ProxyUrl) ->
-    parse_proxy_params(?b2l(ProxyUrl));
-parse_proxy_params([]) ->
-    [];
-parse_proxy_params(ProxyUrl) ->
-    #url{
-        host = Host,
-        port = Port,
-        username = User,
-        password = Passwd,
-        protocol = Protocol
-    } = ibrowse_lib:parse_url(ProxyUrl),
-    [{proxy_protocol, Protocol}, {proxy_host, Host}, {proxy_port, Port}] ++
-        case is_list(User) andalso is_list(Passwd) of
-        false ->
-            [];
-        true ->
-            [{proxy_user, User}, {proxy_password, Passwd}]
-        end.
-
-
-ssl_params(Url) ->
-    case ibrowse_lib:parse_url(Url) of
-    #url{protocol = https} ->
-        Depth = list_to_integer(
-            config:get("replicator", "ssl_certificate_max_depth", "3")
-        ),
-        VerifyCerts = config:get("replicator", "verify_ssl_certificates"),
-        CertFile = config:get("replicator", "cert_file", undefined),
-        KeyFile = config:get("replicator", "key_file", undefined),
-        Password = config:get("replicator", "password", undefined),
-        SslOpts = [{depth, Depth} | ssl_verify_options(VerifyCerts =:= "true")],
-        SslOpts1 = case CertFile /= undefined andalso KeyFile /= undefined of
-            true ->
-                case Password of
-                    undefined ->
-                        [{certfile, CertFile}, {keyfile, KeyFile}] ++ SslOpts;
-                    _ ->
-                        [{certfile, CertFile}, {keyfile, KeyFile},
-                            {password, Password}] ++ SslOpts
-                end;
-            false -> SslOpts
-        end,
-        [{is_ssl, true}, {ssl_options, SslOpts1}];
-    #url{protocol = http} ->
-        []
-    end.
-
-ssl_verify_options(Value) ->
-    ssl_verify_options(Value, erlang:system_info(otp_release)).
-
-ssl_verify_options(true, OTPVersion) when OTPVersion >= "R14" ->
-    CAFile = config:get("replicator", "ssl_trusted_certificates_file"),
-    [{verify, verify_peer}, {cacertfile, CAFile}];
-ssl_verify_options(false, OTPVersion) when OTPVersion >= "R14" ->
-    [{verify, verify_none}];
-ssl_verify_options(true, _OTPVersion) ->
-    CAFile = config:get("replicator", "ssl_trusted_certificates_file"),
-    [{verify, 2}, {cacertfile, CAFile}];
-ssl_verify_options(false, _OTPVersion) ->
-    [{verify, 0}].
-
-
-%% New db record has Options field removed here to enable smoother dbcore migration
 open_db(#db{name = Name, user_ctx = UserCtx}) ->
     {ok, Db} = couch_db:open(Name, [{user_ctx, UserCtx} | []]),
     Db;
@@ -444,103 +68,78 @@
 handle_db_event(_DbName, _Event, Server) ->
     {ok, Server}.
 
-% Obsolete - remove in next release
-sum_stats(S1, S2) ->
-    couch_replicator_stats:sum_stats(S1, S2).
 
+
+rep_error_to_binary(Error) ->
+    couch_util:to_binary(error_reason(Error)).
+
+
+error_reason({shutdown, Error}) ->
+    error_reason(Error);
+error_reason({error, {Error, Reason}})
+  when is_atom(Error), is_binary(Reason) ->
+    io_lib:format("~s: ~s", [Error, Reason]);
+error_reason({error, Reason}) ->
+    Reason;
+error_reason(Reason) ->
+    Reason.
+
+
+
+get_json_value(Key, Props) ->
+    get_json_value(Key, Props, undefined).
+
+get_json_value(Key, Props, Default) when is_atom(Key) ->
+    Ref = make_ref(),
+    case get_value(Key, Props, Ref) of
+        Ref ->
+            get_value(?l2b(atom_to_list(Key)), Props, Default);
+        Else ->
+            Else
+    end;
+get_json_value(Key, Props, Default) when is_binary(Key) ->
+    Ref = make_ref(),
+    case get_value(Key, Props, Ref) of
+        Ref ->
+            get_value(list_to_atom(?b2l(Key)), Props, Default);
+        Else ->
+            Else
+    end.
+
+
+% pretty-print replication id
+-spec pp_rep_id(#rep{} | rep_id()) -> string().
+pp_rep_id(#rep{id = RepId}) ->
+    pp_rep_id(RepId);
+pp_rep_id({Base, Extension}) ->
+    Base ++ Extension.
+
+
+% NV: TODO: this function is not used outside api wrap module
+% consider moving it there during final cleanup
 is_deleted(Change) ->
-    case couch_util:get_value(<<"deleted">>, Change) of
+    case get_value(<<"deleted">>, Change) of
     undefined ->
         % keep backwards compatibility for a while
-        couch_util:get_value(deleted, Change, false);
+        get_value(deleted, Change, false);
     Else ->
         Else
     end.
 
+% NV: TODO: proxy some functions which used to be here, later remove
+% these and replace calls to their respective modules
+replication_id(Rep, Version) ->
+    couch_replicator_ids:replication_id(Rep, Version).
 
-% Sort an EJSON object's properties to attempt
-% to generate a unique representation. This is used
-% to reduce the chance of getting different
-% replication checkpoints for the same Mango selector
-ejsort({V})->
-    ejsort_props(V, []);
-ejsort(V) when is_list(V) ->
-    ejsort_array(V, []);
-ejsort(V) ->
-    V.
+sum_stats(S1, S2) ->
+    couch_replicator_stats:sum_stats(S1, S2).
 
-ejsort_props([], Acc)->
-    {lists:keysort(1, Acc)};
-ejsort_props([{K, V}| R], Acc) ->
-    ejsort_props(R, [{K, ejsort(V)} | Acc]).
-
-ejsort_array([], Acc)->
-    lists:reverse(Acc);
-ejsort_array([V | R], Acc) ->
-    ejsort_array(R, [ejsort(V) | Acc]).
+parse_rep_doc(Props, UserCtx) ->
+    couch_replicator_docs:parse_rep_doc(Props, UserCtx).
 
 
--ifdef(TEST).
-
--include_lib("eunit/include/eunit.hrl").
-
-ejsort_basic_values_test() ->
-    ?assertEqual(ejsort(0), 0),
-    ?assertEqual(ejsort(<<"a">>), <<"a">>),
-    ?assertEqual(ejsort(true), true),
-    ?assertEqual(ejsort([]), []),
-    ?assertEqual(ejsort({[]}), {[]}).
-
-ejsort_compound_values_test() ->
-    ?assertEqual(ejsort([2, 1, 3 ,<<"a">>]), [2, 1, 3, <<"a">>]),
-    Ej1 = {[{<<"a">>, 0}, {<<"c">>, 0},  {<<"b">>, 0}]},
-    Ej1s =  {[{<<"a">>, 0}, {<<"b">>, 0}, {<<"c">>, 0}]},
-    ?assertEqual(ejsort(Ej1), Ej1s),
-    Ej2 = {[{<<"x">>, Ej1}, {<<"z">>, Ej1}, {<<"y">>, [Ej1, Ej1]}]},
-    ?assertEqual(ejsort(Ej2),
-        {[{<<"x">>, Ej1s}, {<<"y">>, [Ej1s, Ej1s]}, {<<"z">>, Ej1s}]}).
-
-check_options_pass_values_test() ->
-    ?assertEqual(check_options([]), []),
-    ?assertEqual(check_options([baz, {other,fiz}]), [baz, {other, fiz}]),
-    ?assertEqual(check_options([{doc_ids, x}]), [{doc_ids, x}]),
-    ?assertEqual(check_options([{filter, x}]), [{filter, x}]),
-    ?assertEqual(check_options([{selector, x}]), [{selector, x}]).
-
-check_options_fail_values_test() ->
-    ?assertThrow({bad_request, _},
-        check_options([{doc_ids, x}, {filter, y}])),
-    ?assertThrow({bad_request, _},
-        check_options([{doc_ids, x}, {selector, y}])),
-    ?assertThrow({bad_request, _},
-        check_options([{filter, x}, {selector, y}])),
-    ?assertThrow({bad_request, _},
-        check_options([{doc_ids, x}, {selector, y}, {filter, z}])).
-
-check_convert_options_pass_test() ->
-    ?assertEqual([], convert_options([])),
-    ?assertEqual([], convert_options([{<<"random">>, 42}])),
-    ?assertEqual([{cancel, true}],
-        convert_options([{<<"cancel">>, true}])),
-    ?assertEqual([{create_target, true}],
-        convert_options([{<<"create_target">>, true}])),
-    ?assertEqual([{continuous, true}],
-        convert_options([{<<"continuous">>, true}])),
-    ?assertEqual([{doc_ids, [<<"id">>]}],
-        convert_options([{<<"doc_ids">>, [<<"id">>]}])),
-    ?assertEqual([{selector, {key, value}}],
-        convert_options([{<<"selector">>, {key, value}}])).
-
-check_convert_options_fail_test() ->
-    ?assertThrow({bad_request, _},
-        convert_options([{<<"cancel">>, <<"true">>}])),
-    ?assertThrow({bad_request, _},
-        convert_options([{<<"create_target">>, <<"true">>}])),
-    ?assertThrow({bad_request, _},
-        convert_options([{<<"continuous">>, <<"true">>}])),
-    ?assertThrow({bad_request, _},
-        convert_options([{<<"doc_ids">>, not_a_list}])),
-    ?assertThrow({bad_request, _},
-        convert_options([{<<"selector">>, [{key, value}]}])).
-
--endif.
+-spec iso8601(erlang:timestamp()) -> binary().
+iso8601({_Mega, _Sec, _Micro} = Timestamp) ->
+    {{Y, Mon, D}, {H, Min, S}} = calendar:now_to_universal_time(Timestamp),
+    Format = "~B-~2..0B-~2..0BT~2..0B:~2..0B:~2..0BZ",
+    iolist_to_binary(io_lib:format(Format, [Y, Mon, D, H, Min, S])).
diff --git a/src/couch_replicator_worker.erl b/src/couch_replicator_worker.erl
index ee0c455..1907879 100644
--- a/src/couch_replicator_worker.erl
+++ b/src/couch_replicator_worker.erl
@@ -196,6 +196,9 @@
         {noreply, State2}
     end;
 
+handle_info({'EXIT', _Pid, max_backoff}, State) ->
+    {stop, {shutdown, max_backoff}, State};
+
 handle_info({'EXIT', Pid, Reason}, State) ->
    {stop, {process_died, Pid, Reason}, State}.
 
diff --git a/test/couch_replicator_compact_tests.erl b/test/couch_replicator_compact_tests.erl
index 7a5a25a..4dce69b 100644
--- a/test/couch_replicator_compact_tests.erl
+++ b/test/couch_replicator_compact_tests.erl
@@ -16,6 +16,10 @@
 -include_lib("couch/include/couch_db.hrl").
 -include_lib("couch_replicator/src/couch_replicator.hrl").
 
+-import(couch_replicator_test_helper, [
+    db_url/1
+]).
+
 -define(ATTFILE, filename:join([?FIXTURESDIR, "logo.png"])).
 -define(DELAY, 100).
 -define(TIMEOUT, 30000).
@@ -92,7 +96,7 @@
 should_ensure_replication_still_running(RepPid, RepId, Source, Target) ->
     ?_test(check_active_tasks(RepPid, RepId, Source, Target)).
 
-check_active_tasks(RepPid, {BaseId, Ext} = _RepId, Src, Tgt) ->
+check_active_tasks(RepPid, {BaseId, Ext} = RepId, Src, Tgt) ->
     Source = case Src of
         {remote, NameSrc} ->
             <<(db_url(NameSrc))/binary, $/>>;
@@ -107,7 +111,7 @@
     end,
     FullRepId = ?l2b(BaseId ++ Ext),
     Pid = ?l2b(pid_to_list(RepPid)),
-    ok = wait_for_replicator(RepPid),
+    ok = wait_for_replicator(RepId),
     [RepTask] = couch_task_status:all(),
     ?assertEqual(Pid, couch_util:get_value(pid, RepTask)),
     ?assertEqual(FullRepId, couch_util:get_value(replication_id, RepTask)),
@@ -124,16 +128,25 @@
     Pending = couch_util:get_value(changes_pending, RepTask),
     ?assert(is_integer(Pending)).
 
-wait_for_replicator(Pid) ->
+
+get_pid(RepId) ->
+    Pid = global:whereis_name({couch_replicator_scheduler_job,RepId}),
+    ?assert(is_pid(Pid)),
+    Pid.
+
+rep_details(RepId) ->
+    gen_server:call(get_pid(RepId), get_details).
+
+wait_for_replicator(RepId) ->
     %% since replicator started asynchronously
     %% we need to wait when it would be in couch_task_status
     %% we query replicator:details to ensure that do_init happen
-    ?assertMatch({ok, _}, couch_replicator:details(Pid)),
+    ?assertMatch({ok, _}, rep_details(RepId)),
     ok.
 
 should_cancel_replication(RepId, RepPid) ->
     ?_assertNot(begin
-        {ok, _} = couch_replicator:cancel_replication(RepId),
+        ok = couch_replicator_scheduler:remove_job(RepId),
         is_process_alive(RepPid)
     end).
 
@@ -295,13 +308,6 @@
                                          " database failed with: ", Reason])}]})
     end.
 
-db_url(DbName) ->
-    iolist_to_binary([
-        "http://", config:get("httpd", "bind_address", "127.0.0.1"),
-        ":", integer_to_list(mochiweb_socket_server:get(couch_httpd, port)),
-        "/", DbName
-    ]).
-
 replicate({remote, Db}, Target) ->
     replicate(db_url(Db), Target);
 
@@ -315,7 +321,9 @@
         {<<"continuous">>, true}
     ]},
     {ok, Rep} = couch_replicator_utils:parse_rep_doc(RepObject, ?ADMIN_USER),
-    {ok, Pid} = couch_replicator:async_replicate(Rep),
+    ok = couch_replicator_scheduler:add_job(Rep),
+    couch_replicator_scheduler:reschedule(),
+    Pid = get_pid(Rep#rep.id),
     {ok, Pid, Rep#rep.id}.
 
 
diff --git a/test/couch_replicator_connection_tests.erl b/test/couch_replicator_connection_tests.erl
new file mode 100644
index 0000000..323a7de
--- /dev/null
+++ b/test/couch_replicator_connection_tests.erl
@@ -0,0 +1,128 @@
+% 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_replicator_connection_tests).
+
+-include_lib("couch/include/couch_eunit.hrl").
+-include_lib("couch/include/couch_db.hrl").
+
+-define(TIMEOUT, 1000).
+
+
+setup() ->
+    Host = config:get("httpd", "bind_address", "127.0.0.1"),
+    Port = config:get("httpd", "port", "5984"),
+    {Host, Port}.
+
+teardown(_) ->
+    ok.
+
+
+httpc_pool_test_() ->
+    {
+        "replicator connection sharing tests",
+        {
+            setup,
+            fun() -> test_util:start_couch([couch_replicator]) end, fun test_util:stop_couch/1,
+            {
+                foreach,
+                fun setup/0, fun teardown/1,
+                [
+                    fun connections_shared_after_relinquish/1,
+                    fun connections_not_shared_after_owner_death/1,
+                    fun idle_connections_closed/1,
+                    fun test_owner_monitors/1
+                ]
+            }
+        }
+    }.
+
+
+connections_shared_after_relinquish({Host, Port}) ->
+    ?_test(begin
+        URL = "http://" ++ Host ++ ":" ++ Port,
+        Self = self(),
+        {ok, Pid} = couch_replicator_connection:acquire(URL),
+        couch_replicator_connection:relinquish(Pid),
+        spawn(fun() ->
+            Self ! couch_replicator_connection:acquire(URL)
+        end),
+        receive
+            {ok, Pid2} ->
+                ?assertEqual(Pid, Pid2)
+        end
+    end).
+
+
+connections_not_shared_after_owner_death({Host, Port}) ->
+    ?_test(begin
+        URL = "http://" ++ Host ++ ":" ++ Port,
+        Self = self(),
+        spawn(fun() ->
+            Self ! couch_replicator_connection:acquire(URL),
+            1/0
+        end),
+        receive
+            {ok, Pid} ->
+                {ok, Pid2} = couch_replicator_connection:acquire(URL),
+                ?assertNotEqual(Pid, Pid2),
+                MRef = monitor(process, Pid),
+                receive {'DOWN', MRef, process, Pid, _Reason} ->
+                    ?assert(not is_process_alive(Pid));
+                    Other -> throw(Other)
+                end
+        end
+    end).
+
+
+idle_connections_closed({Host, Port}) ->
+    ?_test(begin
+        URL = "http://" ++ Host ++ ":" ++ Port,
+        {ok, Pid} = couch_replicator_connection:acquire(URL),
+        couch_replicator_connection ! close_idle_connections,
+        ?assert(ets:member(couch_replicator_connection, Pid)),
+        % block until idle connections have closed
+        sys:get_status(couch_replicator_connection),
+        couch_replicator_connection:relinquish(Pid),
+        couch_replicator_connection ! close_idle_connections,
+        % block until idle connections have closed
+        sys:get_status(couch_replicator_connection),
+        ?assert(not ets:member(couch_replicator_connection, Pid))
+    end).
+
+
+test_owner_monitors({Host, Port}) ->
+    ?_test(begin
+        URL = "http://" ++ Host ++ ":" ++ Port,
+        {ok, Worker0} = couch_replicator_connection:acquire(URL),
+        assert_monitors_equal([{process, self()}]),
+        couch_replicator_connection:relinquish(Worker0),
+        assert_monitors_equal([]),
+        {Workers, Monitors}  = lists:foldl(fun(_, {WAcc, MAcc}) ->
+            {ok, Worker1} = couch_replicator_connection:acquire(URL),
+            MAcc1 = [{process, self()} | MAcc],
+            assert_monitors_equal(MAcc1),
+            {[Worker1 | WAcc], MAcc1}
+        end, {[], []}, lists:seq(1,5)),
+        lists:foldl(fun(Worker2, Acc) ->
+            [_ | NewAcc] = Acc,
+            couch_replicator_connection:relinquish(Worker2),
+            assert_monitors_equal(NewAcc),
+            NewAcc
+        end, Monitors, Workers)
+    end).
+
+
+assert_monitors_equal(ShouldBe) ->
+    sys:get_status(couch_replicator_connection),
+    {monitors, Monitors} = process_info(whereis(couch_replicator_connection), monitors),
+    ?assertEqual(Monitors, ShouldBe).
diff --git a/test/couch_replicator_httpc_pool_tests.erl b/test/couch_replicator_httpc_pool_tests.erl
index ea36f7f..c4ad4e9 100644
--- a/test/couch_replicator_httpc_pool_tests.erl
+++ b/test/couch_replicator_httpc_pool_tests.erl
@@ -30,7 +30,7 @@
         "httpc pool tests",
         {
             setup,
-            fun test_util:start_couch/0, fun test_util:stop_couch/1,
+            fun() -> test_util:start_couch([couch_replicator]) end, fun test_util:stop_couch/1,
             {
                 foreach,
                 fun setup/0, fun teardown/1,
diff --git a/test/couch_replicator_many_leaves_tests.erl b/test/couch_replicator_many_leaves_tests.erl
index bde0e2c..a6999bd 100644
--- a/test/couch_replicator_many_leaves_tests.erl
+++ b/test/couch_replicator_many_leaves_tests.erl
@@ -15,6 +15,11 @@
 -include_lib("couch/include/couch_eunit.hrl").
 -include_lib("couch/include/couch_db.hrl").
 
+-import(couch_replicator_test_helper, [
+    db_url/1,
+    replicate/2
+]).
+
 -define(DOCS_CONFLICTS, [
     {<<"doc1">>, 10},
     {<<"doc2">>, 100},
@@ -199,22 +204,3 @@
     ?assertEqual(length(NewDocs), length(NewRevs)),
     add_attachments(SourceDb, NumAtts, Rest).
 
-db_url(DbName) ->
-    iolist_to_binary([
-        "http://", config:get("httpd", "bind_address", "127.0.0.1"),
-        ":", integer_to_list(mochiweb_socket_server:get(couch_httpd, port)),
-        "/", DbName
-    ]).
-
-replicate(Source, Target) ->
-    RepObject = {[
-        {<<"source">>, Source},
-        {<<"target">>, Target}
-    ]},
-    {ok, Rep} = couch_replicator_utils:parse_rep_doc(RepObject, ?ADMIN_USER),
-    {ok, Pid} = couch_replicator:async_replicate(Rep),
-    MonRef = erlang:monitor(process, Pid),
-    receive
-        {'DOWN', MonRef, process, Pid, _} ->
-            ok
-    end.
diff --git a/test/couch_replicator_modules_load_tests.erl b/test/couch_replicator_modules_load_tests.erl
index 96a9346..a552d14 100644
--- a/test/couch_replicator_modules_load_tests.erl
+++ b/test/couch_replicator_modules_load_tests.erl
@@ -28,13 +28,18 @@
         couch_replicator_httpc,
         couch_replicator_httpd,
         couch_replicator_manager,
+        couch_replicator_scheduler,
+        couch_replicator_scheduler_job,
+        couch_replicator_docs,
+        couch_replicator_clustering,
+        couch_replicator_changes_reader,
+        couch_replicator_ids,
         couch_replicator_notifier,
         couch_replicator,
         couch_replicator_worker,
-        couch_replicator_utils,
-        couch_replicator_job_sup
+        couch_replicator_utils
     ],
     [should_load_module(Mod) || Mod <- Modules].
 
 should_load_module(Mod) ->
-    {atom_to_list(Mod), ?_assertMatch({module, _}, code:load_file(Mod))}.
+    {atom_to_list(Mod), ?_assertMatch({module, _}, code:ensure_loaded(Mod))}.
diff --git a/test/couch_replicator_proxy_tests.erl b/test/couch_replicator_proxy_tests.erl
new file mode 100644
index 0000000..a40e5b1
--- /dev/null
+++ b/test/couch_replicator_proxy_tests.erl
@@ -0,0 +1,69 @@
+% 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_replicator_proxy_tests).
+
+-include_lib("couch/include/couch_eunit.hrl").
+-include_lib("couch_replicator/src/couch_replicator.hrl").
+-include_lib("couch_replicator/src/couch_replicator_api_wrap.hrl").
+
+
+setup() ->
+    ok.
+
+
+teardown(_) ->
+    ok.
+
+
+replicator_proxy_test_() ->
+    {
+        "replicator proxy tests",
+        {
+            setup,
+            fun() -> test_util:start_couch([couch_replicator]) end, fun test_util:stop_couch/1,
+            {
+                foreach,
+                fun setup/0, fun teardown/1,
+                [
+                    fun parse_rep_doc_without_proxy/1,
+                    fun parse_rep_doc_with_proxy/1
+                ]
+            }
+        }
+    }.
+
+
+parse_rep_doc_without_proxy(_) ->
+    ?_test(begin
+        NoProxyDoc = {[
+            {<<"source">>, <<"http://unproxied.com">>},
+            {<<"target">>, <<"http://otherunproxied.com">>}
+        ]},
+        Rep = couch_replicator_docs:parse_rep_doc(NoProxyDoc),
+        ?assertEqual((Rep#rep.source)#httpdb.proxy_url, undefined),
+        ?assertEqual((Rep#rep.target)#httpdb.proxy_url, undefined)
+    end).
+
+
+parse_rep_doc_with_proxy(_) ->
+    ?_test(begin
+        ProxyURL = <<"http://myproxy.com">>,
+        ProxyDoc = {[
+            {<<"source">>, <<"http://unproxied.com">>},
+            {<<"target">>, <<"http://otherunproxied.com">>},
+            {<<"proxy">>, ProxyURL}
+        ]},
+        Rep = couch_replicator_docs:parse_rep_doc(ProxyDoc),
+        ?assertEqual((Rep#rep.source)#httpdb.proxy_url, binary_to_list(ProxyURL)),
+        ?assertEqual((Rep#rep.target)#httpdb.proxy_url, binary_to_list(ProxyURL))
+    end).
diff --git a/test/couch_replicator_rate_limiter_tests.erl b/test/couch_replicator_rate_limiter_tests.erl
new file mode 100644
index 0000000..034550a
--- /dev/null
+++ b/test/couch_replicator_rate_limiter_tests.erl
@@ -0,0 +1,89 @@
+-module(couch_replicator_rate_limiter_tests).
+
+-include_lib("couch/include/couch_eunit.hrl").
+
+
+rate_limiter_test_() ->
+    {
+        foreach,
+        fun setup/0,
+        fun teardown/1,
+        [
+            t_new_key(),
+            t_1_failure(),
+            t_2_failures_back_to_back(),
+            t_2_failures(),
+            t_success_threshold(),
+            t_1_failure_2_successes()
+        ]
+    }.
+
+
+t_new_key() ->
+    ?_test(begin
+        ?assertEqual(0, couch_replicator_rate_limiter:interval({"foo", get}))
+    end).
+
+
+t_1_failure() ->
+    ?_test(begin
+        ?assertEqual(24, couch_replicator_rate_limiter:failure({"foo", get}))
+    end).
+
+
+t_2_failures() ->
+    ?_test(begin
+        couch_replicator_rate_limiter:failure({"foo", get}),
+        low_pass_filter_delay(),
+        Interval = couch_replicator_rate_limiter:failure({"foo", get}),
+        ?assertEqual(29, Interval)
+    end).
+
+
+t_2_failures_back_to_back() ->
+    ?_test(begin
+        couch_replicator_rate_limiter:failure({"foo", get}),
+        Interval = couch_replicator_rate_limiter:failure({"foo", get}),
+        ?assertEqual(24, Interval)
+    end).
+
+
+t_success_threshold() ->
+    ?_test(begin
+        Interval = couch_replicator_rate_limiter:success({"foo", get}),
+        ?assertEqual(0, Interval),
+        Interval = couch_replicator_rate_limiter:success({"foo", get}),
+        ?assertEqual(0, Interval)
+    end).
+
+
+t_1_failure_2_successes() ->
+    ?_test(begin
+        couch_replicator_rate_limiter:failure({"foo", get}),
+        low_pass_filter_delay(),
+        Succ1 = couch_replicator_rate_limiter:success({"foo", get}),
+        ?assertEqual(20, Succ1),
+        low_pass_filter_delay(),
+        Succ2 = couch_replicator_rate_limiter:success({"foo", get}),
+        ?assertEqual(0, Succ2)
+    end).
+
+
+low_pass_filter_delay() ->
+    timer:sleep(100).
+
+
+setup() ->
+    {ok, Pid} = couch_replicator_rate_limiter:start_link(),
+    Pid.
+
+
+teardown(Pid) ->
+    Ref = erlang:monitor(process, Pid),
+    unlink(Pid),
+    exit(Pid, kill),
+    receive
+        {'DOWN', Ref, process, Pid, _} ->
+            ok
+    end,
+    ok.
diff --git a/test/couch_replicator_test_helper.erl b/test/couch_replicator_test_helper.erl
index 398b27b..f52ab11 100644
--- a/test/couch_replicator_test_helper.erl
+++ b/test/couch_replicator_test_helper.erl
@@ -2,7 +2,7 @@
 
 -include_lib("couch/include/couch_eunit.hrl").
 -include_lib("couch/include/couch_db.hrl").
-
+-include_lib("couch_replicator/src/couch_replicator.hrl").
 -export([compare_dbs/2, compare_dbs/3, db_url/1, replicate/1, replicate/2]).
 
 
@@ -103,6 +103,11 @@
         "/", DbName
     ]).
 
+get_pid(RepId) ->
+    Pid = global:whereis_name({couch_replicator_scheduler_job,RepId}),
+    ?assert(is_pid(Pid)),
+    Pid.
+
 replicate(Source, Target) ->
     replicate({[
         {<<"source">>, Source},
@@ -111,9 +116,12 @@
 
 replicate({[_ | _]} = RepObject) ->
     {ok, Rep} = couch_replicator_utils:parse_rep_doc(RepObject, ?ADMIN_USER),
-    {ok, Pid} = couch_replicator:async_replicate(Rep),
+    ok = couch_replicator_scheduler:add_job(Rep),
+    couch_replicator_scheduler:reschedule(),
+    Pid = get_pid(Rep#rep.id),
     MonRef = erlang:monitor(process, Pid),
     receive
         {'DOWN', MonRef, process, Pid, _} ->
             ok
-    end.
+    end,
+    ok = couch_replicator_scheduler:remove_job(Rep#rep.id).
diff --git a/test/couch_replicator_use_checkpoints_tests.erl b/test/couch_replicator_use_checkpoints_tests.erl
index e04488e..73ea7f1 100644
--- a/test/couch_replicator_use_checkpoints_tests.erl
+++ b/test/couch_replicator_use_checkpoints_tests.erl
@@ -15,6 +15,11 @@
 -include_lib("couch/include/couch_eunit.hrl").
 -include_lib("couch/include/couch_db.hrl").
 
+-import(couch_replicator_test_helper, [
+    db_url/1,
+    replicate/1
+]).
+
 -define(DOCS_COUNT, 100).
 -define(TIMEOUT_EUNIT, 30).
 -define(i2l(I), integer_to_list(I)).
@@ -167,23 +172,10 @@
     ok = couch_db:close(SourceDb),
     ok = couch_db:close(TargetDb).
 
-db_url(DbName) ->
-    iolist_to_binary([
-        "http://", config:get("httpd", "bind_address", "127.0.0.1"),
-        ":", integer_to_list(mochiweb_socket_server:get(couch_httpd, port)),
-        "/", DbName
-    ]).
-
 replicate(Source, Target, UseCheckpoints) ->
-    RepObject = {[
+    replicate({[
         {<<"source">>, Source},
         {<<"target">>, Target},
         {<<"use_checkpoints">>, UseCheckpoints}
-    ]},
-    {ok, Rep} = couch_replicator_utils:parse_rep_doc(RepObject, ?ADMIN_USER),
-    {ok, Pid} = couch_replicator:async_replicate(Rep),
-    MonRef = erlang:monitor(process, Pid),
-    receive
-        {'DOWN', MonRef, process, Pid, _} ->
-            ok
-    end.
+    ]}).
+