You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Laurynas Butkus (Jira)" <ji...@apache.org> on 2021/08/23 16:46:00 UTC

[jira] [Created] (KAFKA-13223) Idempotent producer error with Kraft

Laurynas Butkus created KAFKA-13223:
---------------------------------------

             Summary: Idempotent producer error with Kraft 
                 Key: KAFKA-13223
                 URL: https://issues.apache.org/jira/browse/KAFKA-13223
             Project: Kafka
          Issue Type: Bug
          Components: kraft, producer 
            Reporter: Laurynas Butkus


I get an error *"The broker does not support INIT_PRODUCER_ID"* if I try to produce a message idempotence enabled.

Result:
{code:java}
➜  ./bin/kafka-console-producer.sh --bootstrap-server localhost:9092 --topic test --request-required-acks -1 --producer-property enable.idempotence=true
>test
>[2021-08-23 19:40:33,356] ERROR [Producer clientId=console-producer] Aborting producer batches due to fatal error (org.apache.kafka.clients.producer.internals.Sender)
org.apache.kafka.common.errors.UnsupportedVersionException: The broker does not support INIT_PRODUCER_ID
[2021-08-23 19:40:33,358] ERROR Error when sending message to topic test with key: null, value: 4 bytes with error: (org.apache.kafka.clients.producer.internals.ErrorLoggingCallback)
org.apache.kafka.common.errors.UnsupportedVersionException: The broker does not support INIT_PRODUCER_ID
{code}
 

It works fine with idempotence disabled. Also it works fine if using zookeeper.

Tested with altered docker image: 
{code:java}
FROM confluentinc/cp-kafka:6.2.0

RUN sed -i '/KAFKA_ZOOKEEPER_CONNECT/d' /etc/confluent/docker/configure && \
# Docker workaround: Ignore cub zk-ready
sed -i 's/cub zk-ready/echo ignore zk-ready/' /etc/confluent/docker/ensure && \
# KRaft required step: Format the storage directory with a new cluster ID
echo "kafka-storage format --ignore-formatted -t $(kafka-storage random-uuid) -c /etc/kafka/kafka.properties" >> /etc/confluent/docker/ensure
{code}
docker-compose.yml
{code:java}
version: '3.4'

services:
  kafka:
    build: kafka
    restart: unless-stopped
    environment:
      ALLOW_PLAINTEXT_LISTENER: "yes"
      KAFKA_HEAP_OPTS: -Xms256m -Xmx256m
      LOG4J_LOGGER_KAFKA: "WARN"
      KAFKA_BROKER_ID: 1
      KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: 'CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT,PLAINTEXT_HOST:PLAINTEXT'
      KAFKA_ADVERTISED_LISTENERS: 'PLAINTEXT://kafka:29092,PLAINTEXT_HOST://127.0.0.1:9092'
      KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1
      KAFKA_GROUP_INITIAL_REBALANCE_DELAY_MS: 0
      KAFKA_TRANSACTION_STATE_LOG_MIN_ISR: 1
      KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR: 1
      KAFKA_PROCESS_ROLES: 'broker,controller'
      KAFKA_NODE_ID: 1
      KAFKA_CONTROLLER_QUORUM_VOTERS: '1@kafka:29093'
      KAFKA_LISTENERS: 'PLAINTEXT://kafka:29092,CONTROLLER://kafka:29093,PLAINTEXT_HOST://:9092'
      KAFKA_INTER_BROKER_LISTENER_NAME: 'PLAINTEXT'
      KAFKA_CONTROLLER_LISTENER_NAMES: 'CONTROLLER'
      KAFKA_LOG_DIRS: '/tmp/kraft-combined-logs'
    ports:
      - "127.0.0.1:9092:9092/tcp"
    command: "/etc/confluent/docker/run"
{code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)