Ewen,

Thanks for the reply.  We'll proceed while keeping all of your points in
mind.  I looked around for a more focused forum for the jdbc connector
before posting here but didn't come across the confluent-platform group.
I'll direct any more questions about the jdbc connector there.  I'll also
close the github issue with a link to this thread.

Thanks again,
Mark.

On Wed, Dec 16, 2015 at 9:51 PM Ewen Cheslack-Postava <e...@confluent.io>
wrote:

> Mark,
>
> There are definitely limitations to using JDBC for change data capture.
> Using a database-specific implementation, especially if you can read
> directly off the database's log, will be able to handle more situations
> like this. Cases like the one you describe are difficult to address
> efficiently working only with simple queries.
>
> The JDBC connector offers a few different modes for handling incremental
> queries. One of them uses both a timestamp and a unique ID, which will be
> more robust to issues like these. However, even with both, you can still
> come up with variants that can cause issues like the one you describe. You
> also have the option of using a custom query which might help if you can do
> something smarter by making assumptions about your table, but for now
> that's pretty limited for constructing incremental queries since the
> connector doesn't provide a way to track offset columns with custom
> queries. I'd like to improve the support for this in the future, but at
> some point it starts making sense to look at database-specific connectors.
>
> (By the way, this gets even messier once you start thinking about the
> variety of different isolation levels people may be using...)
>
> -Ewen
>
> P.S. Where to ask these questions is a bit confusing since Connect is part
> of Kafka. In general, for specific connectors I'd suggest asking on the
> corresponding mailing list for the project, which in the case of the JDBC
> connector would be the Confluent Platform mailing list here:
> https://groups.google.com/forum/#!forum/confluent-platform
>
> On Wed, Dec 16, 2015 at 5:27 AM, Mark Drago <markdr...@gmail.com> wrote:
>
> > I had asked this in a github issue but I'm reposting here to try and get
> an
> > answer from a wider audience.
> >
> > Has any thought gone into how kafka-connect-jdbc will be impacted by SQL
> > transactions committing IDs and timestamps out-of-order?  Let me give an
> > example with two connections.
> >
> > 1: begin transaction
> > 1: insert (get id 1)
> > 2: begin transaction
> > 2: insert (get id 2)
> > 2: commit (recording id 2)
> > kafka-connect-jdbc runs and thinks it has handled everything through id 2
> > 1: commit (recording id 1)
> >
> > This would result in kafka-connect-jdbc missing id 1. The same thing
> could
> > happen with timestamps. I've read through some of the kafka-connect-jdbc
> > code and I think it may be susceptible to this problem, but I haven't run
> > it or verified that it would be an issue. Has this come up before? Are
> > there plans to deal with this situation?
> >
> > Obviously something like bottled-water for postgresql would handle this
> > nicely as it would get the changes once they're committed.
> >
> >
> > Thanks for any insight,
> >
> > Mark.
> >
> >
> > Original github issue:
> > https://github.com/confluentinc/kafka-connect-jdbc/issues/27
> >
>
>
>
> --
> Thanks,
> Ewen
>

Reply via email to