kafka-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ewe...@apache.org
Subject kafka git commit: KAFKA-3583: Add documentation for Connect status control APIs
Date Thu, 12 May 2016 00:19:32 GMT
Repository: kafka
Updated Branches:
  refs/heads/trunk 1182d61de -> 4807dd1df


KAFKA-3583: Add documentation for Connect status control APIs

Author: Jason Gustafson <jason@confluent.io>

Reviewers: Roger Hoover <roger.hoover@gmail.com>, Ismael Juma <ismael@juma.me.uk>,
Ewen Cheslack-Postava <ewen@confluent.io>

Closes #1358 from hachikuji/KAFKA-3583


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/4807dd1d
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/4807dd1d
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/4807dd1d

Branch: refs/heads/trunk
Commit: 4807dd1df945f6e66c6276f0a85e8abeb24b568a
Parents: 1182d61
Author: Jason Gustafson <jason@confluent.io>
Authored: Wed May 11 17:19:11 2016 -0700
Committer: Ewen Cheslack-Postava <me@ewencp.org>
Committed: Wed May 11 17:19:11 2016 -0700

----------------------------------------------------------------------
 docs/connect.html | 49 +++++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 49 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/4807dd1d/docs/connect.html
----------------------------------------------------------------------
diff --git a/docs/connect.html b/docs/connect.html
index c3cf583..a362dde 100644
--- a/docs/connect.html
+++ b/docs/connect.html
@@ -98,6 +98,10 @@ Since Kafka Connect is intended to be run as a service, it also provides
a REST
     <li><code>GET /connectors/{name}/status</code> - get current status
of the connector, including if it is running, failed, paused, etc., which worker it is assigned
to, error information if it has failed, and the state of all its tasks</li>
     <li><code>GET /connectors/{name}/tasks</code> - get a list of tasks
currently running for a connector</li>
     <li><code>GET /connectors/{name}/tasks/{taskid}/status</code> - get
current status of the task, including if it is running, failed, paused, etc., which worker
it is assigned to, and error information if it has failed</li>
+    <li><code>PUT /connectors/{name}/pause</code> - pause the connector
and its tasks, which stops message processing until the connector is resumed</li>
+    <li><code>PUT /connectors/{name}/resume</code> - resume a paused connector
(or do nothing if the connector is not paused)</li>
+    <li><code>POST /connectors/{name}/restart</code> - restart a connector
(typically because it has failed)</li>
+    <li><code>POST /connectors/{name}/tasks/{taskId}/restart</code> - restart
an individual task (typically because it has failed)</li>
     <li><code>DELETE /connectors/{name}</code> - delete a connector, halting
all tasks and deleting its configuration</li>
 </ul>
 
@@ -338,3 +342,48 @@ However, many connectors will have dynamic schemas. One simple example
of this i
 
 Sink connectors are usually simpler because they are consuming data and therefore do not
need to create schemas. However, they should take just as much care to validate that the schemas
they receive have the expected format. When the schema does not match -- usually indicating
the upstream producer is generating invalid data that cannot be correctly translated to the
destination system -- sink connectors should throw an exception to indicate this error to
the system.
 
+<h4><a id="connect_administration" href="#connect_administration">Kafka Connect
Administration</a></h4>
+
+<p>
+Kafka Connect's <a href="#connect_rest">REST layer</a> provides a set of APIs
to enable administration of the cluster. This includes APIs to view the configuration of connectors
and the status of their tasks, as well as to alter their current behavior (e.g. changing configuration
and restarting tasks).
+</p>
+
+<p>
+When a connector is first submitted to the cluster, the workers rebalance the full set of
connectors in the cluster and their tasks so that each worker has approximately the same amount
of work. This same rebalancing procedure is also used when connectors increase or decrease
the number of tasks they require, or when a connector's configuration is changed. You can
use the REST API to view the current status of a connector and its tasks, including the id
of the worker to which each was assigned. For example, querying the status of a file source
(using <code>GET /connectors/file-source/status</code>) might produce output like
the following:
+</p>
+
+<pre>
+{
+  "name": "file-source",
+  "connector": {
+    "state": "RUNNING",
+    "worker_id": "192.168.1.208:8083"
+  },
+  "tasks": [
+    {
+      "id": 0,
+      "state": "RUNNING",
+      "worker_id": "192.168.1.209:8083"
+    }
+  ]
+}
+</pre>
+
+<p>
+Connectors and their tasks publish status updates to a shared topic (configured with <code>status.storage.topic</code>)
which all workers in the cluster monitor. Because the workers consume this topic asynchronously,
there is typically a (short) delay before a state change is visible through the status API.
The following states are possible for a connector or one of its tasks:
+</p>
+
+<ul>
+  <li><b>UNASSIGNED:</b> The connector/task has not yet been assigned to
a worker.</li>
+  <li><b>RUNNING:</b> The connector/task is running.</li>
+  <li><b>PAUSED:</b> The connector/task has been administratively paused.</li>
+  <li><b>FAILED:</b> The connector/task has failed (usually by raising
an exception, which is reported in the status output).</li>
+</ul>
+
+<p>
+In most cases, connector and task states will match, though they may be different for short
periods of time when changes are occurring or if tasks have failed. For example, when a connector
is first started, there may be a noticeable delay before the connector and its tasks have
all transitioned to the RUNNING state. States will also diverge when tasks fail since Connect
does not automatically restart failed tasks. To restart a connector/task manually, you can
use the restart APIs listed above. Note that if you try to restart a task while a rebalance
is taking place, Connect will return a 409 (Conflict) status code. You can retry after the
rebalance completes, but it might not be necessary since rebalances effectively restart all
the connectors and tasks in the cluster.
+</p>
+
+<p>
+It's sometimes useful to temporarily stop the message processing of a connector. For example,
if the remote system is undergoing maintenance, it would be preferable for source connectors
to stop polling it for new data instead of filling logs with exception spam. For this use
case, Connect offers a pause/resume API. While a source connector is paused, Connect will
stop polling it for additional records. While a sink connector is paused, Connect will stop
pushing new messages to it. The pause state is persistent, so even if you restart the cluster,
the connector will not begin message processing again until the task has been resumed. Note
that there may be a delay before all of a connector's tasks have transitioned to the PAUSED
state since it may take time for them to finish whatever processing they were in the middle
of when being paused. Additionally, failed tasks will not transition to the PAUSED state until
they have been restarted.
+</p>


Mime
View raw message