| <!-- |
| Licensed to the Apache Software Foundation (ASF) under one or more |
| contributor license agreements. See the NOTICE file distributed with |
| this work for additional information regarding copyright ownership. |
| The ASF licenses this file to You under the Apache License, Version 2.0 |
| (the "License"); you may not use this file except in compliance with |
| the License. You may obtain a copy of the License at |
| |
| http://www.apache.org/licenses/LICENSE-2.0 |
| |
| Unless required by applicable law or agreed to in writing, software |
| distributed under the License is distributed on an "AS IS" BASIS, |
| WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| See the License for the specific language governing permissions and |
| limitations under the License. |
| --> |
| |
| <script><!--#include virtual="../js/templateData.js" --></script> |
| |
| <script id="content-template" type="text/x-handlebars-template"> |
| <h1>Upgrade Guide & API Changes</h1> |
| |
| <p> |
| If you want to upgrade from 0.10.1.x to 0.10.2, see the <a href="/{{version}}/documentation/#upgrade_1020_streams">Upgrade Section for 0.10.2</a>. |
| It highlights incompatible changes you need to consider to upgrade your code and application. |
| See <a href="#streams_api_changes_0102">below</a> a complete list of 0.10.2 API and semantical changes that allow you to advance your application and/or simplify your code base, including the usage of new features. |
| </p> |
| |
| <p> |
| If you want to upgrade from 0.10.0.x to 0.10.1, see the <a href="/{{version}}/documentation/#upgrade_1010_streams">Upgrade Section for 0.10.1</a>. |
| It highlights incompatible changes you need to consider to upgrade your code and application. |
| See <a href="#streams_api_changes_0101">below</a> a complete list of 0.10.1 API changes that allow you to advance your application and/or simplify your code base, including the usage of new features. |
| </p> |
| |
| <h3><a id="streams_api_changes_01021" href="#streams_api_changes_0102">Notable changes in 0.10.2.1</a></h3> |
| <p> |
| Parameter updates in <code>StreamsConfig</code>: |
| </p> |
| <ul> |
| <li> of particular importance to improve the resiliency of a Kafka Streams application are two changes to default parameters of producer <code>retries</code> and consumer <code>max.poll.interval.ms</code> </li> |
| </ul> |
| <h3><a id="streams_api_changes_0102" href="#streams_api_changes_0102">Streams API changes in 0.10.2.0</a></h3> |
| |
| <p> |
| New methods in <code>KafkaStreams</code>: |
| </p> |
| <ul> |
| <li> set a listener to react on application state change via <code>#setStateListener(StateListener listener)</code> </li> |
| <li> retrieve the current application state via <code>#state()</code> </li> |
| <li> retrieve the global metrics registry via <code>#metrics()</code> </li> |
| <li> apply a timeout when closing an application via <code>#close(long timeout, TimeUnit timeUnit)</code> </li> |
| <li> specify a custom indent when retrieving Kafka Streams information via <code>#toString(String indent)</code> </li> |
| </ul> |
| |
| <p> |
| Parameter updates in <code>StreamsConfig</code>: |
| </p> |
| <ul> |
| <li> parameter <code>zookeeper.connect</code> was deprecated; a Kafka Streams application does no longer interact with Zookeeper for topic management but uses the new broker admin protocol |
| (cf. <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-TopicAdminSchema.1">KIP-4, Section "Topic Admin Schema"</a>) </li> |
| <li> added many new parameters for metrics, security, and client configurations </li> |
| </ul> |
| |
| <p> Changes in <code>StreamsMetrics</code> interface: </p> |
| <ul> |
| <li> removed methods: <code>#addLatencySensor()</code> </li> |
| <li> added methods: <code>#addLatencyAndThroughputSensor()</code>, <code>#addThroughputSensor()</code>, <code>#recordThroughput()</code>, |
| <code>#addSensor()</code>, <code>#removeSensor()</code> </li> |
| </ul> |
| |
| <p> New methods in <code>TopologyBuilder</code>: </p> |
| <ul> |
| <li> added overloads for <code>#addSource()</code> that allow to define a <code>auto.offset.reset</code> policy per source node </li> |
| <li> added methods <code>#addGlobalStore()</code> to add global <code>StateStore</code>s </li> |
| </ul> |
| |
| <p> New methods in <code>KStreamBuilder</code>: </p> |
| <ul> |
| <li> added overloads for <code>#stream()</code> and <code>#table()</code> that allow to define a <code>auto.offset.reset</code> policy per input stream/table </li> |
| <li> added method <code>#globalKTable()</code> to create a <code>GlobalKTable</code> </li> |
| </ul> |
| |
| <p> New joins for <code>KStream</code>: </p> |
| <ul> |
| <li> added overloads for <code>#join()</code> to join with <code>KTable</code> </li> |
| <li> added overloads for <code>#join()</code> and <code>leftJoin()</code> to join with <code>GlobalKTable</code> </li> |
| <li> note, join semantics in 0.10.2 were improved and thus you might see different result compared to 0.10.0.x and 0.10.1.x |
| (cf. <a href="https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+Join+Semantics">Kafka Streams Join Semantics</a> in the Apache Kafka wiki) |
| </ul> |
| |
| <p> Aligned <code>null</code>-key handling for <code>KTable</code> joins: </p> |
| <ul> |
| <li> like all other KTable operations, <code>KTable-KTable</code> joins do not throw an exception on <code>null</code> key records anymore, but drop those records silently </li> |
| </ul> |
| |
| <p> New window type <em>Session Windows</em>: </p> |
| <ul> |
| <li> added class <code>SessionWindows</code> to specify session windows </li> |
| <li> added overloads for <code>KGroupedStream</code> methods <code>#count()</code>, <code>#reduce()</code>, and <code>#aggregate()</code> |
| to allow session window aggregations </li> |
| </ul> |
| |
| <p> Changes to <code>TimestampExtractor</code>: </p> |
| <ul> |
| <li> method <code>#extract()</code> has a second parameter now </li> |
| <li> new default timestamp extractor class <code>FailOnInvalidTimestamp</code> |
| (it gives the same behavior as old (and removed) default extractor <code>ConsumerRecordTimestampExtractor</code>) </li> |
| <li> new alternative timestamp extractor classes <code>LogAndSkipOnInvalidTimestamp</code> and <code>UsePreviousTimeOnInvalidTimestamps</code> </li> |
| </ul> |
| |
| <p> Relaxed type constraints of many DSL interfaces, classes, and methods (cf. <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-100+-+Relax+Type+constraints+in+Kafka+Streams+API">KIP-100</a>). </p> |
| |
| <h3><a id="streams_api_changes_0101" href="#streams_api_changes_0101">Streams API changes in 0.10.1.0</a></h3> |
| |
| <p> Stream grouping and aggregation split into two methods: </p> |
| <ul> |
| <li> old: KStream #aggregateByKey(), #reduceByKey(), and #countByKey() </li> |
| <li> new: KStream#groupByKey() plus KGroupedStream #aggregate(), #reduce(), and #count() </li> |
| <li> Example: stream.countByKey() changes to stream.groupByKey().count() </li> |
| </ul> |
| |
| <p> Auto Repartitioning: </p> |
| <ul> |
| <li> a call to through() after a key-changing operator and before an aggregation/join is no longer required </li> |
| <li> Example: stream.selectKey(...).through(...).countByKey() changes to stream.selectKey().groupByKey().count() </li> |
| </ul> |
| |
| <p> TopologyBuilder: </p> |
| <ul> |
| <li> methods #sourceTopics(String applicationId) and #topicGroups(String applicationId) got simplified to #sourceTopics() and #topicGroups() </li> |
| </ul> |
| |
| <p> DSL: new parameter to specify state store names: </p> |
| <ul> |
| <li> The new Interactive Queries feature requires to specify a store name for all source KTables and window aggregation result KTables (previous parameter "operator/window name" is now the storeName) </li> |
| <li> KStreamBuilder#table(String topic) changes to #topic(String topic, String storeName) </li> |
| <li> KTable#through(String topic) changes to #through(String topic, String storeName) </li> |
| <li> KGroupedStream #aggregate(), #reduce(), and #count() require additional parameter "String storeName"</li> |
| <li> Example: stream.countByKey(TimeWindows.of("windowName", 1000)) changes to stream.groupByKey().count(TimeWindows.of(1000), "countStoreName") </li> |
| </ul> |
| |
| <p> Windowing: </p> |
| <ul> |
| <li> Windows are not named anymore: TimeWindows.of("name", 1000) changes to TimeWindows.of(1000) (cf. DSL: new parameter to specify state store names) </li> |
| <li> JoinWindows has no default size anymore: JoinWindows.of("name").within(1000) changes to JoinWindows.of(1000) </li> |
| </ul> |
| |
| <div class="pagination"> |
| <a href="/{{version}}/documentation/streams/developer-guide" class="pagination__btn pagination__btn__prev">Previous</a> |
| <a href="#" class="pagination__btn pagination__btn__next pagination__btn--disabled">Next</a> |
| </div> |
| </script> |
| |
| <!--#include virtual="../../includes/_header.htm" --> |
| <!--#include virtual="../../includes/_top.htm" --> |
| <div class="content documentation"> |
| <!--#include virtual="../../includes/_nav.htm" --> |
| <div class="right"> |
| <!--#include virtual="../../includes/_docs_banner.htm" --> |
| <ul class="breadcrumbs"> |
| <li><a href="/documentation">Documentation</a></li> |
| <li><a href="/documentation/streams">Streams</a></li> |
| </ul> |
| <div class="p-content"></div> |
| </div> |
| </div> |
| <!--#include virtual="../../includes/_footer.htm" --> |
| <script> |
| $(function() { |
| // Show selected style on nav item |
| $('.b-nav__streams').addClass('selected'); |
| |
| // Display docs subnav items |
| $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded'); |
| }); |
| </script> |