You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@couchdb.apache.org by GitBox <gi...@apache.org> on 2020/08/12 23:03:32 UTC

[GitHub] [couchdb-documentation] nickva opened a new pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

nickva opened a new pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581


   Draft replicator RFC


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] ksnavely commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
ksnavely commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r470130225



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,359 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.

Review comment:
       It might be worth clarifying that no other activities take place other than replication on a `replication` node.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] rjharmon commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
rjharmon commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r469612352



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,337 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: ''
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs`
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md)
+for centralized scheduling and monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB 2 and 3 replication jobs were maped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of replication job at a
+time on each node. The new design proposes using `couch_jobs`, as described in
+this
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogenous node types as defined in this
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and
+replication jobs will be run only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Replication jobs created with the `"continuous":
+true` parameter. When this job reaches the end of the changes feed it will
+continue waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true`as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs jobs can only be run on these nodes.
+
+`filtered` replications: Replications which use a user-defined filter on the
+source endpoint to filter its changes feed.
+
+`replication_id` : A unique ID defined for replication jobs which is a hash of
+ the source and target endpoints, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ ID will change, if for example, a filter on the source endpoint changes.
+ Computing this value may require a network round-trip to the source endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many
+replication jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies up to how many new jobs
+to spawn during some a configurable scheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the base)
+penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuraiton parameter specifyign the maximum pently
+applied to job which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replicator application's job creation and scheduling works roughly as follows.
+
+ 1) `Persistent` jobs are created by document updates in `_replicator` dbs and
+ `transient` jobs are created by POST-ing to the `_replicate/` endpoint. In
+ both cases a new `couch_jobs` job is created in a separate `couch_jobs`
+ replication subspace. This happens on `api_frontend` nodes only. These are
+ also the nodes which process all the regular HTTP CRUD requests.
+
+   a) Persisent jobs are driven by an EPI callback mechanism which notifies
+   `couch_replicator` application when documents in `_replicator` dbs are
+   updated, when `_replicator` dbs are created and deleted.
+
+   b) Transient jobs are created from the `_replicate` HTTP handler directly.
+ Newly created jobs are in `pending` state.
+
+ 2) Each `replication` node spawns a small (configurable) amount of acceptor
+ processes which wait in `couch_jobs:accept/2` for jobs scheduled to run at a
+ time less or equal to the current time.
+
+ 3) After the job is accepted its state is updated as `running` and then a
+ gen_server server process monitoring these replication jobs will spawn another
+ acceptor to replace it. That happens until the `max_jobs` running jobs have
+ been spawned. The same gen_server will periodically check if there are any
+ pending jobs in the queue, and spawn up to some `max_churn` number of new
+ acceptors. These new acceptors may start new jobs, and if they do, for each
+ one of them, the oldest running job will be stopped and re-enqueued as
+ pending. This in large follows the logic from the replication scheduler in
+ CouchDB <= 3.x except that is uses `couch_jobs` as the central queueing and
+ scheduling mechanism.
+
+ 4) After the job is marked as `running` it computes the `replication_id`,
+ initializes an internal replication state record from the JSON job data
+ record, and start replicating.
+
+ 5) As the job runs, when it checkpoints it periodically recomputes its
+ `replication_id`. In case of filtered replications the `replication_id` maybe
+ change. In that case the job is stopped and moved to the `pending` state so it
+ can be restarted with the new `replication_id`. Also during checkpointing
+ attempts, the `couch_jobs`'s data is updated such that the job stays active
+ and doesn't get re-enqueued by the `couch_jobs` activity monitor.
+
+ 6) If the job crashes, in `gen_server:terminate/2` it reschedules itself via
+ `couch_jobs:resubmit/3` to run at some future time defined roughly as `now +
+ max(min_backoff_penalty * 2^consecutive_errors, max_backoff_penalty)`. In
+ order to avoid job which had intermetent error in teh past but since "healed"
+ if the job runs successfully after some period of time, configurable as
+ `health_threshold`, the `consecutive_errors` count is reset back to 0.
+
+ 7) If the node where replication job runs crashes, or the job is manually
+ killed via `exit(Pid, kill)`, `couch_jobs` activity monitor will automatically
+ re-enqueue the job to run as `pending`.
+
+## Replicator Job States
+
+### Description
+
+The new replicator in CouchDB 4.x will have less replication job states then before. The
+new set of states will be:
+
+ * `pending` : A job is marked as `pending` in these cases:
+    - As soon as a job is created from an `api_frontend` node
+    - Job is stopped to let other replication jobs run
+    - A filtered replication noticed its `replication_id` has changed
+
+ * `running` : When the job accepted by the `couch_jobs:accept/2` call. This
+   generally means the job is actually running on a node, however in case when
+   a node crashes immediately, the job may indicate `running` on that node
+   until `couch_jobs` activity monitor will re-enqueue the job
+
+ * `crashing` : Then job was running, but then crashed with an intermetent
+   error. Job data has an error count which is bumped and then a bakcoff pently
+   is computed and the the job is rescheduled to try again at some point in the
+   future.
+
+ * `completed` : Normal replications which have completed
+
+ * `failed` : This can happen when
+    - A replication job could not be parsed from a replication document or
+      `_replicate/` POST body. For example is the user has not specified a
+      `"source"` field.
+    - There is another persistent replication job running or pending with the
+      same `replication_id`.
+
+
+### Differences from CouchDB <= 3.x
+
+The differences from before are:
+ * `initializing` state was folded into `pending`
+ * `error` state folded into `crashing`
+
+
+### Mapping Between couch_jobs States and Replication States
+
+The state map from these states to `couch_jobs`'s states of (`pending`,
+`running`, `finished`) looks as follows:
+
+replicator state | couch_jobs state
+------------------------------------
+pending          | pending
+running          | running
+crashing         | pending
+completed        | finished
+failed           | finished
+
+### State Transition Diagram
+
+```
++-----+       +-------+
+|start+------>+pending|
++-----+       +-------+
+                  ^
+                  |
+                  |
+                  v
+              +---+---+      +--------+
+    +---------+running+----->|crashing|
+    |         +---+---+      +--------+
+    |             ^
+    |             |
+    v             v
++------+     +---------+
+|failed|     |completed|
++------+     +---------+
+```
+
+
+## Replication ID and Job IDs mapping
+
+Multiple replication jobs may specify a replications which map to the same
+`replication_id`. To handle duplicates and collission there is a `(...,
+LayerPrefix, ?REPLICATION_IDS, ReplicationID) -> JobID` subspace which keeps
+track of mappings from `replication_id` to `job id`. After the `replication_id`
+is computed each replication job checks if there is already another job pending
+or running with the same `replication_id`. If the other job is transient, then
+the current job will reschedule itself as `crashing`. If the other job is
+persistent, the current job will fail permanently as `failed`.
+
+## Transient Jobs Behavior
+
+In CouchDB <= 3.x transient replication jobs ran in memory on the nodes where
+they were assigned to run. If the node where the replication job ran crashed
+the job would simply disappear without a trace. It was up to the user's code to
+periodically monitor the job status and re-create the job. With the current
+design, `transient` jobs are saved as `couch_jobs` records in FDB and so would
+survive if dbcore nodes crashing and restarting. Technically with the new
+design it becomes possible to keep the transient job results even the failed
+and completed jobs, perhaps up to some configurable amount of time to allow
+users to retrieve their status after they stop running. However in the initial
+implementation, in order to keep maximum compatibility with the <= 3.x
+replicator, transient jobs will be removed when they fail and when they
+complete.
+
+
+## Monitoring Endpoints
+
+`_active_tasks`, `_scheduler/jobs` and `_scheduler/docs` endpoint are handled
+by traversing the replication job's data using a new `couch_jobs:fold_jobs/4`
+function and returning subsets of job's data. `_active_tasks` implementation
+already works that way and `_scheduler/*` endpoint will work similarly.
+
+
+# Advantages and Disadvantages
+
+Advantages:
+
+ * Simplicity: re-using `couch_jobs` means having a lot less code to maintain
+   in `couch_replicator`
+
+ * Simpler endpoint and monitoring implementations
+
+ * Less replication job states to keep track of
+
+ * Improved behavior for transient replications, they know survive node

Review comment:
       s/know/now




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r472284237



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.

Review comment:
       Good point. I'll update it to mention that the job may be temporarily paused to let other jobs make progress




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r472320366



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.

Review comment:
       It's described in the `couch_jobs` [RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md#job-creation-api). A job may be scheduled to run at some point in the future.  The `accept/2` call has a parameter to only accept job in a certain time window. And in this case it is used to schedule job execution at a later time.
   
   Another example where we use it is to remove soft-deleted db instances: https://github.com/apache/couchdb/blob/prototype/fdb-layer/src/fabric/src/fabric2_db_expiration.erl#L151




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva merged pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva merged pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r472284721



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the

Review comment:
       Agree, I've been using both `normal` and `one-shot`. I'll go with `one-shot`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r472330819



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recompute their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpointing the job's data value is updated with
+ stats such that the job stays active and doesn't get re-enqueued by the
+ `couch_jobs` activity monitor.
+
+ 7) If the job crashes, it will reschedule itself in `gen_server:terminate/2`

Review comment:
       `terminate` is guaranteed to be called when gen_server handlers return  a `stop` message and if `trap_exit = true ` when it is killed with any exit signal except `kill`. We set `process process_flag(trap_exit, true)` already in that job since we handle exit messages from replication workers and change feed readers. 
   
   In case someone sends the `kill` signal by hand, there is nothing else that would do it otherwise, or the node where job runs crashes, the couch_jobs activity monitor will automatically re-enqueue the job so it is guaranteed to be rescheduled again.
   
   https://blog.differentpla.net/blog/2014/11/13/erlang-terminate/#exit-reasons




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r472330819



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recompute their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpointing the job's data value is updated with
+ stats such that the job stays active and doesn't get re-enqueued by the
+ `couch_jobs` activity monitor.
+
+ 7) If the job crashes, it will reschedule itself in `gen_server:terminate/2`

Review comment:
       `terminate` is guaranteed to be called when gen_server handlers return  a `stop` message and if `trap_exit = true ` when it is killed with any exit signal except (`kill`). We set `process process_flag(trap_exit, true)` already in that job since we handle exit messages from replication workers and change feed readers. 
   
   In case someone sends the `kill` signal by hand, there is nothing else that would do it otherwise, or the node where job runs crashes, the couch_jobs activity monitor will automatically re-enqueue the job so it is guaranteed to be rescheduled again.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r472311961



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.

Review comment:
       Agree. That's a better description. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r472331371



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recompute their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpointing the job's data value is updated with
+ stats such that the job stays active and doesn't get re-enqueued by the
+ `couch_jobs` activity monitor.
+
+ 7) If the job crashes, it will reschedule itself in `gen_server:terminate/2`
+ via `couch_jobs:resubmit/3` call to run again at some future time, defined
+ roughly as `now + max(min_backoff_penalty * 2^consecutive_errors,
+ max_backoff_penalty)`. If a job starts and successfully runs for some
+ predefined period of time without crashing, it is considered to be `"healed"`
+ and its `consecutive_errors` count is reset to 0.
+
+ 8) If the node where replication job runs crashes, or the job is manually
+ killed via `exit(Pid, kill)`, `couch_jobs` activity monitor will automatically
+ re-enqueue the job as `pending`.
+
+## Replicator Job States
+
+### Description
+
+The set of replication job states is defined as:
+
+ * `pending` : A job is marked as `pending` in these cases:
+    - As soon as a job is created from an `api_frontend` node
+    - When it stopped to let other replication jobs run
+    - When a filtered replication's `replication_id` changes
+
+ * `running` : Set when a job is accepted by the `couch_jobs:accept/2`
+   call. This generally means the job is actually running on a node,
+   however, in cases when a node crashes, the job may show as
+   `running` on that node until `couch_jobs` activity monitor
+   re-enqueues the job, and it starts running on another node.
+
+ * `crashing` : The job was running, but then crashed with an intermittent
+   error. Job's data has an error count which is incremented, and then a
+   backoff penalty is computed and the job is rescheduled to try again at some
+   point in the future.
+
+ * `completed` : Normal replications which have completed
+
+ * `failed` : This can happen when:
+    - A replication job could not be parsed from a replication document. For
+      example, if the user has not specified a `"source"` field.
+    - A transient replication job crashes. Transient jobs don't get rescheduled
+      to run again after they crash.
+    - There already is another persistent replication job running or pending
+      with the same `replication_id`.
+
+### State Differences From CouchDB <= 3.x
+
+The set of states is slightly different than the ones from before. There are
+now fewer states as some of them have been combined together:
+
+ * `initializing` was combined with `pending`
+
+ * `error` was combined with `crashing`
+
+### Mapping Between couch_jobs States and Replication States
+
+`couch_jobs` application has its own set of state definitions and they map to
+replicator states like so:
+
+ | Replicator States| `couch_jobs` States
+ | ---              | :--
+ | pending          | pending
+ | running          | running
+ | crashing         | pending
+ | completed        | finished
+ | failed           | finished
+
+### State Transition Diagram
+
+Jobs start in the `pending` state, after either a `_replicator` db doc
+update, or a POST to the `/_replicate` endpoint. Continuous jobs, will
+normally toggle between `pending` and `running` states. Normal jobs
+may toggle between `pending` and running a few times and then end up
+in `completed`.
+
+```
+_replicator doc       +-------+
+POST /_replicate ---->+pending|
+                      +-------+
+                          ^
+                          |
+                          |
+                          v
+                      +---+---+      +--------+
+            +---------+running+<---->|crashing|
+            |         +---+---+      +--------+
+            |             ^
+            |             |
+            v             v
+        +------+     +---------+
+        |failed|     |completed|
+        +------+     +---------+
+```
+
+
+## Replication ID Collisions
+
+Multiple replication jobs may specify replications which map to the same
+`replication_id`. To handle these collisions there is an FDB subspace `(...,
+LayerPrefix, ?REPLICATION_IDS, replication_id) -> job_id` to keep track of
+them. After the `replication_id` is computed, each replication job checks if
+there is already another job pending or running with the same `replication_id`.
+If the other job is transient, then the current job will reschedule itself as
+`crashing`. If the other job is persistent, the current job will fail
+permanently as `failed`.
+
+## Replication Parameter Validation
+
+`_replicator` documents in CouchDB <= 3.x were parsed and validated in a
+two-step process:
+
+  1) In a validate-doc-update (VDU) javascript function from a programmatically
+  inserted _design document. This validation happened when the document was
+  updated, and performed some rough checks on field names and value types. If
+  this validation failed, the document update operation was rejected.
+
+  2) Inside replicator's Erlang code when it was translated to an internal
+ record used by the replication application. This validation was more thorough
+ but didn't have very friendly error messages. If validation failed here, the
+ job would be marked as `failed`.
+
+For CouchDB 4.x the proposal is to use only the Erlang parser. It would be
+called from the `before_doc_update` callback. This is a callback which runs
+before every document update. If validation fails there it would reject the
+document update operation. This should reduce code duplication and also provide
+better feedback to the users directly when they update the `_replicator`
+documents.
+
+## Transient Job Behavior
+
+In CouchDB <= 3.x transient replication jobs ran in memory on a particular node
+in the cluster. If the node where the replication job ran crashes, the job
+would simply disappear without a trace. It was up to the user to periodically
+monitor the job status and re-create the job. In the current design,
+`transient` jobs are persisted to FDB as `couch_jobs` records, and so would
+survive dbcore node restarts. Also after transient jobs complete or failed,

Review comment:
       Good point. That's just a silly name for a node. I'll remove it




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r470684356



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,359 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recomputes their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpoint-ing the job's data value is updated with
+ stats such that the job stays active and doesn't get re-enqueued by the
+ `couch_jobs` activity monitor.
+
+ 7) If the job crashes, it will reschedule itself in `gen_server:terminate/2`
+ via `couch_jobs:resubmit/3` call to run again at some future time, defined
+ roughly as `now + max(min_backoff_penalty * 2^consecutive_errors,
+ max_backoff_penalty)`. If a job starts and successfully runs for some
+ predefined period of time without crashing, it is considered to be `"healed"`
+ and its `consecutive_errors` count is reset to 0.
+
+ 8) If the node where replication job runs crashes, or the job is manually
+ killed via `exit(Pid, kill)`, `couch_jobs` activity monitor will automatically
+ re-enqueue the job as `pending`.
+
+## Replicator Job States
+
+### Description
+
+The set of replication job states is defined as:
+
+ * `pending` : A job is marked as `pending` in these cases:
+    - As soon as a job is created from an `api_frontend` node
+    - When it stopped to let other replication jobs run
+    - When a filtered replication's `replication_id` changes
+
+ * `running` : Set when a job is accepted by the `couch_jobs:accept/2`
+   call. This generally means the job is actually running on a node,
+   however, in cases when a node crashes, the job may show as
+   `running` on that node until `couch_jobs` activity monitor
+   re-enqueues the job, and it starts running on another node.
+
+ * `crashing` : The job was running, but then crashed with an intermittent
+   error. Job's data has an error count which is incremented, and then a
+   backoff penalty is computed and the job is rescheduled to try again at some
+   point in the future.
+
+ * `completed` : Normal replications which have completed
+
+ * `failed` : This can happen when:
+    - A replication job could not be parsed from a replication document. For
+      example, if the user has not specified a `"source"` field.
+    - There already is another persistent replication job running or pending
+      with the same `replication_id`.
+
+### State Differences From CouchDB <= 3.x
+
+The set of states is slightly different than the ones from before. There are
+now less states as some of them have been combined together:
+
+ * `initializing` was combined with `pending`
+
+ * `error` was combined with `crashing`
+
+### Mapping Between couch_jobs States and Replication States
+
+`couch_jobs` application has its own set of state definitions and they map to
+replicator states like so:
+
+ | Replicator States| `couch_jobs` States
+ | ---              | :--
+ | pending          | pending
+ | running          | running
+ | crashing         | pending
+ | completed        | finished
+ | failed           | finished
+
+### State Transition Diagram
+
+Jobs start in the `pending` state, after either a `_replicator` db doc
+update, or a POST to the `/_replicate` endpoint. Continuous jobs, will
+normally toggle between `pending` and `running` states. Normal jobs
+may toggle between `pending` and running a few times and then end up
+in `completed`.
+
+```
+_replicator doc       +-------+
+POST /_replicate ---->+pending|
+                      +-------+
+                          ^
+                          |
+                          |
+                          v
+                      +---+---+      +--------+
+            +---------+running+<---->|crashing|
+            |         +---+---+      +--------+
+            |             ^
+            |             |
+            v             v
+        +------+     +---------+
+        |failed|     |completed|
+        +------+     +---------+
+```
+
+
+## Replication ID Collisions
+
+Multiple replication jobs may specify replications which map to the same
+`replication_id`. To handle these collisions there is an FDB subspace `(...,
+LayerPrefix, ?REPLICATION_IDS, replication_id) -> job_id` to keep track of
+them. After the `replication_id` is computed, each replication job checks if
+there is already another job pending or running with the same `replication_id`.
+If the other job is transient, then the current job will reschedule itself as
+`crashing`. If the other job is persistent, the current job will fail
+permanently as `failed`.
+
+## Transient Job Behavior
+
+In CouchDB <= 3.x transient replication jobs ran in memory on a particular node
+in the cluster. If the node where the replication job ran crashes, the job
+would simply disappear without a trace. It was up to the user to periodically
+monitor the job status and re-create the job. In the current design,
+`transient` jobs are persisted to FDB as `couch_jobs` records, and so would
+survive dbcore node restarts. Technically, with the new design it becomes
+possible to keep the transient job results, including completed job status to
+allow users to retrieve their state later. However in the initial
+implementation, in order to keep maximum compatibility with the <= 3.x
+replicator, transient jobs will be removed when they fail and when they
+complete.
+
+## Monitoring Endpoints
+
+`_active_tasks`, `_scheduler/jobs` and `_scheduler/docs` endpoint are handled
+by traversing the replication job's data using a new `couch_jobs:fold_jobs/4`
+API call and inspect each job's data. `_active_tasks` implementation already
+works that way and `_scheduler/*` endpoint will work similarly.
+
+## Replication Documents Not Updated For Transient Errors
+
+Configuration
+[option](https://docs.couchdb.org/en/latest/replication/replicator.html?highlight=update_docs#compatibility-mode)
+`[replicator] update_docs = false` was introduced with the scheduling
+replicator in a 2.x release. It controls whether to update replication document
+with transient states like `triggered` and `error`. It defaulted to `false` and
+was mainly for compatibility with older monitoring code. That behavior now
+becomes hard-coded such that replication documents are only updated with
+terminal states of `failed` and `completed`. Users should use `_scheduler/docs`
+API to check for completion status instead.
+
+
+# Advantages and Disadvantages
+
+Advantages:
+
+ * Simplicity: re-using `couch_jobs` means having a lot less code to maintain
+   in `couch_replicator`. In the draft implementation there are about 3000
+   lines of code saved compared to the replicator application in CouchDB 3.x
+
+ * Simpler endpoint and monitoring implementation
+
+ * Less replication job states to keep track of
+
+ * Transient replications can now survive node crashes and restarts
+
+ * Using node types allows tightening firewall rules such that only
+   `replication` nodes are the ones which may make arbitrary requests outside
+   the cluster, and `frontend_api` nodes are the only ones that may accept
+   incoming connections.
+
+Disadvantages:
+
+ * Behavior changes for transient jobs
+
+ * Centralized job queue might mean handling some number of conflicts generated
+   in the FDB backend when jobs are accepted. These are mitigated using a
+   `accept_jitter` configuration parameter and a configurable number of max
+   acceptors per node.
+
+ * In monitoring API responses, `running` job state might not immediately
+   reflect the actually running process state on the replication node. If the
+   node crashes, it might take up to a minute or two until the job is
+   re-enqueued by the `couch_jobs` activity monitor.
+
+# Key Changes
+
+ * Behavior changes for transient jobs
+
+ * A delay in `running` state as reflected in monitoring API responses
+
+ * `[replicator] update_docs = false` configuration option becomes hard-coded
+ 
+## Applications and Modules affected
+
+ * couch_jobs : New APIs to fold jobs and get pending count job estimate
+
+ * fabric2_db : Adding EPI db create/delete callbacks
+
+ * couch_replicator :
+    - Remove `couch_replicator_scheduler*` modules
+    - Remove `couch_replicator_doc_processor_*` modules
+    - `couch_replicator` : job creation and a general API entry-point for
+      couch_replicator.
+    - `couch_replicator_job` : runs each replication job
+    - `couch_replicator_job_server` : replication job monitoring gen_server
+    - `couch_replicator_parse` : parses replication document and HTTP
+      `_replicate` POST bodies
+
+## HTTP API additions
+
+N/A
+
+## HTTP API deprecations
+
+N/A
+
+# Security Considerations
+
+None

Review comment:
       Good idea. Updated




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r470898750



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,359 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recomputes their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpoint-ing the job's data value is updated with
+ stats such that the job stays active and doesn't get re-enqueued by the
+ `couch_jobs` activity monitor.
+
+ 7) If the job crashes, it will reschedule itself in `gen_server:terminate/2`
+ via `couch_jobs:resubmit/3` call to run again at some future time, defined
+ roughly as `now + max(min_backoff_penalty * 2^consecutive_errors,
+ max_backoff_penalty)`. If a job starts and successfully runs for some
+ predefined period of time without crashing, it is considered to be `"healed"`
+ and its `consecutive_errors` count is reset to 0.
+
+ 8) If the node where replication job runs crashes, or the job is manually
+ killed via `exit(Pid, kill)`, `couch_jobs` activity monitor will automatically
+ re-enqueue the job as `pending`.
+
+## Replicator Job States
+
+### Description
+
+The set of replication job states is defined as:
+
+ * `pending` : A job is marked as `pending` in these cases:
+    - As soon as a job is created from an `api_frontend` node
+    - When it stopped to let other replication jobs run
+    - When a filtered replication's `replication_id` changes
+
+ * `running` : Set when a job is accepted by the `couch_jobs:accept/2`
+   call. This generally means the job is actually running on a node,
+   however, in cases when a node crashes, the job may show as
+   `running` on that node until `couch_jobs` activity monitor
+   re-enqueues the job, and it starts running on another node.
+
+ * `crashing` : The job was running, but then crashed with an intermittent
+   error. Job's data has an error count which is incremented, and then a
+   backoff penalty is computed and the job is rescheduled to try again at some
+   point in the future.
+
+ * `completed` : Normal replications which have completed
+
+ * `failed` : This can happen when:
+    - A replication job could not be parsed from a replication document. For
+      example, if the user has not specified a `"source"` field.
+    - There already is another persistent replication job running or pending
+      with the same `replication_id`.
+
+### State Differences From CouchDB <= 3.x
+
+The set of states is slightly different than the ones from before. There are
+now less states as some of them have been combined together:
+
+ * `initializing` was combined with `pending`
+
+ * `error` was combined with `crashing`
+
+### Mapping Between couch_jobs States and Replication States
+
+`couch_jobs` application has its own set of state definitions and they map to
+replicator states like so:
+
+ | Replicator States| `couch_jobs` States
+ | ---              | :--
+ | pending          | pending
+ | running          | running
+ | crashing         | pending
+ | completed        | finished
+ | failed           | finished
+
+### State Transition Diagram
+
+Jobs start in the `pending` state, after either a `_replicator` db doc
+update, or a POST to the `/_replicate` endpoint. Continuous jobs, will
+normally toggle between `pending` and `running` states. Normal jobs
+may toggle between `pending` and running a few times and then end up
+in `completed`.
+
+```
+_replicator doc       +-------+
+POST /_replicate ---->+pending|
+                      +-------+
+                          ^
+                          |
+                          |
+                          v
+                      +---+---+      +--------+
+            +---------+running+<---->|crashing|
+            |         +---+---+      +--------+
+            |             ^
+            |             |
+            v             v
+        +------+     +---------+
+        |failed|     |completed|
+        +------+     +---------+
+```
+
+
+## Replication ID Collisions
+
+Multiple replication jobs may specify replications which map to the same
+`replication_id`. To handle these collisions there is an FDB subspace `(...,
+LayerPrefix, ?REPLICATION_IDS, replication_id) -> job_id` to keep track of
+them. After the `replication_id` is computed, each replication job checks if
+there is already another job pending or running with the same `replication_id`.
+If the other job is transient, then the current job will reschedule itself as
+`crashing`. If the other job is persistent, the current job will fail
+permanently as `failed`.
+
+## Transient Job Behavior
+
+In CouchDB <= 3.x transient replication jobs ran in memory on a particular node
+in the cluster. If the node where the replication job ran crashes, the job
+would simply disappear without a trace. It was up to the user to periodically
+monitor the job status and re-create the job. In the current design,
+`transient` jobs are persisted to FDB as `couch_jobs` records, and so would
+survive dbcore node restarts. Technically, with the new design it becomes
+possible to keep the transient job results, including completed job status to
+allow users to retrieve their state later. However in the initial
+implementation, in order to keep maximum compatibility with the <= 3.x
+replicator, transient jobs will be removed when they fail and when they
+complete.

Review comment:
       Playing with running existing elixir replication tests, I notice that if we remove the jobs right away, it becomes hard to maintain the API where wait for the result of a normal replication POST-ed to _replicate.  Currently that's done using couch_job's subscription mechanism, however if the job immediately removes itself, by the time the subscription fires on the `frontend_api` node where the client is listening, the data is already gone so it's a race condition.
   
   I think I'll add the check to the monitoring gen_server to periodically remove these transient jobs older than some configurable time period.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r472315497



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies

Review comment:
       Yeah `couch_epi` (I'll update the name). 
   
   The idea is to re-use the db notification callback from `couch_epi` since we're already using it for indexing: https://github.com/apache/couchdb/blob/prototype/fdb-layer/src/couch_views/src/couch_views_fabric2_plugin.erl#L23




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r470708202



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,359 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recomputes their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpoint-ing the job's data value is updated with
+ stats such that the job stays active and doesn't get re-enqueued by the
+ `couch_jobs` activity monitor.
+
+ 7) If the job crashes, it will reschedule itself in `gen_server:terminate/2`
+ via `couch_jobs:resubmit/3` call to run again at some future time, defined
+ roughly as `now + max(min_backoff_penalty * 2^consecutive_errors,
+ max_backoff_penalty)`. If a job starts and successfully runs for some
+ predefined period of time without crashing, it is considered to be `"healed"`
+ and its `consecutive_errors` count is reset to 0.
+
+ 8) If the node where replication job runs crashes, or the job is manually
+ killed via `exit(Pid, kill)`, `couch_jobs` activity monitor will automatically
+ re-enqueue the job as `pending`.
+
+## Replicator Job States
+
+### Description
+
+The set of replication job states is defined as:
+
+ * `pending` : A job is marked as `pending` in these cases:
+    - As soon as a job is created from an `api_frontend` node
+    - When it stopped to let other replication jobs run
+    - When a filtered replication's `replication_id` changes
+
+ * `running` : Set when a job is accepted by the `couch_jobs:accept/2`
+   call. This generally means the job is actually running on a node,
+   however, in cases when a node crashes, the job may show as
+   `running` on that node until `couch_jobs` activity monitor
+   re-enqueues the job, and it starts running on another node.
+
+ * `crashing` : The job was running, but then crashed with an intermittent
+   error. Job's data has an error count which is incremented, and then a
+   backoff penalty is computed and the job is rescheduled to try again at some
+   point in the future.
+
+ * `completed` : Normal replications which have completed
+
+ * `failed` : This can happen when:
+    - A replication job could not be parsed from a replication document. For
+      example, if the user has not specified a `"source"` field.
+    - There already is another persistent replication job running or pending
+      with the same `replication_id`.
+
+### State Differences From CouchDB <= 3.x
+
+The set of states is slightly different than the ones from before. There are
+now less states as some of them have been combined together:
+
+ * `initializing` was combined with `pending`
+
+ * `error` was combined with `crashing`
+
+### Mapping Between couch_jobs States and Replication States
+
+`couch_jobs` application has its own set of state definitions and they map to
+replicator states like so:
+
+ | Replicator States| `couch_jobs` States
+ | ---              | :--
+ | pending          | pending
+ | running          | running
+ | crashing         | pending
+ | completed        | finished
+ | failed           | finished
+
+### State Transition Diagram
+
+Jobs start in the `pending` state, after either a `_replicator` db doc
+update, or a POST to the `/_replicate` endpoint. Continuous jobs, will
+normally toggle between `pending` and `running` states. Normal jobs
+may toggle between `pending` and running a few times and then end up
+in `completed`.
+
+```
+_replicator doc       +-------+
+POST /_replicate ---->+pending|
+                      +-------+
+                          ^
+                          |
+                          |
+                          v
+                      +---+---+      +--------+
+            +---------+running+<---->|crashing|
+            |         +---+---+      +--------+
+            |             ^
+            |             |
+            v             v
+        +------+     +---------+
+        |failed|     |completed|
+        +------+     +---------+
+```
+
+
+## Replication ID Collisions
+
+Multiple replication jobs may specify replications which map to the same
+`replication_id`. To handle these collisions there is an FDB subspace `(...,
+LayerPrefix, ?REPLICATION_IDS, replication_id) -> job_id` to keep track of
+them. After the `replication_id` is computed, each replication job checks if
+there is already another job pending or running with the same `replication_id`.
+If the other job is transient, then the current job will reschedule itself as
+`crashing`. If the other job is persistent, the current job will fail
+permanently as `failed`.
+
+## Transient Job Behavior
+
+In CouchDB <= 3.x transient replication jobs ran in memory on a particular node
+in the cluster. If the node where the replication job ran crashes, the job
+would simply disappear without a trace. It was up to the user to periodically
+monitor the job status and re-create the job. In the current design,
+`transient` jobs are persisted to FDB as `couch_jobs` records, and so would
+survive dbcore node restarts. Technically, with the new design it becomes
+possible to keep the transient job results, including completed job status to
+allow users to retrieve their state later. However in the initial
+implementation, in order to keep maximum compatibility with the <= 3.x
+replicator, transient jobs will be removed when they fail and when they
+complete.

Review comment:
       It was mainly a concern about users expecting these jobs to disappear if say they are firing them up from a script.  It's also somewhat tricky that completed jobs do not appear in the list of jobs in _scheduler/jobs by default, and they do not appear in _scheduler/docs since those display only replication originating from _replicator databases.
   
   But I could see letting these jobs linger for a configurable amount time after their completion (24 hours?) and allow querying them by their ID as _scheduler/jobs/$ID at the expense of having an extra component which scans the jobs and does that maintenance?
   
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r472302036



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.

Review comment:
       `On` is more appropriate as it is `couch_jobs` terminology and API behavior described [here](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md#job-creation-api)
   
   The idea is that `couch_jobs` API has a job creation section and a job execution section. A job may be `created` on node and `executed` on another. `api_frontend` nodes validate parameters and then `create` a `couch_jobs` job. `replication` nodes then accept and start running these jobs




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r472330819



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recompute their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpointing the job's data value is updated with
+ stats such that the job stays active and doesn't get re-enqueued by the
+ `couch_jobs` activity monitor.
+
+ 7) If the job crashes, it will reschedule itself in `gen_server:terminate/2`

Review comment:
       `terminate` is guaranteed to be called when gen_server handlers return  a `stop` message and if `trap_exit = true ` when it is killed with any exit signal except `kill`. We set `process process_flag(trap_exit, true)` already in that job since we handle exit messages from replication workers and change feed readers. 
   
   In case someone sends the `kill` signal by hand,  supervisor times out waiting during shutdown, or the node where job runs crashes, the couch_jobs activity monitor will automatically re-enqueue the job so it is guaranteed to be rescheduled again.
   
   https://blog.differentpla.net/blog/2014/11/13/erlang-terminate/#exit-reasons




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] ksnavely commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
ksnavely commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r470289014



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,359 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.

Review comment:
       Ah yes, makes total sense.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r472284237



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.

Review comment:
       Good point. I think I'll remove the extra description sentence.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r471250250



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,359 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recomputes their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpoint-ing the job's data value is updated with
+ stats such that the job stays active and doesn't get re-enqueued by the
+ `couch_jobs` activity monitor.
+
+ 7) If the job crashes, it will reschedule itself in `gen_server:terminate/2`
+ via `couch_jobs:resubmit/3` call to run again at some future time, defined
+ roughly as `now + max(min_backoff_penalty * 2^consecutive_errors,
+ max_backoff_penalty)`. If a job starts and successfully runs for some
+ predefined period of time without crashing, it is considered to be `"healed"`
+ and its `consecutive_errors` count is reset to 0.
+
+ 8) If the node where replication job runs crashes, or the job is manually
+ killed via `exit(Pid, kill)`, `couch_jobs` activity monitor will automatically
+ re-enqueue the job as `pending`.
+
+## Replicator Job States
+
+### Description
+
+The set of replication job states is defined as:
+
+ * `pending` : A job is marked as `pending` in these cases:
+    - As soon as a job is created from an `api_frontend` node
+    - When it stopped to let other replication jobs run
+    - When a filtered replication's `replication_id` changes
+
+ * `running` : Set when a job is accepted by the `couch_jobs:accept/2`
+   call. This generally means the job is actually running on a node,
+   however, in cases when a node crashes, the job may show as
+   `running` on that node until `couch_jobs` activity monitor
+   re-enqueues the job, and it starts running on another node.
+
+ * `crashing` : The job was running, but then crashed with an intermittent
+   error. Job's data has an error count which is incremented, and then a
+   backoff penalty is computed and the job is rescheduled to try again at some
+   point in the future.
+
+ * `completed` : Normal replications which have completed
+
+ * `failed` : This can happen when:
+    - A replication job could not be parsed from a replication document. For
+      example, if the user has not specified a `"source"` field.
+    - There already is another persistent replication job running or pending
+      with the same `replication_id`.
+
+### State Differences From CouchDB <= 3.x
+
+The set of states is slightly different than the ones from before. There are
+now less states as some of them have been combined together:
+
+ * `initializing` was combined with `pending`
+
+ * `error` was combined with `crashing`
+
+### Mapping Between couch_jobs States and Replication States
+
+`couch_jobs` application has its own set of state definitions and they map to
+replicator states like so:
+
+ | Replicator States| `couch_jobs` States
+ | ---              | :--
+ | pending          | pending
+ | running          | running
+ | crashing         | pending
+ | completed        | finished
+ | failed           | finished
+
+### State Transition Diagram
+
+Jobs start in the `pending` state, after either a `_replicator` db doc
+update, or a POST to the `/_replicate` endpoint. Continuous jobs, will
+normally toggle between `pending` and `running` states. Normal jobs
+may toggle between `pending` and running a few times and then end up
+in `completed`.
+
+```
+_replicator doc       +-------+
+POST /_replicate ---->+pending|
+                      +-------+
+                          ^
+                          |
+                          |
+                          v
+                      +---+---+      +--------+
+            +---------+running+<---->|crashing|
+            |         +---+---+      +--------+
+            |             ^
+            |             |
+            v             v
+        +------+     +---------+
+        |failed|     |completed|
+        +------+     +---------+
+```
+
+
+## Replication ID Collisions
+
+Multiple replication jobs may specify replications which map to the same
+`replication_id`. To handle these collisions there is an FDB subspace `(...,
+LayerPrefix, ?REPLICATION_IDS, replication_id) -> job_id` to keep track of
+them. After the `replication_id` is computed, each replication job checks if
+there is already another job pending or running with the same `replication_id`.
+If the other job is transient, then the current job will reschedule itself as
+`crashing`. If the other job is persistent, the current job will fail
+permanently as `failed`.
+
+## Transient Job Behavior
+
+In CouchDB <= 3.x transient replication jobs ran in memory on a particular node
+in the cluster. If the node where the replication job ran crashes, the job
+would simply disappear without a trace. It was up to the user to periodically
+monitor the job status and re-create the job. In the current design,
+`transient` jobs are persisted to FDB as `couch_jobs` records, and so would
+survive dbcore node restarts. Technically, with the new design it becomes
+possible to keep the transient job results, including completed job status to
+allow users to retrieve their state later. However in the initial
+implementation, in order to keep maximum compatibility with the <= 3.x
+replicator, transient jobs will be removed when they fail and when they
+complete.

Review comment:
       Added the feature to keep failed and completed transient jobs for a configurable amount of time to the prototype branch and updated the RFC as well.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r472330819



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recompute their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpointing the job's data value is updated with
+ stats such that the job stays active and doesn't get re-enqueued by the
+ `couch_jobs` activity monitor.
+
+ 7) If the job crashes, it will reschedule itself in `gen_server:terminate/2`

Review comment:
       `terminate` is guaranteed to be called when gen_server handlers return  a `stop` message and if `trap_exit = true ` when it is killed with any exit signal except (`kill`). We set `process process_flag(trap_exit, true)` already in that job since we handle exit messages from replication workers and change feed readers. 
   
   In case someone sends the `kill` signal by hand, there is nothing else that would do it otherwise, or the node where job runs crashes, the couch_jobs activity monitor will automatically re-enqueue the job so it is guaranteed to be rescheduled again.
   
   https://blog.differentpla.net/blog/2014/11/13/erlang-terminate/#exit-reasons




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r472260180



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.

Review comment:
       It is silly, I'll remove it




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r470684356



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,359 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recomputes their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpoint-ing the job's data value is updated with
+ stats such that the job stays active and doesn't get re-enqueued by the
+ `couch_jobs` activity monitor.
+
+ 7) If the job crashes, it will reschedule itself in `gen_server:terminate/2`
+ via `couch_jobs:resubmit/3` call to run again at some future time, defined
+ roughly as `now + max(min_backoff_penalty * 2^consecutive_errors,
+ max_backoff_penalty)`. If a job starts and successfully runs for some
+ predefined period of time without crashing, it is considered to be `"healed"`
+ and its `consecutive_errors` count is reset to 0.
+
+ 8) If the node where replication job runs crashes, or the job is manually
+ killed via `exit(Pid, kill)`, `couch_jobs` activity monitor will automatically
+ re-enqueue the job as `pending`.
+
+## Replicator Job States
+
+### Description
+
+The set of replication job states is defined as:
+
+ * `pending` : A job is marked as `pending` in these cases:
+    - As soon as a job is created from an `api_frontend` node
+    - When it stopped to let other replication jobs run
+    - When a filtered replication's `replication_id` changes
+
+ * `running` : Set when a job is accepted by the `couch_jobs:accept/2`
+   call. This generally means the job is actually running on a node,
+   however, in cases when a node crashes, the job may show as
+   `running` on that node until `couch_jobs` activity monitor
+   re-enqueues the job, and it starts running on another node.
+
+ * `crashing` : The job was running, but then crashed with an intermittent
+   error. Job's data has an error count which is incremented, and then a
+   backoff penalty is computed and the job is rescheduled to try again at some
+   point in the future.
+
+ * `completed` : Normal replications which have completed
+
+ * `failed` : This can happen when:
+    - A replication job could not be parsed from a replication document. For
+      example, if the user has not specified a `"source"` field.
+    - There already is another persistent replication job running or pending
+      with the same `replication_id`.
+
+### State Differences From CouchDB <= 3.x
+
+The set of states is slightly different than the ones from before. There are
+now less states as some of them have been combined together:
+
+ * `initializing` was combined with `pending`
+
+ * `error` was combined with `crashing`
+
+### Mapping Between couch_jobs States and Replication States
+
+`couch_jobs` application has its own set of state definitions and they map to
+replicator states like so:
+
+ | Replicator States| `couch_jobs` States
+ | ---              | :--
+ | pending          | pending
+ | running          | running
+ | crashing         | pending
+ | completed        | finished
+ | failed           | finished
+
+### State Transition Diagram
+
+Jobs start in the `pending` state, after either a `_replicator` db doc
+update, or a POST to the `/_replicate` endpoint. Continuous jobs, will
+normally toggle between `pending` and `running` states. Normal jobs
+may toggle between `pending` and running a few times and then end up
+in `completed`.
+
+```
+_replicator doc       +-------+
+POST /_replicate ---->+pending|
+                      +-------+
+                          ^
+                          |
+                          |
+                          v
+                      +---+---+      +--------+
+            +---------+running+<---->|crashing|
+            |         +---+---+      +--------+
+            |             ^
+            |             |
+            v             v
+        +------+     +---------+
+        |failed|     |completed|
+        +------+     +---------+
+```
+
+
+## Replication ID Collisions
+
+Multiple replication jobs may specify replications which map to the same
+`replication_id`. To handle these collisions there is an FDB subspace `(...,
+LayerPrefix, ?REPLICATION_IDS, replication_id) -> job_id` to keep track of
+them. After the `replication_id` is computed, each replication job checks if
+there is already another job pending or running with the same `replication_id`.
+If the other job is transient, then the current job will reschedule itself as
+`crashing`. If the other job is persistent, the current job will fail
+permanently as `failed`.
+
+## Transient Job Behavior
+
+In CouchDB <= 3.x transient replication jobs ran in memory on a particular node
+in the cluster. If the node where the replication job ran crashes, the job
+would simply disappear without a trace. It was up to the user to periodically
+monitor the job status and re-create the job. In the current design,
+`transient` jobs are persisted to FDB as `couch_jobs` records, and so would
+survive dbcore node restarts. Technically, with the new design it becomes
+possible to keep the transient job results, including completed job status to
+allow users to retrieve their state later. However in the initial
+implementation, in order to keep maximum compatibility with the <= 3.x
+replicator, transient jobs will be removed when they fail and when they
+complete.
+
+## Monitoring Endpoints
+
+`_active_tasks`, `_scheduler/jobs` and `_scheduler/docs` endpoint are handled
+by traversing the replication job's data using a new `couch_jobs:fold_jobs/4`
+API call and inspect each job's data. `_active_tasks` implementation already
+works that way and `_scheduler/*` endpoint will work similarly.
+
+## Replication Documents Not Updated For Transient Errors
+
+Configuration
+[option](https://docs.couchdb.org/en/latest/replication/replicator.html?highlight=update_docs#compatibility-mode)
+`[replicator] update_docs = false` was introduced with the scheduling
+replicator in a 2.x release. It controls whether to update replication document
+with transient states like `triggered` and `error`. It defaulted to `false` and
+was mainly for compatibility with older monitoring code. That behavior now
+becomes hard-coded such that replication documents are only updated with
+terminal states of `failed` and `completed`. Users should use `_scheduler/docs`
+API to check for completion status instead.
+
+
+# Advantages and Disadvantages
+
+Advantages:
+
+ * Simplicity: re-using `couch_jobs` means having a lot less code to maintain
+   in `couch_replicator`. In the draft implementation there are about 3000
+   lines of code saved compared to the replicator application in CouchDB 3.x
+
+ * Simpler endpoint and monitoring implementation
+
+ * Less replication job states to keep track of
+
+ * Transient replications can now survive node crashes and restarts
+
+ * Using node types allows tightening firewall rules such that only
+   `replication` nodes are the ones which may make arbitrary requests outside
+   the cluster, and `frontend_api` nodes are the only ones that may accept
+   incoming connections.
+
+Disadvantages:
+
+ * Behavior changes for transient jobs
+
+ * Centralized job queue might mean handling some number of conflicts generated
+   in the FDB backend when jobs are accepted. These are mitigated using a
+   `accept_jitter` configuration parameter and a configurable number of max
+   acceptors per node.
+
+ * In monitoring API responses, `running` job state might not immediately
+   reflect the actually running process state on the replication node. If the
+   node crashes, it might take up to a minute or two until the job is
+   re-enqueued by the `couch_jobs` activity monitor.
+
+# Key Changes
+
+ * Behavior changes for transient jobs
+
+ * A delay in `running` state as reflected in monitoring API responses
+
+ * `[replicator] update_docs = false` configuration option becomes hard-coded
+ 
+## Applications and Modules affected
+
+ * couch_jobs : New APIs to fold jobs and get pending count job estimate
+
+ * fabric2_db : Adding EPI db create/delete callbacks
+
+ * couch_replicator :
+    - Remove `couch_replicator_scheduler*` modules
+    - Remove `couch_replicator_doc_processor_*` modules
+    - `couch_replicator` : job creation and a general API entry-point for
+      couch_replicator.
+    - `couch_replicator_job` : runs each replication job
+    - `couch_replicator_job_server` : replication job monitoring gen_server
+    - `couch_replicator_parse` : parses replication document and HTTP
+      `_replicate` POST bodies
+
+## HTTP API additions
+
+N/A
+
+## HTTP API deprecations
+
+N/A
+
+# Security Considerations
+
+None

Review comment:
       Good idea!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r470133664



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,359 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.

Review comment:
       I thought `/_replicator` since it has to look like `someprefix/_replicator` and say `someprefix_replicator` shouldn't match 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r472302036



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.

Review comment:
       `On` is more appropriate as it is `couch_jobs` terminology and API behavior described [here](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md#job-creation-api)
   
   The idea is that `couch_jobs` API has a job creation section and a job execution section. A job may be `created` on one node and `executed` on another. `api_frontend` nodes validate parameters and then `create` a `couch_jobs` job. `replication` nodes then accept and start running these jobs




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] jaydoane commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
jaydoane commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r470455963



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,359 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recomputes their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpoint-ing the job's data value is updated with

Review comment:
       maybe s/checkpoint-ing/checkpointing ?

##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,359 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recomputes their `replication_id`. In the case of filtered replications the

Review comment:
       s/recomputes/recompute

##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,359 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recomputes their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpoint-ing the job's data value is updated with
+ stats such that the job stays active and doesn't get re-enqueued by the
+ `couch_jobs` activity monitor.
+
+ 7) If the job crashes, it will reschedule itself in `gen_server:terminate/2`
+ via `couch_jobs:resubmit/3` call to run again at some future time, defined
+ roughly as `now + max(min_backoff_penalty * 2^consecutive_errors,
+ max_backoff_penalty)`. If a job starts and successfully runs for some
+ predefined period of time without crashing, it is considered to be `"healed"`
+ and its `consecutive_errors` count is reset to 0.
+
+ 8) If the node where replication job runs crashes, or the job is manually
+ killed via `exit(Pid, kill)`, `couch_jobs` activity monitor will automatically
+ re-enqueue the job as `pending`.
+
+## Replicator Job States
+
+### Description
+
+The set of replication job states is defined as:
+
+ * `pending` : A job is marked as `pending` in these cases:
+    - As soon as a job is created from an `api_frontend` node
+    - When it stopped to let other replication jobs run
+    - When a filtered replication's `replication_id` changes
+
+ * `running` : Set when a job is accepted by the `couch_jobs:accept/2`
+   call. This generally means the job is actually running on a node,
+   however, in cases when a node crashes, the job may show as
+   `running` on that node until `couch_jobs` activity monitor
+   re-enqueues the job, and it starts running on another node.
+
+ * `crashing` : The job was running, but then crashed with an intermittent
+   error. Job's data has an error count which is incremented, and then a
+   backoff penalty is computed and the job is rescheduled to try again at some
+   point in the future.
+
+ * `completed` : Normal replications which have completed
+
+ * `failed` : This can happen when:
+    - A replication job could not be parsed from a replication document. For
+      example, if the user has not specified a `"source"` field.
+    - There already is another persistent replication job running or pending
+      with the same `replication_id`.
+
+### State Differences From CouchDB <= 3.x
+
+The set of states is slightly different than the ones from before. There are
+now less states as some of them have been combined together:
+
+ * `initializing` was combined with `pending`
+
+ * `error` was combined with `crashing`
+
+### Mapping Between couch_jobs States and Replication States
+
+`couch_jobs` application has its own set of state definitions and they map to
+replicator states like so:
+
+ | Replicator States| `couch_jobs` States
+ | ---              | :--
+ | pending          | pending
+ | running          | running
+ | crashing         | pending
+ | completed        | finished
+ | failed           | finished
+
+### State Transition Diagram
+
+Jobs start in the `pending` state, after either a `_replicator` db doc
+update, or a POST to the `/_replicate` endpoint. Continuous jobs, will
+normally toggle between `pending` and `running` states. Normal jobs
+may toggle between `pending` and running a few times and then end up
+in `completed`.
+
+```
+_replicator doc       +-------+
+POST /_replicate ---->+pending|
+                      +-------+
+                          ^
+                          |
+                          |
+                          v
+                      +---+---+      +--------+
+            +---------+running+<---->|crashing|
+            |         +---+---+      +--------+
+            |             ^
+            |             |
+            v             v
+        +------+     +---------+
+        |failed|     |completed|
+        +------+     +---------+
+```
+
+
+## Replication ID Collisions
+
+Multiple replication jobs may specify replications which map to the same
+`replication_id`. To handle these collisions there is an FDB subspace `(...,
+LayerPrefix, ?REPLICATION_IDS, replication_id) -> job_id` to keep track of
+them. After the `replication_id` is computed, each replication job checks if
+there is already another job pending or running with the same `replication_id`.
+If the other job is transient, then the current job will reschedule itself as
+`crashing`. If the other job is persistent, the current job will fail
+permanently as `failed`.
+
+## Transient Job Behavior
+
+In CouchDB <= 3.x transient replication jobs ran in memory on a particular node
+in the cluster. If the node where the replication job ran crashes, the job
+would simply disappear without a trace. It was up to the user to periodically
+monitor the job status and re-create the job. In the current design,
+`transient` jobs are persisted to FDB as `couch_jobs` records, and so would
+survive dbcore node restarts. Technically, with the new design it becomes
+possible to keep the transient job results, including completed job status to
+allow users to retrieve their state later. However in the initial
+implementation, in order to keep maximum compatibility with the <= 3.x
+replicator, transient jobs will be removed when they fail and when they
+complete.

Review comment:
       I have a hard time imagining how retaining compatibility for such an arguably anti-feature is worth it.

##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,359 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recomputes their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpoint-ing the job's data value is updated with
+ stats such that the job stays active and doesn't get re-enqueued by the
+ `couch_jobs` activity monitor.
+
+ 7) If the job crashes, it will reschedule itself in `gen_server:terminate/2`
+ via `couch_jobs:resubmit/3` call to run again at some future time, defined
+ roughly as `now + max(min_backoff_penalty * 2^consecutive_errors,
+ max_backoff_penalty)`. If a job starts and successfully runs for some
+ predefined period of time without crashing, it is considered to be `"healed"`
+ and its `consecutive_errors` count is reset to 0.
+
+ 8) If the node where replication job runs crashes, or the job is manually
+ killed via `exit(Pid, kill)`, `couch_jobs` activity monitor will automatically
+ re-enqueue the job as `pending`.
+
+## Replicator Job States
+
+### Description
+
+The set of replication job states is defined as:
+
+ * `pending` : A job is marked as `pending` in these cases:
+    - As soon as a job is created from an `api_frontend` node
+    - When it stopped to let other replication jobs run
+    - When a filtered replication's `replication_id` changes
+
+ * `running` : Set when a job is accepted by the `couch_jobs:accept/2`
+   call. This generally means the job is actually running on a node,
+   however, in cases when a node crashes, the job may show as
+   `running` on that node until `couch_jobs` activity monitor
+   re-enqueues the job, and it starts running on another node.
+
+ * `crashing` : The job was running, but then crashed with an intermittent
+   error. Job's data has an error count which is incremented, and then a
+   backoff penalty is computed and the job is rescheduled to try again at some
+   point in the future.
+
+ * `completed` : Normal replications which have completed
+
+ * `failed` : This can happen when:
+    - A replication job could not be parsed from a replication document. For
+      example, if the user has not specified a `"source"` field.
+    - There already is another persistent replication job running or pending
+      with the same `replication_id`.
+
+### State Differences From CouchDB <= 3.x
+
+The set of states is slightly different than the ones from before. There are
+now less states as some of them have been combined together:
+
+ * `initializing` was combined with `pending`
+
+ * `error` was combined with `crashing`
+
+### Mapping Between couch_jobs States and Replication States
+
+`couch_jobs` application has its own set of state definitions and they map to
+replicator states like so:
+
+ | Replicator States| `couch_jobs` States
+ | ---              | :--
+ | pending          | pending
+ | running          | running
+ | crashing         | pending
+ | completed        | finished
+ | failed           | finished
+
+### State Transition Diagram
+
+Jobs start in the `pending` state, after either a `_replicator` db doc
+update, or a POST to the `/_replicate` endpoint. Continuous jobs, will
+normally toggle between `pending` and `running` states. Normal jobs
+may toggle between `pending` and running a few times and then end up
+in `completed`.
+
+```
+_replicator doc       +-------+
+POST /_replicate ---->+pending|
+                      +-------+
+                          ^
+                          |
+                          |
+                          v
+                      +---+---+      +--------+
+            +---------+running+<---->|crashing|
+            |         +---+---+      +--------+
+            |             ^
+            |             |
+            v             v
+        +------+     +---------+
+        |failed|     |completed|
+        +------+     +---------+
+```
+
+
+## Replication ID Collisions
+
+Multiple replication jobs may specify replications which map to the same
+`replication_id`. To handle these collisions there is an FDB subspace `(...,
+LayerPrefix, ?REPLICATION_IDS, replication_id) -> job_id` to keep track of
+them. After the `replication_id` is computed, each replication job checks if
+there is already another job pending or running with the same `replication_id`.
+If the other job is transient, then the current job will reschedule itself as
+`crashing`. If the other job is persistent, the current job will fail
+permanently as `failed`.
+
+## Transient Job Behavior
+
+In CouchDB <= 3.x transient replication jobs ran in memory on a particular node
+in the cluster. If the node where the replication job ran crashes, the job
+would simply disappear without a trace. It was up to the user to periodically
+monitor the job status and re-create the job. In the current design,
+`transient` jobs are persisted to FDB as `couch_jobs` records, and so would
+survive dbcore node restarts. Technically, with the new design it becomes
+possible to keep the transient job results, including completed job status to
+allow users to retrieve their state later. However in the initial
+implementation, in order to keep maximum compatibility with the <= 3.x
+replicator, transient jobs will be removed when they fail and when they
+complete.
+
+## Monitoring Endpoints
+
+`_active_tasks`, `_scheduler/jobs` and `_scheduler/docs` endpoint are handled
+by traversing the replication job's data using a new `couch_jobs:fold_jobs/4`
+API call and inspect each job's data. `_active_tasks` implementation already
+works that way and `_scheduler/*` endpoint will work similarly.
+
+## Replication Documents Not Updated For Transient Errors
+
+Configuration
+[option](https://docs.couchdb.org/en/latest/replication/replicator.html?highlight=update_docs#compatibility-mode)
+`[replicator] update_docs = false` was introduced with the scheduling
+replicator in a 2.x release. It controls whether to update replication document
+with transient states like `triggered` and `error`. It defaulted to `false` and
+was mainly for compatibility with older monitoring code. That behavior now
+becomes hard-coded such that replication documents are only updated with
+terminal states of `failed` and `completed`. Users should use `_scheduler/docs`
+API to check for completion status instead.
+
+
+# Advantages and Disadvantages
+
+Advantages:
+
+ * Simplicity: re-using `couch_jobs` means having a lot less code to maintain
+   in `couch_replicator`. In the draft implementation there are about 3000
+   lines of code saved compared to the replicator application in CouchDB 3.x
+
+ * Simpler endpoint and monitoring implementation
+
+ * Less replication job states to keep track of

Review comment:
       Fewer

##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,359 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recomputes their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpoint-ing the job's data value is updated with
+ stats such that the job stays active and doesn't get re-enqueued by the
+ `couch_jobs` activity monitor.
+
+ 7) If the job crashes, it will reschedule itself in `gen_server:terminate/2`
+ via `couch_jobs:resubmit/3` call to run again at some future time, defined
+ roughly as `now + max(min_backoff_penalty * 2^consecutive_errors,
+ max_backoff_penalty)`. If a job starts and successfully runs for some
+ predefined period of time without crashing, it is considered to be `"healed"`
+ and its `consecutive_errors` count is reset to 0.
+
+ 8) If the node where replication job runs crashes, or the job is manually
+ killed via `exit(Pid, kill)`, `couch_jobs` activity monitor will automatically
+ re-enqueue the job as `pending`.
+
+## Replicator Job States
+
+### Description
+
+The set of replication job states is defined as:
+
+ * `pending` : A job is marked as `pending` in these cases:
+    - As soon as a job is created from an `api_frontend` node
+    - When it stopped to let other replication jobs run
+    - When a filtered replication's `replication_id` changes
+
+ * `running` : Set when a job is accepted by the `couch_jobs:accept/2`
+   call. This generally means the job is actually running on a node,
+   however, in cases when a node crashes, the job may show as
+   `running` on that node until `couch_jobs` activity monitor
+   re-enqueues the job, and it starts running on another node.
+
+ * `crashing` : The job was running, but then crashed with an intermittent
+   error. Job's data has an error count which is incremented, and then a
+   backoff penalty is computed and the job is rescheduled to try again at some
+   point in the future.
+
+ * `completed` : Normal replications which have completed
+
+ * `failed` : This can happen when:
+    - A replication job could not be parsed from a replication document. For
+      example, if the user has not specified a `"source"` field.
+    - There already is another persistent replication job running or pending
+      with the same `replication_id`.
+
+### State Differences From CouchDB <= 3.x
+
+The set of states is slightly different than the ones from before. There are
+now less states as some of them have been combined together:
+
+ * `initializing` was combined with `pending`
+
+ * `error` was combined with `crashing`
+
+### Mapping Between couch_jobs States and Replication States
+
+`couch_jobs` application has its own set of state definitions and they map to
+replicator states like so:
+
+ | Replicator States| `couch_jobs` States
+ | ---              | :--
+ | pending          | pending
+ | running          | running
+ | crashing         | pending
+ | completed        | finished
+ | failed           | finished
+
+### State Transition Diagram
+
+Jobs start in the `pending` state, after either a `_replicator` db doc
+update, or a POST to the `/_replicate` endpoint. Continuous jobs, will
+normally toggle between `pending` and `running` states. Normal jobs
+may toggle between `pending` and running a few times and then end up
+in `completed`.
+
+```
+_replicator doc       +-------+
+POST /_replicate ---->+pending|
+                      +-------+
+                          ^
+                          |
+                          |
+                          v
+                      +---+---+      +--------+
+            +---------+running+<---->|crashing|
+            |         +---+---+      +--------+
+            |             ^
+            |             |
+            v             v
+        +------+     +---------+
+        |failed|     |completed|
+        +------+     +---------+
+```
+
+
+## Replication ID Collisions
+
+Multiple replication jobs may specify replications which map to the same
+`replication_id`. To handle these collisions there is an FDB subspace `(...,
+LayerPrefix, ?REPLICATION_IDS, replication_id) -> job_id` to keep track of
+them. After the `replication_id` is computed, each replication job checks if
+there is already another job pending or running with the same `replication_id`.
+If the other job is transient, then the current job will reschedule itself as
+`crashing`. If the other job is persistent, the current job will fail
+permanently as `failed`.
+
+## Transient Job Behavior
+
+In CouchDB <= 3.x transient replication jobs ran in memory on a particular node
+in the cluster. If the node where the replication job ran crashes, the job
+would simply disappear without a trace. It was up to the user to periodically
+monitor the job status and re-create the job. In the current design,
+`transient` jobs are persisted to FDB as `couch_jobs` records, and so would
+survive dbcore node restarts. Technically, with the new design it becomes
+possible to keep the transient job results, including completed job status to
+allow users to retrieve their state later. However in the initial
+implementation, in order to keep maximum compatibility with the <= 3.x
+replicator, transient jobs will be removed when they fail and when they
+complete.
+
+## Monitoring Endpoints
+
+`_active_tasks`, `_scheduler/jobs` and `_scheduler/docs` endpoint are handled
+by traversing the replication job's data using a new `couch_jobs:fold_jobs/4`
+API call and inspect each job's data. `_active_tasks` implementation already
+works that way and `_scheduler/*` endpoint will work similarly.
+
+## Replication Documents Not Updated For Transient Errors
+
+Configuration
+[option](https://docs.couchdb.org/en/latest/replication/replicator.html?highlight=update_docs#compatibility-mode)
+`[replicator] update_docs = false` was introduced with the scheduling
+replicator in a 2.x release. It controls whether to update replication document
+with transient states like `triggered` and `error`. It defaulted to `false` and
+was mainly for compatibility with older monitoring code. That behavior now
+becomes hard-coded such that replication documents are only updated with
+terminal states of `failed` and `completed`. Users should use `_scheduler/docs`
+API to check for completion status instead.
+
+
+# Advantages and Disadvantages
+
+Advantages:
+
+ * Simplicity: re-using `couch_jobs` means having a lot less code to maintain
+   in `couch_replicator`. In the draft implementation there are about 3000
+   lines of code saved compared to the replicator application in CouchDB 3.x
+
+ * Simpler endpoint and monitoring implementation
+
+ * Less replication job states to keep track of
+
+ * Transient replications can now survive node crashes and restarts
+
+ * Using node types allows tightening firewall rules such that only
+   `replication` nodes are the ones which may make arbitrary requests outside
+   the cluster, and `frontend_api` nodes are the only ones that may accept
+   incoming connections.
+
+Disadvantages:
+
+ * Behavior changes for transient jobs
+
+ * Centralized job queue might mean handling some number of conflicts generated
+   in the FDB backend when jobs are accepted. These are mitigated using a
+   `accept_jitter` configuration parameter and a configurable number of max
+   acceptors per node.
+
+ * In monitoring API responses, `running` job state might not immediately
+   reflect the actually running process state on the replication node. If the
+   node crashes, it might take up to a minute or two until the job is
+   re-enqueued by the `couch_jobs` activity monitor.
+
+# Key Changes
+
+ * Behavior changes for transient jobs
+
+ * A delay in `running` state as reflected in monitoring API responses
+
+ * `[replicator] update_docs = false` configuration option becomes hard-coded
+ 
+## Applications and Modules affected
+
+ * couch_jobs : New APIs to fold jobs and get pending count job estimate
+
+ * fabric2_db : Adding EPI db create/delete callbacks
+
+ * couch_replicator :
+    - Remove `couch_replicator_scheduler*` modules
+    - Remove `couch_replicator_doc_processor_*` modules
+    - `couch_replicator` : job creation and a general API entry-point for
+      couch_replicator.
+    - `couch_replicator_job` : runs each replication job
+    - `couch_replicator_job_server` : replication job monitoring gen_server
+    - `couch_replicator_parse` : parses replication document and HTTP
+      `_replicate` POST bodies
+
+## HTTP API additions
+
+N/A
+
+## HTTP API deprecations
+
+N/A
+
+# Security Considerations
+
+None

Review comment:
       I'd say there's potentially increased security by having replication-only nodes which can't receive any external connections, but can only make them.

##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,359 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recomputes their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpoint-ing the job's data value is updated with
+ stats such that the job stays active and doesn't get re-enqueued by the
+ `couch_jobs` activity monitor.
+
+ 7) If the job crashes, it will reschedule itself in `gen_server:terminate/2`
+ via `couch_jobs:resubmit/3` call to run again at some future time, defined
+ roughly as `now + max(min_backoff_penalty * 2^consecutive_errors,
+ max_backoff_penalty)`. If a job starts and successfully runs for some
+ predefined period of time without crashing, it is considered to be `"healed"`
+ and its `consecutive_errors` count is reset to 0.
+
+ 8) If the node where replication job runs crashes, or the job is manually
+ killed via `exit(Pid, kill)`, `couch_jobs` activity monitor will automatically
+ re-enqueue the job as `pending`.
+
+## Replicator Job States
+
+### Description
+
+The set of replication job states is defined as:
+
+ * `pending` : A job is marked as `pending` in these cases:
+    - As soon as a job is created from an `api_frontend` node
+    - When it stopped to let other replication jobs run
+    - When a filtered replication's `replication_id` changes
+
+ * `running` : Set when a job is accepted by the `couch_jobs:accept/2`
+   call. This generally means the job is actually running on a node,
+   however, in cases when a node crashes, the job may show as
+   `running` on that node until `couch_jobs` activity monitor
+   re-enqueues the job, and it starts running on another node.
+
+ * `crashing` : The job was running, but then crashed with an intermittent
+   error. Job's data has an error count which is incremented, and then a
+   backoff penalty is computed and the job is rescheduled to try again at some
+   point in the future.
+
+ * `completed` : Normal replications which have completed
+
+ * `failed` : This can happen when:
+    - A replication job could not be parsed from a replication document. For
+      example, if the user has not specified a `"source"` field.

Review comment:
       It seems strange that we don't just reject obviously invalid replicator documents (e.g. missing or invalid source or target) when they are inserted rather than allowing them to be inserted and then failing. What are the benefits of the latter approach?

##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,359 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recomputes their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpoint-ing the job's data value is updated with
+ stats such that the job stays active and doesn't get re-enqueued by the
+ `couch_jobs` activity monitor.
+
+ 7) If the job crashes, it will reschedule itself in `gen_server:terminate/2`
+ via `couch_jobs:resubmit/3` call to run again at some future time, defined
+ roughly as `now + max(min_backoff_penalty * 2^consecutive_errors,
+ max_backoff_penalty)`. If a job starts and successfully runs for some
+ predefined period of time without crashing, it is considered to be `"healed"`
+ and its `consecutive_errors` count is reset to 0.
+
+ 8) If the node where replication job runs crashes, or the job is manually
+ killed via `exit(Pid, kill)`, `couch_jobs` activity monitor will automatically
+ re-enqueue the job as `pending`.
+
+## Replicator Job States
+
+### Description
+
+The set of replication job states is defined as:
+
+ * `pending` : A job is marked as `pending` in these cases:
+    - As soon as a job is created from an `api_frontend` node
+    - When it stopped to let other replication jobs run
+    - When a filtered replication's `replication_id` changes
+
+ * `running` : Set when a job is accepted by the `couch_jobs:accept/2`
+   call. This generally means the job is actually running on a node,
+   however, in cases when a node crashes, the job may show as
+   `running` on that node until `couch_jobs` activity monitor
+   re-enqueues the job, and it starts running on another node.
+
+ * `crashing` : The job was running, but then crashed with an intermittent
+   error. Job's data has an error count which is incremented, and then a
+   backoff penalty is computed and the job is rescheduled to try again at some
+   point in the future.
+
+ * `completed` : Normal replications which have completed
+
+ * `failed` : This can happen when:
+    - A replication job could not be parsed from a replication document. For
+      example, if the user has not specified a `"source"` field.
+    - There already is another persistent replication job running or pending
+      with the same `replication_id`.
+
+### State Differences From CouchDB <= 3.x
+
+The set of states is slightly different than the ones from before. There are
+now less states as some of them have been combined together:

Review comment:
       s/less/fewer/




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r469682126



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,337 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: ''
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs`
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md)
+for centralized scheduling and monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB 2 and 3 replication jobs were maped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of replication job at a
+time on each node. The new design proposes using `couch_jobs`, as described in
+this
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogenous node types as defined in this
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and
+replication jobs will be run only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Replication jobs created with the `"continuous":
+true` parameter. When this job reaches the end of the changes feed it will
+continue waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true`as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs jobs can only be run on these nodes.
+
+`filtered` replications: Replications which use a user-defined filter on the
+source endpoint to filter its changes feed.
+
+`replication_id` : A unique ID defined for replication jobs which is a hash of
+ the source and target endpoints, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ ID will change, if for example, a filter on the source endpoint changes.
+ Computing this value may require a network round-trip to the source endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many
+replication jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies up to how many new jobs
+to spawn during some a configurable scheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the base)
+penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuraiton parameter specifyign the maximum pently
+applied to job which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replicator application's job creation and scheduling works roughly as follows.
+
+ 1) `Persistent` jobs are created by document updates in `_replicator` dbs and
+ `transient` jobs are created by POST-ing to the `_replicate/` endpoint. In
+ both cases a new `couch_jobs` job is created in a separate `couch_jobs`
+ replication subspace. This happens on `api_frontend` nodes only. These are
+ also the nodes which process all the regular HTTP CRUD requests.
+
+   a) Persisent jobs are driven by an EPI callback mechanism which notifies
+   `couch_replicator` application when documents in `_replicator` dbs are
+   updated, when `_replicator` dbs are created and deleted.
+
+   b) Transient jobs are created from the `_replicate` HTTP handler directly.
+ Newly created jobs are in `pending` state.
+
+ 2) Each `replication` node spawns a small (configurable) amount of acceptor
+ processes which wait in `couch_jobs:accept/2` for jobs scheduled to run at a
+ time less or equal to the current time.
+
+ 3) After the job is accepted its state is updated as `running` and then a
+ gen_server server process monitoring these replication jobs will spawn another
+ acceptor to replace it. That happens until the `max_jobs` running jobs have
+ been spawned. The same gen_server will periodically check if there are any
+ pending jobs in the queue, and spawn up to some `max_churn` number of new
+ acceptors. These new acceptors may start new jobs, and if they do, for each
+ one of them, the oldest running job will be stopped and re-enqueued as
+ pending. This in large follows the logic from the replication scheduler in
+ CouchDB <= 3.x except that is uses `couch_jobs` as the central queueing and
+ scheduling mechanism.
+
+ 4) After the job is marked as `running` it computes the `replication_id`,
+ initializes an internal replication state record from the JSON job data
+ record, and start replicating.
+
+ 5) As the job runs, when it checkpoints it periodically recomputes its
+ `replication_id`. In case of filtered replications the `replication_id` maybe
+ change. In that case the job is stopped and moved to the `pending` state so it
+ can be restarted with the new `replication_id`. Also during checkpointing
+ attempts, the `couch_jobs`'s data is updated such that the job stays active
+ and doesn't get re-enqueued by the `couch_jobs` activity monitor.
+
+ 6) If the job crashes, in `gen_server:terminate/2` it reschedules itself via
+ `couch_jobs:resubmit/3` to run at some future time defined roughly as `now +
+ max(min_backoff_penalty * 2^consecutive_errors, max_backoff_penalty)`. In
+ order to avoid job which had intermetent error in teh past but since "healed"
+ if the job runs successfully after some period of time, configurable as
+ `health_threshold`, the `consecutive_errors` count is reset back to 0.
+
+ 7) If the node where replication job runs crashes, or the job is manually
+ killed via `exit(Pid, kill)`, `couch_jobs` activity monitor will automatically
+ re-enqueue the job to run as `pending`.
+
+## Replicator Job States
+
+### Description
+
+The new replicator in CouchDB 4.x will have less replication job states then before. The
+new set of states will be:
+
+ * `pending` : A job is marked as `pending` in these cases:
+    - As soon as a job is created from an `api_frontend` node
+    - Job is stopped to let other replication jobs run
+    - A filtered replication noticed its `replication_id` has changed
+
+ * `running` : When the job accepted by the `couch_jobs:accept/2` call. This
+   generally means the job is actually running on a node, however in case when
+   a node crashes immediately, the job may indicate `running` on that node
+   until `couch_jobs` activity monitor will re-enqueue the job
+
+ * `crashing` : Then job was running, but then crashed with an intermetent
+   error. Job data has an error count which is bumped and then a bakcoff pently
+   is computed and the the job is rescheduled to try again at some point in the
+   future.
+
+ * `completed` : Normal replications which have completed
+
+ * `failed` : This can happen when
+    - A replication job could not be parsed from a replication document or
+      `_replicate/` POST body. For example is the user has not specified a
+      `"source"` field.
+    - There is another persistent replication job running or pending with the
+      same `replication_id`.
+
+
+### Differences from CouchDB <= 3.x
+
+The differences from before are:
+ * `initializing` state was folded into `pending`
+ * `error` state folded into `crashing`
+
+
+### Mapping Between couch_jobs States and Replication States
+
+The state map from these states to `couch_jobs`'s states of (`pending`,
+`running`, `finished`) looks as follows:
+
+replicator state | couch_jobs state
+------------------------------------
+pending          | pending
+running          | running
+crashing         | pending
+completed        | finished
+failed           | finished
+
+### State Transition Diagram
+
+```
++-----+       +-------+
+|start+------>+pending|
++-----+       +-------+
+                  ^
+                  |
+                  |
+                  v
+              +---+---+      +--------+
+    +---------+running+----->|crashing|
+    |         +---+---+      +--------+
+    |             ^
+    |             |
+    v             v
++------+     +---------+
+|failed|     |completed|
++------+     +---------+
+```
+
+
+## Replication ID and Job IDs mapping
+
+Multiple replication jobs may specify a replications which map to the same
+`replication_id`. To handle duplicates and collission there is a `(...,
+LayerPrefix, ?REPLICATION_IDS, ReplicationID) -> JobID` subspace which keeps
+track of mappings from `replication_id` to `job id`. After the `replication_id`
+is computed each replication job checks if there is already another job pending
+or running with the same `replication_id`. If the other job is transient, then
+the current job will reschedule itself as `crashing`. If the other job is
+persistent, the current job will fail permanently as `failed`.
+
+## Transient Jobs Behavior
+
+In CouchDB <= 3.x transient replication jobs ran in memory on the nodes where
+they were assigned to run. If the node where the replication job ran crashed
+the job would simply disappear without a trace. It was up to the user's code to
+periodically monitor the job status and re-create the job. With the current
+design, `transient` jobs are saved as `couch_jobs` records in FDB and so would
+survive if dbcore nodes crashing and restarting. Technically with the new
+design it becomes possible to keep the transient job results even the failed
+and completed jobs, perhaps up to some configurable amount of time to allow
+users to retrieve their status after they stop running. However in the initial
+implementation, in order to keep maximum compatibility with the <= 3.x
+replicator, transient jobs will be removed when they fail and when they
+complete.
+
+
+## Monitoring Endpoints
+
+`_active_tasks`, `_scheduler/jobs` and `_scheduler/docs` endpoint are handled
+by traversing the replication job's data using a new `couch_jobs:fold_jobs/4`
+function and returning subsets of job's data. `_active_tasks` implementation
+already works that way and `_scheduler/*` endpoint will work similarly.
+
+
+# Advantages and Disadvantages
+
+Advantages:
+
+ * Simplicity: re-using `couch_jobs` means having a lot less code to maintain
+   in `couch_replicator`
+
+ * Simpler endpoint and monitoring implementations
+
+ * Less replication job states to keep track of
+
+ * Improved behavior for transient replications, they know survive node

Review comment:
       Thanks!
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] ksnavely commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
ksnavely commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r470289084



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,359 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.

Review comment:
       Great clarification, thank you.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r470898750



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,359 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recomputes their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpoint-ing the job's data value is updated with
+ stats such that the job stays active and doesn't get re-enqueued by the
+ `couch_jobs` activity monitor.
+
+ 7) If the job crashes, it will reschedule itself in `gen_server:terminate/2`
+ via `couch_jobs:resubmit/3` call to run again at some future time, defined
+ roughly as `now + max(min_backoff_penalty * 2^consecutive_errors,
+ max_backoff_penalty)`. If a job starts and successfully runs for some
+ predefined period of time without crashing, it is considered to be `"healed"`
+ and its `consecutive_errors` count is reset to 0.
+
+ 8) If the node where replication job runs crashes, or the job is manually
+ killed via `exit(Pid, kill)`, `couch_jobs` activity monitor will automatically
+ re-enqueue the job as `pending`.
+
+## Replicator Job States
+
+### Description
+
+The set of replication job states is defined as:
+
+ * `pending` : A job is marked as `pending` in these cases:
+    - As soon as a job is created from an `api_frontend` node
+    - When it stopped to let other replication jobs run
+    - When a filtered replication's `replication_id` changes
+
+ * `running` : Set when a job is accepted by the `couch_jobs:accept/2`
+   call. This generally means the job is actually running on a node,
+   however, in cases when a node crashes, the job may show as
+   `running` on that node until `couch_jobs` activity monitor
+   re-enqueues the job, and it starts running on another node.
+
+ * `crashing` : The job was running, but then crashed with an intermittent
+   error. Job's data has an error count which is incremented, and then a
+   backoff penalty is computed and the job is rescheduled to try again at some
+   point in the future.
+
+ * `completed` : Normal replications which have completed
+
+ * `failed` : This can happen when:
+    - A replication job could not be parsed from a replication document. For
+      example, if the user has not specified a `"source"` field.
+    - There already is another persistent replication job running or pending
+      with the same `replication_id`.
+
+### State Differences From CouchDB <= 3.x
+
+The set of states is slightly different than the ones from before. There are
+now less states as some of them have been combined together:
+
+ * `initializing` was combined with `pending`
+
+ * `error` was combined with `crashing`
+
+### Mapping Between couch_jobs States and Replication States
+
+`couch_jobs` application has its own set of state definitions and they map to
+replicator states like so:
+
+ | Replicator States| `couch_jobs` States
+ | ---              | :--
+ | pending          | pending
+ | running          | running
+ | crashing         | pending
+ | completed        | finished
+ | failed           | finished
+
+### State Transition Diagram
+
+Jobs start in the `pending` state, after either a `_replicator` db doc
+update, or a POST to the `/_replicate` endpoint. Continuous jobs, will
+normally toggle between `pending` and `running` states. Normal jobs
+may toggle between `pending` and running a few times and then end up
+in `completed`.
+
+```
+_replicator doc       +-------+
+POST /_replicate ---->+pending|
+                      +-------+
+                          ^
+                          |
+                          |
+                          v
+                      +---+---+      +--------+
+            +---------+running+<---->|crashing|
+            |         +---+---+      +--------+
+            |             ^
+            |             |
+            v             v
+        +------+     +---------+
+        |failed|     |completed|
+        +------+     +---------+
+```
+
+
+## Replication ID Collisions
+
+Multiple replication jobs may specify replications which map to the same
+`replication_id`. To handle these collisions there is an FDB subspace `(...,
+LayerPrefix, ?REPLICATION_IDS, replication_id) -> job_id` to keep track of
+them. After the `replication_id` is computed, each replication job checks if
+there is already another job pending or running with the same `replication_id`.
+If the other job is transient, then the current job will reschedule itself as
+`crashing`. If the other job is persistent, the current job will fail
+permanently as `failed`.
+
+## Transient Job Behavior
+
+In CouchDB <= 3.x transient replication jobs ran in memory on a particular node
+in the cluster. If the node where the replication job ran crashes, the job
+would simply disappear without a trace. It was up to the user to periodically
+monitor the job status and re-create the job. In the current design,
+`transient` jobs are persisted to FDB as `couch_jobs` records, and so would
+survive dbcore node restarts. Technically, with the new design it becomes
+possible to keep the transient job results, including completed job status to
+allow users to retrieve their state later. However in the initial
+implementation, in order to keep maximum compatibility with the <= 3.x
+replicator, transient jobs will be removed when they fail and when they
+complete.

Review comment:
       Playing with running existing elixir replication tests, I noticed that if we remove the jobs right away, it becomes hard to maintain the API where wait for the result of a normal replication POST-ed to _replicate.  Currently that's done using couch_job's subscription mechanism, however if the job immediately removes itself, by the time the subscription fires on the `frontend_api` node where the client is listening, the data is already gone so it's a race condition.
   
   I think I'll add the check to the monitoring gen_server to periodically remove these transient jobs older than some configurable time period.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] rnewson commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
rnewson commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r471793702



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or

Review comment:
       `created from` to `defined in`?

##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.

Review comment:
       does this sentence add anything?

##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies

Review comment:
       meaning couch_epi? I'm not sure why that would be involved in anything that doesn't require pluggability.

##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.

Review comment:
       this is the first mention of wallclock time being associated with the persisted state of a replication job. Where was it defined? And what's it for?

##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also

Review comment:
       `would` to `will`

##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.

Review comment:
       the current replicator suspends and resumes jobs as it sees fit, we should not contradict that here. the continuous flag declares the users _intention_ for the replication to happen continuously, and the replicator scheduler takes it from there.

##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a

Review comment:
       missing comma after `3.x`.

##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.

Review comment:
       specify that the latter is to fetch the filter function?

##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the

Review comment:
       this seems a new definition to me and `normal` is a very subjective term. I've seen the term `one-shot replication` used for this case and suggest it is a better fit with an established history.

##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.

Review comment:
       `on` to `through`? the jobs aren't created on those nodes at all, the api_frontend endpoint is simply processing the http requests. In the case of the _replicate endpoint, that handler is presumably directing a replication node to start the replication. In the case of the _replicator endpoint, the handler is just writing a document, which the replication scheduler happens to react to after the fact.

##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,

Review comment:
       `, and if they do,` to `and, if they do,`

##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`

Review comment:
       `, and if there are,` to `and, if there are,`

##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a

Review comment:
       `as` to `to`.

##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recompute their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpointing the job's data value is updated with
+ stats such that the job stays active and doesn't get re-enqueued by the
+ `couch_jobs` activity monitor.
+
+ 7) If the job crashes, it will reschedule itself in `gen_server:terminate/2`
+ via `couch_jobs:resubmit/3` call to run again at some future time, defined
+ roughly as `now + max(min_backoff_penalty * 2^consecutive_errors,
+ max_backoff_penalty)`. If a job starts and successfully runs for some
+ predefined period of time without crashing, it is considered to be `"healed"`
+ and its `consecutive_errors` count is reset to 0.
+
+ 8) If the node where replication job runs crashes, or the job is manually
+ killed via `exit(Pid, kill)`, `couch_jobs` activity monitor will automatically
+ re-enqueue the job as `pending`.
+
+## Replicator Job States
+
+### Description
+
+The set of replication job states is defined as:
+
+ * `pending` : A job is marked as `pending` in these cases:
+    - As soon as a job is created from an `api_frontend` node
+    - When it stopped to let other replication jobs run
+    - When a filtered replication's `replication_id` changes
+
+ * `running` : Set when a job is accepted by the `couch_jobs:accept/2`
+   call. This generally means the job is actually running on a node,
+   however, in cases when a node crashes, the job may show as
+   `running` on that node until `couch_jobs` activity monitor
+   re-enqueues the job, and it starts running on another node.
+
+ * `crashing` : The job was running, but then crashed with an intermittent
+   error. Job's data has an error count which is incremented, and then a
+   backoff penalty is computed and the job is rescheduled to try again at some
+   point in the future.
+
+ * `completed` : Normal replications which have completed
+
+ * `failed` : This can happen when:
+    - A replication job could not be parsed from a replication document. For
+      example, if the user has not specified a `"source"` field.
+    - A transient replication job crashes. Transient jobs don't get rescheduled
+      to run again after they crash.
+    - There already is another persistent replication job running or pending
+      with the same `replication_id`.
+
+### State Differences From CouchDB <= 3.x
+
+The set of states is slightly different than the ones from before. There are
+now fewer states as some of them have been combined together:
+
+ * `initializing` was combined with `pending`
+
+ * `error` was combined with `crashing`
+
+### Mapping Between couch_jobs States and Replication States
+
+`couch_jobs` application has its own set of state definitions and they map to
+replicator states like so:
+
+ | Replicator States| `couch_jobs` States
+ | ---              | :--
+ | pending          | pending
+ | running          | running
+ | crashing         | pending
+ | completed        | finished
+ | failed           | finished
+
+### State Transition Diagram
+
+Jobs start in the `pending` state, after either a `_replicator` db doc
+update, or a POST to the `/_replicate` endpoint. Continuous jobs, will
+normally toggle between `pending` and `running` states. Normal jobs
+may toggle between `pending` and running a few times and then end up
+in `completed`.
+
+```
+_replicator doc       +-------+
+POST /_replicate ---->+pending|
+                      +-------+
+                          ^
+                          |
+                          |
+                          v
+                      +---+---+      +--------+
+            +---------+running+<---->|crashing|
+            |         +---+---+      +--------+
+            |             ^
+            |             |
+            v             v
+        +------+     +---------+
+        |failed|     |completed|
+        +------+     +---------+
+```
+
+
+## Replication ID Collisions
+
+Multiple replication jobs may specify replications which map to the same
+`replication_id`. To handle these collisions there is an FDB subspace `(...,
+LayerPrefix, ?REPLICATION_IDS, replication_id) -> job_id` to keep track of
+them. After the `replication_id` is computed, each replication job checks if
+there is already another job pending or running with the same `replication_id`.
+If the other job is transient, then the current job will reschedule itself as
+`crashing`. If the other job is persistent, the current job will fail
+permanently as `failed`.
+
+## Replication Parameter Validation
+
+`_replicator` documents in CouchDB <= 3.x were parsed and validated in a
+two-step process:
+
+  1) In a validate-doc-update (VDU) javascript function from a programmatically
+  inserted _design document. This validation happened when the document was
+  updated, and performed some rough checks on field names and value types. If
+  this validation failed, the document update operation was rejected.
+
+  2) Inside replicator's Erlang code when it was translated to an internal
+ record used by the replication application. This validation was more thorough
+ but didn't have very friendly error messages. If validation failed here, the
+ job would be marked as `failed`.
+
+For CouchDB 4.x the proposal is to use only the Erlang parser. It would be
+called from the `before_doc_update` callback. This is a callback which runs
+before every document update. If validation fails there it would reject the
+document update operation. This should reduce code duplication and also provide
+better feedback to the users directly when they update the `_replicator`
+documents.
+
+## Transient Job Behavior
+
+In CouchDB <= 3.x transient replication jobs ran in memory on a particular node
+in the cluster. If the node where the replication job ran crashes, the job
+would simply disappear without a trace. It was up to the user to periodically
+monitor the job status and re-create the job. In the current design,
+`transient` jobs are persisted to FDB as `couch_jobs` records, and so would
+survive dbcore node restarts. Also after transient jobs complete or failed,

Review comment:
       what's a "dbcore node"?

##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.

Review comment:
       the definition of what's included in the hash is "anything that _could_ affect the result of the replication" btw.

##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recompute their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpointing the job's data value is updated with
+ stats such that the job stays active and doesn't get re-enqueued by the
+ `couch_jobs` activity monitor.
+
+ 7) If the job crashes, it will reschedule itself in `gen_server:terminate/2`

Review comment:
       gen_server:terminate is only for cleanup and only called in rare circumstances, this doesn't seem a good way to manage rescheduling.

##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recompute their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpointing the job's data value is updated with
+ stats such that the job stays active and doesn't get re-enqueued by the
+ `couch_jobs` activity monitor.
+
+ 7) If the job crashes, it will reschedule itself in `gen_server:terminate/2`
+ via `couch_jobs:resubmit/3` call to run again at some future time, defined
+ roughly as `now + max(min_backoff_penalty * 2^consecutive_errors,
+ max_backoff_penalty)`. If a job starts and successfully runs for some
+ predefined period of time without crashing, it is considered to be `"healed"`
+ and its `consecutive_errors` count is reset to 0.
+
+ 8) If the node where replication job runs crashes, or the job is manually
+ killed via `exit(Pid, kill)`, `couch_jobs` activity monitor will automatically
+ re-enqueue the job as `pending`.
+
+## Replicator Job States
+
+### Description
+
+The set of replication job states is defined as:
+
+ * `pending` : A job is marked as `pending` in these cases:
+    - As soon as a job is created from an `api_frontend` node
+    - When it stopped to let other replication jobs run
+    - When a filtered replication's `replication_id` changes
+
+ * `running` : Set when a job is accepted by the `couch_jobs:accept/2`
+   call. This generally means the job is actually running on a node,
+   however, in cases when a node crashes, the job may show as
+   `running` on that node until `couch_jobs` activity monitor
+   re-enqueues the job, and it starts running on another node.
+
+ * `crashing` : The job was running, but then crashed with an intermittent
+   error. Job's data has an error count which is incremented, and then a
+   backoff penalty is computed and the job is rescheduled to try again at some
+   point in the future.
+
+ * `completed` : Normal replications which have completed
+
+ * `failed` : This can happen when:
+    - A replication job could not be parsed from a replication document. For
+      example, if the user has not specified a `"source"` field.
+    - A transient replication job crashes. Transient jobs don't get rescheduled
+      to run again after they crash.
+    - There already is another persistent replication job running or pending
+      with the same `replication_id`.
+
+### State Differences From CouchDB <= 3.x
+
+The set of states is slightly different than the ones from before. There are
+now fewer states as some of them have been combined together:
+
+ * `initializing` was combined with `pending`
+
+ * `error` was combined with `crashing`
+
+### Mapping Between couch_jobs States and Replication States
+
+`couch_jobs` application has its own set of state definitions and they map to
+replicator states like so:
+
+ | Replicator States| `couch_jobs` States
+ | ---              | :--
+ | pending          | pending
+ | running          | running
+ | crashing         | pending
+ | completed        | finished
+ | failed           | finished
+
+### State Transition Diagram
+
+Jobs start in the `pending` state, after either a `_replicator` db doc
+update, or a POST to the `/_replicate` endpoint. Continuous jobs, will
+normally toggle between `pending` and `running` states. Normal jobs
+may toggle between `pending` and running a few times and then end up
+in `completed`.
+
+```
+_replicator doc       +-------+
+POST /_replicate ---->+pending|
+                      +-------+
+                          ^
+                          |
+                          |
+                          v
+                      +---+---+      +--------+
+            +---------+running+<---->|crashing|
+            |         +---+---+      +--------+
+            |             ^
+            |             |
+            v             v
+        +------+     +---------+
+        |failed|     |completed|
+        +------+     +---------+
+```
+
+
+## Replication ID Collisions
+
+Multiple replication jobs may specify replications which map to the same
+`replication_id`. To handle these collisions there is an FDB subspace `(...,
+LayerPrefix, ?REPLICATION_IDS, replication_id) -> job_id` to keep track of
+them. After the `replication_id` is computed, each replication job checks if
+there is already another job pending or running with the same `replication_id`.
+If the other job is transient, then the current job will reschedule itself as
+`crashing`. If the other job is persistent, the current job will fail
+permanently as `failed`.
+
+## Replication Parameter Validation
+
+`_replicator` documents in CouchDB <= 3.x were parsed and validated in a
+two-step process:
+
+  1) In a validate-doc-update (VDU) javascript function from a programmatically
+  inserted _design document. This validation happened when the document was
+  updated, and performed some rough checks on field names and value types. If
+  this validation failed, the document update operation was rejected.
+
+  2) Inside replicator's Erlang code when it was translated to an internal
+ record used by the replication application. This validation was more thorough
+ but didn't have very friendly error messages. If validation failed here, the
+ job would be marked as `failed`.
+
+For CouchDB 4.x the proposal is to use only the Erlang parser. It would be
+called from the `before_doc_update` callback. This is a callback which runs
+before every document update. If validation fails there it would reject the
+document update operation. This should reduce code duplication and also provide
+better feedback to the users directly when they update the `_replicator`
+documents.

Review comment:
       I agree on this. the auto injection of vdu's was cute but never really solid.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r470148839



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,359 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.

Review comment:
       True, in practice, users might want to deploy nodes that only have `replication = true`, and all the other type labels as `false`. But they could also have `replication = true`, `indexing = true` nodes which do background processing, and so in this RFC's terminology we'd still define it as `replication node`. In other words, a node being a replication node is not exclusive with it being a `fontend_api` node.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r472330819



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recompute their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpointing the job's data value is updated with
+ stats such that the job stays active and doesn't get re-enqueued by the
+ `couch_jobs` activity monitor.
+
+ 7) If the job crashes, it will reschedule itself in `gen_server:terminate/2`

Review comment:
       `terminate` is guaranteed to be called when gen_server handlers return  a `stop` message and if `trap_exit = true ` when it is killed with any exit signal except `kill`. We set `process process_flag(trap_exit, true)` already in that job since we handle exit messages from replication workers and change feed readers. 
   
   In case someone sends the `kill` signal by hand (as there is nothing else that would do it otherwise), or the node where job runs crashes, the couch_jobs activity monitor will automatically re-enqueue the job so it is guaranteed to be rescheduled again.
   
   https://blog.differentpla.net/blog/2014/11/13/erlang-terminate/#exit-reasons




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] ksnavely commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
ksnavely commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r473333436



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,384 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x, replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application will manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs defined in document in a
+`_replicator` database.
+
+`continuous` replications : Replication jobs created with the `"continuous":
+true` parameter. These jobs will try to run continuously until the user removes
+them. They may be temporarily paused to allow other jobs to make progress.
+
+`one-shot` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `one-shot`. These jobs will try to run until they reach the end of the
+changes feed, then stop.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs, which is a hash of
+replication parameters that affect the result of the replication. These may
+include source and target endpoint URLs, as well as a filter function specified
+in a design document on the source endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by the `couch_epi` callback mechanism which
+ notifies `couch_replicator` application when documents in `_replicator` DBs
+ are updated, or when `_replicator` DBs are created and deleted. Transient jobs
+ are created from the `_replicate` HTTP handler directly. Newly created jobs
+ are in a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated to `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue and, if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs and, if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recompute their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpointing the job's data value is updated with
+ stats such that the job stays active and doesn't get re-enqueued by the
+ `couch_jobs` activity monitor.
+
+ 7) If the job crashes, it will reschedule itself in `gen_server:terminate/2`
+ via `couch_jobs:resubmit/3` call to run again at some future time, defined
+ roughly as `now + max(min_backoff_penalty * 2^consecutive_errors,
+ max_backoff_penalty)`. If a job starts and successfully runs for some
+ predefined period of time without crashing, it is considered to be `"healed"`
+ and its `consecutive_errors` count is reset to 0.
+
+ 8) If the node where replication job runs crashes, or the job is manually
+ killed via `exit(Pid, kill)`, `couch_jobs` activity monitor will automatically
+ re-enqueue the job as `pending`.
+
+## Replicator Job States
+
+### Description
+
+The set of replication job states is defined as:
+
+ * `pending` : A job is marked as `pending` in these cases:
+    - As soon as a job is created from an `api_frontend` node
+    - When it stopped to let other replication jobs run
+    - When a filtered replication's `replication_id` changes
+
+ * `running` : Set when a job is accepted by the `couch_jobs:accept/2`
+   call. This generally means the job is actually running on a node,
+   however, in cases when a node crashes, the job may show as
+   `running` on that node until `couch_jobs` activity monitor
+   re-enqueues the job, and it starts running on another node.
+
+ * `crashing` : The job was running, but then crashed with an intermittent
+   error. Job's data has an error count which is incremented, and then a
+   backoff penalty is computed and the job is rescheduled to try again at some
+   point in the future.
+
+ * `completed` : One-Shot replications which have completed
+
+ * `failed` : This can happen when:
+    - A replication job could not be parsed from a replication document. For
+      example, if the user has not specified a `"source"` field.
+    - A transient replication job crashes. Transient jobs don't get rescheduled
+      to run again after they crash.
+    - There already is another persistent replication job running or pending
+      with the same `replication_id`.
+
+### State Differences From CouchDB <= 3.x
+
+The set of states is slightly different than the ones from before. There are
+now fewer states as some of them have been combined together:
+
+ * `initializing` was combined with `pending`
+
+ * `error` was combined with `crashing`
+
+### Mapping Between couch_jobs States and Replication States
+
+`couch_jobs` application has its own set of state definitions and they map to
+replicator states like so:
+
+ | Replicator States| `couch_jobs` States
+ | ---              | :--
+ | pending          | pending
+ | running          | running
+ | crashing         | pending
+ | completed        | finished
+ | failed           | finished
+
+### State Transition Diagram
+
+Jobs start in the `pending` state, after either a `_replicator` db doc
+update, or a POST to the `/_replicate` endpoint. Continuous jobs, will
+normally toggle between `pending` and `running` states. One-Shot jobs
+may toggle between `pending` and running a few times and then end up
+in `completed`.
+
+```
+_replicator doc       +-------+
+POST /_replicate ---->+pending|
+                      +-------+
+                          ^
+                          |
+                          |
+                          v
+                      +---+---+      +--------+
+            +---------+running+<---->|crashing|
+            |         +---+---+      +--------+
+            |             |
+            |             |
+            v             v
+        +------+     +---------+
+        |failed|     |completed|
+        +------+     +---------+
+```
+
+
+## Replication ID Collisions
+
+Multiple replication jobs may specify replications which map to the same
+`replication_id`. To handle these collisions there is an FDB subspace `(...,
+LayerPrefix, ?REPLICATION_IDS, replication_id) -> job_id` to keep track of
+them. After the `replication_id` is computed, each replication job checks if
+there is already another job pending or running with the same `replication_id`.
+If the other job is transient, then the current job will reschedule itself as
+`crashing`. If the other job is persistent, the current job will fail
+permanently as `failed`.
+
+## Replication Parameter Validation
+
+`_replicator` documents in CouchDB <= 3.x were parsed and validated in a
+two-step process:
+
+  1) In a validate-doc-update (VDU) javascript function from a programmatically
+  inserted _design document. This validation happened when the document was
+  updated, and performed some rough checks on field names and value types. If
+  this validation failed, the document update operation was rejected.
+
+  2) Inside replicator's Erlang code when it was translated to an internal
+ record used by the replication application. This validation was more thorough
+ but didn't have very friendly error messages. If validation failed here, the
+ job would be marked as `failed`.
+
+For CouchDB 4.x the proposal is to use only the Erlang parser. It would be
+called from the `before_doc_update` callback. This is a callback which runs
+before every document update. If validation fails there it would reject the
+document update operation. This should reduce code duplication and also provide
+better feedback to the users directly when they update the `_replicator`
+documents.
+
+## Transient Job Behavior
+
+In CouchDB <= 3.x transient replication jobs ran in memory on a particular node
+in the cluster. If the node where the replication job ran crashes, the job
+would simply disappear without a trace. It was up to the user to periodically
+monitor the job status and re-create the job. In the current design,
+`transient` jobs are persisted to FDB as `couch_jobs` records, and so would
+survive node restarts. Also after transient jobs complete or failed,

Review comment:
       Nice improvement




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] ksnavely commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
ksnavely commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r470130225



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,359 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.

Review comment:
       It might be worth clarifying that no other activities take place other than replication.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r470684118



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,359 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.
+
+ 3) After a job is accepted, its state is updated as `running`, and then, a
+ gen_server process monitoring these replication jobs will spawn another
+ acceptor. That happens until the `max_jobs` limit is reached.
+
+ 4) The same monitoring gen_server will periodically check if there are any
+ pending jobs in the queue, and if there are, spawn up to some `max_churn`
+ number of new acceptors. These acceptors may start new jobs, and if they do,
+ for each one of them, the oldest running job will be stopped and re-enqueued
+ as `pending`. This in large follows the logic from the replication scheduler
+ in CouchDB <= 3.x except that is uses `couch_jobs` as the central queuing and
+ scheduling mechanism.
+
+ 5) After the job is marked as `running`, it computes its `replication_id`,
+ initializes an internal replication state record from job's data object, and
+ starts replicating. Underneath this level the logic is identical to what's
+ already happening in CouchDB <= 3.x and so it is not described further in this
+ document.
+
+ 6) As jobs run, they periodically checkpoint, and when they do that, they also
+ recomputes their `replication_id`. In the case of filtered replications the
+ `replication_id` may change, and if so, that job is stopped and re-enqueued as
+ `pending`. Also, during checkpoint-ing the job's data value is updated with
+ stats such that the job stays active and doesn't get re-enqueued by the
+ `couch_jobs` activity monitor.
+
+ 7) If the job crashes, it will reschedule itself in `gen_server:terminate/2`
+ via `couch_jobs:resubmit/3` call to run again at some future time, defined
+ roughly as `now + max(min_backoff_penalty * 2^consecutive_errors,
+ max_backoff_penalty)`. If a job starts and successfully runs for some
+ predefined period of time without crashing, it is considered to be `"healed"`
+ and its `consecutive_errors` count is reset to 0.
+
+ 8) If the node where replication job runs crashes, or the job is manually
+ killed via `exit(Pid, kill)`, `couch_jobs` activity monitor will automatically
+ re-enqueue the job as `pending`.
+
+## Replicator Job States
+
+### Description
+
+The set of replication job states is defined as:
+
+ * `pending` : A job is marked as `pending` in these cases:
+    - As soon as a job is created from an `api_frontend` node
+    - When it stopped to let other replication jobs run
+    - When a filtered replication's `replication_id` changes
+
+ * `running` : Set when a job is accepted by the `couch_jobs:accept/2`
+   call. This generally means the job is actually running on a node,
+   however, in cases when a node crashes, the job may show as
+   `running` on that node until `couch_jobs` activity monitor
+   re-enqueues the job, and it starts running on another node.
+
+ * `crashing` : The job was running, but then crashed with an intermittent
+   error. Job's data has an error count which is incremented, and then a
+   backoff penalty is computed and the job is rescheduled to try again at some
+   point in the future.
+
+ * `completed` : Normal replications which have completed
+
+ * `failed` : This can happen when:
+    - A replication job could not be parsed from a replication document. For
+      example, if the user has not specified a `"source"` field.

Review comment:
       Good observation. I'll add a section describing how validation happens. And you're right, based on the current prototype, this error should almost never happen, as we can now re use the main parsing function to validate the document before the doc update happens.
   
   But just in case, users create a non-replicator database, then restore it (rename the deleted snapshot) as a replicator db, the db will be rescanned with the db_created` EPI callback. None of those document would pass through the `before_doc_update` validator and if they fail to parse we'd still want to reflect their state in the _scheduler/docs output.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] nickva commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
nickva commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r472320366



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,386 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in the
+[Background Jobs
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in the [Node Types
+RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.
+
+`transient` replications : Replication jobs created by `POST`-ing to the
+`/_replicate` endpoint.
+
+`persistent` replications : Replication jobs created from a document in a
+`_replicator` database.
+
+`continuous` replications : Jobs created with the `"continuous": true`
+parameter. When this job reaches the end of the changes feed it will continue
+waiting for new changes in a loop until the user removes the job.
+
+`normal` replications : Replication jobs which are not `continuous`. If the
+`"continuous":true` parameter is not specified, by default, replication jobs
+will be `normal`.
+
+`api_frontend node` : Database node which has the `api_frontend` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can be only be created on these nodes.
+
+`replication node` : Database node which has the `replication` type set to
+`true` as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md).
+Replication jobs can only be run on these nodes.
+
+`filtered` replications: Replications with a user-defined filter on the source
+endpoint to filter its changes feed.
+
+`replication_id` : An ID defined by replication jobs which is a hash of the
+ source and target endpoint URLs, some of the options, and for filtered
+ replications, the contents of the filter from the source endpoint. Replication
+ IDs will change, for example, if the filter contents on the source endpoint
+ changes. Computing this value may require a network round-trip to the source
+ endpoint.
+
+`job_id` : A replication job ID derived from the database and document IDs for
+persistent replications, and from source, target endpoint, user name and some
+options for transient replications. Computing a `job_id`, unlike a
+`replication_id`, doesn't require making any network requests. A filtered
+replication with a given `job_id` during its lifetime may change its
+`replication_id` multiple times when filter contents changes on the source.
+
+`max_jobs` : Configuration parameter which specifies up to how many replication
+jobs to run on each `replication` node.
+
+`max_churn` : Configuration parameter which specifies a limit of how many new
+jobs to spawn during each rescheduling interval.
+
+`min_backoff_penalty` : Configuration parameter specifying the minimum (the
+base) penalty applied to jobs which crash repeatedly.
+
+`max_backoff_penalty` : Configuration parameter specifying the maximum penalty
+applied to jobs which crash repeatedly.
+
+---
+
+# Detailed Description
+
+Replication job creation and scheduling works roughly as follows:
+
+ 1) `Persistent` and `transient` jobs both start by creating or updating a
+ `couch_jobs` record in a separate replication key-space on `api_frontend`
+ nodes. Persistent jobs are driven by an EPI callback mechanism which notifies
+ `couch_replicator` application when documents in `_replicator` DBs are
+ updated, or when `_replicator` DBs are created and deleted. Transient jobs are
+ created from the `_replicate` HTTP handler directly. Newly created jobs are in
+ a `pending` state.
+
+ 2) Each `replication` node spawns some acceptor processes which wait in
+ `couch_jobs:accept/2` call for jobs. It will accept only jobs which are
+ scheduled to run at a time less or equal to the current time.

Review comment:
       It's described in the `couch_jobs` [RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md#job-creation-api). A job may be scheduled to run at some point in the future.  The `accept/2` call has a parameter to only accept jobs in a certain time window. And in this case it is used to schedule job execution at a later time.
   
   Another example where we use it is to remove soft-deleted db instances: https://github.com/apache/couchdb/blob/prototype/fdb-layer/src/fabric/src/fabric2_db_expiration.erl#L151




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [couchdb-documentation] ksnavely commented on a change in pull request #581: [RFC] Replicator Implementation for CouchDB 4.x

Posted by GitBox <gi...@apache.org>.
ksnavely commented on a change in pull request #581:
URL: https://github.com/apache/couchdb-documentation/pull/581#discussion_r470126606



##########
File path: rfcs/016-fdb-replicator.md
##########
@@ -0,0 +1,359 @@
+---
+name: Formal RFC
+about: Submit a formal Request For Comments for consideration by the team.
+title: 'Replicator Implementation On FDB'
+labels: rfc, discussion
+assignees: 'vatamane@apache.org'
+
+---
+
+# Introduction
+
+This document describes the design of the replicator application for CouchDB
+4.x. The replicator will rely on `couch_jobs` for centralized scheduling and
+monitoring of replication jobs.
+
+## Abstract
+
+CouchDB replicator is the CouchDB application which runs replication jobs.
+Replication jobs can be created from documents in `_replicator` databases, or
+by `POST`-ing requests to the HTTP `/_replicate` endpoint. Previously, in
+CouchDB <= 3.x replication jobs were mapped to individual cluster nodes and a
+scheduler component would run up to `max_jobs` number of jobs at a time on each
+node. The new design proposes using `couch_jobs`, as described in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/007-background-jobs.md),
+to have a central, FDB-based queue of replication jobs. `couch_jobs`
+application would manage job scheduling and coordination. The new design also
+proposes using heterogeneous node types as defined in
+[RFC](https://github.com/apache/couchdb-documentation/blob/master/rfcs/013-node-types.md)
+such that replication jobs will be created only on `api_frontend` nodes and run
+only on `replication` nodes.
+
+## Requirements Language
+
+The key words "MUST", "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" in this document are to be
+interpreted as described in [RFC
+2119](https://www.rfc-editor.org/rfc/rfc2119.txt).
+
+## Terminology
+
+`_replicator` databases : A database that is either named `_replicator` or ends
+with the `/_replicator` suffix.

Review comment:
       Just double checking, is the suffix identifier `_replicator` or `/_replicator`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org