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.
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<Object, Object> {
String filename;
InputStream stream;
String topic;
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 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 sourcePartition = Collections.singletonMap("filename", filename);
Map 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 and a key.
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.
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) { ... }
public abstract void put(Collection<SinkRecord> records);
public abstract void flush(Map<TopicPartition, Long> offsets);
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 and the event key and value.
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.
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.
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.
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", new SchemaBuilder.boolean().defaultValue(false).build())
.build();
Struct struct = new Struct(schema)
.put("name", "Barbara Liskov")
.put("age", 75)
.build();
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.