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



##########
File path: 
connect/api/src/main/java/org/apache/kafka/connect/source/SourceTaskContext.java
##########
@@ -38,4 +38,30 @@
      * Get the OffsetStorageReader for this SourceTask.
      */
     OffsetStorageReader offsetStorageReader();
+
+    /**
+     * Get a {@link TransactionContext} that can be used to define producer 
transaction boundaries
+     * when exactly-once support is enabled for the connector.
+     *
+     * <p>This method was added in Apache Kafka 3.0. Source tasks that use 
this method but want to
+     * maintain backward compatibility so they can also be deployed to older 
Connect runtimes
+     * should guard the call to this method with a try-catch block, since 
calling this method will result in a
+     * {@link NoSuchMethodException} or {@link NoClassDefFoundError} when the 
source connector is deployed to
+     * Connect runtimes older than Kafka 3.0. For example:
+     * <pre>
+     *     TransactionContext transactionContext;
+     *     try {
+     *         transactionContext = context.transactionContext();
+     *     } catch (NoSuchMethodError | NoClassDefFoundError e) {
+     *         transactionContext = null;
+     *     }
+     * </pre>
+     *
+     * @return the transaction context, or null if the user does not want the 
connector to define
+     * its own transaction boundaries

Review comment:
       No objections to modifying Javadocs here instead of in a KIP (especially 
since you made that clear on the mailing list). It's easier to review these 
sorts of details in a PR IMO anyways.
   
   Can make the change to refer to the connector configuration. Agree that 
"user" is ambiguous.

##########
File path: 
connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java
##########
@@ -28,4 +30,31 @@
     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.
+     * The default implementation will return {@code null}.
+     * @param connectorConfig the configuration that will be used for the 
connector.
+     * @return {@link ExactlyOnceSupport#SUPPORTED} if the connector can 
provide exactly-once support,
+     * and {@link ExactlyOnceSupport#UNSUPPORTED} if it cannot. If {@code 
null}, it is assumed that the
+     * connector cannot.
+     */
+    public ExactlyOnceSupport exactlyOnceSupport(Map<String, String> 
connectorConfig) {
+        return null;
+    }
+
+    /**
+     * Signals whether the connector can define its own transaction boundaries 
with the proposed
+     * configuration. Developers must override this method if they wish to add 
connector-defined
+     * transaction boundary support; if they do not, users will be unable to 
create instances of
+     * this connector that use connector-defined transaction boundaries. The 
default implementation
+     * will return {@code UNSUPPORTED}.

Review comment:
       Can do. I think a reference to the existing `validate` method may help 
clarify things (especially since this method will be used in almost exactly the 
same way); LMK what you think.

##########
File path: 
connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java
##########
@@ -28,4 +30,31 @@
     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.
+     * The default implementation will return {@code null}.

Review comment:
       Ack, done.




-- 
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