C0urante commented on a change in pull request #11773:
URL: https://github.com/apache/kafka/pull/11773#discussion_r818829273



##########
File path: 
connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java
##########
@@ -28,4 +30,46 @@
     protected SourceConnectorContext context() {
         return (SourceConnectorContext) context;
     }
+
+    /**
+     * Signals whether the connector supports exactly-once delivery guarantees 
with a proposed configuration.
+     * Developers can assume that worker-level exactly-once support is enabled 
when this method is invoked.
+     *
+     * <p>For backwards compatibility, the default implementation will return 
{@code null}, but connector developers are
+     * strongly encouraged to override this method to return a non-null value 
such as
+     * {@link ExactlyOnceSupport#SUPPORTED SUPPORTED} or {@link 
ExactlyOnceSupport#UNSUPPORTED UNSUPPORTED}.
+     *
+     * <p>Similar to {@link #validate(Map) validate}, this method may be 
called by the runtime before the
+     * {@link #start(Map) start} method is invoked when the connector will be 
run with exactly-once support.

Review comment:
       Hmmmm... I think this is valid and could be useful but there are a few 
complications.
   
   If we want to guarantee that the config has been validated first (via 
`Connector::validate` and possibly with other checks like making sure that the 
value for `transaction.boundary` is valid), should we then only invoke this 
method if there are no errors in the connector config?
   
   This might make life easier for connector authors who want to be able to 
instantiate an `AbstractConfig` subclass for their connector and then make 
decisions about its exactly-once viability based on the already-parsed values 
from that config class instead of, like you say, having to duplicate that 
validation logic in this method.
   
   On the other hand, it'll prevent issues with exactly-once support from 
surfacing the first time around and will require a follow-up validation attempt 
to discover any, even if the problems with the connector config don't impact 
its ability to provide exactly-once guarantees.
   
   Thinking about just this specific use case (instantiating an 
`AbstractConfig` subclass inside this method), I think it'd be best to do two 
things:
   1. Add a `catch` clause specifically for `ConfigException` instances around 
the calls to `Connector::exactlyOnceSupport` and 
`Connector::canDefineTransactionBoundaries` that gets translated into a 
specific error message stating that the connector config appears to be invalid 
and exactly-once support for the connector cannot be determined (instead of the 
[existing "An unexpected error occurred..." 
message](https://github.com/C0urante/kafka/blob/c687a12af3ec6838140bedf17efae4e9e5c19df1/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java#L907)).
 This should allow connector authors to instantiate their specific 
`AbstractConfig` subclass within this method and not have to worry about 
running into something like 
[KAFKA-13327](https://issues.apache.org/jira/browse/KAFKA-13327), which causes 
500 errors to be thrown during some connector config validations when it's 
possible (and a much better UX) to add an error message to the validated config
 .
   2. Add a `try`/`catch` block around the [parsing logic for 
ConnectorTransactionBoundaries](https://github.com/C0urante/kafka/blob/69d63eda9c96bc76ce29b07c43f0e92c776fa5fb/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java#L67)
 that translates an `IllegalArgumentException` into a `ConfigException`, so 
that connector authors can freely invoke 
`ConnectorTransactionBoundaries::fromProperty` on their connector config inside 
`SourceConnector::exactlyOnceSupport` and, if there's an issue, it'll be 
translated into a helpful error message to the user. And if the connector 
transaction boundary style isn't relevant at all to its exactly-once support, 
we won't double-ding the connector config with two error messages related to 
the `transaction.boundary` property.
   
   Do you think that this is a reasonable way to handle that specific connector 
development style? And are there other development styles that we should aim to 
accommodate?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to