You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by mj...@apache.org on 2023/10/20 23:42:39 UTC

[kafka] branch 3.4 updated: KAFKA-15378: fix streams upgrade system test (#14539)

This is an automated email from the ASF dual-hosted git repository.

mjsax pushed a commit to branch 3.4
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.4 by this push:
     new c2299726439 KAFKA-15378: fix streams upgrade system test (#14539)
c2299726439 is described below

commit c22997264397fe8174d241a611e9892fa93bcabe
Author: Matthias J. Sax <ma...@confluent.io>
AuthorDate: Fri Oct 20 16:20:00 2023 -0700

    KAFKA-15378: fix streams upgrade system test (#14539)
    
    Fixing bad test setup. We tried to fix an upgrade bug for FK-joins in 3.1 release, but it later turned out that the PR was not sufficient to fix it. We finally fixed in 3.4 release.
    
    This PR updates the system test matrix to only test working versions with FK-joins, limited to available test versions.
    
    Reviewers: Guozhang Wang <wa...@gmail.com>, Hao Li <hl...@confluent.io>, Mickael Maison <mi...@gmail.com>
---
 tests/kafkatest/tests/streams/streams_upgrade_test.py | 14 ++++++++++----
 tests/kafkatest/version.py                            |  5 ++++-
 2 files changed, 14 insertions(+), 5 deletions(-)

diff --git a/tests/kafkatest/tests/streams/streams_upgrade_test.py b/tests/kafkatest/tests/streams/streams_upgrade_test.py
index ec47f61c625..b8db2fc0338 100644
--- a/tests/kafkatest/tests/streams/streams_upgrade_test.py
+++ b/tests/kafkatest/tests/streams/streams_upgrade_test.py
@@ -38,9 +38,15 @@ broker_upgrade_versions = [str(LATEST_0_11_0), str(LATEST_1_0), str(LATEST_1_1),
                            str(DEV_BRANCH)]
 
 metadata_1_versions = [str(LATEST_0_10_0)]
-metadata_2_versions = [str(LATEST_0_10_1), str(LATEST_0_10_2), str(LATEST_0_11_0), str(LATEST_1_0), str(LATEST_1_1)]
-fk_join_versions = [str(LATEST_2_4), str(LATEST_2_5), str(LATEST_2_6), str(LATEST_2_7), str(LATEST_2_8), 
-                    str(LATEST_3_0), str(LATEST_3_1), str(LATEST_3_2), str(LATEST_3_3)]
+metadata_2_versions = [str(LATEST_0_10_1), str(LATEST_0_10_2), str(LATEST_0_11_0), str(LATEST_1_0), str(LATEST_1_1),
+                       str(LATEST_2_4), str(LATEST_2_5), str(LATEST_2_6), str(LATEST_2_7), str(LATEST_2_8),
+                       str(LATEST_3_0), str(LATEST_3_1), str(LATEST_3_2), str(LATEST_3_3)]
+# upgrading from version (2.4...3.3) is broken and only fixed later in 3.3.3 (unreleased) and 3.4.0
+# -> https://issues.apache.org/jira/browse/KAFKA-14646
+# thus, we cannot test two bounce rolling upgrade because we know it's broken
+# instead we add version 2.4...3.3 to the `metadata_2_versions` upgrade list
+#fk_join_versions = []
+
 
 """
 After each release one should first check that the released version has been uploaded to 
@@ -198,7 +204,7 @@ class StreamsUpgradeTest(Test):
     @cluster(num_nodes=6)
     @matrix(from_version=metadata_1_versions, to_version=[str(DEV_VERSION)])
     @matrix(from_version=metadata_2_versions, to_version=[str(DEV_VERSION)])
-    @matrix(from_version=fk_join_versions, to_version=[str(DEV_VERSION)])
+    #@matrix(from_version=fk_join_versions, to_version=[str(DEV_VERSION)])
     def test_rolling_upgrade_with_2_bounces(self, from_version, to_version):
         """
         This test verifies that the cluster successfully upgrades despite changes in the metadata and FK
diff --git a/tests/kafkatest/version.py b/tests/kafkatest/version.py
index 01da2c49658..edd82c3d6ed 100644
--- a/tests/kafkatest/version.py
+++ b/tests/kafkatest/version.py
@@ -107,7 +107,10 @@ class KafkaVersion(LooseVersion):
         return self >= V_2_8_0
 
     def supports_fk_joins(self):
-        return hasattr(self, "version") and self >= V_2_4_0
+        # while we support FK joins since 2.4, rolling upgrade is broken in older versions
+        # it's only fixed in 3.3.3 (unreleased) and 3.4.0
+        # -> https://issues.apache.org/jira/browse/KAFKA-14646
+        return hasattr(self, "version") and self >= V_3_4_0
 
 def get_version(node=None):
     """Return the version attached to the given node.