Mirror of Apache Kafka
You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.

552 lines
49 KiB

<!--~
~ 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.
~-->
<script id="connect-template" type="text/x-handlebars-template">
<h3><a id="connect_overview" href="#connect_overview">8.1 Overview</a></h3>
<p>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.</p>
<p>Kafka Connect features include:</p>
<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 group management protocol. More workers can be added to scale up a Kafka Connect cluster.</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" href="#connect_user">8.2 User Guide</a></h3>
<p>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.</p>
<h4><a id="connect_running" href="#connect_running">Running Kafka Connect</a></h4>
<p>Kafka Connect currently supports two modes of execution: standalone (single process) and distributed.</p>
<p>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:</p>
<pre class="brush: bash;">
&gt; bin/connect-standalone.sh config/connect-standalone.properties connector1.properties [connector2.properties ...]
</pre>
<p>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. All workers (both standalone and distributed) require a few configs:</p>
<ul>
<li><code>bootstrap.servers</code> - List of Kafka servers used to bootstrap connections to Kafka</li>
<li><code>key.converter</code> - Converter class used to convert between Kafka Connect format and the serialized form that is written to Kafka. This controls the format of the keys in messages written to or read from Kafka, and since this is independent of connectors it allows any connector to work with any serialization format. Examples of common formats include JSON and Avro.</li>
<li><code>value.converter</code> - Converter class used to convert between Kafka Connect format and the serialized form that is written to Kafka. This controls the format of the values in messages written to or read from Kafka, and since this is independent of connectors it allows any connector to work with any serialization format. Examples of common formats include JSON and Avro.</li>
</ul>
<p>The important configuration options specific to standalone mode are:</p>
<ul>
<li><code>offset.storage.file.filename</code> - File to store offset data in</li>
</ul>
<p>The parameters that are configured here are intended for producers and consumers used by Kafka Connect to access the configuration, offset and status topics. For configuration of Kafka source and Kafka sink tasks, the same parameters can be used but need to be prefixed with <code>consumer.</code> and <code>producer.</code> respectively. The only parameter that is inherited from the worker configuration is <code>bootstrap.servers</code>, which in most cases will be sufficient, since the same cluster is often used for all purposes. A notable exeption is a secured cluster, which requires extra parameters to allow connections. These parameters will need to be set up to three times in the worker configuration, once for management access, once for Kafka sinks and once for Kafka sources.</p>
<p>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).</p>
<p>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:</p>
<pre class="brush: bash;">
&gt; bin/connect-distributed.sh config/connect-distributed.properties
</pre>
<p>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 and task statues. In the distributed mode, Kafka Connect stores the offsets, configs and task statuses in Kafka topics. It is recommended to manually create the topics for offset, configs and statuses in order to achieve the desired the number of partitions and replication factors. If the topics are not yet created when starting Kafka Connect, the topics will be auto created with default number of partitions and replication factor, which may not be best suited for its usage.</p>
<p>In particular, the following configuration parameters, in addition to the common settings mentioned above, are critical to set before starting your cluster:</p>
<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, compacted topic. You may need to manually create the topic to ensure the correct configuration as auto created topics may have multiple partitions or be automatically configured for deletion rather than compaction</li>
<li><code>offset.storage.topic</code> (default <code>connect-offsets</code>) - topic to use for storing offsets; this topic should have many partitions, be replicated, and be configured for compaction</li>
<li><code>status.storage.topic</code> (default <code>connect-status</code>) - topic to use for storing statuses; this topic can have multiple partitions, and should be replicated and configured for compaction</li>
</ul>
<p>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.</p>
<h4><a id="connect_configuring" href="#connect_configuring">Configuring Connectors</a></h4>
<p>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.</p>
<p>Most configurations are connector dependent, so they can't be outlined here. However, there are a few common options:</p>
<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>
<li><code>key.converter</code> - (optional) Override the default key converter set by the worker.</li>
<li><code>value.converter</code> - (optional) Override the default value converter set by the worker.</li>
</ul>
<p>The <code>connector.class</code> config supports several formats: the full name or alias of the class for this connector. If the connector is org.apache.kafka.connect.file.FileStreamSinkConnector, you can either specify this full name or use FileStreamSink or FileStreamSinkConnector to make the configuration a bit shorter.</p>
<p>Sink connectors also have a few additional options to control their input. Each sink connector must set one of the following:</p>
<ul>
<li><code>topics</code> - A comma-separated list of topics to use as input for this connector</li>
<li><code>topics.regex</code> - A Java regular expression of topics to use as input for this connector</li>
</ul>
<p>For any other options, you should consult the documentation for the connector.</p>
<h4><a id="connect_transforms" href="#connect_transforms">Transformations</a></h4>
<p>Connectors can be configured with transformations to make lightweight message-at-a-time modifications. They can be convenient for data massaging and event routing.</p>
<p>A transformation chain can be specified in the connector configuration.</p>
<ul>
<li><code>transforms</code> - List of aliases for the transformation, specifying the order in which the transformations will be applied.</li>
<li><code>transforms.$alias.type</code> - Fully qualified class name for the transformation.</li>
<li><code>transforms.$alias.$transformationSpecificConfig</code> Configuration properties for the transformation</li>
</ul>
<p>For example, lets take the built-in file source connector and use a transformation to add a static field.</p>
<p>Throughout the example we'll use schemaless JSON data format. To use schemaless format, we changed the following two lines in <code>connect-standalone.properties</code> from true to false:</p>
<pre class="brush: text;">
key.converter.schemas.enable
value.converter.schemas.enable
</pre>
<p>The file source connector reads each line as a String. We will wrap each line in a Map and then add a second field to identify the origin of the event. To do this, we use two transformations:</p>
<ul>
<li><b>HoistField</b> to place the input line inside a Map</li>
<li><b>InsertField</b> to add the static field. In this example we'll indicate that the record came from a file connector</li>
</ul>
<p>After adding the transformations, <code>connect-file-source.properties</code> file looks as following:</p>
<pre class="brush: text;">
name=local-file-source
connector.class=FileStreamSource
tasks.max=1
file=test.txt
topic=connect-test
transforms=MakeMap, InsertSource
transforms.MakeMap.type=org.apache.kafka.connect.transforms.HoistField$Value
transforms.MakeMap.field=line
transforms.InsertSource.type=org.apache.kafka.connect.transforms.InsertField$Value
transforms.InsertSource.static.field=data_source
transforms.InsertSource.static.value=test-file-source
</pre>
<p>All the lines starting with <code>transforms</code> were added for the transformations. You can see the two transformations we created: "InsertSource" and "MakeMap" are aliases that we chose to give the transformations. The transformation types are based on the list of built-in transformations you can see below. Each transformation type has additional configuration: HoistField requires a configuration called "field", which is the name of the field in the map that will include the original String from the file. InsertField transformation lets us specify the field name and the value that we are adding.</p>
<p>When we ran the file source connector on my sample file without the transformations, and then read them using <code>kafka-console-consumer.sh</code>, the results were:</p>
<pre class="brush: text;">
"foo"
"bar"
"hello world"
</pre>
<p>We then create a new file connector, this time after adding the transformations to the configuration file. This time, the results will be:</p>
<pre class="brush: json;">
{"line":"foo","data_source":"test-file-source"}
{"line":"bar","data_source":"test-file-source"}
{"line":"hello world","data_source":"test-file-source"}
</pre>
<p>You can see that the lines we've read are now part of a JSON map, and there is an extra field with the static value we specified. This is just one example of what you can do with transformations.</p>
<p>Several widely-applicable data and routing transformations are included with Kafka Connect:</p>
<ul>
<li>InsertField - Add a field using either static data or record metadata</li>
<li>ReplaceField - Filter or rename fields</li>
<li>MaskField - Replace field with valid null value for the type (0, empty string, etc)</li>
<li>ValueToKey</li>
<li>HoistField - Wrap the entire event as a single field inside a Struct or a Map</li>
<li>ExtractField - Extract a specific field from Struct and Map and include only this field in results</li>
<li>SetSchemaMetadata - modify the schema name or version</li>
<li>TimestampRouter - Modify the topic of a record based on original topic and timestamp. Useful when using a sink that needs to write to different tables or indexes based on timestamps</li>
<li>RegexRouter - modify the topic of a record based on original topic, replacement string and a regular expression</li>
</ul>
<p>Details on how to configure each transformation are listed below:</p>
<!--#include virtual="generated/connect_transforms.html" -->
<h4><a id="connect_rest" href="#connect_rest">REST API</a></h4>
<p>Since Kafka Connect is intended to be run as a service, it also provides a REST API for managing connectors. The REST API server can be configured using the <code>listeners</code> configuration option.
This field should contain a list of listeners in the following format: <code>protocol://host:port,protocol2://host2:port2</code>. Currently supported protocols are <code>http</code> and <code>https</code>.
For example:</p>
<pre class="brush: text;">
listeners=http://localhost:8080,https://localhost:8443
</pre>
<p>By default, if no <code>listeners</code> are specified, the REST server runs on port 8083 using the HTTP protocol. When using HTTPS, the configuration has to include the SSL configuration.
By default, it will use the <code>ssl.*</code> settings. In case it is needed to use different configuration for the REST API than for connecting to Kafka brokers, the fields can be prefixed with <code>listeners.https</code>.
When using the prefix, only the prefixed options will be used and the <code>ssl.*</code> options without the prefix will be ignored. Following fields can be used to configure HTTPS for the REST API:</p>
<ul>
<li><code>ssl.keystore.location</code></li>
<li><code>ssl.keystore.password</code></li>
<li><code>ssl.keystore.type</code></li>
<li><code>ssl.key.password</code></li>
<li><code>ssl.truststore.location</code></li>
<li><code>ssl.truststore.password</code></li>
<li><code>ssl.truststore.type</code></li>
<li><code>ssl.enabled.protocols</code></li>
<li><code>ssl.provider</code></li>
<li><code>ssl.protocol</code></li>
<li><code>ssl.cipher.suites</code></li>
<li><code>ssl.keymanager.algorithm</code></li>
<li><code>ssl.secure.random.implementation</code></li>
<li><code>ssl.trustmanager.algorithm</code></li>
<li><code>ssl.endpoint.identification.algorithm</code></li>
<li><code>ssl.client.auth</code></li>
</ul>
<p>The REST API is used not only by users to monitor / manage Kafka Connect. It is also used for the Kafka Connect cross-cluster communication. Requests received on the follower nodes REST API will be forwarded to the leader node REST API.
In case the URI under which is given host reachable is different from the URI which it listens on, the configuration options <code>rest.advertised.host.name</code>, <code>rest.advertised.port</code> and <code>rest.advertised.listener</code>
can be used to change the URI which will be used by the follower nodes to connect with the leader. When using both HTTP and HTTPS listeners, the <code>rest.advertised.listener</code> option can be also used to define which listener
will be used for the cross-cluster communication. When using HTTPS for communication between nodes, the same <code>ssl.*</code> or <code>listeners.https</code> options will be used to configure the HTTPS client.</p>
<p>The following are the currently supported REST API endpoints:</p>
<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 an 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}/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>
<p>Kafka Connect also provides a REST API for getting information about connector plugins:</p>
<ul>
<li><code>GET /connector-plugins</code>- return a list of connector plugins installed in the Kafka Connect cluster. Note that the API only checks for connectors on the worker that handles the request, which means you may see inconsistent results, especially during a rolling upgrade if you add new connector jars</li>
<li><code>PUT /connector-plugins/{connector-type}/config/validate</code> - validate the provided configuration values against the configuration definition. This API performs per config validation, returns suggested values and error messages during validation.</li>
</ul>
<h3><a id="connect_development" href="#connect_development">8.3 Connector Development Guide</a></h3>
<p>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.</p>
<h4><a id="connect_concepts" href="#connect_concepts">Core Concepts and APIs</a></h4>
<h5><a id="connect_connectorsandtasks" href="#connect_connectorsandtasks">Connectors and Tasks</a></h5>
<p>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).</p>
<p><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>.</p>
<p>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.</p>
<h5><a id="connect_streamsandrecords" href="#connect_streamsandrecords">Streams and Records</a></h5>
<p>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.</p>
<p>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.</p>
<h5><a id="connect_dynamicconnectors" href="#connect_dynamicconnectors">Dynamic Connectors</a></h5>
<p>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 any corresponding <code>Tasks</code>.</p>
<h4><a id="connect_developing" href="#connect_developing">Developing a Simple Connector</a></h4>
<p>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.</p>
<p>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.</p>
<h5><a id="connect_connectorexample" href="#connect_connectorexample">Connector Example</a></h5>
<p>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):</p>
<pre class="brush: java;">
public class FileStreamSourceConnector extends SourceConnector {
private String filename;
private String topic;
</pre>
<p>The easiest method to fill in is <code>taskClass()</code>, which defines the class that should be instantiated in worker processes to actually read the data:</p>
<pre class="brush: java;">
@Override
public Class&lt;? extends Task&gt; taskClass() {
return FileStreamSourceTask.class;
}
</pre>
<p>We will define the <code>FileStreamSourceTask</code> class below. Next, we add some standard lifecycle methods, <code>start()</code> and <code>stop()</code></p>:
<pre class="brush: java;">
@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>
<p>Finally, the real core of the implementation is in <code>taskConfigs()</code>. In this case we are 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:</p>
<pre class="brush: java;">
@Override
public List&lt;Map&lt;String, String&gt;&gt; taskConfigs(int maxTasks) {
ArrayList&lt;Map&lt;String, String&gt;&gt; configs = new ArrayList&lt;&gt;();
// Only one input stream makes sense.
Map&lt;String, String&gt; config = new HashMap&lt;&gt;();
if (filename != null)
config.put(FILE_CONFIG, filename);
config.put(TOPIC_CONFIG, topic);
configs.add(config);
return configs;
}
</pre>
<p>Although not used in the example, <code>SourceTask</code> also provides two APIs to commit offsets in the source system: <code>commit</code> and <code>commitRecord</code>. The APIs are provided for source systems which have an acknowledgement mechanism for messages. Overriding these methods allows the source connector to acknowledge messages in the source system, either in bulk or individually, once they have been written to Kafka.
The <code>commit</code> API stores the offsets in the source system, up to the offsets that have been returned by <code>poll</code>. The implementation of this API should block until the commit is complete. The <code>commitRecord</code> API saves the offset in the source system for each <code>SourceRecord</code> after it is written to Kafka. As Kafka Connect will record offsets automatically, <code>SourceTask</code>s are not required to implement them. In cases where a connector does need to acknowledge messages in the source system, only one of the APIs is typically required.</p>
<p>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.</p>
<p>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.</p>
<h5><a id="connect_taskexample" href="#connect_taskexample">Task Example - Source Task</a></h5>
<p>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.</p>
<p>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:</p>
<pre class="brush: java;">
public class FileStreamSourceTask extends SourceTask {
String filename;
InputStream stream;
String topic;
@Override
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>
<p>These are slightly simplified versions, but show 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.</p>
<p>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>:</p>
<pre class="brush: java;">
@Override
public List&lt;SourceRecord&gt; poll() throws InterruptedException {
try {
ArrayList&lt;SourceRecord&gt; records = new ArrayList&lt;&gt;();
while (streamValid(stream) &amp;&amp; records.isEmpty()) {
LineAndOffset line = readToNextLine(stream);
if (line != null) {
Map&lt;String, Object&gt; sourcePartition = Collections.singletonMap("filename", filename);
Map&lt;String, Object&gt; 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>
KAFKA-5142: Add Connect support for message headers (KIP-145) **[KIP-145](https://cwiki.apache.org/confluence/display/KAFKA/KIP-145+-+Expose+Record+Headers+in+Kafka+Connect) has been accepted, and this PR implements KIP-145 except without the SMTs.** Changed the Connect API and runtime to support message headers as described in [KIP-145](https://cwiki.apache.org/confluence/display/KAFKA/KIP-145+-+Expose+Record+Headers+in+Kafka+Connect). The new `Header` interface defines an immutable representation of a Kafka header (key-value pair) with support for the Connect value types and schemas. This interface provides methods for easily converting between many of the built-in primitive, structured, and logical data types. The new `Headers` interface defines an ordered collection of headers and is used to track all headers associated with a `ConnectRecord` (and thus `SourceRecord` and `SinkRecord`). This does allow multiple headers with the same key. The `Headers` contains methods for adding, removing, finding, and modifying headers. Convenience methods allow connectors and transforms to easily use and modify the headers for a record. A new `HeaderConverter` interface is also defined to enable the Connect runtime framework to be able to serialize and deserialize headers between the in-memory representation and Kafka’s byte[] representation. A new `SimpleHeaderConverter` implementation has been added, and this serializes to strings and deserializes by inferring the schemas (`Struct` header values are serialized without the schemas, so they can only be deserialized as `Map` instances without a schema.) The `StringConverter`, `JsonConverter`, and `ByteArrayConverter` have all been extended to also be `HeaderConverter` implementations. Each connector can be configured with a different header converter, although by default the `SimpleHeaderConverter` is used to serialize header values as strings without schemas. Unit and integration tests are added for `ConnectHeader` and `ConnectHeaders`, the two implementation classes for headers. Additional test methods are added for the methods added to the `Converter` implementations. Finally, the `ConnectRecord` object is already used heavily, so only limited tests need to be added while quite a few of the existing tests already cover the changes. Author: Randall Hauch <rhauch@gmail.com> Reviewers: Arjun Satish <arjun@confluent.io>, Ted Yu <yuzhihong@gmail.com>, Magesh Nandakumar <magesh.n.kumar@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io> Closes #4319 from rhauch/kafka-5142-b
7 years ago
<p>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 include a specific output partition, a key, and headers.</p>
<p>Note that this implementation uses the normal Java <code>InputStream</code> interface and may sleep if data is not available. 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.</p>
<h5><a id="connect_sinktasks" href="#connect_sinktasks">Sink Tasks</a></h5>
<p>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:</p>
<pre class="brush: java;">
public abstract class SinkTask implements Task {
public void initialize(SinkTaskContext context) {
this.context = context;
}
public abstract void put(Collection&lt;SinkRecord&gt; records);
public void flush(Map&lt;TopicPartition, OffsetAndMetadata&gt; currentOffsets) {
}
</pre>
KAFKA-5142: Add Connect support for message headers (KIP-145) **[KIP-145](https://cwiki.apache.org/confluence/display/KAFKA/KIP-145+-+Expose+Record+Headers+in+Kafka+Connect) has been accepted, and this PR implements KIP-145 except without the SMTs.** Changed the Connect API and runtime to support message headers as described in [KIP-145](https://cwiki.apache.org/confluence/display/KAFKA/KIP-145+-+Expose+Record+Headers+in+Kafka+Connect). The new `Header` interface defines an immutable representation of a Kafka header (key-value pair) with support for the Connect value types and schemas. This interface provides methods for easily converting between many of the built-in primitive, structured, and logical data types. The new `Headers` interface defines an ordered collection of headers and is used to track all headers associated with a `ConnectRecord` (and thus `SourceRecord` and `SinkRecord`). This does allow multiple headers with the same key. The `Headers` contains methods for adding, removing, finding, and modifying headers. Convenience methods allow connectors and transforms to easily use and modify the headers for a record. A new `HeaderConverter` interface is also defined to enable the Connect runtime framework to be able to serialize and deserialize headers between the in-memory representation and Kafka’s byte[] representation. A new `SimpleHeaderConverter` implementation has been added, and this serializes to strings and deserializes by inferring the schemas (`Struct` header values are serialized without the schemas, so they can only be deserialized as `Map` instances without a schema.) The `StringConverter`, `JsonConverter`, and `ByteArrayConverter` have all been extended to also be `HeaderConverter` implementations. Each connector can be configured with a different header converter, although by default the `SimpleHeaderConverter` is used to serialize header values as strings without schemas. Unit and integration tests are added for `ConnectHeader` and `ConnectHeaders`, the two implementation classes for headers. Additional test methods are added for the methods added to the `Converter` implementations. Finally, the `ConnectRecord` object is already used heavily, so only limited tests need to be added while quite a few of the existing tests already cover the changes. Author: Randall Hauch <rhauch@gmail.com> Reviewers: Arjun Satish <arjun@confluent.io>, Ted Yu <yuzhihong@gmail.com>, Magesh Nandakumar <magesh.n.kumar@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io> Closes #4319 from rhauch/kafka-5142-b
7 years ago
<p>The <code>SinkTask</code> documentation contains full details, but this interface is nearly as simple as 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, the event key and value, and optional headers.</p>
<p>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.</p>
<h5><a id="connect_resuming" href="#connect_resuming">Resuming from Previous Offsets</a></h5>
<p>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.</p>
<p>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:</p>
<pre class="brush: java;">
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>
<p>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.</p>
<h4><a id="connect_dynamicio" href="#connect_dynamicio">Dynamic Input/Output Streams</a></h4>
<p>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.</p>
<p>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>:</p>
<pre class="brush: java;">
if (inputsChanged())
this.context.requestTaskReconfiguration();
</pre>
<p>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.</p>
<p>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.</p>
<p><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 simultaneously 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.</p>
<h4><a id="connect_configs" href="#connect_configs">Connect Configuration Validation</a></h4>
<p>Kafka Connect allows you to validate connector configurations before submitting a connector to be executed and can provide feedback about errors and recommended values. To take advantage of this, connector developers need to provide an implementation of <code>config()</code> to expose the configuration definition to the framework.</p>
<p>The following code in <code>FileStreamSourceConnector</code> defines the configuration and exposes it to the framework.</p>
<pre class="brush: java;">
private static final ConfigDef CONFIG_DEF = new ConfigDef()
.define(FILE_CONFIG, Type.STRING, Importance.HIGH, "Source filename.")
.define(TOPIC_CONFIG, Type.STRING, Importance.HIGH, "The topic to publish data to");
public ConfigDef config() {
return CONFIG_DEF;
}
</pre>
<p><code>ConfigDef</code> class is used for specifying the set of expected configurations. For each configuration, you can specify the name, the type, the default value, the documentation, the group information, the order in the group, the width of the configuration value and the name suitable for display in the UI. Plus, you can provide special validation logic used for single configuration validation by overriding the <code>Validator</code> class. Moreover, as there may be dependencies between configurations, for example, the valid values and visibility of a configuration may change according to the values of other configurations. To handle this, <code>ConfigDef</code> allows you to specify the dependents of a configuration and to provide an implementation of <code>Recommender</code> to get valid values and set visibility of a configuration given the current configuration values.</p>
<p>Also, the <code>validate()</code> method in <code>Connector</code> provides a default validation implementation which returns a list of allowed configurations together with configuration errors and recommended values for each configuration. However, it does not use the recommended values for configuration validation. You may provide an override of the default implementation for customized configuration validation, which may use the recommended values.</p>
<h4><a id="connect_schemas" href="#connect_schemas">Working with Schemas</a></h4>
<p>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.</p>
<p>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>.</p>
<p>The API documentation provides a complete reference, but here is a simple example creating a <code>Schema</code> and <code>Struct</code>:</p>
<pre class="brush: java;">
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);
</pre>
<p>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.</p>
<p>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.</p>
<p>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.</p>
<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 class="brush: json;">
{
"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>
</script>
<div class="p-connect"></div>