Skip to content

KAFKA-18156: VerifiableConsumer should ignore "--session-timeout" when using CONSUMER protocol#18036

Merged
chia7712 merged 13 commits intoapache:trunkfrom
brandboat:KAFKA-18156
Dec 11, 2024
Merged

KAFKA-18156: VerifiableConsumer should ignore "--session-timeout" when using CONSUMER protocol#18036
chia7712 merged 13 commits intoapache:trunkfrom
brandboat:KAFKA-18156

Conversation

@brandboat
Copy link
Copy Markdown
Member

related to KAFKA-18156

The "session.timeout.ms" is unsupported by CONSUMER protocol, so VerifiableConsumer should skip the config when using CONSUMER protocol. Additionally, we should update the docs of "--session-timeout" to highlight this change

Committer Checklist (excluded from commit message)

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

consumerProps.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, Integer.toString(res.getInt("sessionTimeout")));

// session.timeout.ms cannot be set when group.protocol=CONSUMER
if (!groupProtocol.equalsIgnoreCase(GroupProtocol.CONSUMER.name)) {
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

If we set --session-timeout for consumer protocol, can we add warning for this config?

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.

Thanks for the suggestion! However, I’m uncertain whether this change is necessary. When group.protocol=CONSUMER is used, setting a session timeout in the consumer prevents it from starting successfully. This makes me feel the change might be redundant. That said, I’d appreciate hearing others’ thoughts on this. Thanks!

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

maybe we should call consumerProps.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, xx) only if users do set the --session-timeout - that means we should remove the default value of --session-timeout

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.

Sounds good, btw we already set --session-timeout 30sec by default in

static_membership=False, max_messages=-1, session_timeout_sec=30, enable_autocommit=False,

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

The default value of session.timeout.ms is 45 seconds, so I'm unsure if it's necessary to set it to 30 seconds in the end-to-end tests. Could you please remove this setting from the e2e configuration? Additionally, e2e should not use --session-timeout when running AsyncConsumer.

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.

Thanks, I already addressed the comment in the latest commit.

consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, res.getString("resetPolicy"));
consumerProps.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, Integer.toString(res.getInt("sessionTimeout")));

// session.timeout.ms cannot be set when group.protocol=CONSUMER
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Could you please move this check to line#645?

Copy link
Copy Markdown
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.

@brandboat I have left some comments to align with the following ideas:

  1. The default value should not cause an error.
  2. Honor users' configurations—avoid adding extra if-else statements that swallow errors.

// This means we're using the old consumer group protocol.
// This means we're using the CLASSIC consumer group protocol.
consumerProps.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, res.getString("assignmentStrategy"));
Integer sessionTimeout = res.getInt("sessionTimeout");
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

If this tool does not offer the default value, it is fine to throw exception if user does set the --session-timeout

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.

it is fine to throw exception if user does set the --session-timeout

Pardon me, did you mean throw exception if user set session timeout when using CONSUMER group.protocol ? Not sure if I understand correctly 🙏

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

yes, we should honor users' configs to let him encounter the error.

(self.reset_policy, self.group_id, self.topic)

# session timeout is not supported when using CONSUMER group protocol
if self.session_timeout_sec > 0 and self.is_consumer_group_protocol_enabled():
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

this will swallow the incorrect configs in e2e. We can just set the session timeout if e2e test case does set it.

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.

Only VerifiableConsumerTest specifically set session timeout, but I think it's a bit redundant, so I refactor them out.

Comment on lines -254 to -255
self.group_protocol = group_protocol
self.group_remote_assignor = group_remote_assignor
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.

these 2 lines are duplicate code, see L245, L246

def setup_consumer(self, topic, static_membership=False, enable_autocommit=False,
assignment_strategy="org.apache.kafka.clients.consumer.RangeAssignor", group_remote_assignor="range", **kwargs):
return VerifiableConsumer(self.test_context, self.num_consumers, self.kafka,
topic, self.group_id, static_membership=static_membership, session_timeout_sec=self.session_timeout_sec,
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

session_timeout_sec is used by other end-to-end tests, so we need to perform some refactoring for those test cases.

Replace Dynamic Timeouts with Constants
For example, change timeout_sec=self.session_timeout_sec + 5 to timeout_sec=60.

Remove Use Cases of Increasing session_timeout_sec
These cases typically increase session_timeout_sec from 45 seconds to 60 seconds. This adjustment may be unnecessary if the tests can run stably without modifying the timeout.

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.

I tested e2e client again. TC_PATHS="tests/kafkatest/tests/client" bash tests/docker/run_tests.sh and only one test failed.

test_id:    kafkatest.tests.client.consumer_test.OffsetValidationTest.test_broker_rolling_bounce.metadata_quorum=ISOLATED_KRAFT.use_new_coordinator=False
status:     FAIL
run time:   3 minutes 23.970 seconds


    AssertionError('Broker rolling bounce caused 7 unexpected group rebalances')
Traceback (most recent call last):
  File "/usr/local/lib/python3.7/dist-packages/ducktape/tests/runner_client.py", line 351, in _do_run
    data = self.run_test()
  File "/usr/local/lib/python3.7/dist-packages/ducktape/tests/runner_client.py", line 411, in run_test
    return self.test_context.function(self.test)
  File "/usr/local/lib/python3.7/dist-packages/ducktape/mark/_mark.py", line 438, in wrapper
    return functools.partial(f, *args, **kwargs)(*w_args, **w_kwargs)
  File "/opt/kafka-dev/tests/kafkatest/tests/client/consumer_test.py", line 121, in test_broker_rolling_bounce
    "Broker rolling bounce caused %d unexpected group rebalances" % unexpected_rebalances
AssertionError: Broker rolling bounce caused 7 unexpected group rebalances

And looks like it was tracked by https://issues.apache.org/jira/browse/KAFKA-18194

@chia7712
Copy link
Copy Markdown
Member

the hanging test is traced by https://issues.apache.org/jira/browse/KAFKA-17554

@chia7712 chia7712 merged commit d2ad418 into apache:trunk Dec 11, 2024
@brandboat brandboat deleted the KAFKA-18156 branch December 11, 2024 14:41
@kirktrue kirktrue added consumer tests Test fixes (including flaky tests) labels Dec 12, 2024
@kirktrue kirktrue added the KIP-848 The Next Generation of the Consumer Rebalance Protocol label Dec 12, 2024
peterxcli pushed a commit to peterxcli/kafka that referenced this pull request Dec 18, 2024
… using CONSUMER protocol (apache#18036)

Reviewers: TaiJuWu <tjwu1217@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
tedyu pushed a commit to tedyu/kafka that referenced this pull request Jan 6, 2025
… using CONSUMER protocol (apache#18036)

Reviewers: TaiJuWu <tjwu1217@gmail.com>, 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

clients consumer KIP-848 The Next Generation of the Consumer Rebalance Protocol small Small PRs tests Test fixes (including flaky tests) tools

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants