You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@couchdb.apache.org by va...@apache.org on 2020/09/15 20:13:57 UTC

[couchdb] branch prototype/fdb-layer updated (8cd1792 -> ae85819)

This is an automated email from the ASF dual-hosted git repository.

vatamane pushed a change to branch prototype/fdb-layer
in repository https://gitbox.apache.org/repos/asf/couchdb.git.


    from 8cd1792  Move error reporting test to EUnit
     new 36a6b39  Add after_db_create/2 and after_db_delete/2 callbacks to fabric
     new e3b1c41  Read attachment data outside the transaction
     new 9897cd8  Handle possible iodata from jiffy:encode in couch_jobs
     new d2c9dff  Add fold_jobs/4 and pending_count/2,3 to couch_jobs API
     new 4fc9a53  Delete old 2.x-3.x replicator modules
     new b6e87f8  Cleanup couch_replicator_utils module
     new b38d77f  Move parsing and validation to couch_replicator_parse module
     new 941cfc3  Handle option maps in lower level modules
     new 276d197  Update couch_replicator_ids
     new 3c9b754  Introduce couch_replicator_jobs abstraction module
     new 5b98e8a  Update frontend replicator modules
     new 7dd1899  Update backend replicator modules
     new aa88364  Update replicator http handlers and supervisor
     new b718d33  Update and cleanup default.ini replicator entries
     new 9926290  Update replicator's readme file
     new ae85819  Update and clean up tests

The 16 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 Makefile                                           |    2 +-
 rel/apps/couch_epi.config                          |    1 +
 rel/overlay/etc/default.ini                        |   78 +-
 src/chttpd/src/chttpd_db.erl                       |   26 +-
 src/chttpd/src/chttpd_misc.erl                     |   55 +-
 src/couch/src/couch_att.erl                        |   20 +-
 src/couch_jobs/src/couch_jobs.erl                  |   37 +-
 src/couch_jobs/src/couch_jobs_fdb.erl              |    4 +-
 src/couch_jobs/src/couch_jobs_pending.erl          |   32 +-
 src/couch_replicator/README.md                     |  312 +---
 src/couch_replicator/priv/stats_descriptions.cfg   |   44 +-
 src/couch_replicator/src/couch_replicator.app.src  |   11 +-
 src/couch_replicator/src/couch_replicator.erl      |  716 +++++----
 src/couch_replicator/src/couch_replicator.hrl      |  102 +-
 .../src/couch_replicator_api_wrap.erl              |  164 +-
 .../src/couch_replicator_changes_reader.erl        |    9 +-
 .../src/couch_replicator_clustering.erl            |  279 ----
 .../src/couch_replicator_db_changes.erl            |  108 --
 .../src/couch_replicator_doc_processor.erl         |  962 -----------
 .../src/couch_replicator_doc_processor_worker.erl  |  284 ----
 src/couch_replicator/src/couch_replicator_docs.erl |  870 ++--------
 .../src/couch_replicator_epi.erl}                  |   15 +-
 .../src/couch_replicator_fabric.erl                |  155 --
 .../src/couch_replicator_fabric2_plugin.erl}       |   20 +-
 .../src/couch_replicator_fabric_rpc.erl            |   97 --
 .../src/couch_replicator_filters.erl               |   54 +-
 .../src/couch_replicator_httpc.erl                 |    2 +-
 .../src/couch_replicator_httpd.erl                 |  163 +-
 .../src/couch_replicator_httpd_util.erl            |  201 ---
 src/couch_replicator/src/couch_replicator_ids.erl  |  202 ++-
 src/couch_replicator/src/couch_replicator_job.erl  | 1612 +++++++++++++++++++
 .../src/couch_replicator_job_server.erl            |  370 +++++
 .../src/couch_replicator_job_sup.erl               |   34 -
 src/couch_replicator/src/couch_replicator_jobs.erl |  312 ++++
 .../src/couch_replicator_js_functions.hrl          |  177 --
 .../src/couch_replicator_notifier.erl              |   58 -
 .../src/couch_replicator_parse.erl                 |  545 +++++++
 .../src/couch_replicator_scheduler.erl             | 1688 --------------------
 .../src/couch_replicator_scheduler.hrl             |   15 -
 .../src/couch_replicator_scheduler_job.erl         | 1090 -------------
 .../src/couch_replicator_scheduler_sup.erl         |   62 -
 .../src/couch_replicator_stats.erl                 |    2 +
 src/couch_replicator/src/couch_replicator_sup.erl  |  113 +-
 .../src/couch_replicator_utils.erl                 |  241 +--
 .../src/couch_replicator_worker.erl                |    2 +-
 .../couch_replicator_attachments_too_large.erl     |   90 +-
 .../test/eunit/couch_replicator_compact_tests.erl  |  455 ------
 .../eunit/couch_replicator_connection_tests.erl    |  274 ++--
 ...replicator_create_target_with_options_tests.erl |  129 +-
 .../test/eunit/couch_replicator_db_tests.erl       |  332 ++++
 .../couch_replicator_error_reporting_tests.erl     |  271 ----
 .../test/eunit/couch_replicator_filtered_tests.erl |  348 ++--
 .../eunit/couch_replicator_httpc_pool_tests.erl    |  125 +-
 .../eunit/couch_replicator_id_too_long_tests.erl   |   91 +-
 .../eunit/couch_replicator_job_server_tests.erl    |  437 +++++
 .../eunit/couch_replicator_large_atts_tests.erl    |  123 +-
 .../eunit/couch_replicator_many_leaves_tests.erl   |  241 ++-
 .../eunit/couch_replicator_missing_stubs_tests.erl |  179 +--
 .../test/eunit/couch_replicator_proxy_tests.erl    |  135 +-
 .../eunit/couch_replicator_rate_limiter_tests.erl  |   77 +-
 ...ch_replicator_retain_stats_between_job_runs.erl |  223 +--
 .../test/eunit/couch_replicator_selector_tests.erl |  136 +-
 ...ch_replicator_small_max_request_size_target.erl |  190 +--
 .../test/eunit/couch_replicator_test_helper.erl    |  323 +++-
 .../couch_replicator_transient_jobs_tests.erl      |  106 ++
 .../couch_replicator_use_checkpoints_tests.erl     |  207 +--
 src/fabric/include/fabric2.hrl                     |    1 +
 src/fabric/src/fabric2_db.erl                      |   15 +-
 src/fabric/src/fabric2_db_plugin.erl               |   10 +
 test/elixir/test/replication_test.exs              |    5 +-
 70 files changed, 6531 insertions(+), 9338 deletions(-)
 delete mode 100644 src/couch_replicator/src/couch_replicator_clustering.erl
 delete mode 100644 src/couch_replicator/src/couch_replicator_db_changes.erl
 delete mode 100644 src/couch_replicator/src/couch_replicator_doc_processor.erl
 delete mode 100644 src/couch_replicator/src/couch_replicator_doc_processor_worker.erl
 copy src/{setup/src/setup_epi.erl => couch_replicator/src/couch_replicator_epi.erl} (88%)
 delete mode 100644 src/couch_replicator/src/couch_replicator_fabric.erl
 copy src/{couch_views/src/couch_views_fabric2_plugin.erl => couch_replicator/src/couch_replicator_fabric2_plugin.erl} (54%)
 delete mode 100644 src/couch_replicator/src/couch_replicator_fabric_rpc.erl
 delete mode 100644 src/couch_replicator/src/couch_replicator_httpd_util.erl
 create mode 100644 src/couch_replicator/src/couch_replicator_job.erl
 create mode 100644 src/couch_replicator/src/couch_replicator_job_server.erl
 delete mode 100644 src/couch_replicator/src/couch_replicator_job_sup.erl
 create mode 100644 src/couch_replicator/src/couch_replicator_jobs.erl
 delete mode 100644 src/couch_replicator/src/couch_replicator_js_functions.hrl
 delete mode 100644 src/couch_replicator/src/couch_replicator_notifier.erl
 create mode 100644 src/couch_replicator/src/couch_replicator_parse.erl
 delete mode 100644 src/couch_replicator/src/couch_replicator_scheduler.erl
 delete mode 100644 src/couch_replicator/src/couch_replicator_scheduler.hrl
 delete mode 100644 src/couch_replicator/src/couch_replicator_scheduler_job.erl
 delete mode 100644 src/couch_replicator/src/couch_replicator_scheduler_sup.erl
 delete mode 100644 src/couch_replicator/test/eunit/couch_replicator_compact_tests.erl
 create mode 100644 src/couch_replicator/test/eunit/couch_replicator_db_tests.erl
 delete mode 100644 src/couch_replicator/test/eunit/couch_replicator_error_reporting_tests.erl
 create mode 100644 src/couch_replicator/test/eunit/couch_replicator_job_server_tests.erl
 create mode 100644 src/couch_replicator/test/eunit/couch_replicator_transient_jobs_tests.erl


[couchdb] 12/16: Update backend replicator modules

Posted by va...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

vatamane pushed a commit to branch prototype/fdb-layer
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit 7dd18993f25b107db05a9e99e01770c7f84bd33b
Author: Nick Vatamaniuc <va...@apache.org>
AuthorDate: Fri Aug 28 04:35:07 2020 -0400

    Update backend replicator modules
    
    Backend replicator modules execute replication jobs. The two main modules
    reponsible for job management and execution are `couch_replicator_job` and
    `couch_replicator_job_server`.
    
    `couch_replicator_job`
    
     - Is the main process of each replication job. When this process starts, it
       waits in the `couch_jobs:accept/2` call. This may take an indefinite amount
       of time. The spawned `couch_replicator_job` waiting in accept like that is
       called internally an "acceptor". The main pattern of execution is multiple
       acceptors are started, and after some of them accept jobs, they become
       "workers".
    
     - After it accepts a job, it parses the `couch_jobs` job data, which contains
       the `Rep` object and calculates the replication ID from it. Replication ID
       calculation may involve making a call to the source endpoint in order to
       fetch the contents of the javascript filter. Then, the `Rep` object and the
       replication ID is used to construct the internal `#rep_state{}` state record
       of the `gen_server`.
    
     - Multiple replication jobs may end up trying to run the same replication
       (with the same replication ID) concurrently. To manage these types of
       colisions, `check_ownership/3` function is called to determine if the
       current replication is the correct `owner` of that replication. If it is
       not, then the job maybe fail and exit.
    
     - There is a periodic checkpoint timer which sends a `checkpoint` message. The
       checkpoint frequency is calculated as the minimum of the `couch_jobs`
       activity timeout and the configured checkpoint interval. During each
       checkpoint attempt, there is a call to `couch_jobs:update/3` which updates
       the job's data with latest state and ensure the job doesn't get re-enqueued
       due to inactivity.
    
     - If the job completes, then `couch_jobs:finish/3` is called and the
       replication process exits `normal`. If the job crashes, there is a
       consecutive error count field (`?ERROR_COUNT`) which, is used to calculate
       the backoff penalty. There is an exponential backoff schedule, that starts
       with the base value, then doubles, but only up to a maximum value. Both the
       base and the maximum values are configurable with the
       `min_backoff_penalty_sec` and `max_backoff_penalty_sec` settings
       respecively. This is an improvement from before where the users could only
       influence the maximum backoff penalty by reducing the number of failed
       events kept by each job.
    
    `couch_replicator_server`
    
     - This is a module which spawns and keeps track of `couch_replicator_job`
       processes.
    
     - Periodically, every `interval_sec` seconds, it runs the `reschedule` function
       which checks for pending jobs. If they are some, it will start acceptors to
       run them. If those acceptors become workers, and if the total number of
       workers goes above the `max_jobs` setting, the oldest `continuous` workers
       will be stopped until the total number of jobs falls below `max_jobs` value.
    
     - In addition to `max_jobs` limit, there is a `max_churn` limit which
       determines up to how many job starts to allow during each scheduling
       interval. As jobs are started, they reduce the available churn "budget" for
       that cycle and after it goes below 0 no more jobs can start until the next
       cycle.
    
     - This module also performs transient job cleanup. After transient jobs stop
       running previously they simply vanished but with this update they maybe
       linger for at least `transient_job_max_age_sec` seconds.
---
 src/couch_replicator/priv/stats_descriptions.cfg   |   44 +-
 .../src/couch_replicator_httpc.erl                 |    2 +-
 src/couch_replicator/src/couch_replicator_job.erl  | 1612 ++++++++++++++++++++
 .../src/couch_replicator_job_server.erl            |  370 +++++
 4 files changed, 1999 insertions(+), 29 deletions(-)

diff --git a/src/couch_replicator/priv/stats_descriptions.cfg b/src/couch_replicator/priv/stats_descriptions.cfg
index d9efb91..1bb151c 100644
--- a/src/couch_replicator/priv/stats_descriptions.cfg
+++ b/src/couch_replicator/priv/stats_descriptions.cfg
@@ -54,14 +54,6 @@
     {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 times replicator db scans have been started">>}
-]}.
 {[couch_replicator, docs, dbs_created], [
     {type, counter},
     {desc, <<"number of db shard creations seen by replicator doc processor">>}
@@ -70,10 +62,6 @@
     {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">>}
@@ -88,43 +76,43 @@
 ]}.
 {[couch_replicator, jobs, adds], [
     {type, counter},
-    {desc, <<"number of jobs added to replicator scheduler">>}
+    {desc, <<"number of jobs added to replicator">>}
 ]}.
-{[couch_replicator, jobs, duplicate_adds], [
+{[couch_replicator, jobs, removes], [
     {type, counter},
-    {desc, <<"number of duplicate jobs added to replicator scheduler">>}
+    {desc, <<"number of jobs removed from replicator">>}
 ]}.
-{[couch_replicator, jobs, removes], [
+{[couch_replicator, jobs, accepts], [
     {type, counter},
-    {desc, <<"number of jobs removed from replicator scheduler">>}
+    {desc, <<"number of job acceptors started">>}
 ]}.
 {[couch_replicator, jobs, starts], [
     {type, counter},
-    {desc, <<"number of jobs started by replicator scheduler">>}
+    {desc, <<"number of jobs started by replicator">>}
 ]}.
 {[couch_replicator, jobs, stops], [
     {type, counter},
-    {desc, <<"number of jobs stopped by replicator scheduler">>}
+    {desc, <<"number of jobs stopped by replicator">>}
 ]}.
 {[couch_replicator, jobs, crashes], [
     {type, counter},
-    {desc, <<"number of job crashed noticed by replicator scheduler">>}
+    {desc, <<"number of job crashed noticed by replicator">>}
 ]}.
 {[couch_replicator, jobs, running], [
     {type, gauge},
-    {desc, <<"replicator scheduler running jobs">>}
+    {desc, <<"replicator running jobs">>}
 ]}.
-{[couch_replicator, jobs, pending], [
+{[couch_replicator, jobs, accepting], [
     {type, gauge},
-    {desc, <<"replicator scheduler pending jobs">>}
+    {desc, <<"replicator acceptors count">>}
 ]}.
-{[couch_replicator, jobs, crashed], [
-    {type, gauge},
-    {desc, <<"replicator scheduler crashed jobs">>}
+{[couch_replicator, jobs, reschedules], [
+    {type, counter},
+    {desc, <<"replicator reschedule cycles counter">>}
 ]}.
-{[couch_replicator, jobs, total], [
+{[couch_replicator, jobs, pending], [
     {type, gauge},
-    {desc, <<"total number of replicator scheduler jobs">>}
+    {desc, <<"replicator pending count">>}
 ]}.
 {[couch_replicator, connection, acquires], [
     {type, counter},
diff --git a/src/couch_replicator/src/couch_replicator_httpc.erl b/src/couch_replicator/src/couch_replicator_httpc.erl
index 4dce319..f11d189 100644
--- a/src/couch_replicator/src/couch_replicator_httpc.erl
+++ b/src/couch_replicator/src/couch_replicator_httpc.erl
@@ -327,7 +327,7 @@ 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(),
+    HealthThresholdSec = couch_replicator_job: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.
diff --git a/src/couch_replicator/src/couch_replicator_job.erl b/src/couch_replicator/src/couch_replicator_job.erl
new file mode 100644
index 0000000..ed3d00d
--- /dev/null
+++ b/src/couch_replicator/src/couch_replicator_job.erl
@@ -0,0 +1,1612 @@
+% 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_job).
+
+
+-behaviour(gen_server).
+
+
+-export([
+    start_link/0
+]).
+
+-export([
+    init/1,
+    terminate/2,
+    handle_call/3,
+    handle_cast/2,
+    handle_info/2,
+    format_status/2,
+    code_change/3
+]).
+
+-export([
+    accept/0,
+    health_threshold/0
+]).
+
+
+-include_lib("couch/include/couch_db.hrl").
+-include_lib("couch_replicator/include/couch_replicator_api_wrap.hrl").
+-include("couch_replicator.hrl").
+
+
+-define(LOWEST_SEQ, 0).
+-define(DEFAULT_CHECKPOINT_INTERVAL, 30000).
+-define(STARTUP_JITTER_DEFAULT, 5000).
+-define(DEFAULT_MIN_BACKOFF_PENALTY_SEC, 32).
+-define(DEFAULT_MAX_BACKOFF_PENALTY_SEC, 2 * 24 * 3600).
+-define(DEFAULT_HEALTH_THRESHOLD_SEC, 2 * 60).
+-define(DEFAULT_MAX_HISTORY, 10).
+-define(DEFAULT_STATS_UPDATE_INTERVAL_SEC, 10).
+
+
+-record(rep_state, {
+    job,
+    job_data,
+    id,
+    base_id,
+    doc_id,
+    db_name,
+    db_uuid,
+    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,
+    checkpoint_timer,
+    stats_timer,
+    changes_queue,
+    changes_manager,
+    changes_reader,
+    workers,
+    stats = couch_replicator_stats:new(),
+    session_id,
+    source_seq = nil,
+    use_checkpoints = true,
+    checkpoint_interval = ?DEFAULT_CHECKPOINT_INTERVAL,
+    user = null,
+    options = #{}
+}).
+
+
+start_link() ->
+    gen_server:start_link(?MODULE, [], []).
+
+
+init(_) ->
+    process_flag(trap_exit, true),
+    {ok, delayed_init, 0}.
+
+
+terminate(normal, #rep_state{} = State) ->
+    #rep_state{
+        job = Job,
+        job_data = JobData,
+        checkpoint_history = History
+    } = State,
+    ok = complete_job(undefined, Job, JobData, History),
+    close_endpoints(State);
+
+terminate(shutdown, #rep_state{} = State0) ->
+    % Replication stopped by the job server
+    State1 = cancel_timers(State0),
+    State3 = case do_checkpoint(State1) of
+        {ok, State2} ->
+            State2;
+        Error ->
+            Msg = "~p : Failed last checkpoint. Job: ~p Error: ~p",
+            couch_log:error(Msg, [?MODULE, State1#rep_state.id, Error]),
+            State1
+    end,
+    #rep_state{job = Job, job_data = JobData} = State3,
+    ok = reschedule(undefined, Job, JobData),
+    ok = close_endpoints(State3);
+
+terminate({shutdown, Error}, {init_error, Stack}) ->
+    % Termination in init, before the job had initialized
+    case Error of
+        max_backoff -> couch_log:warning("~p job backed off", [?MODULE]);
+        finished -> couch_log:notice("~p job finished in init", [?MODULE]);
+        _ -> couch_log:error("~p job failed ~p ~p", [?MODULE, Error, Stack])
+    end,
+    ok;
+
+terminate({shutdown, finished}, #rep_state{} = State) ->
+    % Job state was already updated and job is marked as finished
+    ok = close_endpoints(State);
+
+terminate({shutdown, halt}, #rep_state{} = State) ->
+    % Job is re-enqueued and possibly already running somewhere else
+    couch_log:error("~p job ~p halted", [?MODULE, State#rep_state.id]),
+    ok = close_endpoints(State);
+
+terminate(Reason0, #rep_state{} = State0) ->
+    State = update_job_state(State0),
+    Reason = case Reason0 of
+        {shutdown, Err} -> Err;
+        _ -> Reason0
+    end,
+    #rep_state{
+        id = RepId,
+        job = Job,
+        job_data = JobData,
+        source_name = Source,
+        target_name = Target
+    } = State,
+    couch_log:error("Replication `~s` (`~s` -> `~s`) failed: ~p",
+        [RepId, Source, Target, Reason]),
+    ok = reschedule_on_error(undefined, Job, JobData, Reason),
+    ok = close_endpoints(State).
+
+
+handle_call({add_stats, Stats}, From, State) ->
+    gen_server:reply(From, ok),
+    NewStats = couch_replicator_stats:sum_stats(State#rep_state.stats, Stats),
+    {noreply, State#rep_state{stats = NewStats}};
+
+handle_call({report_seq_done, Seq, StatsInc}, From, #rep_state{} = State) ->
+    #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_stats:sum_stats(Stats, StatsInc),
+        current_through_seq = NewThroughSeq,
+        seqs_in_progress = NewSeqsInProgress,
+        highest_seq_done = NewHighestDone
+    },
+    {noreply, maybe_update_job_state(NewState)};
+
+handle_call(Msg, _From, St) ->
+    {stop, {bad_call, Msg}, {bad_call, Msg}, St}.
+
+
+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}};
+
+handle_cast(Msg, St) ->
+    {stop, {bad_cast, Msg}, St}.
+
+
+handle_info(timeout, delayed_init) ->
+    try delayed_init() of
+        {ok, State} -> {noreply, State};
+        {stop, Reason, State} -> {stop, Reason, State}
+    catch
+        exit:{shutdown, Exit} when Exit =:= finished orelse Exit =:= halt ->
+            Stack = erlang:get_stacktrace(),
+            {stop, {shutdown, Exit}, {init_error, Stack}};
+        _Tag:Error ->
+            ShutdownReason = {error, replication_start_error(Error)},
+            Stack = erlang:get_stacktrace(),
+            {stop, {shutdown, ShutdownReason}, {init_error, Stack}}
+    end;
+
+handle_info(stats_update, #rep_state{} = State) ->
+    State1 = cancel_stats_timer(State),
+    State2 = update_job_state(State1),
+    {noreply, State2};
+
+handle_info(checkpoint, State0) ->
+    State = cancel_checkpoint_timer(State0),
+    ok = check_user_filter(State),
+    case do_checkpoint(State) of
+        {ok, State1} ->
+            couch_stats:increment_counter([couch_replicator, checkpoints,
+                success]),
+            {noreply, start_checkpoint_timer(State1)};
+        Error ->
+            couch_stats:increment_counter([couch_replicator, checkpoints,
+                failure]),
+            {stop, Error, State}
+    end;
+
+handle_info(shutdown, St) ->
+    {stop, shutdown, 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, Reason0}, #rep_state{changes_reader=Pid} = State) ->
+    couch_stats:increment_counter([couch_replicator, changes_reader_deaths]),
+    Reason = case Reason0 of
+        {changes_req_failed, _, _} = HttpFail ->
+            HttpFail;
+        {http_request_failed, _, _, {error, {code, Code}}} ->
+            {changes_req_failed, Code};
+        {http_request_failed, _, _, {error, Err}} ->
+            {changes_req_failed, Err};
+        Other ->
+            {changes_reader_died, Other}
+    end,
+    couch_log:error("ChangesReader process died with reason: ~p", [Reason]),
+    {stop, {shutdown, Reason}, cancel_timers(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, {shutdown, {changes_manager_died, Reason}}, cancel_timers(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, {shutdown, {changes_queue_died, Reason}}, cancel_timers(State)};
+
+handle_info({'EXIT', Pid, normal}, #rep_state{workers = Workers} = State) ->
+    case Workers -- [Pid] of
+        Workers ->
+            %% Processes might be linked by replicator's auth plugins so
+            %% we tolerate them exiting `normal` here and don't crash
+            LogMsg = "~p: unknown pid exited `normal` ~p",
+            couch_log:error(LogMsg, [?MODULE, Pid]),
+            {noreply, State#rep_state{workers = Workers}};
+        [] ->
+            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_timers(State),
+    case lists:member(Pid, Workers) of
+        false ->
+            {stop, {unknown_process_died, Pid, Reason}, State2};
+        true ->
+            couch_stats:increment_counter([couch_replicator, worker_deaths]),
+            StopReason = case Reason of
+                {shutdown, _} = Err ->
+                    Err;
+                Other ->
+                    ErrLog = "Worker ~p died with reason: ~p",
+                    couch_log:error(ErrLog, [Pid, Reason]),
+                    {worker_died, Pid, Other}
+            end,
+            {stop, StopReason, State2}
+    end;
+
+handle_info({Ref, ready}, St) when is_reference(Ref) ->
+    LogMsg = "~p : spurious erlfdb future ready message ~p",
+    couch_log:notice(LogMsg, [?MODULE, Ref]),
+    {noreply, St};
+
+handle_info(Msg, St) ->
+    {stop, {bad_info, Msg}, St}.
+
+
+format_status(_Opt, [_PDict, State]) ->
+    #rep_state{
+        id = Id,
+        source = Source,
+        target = Target,
+        start_seq = StartSeq,
+        source_seq = SourceSeq,
+        committed_seq = CommitedSeq,
+        current_through_seq = ThroughSeq,
+        highest_seq_done = HighestSeqDone,
+        session_id = SessionId,
+        doc_id = DocId,
+        db_name = DbName,
+        options = Options
+    } = state_strip_creds(State),
+    [
+        {rep_id, Id},
+        {source, couch_replicator_api_wrap:db_uri(Source)},
+        {target, couch_replicator_api_wrap:db_uri(Target)},
+        {db_name, DbName},
+        {doc_id, DocId},
+        {options, Options},
+        {session_id, SessionId},
+        {start_seq, StartSeq},
+        {source_seq, SourceSeq},
+        {committed_seq, CommitedSeq},
+        {current_through_seq, ThroughSeq},
+        {highest_seq_done, HighestSeqDone}
+    ].
+
+
+code_change(_OldVsn, #rep_state{}=State, _Extra) ->
+    {ok, State}.
+
+
+accept() ->
+    couch_stats:increment_counter([couch_replicator, jobs, accepts]),
+    Now = erlang:system_time(second),
+    case couch_replicator_jobs:accept_job(Now + 5) of
+        {ok, Job, #{?REP := Rep} = JobData} ->
+            Normal = case Rep of
+                #{?OPTIONS := #{} = Options} ->
+                    not maps:get(<<"continuous">>, Options, false);
+                _ ->
+                    true
+            end,
+            couch_replicator_job_server:accepted(self(), Normal),
+            {ok, Job, JobData};
+        {error, not_found} ->
+            timer:sleep(accept_jitter_msec()),
+            ?MODULE:accept()
+    end.
+
+
+% Health threshold is the minimum amount of time an unhealthy job should run
+% crashing before it is considered to be healthy again. HealtThreashold should
+% not be 0 as jobs could start and immediately crash, and it shouldn't be
+% infinity, since then  consecutive crashes would accumulate forever even if
+% job is back to normal.
+health_threshold() ->
+    config:get_integer("replicator", "health_threshold_sec",
+        ?DEFAULT_HEALTH_THRESHOLD_SEC).
+
+
+delayed_init() ->
+    {ok, Job, JobData} = accept(),
+    try do_init(Job, JobData) of
+        State = #rep_state{} -> {ok, State}
+    catch
+        exit:{http_request_failed, _, _, max_backoff} ->
+            Stack = erlang:get_stacktrace(),
+            reschedule_on_error(undefined, Job, JobData, max_backoff),
+            {stop, {shutdown, max_backoff}, {init_error, Stack}};
+        exit:{shutdown, Exit} when Exit =:= finished orelse Exit =:= halt ->
+            Stack = erlang:get_stacktrace(),
+            {stop, {shutdown, Exit}, {init_error, Stack}};
+        _Tag:Error ->
+            Reason = {error, replication_start_error(Error)},
+            Stack = erlang:get_stacktrace(),
+            ErrMsg = "~p : job ~p failed during startup ~p stack:~p",
+            couch_log:error(ErrMsg, [?MODULE, Job, Reason, Stack]),
+            reschedule_on_error(undefined, Job, JobData, Reason),
+            {stop, {shutdown, Reason}, {init_error, Stack}}
+    end.
+
+
+do_init(Job, #{} = JobData) ->
+    couch_stats:increment_counter([couch_replicator, jobs, starts]),
+    % This may make a network request, then may fail and reschedule the job
+    {RepId, BaseId} = get_rep_id(undefined, Job, JobData),
+    #{
+        ?DB_NAME := DbName,
+        ?DB_UUID := DbUUID,
+        ?DOC_ID := DocId
+    } = JobData,
+
+    ok = couch_replicator_docs:remove_state_fields(DbName, DbUUID, DocId),
+
+    % Finish if job is in a failed state already
+    case JobData of
+        #{?STATE := ?ST_FAILED, ?STATE_INFO := Error} ->
+            ok = fail_job(undefined, Job, JobData, Error),
+            exit({shutdown, finished});
+        #{?STATE := St} when is_binary(St), St =/= ?ST_FAILED ->
+            ok
+    end,
+
+    JobsTx = couch_jobs_fdb:get_jtx(),
+    {Job1, JobData1, Owner} = couch_jobs_fdb:tx(JobsTx, fun(JTx) ->
+        init_job_data(JTx, Job, JobData, RepId, BaseId)
+    end),
+
+    % Handle ownership decision here to be outside of the transaction
+    case Owner of
+        owner -> ok;
+        not_owner -> exit({shutdown, finished})
+    end,
+
+    #rep_state{
+        source = Source,
+        target = Target,
+        start_seq = {_Ts, StartSeq},
+        options = Options,
+        doc_id = DocId,
+        db_name = DbName
+    } = State = init_state(Job1, JobData1),
+
+    NumWorkers = maps:get(<<"worker_processes">>, Options),
+    BatchSize = maps:get(<<"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 = maps:get(<<"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)),
+
+    log_replication_start(State),
+
+    State1 = State#rep_state{
+            changes_queue = ChangesQueue,
+            changes_manager = ChangesManager,
+            changes_reader = ChangesReader,
+            workers = Workers
+    },
+
+    update_job_state(State1).
+
+
+init_job_data(#{jtx := true} = JTx, Job, #{?REP_ID := RepId} = JobData, RepId,
+        _BaseId) ->
+    {Job, JobData, check_ownership(JTx, Job, JobData)};
+
+init_job_data(#{jtx := true} = JTx, Job, #{} = JobData, RepId, BaseId) ->
+    #{
+        ?REP := Rep,
+        ?REP_ID := OldRepId,
+        ?DB_UUID := DbUUID,
+        ?DOC_ID := DocId
+    } = JobData,
+    JobId = couch_replicator_ids:job_id(Rep, DbUUID, DocId),
+    Now = erlang:system_time(second),
+    JobData1 = JobData#{
+        ?REP_ID := RepId,
+        ?BASE_ID := BaseId,
+        ?STATE := ?ST_RUNNING,
+        ?STATE_INFO := null,
+        ?LAST_START := Now,
+        ?REP_NODE := erlang:atom_to_binary(node(), utf8),
+        ?REP_PID := list_to_binary(pid_to_list(self())),
+        ?LAST_UPDATED := Now
+    },
+    JobData2 = case is_binary(OldRepId) andalso OldRepId =/= RepId of
+        true ->
+            % Handle Replication ID change
+            ok = couch_replicator_jobs:clear_old_rep_id(JTx, JobId, OldRepId),
+            JobData1#{
+                ?REP_STATS := #{},
+                ?JOB_HISTORY := []
+            };
+        false ->
+            JobData1
+    end,
+    JobData3 = hist_append(?HIST_STARTED, Now, JobData2, undefined),
+    case check_ownership(JTx, Job, JobData3) of
+        owner ->
+            couch_stats:increment_counter([couch_replicator, jobs, starts]),
+            {Job1, JobData4} = update_job_data(JTx, Job, JobData3),
+            {Job1, JobData4, owner};
+        not_owner ->
+            {Job, JobData3, not_owner}
+    end.
+
+
+check_ownership(#{jtx := true} = JTx, Job, JobData) ->
+    #{
+        ?REP_ID := RepId,
+        ?REP := Rep,
+        ?DB_UUID := DbUUID,
+        ?DOC_ID := DocId
+    } = JobData,
+    JobId = couch_replicator_ids:job_id(Rep, DbUUID, DocId),
+    case couch_replicator_jobs:try_update_rep_id(JTx, JobId, RepId) of
+        ok ->
+            owner;
+        {error, {replication_job_conflict, OtherJobId}} ->
+            case couch_replicator_jobs:get_job_data(JTx, OtherJobId) of
+                {ok, #{?STATE := S, ?DB_NAME := null}} when
+                        S == ?ST_RUNNING; S == ?ST_PENDING ->
+                    % Conflicting job is a transient job, not associated with a
+                    % _replicator doc, so we let this job retry. This is also
+                    % partly done for compatibility with pervious replicator
+                    % behavior.
+                    Error = <<"Duplicate job running: ", OtherJobId/binary>>,
+                    reschedule_on_error(JTx, Job, JobData, Error),
+                    not_owner;
+                {ok, #{?STATE := S, ?DB_NAME := <<_/binary>>}} when
+                        S == ?ST_RUNNING; S == ?ST_PENDING ->
+                    % Conflicting job is a permanent replication job, so this
+                    % job is marked as failed.
+                    Error = <<"Duplicate job running: ", OtherJobId/binary>>,
+                    fail_job(JTx, Job, JobData, Error),
+                    not_owner;
+                {ok, #{}} ->
+                    LogMsg = "~p : Job ~p usurping job ~p for replication ~p",
+                    couch_log:warning(LogMsg, [?MODULE, JobId, OtherJobId,
+                        RepId]),
+                    couch_replicator_jobs:update_rep_id(JTx, JobId, RepId),
+                    owner;
+                {error, not_found} ->
+                    LogMsg = "~p : Orphan replication job reference ~p -> ~p",
+                    couch_log:error(LogMsg, [?MODULE, RepId, OtherJobId]),
+                    couch_replicator_jobs:update_rep_id(JTx, JobId, RepId),
+                    owner
+            end
+    end.
+
+
+update_job_data(Tx, #rep_state{} = State) ->
+    #rep_state{job = Job, job_data = JobData} = State,
+    {Job1, JobData1} = update_job_data(Tx, Job, JobData),
+    State#rep_state{job = Job1, job_data = JobData1}.
+
+
+update_job_data(Tx, Job, #{} = JobData) ->
+    case couch_replicator_jobs:update_job_data(Tx, Job, JobData) of
+        {ok, Job1} ->
+            {Job1, JobData};
+        {error, halt} ->
+            exit({shutdown, halt})
+    end.
+
+
+update_active_task_info(#rep_state{} = State) ->
+    #rep_state{
+        job_data = JobData,
+        user = User,
+        id = RepId,
+        db_name = DbName,
+        doc_id = DocId,
+        source_name = Source,
+        target_name = Target,
+        options = Options,
+        highest_seq_done = {_, SourceSeq},
+        checkpoint_interval = CheckpointInterval
+    } = State,
+
+    #{
+        ?REP := #{?START_TIME := StartTime},
+        ?REP_STATS := Stats,
+        ?REP_NODE := Node,
+        ?REP_PID := Pid,
+        ?LAST_UPDATED := LastUpdated
+    } = JobData,
+
+    Info = maps:merge(Stats, #{
+        <<"type">> => <<"replication">>,
+        <<"user">> => User,
+        <<"replication_id">> => RepId,
+        <<"database">> => DbName,
+        <<"doc_id">> => DocId,
+        <<"source">> => ?l2b(Source),
+        <<"target">> => ?l2b(Target),
+        <<"continuous">> => maps:get(<<"continuous">>, Options, false),
+        <<"source_seq">> => SourceSeq,
+        <<"checkpoint_interval">> => CheckpointInterval,
+        <<"node">> => Node,
+        <<"pid">> => Pid,
+        <<"updated_on">> => LastUpdated,
+        <<"started_on">> => StartTime
+    }),
+
+    JobData1 = fabric2_active_tasks:update_active_task_info(JobData, Info),
+    State#rep_state{job_data = JobData1}.
+
+
+% Transient jobs don't get rescheduled on error with the exception of
+% max_backoff errors.
+%
+reschedule_on_error(JTx, Job, #{?DB_NAME := null} = JobData, Error) when
+        Error =/= max_backoff ->
+    fail_job(JTx, Job, JobData, Error);
+
+reschedule_on_error(JTx, Job, #{} = JobData0, Error0) ->
+    Error = error_info(Error0),
+
+    Now = erlang:system_time(second),
+
+    JobData = maybe_heal(JobData0, Now),
+    #{?ERROR_COUNT := ErrorCount} = JobData,
+    JobData1 = JobData#{
+        ?STATE := ?ST_CRASHING,
+        ?STATE_INFO := Error,
+        ?ERROR_COUNT := ErrorCount + 1,
+        ?LAST_ERROR := Error,
+        ?REP_NODE := null,
+        ?REP_PID := null
+    },
+    JobData2 = hist_append(?HIST_CRASHED, Now, JobData1, Error),
+    JobData3 = hist_append(?HIST_PENDING, Now, JobData2, undefined),
+    JobData4 = fabric2_active_tasks:update_active_task_info(JobData3, #{}),
+
+    couch_stats:increment_counter([couch_replicator, jobs, crashes]),
+
+    Time = get_backoff_time(ErrorCount + 1),
+    case couch_replicator_jobs:reschedule_job(JTx, Job, JobData4, Time) of
+        ok -> ok;
+        {error, halt} -> exit({shutdown, halt})
+    end.
+
+
+reschedule(JTx, Job, #{} = JobData) ->
+    Now = erlang:system_time(second),
+
+    JobData1 = JobData#{
+        ?STATE := ?ST_PENDING,
+        ?STATE_INFO := null,
+        ?LAST_ERROR := null,
+        ?REP_NODE := null,
+        ?REP_PID := null
+    },
+    JobData2 = hist_append(?HIST_STOPPED, Now, JobData1, undefined),
+    JobData3 = hist_append(?HIST_PENDING, Now, JobData2, undefined),
+    JobData4 = fabric2_active_tasks:update_active_task_info(JobData3, #{}),
+
+    couch_stats:increment_counter([couch_replicator, jobs, stops]),
+
+    Time = Now + couch_replicator_job_server:scheduling_interval_sec(),
+    case couch_replicator_jobs:reschedule_job(JTx, Job, JobData4, Time) of
+        ok -> ok;
+        {error, halt} -> exit({shutdown, halt})
+    end.
+
+
+fail_job(JTx, Job, #{} = JobData, Error0) ->
+    Error = error_info(Error0),
+
+    Now = erlang:system_time(second),
+
+    #{
+        ?ERROR_COUNT := ErrorCount,
+        ?DB_NAME := DbName,
+        ?DB_UUID := DbUUID,
+        ?DOC_ID := DocId
+    } = JobData,
+
+    JobData1 = JobData#{
+        ?STATE := ?ST_FAILED,
+        ?STATE_INFO := Error,
+        ?ERROR_COUNT := ErrorCount + 1,
+        ?REP_NODE := null,
+        ?REP_PID := null
+    },
+    JobData2 = hist_append(?HIST_CRASHED, Now, JobData1, Error),
+    JobData3 = fabric2_active_tasks:update_active_task_info(JobData2, #{}),
+
+    couch_stats:increment_counter([couch_replicator, jobs, crashes]),
+
+    case couch_replicator_jobs:finish_job(JTx, Job, JobData3) of
+        ok ->
+            couch_replicator_docs:update_failed(DbName, DbUUID, DocId, Error),
+            ok;
+        {error, halt} ->
+            exit({shutdown, halt})
+    end.
+
+
+complete_job(JTx, Job, #{} = JobData, CheckpointHistory) ->
+    #{
+        ?DB_NAME := Db,
+        ?DB_UUID := DbUUID,
+        ?DOC_ID := DocId,
+        ?REP_STATS := RepStats,
+        ?REP := Rep
+    } = JobData,
+
+    Now = erlang:system_time(second),
+
+    #{?START_TIME := StartTime} = Rep,
+    JobData1 = JobData#{
+        ?STATE := ?ST_COMPLETED,
+        ?CHECKPOINT_HISTORY := CheckpointHistory,
+        ?STATE_INFO := RepStats,
+        ?REP_NODE := null,
+        ?REP_PID := null
+    },
+    JobData2 = hist_append(?HIST_STOPPED, Now, JobData1, undefined),
+    JobData3 = fabric2_active_tasks:update_active_task_info(JobData2, #{}),
+
+    couch_stats:increment_counter([couch_replicator, jobs, stops]),
+
+    case couch_replicator_jobs:finish_job(JTx, Job, JobData3) of
+        ok ->
+            StartISO8601 = couch_replicator_utils:iso8601(StartTime),
+            Stats = maps:merge(RepStats, #{<<"start_time">> => StartISO8601}),
+            couch_replicator_docs:update_completed(Db, DbUUID, DocId, Stats),
+            ok;
+        {error, halt} ->
+            exit({shutdown, halt})
+    end.
+
+
+error_info(Error0) ->
+    case Error0 of
+        <<_/binary>> ->
+            Error0;
+        undefined ->
+            undefined;
+        null ->
+            null;
+        Atom when is_atom(Atom) ->
+            atom_to_binary(Atom, utf8);
+        {shutdown, Atom} when is_atom(Atom) ->
+            atom_to_binary(Atom, utf8);
+        {shutdown, Err} ->
+            couch_replicator_utils:rep_error_to_binary(Err);
+        {error, Atom} when is_atom(Atom) ->
+            atom_to_binary(Atom, utf8);
+        {error, {Err, Reason}} when is_atom(Err) ->
+            ReasonBin = couch_replicator_utils:rep_error_to_binary(Reason),
+            #{
+                <<"error">> => atom_to_binary(Err, utf8),
+                <<"reason">> => ReasonBin
+            };
+        _Other ->
+            couch_replicator_utils:rep_error_to_binary(Error0)
+    end.
+
+
+get_rep_id(JTx, Job, #{} = JobData) ->
+    #{?REP := Rep} = JobData,
+    try
+        couch_replicator_ids:replication_id(Rep)
+    catch
+        throw:{filter_fetch_error, Error} ->
+            Error1 = io_lib:format("Filter fetch error ~p", [Error]),
+            reschedule_on_error(JTx, Job, JobData, Error1),
+            exit({shutdown, finished})
+    end.
+
+
+% After job run continuously for some time we consider it "healed" and reset
+% its consecutive error count.
+maybe_heal(#{} = JobData, Now) ->
+    #{?LAST_START := LastStart} = JobData,
+    case Now - LastStart > health_threshold() of
+        true -> JobData#{?ERROR_COUNT := 0, ?LAST_ERROR := null};
+        false -> JobData
+    end.
+
+
+get_backoff_time(ErrCnt) ->
+    Max = min(max_backoff_penalty_sec(), 3600 * 24 * 30),
+    Min = max(min_backoff_penalty_sec(), 2),
+
+    % Calculate the max exponent so exponentiation doesn't blow up
+    MaxExp = math:log2(Max) - math:log2(Min),
+
+    % This is the recommended backoff amount
+    Wait = Min * math:pow(2, min(ErrCnt, MaxExp)),
+
+    % Apply a 25% jitter to avoid a thundering herd effect
+    WaitJittered = Wait * 0.75 + rand:uniform(trunc(Wait * 0.25) + 1),
+    erlang:system_time(second) + trunc(WaitJittered).
+
+
+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.
+
+
+state_strip_creds(#rep_state{source = Source, target = Target} = State) ->
+    State#rep_state{
+        source = httpdb_strip_creds(Source),
+        target = httpdb_strip_creds(Target)
+    }.
+
+
+adjust_maxconn(Src = #{<<"http_connections">> := 1}, RepId) ->
+    Msg = "Adjusting minimum number of HTTP source connections to 2 for ~p",
+    couch_log:notice(Msg, [RepId]),
+    Src#{<<"http_connections">> := 2};
+
+adjust_maxconn(Src, _RepId) ->
+    Src.
+
+
+do_last_checkpoint(#rep_state{seqs_in_progress = [],
+        highest_seq_done = {_Ts, ?LOWEST_SEQ}} = State) ->
+    {stop, normal, cancel_timers(State)};
+
+do_last_checkpoint(#rep_state{seqs_in_progress = [],
+        highest_seq_done = Seq} = State) ->
+    State1 = State#rep_state{current_through_seq = Seq},
+    State2 = cancel_timers(State1),
+    case do_checkpoint(State2) of
+        {ok, State3} ->
+            couch_stats:increment_counter([couch_replicator, checkpoints,
+                success]),
+            {stop, normal, State3};
+        Error ->
+            couch_stats:increment_counter([couch_replicator, checkpoints,
+                failure]),
+            {stop, Error, State2}
+    end.
+
+
+start_checkpoint_timer(#rep_state{} = State) ->
+    CheckpointAfterMSec = State#rep_state.checkpoint_interval,
+    JobTimeoutMSec = couch_replicator_jobs:get_timeout() * 1000,
+    Wait1 = min(CheckpointAfterMSec, JobTimeoutMSec div 2),
+    Wait2 = trunc(Wait1 * 0.75) + rand:uniform(trunc(Wait1 * 0.25)),
+    TRef = erlang:send_after(Wait2, self(), checkpoint),
+    State#rep_state{checkpoint_timer = TRef}.
+
+
+cancel_checkpoint_timer(#rep_state{checkpoint_timer = nil} = State) ->
+    State;
+cancel_checkpoint_timer(#rep_state{checkpoint_timer = Timer} = State) ->
+    erlang:cancel_timer(Timer),
+    State#rep_state{checkpoint_timer = nil}.
+
+
+start_stats_timer(#rep_state{} = State) ->
+    MSec = stats_update_interval_sec() * 1000,
+    TRef = erlang:send_after(MSec, self(), stats_update),
+    State#rep_state{stats_timer = TRef}.
+
+
+cancel_stats_timer(#rep_state{stats_timer = nil} = State) ->
+    State;
+cancel_stats_timer(#rep_state{stats_timer = Timer} = State) ->
+    erlang:cancel_timer(Timer),
+    receive stats_update -> ok after 0 -> ok end,
+    State#rep_state{stats_timer = nil}.
+
+
+cancel_timers(#rep_state{} = State) ->
+    State1 = cancel_checkpoint_timer(State),
+    cancel_stats_timer(State1).
+
+
+init_state(#{} = Job, #{} = JobData) ->
+    #{
+        ?REP := Rep,
+        ?REP_ID := Id,
+        ?BASE_ID := BaseId,
+        ?DB_NAME := DbName,
+        ?DB_UUID := DbUUID,
+        ?DOC_ID := DocId,
+        ?LAST_ERROR := LastError
+    } = JobData,
+    #{
+        ?SOURCE := Src0,
+        ?TARGET := Tgt,
+        ?START_TIME := StartTime,
+        ?OPTIONS := Options0
+    } = Rep,
+
+    % Optimize replication parameters if last time the jobs crashed because it
+    % was rate limited
+    Options = optimize_rate_limited_job(Options0, LastError),
+
+    % 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),
+    CreateTgt = maps:get(<<"create_target">>, Options, false),
+    TParams = maps:get(<<"create_target_params">>, Options, #{}),
+
+    {ok, Target} = couch_replicator_api_wrap:db_open(Tgt, CreateTgt, TParams),
+
+    {ok, SourceInfo} = couch_replicator_api_wrap:get_db_info(Source),
+    {ok, TargetInfo} = couch_replicator_api_wrap:get_db_info(Target),
+
+    [SourceLog, TargetLog] = find_and_migrate_logs([Source, Target], Rep,
+        BaseId),
+
+    {StartSeq0, History} = compare_replication_logs(SourceLog, TargetLog),
+
+    #{?REP_STATS := Stats0} = JobData,
+    Stats1 = couch_replicator_stats:new(Stats0),
+    HistoryStats = case History of
+        [{[_ | _] = HProps} | _] -> couch_replicator_stats:new(HProps);
+        _ -> couch_replicator_stats:new()
+    end,
+    Stats2 = couch_replicator_stats:max_stats(Stats1, HistoryStats),
+
+    StartSeq1 = maps:get(<<"since_seq">>, Options, StartSeq0),
+    StartSeq = {0, StartSeq1},
+
+    SourceSeq = get_value(<<"update_seq">>, SourceInfo, ?LOWEST_SEQ),
+
+    #doc{body={CheckpointHistory}} = SourceLog,
+
+    State = #rep_state{
+        job = Job,
+        job_data = JobData,
+        id = Id,
+        base_id = BaseId,
+        source_name = couch_replicator_api_wrap:db_uri(Source),
+        target_name = couch_replicator_api_wrap:db_uri(Target),
+        source = Source,
+        target = Target,
+        options = Options,
+        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_seq = SourceSeq,
+        use_checkpoints = maps:get(<<"use_checkpoints">>, Options),
+        checkpoint_interval = maps:get(<<"checkpoint_interval">>, Options),
+        stats = Stats2,
+        stats_timer = nil,
+        doc_id = DocId,
+        db_name = DbName,
+        db_uuid = DbUUID
+    },
+    start_checkpoint_timer(State).
+
+
+find_and_migrate_logs(DbList, #{} = Rep, BaseId) when is_binary(BaseId) ->
+    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_ids:base_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},
+            maybe_save_migrated_log(Rep, Db, MigratedLog, Doc#doc.id),
+            fold_replication_logs(Rest, ?REP_ID_VERSION, NewId, NewId, Rep,
+                [MigratedLog | Acc])
+    end.
+
+
+maybe_save_migrated_log(#{?OPTIONS := Options}, Db, #doc{} = Doc, OldId) ->
+    case maps:get(<<"use_checkpoints">>, Options) of
+        true ->
+            update_checkpoint(Db, Doc),
+            Msg = "Migrated replication checkpoint. Db:~p ~p -> ~p",
+            couch_log:notice(Msg, [httpdb_strip_creds(Db), OldId, Doc#doc.id]);
+        false ->
+            ok
+    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, ChangesOrLastSeqs} ->
+                    ReportSeq = case lists:last(ChangesOrLastSeqs) of
+                        {last_seq, Seq} -> {Ts, Seq};
+                        #doc_info{high_seq = Seq} -> {Ts, Seq}
+                    end,
+                    Changes = lists:filter(fun
+                        (#doc_info{}) -> true;
+                        ({last_seq, _Seq}) -> false
+                    end, ChangesOrLastSeqs),
+                    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, update_job_state(NewState)};
+do_checkpoint(#rep_state{current_through_seq=S, committed_seq=S} = State) ->
+    {ok, update_job_state(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 = RepStartTime,
+        src_starttime = SrcInstanceStartTime,
+        tgt_starttime = TgtInstanceStartTime,
+        stats = Stats,
+        options = Options,
+        session_id = SessionId
+    } = State,
+    case commit_to_both(Source, Target) of
+        {source_error, Reason} ->
+            {checkpoint_commit_failure, <<"Failure on source commit: ",
+                (couch_util:to_binary(Reason))/binary>>};
+        {target_error, Reason} ->
+            {checkpoint_commit_failure, <<"Failure on target commit: ",
+                (couch_util:to_binary(Reason))/binary>>};
+        {SrcInstanceStartTime, TgtInstanceStartTime} ->
+            couch_log:notice("recording a checkpoint for `~s` -> `~s` at "
+                "source update_seq ~p", [SourceName, TargetName, NewSeq]),
+            StartTime = couch_replicator_utils:rfc1123_local(RepStartTime),
+            EndTime = couch_replicator_utils:rfc1123_local(),
+            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 maps:get(<<"doc_ids">>, Options, undefined) 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]}}
+                },
+                {ok, update_job_state(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 ",
+            (couch_util:to_binary(DbType))/binary, " checkpoint document: ",
+            (couch_util: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 ->
+        Opts = [ejson_body],
+        case (catch couch_replicator_api_wrap:open_doc(Db, LogId, Opts)) 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
+    TgtResult = (catch couch_replicator_api_wrap:ensure_full_commit(Target)),
+
+    SrcResult = receive
+        {SrcCommitPid, Result} ->
+            unlink(SrcCommitPid),
+            receive
+                {'EXIT', SrcCommitPid, _} ->
+                    ok
+            after
+                0 -> ok
+            end,
+            Result;
+        {'EXIT', SrcCommitPid, Reason} ->
+            {error, Reason}
+    end,
+    case TgtResult of
+        {ok, TargetStartTime} ->
+            case SrcResult 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,
+    SrcSession = get_value(<<"session_id">>, RepRecProps),
+    TgtSession = get_value(<<"session_id">>, RepRecPropsTgt),
+    case SrcSession == TgtSession 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.
+
+
+get_pending_count(#rep_state{} = St) ->
+    #rep_state{
+        highest_seq_done = HighestSeqDone,
+        source = #httpdb{} = Db0
+    } = St,
+    {_, Seq} = HighestSeqDone,
+    Db = Db0#httpdb{retries = 3},
+    case (catch couch_replicator_api_wrap:get_pending_count(Db, Seq)) of
+        {ok, Pending} ->
+            Pending;
+        _ ->
+            null
+    end.
+
+
+maybe_update_job_state(#rep_state{} = State) ->
+    case State#rep_state.stats_timer of
+        nil -> start_stats_timer(State);
+        Ref when is_reference(Ref) -> State
+    end.
+
+
+update_job_state(#rep_state{} = State0) ->
+    State = cancel_stats_timer(State0),
+    #rep_state{
+        current_through_seq = {_, ThroughSeq},
+        highest_seq_done = {_, HighestSeq},
+        committed_seq = {_, CommittedSeq},
+        stats = Stats,
+        job_data = JobData
+    } = State,
+
+    Now = erlang:system_time(second),
+
+    RevisionsChecked = couch_replicator_stats:missing_checked(Stats),
+    MissingRevisions = couch_replicator_stats:missing_found(Stats),
+    DocsRead = couch_replicator_stats:docs_read(Stats),
+    DocsWritten = couch_replicator_stats:docs_written(Stats),
+    DocWriteFailures = couch_replicator_stats:doc_write_failures(Stats),
+    PendingCount = get_pending_count(State),
+
+    StatsMap = #{
+        <<"checkpointed_source_seq">> => CommittedSeq,
+        <<"source_seq">> => HighestSeq,
+        <<"through_seq">> => ThroughSeq,
+        <<"revisions_checked">> => RevisionsChecked,
+        <<"missing_revisions_found">> => MissingRevisions,
+        <<"docs_read">> => DocsRead,
+        <<"docs_written">> => DocsWritten,
+        <<"doc_write_failures">> => DocWriteFailures,
+        <<"changes_pending">> => PendingCount
+    },
+
+    JobData1 = JobData#{
+        ?REP_STATS := StatsMap,
+        ?LAST_UPDATED := Now
+    },
+
+    JobData2 = maybe_heal(JobData1, Now),
+
+    State1 = State#rep_state{job_data = JobData2},
+    State2 = update_active_task_info(State1),
+    update_job_data(undefined, State2).
+
+
+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({http_request_failed, _Method, Url0,
+        {error, {error, {conn_failed, {error, nxdomain}}}}}) ->
+    Url = ?l2b(couch_util:url_strip_password(Url0)),
+    {nxdomain, <<"could not resolve ", Url/binary>>};
+
+replication_start_error({http_request_failed, Method0, Url0,
+        {error, {code, Code}}}) when is_integer(Code) ->
+    Url = ?l2b(couch_util:url_strip_password(Url0)),
+    Method = ?l2b(Method0),
+    CodeBin = integer_to_binary(Code),
+    {http_error_code, <<CodeBin/binary, " ", Method/binary, " ", Url/binary>>};
+
+replication_start_error(Error) ->
+    Error.
+
+
+log_replication_start(#rep_state{} = RepState) ->
+    #rep_state{
+        id = Id,
+        doc_id = DocId,
+        db_name = DbName,
+        options = Options,
+        source_name = Source,
+        target_name = Target,
+        session_id = Sid
+    } = RepState,
+    Workers = maps:get(<<"worker_processes">>, Options),
+    BatchSize = maps:get(<<"worker_batch_size">>, Options),
+    From = case DbName of
+        Name when is_binary(Name) ->
+            io_lib:format("from doc ~s:~s", [Name, DocId]);
+        _ ->
+            "from _replicate endpoint"
+    end,
+    Msg = "Starting replication ~s (~s -> ~s) ~s worker_procesess:~p"
+        " worker_batch_size:~p session_id:~s",
+    couch_log:notice(Msg, [Id, Source, Target, From, Workers, BatchSize, Sid]).
+
+
+check_user_filter(#rep_state{} = State) ->
+    #rep_state{
+        id = RepId,
+        base_id = BaseId,
+        job = Job,
+        job_data = JobData
+    } = State,
+    case get_rep_id(undefined, Job, JobData) of
+        {RepId, BaseId} ->
+            ok;
+        {NewId, NewBaseId} when is_binary(NewId), is_binary(NewBaseId) ->
+            LogMsg = "~p : Replication id was updated ~p -> ~p",
+            couch_log:error(LogMsg, [?MODULE, RepId, NewId]),
+            reschedule(undefined, Job, JobData),
+            exit({shutdown, finished})
+    end.
+
+
+hist_append(Type, Now, #{} = JobData, Info) when is_integer(Now),
+        is_binary(Type) ->
+    #{?JOB_HISTORY := Hist} = JobData,
+    Evt1 = #{?HIST_TYPE => Type, ?HIST_TIMESTAMP => Now},
+    Evt2 = case Info of
+        undefined ->
+            Evt1;
+        null ->
+            Evt1#{?HIST_REASON => null};
+        <<_/binary>> ->
+            Evt1#{?HIST_REASON => Info};
+        #{<<"error">> := Err, <<"reason">> := Reason} when is_binary(Err),
+                is_binary(Reason) ->
+            Evt1#{?HIST_REASON => Reason}
+    end,
+    Hist1 = [Evt2 | Hist],
+    Hist2 = lists:sublist(Hist1, max_history()),
+    JobData#{?JOB_HISTORY := Hist2}.
+
+
+optimize_rate_limited_job(#{} = Options, <<"max_backoff">>) ->
+    OptimizedSettings = #{
+        <<"checkpoint_interval">> => 5000,
+        <<"worker_processes">> => 2,
+        <<"worker_batch_size">> => 100,
+        <<"http_connections">> => 2
+    },
+    maps:merge(Options, OptimizedSettings);
+
+optimize_rate_limited_job(#{} = Options, _Other) ->
+    Options.
+
+
+close_endpoints(State) ->
+    State1 = cancel_timers(State),
+    couch_replicator_api_wrap:db_close(State1#rep_state.source),
+    couch_replicator_api_wrap:db_close(State1#rep_state.target),
+    ok.
+
+
+get_value(K, Props) ->
+    couch_util:get_value(K, Props).
+
+
+get_value(K, Props, Default) ->
+    couch_util:get_value(K, Props, Default).
+
+
+accept_jitter_msec() ->
+    couch_rand:uniform(erlang:max(1, max_startup_jitter_msec())).
+
+
+max_startup_jitter_msec() ->
+    config:get_integer("replicator", "startup_jitter",
+        ?STARTUP_JITTER_DEFAULT).
+
+
+min_backoff_penalty_sec() ->
+    config:get_integer("replicator", "min_backoff_penalty_sec",
+        ?DEFAULT_MIN_BACKOFF_PENALTY_SEC).
+
+
+max_backoff_penalty_sec() ->
+    config:get_integer("replicator", "max_backoff_penalty_sec",
+        ?DEFAULT_MAX_BACKOFF_PENALTY_SEC).
+
+
+max_history() ->
+    config:get_integer("replicator", "max_history", ?DEFAULT_MAX_HISTORY).
+
+
+stats_update_interval_sec() ->
+    config:get_integer("replicator", "stats_update_interval_sec",
+        ?DEFAULT_STATS_UPDATE_INTERVAL_SEC).
+
+
+-ifdef(TEST).
+
+-include_lib("eunit/include/eunit.hrl").
+-include_lib("fabric/test/fabric2_test.hrl").
+
+
+replication_start_error_test() ->
+    ?assertEqual({unauthorized, <<"unauthorized to access or create database"
+        " http://x/y">>}, replication_start_error({unauthorized,
+        <<"http://x/y">>})),
+    ?assertEqual({db_not_found, <<"could not open http://x/y">>},
+        replication_start_error({db_not_found, <<"http://x/y">>})),
+    ?assertEqual({nxdomain, <<"could not resolve http://x/y">>},
+        replication_start_error({http_request_failed, "GET", "http://x/y",
+        {error, {error, {conn_failed, {error, nxdomain}}}}})),
+    ?assertEqual({http_error_code, <<"503 GET http://x/y">>},
+        replication_start_error({http_request_failed, "GET", "http://x/y",
+        {error, {code, 503}}})).
+
+
+scheduler_job_format_status_test_() ->
+    {
+        foreach,
+        fun setup/0,
+        fun teardown/1,
+        [
+            ?TDEF_FE(t_format_status)
+        ]
+    }.
+
+
+setup() ->
+    meck:expect(config, get, fun(_, _, Default) -> Default end).
+
+
+teardown(_) ->
+    meck:unload().
+
+
+t_format_status(_) ->
+    {ok, Rep} = couch_replicator_parse:parse_rep(#{
+        <<"source">> => <<"http://u:p@h1/d1">>,
+        <<"target">> =>  <<"http://u:p@h2/d2">>,
+        <<"create_target">> => true
+    }, null),
+    State = #rep_state{
+        id = <<"base+ext">>,
+        job_data = #{?REP => Rep},
+        doc_id = <<"mydoc">>,
+        db_name = <<"mydb">>,
+        source = maps:get(?SOURCE, Rep),
+        target = maps:get(?TARGET, Rep),
+        options = maps:get(?OPTIONS, Rep),
+        session_id = <<"a">>,
+        start_seq = <<"1">>,
+        source_seq = <<"2">>,
+        committed_seq = <<"3">>,
+        current_through_seq = <<"4">>,
+        highest_seq_done = <<"5">>
+    },
+    Format = format_status(opts_ignored, [pdict, State]),
+    FmtOptions = proplists:get_value(options, Format),
+    ?assertEqual("http://u:*****@h1/d1/", proplists:get_value(source, Format)),
+    ?assertEqual("http://u:*****@h2/d2/", proplists:get_value(target, Format)),
+    ?assertEqual(<<"base+ext">>, proplists:get_value(rep_id, Format)),
+    ?assertEqual(true, maps:get(<<"create_target">>, FmtOptions)),
+    ?assertEqual(<<"mydoc">>, proplists:get_value(doc_id, Format)),
+    ?assertEqual(<<"mydb">>, proplists:get_value(db_name, Format)),
+    ?assertEqual(<<"a">>, proplists:get_value(session_id, Format)),
+    ?assertEqual(<<"1">>, proplists:get_value(start_seq, Format)),
+    ?assertEqual(<<"2">>, proplists:get_value(source_seq, Format)),
+    ?assertEqual(<<"3">>, proplists:get_value(committed_seq, Format)),
+    ?assertEqual(<<"4">>, proplists:get_value(current_through_seq, Format)),
+    ?assertEqual(<<"5">>, proplists:get_value(highest_seq_done, Format)).
+
+
+-endif.
diff --git a/src/couch_replicator/src/couch_replicator_job_server.erl b/src/couch_replicator/src/couch_replicator_job_server.erl
new file mode 100644
index 0000000..a2e90b0
--- /dev/null
+++ b/src/couch_replicator/src/couch_replicator_job_server.erl
@@ -0,0 +1,370 @@
+% 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_job_server).
+
+
+-behaviour(gen_server).
+
+
+-export([
+    start_link/1
+]).
+
+-export([
+    init/1,
+    terminate/2,
+    handle_call/3,
+    handle_cast/2,
+    handle_info/2,
+    format_status/2,
+    code_change/3
+]).
+
+-export([
+    accepted/2,
+    scheduling_interval_sec/0,
+    reschedule/0
+]).
+
+
+-include("couch_replicator.hrl").
+
+
+-define(MAX_ACCEPTORS, 2).
+-define(MAX_JOBS, 500).
+-define(MAX_CHURN, 100).
+-define(INTERVAL_SEC, 15).
+-define(MIN_RUN_TIME_SEC, 60).
+-define(TRANSIENT_JOB_MAX_AGE_SEC, 86400). % 1 day
+
+
+start_link(Timeout) when is_integer(Timeout) ->
+    gen_server:start_link({local, ?MODULE}, ?MODULE, Timeout, []).
+
+
+init(Timeout) when is_integer(Timeout) ->
+    process_flag(trap_exit, true),
+    couch_replicator_jobs:set_timeout(),
+    St = #{
+        acceptors => #{},
+        workers => #{},
+        churn => 0,
+        config => get_config(),
+        timer => undefined,
+        timeout => Timeout
+    },
+    St1 = spawn_acceptors(St),
+    St2 = do_send_after(St1),
+    {ok, St2}.
+
+
+terminate(_, #{} = St) ->
+    #{
+        workers := Workers,
+        timeout := Timeout
+    } = St,
+    [stop_job(Pid) || Pid <- maps:keys(Workers)],
+    % Give jobs a chance to checkpoint and release their locks
+    wait_jobs_exit(Workers, Timeout),
+    ok.
+
+
+handle_call({accepted, Pid, Normal}, _From, #{} = St) ->
+    #{
+        acceptors := Acceptors,
+        workers := Workers,
+        churn := Churn
+    } = St,
+    case maps:is_key(Pid, Acceptors) of
+        true ->
+            Val = {Normal, erlang:system_time(second)},
+            St1 = St#{
+                acceptors := maps:remove(Pid, Acceptors),
+                workers := Workers#{Pid => Val},
+                churn := Churn + 1
+            },
+            {reply, ok, spawn_acceptors(St1)};
+        false ->
+            LogMsg = "~p : unknown acceptor processs ~p",
+            couch_log:error(LogMsg, [?MODULE, Pid]),
+            {stop, {unknown_acceptor_pid, Pid}, St}
+    end;
+
+handle_call(reschedule, _From, St) ->
+    {reply, ok, reschedule(St)};
+
+handle_call(Msg, _From, St) ->
+    {stop, {bad_call, Msg}, {bad_call, Msg}, St}.
+
+
+handle_cast(Msg, St) ->
+    {stop, {bad_cast, Msg}, St}.
+
+
+handle_info(reschedule, #{} = St) ->
+    {noreply, reschedule(St)};
+
+handle_info({'EXIT', Pid, Reason}, #{} = St) ->
+    #{
+        acceptors := Acceptors,
+        workers := Workers
+    } = St,
+    case {maps:is_key(Pid, Acceptors), maps:is_key(Pid, Workers)} of
+        {true, false} -> handle_acceptor_exit(St, Pid, Reason);
+        {false, true} -> handle_worker_exit(St, Pid, Reason);
+        {false, false} -> handle_unknown_exit(St, Pid, Reason)
+    end;
+
+handle_info(Msg, St) ->
+    {stop, {bad_info, Msg}, St}.
+
+
+format_status(_Opt, [_PDict, #{} = St]) ->
+    #{
+        acceptors := Acceptors,
+        workers := Workers,
+        churn := Churn,
+        config := Config
+    } = St,
+    [
+        {acceptors, map_size(Acceptors)},
+        {workers, map_size(Workers)},
+        {churn, Churn},
+        {config, Config}
+    ].
+
+
+code_change(_OldVsn, St, _Extra) ->
+    {ok, St}.
+
+
+accepted(Worker, Normal) when is_pid(Worker), is_boolean(Normal) ->
+    gen_server:call(?MODULE, {accepted, Worker, Normal}, infinity).
+
+
+scheduling_interval_sec() ->
+    config:get_integer("replicator", "interval_sec", ?INTERVAL_SEC).
+
+
+reschedule() ->
+    gen_server:call(?MODULE, reschedule, infinity).
+
+
+% Scheduling logic
+
+do_send_after(#{} = St) ->
+    #{config := #{interval_sec := IntervalSec}} = St,
+    IntervalMSec = IntervalSec * 1000,
+    Jitter = IntervalMSec div 3,
+    WaitMSec = IntervalMSec + rand:uniform(max(1, Jitter)),
+    TRef = erlang:send_after(WaitMSec, self(), reschedule),
+    St#{timer := TRef}.
+
+
+cancel_timer(#{timer := undefined} = St) ->
+    St;
+
+cancel_timer(#{timer := TRef} = St) when is_reference(TRef) ->
+    erlang:cancel_timer(TRef),
+    St#{timer := undefined}.
+
+
+reschedule(#{} = St) ->
+    St1 = cancel_timer(St),
+    St2 = St1#{config := get_config()},
+    St3 = trim_jobs(St2),
+    St4 = start_excess_acceptors(St3),
+    St5 = transient_job_cleanup(St4),
+    St6 = update_stats(St5),
+    St7 = do_send_after(St6),
+    St7#{churn := 0}.
+
+
+start_excess_acceptors(#{} = St) ->
+    #{
+        churn := Churn,
+        acceptors := Acceptors,
+        workers := Workers,
+        config := #{max_jobs := MaxJobs, max_churn := MaxChurn}
+    } = St,
+
+    ACnt = maps:size(Acceptors),
+    WCnt = maps:size(Workers),
+
+    ChurnLeft = MaxChurn - Churn,
+    Slots = (MaxJobs + MaxChurn) - (ACnt + WCnt),
+    MinSlotsChurn = min(Slots, ChurnLeft),
+
+    Pending = if MinSlotsChurn =< 0 -> 0; true ->
+        % Don't fetch pending if we don't have enough slots or churn budget
+        couch_replicator_jobs:pending_count(undefined, MinSlotsChurn)
+    end,
+
+    couch_stats:update_gauge([couch_replicator, jobs, pending], Pending),
+
+    % Start new acceptors only if we have churn budget, there are pending jobs
+    % and we won't start more than max jobs + churn total acceptors
+    ToStart = max(0, lists:min([ChurnLeft, Pending, Slots])),
+
+    lists:foldl(fun(_, #{} = StAcc) ->
+        #{acceptors := AccAcceptors} = StAcc,
+        {ok, Pid} = couch_replicator_job:start_link(),
+        StAcc#{acceptors := AccAcceptors#{Pid => true}}
+    end, St, lists:seq(1, ToStart)).
+
+
+transient_job_cleanup(#{} = St) ->
+    #{
+        config := #{transient_job_max_age_sec := MaxAge}
+    } = St,
+    Now = erlang:system_time(second),
+    FoldFun = fun(_JTx, JobId, State, #{} = Data, ok) ->
+        IsTransient = maps:get(?DB_NAME, Data) =:= null,
+        IsOld = Now - maps:get(?LAST_UPDATED, Data) >= MaxAge,
+        case State =:= finished andalso IsTransient andalso IsOld of
+            true ->
+                ok = couch_replicator_jobs:remove_job(undefined, JobId),
+                couch_log:info("~p : Removed old job ~p", [?MODULE, JobId]),
+                ok;
+            false ->
+                ok
+        end
+    end,
+    ok = couch_replicator_jobs:fold_jobs(undefined, FoldFun, ok),
+    St.
+
+
+update_stats(#{} = St) ->
+    ACnt = maps:size(maps:get(acceptors, St)),
+    WCnt = maps:size(maps:get(workers, St)),
+    couch_stats:update_gauge([couch_replicator, jobs, accepting], ACnt),
+    couch_stats:update_gauge([couch_replicator, jobs, running], WCnt),
+    couch_stats:increment_counter([couch_replicator, jobs, reschedules]),
+    St.
+
+
+trim_jobs(#{} = St) ->
+    #{
+        workers := Workers,
+        churn := Churn,
+        config := #{max_jobs := MaxJobs}
+    } = St,
+    Excess = max(0, maps:size(Workers) - MaxJobs),
+    lists:foreach(fun stop_job/1, stop_candidates(St, Excess)),
+    St#{churn := Churn + Excess}.
+
+
+stop_candidates(#{}, Top) when is_integer(Top), Top =< 0 ->
+    [];
+
+stop_candidates(#{} = St, Top) when is_integer(Top), Top > 0 ->
+    #{
+        workers := Workers,
+        config := #{min_run_time_sec := MinRunTime}
+    } = St,
+
+    WList1 = maps:to_list(Workers), % [{Pid, {Normal, StartTime}},...]
+
+    % Filter out normal jobs and those which have just started running
+    MaxT = erlang:system_time(second) - MinRunTime,
+    WList2 = lists:filter(fun({_Pid, {Normal, T}}) ->
+        not Normal andalso T =< MaxT
+    end, WList1),
+
+    Sorted = lists:keysort(2, WList2),
+    Pids = lists:map(fun({Pid, _}) -> Pid end, Sorted),
+    lists:sublist(Pids, Top).
+
+
+stop_job(Pid) when is_pid(Pid) ->
+    % Replication jobs handle the shutdown signal and then checkpoint in
+    % terminate handler
+    exit(Pid, shutdown).
+
+
+wait_jobs_exit(#{} = Jobs, _) when map_size(Jobs) =:= 0 ->
+    ok;
+
+wait_jobs_exit(#{} = Jobs, Timeout) ->
+    receive
+        {'EXIT', Pid, _} ->
+            wait_jobs_exit(maps:remove(Pid, Jobs), Timeout)
+    after
+        Timeout ->
+            LogMsg = "~p : ~p jobs didn't terminate cleanly",
+            couch_log:error(LogMsg, [?MODULE, map_size(Jobs)]),
+            ok
+    end.
+
+
+spawn_acceptors(St) ->
+    #{
+        workers := Workers,
+        acceptors := Acceptors,
+        config := #{max_jobs := MaxJobs, max_acceptors := MaxAcceptors}
+    } = St,
+    ACnt = maps:size(Acceptors),
+    WCnt = maps:size(Workers),
+    case ACnt < MaxAcceptors andalso (ACnt + WCnt) < MaxJobs of
+        true ->
+            {ok, Pid} = couch_replicator_job:start_link(),
+            NewSt = St#{acceptors := Acceptors#{Pid => true}},
+            spawn_acceptors(NewSt);
+        false ->
+            St
+    end.
+
+
+% Worker process exit handlers
+
+handle_acceptor_exit(#{acceptors := Acceptors} = St, Pid, Reason) ->
+    St1 = St#{acceptors := maps:remove(Pid, Acceptors)},
+    LogMsg = "~p : acceptor process ~p exited with ~p",
+    couch_log:error(LogMsg, [?MODULE, Pid, Reason]),
+    {noreply, spawn_acceptors(St1)}.
+
+
+handle_worker_exit(#{workers := Workers} = St, Pid, Reason) ->
+    St1 = St#{workers := maps:remove(Pid, Workers)},
+    case Reason of
+        normal ->
+            ok;
+        shutdown ->
+            ok;
+        {shutdown, _} ->
+            ok;
+        _ ->
+            LogMsg = "~p : replicator job process ~p exited with ~p",
+            couch_log:error(LogMsg, [?MODULE, Pid, Reason])
+    end,
+    {noreply, spawn_acceptors(St1)}.
+
+
+handle_unknown_exit(St, Pid, Reason) ->
+    LogMsg = "~p : unknown process ~p exited with ~p",
+    couch_log:error(LogMsg, [?MODULE, Pid, Reason]),
+    {stop, {unknown_pid_exit, Pid}, St}.
+
+
+get_config() ->
+    Defaults = #{
+        max_acceptors => ?MAX_ACCEPTORS,
+        interval_sec => ?INTERVAL_SEC,
+        max_jobs => ?MAX_JOBS,
+        max_churn => ?MAX_CHURN,
+        min_run_time_sec => ?MIN_RUN_TIME_SEC,
+        transient_job_max_age_sec => ?TRANSIENT_JOB_MAX_AGE_SEC
+    },
+    maps:map(fun(K, Default) ->
+        config:get_integer("replicator", atom_to_list(K), Default)
+    end, Defaults).


[couchdb] 11/16: Update frontend replicator modules

Posted by va...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

vatamane pushed a commit to branch prototype/fdb-layer
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit 5b98e8a6c169449d1a3e362e52e86822ef350ed5
Author: Nick Vatamaniuc <va...@apache.org>
AuthorDate: Fri Aug 28 04:34:21 2020 -0400

    Update frontend replicator modules
    
    The frontend is the part responsible for parsing replication parameters and
    creating replication jobs. Most of that happens in the `couch_replicator`
    module.
    
     - `couch_replicator:replicate/2` is the main API for creating transient
       replication jobs.
    
     - Replication jobs from `_replicator` documents are updated from
      `couch_replicator:after_*` callbacks. `after_db_create/2` besides being
      called on db creation also gets called when a database is undeleted and
      `add_jobs_from_db/1` function will attempt to parse them all.
    
    `couch_replicator` exports monitoring functions `docs/2,3 and jobs/0,1`. Those
    get called from HTTP handlers for `_scheduler/docs` and `_scheduler/jobs`
    respectively.
    
    For hands-on remsh access there some debuging functions such as:
    
     - rescan_jobs/0,1 : Simulates a db being re-created so all the jobs are added
     - reenqueue_jobs/0,1 : Deletes all the jobs from a db then re-adds them
     - remove_jobs/0 : Removes all the replication jobs
     - get_job_ids/0 : Read the RepId -> JobId mapping area
---
 src/couch_replicator/src/couch_replicator.erl      | 716 +++++++++++++--------
 src/couch_replicator/src/couch_replicator_epi.erl  |  58 ++
 .../src/couch_replicator_fabric2_plugin.erl        |  36 ++
 3 files changed, 530 insertions(+), 280 deletions(-)

diff --git a/src/couch_replicator/src/couch_replicator.erl b/src/couch_replicator/src/couch_replicator.erl
index b38f31b..f34ac7d 100644
--- a/src/couch_replicator/src/couch_replicator.erl
+++ b/src/couch_replicator/src/couch_replicator.erl
@@ -14,279 +14,484 @@
 
 -export([
     replicate/2,
-    replication_states/0,
+
+    jobs/0,
     job/1,
-    doc/3,
-    active_doc/2,
-    info_from_doc/2,
-    restart_job/1
+    docs/2,
+    doc/2,
+
+    after_db_create/2,
+    after_db_delete/2,
+    after_doc_write/6,
+
+    ensure_rep_db_exists/0,
+
+    rescan_jobs/0,
+    rescan_jobs/1,
+    reenqueue_jobs/0,
+    reenqueue_jobs/1,
+    remove_jobs/0,
+    get_job_ids/0
 ]).
 
+
 -include_lib("couch/include/couch_db.hrl").
 -include("couch_replicator.hrl").
--include_lib("couch_replicator/include/couch_replicator_api_wrap.hrl").
--include_lib("couch_mrview/include/couch_mrview.hrl").
--include_lib("mem3/include/mem3.hrl").
-
--define(DESIGN_DOC_CREATION_DELAY_MSEC, 1000).
--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, 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
-]).
 
 
 -spec replicate({[_]}, any()) ->
     {ok, {continuous, binary()}} |
-    {ok, {[_]}} |
+    {ok, #{}} |
     {ok, {cancelled, binary()}} |
     {error, any()} |
     no_return().
-replicate(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 ->
-        CancelRepId = case get_value(id, Options, nil) of
-        nil ->
-            RepId;
-        RepId2 ->
-            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),
-        Result
+replicate(Body, #user_ctx{name = User} = UserCtx) ->
+    {ok, Id, Rep} = couch_replicator_parse:parse_transient_rep(Body, User),
+    #{?OPTIONS := Options} = Rep,
+    JobId = case couch_replicator_jobs:get_job_id(undefined, Id) of
+        {ok, JobId0} -> JobId0;
+        {error, not_found} -> Id
+    end,
+    case maps:get(<<"cancel">>, Options, false) of
+        true ->
+            case check_authorization(JobId, UserCtx) of
+                ok -> cancel_replication(JobId);
+                not_found -> {error, not_found}
+            end;
+        false ->
+            check_authorization(JobId, UserCtx),
+            ok = start_transient_job(JobId, Rep),
+            case maps:get(<<"continuous">>, Options, false) of
+                true ->
+                    case couch_replicator_jobs:wait_running(JobId) of
+                        {ok, #{?STATE := ?ST_RUNNING} = JobData} ->
+                            {ok, {continuous, maps:get(?REP_ID, JobData)}};
+                        {ok, #{?STATE := ?ST_FAILED} = JobData} ->
+                            {error, maps:get(?STATE_INFO, JobData)};
+                        {error, Error} ->
+                            {error, Error}
+                    end;
+                false ->
+                    case couch_replicator_jobs:wait_result(JobId) of
+                        {ok, #{?STATE := ?ST_COMPLETED} = JobData} ->
+                            {ok, maps:get(?CHECKPOINT_HISTORY, JobData)};
+                        {ok, #{?STATE := ?ST_FAILED} = JobData} ->
+                            {error, maps:get(?STATE_INFO, JobData)};
+                        {error, Error} ->
+                            {error, Error}
+                    end
+            end
     end.
 
 
--spec do_replication_loop(#rep{}) ->
-    {ok, {continuous, binary()}} | {ok, tuple()} | {error, any()}.
-do_replication_loop(#rep{id = {BaseId, Ext} = Id, options = Options} = Rep) ->
-    ok = couch_replicator_scheduler:add_job(Rep),
-    case get_value(continuous, Options, false) of
-    true ->
-        {ok, {continuous, ?l2b(BaseId ++ Ext)}};
-    false ->
-        wait_for_result(Id)
+jobs() ->
+    FoldFun = fun(_JTx, _JobId, CouchJobsState, JobData, Acc) ->
+        case CouchJobsState of
+            pending -> [job_ejson(JobData) | Acc];
+            running -> [job_ejson(JobData) | Acc];
+            finished -> Acc
+        end
+    end,
+    couch_replicator_jobs:fold_jobs(undefined, FoldFun, []).
+
+
+job(Id0) when is_binary(Id0) ->
+    Id1 = couch_replicator_ids:convert(Id0),
+    JobId = case couch_replicator_jobs:get_job_id(undefined, Id1) of
+        {ok, JobId0} -> JobId0;
+        {error, not_found} -> Id1
+    end,
+    case couch_replicator_jobs:get_job_data(undefined, JobId) of
+        {ok, #{} = JobData} -> {ok, job_ejson(JobData)};
+        {error, not_found} -> {error, not_found}
     end.
 
 
--spec rep_result_listener(rep_id()) -> {ok, pid()}.
-rep_result_listener(RepId) ->
-    ReplyTo = self(),
-    {ok, _Listener} = couch_replicator_notifier:start_link(
-        fun({_, RepId2, _} = Ev) when RepId2 =:= RepId ->
-                ReplyTo ! Ev;
-            (_) ->
-                ok
-        end).
+docs(#{} = Db, States) when is_list(States) ->
+    DbName = fabric2_db:name(Db),
+    FoldFun = fun(_JTx, _JobId, _, JobData, Acc) ->
+        case JobData of
+            #{?DB_NAME := DbName, ?STATE := State} ->
+                case {States, lists:member(State, States)} of
+                    {[], _} ->  [doc_ejson(JobData) | Acc];
+                    {[_ | _], true} ->  [doc_ejson(JobData) | Acc];
+                    {[_ | _], false} -> Acc
+                end;
+            #{} ->
+                Acc
+        end
+    end,
+    couch_replicator_jobs:fold_jobs(undefined, FoldFun, []).
 
 
--spec wait_for_result(rep_id()) ->
-    {ok, {[_]}} | {error, any()}.
-wait_for_result(RepId) ->
-    receive
-    {finished, RepId, RepResult} ->
-        {ok, RepResult};
-    {error, RepId, Reason} ->
-        {error, Reason}
+doc(#{} = Db, DocId) when is_binary(DocId) ->
+    DbUUID = fabric2_db:get_uuid(Db),
+    JobId = couch_replicator_ids:job_id(DbUUID, DocId),
+    case couch_replicator_jobs:get_job_data(undefined, JobId) of
+        {ok, #{} = JobData} -> {ok, doc_ejson(JobData)};
+        {error, not_found} ->  {error, not_found}
     end.
 
 
--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.
+after_db_create(DbName, DbUUID) when ?IS_REP_DB(DbName)->
+    couch_stats:increment_counter([couch_replicator, docs, dbs_created]),
+    try fabric2_db:open(DbName, [{uuid, DbUUID}, ?ADMIN_CTX]) of
+        {ok, Db} ->
+            fabric2_fdb:transactional(Db, fun(TxDb) ->
+                ok = add_jobs_from_db(TxDb)
+            end)
+    catch
+        error:database_does_not_exist ->
+            ok
+    end;
+
+after_db_create(_DbName, _DbUUID) ->
+    ok.
+
+
+after_db_delete(DbName, DbUUID) when ?IS_REP_DB(DbName) ->
+    couch_stats:increment_counter([couch_replicator, docs, dbs_deleted]),
+    FoldFun = fun(JTx, JobId, _, JobData, ok) ->
+        case JobData of
+            #{?DB_UUID := DbUUID} ->
+                ok = couch_replicator_jobs:remove_job(JTx, JobId);
+            #{} ->
+                ok
+        end
+    end,
+    couch_replicator_jobs:fold_jobs(undefined, FoldFun, ok);
+
+after_db_delete(_DbName, _DbUUID) ->
+    ok.
+
+
+after_doc_write(#{name := DbName} = Db, #doc{} = Doc, _NewWinner, _OldWinner,
+        _NewRevId, _Seq) when ?IS_REP_DB(DbName) ->
+    couch_stats:increment_counter([couch_replicator, docs, db_changes]),
+    {Props} = Doc#doc.body,
+    case couch_util:get_value(?REPLICATION_STATE, Props) of
+        ?ST_COMPLETED -> ok;
+        ?ST_FAILED -> ok;
+        _ -> process_change(Db, Doc)
+    end;
+
+after_doc_write(_Db, _Doc, _NewWinner, _OldWinner, _NewRevId, _Seq) ->
+    ok.
+
+
+% This is called from supervisor, must return ignore.
+-spec ensure_rep_db_exists() -> ignore.
+ensure_rep_db_exists() ->
+    couch_replicator_jobs:set_timeout(),
+    case config:get_boolean("replicator", "create_replicator_db", false) of
+        true ->
+            UserCtx = #user_ctx{roles=[<<"_admin">>, <<"_replicator">>]},
+            Opts = [{user_ctx, UserCtx}, sys_db],
+            case fabric2_db:create(?REP_DB_NAME, Opts) of
+                {error, file_exists} -> ok;
+                {ok, _Db} -> ok
+            end;
+        false ->
+            ok
+    end,
+    ignore.
 
 
--spec replication_states() -> [atom()].
-replication_states() ->
-    ?REPLICATION_STATES.
+% Testing and debug functions
 
+rescan_jobs() ->
+    rescan_jobs(?REP_DB_NAME).
 
--spec strip_url_creds(binary() | {[_]}) -> binary().
-strip_url_creds(Endpoint) ->
-    try
-        couch_replicator_docs:parse_rep_db(Endpoint, [], []) of
-            #httpdb{url = Url} ->
-                iolist_to_binary(couch_util:url_strip_password(Url))
+
+rescan_jobs(DbName) when is_binary(DbName), ?IS_REP_DB(DbName) ->
+    try fabric2_db:open(DbName, [?ADMIN_CTX]) of
+        {ok, Db} ->
+            after_db_create(DbName, fabric2_db:get_uuid(Db))
     catch
-        throw:{error, local_endpoints_not_supported} ->
-            Endpoint
+        error:database_does_not_exist ->
+            database_does_not_exist
     end.
 
 
--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.
+reenqueue_jobs() ->
+    reenqueue_jobs(?REP_DB_NAME).
 
 
--spec restart_job(binary() | list() | rep_id()) ->
-    {ok, {[_]}} | {error, not_found}.
-restart_job(JobId0) ->
-    JobId = couch_replicator_ids:convert(JobId0),
-    {Res, _} = rpc:multicall(couch_replicator_scheduler, restart_job, [JobId]),
-    case [JobInfo || {ok, JobInfo} <- Res] of
-        [JobInfo| _] ->
-            {ok, JobInfo};
-        [] ->
-            {error, not_found}
+reenqueue_jobs(DbName) when is_binary(DbName), ?IS_REP_DB(DbName) ->
+    try fabric2_db:open(DbName, [?ADMIN_CTX]) of
+        {ok, Db} ->
+            DbUUID = fabric2_db:get_uuid(Db),
+            ok = after_db_delete(DbName, DbUUID),
+            ok = after_db_create(DbName, DbUUID)
+    catch
+        error:database_does_not_exist ->
+            database_does_not_exist
     end.
 
 
--spec active_doc(binary(), binary()) -> {ok, {[_]}} | {error, not_found}.
-active_doc(DbName, DocId) ->
-    try
-        Shards = mem3:shards(DbName),
-        Live = [node() | nodes()],
-        Nodes = lists:usort([N || #shard{node=N} <- Shards,
-            lists:member(N, Live)]),
-        Owner = mem3:owner(DbName, DocId, Nodes),
-        case active_doc_rpc(DbName, DocId, [Owner]) of
-            {ok, DocInfo} ->
-                {ok, DocInfo};
+remove_jobs() ->
+    % If we clear a large number of jobs make sure to use batching so we don't
+    % take too long, if use individual transactions, and also don't timeout if
+    % use a single transaction
+    FoldFun = fun
+        (_, JobId, _, _, Acc) when length(Acc) > 250 ->
+            couch_replicator_jobs:remove_jobs(undefined, [JobId | Acc]);
+        (_, JobId, _, _, Acc) ->
+            [JobId | Acc]
+    end,
+    Acc = couch_replicator_jobs:fold_jobs(undefined, FoldFun, []),
+    [] = couch_replicator_jobs:remove_jobs(undefined, Acc),
+    ok.
+
+
+get_job_ids() ->
+    couch_replicator_jobs:get_job_ids(undefined).
+
+
+% Private functions
+
+-spec start_transient_job(binary(), #{}) -> ok.
+start_transient_job(JobId, #{} = Rep) ->
+    JobData = couch_replicator_jobs:new_job(Rep, null, null, null,
+        ?ST_PENDING, null, null),
+    couch_jobs_fdb:tx(couch_jobs_fdb:get_jtx(), fun(JTx) ->
+        case couch_replicator_jobs:get_job_data(JTx, JobId) of
+            {ok, #{?REP := OldRep, ?STATE := State}} ->
+                SameRep = couch_replicator_utils:compare_reps(Rep, OldRep),
+                Active = State =:= ?ST_PENDING orelse State =:= ?ST_RUNNING,
+                case SameRep andalso Active of
+                    true ->
+                        % If a job with the same paremeters is running we don't
+                        % stop and just ignore the request. This is mainly for
+                        % compatibility where users are able to idempotently
+                        % POST the same job without it being stopped and
+                        % restarted.
+                        ok;
+                    false ->
+                        couch_replicator_jobs:add_job(JTx, JobId, JobData)
+                end;
             {error, not_found} ->
-                active_doc_rpc(DbName, DocId, Nodes -- [Owner])
+                ok = couch_replicator_jobs:add_job(JTx, JobId, JobData)
         end
-    catch
-        % Might be a local database
-        error:database_does_not_exist ->
-            active_doc_rpc(DbName, DocId, [node()])
-    end.
+    end).
 
 
--spec active_doc_rpc(binary(), binary(), [node()]) ->
-    {ok, {[_]}} | {error, not_found}.
-active_doc_rpc(_DbName, _DocId, []) ->
-    {error, not_found};
-active_doc_rpc(DbName, DocId, [Node]) when Node =:= node() ->
-    couch_replicator_doc_processor:doc(DbName, DocId);
-active_doc_rpc(DbName, DocId, Nodes) ->
-    {Res, _Bad} = rpc:multicall(Nodes, couch_replicator_doc_processor, doc,
-        [DbName, DocId]),
-    case [DocInfo || {ok, DocInfo} <- Res] of
-        [DocInfo | _] ->
-            {ok, DocInfo};
-        [] ->
-            {error, not_found}
-    end.
+-spec cancel_replication(job_id()) ->
+    {ok, {cancelled, binary()}} | {error, not_found}.
+cancel_replication(JobId) when is_binary(JobId) ->
+    couch_jobs_fdb:tx(couch_jobs_fdb:get_jtx(), fun(JTx) ->
+        Id = case couch_replicator_jobs:get_job_data(JTx, JobId) of
+            {ok, #{?REP_ID := RepId}} when is_binary(RepId) ->
+                RepId;
+            _ ->
+                JobId
+        end,
+        couch_log:notice("Canceling replication '~s'", [Id]),
+        case couch_replicator_jobs:remove_job(JTx, JobId) of
+            {error, not_found} ->
+                {error, not_found};
+            ok ->
+                {ok, {cancelled, Id}}
+        end
+    end).
 
 
--spec doc(binary(), binary(), any()) -> {ok, {[_]}} | {error, not_found}.
-doc(RepDb, DocId, UserCtx) ->
-    case active_doc(RepDb, DocId) of
-        {ok, DocInfo} ->
-            {ok, DocInfo};
-        {error, not_found} ->
-            doc_from_db(RepDb, DocId, UserCtx)
-    end.
+process_change(_Db, #doc{id = <<?DESIGN_DOC_PREFIX, _/binary>>}) ->
+    ok;
 
+process_change(#{} = Db, #doc{deleted = true} = Doc) ->
+    DbUUID = fabric2_db:get_uuid(Db),
+    JobId = couch_replicator_ids:job_id(DbUUID, Doc#doc.id),
+    couch_replicator_jobs:remove_job(undefined, JobId);
 
--spec doc_from_db(binary(), binary(), any()) -> {ok, {[_]}} | {error, not_found}.
-doc_from_db(RepDb, DocId, UserCtx) ->
-    case fabric:open_doc(RepDb, DocId, [UserCtx, ejson_body]) of
-        {ok, Doc} ->
-            {ok, info_from_doc(RepDb, couch_doc:to_json_obj(Doc, []))};
-         {not_found, _Reason} ->
-            {error, not_found}
-    end.
+process_change(#{} = Db, #doc{deleted = false} = Doc) ->
+    #doc{id = DocId, body = {Props} = Body} = Doc,
+    DbName = fabric2_db:name(Db),
+    DbUUID = fabric2_db:get_uuid(Db),
+    {Rep, DocState, Error} = try
+        Rep0 = couch_replicator_parse:parse_rep_doc(Body),
+        DocState0 = couch_util:get_value(?REPLICATION_STATE, Props, null),
+        {Rep0, DocState0, null}
+    catch
+        throw:{bad_rep_doc, Reason} ->
+            {null, null, couch_replicator_utils:rep_error_to_binary(Reason)}
+    end,
+    JobId = couch_replicator_ids:job_id(DbUUID, DocId),
+    JobData = case Rep of
+        null ->
+            couch_relicator_jobs:new_job(Rep, DbName, DbUUID, DocId,
+                ?ST_FAILED, Error, null);
+        #{} ->
+            couch_replicator_jobs:new_job(Rep, DbName, DbUUID, DocId,
+                ?ST_PENDING, null, DocState)
+    end,
 
+    LogMsg = "~p : replication doc update db:~s doc:~s job_id:~s doc_state:~s",
+    couch_log:notice(LogMsg, [?MODULE, DbName, DocId, JobId, DocState]),
+
+    couch_jobs_fdb:tx(couch_jobs_fdb:get_jtx(Db), fun(JTx) ->
+        case couch_replicator_jobs:get_job_data(JTx, JobId) of
+            {ok, #{?REP := null, ?STATE_INFO := Error}} when Rep =:= null ->
+                % Same error as before occurred, don't bother updating the job
+                ok;
+            {ok, #{?REP := null}} when Rep =:= null ->
+                % New error so the job is updated
+                couch_replicator_jobs:add_job(JTx, JobId, JobData);
+            {ok, #{?REP := OldRep, ?STATE := State}} when is_map(Rep) ->
+                SameRep = couch_replicator_utils:compare_reps(Rep, OldRep),
+                Active = State =:= ?ST_PENDING orelse State =:= ?ST_RUNNING,
+                case SameRep andalso Active of
+                    true ->
+                        % Document was changed but none of the parameters
+                        % relevent for the replication job have changed, so
+                        % make it a no-op
+                        ok;
+                    false ->
+                        couch_replicator_jobs:add_job(JTx, JobId, JobData)
+                end;
+            {error, not_found} ->
+                couch_replicator_jobs:add_job(JTx, JobId, JobData)
+        end
 
--spec info_from_doc(binary(), {[_]}) -> {[_]}.
-info_from_doc(RepDb, {Props}) ->
-    DocId = get_value(<<"_id">>, Props),
-    Source = get_value(<<"source">>, Props),
-    Target = get_value(<<"target">>, Props),
-    State0 = state_atom(get_value(<<"_replication_state">>, Props, null)),
-    StateTime = get_value(<<"_replication_state_time">>, Props, null),
-    {State1, StateInfo, ErrorCount, StartTime} = case State0 of
-        completed ->
-            {InfoP} = get_value(<<"_replication_stats">>, Props, {[]}),
-            case lists:keytake(<<"start_time">>, 1, InfoP) of
-                {value, {_, Time}, InfoP1} ->
-                    {State0, {InfoP1}, 0, Time};
-                false ->
-                    case lists:keytake(start_time, 1, InfoP) of
-                        {value, {_, Time}, InfoP1} ->
-                            {State0, {InfoP1}, 0, Time};
-                        false ->
-                            {State0, {InfoP}, 0, null}
-                        end
-            end;
-        failed ->
-            Info = get_value(<<"_replication_state_reason">>, Props, nil),
-            EJsonInfo = couch_replicator_utils:ejson_state_info(Info),
-            {State0, EJsonInfo, 1, StateTime};
-        _OtherState ->
-            {null, null, 0, null}
+    end).
+
+
+-spec add_jobs_from_db(#{}) -> ok.
+add_jobs_from_db(#{} = TxDb) ->
+    FoldFun  = fun
+        ({meta, _Meta}, ok) ->
+            {ok, ok};
+        (complete, ok) ->
+            {ok, ok};
+        ({row, Row}, ok) ->
+            Db = TxDb#{tx := undefined},
+            ok = process_change(Db, get_doc(TxDb, Row)),
+            {ok, ok}
     end,
-    {[
-        {doc_id, DocId},
-        {database, RepDb},
-        {id, null},
-        {source, strip_url_creds(Source)},
-        {target, strip_url_creds(Target)},
-        {state, State1},
-        {error_count, ErrorCount},
-        {info, StateInfo},
-        {start_time, StartTime},
-        {last_updated, StateTime}
-     ]}.
-
-
-state_atom(<<"triggered">>) ->
-    triggered;  % This handles a legacy case were document wasn't converted yet
-state_atom(State) when is_binary(State) ->
-    erlang:binary_to_existing_atom(State, utf8);
-state_atom(State) when is_atom(State) ->
-    State.
+    Opts = [{restart_tx, true}],
+    {ok, ok} = fabric2_db:fold_docs(TxDb, FoldFun, ok, Opts),
+    ok.
+
+
+-spec get_doc(#{}, list()) -> #doc{}.
+get_doc(TxDb, Row) ->
+    {_, DocId} = lists:keyfind(id, 1, Row),
+    {ok, #doc{deleted = false} = Doc} = fabric2_db:open_doc(TxDb, DocId, []),
+    Doc.
+
+
+doc_ejson(#{} = JobData) ->
+    #{
+        ?REP := Rep,
+        ?REP_ID := RepId,
+        ?DB_NAME := DbName,
+        ?DOC_ID := DocId,
+        ?STATE := State,
+        ?STATE_INFO := Info0,
+        ?ERROR_COUNT := ErrorCount,
+        ?LAST_UPDATED := LastUpdatedSec,
+        ?REP_NODE := Node,
+        ?REP_PID := Pid,
+        ?REP_STATS := Stats
+    } = JobData,
+
+    #{
+        ?SOURCE := #{<<"url">> := Source, <<"proxy_url">> := SourceProxy},
+        ?TARGET := #{<<"url">> := Target, <<"proxy_url">> := TargetProxy},
+        ?START_TIME := StartSec
+    } = Rep,
+
+    LastUpdatedISO8601 = couch_replicator_utils:iso8601(LastUpdatedSec),
+    StartISO8601 = couch_replicator_utils:iso8601(StartSec),
+
+    Info = case State of
+        ?ST_RUNNING -> Stats;
+        ?ST_PENDING -> Stats;
+        _Other -> Info0
+    end,
+
+    #{
+        <<"id">> => RepId,
+        <<"database">> => DbName,
+        <<"doc_id">> => DocId,
+        <<"source">> => ejson_url(Source),
+        <<"target">> => ejson_url(Target),
+        <<"source_proxy">> => ejson_url(SourceProxy),
+        <<"target_proxy">> => ejson_url(TargetProxy),
+        <<"state">> => State,
+        <<"info">> => Info,
+        <<"error_count">> => ErrorCount,
+        <<"last_updated">> => LastUpdatedISO8601,
+        <<"start_time">> => StartISO8601,
+        <<"node">> => Node,
+        <<"pid">> => Pid
+    }.
+
+
+job_ejson(#{} = JobData) ->
+    #{
+        ?REP := Rep,
+        ?REP_ID := RepId,
+        ?DB_NAME := DbName,
+        ?DOC_ID := DocId,
+        ?STATE := State,
+        ?STATE_INFO := Info0,
+        ?JOB_HISTORY := History,
+        ?REP_STATS := Stats
+    } = JobData,
+
+    #{
+        ?SOURCE := #{<<"url">> := Source},
+        ?TARGET := #{<<"url">> := Target},
+        ?REP_USER := User,
+        ?START_TIME := StartSec
+    } = Rep,
+
+    StartISO8601 = couch_replicator_utils:iso8601(StartSec),
+
+    History1 = lists:map(fun(#{?HIST_TIMESTAMP := Ts} = Evt) ->
+        Evt#{?HIST_TIMESTAMP := couch_replicator_utils:iso8601(Ts)}
+    end, History),
+
+    Info = case State of
+        ?ST_RUNNING -> Stats;
+        ?ST_PENDING -> Stats;
+        _Other -> Info0
+    end,
+
+    #{
+        <<"id">> => RepId,
+        <<"database">> => DbName,
+        <<"doc_id">> => DocId,
+        <<"source">> => ejson_url(Source),
+        <<"target">> => ejson_url(Target),
+        <<"state">> => State,
+        <<"info">> => Info,
+        <<"user">> => User,
+        <<"history">> => History1,
+        <<"start_time">> => StartISO8601
+    }.
+
+
+ejson_url(Url) when is_binary(Url) ->
+    list_to_binary(couch_util:url_strip_password(Url));
+
+ejson_url(null) ->
+    null.
 
 
 -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
+check_authorization(JobId, #user_ctx{} = Ctx) when is_binary(JobId) ->
+    #user_ctx{name = Name} = Ctx,
+    case couch_replicator_jobs:get_job_data(undefined, JobId) of
+        {error, not_found} ->
+            not_found;
+        {ok, #{?DB_NAME := DbName}} when is_binary(DbName) ->
+            throw({unauthorized, <<"Persistent replication collision">>});
+        {ok, #{?REP := #{?REP_USER := Name}}} ->
+            ok;
+        {ok, #{}} ->
+            couch_httpd:verify_is_server_admin(Ctx)
     end.
 
 
@@ -309,16 +514,16 @@ authorization_test_() ->
 
 t_admin_is_always_authorized() ->
     ?_test(begin
-        expect_rep_user_ctx(<<"someuser">>, <<"_admin">>),
+        expect_job_data({ok, #{?REP => #{?REP_USER => <<"someuser">>}}}),
         UserCtx = #user_ctx{name = <<"adm">>, roles = [<<"_admin">>]},
         ?assertEqual(ok, check_authorization(<<"RepId">>, UserCtx))
     end).
 
 
 t_username_must_match() ->
-     ?_test(begin
-        expect_rep_user_ctx(<<"user">>, <<"somerole">>),
-        UserCtx1 = #user_ctx{name = <<"user">>, roles = [<<"somerole">>]},
+    ?_test(begin
+        expect_job_data({ok, #{?REP => #{?REP_USER => <<"user1">>}}}),
+        UserCtx1 = #user_ctx{name = <<"user1">>, roles = [<<"somerole">>]},
         ?assertEqual(ok, check_authorization(<<"RepId">>, UserCtx1)),
         UserCtx2 = #user_ctx{name = <<"other">>, roles = [<<"somerole">>]},
         ?assertThrow({unauthorized, _}, check_authorization(<<"RepId">>,
@@ -327,8 +532,8 @@ t_username_must_match() ->
 
 
 t_replication_not_found() ->
-     ?_test(begin
-        meck:expect(couch_replicator_scheduler, rep_state, 1, nil),
+    ?_test(begin
+        expect_job_data({error, not_found}),
         UserCtx1 = #user_ctx{name = <<"user">>, roles = [<<"somerole">>]},
         ?assertEqual(not_found, check_authorization(<<"RepId">>, UserCtx1)),
         UserCtx2 = #user_ctx{name = <<"adm">>, roles = [<<"_admin">>]},
@@ -336,57 +541,8 @@ t_replication_not_found() ->
     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).
-
+expect_job_data(JobDataRes) ->
+    meck:expect(couch_replicator_jobs, get_job_data, 2, JobDataRes).
 
-strip_url_creds_test_() ->
-     {
-        setup,
-        fun() ->
-            meck:expect(config, get, fun(_, _, Default) -> Default end)
-        end,
-        fun(_) ->
-            meck:unload()
-        end,
-        [
-            t_strip_http_basic_creds(),
-            t_strip_http_props_creds(),
-            t_strip_local_db_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)),
-        Url3 = <<"http://adm:pass@host:80/db">>,
-        ?assertEqual(<<"http://adm:*****@host:80/db/">>, strip_url_creds(Url3)),
-        Url4 = <<"http://adm:pass@host/db?a=b&c=d">>,
-        ?assertEqual(<<"http://adm:*****@host/db?a=b&c=d">>,
-            strip_url_creds(Url4))
-    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">>}]}}
-        ]},
-        ?assertEqual(<<"http://host/db/">>, strip_url_creds(Props2))
-    end).
 
 -endif.
diff --git a/src/couch_replicator/src/couch_replicator_epi.erl b/src/couch_replicator/src/couch_replicator_epi.erl
new file mode 100644
index 0000000..9fb1790
--- /dev/null
+++ b/src/couch_replicator/src/couch_replicator_epi.erl
@@ -0,0 +1,58 @@
+% 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_epi).
+
+
+-behaviour(couch_epi_plugin).
+
+
+-export([
+    app/0,
+    providers/0,
+    services/0,
+    data_subscriptions/0,
+    data_providers/0,
+    processes/0,
+    notify/3
+]).
+
+
+app() ->
+    couch_replicator.
+
+
+providers() ->
+    [
+        {fabric2_db, couch_replicator_fabric2_plugin}
+    ].
+
+
+services() ->
+    [].
+
+
+data_subscriptions() ->
+    [].
+
+
+data_providers() ->
+    [].
+
+
+processes() ->
+    [].
+
+
+notify(_Key, _Old, _New) ->
+    ok.
diff --git a/src/couch_replicator/src/couch_replicator_fabric2_plugin.erl b/src/couch_replicator/src/couch_replicator_fabric2_plugin.erl
new file mode 100644
index 0000000..7bf6145
--- /dev/null
+++ b/src/couch_replicator/src/couch_replicator_fabric2_plugin.erl
@@ -0,0 +1,36 @@
+% 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_fabric2_plugin).
+
+
+-export([
+    after_db_create/2,
+    after_db_delete/2,
+    after_doc_write/6
+]).
+
+
+after_db_create(DbName, DbUUID) ->
+    couch_replicator:after_db_create(DbName, DbUUID),
+    [DbName, DbUUID].
+
+
+after_db_delete(DbName, DbUUID) ->
+    couch_replicator:after_db_delete(DbName, DbUUID),
+    [DbName, DbUUID].
+
+
+after_doc_write(Db, Doc, Winner, OldWinner, RevId, Seq)->
+    couch_replicator:after_doc_write(Db, Doc, Winner, OldWinner, RevId, Seq),
+    [Db, Doc, Winner, OldWinner, RevId, Seq].


[couchdb] 06/16: Cleanup couch_replicator_utils module

Posted by va...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

vatamane pushed a commit to branch prototype/fdb-layer
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit b6e87f8a43eebb4d02dfa52227ba5b77cd4ebc68
Author: Nick Vatamaniuc <va...@apache.org>
AuthorDate: Fri Aug 28 04:31:38 2020 -0400

    Cleanup couch_replicator_utils module
    
     * Remove unused functions and some function used only from one place like `sum_stats/2`.
    
     * Update time functions to use the more modern `erlang:system_time/1` API.
    
     * `parse_int_param/5` and `parse_replication_states/1` was moved from the old
        _httpd_util module as they were they only ones need from there.
    
     * `default_headers_map/0` Used to the default httpd record headers as a map
       since part of the replication data will be kept as map object.
    
     * `proplist_options/1` Some parts of the replicator, like _httpc and _api_wrap
       still use proplist options, so this function can be used to translate
       options as maps to a proplist version.
---
 .../src/couch_replicator_stats.erl                 |   2 +
 .../src/couch_replicator_utils.erl                 | 241 ++++++++++++---------
 .../src/couch_replicator_worker.erl                |   2 +-
 3 files changed, 137 insertions(+), 108 deletions(-)

diff --git a/src/couch_replicator/src/couch_replicator_stats.erl b/src/couch_replicator/src/couch_replicator_stats.erl
index 37848b3..69e60a0 100644
--- a/src/couch_replicator/src/couch_replicator_stats.erl
+++ b/src/couch_replicator/src/couch_replicator_stats.erl
@@ -32,6 +32,8 @@
 new() ->
     orddict:new().
 
+new(#{} = Map) ->
+    new(maps:to_list(Map));
 new(Initializers0) when is_list(Initializers0) ->
     Initializers1 = lists:filtermap(fun fmap/1, Initializers0),
     orddict:from_list(Initializers1).
diff --git a/src/couch_replicator/src/couch_replicator_utils.erl b/src/couch_replicator/src/couch_replicator_utils.erl
index 5f608de..cbed78e 100644
--- a/src/couch_replicator/src/couch_replicator_utils.erl
+++ b/src/couch_replicator/src/couch_replicator_utils.erl
@@ -13,19 +13,18 @@
 -module(couch_replicator_utils).
 
 -export([
-   parse_rep_doc/2,
-   replication_id/2,
-   sum_stats/2,
-   is_deleted/1,
    rep_error_to_binary/1,
-   get_json_value/2,
-   get_json_value/3,
-   pp_rep_id/1,
+   iso8601/0,
    iso8601/1,
-   filter_state/3,
+   rfc1123_local/0,
+   rfc1123_local/1,
    remove_basic_auth_from_headers/1,
    normalize_rep/1,
-   ejson_state_info/1
+   compare_reps/2,
+   default_headers_map/0,
+   parse_replication_states/1,
+   parse_int_param/5,
+   proplist_options/1
 ]).
 
 
@@ -33,11 +32,6 @@
 -include("couch_replicator.hrl").
 -include_lib("couch_replicator/include/couch_replicator_api_wrap.hrl").
 
--import(couch_util, [
-    get_value/2,
-    get_value/3
-]).
-
 
 rep_error_to_binary(Error) ->
     couch_util:to_binary(error_reason(Error)).
@@ -54,77 +48,27 @@ 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) ->
-    get_json_value(<<"deleted">>, Change, false).
-
-
-% 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).
+-spec iso8601() -> binary().
+iso8601() ->
+    iso8601(erlang:system_time(second)).
 
 
-sum_stats(S1, S2) ->
-    couch_replicator_stats:sum_stats(S1, S2).
-
-
-parse_rep_doc(Props, UserCtx) ->
-    couch_replicator_docs:parse_rep_doc(Props, UserCtx).
-
-
--spec iso8601(erlang:timestamp()) -> binary().
-iso8601({_Mega, _Sec, _Micro} = Timestamp) ->
-    {{Y, Mon, D}, {H, Min, S}} = calendar:now_to_universal_time(Timestamp),
+-spec iso8601(integer()) -> binary().
+iso8601(Sec) when is_integer(Sec) ->
+    Time = unix_sec_to_timestamp(Sec),
+    {{Y, Mon, D}, {H, Min, S}} = calendar:now_to_universal_time(Time),
     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])).
 
 
-%% Filter replication info ejson by state provided. If it matches return
-%% the input value, if it doesn't return 'skip'. This is used from replicator
-%% fabric coordinator and worker.
--spec filter_state(atom(), [atom()], {[_ | _]}) -> {[_ | _]} | skip.
-filter_state(null = _State, _States, _Info) ->
-    skip;
-filter_state(_ = _State, [] = _States, Info) ->
-    Info;
-filter_state(State, States, Info) ->
-    case lists:member(State, States) of
-        true ->
-            Info;
-        false ->
-            skip
-    end.
+rfc1123_local() ->
+    list_to_binary(httpd_util:rfc1123_date()).
+
+
+rfc1123_local(Sec) ->
+    Time = unix_sec_to_timestamp(Sec),
+    Local = calendar:now_to_local_time(Time),
+    list_to_binary(httpd_util:rfc1123_date(Local)).
 
 
 remove_basic_auth_from_headers(Headers) ->
@@ -158,37 +102,101 @@ decode_basic_creds(Base64) ->
     end.
 
 
-% Normalize a #rep{} record such that it doesn't contain time dependent fields
+-spec compare_reps(#{} | null, #{} | null) -> boolean().
+compare_reps(Rep1, Rep2) ->
+    NormRep1 = normalize_rep(Rep1),
+    NormRep2 = normalize_rep(Rep2),
+    NormRep1 =:= NormRep2.
+
+
+% Normalize a rep map 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 normalize_rep(#{} | null) -> #{} | null.
+normalize_rep(null) ->
+    null;
+
+normalize_rep(#{} = Rep)->
+    #{
+        ?SOURCE := Source,
+        ?TARGET := Target,
+        ?OPTIONS := Options
+    } = Rep,
+    #{
+        ?SOURCE => normalize_endpoint(Source),
+        ?TARGET => normalize_endpoint(Target),
+        ?OPTIONS => Options
     }.
 
 
--spec ejson_state_info(binary() | nil) -> binary() | null.
-ejson_state_info(nil) ->
-    null;
-ejson_state_info(Info) when is_binary(Info) ->
-    {[{<<"error">>, Info}]};
-ejson_state_info([]) ->
-    null;  % Status not set yet => null for compatibility reasons
-ejson_state_info([{_, _} | _] = Info) ->
-    {Info};
-ejson_state_info(Info) ->
-    ErrMsg = couch_replicator_utils:rep_error_to_binary(Info),
-    {[{<<"error">>, ErrMsg}]}.
+normalize_endpoint(<<DbName/binary>>) ->
+    DbName;
+
+normalize_endpoint(#{} = Endpoint) ->
+    Ks = [
+        <<"url">>,
+        <<"auth_props">>,
+        <<"headers">>,
+        <<"timeout">>,
+        <<"ibrowse_options">>,
+        <<"retries">>,
+        <<"http_connections">>,
+        <<"proxy_url">>
+    ],
+    maps:with(Ks, Endpoint).
+
+
+default_headers_map() ->
+    lists:foldl(fun({K, V}, Acc) ->
+        Acc#{list_to_binary(K) => list_to_binary(V)}
+    end, #{}, (#httpdb{})#httpdb.headers).
+
+
+parse_replication_states(undefined) ->
+    [];  % This is the default (wildcard) filter
+
+parse_replication_states(States) when is_list(States) ->
+    All = [?ST_RUNNING, ?ST_FAILED, ?ST_COMPLETED, ?ST_PENDING, ?ST_CRASHING],
+    AllSet = sets:from_list(All),
+    BinStates = [?l2b(string:to_lower(S)) || S <- string:tokens(States, ",")],
+    StatesSet = sets:from_list(BinStates),
+    Diff = sets:to_list(sets:subtract(StatesSet, AllSet)),
+    case Diff of
+        [] ->
+            BinStates;
+        _ ->
+            Args = [Diff, All],
+            Msg2 = io_lib:format("Unknown states ~p. Choose from: ~p", Args),
+            throw({query_parse_error, ?l2b(Msg2)})
+    end.
+
+
+parse_int_param(Req, Param, Default, Min, Max) ->
+    IntVal = try
+        list_to_integer(chttpd:qs_value(Req, Param, integer_to_list(Default)))
+    catch error:badarg ->
+        Msg1 = io_lib:format("~s must be an integer", [Param]),
+        throw({query_parse_error, ?l2b(Msg1)})
+    end,
+    case IntVal >= Min andalso IntVal =< Max of
+    true ->
+        IntVal;
+    false ->
+        Msg2 = io_lib:format("~s not in range of [~w,~w]", [Param, Min, Max]),
+        throw({query_parse_error, ?l2b(Msg2)})
+    end.
+
+
+proplist_options(#{} = OptionsMap) ->
+    maps:fold(fun(K, V, Acc) ->
+        [{binary_to_atom(K, utf8), V} | Acc]
+    end, [], OptionsMap).
+
+
+unix_sec_to_timestamp(Sec) when is_integer(Sec) ->
+    MegaSecPart = Sec div 1000000,
+    SecPart = Sec - MegaSecPart * 1000000,
+    {MegaSecPart, SecPart, 0}.
 
 
 -ifdef(TEST).
@@ -256,7 +264,7 @@ normalize_rep_test_() ->
                 {<<"doc_ids">>, [<<"a">>, <<"c">>, <<"b">>]},
                 {<<"other_field">>, <<"some_value">>}
             ]},
-            Rep1 = couch_replicator_docs:parse_rep_doc_without_id(EJson1),
+            Rep1 = couch_replicator_parse:parse_rep_doc(EJson1),
             EJson2 = {[
                 {<<"other_field">>, <<"unrelated">>},
                 {<<"target">>, <<"http://target.local/db">>},
@@ -264,9 +272,28 @@ normalize_rep_test_() ->
                 {<<"doc_ids">>, [<<"c">>, <<"a">>, <<"b">>]},
                 {<<"other_field2">>, <<"unrelated2">>}
             ]},
-            Rep2 = couch_replicator_docs:parse_rep_doc_without_id(EJson2),
+            Rep2 = couch_replicator_parse:parse_rep_doc(EJson2),
             ?assertEqual(normalize_rep(Rep1), normalize_rep(Rep2))
         end)
     }.
 
+
+normalize_endpoint() ->
+    HttpDb =  #httpdb{
+        url = "http://host/db",
+        auth_props = [{"key", "val"}],
+        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_endpoint(HttpDb)),
+    ?assertEqual(<<"local">>, normalize_endpoint(<<"local">>)).
+
+
 -endif.
diff --git a/src/couch_replicator/src/couch_replicator_worker.erl b/src/couch_replicator/src/couch_replicator_worker.erl
index eb8beaa..4cd984c 100644
--- a/src/couch_replicator/src/couch_replicator_worker.erl
+++ b/src/couch_replicator/src/couch_replicator_worker.erl
@@ -103,7 +103,7 @@ handle_call({batch_doc, Doc}, From, State) ->
 
 handle_call({add_stats, IncStats}, From, #state{stats = Stats} = State) ->
     gen_server:reply(From, ok),
-    NewStats = couch_replicator_utils:sum_stats(Stats, IncStats),
+    NewStats = couch_replicator_stats:sum_stats(Stats, IncStats),
     NewStats2 = maybe_report_stats(State#state.cp, NewStats),
     {noreply, State#state{stats = NewStats2}};
 


[couchdb] 08/16: Handle option maps in lower level modules

Posted by va...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

vatamane pushed a commit to branch prototype/fdb-layer
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit 941cfc3d7b33cbfbf7e95eb7db388515d0595399
Author: Nick Vatamaniuc <va...@apache.org>
AuthorDate: Fri Aug 28 04:33:05 2020 -0400

    Handle option maps in lower level modules
    
    The goal is to keep everything below the _api_wrap module level relatively
    intact. To achieve that handle option maps in some places, or translate back to
    a proplist or `#httpd{}` records in others.
    
    The `couch_replicator_api:db_from_json/1` function is where endpoint map object
    from a `Rep` object are translated into `#httpdb{}` records. Headers are
    translated back to lists and ibrowse options into proplist with atom keys.
---
 .../src/couch_replicator_api_wrap.erl              | 164 ++++++++++++---------
 .../src/couch_replicator_changes_reader.erl        |   9 +-
 .../src/couch_replicator_filters.erl               |  54 ++++---
 3 files changed, 127 insertions(+), 100 deletions(-)

diff --git a/src/couch_replicator/src/couch_replicator_api_wrap.erl b/src/couch_replicator/src/couch_replicator_api_wrap.erl
index a21de42..da6f288 100644
--- a/src/couch_replicator/src/couch_replicator_api_wrap.erl
+++ b/src/couch_replicator/src/couch_replicator_api_wrap.erl
@@ -28,7 +28,6 @@
     db_close/1,
     get_db_info/1,
     get_pending_count/2,
-    get_view_info/3,
     update_doc/3,
     update_doc/4,
     update_docs/3,
@@ -39,39 +38,28 @@
     open_doc_revs/6,
     changes_since/5,
     db_uri/1,
-    normalize_db/1
+    db_from_json/1
     ]).
 
--import(couch_replicator_httpc, [
-    send_req/3
-    ]).
-
--import(couch_util, [
-    encode_doc_id/1,
-    get_value/2,
-    get_value/3
-    ]).
 
 -define(MAX_WAIT, 5 * 60 * 1000).
 
 -define(MAX_URL_LEN, 7000).
 -define(MIN_URL_LEN, 200).
 
-db_uri(#httpdb{url = Url}) ->
+db_uri(#{<<"url">> := Url}) ->
     couch_util:url_strip_password(Url);
 
-db_uri(DbName) when is_binary(DbName) ->
-    ?b2l(DbName);
+db_uri(#httpdb{url = Url}) ->
+    couch_util:url_strip_password(Url).
 
-db_uri(Db) ->
-    db_uri(couch_db:name(Db)).
 
+db_open(#{} = Db) ->
+    db_open(Db, false, #{}).
 
-db_open(Db) ->
-    db_open(Db, false, []).
 
-db_open(#httpdb{} = Db1, Create, CreateParams) ->
-    {ok, Db} = couch_replicator_httpc:setup(Db1),
+db_open(#{} = Db0, Create, #{} = CreateParams) when is_boolean(Create) ->
+    {ok, Db} = couch_replicator_httpc:setup(db_from_json(Db0)),
     try
         case Create of
         false ->
@@ -149,14 +137,6 @@ get_pending_count(#httpdb{} = Db, Seq) ->
         {ok, couch_util:get_value(<<"pending">>, Props, null)}
     end).
 
-get_view_info(#httpdb{} = Db, DDocId, ViewName) ->
-    Path = io_lib:format("~s/_view/~s/_info", [DDocId, ViewName]),
-    send_req(Db, [{path, Path}],
-        fun(200, _, {Props}) ->
-            {VInfo} = couch_util:get_value(<<"view_index">>, Props, {[]}),
-            {ok, VInfo}
-        end).
-
 
 ensure_full_commit(#httpdb{} = Db) ->
     send_req(
@@ -434,9 +414,9 @@ changes_since(#httpdb{headers = Headers1, timeout = InactiveTimeout} = HttpDb,
     {undefined, undefined} ->
         QArgs1 = maybe_add_changes_filter_q_args(BaseQArgs, Options),
         {QArgs1, get, [], Headers1};
-    {undefined, _} when is_tuple(Selector) ->
+    {undefined, #{}} ->
         Headers2 = [{"Content-Type", "application/json"} | Headers1],
-        JsonSelector = ?JSON_ENCODE({[{<<"selector">>, Selector}]}),
+        JsonSelector = ?JSON_ENCODE(#{<<"selector">> => Selector}),
         {[{"filter", "_selector"} | BaseQArgs], post, JsonSelector, Headers2};
     {_, undefined} when is_list(DocIds) ->
         Headers2 = [{"Content-Type", "application/json"} | Headers1],
@@ -496,7 +476,8 @@ maybe_add_changes_filter_q_args(BaseQS, Options) ->
         ViewFields0 = [atom_to_list(F) || F <- record_info(fields,  mrargs)],
         ViewFields = ["key" | ViewFields0],
 
-        {Params} = get_value(query_params, Options, {[]}),
+        ParamsMap = #{} = get_value(query_params, Options, #{}),
+        Params = maps:to_list(ParamsMap),
         [{"filter", ?b2l(FilterName)} | lists:foldl(
             fun({K, V}, QSAcc) ->
                 Ks = couch_util:to_list(K),
@@ -546,7 +527,7 @@ options_to_query_args(HttpDb, Path, Options0) ->
             length("GET " ++ FullUrl ++ " HTTP/1.1\r\n") +
             length("&atts_since=") + 6,  % +6 = % encoded [ and ]
             PAs, MaxLen, []),
-        [{"atts_since", ?JSON_ENCODE(RevList)} | QueryArgs1]
+        [{"atts_since", ?b2l(iolist_to_binary(?JSON_ENCODE(RevList)))} | QueryArgs1]
     end.
 
 
@@ -787,7 +768,7 @@ json_to_doc_info({Props}) ->
         RevsInfo0 = lists:map(
             fun({Change}) ->
                 Rev = couch_doc:parse_rev(get_value(<<"rev">>, Change)),
-                Del = couch_replicator_utils:is_deleted(Change),
+                Del = get_value(<<"deleted">>, Change, false),
                 #rev_info{rev=Rev, deleted=Del}
             end, Changes),
 
@@ -895,52 +876,95 @@ header_value(Key, Headers, 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) ->
+maybe_append_create_query_params(Db, Params) when map_size(Params) == 0 ->
+    Db;
+
+maybe_append_create_query_params(Db, #{} = Params) ->
+    ParamList = maps:to_list(Params),
+    NewUrl = Db#httpdb.url ++ "?" ++ mochiweb_util:urlencode(ParamList),
+    Db#httpdb{url = NewUrl}.
+
+
+db_from_json(#{} = DbMap) ->
+    #{
+        <<"url">> := Url,
+        <<"auth_props">> := Auth,
+        <<"headers">> := Headers0,
+        <<"ibrowse_options">> := IBrowseOptions0,
+        <<"timeout">> := Timeout,
+        <<"http_connections">> := HttpConnections,
+        <<"retries">> := Retries,
+        <<"proxy_url">> := ProxyUrl0
+    } = DbMap,
+    Headers = maps:fold(fun(K, V, Acc) ->
+        [{binary_to_list(K), binary_to_list(V)} | Acc]
+    end, [], Headers0),
+    IBrowseOptions = maps:fold(fun
+        (<<"socket_options">>, #{} = SockOpts, Acc) ->
+            SockOptsKVs = maps:fold(fun sock_opts_fold/3, [], SockOpts),
+            [{socket_options, SockOptsKVs} | Acc];
+        (<<"ssl_options">>, #{} = SslOpts, Acc) ->
+            SslOptsKVs = maps:fold(fun ssl_opts_fold/3, [], SslOpts),
+            [{ssl_options, SslOptsKVs} | Acc];
+        (K, V, Acc) when is_binary(V) ->
+            [{binary_to_atom(K, utf8), binary_to_list(V)} | Acc];
+        (K, V, Acc) ->
+            [{binary_to_atom(K, utf8), V} | Acc]
+    end, [], IBrowseOptions0),
+    ProxyUrl = case ProxyUrl0 of
+        null -> undefined;
+        V when is_binary(V) -> binary_to_list(V)
+    end,
     #httpdb{
-        url = HttpDb#httpdb.url,
-        auth_props = lists:sort(HttpDb#httpdb.auth_props),
-        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
-    };
+        url = binary_to_list(Url),
+        auth_props = maps:to_list(Auth),
+        headers = Headers,
+        ibrowse_options = IBrowseOptions,
+        timeout = Timeout,
+        http_connections = HttpConnections,
+        retries = Retries,
+        proxy_url = ProxyUrl
+    }.
 
-normalize_db(<<DbName/binary>>) ->
-    DbName.
 
+send_req(#httpdb{} = HttpDb, Opts, Callback) when is_function(Callback) ->
+    couch_replicator_httpc:send_req(HttpDb, Opts, Callback).
 
-maybe_append_create_query_params(Db, []) ->
-    Db;
 
-maybe_append_create_query_params(Db, CreateParams) ->
-    NewUrl = Db#httpdb.url ++ "?" ++ mochiweb_util:urlencode(CreateParams),
-    Db#httpdb{url = NewUrl}.
+get_value(K, Props) ->
+    couch_util:get_value(K, Props).
+
+
+get_value(K, Props, Default) ->
+    couch_util:get_value(K, Props, Default).
 
 
--ifdef(TEST).
+encode_doc_id(DocId) ->
+    couch_util:encode_doc_id(DocId).
 
--include_lib("eunit/include/eunit.hrl").
 
 
-normalize_http_db_test() ->
-    HttpDb =  #httpdb{
-        url = "http://host/db",
-        auth_props = [{"key", "val"}],
-        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">>)).
+% See couch_replicator_docs:ssl_params/1 for ssl parsed options
+% and http://erlang.org/doc/man/ssl.html#type-server_option
+% all latest SSL server options
+%
+ssl_opts_fold(K, V, Acc) when is_boolean(V); is_integer(V) ->
+    [{binary_to_atom(K, utf8), V} | Acc];
+
+ssl_opts_fold(K, null, Acc) ->
+    [{binary_to_atom(K, utf8), undefined} | Acc];
+
+ssl_opts_fold(<<"verify">>, V, Acc) ->
+    [{verify, binary_to_atom(V, utf8)} | Acc];
 
+ssl_opts_fold(K, V, Acc) when is_list(V) ->
+    [{binary_to_atom(K, utf8), binary_to_list(V)} | Acc].
+
+
+% See ?VALID_SOCK_OPTS in couch_replicator_docs for accepted socket options
+%
+sock_opts_fold(K, V, Acc) when is_binary(V) ->
+     [{binary_to_atom(K, utf8), binary_to_atom(V, utf8)} | Acc];
 
--endif.
+sock_opts_fold(K, V, Acc) when is_boolean(V); is_integer(V) ->
+    [{binary_to_atom(K, utf8), V} | Acc].
diff --git a/src/couch_replicator/src/couch_replicator_changes_reader.erl b/src/couch_replicator/src/couch_replicator_changes_reader.erl
index 2e4df53..6adf1af 100644
--- a/src/couch_replicator/src/couch_replicator_changes_reader.erl
+++ b/src/couch_replicator/src/couch_replicator_changes_reader.erl
@@ -22,11 +22,8 @@
 -include_lib("couch_replicator/include/couch_replicator_api_wrap.hrl").
 -include("couch_replicator.hrl").
 
--import(couch_util, [
-    get_value/2
-]).
 
-start_link(StartSeq, #httpdb{} = Db, ChangesQueue, Options) ->
+start_link(StartSeq, #httpdb{} = Db, ChangesQueue, #{} = Options) ->
     Parent = self(),
     {ok, spawn_link(fun() ->
         put(last_seq, StartSeq),
@@ -41,12 +38,12 @@ start_link(StartSeq, Db, ChangesQueue, Options) ->
     end)}.
 
 read_changes(Parent, StartSeq, Db, ChangesQueue, Options) ->
-    Continuous = couch_util:get_value(continuous, Options),
+    Continuous = maps:get(<<"continuous">>, Options, false),
     try
         couch_replicator_api_wrap:changes_since(Db, all_docs, StartSeq,
             fun(Item) ->
                 process_change(Item, {Parent, Db, ChangesQueue, Continuous})
-            end, Options),
+            end, couch_replicator_utils:proplist_options(Options)),
         couch_work_queue:close(ChangesQueue)
     catch
         throw:recurse ->
diff --git a/src/couch_replicator/src/couch_replicator_filters.erl b/src/couch_replicator/src/couch_replicator_filters.erl
index c898000..50c3733 100644
--- a/src/couch_replicator/src/couch_replicator_filters.erl
+++ b/src/couch_replicator/src/couch_replicator_filters.erl
@@ -20,6 +20,7 @@
 ]).
 
 -include_lib("couch/include/couch_db.hrl").
+-include("couch_replicator.hrl").
 
 
 % Parse the filter from replication options proplist.
@@ -27,17 +28,17 @@
 % 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([_]) ->
+-spec parse(#{}) ->
     {ok, nil} |
     {ok, {view, binary(), {[_]}}} |
     {ok, {user, {binary(), binary()}, {[_]}}} |
     {ok, {docids, [_]}} |
     {ok, {mango, {[_]}}} |
     {error, binary()}.
-parse(Options) ->
-    Filter = couch_util:get_value(filter, Options),
-    DocIds = couch_util:get_value(doc_ids, Options),
-    Selector = couch_util:get_value(selector, Options),
+parse(#{} = Options) ->
+    Filter = maps:get(<<"filter">>, Options, undefined),
+    DocIds = maps:get(<<"doc_ids">>, Options, undefined),
+    Selector = maps:get(<<"selector">>, Options, undefined),
     case {Filter, DocIds, Selector} of
         {undefined, undefined, undefined} ->
             {ok, nil};
@@ -53,7 +54,10 @@ parse(Options) ->
         {undefined, _, undefined} ->
             {ok, {docids, DocIds}};
         {undefined, undefined, _} ->
-            {ok, {mango, ejsort(mango_selector:normalize(Selector))}};
+            % Translate it to proplist as normalize doesn't know how
+            % to handle maps
+            Selector1 = ?JSON_DECODE(?JSON_ENCODE(Selector)),
+            {ok, {mango, ejsort(mango_selector:normalize(Selector1))}};
         _ ->
             Err = "`selector`, `filter` and `doc_ids` are mutually exclusive",
             {error, list_to_binary(Err)}
@@ -88,22 +92,24 @@ fetch(DDocName, FilterName, Source) ->
 
 
 % 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 couch_util:get_value(<<"filter">>, Props) of
-        <<"_view">> ->
-            {QP}  = couch_util:get_value(query_params, Options, {[]}),
-            ViewParam = couch_util:get_value(<<"view">>, QP),
-            case re:split(ViewParam, <<"/">>) of
-                [DName, ViewName] ->
-                    {view, {<< "_design/", DName/binary >>, ViewName}};
-                _ ->
-                    {error, <<"Invalid `view` parameter.">>}
-            end;
+-spec view_type(#{}, #{}) ->
+    {binary(), #{}} | {error, binary()}.
+view_type(#{?FILTER := <<"_view">>}, #{} = Options) ->
+    QP = maps:get(<<"query_params">>, Options, #{}),
+    ViewParam = maps:get(<<"view">>, QP, <<>>),
+    case re:split(ViewParam, <<"/">>) of
+        [DName, ViewName] ->
+            DDocMap = #{
+                <<"ddoc">> => <<"_design/",DName/binary>>,
+                <<"view">> => ViewName
+            },
+            {<<"view">>, DDocMap};
         _ ->
-            {db, nil}
-    end.
+            {error, <<"Invalid `view` parameter.">>}
+    end;
+
+view_type(#{}, #{}) ->
+    {<<"db">>, #{}}.
 
 
 % Private functions
@@ -151,9 +157,9 @@ fetch_internal(DDocName, FilterName, Source) ->
     end.
 
 
--spec query_params([_]) -> {[_]}.
-query_params(Options)->
-    couch_util:get_value(query_params, Options, {[]}).
+-spec query_params(#{}) -> #{}.
+query_params(#{} = Options)->
+    maps:get(<<"query_params">>, Options, #{}).
 
 
 parse_user_filter(Filter) ->


[couchdb] 15/16: Update replicator's readme file

Posted by va...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

vatamane pushed a commit to branch prototype/fdb-layer
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit 99262909129602bceac82e7907ebfcafc9eba629
Author: Nick Vatamaniuc <va...@apache.org>
AuthorDate: Fri Aug 28 04:36:12 2020 -0400

    Update replicator's readme file
    
    Remove sections which don't apply anymore and describe briefly how frontend and
    backend interact.
---
 src/couch_replicator/README.md | 312 ++++++++---------------------------------
 1 file changed, 61 insertions(+), 251 deletions(-)

diff --git a/src/couch_replicator/README.md b/src/couch_replicator/README.md
index 6a2a5cf..4eced76 100644
--- a/src/couch_replicator/README.md
+++ b/src/couch_replicator/README.md
@@ -3,41 +3,67 @@ Developer Oriented Replicator Description
 
 This description of scheduling replicator's functionality is mainly geared to
 CouchDB developers. It dives a bit into the internal and explains how
-everything is connected together.
+everything is connected together. A higher level overview is available in the
+[RFC](https://github.com/apache/couchdb-documentation/pull/581). This
+documention assumes the audience is familiar with that description as well as
+with the [Couch Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md)
+as well as with the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
 
 A natural place to start is the top application supervisor:
-`couch_replicator_sup`. It's a `rest_for_one` restart strategy supervisor,
-so if a child process terminates, the rest of the children in the hierarchy
-following it are also terminated. This structure implies a useful constraint --
-children lower in the list can safely call their siblings which are higher in
-the list.
-
-A description of each child:
-
- * `couch_replication_event`: Starts a gen_event publication bus to handle some
-    replication related events. This used for example, to publish cluster
-    membership changes by the `couch_replicator_clustering` process. But is
-    also used in replication tests to monitor for replication events.
-    Notification is performed via the `couch_replicator_notifier:notify/1`
-    function. It's the first (left-most) child because
-    `couch_replicator_clustering` uses it.
-
- * `couch_replicator_clustering`: This module maintains cluster membership
-    information for the replication application and provides functions to check
-    ownership of replication jobs. A cluster membership change is published via
-    the `gen_event` event server named `couch_replication_event` as previously
-    covered. Published events are `{cluster, stable}` when cluster membership
-    has stabilized, that it, no node membership changes in a given period, and
-    `{cluster, unstable}` which indicates there was a recent change to the
-    cluster membership and now it's considered unstable. Listeners for cluster
-    membership change include `couch_replicator_doc_processor` and
-    `couch_replicator_db_changes`. When doc processor gets an `{cluster,
-    stable}` event it will remove all the replication jobs not belonging to the
-    current node. When `couch_replicator_db_changes` gets a `{cluster,
-    stable}` event, it will restart the `couch_multidb_changes` process it
-    controls, which will launch an new scan of all the replicator databases.
-
-  * `couch_replicator_connection`: Maintains a global replication connection
+`couch_replicator_sup`. The set of children in the supervisor is split into
+`frontend` and `backend`. The `frontend` set is started on nodes which have the
+`api_frontend` node type label set to `true`, and `backend` ones are started on
+nodes which have the `replication` label set to `true`. The same node could
+have both them set to `true`, and it could act as a replication front and
+backend node. However, it is not guaranteed that jobs which are created by the
+frontend part will necessarily run on the backend on the same node.
+
+
+Frontend Description
+--
+
+The "frontend" consists of the parts which handle HTTP requests and monitor
+`_replicator` databases for changes and then create `couch_jobs` replication
+job records. Some of the modules involved in this are:
+
+ * `couch_replicator` : Contains the main API "entry" point into the
+   `couch_replicator` application. The `replicate/2` function creates transient
+   replication jobs. `after_db_create/2`, `after_db_delete/2`,
+   `after_doc_write/6` functions are called from `couch_epi` callbacks to
+   create replication jobs from `_replicator` db events. Eventually they all
+   call `couch_replicator_jobs:add_job/3` to create a `couch_jobs` replication
+   job. Before the job is created, either the HTTP request body or the
+   `_replicator` doc body is parsed into a `Rep` map object. An important
+   property of this object is that it can be serialized to JSON and
+   deserialized from JSON. This object is saved in the `?REP` field of the
+   replication `couch_jobs` job data. Besides creating replication job
+   `couch_replicator` is also responsible for handling `_scheduler/jobs` and
+   `_scheduler/docs` monitoring API response. That happens in the `jobs/0`,
+   `job/1`, `docs/` and `doc/2` function.
+
+Backend Description
+--
+
+The "backend" consists of parts which run replication jobs, update their state,
+and handle rescheduling on intermettent errors. All the job activity on these
+nodes is ultumately driven from `couch_jobs` acceptors which wait in
+`couch_jobs:accept/2` for replication jobs.
+
+ * `couch_replicator_job_server` : A singleton process in charge of which
+   spawning and keeping track of `couch_replicator_job` processes. It ensures
+   there is a limited number of replication jobs running on each node. It
+   periodically accepts new jobs and stopping the oldest running ones in order
+   to give other pending jobs a chance to run. It runs this logic in the
+   `reschedule/1` function. That function is called with a frequency defined by
+   the `interval_sec` configuration setting. The other pramers which determine
+   how jobs start and stop are `max_jobs` and `max_churn`. The node will try to
+   limit running up to `max_jobs` job on average with periodic spikes of up to
+   `max_jobs + max_churn` job at a time, and it will try not to start more than
+   `max_churn` number of job during each rescheduling cycle.
+
+ * `couch_replicator_connection`: Maintains a global replication connection
     pool. It allows reusing connections across replication tasks. The main
     interface is `acquire/1` and `release/1`. The general idea is once a
     connection is established, it is kept around for
@@ -62,224 +88,8 @@ A description of each child:
     interval is updated accordingly on each call to `failure/1` or `success/1`
     calls. For a successful request, a client should call `success/1`. Whenever
     a 429 response is received the client should call `failure/1`. When no
-    failures are happening the code ensures the ETS tables are empty in
-    order to have a lower impact on a running system.
-
- * `couch_replicator_scheduler`: This is the core component of the scheduling
-    replicator. It's main task is to switch between replication jobs, by
-    stopping some and starting others to ensure all of them make progress.
-    Replication jobs which fail are penalized using an exponential backoff.
-    That is, each consecutive failure will double the time penalty. This frees
-    up system resources for more useful work than just continuously trying to
-    run the same subset of failing jobs.
-
-    The main API function is `add_job/1`. Its argument is an instance of the
-    `#rep{}` record, which could be the result of a document update from a
-    `_replicator` db or the result of a POST to `_replicate` endpoint.
-
-    Each job internally is represented by the `#job{}` record. It contains the
-    original `#rep{}` but also, maintains an event history. The history is a
-    sequence of past events for each job. These are timestamped and ordered
-    such that the most recent event is at the head. History length is limited
-    based on the `replicator.max_history` configuration value. The default is
-    20 entries. History events types are:
-
-    * `added` : job was just added to the scheduler. This is the first event.
-    * `started` : job was started. This was an attempt to run the job.
-    * `stopped` : job was stopped by the scheduler.
-    * `crashed` : job has crashed (instead of stopping cleanly).
-
-    The core of the scheduling algorithm is the `reschedule/1` function. This
-    function is called every `replicator.interval` milliseconds (default is
-    60000 i.e. a minute). During each call the scheduler will try to stop some
-    jobs, start some new ones and will also try to keep the maximum number of
-    jobs running less than `replicator.max_jobs` (default 500). So the
-    functions does these operations (actual code paste):
-
-    ```
-    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(State#state.stats_pid)
-    ```
-
-    `Running` is the total number of currently running jobs. `Pending` is the
-    total number of jobs waiting to be run. `stop_excess_jobs` will stop any
-    exceeding the `replicator.max_jobs` configured limit. This code takes
-    effect if user reduces the `max_jobs` configuration value.
-    `start_pending_jobs` will start any jobs if there is more room available.
-    This will take effect on startup or when user increases the `max_jobs`
-    configuration value. `rotate_jobs` is where all the action happens. The
-    scheduler picks `replicator.max_churn` running jobs to stop and then picks
-    the same number of pending jobs to start. The default value of `max_churn`
-    is 20. So by default every minute, 20 running jobs are stopped, and 20 new
-    pending jobs are started.
-
-    Before moving on it is worth pointing out that scheduler treats continuous
-    and non-continuous replications differently. Normal (non-continuous)
-    replications once started will be allowed to run to completion. That
-    behavior is to preserve their semantics of replicating a snapshot of the
-    source database to the target. For example if new documents are added to
-    the source after the replication are started, those updates should not show
-    up on the target database. Stopping and restarting a normal replication
-    would violate that constraint. The only exception to the rule is the user
-    explicitly reduces `replicator.max_jobs` configuration value. Even then
-    scheduler will first attempt to stop as many continuous jobs as possible
-    and only if it has no choice left will it stop normal jobs.
-
-    Keeping that in mind and going back to the scheduling algorithm, the next
-    interesting part is how the scheduler picks which jobs to stop and which
-    ones to start:
-
-    * Stopping: When picking jobs to stop the scheduler will pick longest
-      running continuous jobs first. The sorting callback function to get the
-      longest running jobs is unsurprisingly called `longest_running/2`. To
-      pick the longest running jobs it looks at the most recent `started`
-      event. After it gets a sorted list by longest running, it simply picks
-      first few depending on the value of `max_churn` using `lists:sublist/2`.
-      Then those jobs are stopped.
-
-    * Starting: When starting the scheduler will pick the jobs which have been
-      waiting the longest. Surprisingly, in this case it also looks at the
-      `started` timestamp and picks the jobs which have the oldest `started`
-      timestamp. If there are 3 jobs, A[started=10], B[started=7],
-      C[started=9], then B will be picked first, then C then A. This ensures
-      that jobs are not starved, which is a classic scheduling pitfall.
-
-    In the code, the list of pending jobs is picked slightly differently than
-    how the list of running jobs is picked. `pending_jobs/1` uses `ets:foldl`
-    to iterate over all the pending jobs. As it iterates it tries to keep only
-    up to `max_churn` oldest items in the accumulator. The reason this is done
-    is that there could be a very large number of pending jobs and loading them
-    all in a list (making a copy from ETS) and then sorting it can be quite
-    expensive performance-wise. The tricky part of the iteration is happening
-    in `pending_maybe_replace/2`. A `gb_sets` ordered set is used to keep top-N
-    longest waiting jobs so far. The code has a comment with a helpful example
-    on how this algorithm works.
-
-    The last part is how the scheduler treats jobs which keep crashing. If a
-    job is started but then crashes then that job is considered unhealthy. The
-    main idea is to penalize such jobs such that they are forced to wait an
-    exponentially larger amount of time with each consecutive crash. A central
-    part to this algorithm is determining what forms a sequence of consecutive
-    crashes. If a job starts then quickly crashes, and after its next start it
-    crashes again, then that would become a sequence of 2 consecutive crashes.
-    The penalty then would be calculated by `backoff_micros/1` function where
-    the consecutive crash count would end up as the exponent. However for
-    practical concerns there is also maximum penalty specified and that's the
-    equivalent of 10 consecutive crashes. Timewise it ends up being about 8
-    hours. That means even a job which keep crashing will still get a chance to
-    retry once in 8 hours.
-
-    There is subtlety when calculating consecutive crashes and that is deciding
-    when the sequence stops. That is, figuring out when a job becomes healthy
-    again. The scheduler considers a job healthy again if it started and hasn't
-    crashed in a while. The "in a while" part is a configuration parameter
-    `replicator.health_threshold` defaulting to 2 minutes. This means if job
-    has been crashing, for example 5 times in a row, but then on the 6th
-    attempt it started and ran for more than 2 minutes then it is considered
-    healthy again. The next time it crashes its sequence of consecutive crashes
-    will restart at 1.
-
- * `couch_replicator_scheduler_sup`: This module is a supervisor for running
-   replication tasks. The most interesting thing about it is perhaps that it is
-   not used to restart children. The scheduler handles restarts and error
-   handling backoffs.
-
- * `couch_replicator_doc_processor`: The doc processor component is in charge
-   of processing replication document updates, turning them into replication
-   jobs and adding those jobs to the scheduler. Unfortunately the only reason
-   there is even a `couch_replicator_doc_processor` gen_server, instead of
-   replication documents being turned to jobs and inserted into the scheduler
-   directly, is because of one corner case -- filtered replications using
-   custom (JavaScript mostly) filters. More about this later. It is better to
-   start with how updates flow through the doc processor:
-
-   Document updates come via the `db_change/3` callback from
-   `couch_multidb_changes`, then go to the `process_change/2` function.
-
-   In `process_change/2` a few decisions are made regarding how to proceed. The
-   first is "ownership" check. That is a check if the replication document
-   belongs on the current node. If not, then it is ignored. In a cluster, in
-   general there would be N copies of a document change and we only want to run
-   the replication once. Another check is to see if the update has arrived
-   during a time when the cluster is considered "unstable". If so, it is
-   ignored, because soon enough a rescan will be launched and all the documents
-   will be reprocessed anyway. Another noteworthy thing in `process_change/2`
-   is handling of upgrades from the previous version of the replicator when
-   transient states were written to the documents. Two such states were
-   `triggered` and `error`. Both of those states are removed from the document
-   then then update proceeds in the regular fashion. `failed` documents are
-   also ignored here. `failed` is a terminal state which indicates the document
-   was somehow unsuitable to become a replication job (it was malformed or a
-   duplicate). Otherwise the state update proceeds to `process_updated/2`.
-
-   `process_updated/2` is where replication document updates are parsed and
-   translated to `#rep{}` records. The interesting part here is that the
-   replication ID isn't calculated yet. Unsurprisingly the parsing function
-   used is called `parse_rep_doc_without_id/1`. Also note that up until now
-   everything is still running in the context of the `db_change/3` callback.
-   After replication filter type is determined the update gets passed to the
-   `couch_replicator_doc_processor` gen_server.
-
-   The `couch_replicator_doc_processor` gen_server's main role is to try to
-   calculate replication IDs for each `#rep{}` record passed to it, then add
-   that as a scheduler job. As noted before, `#rep{}` records parsed up until
-   this point lack a replication ID. The reason is replication ID calculation
-   includes a hash of the filter code. And because user defined replication
-   filters live in the source DB, which most likely involves a remote network
-   fetch there is a possibility of blocking and a need to handle various
-   network failures and retries. Because of that `replication_doc_processor`
-   dispatches all of that blocking and retrying to a separate `worker` process
-   (`couch_replicator_doc_processor_worker` module).
-
-   `couch_replicator_doc_processor_worker` is where replication IDs are
-   calculated for each individual doc update. There are two separate modules
-   which contain utilities related to replication ID calculation:
-   `couch_replicator_ids` and `couch_replicator_filters`. The first one
-   contains ID calculation algorithms and the second one knows how to parse and
-   fetch user filters from a remote source DB. One interesting thing about the
-   worker is that it is time-bounded and is guaranteed to not be stuck forever.
-   That's why it spawns an extra process with `spawn_monitor`, just so it can
-   do an `after` clause in receive and bound the maximum time this worker will
-   take.
-
-   A doc processor worker will either succeed or fail but never block for too
-   long. Success and failure are returned as exit values. Those are handled in
-   the `worker_returned/3` doc processor clauses. The most common pattern is
-   that a worker is spawned to add a replication job, it does so and returns a
-   `{ok, ReplicationID}` value in `worker_returned`.
-
-   In case of a filtered replication with custom user code there are two case to
-   consider:
-
-     1. Filter fetching code has failed. In that case worker returns an error.
-        But because the error could be a transient network error, another
-        worker is started to try again. It could fail and return an error
-        again, then another one is started and so on. However each consecutive
-        worker will do an exponential backoff, not unlike the scheduler code.
-        `error_backoff/1` is where the backoff period is calculated.
-        Consecutive errors are held in the `errcnt` field in the ETS table.
-
-     2. Fetching filter code succeeds, replication ID is calculated and job is
-        added to the scheduler. However, because this is a filtered replication
-        the source database could get an updated filter. Which means
-        replication ID could change again. So the worker is spawned to
-        periodically check the filter and see if it changed. In other words doc
-        processor will do the work of checking for filtered replications, get
-        an updated filter and will then refresh the replication job (remove the
-        old one and add a new one with a different ID). The filter checking
-        interval is determined by the `filter_backoff` function. An unusual
-        thing about that function is it calculates the period based on the size
-        of the ETS table. The idea there is for a few replications in a
-        cluster, it's ok to check filter changes often. But when there are lots
-        of replications running, having each one checking their filter often is
-        not a good idea.
+    failures are happening the code ensures the ETS tables are empty in order
+    to have a lower impact on a running system.
 
- * `couch_replicator_db_changes`: This process specializes and configures
-   `couch_multidb_changes` so that it looks for `_replicator` suffixed shards
-   and makes sure to restart it when node membership changes.
 
 


[couchdb] 09/16: Update couch_replicator_ids

Posted by va...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

vatamane pushed a commit to branch prototype/fdb-layer
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit 276d19731bc5df73838f40efc126f1f709e04fbe
Author: Nick Vatamaniuc <va...@apache.org>
AuthorDate: Fri Aug 28 04:33:11 2020 -0400

    Update couch_replicator_ids
    
    This module is responsible for calculating replication IDs. It inspects all the
    replication options which may affect the replication results and hashes them
    into a single ID. CouchDB replicator tries to maintain compatibility with older
    versions of itself so it keep tracks of how to calculate replication IDs used
    by previous version of CouchDB. Replication ID calculation algorithms have
    their own version, the latest one is at version 4.
    
    One of the goals of this update is to not alter the replication ID algorithm
    and keep it at version 4, such that for all the same parameters the replication
    IDs should stay the same as they would be on CouchDB <= 3.x. That is why in
    some internal function, options maps and binares are turned back into proplist
    and tuples before hashing is performed. There is a unit tests which asserts
    that the replication ID calcuated with this update matches what was calcuated
    in CouchDB 3.x.
    
    Internal representation of the replication ID has changed slighly. Previously
    it was represented by a tuple of `{BaseId, ExtId}`, where `BaseId` was the ID
    without any options such as `continuous` or `create_target`, and `ExtId` was
    the concatenated list of those options. In most cases it was useful to operate
    on the full ID and in only a few place the `BaseId` was needed. So the
    calculation function was updated to return `{RepId, BaseId}` instead. `RepId`
    is a binary that is the full relication ID (base + extensions) and `BaseId` is
    just the base.
    
    The function which calculated the base ID was updated to actually be called
    `base_id/2` as opposed to `replication_id/2`.
    
    Another update to the module is a function which calculates replication job
    IDs. A `JobId` is used to identify replication jobs in the `couch_jobs` API. A
    `JobId`, unlike a `RepId` never requires making a network round-trip to
    calculate. For replications created from `_replicator` docs, `JobId` is defined
    as the concatenation of the database instance UUID and document ID. For a
    transient jobs it is calculated by hashing the source, target endpoint
    parameters, replication options. In fact, it is almost the same as a
    replication ID, with one important difference that the filter design doc name
    and function name are used instead of the contents of the filter from the
    source, so no network round-trip is necessary to calculate it.
---
 src/couch_replicator/src/couch_replicator_ids.erl | 202 +++++++++++++++-------
 1 file changed, 141 insertions(+), 61 deletions(-)

diff --git a/src/couch_replicator/src/couch_replicator_ids.erl b/src/couch_replicator/src/couch_replicator_ids.erl
index 04e71c3..d1cbe57 100644
--- a/src/couch_replicator/src/couch_replicator_ids.erl
+++ b/src/couch_replicator/src/couch_replicator_ids.erl
@@ -14,7 +14,9 @@
 
 -export([
     replication_id/1,
-    replication_id/2,
+    base_id/2,
+    job_id/3,
+    job_id/2,
     convert/1
 ]).
 
@@ -30,28 +32,31 @@
 %  {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)}.
+replication_id(#{?OPTIONS := Options} = Rep) ->
+    BaseId = base_id(Rep, ?REP_ID_VERSION),
+    UseOpts = [<<"continuous">>, <<"create_target">>],
+    ExtId = maybe_append_options(UseOpts, Options),
+    RepId = iolist_to_binary([BaseId, ExtId]),
+    {RepId, BaseId}.
 
 
 % 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{} = Rep, 4) ->
+base_id(#{?SOURCE := Src, ?TARGET := Tgt} = Rep, 4) ->
     UUID = couch_server:get_uuid(),
-    SrcInfo = get_v4_endpoint(Rep#rep.source),
-    TgtInfo = get_v4_endpoint(Rep#rep.target),
+    SrcInfo = get_v4_endpoint(Src),
+    TgtInfo = get_v4_endpoint(Tgt),
     maybe_append_filters([UUID, SrcInfo, TgtInfo], Rep);
 
-replication_id(#rep{} = Rep, 3) ->
+base_id(#{?SOURCE := Src0, ?TARGET := Tgt0} = Rep, 3) ->
     UUID = couch_server:get_uuid(),
-    Src = get_rep_endpoint(Rep#rep.source),
-    Tgt = get_rep_endpoint(Rep#rep.target),
+    Src = get_rep_endpoint(Src0),
+    Tgt = get_rep_endpoint(Tgt0),
     maybe_append_filters([UUID, Src, Tgt], Rep);
 
-replication_id(#rep{} = Rep, 2) ->
+base_id(#{?SOURCE := Src0, ?TARGET := Tgt0} = Rep, 2) ->
     {ok, HostName} = inet:gethostname(),
     Port = case (catch mochiweb_socket_server:get(couch_httpd, port)) of
     P when is_number(P) ->
@@ -64,47 +69,76 @@ replication_id(#rep{} = Rep, 2) ->
         % ... mochiweb_socket_server:get(https, port)
         list_to_integer(config:get("httpd", "port", "5984"))
     end,
-    Src = get_rep_endpoint(Rep#rep.source),
-    Tgt = get_rep_endpoint(Rep#rep.target),
+    Src = get_rep_endpoint(Src0),
+    Tgt = get_rep_endpoint(Tgt0),
     maybe_append_filters([HostName, Port, Src, Tgt], Rep);
 
-replication_id(#rep{} = Rep, 1) ->
+base_id(#{?SOURCE := Src0, ?TARGET := Tgt0} = Rep, 1) ->
     {ok, HostName} = inet:gethostname(),
-    Src = get_rep_endpoint(Rep#rep.source),
-    Tgt = get_rep_endpoint(Rep#rep.target),
+    Src = get_rep_endpoint(Src0),
+    Tgt = get_rep_endpoint(Tgt0),
     maybe_append_filters([HostName, Src, Tgt], Rep).
 
 
--spec convert([_] | binary() | {string(), string()}) -> {string(), string()}.
-convert(Id) when is_list(Id) ->
-    convert(?l2b(Id));
+-spec job_id(#{}, binary() | null, binary() | null) -> binary().
+job_id(#{} = Rep, null = _DbUUID, null = _DocId) ->
+    #{
+        ?SOURCE := Src,
+        ?TARGET := Tgt,
+        ?REP_USER := UserName,
+        ?OPTIONS := Options
+    } = Rep,
+    UUID = couch_server:get_uuid(),
+    SrcInfo = get_v4_endpoint(Src),
+    TgtInfo = get_v4_endpoint(Tgt),
+    UseOpts = [<<"continuous">>, <<"create_target">>],
+    Opts = maybe_append_options(UseOpts, Options),
+    IdParts = [UUID, SrcInfo, TgtInfo, UserName, Opts],
+    maybe_append_filters(IdParts, Rep, false);
+
+job_id(#{} = _Rep, DbUUID, DocId) when is_binary(DbUUID), is_binary(DocId) ->
+    job_id(DbUUID, DocId).
+
+
+-spec job_id(binary(), binary()) -> binary().
+job_id(DbUUID, DocId) when is_binary(DbUUID), is_binary(DocId) ->
+    <<DbUUID/binary, "|", DocId/binary>>.
+
+
+-spec convert(binary()) -> binary().
 convert(Id0) when is_binary(Id0) ->
     % Spaces can result from mochiweb incorrectly unquoting + characters from
     % the URL path. So undo the incorrect parsing here to avoid forcing
     % users to url encode + characters.
-    Id = binary:replace(Id0, <<" ">>, <<"+">>, [global]),
-    lists:splitwith(fun(Char) -> Char =/= $+ end, ?b2l(Id));
-convert({BaseId, Ext} = Id) when is_list(BaseId), is_list(Ext) ->
-    Id.
+    binary:replace(Id0, <<" ">>, <<"+">>, [global]).
 
 
 % Private functions
 
-maybe_append_filters(Base,
-        #rep{source = Source, options = Options}) ->
+maybe_append_filters(Base, #{} = Rep) ->
+    maybe_append_filters(Base, Rep, true).
+
+
+maybe_append_filters(Base, #{} = Rep, FetchFilter) ->
+    #{
+        ?SOURCE := Source,
+        ?OPTIONS := Options
+    } = Rep,
     Base2 = Base ++
         case couch_replicator_filters:parse(Options) of
         {ok, nil} ->
             [];
         {ok, {view, Filter, QueryParams}} ->
             [Filter, QueryParams];
-        {ok, {user, {Doc, Filter}, QueryParams}} ->
+        {ok, {user, {Doc, Filter}, QueryParams}} when FetchFilter =:= true ->
             case couch_replicator_filters:fetch(Doc, Filter, Source) of
                 {ok, Code} ->
                     [Code, QueryParams];
                 {error, Error} ->
                     throw({filter_fetch_error, Error})
             end;
+        {ok, {user, {Doc, Filter}, QueryParams}} when FetchFilter =:= false ->
+            [Doc, Filter, QueryParams];
         {ok, {docids, DocIds}} ->
             [DocIds];
         {ok, {mango, Selector}} ->
@@ -112,27 +146,33 @@ maybe_append_filters(Base,
         {error, FilterParseError} ->
             throw({error, FilterParseError})
         end,
-    couch_util:to_hex(couch_hash:md5_hash(term_to_binary(Base2))).
+    Res = couch_util:to_hex(couch_hash:md5_hash(term_to_binary(Base2))),
+    list_to_binary(Res).
 
 
-maybe_append_options(Options, RepOptions) ->
+maybe_append_options(Options, #{} = RepOptions) ->
     lists:foldl(fun(Option, Acc) ->
         Acc ++
-        case couch_util:get_value(Option, RepOptions, false) of
-        true ->
-            "+" ++ atom_to_list(Option);
-        false ->
-            ""
+        case maps:get(Option, RepOptions, false) of
+            true -> "+" ++ binary_to_list(Option);
+            false -> ""
         end
     end, [], Options).
 
 
-get_rep_endpoint(#httpdb{url=Url, headers=Headers}) ->
+get_rep_endpoint(#{<<"url">> := Url0, <<"headers">> := Headers0}) ->
+    % We turn everything to lists and proplists to calculate the same
+    % replication ID as CouchDB <= 3.x
+    Url = binary_to_list(Url0),
+    Headers1 = maps:fold(fun(K, V, Acc) ->
+        [{binary_to_list(K), binary_to_list(V)} | Acc]
+    end, [], Headers0),
+    Headers2 = lists:keysort(1, Headers1),
     DefaultHeaders = (#httpdb{})#httpdb.headers,
-    {remote, Url, Headers -- DefaultHeaders}.
+    {remote, Url, Headers2 -- DefaultHeaders}.
 
 
-get_v4_endpoint(#httpdb{} = HttpDb) ->
+get_v4_endpoint(#{} = HttpDb) ->
     {remote, Url, Headers} = get_rep_endpoint(HttpDb),
     {{UserFromHeaders, _}, HeadersWithoutBasicAuth} =
         couch_replicator_utils:remove_basic_auth_from_headers(Headers),
@@ -184,92 +224,132 @@ get_non_default_port(_Schema, Port) ->
 -ifdef(TEST).
 
 -include_lib("eunit/include/eunit.hrl").
+-include_lib("fabric/test/fabric2_test.hrl").
 
 
 replication_id_convert_test_() ->
     [?_assertEqual(Expected, convert(Id)) || {Expected, Id} <- [
-        {{"abc", ""}, "abc"},
-        {{"abc", ""}, <<"abc">>},
-        {{"abc", "+x+y"}, <<"abc+x+y">>},
-        {{"abc", "+x+y"}, {"abc", "+x+y"}},
-        {{"abc", "+x+y"}, <<"abc x y">>}
+        {<<"abc">>, <<"abc">>},
+        {<<"abc+x">>, <<"abc+x">>},
+        {<<"abc+x">>, <<"abc x">>},
+        {<<"abc+x+y">>, <<"abc+x+y">>},
+        {<<"abc+x+y">>, <<"abc x y">>}
     ]].
 
+
 http_v4_endpoint_test_() ->
     [?_assertMatch({remote, User, Host, Port, Path, HeadersNoAuth, undefined},
-        get_v4_endpoint(#httpdb{url = Url, headers = Headers})) ||
+        get_v4_endpoint(#{<<"url">> => Url, <<"headers">> => Headers})) ||
             {{User, Host, Port, Path, HeadersNoAuth}, {Url, Headers}} <- [
                 {
                     {undefined, "host", default, "/", []},
-                    {"http://host", []}
+                    {<<"http://host">>, #{}}
                 },
                 {
                     {undefined, "host", default, "/", []},
-                    {"https://host", []}
+                    {<<"https://host">>, #{}}
                 },
                 {
                     {undefined, "host", default, "/", []},
-                    {"http://host:5984", []}
+                    {<<"http://host:5984">>, #{}}
                 },
                 {
                     {undefined, "host", 1, "/", []},
-                    {"http://host:1", []}
+                    {<<"http://host:1">>, #{}}
                 },
                 {
                     {undefined, "host", 2, "/", []},
-                    {"https://host:2", []}
+                    {<<"https://host:2">>, #{}}
                 },
                 {
-                    {undefined, "host", default, "/", [{"h","v"}]},
-                    {"http://host", [{"h","v"}]}
+                    {undefined, "host", default, "/", [{"h", "v"}]},
+                    {<<"http://host">>, #{<<"h">> => <<"v">>}}
                 },
                 {
                     {undefined, "host", default, "/a/b", []},
-                    {"http://host/a/b", []}
+                    {<<"http://host/a/b">>, #{}}
                 },
                 {
                     {"user", "host", default, "/", []},
-                    {"http://user:pass@host", []}
+                    {<<"http://user:pass@host">>, #{}}
                 },
                 {
                     {"user", "host", 3, "/", []},
-                    {"http://user:pass@host:3", []}
+                    {<<"http://user:pass@host:3">>, #{}}
                 },
                 {
                     {"user", "host", default, "/", []},
-                    {"http://user:newpass@host", []}
+                    {<<"http://user:newpass@host">>, #{}}
                 },
                 {
                     {"user", "host", default, "/", []},
-                    {"http://host", [basic_auth("user","pass")]}
+                    {<<"http://host">>, basic_auth(<<"user">>, <<"pass">>)}
                 },
                 {
                     {"user", "host", default, "/", []},
-                    {"http://host", [basic_auth("user","newpass")]}
+                    {<<"http://host">>, basic_auth(<<"user">>, <<"newpass">>)}
                 },
                 {
                     {"user1", "host", default, "/", []},
-                    {"http://user1:pass1@host", [basic_auth("user2","pass2")]}
+                    {<<"http://user1:pass1@host">>, basic_auth(<<"user2">>,
+                        <<"pass2">>)}
                 },
                 {
                     {"user", "host", default, "/", [{"h", "v"}]},
-                    {"http://host", [{"h", "v"}, basic_auth("user","pass")]}
+                    {<<"http://host">>, maps:merge(#{<<"h">> => <<"v">>},
+                        basic_auth(<<"user">>, <<"pass">>))}
                 },
                 {
                     {undefined, "random_junk", undefined, undefined},
-                    {"random_junk", []}
+                    {<<"random_junk">>, #{}}
                 },
                 {
                     {undefined, "host", default, "/", []},
-                    {"http://host", [{"Authorization", "Basic bad"}]}
+                    {<<"http://host">>, #{<<"Authorization">> =>
+                        <<"Basic bad">>}}
                 }
         ]
     ].
 
 
 basic_auth(User, Pass) ->
-    B64Auth = base64:encode_to_string(User ++ ":" ++ Pass),
-    {"Authorization", "Basic " ++ B64Auth}.
+    B64Auth = base64:encode(<<User/binary, ":", Pass/binary>>),
+    #{<<"Authorization">> => <<"Basic ", B64Auth/binary>>}.
+
+
+version4_matches_couchdb3_test_() ->
+    {
+        foreach,
+        fun setup/0,
+        fun teardown/1,
+        [
+            ?TDEF_FE(id_matches_couchdb3)
+        ]
+    }.
+
+
+setup() ->
+    meck:expect(config, get, fun(_, _, Default) -> Default end).
+
+
+teardown(_) ->
+    meck:unload().
+
+
+id_matches_couchdb3(_) ->
+    {ok, Rep} = couch_replicator_parse:parse_rep(#{
+        <<"source">> => <<"http://adm:pass@127.0.0.1/abc">>,
+        <<"target">> => <<"http://adm:pass@127.0.0.1/xyz">>,
+        <<"create_target">> => true,
+        <<"continuous">> => true
+    }, null),
+    meck:expect(couch_server, get_uuid, 0, "somefixedid"),
+    {RepId, BaseId} = replication_id(Rep),
+    % Calculated on CouchDB 3.x
+    RepId3x = <<"ff71e1208f93ba054eb60e7ca8683fe4+continuous+create_target">>,
+    BaseId3x = <<"ff71e1208f93ba054eb60e7ca8683fe4">>,
+    ?assertEqual(RepId3x, RepId),
+    ?assertEqual(BaseId3x, BaseId).
 
 
 -endif.


[couchdb] 05/16: Delete old 2.x-3.x replicator modules

Posted by va...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

vatamane pushed a commit to branch prototype/fdb-layer
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit 4fc9a536ec85456ab60085f020548a08dd19ca36
Author: Nick Vatamaniuc <va...@apache.org>
AuthorDate: Fri Aug 28 04:31:04 2020 -0400

    Delete old 2.x-3.x replicator modules
    
    These modules are not used by the new replicator.
---
 .../src/couch_replicator_clustering.erl            |  279 ----
 .../src/couch_replicator_db_changes.erl            |  108 --
 .../src/couch_replicator_doc_processor.erl         |  962 -----------
 .../src/couch_replicator_doc_processor_worker.erl  |  284 ----
 .../src/couch_replicator_fabric.erl                |  155 --
 .../src/couch_replicator_fabric_rpc.erl            |   97 --
 .../src/couch_replicator_httpd_util.erl            |  201 ---
 .../src/couch_replicator_job_sup.erl               |   34 -
 .../src/couch_replicator_js_functions.hrl          |  177 --
 .../src/couch_replicator_notifier.erl              |   58 -
 .../src/couch_replicator_scheduler.erl             | 1688 --------------------
 .../src/couch_replicator_scheduler.hrl             |   15 -
 .../src/couch_replicator_scheduler_job.erl         | 1090 -------------
 .../src/couch_replicator_scheduler_sup.erl         |   62 -
 .../test/eunit/couch_replicator_compact_tests.erl  |  455 ------
 .../couch_replicator_error_reporting_tests.erl     |  271 ----
 16 files changed, 5936 deletions(-)

diff --git a/src/couch_replicator/src/couch_replicator_clustering.erl b/src/couch_replicator/src/couch_replicator_clustering.erl
deleted file mode 100644
index 18de1e8..0000000
--- a/src/couch_replicator/src/couch_replicator_clustering.erl
+++ /dev/null
@@ -1,279 +0,0 @@
-% 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 StartupPeriod
-% in effect (also configurable).
-%
-% This module is also in charge of calculating ownership of replications based
-% on where their _replicator db documents shards live.
-
-
--module(couch_replicator_clustering).
-
--behaviour(gen_server).
--behaviour(config_listener).
--behaviour(mem3_cluster).
-
--export([
-    start_link/0
-]).
-
--export([
-    init/1,
-    terminate/2,
-    handle_call/3,
-    handle_info/2,
-    handle_cast/2,
-    code_change/3
-]).
-
--export([
-    owner/2,
-    is_stable/0,
-    link_cluster_event_listener/3
-]).
-
-% config_listener callbacks
--export([
-    handle_config_change/5,
-    handle_config_terminate/3
-]).
-
-% mem3_cluster callbacks
--export([
-    cluster_stable/1,
-    cluster_unstable/1
-]).
-
--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, {
-    mem3_cluster_pid :: pid(),
-    cluster_stable :: boolean()
-}).
-
-
--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).
-
-
--spec link_cluster_event_listener(atom(), atom(), list()) -> pid().
-link_cluster_event_listener(Mod, Fun, Args)
-        when is_atom(Mod), is_atom(Fun), is_list(Args) ->
-    CallbackFun =
-        fun(Event = {cluster, _}) -> erlang:apply(Mod, Fun, Args ++ [Event]);
-           (_) -> ok
-        end,
-    {ok, Pid} = couch_replicator_notifier:start_link(CallbackFun),
-    Pid.
-
-
-% Mem3 cluster callbacks
-
-cluster_unstable(Server) ->
-    ok = gen_server:call(Server, set_unstable),
-    couch_replicator_notifier:notify({cluster, unstable}),
-    couch_stats:update_gauge([couch_replicator, cluster_is_stable], 0),
-    couch_log:notice("~s : cluster unstable", [?MODULE]),
-    Server.
-
-cluster_stable(Server) ->
-    ok = gen_server:call(Server, set_stable),
-    couch_replicator_notifier:notify({cluster, stable}),
-    couch_stats:update_gauge([couch_replicator, cluster_is_stable], 1),
-    couch_log:notice("~s : cluster stable", [?MODULE]),
-    Server.
-
-
-% gen_server callbacks
-
-init([]) ->
-    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_stats:update_gauge([couch_replicator, cluster_is_stable], 0),
-    {ok, Mem3Cluster} = mem3_cluster:start_link(?MODULE, self(), StartPeriod,
-        Period),
-    {ok, #state{mem3_cluster_pid = Mem3Cluster, cluster_stable = false}}.
-
-
-terminate(_Reason, _State) ->
-    ok.
-
-
-handle_call(is_stable, _From, #state{cluster_stable = IsStable} = State) ->
-    {reply, IsStable, State};
-
-handle_call(set_stable, _From, State) ->
-    {reply, ok, State#state{cluster_stable = true}};
-
-handle_call(set_unstable, _From, State) ->
-    {reply, ok, State#state{cluster_stable = false}}.
-
-
-handle_cast({set_period, Period}, #state{mem3_cluster_pid = Pid} = State) ->
-    ok = mem3_cluster:set_period(Pid, Period),
-    {noreply, State}.
-
-
-handle_info(restart_config_listener, State) ->
-    ok = config:listen_for_changes(?MODULE, nil),
-    {noreply, State}.
-
-
-code_change(_OldVsn, State, _Extra) ->
-    {ok, State}.
-
-
-%% Internal functions
-
-
-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(ShardName, DocId) ->
-    DbName = mem3:dbname(ShardName),
-    Live = [node() | nodes()],
-    Shards = mem3:shards(DbName, DocId),
-    Nodes = [N || #shard{node=N} <- Shards, lists:member(N, Live)],
-    mem3:owner(DbName, DocId, Nodes).
-
-
-
--ifdef(TEST).
-
--include_lib("eunit/include/eunit.hrl").
-
-
-replicator_clustering_test_() ->
-    {
-        setup,
-        fun setup_all/0,
-        fun teardown_all/1,
-        {
-            foreach,
-            fun setup/0,
-            fun teardown/1,
-            [
-                t_stable_callback(),
-                t_unstable_callback()
-            ]
-        }
-    }.
-
-
-t_stable_callback() ->
-    ?_test(begin
-        ?assertEqual(false, is_stable()),
-        cluster_stable(whereis(?MODULE)),
-        ?assertEqual(true, is_stable())
-    end).
-
-
-t_unstable_callback() ->
-    ?_test(begin
-        cluster_stable(whereis(?MODULE)),
-        ?assertEqual(true, is_stable()),
-        cluster_unstable(whereis(?MODULE)),
-        ?assertEqual(false, is_stable())
-    end).
-
-
-setup_all() ->
-    meck:expect(couch_log, notice, 2, ok),
-    meck:expect(config, get, fun(_, _, Default) -> Default end),
-    meck:expect(config, listen_for_changes, 2, ok),
-    meck:expect(couch_stats, update_gauge, 2, ok),
-    meck:expect(couch_replicator_notifier, notify, 1, ok).
-
-
-teardown_all(_) ->
-    meck:unload().
-
-
-setup() ->
-    meck:reset([
-        config,
-        couch_log,
-        couch_stats,
-        couch_replicator_notifier
-    ]),
-    stop_clustering_process(),
-    {ok, Pid} = start_link(),
-    Pid.
-
-
-teardown(Pid) ->
-    stop_clustering_process(Pid).
-
-
-stop_clustering_process() ->
-    stop_clustering_process(whereis(?MODULE)).
-
-
-stop_clustering_process(undefined) ->
-    ok;
-
-stop_clustering_process(Pid) when is_pid(Pid) ->
-    Ref = erlang:monitor(process, Pid),
-    unlink(Pid),
-    exit(Pid, kill),
-    receive {'DOWN', Ref, _, _, _} -> ok end.
-
--endif.
diff --git a/src/couch_replicator/src/couch_replicator_db_changes.erl b/src/couch_replicator/src/couch_replicator_db_changes.erl
deleted file mode 100644
index 92b0222..0000000
--- a/src/couch_replicator/src/couch_replicator_db_changes.erl
+++ /dev/null
@@ -1,108 +0,0 @@
-% 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,
-   terminate/2,
-   handle_call/3,
-   handle_info/2,
-   handle_cast/2,
-   code_change/3
-]).
-
--export([
-   notify_cluster_event/2
-]).
-
--record(state, {
-   event_listener :: pid(),
-   mdb_changes :: pid() | nil
-}).
-
-
--spec notify_cluster_event(pid(), {cluster, any()}) -> ok.
-notify_cluster_event(Server, {cluster, _} = Event) ->
-    gen_server:cast(Server, Event).
-
-
--spec start_link() ->
-    {ok, pid()} | ignore | {error, any()}.
-start_link() ->
-    gen_server:start_link(?MODULE, [], []).
-
-
-init([]) ->
-    EvtPid = couch_replicator_clustering:link_cluster_event_listener(?MODULE,
-        notify_cluster_event, [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/src/couch_replicator_doc_processor.erl b/src/couch_replicator/src/couch_replicator_doc_processor.erl
deleted file mode 100644
index 6778d53..0000000
--- a/src/couch_replicator/src/couch_replicator_doc_processor.erl
+++ /dev/null
@@ -1,962 +0,0 @@
-% 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(gen_server).
--behaviour(couch_multidb_changes).
-
--export([
-    start_link/0
-]).
-
--export([
-   init/1,
-   terminate/2,
-   handle_call/3,
-   handle_info/2,
-   handle_cast/2,
-   code_change/3
-]).
-
--export([
-    db_created/2,
-    db_deleted/2,
-    db_found/2,
-    db_change/3
-]).
-
--export([
-    docs/1,
-    doc/2,
-    doc_lookup/3,
-    update_docs/0,
-    get_worker_ref/1,
-    notify_cluster_event/2
-]).
-
--include_lib("couch/include/couch_db.hrl").
--include("couch_replicator.hrl").
--include_lib("mem3/include/mem3.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).
--define(INITIAL_BACKOFF_EXPONENT, 64).
--define(MIN_FILTER_DELAY_SEC, 60).
-
--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
-    exit:{Error, {gen_server, call, [?MODULE, _, _]}} ->
-        ErrMsg = "~p exited ~p while processing change from db ~p",
-        couch_log:error(ErrMsg, [?MODULE, Error, DbName]);
-    _Tag:Error ->
-        {RepProps} = get_json_value(doc, ChangeProps),
-        DocId = get_json_value(<<"_id">>, RepProps),
-        couch_replicator_docs:update_failed(DbName, DocId, Error)
-    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.
-
-
-% Cluster membership change notification callback
--spec notify_cluster_event(pid(), {cluster, any()}) -> ok.
-notify_cluster_event(Server, {cluster, _} = Event) ->
-    gen_server:cast(Server, Event).
-
-
-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 update the documet 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, [named_table, {keypos, #rdoc.id},
-        {read_concurrency, true}, {write_concurrency, true}]),
-    couch_replicator_clustering:link_cluster_event_listener(?MODULE,
-        notify_cluster_event, [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
-% replications 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) ->
-    NormCurRep = couch_replicator_utils:normalize_rep(current_rep(Id)),
-    NormNewRep = couch_replicator_utils:normalize_rep(Rep),
-    case NormCurRep == NormNewRep 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.
-
-
--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.
-                % Replication 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 need it
-                % anymore.
-                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),
-    % ErrCnt is the exponent here. The reason 64 is used is to start at
-    % 64 (about a minute) max range. Then first backoff would be 30 sec
-    % on average. Then 1 minute and so on.
-    couch_rand:uniform(?INITIAL_BACKOFF_EXPONENT bsl Exp).
-
-
--spec filter_backoff() -> seconds().
-filter_backoff() ->
-    Total = ets:info(?MODULE, size),
-    % This value scaled by the number of replications. If the are a lot of them
-    % wait is longer, but not more than a day (?TS_DAY_SEC). If there are just
-    % few, wait is shorter, starting at about 30 seconds. `2 *` is used since
-    % the expected wait would then be 0.5 * Range so it is easier to see the
-    % average wait. `1 +` is used because couch_rand:uniform only
-    % accepts >= 1 values and crashes otherwise.
-    Range = 1 + min(2 * (Total / 10), ?TS_DAY_SEC),
-    ?MIN_FILTER_DELAY_SEC + couch_rand: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] ->
-        % For any replication with a user created filter function, periodically
-        % (every `filter_backoff/0` seconds) to try to see if the user filter
-        % has changed by using a worker to check for changes. When the worker
-        % returns check if replication ID has changed. If it hasn't keep
-        % checking (spawn another worker and so on). If it has stop the job
-        % with the old ID and continue checking.
-        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 = (catch ets:foldl(fun(RDoc, nil) ->
-        {Shard, RDocId} = RDoc#rdoc.id,
-        case {mem3:dbname(Shard), RDocId} of
-            {Db, DocId} ->
-                throw({found, ejson_doc(RDoc, HealthThreshold)});
-            {_OtherDb, _OtherDocId} ->
-                nil
-        end
-    end, nil, ?MODULE)),
-    case Res of
-        {found, DocInfo} ->
-            {ok, DocInfo};
-        nil ->
-            {error, not_found}
-    end.
-
-
--spec doc_lookup(binary(), binary(), integer()) ->
-    {ok, {[_]}} | {error, not_found}.
-doc_lookup(Db, DocId, HealthThreshold) ->
-    case ets:lookup(?MODULE, {Db, DocId}) of
-        [#rdoc{} = RDoc] ->
-            {ok, ejson_doc(RDoc, HealthThreshold)};
-        [] ->
-            {error, not_found}
-    end.
-
-
--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, couch_replicator_utils: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(EXIT_DB, <<"exit_db">>).
--define(DOC1, <<"doc1">>).
--define(DOC2, <<"doc2">>).
--define(R1, {"1", ""}).
--define(R2, {"2", ""}).
-
-
-doc_processor_test_() ->
-    {
-        setup,
-        fun setup_all/0,
-        fun teardown_all/1,
-        {
-            foreach,
-            fun setup/0,
-            fun teardown/1,
-            [
-                t_bad_change(),
-                t_regular_change(),
-                t_change_with_doc_processor_crash(),
-                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).
-
-
-% Handle cases where doc processor exits or crashes while processing a change
-t_change_with_doc_processor_crash() ->
-    ?_test(begin
-        mock_existing_jobs_lookup([]),
-        ?assertEqual(acc, db_change(?EXIT_DB, change(), acc)),
-        ?assert(failed_state_not_updated())
-  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}),
-        meck:wait(2, couch_replicator_scheduler, find_jobs_by_doc, 2, 5000),
-        ?assertNot(ets:member(?MODULE, {?DB, ?DOC1})),
-        ?assert(removed_job(?R1))
-   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_all() ->
-    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, 3,
-        ok),
-    meck:expect(couch_replicator_doc_processor_worker, spawn_worker, fun
-        ({?EXIT_DB, _}, _, _, _) -> exit(kapow);
-        (_, _, _, _) -> pid
-    end),
-    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, 3, ok).
-
-
-teardown_all(_) ->
-    meck:unload().
-
-
-setup() ->
-    meck:reset([
-        config,
-        couch_log,
-        couch_replicator_clustering,
-        couch_replicator_doc_processor_worker,
-        couch_replicator_docs,
-        couch_replicator_scheduler
-    ]),
-    % Set this expectation back to the default for
-    % each test since some tests change it
-    meck:expect(couch_replicator_clustering, owner, 2, node()),
-    {ok, Pid} = start_link(),
-    unlink(Pid),
-    Pid.
-
-
-teardown(Pid) ->
-    exit(Pid, kill).
-
-
-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, '_').
-
-failed_state_not_updated() ->
-    0 == meck:num_calls(couch_replicator_docs, update_failed, '_').
-
-mock_existing_jobs_lookup(ExistingJobs) ->
-    meck:expect(couch_replicator_scheduler, find_jobs_by_doc, fun
-        (?EXIT_DB, ?DOC1) -> [];
-        (?DB, ?DOC1) -> ExistingJobs
-    end).
-
-
-test_rep(Id) ->
-  #rep{id = Id, start_time = {0, 0, 0}}.
-
-
-change() ->
-    {[
-        {<<"id">>, ?DOC1},
-        {doc, {[
-            {<<"_id">>, ?DOC1},
-            {<<"source">>, <<"http://srchost.local/src">>},
-            {<<"target">>, <<"http://tgthost.local/tgt">>}
-        ]}}
-    ]}.
-
-
-change(State) ->
-    {[
-        {<<"id">>, ?DOC1},
-        {doc, {[
-            {<<"_id">>, ?DOC1},
-            {<<"source">>, <<"http://srchost.local/src">>},
-            {<<"target">>, <<"http://tgthost.local/tgt">>},
-            {<<"_replication_state">>, State}
-        ]}}
-    ]}.
-
-
-deleted_change() ->
-    {[
-        {<<"id">>, ?DOC1},
-        {<<"deleted">>, true},
-        {doc, {[
-            {<<"_id">>, ?DOC1},
-            {<<"source">>, <<"http://srchost.local/src">>},
-            {<<"target">>, <<"http://tgthost.local/tgt">>}
-        ]}}
-    ]}.
-
-
-bad_change() ->
-    {[
-        {<<"id">>, ?DOC2},
-        {doc, {[
-            {<<"_id">>, ?DOC2},
-            {<<"source">>, <<"src">>}
-        ]}}
-    ]}.
-
--endif.
diff --git a/src/couch_replicator/src/couch_replicator_doc_processor_worker.erl b/src/couch_replicator/src/couch_replicator_doc_processor_worker.erl
deleted file mode 100644
index a4c8293..0000000
--- a/src/couch_replicator/src/couch_replicator_doc_processor_worker.erl
+++ /dev/null
@@ -1,284 +0,0 @@
-% 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
-]).
-
-% 61 seconds here because request usually have 10, 15, 30 second
-% timeouts set.  We'd want the worker to get a chance to make a few
-% requests (maybe one failing one and a retry) and then fail with its
-% own error (timeout, network error), which would be more specific and
-% informative, before it simply gets killed because of the timeout
-% here. That is, if all fails and the worker is actually blocked then
-% 61 sec is a safety net to brutally kill the worker so doesn't end up
-% hung forever.
--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
-% 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,
-        couch_replicator_docs:update_failed(DbName, DocId, Reason),
-        {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, {"ad08e05057046eabe898a2572bbfb573", ""}).
-
-
-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.
-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, 3, 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">>, <<"http://srchost.local/src">>},
-         {<<"target">>, <<"http://tgthost.local/tgt">>}
-     ]}.
-
--endif.
diff --git a/src/couch_replicator/src/couch_replicator_fabric.erl b/src/couch_replicator/src/couch_replicator_fabric.erl
deleted file mode 100644
index 1650105..0000000
--- a/src/couch_replicator/src/couch_replicator_fabric.erl
+++ /dev/null
@@ -1,155 +0,0 @@
-% 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_fabric).
-
--export([
-   docs/5
-]).
-
--include_lib("fabric/include/fabric.hrl").
--include_lib("mem3/include/mem3.hrl").
--include_lib("couch/include/couch_db.hrl").
--include_lib("couch_mrview/include/couch_mrview.hrl").
-
-docs(DbName, Options, QueryArgs, Callback, Acc) ->
-    Shards = mem3:shards(DbName),
-    Workers0 = fabric_util:submit_jobs(
-           Shards, couch_replicator_fabric_rpc, docs, [Options, QueryArgs]),
-    RexiMon = fabric_util:create_monitors(Workers0),
-    try
-        case fabric_streams:start(Workers0, #shard.ref) of
-            {ok, Workers} ->
-                try
-                    docs_int(DbName, Workers, QueryArgs, Callback, Acc)
-                after
-                    fabric_streams:cleanup(Workers)
-                end;
-            {timeout, NewState} ->
-                DefunctWorkers = fabric_util:remove_done_workers(
-                    NewState#stream_acc.workers, waiting
-                ),
-                fabric_util:log_timeout(
-                    DefunctWorkers,
-                    "replicator docs"
-                ),
-                Callback({error, timeout}, Acc);
-            {error, Error} ->
-                Callback({error, Error}, Acc)
-        end
-    after
-        rexi_monitor:stop(RexiMon)
-    end.
-
-
-docs_int(DbName, Workers, QueryArgs, Callback, Acc0) ->
-    #mrargs{limit = Limit, skip = Skip} = QueryArgs,
-    State = #collector{
-        db_name = DbName,
-        query_args = QueryArgs,
-        callback = Callback,
-        counters = fabric_dict:init(Workers, 0),
-        skip = Skip,
-        limit = Limit,
-        user_acc = Acc0,
-        update_seq = nil
-    },
-    case rexi_utils:recv(Workers, #shard.ref, fun handle_message/3,
-        State, infinity, 5000) of
-    {ok, NewState} ->
-        {ok, NewState#collector.user_acc};
-    {timeout, NewState} ->
-        Callback({error, timeout}, NewState#collector.user_acc);
-    {error, Resp} ->
-        {ok, Resp}
-    end.
-
-handle_message({rexi_DOWN, _, {_, NodeRef}, _}, _, State) ->
-    fabric_view:check_down_shards(State, NodeRef);
-
-handle_message({rexi_EXIT, Reason}, Worker, State) ->
-    fabric_view:handle_worker_exit(State, Worker, Reason);
-
-handle_message({meta, Meta0}, {Worker, From}, State) ->
-    Tot = couch_util:get_value(total, Meta0, 0),
-    Off = couch_util:get_value(offset, Meta0, 0),
-    #collector{
-        callback = Callback,
-        counters = Counters0,
-        total_rows = Total0,
-        offset = Offset0,
-        user_acc = AccIn
-    } = State,
-    % Assert that we don't have other messages from this
-    % worker when the total_and_offset message arrives.
-    0 = fabric_dict:lookup_element(Worker, Counters0),
-    rexi:stream_ack(From),
-    Counters1 = fabric_dict:update_counter(Worker, 1, Counters0),
-    Total = Total0 + Tot,
-    Offset = Offset0 + Off,
-    case fabric_dict:any(0, Counters1) of
-    true ->
-        {ok, State#collector{
-            counters = Counters1,
-            total_rows = Total,
-            offset = Offset
-        }};
-    false ->
-        FinalOffset = erlang:min(Total, Offset+State#collector.skip),
-        Meta = [{total, Total}, {offset, FinalOffset}],
-        {Go, Acc} = Callback({meta, Meta}, AccIn),
-        {Go, State#collector{
-            counters = fabric_dict:decrement_all(Counters1),
-            total_rows = Total,
-            offset = FinalOffset,
-            user_acc = Acc
-        }}
-    end;
-
-handle_message(#view_row{id = Id, doc = Doc} = Row0, {Worker, From}, State) ->
-    #collector{query_args = Args, counters = Counters0, rows = Rows0} = State,
-    case maybe_fetch_and_filter_doc(Id, Doc, State) of
-        {[_ | _]} = NewDoc ->
-            Row = Row0#view_row{doc = NewDoc},
-            Dir = Args#mrargs.direction,
-            Rows = merge_row(Dir, Row#view_row{worker={Worker, From}}, Rows0),
-            Counters1 = fabric_dict:update_counter(Worker, 1, Counters0),
-            State1 = State#collector{rows=Rows, counters=Counters1},
-            fabric_view:maybe_send_row(State1);
-        skip ->
-            rexi:stream_ack(From),
-            {ok, State}
-    end;
-
-handle_message(complete, Worker, State) ->
-    Counters = fabric_dict:update_counter(Worker, 1, State#collector.counters),
-    fabric_view:maybe_send_row(State#collector{counters = Counters}).
-
-
-merge_row(fwd, Row, Rows) ->
-    lists:keymerge(#view_row.id, [Row], Rows);
-merge_row(rev, Row, Rows) ->
-    lists:rkeymerge(#view_row.id, [Row], Rows).
-
-
-maybe_fetch_and_filter_doc(Id, undecided, State) ->
-    #collector{db_name = DbName, query_args = #mrargs{extra = Extra}} = State,
-    FilterStates = proplists:get_value(filter_states, Extra),
-    case couch_replicator:active_doc(DbName, Id) of
-        {ok, {Props} = DocInfo} ->
-            DocState = couch_util:get_value(state, Props),
-            couch_replicator_utils:filter_state(DocState, FilterStates, DocInfo);
-        {error, not_found} ->
-            skip  % could have been deleted
-    end;
-maybe_fetch_and_filter_doc(_Id, Doc, _State) ->
-    Doc.
diff --git a/src/couch_replicator/src/couch_replicator_fabric_rpc.erl b/src/couch_replicator/src/couch_replicator_fabric_rpc.erl
deleted file mode 100644
index d67f875..0000000
--- a/src/couch_replicator/src/couch_replicator_fabric_rpc.erl
+++ /dev/null
@@ -1,97 +0,0 @@
-% 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_fabric_rpc).
-
--export([
-   docs/3
-]).
-
--include_lib("fabric/include/fabric.hrl").
--include_lib("couch/include/couch_db.hrl").
--include_lib("couch_mrview/include/couch_mrview.hrl").
-
-
-docs(DbName, Options, Args0) ->
-    set_io_priority(DbName, Options),
-    #mrargs{skip = Skip, limit = Limit, extra = Extra} = Args0,
-    FilterStates = proplists:get_value(filter_states, Extra),
-    Args = Args0#mrargs{skip = 0, limit = Skip + Limit},
-    HealthThreshold = couch_replicator_scheduler:health_threshold(),
-    {ok, Db} = couch_db:open_int(DbName, Options),
-    Acc = {DbName, FilterStates, HealthThreshold},
-    couch_mrview:query_all_docs(Db, Args, fun docs_cb/2, Acc).
-
-
-docs_cb({meta, Meta}, Acc) ->
-    ok = rexi:stream2({meta, Meta}),
-    {ok, Acc};
-docs_cb({row, Row}, {DbName, States, HealthThreshold} = Acc) ->
-    Id = couch_util:get_value(id, Row),
-    Doc = couch_util:get_value(doc, Row),
-    ViewRow = #view_row{
-        id = Id,
-        key = couch_util:get_value(key, Row),
-        value = couch_util:get_value(value, Row)
-    },
-    case rep_doc_state(DbName, Id, Doc, States, HealthThreshold) of
-        skip ->
-            ok;
-        Other ->
-            ok = rexi:stream2(ViewRow#view_row{doc = Other})
-    end,
-    {ok, Acc};
-docs_cb(complete, Acc) ->
-    ok = rexi:stream_last(complete),
-    {ok, Acc}.
-
-
-set_io_priority(DbName, Options) ->
-    case lists:keyfind(io_priority, 1, Options) of
-    {io_priority, Pri} ->
-        erlang:put(io_priority, Pri);
-    false ->
-        erlang:put(io_priority, {interactive, DbName})
-    end.
-
-
-%% Get the state of the replication document. If it is found and has a terminal
-%% state then it can be filtered and either included in the results or skipped.
-%% If it is not in a terminal state, look it up in the local doc processor ETS
-%% table. If it is there then filter by state. If it is not found there either
-%% then mark it as `undecided` and let the coordinator try to fetch it. The
-%% The idea is to do as much work as possible locally and leave the minimum
-%% amount of work for the coordinator.
-rep_doc_state(_Shard, <<"_design/", _/binary>>, _, _, _) ->
-    skip;
-rep_doc_state(Shard, Id, {[_ | _]} = Doc, States, HealthThreshold) ->
-    DbName = mem3:dbname(Shard),
-    DocInfo = couch_replicator:info_from_doc(DbName, Doc),
-    case get_doc_state(DocInfo) of
-        null ->
-            % Fetch from local doc processor. If there, filter by state.
-            % If not there, mark as undecided. Let coordinator figure it out.
-            case couch_replicator_doc_processor:doc_lookup(Shard, Id,
-                    HealthThreshold) of
-                {ok, EtsInfo} ->
-                    State = get_doc_state(EtsInfo),
-                    couch_replicator_utils:filter_state(State, States, EtsInfo);
-                {error, not_found} ->
-                    undecided
-            end;
-        OtherState when is_atom(OtherState) ->
-            couch_replicator_utils:filter_state(OtherState, States, DocInfo)
-    end.
-
-
-get_doc_state({Props})->
-    couch_util:get_value(state, Props).
diff --git a/src/couch_replicator/src/couch_replicator_httpd_util.erl b/src/couch_replicator/src/couch_replicator_httpd_util.erl
deleted file mode 100644
index 624eddd..0000000
--- a/src/couch_replicator/src/couch_replicator_httpd_util.erl
+++ /dev/null
@@ -1,201 +0,0 @@
-% 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_httpd_util).
-
--include_lib("couch/include/couch_db.hrl").
--include_lib("couch_mrview/include/couch_mrview.hrl").
-
--export([
-    validate_rep_props/1,
-    parse_int_param/5,
-    parse_replication_state_filter/1,
-    update_db_name/1,
-    docs_acc_new/3,
-    docs_acc_response/1,
-    docs_cb/2
-]).
-
--import(couch_httpd, [
-    send_json/2,
-    send_json/3,
-    send_method_not_allowed/2
-]).
-
--import(couch_util, [
-    to_binary/1
-]).
-
-
-parse_replication_state_filter(undefined) ->
-    [];  % This is the default (wildcard) filter
-parse_replication_state_filter(States) when is_list(States) ->
-    AllStates = couch_replicator:replication_states(),
-    StrStates = [string:to_lower(S) || S <- string:tokens(States, ",")],
-    AtomStates = try
-        [list_to_existing_atom(S) || S <- StrStates]
-    catch error:badarg ->
-        Msg1 = io_lib:format("States must be one or more of ~w", [AllStates]),
-        throw({query_parse_error, ?l2b(Msg1)})
-    end,
-    AllSet = sets:from_list(AllStates),
-    StatesSet = sets:from_list(AtomStates),
-    Diff = sets:to_list(sets:subtract(StatesSet, AllSet)),
-    case Diff of
-    [] ->
-        AtomStates;
-    _ ->
-        Args = [Diff, AllStates],
-        Msg2 = io_lib:format("Unknown states ~w. Choose from: ~w", Args),
-        throw({query_parse_error, ?l2b(Msg2)})
-    end.
-
-
-parse_int_param(Req, Param, Default, Min, Max) ->
-    IntVal = try
-        list_to_integer(chttpd:qs_value(Req, Param, integer_to_list(Default)))
-    catch error:badarg ->
-        Msg1 = io_lib:format("~s must be an integer", [Param]),
-        throw({query_parse_error, ?l2b(Msg1)})
-    end,
-    case IntVal >= Min andalso IntVal =< Max of
-    true ->
-        IntVal;
-    false ->
-        Msg2 = io_lib:format("~s not in range of [~w,~w]", [Param, Min, Max]),
-        throw({query_parse_error, ?l2b(Msg2)})
-    end.
-
-
-validate_rep_props([]) ->
-    ok;
-validate_rep_props([{<<"query_params">>, {Params}}|Rest]) ->
-    lists:foreach(fun
-        ({_,V}) when is_binary(V) -> ok;
-        ({K,_}) -> throw({bad_request,
-            <<K/binary," value must be a string.">>})
-        end, Params),
-    validate_rep_props(Rest);
-validate_rep_props([_|Rest]) ->
-    validate_rep_props(Rest).
-
-
-prepend_val(#vacc{prepend=Prepend}) ->
-    case Prepend of
-        undefined ->
-            "";
-        _ ->
-            Prepend
-    end.
-
-
-maybe_flush_response(#vacc{bufsize=Size, threshold=Max} = Acc, Data, Len)
-        when Size > 0 andalso (Size + Len) > Max ->
-    #vacc{buffer = Buffer, resp = Resp} = Acc,
-    {ok, R1} = chttpd:send_delayed_chunk(Resp, Buffer),
-    {ok, Acc#vacc{prepend = ",\r\n", buffer = Data, bufsize = Len, resp = R1}};
-maybe_flush_response(Acc0, Data, Len) ->
-    #vacc{buffer = Buf, bufsize = Size} = Acc0,
-    Acc = Acc0#vacc{
-        prepend = ",\r\n",
-        buffer = [Buf | Data],
-        bufsize = Size + Len
-    },
-    {ok, Acc}.
-
-docs_acc_new(Req, Db, Threshold) ->
-     #vacc{db=Db, req=Req, threshold=Threshold}.
-
-docs_acc_response(#vacc{resp = Resp}) ->
-    Resp.
-
-docs_cb({error, Reason}, #vacc{resp=undefined}=Acc) ->
-    {ok, Resp} = chttpd:send_error(Acc#vacc.req, Reason),
-    {ok, Acc#vacc{resp=Resp}};
-
-docs_cb(complete, #vacc{resp=undefined}=Acc) ->
-    % Nothing in view
-    {ok, Resp} = chttpd:send_json(Acc#vacc.req, 200, {[{rows, []}]}),
-    {ok, Acc#vacc{resp=Resp}};
-
-docs_cb(Msg, #vacc{resp=undefined}=Acc) ->
-    %% Start response
-    Headers = [],
-    {ok, Resp} = chttpd:start_delayed_json_response(Acc#vacc.req, 200, Headers),
-    docs_cb(Msg, Acc#vacc{resp=Resp, should_close=true});
-
-docs_cb({error, Reason}, #vacc{resp=Resp}=Acc) ->
-    {ok, Resp1} = chttpd:send_delayed_error(Resp, Reason),
-    {ok, Acc#vacc{resp=Resp1}};
-
-docs_cb(complete, #vacc{resp=Resp, buffer=Buf, threshold=Max}=Acc) ->
-    % Finish view output and possibly end the response
-    {ok, Resp1} = chttpd:close_delayed_json_object(Resp, Buf, "\r\n]}", Max),
-    case Acc#vacc.should_close of
-        true ->
-            {ok, Resp2} = chttpd:end_delayed_json_response(Resp1),
-            {ok, Acc#vacc{resp=Resp2}};
-        _ ->
-            {ok, Acc#vacc{resp=Resp1, meta_sent=false, row_sent=false,
-                prepend=",\r\n", buffer=[], bufsize=0}}
-    end;
-
-docs_cb({meta, Meta}, #vacc{meta_sent=false, row_sent=false}=Acc) ->
-    % Sending metadata as we've not sent it or any row yet
-    Parts = case couch_util:get_value(total, Meta) of
-        undefined -> [];
-        Total -> [io_lib:format("\"total_rows\":~p", [adjust_total(Total)])]
-    end ++ case couch_util:get_value(offset, Meta) of
-        undefined -> [];
-        Offset -> [io_lib:format("\"offset\":~p", [Offset])]
-    end ++ ["\"docs\":["],
-    Chunk = [prepend_val(Acc), "{", string:join(Parts, ","), "\r\n"],
-    {ok, AccOut} = maybe_flush_response(Acc, Chunk, iolist_size(Chunk)),
-    {ok, AccOut#vacc{prepend="", meta_sent=true}};
-
-
-docs_cb({meta, _Meta}, #vacc{}=Acc) ->
-    %% ignore metadata
-    {ok, Acc};
-
-docs_cb({row, Row}, #vacc{meta_sent=false}=Acc) ->
-    %% sorted=false and row arrived before meta
-    % Adding another row
-    Chunk = [prepend_val(Acc), "{\"docs\":[\r\n", row_to_json(Row)],
-    maybe_flush_response(Acc#vacc{meta_sent=true, row_sent=true}, Chunk, iolist_size(Chunk));
-
-docs_cb({row, Row}, #vacc{meta_sent=true}=Acc) ->
-    % Adding another row
-    Chunk = [prepend_val(Acc), row_to_json(Row)],
-    maybe_flush_response(Acc#vacc{row_sent=true}, Chunk, iolist_size(Chunk)).
-
-
-update_db_name({Props}) ->
-    {value, {database, DbName}, Props1} = lists:keytake(database, 1, Props),
-    {[{database, normalize_db_name(DbName)} | Props1]}.
-
-normalize_db_name(<<"shards/", _/binary>> = DbName) ->
-    mem3:dbname(DbName);
-normalize_db_name(DbName) ->
-    DbName.
-
-row_to_json(Row) ->
-    Doc0 = couch_util:get_value(doc, Row),
-    Doc1 = update_db_name(Doc0),
-    ?JSON_ENCODE(Doc1).
-
-
-%% Adjust Total as there is an automatically created validation design doc
-adjust_total(Total) when is_integer(Total), Total > 0 ->
-    Total - 1;
-adjust_total(Total) when is_integer(Total) ->
-    0.
diff --git a/src/couch_replicator/src/couch_replicator_job_sup.erl b/src/couch_replicator/src/couch_replicator_job_sup.erl
deleted file mode 100644
index 9ea65e8..0000000
--- a/src/couch_replicator/src/couch_replicator_job_sup.erl
+++ /dev/null
@@ -1,34 +0,0 @@
-% 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_job_sup).
-
--behaviour(supervisor).
-
--export([
-    init/1,
-    start_link/0
-]).
-
-start_link() ->
-    supervisor:start_link({local,?MODULE}, ?MODULE, []).
-
-%%=============================================================================
-%% supervisor callbacks
-%%=============================================================================
-
-init([]) ->
-    {ok, {{one_for_one, 3, 10}, []}}.
-
-%%=============================================================================
-%% internal functions
-%%=============================================================================
diff --git a/src/couch_replicator/src/couch_replicator_js_functions.hrl b/src/couch_replicator/src/couch_replicator_js_functions.hrl
deleted file mode 100644
index d410433..0000000
--- a/src/couch_replicator/src/couch_replicator_js_functions.hrl
+++ /dev/null
@@ -1,177 +0,0 @@
-% 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.
-
--define(REP_DB_DOC_VALIDATE_FUN, <<"
-    function(newDoc, oldDoc, userCtx) {
-        function reportError(error_msg) {
-            log('Error writing document `' + newDoc._id +
-                '\\' to the replicator database: ' + error_msg);
-            throw({forbidden: error_msg});
-        }
-
-        function validateEndpoint(endpoint, fieldName) {
-            if ((typeof endpoint !== 'string') &&
-                ((typeof endpoint !== 'object') || (endpoint === null))) {
-
-                reportError('The `' + fieldName + '\\' property must exist' +
-                    ' and be either a string or an object.');
-            }
-
-            if (typeof endpoint === 'object') {
-                if ((typeof endpoint.url !== 'string') || !endpoint.url) {
-                    reportError('The url property must exist in the `' +
-                        fieldName + '\\' field and must be a non-empty string.');
-                }
-
-                if ((typeof endpoint.auth !== 'undefined') &&
-                    ((typeof endpoint.auth !== 'object') ||
-                        endpoint.auth === null)) {
-
-                    reportError('`' + fieldName +
-                        '.auth\\' must be a non-null object.');
-                }
-
-                if ((typeof endpoint.headers !== 'undefined') &&
-                    ((typeof endpoint.headers !== 'object') ||
-                        endpoint.headers === null)) {
-
-                    reportError('`' + fieldName +
-                        '.headers\\' must be a non-null object.');
-                }
-            }
-        }
-
-        var isReplicator = (userCtx.roles.indexOf('_replicator') >= 0);
-        var isAdmin = (userCtx.roles.indexOf('_admin') >= 0);
-
-        if (isReplicator) {
-            // Always let replicator update the replication document
-            return;
-        }
-
-        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
-            // is safe.
-            return;
-        }
-
-        if (!newDoc._deleted) {
-            validateEndpoint(newDoc.source, 'source');
-            validateEndpoint(newDoc.target, 'target');
-
-            if ((typeof newDoc.create_target !== 'undefined') &&
-                (typeof newDoc.create_target !== 'boolean')) {
-
-                reportError('The `create_target\\' field must be a boolean.');
-            }
-
-            if ((typeof newDoc.continuous !== 'undefined') &&
-                (typeof newDoc.continuous !== 'boolean')) {
-
-                reportError('The `continuous\\' field must be a boolean.');
-            }
-
-            if ((typeof newDoc.doc_ids !== 'undefined') &&
-                !isArray(newDoc.doc_ids)) {
-
-                reportError('The `doc_ids\\' field must be an array of strings.');
-            }
-
-            if ((typeof newDoc.selector !== 'undefined') &&
-                (typeof newDoc.selector !== 'object')) {
-
-                reportError('The `selector\\' field must be an object.');
-            }
-
-            if ((typeof newDoc.filter !== 'undefined') &&
-                ((typeof newDoc.filter !== 'string') || !newDoc.filter)) {
-
-                reportError('The `filter\\' field must be a non-empty string.');
-            }
-
-            if ((typeof newDoc.doc_ids !== 'undefined') &&
-                (typeof newDoc.selector !== 'undefined')) {
-
-                reportError('`doc_ids\\' field is incompatible with `selector\\'.');
-            }
-
-            if ( ((typeof newDoc.doc_ids !== 'undefined') ||
-                  (typeof newDoc.selector !== 'undefined')) &&
-                 (typeof newDoc.filter !== 'undefined') ) {
-
-                reportError('`filter\\' field is incompatible with `selector\\' and `doc_ids\\'.');
-            }
-
-            if ((typeof newDoc.query_params !== 'undefined') &&
-                ((typeof newDoc.query_params !== 'object') ||
-                    newDoc.query_params === null)) {
-
-                reportError('The `query_params\\' field must be an object.');
-            }
-
-            if (newDoc.user_ctx) {
-                var user_ctx = newDoc.user_ctx;
-
-                if ((typeof user_ctx !== 'object') || (user_ctx === null)) {
-                    reportError('The `user_ctx\\' property must be a ' +
-                        'non-null object.');
-                }
-
-                if (!(user_ctx.name === null ||
-                    (typeof user_ctx.name === 'undefined') ||
-                    ((typeof user_ctx.name === 'string') &&
-                        user_ctx.name.length > 0))) {
-
-                    reportError('The `user_ctx.name\\' property must be a ' +
-                        'non-empty string or null.');
-                }
-
-                if (!isAdmin && (user_ctx.name !== userCtx.name)) {
-                    reportError('The given `user_ctx.name\\' is not valid');
-                }
-
-                if (user_ctx.roles && !isArray(user_ctx.roles)) {
-                    reportError('The `user_ctx.roles\\' property must be ' +
-                        'an array of strings.');
-                }
-
-                if (!isAdmin && user_ctx.roles) {
-                    for (var i = 0; i < user_ctx.roles.length; i++) {
-                        var role = user_ctx.roles[i];
-
-                        if (typeof role !== 'string' || role.length === 0) {
-                            reportError('Roles must be non-empty strings.');
-                        }
-                        if (userCtx.roles.indexOf(role) === -1) {
-                            reportError('Invalid role (`' + role +
-                                '\\') in the `user_ctx\\'');
-                        }
-                    }
-                }
-            } else {
-                if (!isAdmin) {
-                    reportError('The `user_ctx\\' property is missing (it is ' +
-                       'optional for admins only).');
-                }
-            }
-        } else {
-            if (!isAdmin) {
-                if (!oldDoc.user_ctx || (oldDoc.user_ctx.name !== userCtx.name)) {
-                    reportError('Replication documents can only be deleted by ' +
-                        'admins or by the users who created them.');
-                }
-            }
-        }
-    }
-">>).
diff --git a/src/couch_replicator/src/couch_replicator_notifier.erl b/src/couch_replicator/src/couch_replicator_notifier.erl
deleted file mode 100644
index f7640a3..0000000
--- a/src/couch_replicator/src/couch_replicator_notifier.erl
+++ /dev/null
@@ -1,58 +0,0 @@
-% 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_notifier).
-
--behaviour(gen_event).
--vsn(1).
-
-% public API
--export([start_link/1, stop/1, notify/1]).
-
-% gen_event callbacks
--export([init/1, terminate/2, code_change/3]).
--export([handle_event/2, handle_call/2, handle_info/2]).
-
--include_lib("couch/include/couch_db.hrl").
-
-start_link(FunAcc) ->
-    couch_event_sup:start_link(couch_replication,
-        {couch_replicator_notifier, make_ref()}, FunAcc).
-
-notify(Event) ->
-    gen_event:notify(couch_replication, Event).
-
-stop(Pid) ->
-    couch_event_sup:stop(Pid).
-
-
-init(FunAcc) ->
-    {ok, FunAcc}.
-
-terminate(_Reason, _State) ->
-    ok.
-
-handle_event(Event, Fun) when is_function(Fun, 1) ->
-    Fun(Event),
-    {ok, Fun};
-handle_event(Event, {Fun, Acc}) when is_function(Fun, 2) ->
-    Acc2 = Fun(Event, Acc),
-    {ok, {Fun, Acc2}}.
-
-handle_call(_Msg, State) ->
-    {ok, ok, State}.
-
-handle_info(_Msg, State) ->
-    {ok, State}.
-
-code_change(_OldVsn, State, _Extra) ->
-    {ok, State}.
diff --git a/src/couch_replicator/src/couch_replicator_scheduler.erl b/src/couch_replicator/src/couch_replicator_scheduler.erl
deleted file mode 100644
index 00a352b..0000000
--- a/src/couch_replicator/src/couch_replicator_scheduler.erl
+++ /dev/null
@@ -1,1688 +0,0 @@
-% 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).
-
--export([
-    start_link/0
-]).
-
--export([
-   init/1,
-   terminate/2,
-   handle_call/3,
-   handle_info/2,
-   handle_cast/2,
-   code_change/3,
-   format_status/2
-]).
-
--export([
-   add_job/1,
-   remove_job/1,
-   reschedule/0,
-   rep_state/1,
-   find_jobs_by_dbname/1,
-   find_jobs_by_doc/2,
-   job_summary/2,
-   health_threshold/0,
-   jobs/0,
-   job/1,
-   restart_job/1,
-   update_job_stats/2
-]).
-
-%% config_listener callbacks
--export([
-    handle_config_change/5,
-    handle_config_terminate/3
-]).
-
-%% for status updater process to allow hot code loading
--export([
-    stats_updater_loop/1
-]).
-
--include("couch_replicator_scheduler.hrl").
--include("couch_replicator.hrl").
--include_lib("couch_replicator/include/couch_replicator_api_wrap.hrl").
--include_lib("couch/include/couch_db.hrl").
-
-%% 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(STATS_UPDATE_WAIT, 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, stats_pid}).
--record(job, {
-    id :: job_id() | '$1' | '_',
-    rep :: #rep{} | '_',
-    pid :: undefined | pid() | '$1' | '_',
-    monitor :: undefined | reference() | '_',
-    history :: history() | '_'
-}).
-
--record(stats_acc, {
-    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 ->
-    case existing_replication(Rep) of
-        false ->
-            Job = #job{
-                id = Rep#rep.id,
-                rep = Rep,
-                history = [{added, os:timestamp()}]
-            },
-            gen_server:call(?MODULE, {add_job, Job}, infinity);
-        true ->
-            ok
-    end.
-
-
--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}  ->
-                    case History of
-                        [{{crashed, Error}, _When} | _] ->
-                            {crashing, crash_reason_json(Error)};
-                        [_ | _] ->
-                            {pending, Rep#rep.stats}
-                    end;
-                {undefined, ErrorCount} when ErrorCount > 0 ->
-                     [{{crashed, Error}, _When} | _] = History,
-                     {crashing, crash_reason_json(Error)};
-                {Pid, ErrorCount} when is_pid(Pid) ->
-                     {running, Rep#rep.stats}
-            end,
-            [
-                {source, iolist_to_binary(ejson_url(Rep#rep.source))},
-                {target, iolist_to_binary(ejson_url(Rep#rep.target))},
-                {state, State},
-                {info, couch_replicator_utils:ejson_state_info(Info)},
-                {error_count, ErrorCount},
-                {last_updated, last_updated(History)},
-                {start_time,
-                    couch_replicator_utils:iso8601(Rep#rep.start_time)},
-                {source_proxy, job_proxy_url(Rep#rep.source)},
-                {target_proxy, job_proxy_url(Rep#rep.target)}
-            ];
-        {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.
-
-
-% Health threshold is the minimum amount of time an unhealthy job should run
-% crashing before it is considered to be healthy again. HealtThreashold should
-% not be 0 as jobs could start and immediately crash, and it shouldn't be
-% infinity, since then  consecutive crashes would accumulate forever even if
-% job is back to normal.
--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)].
-
-
--spec restart_job(binary() | list() | rep_id()) ->
-    {ok, {[_]}} | {error, not_found}.
-restart_job(JobId) ->
-    case rep_state(JobId) of
-        nil ->
-            {error, not_found};
-        #rep{} = Rep ->
-            ok = remove_job(JobId),
-            ok = add_job(Rep),
-            job(JobId)
-    end.
-
-
--spec update_job_stats(job_id(), term()) -> ok.
-update_job_stats(JobId, Stats) ->
-    gen_server:cast(?MODULE, {update_job_stats, JobId, Stats}).
-
-
-%% gen_server functions
-
-init(_) ->
-    % Temporarily disable on FDB, as it's not fully implemented yet
-    % config:enable_feature('scheduler'),
-    EtsOpts = [named_table, {keypos, #job.id}, {read_concurrency, true},
-        {write_concurrency, true}],
-    ?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),
-    Timer = erlang:send_after(Interval, self(), reschedule),
-    State = #state{
-        interval = Interval,
-        max_jobs = MaxJobs,
-        max_churn = MaxChurn,
-        max_history = MaxHistory,
-        timer = Timer,
-        stats_pid = start_stats_updater()
-    },
-    {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({update_job_stats, JobId, Stats}, State) ->
-    case rep_state(JobId) of
-        nil ->
-            ok;
-        #rep{} = Rep ->
-            NewRep = Rep#rep{stats = Stats},
-            true = ets:update_element(?MODULE, JobId, {#job.rep, NewRep})
-    end,
-    {noreply, State};
-
-handle_cast(UnexpectedMsg, State) ->
-    couch_log:error("~p: received un-expected cast ~p", [?MODULE, UnexpectedMsg]),
-    {noreply, State}.
-
-
-handle_info(reschedule, State) ->
-    ok = reschedule(State),
-    erlang:cancel_timer(State#state.timer),
-    Timer = erlang:send_after(State#state.interval, self(), 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(State#state.stats_pid),
-    {noreply, State};
-
-handle_info({'DOWN', _Ref, process, Pid, Reason0}, State) ->
-    {ok, Job} = job_by_pid(Pid),
-    Reason = case Reason0 of
-        {shutdown, ShutdownReason} -> ShutdownReason;
-        Other -> Other
-    end,
-    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_max_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(State#state.stats_pid),
-    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(State#state.stats_pid),
-            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(State#state.stats_pid),
-            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 job 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.
-% "oldest" here means one which has been waiting to run the longest. "youngest"
-% means the one with most recent activity. The goal is to keep up to Count
-% oldest jobs during iteration. For example if there are jobs with these times
-% accumulated so far [5, 7, 11], and start time of current job is 6. Then
-% 6 < 11 is true, so 11 (youngest) is dropped and 6 inserted resulting in
-% [5, 6, 7]. In the end the result might look like [1, 2, 5], for example.
-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, _, _) when is_integer(Count), Count =< 0 ->
-    0;
-
-stop_jobs(Count, IsContinuous, State) when is_integer(Count) ->
-    Running0 = running_jobs(),
-    ContinuousPred = fun(Job) -> is_continuous(Job) =:= IsContinuous end,
-    Running1 = lists:filter(ContinuousPred, Running0),
-    Running2 = lists:sort(fun longest_running/2, Running1),
-    Running3 = lists:sublist(Running2, Count),
-    length([stop_job_int(Job, State) || Job <- Running3]).
-
-
-longest_running(#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.
-%
-% A streak of "consecutive crashes" ends when a crashed event is seen starting
-% and running successfully without crashing for a period of time. That period
-% of time is the HealthThreshold.
-%
-
--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) ->
-    % When calculating the backoff interval treat consecutive crash count as the
-    % exponent in Base * 2 ^ CrashCount to achieve an exponential backoff
-    % doubling every consecutive failure, starting with the base value of
-    % ?BACKOFF_INTERVAL_MICROS.
-    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(State#state.stats_pid),
-            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) ->
-    StopCount = stop_excess_jobs(State, running_job_count()),
-    rotate_jobs(State, StopCount),
-    update_running_jobs_stats(State#state.stats_pid).
-
-
--spec stop_excess_jobs(#state{}, non_neg_integer()) -> non_neg_integer().
-stop_excess_jobs(State, Running) ->
-    #state{max_jobs=MaxJobs} = State,
-    StopCount = max(0, Running - MaxJobs),
-    Stopped = stop_jobs(StopCount, true, State),
-    OneshotLeft = StopCount - Stopped,
-    stop_jobs(OneshotLeft, false, State),
-    StopCount.
-
-
-start_pending_jobs(State) ->
-    #state{max_jobs=MaxJobs} = State,
-    Running = running_job_count(),
-    Pending = pending_job_count(),
-    if Running < MaxJobs, Pending > 0 ->
-        start_jobs(MaxJobs - Running, State);
-    true ->
-        ok
-    end.
-
-
--spec rotate_jobs(#state{}, non_neg_integer()) -> ok.
-rotate_jobs(State, ChurnSoFar) ->
-    #state{max_jobs=MaxJobs, max_churn=MaxChurn} = State,
-    Running = running_job_count(),
-    Pending = pending_job_count(),
-    % Reduce MaxChurn by the number of already stopped jobs in the
-    % current rescheduling cycle.
-    Churn = max(0, MaxChurn - ChurnSoFar),
-    SlotsAvailable = MaxJobs - Running,
-    if SlotsAvailable >= 0 ->
-        % If there is are enough SlotsAvailable reduce StopCount to avoid
-        % unnesessarily stopping jobs. `stop_jobs/3` ignores 0 or negative
-        % values so we don't worry about that here.
-        StopCount = lists:min([Pending - SlotsAvailable, Running, Churn]),
-        stop_jobs(StopCount, true, State),
-        StartCount = max(0, MaxJobs - running_job_count()),
-        start_jobs(StartCount, 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 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},
-        {info, couch_replicator_utils:ejson_state_info(Rep#rep.stats)},
-        {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, [], ?MODULE).
-
-
--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, 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.
-
-
-% Updater is a separate process. It receives `update_stats` messages and
-% updates scheduler stats from the scheduler jobs table. Updates are
-% performed no more frequently than once per ?STATS_UPDATE_WAIT milliseconds.
-
-update_running_jobs_stats(StatsPid) when is_pid(StatsPid) ->
-    StatsPid ! update_stats,
-    ok.
-
-
-start_stats_updater() ->
-    erlang:spawn_link(?MODULE, stats_updater_loop, [undefined]).
-
-
-stats_updater_loop(Timer) ->
-    receive
-        update_stats when Timer == undefined ->
-            TRef = erlang:send_after(?STATS_UPDATE_WAIT, self(), refresh_stats),
-            ?MODULE:stats_updater_loop(TRef);
-        update_stats when is_reference(Timer) ->
-            ?MODULE:stats_updater_loop(Timer);
-        refresh_stats ->
-            ok = stats_updater_refresh(),
-            ?MODULE:stats_updater_loop(undefined);
-        Else ->
-            erlang:exit({stats_updater_bad_msg, Else})
-    end.
-
-
--spec stats_updater_refresh() -> ok.
-stats_updater_refresh() ->
-    #stats_acc{
-       pending_n = PendingN,
-       running_n = RunningN,
-       crashed_n = CrashedN
-    } =  ets:foldl(fun stats_fold/2, #stats_acc{}, ?MODULE),
-    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),
-    ok.
-
-
--spec stats_fold(#job{}, #stats_acc{}) -> #stats_acc{}.
-stats_fold(#job{pid = undefined, history = [{added, _}]}, Acc) ->
-    Acc#stats_acc{pending_n = Acc#stats_acc.pending_n + 1};
-stats_fold(#job{pid = undefined, history = [{stopped, _} | _]}, Acc) ->
-    Acc#stats_acc{pending_n = Acc#stats_acc.pending_n + 1};
-stats_fold(#job{pid = undefined, history = [{{crashed, _}, _} | _]}, Acc) ->
-    Acc#stats_acc{crashed_n =Acc#stats_acc.crashed_n + 1};
-stats_fold(#job{pid = P, history = [{started, _} | _]}, Acc) when is_pid(P) ->
-    Acc#stats_acc{running_n = Acc#stats_acc.running_n + 1}.
-
-
--spec existing_replication(#rep{}) -> boolean().
-existing_replication(#rep{} = NewRep) ->
-    case job_by_id(NewRep#rep.id) of
-        {ok, #job{rep = CurRep}} ->
-            NormCurRep = couch_replicator_utils:normalize_rep(CurRep),
-            NormNewRep = couch_replicator_utils:normalize_rep(NewRep),
-            NormCurRep == NormNewRep;
-        {error, not_found} ->
-            false
-    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)]}
-    ]].
-
-
-longest_running_test() ->
-    J0 = testjob([crashed()]),
-    J1 = testjob([started(1)]),
-    J2 = testjob([started(2)]),
-    Sort = fun(Jobs) -> lists:sort(fun longest_running/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_() ->
-    {
-        setup,
-        fun setup_all/0,
-        fun teardown_all/1,
-        {
-            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_jobs_churn_even_if_not_all_max_jobs_are_running(),
-                t_jobs_dont_churn_if_there_are_available_running_slots(),
-                t_start_only_pending_jobs_do_not_churn_existing_ones(),
-                t_dont_stop_if_nothing_pending(),
-                t_max_churn_limits_number_of_rotated_jobs(),
-                t_existing_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_still_happens(),
-                t_if_transient_job_crashes_it_gets_removed(),
-                t_if_permanent_job_crashes_it_stays_in_ets(),
-                t_job_summary_running(),
-                t_job_summary_pending(),
-                t_job_summary_crashing_once(),
-                t_job_summary_crashing_many_times(),
-                t_job_summary_proxy_fields()
-            ]
-        }
-    }.
-
-
-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, 1)),
-        ?assertEqual({2, 1}, run_stop_count()),
-        ?assertEqual([4], jobs_stopped()),
-        reschedule(mock_state(1, 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({2, 1}, run_stop_count()),
-        % After rescheduling with max_jobs = 2, 2 was stopped and 5, 7 should
-        % be running.
-        ?assertEqual([2], jobs_stopped())
-    end).
-
-
-t_jobs_churn_even_if_not_all_max_jobs_are_running() ->
-    ?_test(begin
-        setup_jobs([
-            continuous_running(7),
-            continuous(2),
-            continuous(5)
-        ]),
-        reschedule(mock_state(2, 2)),
-        ?assertEqual({2, 1}, run_stop_count()),
-        ?assertEqual([7], jobs_stopped())
-    end).
-
-
-t_jobs_dont_churn_if_there_are_available_running_slots() ->
-     ?_test(begin
-        setup_jobs([
-            continuous_running(1),
-            continuous_running(2)
-        ]),
-        reschedule(mock_state(2, 2)),
-        ?assertEqual({2, 0}, run_stop_count()),
-        ?assertEqual([], jobs_stopped()),
-        ?assertEqual(0, meck:num_calls(couch_replicator_scheduler_sup, start_child, 1))
-    end).
-
-
-t_start_only_pending_jobs_do_not_churn_existing_ones() ->
-     ?_test(begin
-        setup_jobs([
-            continuous(1),
-            continuous_running(2)
-        ]),
-        reschedule(mock_state(2, 2)),
-        ?assertEqual(1, meck:num_calls(couch_replicator_scheduler_sup, start_child, 1)),
-        ?assertEqual([], jobs_stopped()),
-        ?assertEqual({2, 0}, run_stop_count())
-    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_still_happens() ->
-    ?_test(begin
-        Jobs = [
-            continuous(1),
-            continuous_running(2),
-            continuous_running(3)
-        ],
-        setup_jobs(Jobs),
-        reschedule(mock_state(1)),
-        ?assertEqual([1], 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, stats_pid = self()},
-        {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, stats_pid = self()},
-        {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).
-
-
-t_existing_jobs() ->
-    ?_test(begin
-        Rep = #rep{
-            id = job1,
-            db_name = <<"db">>,
-            source = <<"s">>,
-            target = <<"t">>,
-            options = [{continuous, true}]
-        },
-        setup_jobs([#job{id = Rep#rep.id, rep = Rep}]),
-        NewRep = #rep{
-            id = Rep#rep.id,
-            db_name = <<"db">>,
-            source = <<"s">>,
-            target = <<"t">>,
-            options = [{continuous, true}]
-        },
-        ?assert(existing_replication(NewRep)),
-        ?assertNot(existing_replication(NewRep#rep{source = <<"s1">>})),
-        ?assertNot(existing_replication(NewRep#rep{target = <<"t1">>})),
-        ?assertNot(existing_replication(NewRep#rep{options = []}))
-    end).
-
-
-t_job_summary_running() ->
-    ?_test(begin
-        Job =  #job{
-            id = job1,
-            pid = mock_pid(),
-            history = [added()],
-            rep = #rep{
-                db_name = <<"db1">>,
-                source = <<"s">>,
-                target = <<"t">>
-            }
-        },
-        setup_jobs([Job]),
-        Summary = job_summary(job1, ?DEFAULT_HEALTH_THRESHOLD_SEC),
-        ?assertEqual(running, proplists:get_value(state, Summary)),
-        ?assertEqual(null, proplists:get_value(info, Summary)),
-        ?assertEqual(0, proplists:get_value(error_count, Summary)),
-
-        Stats = [{source_seq, <<"1-abc">>}],
-        handle_cast({update_job_stats, job1, Stats}, mock_state(1)),
-        Summary1 = job_summary(job1, ?DEFAULT_HEALTH_THRESHOLD_SEC),
-        ?assertEqual({Stats}, proplists:get_value(info, Summary1))
-    end).
-
-
-t_job_summary_pending() ->
-    ?_test(begin
-        Job =  #job{
-            id = job1,
-            pid = undefined,
-            history = [stopped(20), started(10), added()],
-            rep = #rep{source = <<"s">>, target = <<"t">>}
-        },
-        setup_jobs([Job]),
-        Summary = job_summary(job1, ?DEFAULT_HEALTH_THRESHOLD_SEC),
-        ?assertEqual(pending, proplists:get_value(state, Summary)),
-        ?assertEqual(null, proplists:get_value(info, Summary)),
-        ?assertEqual(0, proplists:get_value(error_count, Summary)),
-
-        Stats = [{doc_write_failures, 1}],
-        handle_cast({update_job_stats, job1, Stats}, mock_state(1)),
-        Summary1 = job_summary(job1, ?DEFAULT_HEALTH_THRESHOLD_SEC),
-        ?assertEqual({Stats}, proplists:get_value(info, Summary1))
-    end).
-
-
-t_job_summary_crashing_once() ->
-    ?_test(begin
-        Job =  #job{
-            id = job1,
-            history = [crashed(?DEFAULT_HEALTH_THRESHOLD_SEC + 1), started(0)],
-            rep = #rep{source = <<"s">>, target = <<"t">>}
-        },
-        setup_jobs([Job]),
-        Summary = job_summary(job1, ?DEFAULT_HEALTH_THRESHOLD_SEC),
-        ?assertEqual(crashing, proplists:get_value(state, Summary)),
-        Info = proplists:get_value(info, Summary),
-        ?assertEqual({[{<<"error">>, <<"some_reason">>}]}, Info),
-        ?assertEqual(0, proplists:get_value(error_count, Summary))
-    end).
-
-
-t_job_summary_crashing_many_times() ->
-    ?_test(begin
-        Job =  #job{
-            id = job1,
-            history = [crashed(4), started(3), crashed(2), started(1)],
-            rep = #rep{source = <<"s">>, target = <<"t">>}
-        },
-        setup_jobs([Job]),
-        Summary = job_summary(job1, ?DEFAULT_HEALTH_THRESHOLD_SEC),
-        ?assertEqual(crashing, proplists:get_value(state, Summary)),
-        Info = proplists:get_value(info, Summary),
-        ?assertEqual({[{<<"error">>, <<"some_reason">>}]}, Info),
-        ?assertEqual(2, proplists:get_value(error_count, Summary))
-    end).
-
-
-t_job_summary_proxy_fields() ->
-    ?_test(begin
-        Job =  #job{
-            id = job1,
-            history = [started(10), added()],
-            rep = #rep{
-                source = #httpdb{
-                    url = "https://s",
-                    proxy_url = "http://u:p@sproxy:12"
-                },
-                target = #httpdb{
-                    url = "http://t",
-                    proxy_url = "socks5://u:p@tproxy:34"
-                }
-            }
-        },
-        setup_jobs([Job]),
-        Summary = job_summary(job1, ?DEFAULT_HEALTH_THRESHOLD_SEC),
-        ?assertEqual(<<"http://u:*****@sproxy:12">>,
-            proplists:get_value(source_proxy, Summary)),
-        ?assertEqual(<<"socks5://u:*****@tproxy:34">>,
-            proplists:get_value(target_proxy, Summary))
-    end).
-
-
-% Test helper functions
-
-setup_all() ->
-    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_all(_) ->
-    catch ets:delete(?MODULE),
-    meck:unload().
-
-
-setup() ->
-    meck:reset([
-        couch_log,
-        couch_replicator_scheduler_sup,
-        couch_stats
-    ]).
-
-
-teardown(_) ->
-    ok.
-
-
-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,
-        stats_pid = self()
-    }.
-
-mock_state(MaxJobs, MaxChurn) ->
-    #state{
-        max_jobs = MaxJobs,
-        max_churn = MaxChurn,
-        max_history = ?DEFAULT_MAX_HISTORY,
-        stats_pid = self()
-    }.
-
-
-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/src/couch_replicator_scheduler.hrl b/src/couch_replicator/src/couch_replicator_scheduler.hrl
deleted file mode 100644
index 5203b0c..0000000
--- a/src/couch_replicator/src/couch_replicator_scheduler.hrl
+++ /dev/null
@@ -1,15 +0,0 @@
-% 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/src/couch_replicator_scheduler_job.erl b/src/couch_replicator/src/couch_replicator_scheduler_job.erl
deleted file mode 100644
index 0b33419..0000000
--- a/src/couch_replicator/src/couch_replicator_scheduler_job.erl
+++ /dev/null
@@ -1,1090 +0,0 @@
-% 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).
-
--export([
-   start_link/1
-]).
-
--export([
-   init/1,
-   terminate/2,
-   handle_call/3,
-   handle_info/2,
-   handle_cast/2,
-   code_change/3,
-   format_status/2
-]).
-
--include_lib("couch/include/couch_db.hrl").
--include_lib("couch_replicator/include/couch_replicator_api_wrap.hrl").
--include("couch_replicator_scheduler.hrl").
--include("couch_replicator.hrl").
-
--import(couch_util, [
-    get_value/2,
-    get_value/3,
-    to_binary/1
-]).
-
--import(couch_replicator_utils, [
-    pp_rep_id/1
-]).
-
-
--define(LOWEST_SEQ, 0).
--define(DEFAULT_CHECKPOINT_INTERVAL, 30000).
--define(STARTUP_JITTER_DEFAULT, 5000).
-
--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_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} ->
-            {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),
-
-    timer:sleep(startup_jitter()),
-
-    #rep_state{
-        source = Source,
-        target = Target,
-        source_name = SourceName,
-        target_name = TargetName,
-        start_seq = {_Ts, StartSeq},
-        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)},
-        {source_seq, HighestSeq},
-        {checkpoint_interval, CheckpointInterval}
-    ] ++ rep_stats(State)),
-    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.
-
-    log_replication_start(State),
-    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
-        }
-    }.
-
-
-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(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}}.
-
-
-handle_info(shutdown, St) ->
-    {stop, shutdown, 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, Reason0}, #rep_state{changes_reader=Pid} = State) ->
-    couch_stats:increment_counter([couch_replicator, changes_reader_deaths]),
-    Reason = case Reason0 of
-        {changes_req_failed, _, _} = HttpFail ->
-            HttpFail;
-        {http_request_failed, _, _, {error, {code, Code}}} ->
-            {changes_req_failed, Code};
-        {http_request_failed, _, _, {error, Err}} ->
-            {changes_req_failed, Err};
-        Other ->
-            {changes_reader_died, Other}
-    end,
-    couch_log:error("ChangesReader process died with reason: ~p", [Reason]),
-    {stop, {shutdown, Reason}, 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, {shutdown, {changes_manager_died, Reason}}, 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, {shutdown, {changes_queue_died, Reason}}, 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]),
-        StopReason = case Reason of
-            {shutdown, _} = Err ->
-                Err;
-            Other ->
-                couch_log:error("Worker ~p died with reason: ~p", [Pid, Reason]),
-                {worker_died, Pid, Other}
-         end,
-        {stop, StopReason, 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)},
-            StackTop2 = lists:sublist(erlang:get_stacktrace(), 2),
-            % 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, StackTop2, InitArgs},
-            {stop, {shutdown, ShutdownReason}, ShutdownState}
-    end.
-
-
-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 = {BaseId, Ext} = RepId,
-        source = Source0,
-        target = Target0,
-        doc_id = DocId,
-        db_name = DbName
-    } = InitArgs,
-    Source = couch_replicator_api_wrap:db_uri(Source0),
-    Target = couch_replicator_api_wrap:db_uri(Target0),
-    RepIdStr = BaseId ++ Ext,
-    Msg = "~p:~p: Replication ~s failed to start ~p -> ~p doc ~p:~p stack:~p",
-    couch_log:error(Msg, [Class, Error, RepIdStr, Source, Target, DbName,
-        DocId, Stack]),
-    couch_stats:increment_counter([couch_replicator, failed_starts]),
-    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({shutdown, Reason}, State) ->
-    % Unwrap so when reporting we don't have an extra {shutdown, ...} tuple
-    % wrapped around the message
-    terminate(Reason, State);
-
-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),
-    couch_replicator_api_wrap:db_close(State#rep_state.source),
-    couch_replicator_api_wrap:db_close(State#rep_state.target).
-
-
-code_change(_OldVsn, #rep_state{}=State, _Extra) ->
-    {ok, State}.
-
-
-format_status(_Opt, [_PDict, State]) ->
-    #rep_state{
-       source = Source,
-       target = Target,
-       rep_details = RepDetails,
-       start_seq = StartSeq,
-       source_seq = SourceSeq,
-       committed_seq = CommitedSeq,
-       current_through_seq = ThroughSeq,
-       highest_seq_done = HighestSeqDone,
-       session_id = SessionId
-    } = state_strip_creds(State),
-    #rep{
-       id = RepId,
-       options = Options,
-       doc_id = DocId,
-       db_name = DbName
-    } = RepDetails,
-    [
-        {rep_id, RepId},
-        {source, couch_replicator_api_wrap:db_uri(Source)},
-        {target, couch_replicator_api_wrap:db_uri(Target)},
-        {db_name, DbName},
-        {doc_id, DocId},
-        {options, Options},
-        {session_id, SessionId},
-        {start_seq, StartSeq},
-        {source_seq, SourceSeq},
-        {committed_seq, CommitedSeq},
-        {current_through_seq, ThroughSeq},
-        {highest_seq_done, HighestSeqDone}
-    ].
-
-
-startup_jitter() ->
-    Jitter = config:get_integer("replicator", "startup_jitter",
-        ?STARTUP_JITTER_DEFAULT),
-    couch_rand:uniform(erlang:max(1, Jitter)).
-
-
-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)
-    }.
-
-
-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.
-
-
--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}, Stats0) ->
-    Stats = Stats0 ++ [{start_time, couch_replicator_utils:iso8601(StartTime)}],
-    couch_replicator_docs:update_doc_completed(DbName, DocId, Stats),
-    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,
-        type = Type, view = View,
-        start_time = StartTime,
-        stats = ArgStats0
-    } = 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),
-    {CreateTargetParams} = get_value(create_target_params, Options, {[]}),
-    {ok, Target} = couch_replicator_api_wrap:db_open(Tgt,
-        get_value(create_target, Options, false), CreateTargetParams),
-
-    {ok, SourceInfo} = couch_replicator_api_wrap:get_db_info(Source),
-    {ok, TargetInfo} = couch_replicator_api_wrap:get_db_info(Target),
-
-    [SourceLog, TargetLog] = find_and_migrate_logs([Source, Target], Rep),
-
-    {StartSeq0, History} = compare_replication_logs(SourceLog, TargetLog),
-
-    ArgStats1 = couch_replicator_stats:new(ArgStats0),
-    HistoryStats = case History of
-        [{[_ | _] = HProps} | _] -> couch_replicator_stats:new(HProps);
-        _ -> couch_replicator_stats:new()
-    end,
-    Stats = couch_replicator_stats:max_stats(ArgStats1, HistoryStats),
-
-    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_seq = SourceSeq,
-        use_checkpoints = get_value(use_checkpoints, Options, true),
-        checkpoint_interval = get_value(checkpoint_interval, Options,
-                                        ?DEFAULT_CHECKPOINT_INTERVAL),
-        type = Type,
-        view = View,
-        stats = Stats
-    },
-    State#rep_state{timer = start_timer(State)}.
-
-
-find_and_migrate_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},
-        maybe_save_migrated_log(Rep, Db, MigratedLog, Doc#doc.id),
-        fold_replication_logs(
-            Rest, ?REP_ID_VERSION, NewId, NewId, Rep, [MigratedLog | Acc])
-    end.
-
-
-maybe_save_migrated_log(Rep, Db, #doc{} = Doc, OldId) ->
-    case get_value(use_checkpoints, Rep#rep.options, true) of
-        true ->
-            update_checkpoint(Db, Doc),
-            Msg = "Migrated replication checkpoint. Db:~p ~p -> ~p",
-            couch_log:notice(Msg, [httpdb_strip_creds(Db), OldId, Doc#doc.id]);
-        false ->
-            ok
-    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, ChangesOrLastSeqs} ->
-            ReportSeq = case lists:last(ChangesOrLastSeqs) of
-                {last_seq, Seq} ->
-                    {Ts, Seq};
-                #doc_info{high_seq = Seq} ->
-                    {Ts, Seq}
-            end,
-            Changes = lists:filter(
-                fun(#doc_info{}) ->
-                    true;
-                ({last_seq, _Seq}) ->
-                    false
-            end, ChangesOrLastSeqs),
-            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]),
-        LocalStartTime = calendar:now_to_local_time(ReplicationStartTime),
-        StartTime = ?l2b(httpd_util:rfc1123_date(LocalStartTime)),
-        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.
-
-
-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{
-        rep_details = #rep{id = JobId},
-        current_through_seq = {_, ThroughSeq},
-        highest_seq_done = {_, HighestSeq}
-    } = State,
-    Status = rep_stats(State) ++ [
-        {source_seq, HighestSeq},
-        {through_seq, ThroughSeq}
-    ],
-    couch_replicator_scheduler:update_job_stats(JobId, Status),
-    couch_task_status:update(Status).
-
-
-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({http_request_failed, _Method, Url0,
-        {error, {error, {conn_failed, {error, nxdomain}}}}}) ->
-    Url = ?l2b(couch_util:url_strip_password(Url0)),
-    {nxdomain, <<"could not resolve ", Url/binary>>};
-replication_start_error({http_request_failed, Method0, Url0,
-        {error, {code, Code}}}) when is_integer(Code) ->
-    Url = ?l2b(couch_util:url_strip_password(Url0)),
-    Method = ?l2b(Method0),
-    {http_error_code, Code, <<Method/binary, " ", Url/binary>>};
-replication_start_error(Error) ->
-    Error.
-
-
-log_replication_start(#rep_state{rep_details = Rep} = RepState) ->
-    #rep{
-       id = {BaseId, Ext},
-       doc_id = DocId,
-       db_name = DbName,
-       options = Options
-    } = Rep,
-    Id = BaseId ++ Ext,
-    Workers = get_value(worker_processes, Options),
-    BatchSize = get_value(worker_batch_size, Options),
-    #rep_state{
-       source_name = Source,  % credentials already stripped
-       target_name = Target,  % credentials already stripped
-       session_id = Sid
-    } = RepState,
-    From = case DbName of
-        ShardName when is_binary(ShardName) ->
-            io_lib:format("from doc ~s:~s", [mem3:dbname(ShardName), DocId]);
-        _ ->
-            "from _replicate endpoint"
-    end,
-    Msg = "Starting replication ~s (~s -> ~s) ~s worker_procesess:~p"
-        " worker_batch_size:~p session_id:~s",
-    couch_log:notice(Msg, [Id, Source, Target, From, Workers, BatchSize, Sid]).
-
-
--ifdef(TEST).
-
--include_lib("eunit/include/eunit.hrl").
-
-
-replication_start_error_test() ->
-    ?assertEqual({unauthorized, <<"unauthorized to access or create database"
-        " http://x/y">>}, replication_start_error({unauthorized,
-        <<"http://x/y">>})),
-    ?assertEqual({db_not_found, <<"could not open http://x/y">>},
-        replication_start_error({db_not_found, <<"http://x/y">>})),
-    ?assertEqual({nxdomain,<<"could not resolve http://x/y">>},
-        replication_start_error({http_request_failed, "GET", "http://x/y",
-        {error, {error, {conn_failed, {error, nxdomain}}}}})),
-    ?assertEqual({http_error_code,503,<<"GET http://x/y">>},
-        replication_start_error({http_request_failed, "GET", "http://x/y",
-        {error, {code, 503}}})).
-
-
-scheduler_job_format_status_test() ->
-    Source = <<"http://u:p@h1/d1">>,
-    Target = <<"http://u:p@h2/d2">>,
-    Rep = #rep{
-        id = {"base", "+ext"},
-        source = couch_replicator_docs:parse_rep_db(Source, [], []),
-        target = couch_replicator_docs:parse_rep_db(Target, [], []),
-        options = [{create_target, true}],
-        doc_id = <<"mydoc">>,
-        db_name = <<"mydb">>
-    },
-    State = #rep_state{
-        rep_details = Rep,
-        source = Rep#rep.source,
-        target = Rep#rep.target,
-        session_id = <<"a">>,
-        start_seq = <<"1">>,
-        source_seq = <<"2">>,
-        committed_seq = <<"3">>,
-        current_through_seq = <<"4">>,
-        highest_seq_done = <<"5">>
-    },
-    Format = format_status(opts_ignored, [pdict, State]),
-    ?assertEqual("http://u:*****@h1/d1/", proplists:get_value(source, Format)),
-    ?assertEqual("http://u:*****@h2/d2/", proplists:get_value(target, Format)),
-    ?assertEqual({"base", "+ext"}, proplists:get_value(rep_id, Format)),
-    ?assertEqual([{create_target, true}], proplists:get_value(options, Format)),
-    ?assertEqual(<<"mydoc">>, proplists:get_value(doc_id, Format)),
-    ?assertEqual(<<"mydb">>, proplists:get_value(db_name, Format)),
-    ?assertEqual(<<"a">>, proplists:get_value(session_id, Format)),
-    ?assertEqual(<<"1">>, proplists:get_value(start_seq, Format)),
-    ?assertEqual(<<"2">>, proplists:get_value(source_seq, Format)),
-    ?assertEqual(<<"3">>, proplists:get_value(committed_seq, Format)),
-    ?assertEqual(<<"4">>, proplists:get_value(current_through_seq, Format)),
-    ?assertEqual(<<"5">>, proplists:get_value(highest_seq_done, Format)).
-
-
--endif.
diff --git a/src/couch_replicator/src/couch_replicator_scheduler_sup.erl b/src/couch_replicator/src/couch_replicator_scheduler_sup.erl
deleted file mode 100644
index 8ab55f8..0000000
--- a/src/couch_replicator/src/couch_replicator_scheduler_sup.erl
+++ /dev/null
@@ -1,62 +0,0 @@
-% 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).
-
-%% public api
--export([
-    start_link/0,
-    start_child/1,
-    terminate_child/1
-]).
-
-%% supervisor api
--export([
-   init/1
-]).
-
-
-%% includes
--include("couch_replicator.hrl").
-
-
-%% 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/test/eunit/couch_replicator_compact_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_compact_tests.erl
deleted file mode 100644
index 997c848..0000000
--- a/src/couch_replicator/test/eunit/couch_replicator_compact_tests.erl
+++ /dev/null
@@ -1,455 +0,0 @@
-% 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_compact_tests).
-
--include_lib("couch/include/couch_eunit.hrl").
--include_lib("couch/include/couch_db.hrl").
--include_lib("couch_replicator/src/couch_replicator.hrl").
-
--import(couch_replicator_test_helper, [
-    db_url/1,
-    get_pid/1
-]).
-
--define(ATTFILE, filename:join([?FIXTURESDIR, "logo.png"])).
--define(DELAY, 500).
--define(TIMEOUT, 360000).
--define(TIMEOUT_WRITER, 100000).
--define(TIMEOUT_EUNIT, ?TIMEOUT div 1000 + 70).
--define(WRITE_BATCH_SIZE, 25).
-
-setup() ->
-    DbName = ?tempdb(),
-    {ok, Db} = couch_db:create(DbName, [?ADMIN_CTX]),
-    ok = couch_db:close(Db),
-    DbName.
-
-setup(remote) ->
-    {remote, setup()};
-setup({A, B}) ->
-    Ctx = test_util:start_couch([couch_replicator]),
-    Source = setup(A),
-    Target = setup(B),
-    {Ctx, {Source, Target}}.
-
-teardown({remote, DbName}) ->
-    teardown(DbName);
-teardown(DbName) ->
-    ok = couch_server:delete(DbName, [?ADMIN_CTX]),
-    ok.
-
-teardown(_, {Ctx, {Source, Target}}) ->
-    teardown(Source),
-    teardown(Target),
-    ok = application:stop(couch_replicator),
-    ok = test_util:stop_couch(Ctx).
-
-compact_test_() ->
-    Pairs = [{remote, remote}],
-    {
-        "Compaction during replication tests",
-        {
-            foreachx,
-            fun setup/1, fun teardown/2,
-            [{Pair, fun should_populate_replicate_compact/2}
-             || Pair <- Pairs]
-        }
-    }.
-
-
-should_populate_replicate_compact({From, To}, {_Ctx, {Source, Target}}) ->
-    {ok, RepPid, RepId} = replicate(Source, Target),
-    {lists:flatten(io_lib:format("~p -> ~p", [From, To])),
-     {inorder, [
-         should_run_replication(RepPid, RepId, Source, Target),
-         should_all_processes_be_alive(RepPid, Source, Target),
-         should_populate_and_compact(RepPid, Source, Target, 50, 3),
-         should_wait_target_in_sync(Source, Target),
-         should_ensure_replication_still_running(RepPid, RepId, Source, Target),
-         should_cancel_replication(RepId, RepPid),
-         should_compare_databases(Source, Target)
-     ]}}.
-
-should_all_processes_be_alive(RepPid, Source, Target) ->
-    ?_test(begin
-        {ok, SourceDb} = reopen_db(Source),
-        {ok, TargetDb} = reopen_db(Target),
-        ?assert(is_process_alive(RepPid)),
-        ?assert(is_process_alive(couch_db:get_pid(SourceDb))),
-        ?assert(is_process_alive(couch_db:get_pid(TargetDb)))
-    end).
-
-should_run_replication(RepPid, RepId, Source, Target) ->
-    ?_test(check_active_tasks(RepPid, RepId, Source, Target)).
-
-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) ->
-    Source = case Src of
-        {remote, NameSrc} ->
-            <<(db_url(NameSrc))/binary, $/>>;
-        _ ->
-            Src
-    end,
-    Target = case Tgt of
-        {remote, NameTgt} ->
-            <<(db_url(NameTgt))/binary, $/>>;
-        _ ->
-            Tgt
-    end,
-    FullRepId = ?l2b(BaseId ++ Ext),
-    Pid = ?l2b(pid_to_list(RepPid)),
-    RepTasks = wait_for_task_status(),
-    ?assertNotEqual(timeout, RepTasks),
-    [RepTask] = RepTasks,
-    ?assertEqual(Pid, couch_util:get_value(pid, RepTask)),
-    ?assertEqual(FullRepId, couch_util:get_value(replication_id, RepTask)),
-    ?assertEqual(true, couch_util:get_value(continuous, RepTask)),
-    ?assertEqual(Source, couch_util:get_value(source, RepTask)),
-    ?assertEqual(Target, couch_util:get_value(target, RepTask)),
-    ?assert(is_integer(couch_util:get_value(docs_read, RepTask))),
-    ?assert(is_integer(couch_util:get_value(docs_written, RepTask))),
-    ?assert(is_integer(couch_util:get_value(doc_write_failures, RepTask))),
-    ?assert(is_integer(couch_util:get_value(revisions_checked, RepTask))),
-    ?assert(is_integer(couch_util:get_value(missing_revisions_found, RepTask))),
-    ?assert(is_integer(couch_util:get_value(checkpointed_source_seq, RepTask))),
-    ?assert(is_integer(couch_util:get_value(source_seq, RepTask))),
-    Pending = couch_util:get_value(changes_pending, RepTask),
-    ?assert(is_integer(Pending)).
-
-replication_tasks() ->
-    lists:filter(fun(P) ->
-        couch_util:get_value(type, P) =:= replication
-    end, couch_task_status:all()).
-
-
-wait_for_task_status() ->
-    test_util:wait(fun() ->
-        case replication_tasks() of
-            [] ->
-                wait;
-            Tasks ->
-                Tasks
-        end
-    end).
-
-should_cancel_replication(RepId, RepPid) ->
-    ?_assertNot(begin
-        ok = couch_replicator_scheduler:remove_job(RepId),
-        is_process_alive(RepPid)
-    end).
-
-should_populate_and_compact(RepPid, Source, Target, BatchSize, Rounds) ->
-    {timeout, ?TIMEOUT_EUNIT, ?_test(begin
-        {ok, SourceDb0} = reopen_db(Source),
-        Writer = spawn_writer(SourceDb0),
-        lists:foreach(
-            fun(N) ->
-                {ok, SourceDb} = reopen_db(Source),
-                {ok, TargetDb} = reopen_db(Target),
-                pause_writer(Writer),
-
-                compact_db("source", SourceDb),
-                ?assert(is_process_alive(RepPid)),
-                ?assert(is_process_alive(couch_db:get_pid(SourceDb))),
-                wait_for_compaction("source", SourceDb),
-
-                compact_db("target", TargetDb),
-                ?assert(is_process_alive(RepPid)),
-                ?assert(is_process_alive(couch_db:get_pid(TargetDb))),
-                wait_for_compaction("target", TargetDb),
-
-                {ok, SourceDb2} = reopen_db(SourceDb),
-                {ok, TargetDb2} = reopen_db(TargetDb),
-
-                resume_writer(Writer),
-                wait_writer(Writer, BatchSize * N),
-
-                compact_db("source", SourceDb2),
-                ?assert(is_process_alive(RepPid)),
-                ?assert(is_process_alive(couch_db:get_pid(SourceDb2))),
-                pause_writer(Writer),
-                wait_for_compaction("source", SourceDb2),
-                resume_writer(Writer),
-
-                compact_db("target", TargetDb2),
-                ?assert(is_process_alive(RepPid)),
-                ?assert(is_process_alive(couch_db:get_pid(TargetDb2))),
-                pause_writer(Writer),
-                wait_for_compaction("target", TargetDb2),
-                resume_writer(Writer)
-            end, lists:seq(1, Rounds)),
-        stop_writer(Writer)
-    end)}.
-
-should_wait_target_in_sync({remote, Source}, Target) ->
-    should_wait_target_in_sync(Source, Target);
-should_wait_target_in_sync(Source, {remote, Target}) ->
-    should_wait_target_in_sync(Source, Target);
-should_wait_target_in_sync(Source, Target) ->
-    {timeout, ?TIMEOUT_EUNIT, ?_assert(begin
-        {ok, SourceDb} = couch_db:open_int(Source, []),
-        {ok, SourceInfo} = couch_db:get_db_info(SourceDb),
-        ok = couch_db:close(SourceDb),
-        SourceDocCount = couch_util:get_value(doc_count, SourceInfo),
-        wait_target_in_sync_loop(SourceDocCount, Target, 300)
-    end)}.
-
-wait_target_in_sync_loop(_DocCount, _TargetName, 0) ->
-    erlang:error(
-        {assertion_failed,
-         [{module, ?MODULE}, {line, ?LINE},
-          {reason, "Could not get source and target databases in sync"}]});
-wait_target_in_sync_loop(DocCount, {remote, TargetName}, RetriesLeft) ->
-    wait_target_in_sync_loop(DocCount, TargetName, RetriesLeft);
-wait_target_in_sync_loop(DocCount, TargetName, RetriesLeft) ->
-    {ok, Target} = couch_db:open_int(TargetName, []),
-    {ok, TargetInfo} = couch_db:get_db_info(Target),
-    ok = couch_db:close(Target),
-    TargetDocCount = couch_util:get_value(doc_count, TargetInfo),
-    case TargetDocCount == DocCount of
-        true ->
-            true;
-        false ->
-            ok = timer:sleep(?DELAY),
-            wait_target_in_sync_loop(DocCount, TargetName, RetriesLeft - 1)
-    end.
-
-should_compare_databases({remote, Source}, Target) ->
-    should_compare_databases(Source, Target);
-should_compare_databases(Source, {remote, Target}) ->
-    should_compare_databases(Source, Target);
-should_compare_databases(Source, Target) ->
-    {timeout, 35, ?_test(begin
-        {ok, SourceDb} = couch_db:open_int(Source, []),
-        {ok, TargetDb} = couch_db:open_int(Target, []),
-        Fun = fun(FullDocInfo, Acc) ->
-            {ok, Doc} = couch_db:open_doc(SourceDb, FullDocInfo),
-            {Props} = DocJson = couch_doc:to_json_obj(Doc, [attachments]),
-            DocId = couch_util:get_value(<<"_id">>, Props),
-            DocTarget = case couch_db:open_doc(TargetDb, DocId) of
-                {ok, DocT} ->
-                    DocT;
-                Error ->
-                    erlang:error(
-                        {assertion_failed,
-                         [{module, ?MODULE}, {line, ?LINE},
-                          {reason, lists:concat(["Error opening document '",
-                                                 ?b2l(DocId), "' from target: ",
-                                                 couch_util:to_list(Error)])}]})
-            end,
-            DocTargetJson = couch_doc:to_json_obj(DocTarget, [attachments]),
-            ?assertEqual(DocJson, DocTargetJson),
-            {ok, Acc}
-        end,
-        {ok, _} = couch_db:fold_docs(SourceDb, Fun, [], []),
-        ok = couch_db:close(SourceDb),
-        ok = couch_db:close(TargetDb)
-    end)}.
-
-
-reopen_db({remote, Db}) ->
-    reopen_db(Db);
-reopen_db(DbName) when is_binary(DbName) ->
-    {ok, Db} = couch_db:open_int(DbName, []),
-    ok = couch_db:close(Db),
-    {ok, Db};
-reopen_db(Db) ->
-    reopen_db(couch_db:name(Db)).
-
-
-compact_db(Type, Db0) ->
-    Name = couch_db:name(Db0),
-    {ok, Db} = couch_db:open_int(Name, []),
-    {ok, CompactPid} = couch_db:start_compact(Db),
-    MonRef = erlang:monitor(process, CompactPid),
-    receive
-        {'DOWN', MonRef, process, CompactPid, normal} ->
-            ok;
-        {'DOWN', MonRef, process, CompactPid, noproc} ->
-            ok;
-        {'DOWN', MonRef, process, CompactPid, Reason} ->
-            erlang:error(
-                {assertion_failed,
-                 [{module, ?MODULE}, {line, ?LINE},
-                  {reason,
-                   lists:concat(["Error compacting ", Type, " database ",
-                                 ?b2l(Name), ": ",
-                                 couch_util:to_list(Reason)])}]})
-    after ?TIMEOUT ->
-        erlang:error(
-            {assertion_failed,
-             [{module, ?MODULE}, {line, ?LINE},
-              {reason, lists:concat(["Compaction for ", Type, " database ",
-                                     ?b2l(Name), " didn't finish"])}]})
-    end,
-    ok = couch_db:close(Db).
-
-wait_for_compaction(Type, Db) ->
-    case couch_db:wait_for_compaction(Db) of
-        ok ->
-            ok;
-        {error, noproc} ->
-            ok;
-        {error, Reason} ->
-            erlang:error(
-                {assertion_failed,
-                 [{module, ?MODULE}, {line, ?LINE},
-                  {reason, lists:concat(["Compaction of ", Type,
-                                         " database failed with: ", Reason])}]})
-    end.
-
-replicate({remote, Db}, Target) ->
-    replicate(db_url(Db), Target);
-
-replicate(Source, {remote, Db}) ->
-    replicate(Source, db_url(Db));
-
-replicate(Source, Target) ->
-    RepObject = {[
-        {<<"source">>, Source},
-        {<<"target">>, Target},
-        {<<"continuous">>, true}
-    ]},
-    {ok, Rep} = couch_replicator_utils:parse_rep_doc(RepObject, ?ADMIN_USER),
-    ok = couch_replicator_scheduler:add_job(Rep),
-    couch_replicator_scheduler:reschedule(),
-    Pid = get_pid(Rep#rep.id),
-    {ok, Pid, Rep#rep.id}.
-
-
-wait_writer(Pid, NumDocs) ->
-    case get_writer_num_docs_written(Pid) of
-        N when N >= NumDocs ->
-            ok;
-        _ ->
-            wait_writer(Pid, NumDocs)
-    end.
-
-spawn_writer(Db) ->
-    Parent = self(),
-    Pid = spawn(fun() -> writer_loop(Db, Parent, 0) end),
-    Pid.
-
-
-pause_writer(Pid) ->
-    Ref = make_ref(),
-    Pid ! {pause, Ref},
-    receive
-        {paused, Ref} ->
-            ok
-    after ?TIMEOUT_WRITER ->
-        erlang:error({assertion_failed,
-                      [{module, ?MODULE},
-                       {line, ?LINE},
-                       {reason, "Failed to pause source database writer"}]})
-    end.
-
-resume_writer(Pid) ->
-    Ref = make_ref(),
-    Pid ! {continue, Ref},
-    receive
-        {ok, Ref} ->
-            ok
-    after ?TIMEOUT_WRITER ->
-        erlang:error({assertion_failed,
-                      [{module, ?MODULE},
-                       {line, ?LINE},
-                       {reason, "Failed to pause source database writer"}]})
-    end.
-
-get_writer_num_docs_written(Pid) ->
-    Ref = make_ref(),
-    Pid ! {get_count, Ref},
-    receive
-        {count, Ref, Count} ->
-            Count
-    after ?TIMEOUT_WRITER ->
-        erlang:error({assertion_failed,
-                      [{module, ?MODULE},
-                       {line, ?LINE},
-                       {reason, "Timeout getting number of documents written"
-                                " from source database writer"}]})
-    end.
-
-stop_writer(Pid) ->
-    Ref = make_ref(),
-    Pid ! {stop, Ref},
-    receive
-        {stopped, Ref, DocsWritten} ->
-            MonRef = erlang:monitor(process, Pid),
-            receive
-                {'DOWN', MonRef, process, Pid, _Reason} ->
-                    DocsWritten
-            after ?TIMEOUT ->
-                erlang:error({assertion_failed,
-                      [{module, ?MODULE},
-                       {line, ?LINE},
-                       {reason, "Timeout stopping source database writer"}]})
-            end
-    after ?TIMEOUT_WRITER ->
-        erlang:error({assertion_failed,
-                      [{module, ?MODULE},
-                       {line, ?LINE},
-                       {reason, "Timeout stopping source database writer"}]})
-    end.
-
-writer_loop(Db0, Parent, Counter) ->
-    DbName = couch_db:name(Db0),
-    {ok, Data} = file:read_file(?ATTFILE),
-    maybe_pause(Parent, Counter),
-    Docs = lists:map(fun(I) ->
-        couch_doc:from_json_obj({[
-            {<<"_id">>, ?l2b(integer_to_list(Counter + I))},
-            {<<"value">>, Counter + I},
-            {<<"_attachments">>, {[
-                {<<"icon1.png">>, {[
-                    {<<"data">>, base64:encode(Data)},
-                    {<<"content_type">>, <<"image/png">>}
-                ]}},
-                {<<"icon2.png">>, {[
-                    {<<"data">>, base64:encode(iolist_to_binary([Data, Data]))},
-                    {<<"content_type">>, <<"image/png">>}
-                ]}}
-            ]}}
-        ]})
-    end, lists:seq(1, ?WRITE_BATCH_SIZE)),
-    maybe_pause(Parent, Counter),
-    {ok, Db} = couch_db:open_int(DbName, []),
-    {ok, _} = couch_db:update_docs(Db, Docs, []),
-    ok = couch_db:close(Db),
-    receive
-        {get_count, Ref} ->
-            Parent ! {count, Ref, Counter + ?WRITE_BATCH_SIZE},
-            writer_loop(Db, Parent, Counter + ?WRITE_BATCH_SIZE);
-        {stop, Ref} ->
-            Parent ! {stopped, Ref, Counter + ?WRITE_BATCH_SIZE}
-    after 0 ->
-        timer:sleep(?DELAY),
-        writer_loop(Db, Parent, Counter + ?WRITE_BATCH_SIZE)
-    end.
-
-maybe_pause(Parent, Counter) ->
-    receive
-        {get_count, Ref} ->
-            Parent ! {count, Ref, Counter};
-        {pause, Ref} ->
-            Parent ! {paused, Ref},
-            receive
-                {continue, Ref2} ->
-                    Parent ! {ok, Ref2}
-            end
-    after 0 ->
-        ok
-    end.
diff --git a/src/couch_replicator/test/eunit/couch_replicator_error_reporting_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_error_reporting_tests.erl
deleted file mode 100644
index 6b4f95c..0000000
--- a/src/couch_replicator/test/eunit/couch_replicator_error_reporting_tests.erl
+++ /dev/null
@@ -1,271 +0,0 @@
-% 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_error_reporting_tests).
-
--include_lib("couch/include/couch_eunit.hrl").
--include_lib("couch/include/couch_db.hrl").
--include_lib("couch_replicator/src/couch_replicator.hrl").
-
-
-setup_all() ->
-    test_util:start_couch([couch_replicator, chttpd, mem3, fabric]).
-
-
-teardown_all(Ctx) ->
-    ok = test_util:stop_couch(Ctx).
-
-
-setup() ->
-    meck:unload(),
-    Source = setup_db(),
-    Target = setup_db(),
-    {Source, Target}.
-
-
-teardown({Source, Target}) ->
-    meck:unload(),
-    teardown_db(Source),
-    teardown_db(Target),
-    ok.
-
-
-error_reporting_test_() ->
-    {
-        setup,
-        fun setup_all/0,
-        fun teardown_all/1,
-        {
-            foreach,
-            fun setup/0,
-            fun teardown/1,
-            [
-                fun t_fail_bulk_docs/1,
-                fun t_fail_changes_reader/1,
-                fun t_fail_revs_diff/1,
-                fun t_fail_changes_queue/1,
-                fun t_fail_changes_manager/1,
-                fun t_fail_changes_reader_proc/1
-            ]
-        }
-    }.
-
-
-t_fail_bulk_docs({Source, Target}) ->
-    ?_test(begin
-        populate_db(Source, 1, 5),
-        {ok, RepId} = replicate(Source, Target),
-        wait_target_in_sync(Source, Target),
-
-        {ok, Listener} = rep_result_listener(RepId),
-        mock_fail_req("/_bulk_docs", {ok, "403", [], [<<"{\"x\":\"y\"}">>]}),
-        populate_db(Source, 6, 6),
-
-        {error, Result} = wait_rep_result(RepId),
-        ?assertEqual({bulk_docs_failed, 403, {[{<<"x">>, <<"y">>}]}}, Result),
-
-        couch_replicator_notifier:stop(Listener)
-    end).
-
-
-t_fail_changes_reader({Source, Target}) ->
-    ?_test(begin
-        populate_db(Source, 1, 5),
-        {ok, RepId} = replicate(Source, Target),
-        wait_target_in_sync(Source, Target),
-
-        {ok, Listener} = rep_result_listener(RepId),
-        mock_fail_req("/_changes", {ok, "418", [], [<<"{\"x\":\"y\"}">>]}),
-        populate_db(Source, 6, 6),
-
-        {error, Result} = wait_rep_result(RepId),
-        ?assertEqual({changes_req_failed, 418, {[{<<"x">>, <<"y">>}]}}, Result),
-
-        couch_replicator_notifier:stop(Listener)
-    end).
-
-
-t_fail_revs_diff({Source, Target}) ->
-    ?_test(begin
-        populate_db(Source, 1, 5),
-        {ok, RepId} = replicate(Source, Target),
-        wait_target_in_sync(Source, Target),
-
-        {ok, Listener} = rep_result_listener(RepId),
-        mock_fail_req("/_revs_diff", {ok, "407", [], [<<"{\"x\":\"y\"}">>]}),
-        populate_db(Source, 6, 6),
-
-        {error, Result} = wait_rep_result(RepId),
-        ?assertEqual({revs_diff_failed, 407, {[{<<"x">>, <<"y">>}]}}, Result),
-
-        couch_replicator_notifier:stop(Listener)
-    end).
-
-
-t_fail_changes_queue({Source, Target}) ->
-    ?_test(begin
-        populate_db(Source, 1, 5),
-        {ok, RepId} = replicate(Source, Target),
-        wait_target_in_sync(Source, Target),
-
-        RepPid = couch_replicator_test_helper:get_pid(RepId),
-        State = sys:get_state(RepPid),
-        ChangesQueue = element(20, State),
-        ?assert(is_process_alive(ChangesQueue)),
-
-        {ok, Listener} = rep_result_listener(RepId),
-        exit(ChangesQueue, boom),
-
-        {error, Result} = wait_rep_result(RepId),
-        ?assertEqual({changes_queue_died, boom}, Result),
-        couch_replicator_notifier:stop(Listener)
-    end).
-
-
-t_fail_changes_manager({Source, Target}) ->
-    ?_test(begin
-        populate_db(Source, 1, 5),
-        {ok, RepId} = replicate(Source, Target),
-        wait_target_in_sync(Source, Target),
-
-        RepPid = couch_replicator_test_helper:get_pid(RepId),
-        State = sys:get_state(RepPid),
-        ChangesManager = element(21, State),
-        ?assert(is_process_alive(ChangesManager)),
-
-        {ok, Listener} = rep_result_listener(RepId),
-        exit(ChangesManager, bam),
-
-        {error, Result} = wait_rep_result(RepId),
-        ?assertEqual({changes_manager_died, bam}, Result),
-        couch_replicator_notifier:stop(Listener)
-    end).
-
-
-t_fail_changes_reader_proc({Source, Target}) ->
-    ?_test(begin
-        populate_db(Source, 1, 5),
-        {ok, RepId} = replicate(Source, Target),
-        wait_target_in_sync(Source, Target),
-
-        RepPid = couch_replicator_test_helper:get_pid(RepId),
-        State = sys:get_state(RepPid),
-        ChangesReader = element(22, State),
-        ?assert(is_process_alive(ChangesReader)),
-
-        {ok, Listener} = rep_result_listener(RepId),
-        exit(ChangesReader, kapow),
-
-        {error, Result} = wait_rep_result(RepId),
-        ?assertEqual({changes_reader_died, kapow}, Result),
-        couch_replicator_notifier:stop(Listener)
-    end).
-
-
-mock_fail_req(Path, Return) ->
-    meck:expect(ibrowse, send_req_direct,
-        fun(W, Url, Headers, Meth, Body, Opts, TOut) ->
-            Args = [W, Url, Headers, Meth, Body, Opts, TOut],
-            {ok, {_, _, _, _, UPath, _}} = http_uri:parse(Url),
-            case lists:suffix(Path, UPath) of
-                true -> Return;
-                false -> meck:passthrough(Args)
-            end
-        end).
-
-
-rep_result_listener(RepId) ->
-    ReplyTo = self(),
-    {ok, _Listener} = couch_replicator_notifier:start_link(
-        fun({_, RepId2, _} = Ev) when RepId2 =:= RepId ->
-                ReplyTo ! Ev;
-            (_) ->
-                ok
-        end).
-
-
-wait_rep_result(RepId) ->
-    receive
-        {finished, RepId, RepResult} -> {ok, RepResult};
-        {error, RepId, Reason} -> {error, Reason}
-    end.
-
-
-
-setup_db() ->
-    DbName = ?tempdb(),
-    {ok, Db} = couch_db:create(DbName, [?ADMIN_CTX]),
-    ok = couch_db:close(Db),
-    DbName.
-
-
-teardown_db(DbName) ->
-    ok = couch_server:delete(DbName, [?ADMIN_CTX]).
-
-
-populate_db(DbName, Start, End) ->
-    {ok, Db} = couch_db:open_int(DbName, []),
-    Docs = lists:foldl(
-        fun(DocIdCounter, Acc) ->
-            Id = integer_to_binary(DocIdCounter),
-            Doc = #doc{id = Id, body = {[]}},
-            [Doc | Acc]
-        end,
-        [], lists:seq(Start, End)),
-    {ok, _} = couch_db:update_docs(Db, Docs, []),
-    ok = couch_db:close(Db).
-
-
-wait_target_in_sync(Source, Target) ->
-    {ok, SourceDb} = couch_db:open_int(Source, []),
-    {ok, SourceInfo} = couch_db:get_db_info(SourceDb),
-    ok = couch_db:close(SourceDb),
-    SourceDocCount = couch_util:get_value(doc_count, SourceInfo),
-    wait_target_in_sync_loop(SourceDocCount, Target, 300).
-
-
-wait_target_in_sync_loop(_DocCount, _TargetName, 0) ->
-    erlang:error({assertion_failed, [
-          {module, ?MODULE}, {line, ?LINE},
-          {reason, "Could not get source and target databases in sync"}
-    ]});
-
-wait_target_in_sync_loop(DocCount, TargetName, RetriesLeft) ->
-    {ok, Target} = couch_db:open_int(TargetName, []),
-    {ok, TargetInfo} = couch_db:get_db_info(Target),
-    ok = couch_db:close(Target),
-    TargetDocCount = couch_util:get_value(doc_count, TargetInfo),
-    case TargetDocCount == DocCount of
-        true ->
-            true;
-        false ->
-            ok = timer:sleep(500),
-            wait_target_in_sync_loop(DocCount, TargetName, RetriesLeft - 1)
-    end.
-
-
-replicate(Source, Target) ->
-    SrcUrl = couch_replicator_test_helper:db_url(Source),
-    TgtUrl = couch_replicator_test_helper:db_url(Target),
-    RepObject = {[
-        {<<"source">>, SrcUrl},
-        {<<"target">>, TgtUrl},
-        {<<"continuous">>, true},
-        {<<"worker_processes">>, 1},
-        {<<"retries_per_request">>, 1},
-        % Low connection timeout so _changes feed gets restarted quicker
-        {<<"connection_timeout">>, 3000}
-    ]},
-    {ok, Rep} = couch_replicator_utils:parse_rep_doc(RepObject, ?ADMIN_USER),
-    ok = couch_replicator_scheduler:add_job(Rep),
-    couch_replicator_scheduler:reschedule(),
-    {ok, Rep#rep.id}.


[couchdb] 14/16: Update and cleanup default.ini replicator entries

Posted by va...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

vatamane pushed a commit to branch prototype/fdb-layer
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit b718d3380cbc8dc5799f36e19e1a94b8c2e00c2d
Author: Nick Vatamaniuc <va...@apache.org>
AuthorDate: Fri Aug 28 04:36:05 2020 -0400

    Update and cleanup default.ini replicator entries
    
    Update settings with defaults. Also comment out values which are already set to
    default in the code.
---
 rel/overlay/etc/default.ini | 78 ++++++++++++++++++++++++++++++++++-----------
 1 file changed, 59 insertions(+), 19 deletions(-)

diff --git a/rel/overlay/etc/default.ini b/rel/overlay/etc/default.ini
index b837082..712150b 100644
--- a/rel/overlay/etc/default.ini
+++ b/rel/overlay/etc/default.ini
@@ -436,55 +436,99 @@ compression_level = 8 ; from 1 (lowest, fastest) to 9 (highest, slowest), 0 to d
 compressible_types = text/*, application/javascript, application/json, application/xml
 
 [replicator]
-; Random jitter applied on replication job startup (milliseconds)
-startup_jitter = 5000
-; Number of actively running replications
-max_jobs = 500
-;Scheduling interval in milliseconds. During each reschedule cycle
-interval = 60000
+; Number of actively running replications per replication backend node
+;max_jobs = 500
+
+; Scheduling interval in seconds
+;interval_sec = 15
+
 ; Maximum number of replications to start and stop during rescheduling.
-max_churn = 20
+;max_churn = 100
+
+; Max number of acceptors running per node. If they are available job slots
+; left then up to these many acceptors are kept open.
+;max_acceptors = 2
+
+; The amount of jitter (in milliseconds) to apply to replication job acceptors.
+; This will allow multiple acceptors to avoid generating too many transaction
+; conflicts on busy clusters.
+;accept_jitter = 2000
+
+; Minimum time in seconds replication jobs will be left running before being
+; rotated when all the schedule slots are filled. This migth be useful if
+; max_jobs is very low, but jobs should be left running long enough to make at
+; least some progress before being replaced
+;min_run_time_sec = 60
+
+; Health threshold is the minimum amount of time an unhealthy job should run
+; crashing before it is considered to be healthy again.
+;health_threshold_sec = 120
+
+; These are applied when jobs are pentalized after repeatedly crashing. On
+; first error the minimum value is applied. Then the penalty is doubled, but
+; only up to the maximum value.
+;min_backoff_penalty_sec = 32
+;max_backoff_penalty_sec = 172800
+
+; How many per-job history events to keep
+;max_history = 10
+
 ; More worker processes can give higher network throughput but can also
 ; imply more disk and network IO.
-worker_processes = 4
+;worker_processes = 4
+
 ; With lower batch sizes checkpoints are done more frequently. Lower batch sizes
 ; also reduce the total amount of used RAM memory.
-worker_batch_size = 500
+;worker_batch_size = 500
+
 ; Maximum number of HTTP connections per replication.
-http_connections = 20
+;http_connections = 20
+
 ; HTTP connection timeout per replication.
 ; Even for very fast/reliable networks it might need to be increased if a remote
 ; database is too busy.
-connection_timeout = 30000
+;connection_timeout = 30000
+
 ; Request timeout
 ;request_timeout = infinity
 ; If a request fails, the replicator will retry it up to N times.
-retries_per_request = 5
+;retries_per_request = 5
+
 ; Use checkpoints
 ;use_checkpoints = true
+
 ; Checkpoint interval
 ;checkpoint_interval = 30000
+
 ; Some socket options that might boost performance in some scenarios:
 ;       {nodelay, boolean()}
 ;       {sndbuf, integer()}
 ;       {recbuf, integer()}
 ;       {priority, integer()}
 ; See the `inet` Erlang module's man page for the full list of options.
-socket_options = [{keepalive, true}, {nodelay, false}]
+;socket_options = [{keepalive, true}, {nodelay, false}]
+
 ; Path to a file containing the user's certificate.
 ;cert_file = /full/path/to/server_cert.pem
+
 ; Path to file containing user's private PEM encoded key.
 ;key_file = /full/path/to/server_key.pem
+
 ; String containing the user's password. Only used if the private keyfile is password protected.
 ;password = somepassword
+
 ; Set to true to validate peer certificates.
-verify_ssl_certificates = false
+;verify_ssl_certificates = false
+
 ; File containing a list of peer trusted certificates (in the PEM format).
 ;ssl_trusted_certificates_file = /etc/ssl/certs/ca-certificates.crt
+
 ; Maximum peer certificate depth (must be set even if certificate validation is off).
-ssl_certificate_max_depth = 3
+;ssl_certificate_max_depth = 3
+
 ; Maximum document ID length for replication.
 ;max_document_id_length = infinity
+
 ; How much time to wait before retrying after a missing doc exception. This
 ; exception happens if the document was seen in the changes feed, but internal
 ; replication hasn't caught up yet, and fetching document's revisions
@@ -494,10 +538,6 @@ ssl_certificate_max_depth = 3
 ; avoid crashing the whole replication job, which would consume more resources
 ; and add log noise.
 ;missing_doc_retry_msec = 2000
-; Wait this many seconds after startup before attaching changes listeners
-; cluster_start_period = 5
-; Re-check cluster state at least every cluster_quiet_period seconds
-; cluster_quiet_period = 60
 
 ; List of replicator client authentication plugins to try. Plugins will be
 ; tried in order. The first to initialize successfully will be used for that


[couchdb] 01/16: Add after_db_create/2 and after_db_delete/2 callbacks to fabric

Posted by va...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

vatamane pushed a commit to branch prototype/fdb-layer
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit 36a6b390f4f5e236401224ed30dea7282be965a5
Author: Nick Vatamaniuc <va...@apache.org>
AuthorDate: Fri Aug 28 04:30:09 2020 -0400

    Add after_db_create/2 and after_db_delete/2 callbacks to fabric
    
    `after_db_create/2` and `after_db_delete/2` are when databases are created and
    deleted respectively. The callbacks are called with both the database name and
    the database instance UUID values.
---
 rel/apps/couch_epi.config            |  1 +
 src/fabric/src/fabric2_db.erl        | 15 ++++++++++++---
 src/fabric/src/fabric2_db_plugin.erl | 10 ++++++++++
 3 files changed, 23 insertions(+), 3 deletions(-)

diff --git a/rel/apps/couch_epi.config b/rel/apps/couch_epi.config
index d371163..f9f49e1 100644
--- a/rel/apps/couch_epi.config
+++ b/rel/apps/couch_epi.config
@@ -16,6 +16,7 @@
     chttpd_epi,
     couch_index_epi,
     couch_views_epi,
+    couch_replicator_epi,
     dreyfus_epi,
     global_changes_epi,
     mango_epi,
diff --git a/src/fabric/src/fabric2_db.erl b/src/fabric/src/fabric2_db.erl
index b62f26e..b3e510b 100644
--- a/src/fabric/src/fabric2_db.erl
+++ b/src/fabric/src/fabric2_db.erl
@@ -187,6 +187,7 @@ create(DbName, Options) ->
                 #{} = Db0 ->
                     Db1 = maybe_add_sys_db_callbacks(Db0),
                     ok = fabric2_server:store(Db1),
+                    fabric2_db_plugin:after_db_create(DbName, get_uuid(Db1)),
                     {ok, Db1#{tx := undefined}};
                 Error ->
                     Error
@@ -235,6 +236,7 @@ delete(DbName, Options) ->
                 fabric2_fdb:delete(TxDb)
             end),
             if Resp /= ok -> Resp; true ->
+                fabric2_db_plugin:after_db_delete(DbName, get_uuid(Db)),
                 fabric2_server:remove(DbName)
             end
     end.
@@ -243,9 +245,16 @@ delete(DbName, Options) ->
 undelete(DbName, TgtDbName, TimeStamp, Options) ->
     case validate_dbname(TgtDbName) of
         ok ->
-            fabric2_fdb:transactional(DbName, Options, fun(TxDb) ->
-                fabric2_fdb:undelete(TxDb, TgtDbName, TimeStamp)
-            end);
+            Resp = fabric2_fdb:transactional(DbName, Options,
+                fun(TxDb) ->
+                    fabric2_fdb:undelete(TxDb, TgtDbName, TimeStamp)
+                end
+            ),
+            if Resp /= ok -> ok; true ->
+                {ok, Db} = open(TgtDbName, Options),
+                fabric2_db_plugin:after_db_create(TgtDbName, get_uuid(Db))
+            end,
+            Resp;
         Error ->
             Error
     end.
diff --git a/src/fabric/src/fabric2_db_plugin.erl b/src/fabric/src/fabric2_db_plugin.erl
index 1d923dd..095b94c 100644
--- a/src/fabric/src/fabric2_db_plugin.erl
+++ b/src/fabric/src/fabric2_db_plugin.erl
@@ -14,6 +14,8 @@
 
 -export([
     validate_dbname/3,
+    after_db_create/2,
+    after_db_delete/2,
     before_doc_update/3,
     after_doc_write/6,
     after_doc_read/2,
@@ -37,6 +39,14 @@ validate_dbname(DbName, Normalized, Default) ->
     maybe_handle(validate_dbname, [DbName, Normalized], Default).
 
 
+after_db_create(DbName, DbUUID) when is_binary(DbName), is_binary(DbUUID) ->
+    with_pipe(after_db_create, [DbName, DbUUID]).
+
+
+after_db_delete(DbName, DbUUID) when is_binary(DbName), is_binary(DbUUID) ->
+    with_pipe(after_db_delete, [DbName, DbUUID]).
+
+
 before_doc_update(_, #doc{id = <<?LOCAL_DOC_PREFIX, _/binary>>} = Doc, _) ->
     Doc;
 


[couchdb] 16/16: Update and clean up tests

Posted by va...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

vatamane pushed a commit to branch prototype/fdb-layer
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit ae858196848cf9533dfa03a2006227481f47388d
Author: Nick Vatamaniuc <va...@apache.org>
AuthorDate: Fri Aug 28 04:36:18 2020 -0400

    Update and clean up tests
    
    Update tests to use the new replicator. Also clean up redundancy and re-use
    some of the newer macros from fabric2 (?TDEF_FE).
    
    Make sure replicator tests are included in `make check`
---
 Makefile                                           |   2 +-
 .../couch_replicator_attachments_too_large.erl     |  90 ++---
 .../eunit/couch_replicator_connection_tests.erl    | 274 +++++++------
 ...replicator_create_target_with_options_tests.erl | 129 +++---
 .../test/eunit/couch_replicator_db_tests.erl       | 332 ++++++++++++++++
 .../test/eunit/couch_replicator_filtered_tests.erl | 348 ++++++++--------
 .../eunit/couch_replicator_httpc_pool_tests.erl    | 125 +++---
 .../eunit/couch_replicator_id_too_long_tests.erl   |  91 ++---
 .../eunit/couch_replicator_job_server_tests.erl    | 437 +++++++++++++++++++++
 .../eunit/couch_replicator_large_atts_tests.erl    | 123 +++---
 .../eunit/couch_replicator_many_leaves_tests.erl   | 241 +++++-------
 .../eunit/couch_replicator_missing_stubs_tests.erl | 179 ++++-----
 .../test/eunit/couch_replicator_proxy_tests.erl    | 135 +++----
 .../eunit/couch_replicator_rate_limiter_tests.erl  |  77 ++--
 ...ch_replicator_retain_stats_between_job_runs.erl | 223 +++--------
 .../test/eunit/couch_replicator_selector_tests.erl | 136 +++----
 ...ch_replicator_small_max_request_size_target.erl | 190 +++------
 .../test/eunit/couch_replicator_test_helper.erl    | 323 +++++++++++----
 .../couch_replicator_transient_jobs_tests.erl      | 106 +++++
 .../couch_replicator_use_checkpoints_tests.erl     | 207 +++-------
 test/elixir/test/replication_test.exs              |   5 +-
 21 files changed, 2210 insertions(+), 1563 deletions(-)

diff --git a/Makefile b/Makefile
index e8d36629..35b62f9 100644
--- a/Makefile
+++ b/Makefile
@@ -163,7 +163,7 @@ endif
 .PHONY: check
 check:  all
 	@$(MAKE) emilio
-	make eunit apps=couch_eval,couch_expiring_cache,ctrace,couch_jobs,couch_views,fabric,mango,chttpd
+	make eunit apps=couch_eval,couch_expiring_cache,ctrace,couch_jobs,couch_views,fabric,mango,chttpd,couch_replicator
 	make elixir tests=test/elixir/test/basics_test.exs,test/elixir/test/replication_test.exs,test/elixir/test/map_test.exs,test/elixir/test/all_docs_test.exs,test/elixir/test/bulk_docs_test.exs
 	make exunit apps=chttpd
 	make mango-test
diff --git a/src/couch_replicator/test/eunit/couch_replicator_attachments_too_large.erl b/src/couch_replicator/test/eunit/couch_replicator_attachments_too_large.erl
index ac4bb84..0e7e0ea 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_attachments_too_large.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_attachments_too_large.erl
@@ -12,72 +12,60 @@
 
 -module(couch_replicator_attachments_too_large).
 
+
 -include_lib("couch/include/couch_eunit.hrl").
 -include_lib("couch/include/couch_db.hrl").
 -include_lib("couch_replicator/src/couch_replicator.hrl").
-
-
-setup(_) ->
-    Ctx = test_util:start_couch([couch_replicator]),
-    Source = create_db(),
-    create_doc_with_attachment(Source, <<"doc">>, 1000),
-    Target = create_db(),
-    {Ctx, {Source, Target}}.
-
-
-teardown(_, {Ctx, {Source, Target}}) ->
-    delete_db(Source),
-    delete_db(Target),
-    config:delete("couchdb", "max_attachment_size"),
-    ok = test_util:stop_couch(Ctx).
+-include_lib("fabric/test/fabric2_test.hrl").
 
 
 attachment_too_large_replication_test_() ->
-    Pairs = [{remote, remote}],
     {
-        "Attachment size too large replication tests",
+        setup,
+        fun couch_replicator_test_helper:start_couch/0,
+        fun couch_replicator_test_helper:stop_couch/1,
         {
-            foreachx,
-            fun setup/1, fun teardown/2,
-            [{Pair, fun should_succeed/2} || Pair <- Pairs] ++
-            [{Pair, fun should_fail/2} || Pair <- Pairs]
+            foreach,
+            fun setup/0,
+            fun teardown/1,
+            [
+                ?TDEF_FE(t_should_succeed),
+                ?TDEF_FE(t_should_fail)
+            ]
         }
     }.
 
 
-should_succeed({From, To}, {_Ctx, {Source, Target}}) ->
-    RepObject = {[
-        {<<"source">>, db_url(From, Source)},
-        {<<"target">>, db_url(To, Target)}
-    ]},
-    config:set("couchdb", "max_attachment_size", "1000", _Persist = false),
-    {ok, _} = couch_replicator:replicate(RepObject, ?ADMIN_USER),
-    ?_assertEqual(ok, couch_replicator_test_helper:compare_dbs(Source, Target)).
+setup() ->
+    Source = couch_replicator_test_helper:create_db(),
+    create_doc_with_attachment(Source, <<"doc">>, 1000),
+    Target = couch_replicator_test_helper:create_db(),
+    {Source, Target}.
 
 
-should_fail({From, To}, {_Ctx, {Source, Target}}) ->
-    RepObject = {[
-        {<<"source">>, db_url(From, Source)},
-        {<<"target">>, db_url(To, Target)}
-    ]},
-    config:set("couchdb", "max_attachment_size", "999", _Persist = false),
-    {ok, _} = couch_replicator:replicate(RepObject, ?ADMIN_USER),
-    ?_assertError({badmatch, {not_found, missing}},
-        couch_replicator_test_helper:compare_dbs(Source, Target)).
+teardown({Source, Target}) ->
+    config:delete("couchdb", "max_attachment_size", false),
+    couch_replicator_test_helper:delete_db(Source),
+    couch_replicator_test_helper:delete_db(Target).
 
 
-create_db() ->
-    DbName = ?tempdb(),
-    {ok, Db} = couch_db:create(DbName, [?ADMIN_CTX]),
-    ok = couch_db:close(Db),
-    DbName.
+t_should_succeed({Source, Target}) ->
+    config:set("couchdb", "max_attachment_size", "1000", false),
+    {ok, _} = couch_replicator_test_helper:replicate(Source, Target),
+    ?assertEqual(ok, couch_replicator_test_helper:compare_dbs(Source, Target)).
+
+
+t_should_fail({Source, Target}) ->
+    config:set("couchdb", "max_attachment_size", "999", false),
+    {ok, _} = couch_replicator_test_helper:replicate(Source, Target),
+    ExceptIds = [<<"doc">>],
+    ?assertEqual(ok, couch_replicator_test_helper:compare_dbs(Source,
+        Target, ExceptIds)).
 
 
 create_doc_with_attachment(DbName, DocId, AttSize) ->
-    {ok, Db} = couch_db:open(DbName, [?ADMIN_CTX]),
     Doc = #doc{id = DocId, atts = att(AttSize)},
-    {ok, _} = couch_db:update_doc(Db, Doc, []),
-    couch_db:close(Db),
+    couch_replicator_test_helper:create_docs(DbName, [Doc]),
     ok.
 
 
@@ -90,13 +78,3 @@ att(Size) when is_integer(Size), Size >= 1 ->
             << <<"x">> || _ <- lists:seq(1, Size) >>
         end}
     ])].
-
-
-delete_db(DbName) ->
-    ok = couch_server:delete(DbName, [?ADMIN_CTX]).
-
-
-db_url(remote, DbName) ->
-    Addr = config:get("httpd", "bind_address", "127.0.0.1"),
-    Port = mochiweb_socket_server:get(couch_httpd, port),
-    ?l2b(io_lib:format("http://~s:~b/~s", [Addr, Port, DbName])).
diff --git a/src/couch_replicator/test/eunit/couch_replicator_connection_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_connection_tests.erl
index e75cc5a..df30db2 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_connection_tests.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_connection_tests.erl
@@ -12,187 +12,176 @@
 
 -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.
+-include_lib("fabric/test/fabric2_test.hrl").
 
 
 httpc_pool_test_() ->
     {
-        "replicator connection sharing tests",
+        "Replicator connection sharing tests",
         {
             setup,
-            fun() -> test_util:start_couch([couch_replicator]) end, fun test_util:stop_couch/1,
+            fun couch_replicator_test_helper:start_couch/0,
+            fun couch_replicator_test_helper:stop_couch/1,
             {
                 foreach,
-                fun setup/0, fun teardown/1,
+                fun setup/0,
+                fun teardown/1,
                 [
-                    fun connections_shared_after_release/1,
-                    fun connections_not_shared_after_owner_death/1,
-                    fun idle_connections_closed/1,
-                    fun test_owner_monitors/1,
-                    fun worker_discards_creds_on_create/1,
-                    fun worker_discards_url_creds_after_request/1,
-                    fun worker_discards_creds_in_headers_after_request/1,
-                    fun worker_discards_proxy_creds_after_request/1
+                    ?TDEF_FE(connections_shared_after_release),
+                    ?TDEF_FE(connections_not_shared_after_owner_death),
+                    ?TDEF_FE(idle_connections_closed),
+                    ?TDEF_FE(test_owner_monitors),
+                    ?TDEF_FE(worker_discards_creds_on_create),
+                    ?TDEF_FE(worker_discards_url_creds_after_request),
+                    ?TDEF_FE(worker_discards_creds_in_headers_after_request),
+                    ?TDEF_FE(worker_discards_proxy_creds_after_request)
                 ]
             }
         }
     }.
 
 
+setup() ->
+    Host = config:get("chttpd", "bind_address", "127.0.0.1"),
+    Port = config:get("chttpd", "port", "5984"),
+    {Host, Port}.
+
+
+teardown(_) ->
+    ok.
+
+
 connections_shared_after_release({Host, Port}) ->
-    ?_test(begin
-        URL = "http://" ++ Host ++ ":" ++ Port,
-        Self = self(),
-        {ok, Pid} = couch_replicator_connection:acquire(URL),
-        couch_replicator_connection:release(Pid),
-        spawn(fun() ->
-            Self ! couch_replicator_connection:acquire(URL)
-        end),
-        receive
-            {ok, Pid2} ->
-                ?assertEqual(Pid, Pid2)
-        end
-    end).
+    URL = "http://" ++ Host ++ ":" ++ Port,
+    Self = self(),
+    {ok, Pid} = couch_replicator_connection:acquire(URL),
+    couch_replicator_connection:release(Pid),
+    spawn(fun() ->
+        Self ! couch_replicator_connection:acquire(URL)
+    end),
+    receive
+        {ok, Pid2} ->
+            ?assertEqual(Pid, Pid2)
+    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),
-            error("simulate division by zero without compiler warning")
-        end),
-        receive
-            {ok, Pid} ->
-                {ok, Pid2} = couch_replicator_connection:acquire(URL),
-                ?assertNotEqual(Pid, Pid2),
-                MRef = monitor(process, Pid),
-                receive {'DOWN', MRef, process, Pid, _Reason} ->
+    URL = "http://" ++ Host ++ ":" ++ Port,
+    Self = self(),
+    spawn(fun() ->
+        Self ! couch_replicator_connection:acquire(URL),
+        error("simulate division by zero without compiler warning")
+    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).
+                Other ->
+                    throw(Other)
+            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:release(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).
+    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:release(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)).
 
 
 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:release(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:release(Worker2),
-            assert_monitors_equal(NewAcc),
-            NewAcc
-        end, Monitors, Workers)
-    end).
+    URL = "http://" ++ Host ++ ":" ++ Port,
+    {ok, Worker0} = couch_replicator_connection:acquire(URL),
+    assert_monitors_equal([{process, self()}]),
+    couch_replicator_connection:release(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:release(Worker2),
+        assert_monitors_equal(NewAcc),
+        NewAcc
+    end, Monitors, Workers).
 
 
 worker_discards_creds_on_create({Host, Port}) ->
-    ?_test(begin
-        {User, Pass, B64Auth} = user_pass(),
-        URL = "http://" ++ User ++ ":" ++ Pass ++ "@" ++ Host ++ ":" ++ Port,
-        {ok, WPid} = couch_replicator_connection:acquire(URL),
-        Internals = worker_internals(WPid),
-        ?assert(string:str(Internals, B64Auth) =:= 0),
-        ?assert(string:str(Internals, Pass) =:= 0)
-    end).
+    {User, Pass, B64Auth} = user_pass(),
+    URL = "http://" ++ User ++ ":" ++ Pass ++ "@" ++ Host ++ ":" ++ Port,
+    {ok, WPid} = couch_replicator_connection:acquire(URL),
+    Internals = worker_internals(WPid),
+    ?assert(string:str(Internals, B64Auth) =:= 0),
+    ?assert(string:str(Internals, Pass) =:= 0).
 
 
 worker_discards_url_creds_after_request({Host, _}) ->
-    ?_test(begin
-       {User, Pass, B64Auth} = user_pass(),
-       {Port, ServerPid} = server(),
-       PortStr = integer_to_list(Port),
-       URL = "http://" ++ User ++ ":" ++ Pass ++ "@" ++ Host ++ ":" ++ PortStr,
-       {ok, WPid} = couch_replicator_connection:acquire(URL),
-       ?assertMatch({ok, "200", _, _}, send_req(WPid, URL, [], [])),
-       Internals = worker_internals(WPid),
-       ?assert(string:str(Internals, B64Auth) =:= 0),
-       ?assert(string:str(Internals, Pass) =:= 0),
-       couch_replicator_connection:release(WPid),
-       unlink(ServerPid),
-       exit(ServerPid, kill)
-    end).
+    {User, Pass, B64Auth} = user_pass(),
+    {Port, ServerPid} = server(),
+    PortStr = integer_to_list(Port),
+    URL = "http://" ++ User ++ ":" ++ Pass ++ "@" ++ Host ++ ":" ++ PortStr,
+    {ok, WPid} = couch_replicator_connection:acquire(URL),
+    ?assertMatch({ok, "200", _, _}, send_req(WPid, URL, [], [])),
+    Internals = worker_internals(WPid),
+    ?assert(string:str(Internals, B64Auth) =:= 0),
+    ?assert(string:str(Internals, Pass) =:= 0),
+    couch_replicator_connection:release(WPid),
+    unlink(ServerPid),
+    exit(ServerPid, kill).
 
 
 worker_discards_creds_in_headers_after_request({Host, _}) ->
-    ?_test(begin
-       {_User, Pass, B64Auth} = user_pass(),
-       {Port, ServerPid} = server(),
-       PortStr = integer_to_list(Port),
-       URL = "http://" ++ Host ++ ":" ++ PortStr,
-       {ok, WPid} = couch_replicator_connection:acquire(URL),
-       Headers = [{"Authorization", "Basic " ++ B64Auth}],
-       ?assertMatch({ok, "200", _, _}, send_req(WPid, URL, Headers, [])),
-       Internals = worker_internals(WPid),
-       ?assert(string:str(Internals, B64Auth) =:= 0),
-       ?assert(string:str(Internals, Pass) =:= 0),
-       couch_replicator_connection:release(WPid),
-       unlink(ServerPid),
-       exit(ServerPid, kill)
-    end).
+    {_User, Pass, B64Auth} = user_pass(),
+    {Port, ServerPid} = server(),
+    PortStr = integer_to_list(Port),
+    URL = "http://" ++ Host ++ ":" ++ PortStr,
+    {ok, WPid} = couch_replicator_connection:acquire(URL),
+    Headers = [{"Authorization", "Basic " ++ B64Auth}],
+    ?assertMatch({ok, "200", _, _}, send_req(WPid, URL, Headers, [])),
+    Internals = worker_internals(WPid),
+    ?assert(string:str(Internals, B64Auth) =:= 0),
+    ?assert(string:str(Internals, Pass) =:= 0),
+    couch_replicator_connection:release(WPid),
+    unlink(ServerPid),
+    exit(ServerPid, kill).
 
 
 worker_discards_proxy_creds_after_request({Host, _}) ->
-    ?_test(begin
-       {User, Pass, B64Auth} = user_pass(),
-       {Port, ServerPid} = server(),
-       PortStr = integer_to_list(Port),
-       URL = "http://" ++ Host ++ ":" ++ PortStr,
-       {ok, WPid} = couch_replicator_connection:acquire(URL),
-       Opts = [
-           {proxy_host, Host},
-           {proxy_port, Port},
-           {proxy_user, User},
-           {proxy_pass, Pass}
-       ],
-       ?assertMatch({ok, "200", _, _}, send_req(WPid, URL, [], Opts)),
-       Internals = worker_internals(WPid),
-       ?assert(string:str(Internals, B64Auth) =:= 0),
-       ?assert(string:str(Internals, Pass) =:= 0),
-       couch_replicator_connection:release(WPid),
-       unlink(ServerPid),
-       exit(ServerPid, kill)
-    end).
+    {User, Pass, B64Auth} = user_pass(),
+    {Port, ServerPid} = server(),
+    PortStr = integer_to_list(Port),
+    URL = "http://" ++ Host ++ ":" ++ PortStr,
+    {ok, WPid} = couch_replicator_connection:acquire(URL),
+    Opts = [
+        {proxy_host, Host},
+        {proxy_port, Port},
+        {proxy_user, User},
+        {proxy_pass, Pass}
+    ],
+    ?assertMatch({ok, "200", _, _}, send_req(WPid, URL, [], Opts)),
+    Internals = worker_internals(WPid),
+    ?assert(string:str(Internals, B64Auth) =:= 0),
+    ?assert(string:str(Internals, Pass) =:= 0),
+    couch_replicator_connection:release(WPid),
+    unlink(ServerPid),
+    exit(ServerPid, kill).
 
 
 send_req(WPid, URL, Headers, Opts) ->
@@ -237,5 +226,6 @@ server_responder(LSock) ->
 
 assert_monitors_equal(ShouldBe) ->
     sys:get_status(couch_replicator_connection),
-    {monitors, Monitors} = process_info(whereis(couch_replicator_connection), monitors),
+    {monitors, Monitors} = process_info(whereis(couch_replicator_connection),
+        monitors),
     ?assertEqual(Monitors, ShouldBe).
diff --git a/src/couch_replicator/test/eunit/couch_replicator_create_target_with_options_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_create_target_with_options_tests.erl
index 63310d3..c957fc1 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_create_target_with_options_tests.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_create_target_with_options_tests.erl
@@ -12,132 +12,137 @@
 
 -module(couch_replicator_create_target_with_options_tests).
 
+
 -include_lib("couch/include/couch_eunit.hrl").
 -include_lib("couch/include/couch_db.hrl").
 -include_lib("couch_replicator/src/couch_replicator.hrl").
-
--define(USERNAME, "rep_admin").
--define(PASSWORD, "secret").
-
-setup() ->
-    Ctx = test_util:start_couch([fabric, mem3, couch_replicator, chttpd]),
-    Hashed = couch_passwords:hash_admin_password(?PASSWORD),
-    ok = config:set("admins", ?USERNAME, ?b2l(Hashed), _Persist=false),
-    Source = ?tempdb(),
-    Target = ?tempdb(),
-    {Ctx, {Source, Target}}.
-
-
-teardown({Ctx, {_Source, _Target}}) ->
-    config:delete("admins", ?USERNAME),
-    ok = test_util:stop_couch(Ctx).
+-include_lib("fabric/test/fabric2_test.hrl").
 
 
 create_target_with_options_replication_test_() ->
     {
         "Create target with range partitions tests",
         {
-            foreach,
-            fun setup/0, fun teardown/1,
-            [
-                fun should_create_target_with_q_4/1,
-                fun should_create_target_with_q_2_n_1/1,
-                fun should_create_target_with_default/1,
-                fun should_not_create_target_with_q_any/1
-            ]
+            setup,
+            fun couch_replicator_test_helper:start_couch/0,
+            fun couch_replicator_test_helper:stop_couch/1,
+            {
+                foreach,
+                fun setup/0,
+                fun teardown/1,
+                [
+                    ?TDEF_FE(should_create_target_with_q_4),
+                    ?TDEF_FE(should_create_target_with_q_2_n_1),
+                    ?TDEF_FE(should_create_target_with_default),
+                    ?TDEF_FE(should_not_create_target_with_q_any)
+                ]
+            }
         }
     }.
 
 
-should_create_target_with_q_4({_Ctx, {Source, Target}}) ->
+setup() ->
+    Source = ?tempdb(),
+    Target = ?tempdb(),
+    {Source, Target}.
+
+
+teardown({Source, Target}) ->
+    delete_db(Source),
+    delete_db(Target).
+
+
+should_create_target_with_q_4({Source, Target}) ->
     RepObject = {[
-        {<<"source">>, db_url(Source)},
-        {<<"target">>, db_url(Target)},
+        {<<"source">>, Source},
+        {<<"target">>, Target},
         {<<"create_target">>, true},
         {<<"create_target_params">>, {[{<<"q">>, <<"4">>}]}}
     ]},
     create_db(Source),
     create_doc(Source),
-    {ok, _} = couch_replicator:replicate(RepObject, ?ADMIN_USER),
+    {ok, _} = couch_replicator_test_helper:replicate(RepObject),
 
-    {ok, TargetInfo} = fabric:get_db_info(Target),
+    TargetInfo = db_info(Target),
     {ClusterInfo} = couch_util:get_value(cluster, TargetInfo),
     delete_db(Source),
     delete_db(Target),
-    ?_assertEqual(4, couch_util:get_value(q, ClusterInfo)).
+    ?assertEqual(0, couch_util:get_value(q, ClusterInfo)).
 
 
-should_create_target_with_q_2_n_1({_Ctx, {Source, Target}}) ->
+should_create_target_with_q_2_n_1({Source, Target}) ->
     RepObject = {[
-        {<<"source">>, db_url(Source)},
-        {<<"target">>, db_url(Target)},
+        {<<"source">>, Source},
+        {<<"target">>, Target},
         {<<"create_target">>, true},
         {<<"create_target_params">>,
             {[{<<"q">>, <<"2">>}, {<<"n">>, <<"1">>}]}}
     ]},
     create_db(Source),
     create_doc(Source),
-    {ok, _} = couch_replicator:replicate(RepObject, ?ADMIN_USER),
+    {ok, _} = couch_replicator_test_helper:replicate(RepObject),
 
-    {ok, TargetInfo} = fabric:get_db_info(Target),
+    TargetInfo = db_info(Target),
     {ClusterInfo} = couch_util:get_value(cluster, TargetInfo),
     delete_db(Source),
     delete_db(Target),
-    [
-        ?_assertEqual(2, couch_util:get_value(q, ClusterInfo)),
-        ?_assertEqual(1, couch_util:get_value(n, ClusterInfo))
-    ].
+    ?assertEqual(0, couch_util:get_value(q, ClusterInfo)),
+    ?assertEqual(0, couch_util:get_value(n, ClusterInfo)).
 
 
-should_create_target_with_default({_Ctx, {Source, Target}}) ->
+should_create_target_with_default({Source, Target}) ->
     RepObject = {[
-        {<<"source">>, db_url(Source)},
-        {<<"target">>, db_url(Target)},
+        {<<"source">>, Source},
+        {<<"target">>, Target},
         {<<"create_target">>, true}
     ]},
     create_db(Source),
     create_doc(Source),
-    {ok, _} = couch_replicator:replicate(RepObject, ?ADMIN_USER),
+    {ok, _} = couch_replicator_test_helper:replicate(RepObject),
 
-    {ok, TargetInfo} = fabric:get_db_info(Target),
+    TargetInfo = db_info(Target),
     {ClusterInfo} = couch_util:get_value(cluster, TargetInfo),
-    Q = config:get("cluster", "q", "8"),
     delete_db(Source),
     delete_db(Target),
-    ?_assertEqual(list_to_integer(Q), couch_util:get_value(q, ClusterInfo)).
+    ?assertEqual(0, couch_util:get_value(q, ClusterInfo)).
 
 
-should_not_create_target_with_q_any({_Ctx, {Source, Target}}) ->
+should_not_create_target_with_q_any({Source, Target}) ->
     RepObject = {[
-        {<<"source">>, db_url(Source)},
-        {<<"target">>, db_url(Target)},
+        {<<"source">>, Source},
+        {<<"target">>, Target},
         {<<"create_target">>, false},
         {<<"create_target_params">>, {[{<<"q">>, <<"1">>}]}}
     ]},
     create_db(Source),
     create_doc(Source),
-    {error, _} = couch_replicator:replicate(RepObject, ?ADMIN_USER),
-    DbExist = is_list(catch mem3:shards(Target)),
+    {error, _} = couch_replicator_test_helper:replicate(RepObject),
+    Exists = try
+        fabric2_db:open(Target, [?ADMIN_CTX]),
+        ?assert(false)
+    catch
+        error:database_does_not_exist ->
+            database_does_not_exist
+    end,
     delete_db(Source),
-    ?_assertEqual(false, DbExist).
+    ?assertEqual(Exists, database_does_not_exist).
 
 
 create_doc(DbName) ->
-    Body = {[{<<"foo">>, <<"bar">>}]},
-    NewDoc = #doc{body = Body},
-    {ok, _} = fabric:update_doc(DbName, NewDoc, [?ADMIN_CTX]).
+    couch_replicator_test_helper:create_docs(DbName, [
+        #{<<"_id">> => fabric2_util:uuid(), <<"foo">> => <<"bar">>}
+    ]).
 
 
 create_db(DbName) ->
-    ok = fabric:create_db(DbName, [?ADMIN_CTX]).
+    couch_replicator_test_helper:create_db(DbName).
 
 
 delete_db(DbName) ->
-    ok = fabric:delete_db(DbName, [?ADMIN_CTX]).
+    couch_replicator_test_helper:delete_db(DbName).
 
 
-db_url(DbName) ->
-    Addr = config:get("chttpd", "bind_address", "127.0.0.1"),
-    Port = mochiweb_socket_server:get(chttpd, port),
-    ?l2b(io_lib:format("http://~s:~s@~s:~b/~s", [?USERNAME, ?PASSWORD, Addr,
-        Port, DbName])).
+db_info(DbName) ->
+    {ok, Db} = fabric2_db:open(DbName, [?ADMIN_CTX]),
+    {ok, Info} = fabric2_db:get_db_info(Db),
+    Info.
diff --git a/src/couch_replicator/test/eunit/couch_replicator_db_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_db_tests.erl
new file mode 100644
index 0000000..0534410
--- /dev/null
+++ b/src/couch_replicator/test/eunit/couch_replicator_db_tests.erl
@@ -0,0 +1,332 @@
+% 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_tests).
+
+-include_lib("couch/include/couch_eunit.hrl").
+-include_lib("couch/include/couch_db.hrl").
+-include_lib("couch_replicator/src/couch_replicator.hrl").
+-include_lib("fabric/test/fabric2_test.hrl").
+
+
+couch_replicator_db_test_() ->
+    {
+        "Replications are started from docs in _replicator dbs",
+        {
+            setup,
+            fun couch_replicator_test_helper:start_couch/0,
+            fun couch_replicator_test_helper:stop_couch/1,
+            {
+                foreach,
+                fun setup/0,
+                fun teardown/1,
+                [
+                    ?TDEF_FE(default_replicator_db_is_created),
+                    ?TDEF_FE(continuous_replication_created_from_doc, 15),
+                    ?TDEF_FE(normal_replication_created_from_doc, 15),
+                    ?TDEF_FE(replicator_db_deleted, 15),
+                    ?TDEF_FE(replicator_db_recreated, 15),
+                    ?TDEF_FE(invalid_replication_docs),
+                    ?TDEF_FE(duplicate_persistent_replication, 15),
+                    ?TDEF_FE(duplicate_transient_replication, 30)
+                ]
+            }
+        }
+    }.
+
+
+setup() ->
+    Source = couch_replicator_test_helper:create_db(),
+    create_doc(Source, #{<<"_id">> => <<"doc1">>}),
+    Target = couch_replicator_test_helper:create_db(),
+    Name = ?tempdb(),
+    RepDb = couch_replicator_test_helper:create_db(<<Name/binary,
+        "/_replicator">>),
+    config:set("replicator", "stats_update_interval_sec", "0", false),
+    config:set("replicator", "create_replicator_db", "false", false),
+    config:set("couchdb", "enable_database_recovery", "false", false),
+    config:set("replicator", "min_backoff_penalty_sec", "1", false),
+    {Source, Target, RepDb}.
+
+
+teardown({Source, Target, RepDb}) ->
+    config:delete("replicator", "stats_update_interval_sec", false),
+    config:delete("replicator", "create_replicator_db", false),
+    config:delete("couchdb", "enable_database_recovery", false),
+    config:delete("replicator", "min_backoff_penalty_sec", false),
+
+    couch_replicator_test_helper:delete_db(RepDb),
+    couch_replicator_test_helper:delete_db(?REP_DB_NAME),
+    couch_replicator_test_helper:delete_db(Source),
+    couch_replicator_test_helper:delete_db(Target).
+
+
+default_replicator_db_is_created({_, _, _}) ->
+    config:set("replicator", "create_replicator_db", "true", false),
+    ?assertEqual(ignore, couch_replicator:ensure_rep_db_exists()),
+    ?assertMatch({ok, #{}}, fabric2_db:open(?REP_DB_NAME, [])).
+
+
+continuous_replication_created_from_doc({Source, Target, RepDb}) ->
+    DocId = <<"rdoc1">>,
+    RDoc = rep_doc(Source, Target, DocId, #{<<"continuous">> => true}),
+    create_doc(RepDb, RDoc),
+    wait_scheduler_docs_state(RepDb, DocId, <<"running">>),
+
+    {Code, DocInfo} = scheduler_docs(RepDb, DocId),
+    ?assertEqual(200, Code),
+    ?assertMatch(#{
+        <<"database">> := RepDb,
+        <<"doc_id">> := DocId
+    }, DocInfo),
+
+    RepId = maps:get(<<"id">>, DocInfo),
+
+    ?assertMatch([#{
+        <<"database">> := RepDb,
+        <<"doc_id">> := DocId,
+        <<"id">> := RepId,
+        <<"state">> := <<"running">>
+    }], couch_replicator_test_helper:scheduler_jobs()),
+
+    ?assertMatch({200, #{
+        <<"database">> := RepDb,
+        <<"doc_id">> := DocId,
+        <<"id">> := RepId,
+        <<"state">> := <<"running">>
+    }}, scheduler_jobs(RepId)),
+
+    delete_doc(RepDb, DocId),
+    wait_scheduler_docs_not_found(RepDb, DocId),
+    ?assertMatch({404, #{}}, scheduler_jobs(RepId)).
+
+
+normal_replication_created_from_doc({Source, Target, RepDb}) ->
+    DocId = <<"rdoc2">>,
+    RDoc = rep_doc(Source, Target, DocId),
+    create_doc(RepDb, RDoc),
+    wait_scheduler_docs_state(RepDb, DocId, <<"completed">>),
+
+    {Code, DocInfo} = scheduler_docs(RepDb, DocId),
+    ?assertEqual(200, Code),
+    ?assertMatch(#{
+        <<"database">> := RepDb,
+        <<"doc_id">> := DocId,
+        <<"state">> := <<"completed">>,
+        <<"info">> := #{
+            <<"docs_written">> := 1,
+            <<"docs_read">> := 1,
+            <<"missing_revisions_found">> := 1
+        }
+    }, DocInfo),
+
+    wait_doc_state(RepDb, DocId, <<"completed">>),
+    ?assertMatch(#{
+        <<"_replication_state">> := <<"completed">>,
+        <<"_replication_stats">> := #{
+            <<"docs_written">> := 1,
+            <<"docs_read">> := 1,
+            <<"missing_revisions_found">> := 1
+        }
+    }, read_doc(RepDb, DocId)),
+
+    delete_doc(RepDb, DocId),
+    wait_scheduler_docs_not_found(RepDb, DocId).
+
+
+replicator_db_deleted({Source, Target, RepDb}) ->
+    DocId = <<"rdoc3">>,
+    RDoc = rep_doc(Source, Target, DocId, #{<<"continuous">> => true}),
+    create_doc(RepDb, RDoc),
+    wait_scheduler_docs_state(RepDb, DocId, <<"running">>),
+    fabric2_db:delete(RepDb, [?ADMIN_CTX]),
+    wait_scheduler_docs_not_found(RepDb, DocId).
+
+
+replicator_db_recreated({Source, Target, RepDb}) ->
+    DocId = <<"rdoc4">>,
+    RDoc = rep_doc(Source, Target, DocId, #{<<"continuous">> => true}),
+    create_doc(RepDb, RDoc),
+    wait_scheduler_docs_state(RepDb, DocId, <<"running">>),
+
+    config:set("couchdb", "enable_database_recovery", "true", false),
+    fabric2_db:delete(RepDb, [?ADMIN_CTX]),
+    wait_scheduler_docs_not_found(RepDb, DocId),
+
+    Opts = [{start_key, RepDb}, {end_key, RepDb}],
+    {ok, [DbInfo]} = fabric2_db:list_deleted_dbs_info(Opts),
+    {_, Timestamp} = lists:keyfind(timestamp, 1, DbInfo),
+    ok = fabric2_db:undelete(RepDb, RepDb, Timestamp, [?ADMIN_CTX]),
+    wait_scheduler_docs_state(RepDb, DocId, <<"running">>),
+
+    config:set("couchdb", "enable_database_recovery", "false", false),
+    fabric2_db:delete(RepDb, [?ADMIN_CTX]),
+    wait_scheduler_docs_not_found(RepDb, DocId).
+
+
+invalid_replication_docs({_, _, RepDb}) ->
+    Docs = [
+        #{
+            <<"_id">> => <<"1">>,
+            <<"source">> => <<"http://127.0.0.1:1000">>
+        },
+        #{
+            <<"_id">> => <<"1">>,
+            <<"target">> => <<"http://127.0.0.1:1001">>
+        },
+        #{
+            <<"_id">> => <<"1">>
+        },
+        #{
+            <<"_id">> => <<"1">>,
+            <<"source">> => <<"http://127.0.0.1:1002">>,
+            <<"target">> => <<"http://127.0.0.1:1003">>,
+            <<"create_target">> => <<"bad">>
+        },
+        #{
+            <<"_id">> => <<"1">>,
+            <<"source">> => #{<<"junk">> => 42},
+            <<"target">> => <<"http://127.0.0.1:1004">>
+        },
+        #{
+            <<"_id">> => <<"1">>,
+            <<"source">> => <<"http://127.0.0.1:1005">>,
+            <<"target">> => <<"http://127.0.0.1:1006">>,
+            <<"selector">> => #{},
+            <<"filter">> => <<"a/b">>
+        },
+        #{
+            <<"_id">> => <<"1">>,
+            <<"source">> => <<"http://127.0.0.1:1007">>,
+            <<"target">> => <<"https://127.0.0.1:1008">>,
+            <<"doc_ids">> => 42
+        }
+    ],
+    lists:foreach(fun(Doc) ->
+        ?assertThrow({forbidden, _}, create_doc(RepDb, Doc))
+    end, Docs).
+
+
+duplicate_persistent_replication({Source, Target, RepDb}) ->
+    DocId1 = <<"rdoc5">>,
+    RDoc1 = rep_doc(Source, Target, DocId1, #{<<"continuous">> => true}),
+    create_doc(RepDb, RDoc1),
+    wait_scheduler_docs_state(RepDb, DocId1, <<"running">>),
+
+    DocId2 = <<"rdoc6">>,
+    RDoc2 = rep_doc(Source, Target, DocId2, #{<<"continuous">> => true}),
+    create_doc(RepDb, RDoc2),
+    wait_scheduler_docs_state(RepDb, DocId2, <<"failed">>),
+
+    delete_doc(RepDb, DocId1),
+    delete_doc(RepDb, DocId2),
+
+    wait_scheduler_docs_not_found(RepDb, DocId1),
+    wait_scheduler_docs_not_found(RepDb, DocId2).
+
+
+duplicate_transient_replication({Source, Target, RepDb}) ->
+    {ok, _Pid, RepId} = couch_replicator_test_helper:replicate_continuous(
+        Source, Target),
+
+    DocId = <<"rdoc7">>,
+    RDoc = rep_doc(Source, Target, DocId, #{<<"continuous">> => true}),
+    create_doc(RepDb, RDoc),
+    wait_scheduler_docs_state(RepDb, DocId, <<"crashing">>),
+
+    couch_replicator_test_helper:cancel(RepId),
+    wait_reschedule_docs_state(RepDb, DocId, <<"running">>),
+
+    delete_doc(RepDb, DocId),
+    wait_scheduler_docs_not_found(RepDb, DocId).
+
+
+scheduler_jobs(Id) ->
+    SUrl = couch_replicator_test_helper:server_url(),
+    Url = lists:flatten(io_lib:format("~s/_scheduler/jobs/~s", [SUrl, Id])),
+    {ok, Code, _, Body} = test_request:get(Url, []),
+    {Code, jiffy:decode(Body, [return_maps])}.
+
+
+scheduler_docs(DbName, DocId) ->
+    SUrl = couch_replicator_test_helper:server_url(),
+    Fmt = "~s/_scheduler/docs/~s/~s",
+    Url = lists:flatten(io_lib:format(Fmt, [SUrl, DbName, DocId])),
+    {ok, Code, _, Body} = test_request:get(Url, []),
+    {Code, jiffy:decode(Body, [return_maps])}.
+
+
+rep_doc(Source, Target, DocId) ->
+    rep_doc(Source, Target, DocId, #{}).
+
+
+rep_doc(Source, Target, DocId, #{} = Extra) ->
+    maps:merge(#{
+        <<"_id">> => DocId,
+        <<"source">> => couch_replicator_test_helper:db_url(Source),
+        <<"target">> => couch_replicator_test_helper:db_url(Target)
+    }, Extra).
+
+
+create_doc(DbName, Doc) ->
+    couch_replicator_test_helper:create_docs(DbName, [Doc]).
+
+
+delete_doc(DbName, DocId) ->
+    {ok, Db} = fabric2_db:open(DbName, [?ADMIN_CTX]),
+    {ok, Doc} = fabric2_db:open_doc(Db, DocId),
+    Doc1 = Doc#doc{deleted = true},
+    {ok, _} = fabric2_db:update_doc(Db, Doc1, []).
+
+
+read_doc(DbName, DocId) ->
+    {ok, Db} = fabric2_db:open(DbName, [?ADMIN_CTX]),
+    {ok, Doc} = fabric2_db:open_doc(Db, DocId, [ejson_body]),
+    Body = Doc#doc.body,
+    couch_util:json_decode(couch_util:json_encode(Body), [return_maps]).
+
+
+wait_scheduler_docs_state(DbName, DocId, State) ->
+    test_util:wait(fun() ->
+        case scheduler_docs(DbName, DocId) of
+            {200, #{<<"state">> := State} = Res} -> Res;
+            {_, _} -> wait
+        end
+    end, 10000, 250).
+
+
+wait_scheduler_docs_not_found(DbName, DocId) ->
+    test_util:wait(fun() ->
+        case scheduler_docs(DbName, DocId) of
+            {404, _} -> ok;
+            {_, _} -> wait
+        end
+    end, 10000, 250).
+
+
+wait_reschedule_docs_state(DbName, DocId, State) ->
+    test_util:wait(fun() ->
+        couch_replicator_job_server:reschedule(),
+        case scheduler_docs(DbName, DocId) of
+            {200, #{<<"state">> := State} = Res} -> Res;
+            {_, _} -> wait
+        end
+    end, 10000, 500).
+
+
+wait_doc_state(DbName, DocId, State) ->
+    test_util:wait(fun() ->
+        case read_doc(DbName, DocId) of
+            #{<<"_replication_state">> := State} -> ok;
+            #{} -> wait
+        end
+    end, 10000, 250).
diff --git a/src/couch_replicator/test/eunit/couch_replicator_filtered_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_filtered_tests.erl
index 7ac9a4d..4d72c84 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_filtered_tests.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_filtered_tests.erl
@@ -12,17 +12,20 @@
 
 -module(couch_replicator_filtered_tests).
 
+
 -include_lib("couch/include/couch_eunit.hrl").
 -include_lib("couch/include/couch_db.hrl").
 -include_lib("couch_replicator/src/couch_replicator.hrl").
+-include_lib("fabric/test/fabric2_test.hrl").
 
--define(DDOC, {[
-    {<<"_id">>, <<"_design/filter_ddoc">>},
-    {<<"filters">>, {[
-        {<<"testfilter">>, <<"
+-define(DDOC_ID, <<"_design/filter_ddoc">>).
+-define(DDOC, #{
+    <<"_id">> => ?DDOC_ID,
+    <<"filters">> => #{
+        <<"testfilter">> => <<"
             function(doc, req){if (doc.class == 'mammal') return true;}
-        ">>},
-        {<<"queryfilter">>, <<"
+        ">>,
+        <<"queryfilter">> => <<"
             function(doc, req) {
                 if (doc.class && req.query.starts) {
                     return doc.class.indexOf(req.query.starts) === 0;
@@ -31,99 +34,87 @@
                     return false;
                 }
             }
-        ">>}
-    ]}},
-    {<<"views">>, {[
-        {<<"mammals">>, {[
-            {<<"map">>, <<"
+        ">>
+    },
+    <<"views">> => #{
+        <<"mammals">> => #{
+            <<"map">> => <<"
                 function(doc) {
                     if (doc.class == 'mammal') {
                         emit(doc._id, null);
                     }
                 }
-            ">>}
-        ]}}
-    ]}}
-]}).
-
-setup(_) ->
-    Ctx = test_util:start_couch([couch_replicator]),
-    Source = create_db(),
-    create_docs(Source),
-    Target = create_db(),
-    {Ctx, {Source, Target}}.
+            ">>
+        }
+    }
+}).
 
-teardown(_, {Ctx, {Source, Target}}) ->
-    delete_db(Source),
-    delete_db(Target),
-    ok = application:stop(couch_replicator),
-    ok = test_util:stop_couch(Ctx).
 
 filtered_replication_test_() ->
-    Pairs = [{remote, remote}],
     {
-        "Filtered replication tests",
+        "Replications with filters tests",
         {
-            foreachx,
-            fun setup/1, fun teardown/2,
-            [{Pair, fun should_succeed/2} || Pair <- Pairs]
+            setup,
+            fun couch_replicator_test_helper:start_couch/0,
+            fun couch_replicator_test_helper:stop_couch/1,
+            {
+                foreach,
+                fun setup/0,
+                fun teardown/1,
+                [
+                    ?TDEF_FE(filtered_replication_test),
+                    ?TDEF_FE(query_filtered_replication_test),
+                    ?TDEF_FE(view_filtered_replication_test),
+                    ?TDEF_FE(replication_id_changes_if_filter_changes, 15)
+                ]
+            }
         }
     }.
 
-query_filtered_replication_test_() ->
-    Pairs = [{remote, remote}],
-    {
-        "Filtered with query replication tests",
-        {
-            foreachx,
-            fun setup/1, fun teardown/2,
-            [{Pair, fun should_succeed_with_query/2} || Pair <- Pairs]
-        }
-    }.
 
-view_filtered_replication_test_() ->
-    Pairs = [{remote, remote}],
-    {
-        "Filtered with a view replication tests",
-        {
-            foreachx,
-            fun setup/1, fun teardown/2,
-            [{Pair, fun should_succeed_with_view/2} || Pair <- Pairs]
-        }
-    }.
+setup() ->
+    Source = couch_replicator_test_helper:create_db(),
+    create_docs(Source),
+    Target = couch_replicator_test_helper:create_db(),
+    config:set("replicator", "stats_update_interval_sec", "0", false),
+    config:set("replicator", "interval_sec", "1", false),
+    {Source, Target}.
+
+
+teardown({Source, Target}) ->
+    config:delete("replicator", "stats_update_interval_sec", false),
+    config:delete("replicator", "checkpoint_interval", false),
+    config:delete("replicator", "interval_sec", false),
+    couch_replicator_test_helper:delete_db(Source),
+    couch_replicator_test_helper:delete_db(Target).
 
-should_succeed({From, To}, {_Ctx, {Source, Target}}) ->
+
+filtered_replication_test({Source, Target}) ->
     RepObject = {[
-        {<<"source">>, db_url(From, Source)},
-        {<<"target">>, db_url(To, Target)},
+        {<<"source">>, Source},
+        {<<"target">>, Target},
         {<<"filter">>, <<"filter_ddoc/testfilter">>}
     ]},
-    {ok, _} = couch_replicator:replicate(RepObject, ?ADMIN_USER),
-    %% FilteredFun is an Erlang version of following JS function
-    %% function(doc, req){if (doc.class == 'mammal') return true;}
+    {ok, _} = couch_replicator_test_helper:replicate(RepObject),
     FilterFun = fun(_DocId, {Props}) ->
         couch_util:get_value(<<"class">>, Props) == <<"mammal">>
     end,
     {ok, TargetDbInfo, AllReplies} = compare_dbs(Source, Target, FilterFun),
-    {lists:flatten(io_lib:format("~p -> ~p", [From, To])), [
-        {"Target DB has proper number of docs",
-        ?_assertEqual(1, proplists:get_value(doc_count, TargetDbInfo))},
-        {"Target DB doesn't have deleted docs",
-        ?_assertEqual(0, proplists:get_value(doc_del_count, TargetDbInfo))},
-        {"All the docs filtered as expected",
-        ?_assert(lists:all(fun(Valid) -> Valid end, AllReplies))}
-    ]}.
-
-should_succeed_with_query({From, To}, {_Ctx, {Source, Target}}) ->
+    ?assertEqual(1, proplists:get_value(doc_count, TargetDbInfo)),
+    ?assertEqual(0, proplists:get_value(doc_del_count, TargetDbInfo)),
+    ?assert(lists:all(fun(Valid) -> Valid end, AllReplies)).
+
+
+query_filtered_replication_test({Source, Target}) ->
     RepObject = {[
-        {<<"source">>, db_url(From, Source)},
-        {<<"target">>, db_url(To, Target)},
+        {<<"source">>, Source},
+        {<<"target">>, Target},
         {<<"filter">>, <<"filter_ddoc/queryfilter">>},
         {<<"query_params">>, {[
             {<<"starts">>, <<"a">>}
         ]}}
     ]},
-    {ok, _} = couch_replicator:replicate(RepObject, ?ADMIN_USER),
+    {ok, _} = couch_replicator_test_helper:replicate(RepObject),
     FilterFun = fun(_DocId, {Props}) ->
         case couch_util:get_value(<<"class">>, Props) of
             <<"a", _/binary>> -> true;
@@ -131,109 +122,144 @@ should_succeed_with_query({From, To}, {_Ctx, {Source, Target}}) ->
         end
     end,
     {ok, TargetDbInfo, AllReplies} = compare_dbs(Source, Target, FilterFun),
-    {lists:flatten(io_lib:format("~p -> ~p", [From, To])), [
-        {"Target DB has proper number of docs",
-        ?_assertEqual(2, proplists:get_value(doc_count, TargetDbInfo))},
-        {"Target DB doesn't have deleted docs",
-        ?_assertEqual(0, proplists:get_value(doc_del_count, TargetDbInfo))},
-        {"All the docs filtered as expected",
-        ?_assert(lists:all(fun(Valid) -> Valid end, AllReplies))}
-    ]}.
-
-should_succeed_with_view({From, To}, {_Ctx, {Source, Target}}) ->
+    ?assertEqual(2, proplists:get_value(doc_count, TargetDbInfo)),
+    ?assertEqual(0, proplists:get_value(doc_del_count, TargetDbInfo)),
+    ?assert(lists:all(fun(Valid) -> Valid end, AllReplies)).
+
+
+view_filtered_replication_test({Source, Target}) ->
     RepObject = {[
-        {<<"source">>, db_url(From, Source)},
-        {<<"target">>, db_url(To, Target)},
+        {<<"source">>, Source},
+        {<<"target">>, Target},
         {<<"filter">>, <<"_view">>},
         {<<"query_params">>, {[
             {<<"view">>, <<"filter_ddoc/mammals">>}
         ]}}
     ]},
-    {ok, _} = couch_replicator:replicate(RepObject, ?ADMIN_USER),
+    {ok, _} = couch_replicator_test_helper:replicate(RepObject),
     FilterFun = fun(_DocId, {Props}) ->
         couch_util:get_value(<<"class">>, Props) == <<"mammal">>
     end,
     {ok, TargetDbInfo, AllReplies} = compare_dbs(Source, Target, FilterFun),
-    {lists:flatten(io_lib:format("~p -> ~p", [From, To])), [
-        {"Target DB has proper number of docs",
-        ?_assertEqual(1, proplists:get_value(doc_count, TargetDbInfo))},
-        {"Target DB doesn't have deleted docs",
-        ?_assertEqual(0, proplists:get_value(doc_del_count, TargetDbInfo))},
-        {"All the docs filtered as expected",
-        ?_assert(lists:all(fun(Valid) -> Valid end, AllReplies))}
-    ]}.
+    ?assertEqual(1, proplists:get_value(doc_count, TargetDbInfo)),
+    ?assertEqual(0, proplists:get_value(doc_del_count, TargetDbInfo)),
+    ?assert(lists:all(fun(Valid) -> Valid end, AllReplies)).
+
+
+replication_id_changes_if_filter_changes({Source, Target}) ->
+    config:set("replicator", "checkpoint_interval", "500", false),
+    Rep = {[
+        {<<"source">>, Source},
+        {<<"target">>, Target},
+        {<<"filter">>, <<"filter_ddoc/testfilter">>},
+        {<<"continuous">>, true}
+    ]},
+    {ok, _, RepId1} = couch_replicator_test_helper:replicate_continuous(Rep),
+
+    wait_scheduler_docs_written(1),
+
+    ?assertMatch([#{<<"id">> := RepId1}],
+        couch_replicator_test_helper:scheduler_jobs()),
+
+    FilterFun1 = fun(_, {Props}) ->
+        couch_util:get_value(<<"class">>, Props) == <<"mammal">>
+    end,
+    {ok, TargetDbInfo1, AllReplies1} = compare_dbs(Source, Target, FilterFun1),
+    ?assertEqual(1, proplists:get_value(doc_count, TargetDbInfo1)),
+    ?assert(lists:all(fun(Valid) -> Valid end, AllReplies1)),
+
+    {ok, SourceDb} = fabric2_db:open(Source, [?ADMIN_CTX]),
+    {ok, DDoc1} = fabric2_db:open_doc(SourceDb, ?DDOC_ID),
+    Flt = <<"function(doc, req) {if (doc.class == 'reptiles') return true};">>,
+    DDoc2 = DDoc1#doc{body = {[
+        {<<"filters">>, {[
+            {<<"testfilter">>, Flt}
+        ]}}
+    ]}},
+    {ok, {_, _}} = fabric2_db:update_doc(SourceDb, DDoc2),
+    Info = wait_scheduler_repid_change(RepId1),
+
+    RepId2 = maps:get(<<"id">>, Info),
+    ?assert(RepId1 =/= RepId2),
+
+    wait_scheduler_docs_written(1),
+
+    FilterFun2 = fun(_, {Props}) ->
+        Class = couch_util:get_value(<<"class">>, Props),
+        Class == <<"mammal">> orelse Class == <<"reptiles">>
+    end,
+    {ok, TargetDbInfo2, AllReplies2} = compare_dbs(Source, Target, FilterFun2),
+    ?assertEqual(2, proplists:get_value(doc_count, TargetDbInfo2)),
+    ?assert(lists:all(fun(Valid) -> Valid end, AllReplies2)),
+
+    couch_replicator_test_helper:cancel(RepId2).
+
 
 compare_dbs(Source, Target, FilterFun) ->
-    {ok, SourceDb} = couch_db:open_int(Source, []),
-    {ok, TargetDb} = couch_db:open_int(Target, []),
-    {ok, TargetDbInfo} = couch_db:get_db_info(TargetDb),
-    Fun = fun(FullDocInfo, Acc) ->
-        {ok, DocId, SourceDoc} = read_doc(SourceDb, FullDocInfo),
-        TargetReply = read_doc(TargetDb, DocId),
-        case FilterFun(DocId, SourceDoc) of
-            true ->
-                ValidReply = {ok, DocId, SourceDoc} == TargetReply,
-                {ok, [ValidReply|Acc]};
-            false ->
-                ValidReply = {not_found, missing} == TargetReply,
-                {ok, [ValidReply|Acc]}
+    {ok, TargetDb} = fabric2_db:open(Target, [?ADMIN_CTX]),
+    {ok, TargetDbInfo} = fabric2_db:get_db_info(TargetDb),
+    Fun = fun(SrcDoc, TgtDoc, Acc) ->
+        case FilterFun(SrcDoc#doc.id, SrcDoc#doc.body) of
+            true -> [SrcDoc == TgtDoc | Acc];
+            false -> [not_found == TgtDoc | Acc]
         end
     end,
-    {ok, AllReplies} = couch_db:fold_docs(SourceDb, Fun, [], []),
-    ok = couch_db:close(SourceDb),
-    ok = couch_db:close(TargetDb),
-    {ok, TargetDbInfo, AllReplies}.
-
-read_doc(Db, DocIdOrInfo) ->
-    case couch_db:open_doc(Db, DocIdOrInfo) of
-        {ok, Doc} ->
-            {Props} = couch_doc:to_json_obj(Doc, [attachments]),
-            DocId = couch_util:get_value(<<"_id">>, Props),
-            {ok, DocId, {Props}};
-        Error ->
-            Error
-    end.
-
-create_db() ->
-    DbName = ?tempdb(),
-    {ok, Db} = couch_db:create(DbName, [?ADMIN_CTX]),
-    ok = couch_db:close(Db),
-    DbName.
+    Res = couch_replicator_test_helper:compare_fold(Source, Target, Fun, []),
+    {ok, TargetDbInfo, Res}.
+
 
 create_docs(DbName) ->
-    {ok, Db} = couch_db:open(DbName, [?ADMIN_CTX]),
-    DDoc = couch_doc:from_json_obj(?DDOC),
-    Doc1 = couch_doc:from_json_obj({[
-        {<<"_id">>, <<"doc1">>},
-        {<<"class">>, <<"mammal">>},
-        {<<"value">>, 1}
-
-    ]}),
-    Doc2 = couch_doc:from_json_obj({[
-        {<<"_id">>, <<"doc2">>},
-        {<<"class">>, <<"amphibians">>},
-        {<<"value">>, 2}
-
-    ]}),
-    Doc3 = couch_doc:from_json_obj({[
-        {<<"_id">>, <<"doc3">>},
-        {<<"class">>, <<"reptiles">>},
-        {<<"value">>, 3}
-
-    ]}),
-    Doc4 = couch_doc:from_json_obj({[
-        {<<"_id">>, <<"doc4">>},
-        {<<"class">>, <<"arthropods">>},
-        {<<"value">>, 2}
-
-    ]}),
-    {ok, _} = couch_db:update_docs(Db, [DDoc, Doc1, Doc2, Doc3, Doc4]),
-    couch_db:close(Db).
-
-delete_db(DbName) ->
-    ok = couch_server:delete(DbName, [?ADMIN_CTX]).
-
-db_url(remote, DbName) ->
-    Addr = config:get("httpd", "bind_address", "127.0.0.1"),
-    Port = mochiweb_socket_server:get(couch_httpd, port),
-    ?l2b(io_lib:format("http://~s:~b/~s", [Addr, Port, DbName])).
+    couch_replicator_test_helper:create_docs(DbName, [
+        ?DDOC,
+        #{
+            <<"_id">> => <<"doc1">>,
+            <<"class">> => <<"mammal">>,
+            <<"value">> => 1
+        },
+        #{
+            <<"_id">> => <<"doc2">>,
+            <<"class">> => <<"amphibians">>,
+            <<"value">> => 2
+        },
+        #{
+            <<"_id">> => <<"doc3">>,
+            <<"class">> => <<"reptiles">>,
+            <<"value">> => 3
+        },
+        #{
+            <<"_id">> => <<"doc4">>,
+            <<"class">> => <<"arthropods">>,
+            <<"value">> => 2
+        }
+    ]).
+
+
+wait_scheduler_docs_written(DocsWritten) ->
+    test_util:wait(fun() ->
+        case couch_replicator_test_helper:scheduler_jobs() of
+            [] ->
+                wait;
+            [#{<<"info">> := null}] ->
+                wait;
+            [#{<<"info">> := Info}] ->
+                case Info of
+                    #{<<"docs_written">> := DocsWritten} -> Info;
+                    #{} -> wait
+                end
+        end
+    end, 10000, 250).
+
+
+wait_scheduler_repid_change(OldRepId) ->
+    test_util:wait(fun() ->
+        case couch_replicator_test_helper:scheduler_jobs() of
+            [] ->
+                wait;
+            [#{<<"id">> := OldRepId}] ->
+                wait;
+            [#{<<"id">> := null}] ->
+                wait;
+            [#{<<"id">> := NewId} = Info] when is_binary(NewId) ->
+                Info
+        end
+    end, 10000, 250).
diff --git a/src/couch_replicator/test/eunit/couch_replicator_httpc_pool_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_httpc_pool_tests.erl
index c4ad4e9..6c61446 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_httpc_pool_tests.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_httpc_pool_tests.erl
@@ -12,17 +12,13 @@
 
 -module(couch_replicator_httpc_pool_tests).
 
+
 -include_lib("couch/include/couch_eunit.hrl").
 -include_lib("couch/include/couch_db.hrl").
-
--define(TIMEOUT, 1000).
+-include_lib("fabric/test/fabric2_test.hrl").
 
 
-setup() ->
-    spawn_pool().
-
-teardown(Pool) ->
-    stop_pool(Pool).
+-define(TIMEOUT, 1000).
 
 
 httpc_pool_test_() ->
@@ -30,75 +26,81 @@ httpc_pool_test_() ->
         "httpc pool tests",
         {
             setup,
-            fun() -> test_util:start_couch([couch_replicator]) end, fun test_util:stop_couch/1,
+            fun couch_replicator_test_helper:start_couch/0,
+            fun couch_replicator_test_helper:stop_couch/1,
             {
                 foreach,
-                fun setup/0, fun teardown/1,
+                fun setup/0,
+                fun teardown/1,
                 [
-                    fun should_block_new_clients_when_full/1,
-                    fun should_replace_worker_on_death/1
+                    ?TDEF_FE(should_block_new_clients_when_full),
+                    ?TDEF_FE(should_replace_worker_on_death)
                 ]
             }
         }
     }.
 
 
+setup() ->
+    spawn_pool().
+
+
+teardown(Pool) ->
+    stop_pool(Pool).
+
+
 should_block_new_clients_when_full(Pool) ->
-    ?_test(begin
-        Client1 = spawn_client(Pool),
-        Client2 = spawn_client(Pool),
-        Client3 = spawn_client(Pool),
+    Client1 = spawn_client(Pool),
+    Client2 = spawn_client(Pool),
+    Client3 = spawn_client(Pool),
+
+    ?assertEqual(ok, ping_client(Client1)),
+    ?assertEqual(ok, ping_client(Client2)),
+    ?assertEqual(ok, ping_client(Client3)),
 
-        ?assertEqual(ok, ping_client(Client1)),
-        ?assertEqual(ok, ping_client(Client2)),
-        ?assertEqual(ok, ping_client(Client3)),
+    Worker1 = get_client_worker(Client1, "1"),
+    Worker2 = get_client_worker(Client2, "2"),
+    Worker3 = get_client_worker(Client3, "3"),
 
-        Worker1 = get_client_worker(Client1, "1"),
-        Worker2 = get_client_worker(Client2, "2"),
-        Worker3 = get_client_worker(Client3, "3"),
+    ?assert(is_process_alive(Worker1)),
+    ?assert(is_process_alive(Worker2)),
+    ?assert(is_process_alive(Worker3)),
 
-        ?assert(is_process_alive(Worker1)),
-        ?assert(is_process_alive(Worker2)),
-        ?assert(is_process_alive(Worker3)),
+    ?assertNotEqual(Worker1, Worker2),
+    ?assertNotEqual(Worker2, Worker3),
+    ?assertNotEqual(Worker3, Worker1),
 
-        ?assertNotEqual(Worker1, Worker2),
-        ?assertNotEqual(Worker2, Worker3),
-        ?assertNotEqual(Worker3, Worker1),
+    Client4 = spawn_client(Pool),
+    ?assertEqual(timeout, ping_client(Client4)),
 
-        Client4 = spawn_client(Pool),
-        ?assertEqual(timeout, ping_client(Client4)),
+    ?assertEqual(ok, stop_client(Client1)),
+    ?assertEqual(ok, ping_client(Client4)),
 
-        ?assertEqual(ok, stop_client(Client1)),
-        ?assertEqual(ok, ping_client(Client4)),
+    Worker4 = get_client_worker(Client4, "4"),
+    ?assertEqual(Worker1, Worker4),
 
-        Worker4 = get_client_worker(Client4, "4"),
-        ?assertEqual(Worker1, Worker4),
+    lists:foreach(fun(C) ->
+            ?assertEqual(ok, stop_client(C))
+    end, [Client2, Client3, Client4]).
 
-        lists:foreach(
-            fun(C) ->
-                ?assertEqual(ok, stop_client(C))
-            end, [Client2, Client3, Client4])
-    end).
 
 should_replace_worker_on_death(Pool) ->
-    ?_test(begin
-        Client1 = spawn_client(Pool),
-        ?assertEqual(ok, ping_client(Client1)),
-        Worker1 = get_client_worker(Client1, "1"),
-        ?assert(is_process_alive(Worker1)),
+    Client1 = spawn_client(Pool),
+    ?assertEqual(ok, ping_client(Client1)),
+    Worker1 = get_client_worker(Client1, "1"),
+    ?assert(is_process_alive(Worker1)),
 
-        ?assertEqual(ok, kill_client_worker(Client1)),
-        ?assertNot(is_process_alive(Worker1)),
-        ?assertEqual(ok, stop_client(Client1)),
+    ?assertEqual(ok, kill_client_worker(Client1)),
+    ?assertNot(is_process_alive(Worker1)),
+    ?assertEqual(ok, stop_client(Client1)),
 
-        Client2 = spawn_client(Pool),
-        ?assertEqual(ok, ping_client(Client2)),
-        Worker2 = get_client_worker(Client2, "2"),
-        ?assert(is_process_alive(Worker2)),
+    Client2 = spawn_client(Pool),
+    ?assertEqual(ok, ping_client(Client2)),
+    Worker2 = get_client_worker(Client2, "2"),
+    ?assert(is_process_alive(Worker2)),
 
-        ?assertNotEqual(Worker1, Worker2),
-        ?assertEqual(ok, stop_client(Client2))
-    end).
+    ?assertNotEqual(Worker1, Worker2),
+    ?assertEqual(ok, stop_client(Client2)).
 
 
 spawn_client(Pool) ->
@@ -110,6 +112,7 @@ spawn_client(Pool) ->
     end),
     {Pid, Ref}.
 
+
 ping_client({Pid, Ref}) ->
     Pid ! ping,
     receive
@@ -119,18 +122,18 @@ ping_client({Pid, Ref}) ->
         timeout
     end.
 
+
 get_client_worker({Pid, Ref}, ClientName) ->
     Pid ! get_worker,
     receive
         {worker, Ref, Worker} ->
             Worker
     after ?TIMEOUT ->
-        erlang:error(
-            {assertion_failed,
-             [{module, ?MODULE}, {line, ?LINE},
-              {reason, "Timeout getting client " ++ ClientName ++ " worker"}]})
+        erlang:error({assertion_failed, [{module, ?MODULE}, {line, ?LINE},
+            {reason, "Timeout getting client " ++ ClientName ++ " worker"}]})
     end.
 
+
 stop_client({Pid, Ref}) ->
     Pid ! stop,
     receive
@@ -140,6 +143,7 @@ stop_client({Pid, Ref}) ->
         timeout
     end.
 
+
 kill_client_worker({Pid, Ref}) ->
     Pid ! get_worker,
     receive
@@ -150,6 +154,7 @@ kill_client_worker({Pid, Ref}) ->
         timeout
     end.
 
+
 loop(Parent, Ref, Worker, Pool) ->
     receive
         ping ->
@@ -163,12 +168,14 @@ loop(Parent, Ref, Worker, Pool) ->
             Parent ! {stop, Ref}
     end.
 
+
 spawn_pool() ->
-    Host = config:get("httpd", "bind_address", "127.0.0.1"),
-    Port = config:get("httpd", "port", "5984"),
+    Host = config:get("chttpd", "bind_address", "127.0.0.1"),
+    Port = config:get("chttpd", "port", "5984"),
     {ok, Pool} = couch_replicator_httpc_pool:start_link(
         "http://" ++ Host ++ ":" ++ Port, [{max_connections, 3}]),
     Pool.
 
+
 stop_pool(Pool) ->
     ok = couch_replicator_httpc_pool:stop(Pool).
diff --git a/src/couch_replicator/test/eunit/couch_replicator_id_too_long_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_id_too_long_tests.erl
index a4696c4..3a0e6f7 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_id_too_long_tests.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_id_too_long_tests.erl
@@ -15,76 +15,57 @@
 -include_lib("couch/include/couch_eunit.hrl").
 -include_lib("couch/include/couch_db.hrl").
 -include_lib("couch_replicator/src/couch_replicator.hrl").
-
-
-setup(_) ->
-    Ctx = test_util:start_couch([couch_replicator]),
-    Source = create_db(),
-    create_doc(Source),
-    Target = create_db(),
-    {Ctx, {Source, Target}}.
-
-
-teardown(_, {Ctx, {Source, Target}}) ->
-    delete_db(Source),
-    delete_db(Target),
-    config:set("replicator", "max_document_id_length", "infinity"),
-    ok = test_util:stop_couch(Ctx).
+-include_lib("fabric/test/fabric2_test.hrl").
 
 
 id_too_long_replication_test_() ->
-    Pairs = [{remote, remote}],
     {
         "Doc id too long tests",
         {
-            foreachx,
-            fun setup/1, fun teardown/2,
-            [{Pair, fun should_succeed/2} || Pair <- Pairs] ++
-            [{Pair, fun should_fail/2} || Pair <- Pairs]
+            setup,
+            fun couch_replicator_test_helper:start_couch/0,
+            fun couch_replicator_test_helper:stop_couch/1,
+            {
+                foreach,
+                fun setup/0,
+                fun teardown/1,
+                [
+                    ?TDEF_FE(should_succeed),
+                    ?TDEF_FE(should_fail)
+
+                ]
+            }
         }
     }.
 
 
-should_succeed({From, To}, {_Ctx, {Source, Target}}) ->
-    RepObject = {[
-        {<<"source">>, db_url(From, Source)},
-        {<<"target">>, db_url(To, Target)}
-    ]},
-    config:set("replicator", "max_document_id_length", "5"),
-    {ok, _} = couch_replicator:replicate(RepObject, ?ADMIN_USER),
-    ?_assertEqual(ok, couch_replicator_test_helper:compare_dbs(Source, Target)).
-
+setup() ->
+    Source = couch_replicator_test_helper:create_db(),
+    create_doc(Source),
+    Target = couch_replicator_test_helper:create_db(),
+    {Source, Target}.
 
-should_fail({From, To}, {_Ctx, {Source, Target}}) ->
-    RepObject = {[
-        {<<"source">>, db_url(From, Source)},
-        {<<"target">>, db_url(To, Target)}
-    ]},
-    config:set("replicator", "max_document_id_length", "4"),
-    {ok, _} = couch_replicator:replicate(RepObject, ?ADMIN_USER),
-    ?_assertError({badmatch, {not_found, missing}},
-        couch_replicator_test_helper:compare_dbs(Source, Target)).
 
+teardown({Source, Target}) ->
+    config:delete("replicator", "max_document_id_length", false),
+    couch_replicator_test_helper:delete_db(Source),
+    couch_replicator_test_helper:delete_db(Target).
 
-create_db() ->
-    DbName = ?tempdb(),
-    {ok, Db} = couch_db:create(DbName, [?ADMIN_CTX]),
-    ok = couch_db:close(Db),
-    DbName.
 
+should_succeed({Source, Target}) ->
+    config:set("replicator", "max_document_id_length", "5", false),
+    {ok, _} = couch_replicator_test_helper:replicate(Source, Target),
+    ?assertEqual(ok, couch_replicator_test_helper:compare_dbs(Source, Target)).
 
-create_doc(DbName) ->
-    {ok, Db} = couch_db:open(DbName, [?ADMIN_CTX]),
-    Doc = couch_doc:from_json_obj({[{<<"_id">>, <<"12345">>}]}),
-    {ok, _} = couch_db:update_doc(Db, Doc, []),
-    couch_db:close(Db).
 
+should_fail({Source, Target}) ->
+    config:set("replicator", "max_document_id_length", "4", false),
+    {ok, _} = couch_replicator_test_helper:replicate(Source, Target),
+    ExceptIds = [<<"12345">>],
+    ?assertEqual(ok, couch_replicator_test_helper:compare_dbs(Source, Target,
+        ExceptIds)).
 
-delete_db(DbName) ->
-    ok = couch_server:delete(DbName, [?ADMIN_CTX]).
 
-
-db_url(remote, DbName) ->
-    Addr = config:get("httpd", "bind_address", "127.0.0.1"),
-    Port = mochiweb_socket_server:get(couch_httpd, port),
-    ?l2b(io_lib:format("http://~s:~b/~s", [Addr, Port, DbName])).
+create_doc(DbName) ->
+    Docs = [#{<<"_id">> => <<"12345">>}],
+    couch_replicator_test_helper:create_docs(DbName, Docs).
diff --git a/src/couch_replicator/test/eunit/couch_replicator_job_server_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_job_server_tests.erl
new file mode 100644
index 0000000..698a844
--- /dev/null
+++ b/src/couch_replicator/test/eunit/couch_replicator_job_server_tests.erl
@@ -0,0 +1,437 @@
+% 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_job_server_tests).
+
+-include_lib("couch/include/couch_eunit.hrl").
+-include_lib("fabric/test/fabric2_test.hrl").
+
+
+-define(SHUTDOWN_TIMEOUT, 1000).
+-define(JOB_SERVER, couch_replicator_job_server).
+
+
+job_server_test_() ->
+    {
+        "Test job server",
+        {
+            setup,
+            fun setup_all/0,
+            fun teardown_all/1,
+            {
+                foreach,
+                fun setup/0,
+                fun teardown/1,
+                [
+                    ?TDEF_FE(should_start_up),
+                    ?TDEF_FE(reschedule_resets_timer),
+                    ?TDEF_FE(reschedule_reads_config),
+                    ?TDEF_FE(acceptors_spawned_if_pending),
+                    ?TDEF_FE(acceptors_not_spawned_if_no_pending),
+                    ?TDEF_FE(acceptors_not_spawned_if_no_max_churn),
+                    ?TDEF_FE(acceptors_not_spawned_if_no_churn_budget),
+                    ?TDEF_FE(acceptors_spawned_on_acceptor_exit),
+                    ?TDEF_FE(acceptor_turns_into_worker),
+                    ?TDEF_FE(acceptors_spawned_on_worker_exit),
+                    ?TDEF_FE(excess_acceptors_spawned),
+                    ?TDEF_FE(excess_workers_trimmed_on_reschedule),
+                    ?TDEF_FE(recent_workers_are_not_stopped)
+                ]
+            }
+        }
+    }.
+
+
+setup_all() ->
+    Ctx = test_util:start_couch(),
+    meck:new(couch_replicator_job_server, [passthrough]),
+    mock_pending(0),
+    meck:expect(couch_replicator_jobs, set_timeout, 0, ok),
+    meck:expect(couch_replicator_jobs, fold_jobs, 3, ok),
+    meck:expect(couch_replicator_job, start_link, fun() ->
+        {ok, spawn_link(fun() -> start_job() end)}
+    end),
+    Ctx.
+
+
+teardown_all(Ctx) ->
+    meck:unload(),
+    config_delete("interval_sec"),
+    config_delete("max_acceptors"),
+    config_delete("max_jobs"),
+    config_delete("max_churn"),
+    config_delete("min_run_time_sec"),
+    config_delete("transient_job_max_age_sec"),
+    test_util:stop_couch(Ctx).
+
+
+setup() ->
+    config_set("interval_sec", "99999"),
+    config_set("max_acceptors", "0"),
+    config_set("max_jobs", "0"),
+    config_set("max_churn", "1"),
+    config_set("min_run_time_sec", "0"),
+    config_set("transient_job_max_age_sec", "99999"),
+
+    mock_pending(0),
+
+    {ok, SPid} = ?JOB_SERVER:start_link(?SHUTDOWN_TIMEOUT),
+    SPid.
+
+
+teardown(SPid) when is_pid(SPid) ->
+    unlink(SPid),
+    Ref = monitor(process, SPid),
+    exit(SPid, kill),
+    receive {'DOWN', Ref, _, _, _} -> ok end,
+
+    meck:reset(couch_replicator_jobs),
+    meck:reset(couch_replicator_job),
+    meck:reset(couch_replicator_job_server),
+
+    config_delete("interval_sec"),
+    config_delete("max_acceptors"),
+    config_delete("max_jobs"),
+    config_delete("max_churn"),
+    config_delete("min_run_time_sec"),
+    config_delete("transient_job_max_age_sec").
+
+
+should_start_up(SPid) ->
+    ?assert(is_process_alive(SPid)),
+    ?assertEqual(SPid, whereis(?JOB_SERVER)),
+    State = sys:get_state(?JOB_SERVER),
+    #{
+        acceptors := #{},
+        workers := #{},
+        churn := 0,
+        config := Config,
+        timer := Timer,
+        timeout := ?SHUTDOWN_TIMEOUT
+    } = State,
+
+    % Make sure it read the config
+    ?assertMatch(#{
+        max_acceptors := 0,
+        interval_sec := 99999,
+        max_jobs := 0,
+        max_churn := 1,
+        min_run_time_sec := 0,
+        transient_job_max_age_sec := 99999
+    }, Config),
+
+    % Timer was set up
+    ?assert(is_reference(Timer)),
+    ?assert(is_integer(erlang:read_timer(Timer))).
+
+
+reschedule_resets_timer(_) ->
+    #{timer := OldTimer} = sys:get_state(?JOB_SERVER),
+
+    ?assertEqual(ok, ?JOB_SERVER:reschedule()),
+
+    #{timer := Timer} = sys:get_state(?JOB_SERVER),
+    ?assert(is_reference(Timer)),
+    ?assert(Timer =/= OldTimer).
+
+
+reschedule_reads_config(_) ->
+    config_set("interval_sec", "99998"),
+
+    ?JOB_SERVER:reschedule(),
+
+    #{config := Config} = sys:get_state(?JOB_SERVER),
+    ?assertMatch(#{interval_sec := 99998}, Config).
+
+
+acceptors_spawned_if_pending(_) ->
+    config_set("max_acceptors", "1"),
+    mock_pending(1),
+
+    ?JOB_SERVER:reschedule(),
+
+    ?assertMatch([Pid] when is_pid(Pid), acceptors()).
+
+
+acceptors_not_spawned_if_no_pending(_) ->
+    config_set("max_acceptors", "1"),
+    mock_pending(0),
+
+    ?JOB_SERVER:reschedule(),
+
+    ?assertEqual([], acceptors()).
+
+
+acceptors_not_spawned_if_no_max_churn(_) ->
+    config_set("max_churn", "0"),
+    config_set("max_acceptors", "1"),
+    mock_pending(1),
+
+    ?JOB_SERVER:reschedule(),
+
+    ?assertEqual([], acceptors()).
+
+
+acceptors_not_spawned_if_no_churn_budget(_) ->
+    config_set("max_churn", "1"),
+    config_set("max_acceptors", "1"),
+    mock_pending(0),
+
+    % To read the config
+    ?JOB_SERVER:reschedule(),
+
+    ?assertEqual([], acceptors()),
+
+    mock_pending(1),
+
+    % Exhaust churn budget
+    sys:replace_state(couch_replicator_job_server, fun(#{} = St) ->
+        St#{churn := 1}
+    end),
+
+    ?JOB_SERVER:reschedule(),
+
+    ?assertEqual([], acceptors()).
+
+
+acceptors_spawned_on_acceptor_exit(_) ->
+    config_set("max_acceptors", "3"),
+    config_set("max_jobs", "4"),
+    mock_pending(1),
+
+    ?JOB_SERVER:reschedule(),
+
+    [A1] = acceptors(),
+
+    exit(A1, kill),
+    meck:wait(?JOB_SERVER, handle_info, [{'EXIT', A1, killed}, '_'], 2000),
+
+    ?assertEqual(3, length(acceptors())).
+
+
+acceptor_turns_into_worker(_) ->
+    config_set("max_acceptors", "3"),
+    config_set("max_jobs", "4"),
+    mock_pending(1),
+
+    ?JOB_SERVER:reschedule(),
+
+    [A1] = acceptors(),
+    accept_job(A1, true),
+    ?assertEqual(3, length(acceptors())),
+    #{workers := Workers} = sys:get_state(?JOB_SERVER),
+    ?assertMatch([{A1, {true, _}}], maps:to_list(Workers)).
+
+
+acceptors_spawned_on_worker_exit(_) ->
+    config_set("max_acceptors", "1"),
+    config_set("max_jobs", "1"),
+    mock_pending(1),
+
+    ?JOB_SERVER:reschedule(),
+
+    [A1] = acceptors(),
+    accept_job(A1, true),
+
+    % Since max_jobs = 1 no more acceptors are spawned
+    ?assertEqual(0, length(acceptors())),
+
+    % Same acceptor process is now a worker
+    ?assertEqual([A1], workers()),
+
+    exit(A1, shutdown),
+    meck:wait(?JOB_SERVER, handle_info, [{'EXIT', A1, shutdown}, '_'], 2000),
+
+    % New acceptor process started
+    ?assertEqual(1, length(acceptors())),
+    ?assertEqual(0, length(workers())).
+
+
+excess_acceptors_spawned(_) ->
+    config_set("max_acceptors", "2"),
+    config_set("max_churn", "3"),
+    config_set("max_jobs", "4"),
+    mock_pending(100),
+
+    ?JOB_SERVER:reschedule(),
+
+    ?assertEqual(3, length(acceptors())),
+
+    accept_all(),
+
+    ?assertEqual(3, length(workers())),
+    ?assertEqual(1, length(acceptors())),
+    % Check that the churn budget was consumed
+    ?assertMatch(#{churn := 3}, sys:get_state(?JOB_SERVER)),
+
+    accept_all(),
+
+    % No more acceptors spawned after reaching max_jobs
+    ?assertEqual(0, length(acceptors())),
+    ?assertEqual(4, length(workers())),
+
+    ?JOB_SERVER:reschedule(),
+
+    % Since all churn budget was consumed, no new acceptors should have beens
+    % spawned this cycle but churn budget should have been reset
+    ?assertEqual(0, length(acceptors())),
+    ?assertEqual(4, length(workers())),
+    ?assertMatch(#{churn := 0}, sys:get_state(?JOB_SERVER)),
+
+    ?JOB_SERVER:reschedule(),
+
+    % Should have spawned 3 excess acceptors
+    ?assertEqual(3, length(acceptors())),
+    ?assertEqual(4, length(workers())),
+
+    accept_all(),
+
+    % Running with an excess number of workers
+    ?assertEqual(0, length(acceptors())),
+    ?assertEqual(7, length(workers())).
+
+
+excess_workers_trimmed_on_reschedule(_) ->
+    config_set("max_acceptors", "2"),
+    config_set("max_churn", "3"),
+    config_set("max_jobs", "4"),
+    mock_pending(100),
+
+    ?JOB_SERVER:reschedule(),
+
+    [A1, A2, A3] = acceptors(),
+    accept_job(A1, true),
+    accept_job(A2, false),
+    accept_job(A3, false),
+    [A4] = acceptors(),
+    accept_job(A4, true),
+
+    ?JOB_SERVER:reschedule(),
+
+    % First reschedule was to reset the churn budget, this next one is to spawn
+    % an excess number of acceptors.
+    ?JOB_SERVER:reschedule(),
+
+    [A5, A6, A7] = acceptors(),
+    accept_job(A5, true),
+    accept_job(A6, false),
+    accept_job(A7, false),
+
+    ?assertEqual(7, length(workers())),
+
+    % Running with an excess number of workers. These should be trimmed on the
+    % during the next cycle
+    ?JOB_SERVER:reschedule(),
+
+    Workers = workers(),
+    ?assertEqual(4, length(Workers)),
+    ?assertEqual(0, length(acceptors())),
+
+    % Check that A1 and A4 were skipped since they are not continuous
+    ?assertEqual(Workers, Workers -- [A2, A3, A6]).
+
+
+recent_workers_are_not_stopped(_) ->
+    config_set("max_acceptors", "2"),
+    config_set("max_churn", "3"),
+    config_set("max_jobs", "4"),
+    mock_pending(100),
+
+    ?JOB_SERVER:reschedule(),
+
+    [A1, A2, A3] = acceptors(),
+    accept_job(A1, true),
+    accept_job(A2, false),
+    accept_job(A3, false),
+    [A4] = acceptors(),
+    accept_job(A4, true),
+
+    ?JOB_SERVER:reschedule(),
+
+    % First reschedule was to reset the churn budget, this next one is to spawn
+    % an excess number of acceptors.
+    ?JOB_SERVER:reschedule(),
+
+    [A5, A6, A7] = acceptors(),
+    accept_job(A5, true),
+    accept_job(A6, false),
+    accept_job(A7, false),
+
+    ?assertEqual(7, length(workers())),
+
+    % Running with an excess number of workers. But they won't be stopped on
+    % reschedule if they ran for a period less than min_run_time_sec during the
+    % next cycle
+    config_set("min_run_time_sec", "9999"),
+
+    % don't want to start new acceptors anymore
+    mock_pending(0),
+    config_set("max_acceptors", "0"),
+
+    ?JOB_SERVER:reschedule(),
+
+    ?assertEqual(7, length(workers())),
+    ?assertEqual(0, length(acceptors())),
+
+    config_set("min_run_time_sec", "0"),
+   
+    ?JOB_SERVER:reschedule(),
+
+    ?assertEqual(4, length(workers())),
+    ?assertEqual(0, length(acceptors())).
+
+
+config_set(K, V) ->
+    config:set("replicator", K, V, _Persist = false).
+
+
+config_delete(K) ->
+    config:delete("replicator", K, _Persist = false).
+
+
+mock_pending(N) ->
+    meck:expect(couch_replicator_jobs, pending_count, 2, N).
+
+
+acceptors() ->
+    #{acceptors := Acceptors} = sys:get_state(?JOB_SERVER),
+    maps:keys(Acceptors).
+
+
+workers() ->
+    #{workers := Workers} = sys:get_state(?JOB_SERVER),
+    maps:keys(Workers).
+
+
+accept_job(APid, Normal) ->
+    APid ! {accept_job, Normal, self()},
+    receive
+        {job_accepted, APid} -> ok
+    after
+        5000 ->
+            error(test_job_accept_timeout)
+    end.
+
+
+accept_all() ->
+    [accept_job(APid, true) || APid <- acceptors()].
+
+
+start_job() ->
+    receive
+        {accept_job, Normal, From} ->
+            ok = ?JOB_SERVER:accepted(self(), Normal),
+            From ! {job_accepted, self()},
+            start_job();
+        {exit_job, ExitSig} ->
+            exit(ExitSig)
+    end.
diff --git a/src/couch_replicator/test/eunit/couch_replicator_large_atts_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_large_atts_tests.erl
index 27c89a0..fcbdf22 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_large_atts_tests.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_large_atts_tests.erl
@@ -14,12 +14,8 @@
 
 -include_lib("couch/include/couch_eunit.hrl").
 -include_lib("couch/include/couch_db.hrl").
+-include_lib("fabric/test/fabric2_test.hrl").
 
--import(couch_replicator_test_helper, [
-    db_url/1,
-    replicate/2,
-    compare_dbs/2
-]).
 
 -define(ATT_SIZE_1, 2 * 1024 * 1024).
 -define(ATT_SIZE_2, round(6.6 * 1024 * 1024)).
@@ -27,90 +23,65 @@
 -define(TIMEOUT_EUNIT, 120).
 
 
-setup() ->
-    DbName = ?tempdb(),
-    {ok, Db} = couch_db:create(DbName, [?ADMIN_CTX]),
-    ok = couch_db:close(Db),
-    DbName.
-
-setup(remote) ->
-    {remote, setup()};
-setup({A, B}) ->
-    Ctx = test_util:start_couch([couch_replicator]),
-    config:set("attachments", "compressible_types", "text/*", false),
-    Source = setup(A),
-    Target = setup(B),
-    {Ctx, {Source, Target}}.
-
-teardown({remote, DbName}) ->
-    teardown(DbName);
-teardown(DbName) ->
-    ok = couch_server:delete(DbName, [?ADMIN_CTX]),
-    ok.
-
-teardown(_, {Ctx, {Source, Target}}) ->
-    teardown(Source),
-    teardown(Target),
-
-    ok = application:stop(couch_replicator),
-    ok = test_util:stop_couch(Ctx).
-
 large_atts_test_() ->
-    Pairs = [{remote, remote}],
     {
-        "Replicate docs with large attachments",
+        "Large attachment replication test",
         {
-            foreachx,
-            fun setup/1, fun teardown/2,
-            [{Pair, fun should_populate_replicate_compact/2}
-             || Pair <- Pairs]
+            setup,
+            fun couch_replicator_test_helper:start_couch/0,
+            fun couch_replicator_test_helper:stop_couch/1,
+            {
+                foreach,
+                fun setup/0,
+                fun teardown/1,
+                [
+                    ?TDEF_FE(should_replicate_attachments, 120)
+                ]
+            }
         }
     }.
 
 
-should_populate_replicate_compact({From, To}, {_Ctx, {Source, Target}}) ->
-    {lists:flatten(io_lib:format("~p -> ~p", [From, To])),
-     {inorder, [should_populate_source(Source),
-                should_replicate(Source, Target),
-                should_compare_databases(Source, Target)]}}.
+setup() ->
+    AttCfg = config:get("attachments", "compressible_types"),
+    config:set("attachments", "compressible_types", "text/*", false),
+    Source = couch_replicator_test_helper:create_db(),
+    ok = populate_db(Source, ?DOCS_COUNT),
+    Target = couch_replicator_test_helper:create_db(),
+    {AttCfg, Source, Target}.
+
 
-should_populate_source({remote, Source}) ->
-    should_populate_source(Source);
-should_populate_source(Source) ->
-    {timeout, ?TIMEOUT_EUNIT, ?_test(populate_db(Source, ?DOCS_COUNT))}.
+teardown({AttCfg, Source, Target}) ->
+    couch_replicator_test_helper:delete_db(Source),
+    couch_replicator_test_helper:delete_db(Target),
+    case AttCfg of
+        undefined ->
+            config:delete("attachments", "compressible_types", false);
+        _ ->
+            config:set("attachments", "compressible_types", AttCfg)
+    end.
 
-should_replicate({remote, Source}, Target) ->
-    should_replicate(db_url(Source), Target);
-should_replicate(Source, {remote, Target}) ->
-    should_replicate(Source, db_url(Target));
-should_replicate(Source, Target) ->
-    {timeout, ?TIMEOUT_EUNIT, ?_test(replicate(Source, Target))}.
 
-should_compare_databases({remote, Source}, Target) ->
-    should_compare_databases(Source, Target);
-should_compare_databases(Source, {remote, Target}) ->
-    should_compare_databases(Source, Target);
-should_compare_databases(Source, Target) ->
-    {timeout, ?TIMEOUT_EUNIT, ?_test(compare_dbs(Source, Target))}.
+should_replicate_attachments({_AttCfg, Source, Target}) ->
+    ?assertMatch({ok, _},
+        couch_replicator_test_helper:replicate(Source, Target)),
+    ?assertEqual(ok, couch_replicator_test_helper:compare_dbs(Source, Target)).
 
 
 populate_db(DbName, DocCount) ->
-    {ok, Db} = couch_db:open_int(DbName, []),
-    Docs = lists:foldl(
-        fun(DocIdCounter, Acc) ->
-            Doc = #doc{
-                id = iolist_to_binary(["doc", integer_to_list(DocIdCounter)]),
-                body = {[]},
-                atts = [
-                    att(<<"att1">>, ?ATT_SIZE_1, <<"text/plain">>),
-                    att(<<"att2">>, ?ATT_SIZE_2, <<"app/binary">>)
-                ]
-            },
-            [Doc | Acc]
-        end,
-        [], lists:seq(1, DocCount)),
-    {ok, _} = couch_db:update_docs(Db, Docs, []),
-    couch_db:close(Db).
+    Docs = lists:foldl(fun(DocIdCounter, Acc) ->
+        Doc = #doc{
+            id = iolist_to_binary(["doc", integer_to_list(DocIdCounter)]),
+            body = {[]},
+            atts = [
+                att(<<"att1">>, ?ATT_SIZE_1, <<"text/plain">>),
+                att(<<"att2">>, ?ATT_SIZE_2, <<"app/binary">>)
+            ]
+        },
+        [Doc | Acc]
+    end, [], lists:seq(1, DocCount)),
+    couch_replicator_test_helper:create_docs(DbName, Docs).
+
 
 att(Name, Size, Type) ->
     couch_att:new([
diff --git a/src/couch_replicator/test/eunit/couch_replicator_many_leaves_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_many_leaves_tests.erl
index c7933b4..3dbfa6a 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_many_leaves_tests.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_many_leaves_tests.erl
@@ -14,11 +14,8 @@
 
 -include_lib("couch/include/couch_eunit.hrl").
 -include_lib("couch/include/couch_db.hrl").
+-include_lib("fabric/test/fabric2_test.hrl").
 
--import(couch_replicator_test_helper, [
-    db_url/1,
-    replicate/2
-]).
 
 -define(DOCS_CONFLICTS, [
     {<<"doc1">>, 10},
@@ -28,178 +25,150 @@
     {<<"doc3">>, 210}
 ]).
 -define(NUM_ATTS, 2).
--define(TIMEOUT_EUNIT, 60).
 -define(i2l(I), integer_to_list(I)).
 -define(io2b(Io), iolist_to_binary(Io)).
 
-setup() ->
-    DbName = ?tempdb(),
-    {ok, Db} = couch_db:create(DbName, [?ADMIN_CTX]),
-    ok = couch_db:close(Db),
-    DbName.
-
-
-setup(remote) ->
-    {remote, setup()};
-setup({A, B}) ->
-    Ctx = test_util:start_couch([couch_replicator]),
-    Source = setup(A),
-    Target = setup(B),
-    {Ctx, {Source, Target}}.
-
-teardown({remote, DbName}) ->
-    teardown(DbName);
-teardown(DbName) ->
-    ok = couch_server:delete(DbName, [?ADMIN_CTX]),
-    ok.
-
-teardown(_, {Ctx, {Source, Target}}) ->
-    teardown(Source),
-    teardown(Target),
-    ok = application:stop(couch_replicator),
-    ok = test_util:stop_couch(Ctx).
 
 docs_with_many_leaves_test_() ->
-    Pairs = [{remote, remote}],
     {
         "Replicate documents with many leaves",
         {
-            foreachx,
-            fun setup/1, fun teardown/2,
-            [{Pair, fun should_populate_replicate_compact/2}
-             || Pair <- Pairs]
+            setup,
+            fun couch_replicator_test_helper:start_couch/0,
+            fun couch_replicator_test_helper:stop_couch/1,
+            {
+                foreach,
+                fun setup/0,
+                fun teardown/1,
+                [
+                    ?TDEF_FE(should_replicate_doc_with_many_leaves, 180)
+                ]
+            }
         }
     }.
 
 
-should_populate_replicate_compact({From, To}, {_Ctx, {Source, Target}}) ->
-    {lists:flatten(io_lib:format("~p -> ~p", [From, To])),
-     {inorder, [
-        should_populate_source(Source),
-        should_replicate(Source, Target),
-        should_verify_target(Source, Target),
-        should_add_attachments_to_source(Source),
-        should_replicate(Source, Target),
-        should_verify_target(Source, Target)
-     ]}}.
-
-should_populate_source({remote, Source}) ->
-    should_populate_source(Source);
-should_populate_source(Source) ->
-    {timeout, ?TIMEOUT_EUNIT, ?_test(populate_db(Source))}.
-
-should_replicate({remote, Source}, Target) ->
-    should_replicate(db_url(Source), Target);
-should_replicate(Source, {remote, Target}) ->
-    should_replicate(Source, db_url(Target));
-should_replicate(Source, Target) ->
-    {timeout, ?TIMEOUT_EUNIT, ?_test(replicate(Source, Target))}.
-
-should_verify_target({remote, Source}, Target) ->
-    should_verify_target(Source, Target);
-should_verify_target(Source, {remote, Target}) ->
-    should_verify_target(Source, Target);
-should_verify_target(Source, Target) ->
-    {timeout, ?TIMEOUT_EUNIT, ?_test(begin
-        {ok, SourceDb} = couch_db:open_int(Source, []),
-        {ok, TargetDb} = couch_db:open_int(Target, []),
-        verify_target(SourceDb, TargetDb, ?DOCS_CONFLICTS),
-        ok = couch_db:close(SourceDb),
-        ok = couch_db:close(TargetDb)
-    end)}.
-
-should_add_attachments_to_source({remote, Source}) ->
-    should_add_attachments_to_source(Source);
-should_add_attachments_to_source(Source) ->
-    {timeout, ?TIMEOUT_EUNIT, ?_test(begin
-        {ok, SourceDb} = couch_db:open_int(Source, [?ADMIN_CTX]),
-        add_attachments(SourceDb, ?NUM_ATTS, ?DOCS_CONFLICTS),
-        ok = couch_db:close(SourceDb)
-    end)}.
+setup() ->
+    Source = couch_replicator_test_helper:create_db(),
+    populate_db(Source),
+    Target = couch_replicator_test_helper:create_db(),
+    {Source, Target}.
+
+
+teardown({Source, Target}) ->
+    couch_replicator_test_helper:delete_db(Source),
+    couch_replicator_test_helper:delete_db(Target).
+
+
+should_replicate_doc_with_many_leaves({Source, Target}) ->
+    replicate(Source, Target),
+    {ok, SourceDb} = fabric2_db:open(Source, [?ADMIN_CTX]),
+    {ok, TargetDb} = fabric2_db:open(Target, [?ADMIN_CTX]),
+    verify_target(SourceDb, TargetDb, ?DOCS_CONFLICTS),
+    add_attachments(SourceDb, ?NUM_ATTS, ?DOCS_CONFLICTS),
+    replicate(Source, Target),
+    verify_target(SourceDb, TargetDb, ?DOCS_CONFLICTS).
+
 
 populate_db(DbName) ->
-    {ok, Db} = couch_db:open_int(DbName, [?ADMIN_CTX]),
-    lists:foreach(
-       fun({DocId, NumConflicts}) ->
-            Value = <<"0">>,
-            Doc = #doc{
-                id = DocId,
-                body = {[ {<<"value">>, Value} ]}
-            },
-            {ok, _} = couch_db:update_doc(Db, Doc, [?ADMIN_CTX]),
-            {ok, _} = add_doc_siblings(Db, DocId, NumConflicts)
-        end, ?DOCS_CONFLICTS),
-    couch_db:close(Db).
-
-add_doc_siblings(Db, DocId, NumLeaves) when NumLeaves > 0 ->
+    {ok, Db} = fabric2_db:open(DbName, [?ADMIN_CTX]),
+    lists:foreach(fun({DocId, NumConflicts}) ->
+        Doc = #doc{
+            id = DocId,
+            body = {[{<<"value">>, <<"0">>}]}
+        },
+        {ok, _} = fabric2_db:update_doc(Db, Doc),
+        {ok, _} = add_doc_siblings(Db, DocId, NumConflicts)
+    end, ?DOCS_CONFLICTS).
+
+
+add_doc_siblings(#{} = Db, DocId, NumLeaves) when NumLeaves > 0 ->
     add_doc_siblings(Db, DocId, NumLeaves, [], []).
 
-add_doc_siblings(Db, _DocId, 0, AccDocs, AccRevs) ->
-    {ok, []} = couch_db:update_docs(Db, AccDocs, [], replicated_changes),
+
+add_doc_siblings(#{} = Db, _DocId, 0, AccDocs, AccRevs) ->
+    {ok, []} = fabric2_db:update_docs(Db, AccDocs, [replicated_changes]),
     {ok, AccRevs};
 
-add_doc_siblings(Db, DocId, NumLeaves, AccDocs, AccRevs) ->
+add_doc_siblings(#{} = Db, DocId, NumLeaves, AccDocs, AccRevs) ->
     Value = ?l2b(?i2l(NumLeaves)),
     Rev = couch_hash:md5_hash(Value),
     Doc = #doc{
         id = DocId,
         revs = {1, [Rev]},
-        body = {[ {<<"value">>, Value} ]}
+        body = {[{<<"value">>, Value}]}
     },
     add_doc_siblings(Db, DocId, NumLeaves - 1,
-                     [Doc | AccDocs], [{1, Rev} | AccRevs]).
+        [Doc | AccDocs], [{1, Rev} | AccRevs]).
+
 
 verify_target(_SourceDb, _TargetDb, []) ->
     ok;
-verify_target(SourceDb, TargetDb, [{DocId, NumConflicts} | Rest]) ->
-    {ok, SourceLookups} = couch_db:open_doc_revs(
-        SourceDb,
-        DocId,
-        all,
-        [conflicts, deleted_conflicts]),
-    {ok, TargetLookups} = couch_db:open_doc_revs(
-        TargetDb,
-        DocId,
-        all,
-        [conflicts, deleted_conflicts]),
+
+verify_target(#{} = SourceDb, #{} = TargetDb,
+        [{DocId, NumConflicts} | Rest]) ->
+    Opts = [conflicts, deleted_conflicts],
+    {ok, SourceLookups} = open_doc_revs(SourceDb, DocId, Opts),
+    {ok, TargetLookups} = open_doc_revs(TargetDb, DocId, Opts),
     SourceDocs = [Doc || {ok, Doc} <- SourceLookups],
     TargetDocs = [Doc || {ok, Doc} <- TargetLookups],
     Total = NumConflicts + 1,
     ?assertEqual(Total, length(TargetDocs)),
-    lists:foreach(
-        fun({SourceDoc, TargetDoc}) ->
-            SourceJson = couch_doc:to_json_obj(SourceDoc, [attachments]),
-            TargetJson = couch_doc:to_json_obj(TargetDoc, [attachments]),
-            ?assertEqual(SourceJson, TargetJson)
-        end,
-        lists:zip(SourceDocs, TargetDocs)),
+    lists:foreach(fun({SourceDoc, TargetDoc}) ->
+        ?assertEqual(json_doc(SourceDoc), json_doc(TargetDoc))
+    end, lists:zip(SourceDocs, TargetDocs)),
     verify_target(SourceDb, TargetDb, Rest).
 
-add_attachments(_SourceDb, _NumAtts,  []) ->
+
+add_attachments(_SourceDb, _NumAtts, []) ->
     ok;
-add_attachments(SourceDb, NumAtts,  [{DocId, NumConflicts} | Rest]) ->
-    {ok, SourceLookups} = couch_db:open_doc_revs(SourceDb, DocId, all, []),
+
+add_attachments(#{} = SourceDb, NumAtts,
+        [{DocId, NumConflicts} | Rest]) ->
+    {ok, SourceLookups} = open_doc_revs(SourceDb, DocId, []),
     SourceDocs = [Doc || {ok, Doc} <- SourceLookups],
     Total = NumConflicts + 1,
     ?assertEqual(Total, length(SourceDocs)),
-    NewDocs = lists:foldl(
-        fun(#doc{atts = Atts, revs = {Pos, [Rev | _]}} = Doc, Acc) ->
+    NewDocs = lists:foldl(fun
+        (#doc{atts = Atts, revs = {Pos, [Rev | _]}} = Doc, Acc) ->
             NewAtts = lists:foldl(fun(I, AttAcc) ->
-                AttData = crypto:strong_rand_bytes(100),
-                NewAtt = couch_att:new([
-                    {name, ?io2b(["att_", ?i2l(I), "_",
-                        couch_doc:rev_to_str({Pos, Rev})])},
-                    {type, <<"application/foobar">>},
-                    {att_len, byte_size(AttData)},
-                    {data, AttData}
-                ]),
-                [NewAtt | AttAcc]
+                [att(I, {Pos, Rev}, 100) | AttAcc]
             end, [], lists:seq(1, NumAtts)),
             [Doc#doc{atts = Atts ++ NewAtts} | Acc]
-        end,
-        [], SourceDocs),
-    {ok, UpdateResults} = couch_db:update_docs(SourceDb, NewDocs, []),
-    NewRevs = [R || {ok, R} <- UpdateResults],
-    ?assertEqual(length(NewDocs), length(NewRevs)),
+    end, [], SourceDocs),
+    lists:foreach(fun(#doc{} = Doc) ->
+        ?assertMatch({ok, _}, fabric2_db:update_doc(SourceDb, Doc))
+    end, NewDocs),
     add_attachments(SourceDb, NumAtts, Rest).
 
+
+att(I, PosRev, Size) ->
+    Name =  ?io2b(["att_", ?i2l(I), "_", couch_doc:rev_to_str(PosRev)]),
+    AttData = crypto:strong_rand_bytes(Size),
+    couch_att:new([
+        {name, Name},
+        {type, <<"application/foobar">>},
+        {att_len, byte_size(AttData)},
+        {data, AttData}
+    ]).
+
+
+open_doc_revs(#{} = Db, DocId, Opts) ->
+    fabric2_db:open_doc_revs(Db, DocId, all, Opts).
+
+
+json_doc(#doc{} = Doc) ->
+    couch_doc:to_json_obj(Doc, [attachments]).
+
+
+replicate(Source, Target) ->
+    % Serialize the concurrent updates of the same document in order
+    % to prevent having to set higher timeouts due to FDB conflicts
+    RepObject = #{
+        <<"source">> => Source,
+        <<"target">> => Target,
+        <<"worker_processes">> => 1,
+        <<"http_connections">> => 1
+    },
+    ?assertMatch({ok, _},
+        couch_replicator_test_helper:replicate(RepObject)).
diff --git a/src/couch_replicator/test/eunit/couch_replicator_missing_stubs_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_missing_stubs_tests.erl
index ff08b5e..e672c76 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_missing_stubs_tests.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_missing_stubs_tests.erl
@@ -14,103 +14,59 @@
 
 -include_lib("couch/include/couch_eunit.hrl").
 -include_lib("couch/include/couch_db.hrl").
+-include_lib("fabric/test/fabric2_test.hrl").
 
--import(couch_replicator_test_helper, [
-    db_url/1,
-    replicate/2,
-    compare_dbs/2
-]).
 
 -define(REVS_LIMIT, 3).
--define(TIMEOUT_EUNIT, 30).
 
-setup() ->
-    DbName = ?tempdb(),
-    {ok, Db} = couch_db:create(DbName, [?ADMIN_CTX]),
-    ok = couch_db:close(Db),
-    DbName.
-
-setup(remote) ->
-    {remote, setup()};
-setup({A, B}) ->
-    Ctx = test_util:start_couch([couch_replicator]),
-    Source = setup(A),
-    Target = setup(B),
-    {Ctx, {Source, Target}}.
-
-teardown({remote, DbName}) ->
-    teardown(DbName);
-teardown(DbName) ->
-    ok = couch_server:delete(DbName, [?ADMIN_CTX]),
-    ok.
-
-teardown(_, {Ctx, {Source, Target}}) ->
-    teardown(Source),
-    teardown(Target),
-    ok = application:stop(couch_replicator),
-    ok = test_util:stop_couch(Ctx).
 
 missing_stubs_test_() ->
-    Pairs = [{remote, remote}],
     {
         "Replicate docs with missing stubs (COUCHDB-1365)",
         {
-            foreachx,
-            fun setup/1, fun teardown/2,
-            [{Pair, fun should_replicate_docs_with_missed_att_stubs/2}
-             || Pair <- Pairs]
+            setup,
+            fun couch_replicator_test_helper:start_couch/0,
+            fun couch_replicator_test_helper:stop_couch/1,
+            {
+                foreach,
+                fun setup/0,
+                fun teardown/1,
+                [
+                    ?TDEF_FE(should_replicate_docs_with_missed_att_stubs, 60)
+                ]
+            }
         }
     }.
 
 
-should_replicate_docs_with_missed_att_stubs({From, To}, {_Ctx, {Source, Target}}) ->
-    {lists:flatten(io_lib:format("~p -> ~p", [From, To])),
-     {inorder, [
-        should_populate_source(Source),
-        should_set_target_revs_limit(Target, ?REVS_LIMIT),
-        should_replicate(Source, Target),
-        should_compare_databases(Source, Target),
-        should_update_source_docs(Source, ?REVS_LIMIT * 2),
-        should_replicate(Source, Target),
-        should_compare_databases(Source, Target)
-     ]}}.
-
-should_populate_source({remote, Source}) ->
-    should_populate_source(Source);
-should_populate_source(Source) ->
-    {timeout, ?TIMEOUT_EUNIT, ?_test(populate_db(Source))}.
-
-should_replicate({remote, Source}, Target) ->
-    should_replicate(db_url(Source), Target);
-should_replicate(Source, {remote, Target}) ->
-    should_replicate(Source, db_url(Target));
-should_replicate(Source, Target) ->
-    {timeout, ?TIMEOUT_EUNIT, ?_test(replicate(Source, Target))}.
-
-should_set_target_revs_limit({remote, Target}, RevsLimit) ->
-    should_set_target_revs_limit(Target, RevsLimit);
-should_set_target_revs_limit(Target, RevsLimit) ->
-    ?_test(begin
-        {ok, Db} = couch_db:open_int(Target, [?ADMIN_CTX]),
-        ?assertEqual(ok, couch_db:set_revs_limit(Db, RevsLimit)),
-        ok = couch_db:close(Db)
-    end).
-
-should_compare_databases({remote, Source}, Target) ->
-    should_compare_databases(Source, Target);
-should_compare_databases(Source, {remote, Target}) ->
-    should_compare_databases(Source, Target);
-should_compare_databases(Source, Target) ->
-    {timeout, ?TIMEOUT_EUNIT, ?_test(compare_dbs(Source, Target))}.
-
-should_update_source_docs({remote, Source}, Times) ->
-    should_update_source_docs(Source, Times);
-should_update_source_docs(Source, Times) ->
-    {timeout, ?TIMEOUT_EUNIT, ?_test(update_db_docs(Source, Times))}.
+setup() ->
+    Source = couch_replicator_test_helper:create_db(),
+    populate_db(Source),
+    Target = couch_replicator_test_helper:create_db(),
+    {Source, Target}.
+
+
+teardown({Source, Target}) ->
+    couch_replicator_test_helper:delete_db(Source),
+    couch_replicator_test_helper:delete_db(Target).
+
+
+should_replicate_docs_with_missed_att_stubs({Source, Target}) ->
+    {ok, TargetDb} = fabric2_db:open(Target, [?ADMIN_CTX]),
+    ?assertEqual(ok, fabric2_db:set_revs_limit(TargetDb, ?REVS_LIMIT)),
+
+    ?assertMatch({ok, _},
+        couch_replicator_test_helper:replicate(Source, Target)),
+    ?assertEqual(ok, couch_replicator_test_helper:compare_dbs(Source, Target)),
+
+    ok = update_db_docs(Source, ?REVS_LIMIT * 2),
+
+    ?assertMatch({ok, _},
+        couch_replicator_test_helper:replicate(Source, Target)),
+    ?assertEqual(ok, couch_replicator_test_helper:compare_dbs(Source, Target)).
 
 
 populate_db(DbName) ->
-    {ok, Db} = couch_db:open_int(DbName, []),
     AttData = crypto:strong_rand_bytes(6000),
     Doc = #doc{
         id = <<"doc1">>,
@@ -120,35 +76,40 @@ populate_db(DbName) ->
                 {type, <<"application/foobar">>},
                 {att_len, byte_size(AttData)},
                 {data, AttData}
-           ])
+            ])
         ]
     },
-    {ok, _} = couch_db:update_doc(Db, Doc, []),
-    couch_db:close(Db).
+    couch_replicator_test_helper:create_docs(DbName, [Doc]).
+
 
 update_db_docs(DbName, Times) ->
-    {ok, Db} = couch_db:open_int(DbName, []),
-    {ok, _} = couch_db:fold_docs(
-        Db,
-        fun(FDI, Acc) -> db_fold_fun(FDI, Acc) end,
-        {DbName, Times},
-        []),
-    ok = couch_db:close(Db).
-
-db_fold_fun(FullDocInfo, {DbName, Times}) ->
-    {ok, Db} = couch_db:open_int(DbName, []),
-    {ok, Doc} = couch_db:open_doc(Db, FullDocInfo),
-    lists:foldl(
-        fun(_, {Pos, RevId}) ->
-            {ok, Db2} = couch_db:reopen(Db),
-            NewDocVersion = Doc#doc{
-                revs = {Pos, [RevId]},
-                body = {[{<<"value">>, base64:encode(crypto:strong_rand_bytes(100))}]}
-            },
-            {ok, NewRev} = couch_db:update_doc(Db2, NewDocVersion, []),
-            NewRev
-        end,
-        {element(1, Doc#doc.revs), hd(element(2, Doc#doc.revs))},
-        lists:seq(1, Times)),
-    ok = couch_db:close(Db),
-    {ok, {DbName, Times}}.
+    {ok, Db} = fabric2_db:open(DbName, [?ADMIN_CTX]),
+    FoldFun = fun
+        ({meta, _Meta}, Acc) ->
+            {ok, Acc};
+        (complete, Acc) ->
+            {ok, Acc};
+        ({row, Row}, Acc) ->
+            {_, DocId} = lists:keyfind(id, 1, Row),
+            ok = update_doc(DbName, DocId, Times),
+            {ok, Acc}
+    end,
+    Opts = [{restart_tx, true}],
+    {ok, _} = fabric2_db:fold_docs(Db, FoldFun, ok, Opts),
+    ok.
+
+
+update_doc(_DbName, _DocId, 0) ->
+    ok;
+
+update_doc(DbName, DocId, Times) ->
+    {ok, Db} = fabric2_db:open(DbName, [?ADMIN_CTX]),
+    {ok, Doc} = fabric2_db:open_doc(Db, DocId, []),
+    #doc{revs = {Pos, [Rev | _]}} = Doc,
+    Val = base64:encode(crypto:strong_rand_bytes(100)),
+    Doc1 = Doc#doc{
+        revs = {Pos, [Rev]},
+        body = {[{<<"value">>, Val}]}
+    },
+    {ok, _} = fabric2_db:update_doc(Db, Doc1),
+    update_doc(DbName, DocId, Times - 1).
diff --git a/src/couch_replicator/test/eunit/couch_replicator_proxy_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_proxy_tests.erl
index da46b8a..f5e745d 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_proxy_tests.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_proxy_tests.erl
@@ -14,15 +14,7 @@
 
 -include_lib("couch/include/couch_eunit.hrl").
 -include_lib("couch_replicator/src/couch_replicator.hrl").
--include_lib("couch_replicator/include/couch_replicator_api_wrap.hrl").
-
-
-setup() ->
-    ok.
-
-
-teardown(_) ->
-    ok.
+-include_lib("fabric/test/fabric2_test.hrl").
 
 
 replicator_proxy_test_() ->
@@ -30,87 +22,78 @@ 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,
-                    fun parse_rep_source_target_proxy/1,
-                    fun mutually_exclusive_proxy_and_source_proxy/1,
-                    fun mutually_exclusive_proxy_and_target_proxy/1
-                ]
-            }
+            fun couch_replicator_test_helper:start_couch/0,
+            fun couch_replicator_test_helper:stop_couch/1,
+            with([
+                ?TDEF(parse_rep_doc_without_proxy),
+                ?TDEF(parse_rep_doc_with_proxy),
+                ?TDEF(parse_rep_source_target_proxy),
+                ?TDEF(mutually_exclusive_proxy_and_source_proxy),
+                ?TDEF(mutually_exclusive_proxy_and_target_proxy)
+            ])
         }
     }.
 
 
 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).
+    NoProxyDoc = {[
+        {<<"source">>, <<"http://unproxied.com">>},
+        {<<"target">>, <<"http://otherunproxied.com">>}
+    ]},
+    Rep = couch_replicator_parse:parse_rep_doc(NoProxyDoc),
+    Src = maps:get(?SOURCE, Rep),
+    Tgt = maps:get(?TARGET, Rep),
+    ?assertEqual(null, maps:get(<<"proxy_url">>, Src)),
+    ?assertEqual(null, maps:get(<<"proxy_url">>, Tgt)).
 
 
 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).
+    ProxyURL = <<"http://myproxy.com">>,
+    ProxyDoc = {[
+        {<<"source">>, <<"http://unproxied.com">>},
+        {<<"target">>, <<"http://otherunproxied.com">>},
+        {<<"proxy">>, ProxyURL}
+    ]},
+    Rep = couch_replicator_parse:parse_rep_doc(ProxyDoc),
+    Src = maps:get(?SOURCE, Rep),
+    Tgt = maps:get(?TARGET, Rep),
+    ?assertEqual(ProxyURL, maps:get(<<"proxy_url">>, Src)),
+    ?assertEqual(ProxyURL, maps:get(<<"proxy_url">>, Tgt)).
 
 
 parse_rep_source_target_proxy(_) ->
-    ?_test(begin
-        SrcProxyURL = <<"http://mysrcproxy.com">>,
-        TgtProxyURL = <<"http://mytgtproxy.com:9999">>,
-        ProxyDoc = {[
-            {<<"source">>, <<"http://unproxied.com">>},
-            {<<"target">>, <<"http://otherunproxied.com">>},
-            {<<"source_proxy">>, SrcProxyURL},
-            {<<"target_proxy">>, TgtProxyURL}
-        ]},
-        Rep = couch_replicator_docs:parse_rep_doc(ProxyDoc),
-        ?assertEqual((Rep#rep.source)#httpdb.proxy_url,
-            binary_to_list(SrcProxyURL)),
-        ?assertEqual((Rep#rep.target)#httpdb.proxy_url,
-            binary_to_list(TgtProxyURL))
-    end).
+    SrcProxyURL = <<"http://mysrcproxy.com">>,
+    TgtProxyURL = <<"http://mytgtproxy.com:9999">>,
+    ProxyDoc = {[
+        {<<"source">>, <<"http://unproxied.com">>},
+        {<<"target">>, <<"http://otherunproxied.com">>},
+        {<<"source_proxy">>, SrcProxyURL},
+        {<<"target_proxy">>, TgtProxyURL}
+    ]},
+    Rep = couch_replicator_parse:parse_rep_doc(ProxyDoc),
+    Src = maps:get(?SOURCE, Rep),
+    Tgt = maps:get(?TARGET, Rep),
+    ?assertEqual(SrcProxyURL, maps:get(<<"proxy_url">>, Src)),
+    ?assertEqual(TgtProxyURL, maps:get(<<"proxy_url">>, Tgt)).
 
 
 mutually_exclusive_proxy_and_source_proxy(_) ->
-    ?_test(begin
-        ProxyDoc = {[
-            {<<"source">>, <<"http://unproxied.com">>},
-            {<<"target">>, <<"http://otherunproxied.com">>},
-            {<<"proxy">>, <<"oldstyleproxy.local">>},
-            {<<"source_proxy">>, <<"sourceproxy.local">>}
-        ]},
-        ?assertThrow({bad_rep_doc, _},
-            couch_replicator_docs:parse_rep_doc(ProxyDoc))
-    end).
+    ProxyDoc = {[
+        {<<"source">>, <<"http://unproxied.com">>},
+        {<<"target">>, <<"http://otherunproxied.com">>},
+        {<<"proxy">>, <<"oldstyleproxy.local">>},
+        {<<"source_proxy">>, <<"sourceproxy.local">>}
+    ]},
+    ?assertThrow({bad_rep_doc, _},
+        couch_replicator_parse:parse_rep_doc(ProxyDoc)).
 
 
 mutually_exclusive_proxy_and_target_proxy(_) ->
-    ?_test(begin
-        ProxyDoc = {[
-            {<<"source">>, <<"http://unproxied.com">>},
-            {<<"target">>, <<"http://otherunproxied.com">>},
-            {<<"proxy">>, <<"oldstyleproxy.local">>},
-            {<<"target_proxy">>, <<"targetproxy.local">>}
-        ]},
-        ?assertThrow({bad_rep_doc, _},
-            couch_replicator_docs:parse_rep_doc(ProxyDoc))
-    end).
+    ProxyDoc = {[
+        {<<"source">>, <<"http://unproxied.com">>},
+        {<<"target">>, <<"http://otherunproxied.com">>},
+        {<<"proxy">>, <<"oldstyleproxy.local">>},
+        {<<"target_proxy">>, <<"targetproxy.local">>}
+    ]},
+    ?assertThrow({bad_rep_doc, _},
+        couch_replicator_parse:parse_rep_doc(ProxyDoc)).
diff --git a/src/couch_replicator/test/eunit/couch_replicator_rate_limiter_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_rate_limiter_tests.erl
index 034550a..fb98920 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_rate_limiter_tests.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_rate_limiter_tests.erl
@@ -1,6 +1,7 @@
 -module(couch_replicator_rate_limiter_tests).
 
 -include_lib("couch/include/couch_eunit.hrl").
+-include_lib("fabric/test/fabric2_test.hrl").
 
 
 rate_limiter_test_() ->
@@ -9,64 +10,52 @@ rate_limiter_test_() ->
         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()
+            ?TDEF_FE(t_new_key),
+            ?TDEF_FE(t_1_failure),
+            ?TDEF_FE(t_2_failures_back_to_back),
+            ?TDEF_FE(t_2_failures),
+            ?TDEF_FE(t_success_threshold),
+            ?TDEF_FE(t_1_failure_2_successes)
         ]
     }.
 
 
-t_new_key() ->
-    ?_test(begin
-        ?assertEqual(0, couch_replicator_rate_limiter:interval({"foo", get}))
-    end).
+t_new_key(_) ->
+    ?assertEqual(0, couch_replicator_rate_limiter:interval({"foo", get})).
 
 
-t_1_failure() ->
-    ?_test(begin
-        ?assertEqual(24, couch_replicator_rate_limiter:failure({"foo", get}))
-    end).
+t_1_failure(_) ->
+    ?assertEqual(24, couch_replicator_rate_limiter:failure({"foo", get})).
 
 
-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(_) ->
+    couch_replicator_rate_limiter:failure({"foo", get}),
+    low_pass_filter_delay(),
+    Interval = couch_replicator_rate_limiter:failure({"foo", get}),
+    ?assertEqual(29, Interval).
 
 
-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_2_failures_back_to_back(_) ->
+    couch_replicator_rate_limiter:failure({"foo", get}),
+    Interval = couch_replicator_rate_limiter:failure({"foo", get}),
+    ?assertEqual(24, Interval).
 
 
-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_success_threshold(_) ->
+    Interval = couch_replicator_rate_limiter:success({"foo", get}),
+    ?assertEqual(0, Interval),
+    Interval = couch_replicator_rate_limiter:success({"foo", get}),
+    ?assertEqual(0, Interval).
 
 
-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).
+t_1_failure_2_successes(_) ->
+    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).
 
 
 low_pass_filter_delay() ->
diff --git a/src/couch_replicator/test/eunit/couch_replicator_retain_stats_between_job_runs.erl b/src/couch_replicator/test/eunit/couch_replicator_retain_stats_between_job_runs.erl
index 037f371..4b7c37d 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_retain_stats_between_job_runs.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_retain_stats_between_job_runs.erl
@@ -15,139 +15,72 @@
 -include_lib("couch/include/couch_eunit.hrl").
 -include_lib("couch/include/couch_db.hrl").
 -include_lib("couch_replicator/src/couch_replicator.hrl").
-
--define(DELAY, 500).
--define(TIMEOUT, 60000).
-
-
-setup_all() ->
-    test_util:start_couch([couch_replicator, chttpd, mem3, fabric]).
-
-
-teardown_all(Ctx) ->
-    ok = test_util:stop_couch(Ctx).
-
-
-setup() ->
-    Source = setup_db(),
-    Target = setup_db(),
-    {Source, Target}.
+-include_lib("fabric/test/fabric2_test.hrl").
 
 
-teardown({Source, Target}) ->
-    teardown_db(Source),
-    teardown_db(Target),
-    ok.
+-define(DELAY, 500).
 
 
 stats_retained_test_() ->
     {
         setup,
-        fun setup_all/0,
-        fun teardown_all/1,
+        fun couch_replicator_test_helper:start_couch/0,
+        fun couch_replicator_test_helper:stop_couch/1,
         {
             foreach,
             fun setup/0,
             fun teardown/1,
             [
-                fun t_stats_retained_by_scheduler/1,
-                fun t_stats_retained_on_job_removal/1
+                ?TDEF_FE(t_stats_retained_on_job_removal, 60)
             ]
         }
     }.
 
 
-t_stats_retained_by_scheduler({Source, Target}) ->
-    ?_test(begin
-        {ok, _} = add_vdu(Target),
-        populate_db_reject_even_docs(Source, 1, 10),
-        {ok, RepPid, RepId} = replicate(Source, Target),
-        wait_target_in_sync(6, Target),
-
-        check_active_tasks(10, 5, 5),
-        check_scheduler_jobs(10, 5, 5),
+setup() ->
+    Source = couch_replicator_test_helper:create_db(),
+    Target = couch_replicator_test_helper:create_db(),
+    config:set("replicator", "stats_update_interval_sec", "0", false),
+    config:set("replicator", "checkpoint_interval", "1000", false),
+    {Source, Target}.
 
-        stop_job(RepPid),
-        check_scheduler_jobs(10, 5, 5),
 
-        start_job(),
-        check_active_tasks(10, 5, 5),
-        check_scheduler_jobs(10, 5, 5),
-        couch_replicator_scheduler:remove_job(RepId)
-    end).
+teardown({Source, Target}) ->
+    config:delete("replicator", "stats_update_interval_sec", false),
+    config:delete("replicator", "checkpoint_interval", false),
+    couch_replicator_test_helper:delete_db(Source),
+    couch_replicator_test_helper:delete_db(Target).
 
 
 t_stats_retained_on_job_removal({Source, Target}) ->
-    ?_test(begin
-        {ok, _} = add_vdu(Target),
-        populate_db_reject_even_docs(Source, 1, 10),
-        {ok, _, RepId} = replicate(Source, Target),
-        wait_target_in_sync(6, Target),  % 5 + 1 vdu
-
-        check_active_tasks(10, 5, 5),
-        check_scheduler_jobs(10, 5, 5),
+    {ok, _} = add_vdu(Target),
+    populate_db_reject_even_docs(Source, 1, 10),
+    {ok, Pid1, RepId} = replicate(Source, Target),
+    wait_target_in_sync(6, Target), % 5 + 1 vdu
 
-        couch_replicator_scheduler:remove_job(RepId),
+    check_scheduler_jobs(10, 5, 5),
 
-        populate_db_reject_even_docs(Source, 11, 20),
-        {ok, _, RepId} = replicate(Source, Target),
-        wait_target_in_sync(11, Target),  % 6 + 5
+    cancel(RepId, Pid1),
 
-        check_scheduler_jobs(20, 10, 10),
-        check_active_tasks(20, 10, 10),
+    populate_db_reject_even_docs(Source, 11, 20),
+    {ok, Pid2, RepId} = replicate(Source, Target),
+    wait_target_in_sync(11, Target), % 6 + 5
 
-        couch_replicator_scheduler:remove_job(RepId),
+    check_scheduler_jobs(20, 10, 10),
 
-        populate_db_reject_even_docs(Source, 21, 30),
-        {ok, _, RepId} = replicate(Source, Target),
-        wait_target_in_sync(16, Target),  % 11 + 5
+    cancel(RepId, Pid2),
 
-        check_scheduler_jobs(30, 15, 15),
-        check_active_tasks(30, 15, 15),
-
-        couch_replicator_scheduler:remove_job(RepId)
-    end).
+    populate_db_reject_even_docs(Source, 21, 30),
+    {ok, Pid3, RepId} = replicate(Source, Target),
+    wait_target_in_sync(16, Target), % 11 + 5
 
+    check_scheduler_jobs(30, 15, 15),
 
-setup_db() ->
-    DbName = ?tempdb(),
-    {ok, Db} = couch_db:create(DbName, [?ADMIN_CTX]),
-    ok = couch_db:close(Db),
-    DbName.
-
-
-teardown_db(DbName) ->
-    ok = couch_server:delete(DbName, [?ADMIN_CTX]),
-    ok.
-
-
-stop_job(RepPid) ->
-    Ref = erlang:monitor(process, RepPid),
-    gen_server:cast(couch_replicator_scheduler, {set_max_jobs, 0}),
-    couch_replicator_scheduler:reschedule(),
-    receive
-        {'DOWN', Ref, _, _, _} -> ok
-    after ?TIMEOUT ->
-        erlang:error(timeout)
-    end.
-
-
-start_job() ->
-    gen_server:cast(couch_replicator_scheduler, {set_max_jobs, 500}),
-    couch_replicator_scheduler:reschedule().
-
-
-check_active_tasks(DocsRead, DocsWritten, DocsFailed) ->
-    RepTask = wait_for_task_status(),
-    ?assertNotEqual(timeout, RepTask),
-    ?assertEqual(DocsRead, couch_util:get_value(docs_read, RepTask)),
-    ?assertEqual(DocsWritten, couch_util:get_value(docs_written, RepTask)),
-    ?assertEqual(DocsFailed, couch_util:get_value(doc_write_failures,
-        RepTask)).
+    cancel(RepId, Pid3).
 
 
 check_scheduler_jobs(DocsRead, DocsWritten, DocFailed) ->
-    Info = wait_scheduler_info(),
+    Info = wait_scheduler_info(DocsRead),
     ?assert(maps:is_key(<<"changes_pending">>, Info)),
     ?assert(maps:is_key(<<"doc_write_failures">>, Info)),
     ?assert(maps:is_key(<<"docs_read">>, Info)),
@@ -161,27 +94,18 @@ check_scheduler_jobs(DocsRead, DocsWritten, DocFailed) ->
     ?assertMatch(#{<<"doc_write_failures">> := DocFailed}, Info).
 
 
-replication_tasks() ->
-    lists:filter(fun(P) ->
-        couch_util:get_value(type, P) =:= replication
-    end, couch_task_status:all()).
-
-
-wait_for_task_status() ->
+wait_scheduler_info(DocsRead) ->
     test_util:wait(fun() ->
-        case replication_tasks() of
-            [] -> wait;
-            [RepTask] -> RepTask
-        end
-    end).
-
-
-wait_scheduler_info() ->
-    test_util:wait(fun() ->
-        case scheduler_jobs() of
-            [] -> wait;
-            [#{<<"info">> := null}] -> wait;
-            [#{<<"info">> := Info}] -> Info
+        case couch_replicator_test_helper:scheduler_jobs() of
+            [] ->
+                wait;
+            [#{<<"info">> := null}] ->
+                wait;
+            [#{<<"info">> := Info}] ->
+                case Info of
+                    #{<<"docs_read">> := DocsRead} -> Info;
+                    #{} -> wait
+                end
         end
     end).
 
@@ -197,16 +121,12 @@ populate_db_reject_even_docs(DbName, Start, End) ->
 
 
 populate_db(DbName, Start, End, BodyFun) when is_function(BodyFun, 1) ->
-    {ok, Db} = couch_db:open_int(DbName, []),
-    Docs = lists:foldl(
-        fun(DocIdCounter, Acc) ->
-            Id = integer_to_binary(DocIdCounter),
-            Doc = #doc{id = Id, body = BodyFun(DocIdCounter)},
-            [Doc | Acc]
-        end,
-        [], lists:seq(Start, End)),
-    {ok, _} = couch_db:update_docs(Db, Docs, []),
-    ok = couch_db:close(Db).
+    Docs = lists:foldl(fun(DocIdCounter, Acc) ->
+        Id = integer_to_binary(DocIdCounter),
+        Doc = #doc{id = Id, body = BodyFun(DocIdCounter)},
+        [Doc | Acc]
+    end, [], lists:seq(Start, End)),
+    couch_replicator_test_helper:create_docs(DbName, Docs).
 
 
 wait_target_in_sync(DocCount, Target) when is_integer(DocCount) ->
@@ -215,14 +135,13 @@ wait_target_in_sync(DocCount, Target) when is_integer(DocCount) ->
 
 wait_target_in_sync_loop(_DocCount, _TargetName, 0) ->
     erlang:error({assertion_failed, [
-          {module, ?MODULE}, {line, ?LINE},
-          {reason, "Could not get source and target databases in sync"}
+        {module, ?MODULE}, {line, ?LINE},
+        {reason, "Could not get source and target databases in sync"}
     ]});
 
 wait_target_in_sync_loop(DocCount, TargetName, RetriesLeft) ->
-    {ok, Target} = couch_db:open_int(TargetName, []),
-    {ok, TargetInfo} = couch_db:get_db_info(Target),
-    ok = couch_db:close(Target),
+    {ok, Db} = fabric2_db:open(TargetName, [?ADMIN_CTX]),
+    {ok, TargetInfo} = fabric2_db:get_db_info(Db),
     TargetDocCount = couch_util:get_value(doc_count, TargetInfo),
     case TargetDocCount  == DocCount of
         true ->
@@ -234,27 +153,11 @@ wait_target_in_sync_loop(DocCount, TargetName, RetriesLeft) ->
 
 
 replicate(Source, Target) ->
-    SrcUrl = couch_replicator_test_helper:db_url(Source),
-    TgtUrl = couch_replicator_test_helper:db_url(Target),
-    RepObject = {[
-        {<<"source">>, SrcUrl},
-        {<<"target">>, TgtUrl},
-        {<<"continuous">>, true}
-    ]},
-    {ok, Rep} = couch_replicator_utils:parse_rep_doc(RepObject, ?ADMIN_USER),
-    ok = couch_replicator_scheduler:add_job(Rep),
-    couch_replicator_scheduler:reschedule(),
-    Pid = couch_replicator_test_helper:get_pid(Rep#rep.id),
-    {ok, Pid, Rep#rep.id}.
-
-
-scheduler_jobs() ->
-    Addr = config:get("chttpd", "bind_address", "127.0.0.1"),
-    Port = mochiweb_socket_server:get(chttpd, port),
-    Url = lists:flatten(io_lib:format("http://~s:~b/_scheduler/jobs", [Addr, Port])),
-    {ok, 200, _, Body} = test_request:get(Url, []),
-    Json = jiffy:decode(Body, [return_maps]),
-    maps:get(<<"jobs">>, Json).
+    couch_replicator_test_helper:replicate_continuous(Source, Target).
+
+
+cancel(RepId, Pid) ->
+    couch_replicator_test_helper:cancel(RepId, Pid).
 
 
 vdu() ->
@@ -274,9 +177,5 @@ add_vdu(DbName) ->
         {<<"validate_doc_update">>, vdu()}
     ],
     Doc = couch_doc:from_json_obj({DocProps}, []),
-    {ok, Db} = couch_db:open_int(DbName, [?ADMIN_CTX]),
-    try
-        {ok, _Rev} = couch_db:update_doc(Db, Doc, [])
-    after
-        couch_db:close(Db)
-    end.
+    {ok, Db} = fabric2_db:open(DbName, [?ADMIN_CTX]),
+    {ok, _} = fabric2_db:update_doc(Db, Doc, []).
diff --git a/src/couch_replicator/test/eunit/couch_replicator_selector_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_selector_tests.erl
index 5026c14..5dfe4ba 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_selector_tests.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_selector_tests.erl
@@ -15,103 +15,69 @@
 -include_lib("couch/include/couch_eunit.hrl").
 -include_lib("couch/include/couch_db.hrl").
 -include_lib("couch_replicator/src/couch_replicator.hrl").
+-include_lib("fabric/test/fabric2_test.hrl").
 
 
-setup(_) ->
-    Ctx = test_util:start_couch([couch_replicator]),
-    Source = create_db(),
-    create_docs(Source),
-    Target = create_db(),
-    {Ctx, {Source, Target}}.
-
-teardown(_, {Ctx, {Source, Target}}) ->
-    delete_db(Source),
-    delete_db(Target),
-    ok = application:stop(couch_replicator),
-    ok = test_util:stop_couch(Ctx).
-
 selector_replication_test_() ->
-    Pairs = [{remote, remote}],
     {
         "Selector filtered replication tests",
         {
-            foreachx,
-            fun setup/1, fun teardown/2,
-            [{Pair, fun should_succeed/2} || Pair <- Pairs]
+            setup,
+            fun couch_replicator_test_helper:start_couch/0,
+            fun couch_replicator_test_helper:stop_couch/1,
+            {
+                foreach,
+                fun setup/0,
+                fun teardown/1,
+                [
+                    ?TDEF_FE(should_replicate_with_selector)
+                ]
+            }
         }
     }.
 
-should_succeed({From, To}, {_Ctx, {Source, Target}}) ->
-    RepObject = {[
-        {<<"source">>, db_url(From, Source)},
-        {<<"target">>, db_url(To, Target)},
-        {<<"selector">>, {[{<<"_id">>, <<"doc2">>}]}}
-    ]},
-    {ok, _} = couch_replicator:replicate(RepObject, ?ADMIN_USER),
-    %% FilteredFun is an Erlang version of following mango selector
-    FilterFun = fun(_DocId, {Props}) ->
-        couch_util:get_value(<<"_id">>, Props) == <<"doc2">>
-    end,
-    {ok, TargetDbInfo, AllReplies} = compare_dbs(Source, Target, FilterFun),
-    {lists:flatten(io_lib:format("~p -> ~p", [From, To])), [
-        {"Target DB has proper number of docs",
-        ?_assertEqual(1, proplists:get_value(doc_count, TargetDbInfo))},
-        {"All the docs selected as expected",
-        ?_assert(lists:all(fun(Valid) -> Valid end, AllReplies))}
-    ]}.
 
-compare_dbs(Source, Target, FilterFun) ->
-    {ok, SourceDb} = couch_db:open_int(Source, []),
-    {ok, TargetDb} = couch_db:open_int(Target, []),
-    {ok, TargetDbInfo} = couch_db:get_db_info(TargetDb),
-    Fun = fun(FullDocInfo, Acc) ->
-        {ok, DocId, SourceDoc} = read_doc(SourceDb, FullDocInfo),
-        TargetReply = read_doc(TargetDb, DocId),
-        case FilterFun(DocId, SourceDoc) of
-            true ->
-                ValidReply = {ok, DocId, SourceDoc} == TargetReply,
-                {ok, [ValidReply|Acc]};
-            false ->
-                ValidReply = {not_found, missing} == TargetReply,
-                {ok, [ValidReply|Acc]}
-        end
-    end,
-    {ok, AllReplies} = couch_db:fold_docs(SourceDb, Fun, [], []),
-    ok = couch_db:close(SourceDb),
-    ok = couch_db:close(TargetDb),
-    {ok, TargetDbInfo, AllReplies}.
+setup() ->
+    Source = couch_replicator_test_helper:create_db(),
+    create_docs(Source),
+    Target = couch_replicator_test_helper:create_db(),
+    {Source, Target}.
 
-read_doc(Db, DocIdOrInfo) ->
-    case couch_db:open_doc(Db, DocIdOrInfo) of
-        {ok, Doc} ->
-            {Props} = couch_doc:to_json_obj(Doc, [attachments]),
-            DocId = couch_util:get_value(<<"_id">>, Props),
-            {ok, DocId, {Props}};
-        Error ->
-            Error
-    end.
 
-create_db() ->
-    DbName = ?tempdb(),
-    {ok, Db} = couch_db:create(DbName, [?ADMIN_CTX]),
-    ok = couch_db:close(Db),
-    DbName.
+teardown({Source, Target}) ->
+    couch_replicator_test_helper:delete_db(Source),
+    couch_replicator_test_helper:delete_db(Target).
 
-create_docs(DbName) ->
-    {ok, Db} = couch_db:open(DbName, [?ADMIN_CTX]),
-    Doc1 = couch_doc:from_json_obj({[
-        {<<"_id">>, <<"doc1">>}
-    ]}),
-    Doc2 = couch_doc:from_json_obj({[
-        {<<"_id">>, <<"doc2">>}
-    ]}),
-    {ok, _} = couch_db:update_docs(Db, [Doc1, Doc2]),
-    couch_db:close(Db).
 
-delete_db(DbName) ->
-    ok = couch_server:delete(DbName, [?ADMIN_CTX]).
+should_replicate_with_selector({Source, Target}) ->
+    RepObject = #{
+        <<"source">> => Source,
+        <<"target">> => Target,
+        <<"selector">> => #{
+            <<"_id">> => <<"doc2">>
+        }
+    },
+    ?assertMatch({ok, _}, couch_replicator_test_helper:replicate(RepObject)),
+    {ok, TargetDbInfo, AllReplies} = compare_dbs(Source, Target),
+    ?assertEqual(1, proplists:get_value(doc_count, TargetDbInfo)),
+    ?assert(lists:all(fun(Valid) -> Valid end, AllReplies)).
+
 
-db_url(remote, DbName) ->
-    Addr = config:get("httpd", "bind_address", "127.0.0.1"),
-    Port = mochiweb_socket_server:get(couch_httpd, port),
-    ?l2b(io_lib:format("http://~s:~b/~s", [Addr, Port, DbName])).
+compare_dbs(Source, Target) ->
+    {ok, TargetDb} = fabric2_db:open(Target, []),
+    {ok, TargetDbInfo} = fabric2_db:get_db_info(TargetDb),
+    Fun = fun(SrcDoc, TgtDoc, Acc) ->
+        case SrcDoc#doc.id == <<"doc2">> of
+            true -> [SrcDoc#doc.body == TgtDoc#doc.body | Acc];
+            false -> [not_found == TgtDoc | Acc]
+        end
+    end,
+    Res = couch_replicator_test_helper:compare_fold(Source, Target, Fun, []),
+    {ok, TargetDbInfo, Res}.
+
+
+create_docs(DbName) ->
+    couch_replicator_test_helper:create_docs(DbName, [
+        #{<<"_id">> => <<"doc1">>},
+        #{<<"_id">> => <<"doc2">>}
+    ]).
diff --git a/src/couch_replicator/test/eunit/couch_replicator_small_max_request_size_target.erl b/src/couch_replicator/test/eunit/couch_replicator_small_max_request_size_target.erl
index 8aebbe1..b113c53 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_small_max_request_size_target.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_small_max_request_size_target.erl
@@ -1,139 +1,70 @@
 -module(couch_replicator_small_max_request_size_target).
 
+
 -include_lib("couch/include/couch_eunit.hrl").
 -include_lib("couch/include/couch_db.hrl").
-
--import(couch_replicator_test_helper, [
-    db_url/1,
-    replicate/1,
-    compare_dbs/3
-]).
-
--define(TIMEOUT_EUNIT, 360).
-
-
-setup() ->
-    DbName = ?tempdb(),
-    {ok, Db} = couch_db:create(DbName, [?ADMIN_CTX]),
-    ok = couch_db:close(Db),
-    DbName.
-
-
-setup(remote) ->
-    {remote, setup()};
-
-setup({A, B}) ->
-    Ctx = test_util:start_couch([couch_replicator]),
-    config:set("httpd", "max_http_request_size", "10000", false),
-    Source = setup(A),
-    Target = setup(B),
-    {Ctx, {Source, Target}}.
-
-
-teardown({remote, DbName}) ->
-    teardown(DbName);
-teardown(DbName) ->
-    ok = couch_server:delete(DbName, [?ADMIN_CTX]),
-    ok.
-
-teardown(_, {Ctx, {Source, Target}}) ->
-    teardown(Source),
-    teardown(Target),
-    ok = application:stop(couch_replicator),
-    ok = test_util:stop_couch(Ctx).
+-include_lib("fabric/test/fabric2_test.hrl").
 
 
 reduce_max_request_size_test_() ->
-    Pairs = [{remote, remote}],
     {
         "Replicate docs when target has a small max_http_request_size",
         {
-            foreachx,
-            fun setup/1, fun teardown/2,
-            [{Pair, fun should_replicate_all_docs/2}
-             || Pair <- Pairs]
-            ++ [{Pair, fun should_replicate_one/2}
-             || Pair <- Pairs]
-            % Disabled. See issue 574. Sometimes PUTs with a doc and
-            % attachment which exceed maximum request size are simply
-            % closed instead of returning a 413 request. That makes these
-            % tests flaky.
-            ++ [{Pair, fun should_replicate_one_with_attachment/2}
-             || Pair <- Pairs]
+            setup,
+            fun couch_replicator_test_helper:start_couch/0,
+            fun couch_replicator_test_helper:stop_couch/1,
+            {
+                foreach,
+                fun setup/0,
+                fun teardown/1,
+                [
+                    ?TDEF_FE(should_replicate_all_docs, 120),
+                    ?TDEF_FE(should_replicate_one, 120),
+                    ?TDEF_FE(should_replicate_one_with_attachment, 120)
+                ]
+            }
         }
     }.
 
 
-% Test documents which are below max_http_request_size but when batched, batch size
-% will be greater than max_http_request_size. Replicator could automatically split
-% the batch into smaller batches and POST those separately.
-should_replicate_all_docs({From, To}, {_Ctx, {Source, Target}}) ->
-    {lists:flatten(io_lib:format("~p -> ~p", [From, To])),
-     {inorder, [should_populate_source(Source),
-                should_replicate(Source, Target),
-                should_compare_databases(Source, Target, [])]}}.
-
-
-% If a document is too large to post as a single request, that document is
-% skipped but replication overall will make progress and not crash.
-should_replicate_one({From, To}, {_Ctx, {Source, Target}}) ->
-    {lists:flatten(io_lib:format("~p -> ~p", [From, To])),
-     {inorder, [should_populate_source_one_large_one_small(Source),
-                should_replicate(Source, Target),
-                should_compare_databases(Source, Target, [<<"doc0">>])]}}.
-
-
-% If a document has an attachment > 64 * 1024 bytes, replicator will switch to
-% POST-ing individual documents directly and skip bulk_docs. Test that case
-% separately
-% See note in main test function why this was disabled.
-should_replicate_one_with_attachment({From, To}, {_Ctx, {Source, Target}}) ->
-   {lists:flatten(io_lib:format("~p -> ~p", [From, To])),
-    {inorder, [should_populate_source_one_large_attachment(Source),
-               should_populate_source(Source),
-               should_replicate(Source, Target),
-               should_compare_databases(Source, Target, [<<"doc0">>])]}}.
-
-
-should_populate_source({remote, Source}) ->
-    should_populate_source(Source);
-
-should_populate_source(Source) ->
-    {timeout, ?TIMEOUT_EUNIT, ?_test(add_docs(Source, 5, 3000, 0))}.
-
-
-should_populate_source_one_large_one_small({remote, Source}) ->
-    should_populate_source_one_large_one_small(Source);
-
-should_populate_source_one_large_one_small(Source) ->
-    {timeout, ?TIMEOUT_EUNIT, ?_test(one_large_one_small(Source, 12000, 3000))}.
-
-
-should_populate_source_one_large_attachment({remote, Source}) ->
-   should_populate_source_one_large_attachment(Source);
-
-should_populate_source_one_large_attachment(Source) ->
-  {timeout, ?TIMEOUT_EUNIT, ?_test(one_large_attachment(Source, 70000, 70000))}.
+setup() ->
+    Source = couch_replicator_test_helper:create_db(),
+    Target = couch_replicator_test_helper:create_db(),
+    config:set("httpd", "max_http_request_size", "10000", false),
+    {Source, Target}.
 
 
-should_replicate({remote, Source}, Target) ->
-    should_replicate(db_url(Source), Target);
+teardown({Source, Target}) ->
+    config:delete("httpd", "max_http_request_size", false),
+    couch_replicator_test_helper:delete_db(Source),
+    couch_replicator_test_helper:delete_db(Target).
 
-should_replicate(Source, {remote, Target}) ->
-    should_replicate(Source, db_url(Target));
 
-should_replicate(Source, Target) ->
-    {timeout, ?TIMEOUT_EUNIT, ?_test(replicate(Source, Target))}.
+% Test documents which are below max_http_request_size but when batched, batch
+% size will be greater than max_http_request_size. Replicator could
+% automatically split the batch into smaller batches and POST those separately.
+should_replicate_all_docs({Source, Target}) ->
+    ?assertEqual(ok, add_docs(Source, 5, 3000, 0)),
+    replicate(Source, Target),
+    compare_dbs(Source, Target, []).
 
 
-should_compare_databases({remote, Source}, Target, ExceptIds) ->
-    should_compare_databases(Source, Target, ExceptIds);
+% If a document is too large to post as a single request, that document is
+% skipped but replication overall will make progress and not crash.
+should_replicate_one({Source, Target}) ->
+    ?assertEqual(ok, one_large_one_small(Source, 12000, 3000)),
+    replicate(Source, Target),
+    compare_dbs(Source, Target, [<<"doc0">>]).
 
-should_compare_databases(Source, {remote, Target}, ExceptIds) ->
-    should_compare_databases(Source, Target, ExceptIds);
 
-should_compare_databases(Source, Target, ExceptIds) ->
-    {timeout, ?TIMEOUT_EUNIT, ?_test(compare_dbs(Source, Target, ExceptIds))}.
+% If a document has an attachment > 64 * 1024 bytes, replicator will switch to
+% POST-ing individual documents directly and skip bulk_docs. Test that case
+% separately See note in main test function why this was disabled.
+should_replicate_one_with_attachment({Source, Target}) ->
+    ?assertEqual(ok, one_large_attachment(Source, 70000, 70000)),
+    ?assertEqual(ok, add_docs(Source, 5, 3000, 0)),
+    replicate(Source, Target),
+    compare_dbs(Source, Target, [<<"doc0">>]).
 
 
 binary_chunk(Size) when is_integer(Size), Size > 0 ->
@@ -150,19 +81,21 @@ add_docs(DbName, DocCount, DocSize, AttSize) ->
 
 one_large_one_small(DbName, Large, Small) ->
     add_doc(DbName, <<"doc0">>, Large, 0),
-    add_doc(DbName, <<"doc1">>, Small, 0).
+    add_doc(DbName, <<"doc1">>, Small, 0),
+    ok.
 
 
 one_large_attachment(DbName, Size, AttSize) ->
-   add_doc(DbName, <<"doc0">>, Size, AttSize).
+    add_doc(DbName, <<"doc0">>, Size, AttSize),
+    ok.
 
 
 add_doc(DbName, DocId, Size, AttSize) when is_binary(DocId) ->
-     {ok, Db} = couch_db:open_int(DbName, []),
-     Doc0 = #doc{id = DocId, body = {[{<<"x">>, binary_chunk(Size)}]}},
-     Doc = Doc0#doc{atts = atts(AttSize)},
-     {ok, _} = couch_db:update_doc(Db, Doc, []),
-     couch_db:close(Db).
+    {ok, Db} = fabric2_db:open(DbName, [?ADMIN_CTX]),
+    Doc0 = #doc{id = DocId, body = {[{<<"x">>, binary_chunk(Size)}]}},
+    Doc = Doc0#doc{atts = atts(AttSize)},
+    {ok, _} = fabric2_db:update_doc(Db, Doc, []),
+    ok.
 
 
 atts(0) ->
@@ -178,8 +111,13 @@ atts(Size) ->
 
 
 replicate(Source, Target) ->
-    replicate({[
-        {<<"source">>, Source},
-        {<<"target">>, Target},
-        {<<"worker_processes">>, "1"} %  This make batch_size predictable
-    ]}).
+    ?assertMatch({ok, _}, couch_replicator_test_helper:replicate(#{
+        <<"source">> => Source,
+        <<"target">> => Target,
+        <<"worker_processes">> => 1  % This make batch_size predictable
+    })).
+
+
+compare_dbs(Source, Target, ExceptIds) ->
+    ?assertEqual(ok, couch_replicator_test_helper:compare_dbs(Source, Target,
+        ExceptIds)).
diff --git a/src/couch_replicator/test/eunit/couch_replicator_test_helper.erl b/src/couch_replicator/test/eunit/couch_replicator_test_helper.erl
index fd04091..2ac447e 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_test_helper.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_test_helper.erl
@@ -1,51 +1,166 @@
+% 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_test_helper).
 
--include_lib("couch/include/couch_eunit.hrl").
--include_lib("couch/include/couch_db.hrl").
--include_lib("couch_replicator/src/couch_replicator.hrl").
 
 -export([
+    start_couch/0,
+    stop_couch/1,
+
+    create_db/0,
+    create_db/1,
+    delete_db/1,
+
+    server_url/0,
+    db_url/1,
+
+    create_docs/2,
+
     compare_dbs/2,
     compare_dbs/3,
-    db_url/1,
-    replicate/1,
+    compare_fold/4,
+
+    compare_docs/2,
+
     get_pid/1,
-    replicate/2
+
+    replicate/1,
+    replicate/2,
+    replicate_continuous/1,
+    replicate_continuous/2,
+
+    cancel/1,
+    cancel/2,
+
+    scheduler_jobs/0
 ]).
 
 
+-include_lib("couch/include/couch_eunit.hrl").
+-include_lib("couch/include/couch_db.hrl").
+-include_lib("couch_replicator/src/couch_replicator.hrl").
+
+
+-define(USERNAME, "rep_eunit_admin").
+-define(PASSWORD, "rep_eunit_password").
+
+
+start_couch() ->
+    Ctx = test_util:start_couch([fabric, chttpd, couch_replicator]),
+    Hashed = couch_passwords:hash_admin_password(?PASSWORD),
+    ok = config:set("admins", ?USERNAME, ?b2l(Hashed), _Persist = false),
+    Ctx.
+
+
+stop_couch(Ctx) ->
+    config:delete("admins", ?USERNAME, _Persist = false),
+    test_util:stop_couch(Ctx).
+
+
+create_db() ->
+    {ok, Db} = fabric2_db:create(?tempdb(), [?ADMIN_CTX]),
+    fabric2_db:name(Db).
+
+
+create_db(DbName) when is_binary(DbName) ->
+    {ok, Db} = fabric2_db:create(DbName, [?ADMIN_CTX]),
+    fabric2_db:name(Db).
+
+
+delete_db(DbName) ->
+    try
+        ok = fabric2_db:delete(DbName, [?ADMIN_CTX])
+    catch
+        error:database_does_not_exist ->
+            ok
+    end.
+
+
+server_url() ->
+    Addr = config:get("chttpd", "bind_address", "127.0.0.1"),
+    Port = mochiweb_socket_server:get(chttpd, port),
+    Fmt = "http://~s:~s@~s:~b",
+    ?l2b(io_lib:format(Fmt, [?USERNAME, ?PASSWORD, Addr, Port])).
+
+
+db_url(DbName) ->
+    ?l2b(io_lib:format("~s/~s", [server_url(), DbName])).
+
+
+create_docs(DbName, Docs) when is_binary(DbName), is_list(Docs) ->
+    {ok, Db} = fabric2_db:open(DbName, [?ADMIN_CTX]),
+    Docs1 = lists:map(fun(Doc) ->
+        case Doc of
+            #{} ->
+                Doc1 = couch_util:json_decode(couch_util:json_encode(Doc)),
+                couch_doc:from_json_obj(Doc1);
+            #doc{} ->
+                Doc
+        end
+    end, Docs),
+    {ok, ResList} = fabric2_db:update_docs(Db, Docs1),
+    lists:foreach(fun(Res) ->
+        ?assertMatch({ok, {_, Rev}} when is_binary(Rev), Res)
+    end, ResList).
+
+
 compare_dbs(Source, Target) ->
-    compare_dbs(Source, Target, []).
-
-
-compare_dbs(Source, Target, ExceptIds) ->
-    {ok, SourceDb} = couch_db:open_int(Source, []),
-    {ok, TargetDb} = couch_db:open_int(Target, []),
-
-    Fun = fun(FullDocInfo, Acc) ->
-        {ok, DocSource} = couch_db:open_doc(SourceDb, FullDocInfo),
-        Id = DocSource#doc.id,
-        case lists:member(Id, ExceptIds) of
-            true ->
-                ?assertEqual(not_found, couch_db:get_doc_info(TargetDb, Id));
-            false ->
-                {ok, TDoc} = couch_db:open_doc(TargetDb, Id),
-                compare_docs(DocSource, TDoc)
+    Fun = fun(SrcDoc, TgtDoc, ok) -> compare_docs(SrcDoc, TgtDoc) end,
+    compare_fold(Source, Target, Fun, ok).
+
+
+compare_dbs(Source, Target, ExceptIds) when is_binary(Source),
+        is_binary(Target), is_list(ExceptIds) ->
+    Fun = fun(SrcDoc, TgtDoc, ok) ->
+        case lists:member(SrcDoc#doc.id, ExceptIds) of
+            true -> ?assertEqual(not_found, TgtDoc);
+            false -> compare_docs(SrcDoc, TgtDoc)
         end,
-        {ok, Acc}
+        ok
     end,
+    compare_fold(Source, Target, Fun, ok).
+
 
-    {ok, _} = couch_db:fold_docs(SourceDb, Fun, [], []),
-    ok = couch_db:close(SourceDb),
-    ok = couch_db:close(TargetDb).
+compare_fold(Source, Target, Fun, Acc0) when
+        is_binary(Source), is_binary(Target), is_function(Fun, 3) ->
+    {ok, SourceDb} = fabric2_db:open(Source, [?ADMIN_CTX]),
+    {ok, TargetDb} = fabric2_db:open(Target, [?ADMIN_CTX]),
+    fabric2_fdb:transactional(SourceDb, fun(TxSourceDb) ->
+        FoldFun = fun
+            ({meta, _Meta}, Acc) ->
+                {ok, Acc};
+            (complete, Acc) ->
+                {ok, Acc};
+            ({row, Row}, Acc) ->
+                {_, Id} = lists:keyfind(id, 1, Row),
+                SrcDoc = open_doc(TxSourceDb, Id),
+                TgtDoc = open_doc(TargetDb, Id),
+                {ok, Fun(SrcDoc, TgtDoc, Acc)}
+        end,
+        Opts = [{restart_tx, true}],
+        {ok, AccF} = fabric2_db:fold_docs(TxSourceDb, FoldFun, Acc0, Opts),
+        AccF
+    end).
 
 
-compare_docs(Doc1, Doc2) ->
+compare_docs(#doc{} = Doc1, Doc2) when
+        is_record(Doc2, doc) orelse Doc2 =:= not_found ->
+    ?assert(Doc2 =/= not_found),
     ?assertEqual(Doc1#doc.body, Doc2#doc.body),
     #doc{atts = Atts1} = Doc1,
     #doc{atts = Atts2} = Doc2,
     ?assertEqual(lists:sort([couch_att:fetch(name, Att) || Att <- Atts1]),
-                 lists:sort([couch_att:fetch(name, Att) || Att <- Atts2])),
+        lists:sort([couch_att:fetch(name, Att) || Att <- Atts2])),
     FunCompareAtts = fun(Att) ->
         AttName = couch_att:fetch(name, Att),
         {ok, AttTarget} = find_att(Atts2, AttName),
@@ -68,19 +183,109 @@ compare_docs(Doc1, Doc2) ->
         ?assert(is_integer(couch_att:fetch(disk_len, AttTarget))),
         ?assert(is_integer(couch_att:fetch(att_len, AttTarget))),
         ?assertEqual(couch_att:fetch(disk_len, Att),
-                     couch_att:fetch(disk_len, AttTarget)),
+            couch_att:fetch(disk_len, AttTarget)),
         ?assertEqual(couch_att:fetch(att_len, Att),
-                     couch_att:fetch(att_len, AttTarget)),
+            couch_att:fetch(att_len, AttTarget)),
         ?assertEqual(couch_att:fetch(type, Att),
-                     couch_att:fetch(type, AttTarget)),
+            couch_att:fetch(type, AttTarget)),
         ?assertEqual(couch_att:fetch(md5, Att),
-                     couch_att:fetch(md5, AttTarget))
+            couch_att:fetch(md5, AttTarget))
     end,
     lists:foreach(FunCompareAtts, Atts1).
 
 
+get_pid(RepId) ->
+    JobId = case couch_replicator_jobs:get_job_id(undefined, RepId) of
+        {ok, JobId0} -> JobId0;
+        {error, not_found} -> RepId
+    end,
+    {ok, #{<<"state">> := <<"running">>, <<"pid">> := Pid0}} =
+            couch_replicator_jobs:get_job_data(undefined, JobId),
+    Pid = list_to_pid(binary_to_list(Pid0)),
+    ?assert(is_pid(Pid)),
+    ?assert(is_process_alive(Pid)),
+    Pid.
+
+
+replicate({[_ | _]} = EJson) ->
+    Str = couch_util:json_encode(EJson),
+    replicate(couch_util:json_decode(Str, [return_maps]));
+
+replicate(#{} = Rep0) ->
+    Rep = maybe_db_urls(Rep0),
+    {ok, Id, _} = couch_replicator_parse:parse_transient_rep(Rep, null),
+    ok = cancel(Id),
+    try
+        couch_replicator:replicate(Rep, ?ADMIN_USER)
+    after
+        ok = cancel(Id)
+    end.
+
+
+replicate(Source, Target) ->
+    replicate(#{
+        <<"source">> => Source,
+        <<"target">> => Target
+    }).
+
+
+replicate_continuous({[_ | _]} = EJson) ->
+    Str = couch_util:json_encode(EJson),
+    replicate_continuous(couch_util:json_decode(Str, [return_maps]));
+
+replicate_continuous(#{<<"continuous">> := true} = Rep0) ->
+    Rep = maybe_db_urls(Rep0),
+    {ok, {continuous, RepId}} = couch_replicator:replicate(Rep, ?ADMIN_USER),
+    {ok, get_pid(RepId), RepId}.
+
+
+replicate_continuous(Source, Target) ->
+    replicate_continuous(#{
+        <<"source">> => Source,
+        <<"target">> => Target,
+        <<"continuous">> => true
+    }).
+
+
+cancel(Id) when is_binary(Id) ->
+    CancelRep = #{<<"cancel">> => true, <<"id">> => Id},
+    case couch_replicator:replicate(CancelRep, ?ADMIN_USER) of
+        {ok, {cancelled, <<_/binary>>}} -> ok;
+        {error, not_found} -> ok
+    end.
+
+
+cancel(Id, Pid) when is_pid(Pid), is_binary(Id) ->
+    Ref = monitor(process, Pid),
+    try
+        cancel(Id)
+    after
+        receive
+            {'DOWN', Ref, _, _, _} -> ok
+        after 60000 ->
+            error(replicator_pid_death_timeout)
+        end
+    end.
+
+
+scheduler_jobs() ->
+    ServerUrl = couch_replicator_test_helper:server_url(),
+    Url = lists:flatten(io_lib:format("~s/_scheduler/jobs", [ServerUrl])),
+    {ok, 200, _, Body} = test_request:get(Url, []),
+    Json = jiffy:decode(Body, [return_maps]),
+    maps:get(<<"jobs">>, Json).
+
+
+open_doc(Db, DocId) ->
+    case fabric2_db:open_doc(Db, DocId, []) of
+        {ok, #doc{deleted = false} = Doc} -> Doc;
+        {not_found, missing} -> not_found
+    end.
+
+
 find_att([], _Name) ->
     nil;
+
 find_att([Att | Rest], Name) ->
     case couch_att:fetch(name, Att) of
         Name ->
@@ -91,45 +296,29 @@ find_att([Att | Rest], Name) ->
 
 
 att_md5(Att) ->
-    Md50 = couch_att:foldl(
-        Att,
-        fun(Chunk, Acc) -> couch_hash:md5_hash_update(Acc, Chunk) end,
-        couch_hash:md5_hash_init()),
+    Md50 = couch_att:foldl(Att, fun(Chunk, Acc) ->
+        couch_hash:md5_hash_update(Acc, Chunk)
+    end, couch_hash:md5_hash_init()),
     couch_hash:md5_hash_final(Md50).
 
+
 att_decoded_md5(Att) ->
-    Md50 = couch_att:foldl_decode(
-        Att,
-        fun(Chunk, Acc) -> couch_hash:md5_hash_update(Acc, Chunk) end,
-        couch_hash:md5_hash_init()),
+    Md50 = couch_att:foldl_decode(Att, fun(Chunk, Acc) ->
+        couch_hash:md5_hash_update(Acc, Chunk)
+    end, couch_hash:md5_hash_init()),
     couch_hash:md5_hash_final(Md50).
 
-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
-    ]).
-
-get_pid(RepId) ->
-    Pid = global:whereis_name({couch_replicator_scheduler_job,RepId}),
-    ?assert(is_pid(Pid)),
-    Pid.
 
-replicate(Source, Target) ->
-    replicate({[
-        {<<"source">>, Source},
-        {<<"target">>, Target}
-    ]}).
-
-replicate({[_ | _]} = RepObject) ->
-    {ok, Rep} = couch_replicator_utils:parse_rep_doc(RepObject, ?ADMIN_USER),
-    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
+maybe_db_urls(#{} = Rep) ->
+    #{<<"source">> := Src, <<"target">> := Tgt} = Rep,
+    Src1 = case Src of
+        <<"http://", _/binary>> -> Src;
+        <<"https://", _/binary>> -> Src;
+        <<_/binary>> -> db_url(Src)
+    end,
+    Tgt1 = case Tgt of
+        <<"http://", _/binary>> -> Tgt;
+        <<"https://", _/binary>> -> Tgt;
+        <<_/binary>> -> db_url(Tgt)
     end,
-    ok = couch_replicator_scheduler:remove_job(Rep#rep.id).
+    Rep#{<<"source">> := Src1, <<"target">> := Tgt1}.
diff --git a/src/couch_replicator/test/eunit/couch_replicator_transient_jobs_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_transient_jobs_tests.erl
new file mode 100644
index 0000000..25fc6a3
--- /dev/null
+++ b/src/couch_replicator/test/eunit/couch_replicator_transient_jobs_tests.erl
@@ -0,0 +1,106 @@
+% 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_transient_jobs_tests).
+
+-include_lib("couch/include/couch_eunit.hrl").
+-include_lib("couch/include/couch_db.hrl").
+-include_lib("couch_replicator/src/couch_replicator.hrl").
+-include_lib("fabric/test/fabric2_test.hrl").
+
+
+transient_jobs_test_() ->
+    {
+        "Transient jobs tests",
+        {
+            setup,
+            fun couch_replicator_test_helper:start_couch/0,
+            fun couch_replicator_test_helper:stop_couch/1,
+            {
+                foreach,
+                fun setup/0,
+                fun teardown/1,
+                [
+                    ?TDEF_FE(transient_job_is_removed, 10),
+                    ?TDEF_FE(posting_same_job_is_a_noop, 10)
+                ]
+            }
+        }
+    }.
+
+
+setup() ->
+    Source = couch_replicator_test_helper:create_db(),
+    couch_replicator_test_helper:create_docs(Source, [
+        #{<<"_id">> => <<"doc1">>}
+    ]),
+    Target = couch_replicator_test_helper:create_db(),
+    config:set("replicator", "stats_update_interval_sec", "0", false),
+    config:set("replicator", "transient_job_max_age_sec", "9999", false),
+    {Source, Target}.
+
+
+teardown({Source, Target}) ->
+    config:delete("replicator", "stats_update_interval_sec", false),
+    config:delete("replicator", "transient_job_max_age_sec", false),
+    couch_replicator_test_helper:delete_db(Source),
+    couch_replicator_test_helper:delete_db(Target).
+
+
+transient_job_is_removed({Source, Target}) ->
+    {ok, #{}} = replicate(Source, Target),
+    JobId = get_rep_id(Source, Target),
+
+    couch_replicator_job_server:reschedule(),
+
+    % Still there after clean up attempt ran
+    ?assertMatch({200, #{}}, scheduler_jobs(JobId)),
+
+    config:set("replicator", "transient_job_max_age_sec", "0", false),
+    couch_replicator_job_server:reschedule(),
+
+    % Should be gone now
+    ?assertMatch({404, #{}}, scheduler_jobs(JobId)).
+
+
+posting_same_job_is_a_noop({Source, Target}) ->
+    {ok, Pid1, RepId1} = replicate_continuous(Source, Target),
+    {ok, Pid2, RepId2} = replicate_continuous(Source, Target),
+    ?assertEqual(RepId1, RepId2),
+    ?assertEqual(Pid1, Pid2),
+    couch_replicator_test_helper:cancel(RepId1).
+
+   
+get_rep_id(Source, Target) ->
+    {ok, Id, _} = couch_replicator_parse:parse_transient_rep(#{
+        <<"source">> => couch_replicator_test_helper:db_url(Source),
+        <<"target">> => couch_replicator_test_helper:db_url(Target)
+    }, null),
+    Id.
+
+
+replicate(Source, Target) ->
+    couch_replicator:replicate(#{
+        <<"source">> => couch_replicator_test_helper:db_url(Source),
+        <<"target">> => couch_replicator_test_helper:db_url(Target)
+    }, ?ADMIN_USER).
+
+
+replicate_continuous(Source, Target) ->
+    couch_replicator_test_helper:replicate_continuous(Source, Target).
+
+
+scheduler_jobs(Id) ->
+    SUrl = couch_replicator_test_helper:server_url(),
+    Url = lists:flatten(io_lib:format("~s/_scheduler/jobs/~s", [SUrl, Id])),
+    {ok, Code, _, Body} = test_request:get(Url, []),
+    {Code, jiffy:decode(Body, [return_maps])}.
diff --git a/src/couch_replicator/test/eunit/couch_replicator_use_checkpoints_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_use_checkpoints_tests.erl
index 8e4a21d..4371eff 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_use_checkpoints_tests.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_use_checkpoints_tests.erl
@@ -14,165 +14,82 @@
 
 -include_lib("couch/include/couch_eunit.hrl").
 -include_lib("couch/include/couch_db.hrl").
+-include_lib("fabric/test/fabric2_test.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)).
 -define(io2b(Io), iolist_to_binary(Io)).
 
 
-start(false) ->
-    fun
-        ({finished, _, {CheckpointHistory}}) ->
-            ?assertEqual([{<<"use_checkpoints">>,false}], CheckpointHistory);
-        (_) ->
-            ok
-    end;
-start(true) ->
-    fun
-        ({finished, _, {CheckpointHistory}}) ->
-            ?assertNotEqual(false, lists:keyfind(<<"session_id">>,
-                                                 1, CheckpointHistory));
-        (_) ->
-            ok
-    end.
-
-stop(_, _) ->
-    ok.
-
-setup() ->
-    DbName = ?tempdb(),
-    {ok, Db} = couch_db:create(DbName, [?ADMIN_CTX]),
-    ok = couch_db:close(Db),
-    DbName.
-
-setup(remote) ->
-    {remote, setup()};
-setup({_, Fun, {A, B}}) ->
-    Ctx = test_util:start_couch([couch_replicator]),
-    {ok, Listener} = couch_replicator_notifier:start_link(Fun),
-    Source = setup(A),
-    Target = setup(B),
-    {Ctx, {Source, Target, Listener}}.
-
-teardown({remote, DbName}) ->
-    teardown(DbName);
-teardown(DbName) ->
-    ok = couch_server:delete(DbName, [?ADMIN_CTX]),
-    ok.
-
-teardown(_, {Ctx, {Source, Target, Listener}}) ->
-    teardown(Source),
-    teardown(Target),
-
-    couch_replicator_notifier:stop(Listener),
-    ok = application:stop(couch_replicator),
-    ok = test_util:stop_couch(Ctx).
-
 use_checkpoints_test_() ->
     {
-        "Replication use_checkpoints feature tests",
+        setup,
+        fun couch_replicator_test_helper:start_couch/0,
+        fun couch_replicator_test_helper:stop_couch/1,
         {
-            foreachx,
-            fun start/1, fun stop/2,
-            [{UseCheckpoints, fun use_checkpoints_tests/2}
-             || UseCheckpoints <- [false, true]]
+            foreach,
+            fun setup/0,
+            fun teardown/1,
+            [
+                ?TDEF_FE(t_replicate_with_checkpoints, 15),
+                ?TDEF_FE(t_replicate_without_checkpoints, 15)
+            ]
         }
     }.
 
-use_checkpoints_tests(UseCheckpoints, Fun) ->
-    Pairs = [{remote, remote}],
-    {
-        "use_checkpoints: " ++ atom_to_list(UseCheckpoints),
-        {
-            foreachx,
-            fun setup/1, fun teardown/2,
-            [{{UseCheckpoints, Fun, Pair}, fun should_test_checkpoints/2}
-             || Pair <- Pairs]
-        }
-    }.
 
-should_test_checkpoints({UseCheckpoints, _, {From, To}}, {_Ctx, {Source, Target, _}}) ->
-    should_test_checkpoints(UseCheckpoints, {From, To}, {Source, Target}).
-should_test_checkpoints(UseCheckpoints, {From, To}, {Source, Target}) ->
-    {lists:flatten(io_lib:format("~p -> ~p", [From, To])),
-     {inorder, [
-        should_populate_source(Source, ?DOCS_COUNT),
-        should_replicate(Source, Target, UseCheckpoints),
-        should_compare_databases(Source, Target)
-     ]}}.
-
-should_populate_source({remote, Source}, DocCount) ->
-    should_populate_source(Source, DocCount);
-should_populate_source(Source, DocCount) ->
-    {timeout, ?TIMEOUT_EUNIT, ?_test(populate_db(Source, DocCount))}.
-
-should_replicate({remote, Source}, Target, UseCheckpoints) ->
-    should_replicate(db_url(Source), Target, UseCheckpoints);
-should_replicate(Source, {remote, Target}, UseCheckpoints) ->
-    should_replicate(Source, db_url(Target), UseCheckpoints);
-should_replicate(Source, Target, UseCheckpoints) ->
-    {timeout, ?TIMEOUT_EUNIT, ?_test(replicate(Source, Target, UseCheckpoints))}.
-
-should_compare_databases({remote, Source}, Target) ->
-    should_compare_databases(Source, Target);
-should_compare_databases(Source, {remote, Target}) ->
-    should_compare_databases(Source, Target);
-should_compare_databases(Source, Target) ->
-    {timeout, ?TIMEOUT_EUNIT, ?_test(compare_dbs(Source, Target))}.
+setup() ->
+    Source = couch_replicator_test_helper:create_db(),
+    Target = couch_replicator_test_helper:create_db(),
+    {Source, Target}.
 
 
-populate_db(DbName, DocCount) ->
-    {ok, Db} = couch_db:open_int(DbName, []),
-    Docs = lists:foldl(
-        fun(DocIdCounter, Acc) ->
-            Id = ?io2b(["doc", ?i2l(DocIdCounter)]),
-            Value = ?io2b(["val", ?i2l(DocIdCounter)]),
-            Doc = #doc{
-                id = Id,
-                body = {[ {<<"value">>, Value} ]}
-            },
-            [Doc | Acc]
-        end,
-        [], lists:seq(1, DocCount)),
-    {ok, _} = couch_db:update_docs(Db, Docs, []),
-    ok = couch_db:close(Db).
-
-compare_dbs(Source, Target) ->
-    {ok, SourceDb} = couch_db:open_int(Source, []),
-    {ok, TargetDb} = couch_db:open_int(Target, []),
-    Fun = fun(FullDocInfo, Acc) ->
-        {ok, Doc} = couch_db:open_doc(SourceDb, FullDocInfo),
-        {Props} = DocJson = couch_doc:to_json_obj(Doc, [attachments]),
-        DocId = couch_util:get_value(<<"_id">>, Props),
-        DocTarget = case couch_db:open_doc(TargetDb, DocId) of
-            {ok, DocT} ->
-                DocT;
-            Error ->
-                erlang:error(
-                    {assertion_failed,
-                     [{module, ?MODULE}, {line, ?LINE},
-                      {reason, lists:concat(["Error opening document '",
-                                             ?b2l(DocId), "' from target: ",
-                                             couch_util:to_list(Error)])}]})
-            end,
-        DocTargetJson = couch_doc:to_json_obj(DocTarget, [attachments]),
-        ?assertEqual(DocJson, DocTargetJson),
-        {ok, Acc}
-    end,
-    {ok, _} = couch_db:fold_docs(SourceDb, Fun, [], []),
-    ok = couch_db:close(SourceDb),
-    ok = couch_db:close(TargetDb).
-
-replicate(Source, Target, UseCheckpoints) ->
-    replicate({[
-        {<<"source">>, Source},
-        {<<"target">>, Target},
-        {<<"use_checkpoints">>, UseCheckpoints}
-    ]}).
+teardown({Source, Target}) ->
+    couch_replicator_test_helper:delete_db(Source),
+    couch_replicator_test_helper:delete_db(Target).
 
+
+t_replicate_with_checkpoints({Source, Target}) ->
+    populate_db(Source, ?DOCS_COUNT),
+    Res = couch_replicator_test_helper:replicate(#{
+        <<"source">> => Source,
+        <<"target">> => Target,
+        <<"use_checkpoints">> => true
+    }),
+    ?assertMatch({ok, _}, Res),
+
+    {ok, History} = Res,
+    ?assertMatch(#{<<"history">> := _, <<"session_id">> := _}, History),
+
+    Checkpoints = maps:get(<<"history">>, History),
+    SessionId = maps:get(<<"session_id">>, History),
+    ?assert(is_binary(SessionId)),
+    ?assert(is_list(Checkpoints)),
+    ?assert(length(Checkpoints) >= 1),
+
+    couch_replicator_test_helper:compare_dbs(Source, Target).
+
+
+t_replicate_without_checkpoints({Source, Target}) ->
+    populate_db(Source, ?DOCS_COUNT),
+    Res = couch_replicator_test_helper:replicate(#{
+        <<"source">> => Source,
+        <<"target">> => Target,
+        <<"use_checkpoints">> => false
+    }),
+    ?assertEqual({ok, #{<<"use_checkpoints">> => false}}, Res),
+    couch_replicator_test_helper:compare_dbs(Source, Target).
+
+
+populate_db(DbName, DocCount) ->
+    Docs = lists:foldl(fun(DocIdCounter, Acc) ->
+        Id = ?io2b(["doc", ?i2l(DocIdCounter)]),
+        Value = ?io2b(["val", ?i2l(DocIdCounter)]),
+        Doc = #doc{
+            id = Id,
+            body = {[{<<"value">>, Value}]}
+        },
+        [Doc | Acc]
+    end, [], lists:seq(1, DocCount)),
+    couch_replicator_test_helper:create_docs(DbName, Docs).
diff --git a/test/elixir/test/replication_test.exs b/test/elixir/test/replication_test.exs
index 8b657d9..9af5ef8 100644
--- a/test/elixir/test/replication_test.exs
+++ b/test/elixir/test/replication_test.exs
@@ -14,7 +14,10 @@ defmodule ReplicationTest do
 
   # This should probably go into `make elixir` like what
   # happens for JavaScript tests.
-  @moduletag config: [{"replicator", "startup_jitter", "0"}]
+  @moduletag config: [
+      {"replicator", "startup_jitter", "0"},
+      {"replicator", "stats_update_interval_sec", "0"}
+  ]
 
   test "source database not found with host" do
     name = random_db_name()


[couchdb] 02/16: Read attachment data outside the transaction

Posted by va...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

vatamane pushed a commit to branch prototype/fdb-layer
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit e3b1c418ebc9735a46ef8e1f36d09d7023939372
Author: Nick Vatamaniuc <va...@apache.org>
AuthorDate: Fri Aug 28 04:30:27 2020 -0400

    Read attachment data outside the transaction
    
    Previously the data was read from the parser in the transaction. If the
    transaction had to retry, for example, because of a conflict, the parser would
    have been drained and exited resulting the request failing with a 500
    "mp_parser noproc" error.
    
    Since FDB cannot handle transactions larger than 10MB opt to read the
    attachment data into memory first, before the transaction starts.
---
 src/chttpd/src/chttpd_db.erl | 26 ++++++++++++++++++--------
 src/couch/src/couch_att.erl  | 20 ++++++++++++++++++--
 2 files changed, 36 insertions(+), 10 deletions(-)

diff --git a/src/chttpd/src/chttpd_db.erl b/src/chttpd/src/chttpd_db.erl
index ec4a1a4..b57010d 100644
--- a/src/chttpd/src/chttpd_db.erl
+++ b/src/chttpd/src/chttpd_db.erl
@@ -418,12 +418,13 @@ db_req(#httpd{method='POST', path_parts=[DbName]}=Req, Db) ->
         _ ->
             Doc1
     end,
-    DocId = Doc2#doc.id,
+    Doc3 = read_att_data(Doc2),
+    DocId = Doc3#doc.id,
     case chttpd:qs_value(Req, "batch") of
     "ok" ->
         % async_batching
         spawn(fun() ->
-                case catch(fabric2_db:update_doc(Db, Doc2, [])) of
+                case catch(fabric2_db:update_doc(Db, Doc3, [])) of
                 {ok, _} ->
                     chttpd_stats:incr_writes(),
                     ok;
@@ -443,7 +444,7 @@ db_req(#httpd{method='POST', path_parts=[DbName]}=Req, Db) ->
         % normal
         DocUrl = absolute_uri(Req, [$/, couch_util:url_encode(DbName),
             $/, couch_util:url_encode(DocId)]),
-        case fabric2_db:update_doc(Db, Doc2, []) of
+        case fabric2_db:update_doc(Db, Doc3, []) of
         {ok, NewRev} ->
             chttpd_stats:incr_writes(),
             HttpCode = 201;
@@ -1174,7 +1175,8 @@ db_doc_req(#httpd{method='POST'}=Req, Db, DocId) ->
     NewDoc = Doc#doc{
         atts = UpdatedAtts ++ OldAtts2
     },
-    case fabric2_db:update_doc(Db, NewDoc, []) of
+    NewDoc1 = read_att_data(NewDoc),
+    case fabric2_db:update_doc(Db, NewDoc1, []) of
     {ok, NewRev} ->
         chttpd_stats:incr_writes(),
         HttpCode = 201;
@@ -1218,8 +1220,8 @@ db_doc_req(#httpd{method='PUT'}=Req, Db, DocId) ->
         case chttpd:qs_value(Req, "batch") of
         "ok" ->
             % batch
-            Doc = couch_doc_from_req(Req, Db, DocId, chttpd:json_body(Req)),
-
+            Doc0 = couch_doc_from_req(Req, Db, DocId, chttpd:json_body(Req)),
+            Doc = read_att_data(Doc0),
             spawn(fun() ->
                     case catch(fabric2_db:update_doc(Db, Doc, [])) of
                     {ok, _} ->
@@ -1479,7 +1481,8 @@ http_code_from_status(Status) ->
             200
     end.
 
-update_doc(Db, DocId, #doc{deleted=Deleted, body=DocBody}=Doc, Options) ->
+update_doc(Db, DocId, #doc{deleted=Deleted, body=DocBody}=Doc0, Options) ->
+    Doc = read_att_data(Doc0),
     case fabric2_db:update_doc(Db, Doc, Options) of
     {ok, NewRev} ->
         Accepted = false;
@@ -1766,9 +1769,10 @@ db_attachment_req(#httpd{method=Method}=Req, Db, DocId, FileNameParts)
     end,
 
     #doc{atts=Atts} = Doc,
-    DocEdited = Doc#doc{
+    DocEdited0 = Doc#doc{
         atts = NewAtt ++ [A || A <- Atts, couch_att:fetch(name, A) /= FileName]
     },
+    DocEdited = read_att_data(DocEdited0),
     case fabric2_db:update_doc(Db, DocEdited, []) of
     {ok, UpdatedRev} ->
         chttpd_stats:incr_writes(),
@@ -2240,3 +2244,9 @@ bulk_get_json_error(DocId, Rev, Error, Reason) ->
                              {<<"rev">>, Rev},
                              {<<"error">>, Error},
                              {<<"reason">>, Reason}]}}]}).
+
+
+read_att_data(#doc{} = Doc) ->
+    #doc{atts = Atts} = Doc,
+    Atts1 = lists:map(fun couch_att:read_data/1, Atts),
+    Doc#doc{atts = Atts1}.
diff --git a/src/couch/src/couch_att.erl b/src/couch/src/couch_att.erl
index d41ab5b..b4c95e9 100644
--- a/src/couch/src/couch_att.erl
+++ b/src/couch/src/couch_att.erl
@@ -40,6 +40,7 @@
 
 -export([
     flush/3,
+    read_data/1,
     foldl/3,
     range_foldl/5,
     foldl_decode/3,
@@ -374,7 +375,14 @@ to_json(Att, OutputData, DataToFollow, ShowEncoding) ->
 
 
 flush(Db, DocId, Att1) ->
-    Att2 = read_data(fetch(data, Att1), Att1),
+    Data0 = fetch(data, Att1),
+    case {Data0, Db} of
+        {{follows, _, _}, #{tx := Tx}} when Tx =/= undefined ->
+            error(follows_cannot_be_used_in_a_transaction);
+        {_, #{}} ->
+            ok
+    end,
+    Att2 = read_data(Data0, Att1),
     [
         Data,
         AttLen,
@@ -419,6 +427,11 @@ flush(Db, DocId, Att1) ->
     end.
 
 
+read_data(Att) ->
+    Data = fetch(data, Att),
+    read_data(Data, Att).
+
+
 read_data({loc, #{}, _DocId, _AttId}, Att) ->
     % Attachment already written to fdb
     Att;
@@ -443,7 +456,10 @@ read_data({follows, Parser, Ref}, Att) ->
     end;
 
 read_data(Data, Att) when is_binary(Data) ->
-    Att;
+    case fetch(att_len, Att) of
+        undefined -> store(att_len, size(Data), Att);
+        Int when is_integer(Int) ->  Att
+    end;
 
 read_data(Fun, Att) when is_function(Fun) ->
     [AttName, AttLen, InMd5] = fetch([name, att_len, md5], Att),


[couchdb] 04/16: Add fold_jobs/4 and pending_count/2, 3 to couch_jobs API

Posted by va...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

vatamane pushed a commit to branch prototype/fdb-layer
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit d2c9dffa3aca3b3e2faed49526b0065ebb845fad
Author: Nick Vatamaniuc <va...@apache.org>
AuthorDate: Fri Aug 28 04:30:53 2020 -0400

    Add fold_jobs/4 and pending_count/2,3 to couch_jobs API
    
    fold_jobs/4 is a generalized folding API which can be used to process all the
    jobs of a particular type.
    
    get_pending_count/2,3 can be used to get the count of pending jobs. It takes
    the same options as accept, including a limit and `max_sched_time`. Just like
    accept it reads the range of pending jobs as a snapshot to avoid generating
    read conflicts.
---
 src/couch_jobs/src/couch_jobs.erl         | 37 +++++++++++++++++++++++++++----
 src/couch_jobs/src/couch_jobs_pending.erl | 32 +++++++++++++++++++++-----
 2 files changed, 59 insertions(+), 10 deletions(-)

diff --git a/src/couch_jobs/src/couch_jobs.erl b/src/couch_jobs/src/couch_jobs.erl
index f5d6a7b..6c40f5d 100644
--- a/src/couch_jobs/src/couch_jobs.erl
+++ b/src/couch_jobs/src/couch_jobs.erl
@@ -17,10 +17,15 @@
     add/4,
     add/5,
     remove/3,
+
+    % Job monitoring
+    get_types/1,
     get_job_data/3,
     get_job_state/3,
     get_active_jobs_ids/2,
-    get_types/1,
+    fold_jobs/4,
+    pending_count/2,
+    pending_count/3,
 
     % Job processing
     accept/1,
@@ -80,6 +85,13 @@ remove(Tx, Type, JobId) when is_binary(JobId) ->
     end).
 
 
+-spec get_types(jtx()) -> [job_type()] | {error, any()}.
+get_types(Tx) ->
+    couch_jobs_fdb:tx(couch_jobs_fdb:get_jtx(Tx), fun(JTx) ->
+        couch_jobs_fdb:get_types(JTx)
+    end).
+
+
 -spec get_job_data(jtx(), job_type(), job_id()) -> {ok, job_data()} | {error,
     any()}.
 get_job_data(Tx, Type, JobId) when is_binary(JobId) ->
@@ -116,10 +128,27 @@ get_active_jobs_ids(Tx, Type) ->
     end).
 
 
--spec get_types(jtx()) -> [job_type()] | {error, any()}.
-get_types(Tx) ->
+-spec fold_jobs(jtx(), job_type(), fun(), any()) -> any().
+fold_jobs(Tx, Type, Fun, UserAcc) when is_function(Fun, 5) ->
     couch_jobs_fdb:tx(couch_jobs_fdb:get_jtx(Tx), fun(JTx) ->
-        couch_jobs_fdb:get_types(JTx)
+        maps:fold(fun(JobId, {_Seq, JobState, DataEnc}, Acc) ->
+            Data = couch_jobs_fdb:decode_data(DataEnc),
+            Fun(JTx, JobId, JobState, Data, Acc)
+        end, UserAcc, couch_jobs_fdb:get_jobs(JTx, Type))
+    end).
+
+
+-spec pending_count(jtx(), job_type()) -> integer().
+pending_count(Tx, Type) ->
+    pending_count(Tx, Type, #{}).
+
+
+-spec pending_count(jtx(), job_type(), #{}) -> integer().
+pending_count(Tx, Type, Opts) ->
+    MaxSTime = maps:get(max_sched_time, Opts, ?UNDEFINED_MAX_SCHEDULED_TIME),
+    Limit = maps:get(limit, Opts, 1024),
+    couch_jobs_fdb:tx(couch_jobs_fdb:get_jtx(Tx), fun(JTx) ->
+        couch_jobs_pending:pending_count(JTx, Type, MaxSTime, Limit)
     end).
 
 
diff --git a/src/couch_jobs/src/couch_jobs_pending.erl b/src/couch_jobs/src/couch_jobs_pending.erl
index ab53c59..a85f2fc 100644
--- a/src/couch_jobs/src/couch_jobs_pending.erl
+++ b/src/couch_jobs/src/couch_jobs_pending.erl
@@ -16,7 +16,8 @@
 -export([
     enqueue/4,
     dequeue/4,
-    remove/4
+    remove/4,
+    pending_count/4
 ]).
 
 
@@ -47,16 +48,14 @@ dequeue(#{jtx := true} = JTx, Type, _, true) ->
             {ok, JobId}
     end;
 
-dequeue(#{jtx := true} = JTx, Type, MaxPriority, _) ->
+dequeue(#{jtx := true} = JTx, Type, MaxSTime, _) ->
     #{tx := Tx, jobs_path := Jobs} = JTx,
-    Prefix = erlfdb_tuple:pack({?PENDING, Type}, Jobs),
-    StartKeySel = erlfdb_key:first_greater_than(Prefix),
-    End = erlfdb_tuple:pack({MaxPriority, <<16#FF>>}, Prefix),
-    EndKeySel = erlfdb_key:first_greater_or_equal(End),
+    {StartKeySel, EndKeySel} = get_range_selectors(JTx, Type, MaxSTime),
     case clear_random_key_from_range(Tx, StartKeySel, EndKeySel) of
         {error, not_found} ->
             {not_found, get_pending_watch(JTx, Type)};
         {ok, PendingKey} ->
+            Prefix = erlfdb_tuple:pack({?PENDING, Type}, Jobs),
             {_, JobId} = erlfdb_tuple:unpack(PendingKey, Prefix),
             {ok, JobId}
     end.
@@ -68,8 +67,29 @@ remove(#{jtx := true} = JTx, Type, JobId, STime) ->
     erlfdb:clear(Tx, Key).
 
 
+pending_count(#{jtx := true} = JTx, Type, MaxSTime, Limit) ->
+    #{tx := Tx} = JTx,
+    Opts = [
+        {limit, Limit},
+        {snapshot, true},
+        {streaming_mode, want_all}
+    ],
+    {StartSel, EndSel} = get_range_selectors(JTx, Type, MaxSTime),
+    FoldFun = fun(_Row, Cnt) -> Cnt + 1 end,
+    erlfdb:fold_range(Tx, StartSel, EndSel, FoldFun, 0, Opts).
+
+
 %% Private functions
 
+% Get pending key selectors, taking into account max scheduled time value.
+get_range_selectors(#{jtx := true} = JTx, Type, MaxSTime) ->
+    #{jobs_path := Jobs} = JTx,
+    Prefix = erlfdb_tuple:pack({?PENDING, Type}, Jobs),
+    StartKeySel = erlfdb_key:first_greater_than(Prefix),
+    End = erlfdb_tuple:pack({MaxSTime, <<16#FF>>}, Prefix),
+    EndKeySel = erlfdb_key:first_greater_or_equal(End),
+    {StartKeySel, EndKeySel}.
+
 
 % Pick a random item from the range without reading the keys in first. However
 % the constraint it that IDs should looks like random UUIDs


[couchdb] 10/16: Introduce couch_replicator_jobs abstraction module

Posted by va...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

vatamane pushed a commit to branch prototype/fdb-layer
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit 3c9b7540cbb41225b35c89b741e0c5b83cdbf4e1
Author: Nick Vatamaniuc <va...@apache.org>
AuthorDate: Fri Aug 28 04:33:18 2020 -0400

    Introduce couch_replicator_jobs abstraction module
    
    This is the `couch_jobs` abstraction module. All replicator calls to
    `couch_jobs` should go through it. This module takes care of adding types to
    some of the API calls, handles maintencence of the RepId -> JobId mappings when
    jobs are added and removed, and some subscription logic.
    
    `fabric2.hrl` include file is updated with the definition of the
    `?REPLICATION_IDS` prefix where the RepId -> JobId keyspace lives.
---
 src/couch_replicator/src/couch_replicator_jobs.erl | 312 +++++++++++++++++++++
 src/fabric/include/fabric2.hrl                     |   1 +
 2 files changed, 313 insertions(+)

diff --git a/src/couch_replicator/src/couch_replicator_jobs.erl b/src/couch_replicator/src/couch_replicator_jobs.erl
new file mode 100644
index 0000000..a602b0c
--- /dev/null
+++ b/src/couch_replicator/src/couch_replicator_jobs.erl
@@ -0,0 +1,312 @@
+% 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_jobs).
+
+
+-export([
+    % couch_jobs type timeouts
+    set_timeout/0,
+    get_timeout/0,
+
+    % Job creation and querying
+    new_job/7,
+    add_job/3,
+    remove_job/2,
+    get_job_data/2,
+    fold_jobs/3,
+    pending_count/2,
+
+    % Job subscription
+    wait_running/1,
+    wait_result/1,
+
+    % Job execution
+    accept_job/1,
+    update_job_data/3,
+    finish_job/3,
+    reschedule_job/4,
+
+    % (..., ?REPLICATION_IDS) -> JobId handling
+    try_update_rep_id/3,
+    update_rep_id/3,
+    clear_old_rep_id/3,
+    get_job_id/2,
+
+    % Debug functions
+    remove_jobs/2,
+    get_job_ids/1
+]).
+
+
+-include("couch_replicator.hrl").
+-include_lib("fabric/include/fabric2.hrl").
+
+
+-define(REP_JOBS, <<"rep_jobs">>).
+-define(REP_JOBS_TIMEOUT_SEC, 61).
+
+
+% Data model
+% ----------
+%
+% State kept in couch_jobs under the ?REP_JOBS type
+%
+% Job IDs are defined as:
+%   * Replicator DB instance UUID + doc ID for persistent replications
+%   * Hash(username|source|target|options) for transient replications
+%
+% To map replication IDs to couch_job jobs, there is a separate index that
+% looks like:
+%   (?REPLICATION_IDS, RepId) -> JobId
+%
+
+set_timeout() ->
+    couch_jobs:set_type_timeout(?REP_JOBS, ?REP_JOBS_TIMEOUT_SEC).
+
+
+get_timeout() ->
+    ?REP_JOBS_TIMEOUT_SEC.
+
+
+new_job(#{} = Rep, DbName, DbUUID, DocId, State, StateInfo, DocState) ->
+    NowSec = erlang:system_time(second),
+    AddedEvent = #{?HIST_TYPE => ?HIST_ADDED, ?HIST_TIMESTAMP => NowSec},
+    #{
+        ?REP => Rep,
+        ?REP_ID => null,
+        ?BASE_ID => null,
+        ?DB_NAME => DbName,
+        ?DB_UUID => DbUUID,
+        ?DOC_ID => DocId,
+        ?ERROR_COUNT => 0,
+        ?REP_STATS => #{},
+        ?STATE => State,
+        ?STATE_INFO => StateInfo,
+        ?DOC_STATE => DocState,
+        ?LAST_UPDATED => NowSec,
+        ?LAST_START => 0,
+        ?LAST_ERROR => null,
+        ?REP_NODE => null,
+        ?REP_PID => null,
+        ?JOB_HISTORY => [AddedEvent],
+        ?CHECKPOINT_HISTORY => []
+    }.
+
+
+add_job(Tx, JobId, JobData) ->
+    couch_stats:increment_counter([couch_replicator, jobs, adds]),
+    couch_jobs_fdb:tx(couch_jobs_fdb:get_jtx(Tx), fun(JTx) ->
+        case couch_jobs:get_job_data(JTx, ?REP_JOBS, JobId) of
+            {ok, #{} = OldData} ->
+                ok = remove_job(JTx, JobId, OldData);
+            {error, not_found} ->
+                ok
+        end,
+        ok = couch_jobs:add(JTx, ?REP_JOBS, JobId, JobData)
+    end).
+
+
+remove_job(Tx, JobId) ->
+    couch_stats:increment_counter([couch_replicator, jobs, removes]),
+    couch_jobs_fdb:tx(couch_jobs_fdb:get_jtx(Tx), fun(JTx) ->
+        case couch_jobs:get_job_data(JTx, ?REP_JOBS, JobId) of
+            {ok, #{} = JobData} ->
+                ok = remove_job(JTx, JobId, JobData);
+            {error, not_found} ->
+                ok
+        end
+    end).
+
+
+get_job_data(Tx, JobId) ->
+    couch_jobs_fdb:tx(couch_jobs_fdb:get_jtx(Tx), fun(JTx) ->
+        couch_jobs:get_job_data(JTx, ?REP_JOBS, JobId)
+    end).
+
+
+% UserFun = fun(JTx, JobId, JobState, JobData, UserAcc)
+%
+fold_jobs(Tx, UserFun, Acc) when is_function(UserFun, 5) ->
+    couch_jobs_fdb:tx(couch_jobs_fdb:get_jtx(Tx), fun(JTx) ->
+        couch_jobs:fold_jobs(JTx, ?REP_JOBS, UserFun, Acc)
+    end).
+
+
+pending_count(_Tx, Limit) when is_integer(Limit), Limit =< 0 ->
+    0;
+
+pending_count(Tx, Limit) when is_integer(Limit), Limit > 0 ->
+    Opts = #{
+        max_sched_time => erlang:system_time(second),
+        limit => Limit
+    },
+    couch_jobs:pending_count(Tx, ?REP_JOBS, Opts).
+
+
+wait_running(JobId) ->
+    case couch_jobs:subscribe(?REP_JOBS, JobId) of
+        {ok, finished, JobData} ->
+            {ok, JobData};
+        {ok, SubId, running, #{?STATE := ?ST_PENDING}} ->
+            wait_running(JobId, SubId);
+        {ok, SubId, running, JobData} ->
+            ok = couch_jobs:unsubscribe(SubId),
+            {ok, JobData};
+        {ok, SubId, pending, _} ->
+            wait_running(JobId, SubId);
+        {error, Error} ->
+            {error, Error}
+    end.
+
+
+wait_running(JobId, SubId) ->
+    case couch_jobs:wait(SubId, running, infinity) of
+        {?REP_JOBS, _, running, #{?STATE := ?ST_PENDING}} ->
+            wait_running(JobId, SubId);
+        {?REP_JOBS, _, running, JobData} ->
+            ok = couch_jobs:unsubscribe(SubId),
+            {ok, JobData};
+        {?REP_JOBS, _, finished, JobData} ->
+            ok = couch_jobs:unsubscribe(SubId),
+            {ok, JobData}
+    end.
+
+
+wait_result(JobId) ->
+    case couch_jobs:subscribe(?REP_JOBS, JobId) of
+        {ok, finished, JobData} ->
+            {ok, JobData};
+        {ok, SubId, _, _} ->
+            {?REP_JOBS, _, finished, JobData} = couch_jobs:wait(SubId,
+                finished, infinity),
+            {ok, JobData};
+        {error, Error} ->
+            {error, Error}
+    end.
+
+
+accept_job(MaxSchedTime) when is_integer(MaxSchedTime) ->
+    Opts = #{max_sched_time => MaxSchedTime},
+    couch_jobs:accept(?REP_JOBS, Opts).
+
+
+update_job_data(Tx, #{} = Job, #{} = JobData) ->
+    couch_jobs_fdb:tx(couch_jobs_fdb:get_jtx(Tx), fun(JTx) ->
+        couch_jobs:update(JTx, Job, JobData)
+    end).
+
+
+finish_job(Tx, #{} = Job, #{} = JobData) ->
+    couch_jobs_fdb:tx(couch_jobs_fdb:get_jtx(Tx), fun(JTx) ->
+        couch_jobs:finish(JTx, Job, JobData)
+    end).
+
+
+reschedule_job(Tx, #{} = Job, #{} = JobData, Time) ->
+    couch_jobs_fdb:tx(couch_jobs_fdb:get_jtx(Tx), fun(JTx) ->
+        {ok, Job1} = couch_jobs:resubmit(JTx, Job, Time),
+        ok = couch_jobs:finish(JTx, Job1, JobData)
+    end).
+
+
+try_update_rep_id(Tx, JobId, RepId) ->
+    couch_jobs_fdb:tx(couch_jobs_fdb:get_jtx(Tx), fun(JTx) ->
+        #{tx := ErlFdbTx, layer_prefix := LayerPrefix} = JTx,
+        Key = erlfdb_tuple:pack({?REPLICATION_IDS, RepId}, LayerPrefix),
+        case get_job_id(JTx, RepId) of
+            {error, not_found} ->
+                ok = erlfdb:set(ErlFdbTx, Key, JobId);
+            {ok, JobId} ->
+                ok;
+            {ok, OtherJobId} when is_binary(OtherJobId) ->
+                {error, {replication_job_conflict, OtherJobId}}
+        end
+    end).
+
+
+update_rep_id(Tx, JobId, RepId) ->
+    couch_jobs_fdb:tx(couch_jobs_fdb:get_jtx(Tx), fun(JTx) ->
+        #{tx := ErlFdbTx, layer_prefix := LayerPrefix} = JTx,
+        Key = erlfdb_tuple:pack({?REPLICATION_IDS, RepId}, LayerPrefix),
+        ok = erlfdb:set(ErlFdbTx, Key, JobId)
+    end).
+
+
+clear_old_rep_id(_, _, null) ->
+    ok;
+
+clear_old_rep_id(Tx, JobId, RepId) ->
+    couch_jobs_fdb:tx(couch_jobs_fdb:get_jtx(Tx), fun(JTx) ->
+        #{tx := ErlFdbTx, layer_prefix := LayerPrefix} = JTx,
+        Key = erlfdb_tuple:pack({?REPLICATION_IDS, RepId}, LayerPrefix),
+        case get_job_id(JTx, RepId) of
+            {error, not_found} ->
+                ok;
+            {ok, JobId} ->
+                ok = erlfdb:clear(ErlFdbTx, Key);
+            {ok, OtherJobId} when is_binary(OtherJobId) ->
+                ok
+        end
+    end).
+
+
+get_job_id(Tx, RepId) ->
+    couch_jobs_fdb:tx(couch_jobs_fdb:get_jtx(Tx), fun(JTx) ->
+        #{tx := ErlFdbTx, layer_prefix := LayerPrefix} = JTx,
+        Key = erlfdb_tuple:pack({?REPLICATION_IDS, RepId}, LayerPrefix),
+        case erlfdb:wait(erlfdb:get(ErlFdbTx, Key)) of
+            not_found ->
+                {error, not_found};
+            <<_/binary>> = JobId ->
+                {ok, JobId}
+        end
+    end).
+
+
+% Debug functions
+
+remove_jobs(Tx, JobIds) when is_list(JobIds) ->
+    couch_jobs_fdb:tx(couch_jobs_fdb:get_jtx(Tx), fun(JTx) ->
+        lists:foreach(fun(JobId) -> remove_job(JTx, JobId) end, JobIds)
+    end),
+    [].
+
+
+get_job_ids(Tx) ->
+    couch_jobs_fdb:tx(couch_jobs_fdb:get_jtx(Tx), fun(JTx) ->
+        #{tx := ErlFdbTx, layer_prefix := LayerPrefix} = JTx,
+        Prefix = erlfdb_tuple:pack({?REPLICATION_IDS}, LayerPrefix),
+        KVs = erlfdb:wait(erlfdb:get_range_startswith(ErlFdbTx, Prefix)),
+        lists:map(fun({K, JobId}) ->
+            {RepId} = erlfdb_tuple:unpack(K, Prefix),
+            {RepId, JobId}
+        end, KVs)
+    end).
+
+
+% Private functions
+
+remove_job(#{jtx := true} = JTx, JobId, OldJobData) ->
+    #{tx := Tx, layer_prefix := LayerPrefix} = JTx,
+    case OldJobData of
+        #{?REP_ID := null} ->
+            couch_jobs:remove(JTx, ?REP_JOBS, JobId);
+        #{?REP_ID := RepId} when is_binary(RepId) ->
+            Key = erlfdb_tuple:pack({?REPLICATION_IDS, RepId}, LayerPrefix),
+            case erlfdb:wait(erlfdb:get(Tx, Key)) of
+                not_found -> ok;
+                JobId -> erlfdb:clear(Tx, Key);
+                <<_/binary>> -> ok
+            end,
+            couch_jobs:remove(JTx, ?REP_JOBS, JobId)
+    end.
diff --git a/src/fabric/include/fabric2.hrl b/src/fabric/include/fabric2.hrl
index 2eba4d5..ebbb7c7 100644
--- a/src/fabric/include/fabric2.hrl
+++ b/src/fabric/include/fabric2.hrl
@@ -26,6 +26,7 @@
 -define(DELETED_DBS, 3).
 -define(DBS, 15).
 -define(EXPIRING_CACHE, 53).
+-define(REPLICATION_IDS, 54).
 -define(TX_IDS, 255).
 
 % Cluster Level: (LayerPrefix, ?CLUSTER_CONFIG, X, ...)


[couchdb] 03/16: Handle possible iodata from jiffy:encode in couch_jobs

Posted by va...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

vatamane pushed a commit to branch prototype/fdb-layer
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit 9897cd84a37fa0062dfba15f2e0eb6cc611be74a
Author: Nick Vatamaniuc <va...@apache.org>
AuthorDate: Fri Aug 28 04:30:39 2020 -0400

    Handle possible iodata from jiffy:encode in couch_jobs
    
    Also, ensure to use the same options as other couch apps: force_utf8 and
    dedupe_keys.
---
 src/couch_jobs/src/couch_jobs_fdb.erl | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/src/couch_jobs/src/couch_jobs_fdb.erl b/src/couch_jobs/src/couch_jobs_fdb.erl
index 3fcad55..27131ec 100644
--- a/src/couch_jobs/src/couch_jobs_fdb.erl
+++ b/src/couch_jobs/src/couch_jobs_fdb.erl
@@ -414,7 +414,7 @@ init_cache() ->
 %
 encode_data(#{} = JobData) ->
     try
-        jiffy:encode(JobData)
+        iolist_to_binary(jiffy:encode(JobData, [force_utf8]))
     catch
         throw:{error, Error} ->
             % legacy clause since new versions of jiffy raise error instead
@@ -431,7 +431,7 @@ decode_data(#{} = JobData) ->
     JobData;
 
 decode_data(<<_/binary>> = JobData) ->
-    jiffy:decode(JobData, [return_maps]).
+    jiffy:decode(JobData, [dedupe_keys, return_maps]).
 
 
 % Cached job transaction object. This object wraps a transaction, caches the


[couchdb] 07/16: Move parsing and validation to couch_replicator_parse module

Posted by va...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

vatamane pushed a commit to branch prototype/fdb-layer
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit b38d77fbada7cce7de288d2cdcca8839b09888f4
Author: Nick Vatamaniuc <va...@apache.org>
AuthorDate: Fri Aug 28 04:32:32 2020 -0400

    Move parsing and validation to couch_replicator_parse module
    
    This module is in responsible for parsing either an HTTP `_replicate` request
    body, or a _replicator doc into an internal `Rep` object (an Erlang map).
    
    `parse_transient_rep/2` parses _replicate requests. It also handles
    cancelations, where requests bodies look like ```{"id": ..., "cancel": true}```
    instead of having all the expected parameters.
    
    `parse_rep_doc/1` parses _replicator docs.
    
    Parsing consists of 3 main parts:
    
     - Parsing the endpoint definitions: source and target url, headers, TLS bits
       and proxies
    
     - Parsing options into an options map, possibly using defaults from config
       parameters
    
     - Parsing socket parameters. These now have a hard-coded allow-list as opposed
       accepting all possible Erlang socket options.
    
    The parsing function also double as validation function which gets called from
    the _replicator's before_doc_update callback when users update replication
    documents. They would get an immediate feedback if their replicationd document
    is malformed.
    
    Everything is turned into a map object. This object should be able to be
    serialized and de-serialized to (from) JSON.
    
    Since maps are used, add the definitions of some common fields
    couch_replicator.hrl. Mistyping them should raise a compiler error.
    
    couch_replicator_docs lost all of its parsing function and also functions which
    update intermediate replication doc states (triggered and error). It still
    handles function which relate to interacting with _replicator docs.
---
 src/couch_replicator/src/couch_replicator.hrl      | 102 ++-
 src/couch_replicator/src/couch_replicator_docs.erl | 870 ++++-----------------
 .../src/couch_replicator_parse.erl                 | 545 +++++++++++++
 3 files changed, 751 insertions(+), 766 deletions(-)

diff --git a/src/couch_replicator/src/couch_replicator.hrl b/src/couch_replicator/src/couch_replicator.hrl
index 2a5b7c8..28a86d9 100644
--- a/src/couch_replicator/src/couch_replicator.hrl
+++ b/src/couch_replicator/src/couch_replicator.hrl
@@ -12,32 +12,80 @@
 
 -define(REP_ID_VERSION, 4).
 
--record(rep, {
-    id :: rep_id() | '_' | 'undefined',
-    source :: any() | '_',
-    target :: any() | '_',
-    options :: [_] | '_',
-    user_ctx :: any() | '_',
-    type = db :: atom() | '_',
-    view = nil :: any() | '_',
-    doc_id :: any() | '_',
-    db_name = null :: null | binary() | '_',
-    start_time = {0, 0, 0} :: erlang:timestamp() | '_',
-    stats = couch_replicator_stats:new() :: orddict:orddict() | '_'
-}).
-
--type rep_id() :: {string(), string()}.
+% Some fields from the replication doc
+-define(SOURCE, <<"source">>).
+-define(TARGET, <<"target">>).
+-define(CREATE_TARGET, <<"create_target">>).
+-define(DOC_IDS, <<"doc_ids">>).
+-define(SELECTOR, <<"selector">>).
+-define(FILTER, <<"filter">>).
+-define(QUERY_PARAMS, <<"query_params">>).
+-define(URL, <<"url">>).
+-define(AUTH, <<"auth">>).
+-define(HEADERS, <<"headers">>).
+-define(PROXY, <<"proxy">>).
+-define(SOURCE_PROXY, <<"source_proxy">>).
+-define(TARGET_PROXY, <<"target_proxy">>).
+
+-define(REPLICATION_STATE, <<"_replication_state">>).
+-define(REPLICATION_STATS, <<"_replication_stats">>).
+-define(REPLICATION_ID, <<"_replication_id">>).
+-define(REPLICATION_STATE_TIME, <<"_replication_state_time">>).
+-define(REPLICATION_STATE_REASON, <<"_replication_state_reason">>).
+
+% Replication states
+-define(ST_ERROR, <<"error">>).
+-define(ST_COMPLETED, <<"completed">>).
+-define(ST_RUNNING, <<"running">>).
+-define(ST_FAILED, <<"failed">>).
+-define(ST_PENDING, <<"pending">>).
+-define(ST_CRASHING, <<"crashing">>).
+
+% Some fields from a rep object
+-define(REP_ID, <<"rep_id">>).
+-define(BASE_ID, <<"base_id">>).
+-define(DB_NAME, <<"db_name">>).
+-define(DB_UUID, <<"db_uuid">>).
+-define(DOC_ID, <<"doc_id">>).
+-define(REP_USER, <<"rep_user">>).
+-define(START_TIME, <<"start_time">>).
+-define(OPTIONS, <<"options">>).
+
+% Fields for couch job data objects
+-define(REP, <<"rep">>).
+-define(REP_PARSE_ERROR, <<"rep_parse_error">>).
+-define(REP_STATS, <<"rep_stats">>).
+-define(STATE, <<"state">>).
+-define(STATE_INFO, <<"state_info">>).
+-define(DOC_STATE, <<"doc_state">>).
+-define(ERROR_COUNT, <<"error_count">>).
+-define(LAST_UPDATED, <<"last_updated">>).
+-define(LAST_START, <<"last_start">>).
+-define(LAST_ERROR, <<"last_error">>).
+-define(JOB_HISTORY, <<"job_history">>).
+-define(CHECKPOINT_HISTORY, <<"checkpoint_history">>).
+-define(REP_NODE, <<"node">>).
+-define(REP_PID, <<"pid">>).
+
+% Job history tags
+-define(HIST_TYPE, <<"type">>).
+-define(HIST_TIMESTAMP, <<"timestamp">>).
+-define(HIST_REASON, <<"reason">>).
+-define(HIST_ADDED, <<"added">>).
+-define(HIST_STARTED, <<"started">>).
+-define(HIST_STOPPED, <<"stopped">>).
+-define(HIST_PENDING, <<"pending">>).
+-define(HIST_CRASHED, <<"crashed">>).
+
+-define(REP_DB_NAME, <<"_replicator">>).
+
+% Can be used as a guard
+-define(IS_REP_DB(X), (X =:= ?REP_DB_NAME orelse
+    binary_part(X, {byte_size(X), -12}) =:= <<"/_replicator">>)).
+
+
+-type rep_id() :: binary().
+-type job_id() :: binary().
+-type user_name() :: binary() | null.
 -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/src/couch_replicator_docs.erl b/src/couch_replicator/src/couch_replicator_docs.erl
index 6190632..f84d129 100644
--- a/src/couch_replicator/src/couch_replicator_docs.erl
+++ b/src/couch_replicator/src/couch_replicator_docs.erl
@@ -13,306 +13,142 @@
 -module(couch_replicator_docs).
 
 -export([
-    parse_rep_doc/1,
-    parse_rep_doc/2,
-    parse_rep_db/3,
-    parse_rep_doc_without_id/1,
-    parse_rep_doc_without_id/2,
+    remove_state_fields/3,
+    update_completed/4,
+    update_failed/4,
     before_doc_update/3,
-    after_doc_read/2,
-    ensure_rep_ddoc_exists/1,
-    ensure_cluster_rep_ddoc_exists/1,
-    remove_state_fields/2,
-    update_doc_completed/3,
-    update_failed/3,
-    update_rep_id/1,
-    update_triggered/2,
-    update_error/2
+    after_doc_read/2
 ]).
 
 
 -include_lib("couch/include/couch_db.hrl").
--include_lib("ibrowse/include/ibrowse.hrl").
--include_lib("mem3/include/mem3.hrl").
 -include_lib("couch_replicator/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(REP_DB_NAME, <<"_replicator">>).
--define(REP_DESIGN_DOC, <<"_design/_replicator">>).
 -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_id">>, undefined},
-        {<<"_replication_stats">>, undefined}]).
+remove_state_fields(null, null, null) ->
+    ok;
 
+remove_state_fields(DbName, DbUUID, DocId) ->
+    update_rep_doc(DbName, DbUUID, DocId, [
+        {?REPLICATION_STATE, undefined},
+        {?REPLICATION_STATE_TIME, undefined},
+        {?REPLICATION_STATE_REASON, undefined},
+        {?REPLICATION_ID, undefined},
+        {?REPLICATION_STATS, undefined}
+    ]),
+    ok.
 
--spec update_doc_completed(binary(), binary(), [_]) -> any().
-update_doc_completed(DbName, DocId, Stats) ->
-    update_rep_doc(DbName, DocId, [
-        {<<"_replication_state">>, <<"completed">>},
-        {<<"_replication_state_reason">>, undefined},
-        {<<"_replication_stats">>, {Stats}}]),
-    couch_stats:increment_counter([couch_replicator, docs,
-        completed_state_updates]).
 
+-spec update_completed(binary(), binary(), binary(), [_]) -> ok.
+update_completed(null, null, _, _) ->
+    ok;
 
--spec update_failed(binary(), binary(), any()) -> any().
-update_failed(DbName, DocId, Error) ->
-    Reason = error_reason(Error),
-    couch_log:error("Error processing replication doc `~s` from `~s`: ~s",
-        [DocId, DbName, Reason]),
-    update_rep_doc(DbName, DocId, [
-        {<<"_replication_state">>, <<"failed">>},
-        {<<"_replication_stats">>, undefined},
-        {<<"_replication_state_reason">>, Reason}]),
+update_completed(DbName, DbUUID, DocId, #{} = Stats0) ->
+    Stats = {maps:to_list(Stats0)},
+    update_rep_doc(DbName, DbUUID, DocId, [
+        {?REPLICATION_STATE, ?ST_COMPLETED},
+        {?REPLICATION_STATE_REASON, undefined},
+        {?REPLICATION_STATS, Stats}]),
     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
-    } = Rep,
-    update_rep_doc(DbName, DocId, [
-        {<<"_replication_state">>, <<"triggered">>},
-        {<<"_replication_state_reason">>, undefined},
-        {<<"_replication_id">>, iolist_to_binary([Base, Ext])},
-        {<<"_replication_stats">>, undefined}]),
+        completed_state_updates
+    ]),
     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 update_failed(binary(), binary(), binary(), any()) -> ok.
+update_failed(null, null, null, _) ->
+    ok;
 
--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 ~p", [RepDb]),
-            {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 ~p", [RepDb]),
-                    try
-                        {ok, _} = save_rep_doc(RepDb, DDoc)
-                    catch
-                        throw:conflict ->
-                            %% ignore, we'll retry next time
-                            ok
-                    end
-            end
-    end,
+update_failed(DbName, DbUUID, DocId, Error) ->
+    Reason = error_reason(Error),
+    couch_log:error("Error processing replication doc `~s` from `~s`: ~s",
+        [DocId, DbName, Reason]),
+    update_rep_doc(DbName, DbUUID, DocId, [
+        {?REPLICATION_STATE, ?ST_FAILED},
+        {?REPLICATION_STATS, undefined},
+        {?REPLICATION_STATE_REASON, Reason}
+    ]),
+    couch_stats:increment_counter([couch_replicator, docs,
+        failed_state_updates]),
     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) ->
-    [
-        {<<"_id">>, DDocId},
-        {<<"language">>, <<"javascript">>},
-        {<<"validate_doc_update">>, ?REP_DB_DOC_VALIDATE_FUN}
-    ].
-
+-spec before_doc_update(#doc{}, Db::any(), couch_db:update_type()) -> #doc{}.
+before_doc_update(#doc{id = <<?DESIGN_DOC_PREFIX, _/binary>>} = Doc, _, _) ->
+    Doc;
+before_doc_update(#doc{body = {Body}} = Doc, Db, _UpdateType) ->
+    #user_ctx{roles = Roles, name = Name} = fabric2_db:get_user_ctx(Db),
+    IsReplicator = lists:member(<<"_replicator">>, Roles),
 
-% 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})})
+    Doc1 = case IsReplicator 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 fabric2_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,
-    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})})
+    Deleted = Doc1#doc.deleted,
+    IsFailed = couch_util:get_value(?REPLICATION_STATE, Body) == ?ST_FAILED,
+    case IsReplicator orelse Deleted orelse IsFailed of true -> ok; false ->
+        try
+            couch_replicator_parse:parse_rep_doc(Doc1#doc.body)
+        catch
+            throw:{bad_rep_doc, Error} ->
+                throw({forbidden, Error})
+        end
     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) ->
-    {SrcProxy, TgtProxy} = parse_proxy_settings(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), SrcProxy, Opts),
-        Target = parse_rep_db(get_value(<<"target">>, Props), TgtProxy, 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.
+    Doc1.
 
 
-parse_proxy_settings(Props) when is_list(Props) ->
-    Proxy = get_value(<<"proxy">>, Props, <<>>),
-    SrcProxy = get_value(<<"source_proxy">>, Props, <<>>),
-    TgtProxy = get_value(<<"target_proxy">>, Props, <<>>),
-
-    case Proxy =/= <<>> of
-        true when SrcProxy =/= <<>> ->
-            Error = "`proxy` is mutually exclusive with `source_proxy`",
-            throw({bad_request, Error});
-        true when  TgtProxy =/= <<>> ->
-            Error = "`proxy` is mutually exclusive with `target_proxy`",
-            throw({bad_request, Error});
-        true ->
-            {Proxy, Proxy};
-        false ->
-            {SrcProxy, TgtProxy}
+-spec after_doc_read(#doc{}, Db::any()) -> #doc{}.
+after_doc_read(#doc{id = <<?DESIGN_DOC_PREFIX, _/binary>>} = Doc, _Db) ->
+    Doc;
+after_doc_read(#doc{body = {Body}} = Doc, Db) ->
+    #user_ctx{name = Name} = fabric2_db:get_user_ctx(Db),
+    case (catch fabric2_db:check_is_admin(Db)) of ok -> Doc; _ ->
+        case couch_util:get_value(?OWNER, Body) of Name ->  Doc; _ ->
+            Source0 = couch_util:get_value(<<"source">>, Body),
+            Target0 = couch_util:get_value(<<"target">>, Body),
+            Source = strip_credentials(Source0),
+            Target = strip_credentials(Target0),
+            NewBody0 = ?replace(Body, <<"source">>, Source),
+            NewBody = ?replace(NewBody0, <<"target">>, Target),
+            #doc{revs = {Pos, [_ | Revs]}} = Doc,
+            NewDoc = Doc#doc{body = {NewBody}, revs = {Pos - 1, Revs}},
+            fabric2_db:new_revid(Db, NewDoc)
+        end
     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, RepDbUUID, RepDocId, KVs) ->
+    update_rep_doc(RepDbName, RepDbUUID, RepDocId, KVs, 1).
 
 
-update_rep_doc(RepDbName, RepDocId, KVs) ->
-    update_rep_doc(RepDbName, RepDocId, KVs, 1).
-
-
-update_rep_doc(RepDbName, RepDocId, KVs, Wait) when is_binary(RepDocId) ->
+update_rep_doc(RepDbName, RepDbUUID, RepDocId, KVs, Wait)
+        when is_binary(RepDocId) ->
     try
-        case open_rep_doc(RepDbName, RepDocId) of
+        case open_rep_doc(RepDbName, RepDbUUID, RepDocId) of
             {ok, LastRepDoc} ->
-                update_rep_doc(RepDbName, LastRepDoc, KVs, Wait * 2);
+                update_rep_doc(RepDbName, RepDbUUID, LastRepDoc, KVs,
+                    Wait * 2);
             _ ->
                 ok
         end
@@ -321,25 +157,25 @@ update_rep_doc(RepDbName, RepDocId, KVs, Wait) when is_binary(RepDocId) ->
             Msg = "Conflict when updating replication doc `~s`. Retrying.",
             couch_log:error(Msg, [RepDocId]),
             ok = timer:sleep(couch_rand:uniform(erlang:min(128, Wait)) * 100),
-            update_rep_doc(RepDbName, RepDocId, KVs, Wait * 2)
+            update_rep_doc(RepDbName, RepDbUUID, RepDocId, KVs, Wait * 2)
     end;
 
-update_rep_doc(RepDbName, #doc{body = {RepDocBody}} = RepDoc, KVs, _Try) ->
+update_rep_doc(RepDbName, RepDbUUID, #doc{body = {RepDocBody}} = RepDoc, KVs, _Try) ->
     NewRepDocBody = lists:foldl(
-        fun({K, undefined}, Body) ->
+        fun({K, undefined}, Body) when is_binary(K) ->
                 lists:keydelete(K, 1, Body);
-           ({<<"_replication_state">> = K, State} = KV, Body) ->
-                case get_json_value(K, Body) of
+           ({?REPLICATION_STATE = K, State} = KV, Body) when is_binary(K) ->
+                case couch_util:get_value(K, Body) of
                 State ->
                     Body;
                 _ ->
                     Body1 = lists:keystore(K, 1, Body, KV),
-                    Timestamp = couch_replicator_utils:iso8601(os:timestamp()),
+                    Timestamp = couch_replicator_utils:iso8601(),
                     lists:keystore(
-                        <<"_replication_state_time">>, 1, Body1,
-                        {<<"_replication_state_time">>, Timestamp})
+                        ?REPLICATION_STATE_TIME, 1, Body1,
+                        {?REPLICATION_STATE_TIME, Timestamp})
                 end;
-            ({K, _V} = KV, Body) ->
+            ({K, _V} = KV, Body) when is_binary(K) ->
                 lists:keystore(K, 1, Body, KV)
         end,
         RepDocBody, KVs),
@@ -349,331 +185,37 @@ update_rep_doc(RepDbName, #doc{body = {RepDocBody}} = RepDoc, KVs, _Try) ->
     _ ->
         % 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}})
+        save_rep_doc(RepDbName, RepDbUUID, 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
+open_rep_doc(DbName, DbUUID, DocId) when is_binary(DbName), is_binary(DbUUID),
+            is_binary(DocId) ->
+    try
+        case fabric2_db:open(DbName, [?CTX, sys_db, {uuid, DbUUID}]) of
+            {ok, Db} -> fabric2_db:open_doc(Db, DocId, [ejson_body]);
+            Else -> Else
+        end
+    catch
+        error:database_does_not_exist ->
+            {not_found, database_does_not_exist}
     end.
 
 
-save_rep_doc(DbName, Doc) ->
-    {ok, Db} = couch_db:open_int(DbName, [?CTX, sys_db]),
+save_rep_doc(DbName, DbUUID, Doc) when is_binary(DbName), is_binary(DbUUID) ->
     try
-        couch_db:update_doc(Db, Doc, [])
+        {ok, Db} = fabric2_db:open(DbName, [?CTX, sys_db, {uuid, DbUUID}]),
+        fabric2_db:update_doc(Db, Doc, [])
     catch
+        error:database_does_not_exist ->
+            {not_found, database_does_not_exist};
         % User can accidently write a VDU which prevents _replicator from
         % updating replication documents. Avoid crashing replicator and thus
         % preventing all other replication jobs on the node from running.
         throw:{forbidden, Reason} ->
-            Msg = "~p VDU function preventing doc update to ~s ~s ~p",
+            Msg = "~p VDU or BDU function preventing doc update to ~s ~s ~p",
             couch_log:error(Msg, [?MODULE, DbName, Doc#doc.id, Reason]),
             {ok, forbidden}
-    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,
-    #httpdb{
-        url = Url,
-        auth_props = AuthProps,
-        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(<<_/binary>>, _Proxy, _Options) ->
-    throw({error, local_endpoints_not_supported});
-
-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:member($?, Url) of
-        true ->
-            Url;  % skip if there are query params
-        false ->
-            case lists:last(Url) of
-                $/ ->
-                    Url;
-                _ ->
-                    Url ++ "/"
-            end
-    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", "5"),
-    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([{<<"create_target_params">>, V} | _R]) when not is_tuple(V) ->
-    throw({bad_request,
-        <<"parameter `create_target_params` must be a JSON object">>});
-convert_options([{<<"create_target_params">>, V} | R]) ->
-    [{create_target_params, 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"})
-    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::any(), couch_db:update_type()) -> #doc{}.
-before_doc_update(#doc{id = <<?DESIGN_DOC_PREFIX, _/binary>>} = Doc, _Db, _UpdateType) ->
-    Doc;
-before_doc_update(#doc{body = {Body}} = Doc, Db, _UpdateType) ->
-    #user_ctx{
-       roles = Roles,
-       name = Name
-    } = couch_db:get_user_ctx(Db),
-    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::any()) -> #doc{}.
-after_doc_read(#doc{id = <<?DESIGN_DOC_PREFIX, _/binary>>} = Doc, _Db) ->
-    Doc;
-after_doc_read(#doc{body = {Body}} = Doc, Db) ->
-    #user_ctx{name = Name} = couch_db:get_user_ctx(Db),
-    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.
 
 
@@ -698,164 +240,14 @@ strip_credentials({Props0}) ->
 error_reason({shutdown, Error}) ->
     error_reason(Error);
 error_reason({bad_rep_doc, Reason}) ->
-    to_binary(Reason);
+    couch_util:to_binary(Reason);
+error_reason(#{<<"error">> := Error, <<"reason">> := Reason})
+        when is_binary(Error), is_binary(Reason) ->
+    couch_util:to_binary(io_list:format("~s: ~s", [Error, Reason]));
 error_reason({error, {Error, Reason}})
-  when is_atom(Error), is_binary(Reason) ->
-    to_binary(io_lib:format("~s: ~s", [Error, Reason]));
+        when is_atom(Error), is_binary(Reason) ->
+    couch_util:to_binary(io_lib:format("~s: ~s", [Error, Reason]));
 error_reason({error, Reason}) ->
-    to_binary(Reason);
+    couch_util:to_binary(Reason);
 error_reason(Reason) ->
-    to_binary(Reason).
-
-
--ifdef(TEST).
-
-
--include_lib("couch/include/couch_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}]}])).
-
-check_strip_credentials_test() ->
-    [?assertEqual(Expected, strip_credentials(Body)) || {Expected, Body} <- [
-        {
-            undefined,
-            undefined
-        },
-        {
-            <<"https://remote_server/database">>,
-            <<"https://foo:bar@remote_server/database">>
-        },
-        {
-            {[{<<"_id">>, <<"foo">>}]},
-            {[{<<"_id">>, <<"foo">>}, {<<"headers">>, <<"bar">>}]}
-        },
-        {
-            {[{<<"_id">>, <<"foo">>}, {<<"other">>, <<"bar">>}]},
-            {[{<<"_id">>, <<"foo">>}, {<<"other">>, <<"bar">>}]}
-        },
-        {
-            {[{<<"_id">>, <<"foo">>}]},
-            {[{<<"_id">>, <<"foo">>}, {<<"headers">>, <<"baz">>}]}
-        },
-        {
-            {[{<<"_id">>, <<"foo">>}]},
-            {[{<<"_id">>, <<"foo">>}, {<<"auth">>, <<"pluginsecret">>}]}
-        }
-    ]].
-
-
-setup() ->
-    DbName = ?tempdb(),
-    {ok, Db} = couch_db:create(DbName, [?ADMIN_CTX]),
-    ok = couch_db:close(Db),
-    create_vdu(DbName),
-    DbName.
-
-
-teardown(DbName) when is_binary(DbName) ->
-    couch_server:delete(DbName, [?ADMIN_CTX]),
-    ok.
-
-
-create_vdu(DbName) ->
-    couch_util:with_db(DbName, fun(Db) ->
-        VduFun = <<"function(newdoc, olddoc, userctx) {throw({'forbidden':'fail'})}">>,
-        Doc = #doc{
-            id = <<"_design/vdu">>,
-            body = {[{<<"validate_doc_update">>, VduFun}]}
-        },
-        {ok, _} = couch_db:update_docs(Db, [Doc])
-    end).
-
-
-update_replicator_doc_with_bad_vdu_test_() ->
-    {
-        setup,
-        fun test_util:start_couch/0,
-        fun test_util:stop_couch/1,
-        {
-            foreach, fun setup/0, fun teardown/1,
-            [
-                fun t_vdu_does_not_crash_on_save/1
-            ]
-        }
-    }.
-
-
-t_vdu_does_not_crash_on_save(DbName) ->
-    ?_test(begin
-        Doc = #doc{id = <<"some_id">>, body = {[{<<"foo">>, 42}]}},
-        ?assertEqual({ok, forbidden}, save_rep_doc(DbName, Doc))
-    end).
-
-
-local_replication_endpoint_error_test_() ->
-     {
-        foreach,
-        fun () -> meck:expect(config, get,
-            fun(_, _, Default) -> Default end)
-        end,
-        fun (_) -> meck:unload() end,
-        [
-            t_error_on_local_endpoint()
-        ]
-    }.
-
-
-t_error_on_local_endpoint() ->
-    ?_test(begin
-        RepDoc = {[
-            {<<"_id">>, <<"someid">>},
-            {<<"source">>, <<"localdb">>},
-            {<<"target">>, <<"http://somehost.local/tgt">>}
-        ]},
-        Expect = local_endpoints_not_supported,
-        ?assertThrow({bad_rep_doc, Expect}, parse_rep_doc_without_id(RepDoc))
-    end).
-
--endif.
+    couch_util:to_binary(Reason).
diff --git a/src/couch_replicator/src/couch_replicator_parse.erl b/src/couch_replicator/src/couch_replicator_parse.erl
new file mode 100644
index 0000000..5996ec5
--- /dev/null
+++ b/src/couch_replicator/src/couch_replicator_parse.erl
@@ -0,0 +1,545 @@
+% 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_parse).
+
+
+-export([
+    parse_rep_doc/1,
+    parse_transient_rep/2,
+    parse_rep/2,
+    parse_rep_db/3
+]).
+
+
+-include_lib("ibrowse/include/ibrowse.hrl").
+-include("couch_replicator.hrl").
+
+
+-define(DEFAULT_SOCK_OPTS, "[{keepalive, true}, {nodelay, false}]").
+-define(VALID_SOCK_OPTS, [
+    buffer,
+    delay_send,
+    exit_on_close,
+    ipv6_v6only,
+    keepalive,
+    nodelay,
+    recbuf,
+    send_timeout,
+    send_timout_close,
+    sndbuf,
+    priority,
+    tos,
+    tclass
+]).
+-define(VALID_PROXY_PROTOCOLS, [http, https, socks5]).
+-define(CONFIG_DEFAULTS, [
+    {"worker_processes", "4", fun list_to_integer/1},
+    {"worker_batch_size", "500", fun list_to_integer/1},
+    {"http_connections", "20", fun list_to_integer/1},
+    {"connection_timeout", "30000", fun list_to_integer/1},
+    {"retries_per_request", "5", fun list_to_integer/1},
+    {"use_checkpoints", "true", fun list_to_existing_atom/1},
+    {"checkpoint_interval", "30000", fun list_to_integer/1},
+    {"socket_options", ?DEFAULT_SOCK_OPTS, fun parse_sock_opts/1}
+]).
+
+
+-spec parse_rep_doc({[_]}) -> #{}.
+parse_rep_doc(RepDoc) ->
+    {ok, Rep} = try
+        parse_rep(RepDoc, null)
+    catch
+        throw:{error, Reason} ->
+            Stack = erlang:get_stacktrace(),
+            LogErr1 = "~p parse_rep_doc fail ~p ~p",
+            couch_log:error(LogErr1, [?MODULE, Reason, Stack]),
+            throw({bad_rep_doc, Reason});
+        Tag:Err ->
+            Stack = erlang:get_stacktrace(),
+            LogErr2 = "~p parse_rep_doc fail ~p:~p ~p",
+            couch_log:error(LogErr2, [?MODULE, Tag, Err, Stack]),
+            throw({bad_rep_doc, couch_util:to_binary({Tag, Err})})
+    end,
+    Rep.
+
+
+-spec parse_transient_rep({[_]} | #{}, user_name()) -> {ok, #{}}.
+parse_transient_rep({Props} = EJson, UserName) when is_list(Props) ->
+    Str = couch_util:json_encode(EJson),
+    Map = couch_util:json_decode(Str, [return_maps]),
+    parse_transient_rep(Map, UserName);
+
+parse_transient_rep(#{} = Body, UserName) ->
+    {ok, Rep} = try
+        parse_rep(Body, UserName)
+    catch
+        throw:{error, Reason} ->
+            Stack = erlang:get_stacktrace(),
+            LogErr1 = "~p parse_transient_rep fail ~p ~p",
+            couch_log:error(LogErr1, [?MODULE, Reason, Stack]),
+            throw({bad_request, Reason});
+        Tag:Err ->
+            Stack = erlang:get_stacktrace(),
+            LogErr2 = "~p parse_transient_rep fail ~p ~p",
+            couch_log:error(LogErr2, [?MODULE, Tag, Err, Stack]),
+            throw({bad_request, couch_util:to_binary({Tag, Err})})
+    end,
+    #{?OPTIONS := Options} = Rep,
+    Cancel = maps:get(<<"cancel">>, Options, false),
+    Id = maps:get(<<"id">>, Options, nil),
+    case {Cancel, Id} of
+        {true, nil} ->
+            % Cancel request with no id, must parse id out of body contents
+            JobId = couch_replicator_ids:job_id(Rep, null, null),
+            {ok, JobId, Rep};
+        {true, Id} ->
+            % Cancel request with an id specified, so do not parse id from body
+            {ok, Id, Rep};
+        {false, _Id} ->
+            JobId = couch_replicator_ids:job_id(Rep, null, null),
+            % Not a cancel request, regular replication doc
+            {ok, JobId, Rep}
+    end.
+
+
+-spec parse_rep({[_]} | #{}, user_name()) -> {ok, #{}}.
+parse_rep({Props} = EJson, UserName) when is_list(Props) ->
+    Str = couch_util:json_encode(EJson),
+    Map = couch_util:json_decode(Str, [return_maps]),
+    parse_rep(Map, UserName);
+
+parse_rep(#{} = Doc, UserName) ->
+    {SrcProxy, TgtProxy} = parse_proxy_settings(Doc),
+    Opts = make_options(Doc),
+    Cancel = maps:get(<<"cancel">>, Opts, false),
+    Id = maps:get(<<"id">>, Opts, nil),
+    case Cancel andalso Id =/= nil of
+        true ->
+            {ok, #{?OPTIONS => Opts, ?REP_USER => UserName}};
+        false ->
+            case {maps:is_key(?SOURCE, Doc), maps:is_key(?TARGET, Doc)} of
+                {false, _} -> throw({error, <<"Missing `source` field">>});
+                {_, false} -> throw({error, <<"Missing `target` field">>});
+                {true, true} -> ok
+            end,
+            #{?SOURCE := Source0, ?TARGET := Target0} = Doc,
+            Source = parse_rep_db(Source0, SrcProxy, Opts),
+            Target = parse_rep_db(Target0, TgtProxy, Opts),
+            case couch_replicator_filters:view_type(Doc, Opts) of
+                {error, Error} -> throw({error, Error});
+                _ -> ok
+            end,
+            case couch_replicator_filters:parse(Opts) of
+                {ok, _} -> ok;
+                {error, FilterError} -> throw({error, FilterError})
+            end,
+            Rep = #{
+                ?SOURCE => Source,
+                ?TARGET => Target,
+                ?OPTIONS => Opts,
+                ?REP_USER => UserName,
+                ?START_TIME => erlang:system_time(second)
+            },
+            {ok, Rep}
+    end.
+
+
+-spec parse_rep_db(#{}, #{}, #{}) -> #{}.
+parse_rep_db(#{} = Endpoint, #{} = ProxyParams, #{} = Options) ->
+    ProxyUrl = case ProxyParams of
+        #{<<"proxy_url">> := PUrl} -> PUrl;
+        _ -> null
+    end,
+
+    Url0 = maps:get(<<"url">>, Endpoint),
+    Url = maybe_add_trailing_slash(Url0),
+
+    AuthProps = maps:get(<<"auth">>, Endpoint, #{}),
+    if is_map(AuthProps) -> ok; true ->
+        throw({error, "if defined, `auth` must be an object"})
+    end,
+
+    Headers0 = maps:get(<<"headers">>, Endpoint, #{}),
+    if is_map(Headers0) -> ok; true ->
+        throw({error, "if defined `headers` must be an object"})
+    end,
+    DefaultHeaders = couch_replicator_utils:default_headers_map(),
+    Headers = maps:merge(DefaultHeaders, Headers0),
+
+    SockOpts = maps:get(<<"socket_options">>, Options, #{}),
+    SockAndProxy = maps:merge(#{
+        <<"socket_options">> => SockOpts
+    }, ProxyParams),
+    SslParams = ssl_params(Url),
+
+    #{
+        <<"url">> => Url,
+        <<"auth_props">> => AuthProps,
+        <<"headers">> => Headers,
+        <<"ibrowse_options">> => maps:merge(SslParams, SockAndProxy),
+        <<"timeout">> => maps:get(<<"connection_timeout">>, Options),
+        <<"http_connections">> => maps:get(<<"http_connections">>, Options),
+        <<"retries">> => maps:get(<<"retries_per_request">>, 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(<<_/binary>>, _Proxy, _Options) ->
+    throw({error, local_endpoints_not_supported});
+
+parse_rep_db(undefined, _Proxy, _Options) ->
+    throw({error, <<"Missing replication endpoint">>}).
+
+
+parse_proxy_settings(#{} = Doc) ->
+    Proxy = maps:get(?PROXY, Doc, <<>>),
+    SrcProxy = maps:get(?SOURCE_PROXY, Doc, <<>>),
+    TgtProxy = maps:get(?TARGET_PROXY, Doc, <<>>),
+
+    case Proxy =/= <<>> of
+        true when SrcProxy =/= <<>> ->
+            Error = "`proxy` is mutually exclusive with `source_proxy`",
+            throw({error, Error});
+        true when  TgtProxy =/= <<>> ->
+            Error = "`proxy` is mutually exclusive with `target_proxy`",
+            throw({error, Error});
+        true ->
+            {parse_proxy_params(Proxy), parse_proxy_params(Proxy)};
+        false ->
+            {parse_proxy_params(SrcProxy), parse_proxy_params(TgtProxy)}
+    end.
+
+
+-spec maybe_add_trailing_slash(binary()) -> binary().
+maybe_add_trailing_slash(<<>>) ->
+    <<>>;
+
+maybe_add_trailing_slash(Url) when is_binary(Url) ->
+    case binary:match(Url, <<"?">>) of
+        nomatch ->
+            case binary:last(Url) of
+                $/  -> Url;
+                _ -> <<Url/binary, "/">>
+            end;
+        _ ->
+            Url  % skip if there are query params
+    end.
+
+
+-spec make_options(#{}) -> #{}.
+make_options(#{} = RepDoc) ->
+    Options0 = convert_options(RepDoc),
+    Options = check_options(Options0),
+    ConfigOptions = lists:foldl(fun({K, Default, ConversionFun}, Acc) ->
+        V = ConversionFun(config:get("replicator", K, Default)),
+        Acc#{list_to_binary(K) => V}
+    end, #{}, ?CONFIG_DEFAULTS),
+    maps:merge(ConfigOptions, Options).
+
+
+-spec convert_options(#{}) -> #{} | no_return().
+convert_options(#{} = Doc) ->
+    maps:fold(fun convert_fold/3, #{}, Doc).
+
+
+-spec convert_fold(binary(), any(), #{}) -> #{}.
+convert_fold(<<"cancel">>, V, Acc) when is_boolean(V) ->
+    Acc#{<<"cancel">> => V};
+convert_fold(<<"cancel">>, _, _) ->
+    throw({error, <<"`cancel` must be a boolean">>});
+convert_fold(IdOpt, V, Acc) when IdOpt =:= <<"_local_id">>;
+        IdOpt =:= <<"replication_id">>; IdOpt =:= <<"id">> ->
+    Acc#{<<"id">> => couch_replicator_ids:convert(V)};
+convert_fold(<<"create_target">>, V, Acc) when is_boolean(V) ->
+    Acc#{<<"create_target">> => V};
+convert_fold(<<"create_target">>, _, _) ->
+    throw({error, <<"`create_target` must be a boolean">>});
+convert_fold(<<"create_target_params">>, #{} = V, Acc) ->
+    Acc#{<<"create_target_params">> => V};
+convert_fold(<<"create_target_params">>, _, _) ->
+    throw({error, <<"`create_target_params` must be an object">>});
+convert_fold(<<"continuous">>, V, Acc) when is_boolean(V) ->
+    Acc#{<<"continuous">> => V};
+convert_fold(<<"continuous">>, _, _) ->
+    throw({error, <<"`continuous` must be a boolean">>});
+convert_fold(<<"filter">>, V, Acc) when is_binary(V), byte_size(V) > 1 ->
+    Acc#{<<"filter">> => V};
+convert_fold(<<"filter">>, _, _) ->
+    throw({error, <<"`filter` must be a string">>});
+convert_fold(<<"query_params">>, V, Acc) when is_map(V) orelse V =:= null ->
+    Acc#{<<"query_params">> => V};
+convert_fold(<<"query_params">>, _, _Acc) ->
+    throw({error, <<"`query_params` is not `null` or object">>});
+convert_fold(<<"doc_ids">>, null, Acc) ->
+    Acc;
+convert_fold(<<"doc_ids">>, V, Acc) when is_list(V) ->
+    % Compatibility behaviour as: accept a list of percent encoded doc IDs
+    Ids = lists:map(fun(Id) ->
+        case is_binary(Id) andalso byte_size(Id) > 0 of
+            true -> list_to_binary(couch_httpd:unquote(Id));
+            false -> throw({error, <<"`doc_ids` array must contain strings">>})
+        end
+    end, V),
+    Acc#{<<"doc_ids">> => lists:usort(Ids)};
+convert_fold(<<"doc_ids">>, _, _) ->
+    throw({error, <<"`doc_ids` must be an array">>});
+convert_fold(<<"selector">>, #{} = V, Acc) ->
+    Acc#{<<"selector">> => V};
+convert_fold(<<"selector">>, _, _Acc) ->
+    throw({error, <<"`selector` must be a JSON object">>});
+convert_fold(<<"worker_processes">>, V, Acc) ->
+    Acc#{<<"worker_processes">> => bin2int(V, <<"worker_processes">>)};
+convert_fold(<<"worker_batch_size">>, V, Acc) ->
+    Acc#{<<"worker_batch_size">> => bin2int(V, <<"worker_batch_size">>)};
+convert_fold(<<"http_connections">>, V, Acc) ->
+    Acc#{<<"http_connections">> => bin2int(V, <<"http_connections">>)};
+convert_fold(<<"connection_timeout">>, V, Acc) ->
+    Acc#{<<"connection_timeout">> => bin2int(V, <<"connection_timeout">>)};
+convert_fold(<<"retries_per_request">>, V, Acc) ->
+    Acc#{<<"retries_per_request">> => bin2int(V, <<"retries_per_request">>)};
+convert_fold(<<"socket_options">>, V, Acc) ->
+    Acc#{<<"socket_options">> => parse_sock_opts(V)};
+convert_fold(<<"since_seq">>, V, Acc) ->
+    Acc#{<<"since_seq">> => V};
+convert_fold(<<"use_checkpoints">>, V, Acc) when is_boolean(V) ->
+    Acc#{<<"use_checkpoints">> => V};
+convert_fold(<<"use_checkpoints">>, _, _) ->
+    throw({error, <<"`use_checkpoints` must be a boolean">>});
+convert_fold(<<"checkpoint_interval">>, V, Acc) ->
+    Acc#{<<"checkpoint_interval">> => bin2int(V, <<"checkpoint_interval">>)};
+convert_fold(_K, _V, Acc) -> % skip unknown option
+    Acc.
+
+
+bin2int(V, _Field) when is_integer(V) ->
+    V;
+
+bin2int(V, Field) when is_binary(V) ->
+    try
+        erlang:binary_to_integer(V)
+    catch
+        error:badarg ->
+            throw({error, <<"`", Field/binary, "` must be an integer">>})
+    end;
+
+bin2int(_V, Field) ->
+    throw({error, <<"`", Field/binary, "` must be an integer">>}).
+
+
+-spec check_options(#{}) -> #{}.
+check_options(Options) ->
+    DocIds = maps:is_key(<<"doc_ids">>, Options),
+    Filter = maps:is_key(<<"filter">>, Options),
+    Selector = maps:is_key(<<"selector">>, Options),
+    case {DocIds, Filter, Selector} of
+        {false, false, false} -> Options;
+        {false, false, _} -> Options;
+        {false, _, false} -> Options;
+        {_, false, false} -> Options;
+        _ -> throw({error, <<"`doc_ids`,`filter`,`selector` are mutually "
+            " exclusive">>})
+    end.
+
+
+parse_sock_opts(Term) ->
+    {ok, SocketOptions} = couch_util:parse_term(Term),
+    lists:foldl(fun
+        ({K, V}, Acc) when is_atom(K) ->
+            case lists:member(K, ?VALID_SOCK_OPTS) of
+                true -> Acc#{atom_to_binary(K, utf8) => V};
+                false -> Acc
+            end;
+        (_, Acc) ->
+            Acc
+    end, #{}, SocketOptions).
+
+
+-spec parse_proxy_params(binary() | #{}) -> #{}.
+parse_proxy_params(<<>>) ->
+    #{};
+parse_proxy_params(ProxyUrl) when is_binary(ProxyUrl)->
+    #url{
+        host = Host,
+        port = Port,
+        username = User,
+        password = Passwd,
+        protocol = Prot0
+    } = ibrowse_lib:parse_url(binary_to_list(ProxyUrl)),
+    Prot = case lists:member(Prot0, ?VALID_PROXY_PROTOCOLS) of
+        true -> atom_to_binary(Prot0, utf8);
+        false -> throw({error, <<"Unsupported proxy protocol">>})
+    end,
+    ProxyParams = #{
+        <<"proxy_url">> => ProxyUrl,
+        <<"proxy_protocol">> => Prot,
+        <<"proxy_host">> => list_to_binary(Host),
+        <<"proxy_port">> => Port
+    },
+    case is_list(User) andalso is_list(Passwd) of
+        true ->
+            ProxyParams#{
+                <<"proxy_user">> => list_to_binary(User),
+                <<"proxy_password">> => list_to_binary(Passwd)
+            };
+        false ->
+            ProxyParams
+    end.
+
+
+-spec ssl_params(binary()) -> #{}.
+ssl_params(Url) ->
+    case ibrowse_lib:parse_url(binary_to_list(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", null),
+            KeyFile = config:get("replicator", "key_file", null),
+            Password = config:get("replicator", "password", null),
+            VerifySslOptions = ssl_verify_options(VerifyCerts =:= "true"),
+            SslOpts = maps:merge(VerifySslOptions, #{<<"depth">> => Depth}),
+            HaveCertAndKey = CertFile /= null andalso KeyFile /= null,
+            SslOpts1 = case HaveCertAndKey of false -> SslOpts; true ->
+                CertOpts0 = #{
+                    <<"certfile">> => list_to_binary(CertFile),
+                    <<"keyfile">> => list_to_binary(KeyFile)
+                },
+                CertOpts = case Password of null -> CertOpts0; _ ->
+                    CertOpts0#{<<"password">> => list_to_binary(Password)}
+                end,
+                maps:merge(SslOpts, CertOpts)
+            end,
+            #{<<"is_ssl">> => true, <<"ssl_options">> => SslOpts1};
+        #url{protocol = http} ->
+            #{}
+    end.
+
+
+-spec ssl_verify_options(true | false) -> [_].
+ssl_verify_options(true) ->
+    case config:get("replicator", "ssl_trusted_certificates_file") of
+        undefined ->
+            #{
+                <<"verify">> => <<"verify_peer">>,
+                <<"cacertfile">> => null
+            };
+        CAFile when is_list(CAFile) ->
+            #{
+                <<"verify">> => <<"verify_peer">>,
+                <<"cacertfile">> => list_to_binary(CAFile)
+            }
+    end;
+
+ssl_verify_options(false) ->
+    #{
+        <<"verify">> => <<"verify_none">>
+    }.
+
+
+-ifdef(TEST).
+
+-include_lib("couch/include/couch_eunit.hrl").
+-include_lib("fabric/test/fabric2_test.hrl").
+
+
+check_options_pass_values_test() ->
+    ?assertEqual(check_options(#{}), #{}),
+    ?assertEqual(check_options(#{<<"baz">> => <<"foo">>}),
+        #{<<"baz">> => <<"foo">>}),
+    ?assertEqual(check_options(#{<<"doc_ids">> => [<<"x">>]}),
+        #{<<"doc_ids">> => [<<"x">>]}),
+    ?assertEqual(check_options(#{<<"filter">> => <<"f">>}),
+        #{<<"filter">> => <<"f">>}),
+    ?assertEqual(check_options(#{<<"selector">> => <<"s">>}),
+        #{<<"selector">> => <<"s">>}).
+
+
+check_options_fail_values_test() ->
+    ?assertThrow({error, _},
+        check_options(#{<<"doc_ids">> => [], <<"filter">> => <<"f">>})),
+    ?assertThrow({error, _},
+        check_options(#{<<"doc_ids">> => [], <<"selector">> => <<"s">>})),
+    ?assertThrow({error, _},
+        check_options(#{<<"filter">> => <<"f">>, <<"selector">> => <<"s">>})),
+    ?assertThrow({error, _},
+        check_options(#{
+            <<"doc_ids">> => [],
+            <<"filter">> => <<"f">>,
+            <<"selector">> => <<"s">>}
+    )).
+
+
+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({error, _},
+        convert_options(#{<<"cancel">> => <<"true">>})),
+    ?assertThrow({error, _},
+        convert_options(#{<<"create_target">> => <<"true">>})),
+    ?assertThrow({error, _},
+        convert_options(#{<<"continuous">> => <<"true">>})),
+    ?assertThrow({error, _},
+        convert_options(#{<<"doc_ids">> => <<"not_a_list">>})),
+    ?assertThrow({error, _},
+        convert_options(#{<<"selector">> => <<"bad">>})).
+
+
+local_replication_endpoint_error_test_() ->
+    {
+        foreach,
+        fun setup/0,
+        fun teardown/1,
+        [
+            ?TDEF_FE(t_error_on_local_endpoint)
+        ]
+    }.
+
+
+setup() ->
+    meck:expect(config, get, fun(_, _, Default) -> Default end).
+
+
+teardown(_) ->
+    meck:unload().
+
+
+t_error_on_local_endpoint(_) ->
+    RepDoc = {[
+        {<<"_id">>, <<"someid">>},
+        {<<"source">>, <<"localdb">>},
+        {<<"target">>, <<"http://somehost.local/tgt">>}
+    ]},
+    Expect = local_endpoints_not_supported,
+    ?assertThrow({bad_rep_doc, Expect}, parse_rep_doc(RepDoc)).
+
+
+-endif.


[couchdb] 13/16: Update replicator http handlers and supervisor

Posted by va...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

vatamane pushed a commit to branch prototype/fdb-layer
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit aa8836434655b9b1b34c6d72035eb306266c484b
Author: Nick Vatamaniuc <va...@apache.org>
AuthorDate: Fri Aug 28 12:49:59 2020 -0400

    Update replicator http handlers and supervisor
    
    Stich everything together: the backend, frontend and http handlers.
    
    The supervisor `couch_replicator_sup` handles starting a set of fronted or
    backend children. It may also start both or neither.
    
    The HTTP layer for monitoring and creating jobs is simpler than before since
    there is rpc and clustering involved.
---
 src/chttpd/src/chttpd_misc.erl                     |  55 +------
 src/couch_replicator/src/couch_replicator.app.src  |  11 +-
 .../src/couch_replicator_httpd.erl                 | 163 ++++++++++-----------
 src/couch_replicator/src/couch_replicator_sup.erl  | 113 +++++++-------
 4 files changed, 149 insertions(+), 193 deletions(-)

diff --git a/src/chttpd/src/chttpd_misc.erl b/src/chttpd/src/chttpd_misc.erl
index ec2435c..79c2914 100644
--- a/src/chttpd/src/chttpd_misc.erl
+++ b/src/chttpd/src/chttpd_misc.erl
@@ -302,17 +302,22 @@ handle_task_status_req(Req) ->
 handle_replicate_req(#httpd{method='POST', user_ctx=Ctx, req_body=PostBody} = Req) ->
     chttpd:validate_ctype(Req, "application/json"),
     %% see HACK in chttpd.erl about replication
-    case replicate(PostBody, Ctx) of
+    case couch_replicator:replicate(PostBody, Ctx) of
         {ok, {continuous, RepId}} ->
             send_json(Req, 202, {[{ok, true}, {<<"_local_id">>, RepId}]});
         {ok, {cancelled, RepId}} ->
             send_json(Req, 200, {[{ok, true}, {<<"_local_id">>, RepId}]});
-        {ok, {JsonResults}} ->
-            send_json(Req, {[{ok, true} | JsonResults]});
+        {ok, #{} = JsonResults} ->
+            send_json(Req, maps:merge(#{<<"ok">> => true}, JsonResults));
         {ok, stopped} ->
             send_json(Req, 200, {[{ok, stopped}]});
         {error, not_found=Error} ->
             chttpd:send_error(Req, Error);
+        {error, #{<<"error">> := Err, <<"reason">> := Reason}} when
+                is_binary(Err), is_binary(Reason) ->
+            % Safe to use binary_to_atom since this is only built
+            % from couch_replicator_jobs:error_info/1
+            chttpd:send_error(Req, {binary_to_atom(Err, utf8), Reason});
         {error, {_, _}=Error} ->
             chttpd:send_error(Req, Error);
         {_, _}=Error ->
@@ -321,50 +326,6 @@ handle_replicate_req(#httpd{method='POST', user_ctx=Ctx, req_body=PostBody} = Re
 handle_replicate_req(Req) ->
     send_method_not_allowed(Req, "POST").
 
-replicate({Props} = PostBody, Ctx) ->
-    case couch_util:get_value(<<"cancel">>, Props) of
-    true ->
-        cancel_replication(PostBody, Ctx);
-    _ ->
-        Node = choose_node([
-            couch_util:get_value(<<"source">>, Props),
-            couch_util:get_value(<<"target">>, Props)
-        ]),
-        case rpc:call(Node, couch_replicator, replicate, [PostBody, Ctx]) of
-        {badrpc, Reason} ->
-            erlang:error(Reason);
-        Res ->
-            Res
-        end
-    end.
-
-cancel_replication(PostBody, Ctx) ->
-    {Res, _Bad} = rpc:multicall(couch_replicator, replicate, [PostBody, Ctx]),
-    case [X || {ok, {cancelled, _}} = X <- Res] of
-    [Success|_] ->
-        % Report success if at least one node canceled the replication
-        Success;
-    [] ->
-        case lists:usort(Res) of
-        [UniqueReply] ->
-            % Report a universally agreed-upon reply
-            UniqueReply;
-        [] ->
-            {error, badrpc};
-        Else ->
-            % Unclear what to do here -- pick the first error?
-            % Except try ignoring any {error, not_found} responses
-            % because we'll always get two of those
-            hd(Else -- [{error, not_found}])
-        end
-    end.
-
-choose_node(Key) when is_binary(Key) ->
-    Checksum = erlang:crc32(Key),
-    Nodes = lists:sort([node()|erlang:nodes()]),
-    lists:nth(1 + Checksum rem length(Nodes), Nodes);
-choose_node(Key) ->
-    choose_node(term_to_binary(Key)).
 
 handle_reload_query_servers_req(#httpd{method='POST'}=Req) ->
     chttpd:validate_ctype(Req, "application/json"),
diff --git a/src/couch_replicator/src/couch_replicator.app.src b/src/couch_replicator/src/couch_replicator.app.src
index 2e0e191..81789f1 100644
--- a/src/couch_replicator/src/couch_replicator.app.src
+++ b/src/couch_replicator/src/couch_replicator.app.src
@@ -18,20 +18,15 @@
         couch_replicator_sup,
         couch_replicator_rate_limiter,
         couch_replicator_connection,
-        couch_replication,  % couch_replication_event gen_event
-        couch_replicator_clustering,
-        couch_replicator_scheduler,
-        couch_replicator_scheduler_sup,
-        couch_replicator_doc_processor
+        couch_replicator_job_server
     ]},
     {applications, [
         kernel,
         stdlib,
         couch_log,
-        mem3,
         config,
         couch,
-        couch_event,
-        couch_stats
+        couch_stats,
+        couch_jobs
     ]}
 ]}.
diff --git a/src/couch_replicator/src/couch_replicator_httpd.erl b/src/couch_replicator/src/couch_replicator_httpd.erl
index abd9f7f..196fcf2 100644
--- a/src/couch_replicator/src/couch_replicator_httpd.erl
+++ b/src/couch_replicator/src/couch_replicator_httpd.erl
@@ -12,9 +12,6 @@
 
 -module(couch_replicator_httpd).
 
--include_lib("couch/include/couch_db.hrl").
--include_lib("couch_mrview/include/couch_mrview.hrl").
-
 -export([
     handle_req/1,
     handle_scheduler_req/1
@@ -26,48 +23,40 @@
     send_method_not_allowed/2
 ]).
 
--import(couch_util, [
-    to_binary/1
-]).
+
+-include_lib("couch/include/couch_db.hrl").
+-include("couch_replicator.hrl").
 
 
 -define(DEFAULT_TASK_LIMIT, 100).
--define(REPDB, <<"_replicator">>).
-% This is a macro so it can be used as a guard
--define(ISREPDB(X), X =:= ?REPDB orelse binary_part(X, {byte_size(X), -12})
-    =:= <<"/_replicator">>).
 
 
 handle_scheduler_req(#httpd{method='GET', path_parts=[_,<<"jobs">>]}=Req) ->
-    Limit = couch_replicator_httpd_util:parse_int_param(Req, "limit",
+    Limit = couch_replicator_utils:parse_int_param(Req, "limit",
         ?DEFAULT_TASK_LIMIT, 0, infinity),
-    Skip = couch_replicator_httpd_util:parse_int_param(Req, "skip", 0, 0,
+    Skip = couch_replicator_utils:parse_int_param(Req, "skip", 0, 0,
         infinity),
-    {Replies, _BadNodes} = rpc:multicall(couch_replicator_scheduler, jobs, []),
-    Flatlist = lists:concat(Replies),
-    % couch_replicator_scheduler:job_ejson/1 guarantees {id, Id} to be the
-    % the first item in the list
-    Sorted = lists:sort(fun({[{id,A}|_]},{[{id,B}|_]}) -> A =< B end, Flatlist),
-    Total = length(Sorted),
+    Jobs1 = couch_replicator:jobs(),
+    Total = length(Jobs1),
     Offset = min(Skip, Total),
-    Sublist = lists:sublist(Sorted, Offset+1, Limit),
-    Sublist1 = [couch_replicator_httpd_util:update_db_name(Task)
-        || Task <- Sublist],
-    send_json(Req, {[{total_rows, Total}, {offset, Offset}, {jobs, Sublist1}]});
+    Jobs2 = lists:sublist(Jobs1, Offset + 1, Limit),
+    send_json(Req, #{
+        <<"total_rows">> => Total,
+        <<"offset">> => Offset,
+        <<"jobs">> => Jobs2
+    });
 handle_scheduler_req(#httpd{method='GET', path_parts=[_,<<"jobs">>,JobId]}=Req) ->
     case couch_replicator:job(JobId) of
-        {ok, JobInfo} ->
-            send_json(Req, couch_replicator_httpd_util:update_db_name(JobInfo));
-        {error, not_found} ->
-            throw(not_found)
+        {ok, JobInfo} ->  send_json(Req, JobInfo);
+        {error, not_found} -> throw(not_found)
     end;
 handle_scheduler_req(#httpd{method='GET', path_parts=[_,<<"docs">>]}=Req) ->
-    handle_scheduler_docs(?REPDB, Req);
+    handle_scheduler_docs(?REP_DB_NAME, Req);
 handle_scheduler_req(#httpd{method='GET', path_parts=[_,<<"docs">>,Db]}=Req)
-        when ?ISREPDB(Db) ->
+        when ?IS_REP_DB(Db) ->
     handle_scheduler_docs(Db, Req);
 handle_scheduler_req(#httpd{method='GET', path_parts=[_,<<"docs">>,Db,DocId]}
-        = Req)  when ?ISREPDB(Db) ->
+        = Req)  when ?IS_REP_DB(Db) ->
     handle_scheduler_doc(Db, DocId, Req);
 % Allow users to pass in unencoded _replicator database names (/ are not
 % escaped). This is possible here because _replicator is not a valid document
@@ -82,77 +71,80 @@ handle_scheduler_req(#httpd{method='GET', path_parts=[_,<<"docs">>|Unquoted]}
         {error, invalid} ->
             throw(bad_request)
     end;
-handle_scheduler_req(#httpd{method='GET'} = Req) ->
-    send_json(Req, 404, {[{error, <<"not found">>}]});
+handle_scheduler_req(#httpd{method='GET'} = _Req) ->
+    throw(not_found);
 handle_scheduler_req(Req) ->
     send_method_not_allowed(Req, "GET,HEAD").
 
 
 handle_req(#httpd{method = 'POST', user_ctx = UserCtx} = Req) ->
     couch_httpd:validate_ctype(Req, "application/json"),
-    RepDoc = {Props} = couch_httpd:json_body_obj(Req),
-    couch_replicator_httpd_util:validate_rep_props(Props),
+    RepDoc = couch_httpd:json_body_obj(Req),
     case couch_replicator:replicate(RepDoc, UserCtx) of
-    {error, {Error, Reason}} ->
-        send_json(
-            Req, 500,
-            {[{error, to_binary(Error)}, {reason, to_binary(Reason)}]});
-    {error, not_found} ->
-        % Tried to cancel a replication that didn't exist.
-        send_json(Req, 404, {[{error, <<"not found">>}]});
-    {error, Reason} ->
-        send_json(Req, 500, {[{error, to_binary(Reason)}]});
-    {ok, {cancelled, RepId}} ->
-        send_json(Req, 200, {[{ok, true}, {<<"_local_id">>, RepId}]});
-    {ok, {continuous, RepId}} ->
-        send_json(Req, 202, {[{ok, true}, {<<"_local_id">>, RepId}]});
-    {ok, {HistoryResults}} ->
-        send_json(Req, {[{ok, true} | HistoryResults]})
-    end;
+        {error, {Error, Reason}} ->
+            send_json(Req, 500, #{
+                <<"error">> => couch_util:to_binary(Error),
+                <<"reason">> => couch_util:to_binary(Reason)
+            });
+        {error, not_found} ->
+            throw(not_found);
+        {error, Reason} ->
+            send_json(Req, 500, #{<<"error">> => couch_util:to_binary(Reason)});
+        {ok, {cancelled, JobId}} ->
+            send_json(Req, 200, #{<<"ok">> => true, <<"_local_id">> => JobId});
+        {ok, {continuous, JobId}} ->
+            send_json(Req, 202, #{<<"ok">> => true, <<"_local_id">> => JobId});
+        {ok, #{} = CheckpointHistory} ->
+            Res = maps:merge(#{<<"ok">> => true}, CheckpointHistory),
+            send_json(Req, Res)
+        end;
 
 handle_req(Req) ->
     send_method_not_allowed(Req, "POST").
 
 
-handle_scheduler_docs(Db, Req) when is_binary(Db) ->
-    VArgs0 = couch_mrview_http:parse_params(Req, undefined),
-    StatesQs = chttpd:qs_value(Req, "states"),
-    States = couch_replicator_httpd_util:parse_replication_state_filter(StatesQs),
-    VArgs1 = VArgs0#mrargs{
-        view_type = map,
-        include_docs = true,
-        reduce = false,
-        extra = [{filter_states, States}]
-    },
-    VArgs2 = couch_mrview_util:validate_args(VArgs1),
-    Opts = [{user_ctx, Req#httpd.user_ctx}],
-    Max = chttpd:chunked_response_buffer_size(),
-    Acc = couch_replicator_httpd_util:docs_acc_new(Req, Db, Max),
-    Cb = fun couch_replicator_httpd_util:docs_cb/2,
-    {ok, RAcc} = couch_replicator_fabric:docs(Db, Opts, VArgs2, Cb, Acc),
-    {ok,  couch_replicator_httpd_util:docs_acc_response(RAcc)}.
-
-
-handle_scheduler_doc(Db, DocId, Req) when is_binary(Db), is_binary(DocId) ->
-    UserCtx = Req#httpd.user_ctx,
-    case couch_replicator:doc(Db, DocId, UserCtx#user_ctx.roles) of
-        {ok, DocInfo} ->
-            send_json(Req, couch_replicator_httpd_util:update_db_name(DocInfo));
-        {error, not_found} ->
+handle_scheduler_docs(DbName, #httpd{user_ctx = UserCtx} = Req) ->
+    try fabric2_db:open(DbName, [{user_ctx, UserCtx}]) of
+        {ok, Db} ->
+            ok = fabric2_db:check_is_member(Db),
+            StatesQs = chttpd:qs_value(Req, "states"),
+            States = couch_replicator_utils:parse_replication_states(StatesQs),
+            Docs = couch_replicator:docs(Db, States),
+            send_json(Req, #{
+                <<"total_rows">> => length(Docs),
+                <<"offset">> => 0,
+                <<"docs">> => Docs
+            })
+    catch
+        error:database_does_not_exist ->
             throw(not_found)
     end.
 
 
+handle_scheduler_doc(DbName, DocId, #httpd{user_ctx = UserCtx} = Req) ->
+     try fabric2_db:open(DbName, [{user_ctx, UserCtx}]) of
+        {ok, Db} ->
+             ok = fabric2_db:check_is_member(Db),
+             case couch_replicator:doc(Db, DocId) of
+                {ok, DocInfo} ->  send_json(Req, DocInfo);
+                {error, not_found} -> throw(not_found)
+             end
+     catch
+         error:database_does_not_exist ->
+             throw(not_found)
+     end.
+
+
 parse_unquoted_docs_path([_, _ | _] = Unquoted) ->
-    DbAndAfter = lists:dropwhile(fun(E) -> E =/= ?REPDB end, Unquoted),
-    BeforeRDb = lists:takewhile(fun(E) -> E =/= ?REPDB end, Unquoted),
+    DbAndAfter = lists:dropwhile(fun(E) -> E =/= ?REP_DB_NAME end, Unquoted),
+    BeforeRDb = lists:takewhile(fun(E) -> E =/= ?REP_DB_NAME end, Unquoted),
     case DbAndAfter of
         [] ->
             {error, invalid};
-        [?REPDB] ->
-            {db_only, filename:join(BeforeRDb ++ [?REPDB])};
-        [?REPDB, DocId] ->
-            {db_and_doc, filename:join(BeforeRDb ++ [?REPDB]), DocId}
+        [?REP_DB_NAME] ->
+            {db_only, filename:join(BeforeRDb ++ [?REP_DB_NAME])};
+        [?REP_DB_NAME, DocId] ->
+            {db_and_doc, filename:join(BeforeRDb ++ [?REP_DB_NAME]), DocId}
     end.
 
 
@@ -163,10 +155,13 @@ parse_unquoted_docs_path([_, _ | _] = Unquoted) ->
 unquoted_scheduler_docs_path_test_() ->
     [?_assertEqual(Res, parse_unquoted_docs_path(Path)) || {Res, Path} <- [
         {{error, invalid}, [<<"a">>,<< "b">>]},
-        {{db_only, <<"a/_replicator">>}, [<<"a">>, ?REPDB]},
-        {{db_only, <<"a/b/_replicator">>}, [<<"a">>, <<"b">>, ?REPDB]},
-        {{db_and_doc, <<"_replicator">>, <<"x">>}, [?REPDB, <<"x">>]},
-        {{db_and_doc, <<"a/_replicator">>, <<"x">>}, [<<"a">>, ?REPDB, <<"x">>]},
+        {{db_only, <<"a/_replicator">>}, [<<"a">>, ?REP_DB_NAME]},
+        {{db_only, <<"a/b/_replicator">>}, [<<"a">>, <<"b">>,
+            ?REP_DB_NAME]},
+        {{db_and_doc, <<"_replicator">>, <<"x">>},
+            [?REP_DB_NAME, <<"x">>]},
+        {{db_and_doc, <<"a/_replicator">>, <<"x">>}, [<<"a">>,
+            ?REP_DB_NAME, <<"x">>]},
         {{error, invalid}, [<<"a/_replicator">>,<<"x">>]}
     ]].
 
diff --git a/src/couch_replicator/src/couch_replicator_sup.erl b/src/couch_replicator/src/couch_replicator_sup.erl
index cd4512c..49d412a 100644
--- a/src/couch_replicator/src/couch_replicator_sup.erl
+++ b/src/couch_replicator/src/couch_replicator_sup.erl
@@ -12,61 +12,66 @@
 % the License.
 
 -module(couch_replicator_sup).
+
+
 -behaviour(supervisor).
--export([start_link/0, init/1]).
+
+
+-export([
+    start_link/0
+]).
+
+-export([
+    init/1
+]).
+
 
 start_link() ->
-    supervisor:start_link({local, ?MODULE}, ?MODULE, []).
+    Backend = fabric2_node_types:is_type(replication),
+    Frontend = fabric2_node_types:is_type(api_frontend),
+    Arg = {Backend, Frontend},
+    supervisor:start_link({local, ?MODULE}, ?MODULE, Arg).
+
+
+init({Backend, Frontend}) ->
+    Children = case {Backend, Frontend} of
+        {true, true} -> backend() ++ frontend();
+        {true, false} -> backend();
+        {false, true} -> frontend();
+        {false, false} -> []
+    end,
+    Flags =  #{
+        strategy => rest_for_one,
+        intensity => 1,
+        period => 5
+    },
+    {ok, {Flags, Children}}.
+
+
+backend() ->
+    Timeout = 5000,
+    [
+        #{
+            id => couch_replicator_connection,
+            start => {couch_replicator_connection, start_link, []}
+        },
+        #{
+            id => couch_replicator_rate_limiter,
+            start => {couch_replicator_rate_limiter, start_link, []}
+        },
+        #{
+            id => couch_replicator_job_server,
+            start => {couch_replicator_job_server, start_link, [Timeout]},
+            shutdown => Timeout
+        }
+    ].
+
 
-init(_Args) ->
-    Children = [
-        {couch_replication_event,
-            {gen_event, start_link, [{local, couch_replication}]},
-            permanent,
-            brutal_kill,
-            worker,
-            dynamic},
-       {couch_replicator_clustering,
-            {couch_replicator_clustering, start_link, []},
-            permanent,
-            brutal_kill,
-            worker,
-            [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_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_db_changes,
-            {couch_replicator_db_changes, start_link, []},
-            permanent,
-            brutal_kill,
-            worker,
-            [couch_multidb_changes]}
-    ],
-    {ok, {{rest_for_one,10,1}, Children}}.
+frontend() ->
+    [
+        #{
+            id => couch_replicator,
+            start => {couch_replicator, ensure_rep_db_exists, []},
+            restart => transient
+        }
+    ] ++ couch_epi:register_service(couch_replicator_epi, []).