Skip to content

Conversation

@clarkwtc
Copy link
Contributor

@clarkwtc clarkwtc commented Mar 7, 2025

Migrate ConsumerRebootstrapTest to the new test infra and remove the old Scala test.

The PR changed three things.

  • Migrated ConsumerRebootstrapTest to new test infra and removed the old Scala test.
  • Updated the original test case to cover rebootstrap scenarios.
  • Integrated ConsumerRebootstrapTest into ClientRebootstrapTest in the client-integration-tests module.
  • Removed the RebootstrapTest.scala.

Default ConsumerRebootstrap config:

properties.put(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG, "rebootstrap");
properties.put(CommonClientConfigs.METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_CONFIG, "300000");
properties.put(CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG, "10000");
properties.put(CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG, "30000");
properties.put(CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG, "50L");
properties.put(CommonClientConfigs.RECONNECT_BACKOFF_MAX_MS_CONFIG, "1000L");

The test case for the consumer with enabled rebootstrap
Screenshot 2025-03-22 at 9 48 13 PM

The test case for the consumer with disabled rebootstrap
Screenshot 2025-03-22 at 9 47 22 PM

Reviewers: Chia-Ping Tsai chia7712@gmail.com

@github-actions github-actions bot added triage PRs from the community core Kafka Broker tests Test fixes (including flaky tests) labels Mar 7, 2025
@github-actions
Copy link

A label of 'needs-attention' was automatically added to this PR in order to raise the
attention of the committers. Once this issue has been triaged, the triage label
should be removed to prevent this automation from happening again.

@chia7712
Copy link
Member

@clarkwtc could you please merge it into ClientRebootstrapTest?

@clarkwtc
Copy link
Contributor Author

clarkwtc commented Mar 19, 2025

@chia7712
I merged it into ClientRebootstrapTest.

@github-actions
Copy link

A label of 'needs-attention' was automatically added to this PR in order to raise the
attention of the committers. Once this issue has been triaged, the triage label
should be removed to prevent this automation from happening again.

Copy link
Member

@chia7712 chia7712 left a comment

Choose a reason for hiding this comment

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

@clarkwtc thanks for this patch.

try (var producer = clusterInstance.producer()) {
var recordMetadata = producer.send(new ProducerRecord<>(TOPIC, part, "key 1".getBytes(), "value 1".getBytes())).get();
assertEquals(1, recordMetadata.offset());
producer.flush();
Copy link
Member

Choose a reason for hiding this comment

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

we don't need to call flush after you call the get

Copy link
Contributor Author

@clarkwtc clarkwtc Mar 21, 2025

Choose a reason for hiding this comment

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

Maybe we should remove to call get?
I need to call flush; without it, if the broker restarts on rebootstrap, the consumer polls nothing.

Copy link
Member

Choose a reason for hiding this comment

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

Given that acks is not set to 0, the get method guarantees that the record is transmitted to the server. This behavior is equivalent to a flush operation.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Sure, set acks=-1 to guarantee that the record will not be lost.

var tp = new TopicPartition(TOPIC, part);

try (var producer = clusterInstance.producer()) {
var recordMetadata = producer.send(new ProducerRecord<>(TOPIC, part, "key 0".getBytes(), "value 0".getBytes())).get();
Copy link
Member

Choose a reason for hiding this comment

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

there is only one partition, so we can streamline the code:

var recordMetadata = producer.send(new ProducerRecord<>(TOPIC, "value 0".getBytes())).get();

Copy link
Contributor Author

@clarkwtc clarkwtc Mar 22, 2025

Choose a reason for hiding this comment

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

Got it; make it simple.

clusterInstance.startBroker(broker0);

try (var producer = clusterInstance.producer()) {
var recordMetadata = producer.send(new ProducerRecord<>(TOPIC, part, "key 1".getBytes(), "value 1".getBytes())).get();
Copy link
Member

Choose a reason for hiding this comment

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

ditto

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 fixed it.

// Only the server 1 is available for the consumer during the bootstrap.
consumer.assign(partitions);
consumer.seekToBeginning(partitions);
assertEquals(1, consumer.poll(Duration.ofSeconds(1)).count());
Copy link
Member

Choose a reason for hiding this comment

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

we can't assume the records get returned by only one poll. could you please add loop to avoid flaky?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Okay, I use TestUtils.waitForCondition to prevent flaky.


// The server 1 originally cached during the bootstrap, is offline.
// However, the server 0 from the bootstrap list is online.
assertEquals(1, consumer.poll(Duration.ofSeconds(1)).count());
Copy link
Member

Choose a reason for hiding this comment

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

ditto

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 fixed it.

var partitions = List.of(new TopicPartition(TOPIC, part));

try (var producer = clusterInstance.producer()) {
var recordMetadata = producer.send(new ProducerRecord<>(TOPIC, part, "key 0".getBytes(), "value 0".getBytes())).get();
Copy link
Member

Choose a reason for hiding this comment

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

ditto

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 fixed it.

@github-actions github-actions bot removed needs-attention triage PRs from the community labels Mar 21, 2025
- Add ack equal 1 writing the record to local log.
- Wait consumer to poll record, that avoid flaky.
- Streamline ProducerRecord because only one partition.
@clarkwtc clarkwtc changed the title Kafka-18914 Migrate ConsumerRebootstrapTest to use new test infra KAFKA-18914 Migrate ConsumerRebootstrapTest to use new test infra Mar 22, 2025
@chia7712 chia7712 merged commit 1547204 into apache:trunk Mar 25, 2025
24 checks passed
ShivsundarR pushed a commit to ShivsundarR/kafka that referenced this pull request Mar 26, 2025
…ache#19154)

Migrate ConsumerRebootstrapTest to the new test infra and remove the old
Scala test.

The PR changed three things.
* Migrated `ConsumerRebootstrapTest` to new test infra and removed the
old Scala test.
* Updated the original test case to cover rebootstrap scenarios.
* Integrated `ConsumerRebootstrapTest` into `ClientRebootstrapTest` in
the `client-integration-tests` module.
* Removed the `RebootstrapTest.scala`.

Default `ConsumerRebootstrap` config:
> properties.put(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG,
"rebootstrap");

properties.put(CommonClientConfigs.METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_CONFIG,
"300000");

properties.put(CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG,
"10000");

properties.put(CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG,
"30000");
properties.put(CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG, "50L");
properties.put(CommonClientConfigs.RECONNECT_BACKOFF_MAX_MS_CONFIG,
"1000L");

The test case for the consumer with enabled rebootstrap
![Screenshot 2025-03-22 at 9 48
13 PM](https://github.com/user-attachments/assets/8470549f-a24c-43fa-ae44-789cbf422a63)


The test case for the consumer with disabled rebootstrap
![Screenshot 2025-03-22 at 9 47
22 PM](https://github.com/user-attachments/assets/0a183464-6a74-449f-8e71-d641a6ea5bb1)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
TaiJuWu pushed a commit to TaiJuWu/kafka that referenced this pull request Mar 31, 2025
…ache#19154)

Migrate ConsumerRebootstrapTest to the new test infra and remove the old
Scala test.

The PR changed three things.
* Migrated `ConsumerRebootstrapTest` to new test infra and removed the
old Scala test.
* Updated the original test case to cover rebootstrap scenarios.
* Integrated `ConsumerRebootstrapTest` into `ClientRebootstrapTest` in
the `client-integration-tests` module.
* Removed the `RebootstrapTest.scala`.

Default `ConsumerRebootstrap` config:
> properties.put(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG,
"rebootstrap");

properties.put(CommonClientConfigs.METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_CONFIG,
"300000");

properties.put(CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG,
"10000");

properties.put(CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG,
"30000");
properties.put(CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG, "50L");
properties.put(CommonClientConfigs.RECONNECT_BACKOFF_MAX_MS_CONFIG,
"1000L");

The test case for the consumer with enabled rebootstrap
![Screenshot 2025-03-22 at 9 48
13 PM](https://github.com/user-attachments/assets/8470549f-a24c-43fa-ae44-789cbf422a63)


The test case for the consumer with disabled rebootstrap
![Screenshot 2025-03-22 at 9 47
22 PM](https://github.com/user-attachments/assets/0a183464-6a74-449f-8e71-d641a6ea5bb1)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
janchilling pushed a commit to janchilling/kafka that referenced this pull request Apr 4, 2025
…ache#19154)

Migrate ConsumerRebootstrapTest to the new test infra and remove the old
Scala test.

The PR changed three things.
* Migrated `ConsumerRebootstrapTest` to new test infra and removed the
old Scala test.
* Updated the original test case to cover rebootstrap scenarios.
* Integrated `ConsumerRebootstrapTest` into `ClientRebootstrapTest` in
the `client-integration-tests` module.
* Removed the `RebootstrapTest.scala`.

Default `ConsumerRebootstrap` config:
> properties.put(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG,
"rebootstrap");

properties.put(CommonClientConfigs.METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_CONFIG,
"300000");

properties.put(CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG,
"10000");

properties.put(CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG,
"30000");
properties.put(CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG, "50L");
properties.put(CommonClientConfigs.RECONNECT_BACKOFF_MAX_MS_CONFIG,
"1000L");

The test case for the consumer with enabled rebootstrap
![Screenshot 2025-03-22 at 9 48
13 PM](https://github.com/user-attachments/assets/8470549f-a24c-43fa-ae44-789cbf422a63)


The test case for the consumer with disabled rebootstrap
![Screenshot 2025-03-22 at 9 47
22 PM](https://github.com/user-attachments/assets/0a183464-6a74-449f-8e71-d641a6ea5bb1)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

ci-approved clients core Kafka Broker tests Test fixes (including flaky tests)

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants