You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2022/02/22 23:18:24 UTC

[GitHub] [kafka] C0urante commented on a change in pull request #11773: KAFKA-10000: Add new source connector APIs related to exactly-once support (KIP-618)

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



##########
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.
+     *
+     * @param connectorConfig the configuration that will be used for the connector.
+     * @return {@link ExactlyOnceSupport#SUPPORTED} if the connector can provide exactly-once support with the given
+     * configuration, and {@link ExactlyOnceSupport#UNSUPPORTED} if it cannot. If this method is overridden by a
+     * connector, should not be {@code null}, but if {@code null}, it will be assumed that the connector cannot provide
+     * exactly-once guarantees.
+     * @since 3.2
+     */
+    public ExactlyOnceSupport exactlyOnceSupport(Map<String, String> connectorConfig) {
+        return null;

Review comment:
       This follows the behavior specified in the KIP, so changing it now would be a little difficult (although it's not out of the question as long as the change is warranted).
   
   However, I think this is better as-is. We default to `null` here in order to distinguish between connectors that might be able to provide exactly-once guarantees but just haven't implemented this method yet, and connectors that definitely cannot provide exactly-once guarantees. This comes in handy during preflight validation when a user has set `exactly.once.support ` to `required` in their connector config; we can give users a more informative error message if the connector doesn't return `ExactlyOnceSupport.Supported` from this method.
   
   See the preflight validation logic in [DistributedHerder](https://github.com/C0urante/kafka/blob/fbd06b01ff168380341aaa56252a0a0f0556ae4c/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java#L892-L899) that will be added in https://github.com/apache/kafka/pull/11776 for more detail.
   
   The reason that we can provide a default for `canDefineTransactionBoundaries` is that none of the existing connectors that are already built, published, and part of the Connect ecosystem have had a chance to implement the API necessary for connector-defined transaction boundaries yet, since that API hasn't been published in an official release.




-- 
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-unsubscribe@kafka.apache.org

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