bryanck commented on code in PR #12372: URL: https://github.com/apache/iceberg/pull/12372#discussion_r1966910819
########## kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CommitterImpl.java: ########## @@ -57,49 +83,67 @@ public int compare(TopicPartition o1, TopicPartition o2) { } @Override - public void start(Catalog catalog, IcebergSinkConfig config, SinkTaskContext context) { - KafkaClientFactory clientFactory = new KafkaClientFactory(config.kafkaProps()); - + public boolean isLeader(Collection<TopicPartition> currentAssignedPartitions) { ConsumerGroupDescription groupDesc; try (Admin admin = clientFactory.createAdmin()) { groupDesc = KafkaUtils.consumerGroupDescription(config.connectGroupId(), admin); } - - if (groupDesc.state() == ConsumerGroupState.STABLE) { + if (groupDesc.state() == ConsumerGroupState.STABLE) { Collection<MemberDescription> members = groupDesc.members(); Set<TopicPartition> partitions = context.assignment(); if (isLeader(members, partitions)) { - LOG.info("Task elected leader, starting commit coordinator"); - Coordinator coordinator = new Coordinator(catalog, config, members, clientFactory, context); - coordinatorThread = new CoordinatorThread(coordinator); - coordinatorThread.start(); + membersWhenWorkerIsCoordinator = members; + return true; } } + return false; + } - LOG.info("Starting commit worker"); - SinkWriter sinkWriter = new SinkWriter(catalog, config); - worker = new Worker(config, clientFactory, sinkWriter, context); - worker.start(); + @Override + public void start(ResourceType resourceType) { + switch (resourceType) { + case WORKER: + startWorker(); + break; + case COORDINATOR: + startCoordinator(); + } } @Override public void save(Collection<SinkRecord> sinkRecords) { if (sinkRecords != null && !sinkRecords.isEmpty()) { + startWorker(); worker.save(sinkRecords); } processControlEvents(); } @Override - public void stop() { - if (worker != null) { - worker.stop(); - worker = null; + public void syncLastCommittedOffsets() { + Map<TopicPartition, Long> stableConsumerOffsets; + try (Admin admin = clientFactory.createAdmin()) { + ListConsumerGroupOffsetsResult response = + admin.listConsumerGroupOffsets( + config.connectGroupId(), new ListConsumerGroupOffsetsOptions().requireStable(true)); + stableConsumerOffsets = + response.partitionsToOffsetAndMetadata().get().entrySet().stream() + .filter(entry -> context.assignment().contains(entry.getKey())) + .collect(toMap(Map.Entry::getKey, entry -> entry.getValue().offset())); + } catch (InterruptedException | ExecutionException e) { + throw new ConnectException(e); } + context.offset(stableConsumerOffsets); + } - if (coordinatorThread != null) { - coordinatorThread.terminate(); - coordinatorThread = null; + @Override + public void stop(ResourceType resourceType) { Review Comment: Likewise, we can just call the type-specific methods directly instead. -- 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: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org