kumarpritam863 commented on code in PR #12372: URL: https://github.com/apache/iceberg/pull/12372#discussion_r1967989819
########## kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkTask.java: ########## @@ -46,26 +45,38 @@ public String version() { @Override public void start(Map<String, String> props) { this.config = new IcebergSinkConfig(props); + /* + catalog and committer are global resource and does not depend on the topic partition, + hence we should open this with the start call only and should only close these if the task is closed by connect + framework. + */ + catalog = CatalogUtils.loadCatalog(config); + committer = CommitterFactory.createCommitter(catalog, config, context); } @Override public void open(Collection<TopicPartition> partitions) { - Preconditions.checkArgument(catalog == null, "Catalog already open"); - Preconditions.checkArgument(committer == null, "Committer already open"); - - catalog = CatalogUtils.loadCatalog(config); - committer = CommitterFactory.createCommitter(config); - committer.start(catalog, config, context); + // We should be starting co-ordinator only the list of partitions has the zeroth partition. + if(committer.isCoordinator(partitions)) { Review Comment: If we want to move this logic to the committer then we need to somehow pass the the partitions information and that would require modification to the commuter method signatures ########## kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkTask.java: ########## @@ -46,26 +45,38 @@ public String version() { @Override public void start(Map<String, String> props) { this.config = new IcebergSinkConfig(props); + /* + catalog and committer are global resource and does not depend on the topic partition, + hence we should open this with the start call only and should only close these if the task is closed by connect + framework. + */ + catalog = CatalogUtils.loadCatalog(config); + committer = CommitterFactory.createCommitter(catalog, config, context); } @Override public void open(Collection<TopicPartition> partitions) { - Preconditions.checkArgument(catalog == null, "Catalog already open"); - Preconditions.checkArgument(committer == null, "Committer already open"); - - catalog = CatalogUtils.loadCatalog(config); - committer = CommitterFactory.createCommitter(config); - committer.start(catalog, config, context); + // We should be starting co-ordinator only the list of partitions has the zeroth partition. + if(committer.isCoordinator(partitions)) { + committer.startCoordinator(); + } + committer.syncLastCommittedOffsets(); } @Override public void close(Collection<TopicPartition> partitions) { - close(); + // We need to close worker here in every case to ensure exactly once otherwise this will lead to duplicate records. + committer.stopWorker(); + // Coordinator should only be closed if this received closed partitions has the partition which elected this task as coordinator. + if(committer.isCoordinator(partitions)) { Review Comment: If we want to move this logic to the committer then we need to somehow pass the the partitions information and that would require modification to the commuter method signatures -- 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