Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

Table of Contents

Status

Current state: Under Discussion ADOPTED (in 3.0)

Discussion thread: here TODO 

JIRA:

Jira
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyKAFKA-12574

Jira
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyKAFKA-12689

Jira
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyKAFKA-12690


Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

...

Code Block
languagejava
public class StreamsConfig {

    /**
     * @deprecated Since 3.0.0, will be removed in 4.0. Use {@link #EXACTLY_ONCE_V2 "exactly_once_v2"} instead.
     */
    @Deprecated 
    public static final String EXACTLY_ONCE = "exactly_once"; 

    /**
     * @deprecated Since 3.0.0, will be removed in 4.0. Use {@link #EXACTLY_ONCE_V2 "exactly_once_v2"} instead.
     */
    @Deprecated
    public static final String EXACTLY_ONCE_BETA = "exactly_once_beta"; 

    /**
     * Config value for parameter {@link #PROCESSING_GUARANTEE_CONFIG "processing.guarantee"} for exactly-once processing guarantees.
     * <p>
     * Enabling exactly-once-v2 requires broker version 2.5 or higher.
     * If you enable this feature Kafka Streams will use fewer resources (like broker connections)
     * compared to the {@link #EXACTLY_ONCE} case. This config corresponds
     * to the old {@link #EXACTLY_ONCE_BETA} processing mode, which has been deprecated 
     * in favor of this config.
     */
    public static final String EXACTLY_ONCE_V2 = "exactly_once_v2";

}


Code Block
languagejava
class Producer {

    /**
     * @deprecated Since 3.0.0, will be removed in 4.0. Use {@link #sendOffsetsToTransaction(Map, ConsumerGroupMetadata)} instead.
     */
    @Deprecated
    public void sendOffsetsToTransaction(Map<TopicPartition, OffsetAndMetadata> offsets,
                                         String consumerGroupId);
}

Proposed Changes

We plan to deprecate both existing EOS configs in 3.0, and add a new config which corresponds to the existing "eos-beta" option. What we now call "eos-beta" will become "eos-v2", in code, comments, configs, and docs. No changes to the actual eos implementation will be made at this time – when time, and the configs will remain until 4.0. At that time, both the deprecated configs and the underlying eos-alpha is finally removed (likely in 4.0) then we can implementation will be removed. This will allow us to significantly clean up the task and producer management code, since it there will only be all thread-level producers remaining.

Since some users may skip the StreamsConfig and set the string value of "exactly_once" or "exactly_once_beta" directly, we will also log a warning if those configs are detected to raise the visibility for users who may have missed the deprecation.

In addition, we plan to deprecate the old Producer#sendOffsetsToTransaction overload which accepts a String consumerGroupId parameter, in favor of the new overload which accepts a ConsumerGroupMetadata. The deprecated method currently just invokes the new method anyway, so this should not have any impact on users (ie, both methods apply to the same cases)

Compatibility, Deprecation, and Migration Plan

...