Skip to content

KAFKA-5291: AdminClient should not trigger auto creation of topics#3098

Closed
ijuma wants to merge 5 commits intoapache:trunkfrom
ijuma:kafka-5291-admin-client-no-auto-topic-creation
Closed

KAFKA-5291: AdminClient should not trigger auto creation of topics#3098
ijuma wants to merge 5 commits intoapache:trunkfrom
ijuma:kafka-5291-admin-client-no-auto-topic-creation

Conversation

@ijuma
Copy link
Copy Markdown
Member

@ijuma ijuma commented May 19, 2017

  • Added a boolean allow_auto_topic_creation to MetadataRequest and
    bumped the protocol version to V4.

  • When connecting to brokers older than 0.11.0.0, the allow_auto_topic_creation
    field won't be considered, so we send a metadata request for all topics
    to keep the behavior consistent.

  • Set allow_auto_topic_creation to false in the new AdminClient and
    StreamsKafkaClient (which exists for the purpose of creating topics
    manually); set it to true everywhere else for now. Other clients will eventually
    rely on client-side auto topic creation, but that’s not there yet.

  • Add allowAutoTopicCreation field to Metadata, which is used by
    DefaultMetadataUpdater. This is not strictly needed for the new
    AdminClient, but it avoids surprises if it ever adds a topic to Metadata
    via setTopics or addTopic.

@asfbot
Copy link
Copy Markdown

asfbot commented May 19, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/4180/
Test PASSed (JDK 7 and Scala 2.11).

@asfbot
Copy link
Copy Markdown

asfbot commented May 19, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/4167/
Test PASSed (JDK 8 and Scala 2.12).

@ijuma ijuma force-pushed the kafka-5291-admin-client-no-auto-topic-creation branch from e960814 to 8194071 Compare May 19, 2017 12:46
@asfbot
Copy link
Copy Markdown

asfbot commented May 19, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/4171/
Test FAILed (JDK 8 and Scala 2.12).

@asfbot
Copy link
Copy Markdown

asfbot commented May 19, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/4184/
Test FAILed (JDK 7 and Scala 2.11).

@ijuma ijuma force-pushed the kafka-5291-admin-client-no-auto-topic-creation branch 2 times, most recently from f29e855 to a50f24e Compare May 19, 2017 13:13
@asfbot
Copy link
Copy Markdown

asfbot commented May 19, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/4186/
Test PASSed (JDK 7 and Scala 2.11).

@asfbot
Copy link
Copy Markdown

asfbot commented May 19, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/4173/
Test PASSed (JDK 8 and Scala 2.12).

@ijuma
Copy link
Copy Markdown
Member Author

ijuma commented May 19, 2017

Review by @junrao.

@hachikuji
Copy link
Copy Markdown
Contributor

@ijuma Took a quick glance and it seems reasonable. I guess if you connect to a pre-0.11 broker, then we'll downgrade to the old behavior with auto-creation allowed?

@ijuma
Copy link
Copy Markdown
Member Author

ijuma commented May 19, 2017

@hachikuji, yes, that's right. I mentioned that in the KIP thread, but forgot to update the description. Did it now.

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.

Is there a race condition here where we might not get Errors.LEADER_NOT_AVAILABLE, if the brokers are quicker than usual?

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 don't think so. The call that initiates auto creation always returns LEADER_NOT_AVAILABLE.

@cmccabe
Copy link
Copy Markdown
Contributor

cmccabe commented May 22, 2017

@ijuma: great idea for an improvement. As I mentioned on the mailing list thread, I think it would be more reasonable to use MetadataRequest(topics=ALL) when doing a protocol downgrade. This would avoid having such surprisingly different behavior when talking to 0.10.x. It does mean that the scalability of old versions is lower, but that's why we have upgrades, right?

@ijuma ijuma force-pushed the kafka-5291-admin-client-no-auto-topic-creation branch from a50f24e to d358ddc Compare May 25, 2017 16:13
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.

This was just a hack to demonstrate the issue, but without this change the fallback times out.

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.

Removed this hack as it seems to be fixed by the @cmccabe's timeout fixes PR that I just merged.

@ijuma ijuma force-pushed the kafka-5291-admin-client-no-auto-topic-creation branch from d358ddc to f3427b9 Compare May 25, 2017 16:23
@ijuma
Copy link
Copy Markdown
Member Author

ijuma commented May 25, 2017

@cmccabe, can you please take a look?

@asfbot
Copy link
Copy Markdown

asfbot commented May 25, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/4381/
Test PASSed (JDK 8 and Scala 2.12).

@asfbot
Copy link
Copy Markdown

asfbot commented May 25, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/4395/
Test PASSed (JDK 7 and Scala 2.11).

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.

Shouldn't this default to true here? I think all the places that use allTopics() are expecting this to be true (except the new use in adminclient)

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.

The behaviour of ALL_TOPICS is that auto creation never happens. So, I had set it to false to make that obvious. All the code handles this properly. The one downside is that if you serialize a version that doesn't have this field and then deserialize, the deserialized version will have the boolean as true. This may be confusing, so I will change it to true like you suggested.

@ijuma ijuma force-pushed the kafka-5291-admin-client-no-auto-topic-creation branch from f3427b9 to bf65707 Compare May 31, 2017 02:35
@asfbot
Copy link
Copy Markdown

asfbot commented May 31, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/4592/
Test FAILed (JDK 8 and Scala 2.12).

@asfbot
Copy link
Copy Markdown

asfbot commented May 31, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/4607/
Test FAILed (JDK 7 and Scala 2.11).

@ijuma ijuma force-pushed the kafka-5291-admin-client-no-auto-topic-creation branch from bf65707 to f1df03b Compare May 31, 2017 02:37
@ijuma
Copy link
Copy Markdown
Member Author

ijuma commented May 31, 2017

@cmccabe @junrao This is ready for review.

@asfbot
Copy link
Copy Markdown

asfbot commented May 31, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/4609/
Test PASSed (JDK 7 and Scala 2.11).

@asfbot
Copy link
Copy Markdown

asfbot commented May 31, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/4594/
Test PASSed (JDK 8 and Scala 2.12).

ijuma added 2 commits May 31, 2017 20:06
- Added a boolean allow_auto_topic_creation to MetadataRequest.
I didn’t bump the version a second time since we did it once for this
release already, but this needs to be verified.

- Set it to false in the new AdminClient and StreamsKafkaClient
(which exists for the purpose of creating topics manually); set it
to true everywhere else for now. Other clients will eventually
rely on client-side auto topic creation, but that’s not there yet.
@ijuma ijuma force-pushed the kafka-5291-admin-client-no-auto-topic-creation branch from 04560e7 to be6cb0d Compare May 31, 2017 19:06
@asfbot
Copy link
Copy Markdown

asfbot commented May 31, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/4649/
Test FAILed (JDK 8 and Scala 2.12).

@asfbot
Copy link
Copy Markdown

asfbot commented May 31, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/4665/
Test PASSed (JDK 7 and Scala 2.11).

@ijuma ijuma force-pushed the kafka-5291-admin-client-no-auto-topic-creation branch from d1d5ef9 to ef816cc Compare May 31, 2017 20:56
Copy link
Copy Markdown
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@ijuma : Thanks for the patch. LGTM. Just a couple of minor comments.

val response1 = sendMetadataRequest(new MetadataRequest(Seq(topic1, topic2).asJava, true, ApiKeys.METADATA.latestVersion))
checkAutoCreatedTopic(topic1, topic2, response1)

// V2 doesn't support a configurable allowAutoTopicCreation, so the fact that we set it to `false` has no effect
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.

v2 should be v3 now?

val response2 = sendMetadataRequest(new MetadataRequest(Seq(topic2, topic3).asJava, false, 2))
checkAutoCreatedTopic(topic2, topic3, response2)

// V3 and higher support a configurable allowAutoTopicCreation
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.

v3 should be v4 now?

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.

Yes, the test fails (as expected) without that change.

@ijuma
Copy link
Copy Markdown
Member Author

ijuma commented May 31, 2017

@junrao, updated the PR. I'll merge it after the tests pass if you're OK with that.

@asfbot
Copy link
Copy Markdown

asfbot commented May 31, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/4659/
Test FAILed (JDK 8 and Scala 2.12).


@Override
boolean handleUnsupportedVersionException(UnsupportedVersionException exception) {
if (supportsDisablingTopicCreation) {
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 works for now since we only support disabling auto topic creation in the latest version. However, if we bump up the version of metadata request again in the future, this check may be too restrictive. If this requires more changes, perhaps we can at least add a comment for now.

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 think that would still work in that case. We only ever need to fall back once: if the highest version of metadata request supported by the broker is v3 or lower. The first attempt will fail and the subsequent one will work.

To make it more concrete, say we introduce metadata request v5. If the broker supports v4 or v5, then it will just work. If the broker only supports v3, it's the same as now: we fail on the first attempt and then we fallback to not using the flag. Does that make sense?

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.

Thanks for the explanation. That works then.

@junrao
Copy link
Copy Markdown
Contributor

junrao commented May 31, 2017

@ijuma : Yes, LGTM. Just left another minor comment. You can merge the PR once it's addressed.

@asfbot
Copy link
Copy Markdown

asfbot commented May 31, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/4666/
Test PASSed (JDK 8 and Scala 2.12).

@asfbot
Copy link
Copy Markdown

asfbot commented May 31, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/4682/
Test PASSed (JDK 7 and Scala 2.11).

asfgit pushed a commit that referenced this pull request Jun 1, 2017
- Added a boolean `allow_auto_topic_creation` to MetadataRequest and
bumped the protocol version to V4.

- When connecting to brokers older than 0.11.0.0, the `allow_auto_topic_creation`
field won't be considered, so we send a metadata request for all topics
to keep the behavior consistent.

- Set `allow_auto_topic_creation` to false in the new AdminClient and
StreamsKafkaClient (which exists for the purpose of creating topics
manually); set it to true everywhere else for now. Other clients will eventually
rely on client-side auto topic creation, but that’s not there yet.

- Add `allowAutoTopicCreation` field to `Metadata`, which is used by
`DefaultMetadataUpdater`. This is not strictly needed for the new
`AdminClient`, but it avoids surprises if it ever adds a topic to `Metadata`
via `setTopics` or `addTopic`.

Author: Ismael Juma <ismael@juma.me.uk>

Reviewers: Jun Rao <junrao@gmail.com>

Closes #3098 from ijuma/kafka-5291-admin-client-no-auto-topic-creation

(cherry picked from commit 7311dcb)
Signed-off-by: Ismael Juma <ismael@juma.me.uk>
@asfgit asfgit closed this in 7311dcb Jun 1, 2017
@ijuma ijuma deleted the kafka-5291-admin-client-no-auto-topic-creation branch June 18, 2017 09:30
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.

5 participants