diff --git a/docs/streams/developer-guide/dsl-api.html b/docs/streams/developer-guide/dsl-api.html index 8e8a36f76c0..4de5389ac75 100644 --- a/docs/streams/developer-guide/dsl-api.html +++ b/docs/streams/developer-guide/dsl-api.html @@ -3130,15 +3130,20 @@ grouped Processor (provided by a given ProcessorSupplier);
  • KStream#processValues: Process all records in a stream, one record at a time, by applying a - FixedKeyProcessor (provided by a given FixedKeyProcessorSupplier); + FixedKeyProcessor (provided by a given FixedKeyProcessorSupplier) + [CAUTION: If you are deploying a new Kafka Streams application, and you are using the + "merge repartition topics" optimization, you should enable the fix for + KAFKA-19668 to avoid compatibility + issues for future upgrades to newer versions of Kafka Streams; + For more details, see the migration guide below];
  • Processor: A processor of key-value pair records;
  • ContextualProcessor: An abstract implementation of Processor that manages the - ProcessorContext instance. + ProcessorContext instance;
  • FixedKeyProcessor: A processor of key-value pair records where keys are immutable;
  • ContextualFixedKeyProcessor: An abstract implementation of FixedKeyProcessor that - manages the FixedKeyProcessorContext instance. + manages the FixedKeyProcessorContext instance;
  • ProcessorSupplier: A processor supplier that can create one or more Processor instances; and
  • @@ -3456,6 +3461,25 @@ grouped

    The Processor API now serves as a unified replacement for all these methods. It simplifies the API surface while maintaining support for both stateless and stateful operations.

    + +

    CAUTION: If you are using KStream.transformValues() and you have the "merge repartition topics" + optimization enabled, rewriting your program to KStream.processValues() might not be safe due to + KAFKA-19668. For this case, you should not upgrade + to Kafka Streams 4.0.0 or 4.1.0, but use Kafka Streams 4.0.1 instead, which contains a fix. + Note, that the fix is not enabled by default for backward compatibility reasons, and you would need to + enable the fix by setting config __enable.process.processValue.fix__ = true and pass it + into StreamsBuilder() constructor.

    +
    final Properties properties = new Properties();
    +properties.put(StreamsConfig.APPLICATION_ID_CONFIG, ...);
    +properties.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, ...);
    +properties.put(TopologyConfig.InternalConfig.ENABLE_PROCESS_PROCESSVALUE_FIX, true);
    +
    +final StreamsBuilder builder = new StreamsBuilder(new TopologyConfig(new StreamsConfig(properties)));
    + +

    It is recommended, that you compare the output of Topology.describe() for the old and new topology, + to verify if the rewrite to processValues() is correct, and that it does not introduce any incompatibilities. + You should also test the upgrade in a non-production environment.

    +

    Migration Examples

    To migrate from the deprecated transform, transformValues, flatTransform, and flatTransformValues methods to the Processor API (PAPI) in Kafka Streams, let's resume the diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html index 2230ffb8aaa..d5c926923e0 100644 --- a/docs/streams/upgrade-guide.html +++ b/docs/streams/upgrade-guide.html @@ -262,7 +262,15 @@

    Notable changes in 4.0.0