Indeed, implying that the flushing and acknowledgement of records happens
in order reveals an implementation detail that is not required by the
interface. Strictly speaking if that was required then you'd only need a
single record as an argument to offstesFlushedAndAcked to indicate up to
which record from the list of polled records the task managed to flush and
ack. But this wouldn't be elegant nor is certain that the task is aware at
the point when offstesFlushedAndAcked is called what was the order of the
records that it returned with poll.

I don't feel strong about it and I acknowledge the symmetry with poll. I
made this comment since we are passing a complete list of flushed records
and retaining the order seemed to restrict future implementations. But
probably makes things simpler for source connectors too.

Regarding the text sections I was mainly referring to importing information
from the jira discussion to the KIP. And my intention is the same: that
people will understand the improvement and the motivation just by reading
the KIP, without having to go over the code changes or the jira comments.
I'll let you to it and let's see where we are with implementation later in
the current KIP cycle. I find it a useful improvement, I hope it makes it
in.

-Konstantine



On Wed, Oct 17, 2018 at 5:55 AM Per Steffensen <perst...@gmail.com> wrote:

> The fix will definitely "facilitate" the source-connectors I have
> written. It will make them work 100% correctly. Today they dont.
>
> Fine for me to change "Acknowledged" to "Acked" in the method-naming.
>
> Not sure I would like to give a Collection instead of a List as the
> argument to offstesFlushedAndAck(nowledg)ed. poll() returns a List
> (ordered records), the records are handled in that order and I would
> like to hand the records back in that order as well. Handling back a
> Collection may indicate that order does not matter. Besides that it is
> likely to help the implementation of offstesFlushedAndAck(nowledg)ed
> that you get records back in order.
>
> Regarding adding stuff to the "rejected approaches" and "motivation"
> sections of the KIP, I am not sure I will get the time anytime soon.
> Please feel free to help adding this to the KIP. This way we also have
> at least two persons who really understands what this is about. Some
> times you only really understand what something is about, when you are
> forced to write about it (at least that is my excuse ).
>
> Regards, Per Steffensen
>
> On 16/10/2018 05.57, Konstantine Karantasis wrote:
> > This is a significant improvement to the semantics of source connectors.
> > I'm expecting that it will facilitate source connector implementations
> and
> > even enrich the application uses cases that we see. I only have a few
> minor
> > suggestions at the moment.
> >
> > I believe that Acked is a common abbreviation for Acknowledged and that
> we
> > could use it in this context. And this suggestion is coming from a big
> > proponent of complete words in variables and method names. Thus, feel
> free
> > to consider 'offsetsFlushedAndAcked' as well as 'recordSentAndAcked'.
> Since
> > this is a public interface, I'd also make the implementation specific
> > comment that a Collection<SourceRecord> might be more versatile than
> > List<SourceRecord> as argument in offsetsFlushedAndAcknowledged.
> >
> > The rejected approaches section could use some of the material in the
> > original jira ticket, which is pretty insightful in order to understand
> how
> > we arrived to this KIP. For example, I think it'd be useful to state
> > briefly why the 'commit' method is not getting removed completely but
> it's
> > substituted with 'offsetsFlushedAndAcked'. Also useful I believe it would
> > be to include in the motivation section some info related to why and how
> a
> > source system could use these method calls to safely recycle data that
> have
> > been surely imported to Kafka. I see this type of use cases having an
> > increased importance as Kafka is used more and more as the source of
> truth
> > and persistence layer for an increasing number of applications.
> >
> > These suggestions, although they are not strictly required in order to
> move
> > forward with this improvement, I believe can help a lot to understand the
> > context of this proposed changed, without having to read the complete
> > history in the jira ticket.
> >
> > Thanks for the KIP Per!
> >
> > -Konstantine
> >
> >
> > On Wed, Oct 10, 2018 at 6:50 AM Per Steffensen <perst...@gmail.com>
> wrote:
> >
> >> Please help make the proposed changes in KIP-381 become reality. Please
> >> comment.
> >>
> >> KIP:
> >>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-381%3A+Connect%3A+Tell+about+records+that+had+their+offsets+flushed+in+callback
> >>
> >> JIRA: https://issues.apache.org/jira/browse/KAFKA-5716
> >>
> >> PR: https://github.com/apache/kafka/pull/3872
> >>
> >> Thanks!
> >>
> >>
> >>
>
>

Reply via email to