Skip to content
Open
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
2 changes: 1 addition & 1 deletion docs/streams/developer-guide/config-streams.html
Original file line number Diff line number Diff line change
Expand Up @@ -1235,7 +1235,7 @@ <h4><a class="toc-backref" href="#id31">topology.optimization</a><a class="heade
The version you are upgrading from. It is important to set this config when performing a rolling upgrade to certain versions, as described in the upgrade guide.
You should set this config to the appropriate version before bouncing your instances and upgrading them to the newer version. Once everyone is on the
newer version, you should remove this config and do a second rolling bounce. It is only necessary to set this config and follow the two-bounce upgrade path
when upgrading from below version 2.0, or when upgrading to 2.4+ from any version lower than 2.4.
when upgrading to 3.4+ from any version lower than 3.4.
Copy link
Contributor Author

Choose a reason for hiding this comment

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

I changed to 3.4 to align with the upgrade guide. if it's incorrect, please let me know, I will fix it

</div>
</blockquote>
</div>
Expand Down
17 changes: 2 additions & 15 deletions docs/streams/upgrade-guide.html
Original file line number Diff line number Diff line change
Expand Up @@ -35,9 +35,8 @@ <h1>Upgrade Guide and API Changes</h1>

<p>
Upgrading from any older version to {{fullDotVersion}} is possible: if upgrading from 3.4 or below, you will need to do two rolling bounces, where during the first rolling bounce phase you set the config <code>upgrade.from="older version"</code>
(possible values are <code>"0.10.0" - "3.4"</code>) and during the second you remove it. This is required to safely handle 3 changes. The first is introduction of the new cooperative rebalancing protocol of the embedded consumer. The second is a change in foreign-key join serialization format.
Note that you will remain using the old eager rebalancing protocol if you skip or delay the second rolling bounce, but you can safely switch over to cooperative at any time once the entire group is on 2.4+ by removing the config value and bouncing. For more details please refer to
<a href="https://cwiki.apache.org/confluence/x/vAclBg">KIP-429</a>. The third is a change in the serialization format for an internal repartition topic. For more details, please refer to <a href="https://cwiki.apache.org/confluence/x/P5VbDg">KIP-904</a>:
(possible values are <code>"2.4" - "3.4"</code>) and during the second you remove it. This is required to safely handle 2 changes. The first is a change in foreign-key join serialization format.
The second is a change in the serialization format for an internal repartition topic. For more details, please refer to <a href="https://cwiki.apache.org/confluence/x/P5VbDg">KIP-904</a>:
</p>
<ul>
<li> prepare your application instances for a rolling bounce and make sure that config <code>upgrade.from</code> is set to the version from which it is being upgrade.</li>
Expand All @@ -51,18 +50,6 @@ <h1>Upgrade Guide and API Changes</h1>
<li> update your code and swap old code and jar file with new code and new jar file </li>
Copy link
Member

Choose a reason for hiding this comment

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

Cannot comment above, but it say:

As an alternative, an offline upgrade is also possible. Upgrading from any versions as old as 0.10.0.x to {{fullDotVersion}} in offline mode require the following steps:

And stop all old (e.g., 0.10.0.x) application instances.

That's not correct any longer. Let's bump 0.10.0.x to `0.11.0.

I am also wondering if we should whole paragraphs Streams API changes in XXX for at least all 0.10.0.x and 0.10.1.x releases. Maybe even more (ie, up-to-including 2.3.x, we we only support rolling-bounce upgrade from 2.4+ as oldest version to 4.0+ now), and add a sentence + link to eg 3.9 docs for these older versions?

\cc @lucasbru @bbejeck WDYT?

<li> restart all new ({{fullDotVersion}}) application instances </li>
</ul>
<p>
Note: The cooperative rebalancing protocol has been the default since 2.4, but we have continued to support the
eager rebalancing protocol to provide users an upgrade path. This support will be dropped in a future release,
so any users still on the eager protocol should prepare to finish upgrading their applications to the cooperative protocol in version 3.1.
This only affects users who are still on a version older than 2.4, and users who have upgraded already but have not yet
removed the <code>upgrade.from</code> config that they set when upgrading from a version below 2.4.
Users fitting into the latter case will simply need to unset this config when upgrading beyond 3.1,
while users in the former case will need to follow a slightly different upgrade path if they attempt to upgrade from 2.3 or below to a version above 3.1.
Those applications will need to go through a bridge release, by first upgrading to a version between 2.4 - 3.1 and setting the <code>upgrade.from</code> config,
then removing that config and upgrading to the final version above 3.1. See <a href="https://issues.apache.org/jira/browse/KAFKA-8575">KAFKA-8575</a>
for more details.
</p>

<p>For a table that shows Streams API compatibility with Kafka broker versions, see <a href="#streams_api_broker_compat">Broker Compatibility</a>.</p>
Copy link
Member

Choose a reason for hiding this comment

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

Further below, there is more stuff, that seems to be old and we can remove it?

To run a Kafka Streams application version 2.2.1, 2.3.0, or higher a broker version 0.11.0 or higher is required and the on-disk message format must be 0.11 or higher. [...]

Ie, the whole two paragraphs?


Expand Down
60 changes: 0 additions & 60 deletions streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
Original file line number Diff line number Diff line change
Expand Up @@ -287,66 +287,6 @@ public class StreamsConfig extends AbstractConfig {
OPTIMIZE, NO_OPTIMIZATION, REUSE_KTABLE_SOURCE_TOPICS, MERGE_REPARTITION_TOPICS,
SINGLE_STORE_SELF_JOIN);

/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 0.10.0.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_0100 = UpgradeFromValues.UPGRADE_FROM_0100.toString();

/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 0.10.1.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_0101 = UpgradeFromValues.UPGRADE_FROM_0101.toString();

/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 0.10.2.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_0102 = UpgradeFromValues.UPGRADE_FROM_0102.toString();

/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 0.11.0.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_0110 = UpgradeFromValues.UPGRADE_FROM_0110.toString();

/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 1.0.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_10 = UpgradeFromValues.UPGRADE_FROM_10.toString();

/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 1.1.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_11 = UpgradeFromValues.UPGRADE_FROM_11.toString();

/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 2.0.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_20 = UpgradeFromValues.UPGRADE_FROM_20.toString();

/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 2.1.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_21 = UpgradeFromValues.UPGRADE_FROM_21.toString();

/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 2.2.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_22 = UpgradeFromValues.UPGRADE_FROM_22.toString();

/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 2.3.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_23 = UpgradeFromValues.UPGRADE_FROM_23.toString();

/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 2.4.x}.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,16 +17,6 @@
package org.apache.kafka.streams.internals;

public enum UpgradeFromValues {
UPGRADE_FROM_0100("0.10.0"),
UPGRADE_FROM_0101("0.10.1"),
UPGRADE_FROM_0102("0.10.2"),
UPGRADE_FROM_0110("0.11.0"),
UPGRADE_FROM_10("1.0"),
UPGRADE_FROM_11("1.1"),
UPGRADE_FROM_20("2.0"),
UPGRADE_FROM_21("2.1"),
UPGRADE_FROM_22("2.2"),
UPGRADE_FROM_23("2.3"),
UPGRADE_FROM_24("2.4"),
UPGRADE_FROM_25("2.5"),
UPGRADE_FROM_26("2.6"),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,16 +58,6 @@ private static boolean isUpgrade(final Map<String, ?> configs) {
}

switch (UpgradeFromValues.fromString((String) upgradeFrom)) {
case UPGRADE_FROM_0100:
case UPGRADE_FROM_0101:
case UPGRADE_FROM_0102:
case UPGRADE_FROM_0110:
case UPGRADE_FROM_10:
case UPGRADE_FROM_11:
case UPGRADE_FROM_20:
case UPGRADE_FROM_21:
case UPGRADE_FROM_22:
case UPGRADE_FROM_23:
case UPGRADE_FROM_24:
case UPGRADE_FROM_25:
case UPGRADE_FROM_26:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -100,16 +100,6 @@ private boolean isNotUpgrade(final Map<String, ?> configs) {
}

switch (UpgradeFromValues.fromString((String) upgradeFrom)) {
case UPGRADE_FROM_0100:
case UPGRADE_FROM_0101:
case UPGRADE_FROM_0102:
case UPGRADE_FROM_0110:
case UPGRADE_FROM_10:
case UPGRADE_FROM_11:
case UPGRADE_FROM_20:
case UPGRADE_FROM_21:
case UPGRADE_FROM_22:
case UPGRADE_FROM_23:
case UPGRADE_FROM_24:
case UPGRADE_FROM_25:
case UPGRADE_FROM_26:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,16 +77,6 @@ private static boolean upgradeFromV0(final Map<String, ?> configs) {
}

switch (UpgradeFromValues.fromString((String) upgradeFrom)) {
case UPGRADE_FROM_0100:
case UPGRADE_FROM_0101:
case UPGRADE_FROM_0102:
case UPGRADE_FROM_0110:
case UPGRADE_FROM_10:
case UPGRADE_FROM_11:
case UPGRADE_FROM_20:
case UPGRADE_FROM_21:
case UPGRADE_FROM_22:
case UPGRADE_FROM_23:
case UPGRADE_FROM_24:
case UPGRADE_FROM_25:
case UPGRADE_FROM_26:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.StreamsConfig.InternalConfig;
import org.apache.kafka.streams.internals.UpgradeFromValues;
import org.apache.kafka.streams.processor.assignment.AssignmentConfigs;
import org.apache.kafka.streams.processor.internals.ClientUtils;
import org.apache.kafka.streams.processor.internals.InternalTopicManager;
Expand Down Expand Up @@ -59,8 +58,6 @@ public AssignorConfiguration(final Map<String, ?> configs) {
final LogContext logContext = new LogContext(logPrefix);
log = logContext.logger(getClass());

validateUpgradeFrom();

{
final Object o = configs.get(InternalConfig.REFERENCE_CONTAINER_PARTITION_ASSIGNOR);
if (o == null) {
Expand Down Expand Up @@ -94,32 +91,6 @@ public ReferenceContainer referenceContainer() {
return referenceContainer;
}

// cooperative rebalancing was introduced in 2.4 and the old protocol (eager rebalancing) was removed
// in 4.0, meaning live upgrades from 2.3 or below to 4.0+ are no longer possible without a bridge release
public void validateUpgradeFrom() {
final String upgradeFrom = streamsConfig.getString(StreamsConfig.UPGRADE_FROM_CONFIG);
if (upgradeFrom != null) {
switch (UpgradeFromValues.fromString(upgradeFrom)) {
case UPGRADE_FROM_0100:
case UPGRADE_FROM_0101:
case UPGRADE_FROM_0102:
case UPGRADE_FROM_0110:
case UPGRADE_FROM_10:
case UPGRADE_FROM_11:
case UPGRADE_FROM_20:
case UPGRADE_FROM_21:
case UPGRADE_FROM_22:
case UPGRADE_FROM_23:
final String errMsg = String.format(
"The eager rebalancing protocol is no longer supported in 4.0 which means live upgrades from 2.3 or below are not possible."
+ " Please see the Streams upgrade guide for the bridge releases and recommended upgrade path. Got upgrade.from='%s'", upgradeFrom);
log.error(errMsg);
throw new ConfigException(errMsg);

}
}
}

public String logPrefix() {
return logPrefix;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -570,28 +570,6 @@ public void shouldInterleaveTasksByGroupIdDuringNewAssignment(final Map<String,
assertThat(interleavedTaskIds, equalTo(assignment));
}

@ParameterizedTest
@MethodSource("parameter")
public void shouldThrowOnEagerSubscription(final Map<String, Object> parameterizedConfig) {
setUp(parameterizedConfig, false);
builder.addSource(null, "source1", null, null, null, "topic1");
builder.addSource(null, "source2", null, null, null, "topic2");
builder.addProcessor("processor", new MockApiProcessorSupplier<>(), "source1", "source2");

final Set<TaskId> prevTasks = Set.of(
new TaskId(0, 1), new TaskId(1, 1), new TaskId(2, 1)
);
final Set<TaskId> standbyTasks = Set.of(
new TaskId(0, 2), new TaskId(1, 2), new TaskId(2, 2)
);

createMockTaskManager(prevTasks, standbyTasks);
assertThrows(
ConfigException.class,
() -> configurePartitionAssignorWith(Collections.singletonMap(StreamsConfig.UPGRADE_FROM_CONFIG, StreamsConfig.UPGRADE_FROM_23), parameterizedConfig)
);
}

@ParameterizedTest
@MethodSource("parameter")
public void testCooperativeSubscription(final Map<String, Object> parameterizedConfig) {
Expand Down