[jira] [Comment Edited] (KAFKA-7403) Offset commit failure after upgrading brokers past KIP-211/KAFKA-4682

2018-09-17 Thread Jon Lee (JIRA)


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

Jon Lee edited comment on KAFKA-7403 at 9/18/18 4:41 AM:
-

[~vahid] Did you set inter.broker.protocol.version to something lower than 2.1 
for your brokers and commit an offset from a consumer? 

BTW, it looks to me that the problem is not 0.10.2 consumer specific. Any 
consumer who would send out an OffsetCommitRequest with the retentionTime field 
set to DEFAULT_RETENTION_TIME will experience this error.

I was able to reproduce the issue with the following setup:
 * 2.0 broker with KAFKA-4682 patch
 ** inter.broker.protocol.version = 1.0
 * 2.0 client
 ** enable.auto.commit = true
 ** auto.commit.interval.ms = 1000

Then the client gets an error when it commits an offset:
{code:java}
$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9093 
--consumer.config config/consumer.properties --topic t --from-beginning
:

:
[2018-09-17 21:17:30,828] ERROR [Consumer clientId=consumer-1, 
groupId=test-consumer-group] Offset commit failed on partition t-0 at offset 5: 
The server experienced an unexpected error when processing the request 
(org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
[2018-09-17 21:17:30,830] WARN [Consumer clientId=consumer-1, 
groupId=test-consumer-group] Asynchronous auto-commit of offsets 
{t-0=OffsetAndMetadata{offset=5, metadata=''}} failed: Unexpected error in 
commit: The server experienced an unexpected error when processing the request 
(org.apache.kafka.clients.consumer.internals.ConsumerCoordinator){code}
 

As mentioned above, to reproduce this in unit tests, pass KAFKA_0_11_0_IV2 
instead of ApiVersion.latestVersion as the second parameter of the 
GroupMetadataManager constructor in GroupMetadataManagerTest.scala. Then, 
multiple tests including testCommitOffset fail.


was (Author: jonlee2):
[~vahid] Did you set inter.broker.protocol.version to something lower than 2.1 
for your brokers and commit an offset from a consumer? 

BTW, it looks to me that the problem is not 0.10.2 consumer specific. Any 
consumer who would send out an OffsetCommitRequest with the retentionTime field 
set to DEFAULT_RETENTION_TIME will experience this error.

I was able to reproduce the issue with the following setup:
 * 2.0 broker with KAFKA-4682 patch
 ** inter.broker.protocol.version = 1.0
 * 2.0 client
 ** enable.auto.commit = true
 ** auto.commit.interval.ms = 1000

Then the client gets an error when it commits an offset:
{code:java}
$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9093 
--consumer.config config/consumer.properties --topic t --from-beginning
:

:
[2018-09-17 21:17:30,828] ERROR [Consumer clientId=consumer-1, 
groupId=test-consumer-group] Offset commit failed on partition t-0 at offset 5: 
The server experienced an unexpected error when processing the request 
(org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
[2018-09-17 21:17:30,830] WARN [Consumer clientId=consumer-1, 
groupId=test-consumer-group] Asynchronous auto-commit of offsets 
{t-0=OffsetAndMetadata{offset=5, metadata=''}} failed: Unexpected error in 
commit: The server experienced an unexpected error when processing the request 
(org.apache.kafka.clients.consumer.internals.ConsumerCoordinator){code}
 

As mentioned above, to reproduce this in a unit test, pass KAFKA_0_11_0_IV2 as 
the ApiVersion (the second parameter) to the constructor of 
GroupMetadataManager in GroupMetadataManagerTest.scala.

> Offset commit failure after upgrading brokers past KIP-211/KAFKA-4682
> -
>
> Key: KAFKA-7403
> URL: https://issues.apache.org/jira/browse/KAFKA-7403
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 2.1.0
>Reporter: Jon Lee
>Priority: Major
>
> I am currently trying broker upgrade from 0.11 to 2.0 with some patches 
> including KIP-211/KAFKA-4682. After the upgrade, however, applications with 
> 0.10.2 Kafka clients failed with the following error:
> {code:java}
> 2018/09/11 19:34:52.814 ERROR Failed to commit offsets. Exiting. 
> org.apache.kafka.common.KafkaException: Unexpected error in commit: The 
> server experienced an unexpected error when processing the request at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator$OffsetCommitResponseHandler.handle(ConsumerCoordinator.java:784)
>  ~[kafka-clients-0.10.2.86.jar:?] at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator$OffsetCommitResponseHandler.handle(ConsumerCoordinator.java:722)
>  ~[kafka-clients-0.10.2.86.jar:?] at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:784)
>  ~[kafka-clients-0.10.2.8

[jira] [Comment Edited] (KAFKA-7403) Offset commit failure after upgrading brokers past KIP-211/KAFKA-4682

2018-09-17 Thread Jon Lee (JIRA)


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

Jon Lee edited comment on KAFKA-7403 at 9/18/18 4:32 AM:
-

[~vahid] Did you set inter.broker.protocol.version to something lower than 2.1 
for your brokers and commit an offset from a consumer? 

BTW, it looks to me that the problem is not 0.10.2 consumer specific. Any 
consumer who would send out an OffsetCommitRequest with the retentionTime field 
set to DEFAULT_RETENTION_TIME will experience this error.

I was able to reproduce the issue with the following setup:
 * 2.0 broker with KAFKA-4682 patch
 ** inter.broker.protocol.version = 1.0
 * 2.0 client
 ** enable.auto.commit = true
 ** auto.commit.interval.ms = 1000

Then the client gets an error when it commits an offset:
{code:java}
$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9093 
--consumer.config config/consumer.properties --topic t --from-beginning
:

:
[2018-09-17 21:17:30,828] ERROR [Consumer clientId=consumer-1, 
groupId=test-consumer-group] Offset commit failed on partition t-0 at offset 5: 
The server experienced an unexpected error when processing the request 
(org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
[2018-09-17 21:17:30,830] WARN [Consumer clientId=consumer-1, 
groupId=test-consumer-group] Asynchronous auto-commit of offsets 
{t-0=OffsetAndMetadata{offset=5, metadata=''}} failed: Unexpected error in 
commit: The server experienced an unexpected error when processing the request 
(org.apache.kafka.clients.consumer.internals.ConsumerCoordinator){code}
 

As mentioned above, to reproduce this in a unit test, pass KAFKA_0_11_0_IV2 as 
the ApiVersion (the second parameter) to the constructor of 
GroupMetadataManager in GroupMetadataManagerTest.scala.


was (Author: jonlee2):
[~vahid] It looks to me that the problem is not 0.10.2 consumer specific. Any 
consumer who would send out an OffsetCommitRequest with the retentionTime field 
set to DEFAULT_RETENTION_TIME will experience this error.

I was able to reproduce the issue with the following setup:
 * 2.0 broker with KAFKA-4682 patch
 ** inter.broker.protocol.version = 1.0
 * 2.0 client
 ** enable.auto.commit = true
 ** auto.commit.interval.ms = 1000

Then the client gets an error when it commits an offset:
{code:java}
$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9093 
--consumer.config config/consumer.properties --topic t --from-beginning
:

:
[2018-09-17 21:17:30,828] ERROR [Consumer clientId=consumer-1, 
groupId=test-consumer-group] Offset commit failed on partition t-0 at offset 5: 
The server experienced an unexpected error when processing the request 
(org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
[2018-09-17 21:17:30,830] WARN [Consumer clientId=consumer-1, 
groupId=test-consumer-group] Asynchronous auto-commit of offsets 
{t-0=OffsetAndMetadata{offset=5, metadata=''}} failed: Unexpected error in 
commit: The server experienced an unexpected error when processing the request 
(org.apache.kafka.clients.consumer.internals.ConsumerCoordinator){code}
 

As mentioned above, to reproduce this in a unit test, pass KAFKA_0_11_0_IV2 as 
the ApiVersion (the second parameter) to the constructor of 
GroupMetadataManager in GroupMetadataManagerTest.scala.

> Offset commit failure after upgrading brokers past KIP-211/KAFKA-4682
> -
>
> Key: KAFKA-7403
> URL: https://issues.apache.org/jira/browse/KAFKA-7403
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 2.1.0
>Reporter: Jon Lee
>Priority: Major
>
> I am currently trying broker upgrade from 0.11 to 2.0 with some patches 
> including KIP-211/KAFKA-4682. After the upgrade, however, applications with 
> 0.10.2 Kafka clients failed with the following error:
> {code:java}
> 2018/09/11 19:34:52.814 ERROR Failed to commit offsets. Exiting. 
> org.apache.kafka.common.KafkaException: Unexpected error in commit: The 
> server experienced an unexpected error when processing the request at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator$OffsetCommitResponseHandler.handle(ConsumerCoordinator.java:784)
>  ~[kafka-clients-0.10.2.86.jar:?] at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator$OffsetCommitResponseHandler.handle(ConsumerCoordinator.java:722)
>  ~[kafka-clients-0.10.2.86.jar:?] at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:784)
>  ~[kafka-clients-0.10.2.86.jar:?] at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:765)
>  ~[kafka-clients-0.10.2.86.jar:?] at 
> org.apache.kafk