Skip to content

KAFKA-8531: Change default replication factor config#10532

Merged
mjsax merged 3 commits intoapache:trunkfrom
mjsax:kafka-8531-default-replication-factor
May 5, 2021
Merged

KAFKA-8531: Change default replication factor config#10532
mjsax merged 3 commits intoapache:trunkfrom
mjsax:kafka-8531-default-replication-factor

Conversation

@mjsax
Copy link
Member

@mjsax mjsax commented Apr 13, 2021

Call for review @cadonna @ableegoldman

@mjsax mjsax added the streams label Apr 13, 2021
@mjsax mjsax force-pushed the kafka-8531-default-replication-factor branch from 8f2016a to 2480dcb Compare April 13, 2021 05:28
@mjsax
Copy link
Member Author

mjsax commented Apr 13, 2021

As test against older broker versions, I started https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4463/ -- maybe we need to update some system tests...

@mjsax
Copy link
Member Author

mjsax commented Apr 13, 2021

Was also wondering about a potential error message -- not sure atm what error message a user would get if they run against 2.3 brokers and if the error message would be clear. Should we do anything about it?

@ableegoldman
Copy link
Member

Was also wondering about a potential error message -- not sure atm what error message a user would get if they run against 2.3 brokers and if the error message would be clear. Should we do anything about it?

Not sure either -- maybe you can use the soak test to spin up brokers on 2.2 against this PR and check out the error message + stack trace? I definitely think we should try to catch the error and log a more helpful error message (eg in 3.0 we changed the default replication factor to -1 which requires brokers to be on 2.3+, please update your brokers or manually set the replication factor config)

@mjsax
Copy link
Member Author

mjsax commented Apr 13, 2021

The logs show the following error:

[2021-04-13 15:52:17,685] ERROR stream-thread [main] Unexpected error during topic creation for streams-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition.
Error message was: org.apache.kafka.common.errors.UnsupportedVersionException: Creating topics with default partitions/replication factor are only supported in CreateTopicRequest version 4+. The following topics need values for partitions and replicas: [streams-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition] (org.apache.kafka.streams.processor.internals.InternalTopicManager:446)
[2021-04-13 15:52:17,687] ERROR stream-client [streams-wordcount-4b9ea556-bc49-4671-83e8-4d2adaba3677] Encountered the following exception during processing and the registered exception handler opted to SHUTDOWN_CLIENT. The streams client is going to shut down now.  (org.apache.kafka.streams.KafkaStreams:484)
org.apache.kafka.streams.errors.StreamsException: Could not create topic streams-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition.
...
Caused by: org.apache.kafka.common.errors.UnsupportedVersionException: Creating topics with default partitions/replication factor are only supported in CreateTopicRequest version 4+. The following topics need values for partitions and replicas: [streams-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition]

Do we think this would be sufficient? Or should we try to handle this specific UnsupportedVersionException explicitly?

@ableegoldman
Copy link
Member

Yikes -- no, I think we definitely need to handle that ourselves. I don't think users will have any idea what that means -- for one thing they'll wonder why they're suddenly seeing it now, and probably report it as a bug if we don't explicitly tell them that we changed the default replication factor. For another, how are they supposed to know how to interpret Creating topics with default partitions/replication factor are only supported in CreateTopicRequest version 4+ -- it doesn't even clearly state that the problem is older brokers, much less what version specifically the brokers need to be

@mjsax
Copy link
Member Author

mjsax commented Apr 14, 2021

Updated the PR.

The log now show:

[2021-04-14 14:10:02,176] ERROR stream-thread [main] Unexpected error during topic creation for streams-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition.
Error message was: org.apache.kafka.common.errors.UnsupportedVersionException: Creating topics with default partitions/replication factor are only supported in CreateTopicRequest version 4+. The following topics need values for partitions and replicas: [streams-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition] (org.apache.kafka.streams.processor.internals.InternalTopicManager:451)
[2021-04-14 14:10:02,183] ERROR stream-client [streams-wordcount-4b9ea556-bc49-4671-83e8-4d2adaba3677] Encountered the following exception during processing and the registered exception handler opted to SHUTDOWN_CLIENT. The streams client is going to shut down now.  (org.apache.kafka.streams.KafkaStreams:484)
org.apache.kafka.streams.errors.StreamsException: Could not create topic streams-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition, because brokers don't support configuration replication.factor=-1. You can change the replication.factor config or upgrade your brokers to version 2.4 or newer to avoid this error.
	at org.apache.kafka.streams.processor.internals.InternalTopicManager.makeReady(InternalTopicManager.java:459)
[...]

Copy link
Member

@ableegoldman ableegoldman left a comment

Choose a reason for hiding this comment

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

Two nits, otherwise LGTM. Thanks for adding a more informative error

*/
package org.apache.kafka.streams.processor.internals;

import java.util.Collection;
Copy link
Member

Choose a reason for hiding this comment

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

nit: import in the wrong place

@mjsax mjsax added the kip Requires or implements a KIP label Apr 14, 2021
@mjsax
Copy link
Member Author

mjsax commented Apr 14, 2021

I started a quick KIP for this change -- will wait until the KIP is accepted before merging.

@mjsax mjsax merged commit 6a5992a into apache:trunk May 5, 2021
@mjsax mjsax deleted the kafka-8531-default-replication-factor branch May 5, 2021 23:16
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

kip Requires or implements a KIP streams

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants