Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-12724: Add 2.8.0 to system tests and streams upgrade tests. #10602

Merged
merged 6 commits into from
Aug 4, 2021

Conversation

kamalcph
Copy link
Contributor

  • Ran ./gradlew streams:testAll to verify the streams upgrade test.
  • Ran individual system test in my local setup.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

}
});

streams.setUncaughtExceptionHandler(e -> {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Updated this part of the code as the streams#setUncaughtExceptionHandler(Thread.UncaughtExceptionHandler handler) method is deprecated

@kamalcph
Copy link
Contributor Author

@chia7712 @mjsax @ijuma
Ping for review


final Properties streamsProperties = Utils.loadProps(propFileName);

System.out.println("StreamsTest instance started (StreamsUpgradeTest v2.7)");
Copy link
Member

Choose a reason for hiding this comment

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

Should be updated to 2.8

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ack


@Override
public void init(final ProcessorContext context) {
System.out.println("[2.6] initializing processor: topic=data taskId=" context.taskId());
Copy link
Member

Choose a reason for hiding this comment

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

Should be updated to 2.8 (Seem to be a c&p error) could you fix this in 2.7 and update to 2.7 ?

Maybe worth to introduce a variable that encodes the version?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Updated the version to 2.8.

@mjsax
Copy link
Member

mjsax commented Apr 28, 2021

\cc @vvcephei who was RM for 2.8 for tracking.

tests/docker/Dockerfile Outdated Show resolved Hide resolved
@jolshan
Copy link
Member

jolshan commented Apr 29, 2021

Can we also add some of the newer versions todowngrade_test.py? We've fallen behind on some of the versions there.


# broker 0.10.0 is not compatible with newer Kafka Streams versions
broker_upgrade_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_0), str(LATEST_2_1), str(LATEST_2_2), str(LATEST_2_3), \
str(LATEST_2_4), str(LATEST_2_5), str(LATEST_2_6), str(LATEST_2_7), str(DEV_BRANCH)]
str(LATEST_2_4), str(LATEST_2_5), str(LATEST_2_6), str(LATEST_2_7), str(LATEST_2_8), str(DEV_BRANCH)]
Copy link
Member

Choose a reason for hiding this comment

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

I noticed we are updating the versions for one test (test_upgrade_downgrade_brokers), but not for the other one (test_metadata_upgrade)

The metadata_1_versions and metadata_2_versions are the versions used in this test. Should we keep updating those as well? I'm not super familiar with the streams tests here, only noticed this when running the system test files changed in this PR.

The tests that were not ignored passed btw :)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Added v2.0 and above to the metadata_upgrade test. As per KIP-268, one rolling restart is enough for apps upgrading from 2.x to the latest version. I kept the test behavior unchanged, we can fix this later if required in a follow-up PR. Yeah, the upgrade_downgrade_brokers test was already disabled, so adding versions to this test won't help.

Sorry for late reply.

@vvcephei
Copy link
Contributor

vvcephei commented Jul 6, 2021

Hi @kamalcph , what's the status of this PR? I noticed there are some unresolved code review comments, and there are some conflicts.

@kamalcph
Copy link
Contributor Author

kamalcph commented Jul 7, 2021

Hi @kamalcph , what's the status of this PR? I noticed there are some unresolved code review comments, and there are some conflicts.

I'll address the pending review comments and resolve the conflicts.

@kamalcph
Copy link
Contributor Author

kamalcph commented Jul 8, 2021

@vvcephei
Resolved the conflicts. Ran the below tests:

./gradlew :streams:testAll
 TC_PATHS="tests/kafkatest/tests/streams/streams_upgrade_test.py" bash tests/docker/run_tests.sh

The metadata_upgrade test fails for versions above 2.4.1 with Could not detect Kafka Streams version 3.1.0-SNAPSHOT. Please take a look when you get a chance.

@vvcephei
Copy link
Contributor

vvcephei commented Jul 8, 2021

Thanks, @kamalcph ! I'll take a look now.

@vvcephei
Copy link
Contributor

vvcephei commented Jul 8, 2021

Hmm, I had some trouble with the required pip version... I went ahead and kicked off https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4586/ while I debug my local env.

@vvcephei
Copy link
Contributor

vvcephei commented Jul 8, 2021

I haven't verified it, but I think your problem is that you needed to do ./gradlew clean systemTestLibs instead of :streams:testAll.


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)]
metadata_3_10_versions = [str(LATEST_2_0), str(LATEST_2_1), str(LATEST_2_2), str(LATEST_2_3), str(LATEST_2_4),
Copy link
Member

Choose a reason for hiding this comment

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

What is 3_10 ?

Copy link
Contributor

Choose a reason for hiding this comment

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

It's probably a typo, but it didn't matter, since this was the wrong test to add new versions to, anyway. I backed out these changes and added the missing versions to streams_application_upgrade_test.py.

System.out.println(name ": FATAL: An unexpected exception is encountered: " e);
e.printStackTrace(System.out);
uncaughtException = true;
return StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT;
Copy link
Member

Choose a reason for hiding this comment

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

Wondering if we should stop the whole app for this case? \cc @vvcephei

Copy link
Contributor

Choose a reason for hiding this comment

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

That probably makes more sense, but this more closely mimics the previous behavior of the test. I think I'll leave it to someone who's feeling energetic to make this change and test it.

@Override
public void init(final ProcessorContext context) {
super.init(context);
System.out.println("[DEV] initializing processor: topic=" topic " taskId=" context.taskId());
Copy link
Member

Choose a reason for hiding this comment

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

Why DEV -- should be 2.8?

Copy link
Contributor

Choose a reason for hiding this comment

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

Good eye :)

@vvcephei
Copy link
Contributor

vvcephei commented Aug 4, 2021

Ok, I figured it out. The problem was that this change was adding the new version(s) to the metadata upgrade test, but there isn't a metadata upgrade between 2.x and 3.0. Since we're down to the wire now, I just went ahead and pushed a fix to this branch.

It passes for me locally now:

[john@arcturus kafka]$ TC_PATHS="tests/kafkatest/tests/streams/streams_application_upgrade_test.py::StreamsUpgradeTest.test_app_upgrade" _DUCKTAPE_OPTIONS='--parameters '\''{"from_version":"2.8.0","to_version":"3.1.0-SNAPSHOT","bounce_type":"full"}'\' bash tests/docker/run_tests.sh

docker exec ducker01 bash -c "cd /opt/kafka-dev && ducktape --cluster-file /opt/kafka-dev/tests/docker/build/cluster.json  ./tests/kafkatest/tests/streams/streams_application_upgrade_test.py::StreamsUpgradeTest.test_app_upgrade --parameters '{"from_version":"2.8.0","to_version":"3.1.0-SNAPSHOT","bounce_type":"full"}'"
[INFO:2021-08-03 21:27:59,868]: starting test run with session id 2021-08-03--012...
[INFO:2021-08-03 21:27:59,868]: running 1 tests...
[INFO:2021-08-03 21:27:59,869]: Triggering test 1 of 1...
[INFO:2021-08-03 21:27:59,878]: RunnerClient: Loading test {'directory': '/opt/kafka-dev/tests/kafkatest/tests/streams', 'file_name': 'streams_application_upgrade_test.py', 'cls_name': 'StreamsUpgradeTest', 'method_name': 'test_app_upgrade', 'injected_args': {'from_version': '2.8.0', 'to_version': '3.1.0-SNAPSHOT', 'bounce_type': 'full'}}
[INFO:2021-08-03 21:27:59,883]: RunnerClient: kafkatest.tests.streams.streams_application_upgrade_test.StreamsUpgradeTest.test_app_upgrade.from_version=2.8.0.to_version=3.1.0-SNAPSHOT.bounce_type=full: Setting up...
[INFO:2021-08-03 21:27:59,884]: RunnerClient: kafkatest.tests.streams.streams_application_upgrade_test.StreamsUpgradeTest.test_app_upgrade.from_version=2.8.0.to_version=3.1.0-SNAPSHOT.bounce_type=full: Running...
[INFO:2021-08-03 21:29:44,814]: RunnerClient: kafkatest.tests.streams.streams_application_upgrade_test.StreamsUpgradeTest.test_app_upgrade.from_version=2.8.0.to_version=3.1.0-SNAPSHOT.bounce_type=full: PASS
[INFO:2021-08-03 21:29:44,816]: RunnerClient: kafkatest.tests.streams.streams_application_upgrade_test.StreamsUpgradeTest.test_app_upgrade.from_version=2.8.0.to_version=3.1.0-SNAPSHOT.bounce_type=full: Tearing down...
[INFO:2021-08-03 21:30:03,898]: RunnerClient: kafkatest.tests.streams.streams_application_upgrade_test.StreamsUpgradeTest.test_app_upgrade.from_version=2.8.0.to_version=3.1.0-SNAPSHOT.bounce_type=full: Summary: 
[INFO:2021-08-03 21:30:03,899]: RunnerClient: kafkatest.tests.streams.streams_application_upgrade_test.StreamsUpgradeTest.test_app_upgrade.from_version=2.8.0.to_version=3.1.0-SNAPSHOT.bounce_type=full: Data: None
================================================================================
SESSION REPORT (ALL TESTS)
ducktape version: 0.8.1
session_id:       2021-08-03--012
run time:         2 minutes 4.072 seconds
tests run:        1
passed:           1
failed:           0
ignored:          0
================================================================================
test_id:    kafkatest.tests.streams.streams_application_upgrade_test.StreamsUpgradeTest.test_app_upgrade.from_version=2.8.0.to_version=3.1.0-SNAPSHOT.bounce_type=full
status:     PASS
run time:   2 minutes 4.014 seconds
--------------------------------------------------------------------------------

@vvcephei
Copy link
Contributor

vvcephei commented Aug 4, 2021

@kamalcph
Copy link
Contributor Author

kamalcph commented Aug 4, 2021

@vvcephei
Thanks for figuring out the issue. Feel free to take over this patch.

@vvcephei
Copy link
Contributor

vvcephei commented Aug 4, 2021

Thanks @kamalcph . Sorry it took me so long to look into it.

@vvcephei
Copy link
Contributor

vvcephei commented Aug 4, 2021

There were a number of failures in the prior run. Some were timeouts, and some of the Streams failures were due to the HA assignor giving only standbys to some nodes. I spot-checked by re-running a few of the tests locally and confirmed that they were flaky failures.

I re-ran the tests (https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4635/)

  • with "spot instance" disabled
  • having added a change to the Streams tests intended to disable the HA assignor logic in favor of a one-shot balanced assignment.

I'll keep an eye on that test run to make sure it's clean.

@vvcephei
Copy link
Contributor

vvcephei commented Aug 4, 2021

There were still 21 test failures, but they do not look related to this change. There are also fixes ongoing in parallel to other tests, so I'm going to go ahead and merge this one. If it turns out to be the cause of new problems, we should just revert it.

@vvcephei vvcephei merged commit a103c95 into apache:trunk Aug 4, 2021
vvcephei pushed a commit that referenced this pull request Aug 4, 2021
…0602)

Also adjusted the acceptable recovery lag to stabilize Streams tests.

Reviewers: Justine Olshan <[email protected]>, Matthias J. Sax <[email protected]>, John Roesler <[email protected]>
@kamalcph kamalcph deleted the KAFKA-12724 branch August 9, 2021 06:30
xdgrulez pushed a commit to xdgrulez/kafka that referenced this pull request Dec 22, 2021
…ache#10602)

Also adjusted the acceptable recovery lag to stabilize Streams tests.

Reviewers: Justine Olshan <[email protected]>, Matthias J. Sax <[email protected]>, John Roesler <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants