Skip to content

KAFKA-8471: Replace control requests/responses with automated protocol#7353

Merged
ijuma merged 37 commits intoapache:trunkfrom
ijuma:use-generated-control-requests
Sep 29, 2019
Merged

KAFKA-8471: Replace control requests/responses with automated protocol#7353
ijuma merged 37 commits intoapache:trunkfrom
ijuma:use-generated-control-requests

Conversation

@ijuma
Copy link
Copy Markdown
Member

@ijuma ijuma commented Sep 17, 2019

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:

  • KAFKA-8956: Refactor DelayedCreatePartitions#updateWaiting to avoid modifying
    collection in foreach.
  • Avoid unnecessary allocation for state change trace logging if trace logging is not enabled
  • Use toBuffer instead of toList, toIndexedSeq or toSeq as it generally performs
    better and it matches the performance characteristics of java.util.ArrayList. This is
    particularly important when passing such instances to Java code.
  • Minor refactoring for clarity and readability.
  • Removed usage of deprecated /:, unused imports and unnecessary vars.
  • Include exception in AdminClientIntegrationTest failure message.
  • Move StopReplicaRequest verification in AuthorizerIntegrationTest to the end
    to match the comment.

Committer Checklist (excluded from commit message)

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

Comment thread clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java Outdated
Comment thread clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java Outdated
@cmccabe
Copy link
Copy Markdown
Contributor

cmccabe commented Sep 17, 2019

Thanks for tackling this.

Just a note, using common structures in the spec files will make this work much better...

@ijuma ijuma force-pushed the use-generated-control-requests branch 5 times, most recently from 19af926 to f3025d6 Compare September 24, 2019 13:26
@ijuma ijuma marked this pull request as ready for review September 24, 2019 16:20
Comment thread clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java Outdated
Comment thread clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java Outdated
Comment thread clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java Outdated
Comment thread clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java Outdated
Comment thread clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java Outdated
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.

It seems like this is the source of the mysterious Jenkins issue. typo? 🤔

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. Interesting, I wonder how that happened, I didn't intentionally change it. Will fix.

@cmccabe
Copy link
Copy Markdown
Contributor

cmccabe commented Sep 24, 2019

Thanks-- this looks good overall.

I noticed that in a few requests we have the pattern of caching a List<T> or something, which gets created when it is needed. I took a look at a few places in the code where these are being used, and it seems like in all the cases I saw, the code could really just be using an Iterator<T>, since all it wants to do is iterate over all the members.

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 seq or list with indexedSeq. Does this provide performance advantages? Would this be a good newbie cleanup JIRA?

@ijuma
Copy link
Copy Markdown
Member Author

ijuma commented Sep 25, 2019

retest this please

@ijuma ijuma force-pushed the use-generated-control-requests branch 2 times, most recently from c677cec to 7c9b8bf Compare September 27, 2019 16:16
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.

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.

@cmccabe
Copy link
Copy Markdown
Contributor

cmccabe commented Sep 27, 2019

Looks good. Thanks for fixing the warnings in some of the tests.

The test failure on JDK 11 and Scala 2.13 looks like https://issues.apache.org/jira/browse/INFRA-19155

09:29:13 ERROR: Error fetching remote repo 'origin'
09:29:13 hudson.plugins.git.GitException: Failed to fetch from git://github.com/apache/kafka.git

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.

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.

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.

Great catch, I removed it accidentally. I added them back.

Comment thread clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java Outdated
Copy link
Copy Markdown
Contributor

@cmccabe cmccabe Sep 27, 2019

Choose a reason for hiding this comment

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

it would be nice to have a short comment here with what was different in version 0,1,2

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.

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.

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.

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.

@cmccabe
Copy link
Copy Markdown
Contributor

cmccabe commented Sep 27, 2019

LGTM. I left some minor comments which we can fix here or in a follow-on PR-- whatever is more convenient

@ijuma
Copy link
Copy Markdown
Member Author

ijuma commented Sep 28, 2019

Tests passed locally.

@ijuma
Copy link
Copy Markdown
Member Author

ijuma commented Sep 28, 2019

2 of the jobs passed. The Scala 2.13 job failed with a compiler error. Will fix.

@ijuma ijuma force-pushed the use-generated-control-requests branch from ba51f59 to a971fb1 Compare September 28, 2019 20:28
@ijuma
Copy link
Copy Markdown
Member Author

ijuma commented Sep 28, 2019

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):

http://confluent-kafka-branch-builder-system-test-results.s3-us-west-2.amazonaws.com/2019-09-28--001.1569687424--ijuma--use-generated-control-requests--6052105/report.html

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:

http://confluent-kafka-branch-builder-system-test-results.s3-us-west-2.amazonaws.com/2019-09-28--001.1569701884--ijuma--use-generated-control-requests--9707c78/report.html

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:

http://confluent-kafka-system-test-results.s3-us-west-2.amazonaws.com/2019-09-28--001.1569711663--apache--trunk--8818a70/report.html

These results with the nature of the failures indicate that they are unrelated to this PR.

@ijuma
Copy link
Copy Markdown
Member Author

ijuma commented Sep 29, 2019

Test results:

  • Scala 2.13 passed
  • Scala 2.12 timed out due to a slow build machine
  • Scala 2.11 has an unrelated flaky test failure: kafka.admin.ResetConsumerGroupOffsetTest.testResetOffsetsToEarliestOnTopics

Since Scala 2.11 and 2.12 passed previously, I'm going to merge.

@ijuma ijuma merged commit 66183f7 into apache:trunk Sep 29, 2019
@ijuma ijuma deleted the use-generated-control-requests branch September 29, 2019 02:39
ijuma added a commit to confluentinc/kafka that referenced this pull request Sep 29, 2019
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)
  ...
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.

2 participants