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 were switched on. > 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. If we had the > intermediate producer you describe at our disposal, and it were in use by > every running source task for a given connector, we could probably enable > users to toggle exactly-once on a per-connector basis, but it would also > require new ACLs for all connectors. > That will be the case for the new transactional cluster anyway. I think there is value to supporting connectors that don't use full-blown transactions in an exactly-once cluster, because the overhead in a fencing producer should be similar to an idempotent producer (which IIRC is about 3% above a non-idempotent producer). That's because we only need to make a single InitProducerIdRequest, and thereafter the epoch check is tiny. If that's right then many people would then be able to use a single cluster for both exactly once and non-exactly once connectors (i.e. it would get rid of the performance cost of running a non-EOS connector in an exactly-once cluster). Only people who cared about the ~3% would need to run "old-style" clusters using unfenced producers. Gouzhang also has a (possible) use case for a fencing-only producer ( https://issues.apache.org/jira/browse/KAFKA-12693), and as he points out there, you should be able to get these semantics today by calling initTransactions() and then just using the producer as normal (no beginTransaction()/abortTransaction()/endTransaction()). In light of that (and assuming you buy these arguments), I wonder how much extra effort it would be to do for EOS-enabled clusters as part of this KIP? Thanks again, Tom On Fri, May 14, 2021 at 2:14 AM Chris Egerton <chr...@confluent.io.invalid> wrote: > 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 configurations such as invoking "Connector::validate" and > ensuring that all of the referenced SMTs, Predicates, and Converter classes > are present on the worker. In fact, I wonder if that's how we could > implement it--as a preflight check. That way, Connector and Task instances > won't even have the chance to fail; if the user states a requirement for > exactly-once support but their connector configuration doesn't meet that > requirement, we can fail the connector creation/reconfiguration request > before even writing the new config to the config topic. We could also add > this support to the "PUT /{connectorType}/config/validate" endpoint so that > users could test exactly-once support for various configurations without > having to actually create or reconfigure a connector. We could still fail > tasks on startup if something slipped by (possibly due to connector > upgrade) but it'd make the UX a bit smoother in most cases to fail faster. > > Since a possible use of the property is to allow future users to control > exactly-once support on a per-connector basis, I wonder whether a binary > property is sufficient here. Even if a connector doesn't support > exactly-once, there could still be benefits to using a transactional > producer with rounds of zombie fencing; for example, preventing duplicate > task instances from producing data, which could be leveraged to provide > at-most-once delivery guarantees. In that case, we'd want a way to signal > to Connect that the framework should do everything it does to provide > exactly-once source support, but not make the assertion on the connector > config, and we'd end up providing three possibilities to users: required, > best-effort, and disabled. It sounds like right now what we're proposing is > that we expose only the first two and don't allow users to actually disable > exactly-once support on a per-connector basis, but want to leave room for > the third option in the future. With that in mind, "required/not_required" > might not be the best fit. Perhaps "required"/"requested" for now, with > "disabled" as the value that could be implemented later? > > RE: "Is the problem here simply that the zombie fencing provided by the > producer is only available when using transactions, and therefore having a > non-transactional producer in the cluster poses a risk of a zombie not > being fenced?"--that's half of it. 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 were switched on. If we had the > intermediate producer you describe at our disposal, and it were in use by > every running source task for a given connector, we could probably enable > users to toggle exactly-once on a per-connector basis, but it would also > require new ACLs for all connectors. Even though we're allowed to make > breaking changes with the upcoming 3.0 release, I'm not sure the tradeoff > is worth it. I suppose we could break down exactly-once support into two > separate config properties--a worker-level property, that causes all source > tasks on the worker to use producers that can be fenced (either full-on > transactional producers or "intermediate" producers), and a per-connector > property, that toggles whether the connector itself uses a full-on > transactional producer or just an intermediate producer (and whether or not > zombie fencing is performed for new task configs). This seems like it might > be overkill for now, though. > > As far as the zombie fencing endpoint goes--the behavior will be the same > either way w/r/t the exactly.once.source.enabled property. The property > will dictate whether the endpoint is used by tasks, but it'll be available > for use no matter what. This is how a rolling upgrade becomes possible; > even if the leader hasn't been upgraded yet (to set > exactly.once.source.enabled to true), it will still be capable of handling > fencing requests from workers that have already been upgraded. > > Cheers, > > Chris > > On Wed, May 12, 2021 at 5:33 AM Tom Bentley <tbent...@redhat.com> wrote: > > > 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. Alternatively we could omit it > > from the enum and use null to mean unknown (we'd have to check for a null > > result anyway), with the contract for the method being that it should > > return non-null. Of course, this doesn't remove the ambiguous case, but > > avoids the need to eventually remove UNKNOWN in the future. > > > > I think there's another way for a worker to use the value too: Imagine > > you're deploying a connector that you need to be exactly once. It's > awkward > > to have to query the REST API to determine that exactly once was working, > > especially if you need to do this after config changes too. What you > > actually want is to make an EOS assertion, via a connector config (e.g. > > require.exactly.once=true, or perhaps > exactly.once=required/not_required), > > which would fail the connector/task if exactly once could not be > provided. > > > > The not_required case wouldn't disable the transactional runtime > > environment, simply not guarantee that it was providing EOS. Although it > > would leave the door open to supporting mixed EOS/non-transactional > > deployments in the cluster in the future, if that became possible (i.e. > we > > could retrospectively make not_required mean no transactions). > > > > On the subject of why it's not possible to enabled exactly once on a > > per-connector basis: Is the problem here simply that the zombie fencing > > provided by the producer is only available when using transactions, and > > therefore having a non-transactional producer in the cluster poses a risk > > of a zombie not being fenced? This makes me wonder whether there's a case > > for a producer with zombie fencing that is not transactional > (intermediate > > between idempotent and transactional producer). IIUC this would need to > > make a InitProducerId request and use the PID in produce requests, but > > could dispense with the other transactional RPCs. If such a thing existed > > would the zombie fencing it provided be sufficient to provide safe > > semantics for running a non-EOS connector in an EOS-capable cluster? > > > > The endpoint for zombie fencing: It's not described how this works when > > exactly.once.source.enabled=false > > > > Cheers, > > > > Tom > > >