Versions Compared

Key

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

Table of Contents

Status

Current state: Accepted Adopted Accepted but temporarily reverted (see KAFKA-12508)

Discussion thread: Voting Thread 

JIRA:

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

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

...

  1. Emit on change: The objective of this KIP is to implement this reporting strategy. The primary benefit of such an emission model is that it will drop idempotent updates, so it will be superior to emit on update in that traffic can be significantly reduced in high idempotent update traffic situations.
  2. Emit on update / non-empty content: It has already been implemented by Kafka Streams. Basically, as long as the content or processed result is not empty, we will return the processed result. That could however lead to high idempotent update traffic, which is the primary motivation for adding emit on change.
  3. Emit on window close: This has been implemented some time ago with the suppress operator which allows users to combine results in a window, and then we emit only the final result of the active window. 
  4. Periodic Emission: This emission model is unique in that it is solely dependent on time to emit a given result. Essentially, every time a fixed time interval has passed, then a result will be emitted. 

...

Note: an "update" is a concept that only applies to Table operations, so the concept of an "idempotent update" also only applies to Table operations.
See https://kafka.apache.org/documentation/streams/developer-guide/dsl-api.html#streams_concepts_ktable for more information.

Given that definition, we propose for Streams to drop idempotent updates in any situation where it's possible and convenient to do so. For example, any time we already have both the prior and new results serialized, we may compare them, and drop the update if it is idempotent.

Note that under this proposal, we can implement idempotence checking in the following situations:
1. Any aggregation (for example, KGroupedStream, KGroupedTable, TimeWindowedKStream, and SessionWindowedKStream operations)

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

2. Any Materialized source KTable operation
Jira
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyKAFKA-10248

3. Repartition operations, when we need to send both prior and new results
Jira
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyKAFKA-12547

Metrics

The following metric will be added to record the number of idempotent updates dropped:

...

  • The rate option measures the average amount of idempotent records dropped in a one second interval.
  • The total option will just give a raw count of the number of records dropped. we have decided that we will forward aggregation results if and

Design Reasoning

With the current default model of emission, we are forwarding the processed results downstream regardless if  it has changed or not. After some discussion, there are a couple of points that I would like to emphasize:

  1. There has been some thoughts as to only coupling emit on change with the suppress operator. However, that approach has been abandoned in favor of a more extensive change which would impact almost all KTable operations supported by Kafka Streams. Our justification is that idempotent update changes should be suppressed even before they are forwarded downstream, as the duplication of these useless results has the potential to explode across multiple downstream nodes once it has been forwarded. The expected behavior is the following:
    1. Any operations that have resulted in a idempotent update would be discarded. Exceptions would be made for non-materialized KTables for performance reasons. Some might have concerns about a KTable constructed through aggregations. In this case, if a idempotent update result is produced, how do we determine which timestamp to use? To keep things simple, we have decided that we will forward drop aggregation results if and only and only if the timestamp and the value had not changed.
  2. About configurations. As noted, we intend that Kafka Streams be as simple as possible when it comes to configurations. In that case, this KIP proposes that instead, emit-on-change becomes the new behavior of Kafka Streams. Since emit-on-change is more efficient (idempotent updates shouldn't really be sent in the first place), there is a strong case for no additional config that allows the user to "opt-out" of this change (and remain with emit-on-update).

...