KAFKA-8471: Replace control requests/responses with automated protocol#7353
KAFKA-8471: Replace control requests/responses with automated protocol#7353ijuma merged 37 commits intoapache:trunkfrom
Conversation
|
Thanks for tackling this. Just a note, using common structures in the spec files will make this work much better... |
19af926 to
f3025d6
Compare
There was a problem hiding this comment.
It seems like this is the source of the mysterious Jenkins issue. typo? 🤔
There was a problem hiding this comment.
Thanks. Interesting, I wonder how that happened, I didn't intentionally change it. Will fix.
|
Thanks-- this looks good overall. I noticed that in a few requests we have the pattern of caching a I think this would be much preferrable from a code cleanliness (no volatiles, no locking, no duplicate data, no confusing issues with mutable lists), and also from an efficiency standpoint (we can avoid copying the list(s)). It's usually pretty straightforward to create a composite iterator (although the java standard library inexplicably doesn't have a method for composing iterators, it's trivial to create one)... what do you think? Also, I notice that in a few places n Scala you are replacing |
|
retest this please |
c677cec to
7c9b8bf
Compare
There was a problem hiding this comment.
We should wrap this in an if (stateChangeLogger.isTraceEnabled) so that we don't execute the "for" loop unless there is actually something to do in the loop body. This is an existing issue, of course, and probably better as a follow-on change.
|
Looks good. Thanks for fixing the warnings in some of the tests. The test failure on |
There was a problem hiding this comment.
Did you mean to set addingReplicas or removingReplicas here? Currently they're not being set anywhere, which makes the code below to zero them out for versions less than 3 unnecessary.
There was a problem hiding this comment.
Great catch, I removed it accidentally. I added them back.
There was a problem hiding this comment.
it would be nice to have a short comment here with what was different in version 0,1,2
There was a problem hiding this comment.
This is very nitpicky, I know, but I would really prefer to just extend the C-style comment than to mix C and C++ styles here.
There was a problem hiding this comment.
While we're nitpicking, the former is a javadoc comment. :) That's why I didn't include the implementation node in it. Since this is all internal anyway, I made the change you suggested.
|
LGTM. I left some minor comments which we can fix here or in a follow-on PR-- whatever is more convenient |
|
Tests passed locally. |
|
2 of the jobs passed. The Scala 2.13 job failed with a compiler error. Will fix. |
ba51f59 to
a971fb1
Compare
|
I ran the system tests on this PR and there were 7 seemingly unrelated failures (4 variants of test_transactions, 1 variant of test_reassign_partitions, 1 variant of ConnectDistributedTest.test_bounce, 1 variant of ConnectRestApiTest.test_rest_api): I saw recent failures in trunk for all of the above aside from the transactions tests. So I re-ran them and two variants ({"bounce_target": "broker", "check_order": true, "failure_mode": "hard_bounce"}, {"bounce_target": "clients", "check_order": true, "failure_mode": "hard_bounce"} and {"bounce_target": "clients", "check_order": false, "failure_mode": "hard_bounce"}) failed: I then re-ran the transactions tests on the master branch and a couple of the variants ({"bounce_target": "clients", "check_order": true, "failure_mode": "hard_bounce"} and {"bounce_target": "clients", "check_order": false, "failure_mode": "hard_bounce"}) failed: These results with the nature of the failures indicate that they are unrelated to this PR. |
|
Test results:
Since Scala 2.11 and 2.12 passed previously, I'm going to merge. |
Conflicts: * .gitignore: addition of clients/src/generated-test was near local additions for support-metrics. * checkstyle/suppressions.xml: upstream refactoring of exclusions for generator were near the local changes for support-metrics. * gradle.properties: scala version bump caused a minor conflict due to the kafka version change locally. gradle/dependencies.gradle: bcpkix version bump was near avro additions in the local version. * apache-github/trunk: (49 commits) KAFKA-8471: Replace control requests/responses with automated protocol (apache#7353) MINOR: Don't generate unnecessary strings for debug logging in FetchSessionHandler (apache#7394) MINOR:fixed typo and removed outdated varilable name (apache#7402) KAFKA-8934: Create version file during build for Streams (apache#7397) KAFKA-8319: Make KafkaStreamsTest a non-integration test class (apache#7382) KAFKA-6883: Add toUpperCase support to sasl.kerberos.principal.to.local rule (KIP-309) KAFKA-8907; Return topic configs in CreateTopics response (KIP-525) (apache#7380) MINOR: Address review comments for KIP-504 authorizer changes (apache#7379) MINOR: add versioning to request and response headers (apache#7372) KAFKA-7273: Extend Connect Converter to support headers (apache#6362) MINOR: improve the Kafka RPC code generator (apache#7340) MINOR: Improve the org.apache.kafka.common.protocol code (apache#7344) KAFKA-8880: Docs on upgrade-guide (apache#7385) KAFKA-8179: do not suspend standby tasks during rebalance (apache#7321) KAFKA-8580: Compute RocksDB metrics (apache#7263) KAFKA-8880: Add overloaded function of Consumer.committed (apache#7304) HOTFIX: fix Kafka Streams upgrade note for broker backward compatibility (apache#7363) KAFKA-8848; Update system tests to use new AclAuthorizer (apache#7374) MINOR: remove unnecessary null check (apache#7299) KAFKA-6958: Overload methods for group and windowed stream to allow to name operation name using the new Named class (apache#6413) ...
Replaced UpdateMetadata{Request, Response}, LeaderAndIsr{Request, Response}
and StopReplica{Request, Response} with the automated protocol classes.
Updated the JSON schema for the 3 request types to be more consistent and
less strict (if needed to avoid duplication).
The general approach is to avoid generating new collections in the request
classes. Normalization happens in the constructor to make this possible. Builders
still have to group by topic to maintain the external ungrouped view.
Introduced new tests for LeaderAndIsrRequest and UpdateMetadataRequest to
verify that the new logic is correct.
A few other clean-ups/fixes in code that was touched due to these changes:
collection in foreach.
toBufferinstead oftoList,toIndexedSeqortoSeqas it generally performsbetter and it matches the performance characteristics of
java.util.ArrayList. This isparticularly important when passing such instances to Java code.
/:, unused imports and unnecessaryvars.AdminClientIntegrationTestfailure message.AuthorizerIntegrationTestto the endto match the comment.
Committer Checklist (excluded from commit message)