Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
40 changes: 31 additions & 9 deletions bin/kafka-run-class.sh
Original file line number Diff line number Diff line change
Expand Up @@ -69,28 +69,50 @@ do
fi
done

for file in "$base_dir"/clients/build/libs/kafka-clients*.jar;
do
if should_include_file "$file"; then
CLASSPATH="$CLASSPATH":"$file"
fi
done
if [ -z "$UPGRADE_KAFKA_STREAMS_TEST_VERSION" ]; then
clients_lib_dir=$(dirname $0)/../clients/build/libs
streams_lib_dir=$(dirname $0)/../streams/build/libs
rocksdb_lib_dir=$(dirname $0)/../streams/build/dependant-libs-${SCALA_VERSION}
else
clients_lib_dir=/opt/kafka-$UPGRADE_KAFKA_STREAMS_TEST_VERSION/libs
streams_lib_dir=$clients_lib_dir
rocksdb_lib_dir=$streams_lib_dir
fi


for file in "$base_dir"/streams/build/libs/kafka-streams*.jar;
for file in "$clients_lib_dir"/kafka-clients*.jar;
do
if should_include_file "$file"; then
CLASSPATH="$CLASSPATH":"$file"
fi
done

for file in "$base_dir"/streams/examples/build/libs/kafka-streams-examples*.jar;
for file in "$streams_lib_dir"/kafka-streams*.jar;
do
if should_include_file "$file"; then
CLASSPATH="$CLASSPATH":"$file"
fi
done

for file in "$base_dir"/streams/build/dependant-libs-${SCALA_VERSION}/rocksdb*.jar;
if [ -z "$UPGRADE_KAFKA_STREAMS_TEST_VERSION" ]; then
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just for my own education: why we want to only include example jars if it is not for upgrade tests?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We don't need them for upgrade test -- it's just to reduce CLASSPATH to the minimum for this case.

for file in "$base_dir"/streams/examples/build/libs/kafka-streams-examples*.jar;
do
if should_include_file "$file"; then
CLASSPATH="$CLASSPATH":"$file"
fi
done
else
VERSION_NO_DOTS=`echo $UPGRADE_KAFKA_STREAMS_TEST_VERSION | sed 's/\.//g'`
SHORT_VERSION_NO_DOTS=${VERSION_NO_DOTS:0:((${#VERSION_NO_DOTS} - 1))} # remove last char, ie, bug-fix number
for file in "$base_dir"/streams/upgrade-system-tests-$SHORT_VERSION_NO_DOTS/build/libs/kafka-streams-upgrade-system-tests*.jar;
do
if should_include_file "$file"; then
CLASSPATH="$CLASSPATH":"$file"
fi
done
fi

for file in "$rocksdb_lib_dir"/rocksdb*.jar;
do
CLASSPATH="$CLASSPATH":"$file"
done
Expand Down
60 changes: 60 additions & 0 deletions build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -1027,6 +1027,66 @@ project(':streams:examples') {
}
}

project(':streams:upgrade-system-tests-0100') {
archivesBaseName = "kafka-streams-upgrade-system-tests-0100"

dependencies {
testCompile libs.kafkaStreams_0100
}

systemTestLibs {
dependsOn testJar
}
}

project(':streams:upgrade-system-tests-0101') {
archivesBaseName = "kafka-streams-upgrade-system-tests-0101"

dependencies {
testCompile libs.kafkaStreams_0101
}

systemTestLibs {
dependsOn testJar
}
}

project(':streams:upgrade-system-tests-0102') {
archivesBaseName = "kafka-streams-upgrade-system-tests-0102"

dependencies {
testCompile libs.kafkaStreams_0102
}

systemTestLibs {
dependsOn testJar
}
}

project(':streams:upgrade-system-tests-0110') {
archivesBaseName = "kafka-streams-upgrade-system-tests-0110"

dependencies {
testCompile libs.kafkaStreams_0110
}

systemTestLibs {
dependsOn testJar
}
}

project(':streams:upgrade-system-tests-10') {
archivesBaseName = "kafka-streams-upgrade-system-tests-10"

dependencies {
testCompile libs.kafkaStreams_10
}

systemTestLibs {
dependsOn testJar
}
}

project(':jmh-benchmarks') {

apply plugin: 'com.github.johnrengelman.shadow'
Expand Down
2 changes: 1 addition & 1 deletion checkstyle/suppressions.xml
Original file line number Diff line number Diff line change
Expand Up @@ -189,7 +189,7 @@
files="SmokeTestDriver.java"/>

<suppress checks="NPathComplexity"
files="KStreamKStreamJoinTest.java"/>
files="KStreamKStreamJoinTest.java|SmokeTestDriver.java"/>
<suppress checks="NPathComplexity"
files="KStreamKStreamLeftJoinTest.java"/>

Expand Down
59 changes: 43 additions & 16 deletions docs/streams/upgrade-guide.html
Original file line number Diff line number Diff line change
Expand Up @@ -40,37 +40,64 @@ <h1>Upgrade Guide and API Changes</h1>
</p>

<p>
If you want to upgrade from 1.0.x to 1.1.0 and you have customized window store implementations on the <code>ReadOnlyWindowStore</code> interface
If you want to upgrade from 1.0.x to 1.2.0 and you have customized window store implementations on the <code>ReadOnlyWindowStore</code> interface
you'd need to update your code to incorporate the newly added public APIs.
Otherwise, if you are using Java 7 you don't need to make any code changes as the public API is fully backward compatible;
but if you are using Java 8 method references in your Kafka Streams code you might need to update your code to resolve method ambiguities.
Hot-swaping the jar-file only might not work for this case.
See <a href="#streams_api_changes_110">below</a> for a complete list of 1.1.0 API and semantic changes that allow you to advance your application and/or simplify your code base.
See below a complete list of <a href="#streams_api_changes_120">1.2.0</a> and <a href="#streams_api_changes_110">1.1.0</a>
API and semantic changes that allow you to advance your application and/or simplify your code base.
</p>

<p>
If you want to upgrade from 0.11.0.x to 1.0.0 you don't need to make any code changes as the public API is fully backward compatible.
If you want to upgrade from 0.10.2.x or 0.11.0.x to 1.2.x and you have customized window store implementations on the <code>ReadOnlyWindowStore</code> interface
you'd need to update your code to incorporate the newly added public APIs.
Otherwise, if you are using Java 7 you don't need to do any code changes as the public API is fully backward compatible;
but if you are using Java 8 method references in your Kafka Streams code you might need to update your code to resolve method ambiguities.
However, some public APIs were deprecated and thus it is recommended to update your code eventually to allow for future upgrades.
See <a href="#streams_api_changes_100">below</a> for a complete list of 1.0.0 API and semantic changes that allow you to advance your application and/or simplify your code base.
</p>

<p>
If you want to upgrade from 0.10.2.x to 0.11.0 you don't need to make any code changes as the public API is fully backward compatible.
However, some configuration parameters were deprecated and thus it is recommended to update your code eventually to allow for future upgrades.
See <a href="#streams_api_changes_0110">below</a> for a complete list of 0.11.0 API and semantic changes that allow you to advance your application and/or simplify your code base.
See below a complete list of <a href="#streams_api_changes_120">1.2</a>, <a href="#streams_api_changes_110">1.1</a>,
<a href="#streams_api_changes_100">1.0</a>, and <a href="#streams_api_changes_0110">0.11.0</a> API
and semantic changes that allow you to advance your application and/or simplify your code base, including the usage of new features.
Additionally, Streams API 1.1.x requires broker on-disk message format version 0.10 or higher; thus, you need to make sure that the message
format is configured correctly before you upgrade your Kafka Streams application.
</p>

<p>
If you want to upgrade from 0.10.1.x to 0.10.2, see the <a href="/{{version}}/documentation/#upgrade_1020_streams"><b>Upgrade Section for 0.10.2</b></a>.
It highlights incompatible changes you need to consider to upgrade your code and application.
See <a href="#streams_api_changes_0102">below</a> for a complete list of 0.10.2 API and semantic changes that allow you to advance your application and/or simplify your code base.
If you want to upgrade from 0.10.1.x to 1.2.x see the Upgrade Sections for <a href="/{{version}}/documentation/#upgrade_1020_streams"><b>0.10.2</b></a>,
<a href="/{{version}}/documentation/#upgrade_1100_streams"><b>0.11.0</b></a>,
<a href="/{{version}}/documentation/#upgrade_100_streams"><b>1.0</b></a>,
<a href="/{{version}}/documentation/#upgrade_100_streams"><b>1.0</b></a>, and
<a href="/{{version}}/documentation/#upgrade_110_streams"><b>1.2</b></a>.
Note, that a brokers on-disk message format must be on version 0.10 or higher to run a Kafka Streams application version 1.2 or higher.
See below a complete list of <a href="#streams_api_changes_0102">0.10.2</a>, <a href="#streams_api_changes_0110">0.11.0</a>,
<a href="#streams_api_changes_100">1.0</a>, <a href="#streams_api_changes_110">1.1</a>, and <a href="#streams_api_changes_120">1.2</a>
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"><b>Upgrade Section for 0.10.1</b></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.
Upgrading from 0.10.0.x to 1.2.0 directly is also possible.
Note, that a brokers must be on version 0.10.1 or higher and on-disk message format must be on version 0.10 or higher
to run a Kafka Streams application version 1.2 or higher.
See <a href="#streams_api_changes_0101">Streams API changes in 0.10.1</a>, <a href="#streams_api_changes_0102">Streams API changes in 0.10.2</a>,
<a href="#streams_api_changes_0110">Streams API changes in 0.11.0</a>, <a href="#streams_api_changes_100">Streams API changes in 1.0</a>, and
<a href="#streams_api_changes_110">Streams API changes in 1.1</a>, and <a href="#streams_api_changes_120">Streams API changes in 1.2</a>
for a complete list of API changes.
Upgrading to 1.2.0 requires two rolling bounces with config <code>upgrade.from="0.10.0"</code> set for first upgrade phase
(cf. <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-268%3A+Simplify+Kafka+Streams+Rebalance+Metadata+Upgrade">KIP-268</a>).
As an alternative, an offline upgrade is also possible.
</p>
<ul>
<li> prepare your application instances for a rolling bounce and make sure that config <code>upgrade.from</code> is set to <code>"0.10.0"</code> for new version 1.2.0</li>
<li> bounce each instance of your application once </li>
<li> prepare your newly deployed 1.2.0 application instances for a second round of rolling bounces; make sure to remove the value for config <code>upgrade.mode</code> </li>
<li> bounce each instance of your application once more to complete the upgrade </li>
</ul>
<p> Upgrading from 0.10.0.x to 1.2.0 in offline mode: </p>
<ul>
<li> stop all old (0.10.0.x) application instances </li>
<li> update your code and swap old code and jar file with new code and new jar file </li>
<li> restart all new (1.2.0) application instances </li>
</ul>

<!-- TODO: verify release verion and update `id` and `href` attributes (also at other places that link to this headline) -->
<h3><a id="streams_api_changes_120" href="#streams_api_changes_120">Streams API changes in 1.2.0</a></h3>
Expand Down
Loading