Skip to content

KAFKA-9046: Use top-level worker configs for connector admin clients#7525

Closed
C0urante wants to merge 4 commits intoapache:trunkfrom
C0urante:kafka-9046
Closed

KAFKA-9046: Use top-level worker configs for connector admin clients#7525
C0urante wants to merge 4 commits intoapache:trunkfrom
C0urante:kafka-9046

Conversation

@C0urante
Copy link
Copy Markdown
Contributor

Jira

The changes here are meant to find a healthy compromise between the pre- and post-KIP-458 functionality of Connect workers when configuring admin clients for use with DLQs. Before KIP-458, admin clients were configured using the top-level worker configs; after KIP-458, they are configured using worker configs with a prefix of admin. and then optionally overridden by connector configs with a prefix of admin.override.. The behavior proposed here is to use, in ascending order of precedence, the top-level worker configs, worker configs prefixed with admin., and connector configs prefixed with admin.override.; essentially, use the pre-KIP-458 behavior by default but allow it to be overridden by the post-KIP-458 behavior.

Committer Checklist (excluded from commit message)

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

connConfig.put("metadata.max.age.ms", "10000");

Map<String, String> expectedConfigs = new HashMap<>();
expectedConfigs.putAll(props);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

The admin configs after overrides will contain both "admin.metadata.max.age.ms", and "metadata.max.age.ms", with different values. I feel like that might be confusing to debug. Could we include just the configs that don't start with "admin." prefix so they would not be overridden?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Part of this test is to ensure that overrides happen as expected; leaving out to-be-overridden configs would leave that case uncovered. If anything, it might make sense to expand the tests here to make sure that three distinct sources of configs for the admin client are recognized and handled with the expected precedence. In ascending order, that would be: from the worker config with no prefix, from the worker config with a (stripped) prefix of admin., and from the connector config with a (stripped) prefix of admin.override..

Copy link
Copy Markdown
Contributor Author

@C0urante C0urante Oct 23, 2019

Choose a reason for hiding this comment

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

Or are you suggesting changes in the actual Worker class as opposed to this test class?

Copy link
Copy Markdown
Contributor

@ncliang ncliang Oct 23, 2019

Choose a reason for hiding this comment

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

Oh, yeah, the comment was not about the test, just happen to realize this when looking at the testcase. Could we have something like originalsWithoutPrefix on AbstractConfig and use it to pre-populate adminProps with just original configs not containing override ones? Or even just calling originalsWithPrefix without stripping the prefix and removing those keys from adminProps

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Yeah, that should be fine. Probably not worth it to alter the AbstractConfig since that's public API but engaging in some effort to make sure that both admin.foo and foo aren't present in the props given to the admin client seems like a good idea.

@C0urante
Copy link
Copy Markdown
Contributor Author

Thanks, @ncliang. I've incorporated your suggestion of not including admin.-prefixed properties twice (both with and without the prefix) in admin client configs; LMK what you think

Copy link
Copy Markdown
Contributor

@ncliang ncliang left a comment

Choose a reason for hiding this comment

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

LGTM!

Map<String, String> expectedConfigs = new HashMap<>();
expectedConfigs.putAll(props);
expectedConfigs.remove("admin.client.id");
expectedConfigs.remove("admin.metadata.max.age.ms");
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

nit: these 2 are added by the testcase just a few lines above. You can construct expectedConfigs from workerProps directly.

Copy link
Copy Markdown
Contributor

@kkonstantine kkonstantine left a comment

Choose a reason for hiding this comment

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

LGTM overall, this is an important fix.
I agree with the recommendation to exclude producer. consumer. prefixed properties and of course give always precedence to admin. prefixed properties.

@C0urante
Copy link
Copy Markdown
Contributor Author

Thanks for the discussion, @rhauch and @kkonstantine. Adding a check to prevent producer and consumer properties from being passed to the admin client seems safe enough, so I've added one in my latest commit. LMK what you think.

@rhauch
Copy link
Copy Markdown
Contributor

rhauch commented Nov 13, 2019

Just to be thoroughly transparent, in AK 2.2 the Worker class passed the WorkerConfig and the producer.* properties to DeadLetterQueueReporter.createAndSetup(...) method, which then created an admin client with the WorkerConfig properties and created a producer with the supplied producer.* properties in the worker config.

In AK 2.3 this changed to compute the admin properties just from the admin.* properties and any connector-specific overrides and pass them to the DeadLetterQueueReporter.createAndSetup(...) method, which then created an admin client with the supplied admin.* properties and created a producer with the supplied producer.* properties in the worker config.

This change reverts the behavior so that the worker config properties (minus any producer. or consumer. properties), overwritten with any admin.* properties and any connector-specific admin property overrides, are passed as the properties for the admin client to the DeadLetterQueueReporter.createAndSetup(...) method. This restores the AK 2.2 behavior yet is still compatible with the AK 2.3 behavior.

After merging, this should be backported to the 2.3 branch.

Copy link
Copy Markdown
Contributor

@rhauch rhauch left a comment

Choose a reason for hiding this comment

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

LGTM. Thanks for identifying this problem and providing the fix, @C0urante!

Waiting for a green build before merging.

@C0urante
Copy link
Copy Markdown
Contributor Author

@rhauch the only failure on the JDK8/Scala 2.11 branch appears to be unrelated:

kafka.api.SslAdminClientIntegrationTest > testCreatePartitions FAILED
     java.lang.AssertionError: validateOnly expected:<3> but was:<1>

@omkreddy omkreddy closed this in 38d243b Nov 14, 2019
omkreddy pushed a commit that referenced this pull request Nov 14, 2019
[Jira](https://issues.apache.org/jira/browse/KAFKA-9046)

The changes here are meant to find a healthy compromise between the pre- and post-KIP-458 functionality of Connect workers when configuring admin clients for use with DLQs. Before KIP-458, admin clients were configured using the top-level worker configs; after KIP-458, they are configured using worker configs with a prefix of `admin.` and then optionally overridden by connector configs with a prefix of `admin.override.`. The behavior proposed here is to use, in ascending order of precedence, the top-level worker configs, worker configs prefixed with `admin.`, and connector configs prefixed with `admin.override.`; essentially, use the pre-KIP-458 behavior by default but allow it to be overridden by the post-KIP-458 behavior.

Author: Chris Egerton <chrise@confluent.io>

Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Randall Hauch <rhauch@gmail.com>, Nigel Liang <nigel@nigelliang.com>

Closes #7525 from C0urante/kafka-9046

(cherry picked from commit 38d243b)
Signed-off-by: Manikumar Reddy <manikumar@confluent.io>
omkreddy pushed a commit that referenced this pull request Nov 14, 2019
[Jira](https://issues.apache.org/jira/browse/KAFKA-9046)

The changes here are meant to find a healthy compromise between the pre- and post-KIP-458 functionality of Connect workers when configuring admin clients for use with DLQs. Before KIP-458, admin clients were configured using the top-level worker configs; after KIP-458, they are configured using worker configs with a prefix of `admin.` and then optionally overridden by connector configs with a prefix of `admin.override.`. The behavior proposed here is to use, in ascending order of precedence, the top-level worker configs, worker configs prefixed with `admin.`, and connector configs prefixed with `admin.override.`; essentially, use the pre-KIP-458 behavior by default but allow it to be overridden by the post-KIP-458 behavior.

Author: Chris Egerton <chrise@confluent.io>

Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Randall Hauch <rhauch@gmail.com>, Nigel Liang <nigel@nigelliang.com>

Closes #7525 from C0urante/kafka-9046

(cherry picked from commit 38d243b)
Signed-off-by: Manikumar Reddy <manikumar@confluent.io>
@omkreddy
Copy link
Copy Markdown
Contributor

Merged the PR to trunk, 2.4, 2.3 branches

@C0urante
Copy link
Copy Markdown
Contributor Author

Thanks, @omkreddy!

@C0urante C0urante deleted the kafka-9046 branch November 14, 2019 19:08
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants