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

Compare with Current View Page History

« Previous Version 6 Next »

Status

Current stateUnder Discussion

Discussion thread: TBD 

JIRA: KAFKA-8770 

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

Motivation

Currently, in Kafka Streams, we support two modes of processing:

  • emit on update
  • emit on window close

The first one has been present for the majority of Kafka Stream's inception, when results are forwarded regardless . The second option has been added by implementing the so-called "suppression" operator which collects results throughout the window, and extending into the grace period. However, one mode of processing should've been implemented long ago, and that is emit on change. As mentioned in the JIRA, in many cases, the user would process a prior result to yield a new one, yet, this new result is forwarded to a downstream processor even though the operation performed had been effectively a no-op. This is problematic in certain circumstances where there is a high number of such operations, leading to a number of useless results being forwarded down the topology.

Therefore, with this KIP, we should consider a few things:

  1. We should decide whether or not emit on change should be considered Kafka's default behavior (i.e. we completely forgo the current emit on update design and migrate to emit on change).
  2. The second option would be to add configuration to disable emit on change, but this could potentially add undesired complexity to our current API. We will need to add this config if it has been determined by a performance benchmark that emit on change severely impacts processing throughput / latency (the main latency incurred here would probably be loading prior results, if we are going to follow through with this approach).

Proposed Behavior Changes

With the current default model of emission, we are forwarding the processed results downstream regardless if  it has changed or not. In the future, this KIP aims to remove noop results from making it further down the Kafka Streams topology. There is no other major change that would result in from this proposal.

Implementation

This section is to discuss some points on how we should approach the implementation of this KIP. Some extra configurations will probably result from the implementation, but that is still debatable. See below for possible additions. 

Details on Core Improvement

Emit on change, in many aspects, is basically an improved version of emit on update, except the crucial improvement is that we check if the new result had been changed by a given operation. And for doing such a check, if we want to be thorough, load the previous result and compare it with the most recent one. However, this would not be doable in some cases without incurring extra performance hits. Therefore, rather than load the entire result, we should compare the previous result's hash code with the current result's hash code. Admittedly, there might be a small chance of collisions where the new result with changed values would still yield the same hash code. Yet, it could be safe to say that a good hash function would never let such a thing happen.

However, the main constraint for this approach, if we were to follow through, is that the hash code must reflect any changes in the individual component fields of the result. The key building block for this entire optimization is that the user provides a good hash code override of the given value type. Of course, this isn't always the case.  After all, it isn't safe to assume that the user has provided a safe hash code function for us to take advantage of. In conclusion, we can probably come up with the following choices for the user:

  1. Completely disregard the hash code and load the prior result and compare it to the new one.
  2. Compare only the hash codes of the given objects instead.

The second option quite obviously is the recommended one. While it is uncertain if we will need to give the user the ability to choose between these two options, it would probably be necessary. In the latter option, it should be emphasized that the hash function must be chosen carefully. In many cases, Kafka already loads the prior result anyway. So for instances where we already load the previous result, we will just use that instead of the hash code (if we are to implement option 2). 

Discourse on other areas

The timestamps of the results processed would also present a serious design challenge for this KIP. The requirements of emit on change would entail the following behavior:

  1. Let's say we receive record 1 with timestamp 0 which leads to a change in the result processed. 
  2. Afterwards, we receive another record with timestamp 100 which would lead to a no-op. 

It is required that the timestamp be 0. 

To resolve this situation, the current best bet is probably to load the timestamp along with the hash code / full prior result. 

Compatibility, Deprecation, and Migration Plan

If the user has logic that involves actively filtering out no-op operations, then they will be able to remove this code in a future release once this KIP has been integrated into Kafka Streams. For now, this KIP will aim to add a minimum number of configurations, and avoid adding APIs. Therefore, no major accomodations has to be made to change your current method calls. 

Rejected Alternatives

None at the moment.

  • No labels