You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 5 Next »

Status

Current stateUnder Discussion

Discussion thread: here TODO

JIRA: Unable to render Jira issues macro, execution error.

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

Motivation

In KIP-447 we introduced a new thread-producer which is capable of exactly-once semantics across multiple tasks. The new mode of EOS, called eos-beta, is intended to eventually be the preferred processing mode for EOS as it improves the performance and scaling of partitions/tasks. The only downside is that it requires brokers to be on version 2.5+ in order to understand the latest APIs that are necessary for this thread-producer.

Eventually we would like to be able to remove the eos-alpha code paths from Streams as this will help to simplify the logic and reduce the processing mode branching. We should deprecate it now to open the door for its removal in a later version, and encourage users to upgrade their brokers and migrate away from eos-alpha.

Additionally, the current name given to the new-and-improved processing guarantee may give users the impression that it's not stable or production-ready. We should also deprecate eos-beta, although in name only, to remove the "-beta" and all connotations.

Public Interfaces

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

}
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, and the configs will remain until 4.0. At that time, both the deprecated configs and the underlying eos-alpha implementation will be removed. This will allow us to significantly clean up the task and producer management code, since there will only be 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

Existing users of eos-alpha are encouraged to upgrade their brokers in preparation for its removal in a later version. Existing eos-beta users will simply need to swap in the new config name.

Rejected Alternatives

A few options were discussed on the original ticket:

  • Keep eos-alpha around forever
    • Rejected for the reasons outlined in the Motivation section: it complicates the code path, increases maintenance cost, and provides no advantage over eos-beta.
  • Deprecate only the eos-beta config, and just change the meaning of the existing EXACTLY_ONCE config to mean "eos-beta" rather than "eos-alpha"
    • Rejected because this doesn't follow the spirit (or even the letter) of the compatibility policy in Apache Kafka
    • It's an extremely harsh user experience for anyone who hasn't yet upgraded their brokers: the app would crash after the upgrade, and there would be no way to configure it to work with older brokers since eos-alpha would be effectively gone
  • Deprecate the EXACTLY_ONCE_V2 config in 4.0, then change the meaning of EXACTLY_ONCE to correspond to eos-beta/eos-v2 and un-deprecate it instead of removing it
    • Rejected because this creates unnecessary user toil in requiring them to migrate from eos-beta to eos-v2 and then back to just eos
    • Also, eos-v2 is nice because it leaves the door open to future eos versions which may require feature flagging for compatibility reasons or due to tradeoffs which are up to the user, so we can just continue the line with eos-v3 rather than going through another weird deprecation cycle and/or re-introducing eos-v2 all over again, but with yet another different meaning.




  • No labels