Randall,

Great question. Ideally you wouldn't need this type of state since it
should really be available in the source system. In your case, it might
actually make sense to be able to grab that information from the DB itself,
although that will also have issues if, for example, there have been
multiple schema changes and you can no longer get a previous schema from
the current state of the tables.

The offset storage is probably pretty close to what you're looking for,
although we obviously structure that very narrowly. Adding in some sort of
other state store is an interesting idea, though I'd be curious how many
other systems encounter similar challenges. I think one way to do this
without huge changes and in a way that properly handles offset commits
would be to expose a small API for setting local state and have Connect
store that state right in the same topic (and message) as offsets. To
handle offset commits and reviving tasks that hit a fault, we would just
grab the current state as part of the process of committing offsets. Then
offsets would just be a special case of that more general state.

However, I'm also wary of doing something like this. Right now every worker
has to consume the entire offsets topic. This works fine because offsets,
while capable of being pretty complex, are generally pretty small such that
there's no concern having to tail it on all workers (and no concern for the
load on brokers leading those partitions). Once you provide a generic state
storage mechanism without clear constraints on how it should be used,
someone will come along and abuse it. Also, with offsets it is very clear
(to the connector developer) which task should write to which keys (where
the key here is the partition of the source partitioned stream). With
support for arbitrary state, ownership of different subsets of the key
space is very unclear. I think you may not have that particular problem
because you probably only have 1 partition anyway since you are reading a
transaction log.

In any case, you're right that this doesn't exist today. There is one very
hacky way to get around this, which is to store that schema information in
your "offsets". This may not be *too* bad -- it'll increase the size of
offset data, but probably doesn't affect much else. The data size may not
be that bad as long as offsets aren't committed too frequently. In terms of
performance, I'm assuming these schema changes are relatively rare, and you
can just include the same schema object in every offset you create during
the periods between schema changes so you (and the GC) are probably only
doing a negligible amount of extra work.

Re: creating a consumer, Connect doesn't provide any utilities to do that
since the goal is to handle everything Kafka-related for the connector
developer so they can just focus on getting the data from the other system!
We could consider exposing some of the worker config though, which I
imagine is all you really need -- it'd just be convenient to have the
connection info for the Kafka brokers.

Finally, I'd love to know which DB you're reading the transaction log from
and if you're planning on open sourcing the connector:)

-Ewen

On Thu, Jan 28, 2016 at 6:12 AM, Randall Hauch <rha...@gmail.com> wrote:

> Rather than leave this thread so open ended, perhaps I can narrow down to
> what I think is the best approach. These accumulations are really just
> additional information from the source that don’t get written to the normal
> topics. Instead, each change to the accumulated state can be emitted as
> source records on a dedicated topic. That is very straightforward with the
> existing Kafka Connect.
>
> The challenge I’m struggling with is how a task can/should, upon startup,
> *consume* that stream to rebuild its state. I can set up my own Kafka
> consumer for that topic, but IIUC now my connector config has to include
> much of the same information included in the Kafka Connect workers
> configuration.
>
> Am I just missing how a connector can see the worker configuration
> properties? Or is there a way that Kafka Connect can help me create a Kafka
> consumer?
>
> Best regards,
>
> Randall Hauch
>
> On January 28, 2016 at 12:11:07 AM, Randall Hauch (rha...@gmail.com)
> wrote:
> I’m creating a custom Kafka Connect source connector, and I’m running into
> a situation for which Kafka Connect doesn’t seem to provide a solution out
> of the box. I thought I’d first post to the users list in case I’m just
> missing a feature that’s already there.
>
> My connector’s SourceTask implementation is reading a relational database
> transaction log. That log contains schema changes and row changes, and the
> row changes include a reference to the table and the row values. Thus, as
> the task processes the log, it has to use any schema changes in the log to
> adjust how it converts subsequent row changes into Kafka source records.
> Should the task stop and be restarted elsewhere, it can continue reading
> the transaction log where it left off only if that new task instance can
> recover the schema state accumulated by an earlier task.
>
> While I certainly can use a custom solution to store this state somewhere,
> it seems like other connectors might benefit from having Kafka Connect
> include something out of the box. And, this accumulated state (and its
> history with respect to the source offset at which the state changes) seems
> like a perfect fit for storing in a Kafka topic.
>
> Does Kafka Connect already have a mechanism for tasks to store and recover
> arbitrary state? If not, then is there interest in adding this capability
> to Kafka Connect? (If there is interest, then perhaps the dev list is a
> better venue.)
>
> Best regards,
>
> Randall Hauch
>



-- 
Thanks,
Ewen

Reply via email to