Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.
Comment: Update method name per discussion

...

In tickets KAFKA-4835 and KAFKA-10844 the option for canceling the unneeded repartition is being requested. Repartition canceling is also needed for efficient usage of distinct() operator proposed in KIP-655: groupBy(...).windowedBy(...).distinct() will always repartition by default, while in practice this is not always needed in practice.

Public Interfaces

In accordance with KStreams DSL Grammar, we introduce a new parameterless DSLOperation cancelRepartition markAsPartitioned() on KStream, which resets keyChangingOperation flag for the upstream node.

...

  • This operation is unsafe: the wrong usage can lead to undefined behaviourbehavior. Thus it should be used with care for performance tuning only. This warning should be given in the documentation.
  •  The usage of this operation complicates the usage of IQ(Interactive Query). Without repartitioning, the mapped key does not contain the information of the correct partition anymore, since the hash of the mapped key might compute to a wrong instance. However, if we use cancelRepartition markAsPartitioned correctly, it is always possible to provide the 'reverse mapping' for the key that restores the original key, which can then can be used for obtaining the metadata.

...

  • CompositeKey<H, P> consists of a head and a postfix, and the partition of a composite key is always defined by its 'head' only.
  • Also, k and CompositeKey(k, v) must have the same partition for each k.
  • We will need to introduce selectCompositeKey operation which operations that will not lead to repartition.

CompositeKey usage design will be safe both from the pov of data locality and IQ , however it is questionable, whether it will be convenient to use it.but adds complexity to the usage.