You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
I am trying to instantiate the Iceberg Kafka Connect, but the Kafka Cluster has specific ACLs that authorize the users with transactional ids following naming conventions with a specific prefix. I tried to set the property "iceberg.coordinator.transactional.suffix", but did not work properly.
Error Log
[2025-02-19 16:16:55,690] INFO [iceberg-sink-connector|task-1] [Producer clientId=c9e05765-a588-49fe-a894-e377f3319282, transactionalId=committer-txn-be446910-f4bb-4b08-a0e7-3985d8da1dc4-1] Transiting to fatal error state due to org.apache.kafka.common.errors.TransactionalIdAuthorizationException: Transactional Id authorization failed. (org.apache.kafka.clients.producer.internals.TransactionManager:455)
Analysis
Looking at the code in Channel.java the transactional ID is currently constructed by appending the transactional.suffix to the name.
Update CommitterImpl.java to use the new configuration parameter in the superclass constructor:
CommitterImpl(
SinkTaskContextcontext,
IcebergSinkConfigconfig,
KafkaClientFactoryclientFactory,
CoordinatorThreadFactorycoordinatorThreadFactory) {
// pass transient consumer group ID to which we never commit offsetssuper(
"committer",
IcebergSinkConfig.DEFAULT_CONTROL_GROUP_PREFIX + UUID.randomUUID(),
config.committerTransactionalId(),
config,
clientFactory);
Update Coordinator.java to use the new configuration parameter in the superclass constructor:
publicCoordinator(
Catalogcatalog,
IcebergSinkConfigconfig,
Collection<MemberDescription> members,
KafkaClientFactoryclientFactory) {
// pass consumer group ID to which we commit low watermark offsetssuper("coordinator",
config.controlGroupId() + "-coord",
config.coordinatorTransactionalId(),
config,
clientFactory);
Query engine
Kafka Connect
Willingness to contribute
I can contribute this improvement/feature independently
I would be willing to contribute this improvement/feature with guidance from the Iceberg community
I cannot contribute this improvement/feature at this time
The text was updated successfully, but these errors were encountered:
Feature Request / Improvement
I am trying to instantiate the Iceberg Kafka Connect, but the Kafka Cluster has specific ACLs that authorize the users with transactional ids following naming conventions with a specific prefix. I tried to set the property "iceberg.coordinator.transactional.suffix", but did not work properly.
Error Log
Analysis
Looking at the code in Channel.java the transactional ID is currently constructed by appending the transactional.suffix to the name.
The constructor of Channel class is invoked by its subclasses CommitterImpl.java and Coordinator.java.
Proposed Changes
Query engine
Kafka Connect
Willingness to contribute
The text was updated successfully, but these errors were encountered: