8.3 Connector Development Guide
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.
Core Concepts and APIs
Connectors and Tasks
To copy data between Kafka and another system, users create a Connector
for the system they want to pull data from or push data to. Connectors come in two flavors: SourceConnectors
import data from another system (e.g. JDBCSourceConnector
would import a relational database into Kafka) and SinkConnectors
export data (e.g. HDFSSinkConnector
would export the contents of a Kafka topic to an HDFS file).
Connectors
do not perform any data copying themselves: their configuration describes the data to be copied, and the Connector
is responsible for breaking that job into a set of Tasks
that can be distributed to workers. These Tasks
also come in two corresponding flavors: SourceTask
and SinkTask
.
With an assignment in hand, each Task
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.
Streams and Records
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.
Dynamic Connectors
Not all jobs are static, so Connector
implementations are also responsible for monitoring the external system for any changes that might require reconfiguration. For example, in the JDBCSourceConnector
example, the Connector
might assign a set of tables to each Task
. When a new table is created, it must discover this so it can assign the new table to one of the Tasks
by updating its configuration. When it notices a change that requires reconfiguration (or a change in the number of Tasks
), it notifies the framework and the framework updates any corresponding Tasks
.
Developing a Simple Connector
Developing a connector only requires implementing two interfaces, the Connector
and Task
. A simple example is included with the source code for Kafka in the file
package. This connector is meant for use in standalone mode and has implementations of a SourceConnector
/SourceTask
to read each line of a file and emit it as a record and a SinkConnector
/SinkTask
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.
Connector Example
We’ll cover the SourceConnector
as a simple example. SinkConnector
implementations are very similar. Start by creating the class that inherits from SourceConnector
and add a couple of fields that will store parsed configuration information (the filename to read from and the topic to send data to):
public class FileStreamSourceConnector extends SourceConnector {
private String filename;
private String topic;
The easiest method to fill in is taskClass()
, which defines the class that should be instantiated in worker processes to actually read the data:
@Override
public Class<? extends Task> taskClass() {
return FileStreamSourceTask.class;
}
We will define the FileStreamSourceTask
class below. Next, we add some standard lifecycle methods, start()
and stop()
:
@Override
public void start(Map<String, String> 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.
}
Finally, the real core of the implementation is in taskConfigs()
. In this case we are only handling a single file, so even though we may be permitted to generate more tasks as per the maxTasks
argument, we return a list with only one entry:
@Override
public List<Map<String, String>> taskConfigs(int maxTasks) {
ArrayList<Map<String, String>> configs = new ArrayList<>();
// Only one input stream makes sense.
Map<String, String> config = new HashMap<>();
if (filename != null)
config.put(FILE_CONFIG, filename);
config.put(TOPIC_CONFIG, topic);
configs.add(config);
return configs;
}
Although not used in the example, SourceTask
also provides two APIs to commit offsets in the source system: commit
and commitRecord
. 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 commit
API stores the offsets in the source system, up to the offsets that have been returned by poll
. The implementation of this API should block until the commit is complete. The commitRecord
API saves the offset in the source system for each SourceRecord
after it is written to Kafka. As Kafka Connect will record offsets automatically, SourceTask
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.
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 ConnectorUtils
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.
Task Example - Source Task
Next we’ll describe the implementation of the corresponding SourceTask
. 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 Task
class. It also has some standard lifecycle methods:
public class FileStreamSourceTask extends SourceTask {
String filename;
InputStream stream;
String topic;
@Override
public void start(Map<String, String> props) {
filename = props.get(FileStreamSourceConnector.FILE_CONFIG);
stream = openOrThrowError(filename);
topic = props.get(FileStreamSourceConnector.TOPIC_CONFIG);
}
@Override
public synchronized void stop() {
stream.close();
}
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 start()
method does not yet handle resuming from a previous offset, which will be addressed in a later section. Second, the stop()
method is synchronized. This will be necessary because SourceTasks
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 poll()
method which gets events from the input system and returns a List<SourceRecord>
:
@Override
public List<SourceRecord> poll() throws InterruptedException {
try {
ArrayList<SourceRecord> records = new ArrayList<>();
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;
}
Again, we’ve omitted some details, but we can see the important steps: the poll()
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 SourceRecord
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 SourceRecord
constructor can also include a specific output partition, a key, and headers.
Note that this implementation uses the normal Java InputStream
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 poll()
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.
Sink Tasks
The previous section described how to implement a simple SourceTask
. Unlike SourceConnector
and SinkConnector
, SourceTask
and SinkTask
have very different interfaces because SourceTask
uses a pull interface and SinkTask
uses a push interface. Both share the common lifecycle methods, but the SinkTask
interface is quite different:
public abstract class SinkTask implements Task {
public void initialize(SinkTaskContext context) {
this.context = context;
}
public abstract void put(Collection<SinkRecord> records);
public void flush(Map<TopicPartition, OffsetAndMetadata> currentOffsets) {
}
The SinkTask
documentation contains full details, but this interface is nearly as simple as the SourceTask
. The put()
method should contain most of the implementation, accepting sets of SinkRecords
, 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 SinkRecords
contain essentially the same information as SourceRecords
: Kafka topic, partition, offset, the event key and value, and optional headers.
The flush()
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 offsets
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 flush()
operation atomically commits the data and offsets to a final location in HDFS.
Errant Record Reporter
When error reporting is enabled for a connector, the connector can use an ErrantRecordReporter
to report problems with individual records sent to a sink connector. The following example shows how a connector’s SinkTask
subclass might obtain and use the ErrantRecordReporter
, safely handling a null reporter when the DLQ is not enabled or when the connector is installed in an older Connect runtime that doesn’t have this reporter feature:
private ErrantRecordReporter reporter;
@Override
public void start(Map<String, String> props) {
...
try {
reporter = context.errantRecordReporter(); // may be null if DLQ not enabled
} catch (NoSuchMethodException | NoClassDefFoundError e) {
// Will occur in Connect runtimes earlier than 2.6
reporter = null;
}
}
@Override
public void put(Collection<SinkRecord> records) {
for (SinkRecord record: records) {
try {
// attempt to process and send record to data sink
process(record);
} catch(Exception e) {
if (reporter != null) {
// Send errant record to error reporter
reporter.report(record, e);
} else {
// There's no error reporter, so fail
throw new ConnectException("Failed on record", e);
}
}
}
}
Resuming from Previous Offsets
The SourceTask
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 SourceContext
passed into its initialize()
method to access the offset data. In initialize()
, we would add a bit more code to read the offset (if it exists) and seek to that position:
stream = new FileInputStream(filename);
Map<String, Object> offset = context.offsetStorageReader().offset(Collections.singletonMap(FILENAME_FIELD, filename));
if (offset != null) {
Long lastRecordedOffset = (Long) offset.get("position");
if (lastRecordedOffset != null)
seekToOffset(stream, lastRecordedOffset);
}
Of course, you might need to read many keys for each of the input streams. The OffsetStorageReader
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.
Dynamic Input/Output Streams
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 ConnectorContext
object that reconfiguration is necessary. For example, in a SourceConnector
:
if (inputsChanged())
this.context.requestTaskReconfiguration();
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 SourceConnector
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 Connector
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 Task
encounters the issue before the Connector
, which will be common if the Connector
needs to poll for changes, the Task
will need to handle the subsequent error. Thankfully, this can usually be handled simply by catching and handling the appropriate exception.
SinkConnectors
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. SinkTasks
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 SinkTasks
seeing a new input stream for the first time and simultaneously trying to create the new resource. SinkConnectors
, on the other hand, will generally require no special code for handling a dynamic set of streams.
Connect Configuration Validation
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 config()
to expose the configuration definition to the framework.
The following code in FileStreamSourceConnector
defines the configuration and exposes it to the framework.
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;
}
ConfigDef
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 Validator
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, ConfigDef
allows you to specify the dependents of a configuration and to provide an implementation of Recommender
to get valid values and set visibility of a configuration given the current configuration values.
Also, the validate()
method in Connector
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.
Working with Schemas
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 data
API. Most structured records will need to interact with two classes in addition to primitive types: Schema
and Struct
.
The API documentation provides a complete reference, but here is a simple example creating a Schema
and Struct
:
Schema schema = SchemaBuilder.struct().name(NAME)
.field("name", Schema.STRING_SCHEMA)
.field("age", Schema.INT_SCHEMA)
.field("admin", SchemaBuilder.bool().defaultValue(false).build())
.build();
Struct struct = new Struct(schema)
.put("name", "Barbara Liskov")
.put("age", 75);
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 ALTER TABLE
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.
Kafka Connect Administration
Kafka Connect’s REST layer 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).
When a connector is first submitted to the cluster, a rebalance is triggered between the Connect workers in order to distribute the load that consists of the tasks of the new connector. This same rebalancing procedure is also used when connectors increase or decrease the number of tasks they require, when a connector’s configuration is changed, or when a worker is added or removed from the group as part of an intentional upgrade of the Connect cluster or due to a failure.
In versions prior to 2.3.0, the Connect workers would rebalance the full set of connectors and their tasks in the cluster as a simple way to make sure that each worker has approximately the same amount of work. This behavior can be still enabled by setting connect.protocol=eager
.
Starting with 2.3.0, Kafka Connect is using by default a protocol that performs incremental cooperative rebalancing that incrementally balances the connectors and tasks across the Connect workers, affecting only tasks that are new, to be removed, or need to move from one worker to another. Other tasks are not stopped and restarted during the rebalance, as they would have been with the old protocol.
If a Connect worker leaves the group, intentionally or due to a failure, Connect waits for scheduled.rebalance.max.delay.ms
before triggering a rebalance. This delay defaults to five minutes (300000ms
) to tolerate failures or upgrades of workers without immediately redistributing the load of a departing worker. If this worker returns within the configured delay, it gets its previously assigned tasks in full. However, this means that the tasks will remain unassigned until the time specified by scheduled.rebalance.max.delay.ms
elapses. If a worker does not return within that time limit, Connect will reassign those tasks among the remaining workers in the Connect cluster.
The new Connect protocol is enabled when all the workers that form the Connect cluster are configured with connect.protocol=compatible
, which is also the default value when this property is missing. Therefore, upgrading to the new Connect protocol happens automatically when all the workers upgrade to 2.3.0. A rolling upgrade of the Connect cluster will activate incremental cooperative rebalancing when the last worker joins on version 2.3.0.
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, the GET /connectors/file-source/status
request shows the status of a connector named file-source
:
{
"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"
}
]
}
Connectors and their tasks publish status updates to a shared topic (configured with status.storage.topic
) 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:
- UNASSIGNED: The connector/task has not yet been assigned to a worker.
- RUNNING: The connector/task is running.
- PAUSED: The connector/task has been administratively paused.
- FAILED: The connector/task has failed (usually by raising an exception, which is reported in the status output).
- DESTROYED: The connector/task has been administratively removed and will stop appearing in the Connect cluster.
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.
Starting with 2.5.0, Kafka Connect uses the status.storage.topic
to also store information related to the topics that each connector is using. Connect Workers use these per-connector topic status updates to respond to requests to the REST endpoint GET /connectors/{name}/topics
by returning the set of topic names that a connector is using. A request to the REST endpoint PUT /connectors/{name}/topics/reset
resets the set of active topics for a connector and allows a new set to be populated, based on the connector’s latest pattern of topic usage. Upon connector deletion, the set of the connector’s active topics is also deleted. Topic tracking is enabled by default but can be disabled by setting topic.tracking.enable=false
. If you want to disallow requests to reset the active topics of connectors during runtime, set the Worker property topic.tracking.allow.reset=false
.
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.