kafka-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From gwens...@apache.org
Subject kafka git commit: KAFKA-2379: Add basic documentation for Kafka Connect.
Date Tue, 10 Nov 2015 00:31:46 GMT
Repository: kafka
Updated Branches:
  refs/heads/0.9.0 2b382d6f9 -> 2b97b6cc0


KAFKA-2379: Add basic documentation for Kafka Connect.

Author: Ewen Cheslack-Postava <me@ewencp.org>

Reviewers: Gwen Shapira

Closes #475 from ewencp/kafka-2379-connect-docs

(cherry picked from commit 83eaf3284fe8535bd06743a367a469d3d01946b1)
Signed-off-by: Gwen Shapira <cshapi@gmail.com>


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

Branch: refs/heads/0.9.0
Commit: 2b97b6cc059d1d8f33d9ef943c758682015e776b
Parents: 2b382d6
Author: Ewen Cheslack-Postava <me@ewencp.org>
Authored: Mon Nov 9 16:29:30 2015 -0800
Committer: Gwen Shapira <cshapi@gmail.com>
Committed: Mon Nov 9 16:31:16 2015 -0800

----------------------------------------------------------------------
 .gitignore                                      |   4 +
 build.gradle                                    |   8 +-
 config/connect-console-sink.properties          |   2 +-
 config/connect-console-source.properties        |   2 +-
 config/connect-file-sink.properties             |   2 +-
 config/connect-file-source.properties           |   2 +-
 .../runtime/distributed/DistributedConfig.java  |   3 +
 docs/configuration.html                         |   5 +-
 docs/connect.html                               | 328 +++++++++++++++++++
 docs/documentation.html                         |  21 +-
 docs/quickstart.html                            |  62 ++++
 11 files changed, 431 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/2b97b6cc/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index a9dad34..5a8ba0f 100644
--- a/.gitignore
+++ b/.gitignore
@@ -37,3 +37,7 @@ results
 tests/results
 .ducktape
 tests/.ducktape
+docs/producer_config.html
+docs/consumer_config.html
+docs/kafka_config.html
+docs/connect_config.html

http://git-wip-us.apache.org/repos/asf/kafka/blob/2b97b6cc/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index fc7dd79..c694c59 100644
--- a/build.gradle
+++ b/build.gradle
@@ -319,7 +319,7 @@ project(':core') {
     standardOutput = new File('docs/kafka_config.html').newOutputStream()
   }
 
-  task siteDocsTar(dependsOn: ['genProducerConfigDocs', 'genConsumerConfigDocs', 'genKafkaConfigDocs'],
type: Tar) {
+  task siteDocsTar(dependsOn: ['genProducerConfigDocs', 'genConsumerConfigDocs', 'genKafkaConfigDocs',
':connect:runtime:genConnectConfigDocs'], type: Tar) {
     classifier = 'site-docs'
     compression = Compression.GZIP
     from project.file("../docs")
@@ -761,6 +761,12 @@ project(':connect:runtime') {
     configFile = new File(rootDir, "checkstyle/checkstyle.xml")
   }
   test.dependsOn('checkstyleMain', 'checkstyleTest')
+
+  tasks.create(name: "genConnectConfigDocs", dependsOn:jar, type: JavaExec) {
+    classpath = sourceSets.main.runtimeClasspath
+    main = 'org.apache.kafka.connect.runtime.distributed.DistributedConfig'
+    standardOutput = new File('docs/connect_config.html').newOutputStream()
+  }
 }
 
 project(':connect:file') {

http://git-wip-us.apache.org/repos/asf/kafka/blob/2b97b6cc/config/connect-console-sink.properties
----------------------------------------------------------------------
diff --git a/config/connect-console-sink.properties b/config/connect-console-sink.properties
index 30dbe2f..e240a8f 100644
--- a/config/connect-console-sink.properties
+++ b/config/connect-console-sink.properties
@@ -16,4 +16,4 @@
 name=local-console-sink
 connector.class=org.apache.kafka.connect.file.FileStreamSinkConnector
 tasks.max=1
-topics=test
\ No newline at end of file
+topics=connect-test
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/2b97b6cc/config/connect-console-source.properties
----------------------------------------------------------------------
diff --git a/config/connect-console-source.properties b/config/connect-console-source.properties
index d161e4e..d0e2069 100644
--- a/config/connect-console-source.properties
+++ b/config/connect-console-source.properties
@@ -16,4 +16,4 @@
 name=local-console-source
 connector.class=org.apache.kafka.connect.file.FileStreamSourceConnector
 tasks.max=1
-topic=test
\ No newline at end of file
+topic=connect-test
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/2b97b6cc/config/connect-file-sink.properties
----------------------------------------------------------------------
diff --git a/config/connect-file-sink.properties b/config/connect-file-sink.properties
index 17275b3..e2cf361 100644
--- a/config/connect-file-sink.properties
+++ b/config/connect-file-sink.properties
@@ -17,4 +17,4 @@ name=local-file-sink
 connector.class=org.apache.kafka.connect.file.FileStreamSinkConnector
 tasks.max=1
 file=test.sink.txt
-topics=test
\ No newline at end of file
+topics=connect-test
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/2b97b6cc/config/connect-file-source.properties
----------------------------------------------------------------------
diff --git a/config/connect-file-source.properties b/config/connect-file-source.properties
index 31fa96c..df92d44 100644
--- a/config/connect-file-source.properties
+++ b/config/connect-file-source.properties
@@ -17,4 +17,4 @@ name=local-file-source
 connector.class=org.apache.kafka.connect.file.FileStreamSourceConnector
 tasks.max=1
 file=test.txt
-topic=test
\ No newline at end of file
+topic=connect-test
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/2b97b6cc/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java
----------------------------------------------------------------------
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java
b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java
index 989f8c5..043c542 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java
@@ -184,4 +184,7 @@ public class DistributedConfig extends WorkerConfig {
         super(CONFIG, props);
     }
 
+    public static void main(String[] args) {
+        System.out.println(CONFIG.toHtmlTable());
+    }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/2b97b6cc/docs/configuration.html
----------------------------------------------------------------------
diff --git a/docs/configuration.html b/docs/configuration.html
index b33cf6a..abaff63 100644
--- a/docs/configuration.html
+++ b/docs/configuration.html
@@ -329,4 +329,7 @@ The essential consumer configurations are the following:
 
 <h3><a id="newconsumerconfigs">3.4 New Consumer Configs</a></h3>
 Since 0.9.0.0 we have been working on a replacement for our existing simple and high-level
consumers. The code can be considered beta quality. Below is the configuration for the new
consumer:
-<!--#include virtual="consumer_config.html" -->
\ No newline at end of file
+<!--#include virtual="consumer_config.html" -->
+
+<h3><a id="connectconfigs">3.5 Kafka Connect Configs</a></h3>
+<!--#include virtual="connect_config.html" -->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/2b97b6cc/docs/connect.html
----------------------------------------------------------------------
diff --git a/docs/connect.html b/docs/connect.html
new file mode 100644
index 0000000..8791ab0
--- /dev/null
+++ b/docs/connect.html
@@ -0,0 +1,328 @@
+<!--~
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  ~-->
+
+<h3><a id="connect_overview">8.1 Overview</a></h3>
+
+Kafka Connect is a tool for scalably and reliably streaming data between Apache Kafka and
other systems. It makes it simple to quickly define <i>connectors</i> that move
large collections of data into and out of Kafka. Kafka Connect can ingest entire databases
or collect metrics from all your application servers into Kafka topics, making the data available
for stream processing with low latency. An export job can deliver data from Kafka topics into
secondary storage and query systems or into batch systems for offline analysis.
+
+Kafka Connect features include:
+<ul>
+    <li><b>A common framework for Kafka connectors</b> - Kafka Connect
standardizes integration of other data systems with Kafka, simplifying connector development,
deployment, and management</li>
+    <li><b>Distributed and standalone modes</b> - scale up to a large,
centrally managed service supporting an entire organization or scale down to development,
testing, and small production deployments</li>
+    <li><b>REST interface</b> - submit and manage connectors to your Kafka
Connect cluster via an easy to use REST API</li>
+    <li><b>Automatic offset management</b> - with just a little information
from connectors, Kafka Connect can manage the offset commit process automatically so connector
developers do not need to worry about this error prone part of connector development</li>
+    <li><b>Distributed and scalable by default</b> - Kafka Connect builds
on the existing </li>
+    <li><b>Streaming/batch integration</b> - leveraging Kafka's existing
capabilities, Kafka Connect is an ideal solution for bridging streaming and batch data systems</li>
+</ul>
+
+<h3><a id="connect_user">8.2 User Guide</a></h3>
+
+The quickstart provides a brief example of how to run a standalone version of Kafka Connect.
This section describes how to configure, run, and manage Kafka Connect in more detail.
+
+<h4>Running Kafka Connect</h4>
+
+Kafka Connect currently supports two modes of execution: standalone (single process) and
distributed.
+
+In standalone mode all work is performed in a single process. This configuration is simpler
to setup and get started with and may be useful in situations where only one worker makes
sense (e.g. collecting log files), but it does not benefit from some of the features of Kafka
Connect such as fault tolerance. You can start a standalone process with the following command:
+
+<pre>
+&gt; bin/connect-standalone.sh config/connect-standalone.properties connector1.properties
[connector2.properties ...]
+</pre>
+
+The first parameter is the configuration for the worker. This includes settings such as the
Kafka connection parameters, serialization format, and how frequently to commit offsets. The
provided example should work well with a local cluster running with the default configuration
provided by <code>config/server.properties</code>. It will require tweaking to
use with a different configuration or production deployment.
+
+The remaining parameters are connector configuration files. You may include as many as you
want, but all will execute within the same process (on different threads).
+
+Distributed mode handles automatic balancing of work, allows you to scale up (or down) dynamically,
and offers fault tolerance both in the active tasks and for configuration and offset commit
data. Execution is very similar to standalone mode:
+
+<pre>
+&gt; bin/connect-distributed.sh config/connect-distributed.properties
+</pre>
+
+The difference is in the class which is started and the configuration parameters which change
how the Kafka Connect process decides where to store configurations, how to assign work, and
where to store offsets. In particular, the following configuration parameters are critical
to set before starting your cluster:
+
+<ul>
+    <li><code>group.id</code> (default <code>connect-cluster</code>)
- unique name for the cluster, used in forming the Connect cluster group; note that this <b>must
not conflict</b> with consumer group IDs</li>
+    <li><code>config.storage.topic</code> (default <code>connect-configs</code>)
- topic to use for storing connector and task configurations; note that this should be a single
partition, highly replicated topic</li>
+    <li><code>offset.storage.topic</code> (default <code>connect-offsets</code>)
- topic to use for ; this topic should have many partitions and be replicated</li>
+</ul>
+
+Note that in distributed mode the connector configurations are not passed on the command
line. Instead, use the REST API described below to create, modify, and destroy connectors.
+
+
+<h4>Configuring Connectors</h4>
+
+Connector configurations are simple key-value mappings. For standalone mode these are defined
in a properties file and passed to the Connect process on the command line. In distributed
mode, they will be included in the JSON payload for the request that creates (or modifies)
the connector.
+
+Most configurations are connector dependent, so they can't be outlined here. However, there
are a few common options:
+
+<ul>
+    <li><code>name</code> - Unique name for the connector. Attempting to
register again with the same name will fail.</li>
+    <li><code>connector.class</code> - The Java class for the connector</li>
+    <li><code>tasks.max</code> - The maximum number of tasks that should
be created for this connector. The connector may create fewer tasks if it cannot achieve this
level of parallelism.</li>
+</ul>
+
+Sink connectors also have one additional option to control their input:
+<ul>
+    <li><code>topics</code> - A list of topics to use as input for this
connector</li>
+</ul>
+
+For any other options, you should consult the documentation for the connector.
+
+
+<h4>REST API</h4>
+
+Since Kafka Connect is intended to be run as a service, it also supports a REST API for managing
connectors. By default this service runs on port 8083. The following are the currently supported
endpoints:
+
+<ul>
+    <li><code>GET /connectors</code> - return a list of active connectors</li>
+    <li><code>POST /connectors</code> - create a new connector; the request
body should be a JSON object containing a string <code>name</code> field and a
object <code>config</code> field with the connector configuration parameters</li>
+    <li><code>GET /connectors/{name}</code> - get information about a specific
connector</li>
+    <li><code>GET /connectors/{name}/config</code> - get the configuration
parameters for a specific connector</li>
+    <li><code>PUT /connectors/{name}/config</code> - update the configuration
parameters for a specific connector</li>
+    <li><code>GET /connectors/{name}/tasks</code> - get a list of tasks
currently running for a connector</li>
+    <li><code>DELETE /connectors/{name}</code> - delete a connector, halting
all tasks and deleting its configuration</li>
+</ul>
+
+<h3><a id="connect_development">8.3 Connector Development Guide</a></h3>
+
+This guide describes how developers can write new connectors for Kafka Connect to move data
between Kafka and other systems. It briefly reviews a few key concepts and then describes
how to create a simple connector.
+
+<h4>Core Concepts and APIs</h4>
+
+<h5>Connectors and Tasks</h5>
+
+To copy data between Kafka and another system, users create a <code>Connector</code>
for the system they want to pull data from or push data to. Connectors come in two flavors:
<code>SourceConnectors</code> import data from another system (e.g. <code>JDBCSourceConnector</code>
would import a relational database into Kafka) and <code>SinkConnectors</code>
export data (e.g. <code>HDFSSinkConnector</code> would export the contents of
a Kafka topic to an HDFS file).
+
+<code>Connectors</code> do not perform any data copying themselves: their configuration
describes the data to be copied, and the <code>Connector</code> is responsible
for breaking that job into a set of <code>Tasks</code> that can be distributed
to workers. These <code>Tasks</code> also come in two corresponding flavors: <code>SourceTask</code>and
<code>SinkTask</code>.
+
+With an assignment in hand, each <code>Task</code> must copy its subset of the
data to or from Kafka. In Kafka Connect, it should always be possible to frame these assignments
as a set of input and output streams consisting of records with consistent schemas. Sometimes
this mapping is obvious: each file in a set of log files can be considered a stream with each
parsed line forming a record using the same schema and offsets stored as byte offsets in the
file. In other cases it may require more effort to map to this model: a JDBC connector can
map each table to a stream, but the offset is less clear. One possible mapping uses a timestamp
column to generate queries incrementally returning new data, and the last queried timestamp
can be used as the offset.
+
+
+<h5>Streams and Records</h5>
+
+Each stream should be a sequence of key-value records. Both the keys and values can have
complex structure -- many primitive types are provided, but arrays, objects, and nested data
structures can be represented as well. The runtime data format does not assume any particular
serialization format; this conversion is handled internally by the framework.
+
+In addition to the key and value, records (both those generated by sources and those delivered
to sinks) have associated stream IDs and offsets. These are used by the framework to periodically
commit the offsets of data that have been processed so that in the event of failures, processing
can resume from the last committed offsets, avoiding unnecessary reprocessing and duplication
of events.
+
+<h5>Dynamic Connectors</h5>
+
+Not all jobs are static, so <code>Connector</code> implementations are also responsible
for monitoring the external system for any changes that might require reconfiguration. For
example, in the <code>JDBCSourceConnector</code> example, the <code>Connector</code>
might assign a set of tables to each <code>Task</code>. When a new table is created,
it must discover this so it can assign the new table to one of the <code>Tasks</code>
by updating its configuration. When it notices a change that requires reconfiguration (or
a change in the number of <code>Tasks</code>), it notifies the framework and the
framework updates anycorresponding <code>Tasks</code>.
+
+
+<h4>Developing a Simple Connector</h4>
+
+Developing a connector only requires implementing two interfaces, the <code>Connector</code>
and <code>Task</code>. A simple example is included with the source code for Kafka
in the <code>file</code> package. This connector is meant for use in standalone
mode and has implementations of a <code>SourceConnector</code>/<code>SourceTask</code>
to read each line of a file and emit it as a record and a <code>SinkConnector</code>/<code>SinkTask</code>
that writes each record to a file.
+
+The rest of this section will walk through some code to demonstrate the key steps in creating
a connector, but developers should also refer to the full example source code as many details
are omitted for brevity.
+
+<h5>Connector Example</h5>
+
+We'll cover the <code>SourceConnector</code> as a simple example. <code>SinkConnector</code>
implementations are very similar. Start by creating the class that inherits from <code>SourceConnector</code>
and add a couple of fields that will store parsed configuration information (the filename
to read from and the topic to send data to):
+
+<pre>
+public class FileStreamSourceConnector extends SourceConnector {
+    private String filename;
+    private String topic;
+</pre>
+
+The easiest method to fill in is <code>getTaskClass()</code>, which defines the
class that should be instantiated in worker processes to actually read the data:
+
+<pre>
+@Override
+public Class<? extends Task> getTaskClass() {
+    return FileStreamSourceTask.class;
+}
+</pre>
+
+We will define the <code>FileStreamSourceTask</code> class below. Next, we add
some standard lifecycle methods, <code>start()</code> and <code>stop()</code>:
+
+<pre>
+@Override
+public void start(Map&lt;String, String&gt; props) {
+    // The complete version includes error handling as well.
+    filename = props.get(FILE_CONFIG);
+    topic = props.get(TOPIC_CONFIG);
+}
+
+@Override
+public void stop() {
+    // Nothing to do since no background monitoring is required.
+}
+</pre>
+
+Finally, the real core of the implementation is in <code>getTaskConfigs()</code>.
In this case we're only
+handling a single file, so even though we may be permitted to generate more tasks as per
the
+<code>maxTasks</code> argument, we return a list with only one entry:
+
+<pre>
+@Override
+public List&lt;Map&lt;String, String&gt;&gt; getTaskConfigs(int maxTasks)
{
+    ArrayList&gt;Map&lt;String, String&gt;&gt; configs = new ArrayList&lt;&gt;();
+    // Only one input stream makes sense.
+    Map&lt;String, String&gt; config = new Map&lt;&gt;();
+    if (filename != null)
+        config.put(FILE_CONFIG, filename);
+    config.put(TOPIC_CONFIG, topic);
+    configs.add(config);
+    return configs;
+}
+</pre>
+
+Even with multiple tasks, this method implementation is usually pretty simple. It just has
to determine the number of input tasks, which may require contacting the remote service it
is pulling data from, and then divvy them up. Because some patterns for splitting work among
tasks are so common, some utilities are provided in <code>ConnectorUtils</code>
to simplify these cases.
+
+Note that this simple example does not include dynamic input. See the discussion in the next
section for how to trigger updates to task configs.
+
+<h5>Task Example - Source Task</h5>
+
+Next we'll describe the implementation of the corresponding <code>SourceTask</code>.
The implementation is short, but too long to cover completely in this guide. We'll use pseudo-code
to describe most of the implementation, but you can refer to the source code for the full
example.
+
+Just as with the connector, we need to create a class inheriting from the appropriate base
<code>Task</code> class. It also has some standard lifecycle methods:
+
+
+<pre>
+public class FileStreamSourceTask extends SourceTask&lt;Object, Object&gt; {
+    String filename;
+    InputStream stream;
+    String topic;
+
+    public void start(Map&lt;String, String&gt; props) {
+        filename = props.get(FileStreamSourceConnector.FILE_CONFIG);
+        stream = openOrThrowError(filename);
+        topic = props.get(FileStreamSourceConnector.TOPIC_CONFIG);
+    }
+
+    @Override
+    public synchronized void stop() {
+        stream.close()
+    }
+</pre>
+
+These are slightly simplified versions, but show that that these methods should be relatively
simple and the only work they should perform is allocating or freeing resources. There are
two points to note about this implementation. First, the <code>start()</code>
method does not yet handle resuming from a previous offset, which will be addressed in a later
section. Second, the <code>stop()</code> method is synchronized. This will be
necessary because <code>SourceTasks</code> are given a dedicated thread which
they can block indefinitely, so they need to be stopped with a call from a different thread
in the Worker.
+
+Next, we implement the main functionality of the task, the <code>poll()</code>
method which gets events from the input system and returns a <code>List&lt;SourceRecord&gt;</code>:
+
+<pre>
+@Override
+public List&lt;SourceRecord&gt; poll() throws InterruptedException {
+    try {
+        ArrayList&lt;SourceRecord&gt; records = new ArrayList&lt;&gt;();
+        while (streamValid(stream) && records.isEmpty()) {
+            LineAndOffset line = readToNextLine(stream);
+            if (line != null) {
+                Map<String, Object> sourcePartition = Collections.singletonMap("filename",
filename);
+                Map<String, Object> sourceOffset = Collections.singletonMap("position",
streamOffset);
+                records.add(new SourceRecord(sourcePartition, sourceOffset, topic, Schema.STRING_SCHEMA,
line));
+            } else {
+                Thread.sleep(1);
+            }
+        }
+        return records;
+    } catch (IOException e) {
+        // Underlying stream was killed, probably as a result of calling stop. Allow to return
+        // null, and driving thread will handle any shutdown if necessary.
+    }
+    return null;
+}
+</pre>
+
+Again, we've omitted some details, but we can see the important steps: the <code>poll()</code>
method is going to be called repeatedly, and for each call it will loop trying to read records
from the file. For each line it reads, it also tracks the file offset. It uses this information
to create an output <code>SourceRecord</code> with four pieces of information:
the source partition (there is only one, the single file being read), source offset (byte
offset in the file), output topic name, and output value (the line, and we include a schema
indicating this value will always be a string). Other variants of the <code>SourceRecord</code>
constructor can also inclue a specific output partition and a key.
+
+Note that this implementation uses the normal Java <code>InputStream</code>interface
and may sleep if data is not avaiable. This is acceptable because Kafka Connect provides each
task with a dedicated thread. While task implementations have to conform to the basic <code>poll()</code>interface,
they have a lot of flexibility in how they are implemented. In this case, an NIO-based implementation
would be more efficient, but this simple approach works, is quick to implement, and is compatible
with older versions of Java.
+
+<h5>Sink Tasks</h5>
+
+The previous section described how to implement a simple <code>SourceTask</code>.
Unlike <code>SourceConnector</code>and <code>SinkConnector</code>,
<code>SourceTask</code>and <code>SinkTask</code>have very different
interfaces because <code>SourceTask</code>uses a pull interface and <code>SinkTask</code>uses
a push interface. Both share the common lifecycle methods, but the <code>SinkTask</code>interface
is quite different:
+
+<pre>
+public abstract class SinkTask implements Task {
+public void initialize(SinkTaskContext context) { ... }
+
+public abstract void put(Collection&lt;SinkRecord&gt; records);
+
+public abstract void flush(Map&lt;TopicPartition, Long&gt; offsets);
+</pre>
+
+The <code>SinkTask</code> documentation contains full details, but this interface
is nearly as simple as the the <code>SourceTask</code>. The <code>put()</code>method
should contain most of the implementation, accepting sets of <code>SinkRecords</code>,
performing any required translation, and storing them in the destination system. This method
does not need to ensure the data has been fully written to the destination system before returning.
In fact, in many cases internal buffering will be useful so an entire batch of records can
be sent at once, reducing the overhead of inserting events into the downstream data store.
The <code>SinkRecords</code>contain essentially the same information as <code>SourceRecords</code>:
Kafka topic, partition, offset and the event key and value.
+
+The <code>flush()</code>method is used during the offset commit process, which
allows tasks to recover from failures and resume from a safe point such that no events will
be missed. The method should push any outstanding data to the destination system and then
block until the write has been acknowledged. The <code>offsets</code>parameter
can often be ignored, but is useful in some cases where implementations want to store offset
information in the destination store to provide exactly-once
+delivery. For example, an HDFS connector could do this and use atomic move operations to
make sure the <code>flush()</code>operation atomically commits the data and offsets
to a final location in HDFS.
+
+
+<h5>Resuming from Previous Offsets</h5>
+
+The <code>SourceTask</code>implementation included a stream ID (the input filename)
and offset (position in the file) with each record. The framework uses this to commit offsets
periodically so that in the case of a failure, the task can recover and minimize the number
of events that are reprocessed and possibly duplicated (or to resume from the most recent
offset if Kafka Connect was stopped gracefully, e.g. in standalone mode or due to a job reconfiguration).
This commit process is completely automated by the framework, but only the connector knows
how to seek back to the right position in the input stream to resume from that location.
+
+To correctly resume upon startup, the task can use the <code>SourceContext</code>passed
into its <code>initialize()</code>method to access the offset data. In <code>initialize()</code>,
we would add a bit more code to read the offset (if it exists) and seek to that position:
+
+<pre>
+    stream = new FileInputStream(filename);
+    Map&lt;String, Object&gt; offset = context.offsetStorageReader().offset(Collections.singletonMap(FILENAME_FIELD,
filename));
+    if (offset != null) {
+        Long lastRecordedOffset = (Long) offset.get("position");
+        if (lastRecordedOffset != null)
+            seekToOffset(stream, lastRecordedOffset);
+    }
+</pre>
+
+Of course, you might need to read many keys for each of the input streams. The <code>OffsetStorageReader</code>
interface also allows you to issue bulk reads to efficiently load all offsets, then apply
them by seeking each input stream to the appropriate position.
+
+<h4>Dynamic Input/Output Streams</h4>
+
+Kafka Connect is intended to define bulk data copying jobs, such as copying an entire database
rather than creating many jobs to copy each table individually. One consequence of this design
is that the set of input or output streams for a connector can vary over time.
+
+Source connectors need to monitor the source system for changes, e.g. table additions/deletions
in a database. When they pick up changes, they should notify the framework via the <code>ConnectorContext</code>object
that reconfiguration is necessary. For example, in a <code>SourceConnector</code>:
+
+
+<pre>
+if (inputsChanged())
+    this.context.requestTaskReconfiguration();
+</pre>
+
+The framework will promptly request new configuration information and update the tasks, allowing
them to gracefully commit their progress before reconfiguring them. Note that in the <code>SourceConnector</code>this
monitoring is currently left up to the connector implementation. If an extra thread is required
to perform this monitoring, the connector must allocate it itself.
+
+Ideally this code for monitoring changes would be isolated to the <code>Connector</code>and
tasks would not need to worry about them. However, changes can also affect tasks, most commonly
when one of their input streams is destroyed in the input system, e.g. if a table is dropped
from a database. If the <code>Task</code>encounters the issue before the <code>Connector</code>,
which will be common if the <code>Connector</code>needs to poll for changes, the
<code>Task</code>will need to handle the subsequent error. Thankfully, this can
usually be handled simply by catching and handling the appropriate exception.
+
+<code>SinkConnectors</code> usually only have to handle the addition of streams,
which may translate to new entries in their outputs (e.g., a new database table). The framework
manages any changes to the Kafka input, such as when the set of input topics changes because
of a regex subscription. <code>SinkTasks</code>should expect new input streams,
which may require creating new resources in the downstream system, such as a new table in
a database. The trickiest situation to handle in these cases may be conflicts between multiple
<code>SinkTasks</code>seeing a new input stream for the first time and simultaneoulsy
trying to create the new resource. <code>SinkConnectors</code>, on the other hand,
will generally require no special code for handling a dynamic set of streams.
+
+<h4>Working with Schemas</h4>
+
+The FileStream connectors are good examples because they are simple, but they also have trivially
structured data -- each line is just a string. Almost all practical connectors will need schemas
with more complex data formats.
+
+To create more complex data, you'll need to work with the Kafka Connect <code>data</code>
API. Most structured records will need to interact with two classes in addition to primitive
types: <code>Schema</code> and <code>Struct</code>.
+
+The API documentation provides a complete reference, but here is a simple example creating
a <code>Schema</code>and <code>Struct</code>:
+
+<pre>
+Schema schema = SchemaBuilder.struct().name(NAME)
+                    .field("name", Schema.STRING_SCHEMA)
+                    .field("age", Schema.INT_SCHEMA)
+                    .field("admin", new SchemaBuilder.boolean().defaultValue(false).build())
+                    .build();
+
+Struct struct = new Struct(schema)
+                           .put("name", "Barbara Liskov")
+                           .put("age", 75)
+                           .build();
+</pre>
+
+If you are implementing a source connector, you'll need to decide when and how to create
schemas. Where possible, you should avoid recomputing them as much as possible. For example,
if your connector is guaranteed to have a fixed schema, create it statically and reuse a single
instance.
+
+However, many connectors will have dynamic schemas. One simple example of this is a database
connector. Considering even just a single table, the schema will not be predefined for the
entire connector (as it varies from table to table). But it also may not be fixed for a single
table over the lifetime of the connector since the user may execute an <code>ALTER TABLE</code>command.
The connector must be able to detect these changes and react appropriately.
+
+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.
+

http://git-wip-us.apache.org/repos/asf/kafka/blob/2b97b6cc/docs/documentation.html
----------------------------------------------------------------------
diff --git a/docs/documentation.html b/docs/documentation.html
index a1ac463..c64e67f 100644
--- a/docs/documentation.html
+++ b/docs/documentation.html
@@ -30,6 +30,7 @@ Prior releases: <a href="/07/documentation.html">0.7.x</a>,
<a href="/08/documen
              <li><a href="#ecosystem">1.4 Ecosystem</a>
              <li><a href="#upgrade">1.5 Upgrading</a>
          </ul>
+    </li>
     <li><a href="#api">2. API</a>
           <ul>
               <li><a href="#producerapi">2.1 Producer API</a>
@@ -37,13 +38,16 @@ Prior releases: <a href="/07/documentation.html">0.7.x</a>,
<a href="/08/documen
               <li><a href="#simpleconsumerapi">2.3 Simple Consumer API</a>
               <li><a href="#newconsumerapi">2.4 New Consumer API</a>
           </ul>
+    </li>
     <li><a href="#configuration">3. Configuration</a>
         <ul>
             <li><a href="#brokerconfigs">3.1 Broker Configs</a>
             <li><a href="#producerconfigs">3.2 Producer Configs</a>
             <li><a href="#consumerconfigs">3.3 Consumer Configs</a>
             <li><a href="#newconsumerconfigs">3.4 New Consumer Configs</a>
+            <li><a href="#connectconfigs">3.5 Kafka Connect Configs</a>
         </ul>
+    </li>
     <li><a href="#design">4. Design</a>
         <ul>
              <li><a href="#majordesignelements">4.1 Motivation</a>
@@ -55,6 +59,7 @@ Prior releases: <a href="/07/documentation.html">0.7.x</a>,
<a href="/08/documen
              <li><a href="#replication">4.7 Replication</a>
              <li><a href="#compaction">4.8 Log Compaction</a>
         </ul>
+    </li>
     <li><a href="#implementation">5. Implementation</a>
         <ul>
               <li><a href="#apidesign">5.1 API Design</a>
@@ -64,6 +69,7 @@ Prior releases: <a href="/07/documentation.html">0.7.x</a>,
<a href="/08/documen
               <li><a href="#log">5.5 Log</a>
               <li><a href="#distributionimpl">5.6 Distribution</a>
         </ul>
+    </li>
     <li><a href="#operations">6. Operations</a>
         <ul>
              <li><a href="#basic_ops">6.1 Basic Kafka Operations</a>
@@ -101,13 +107,22 @@ Prior releases: <a href="/07/documentation.html">0.7.x</a>,
<a href="/08/documen
                     <li><a href="#zkops">Operationalization</a>
                 </ul>
         </ul>
-    <li><a href="#security">7. Security</a></li>
+    </li>
+    <li><a href="#security">7. Security</a>
         <ul>
             <li><a href="#security_overview">7.1 Security Overview</a></li>
             <li><a href="#security_ssl">7.2 Encryption and Authentication using
SSL</a></li>
             <li><a href="#security_sasl">7.3 Authentication using SASL</a></li>
             <li><a href="#security_authz">7.4 Authorization and ACLs</a></li>
         </ul>
+    </li>
+    <li><a href="#connect">8. Kafka Connect</a>
+        <ul>
+            <li><a href="#connect_overview">8.1 Overview</a></li>
+            <li><a href="#connect_user">8.2 User Guide</a></li>
+            <li><a href="#connect_development">8.3 Connector Development Guide</a></li>
+        </ul>
+    </li>
 </ul>
 
 <h2><a id="gettingStarted">1. Getting Started</a></h2>
@@ -140,5 +155,7 @@ Prior releases: <a href="/07/documentation.html">0.7.x</a>,
<a href="/08/documen
 <h2><a id="security">7. Security</a></h2>
 <!--#include virtual="security.html" -->
 
-<!--#include virtual="../includes/footer.html" -->
+<h2><a id="connect">8. Kafka Connect</a></h2>
+<!--#include virtual="connect.html" -->
 
+<!--#include virtual="../includes/footer.html" -->

http://git-wip-us.apache.org/repos/asf/kafka/blob/2b97b6cc/docs/quickstart.html
----------------------------------------------------------------------
diff --git a/docs/quickstart.html b/docs/quickstart.html
index 18f610b..ac5623b 100644
--- a/docs/quickstart.html
+++ b/docs/quickstart.html
@@ -187,3 +187,65 @@ my test message 1
 my test message 2
 <b>^C</b>
 </pre>
+
+
+<h4>Step 7: Use Kafka Connect to import/export data</h4>
+
+Writing data from the console and writing it back to the console is a convenient place to
start, but you'll probably want
+to use data from other sources or export data from Kafka to other systems. For many systems,
instead of writing custom
+integration code you can use Kafka Connect to import or export data.
+
+Kafka Connect is a tool included with Kafka that imports and exports data to Kafka. It is
an extensible tool that runs
+<i>connectors</i>, which implement the custom logic for interacting with an external
system. In this quickstart we'll see
+how to run Kafka Connect with simple connectors that import data from a file to a Kafka topic
and export data from a
+Kafka topic to a file.
+
+First, we’ll start by creating some seed data to test with:
+
+<pre>
+&gt; <b>echo -e "foo\nbar" > test.txt</b>
+</pre>
+
+Next, we'll start two connectors running in <i>standalone</i> mode, which means
they run in a single, local, dedicated
+process. We provide three configuration files as parameters. The first is always the configuration
for the Kafka Connect
+process, containing common configuration such as the Kafka brokers to connect to and the
serialization format for data.
+The remaining configuration files each specify a connector to create. These files include
a unique connector name, the connector
+class to instantiate, and any other configuration required by the connector.
+
+<pre>
+&gt; <b>bin/connect-standalone.sh config/connect-standalone.properties config/connect-file-source.properties
config/connect-file-sink.properties</b>
+</pre>
+
+These sample configuration files, included with Kafka, use the default local cluster configuration
you started earlier
+and create two connectors: the first is a source connector that reads lines from an input
file and produces each to a Kafka topic
+and the second is a sink connector that reads messages from a Kafka topic and produces each
as a line in an output file.
+
+During startup you'll see a number of log messages, including some indicating that the connectors
are being instantiated.
+Once the Kafka Connect process has started, the source connector should start reading lines
from <pre>test.txt</pre> and
+producing them to the topic <pre>connect-test</pre>, and the sink connector should
start reading messages from the topic <pre>connect-test</pre>
+and write them to the file <pre>test.sink.txt</pre>. We can verify the data has
been delivered through the entire pipeline
+by examining the contents of the output file:
+
+<pre>
+&gt; <b>cat test.sink.txt</b>
+foo
+bar
+</pre>
+
+Note that the data is being stored in the Kafka topic <pre>connect-test</pre>,
so we can also run a console consumer to see the
+data in the topic (or use custom consumer code to process it):
+
+<pre>
+&gt; <b>bin/kafka-console-consumer.sh --zookeeper localhost:2181 --topic connect-test
--from-beginning</b>
+{"schema":{"type":"string","optional":false},"payload":"foo"}
+{"schema":{"type":"string","optional":false},"payload":"bar"}
+...
+</pre>
+
+The connectors continue to process data, so we can add data to the file and see it move through
the pipeline:
+
+<pre>
+&gt; <b>echo "Another line" >> test.txt</b>
+</pre>
+
+You should see the line appear in the console consumer output and in the sink file.


Mime
View raw message