Hi Randall,
That's a fair assessment; if a user upgrades their cluster to 3.0 with no
changes to worker or connector configs, it's possible that their cluster
will break if their worker principal(s) lack the necessary ACLs on the
Kafka cluster that hosts the config topic.
If we wanted to take a
Chris,
Sorry for the late question/comment. But the "Breaking Changes" concerns
me. IIUC, when a user upgrades their 1.x or 2.x Connect cluster, then when
they restart their 3.0 worker(s) the workers will fail due to this producer
requirement even if they make no changes to their worker configs
Hi Tom,
I do agree that it'd be safer to default to "required", but since at the
time of the 3.0 release no existing connectors will have implemented the
"SourceConnector::exactlyOnceSupport" method, it'd require all Connect
users to downgrade to "requested" anyways in order to enable
Hi Chris,
Just a minor question: I can see why the default for exactly.once.support
is requested (you want a good first-run experience, I assume), but it's a
little like engineering a safety catch and then not enabling it. Wouldn't
it be safer to default to required, so that there's no way
Hi Gunnar,
Thanks for taking a look! I've addressed the low-hanging fruit in the KIP;
responses to other comments inline here:
> * TransactionContext: What's the use case for the methods accepting a
source record (commitTransaction(SourceRecord
record), abortTransaction(SourceRecord record))?
Chris,
One follow-up question after thinking some more about this; is there any
limit in terms of duration or size of in-flight, connector-controlled
transactions? In case of Debezium for instance, there may be cases where we
tail the TX log from an upstream source database, not knowing whether
Chris, all,
I've just read KIP-618, and let me congratulate you first of all for this
impressive piece of work! Here's a few small suggestions and questions I
had while reading:
* TransactionContext: What's the use case for the methods accepting a
source record (commitTransaction(SourceRecord
Hi all,
Wanted to note here that I've updated the KIP document to include the
changes discussed recently. They're mostly located in the "Public
Interfaces" section. I suspect discussion hasn't concluded yet and there
will probably be a few more changes to come, but wanted to take the
opportunity
Hi Tom,
Wow, I was way off base! I was thinking that the intent of the fencible
producer was to employ it by default with 3.0, as opposed to only after the
worker-level
"exactly.once.source.enabled" property was flipped on. You are correct that
with the case you were actually describing, there
Hi Chris,
Thanks for continuing to entertain some of these ideas.
On Fri, May 14, 2021 at 5:06 PM Chris Egerton
wrote:
> [...]
>
That's true, but we do go from three static ACLs (write/describe on a fixed
> transactional ID, and idempotent write on a fixed cluster) to a dynamic
> collection of
Hey Jeremy,
Thanks for taking a look! Always nice to have input from connector
developers, especially ones as prolific as you.
I was hoping to leave connector-defined transaction boundaries for future
work as the use cases for them were unclear. For example, with transactions
in an upstream data
Hey Chris!
Nice work on this KIP!
What are the thoughts about letting the connector developer control
the boundaries of the transaction? For example kafka-connect-spooldir
is used to parse and import files to kafka. It would be amazing if I
could begin and end the transaction as I open and close
Hi Randall,
Thanks for clarifying the issues with large transactions. I think we're
starting to converge on an understanding of the problem space, if not quite
on an approach to tackle it.
I think my biggest issue with defining transaction boundaries on a
per-task-poll basis are these points
On Mon, May 10, 2021 at 10:26 AM Chris Egerton
wrote:
> RE transaction boundaries:
>
> > First, the offset commit interval is currently 60 seconds by default, and
> source connectors can produce a lot of records in that time. Until someone
> operating a Connect cluster changed that to a much
Hi Tom,
Really interesting turn this has taken! Responses inline.
> I'm not quite sure I follow what you mean here. Can you explain? AFAICT it
doesn't apply in a cluster where all tasks are using fencible producers,
but maybe I'm missing something.
Imagine this (unlikely but possible) scenario:
Hi Chris,
Thanks for the reply.
"required"/"requested" sounds good to me. Likewise the pre-flight check and
"PUT /{connectorType}/config/validate".
The other half is we'd still need to
> track the number of tasks for that connector that would need to be fenced
> out if/when exactly-once for it
Hi Tom,
I'm fine with an implicit mapping of connector-provided null to
user-exposed UNKNOWN, if the design continues down that overall path.
Allowing users to assert that a connector should support exactly-once
sounds reasonable; it's similar to the pre-flight checks we already do for
connector
Hi Chris and Randall,
I can see that for connectors where exactly once is configuration-dependent
it makes sense to use a default method. The problem with having an explicit
UNKNOWN case is we really want connector developers to _not_ use it. That
could mean it's deprecated from the start.
Thanks for your thoughts, Randall. Responses below:
RE exactly-once support in the REST API:
> The worker has to do (or plan to do) something with the information about
a
connector's support for EOS, whether that's via an annotation or a method.
Otherwise, what's the point of requiring the
Thanks once again for the KIP and the updates, Chris. As mentioned in my
prior email, here are my more general comments/questions (other than my
earlier question about the transaction boundaries). Hopefully these are
readable:
The worker-level offset.flush.timeout.ms property will be ignored for
Thanks again for the continued improvements to this KIP, Chris. I have a
number of questions that I'll enumerate in a subsequent email, but first I
wanted to make a higher-level comment.
I don't know what to make about the proposal to rely upon the existing
offset commit interval as the way to
Thanks for continuing to work on this KIP, Chris.
On Fri, May 7, 2021 at 12:58 PM Chris Egerton
wrote:
> b) An annotation is a really cool way to allow connector developers to
> signal eligibility for exactly-once to Connect (and possibly, through
> Connect, to users). Like you mention,
Hi Tom,
Thanks for taking a look! Really appreciate it. Answers below:
1.
a) This is possible, but it would likely require reworking the rebalance
protocol, and would not provide comprehensive guarantees about exactly-once
delivery. If the group coordination protocol used for rebalancing is also
Hi Chris,
Thanks for this KIP. I've taken an initial look and have a few questions.
1. The doc for exactly.once.source.enabled says "Note that this must be
enabled on every worker in a cluster in order for exactly-once delivery to
be guaranteed, and that some source connectors may still not be
Hi all,
Good news everyone! I've reworked the design one more time and hopefully
some of the improvements here should make the proposal more palatable.
TL;DR:
- Rolling upgrades are now possible, in at most two phases; workers will
first have to be given a binary upgrade to 3.0 (the targeted
Hi Randall,
After thinking things over carefully, I've done some reworking of the
design. Instead of performing zombie fencing during rebalance, the leader
will expose an internal REST endpoint that will allow workers to request a
round of zombie fencing on demand, at any time. Workers will then
Hi Randall,
No complaints about email size from me. Let's dive in!
1. Sure! Especially important in my mind is that this is already possible
with Connect as it is today, and users can benefit from this with or
without the expanded exactly-once souce support we're trying to add with
this KIP.
Hi Randall,
Thanks for taking a look! Responses inline:
> This would seem to make the transactions based upon time rather than
record
count. Is that really the intent? Did you consider using one transaction
per "batch" of records, whatever number of records that means? One issue
with the
Hey, Chris.
Thanks again for all your hard work figuring out this complicated problem.
While there are still issues to iron out, at the moment it seems pretty
sensible.
Here's my follow up email with additional comments/questions. I've
organized these by section to help provide context, and I've
Chris,
Thanks for the well-written KIP and the more recent updates. It's exciting
to envision EOS for Connect source connectors!
I'll follow up this email with another that goes into specific comments and
questions not already touched on by others. This email addresses comments
and questions
This is a great explanation, thank you!
On Mon, Feb 22, 2021 at 2:44 PM Chris Egerton wrote:
> Hi Guozhang,
>
> Your understanding should be correct in most cases, but there are two finer
> points that may interest you:
>
> 1. It's technically dependent on the implementation of the connector
Hi Guozhang,
Your understanding should be correct in most cases, but there are two finer
points that may interest you:
1. It's technically dependent on the implementation of the connector and
how it chooses to allocate source partitions across its tasks; even if the
number of tasks and source
Thanks Chris, yeah I think I agree with you that this does not necessarily
have to be in the scope of this KIP.
My understanding was that the source partitions -> tasks are not static but
dynamic, but they are only changed when either the number of partitions
changed or "tasks.max" config changed
Hi Guozhang,
Thanks for taking a look, and for your suggestion!
I think there is room for more intelligent fencing strategies, but I think
that it'd have to be more nuanced than one based on task->worker
assignments. Connectors can arbitrarily reassign source partitions (such as
database tables,
Hi Gwen,
Thanks for the feedback!
0.
That's a great point; I've updated the motivation section with that
rationale.
1.
This enables safe "hard downgrades" of clusters where, instead of just
disabling exactly-once support on each worker, each worker is rolled back
to an earlier version of the
Hello Chris,
Thanks for the great write-up! I was mainly reviewing the admin
fenceProducers API of the KIP. I think it makes sense overall. I'm just
wondering if we can go one step further, that instead of forcing to fence
out all producers of the previous generation, could we try to achieve
Hey Chris,
Thank you for the proposal. Few questions / comments:
0. It may be worth pointing out in the motivation section that
source-connector exactly once is more important than sink connector
exactly once, since many target systems will have unique key
constraint mechanisms that will prevent
Hi Jason,
Fine by me; I wanted to be conservative with the return type but the case
you've outlined sounds enticing enough that adding a little flexibility to
the API seems warranted. I've added your suggestion to the proposed admin
API expansions; let me know what you think.
Cheers,
Chris
On
Hi Chris,
If we add the new `fenceProducers` admin API, can we return the information
from the `InitProducerId` response (i.e. producer id and epoch)? We may not
have a use case for it yet, but I don't see any harm exposing it for the
future. For example, we could allow this state to be provided
Hi Ning,
Apologies for the delay in response. I realized after publishing the KIP
that there were some finer points I hadn't considered in my design and that
it was far from providing exactly-once guarantees. In response to your
questions:
1) The goal of the KIP is to ensure the accuracy of the
Hello Chris,
That is an interesting KIP. I have a couple of questions:
(1) in section of pseudo-code, what if the failure happens between 4(b) and
5(a), meaning after the producer commit the transaction, and before
task.commitRecord().
(2) in section "source task life time", what is the
Hi all,
I know it's a busy time with the upcoming 2.6 release and I don't expect
this to get a lot of traction until that's done, but I've published a KIP
for allowing atomic commit of offsets and records for source connectors and
would appreciate your feedback:
42 matches
Mail list logo