rainerschamm opened a new issue, #14818:
URL: https://github.com/apache/iceberg/issues/14818

   ### Apache Iceberg version
   
   1.9.2
   
   ### Query engine
   
   Kafka Connect
   
   ### Please describe the bug 🐞
   
   We see this error in the connect logs
   
   ```
   2025-12-11 07:43:52 INFO  [task-thread-iceberg-sink-connector2-0] 
ConsumerCoordinator:666 - [Consumer 
clientId=2accb103-ddfa-47a0-a6bd-5ac68a48e6bc, 
groupId=cg-control-5c866299-5107-4755-97d4-6fd9ff9d6eb4] Successfully joined 
group with generation Generation{generationId=1, 
memberId='2accb103-ddfa-47a0-a6bd-5ac68a48e6bc-4ac392c6-7224-4c94-b878-6218882faefa',
 protocol='range'}
   2025-12-11 07:43:52 INFO  [task-thread-iceberg-sink-connector2-0] 
ConsumerCoordinator:664 - [Consumer 
clientId=2accb103-ddfa-47a0-a6bd-5ac68a48e6bc, 
groupId=cg-control-5c866299-5107-4755-97d4-6fd9ff9d6eb4] Finished assignment 
for group at generation 1: 
{2accb103-ddfa-47a0-a6bd-5ac68a48e6bc-4ac392c6-7224-4c94-b878-6218882faefa=Assignment(partitions=[control-iceberg-0,
 control-iceberg-1, control-iceberg-2, control-iceberg-3, control-iceberg-4, 
control-iceberg-5])}
   2025-12-11 07:43:52 INFO  [kafka-coordinator-heartbeat-thread | 
cg-control-5c866299-5107-4755-97d4-6fd9ff9d6eb4] ConsumerCoordinator:878 - 
[Consumer clientId=2accb103-ddfa-47a0-a6bd-5ac68a48e6bc, 
groupId=cg-control-5c866299-5107-4755-97d4-6fd9ff9d6eb4] SyncGroup failed: The 
coordinator is not aware of this member. Need to re-join the group. Sent 
generation was Generation{generationId=1, 
memberId='2accb103-ddfa-47a0-a6bd-5ac68a48e6bc-4ac392c6-7224-4c94-b878-6218882faefa',
 protocol='range'}
   2025-12-11 07:43:52 INFO  [kafka-coordinator-heartbeat-thread | 
cg-control-5c866299-5107-4755-97d4-6fd9ff9d6eb4] ConsumerCoordinator:1056 - 
[Consumer clientId=2accb103-ddfa-47a0-a6bd-5ac68a48e6bc, 
groupId=cg-control-5c866299-5107-4755-97d4-6fd9ff9d6eb4] Resetting generation 
and member id due to: encountered UNKNOWN_MEMBER_ID from SYNC_GROUP response
   2025-12-11 07:43:52 INFO  [kafka-coordinator-heartbeat-thread | 
cg-control-5c866299-5107-4755-97d4-6fd9ff9d6eb4] ConsumerCoordinator:1103 - 
[Consumer clientId=2accb103-ddfa-47a0-a6bd-5ac68a48e6bc, 
groupId=cg-control-5c866299-5107-4755-97d4-6fd9ff9d6eb4] Request joining group 
due to: encountered UNKNOWN_MEMBER_ID from SYNC_GROUP response
   ```
   
   and the corresponding error in the broker logs:
   
   ```
   2025-12-11 07:44:52 INFO  [group-coordinator-event-processor-3] 
GroupMetadataManager:4675 - [GroupCoordinator id=0 topic=__consumer_offsets 
partition=40] Dynamic member with unknown member id joins group 
cg-control-5c866299-5107-4755-97d4-6fd9ff9d6eb4 in Empty state. Created a new 
member id 
2accb103-ddfa-47a0-a6bd-5ac68a48e6bc-75a7f486-776c-4c9e-8063-f8dd6a9bea15 and 
requesting the member to rejoin with this id.
   2025-12-11 07:45:02 INFO  [group-coordinator-event-processor-1] 
GroupMetadataManager:5022 - [GroupCoordinator id=0 topic=__consumer_offsets 
partition=40] Pending member 
2accb103-ddfa-47a0-a6bd-5ac68a48e6bc-75a7f486-776c-4c9e-8063-f8dd6a9bea15 in 
group cg-control-5c866299-5107-4755-97d4-6fd9ff9d6eb4 has been removed after 
session timeout expiration.
   ```
   
   So 10 seconds after join request the broker removed the member since the 
consumer never sent  a heartbeat.
   
   We experimented with some code changes, and increasing the poll duration in 
the Worker to the same value used in the Coordinator, the error went away 
completely;
   
   Coordinator.java line number 68:
   ```
     private static final Duration POLL_DURATION = Duration.ofSeconds(1);
   ```
   Coordinator.java line number 108:
   ```
      consumeAvailable(POLL_DURATION);
   ```
   
   In the Worker.java we changed line 65 from:
   ```
       consumeAvailable(Duration.ZERO);
   ```
   to:
   ```
       consumeAvailable(Duration.ofSeconds(1));
   ```
   
   These are some AI responses; which confirm our own finding while testing:
   
   ```
   Why poll(Duration.ZERO) breaks Kafka consumer groups
   1. No Heartbeats
   
   The heartbeat only happens inside poll(), when Kafka decides it can send 
heartbeats.
   With poll(Duration.ZERO): Kafka cannot block.
   It often does not send heartbeats effectively
   
   The broker kicks the consumer out:
   coordinator is not aware of this member UnknownMemberIdException
   
   2. Rebalance Never Completes
   
   During a rebalance, the consumer must call poll() with enough duration to 
allow the coordinator to finish assignment.
   With zero duration, the join request cannot complete before the session 
timeout.
   ```
   
   Our proposed fix is to simply increase the poll duration from ZERO to 1 
second.
    
   
   
   ### Willingness to contribute
   
   - [x] I can contribute a fix for this bug independently
   - [ ] I would be willing to contribute a fix for this bug with guidance from 
the Iceberg community
   - [ ] I cannot contribute a fix for this bug at this time


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to