-
Notifications
You must be signed in to change notification settings - Fork 74
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Replace sync Kafka Producers with confluent_kafka one #765
Merged
Merged
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Empty file.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,178 @@ | ||
""" | ||
Copyright (c) 2023 Aiven Ltd | ||
See LICENSE for details | ||
""" | ||
|
||
from __future__ import annotations | ||
|
||
from collections.abc import Iterable | ||
from concurrent.futures import Future | ||
from confluent_kafka import TopicPartition | ||
from confluent_kafka.admin import ( | ||
AdminClient, | ||
BrokerMetadata, | ||
ClusterMetadata, | ||
ConfigResource, | ||
ConfigSource, | ||
NewTopic, | ||
OffsetSpec, | ||
ResourceType, | ||
TopicMetadata, | ||
) | ||
from confluent_kafka.error import KafkaException | ||
from karapace.constants import TOPIC_CREATION_TIMEOUT_S | ||
from karapace.kafka.common import ( | ||
_KafkaConfigMixin, | ||
raise_from_kafkaexception, | ||
single_futmap_result, | ||
UnknownTopicOrPartitionError, | ||
) | ||
from typing import Container | ||
|
||
|
||
class KafkaAdminClient(_KafkaConfigMixin, AdminClient): | ||
def new_topic( | ||
self, | ||
name: str, | ||
*, | ||
num_partitions: int = 1, | ||
replication_factor: int = 1, | ||
config: dict[str, str] | None = None, | ||
request_timeout: float = TOPIC_CREATION_TIMEOUT_S, | ||
) -> NewTopic: | ||
new_topic = NewTopic( | ||
topic=name, | ||
num_partitions=num_partitions, | ||
replication_factor=replication_factor, | ||
config=config if config is not None else {}, | ||
) | ||
self.log.info("Creating new topic %s with replication factor %s", new_topic, replication_factor) | ||
futmap: dict[str, Future] = self.create_topics([new_topic], request_timeout=request_timeout) | ||
try: | ||
single_futmap_result(futmap) | ||
return new_topic | ||
except KafkaException as exc: | ||
raise_from_kafkaexception(exc) | ||
|
||
def update_topic_config(self, name: str, config: dict[str, str]) -> None: | ||
self.log.info("Updating topic '%s' configuration with %s", name, config) | ||
futmap = self.alter_configs([ConfigResource(ResourceType.TOPIC, name, set_config=config)]) | ||
try: | ||
single_futmap_result(futmap) | ||
except KafkaException as exc: | ||
raise_from_kafkaexception(exc) | ||
|
||
def delete_topic(self, name: str) -> None: | ||
self.log.info("Deleting topic '%s'", name) | ||
futmap = self.delete_topics([name]) | ||
try: | ||
single_futmap_result(futmap) | ||
except KafkaException as exc: | ||
raise_from_kafkaexception(exc) | ||
|
||
def cluster_metadata(self, topics: Iterable[str] | None = None) -> dict: | ||
"""Fetch cluster metadata and topic information for given topics or all topics if not given. | ||
|
||
Using the `list_topics` method of the `AdminClient`, as this actually provides | ||
metadata for the entire cluster, not just topics, as suggested by the name. | ||
|
||
The topics filter is only applied _after_ fetching the cluster metadata, | ||
due to `list_topics` only accepting a single topic as a filter. | ||
""" | ||
self.log.info("Fetching cluster metadata with topic filter: %s", topics) | ||
cluster_metadata: ClusterMetadata = self.list_topics() | ||
topics_metadata: dict[str, TopicMetadata] = cluster_metadata.topics | ||
brokers_metadata: dict[int, BrokerMetadata] = cluster_metadata.brokers | ||
|
||
if topics is not None and any(topic not in topics_metadata.keys() for topic in topics): | ||
raise UnknownTopicOrPartitionError() | ||
|
||
topics_data: dict[str, dict] = {} | ||
for topic, topic_metadata in topics_metadata.items(): | ||
if topics is not None and topic not in topics: | ||
continue | ||
|
||
partitions_data = [] | ||
for partition_id, partition_metadata in topic_metadata.partitions.items(): | ||
partition_data = { | ||
"partition": partition_id, | ||
"leader": partition_metadata.leader, | ||
"replicas": [ | ||
{ | ||
"broker": replica_id, | ||
"leader": replica_id == partition_metadata.leader, | ||
"in_sync": replica_id in partition_metadata.isrs, | ||
} | ||
for replica_id in partition_metadata.replicas | ||
], | ||
} | ||
partitions_data.append(partition_data) | ||
|
||
topics_data[topic] = {"partitions": partitions_data} | ||
|
||
return {"topics": topics_data, "brokers": list(brokers_metadata.keys())} | ||
|
||
def get_topic_config( | ||
self, | ||
name: str, | ||
config_name_filter: Container[str] | None = None, | ||
config_source_filter: Container[ConfigSource] | None = None, | ||
) -> dict[str, str]: | ||
"""Fetches, filters and returns topic configuration. | ||
|
||
The two filters, `config_name_filter` and `config_source_filter` work together | ||
so if a config entry matches either of them, it'll be returned. | ||
If a filter is not provided (ie. is `None`), it'll act as if matching all | ||
config entries. | ||
""" | ||
self.log.info( | ||
"Fetching config for topic '%s' with name filter %s and source filter %s", | ||
name, | ||
config_name_filter, | ||
config_source_filter, | ||
) | ||
futmap: dict[ConfigResource, Future] = self.describe_configs([ConfigResource(ResourceType.TOPIC, name)]) | ||
try: | ||
topic_configs = single_futmap_result(futmap) | ||
except KafkaException as exc: | ||
raise_from_kafkaexception(exc) | ||
|
||
config: dict[str, str] = {} | ||
for config_name, config_entry in topic_configs.items(): | ||
matches_name_filter: bool = config_name_filter is None or config_name in config_name_filter | ||
matches_source_filter: bool = ( | ||
config_source_filter is None or ConfigSource(config_entry.source) in config_source_filter | ||
) | ||
|
||
if matches_name_filter or matches_source_filter: | ||
config[config_name] = config_entry.value | ||
|
||
return config | ||
|
||
def get_offsets(self, topic: str, partition_id: int) -> dict[str, int]: | ||
"""Returns the beginning and end offsets for a topic partition. | ||
|
||
Making two separate requests for beginning and end offsets, due to the | ||
`AdminClient.list_offsets` behaviour: it expects a dictionary of topic | ||
partitions as keys, thus unable to fetch different values in one request | ||
for the same topic and partition. | ||
""" | ||
try: | ||
self.log.info("Fetching latest offset for topic '%s' partition %s", topic, partition_id) | ||
latest_offset_futmap: dict[TopicPartition, Future] = self.list_offsets( | ||
{ | ||
TopicPartition(topic, partition_id): OffsetSpec.latest(), | ||
} | ||
) | ||
endoffset = single_futmap_result(latest_offset_futmap) | ||
|
||
self.log.info("Fetching earliest offset for topic '%s' partition %s", topic, partition_id) | ||
earliest_offset_futmap: dict[TopicPartition, Future] = self.list_offsets( | ||
{ | ||
TopicPartition(topic, partition_id): OffsetSpec.earliest(), | ||
} | ||
) | ||
startoffset = single_futmap_result(earliest_offset_futmap) | ||
except KafkaException as exc: | ||
raise_from_kafkaexception(exc) | ||
return {"beginning_offset": startoffset.offset, "end_offset": endoffset.offset} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Note for posterity, this was discussed out of bands: header keys cannot be null, so it's correct to simplify handling here.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
why doing that instead of asserting that isn't
None
?In that way if the assumptions isn't true we are skipping a record, IMO we should fail if an assumption is violated rather than proving that currently isn't violated and if the implementation changes in the future we have a skip instead of an error