Hi All,

An update on this. Finally I could figure out the cause for this. I have a
consumer with *MAX_POLL_INTERVAL_MS_CONFIG* set to *Integer.MAX_VALUE*,
which was causing the problem. Looks like its a combination of
*group.initial.rebalance.delay.ms
<http://group.initial.rebalance.delay.ms>* in kafka + the *max.poll.interval.ms
<http://max.poll.interval.ms>* causing the *Rebalance failed.
org.apache.kafka.common.errors.DisconnectException*. After debugging I
could see the below line from AbstractCoordinator class (line #337), which
leads integer overflow if *max.poll.interval.ms
<http://max.poll.interval.ms>* is greater than (Integer.MAX_VALUE - 5000)
and thus *joinGroupTimeoutMs* defaults the request timeout. Now if
*request.time.out* is less than *group.initial.rebalance.delay.ms
<http://group.initial.rebalance.delay.ms>* then the issue occurs. Let me
know what you think. For now I can get away with changing the
max.poll.interval.ms

AbstractCoordinator #337
int joinGroupTimeoutMs = Math.max(this.client.defaultRequestTimeoutMs(),
this.rebalanceConfig.rebalanceTimeoutMs + 5000);

Thanks,
Tony



On Wed, Jul 14, 2021 at 10:56 PM Tony John <tonyjohnant...@gmail.com> wrote:

> Hi Shilin,
>
> Thanks for the suggestion. But I am not upgrading an existing cluster.
> I've got a fresh broker and application cluster and there are no consumer
> offsets or topics present. When the app starts it creates the topics and
> once it moves to RUNNING state I see the rebalance failed log every 30
> seconds. As I understand, the steps in the doc needs to be followed only if
> an existing cluster is being migrated to the new version. Am I missing
> something here? Below is my KafkaConfig from one the broker during start
> up.
>
>
> [2021-07-14 07:27:06,271] INFO KafkaConfig values:
>         advertised.host.name = null
>         advertised.listeners = PLAINTEXT://broker100:9092
>         advertised.port = null
>         alter.config.policy.class.name = null
>         alter.log.dirs.replication.quota.window.num = 11
>         alter.log.dirs.replication.quota.window.size.seconds = 1
>         authorizer.class.name =
>         auto.create.topics.enable = true
>         auto.leader.rebalance.enable = true
>         background.threads = 10
>         broker.id = 100
>         broker.id.generation.enable = true
>         broker.rack = null
>         client.quota.callback.class = null
>         compression.type = producer
>         connection.failed.authentication.delay.ms = 100
>         connections.max.idle.ms = 1080000
>         connections.max.reauth.ms = 0
>         control.plane.listener.name = null
>         controlled.shutdown.enable = true
>         controlled.shutdown.max.retries = 3
>         controlled.shutdown.retry.backoff.ms = 5000
>         controller.quota.window.num = 11
>         controller.quota.window.size.seconds = 1
>         controller.socket.timeout.ms = 30000
>         create.topic.policy.class.name = null
>         default.replication.factor = 1
>         delegation.token.expiry.check.interval.ms = 3600000
>         delegation.token.expiry.time.ms = 86400000
>         delegation.token.master.key = null
>         delegation.token.max.lifetime.ms = 604800000
>         delete.records.purgatory.purge.interval.requests = 1
>         delete.topic.enable = true
>         fetch.max.bytes = 57671680
>         fetch.purgatory.purge.interval.requests = 1000
>         group.initial.rebalance.delay.ms = 120000
>         group.max.session.timeout.ms = 1200000
>         group.max.size = 2147483647
>         group.min.session.timeout.ms = 6000
>         host.name =
>         inter.broker.listener.name = null
>         inter.broker.protocol.version = 2.7-IV2
>         kafka.metrics.polling.interval.secs = 10
>         kafka.metrics.reporters = []
>         leader.imbalance.check.interval.seconds = 300
>         leader.imbalance.per.broker.percentage = 10
>         listener.security.protocol.map =
> PLAINTEXT:PLAINTEXT,SSL:SSL,SASL_PLAINTEXT:SASL_PLAINTEXT,SASL_SSL:SASL_SSL
>         listeners = PLAINTEXT://broker100:9092
>         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.max.compaction.lag.ms = 9223372036854775807
>         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 = /mnt/store/latest/kafka/kafka-logs
>         log.flush.interval.messages = 9223372036854775807
>         log.flush.interval.ms = null
>         log.flush.offset.checkpoint.interval.ms = 60000
>         log.flush.scheduler.interval.ms = 9223372036854775807
>         log.flush.start.offset.checkpoint.interval.ms = 60000
>         log.index.interval.bytes = 4096
>         log.index.size.max.bytes = 10485760
>         log.message.downconversion.enable = true
>         log.message.format.version = 2.7-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.connection.creation.rate = 2147483647
>         max.connections = 2147483647
>         max.connections.per.ip = 2147483647
>         max.connections.per.ip.overrides =
>         max.incremental.fetch.session.cache.slots = 1000
>         message.max.bytes = 31457280
>         metric.reporters = []
>         metrics.num.samples = 2
>         metrics.recording.level = INFO
>         metrics.sample.window.ms = 30000
>         min.insync.replicas = 1
>         num.io.threads = 4
>         num.network.threads = 4
>         num.partitions = 1
>         num.recovery.threads.per.data.dir = 4
>         num.replica.alter.log.dirs.threads = null
>         num.replica.fetchers = 1
>         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 = 10080
>         offsets.topic.compression.codec = 0
>         offsets.topic.num.partitions = 50
>         offsets.topic.replication.factor = 2
>         offsets.topic.segment.bytes = 104857600
>         password.encoder.cipher.algorithm = AES/CBC/PKCS5Padding
>         password.encoder.iterations = 4096
>         password.encoder.key.length = 128
>         password.encoder.keyfactory.algorithm = null
>         password.encoder.old.secret = null
>         password.encoder.secret = null
>         port = 9092
>         principal.builder.class = null
>         producer.purgatory.purge.interval.requests = 1000
>         queued.max.request.bytes = -1
>         queued.max.requests = 500
>         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 = 31457280
>         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 = 30000
>         replica.selector.class = null
>         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 = 600000
>         reserved.broker.max.id = 1000
>         sasl.client.callback.handler.class = null
>         sasl.enabled.mechanisms = [GSSAPI]
>         sasl.jaas.config = null
>         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.login.callback.handler.class = null
>         sasl.login.class = null
>         sasl.login.refresh.buffer.seconds = 300
>         sasl.login.refresh.min.period.seconds = 60
>         sasl.login.refresh.window.factor = 0.8
>         sasl.login.refresh.window.jitter = 0.05
>         sasl.mechanism.inter.broker.protocol = GSSAPI
>         sasl.server.callback.handler.class = null
>         security.inter.broker.protocol = PLAINTEXT
>         security.providers = null
>         socket.connection.setup.timeout.max.ms = 127000
>         socket.connection.setup.timeout.ms = 10000
>         socket.receive.buffer.bytes = 102400
>         socket.request.max.bytes = 104857600
>         socket.send.buffer.bytes = 102400
>         ssl.cipher.suites = []
>         ssl.client.auth = none
>         ssl.enabled.protocols = [TLSv1.2]
>         ssl.endpoint.identification.algorithm = https
>         ssl.engine.factory.class = null
>         ssl.key.password = null
>         ssl.keymanager.algorithm = SunX509
>         ssl.keystore.certificate.chain = null
>         ssl.keystore.key = null
>         ssl.keystore.location = null
>         ssl.keystore.password = null
>         ssl.keystore.type = JKS
>         ssl.principal.mapping.rules = DEFAULT
>         ssl.protocol = TLSv1.2
>         ssl.provider = null
>         ssl.secure.random.implementation = null
>         ssl.trustmanager.algorithm = PKIX
>         ssl.truststore.certificates = null
>         ssl.truststore.location = null
>         ssl.truststore.password = null
>         ssl.truststore.type = JKS
>         transaction.abort.timed.out.transaction.cleanup.interval.ms =
> 10000
>         transaction.max.timeout.ms = 900000
>         transaction.remove.expired.transaction.cleanup.interval.ms =
> 3600000
>         transaction.state.log.load.buffer.size = 5242880
>         transaction.state.log.min.isr = 2
>         transaction.state.log.num.partitions = 50
>         transaction.state.log.replication.factor = 2
>         transaction.state.log.segment.bytes = 104857600
>         transactional.id.expiration.ms = 604800000
>         unclean.leader.election.enable = false
>         zookeeper.clientCnxnSocket = null
>         zookeeper.connect =
> broker100:2181,broker101:2181,broker102:2181,broker103:2181,broker104:2181
>         zookeeper.connection.timeout.ms = 30000
>         zookeeper.max.in.flight.requests = 10
>         zookeeper.session.timeout.ms = 18000
>         zookeeper.set.acl = false
>         zookeeper.ssl.cipher.suites = null
>         zookeeper.ssl.client.enable = false
>         zookeeper.ssl.crl.enable = false
>         zookeeper.ssl.enabled.protocols = null
>         zookeeper.ssl.endpoint.identification.algorithm = HTTPS
>         zookeeper.ssl.keystore.location = null
>         zookeeper.ssl.keystore.password = null
>         zookeeper.ssl.keystore.type = null
>         zookeeper.ssl.ocsp.enable = false
>         zookeeper.ssl.protocol = TLSv1.2
>         zookeeper.ssl.truststore.location = null
>         zookeeper.ssl.truststore.password = null
>         zookeeper.ssl.truststore.type = null
>         zookeeper.sync.time.ms = 2000
>
> Thanks,
> Tony
>
> On Wed, Jul 14, 2021 at 4:58 PM Shilin Wu <s...@confluent.io.invalid>
> wrote:
>
>> Depending on your original version, you may have to consult the upgrade
>> guide.
>> https://kafka.apache.org/27/documentation.html#upgrade
>>
>> Didn't see important compatibility settings like:
>> [image: image.png]
>>
>>
>> Perhaps you are not doing it correctly.
>>
>>
>> [image: Confluent] <https://www.confluent.io>
>> Wu Shilin
>> Solution Architect
>> +6581007012
>> Follow us: [image: Blog]
>> <https://www.confluent.io/blog?utm_source=footer&utm_medium=email&utm_campaign=ch.email-signature_type.community_content.blog>[image:
>> Twitter] <https://twitter.com/ConfluentInc>[image: LinkedIn]
>> <https://www.linkedin.com/company/confluent/>[image: Slack]
>> <https://slackpass.io/confluentcommunity>[image: YouTube]
>> <https://youtube.com/confluent>
>> [image: Kafka Summit] <https://www.kafka-summit.org/>
>>
>>
>> On Wed, Jul 14, 2021 at 7:21 PM Tony John <tonyjohnant...@gmail.com>
>> wrote:
>>
>>> Can someone help me on this.
>>>
>>> Thanks,
>>> Tony
>>>
>>> On Fri, Jul 9, 2021 at 8:15 PM Tony John <tonyjohnant...@gmail.com>
>>> wrote:
>>>
>>> > Hi All,
>>> >
>>> > I am trying to upgrade my Kafka streams application to 2.7.1 version of
>>> > Kafka. The brokers are upgraded to 2.7.1 and kafka dependencies are
>>> also on
>>> > 2.7.1. But when I start the application, rebalance is failing with the
>>> > following message
>>> >
>>> > Rebalance failed. org.apache.kafka.common.errors.DisconnectException
>>> >
>>> > I am also seeing Group coordinator broker102:9092 (id: 2147483645 rack:
>>> > null) is unavailable or invalid due to cause: coordinator
>>> > unavailable.isDisconnected: false. Rediscovery will be attempted.
>>> >
>>> > The full set of logs (which gets printed every 30 seconds) is given
>>> below
>>> >
>>> > INFO  2021-07-09 09:33:20.229 | internals.AbstractCoordinator [Consumer
>>> > clientId=consumer-my-app-v1-task-coordinator-consumer-app-node100-9,
>>> > groupId=my-app-v1-task-coordinator-consumer-app-node100] Group
>>> coordinator
>>> > broker102:9092 (id: 2147483645 rack: null) is unavailable or invalid
>>> due to
>>> > cause: null.isDisconnected: true. Rediscovery will be attempted.
>>> > INFO  2021-07-09 09:33:20.230 | internals.AbstractCoordinator [Consumer
>>> > clientId=consumer-my-app-v1-task-coordinator-consumer-app-node100-9,
>>> > groupId=my-app-v1-task-coordinator-consumer-app-node100] Discovered
>>> group
>>> > coordinator broker102:9092 (id: 2147483645 rack: null)
>>> > INFO  2021-07-09 09:33:20.230 | internals.AbstractCoordinator [Consumer
>>> > clientId=consumer-my-app-v1-task-coordinator-consumer-app-node100-9,
>>> > groupId=my-app-v1-task-coordinator-consumer-app-node100] Group
>>> coordinator
>>> > broker102:9092 (id: 2147483645 rack: null) is unavailable or invalid
>>> due to
>>> > cause: coordinator unavailable.isDisconnected: false. Rediscovery will
>>> be
>>> > attempted.
>>> > INFO  2021-07-09 09:33:20.330 | internals.AbstractCoordinator [Consumer
>>> > clientId=consumer-my-app-v1-task-coordinator-consumer-app-node100-9,
>>> > groupId=my-app-v1-task-coordinator-consumer-app-node100] Discovered
>>> group
>>> > coordinator broker102:9092 (id: 2147483645 rack: null)
>>> > INFO  2021-07-09 09:33:20.331 | internals.AbstractCoordinator [Consumer
>>> > clientId=consumer-my-app-v1-task-coordinator-consumer-app-node100-9,
>>> > groupId=my-app-v1-task-coordinator-consumer-app-node100] Rebalance
>>> failed.
>>> > org.apache.kafka.common.errors.DisconnectException
>>> > INFO  2021-07-09 09:33:20.331 | internals.AbstractCoordinator [Consumer
>>> > clientId=consumer-my-app-v1-task-coordinator-consumer-app-node100-9,
>>> > groupId=my-app-v1-task-coordinator-consumer-app-node100] (Re-)joining
>>> group
>>> > INFO  2021-07-09 09:33:20.333 | internals.AbstractCoordinator [Consumer
>>> > clientId=consumer-my-app-v1-task-coordinator-consumer-app-node100-9,
>>> > groupId=my-app-v1-task-coordinator-consumer-app-node100] (Re-)joining
>>> group
>>> > INFO  2021-07-09 09:33:20.419 | internals.AbstractCoordinator [Consumer
>>> > clientId=consumer-my-app-v1-master-coordinator-consumer-8,
>>> > groupId=my-app-v1-master-coordinator-consumer] Group coordinator
>>> > broker101:9092 (id: 2147483646 rack: null) is unavailable or invalid
>>> due to
>>> > cause: null.isDisconnected: true. Rediscovery will be attempted.
>>> > INFO  2021-07-09 09:33:20.420 | internals.AbstractCoordinator [Consumer
>>> > clientId=consumer-my-app-v1-master-coordinator-consumer-8,
>>> > groupId=my-app-v1-master-coordinator-consumer] Discovered group
>>> coordinator
>>> > broker101:9092 (id: 2147483646 rack: null)
>>> > INFO  2021-07-09 09:33:20.420 | internals.AbstractCoordinator [Consumer
>>> > clientId=consumer-my-app-v1-master-coordinator-consumer-8,
>>> > groupId=my-app-v1-master-coordinator-consumer] Group coordinator
>>> > broker101:9092 (id: 2147483646 rack: null) is unavailable or invalid
>>> due to
>>> > cause: coordinator unavailable.isDisconnected: false. Rediscovery will
>>> be
>>> > attempted.
>>> > INFO  2021-07-09 09:33:20.521 | internals.AbstractCoordinator [Consumer
>>> > clientId=consumer-my-app-v1-master-coordinator-consumer-8,
>>> > groupId=my-app-v1-master-coordinator-consumer] Discovered group
>>> coordinator
>>> > broker101:9092 (id: 2147483646 rack: null)
>>> > INFO  2021-07-09 09:33:20.522 | internals.AbstractCoordinator [Consumer
>>> > clientId=consumer-my-app-v1-master-coordinator-consumer-8,
>>> > groupId=my-app-v1-master-coordinator-consumer] Rebalance failed.
>>> > org.apache.kafka.common.errors.DisconnectException
>>> > INFO  2021-07-09 09:33:20.523 | internals.AbstractCoordinator [Consumer
>>> > clientId=consumer-my-app-v1-master-coordinator-consumer-8,
>>> > groupId=my-app-v1-master-coordinator-consumer] (Re-)joining group
>>> > INFO  2021-07-09 09:33:20.524 | internals.AbstractCoordinator [Consumer
>>> > clientId=consumer-my-app-v1-master-coordinator-consumer-8,
>>> > groupId=my-app-v1-master-coordinator-consumer] (Re-)joining group
>>> >
>>> > The application was working fine on 2.5.1. Please note with 2.5.1 the
>>> > build used was kafka_*2.12*-2.5.1, but with 2.7.1 I used kafka_*2.13*
>>> > -2.7.1
>>> >
>>> > *The broker config is *
>>> >
>>> > broker.id=2
>>> > listeners=PLAINTEXT://broker102:9092
>>> > advertised.listeners=PLAINTEXT://broker102:9092
>>> > num.network.threads=4
>>> > num.io.threads=4
>>> > socket.send.buffer.bytes=102400
>>> > socket.receive.buffer.bytes=102400
>>> > socket.request.max.bytes=104857600
>>> > log.dirs=/mnt/store/kafka/kafka-logs
>>> > num.partitions=1
>>> > num.recovery.threads.per.data.dir=4
>>> > offsets.topic.replication.factor=2
>>> > transaction.state.log.replication.factor=2
>>> > transaction.state.log.min.isr=2
>>> > log.retention.hours=1
>>> > log.segment.bytes=1073741824
>>> > log.retention.check.interval.ms=300000
>>> >
>>> >
>>> zookeeper.connect=broker100:2181,broker101:2181,broker102:2181,broker103:2181,broker104:2181
>>> > zookeeper.connection.timeout.ms=30000
>>> > group.initial.rebalance.delay.ms=120000
>>> > offsets.retention.minutes=10080
>>> > message.max.bytes=31457280
>>> > replica.fetch.max.bytes=31457280
>>> > group.max.session.timeout.ms=1200000
>>> > request.timeout.ms=600000
>>> > connections.max.idle.ms=1080000
>>> >
>>> > What could be wrong? Should I switch to kafka_*2.12*-2.7.1 ?
>>> >
>>> > Thanks,
>>> > Tony
>>> >
>>> >
>>>
>>

Reply via email to