You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "Michael Golovanov (JIRA)" <ji...@apache.org> on 2017/12/02 14:09:00 UTC

[jira] [Commented] (KAFKA-6221) ReplicaFetcherThread throws UnknownTopicOrPartitionException on topic creation

    [ https://issues.apache.org/jira/browse/KAFKA-6221?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16275568#comment-16275568 ] 

Michael Golovanov commented on KAFKA-6221:
------------------------------------------

This issue reproduce on bigger cluster

After starting kafka cluster with clean state (no topics) we start our consumer java application thats create topics via kafka admin api. And see this errors again.
This error may cause another errors with replication fetcher requests timeouts.And after one or two days brokers delivery of messages have degrade 

server config is below with mask some keys with <hidden ....> 

	advertised.host.name = null
	advertised.listeners = null
	advertised.port = null
	authorizer.class.name = 
	auto.create.topics.enable = false
	auto.leader.rebalance.enable = true
	background.threads = 10
	broker.id = 5
	broker.id.generation.enable = true
	broker.rack = null
	compression.type = producer
	connections.max.idle.ms = 600000
	controlled.shutdown.enable = true
	controlled.shutdown.max.retries = 3
	controlled.shutdown.retry.backoff.ms = 5000
	controller.socket.timeout.ms = 30000
	default.replication.factor = 2
	delete.topic.enable = false
	fetch.purgatory.purge.interval.requests = 1000
	group.max.session.timeout.ms = 900000
	group.min.session.timeout.ms = 6000
	host.name = 
	inter.broker.protocol.version = 0.10.1-IV2
	leader.imbalance.check.interval.seconds = 300
	leader.imbalance.per.broker.percentage = 10
	listeners = PLAINTEXT://<hidden>:9092,SSL://<hidden>:9093
	log.cleaner.backoff.ms = 15000
	log.cleaner.dedupe.buffer.size = 134217728
	log.cleaner.delete.retention.ms = 86400000
	log.cleaner.enable = true
	log.cleaner.io.buffer.load.factor = 0.9
	log.cleaner.io.buffer.size = 524288
	log.cleaner.io.max.bytes.per.second = 1.7976931348623157E308
	log.cleaner.min.cleanable.ratio = 0.5
	log.cleaner.min.compaction.lag.ms = 0
	log.cleaner.threads = 1
	log.cleanup.policy = [delete]
	log.dir = /tmp/kafka-logs
	log.dirs = /KAFKA/51.PPRB/v10,/KAFKA/52.PPRB/v10,/KAFKA/53.PPRB/v10
	log.flush.interval.messages = 20000
	log.flush.interval.ms = 10000
	log.flush.offset.checkpoint.interval.ms = 60000
	log.flush.scheduler.interval.ms = 9223372036854775807
	log.index.interval.bytes = 4096
	log.index.size.max.bytes = 10485760
	log.message.format.version = 0.10.1-IV2
	log.message.timestamp.difference.max.ms = 9223372036854775807
	log.message.timestamp.type = CreateTime
	log.preallocate = false
	log.retention.bytes = -1
	log.retention.check.interval.ms = 300000
	log.retention.hours = 1
	log.retention.minutes = null
	log.retention.ms = null
	log.roll.hours = 168
	log.roll.jitter.hours = 0
	log.roll.jitter.ms = null
	log.roll.ms = null
	log.segment.bytes = 1073741824
	log.segment.delete.delay.ms = 60000
	max.connections.per.ip = 2147483647
	max.connections.per.ip.overrides = 
	message.max.bytes = 5242880
	metric.reporters = []
	metrics.num.samples = 2
	metrics.sample.window.ms = 30000
	min.insync.replicas = 1
	num.io.threads = 28
	num.network.threads = 28
	num.partitions = 1
	num.recovery.threads.per.data.dir = 4
	num.replica.fetchers = 28
	offset.metadata.max.bytes = 4096
	offsets.commit.required.acks = -1
	offsets.commit.timeout.ms = 5000
	offsets.load.buffer.size = 5242880
	offsets.retention.check.interval.ms = 600000
	offsets.retention.minutes = 1440
	offsets.topic.compression.codec = 0
	offsets.topic.num.partitions = 50
	offsets.topic.replication.factor = 3
	offsets.topic.segment.bytes = 104857600
	port = 9092
	principal.builder.class = class org.apache.kafka.common.security.auth.DefaultPrincipalBuilder
	producer.purgatory.purge.interval.requests = 1000
	queued.max.requests = 1000
	quota.consumer.default = 9223372036854775807
	quota.producer.default = 9223372036854775807
	quota.window.num = 11
	quota.window.size.seconds = 1
	replica.fetch.backoff.ms = 1000
	replica.fetch.max.bytes = 5242880
	replica.fetch.min.bytes = 1
	replica.fetch.response.max.bytes = 10485760
	replica.fetch.wait.max.ms = 500
	replica.high.watermark.checkpoint.interval.ms = 5000
	replica.lag.time.max.ms = 10000
	replica.socket.receive.buffer.bytes = 65536
	replica.socket.timeout.ms = 30000
	replication.quota.window.num = 11
	replication.quota.window.size.seconds = 1
	request.timeout.ms = 30000
	reserved.broker.max.id = 1000
	sasl.enabled.mechanisms = [GSSAPI]
	sasl.kerberos.kinit.cmd = /usr/bin/kinit
	sasl.kerberos.min.time.before.relogin = 60000
	sasl.kerberos.principal.to.local.rules = [DEFAULT]
	sasl.kerberos.service.name = null
	sasl.kerberos.ticket.renew.jitter = 0.05
	sasl.kerberos.ticket.renew.window.factor = 0.8
	sasl.mechanism.inter.broker.protocol = GSSAPI
	security.inter.broker.protocol = SSL
	socket.receive.buffer.bytes = 102400
	socket.request.max.bytes = 104857600
	socket.send.buffer.bytes = 102400
	ssl.cipher.suites = null
	ssl.client.auth = required
	ssl.enabled.protocols = [TLSv1.2]
	ssl.endpoint.identification.algorithm = null
	ssl.key.password = [hidden]
	ssl.keymanager.algorithm = SunX509
	ssl.keystore.location = <hidden path>
	ssl.keystore.password = [hidden]
	ssl.keystore.type = JKS
	ssl.protocol = TLS
	ssl.provider = null
	ssl.secure.random.implementation = null
	ssl.trustmanager.algorithm = PKIX
	ssl.truststore.location = <hidden path>
	ssl.truststore.password = [hidden]
	ssl.truststore.type = JKS
	unclean.leader.election.enable = true
	zookeeper.connect = <hidden>
	zookeeper.connection.timeout.ms = 6000
	zookeeper.session.timeout.ms = 6000
	zookeeper.set.acl = false
	zookeeper.sync.time.ms = 2000


> ReplicaFetcherThread throws UnknownTopicOrPartitionException on topic creation 
> -------------------------------------------------------------------------------
>
>                 Key: KAFKA-6221
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6221
>             Project: Kafka
>          Issue Type: Bug
>          Components: core
>    Affects Versions: 0.10.2.0, 0.10.2.1, 0.11.0.1, 1.0.0
>         Environment: RHEL 7
>            Reporter: Alex Dunayevsky
>            Priority: Minor
>   Original Estimate: 336h
>  Remaining Estimate: 336h
>
> This issue appeared to happen frequently on 0.10.2.0. 
> On 0.10.2.1 and 1.0.0 it's a way harder to reproduce. 
> We'll focus on reproducing it on 0.10.2.1 and 1.0.0.
> *TOPOLOGY:* 
>   3 brokers, 1 zk.
> *REPRODUCING STRATEGY:* 
> Create a few dozens topics (say, 40) one by one, each with replication factor 2. Number of partitions, generally, does not matter but, for easier reproduction, should not be too small (around 30 or so). 
> *CREATE 40 TOPICS:*
> {code:java} for i in {1..40}; do bin/kafka-topics.sh --create --topic "topic${i}_p28_r2" --partitions 28 --replication-factor 2 --zookeeper :2165; done {code}
> *ERRORS*
> {code:java}
> *BROKER 1*
> [2017-11-15 16:46:00,853] ERROR [ReplicaFetcherThread-0-2], Error for partition [topic1_p28_r2,27] to broker 2:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:46:00,853] ERROR [ReplicaFetcherThread-0-2], Error for partition [topic1_p28_r2,27] to broker 2:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:46:00,854] ERROR [ReplicaFetcherThread-0-2], Error for partition [topic1_p28_r2,9] to broker 2:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:46:00,854] ERROR [ReplicaFetcherThread-0-2], Error for partition [topic1_p28_r2,9] to broker 2:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:46:00,854] ERROR [ReplicaFetcherThread-0-2], Error for partition [topic1_p28_r2,3] to broker 2:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:46:00,854] ERROR [ReplicaFetcherThread-0-2], Error for partition [topic1_p28_r2,3] to broker 2:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:46:00,854] ERROR [ReplicaFetcherThread-0-2], Error for partition [topic1_p28_r2,15] to broker 2:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:46:00,854] ERROR [ReplicaFetcherThread-0-2], Error for partition [topic1_p28_r2,15] to broker 2:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:46:00,854] ERROR [ReplicaFetcherThread-0-2], Error for partition [topic1_p28_r2,21] to broker 2:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:46:00,854] ERROR [ReplicaFetcherThread-0-2], Error for partition [topic1_p28_r2,21] to broker 2:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> *BROKER 2*
> [2017-11-15 16:46:36,408] ERROR [ReplicaFetcherThread-0-3], Error for partition [topic20_p28_r2,12] to broker 3:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:46:36,408] ERROR [ReplicaFetcherThread-0-3], Error for partition [topic20_p28_r2,12] to broker 3:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:46:36,410] ERROR [ReplicaFetcherThread-0-3], Error for partition [topic20_p28_r2,0] to broker 3:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:46:36,410] ERROR [ReplicaFetcherThread-0-3], Error for partition [topic20_p28_r2,0] to broker 3:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:46:36,410] ERROR [ReplicaFetcherThread-0-3], Error for partition [topic20_p28_r2,6] to broker 3:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:46:36,410] ERROR [ReplicaFetcherThread-0-3], Error for partition [topic20_p28_r2,6] to broker 3:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:46:36,410] ERROR [ReplicaFetcherThread-0-3], Error for partition [topic20_p28_r2,24] to broker 3:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:46:36,410] ERROR [ReplicaFetcherThread-0-3], Error for partition [topic20_p28_r2,24] to broker 3:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:46:36,410] ERROR [ReplicaFetcherThread-0-3], Error for partition [topic20_p28_r2,18] to broker 3:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:46:36,410] ERROR [ReplicaFetcherThread-0-3], Error for partition [topic20_p28_r2,18] to broker 3:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> *BROKER 3*
> [2017-11-15 16:47:00,393] ERROR [ReplicaFetcherThread-0-2], Error for partition [topic30_p28_r2,16] to broker 2:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:47:00,393] ERROR [ReplicaFetcherThread-0-2], Error for partition [topic30_p28_r2,16] to broker 2:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:47:00,393] ERROR [ReplicaFetcherThread-0-2], Error for partition [topic30_p28_r2,10] to broker 2:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:47:00,393] ERROR [ReplicaFetcherThread-0-2], Error for partition [topic30_p28_r2,10] to broker 2:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:47:00,393] ERROR [ReplicaFetcherThread-0-2], Error for partition [topic30_p28_r2,4] to broker 2:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:47:00,393] ERROR [ReplicaFetcherThread-0-2], Error for partition [topic30_p28_r2,4] to broker 2:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:47:00,393] ERROR [ReplicaFetcherThread-0-2], Error for partition [topic30_p28_r2,22] to broker 2:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> [2017-11-15 16:47:00,393] ERROR [ReplicaFetcherThread-0-2], Error for partition [topic30_p28_r2,22] to broker 2:org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition. (kafka.server.ReplicaFetcherThread)
> {code}
> *RESULTS:*
> As you can see, the problem occurs on 
> * partitions [3,9,15,21,27] of topic `{{topic1_p28_r2}}`
> * partitions [0,6,12,18,14] of topic `{{topic20_p28_r2}}`
> * partitions [4,10,16,22]    of topic `{{topic30_p28_r2}}`
> Partitions are either odd or even.
> Every "unlucky" partition, except the starting one, for each topic, appears to be the 6th in the row. For instance: 
> * 3, 9(3+6), 15(3+6+6), ...
> * 0, 6(0+6), 12(0+6+6), ...
> * 4, ...



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)