You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 14 Next »

Status

Current state: draft [One of "Under Discussion", "Accepted", "Rejected"]

Discussion thread: here

JIRA: here

Released: [0.10.1.0]

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

Certain datasets require modifications to the messages either coming into Kafka (source) or going to another data store (sink). For example, we may require to remove user identifiable information like SSN from Kafka messages while exporting data into a Hadoop cluster for analysis.

Currently, the "data transfer" logic is locked within a single method of the Task classes, thus Kafka Connect API is inflexible for making any modifications. This is more so when a Kafka Connect jar is provided via third party where the user cannot modify the source. Custom modifications from user to user would require modifying the source of Connect API. 

Public Interfaces

For the transformers, we require to introduce an abstract class which others can extend to create their own custom Transformer. 

public abstract class Transformer<T1, T2> {
    public abstract T2 transform(T1 t1);
    public void initialize(Map<String, String> props) {}
}
 

I was thinking if we should restrict the signature of the transform(). We could pass a standard List<*Record> but that would restrict what the user would be interested in doing.
I propose they could pass around from any object to any object they want. (I see this is mostly going to be a List<> of custom objects, but we could open this up to other structures too, like a HashMap if that’s what they need and say the order is not important like for a Sink to a database system.) This includes modifying all fields in ConnectRecord, assuming that the transformer knows what it’s doing. 
It will be the user who takes care of ensuring that the objects that the transformer produces and the next one that consumes can handle the datatype, and we could provide helpers for standard transformations. We can use generics for that to signify compatibility.

Also, this would need a :

transformers=org.AbcTransformer,org2.XyzTransformer

property in the Kafka Connect config files to initialise the transformers, in the order specified in the above property. 

 

To accommodate the transformers, we can split the “functional” methods into separate parts from a user logical POV:

SourceTask’s poll() method can broken into:
  1. fetch(): Fetch the data from the source, potentially as a list of objects in the native representation of the source
  2. Optional transformer phase.
  3. push(): Convert the list of object or any other format to List<SourceRecord> which gets saved in Kafka

Code POV:
List<SourceRecord> poll() { 
   obj = fetch();
   for(transformer : transformers) 
       obj = transformer.transform(obj);
   push(obj);
}

SinkTask’s push() method can be broken down into:
  1. pull(): Transform the Collection<SinkRecord> to a native type that the destination recognises
  2. Optional transformer phase.
  3. store(): Get the transformed or the native type from pull to store into the destination

Code POV:
void put(Collection<SinkRecord> records) { 
   obj = pull(records);
   for(transformer : transformers) 
      obj = transformer.transform(obj);
   store(obj);
}

Passing the complete collection from Transformer to Transformer will allow modification across a set of messages. For example if someone wants to filter messages based on certain messages received before the current one, that would be possible easily.
Alternatively, we could use a message to message transformation.

Proposed Changes

Besides breaking the functions push() and poll() of SinkTask and SourceTask respectively, the change introduced is the optional Transformers chain that users can introduce to operate on the message when being passed from one source to another, one of them being Kafka (as consistent with the Kafka Connect API.) 

Users can define and add their own Transformers and the chain based on what modifications they would be interested in performing on the messages.

Compatibility, Deprecation, and Migration Plan

This change will break the interface for SourceTask and SinkTask. 

If we do not want to deprecate this, we could use another class to derive the API from, and do a runtime check of type and call the methods accordingly. This way neither the old code nor the newer API would break, and newer API will support Transformers. 

Test Plan

Tests could include executing Kafka Connect code with and without transformers. With transformers, the modifications that the transformers make in order of the chain should appear in the output compared to the input messages. Without transformers, there should be no change in the messages.

Rejected Alternatives

If there are alternative ways of accomplishing the same thing, what were they? The purpose of this section is to motivate why the design is the way it is and not some other way.

  • No labels