From 8449c229e751de0d6ba8b8e37368142a67bfc58f Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Wed, 3 Jul 2019 17:49:08 +0200 Subject: [PATCH 01/94] Add consumer and producer method to write to and read from file --- esque/cli/commands.py | 57 +++++++++++++++++++++++++++++-- esque/clients.py | 78 +++++++++++++++++++++++++++++++++++++------ esque/message.py | 16 +++++++++ 3 files changed, 138 insertions(+), 13 deletions(-) create mode 100644 esque/message.py diff --git a/esque/cli/commands.py b/esque/cli/commands.py index 54d02d3e..8172665e 100644 --- a/esque/cli/commands.py +++ b/esque/cli/commands.py @@ -1,3 +1,5 @@ +import os +import random from time import sleep import click @@ -12,7 +14,7 @@ from esque.cli.output import bold, pretty, get_output_topic_diffs, get_output_new_topics from esque.clients import Consumer, Producer from esque.cluster import Cluster -from esque.config import PING_TOPIC, Config +from esque.config import PING_TOPIC, Config, PING_GROUP_ID from esque.consumergroup import ConsumerGroupController from esque.errors import ( ConsumerGroupDoesNotExistException, @@ -164,6 +166,26 @@ def apply(state: State, file: str): click.echo("No new topics to create.") +@esque.command(help="Consume messages from a topic to file.") +@click.option("-t", "--topic", help="Source Topic name", required=True) +@click.option("-f", "--file", help="Destination file path", required=True) +@click.argument("amount", required=True) +def consume(topic:str, file: str, amount: str): + your_letters = 'abcdefghijklmnopqrstuvwxyz' + group_id = topic.join((random.choice(your_letters) for i in range(7))) + + consumer = Consumer(group_id, topic, "earliest") + consumer.consume_to_file(file, int(amount)) + + +@esque.command(help="Produce messages from a file to topic.") +@click.option("-f", "--file", help="Source file path", required=True) +@click.option("-t", "--topic", help="Destination Topic name", required=True) +def produce(topic:str, file: str): + producer = Producer() + producer.produce_from_file(file, topic) + + @delete.command("topic") @click.argument( "topic-name", required=True, type=click.STRING, autocompletion=list_topics @@ -261,6 +283,35 @@ def get_topics(state, topic): click.echo(topic.name) +@esque.command("transfer") +@click.argument("topic", required=True) +@click.option("-f", "--from", "from_context", help="Source Context", required=True) +@click.option("-t", "--to", "to_context", help="Destination context", required=True) +@click.option("-n", "--numbers", help="Number of messages", type=click.INT, required=True) +@click.option('--last/--first', default=False) +def transfer(topic: str, from_context: str, to_context: str, numbers: int, last: bool): + your_letters = 'abcdefghijklmnopqrstuvwxyz' + group_id = topic.join((random.choice(your_letters) for i in range(7))) + file_name = group_id + + ctx(from_context) + print("Start consuming from source context " + from_context) + consumer = Consumer(group_id, topic, last) + number_consumed_messages = consumer.consume_to_file(file_name, int(numbers)) + print(str(number_consumed_messages) + " messages consumed successfully.") + print("Ready to produce to context " + to_context + " and target topic " + topic + "\n") + + if ensure_approval("Do you want to proceed?", no_verify=state.no_verify): + ctx(to_context) + producer = Producer() + number_produced_messages = producer.produce_from_file(file_name, topic) + print(str(number_produced_messages) + " messages successfully produced.") + + os.remove(file_name) + + + + @esque.command("ping", help="Tests the connection to the kafka cluster.") @click.option("-t", "--times", help="Number of pings.", default=10) @click.option("-w", "--wait", help="Seconds to wait between pings.", default=1) @@ -275,12 +326,12 @@ def ping(state, times, wait): click.echo("Topic already exists.") producer = Producer() - consumer = Consumer() + consumer = Consumer(PING_GROUP_ID, PING_TOPIC, "latest") click.echo(f"Ping with {state.cluster.bootstrap_servers}") for i in range(times): - producer.produce_ping() + producer.produce_ping(PING_TOPIC) _, delta = consumer.consume_ping() deltas.append(delta) click.echo(f"m_seq={i} time={delta:.2f}ms") diff --git a/esque/clients.py b/esque/clients.py index af69cdc2..89b74442 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -1,23 +1,28 @@ +import pickle from typing import Optional, Tuple import click import confluent_kafka import pendulum -from confluent_kafka import TopicPartition +from confluent_kafka import TopicPartition, Message -from esque.config import Config, PING_GROUP_ID, PING_TOPIC -from esque.errors import raise_for_kafka_error, raise_for_message +from esque.config import Config +from esque.errors import raise_for_kafka_error, raise_for_message, MessageEmptyException, KafkaException from esque.helpers import delivery_callback, delta_t +from esque.message import decode_message DEFAULT_RETENTION_MS = 7 * 24 * 60 * 60 * 1000 - class Consumer: - def __init__(self): + def __init__(self, group_id: str, topic_name: str, last: bool): + offset_reset = "earliest" + if last: + offset_reset = "latest" + self._config = Config().create_confluent_config() self._config.update( { - "group.id": PING_GROUP_ID, + "group.id": group_id, "error_cb": raise_for_kafka_error, # We need to commit offsets manually once we"re sure it got saved # to the sink @@ -25,11 +30,11 @@ def __init__(self): "enable.partition.eof": False, # We need this to start at the last committed offset instead of the # latest when subscribing for the first time - "default.topic.config": {"auto.offset.reset": "latest"}, + "default.topic.config": {"auto.offset.reset": offset_reset}, } ) self._consumer = confluent_kafka.Consumer(self._config) - self._assign_exact_partitions(PING_TOPIC) + self._assign_exact_partitions(topic_name) def consume_ping(self) -> Optional[Tuple[str, int]]: msg = self._consumer.consume(timeout=10)[0] @@ -40,6 +45,38 @@ def consume_ping(self) -> Optional[Tuple[str, int]]: delta_sent = pendulum.now() - msg_sent_at return msg.key(), delta_sent.microseconds / 1000 + def consume_to_file(self, file_path: str, amount: int) -> int: + counter = 0 + with open(file_path, "wb") as file: + while counter < amount: + try: + message = self._consume_single_message() + except KafkaException as ex: + print("An error occurred: " + ex.message) + continue + + decoded_message = decode_message(message) + serializable_message = {"key": decoded_message.key, "value": decoded_message.value} + pickle.dump(serializable_message, file) + counter += 1 + + return counter + + def _consume_single_message(self) -> Message: + poll_limit = 10 + counter = 0 + try: + while counter < poll_limit: + message = self._consumer.poll(timeout=10) + if message is None: + counter += 1 + continue + raise_for_message(message) + return message + except KafkaException as ex: + print("An error occurred: " + ex.message) + + def _assign_exact_partitions(self, topic: str) -> None: self._consumer.assign([TopicPartition(topic=topic, partition=0, offset=0)]) @@ -52,10 +89,10 @@ def __init__(self): ) self._producer = confluent_kafka.Producer(self._config) - def produce_ping(self): + def produce_ping(self, topic_name: str): start = pendulum.now() self._producer.produce( - topic=PING_TOPIC, key=str(0), value=str(pendulum.now().timestamp()) + topic=topic_name, key=str(0), value=str(pendulum.now().timestamp()) ) while True: left_messages = self._producer.flush(1) @@ -64,3 +101,24 @@ def produce_ping(self): click.echo( f"{delta_t(start)} | Still {left_messages} messages left, flushing..." ) + + def produce_from_file(self, file_path: str, topic_name: str) -> int: + with open(file_path, "rb") as file: + counter = 0 + while True: + try: + record = pickle.load(file) + except EOFError: + break + + self._producer.produce( + topic=topic_name, key=record["key"], value=record["value"] + ) + counter += 1 + + while True: + left_messages = self._producer.flush(1) + if left_messages == 0: + break + + return counter diff --git a/esque/message.py b/esque/message.py new file mode 100644 index 00000000..40b13a3f --- /dev/null +++ b/esque/message.py @@ -0,0 +1,16 @@ +from confluent_kafka.cimpl import Message + + +class DecodedMessage: + def __init__(self, key: str, value: str): + self.key = key + self.value = value + + +def decode_message(message: Message) -> DecodedMessage: + decoded_key = message.key().decode("utf-8") + decoded_value = message.value().decode("utf-8") + + return DecodedMessage(decoded_key, decoded_value) + + From d835438c20cd9f31a5c47162a83750b642a22d84 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Fri, 5 Jul 2019 16:10:38 +0200 Subject: [PATCH 02/94] Add schema registry to config --- config/sample_config.cfg | 2 ++ docker-compose.yml | 28 +++++++++++++++++++++++++ esque/cli/commands.py | 44 ++++++++++++++++++++++++---------------- esque/cli/output.py | 6 +++++- esque/config.py | 5 +++++ 5 files changed, 66 insertions(+), 19 deletions(-) diff --git a/config/sample_config.cfg b/config/sample_config.cfg index 4d134d11..bcbe2d8d 100644 --- a/config/sample_config.cfg +++ b/config/sample_config.cfg @@ -5,8 +5,10 @@ current = docker bootstrap_hosts = localhost bootstrap_port = 9092 security_protocol = LOCAL +schema_registry = localhost:8081 [Context.docker] bootstrap_hosts = kafka bootstrap_port = 9093 security_protocol = PLAINTEXT +schema_registry = schema-registry:8081 diff --git a/docker-compose.yml b/docker-compose.yml index 877f3ceb..08f173ba 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -37,6 +37,34 @@ services: KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 KAFKA_GROUP_INITIAL_REBALANCE_DELAY_MS: 0 + schema_registry_source: + image: confluentinc/cp-schema-registry:5.2.2 + container_name: schema_registry_source + environment: + - SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL=zookeeper:2181 + - SCHEMA_REGISTRY_HOST_NAME=schema-registry + - SCHEMA_REGISTRY_LISTENERS=http://0.0.0.0:8081 + - SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS=PLAINTEXT://kafka:9092 + links: + - zookeeper + - kafka + ports: + - 8081:8081 + + schema_registry_target: + image: confluentinc/cp-schema-registry:5.2.2 + container_name: schema_registry_target + environment: + - SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL=zookeeper:2181 + - SCHEMA_REGISTRY_HOST_NAME=schema-registry + - SCHEMA_REGISTRY_LISTENERS=http://0.0.0.0:8082 + - SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS=PLAINTEXT://kafka:9092 + links: + - zookeeper + - kafka + ports: + - 8082:8081 + esque: build: . volumes: diff --git a/esque/cli/commands.py b/esque/cli/commands.py index 8172665e..5446458b 100644 --- a/esque/cli/commands.py +++ b/esque/cli/commands.py @@ -1,5 +1,6 @@ import os import random +import time from time import sleep import click @@ -11,7 +12,7 @@ from esque.broker import Broker from esque.cli.helpers import ensure_approval from esque.cli.options import State, no_verify_option, pass_state -from esque.cli.output import bold, pretty, get_output_topic_diffs, get_output_new_topics +from esque.cli.output import bold, pretty, get_output_topic_diffs, get_output_new_topics, blue_bold, green_bold from esque.clients import Consumer, Producer from esque.cluster import Cluster from esque.config import PING_TOPIC, Config, PING_GROUP_ID @@ -285,33 +286,40 @@ def get_topics(state, topic): @esque.command("transfer") @click.argument("topic", required=True) -@click.option("-f", "--from", "from_context", help="Source Context", required=True) -@click.option("-t", "--to", "to_context", help="Destination context", required=True) +@click.option("-f", "--from", "from_context", help="Source Context", type=click.STRING, required=True) +@click.option("-t", "--to", "to_context", help="Destination context", type=click.STRING, required=True) @click.option("-n", "--numbers", help="Number of messages", type=click.INT, required=True) @click.option('--last/--first', default=False) -def transfer(topic: str, from_context: str, to_context: str, numbers: int, last: bool): - your_letters = 'abcdefghijklmnopqrstuvwxyz' - group_id = topic.join((random.choice(your_letters) for i in range(7))) - file_name = group_id - - ctx(from_context) - print("Start consuming from source context " + from_context) +@pass_state +def transfer(state: State, topic: str, from_context: str, to_context: str, numbers: int, last: bool): + current_timestamp_milliseconds = int(round(time.time() * 1000)) + temp_name = topic + '_' + str(current_timestamp_milliseconds) + group_id = "group_for_" + temp_name + file_name = "temp-file_" + temp_name + + state.config.context_switch(from_context) + click.echo("\nStart consuming from source context " + blue_bold(from_context)) consumer = Consumer(group_id, topic, last) number_consumed_messages = consumer.consume_to_file(file_name, int(numbers)) - print(str(number_consumed_messages) + " messages consumed successfully.") - print("Ready to produce to context " + to_context + " and target topic " + topic + "\n") + click.echo(blue_bold(str(number_consumed_messages)) + " messages consumed successfully.") + click.echo("\nReady to produce to context " + blue_bold(to_context) + " and target topic " + blue_bold(topic)) - if ensure_approval("Do you want to proceed?", no_verify=state.no_verify): - ctx(to_context) + if ensure_approval("Do you want to proceed?\n", no_verify=state.no_verify): + state.config.context_switch(to_context) producer = Producer() number_produced_messages = producer.produce_from_file(file_name, topic) - print(str(number_produced_messages) + " messages successfully produced.") + click.echo( + green_bold(str(number_produced_messages)) + + " messages successfully produced to context " + + green_bold(to_context) + + " and topic " + + green_bold(topic) + + "." + ) os.remove(file_name) - - @esque.command("ping", help="Tests the connection to the kafka cluster.") @click.option("-t", "--times", help="Number of pings.", default=10) @click.option("-w", "--wait", help="Seconds to wait between pings.", default=1) @@ -326,7 +334,7 @@ def ping(state, times, wait): click.echo("Topic already exists.") producer = Producer() - consumer = Consumer(PING_GROUP_ID, PING_TOPIC, "latest") + consumer = Consumer(PING_GROUP_ID, PING_TOPIC, True) click.echo(f"Ping with {state.cluster.bootstrap_servers}") diff --git a/esque/cli/output.py b/esque/cli/output.py index b11da850..af40898f 100644 --- a/esque/cli/output.py +++ b/esque/cli/output.py @@ -180,7 +180,11 @@ def bold(s: str) -> str: def blue_bold(s: str) -> str: - return click.style(s, fg="blue", bold=True) + return bold(click.style(s, fg="blue")) + + +def green_bold(s: str) -> str: + return bold(click.style(s, fg="green")) STYLE_MAPPING = { diff --git a/esque/config.py b/esque/config.py index 0a1c312d..c2ef0ead 100644 --- a/esque/config.py +++ b/esque/config.py @@ -83,6 +83,11 @@ def bootstrap_hosts(self) -> List[str]: config_dict = self.current_context_dict return config_dict["bootstrap_hosts"].split(",") + @property + def schema_registry(self) -> str: + config_dict = self.current_context_dict + return config_dict["schema_registry"] + @property def bootstrap_servers(self): if self.bootstrap_domain: From 064876f21d2d703d52194afd89f793a9e826c35a Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sun, 7 Jul 2019 23:30:03 +0200 Subject: [PATCH 03/94] Add serializer for avro messages --- esque/avromessage.py | 70 +++++++++++++++++++++++++++++++++++++++++ esque/clients.py | 13 ++++---- esque/message.py | 19 +++++++++-- esque/schemaregistry.py | 43 +++++++++++++++++++++++++ setup.py | 2 ++ 5 files changed, 138 insertions(+), 9 deletions(-) create mode 100644 esque/avromessage.py create mode 100644 esque/schemaregistry.py diff --git a/esque/avromessage.py b/esque/avromessage.py new file mode 100644 index 00000000..4d027a52 --- /dev/null +++ b/esque/avromessage.py @@ -0,0 +1,70 @@ +import json +import pathlib +import pickle +import struct +from io import BytesIO +from typing import Optional, Tuple, Dict, BinaryIO +import itertools as it + +import fastavro +from confluent_kafka.cimpl import Message + +from esque.schemaregistry import SchemaRegistryClient + + +class DecodedAvroMessage: + def __init__(self, key: Optional[Dict], value: Optional[Dict], key_schema_id: int, value_schema_id: int): + self.key = key + self.value = value + self.key_schema_id = key_schema_id + self.value_schema_id = value_schema_id + + +class AvroSerializer: + + def __init__(self, schema_registry_client: SchemaRegistryClient, working_dir: pathlib.Path): + self.working_dir = working_dir + self.schema_registry_client = schema_registry_client + self.current_key_schema_id = None + self.current_value_schema_id = None + self.schema_version = it.count(1) + + def serialize(self, message: Message, file: BinaryIO): + key_schema_id, decoded_key = self.decode_bytes(message.key()) + value_schema_id, decoded_value = self.decode_bytes(message.value()) + decoded_message = DecodedAvroMessage(decoded_key, decoded_value, key_schema_id, value_schema_id) + + if self.schema_changed(decoded_message): + schema_dir_name = f"{next(self.schema_version):04}_{key_schema_id}_{value_schema_id}" + directory = self.working_dir / schema_dir_name + directory.mkdir() + + (directory / "key_schema.avsc").write_text( + json.dumps(self.schema_registry_client.get_schema_from_id(key_schema_id).original_schema) + ) + + (directory / "value_schema.avsc").write_text( + json.dumps(self.schema_registry_client.get_schema_from_id(value_schema_id).original_schema) + ) + + serializable_message = {"key": decoded_key, "value": decoded_value} + pickle.dump(serializable_message, file) + + def decode_bytes(self, raw_data: Optional[bytes]) -> Tuple[int, Optional[Dict]]: + if raw_data is None: + return -1, None + + with BytesIO(raw_data) as fake_stream: + schema_id = extract_schema_id(fake_stream.read(5)) + parsed_schema = self.schema_registry_client.get_schema_from_id(schema_id).parsed_schema + record = fastavro.schemaless_reader(fake_stream, parsed_schema) + return schema_id, record + + def schema_changed(self, decoded_message: DecodedAvroMessage) -> bool: + return (self.current_value_schema_id != decoded_message.value_schema_id and decoded_message.value is not None) \ + or self.current_key_schema_id != decoded_message.key_schema_id + + +def extract_schema_id(message: bytes) -> int: + _, schema_id = struct.unpack(">bI", message[:5]) + return schema_id diff --git a/esque/clients.py b/esque/clients.py index 89b74442..01c622c2 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -1,3 +1,4 @@ +import pathlib import pickle from typing import Optional, Tuple @@ -7,9 +8,9 @@ from confluent_kafka import TopicPartition, Message from esque.config import Config -from esque.errors import raise_for_kafka_error, raise_for_message, MessageEmptyException, KafkaException +from esque.errors import raise_for_kafka_error, raise_for_message, KafkaException from esque.helpers import delivery_callback, delta_t -from esque.message import decode_message +from esque.avromessage import AvroSerializer DEFAULT_RETENTION_MS = 7 * 24 * 60 * 60 * 1000 @@ -45,9 +46,9 @@ def consume_ping(self) -> Optional[Tuple[str, int]]: delta_sent = pendulum.now() - msg_sent_at return msg.key(), delta_sent.microseconds / 1000 - def consume_to_file(self, file_path: str, amount: int) -> int: + def consume_to_file(self, working_dir: pathlib.Path, serializer: AvroSerializer, amount: int) -> int: counter = 0 - with open(file_path, "wb") as file: + with (working_dir / "data").open("wb") as file: while counter < amount: try: message = self._consume_single_message() @@ -55,9 +56,7 @@ def consume_to_file(self, file_path: str, amount: int) -> int: print("An error occurred: " + ex.message) continue - decoded_message = decode_message(message) - serializable_message = {"key": decoded_message.key, "value": decoded_message.value} - pickle.dump(serializable_message, file) + serializer.serialize(message, file) counter += 1 return counter diff --git a/esque/message.py b/esque/message.py index 40b13a3f..73626933 100644 --- a/esque/message.py +++ b/esque/message.py @@ -1,3 +1,6 @@ +import pickle +from typing import BinaryIO + from confluent_kafka.cimpl import Message @@ -7,10 +10,22 @@ def __init__(self, key: str, value: str): self.value = value +class Serializer: + + def serialize(self, message: Message, file: BinaryIO): + pass + + +class JsonSerializer: + + def serialize(self, message: Message, file: BinaryIO): + decoded_message = decode_message(message) + serializable_message = {"key": decoded_message.key, "value": decoded_message.value} + pickle.dump(serializable_message, file) + + def decode_message(message: Message) -> DecodedMessage: decoded_key = message.key().decode("utf-8") decoded_value = message.value().decode("utf-8") return DecodedMessage(decoded_key, decoded_value) - - diff --git a/esque/schemaregistry.py b/esque/schemaregistry.py new file mode 100644 index 00000000..f444d12a --- /dev/null +++ b/esque/schemaregistry.py @@ -0,0 +1,43 @@ +import json +import struct +from collections import namedtuple +from io import BytesIO +from typing import Dict, Optional + +import fastavro +import requests +from functools import lru_cache + +SchemaPair = namedtuple("SchemaPair", ["original_schema", "parsed_schema"]) + + +class SchemaRegistryClient: + + def __init__(self, schema_registry_uri: str): + self.schema_registry_uri = schema_registry_uri + + @lru_cache(maxsize=100) + def get_schema_from_id(self, schema_id: int) -> SchemaPair: + return self.get_schema_from_server(schema_id, self.schema_registry_uri) + + def get_schema_from_server(self, schema_id: int, server_url: str) -> SchemaPair: + url = "{server_url}/schemas/ids/{schema_id}" + response = requests.get(url.format(**locals())) + response.raise_for_status() + schema: Dict = json.loads(response.json()["schema"]) + print("Schema received for id {schema_id}\n{schema}".format(**locals())) + return SchemaPair(schema, fastavro.schema.parse_schema(schema)) + + def get_schema_for_bytes(self, avro_content: Optional[bytes]) -> SchemaPair: + if avro_content is None: + return -1, None + + with BytesIO(avro_content) as fake_stream: + schema_id = self._extract_schema_id(fake_stream.read(5)) + parsed_schema = self.get_schema_from_id(schema_id).parsed_schema + record = fastavro.schemaless_reader(fake_stream, parsed_schema) + return schema_id, record + + def _extract_schema_id(self, message: bytes) -> int: + _, schema_id = struct.unpack(">bI", message[:5]) + return schema_id diff --git a/setup.py b/setup.py index a8f630a0..c31a10e3 100644 --- a/setup.py +++ b/setup.py @@ -31,6 +31,8 @@ "pykafka", "pendulum", "pyyaml", + 'requests', + 'fastavro', ] From df51bf1ae3aa321bde8dce5cf6dad8c6e893b23a Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sun, 7 Jul 2019 23:41:04 +0200 Subject: [PATCH 04/94] Set Serializer as super class for all serializer --- esque/avromessage.py | 7 ++++--- esque/cli/commands.py | 7 ++++++- esque/clients.py | 7 ++++--- esque/message.py | 12 ++++++++++-- 4 files changed, 24 insertions(+), 9 deletions(-) diff --git a/esque/avromessage.py b/esque/avromessage.py index 4d027a52..c746a9b1 100644 --- a/esque/avromessage.py +++ b/esque/avromessage.py @@ -9,6 +9,7 @@ import fastavro from confluent_kafka.cimpl import Message +from esque.message import Serializer from esque.schemaregistry import SchemaRegistryClient @@ -20,10 +21,10 @@ def __init__(self, key: Optional[Dict], value: Optional[Dict], key_schema_id: in self.value_schema_id = value_schema_id -class AvroSerializer: +class AvroSerializer(Serializer): - def __init__(self, schema_registry_client: SchemaRegistryClient, working_dir: pathlib.Path): - self.working_dir = working_dir + def __init__(self, working_dir: pathlib.Path, schema_registry_client: SchemaRegistryClient): + super().__init__(working_dir) self.schema_registry_client = schema_registry_client self.current_key_schema_id = None self.current_value_schema_id = None diff --git a/esque/cli/commands.py b/esque/cli/commands.py index 5a79d984..2fd400a1 100644 --- a/esque/cli/commands.py +++ b/esque/cli/commands.py @@ -1,6 +1,7 @@ import os import random import time +from pathlib import Path from time import sleep import click @@ -9,6 +10,7 @@ import yaml from esque.__version__ import __version__ +from esque.avromessage import AvroSerializer from esque.broker import Broker from esque.cli.helpers import ensure_approval from esque.cli.options import State, no_verify_option, pass_state @@ -22,6 +24,7 @@ ContextNotDefinedException, TopicAlreadyExistsException, ) +from esque.schemaregistry import SchemaRegistryClient from esque.topic import TopicController @@ -319,7 +322,9 @@ def transfer(state: State, topic: str, from_context: str, to_context: str, numbe state.config.context_switch(from_context) click.echo("\nStart consuming from source context " + blue_bold(from_context)) consumer = Consumer(group_id, topic, last) - number_consumed_messages = consumer.consume_to_file(file_name, int(numbers)) + schema_registry = SchemaRegistryClient(state.config.schema_registry) + serializer = AvroSerializer(Path('temp'), schema_registry) + number_consumed_messages = consumer.consume_to_file(serializer, int(numbers)) click.echo(blue_bold(str(number_consumed_messages)) + " messages consumed successfully.") click.echo("\nReady to produce to context " + blue_bold(to_context) + " and target topic " + blue_bold(topic)) diff --git a/esque/clients.py b/esque/clients.py index 01c622c2..91f1249b 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -10,10 +10,11 @@ from esque.config import Config from esque.errors import raise_for_kafka_error, raise_for_message, KafkaException from esque.helpers import delivery_callback, delta_t -from esque.avromessage import AvroSerializer +from esque.message import Serializer DEFAULT_RETENTION_MS = 7 * 24 * 60 * 60 * 1000 + class Consumer: def __init__(self, group_id: str, topic_name: str, last: bool): offset_reset = "earliest" @@ -46,9 +47,9 @@ def consume_ping(self) -> Optional[Tuple[str, int]]: delta_sent = pendulum.now() - msg_sent_at return msg.key(), delta_sent.microseconds / 1000 - def consume_to_file(self, working_dir: pathlib.Path, serializer: AvroSerializer, amount: int) -> int: + def consume_to_file(self, serializer: Serializer, amount: int) -> int: counter = 0 - with (working_dir / "data").open("wb") as file: + with (serializer.get_working_directory_path() / "data").open("wb") as file: while counter < amount: try: message = self._consume_single_message() diff --git a/esque/message.py b/esque/message.py index 73626933..c894c4cb 100644 --- a/esque/message.py +++ b/esque/message.py @@ -1,4 +1,6 @@ +import pathlib import pickle +from pathlib import Path from typing import BinaryIO from confluent_kafka.cimpl import Message @@ -10,13 +12,19 @@ def __init__(self, key: str, value: str): self.value = value -class Serializer: +class Serializer(object): + + def __init__(self, working_dir: pathlib.Path): + self.working_dir = working_dir def serialize(self, message: Message, file: BinaryIO): pass + def get_working_directory_path(self) -> Path: + return self.working_dir + -class JsonSerializer: +class JsonSerializer(Serializer): def serialize(self, message: Message, file: BinaryIO): decoded_message = decode_message(message) From 801072d98fc958b7f8009f2fc5a36f69fc0e2a47 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 8 Jul 2019 15:24:23 +0200 Subject: [PATCH 05/94] Refactor Producer and Consumer for PlainText and Avro --- esque/avromessage.py | 44 +++++++++++++---- esque/cli/commands.py | 96 ++++++++++++++++++------------------- esque/clients.py | 103 +++++++++++++++++++++++++++++++++------- esque/errors.py | 14 ++++++ esque/message.py | 39 ++++++++++----- esque/schemaregistry.py | 3 +- setup.py | 2 +- 7 files changed, 212 insertions(+), 89 deletions(-) diff --git a/esque/avromessage.py b/esque/avromessage.py index c746a9b1..e0ee1e77 100644 --- a/esque/avromessage.py +++ b/esque/avromessage.py @@ -8,8 +8,9 @@ import fastavro from confluent_kafka.cimpl import Message +from confluent_kafka.avro import loads as load_schema -from esque.message import Serializer +from esque.message import FileWriter, FileReader, KafkaMessage from esque.schemaregistry import SchemaRegistryClient @@ -21,23 +22,27 @@ def __init__(self, key: Optional[Dict], value: Optional[Dict], key_schema_id: in self.value_schema_id = value_schema_id -class AvroSerializer(Serializer): +class AvroFileWriter(FileWriter): def __init__(self, working_dir: pathlib.Path, schema_registry_client: SchemaRegistryClient): - super().__init__(working_dir) + self.working_dir = working_dir self.schema_registry_client = schema_registry_client self.current_key_schema_id = None self.current_value_schema_id = None + self.schema_dir_name = None self.schema_version = it.count(1) - def serialize(self, message: Message, file: BinaryIO): + def write_message_to_file(self, message: Message, file: BinaryIO): key_schema_id, decoded_key = self.decode_bytes(message.key()) value_schema_id, decoded_value = self.decode_bytes(message.value()) decoded_message = DecodedAvroMessage(decoded_key, decoded_value, key_schema_id, value_schema_id) - if self.schema_changed(decoded_message): - schema_dir_name = f"{next(self.schema_version):04}_{key_schema_id}_{value_schema_id}" - directory = self.working_dir / schema_dir_name + if self.schema_changed(decoded_message) or self.schema_dir_name is None: + self.schema_dir_name = f"{next(self.schema_version):04}_{key_schema_id}_{value_schema_id}" + self.current_key_schema_id = key_schema_id + self.current_value_schema_id = value_schema_id + + directory = self.working_dir / self.schema_dir_name directory.mkdir() (directory / "key_schema.avsc").write_text( @@ -48,7 +53,11 @@ def serialize(self, message: Message, file: BinaryIO): json.dumps(self.schema_registry_client.get_schema_from_id(value_schema_id).original_schema) ) - serializable_message = {"key": decoded_key, "value": decoded_value} + serializable_message = { + "key": decoded_key, + "value": decoded_value, + "schema_directory_name": self.schema_dir_name, + } pickle.dump(serializable_message, file) def decode_bytes(self, raw_data: Optional[bytes]) -> Tuple[int, Optional[Dict]]: @@ -66,6 +75,25 @@ def schema_changed(self, decoded_message: DecodedAvroMessage) -> bool: or self.current_key_schema_id != decoded_message.key_schema_id +class AvroFileReader(FileReader): + + def __init__(self, working_dir: pathlib.Path): + self.working_dir = working_dir + + def read_from_file(self, file: BinaryIO) -> Optional[KafkaMessage]: + try: + record = pickle.load(file) + except EOFError: + return None + + schema_directory = (self.working_dir / record["schema_directory_name"]) + + key_schema = load_schema((schema_directory / "key_schema.avsc").read_text()) + value_schema = load_schema((schema_directory / "value_schema.avsc").read_text()) + + return KafkaMessage(record["key"], record["value"], key_schema, value_schema) + + def extract_schema_id(message: bytes) -> int: _, schema_id = struct.unpack(">bI", message[:5]) return schema_id diff --git a/esque/cli/commands.py b/esque/cli/commands.py index 2fd400a1..d5e39783 100644 --- a/esque/cli/commands.py +++ b/esque/cli/commands.py @@ -1,21 +1,18 @@ -import os -import random +import shutil import time from pathlib import Path from time import sleep import click -from click import version_option - import yaml +from click import version_option from esque.__version__ import __version__ -from esque.avromessage import AvroSerializer from esque.broker import Broker from esque.cli.helpers import ensure_approval from esque.cli.options import State, no_verify_option, pass_state from esque.cli.output import bold, pretty, pretty_topic_diffs, get_output_new_topics, blue_bold, green_bold -from esque.clients import Consumer, Producer +from esque.clients import Consumer, Producer, FileConsumer, FileProducer, AvroFileProducer, AvroFileConsumer from esque.cluster import Cluster from esque.config import PING_TOPIC, Config, PING_GROUP_ID from esque.consumergroup import ConsumerGroupController @@ -23,7 +20,7 @@ ConsumerGroupDoesNotExistException, ContextNotDefinedException, TopicAlreadyExistsException, -) + DeleteOnException) from esque.schemaregistry import SchemaRegistryClient from esque.topic import TopicController @@ -189,26 +186,6 @@ def apply(state: State, file: str): click.echo("No new topics to create.") -@esque.command(help="Consume messages from a topic to file.") -@click.option("-t", "--topic", help="Source Topic name", required=True) -@click.option("-f", "--file", help="Destination file path", required=True) -@click.argument("amount", required=True) -def consume(topic:str, file: str, amount: str): - your_letters = 'abcdefghijklmnopqrstuvwxyz' - group_id = topic.join((random.choice(your_letters) for i in range(7))) - - consumer = Consumer(group_id, topic, "earliest") - consumer.consume_to_file(file, int(amount)) - - -@esque.command(help="Produce messages from a file to topic.") -@click.option("-f", "--file", help="Source file path", required=True) -@click.option("-t", "--topic", help="Destination Topic name", required=True) -def produce(topic:str, file: str): - producer = Producer() - producer.produce_from_file(file, topic) - - @delete.command("topic") @click.argument( "topic-name", required=True, type=click.STRING, autocompletion=list_topics @@ -306,42 +283,59 @@ def get_topics(state, topic, o): click.echo(topic.name) -@esque.command("transfer") +@esque.command("transfer", help="Transfer messages of a topic from one environment to another.") @click.argument("topic", required=True) @click.option("-f", "--from", "from_context", help="Source Context", type=click.STRING, required=True) @click.option("-t", "--to", "to_context", help="Destination context", type=click.STRING, required=True) @click.option("-n", "--numbers", help="Number of messages", type=click.INT, required=True) @click.option('--last/--first', default=False) +@click.option( + "-a", "--avro", help="Set this flag if the topic contains avro data", default=False, is_flag=True +) @pass_state -def transfer(state: State, topic: str, from_context: str, to_context: str, numbers: int, last: bool): +def transfer(state: State, topic: str, from_context: str, to_context: str, numbers: int, last: bool, avro: bool): current_timestamp_milliseconds = int(round(time.time() * 1000)) temp_name = topic + '_' + str(current_timestamp_milliseconds) group_id = "group_for_" + temp_name - file_name = "temp-file_" + temp_name + directory_name = "temp-file_" + temp_name state.config.context_switch(from_context) - click.echo("\nStart consuming from source context " + blue_bold(from_context)) - consumer = Consumer(group_id, topic, last) - schema_registry = SchemaRegistryClient(state.config.schema_registry) - serializer = AvroSerializer(Path('temp'), schema_registry) - number_consumed_messages = consumer.consume_to_file(serializer, int(numbers)) - click.echo(blue_bold(str(number_consumed_messages)) + " messages consumed successfully.") - click.echo("\nReady to produce to context " + blue_bold(to_context) + " and target topic " + blue_bold(topic)) - - if ensure_approval("Do you want to proceed?\n", no_verify=state.no_verify): - state.config.context_switch(to_context) - producer = Producer() - number_produced_messages = producer.produce_from_file(file_name, topic) - click.echo( - green_bold(str(number_produced_messages)) - + " messages successfully produced to context " - + green_bold(to_context) - + " and topic " - + green_bold(topic) - + "." - ) - os.remove(file_name) + base_dir = Path(directory_name) + + with DeleteOnException(base_dir) as working_dir: + click.echo("\nStart consuming from source context " + blue_bold(from_context)) + + if avro: + schema_registry = SchemaRegistryClient(state.config.schema_registry) + consumer = AvroFileConsumer(group_id, topic, working_dir, schema_registry, last) + else: + consumer = FileConsumer(group_id, topic, working_dir, last) + + number_consumed_messages = consumer.consume_to_file(int(numbers)) + click.echo(blue_bold(str(number_consumed_messages)) + " messages consumed successfully.") + click.echo("\nReady to produce to context " + blue_bold(to_context) + " and target topic " + blue_bold(topic)) + + if ensure_approval("Do you want to proceed?\n", no_verify=state.no_verify): + state.config.context_switch(to_context) + + if avro: + producer = AvroFileProducer(working_dir) + else: + producer = FileProducer(working_dir) + + number_produced_messages = producer.produce_from_file("test_write_from_file_target2") + click.echo( + green_bold(str(number_produced_messages)) + + " messages successfully produced to context " + + green_bold(to_context) + + " and topic " + + green_bold(topic) + + "." + ) + + if base_dir.exists(): + shutil.rmtree(base_dir) @esque.command("ping", help="Tests the connection to the kafka cluster.") diff --git a/esque/clients.py b/esque/clients.py index 91f1249b..72f4ad0c 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -1,16 +1,18 @@ import pathlib -import pickle from typing import Optional, Tuple import click import confluent_kafka import pendulum from confluent_kafka import TopicPartition, Message +from confluent_kafka.avro import AvroProducer +from esque.avromessage import AvroFileReader, AvroFileWriter from esque.config import Config from esque.errors import raise_for_kafka_error, raise_for_message, KafkaException from esque.helpers import delivery_callback, delta_t -from esque.message import Serializer +from esque.message import KafkaMessage, PlainTextFileReader, PlainTextFileWriter +from esque.schemaregistry import SchemaRegistryClient DEFAULT_RETENTION_MS = 7 * 24 * 60 * 60 * 1000 @@ -38,6 +40,12 @@ def __init__(self, group_id: str, topic_name: str, last: bool): self._consumer = confluent_kafka.Consumer(self._config) self._assign_exact_partitions(topic_name) + def _assign_exact_partitions(self, topic: str) -> None: + self._consumer.assign([TopicPartition(topic=topic, partition=0, offset=0)]) + + +class PingConsumer(Consumer): + def consume_ping(self) -> Optional[Tuple[str, int]]: msg = self._consumer.consume(timeout=10)[0] @@ -47,9 +55,24 @@ def consume_ping(self) -> Optional[Tuple[str, int]]: delta_sent = pendulum.now() - msg_sent_at return msg.key(), delta_sent.microseconds / 1000 - def consume_to_file(self, serializer: Serializer, amount: int) -> int: + +class FileConsumer(Consumer): + + def __init__(self, group_id: str, topic_name: str, working_dir: pathlib.Path, last: bool): + super().__init__(group_id, topic_name, last) + self.working_dir = working_dir + offset_reset = "earliest" + if last: + offset_reset = "latest" + + self._config.update({"default.topic.config": {"auto.offset.reset": offset_reset}}) + self._consumer = confluent_kafka.Consumer(self._config) + self._assign_exact_partitions(topic_name) + self.writer = PlainTextFileWriter() + + def consume_to_file(self, amount: int) -> int: counter = 0 - with (serializer.get_working_directory_path() / "data").open("wb") as file: + with (self.working_dir / "data").open("wb") as file: while counter < amount: try: message = self._consume_single_message() @@ -57,7 +80,7 @@ def consume_to_file(self, serializer: Serializer, amount: int) -> int: print("An error occurred: " + ex.message) continue - serializer.serialize(message, file) + self.writer.write_message_to_file(message, file) counter += 1 return counter @@ -77,16 +100,28 @@ def _consume_single_message(self) -> Message: print("An error occurred: " + ex.message) - def _assign_exact_partitions(self, topic: str) -> None: - self._consumer.assign([TopicPartition(topic=topic, partition=0, offset=0)]) +class AvroFileConsumer(FileConsumer): + def __init__( + self, + group_id: str, + topic_name: str, + working_dir: pathlib.Path, + schema_registry_client: SchemaRegistryClient, + last: bool + ): + super().__init__(group_id, topic_name, working_dir, last) + self.writer = AvroFileWriter(working_dir, schema_registry_client) -class Producer: + + +class Producer(object): def __init__(self): self._config = Config().create_confluent_config() self._config.update( {"on_delivery": delivery_callback, "error_cb": raise_for_kafka_error} ) + self._producer = confluent_kafka.Producer(self._config) def produce_ping(self, topic_name: str): @@ -102,18 +137,31 @@ def produce_ping(self, topic_name: str): f"{delta_t(start)} | Still {left_messages} messages left, flushing..." ) - def produce_from_file(self, file_path: str, topic_name: str) -> int: - with open(file_path, "rb") as file: + +class FileProducer(object): + + def __init__(self, working_dir: pathlib.Path): + self._config = Config().create_confluent_config() + self._config.update( + { + "on_delivery": delivery_callback, + "error_cb": raise_for_kafka_error, + } + ) + + self._producer = confluent_kafka.Producer(self._config) + self.working_dir = working_dir + self.reader = PlainTextFileReader() + + def produce_from_file(self, topic_name: str) -> int: + with (self.working_dir / "data").open("rb") as file: counter = 0 while True: - try: - record = pickle.load(file) - except EOFError: + message = self.reader.read_from_file(file) + if message is None: break - self._producer.produce( - topic=topic_name, key=record["key"], value=record["value"] - ) + self.produce(topic_name, message) counter += 1 while True: @@ -122,3 +170,26 @@ def produce_from_file(self, file_path: str, topic_name: str) -> int: break return counter + + def produce(self, topic_name: str, message: KafkaMessage): + self._producer.produce(topic=topic_name, key=message.key, value=message.value) + + +class AvroFileProducer(FileProducer): + + def __init__(self, working_dir: pathlib.Path): + super().__init__(working_dir) + self._config.update( + {"schema.registry.url": Config().schema_registry} + ) + self._producer = AvroProducer(self._config) + self.reader = AvroFileReader(working_dir) + + def produce(self, topic_name: str, message: KafkaMessage): + self._producer.produce( + topic=topic_name, + key=message.key, + value=message.value, + key_schema=message.key_schema, + value_schema=message.value_schema + ) diff --git a/esque/errors.py b/esque/errors.py index 81a690f0..3b016520 100644 --- a/esque/errors.py +++ b/esque/errors.py @@ -1,4 +1,6 @@ import functools +import pathlib +import shutil from typing import Dict, Type import confluent_kafka @@ -68,6 +70,18 @@ class TopicDoesNotExistException(Exception): pass +class DeleteOnException: + def __init__(self, dir_: pathlib.Path): + self._dir = dir_ + self._dir.mkdir(parents=True) + + def __enter__(self) -> pathlib.Path: + return self._dir + + def __exit__(self, exc_type, exc_val, exc_tb): + if exc_val is not None and self._dir.exists(): + shutil.rmtree(self._dir) + ERROR_LOOKUP: Dict[int, Type[KafkaException]] = { 36: TopicAlreadyExistsException, -191: EndOfPartitionReachedException, diff --git a/esque/message.py b/esque/message.py index c894c4cb..57bed85f 100644 --- a/esque/message.py +++ b/esque/message.py @@ -1,7 +1,5 @@ -import pathlib import pickle -from pathlib import Path -from typing import BinaryIO +from typing import BinaryIO, Optional from confluent_kafka.cimpl import Message @@ -12,26 +10,45 @@ def __init__(self, key: str, value: str): self.value = value -class Serializer(object): +class KafkaMessage: + def __init__(self, key: str, value: str, key_schema=None, value_schema=None): + self.key = key + self.value = value + self.key_schema = key_schema + self.value_schema = value_schema + - def __init__(self, working_dir: pathlib.Path): - self.working_dir = working_dir +class FileWriter(object): - def serialize(self, message: Message, file: BinaryIO): + def write_message_to_file(self, message: Message, file: BinaryIO): pass - def get_working_directory_path(self) -> Path: - return self.working_dir + +class FileReader(object): + + def read_from_file(self, file: BinaryIO) -> Optional[KafkaMessage]: + pass -class JsonSerializer(Serializer): +class PlainTextFileWriter(FileWriter): - def serialize(self, message: Message, file: BinaryIO): + def write_message_to_file(self, message: Message, file: BinaryIO): decoded_message = decode_message(message) serializable_message = {"key": decoded_message.key, "value": decoded_message.value} pickle.dump(serializable_message, file) +class PlainTextFileReader(FileReader): + + def reade_from_file(self, file: BinaryIO) -> Optional[KafkaMessage]: + try: + record = pickle.load(file) + except EOFError: + return None + + return KafkaMessage(record["key"], record["value"]) + + def decode_message(message: Message) -> DecodedMessage: decoded_key = message.key().decode("utf-8") decoded_value = message.value().decode("utf-8") diff --git a/esque/schemaregistry.py b/esque/schemaregistry.py index f444d12a..d18b5350 100644 --- a/esque/schemaregistry.py +++ b/esque/schemaregistry.py @@ -1,12 +1,12 @@ import json import struct from collections import namedtuple +from functools import lru_cache from io import BytesIO from typing import Dict, Optional import fastavro import requests -from functools import lru_cache SchemaPair = namedtuple("SchemaPair", ["original_schema", "parsed_schema"]) @@ -25,7 +25,6 @@ def get_schema_from_server(self, schema_id: int, server_url: str) -> SchemaPair: response = requests.get(url.format(**locals())) response.raise_for_status() schema: Dict = json.loads(response.json()["schema"]) - print("Schema received for id {schema_id}\n{schema}".format(**locals())) return SchemaPair(schema, fastavro.schema.parse_schema(schema)) def get_schema_for_bytes(self, avro_content: Optional[bytes]) -> SchemaPair: diff --git a/setup.py b/setup.py index c31a10e3..fe2dcec9 100644 --- a/setup.py +++ b/setup.py @@ -32,7 +32,7 @@ "pendulum", "pyyaml", 'requests', - 'fastavro', + 'fastavro>=0.21.8', ] From d2a936ef7de18d6c57d39d43a4bfaf0b84bdd028 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 8 Jul 2019 15:46:36 +0200 Subject: [PATCH 06/94] Fix coding styles --- esque/avromessage.py | 46 ++++++++++++++------ esque/cli/commands.py | 93 +++++++++++++++++++++++++++++++++-------- esque/clients.py | 39 +++++++---------- esque/errors.py | 1 + esque/message.py | 9 ++-- esque/schemaregistry.py | 1 - setup.py | 4 +- 7 files changed, 133 insertions(+), 60 deletions(-) diff --git a/esque/avromessage.py b/esque/avromessage.py index e0ee1e77..33223cc3 100644 --- a/esque/avromessage.py +++ b/esque/avromessage.py @@ -15,7 +15,13 @@ class DecodedAvroMessage: - def __init__(self, key: Optional[Dict], value: Optional[Dict], key_schema_id: int, value_schema_id: int): + def __init__( + self, + key: Optional[Dict], + value: Optional[Dict], + key_schema_id: int, + value_schema_id: int, + ): self.key = key self.value = value self.key_schema_id = key_schema_id @@ -23,8 +29,9 @@ def __init__(self, key: Optional[Dict], value: Optional[Dict], key_schema_id: in class AvroFileWriter(FileWriter): - - def __init__(self, working_dir: pathlib.Path, schema_registry_client: SchemaRegistryClient): + def __init__( + self, working_dir: pathlib.Path, schema_registry_client: SchemaRegistryClient + ): self.working_dir = working_dir self.schema_registry_client = schema_registry_client self.current_key_schema_id = None @@ -35,10 +42,14 @@ def __init__(self, working_dir: pathlib.Path, schema_registry_client: SchemaRegi def write_message_to_file(self, message: Message, file: BinaryIO): key_schema_id, decoded_key = self.decode_bytes(message.key()) value_schema_id, decoded_value = self.decode_bytes(message.value()) - decoded_message = DecodedAvroMessage(decoded_key, decoded_value, key_schema_id, value_schema_id) + decoded_message = DecodedAvroMessage( + decoded_key, decoded_value, key_schema_id, value_schema_id + ) if self.schema_changed(decoded_message) or self.schema_dir_name is None: - self.schema_dir_name = f"{next(self.schema_version):04}_{key_schema_id}_{value_schema_id}" + self.schema_dir_name = ( + f"{next(self.schema_version):04}_{key_schema_id}_{value_schema_id}" + ) self.current_key_schema_id = key_schema_id self.current_value_schema_id = value_schema_id @@ -46,11 +57,19 @@ def write_message_to_file(self, message: Message, file: BinaryIO): directory.mkdir() (directory / "key_schema.avsc").write_text( - json.dumps(self.schema_registry_client.get_schema_from_id(key_schema_id).original_schema) + json.dumps( + self.schema_registry_client.get_schema_from_id( + key_schema_id + ).original_schema + ) ) (directory / "value_schema.avsc").write_text( - json.dumps(self.schema_registry_client.get_schema_from_id(value_schema_id).original_schema) + json.dumps( + self.schema_registry_client.get_schema_from_id( + value_schema_id + ).original_schema + ) ) serializable_message = { @@ -66,17 +85,20 @@ def decode_bytes(self, raw_data: Optional[bytes]) -> Tuple[int, Optional[Dict]]: with BytesIO(raw_data) as fake_stream: schema_id = extract_schema_id(fake_stream.read(5)) - parsed_schema = self.schema_registry_client.get_schema_from_id(schema_id).parsed_schema + parsed_schema = self.schema_registry_client.get_schema_from_id( + schema_id + ).parsed_schema record = fastavro.schemaless_reader(fake_stream, parsed_schema) return schema_id, record def schema_changed(self, decoded_message: DecodedAvroMessage) -> bool: - return (self.current_value_schema_id != decoded_message.value_schema_id and decoded_message.value is not None) \ - or self.current_key_schema_id != decoded_message.key_schema_id + return ( + self.current_value_schema_id != decoded_message.value_schema_id + and decoded_message.value is not None + ) or self.current_key_schema_id != decoded_message.key_schema_id class AvroFileReader(FileReader): - def __init__(self, working_dir: pathlib.Path): self.working_dir = working_dir @@ -86,7 +108,7 @@ def read_from_file(self, file: BinaryIO) -> Optional[KafkaMessage]: except EOFError: return None - schema_directory = (self.working_dir / record["schema_directory_name"]) + schema_directory = self.working_dir / record["schema_directory_name"] key_schema = load_schema((schema_directory / "key_schema.avsc").read_text()) value_schema = load_schema((schema_directory / "value_schema.avsc").read_text()) diff --git a/esque/cli/commands.py b/esque/cli/commands.py index d5e39783..c560a1c6 100644 --- a/esque/cli/commands.py +++ b/esque/cli/commands.py @@ -11,8 +11,24 @@ from esque.broker import Broker from esque.cli.helpers import ensure_approval from esque.cli.options import State, no_verify_option, pass_state -from esque.cli.output import bold, pretty, pretty_topic_diffs, get_output_new_topics, blue_bold, green_bold -from esque.clients import Consumer, Producer, FileConsumer, FileProducer, AvroFileProducer, AvroFileConsumer +from esque.cli.output import ( + bold, + pretty, + pretty_topic_diffs, + get_output_new_topics, + blue_bold, + green_bold, +) +from esque.clients import ( + Consumer, + Producer, + FileConsumer, + FileProducer, + AvroFileProducer, + AvroFileConsumer, + PingConsumer, + PingProducer, +) from esque.cluster import Cluster from esque.config import PING_TOPIC, Config, PING_GROUP_ID from esque.consumergroup import ConsumerGroupController @@ -20,7 +36,8 @@ ConsumerGroupDoesNotExistException, ContextNotDefinedException, TopicAlreadyExistsException, - DeleteOnException) + DeleteOnException, +) from esque.schemaregistry import SchemaRegistryClient from esque.topic import TopicController @@ -283,19 +300,49 @@ def get_topics(state, topic, o): click.echo(topic.name) -@esque.command("transfer", help="Transfer messages of a topic from one environment to another.") +@esque.command( + "transfer", help="Transfer messages of a topic from one environment to another." +) @click.argument("topic", required=True) -@click.option("-f", "--from", "from_context", help="Source Context", type=click.STRING, required=True) -@click.option("-t", "--to", "to_context", help="Destination context", type=click.STRING, required=True) -@click.option("-n", "--numbers", help="Number of messages", type=click.INT, required=True) -@click.option('--last/--first', default=False) @click.option( - "-a", "--avro", help="Set this flag if the topic contains avro data", default=False, is_flag=True + "-f", + "--from", + "from_context", + help="Source Context", + type=click.STRING, + required=True, +) +@click.option( + "-t", + "--to", + "to_context", + help="Destination context", + type=click.STRING, + required=True, +) +@click.option( + "-n", "--numbers", help="Number of messages", type=click.INT, required=True +) +@click.option("--last/--first", default=False) +@click.option( + "-a", + "--avro", + help="Set this flag if the topic contains avro data", + default=False, + is_flag=True, ) @pass_state -def transfer(state: State, topic: str, from_context: str, to_context: str, numbers: int, last: bool, avro: bool): +def transfer( + state: State, + topic: str, + from_context: str, + to_context: str, + numbers: int, + last: bool, + avro: bool, +): current_timestamp_milliseconds = int(round(time.time() * 1000)) - temp_name = topic + '_' + str(current_timestamp_milliseconds) + temp_name = topic + "_" + str(current_timestamp_milliseconds) group_id = "group_for_" + temp_name directory_name = "temp-file_" + temp_name @@ -308,13 +355,23 @@ def transfer(state: State, topic: str, from_context: str, to_context: str, numbe if avro: schema_registry = SchemaRegistryClient(state.config.schema_registry) - consumer = AvroFileConsumer(group_id, topic, working_dir, schema_registry, last) + consumer = AvroFileConsumer( + group_id, topic, working_dir, schema_registry, last + ) else: consumer = FileConsumer(group_id, topic, working_dir, last) number_consumed_messages = consumer.consume_to_file(int(numbers)) - click.echo(blue_bold(str(number_consumed_messages)) + " messages consumed successfully.") - click.echo("\nReady to produce to context " + blue_bold(to_context) + " and target topic " + blue_bold(topic)) + click.echo( + blue_bold(str(number_consumed_messages)) + + " messages consumed successfully." + ) + click.echo( + "\nReady to produce to context " + + blue_bold(to_context) + + " and target topic " + + blue_bold(topic) + ) if ensure_approval("Do you want to proceed?\n", no_verify=state.no_verify): state.config.context_switch(to_context) @@ -324,7 +381,9 @@ def transfer(state: State, topic: str, from_context: str, to_context: str, numbe else: producer = FileProducer(working_dir) - number_produced_messages = producer.produce_from_file("test_write_from_file_target2") + number_produced_messages = producer.produce_from_file( + "test_write_from_file_target2" + ) click.echo( green_bold(str(number_produced_messages)) + " messages successfully produced to context " @@ -351,8 +410,8 @@ def ping(state, times, wait): except TopicAlreadyExistsException: click.echo("Topic already exists.") - producer = Producer() - consumer = Consumer(PING_GROUP_ID, PING_TOPIC, True) + producer = PingProducer() + consumer = PingConsumer(PING_GROUP_ID, PING_TOPIC, True) click.echo(f"Ping with {state.cluster.bootstrap_servers}") diff --git a/esque/clients.py b/esque/clients.py index 72f4ad0c..54bb6d1e 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -45,7 +45,6 @@ def _assign_exact_partitions(self, topic: str) -> None: class PingConsumer(Consumer): - def consume_ping(self) -> Optional[Tuple[str, int]]: msg = self._consumer.consume(timeout=10)[0] @@ -57,15 +56,18 @@ def consume_ping(self) -> Optional[Tuple[str, int]]: class FileConsumer(Consumer): - - def __init__(self, group_id: str, topic_name: str, working_dir: pathlib.Path, last: bool): + def __init__( + self, group_id: str, topic_name: str, working_dir: pathlib.Path, last: bool + ): super().__init__(group_id, topic_name, last) self.working_dir = working_dir offset_reset = "earliest" if last: offset_reset = "latest" - self._config.update({"default.topic.config": {"auto.offset.reset": offset_reset}}) + self._config.update( + {"default.topic.config": {"auto.offset.reset": offset_reset}} + ) self._consumer = confluent_kafka.Consumer(self._config) self._assign_exact_partitions(topic_name) self.writer = PlainTextFileWriter() @@ -101,21 +103,19 @@ def _consume_single_message(self) -> Message: class AvroFileConsumer(FileConsumer): - def __init__( - self, - group_id: str, - topic_name: str, - working_dir: pathlib.Path, - schema_registry_client: SchemaRegistryClient, - last: bool + self, + group_id: str, + topic_name: str, + working_dir: pathlib.Path, + schema_registry_client: SchemaRegistryClient, + last: bool, ): super().__init__(group_id, topic_name, working_dir, last) self.writer = AvroFileWriter(working_dir, schema_registry_client) - -class Producer(object): +class PingProducer(object): def __init__(self): self._config = Config().create_confluent_config() self._config.update( @@ -139,14 +139,10 @@ def produce_ping(self, topic_name: str): class FileProducer(object): - def __init__(self, working_dir: pathlib.Path): self._config = Config().create_confluent_config() self._config.update( - { - "on_delivery": delivery_callback, - "error_cb": raise_for_kafka_error, - } + {"on_delivery": delivery_callback, "error_cb": raise_for_kafka_error} ) self._producer = confluent_kafka.Producer(self._config) @@ -176,12 +172,9 @@ def produce(self, topic_name: str, message: KafkaMessage): class AvroFileProducer(FileProducer): - def __init__(self, working_dir: pathlib.Path): super().__init__(working_dir) - self._config.update( - {"schema.registry.url": Config().schema_registry} - ) + self._config.update({"schema.registry.url": Config().schema_registry}) self._producer = AvroProducer(self._config) self.reader = AvroFileReader(working_dir) @@ -191,5 +184,5 @@ def produce(self, topic_name: str, message: KafkaMessage): key=message.key, value=message.value, key_schema=message.key_schema, - value_schema=message.value_schema + value_schema=message.value_schema, ) diff --git a/esque/errors.py b/esque/errors.py index 3b016520..0fc6fee3 100644 --- a/esque/errors.py +++ b/esque/errors.py @@ -82,6 +82,7 @@ def __exit__(self, exc_type, exc_val, exc_tb): if exc_val is not None and self._dir.exists(): shutil.rmtree(self._dir) + ERROR_LOOKUP: Dict[int, Type[KafkaException]] = { 36: TopicAlreadyExistsException, -191: EndOfPartitionReachedException, diff --git a/esque/message.py b/esque/message.py index 57bed85f..2b45968c 100644 --- a/esque/message.py +++ b/esque/message.py @@ -19,27 +19,26 @@ def __init__(self, key: str, value: str, key_schema=None, value_schema=None): class FileWriter(object): - def write_message_to_file(self, message: Message, file: BinaryIO): pass class FileReader(object): - def read_from_file(self, file: BinaryIO) -> Optional[KafkaMessage]: pass class PlainTextFileWriter(FileWriter): - def write_message_to_file(self, message: Message, file: BinaryIO): decoded_message = decode_message(message) - serializable_message = {"key": decoded_message.key, "value": decoded_message.value} + serializable_message = { + "key": decoded_message.key, + "value": decoded_message.value, + } pickle.dump(serializable_message, file) class PlainTextFileReader(FileReader): - def reade_from_file(self, file: BinaryIO) -> Optional[KafkaMessage]: try: record = pickle.load(file) diff --git a/esque/schemaregistry.py b/esque/schemaregistry.py index d18b5350..51a50ccb 100644 --- a/esque/schemaregistry.py +++ b/esque/schemaregistry.py @@ -12,7 +12,6 @@ class SchemaRegistryClient: - def __init__(self, schema_registry_uri: str): self.schema_registry_uri = schema_registry_uri diff --git a/setup.py b/setup.py index fe2dcec9..6215342b 100644 --- a/setup.py +++ b/setup.py @@ -31,8 +31,8 @@ "pykafka", "pendulum", "pyyaml", - 'requests', - 'fastavro>=0.21.8', + "requests", + "fastavro>=0.21.8", ] From 70057474a970beb910edd27c821125097abfe2cf Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 8 Jul 2019 15:47:23 +0200 Subject: [PATCH 07/94] Remove unused imports --- esque/cli/commands.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/esque/cli/commands.py b/esque/cli/commands.py index c560a1c6..23fb7e30 100644 --- a/esque/cli/commands.py +++ b/esque/cli/commands.py @@ -20,8 +20,6 @@ green_bold, ) from esque.clients import ( - Consumer, - Producer, FileConsumer, FileProducer, AvroFileProducer, From f433564cd2f5dea917bc831159a81127008ecf02 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 8 Jul 2019 16:26:08 +0200 Subject: [PATCH 08/94] Add create methods for Producer and Consumer --- esque/cli/commands.py | 16 ++++-------- esque/clients.py | 59 +++++++++++++++++++++++++++++++++---------- esque/config.py | 4 +-- esque/message.py | 2 +- 4 files changed, 53 insertions(+), 28 deletions(-) diff --git a/esque/cli/commands.py b/esque/cli/commands.py index 23fb7e30..4d0c2593 100644 --- a/esque/cli/commands.py +++ b/esque/cli/commands.py @@ -36,7 +36,6 @@ TopicAlreadyExistsException, DeleteOnException, ) -from esque.schemaregistry import SchemaRegistryClient from esque.topic import TopicController @@ -343,21 +342,16 @@ def transfer( temp_name = topic + "_" + str(current_timestamp_milliseconds) group_id = "group_for_" + temp_name directory_name = "temp-file_" + temp_name - - state.config.context_switch(from_context) - base_dir = Path(directory_name) + state.config.context_switch(from_context) with DeleteOnException(base_dir) as working_dir: click.echo("\nStart consuming from source context " + blue_bold(from_context)) if avro: - schema_registry = SchemaRegistryClient(state.config.schema_registry) - consumer = AvroFileConsumer( - group_id, topic, working_dir, schema_registry, last - ) + consumer = AvroFileConsumer.create(group_id, topic, working_dir, last) else: - consumer = FileConsumer(group_id, topic, working_dir, last) + consumer = FileConsumer.create(group_id, topic, working_dir, last) number_consumed_messages = consumer.consume_to_file(int(numbers)) click.echo( @@ -375,9 +369,9 @@ def transfer( state.config.context_switch(to_context) if avro: - producer = AvroFileProducer(working_dir) + producer = AvroFileProducer.create(working_dir) else: - producer = FileProducer(working_dir) + producer = FileProducer.create(working_dir) number_produced_messages = producer.produce_from_file( "test_write_from_file_target2" diff --git a/esque/clients.py b/esque/clients.py index 54bb6d1e..01dda2f5 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -11,7 +11,13 @@ from esque.config import Config from esque.errors import raise_for_kafka_error, raise_for_message, KafkaException from esque.helpers import delivery_callback, delta_t -from esque.message import KafkaMessage, PlainTextFileReader, PlainTextFileWriter +from esque.message import ( + KafkaMessage, + PlainTextFileReader, + PlainTextFileWriter, + FileReader, + FileWriter, +) from esque.schemaregistry import SchemaRegistryClient DEFAULT_RETENTION_MS = 7 * 24 * 60 * 60 * 1000 @@ -57,7 +63,12 @@ def consume_ping(self) -> Optional[Tuple[str, int]]: class FileConsumer(Consumer): def __init__( - self, group_id: str, topic_name: str, working_dir: pathlib.Path, last: bool + self, + group_id: str, + topic_name: str, + working_dir: pathlib.Path, + file_writer: FileWriter, + last: bool, ): super().__init__(group_id, topic_name, last) self.working_dir = working_dir @@ -70,7 +81,13 @@ def __init__( ) self._consumer = confluent_kafka.Consumer(self._config) self._assign_exact_partitions(topic_name) - self.writer = PlainTextFileWriter() + self.file_writer = file_writer + + @classmethod + def create( + cls, group_id: str, topic_name: str, working_dir: pathlib.Path, last: bool + ): + return cls(group_id, topic_name, working_dir, PlainTextFileWriter(), last) def consume_to_file(self, amount: int) -> int: counter = 0 @@ -82,7 +99,7 @@ def consume_to_file(self, amount: int) -> int: print("An error occurred: " + ex.message) continue - self.writer.write_message_to_file(message, file) + self.file_writer.write_message_to_file(message, file) counter += 1 return counter @@ -108,11 +125,19 @@ def __init__( group_id: str, topic_name: str, working_dir: pathlib.Path, - schema_registry_client: SchemaRegistryClient, + file_writer: FileWriter, last: bool, ): - super().__init__(group_id, topic_name, working_dir, last) - self.writer = AvroFileWriter(working_dir, schema_registry_client) + super().__init__(group_id, topic_name, working_dir, file_writer, last) + self.writer = file_writer + + @classmethod + def create( + cls, group_id: str, topic_name: str, working_dir: pathlib.Path, last: bool + ): + schema_registry_client = SchemaRegistryClient(Config().schema_registry) + file_writer = AvroFileWriter(working_dir, schema_registry_client) + return cls(group_id, topic_name, working_dir, file_writer, last) class PingProducer(object): @@ -139,7 +164,7 @@ def produce_ping(self, topic_name: str): class FileProducer(object): - def __init__(self, working_dir: pathlib.Path): + def __init__(self, working_dir: pathlib.Path, file_reader: FileReader): self._config = Config().create_confluent_config() self._config.update( {"on_delivery": delivery_callback, "error_cb": raise_for_kafka_error} @@ -147,13 +172,17 @@ def __init__(self, working_dir: pathlib.Path): self._producer = confluent_kafka.Producer(self._config) self.working_dir = working_dir - self.reader = PlainTextFileReader() + self.file_reader = file_reader + + @classmethod + def create(cls, working_dir: pathlib.Path): + return cls(working_dir, PlainTextFileReader()) def produce_from_file(self, topic_name: str) -> int: with (self.working_dir / "data").open("rb") as file: counter = 0 while True: - message = self.reader.read_from_file(file) + message = self.file_reader.read_from_file(file) if message is None: break @@ -172,11 +201,15 @@ def produce(self, topic_name: str, message: KafkaMessage): class AvroFileProducer(FileProducer): - def __init__(self, working_dir: pathlib.Path): - super().__init__(working_dir) + def __init__(self, working_dir: pathlib.Path, file_reader: FileReader): + super().__init__(working_dir, file_reader) self._config.update({"schema.registry.url": Config().schema_registry}) self._producer = AvroProducer(self._config) - self.reader = AvroFileReader(working_dir) + self.file_reader = file_reader + + @classmethod + def create(cls, working_dir: pathlib.Path): + return cls(working_dir, AvroFileReader(working_dir)) def produce(self, topic_name: str, message: KafkaMessage): self._producer.produce( diff --git a/esque/config.py b/esque/config.py index c2ef0ead..adb21fe9 100644 --- a/esque/config.py +++ b/esque/config.py @@ -18,9 +18,7 @@ def config_dir() -> Path: - if ESQUE_ENV == "dev": - return Path(__file__).parent.parent - return Path(click.get_app_dir("esque", force_posix=True)) + return Path(__file__).parent.parent def config_path() -> Path: diff --git a/esque/message.py b/esque/message.py index 2b45968c..8a861368 100644 --- a/esque/message.py +++ b/esque/message.py @@ -39,7 +39,7 @@ def write_message_to_file(self, message: Message, file: BinaryIO): class PlainTextFileReader(FileReader): - def reade_from_file(self, file: BinaryIO) -> Optional[KafkaMessage]: + def read_from_file(self, file: BinaryIO) -> Optional[KafkaMessage]: try: record = pickle.load(file) except EOFError: From 4b65ca6cf15e5ab647c6167e6c312843a503a66b Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 8 Jul 2019 16:26:48 +0200 Subject: [PATCH 09/94] Do not use test topic anymore --- esque/cli/commands.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/esque/cli/commands.py b/esque/cli/commands.py index 4d0c2593..5357c1e8 100644 --- a/esque/cli/commands.py +++ b/esque/cli/commands.py @@ -373,9 +373,7 @@ def transfer( else: producer = FileProducer.create(working_dir) - number_produced_messages = producer.produce_from_file( - "test_write_from_file_target2" - ) + number_produced_messages = producer.produce_from_file(topic) click.echo( green_bold(str(number_produced_messages)) + " messages successfully produced to context " From edfa8603331ed80eeee812400de7a2dc18dbdaf8 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 8 Jul 2019 16:28:12 +0200 Subject: [PATCH 10/94] Revert config change for testing --- esque/config.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/esque/config.py b/esque/config.py index adb21fe9..c2ef0ead 100644 --- a/esque/config.py +++ b/esque/config.py @@ -18,7 +18,9 @@ def config_dir() -> Path: - return Path(__file__).parent.parent + if ESQUE_ENV == "dev": + return Path(__file__).parent.parent + return Path(click.get_app_dir("esque", force_posix=True)) def config_path() -> Path: From c209d124824e71d2c57646607eb8335a429b4bf8 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 8 Jul 2019 16:33:46 +0200 Subject: [PATCH 11/94] Extract methods in commands --- esque/cli/commands.py | 73 ++++++++++++++++++++++++------------------- 1 file changed, 40 insertions(+), 33 deletions(-) diff --git a/esque/cli/commands.py b/esque/cli/commands.py index 5357c1e8..a48df531 100644 --- a/esque/cli/commands.py +++ b/esque/cli/commands.py @@ -346,47 +346,54 @@ def transfer( state.config.context_switch(from_context) with DeleteOnException(base_dir) as working_dir: - click.echo("\nStart consuming from source context " + blue_bold(from_context)) - - if avro: - consumer = AvroFileConsumer.create(group_id, topic, working_dir, last) - else: - consumer = FileConsumer.create(group_id, topic, working_dir, last) - - number_consumed_messages = consumer.consume_to_file(int(numbers)) - click.echo( - blue_bold(str(number_consumed_messages)) - + " messages consumed successfully." - ) - click.echo( - "\nReady to produce to context " - + blue_bold(to_context) - + " and target topic " - + blue_bold(topic) + _consume_to_file( + avro, from_context, group_id, last, numbers, to_context, topic, working_dir ) if ensure_approval("Do you want to proceed?\n", no_verify=state.no_verify): - state.config.context_switch(to_context) - - if avro: - producer = AvroFileProducer.create(working_dir) - else: - producer = FileProducer.create(working_dir) - - number_produced_messages = producer.produce_from_file(topic) - click.echo( - green_bold(str(number_produced_messages)) - + " messages successfully produced to context " - + green_bold(to_context) - + " and topic " - + green_bold(topic) - + "." - ) + _produce_from_file(avro, state, to_context, topic, working_dir) if base_dir.exists(): shutil.rmtree(base_dir) +def _produce_from_file(avro, state, to_context, topic, working_dir): + state.config.context_switch(to_context) + if avro: + producer = AvroFileProducer.create(working_dir) + else: + producer = FileProducer.create(working_dir) + number_produced_messages = producer.produce_from_file(topic) + click.echo( + green_bold(str(number_produced_messages)) + + " messages successfully produced to context " + + green_bold(to_context) + + " and topic " + + green_bold(topic) + + "." + ) + + +def _consume_to_file( + avro, from_context, group_id, last, numbers, to_context, topic, working_dir +): + click.echo("\nStart consuming from source context " + blue_bold(from_context)) + if avro: + consumer = AvroFileConsumer.create(group_id, topic, working_dir, last) + else: + consumer = FileConsumer.create(group_id, topic, working_dir, last) + number_consumed_messages = consumer.consume_to_file(int(numbers)) + click.echo( + blue_bold(str(number_consumed_messages)) + " messages consumed successfully." + ) + click.echo( + "\nReady to produce to context " + + blue_bold(to_context) + + " and target topic " + + blue_bold(topic) + ) + + @esque.command("ping", help="Tests the connection to the kafka cluster.") @click.option("-t", "--times", help="Number of pings.", default=10) @click.option("-w", "--wait", help="Seconds to wait between pings.", default=1) From 3751aff4b90f424b20911b8475bcb29d1904ed38 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 8 Jul 2019 16:47:17 +0200 Subject: [PATCH 12/94] Remove unused method --- esque/schemaregistry.py | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/esque/schemaregistry.py b/esque/schemaregistry.py index 51a50ccb..f57624da 100644 --- a/esque/schemaregistry.py +++ b/esque/schemaregistry.py @@ -26,16 +26,6 @@ def get_schema_from_server(self, schema_id: int, server_url: str) -> SchemaPair: schema: Dict = json.loads(response.json()["schema"]) return SchemaPair(schema, fastavro.schema.parse_schema(schema)) - def get_schema_for_bytes(self, avro_content: Optional[bytes]) -> SchemaPair: - if avro_content is None: - return -1, None - - with BytesIO(avro_content) as fake_stream: - schema_id = self._extract_schema_id(fake_stream.read(5)) - parsed_schema = self.get_schema_from_id(schema_id).parsed_schema - record = fastavro.schemaless_reader(fake_stream, parsed_schema) - return schema_id, record - def _extract_schema_id(self, message: bytes) -> int: _, schema_id = struct.unpack(">bI", message[:5]) return schema_id From fafe570634d9a3bc89816353ad7a88af39383aa2 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 8 Jul 2019 22:52:30 +0200 Subject: [PATCH 13/94] Better process handling of edge cases --- esque/cli/commands.py | 61 +++++++++++++++++++++++++---------------- esque/cli/helpers.py | 16 +++++++++++ esque/clients.py | 24 ++++++++-------- esque/errors.py | 15 ---------- esque/message.py | 19 +++++++------ esque/schemaregistry.py | 3 +- 6 files changed, 78 insertions(+), 60 deletions(-) diff --git a/esque/cli/commands.py b/esque/cli/commands.py index a48df531..949fea03 100644 --- a/esque/cli/commands.py +++ b/esque/cli/commands.py @@ -1,4 +1,4 @@ -import shutil +import pathlib import time from pathlib import Path from time import sleep @@ -9,7 +9,7 @@ from esque.__version__ import __version__ from esque.broker import Broker -from esque.cli.helpers import ensure_approval +from esque.cli.helpers import ensure_approval, DeleteOnFinished from esque.cli.options import State, no_verify_option, pass_state from esque.cli.output import ( bold, @@ -34,7 +34,6 @@ ConsumerGroupDoesNotExistException, ContextNotDefinedException, TopicAlreadyExistsException, - DeleteOnException, ) from esque.topic import TopicController @@ -345,20 +344,36 @@ def transfer( base_dir = Path(directory_name) state.config.context_switch(from_context) - with DeleteOnException(base_dir) as working_dir: - _consume_to_file( - avro, from_context, group_id, last, numbers, to_context, topic, working_dir + with DeleteOnFinished(base_dir) as working_dir: + number_consumed_messages = _consume_to_file( + working_dir, topic, group_id, from_context, numbers, avro, last ) - if ensure_approval("Do you want to proceed?\n", no_verify=state.no_verify): - _produce_from_file(avro, state, to_context, topic, working_dir) + if number_consumed_messages == 0: + click.echo( + click.style( + "Execution stopped, because no messages consumed.", fg="red" + ) + ) + return + + click.echo( + "\nReady to produce to context " + + blue_bold(to_context) + + " and target topic " + + blue_bold(topic) + ) + + if not ensure_approval("Do you want to proceed?\n", no_verify=state.no_verify): + return - if base_dir.exists(): - shutil.rmtree(base_dir) + state.config.context_switch(to_context) + _produce_from_file(topic, to_context, working_dir, avro) -def _produce_from_file(avro, state, to_context, topic, working_dir): - state.config.context_switch(to_context) +def _produce_from_file( + topic: str, to_context: str, working_dir: pathlib.Path, avro: bool +): if avro: producer = AvroFileProducer.create(working_dir) else: @@ -375,23 +390,23 @@ def _produce_from_file(avro, state, to_context, topic, working_dir): def _consume_to_file( - avro, from_context, group_id, last, numbers, to_context, topic, working_dir -): + working_dir: pathlib.Path, + topic: str, + group_id: str, + from_context: str, + numbers: int, + avro: bool, + last: bool, +) -> int: click.echo("\nStart consuming from source context " + blue_bold(from_context)) if avro: consumer = AvroFileConsumer.create(group_id, topic, working_dir, last) else: consumer = FileConsumer.create(group_id, topic, working_dir, last) number_consumed_messages = consumer.consume_to_file(int(numbers)) - click.echo( - blue_bold(str(number_consumed_messages)) + " messages consumed successfully." - ) - click.echo( - "\nReady to produce to context " - + blue_bold(to_context) - + " and target topic " - + blue_bold(topic) - ) + click.echo(blue_bold(str(number_consumed_messages)) + " messages consumed.") + + return number_consumed_messages @esque.command("ping", help="Tests the connection to the kafka cluster.") diff --git a/esque/cli/helpers.py b/esque/cli/helpers.py index d0ef2fd8..1971da83 100644 --- a/esque/cli/helpers.py +++ b/esque/cli/helpers.py @@ -1,3 +1,6 @@ +import pathlib +import shutil + import click @@ -5,3 +8,16 @@ def ensure_approval(question: str, *, no_verify: bool = False) -> bool: if no_verify: return True return click.confirm(question) + + +class DeleteOnFinished: + def __init__(self, dir_: pathlib.Path): + self._dir = dir_ + self._dir.mkdir(parents=True) + + def __enter__(self) -> pathlib.Path: + return self._dir + + def __exit__(self, exc_type, exc_val, exc_tb): + if self._dir.exists(): + shutil.rmtree(self._dir) diff --git a/esque/clients.py b/esque/clients.py index 01dda2f5..c347a3a6 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -6,6 +6,7 @@ import pendulum from confluent_kafka import TopicPartition, Message from confluent_kafka.avro import AvroProducer +from confluent_kafka.cimpl import KafkaError from esque.avromessage import AvroFileReader, AvroFileWriter from esque.config import Config @@ -93,18 +94,16 @@ def consume_to_file(self, amount: int) -> int: counter = 0 with (self.working_dir / "data").open("wb") as file: while counter < amount: - try: - message = self._consume_single_message() - except KafkaException as ex: - print("An error occurred: " + ex.message) - continue + message = self._consume_single_message() + if message is None: + return counter self.file_writer.write_message_to_file(message, file) counter += 1 return counter - def _consume_single_message(self) -> Message: + def _consume_single_message(self) -> Optional[Message]: poll_limit = 10 counter = 0 try: @@ -113,10 +112,17 @@ def _consume_single_message(self) -> Message: if message is None: counter += 1 continue + if message.error() is not None: + if message.error().code() == KafkaError._PARTITION_EOF: + print("\nEnd of partition reached!".format(**locals())) + break + else: + raise RuntimeError(message.error().str()) raise_for_message(message) return message except KafkaException as ex: print("An error occurred: " + ex.message) + return None class AvroFileConsumer(FileConsumer): @@ -181,11 +187,7 @@ def create(cls, working_dir: pathlib.Path): def produce_from_file(self, topic_name: str) -> int: with (self.working_dir / "data").open("rb") as file: counter = 0 - while True: - message = self.file_reader.read_from_file(file) - if message is None: - break - + for message in self.file_reader.read_from_file(file): self.produce(topic_name, message) counter += 1 diff --git a/esque/errors.py b/esque/errors.py index 0fc6fee3..81a690f0 100644 --- a/esque/errors.py +++ b/esque/errors.py @@ -1,6 +1,4 @@ import functools -import pathlib -import shutil from typing import Dict, Type import confluent_kafka @@ -70,19 +68,6 @@ class TopicDoesNotExistException(Exception): pass -class DeleteOnException: - def __init__(self, dir_: pathlib.Path): - self._dir = dir_ - self._dir.mkdir(parents=True) - - def __enter__(self) -> pathlib.Path: - return self._dir - - def __exit__(self, exc_type, exc_val, exc_tb): - if exc_val is not None and self._dir.exists(): - shutil.rmtree(self._dir) - - ERROR_LOOKUP: Dict[int, Type[KafkaException]] = { 36: TopicAlreadyExistsException, -191: EndOfPartitionReachedException, diff --git a/esque/message.py b/esque/message.py index 8a861368..73cac7ff 100644 --- a/esque/message.py +++ b/esque/message.py @@ -1,5 +1,5 @@ import pickle -from typing import BinaryIO, Optional +from typing import BinaryIO, Iterable from confluent_kafka.cimpl import Message @@ -24,7 +24,7 @@ def write_message_to_file(self, message: Message, file: BinaryIO): class FileReader(object): - def read_from_file(self, file: BinaryIO) -> Optional[KafkaMessage]: + def read_from_file(self, file: BinaryIO) -> Iterable[KafkaMessage]: pass @@ -39,13 +39,14 @@ def write_message_to_file(self, message: Message, file: BinaryIO): class PlainTextFileReader(FileReader): - def read_from_file(self, file: BinaryIO) -> Optional[KafkaMessage]: - try: - record = pickle.load(file) - except EOFError: - return None - - return KafkaMessage(record["key"], record["value"]) + def read_from_file(self, file: BinaryIO) -> Iterable[KafkaMessage]: + while True: + try: + record = pickle.load(file) + except EOFError: + continue + + yield KafkaMessage(record["key"], record["value"]) def decode_message(message: Message) -> DecodedMessage: diff --git a/esque/schemaregistry.py b/esque/schemaregistry.py index f57624da..ca70f607 100644 --- a/esque/schemaregistry.py +++ b/esque/schemaregistry.py @@ -2,8 +2,7 @@ import struct from collections import namedtuple from functools import lru_cache -from io import BytesIO -from typing import Dict, Optional +from typing import Dict import fastavro import requests From 49134b304fb603afb27c06e7051f1977bfab9f78 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 8 Jul 2019 23:03:58 +0200 Subject: [PATCH 14/94] Refactor write_to_file method --- esque/avromessage.py | 38 +++++++++++++++++++------------------- 1 file changed, 19 insertions(+), 19 deletions(-) diff --git a/esque/avromessage.py b/esque/avromessage.py index 33223cc3..49c99fd8 100644 --- a/esque/avromessage.py +++ b/esque/avromessage.py @@ -52,25 +52,7 @@ def write_message_to_file(self, message: Message, file: BinaryIO): ) self.current_key_schema_id = key_schema_id self.current_value_schema_id = value_schema_id - - directory = self.working_dir / self.schema_dir_name - directory.mkdir() - - (directory / "key_schema.avsc").write_text( - json.dumps( - self.schema_registry_client.get_schema_from_id( - key_schema_id - ).original_schema - ) - ) - - (directory / "value_schema.avsc").write_text( - json.dumps( - self.schema_registry_client.get_schema_from_id( - value_schema_id - ).original_schema - ) - ) + self._dump_schemata(key_schema_id, value_schema_id) serializable_message = { "key": decoded_key, @@ -79,6 +61,24 @@ def write_message_to_file(self, message: Message, file: BinaryIO): } pickle.dump(serializable_message, file) + def _dump_schemata(self, key_schema_id, value_schema_id): + directory = self.working_dir / self.schema_dir_name + directory.mkdir() + (directory / "key_schema.avsc").write_text( + json.dumps( + self.schema_registry_client.get_schema_from_id( + key_schema_id + ).original_schema + ) + ) + (directory / "value_schema.avsc").write_text( + json.dumps( + self.schema_registry_client.get_schema_from_id( + value_schema_id + ).original_schema + ) + ) + def decode_bytes(self, raw_data: Optional[bytes]) -> Tuple[int, Optional[Dict]]: if raw_data is None: return -1, None From 14dba5c8c2631b4827470f6731f0c1e314fed7a4 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 8 Jul 2019 23:23:11 +0200 Subject: [PATCH 15/94] Add more output and fix bugs --- esque/avromessage.py | 21 +++++++++++---------- esque/cli/commands.py | 3 ++- esque/clients.py | 1 + esque/message.py | 2 +- 4 files changed, 15 insertions(+), 12 deletions(-) diff --git a/esque/avromessage.py b/esque/avromessage.py index 49c99fd8..8a75001d 100644 --- a/esque/avromessage.py +++ b/esque/avromessage.py @@ -3,7 +3,7 @@ import pickle import struct from io import BytesIO -from typing import Optional, Tuple, Dict, BinaryIO +from typing import Optional, Tuple, Dict, BinaryIO, Iterable import itertools as it import fastavro @@ -102,18 +102,19 @@ class AvroFileReader(FileReader): def __init__(self, working_dir: pathlib.Path): self.working_dir = working_dir - def read_from_file(self, file: BinaryIO) -> Optional[KafkaMessage]: - try: - record = pickle.load(file) - except EOFError: - return None + def read_from_file(self, file: BinaryIO) -> Iterable[KafkaMessage]: + while True: + try: + record = pickle.load(file) + except EOFError: + return - schema_directory = self.working_dir / record["schema_directory_name"] + schema_directory = self.working_dir / record["schema_directory_name"] - key_schema = load_schema((schema_directory / "key_schema.avsc").read_text()) - value_schema = load_schema((schema_directory / "value_schema.avsc").read_text()) + key_schema = load_schema((schema_directory / "key_schema.avsc").read_text()) + value_schema = load_schema((schema_directory / "value_schema.avsc").read_text()) - return KafkaMessage(record["key"], record["value"], key_schema, value_schema) + yield KafkaMessage(record["key"], record["value"], key_schema, value_schema) def extract_schema_id(message: bytes) -> int: diff --git a/esque/cli/commands.py b/esque/cli/commands.py index 949fea03..fed8f348 100644 --- a/esque/cli/commands.py +++ b/esque/cli/commands.py @@ -378,6 +378,7 @@ def _produce_from_file( producer = AvroFileProducer.create(working_dir) else: producer = FileProducer.create(working_dir) + click.echo("\nStart producing to topic " + blue_bold(topic) + " in target context " + blue_bold(to_context)) number_produced_messages = producer.produce_from_file(topic) click.echo( green_bold(str(number_produced_messages)) @@ -398,11 +399,11 @@ def _consume_to_file( avro: bool, last: bool, ) -> int: - click.echo("\nStart consuming from source context " + blue_bold(from_context)) if avro: consumer = AvroFileConsumer.create(group_id, topic, working_dir, last) else: consumer = FileConsumer.create(group_id, topic, working_dir, last) + click.echo("\nStart consuming from topic " + blue_bold(topic) + " in source context " + blue_bold(from_context)) number_consumed_messages = consumer.consume_to_file(int(numbers)) click.echo(blue_bold(str(number_consumed_messages)) + " messages consumed.") diff --git a/esque/clients.py b/esque/clients.py index c347a3a6..273ee85f 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -195,6 +195,7 @@ def produce_from_file(self, topic_name: str) -> int: left_messages = self._producer.flush(1) if left_messages == 0: break + click.echo("Still {left_messages} messages left, flushing...") return counter diff --git a/esque/message.py b/esque/message.py index 73cac7ff..016ed68b 100644 --- a/esque/message.py +++ b/esque/message.py @@ -44,7 +44,7 @@ def read_from_file(self, file: BinaryIO) -> Iterable[KafkaMessage]: try: record = pickle.load(file) except EOFError: - continue + return yield KafkaMessage(record["key"], record["value"]) From 8e1777bf745e9030ab0ba9ab25b354568cc407bc Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Tue, 9 Jul 2019 07:21:55 +0200 Subject: [PATCH 16/94] Fix coding styles --- esque/avromessage.py | 4 +++- esque/cli/commands.py | 14 ++++++++++++-- 2 files changed, 15 insertions(+), 3 deletions(-) diff --git a/esque/avromessage.py b/esque/avromessage.py index 8a75001d..868f40f0 100644 --- a/esque/avromessage.py +++ b/esque/avromessage.py @@ -112,7 +112,9 @@ def read_from_file(self, file: BinaryIO) -> Iterable[KafkaMessage]: schema_directory = self.working_dir / record["schema_directory_name"] key_schema = load_schema((schema_directory / "key_schema.avsc").read_text()) - value_schema = load_schema((schema_directory / "value_schema.avsc").read_text()) + value_schema = load_schema( + (schema_directory / "value_schema.avsc").read_text() + ) yield KafkaMessage(record["key"], record["value"], key_schema, value_schema) diff --git a/esque/cli/commands.py b/esque/cli/commands.py index fed8f348..bb1fca8c 100644 --- a/esque/cli/commands.py +++ b/esque/cli/commands.py @@ -378,7 +378,12 @@ def _produce_from_file( producer = AvroFileProducer.create(working_dir) else: producer = FileProducer.create(working_dir) - click.echo("\nStart producing to topic " + blue_bold(topic) + " in target context " + blue_bold(to_context)) + click.echo( + "\nStart producing to topic " + + blue_bold(topic) + + " in target context " + + blue_bold(to_context) + ) number_produced_messages = producer.produce_from_file(topic) click.echo( green_bold(str(number_produced_messages)) @@ -403,7 +408,12 @@ def _consume_to_file( consumer = AvroFileConsumer.create(group_id, topic, working_dir, last) else: consumer = FileConsumer.create(group_id, topic, working_dir, last) - click.echo("\nStart consuming from topic " + blue_bold(topic) + " in source context " + blue_bold(from_context)) + click.echo( + "\nStart consuming from topic " + + blue_bold(topic) + + " in source context " + + blue_bold(from_context) + ) number_consumed_messages = consumer.consume_to_file(int(numbers)) click.echo(blue_bold(str(number_consumed_messages)) + " messages consumed.") From b08b0a3cb716601eca0101f945ebd9a593ddd10f Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Wed, 10 Jul 2019 20:49:16 +0200 Subject: [PATCH 17/94] Remove unused method in SchemaRegistryClient --- esque/schemaregistry.py | 4 ---- 1 file changed, 4 deletions(-) diff --git a/esque/schemaregistry.py b/esque/schemaregistry.py index ca70f607..b92b9398 100644 --- a/esque/schemaregistry.py +++ b/esque/schemaregistry.py @@ -24,7 +24,3 @@ def get_schema_from_server(self, schema_id: int, server_url: str) -> SchemaPair: response.raise_for_status() schema: Dict = json.loads(response.json()["schema"]) return SchemaPair(schema, fastavro.schema.parse_schema(schema)) - - def _extract_schema_id(self, message: bytes) -> int: - _, schema_id = struct.unpack(">bI", message[:5]) - return schema_id From 2b0430974dec0033542e18a2b851c7892208c6d0 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Wed, 10 Jul 2019 20:50:37 +0200 Subject: [PATCH 18/94] Adjust max line length to 119 --- .flake8 | 2 +- esque/avromessage.py | 43 ++-------- esque/cli/commands.py | 190 +++++++---------------------------------- esque/cli/options.py | 4 +- esque/cli/output.py | 20 +---- esque/clients.py | 54 +++--------- esque/cluster.py | 5 +- esque/config.py | 50 ++--------- esque/consumergroup.py | 45 +++------- esque/errors.py | 5 +- esque/message.py | 5 +- esque/topic.py | 26 ++---- 12 files changed, 86 insertions(+), 363 deletions(-) diff --git a/.flake8 b/.flake8 index 52c1e80b..cc0c6d75 100644 --- a/.flake8 +++ b/.flake8 @@ -1,5 +1,5 @@ [flake8] ignore = E203, E266, E501, W503 -max-line-length = 80 +max-line-length = 119 max-complexity = 18 select = B,C,E,F,W,T4,B9 \ No newline at end of file diff --git a/esque/avromessage.py b/esque/avromessage.py index 868f40f0..45bda51e 100644 --- a/esque/avromessage.py +++ b/esque/avromessage.py @@ -15,13 +15,7 @@ class DecodedAvroMessage: - def __init__( - self, - key: Optional[Dict], - value: Optional[Dict], - key_schema_id: int, - value_schema_id: int, - ): + def __init__(self, key: Optional[Dict], value: Optional[Dict], key_schema_id: int, value_schema_id: int): self.key = key self.value = value self.key_schema_id = key_schema_id @@ -29,9 +23,7 @@ def __init__( class AvroFileWriter(FileWriter): - def __init__( - self, working_dir: pathlib.Path, schema_registry_client: SchemaRegistryClient - ): + def __init__(self, working_dir: pathlib.Path, schema_registry_client: SchemaRegistryClient): self.working_dir = working_dir self.schema_registry_client = schema_registry_client self.current_key_schema_id = None @@ -42,14 +34,10 @@ def __init__( def write_message_to_file(self, message: Message, file: BinaryIO): key_schema_id, decoded_key = self.decode_bytes(message.key()) value_schema_id, decoded_value = self.decode_bytes(message.value()) - decoded_message = DecodedAvroMessage( - decoded_key, decoded_value, key_schema_id, value_schema_id - ) + decoded_message = DecodedAvroMessage(decoded_key, decoded_value, key_schema_id, value_schema_id) if self.schema_changed(decoded_message) or self.schema_dir_name is None: - self.schema_dir_name = ( - f"{next(self.schema_version):04}_{key_schema_id}_{value_schema_id}" - ) + self.schema_dir_name = f"{next(self.schema_version):04}_{key_schema_id}_{value_schema_id}" self.current_key_schema_id = key_schema_id self.current_value_schema_id = value_schema_id self._dump_schemata(key_schema_id, value_schema_id) @@ -65,18 +53,10 @@ def _dump_schemata(self, key_schema_id, value_schema_id): directory = self.working_dir / self.schema_dir_name directory.mkdir() (directory / "key_schema.avsc").write_text( - json.dumps( - self.schema_registry_client.get_schema_from_id( - key_schema_id - ).original_schema - ) + json.dumps(self.schema_registry_client.get_schema_from_id(key_schema_id).original_schema) ) (directory / "value_schema.avsc").write_text( - json.dumps( - self.schema_registry_client.get_schema_from_id( - value_schema_id - ).original_schema - ) + json.dumps(self.schema_registry_client.get_schema_from_id(value_schema_id).original_schema) ) def decode_bytes(self, raw_data: Optional[bytes]) -> Tuple[int, Optional[Dict]]: @@ -85,16 +65,13 @@ def decode_bytes(self, raw_data: Optional[bytes]) -> Tuple[int, Optional[Dict]]: with BytesIO(raw_data) as fake_stream: schema_id = extract_schema_id(fake_stream.read(5)) - parsed_schema = self.schema_registry_client.get_schema_from_id( - schema_id - ).parsed_schema + parsed_schema = self.schema_registry_client.get_schema_from_id(schema_id).parsed_schema record = fastavro.schemaless_reader(fake_stream, parsed_schema) return schema_id, record def schema_changed(self, decoded_message: DecodedAvroMessage) -> bool: return ( - self.current_value_schema_id != decoded_message.value_schema_id - and decoded_message.value is not None + self.current_value_schema_id != decoded_message.value_schema_id and decoded_message.value is not None ) or self.current_key_schema_id != decoded_message.key_schema_id @@ -112,9 +89,7 @@ def read_from_file(self, file: BinaryIO) -> Iterable[KafkaMessage]: schema_directory = self.working_dir / record["schema_directory_name"] key_schema = load_schema((schema_directory / "key_schema.avsc").read_text()) - value_schema = load_schema( - (schema_directory / "value_schema.avsc").read_text() - ) + value_schema = load_schema((schema_directory / "value_schema.avsc").read_text()) yield KafkaMessage(record["key"], record["value"], key_schema, value_schema) diff --git a/esque/cli/commands.py b/esque/cli/commands.py index bb1fca8c..8e711118 100644 --- a/esque/cli/commands.py +++ b/esque/cli/commands.py @@ -11,30 +11,12 @@ from esque.broker import Broker from esque.cli.helpers import ensure_approval, DeleteOnFinished from esque.cli.options import State, no_verify_option, pass_state -from esque.cli.output import ( - bold, - pretty, - pretty_topic_diffs, - get_output_new_topics, - blue_bold, - green_bold, -) -from esque.clients import ( - FileConsumer, - FileProducer, - AvroFileProducer, - AvroFileConsumer, - PingConsumer, - PingProducer, -) +from esque.cli.output import bold, pretty, pretty_topic_diffs, get_output_new_topics, blue_bold, green_bold +from esque.clients import FileConsumer, FileProducer, AvroFileProducer, AvroFileConsumer, PingConsumer, PingProducer from esque.cluster import Cluster from esque.config import PING_TOPIC, Config, PING_GROUP_ID from esque.consumergroup import ConsumerGroupController -from esque.errors import ( - ConsumerGroupDoesNotExistException, - ContextNotDefinedException, - TopicAlreadyExistsException, -) +from esque.errors import ConsumerGroupDoesNotExistException, ContextNotDefinedException, TopicAlreadyExistsException from esque.topic import TopicController @@ -72,19 +54,12 @@ def edit(): # TODO: Figure out how to pass the state object def list_topics(ctx, args, incomplete): cluster = Cluster() - return [ - topic["name"] - for topic in TopicController(cluster).list_topics(search_string=incomplete) - ] + return [topic["name"] for topic in TopicController(cluster).list_topics(search_string=incomplete)] def list_contexts(ctx, args, incomplete): config = Config() - return [ - context - for context in config.available_contexts - if context.startswith(incomplete) - ] + return [context for context in config.available_contexts if context.startswith(incomplete)] @edit.command("topic") @@ -125,9 +100,7 @@ def ctx(state, context): def create_topic(state: State, topic_name: str): if ensure_approval("Are you sure?", no_verify=state.no_verify): topic_controller = TopicController(state.cluster) - TopicController(state.cluster).create_topics( - [(topic_controller.get_topic(topic_name))] - ) + TopicController(state.cluster).create_topics([(topic_controller.get_topic(topic_name))]) @esque.command("apply", help="Apply a configuration") @@ -148,29 +121,16 @@ def apply(state: State, file: str): ) ) editable_topics = topic_controller.filter_existing_topics(topics) - topics_to_be_changed = [ - topic for topic in editable_topics if topic.config_diff() != {} - ] - topic_config_diffs = { - topic.name: topic.config_diff() for topic in topics_to_be_changed - } + topics_to_be_changed = [topic for topic in editable_topics if topic.config_diff() != {}] + topic_config_diffs = {topic.name: topic.config_diff() for topic in topics_to_be_changed} if len(topic_config_diffs) > 0: click.echo(pretty_topic_diffs(topic_config_diffs)) - if ensure_approval( - "Are you sure to change configs?", no_verify=state.no_verify - ): + if ensure_approval("Are you sure to change configs?", no_verify=state.no_verify): topic_controller.alter_configs(topics_to_be_changed) click.echo( click.style( - pretty( - { - "Successfully changed topics": [ - topic.name for topic in topics_to_be_changed - ] - } - ), - fg="green", + pretty({"Successfully changed topics": [topic.name for topic in topics_to_be_changed]}), fg="green" ) ) else: @@ -179,30 +139,17 @@ def apply(state: State, file: str): new_topics = [topic for topic in topics if topic not in editable_topics] if len(new_topics) > 0: click.echo(get_output_new_topics(new_topics)) - if ensure_approval( - "Are you sure to create the new topics?", no_verify=state.no_verify - ): + if ensure_approval("Are you sure to create the new topics?", no_verify=state.no_verify): topic_controller.create_topics(new_topics) click.echo( - click.style( - pretty( - { - "Successfully created topics": [ - topic.name for topic in new_topics - ] - } - ), - fg="green", - ) + click.style(pretty({"Successfully created topics": [topic.name for topic in new_topics]}), fg="green") ) else: click.echo("No new topics to create.") @delete.command("topic") -@click.argument( - "topic-name", required=True, type=click.STRING, autocompletion=list_topics -) +@click.argument("topic-name", required=True, type=click.STRING, autocompletion=list_topics) @no_verify_option @pass_state def delete_topic(state: State, topic_name: str): @@ -214,9 +161,7 @@ def delete_topic(state: State, topic_name: str): @describe.command("topic") -@click.argument( - "topic-name", required=True, type=click.STRING, autocompletion=list_topics -) +@click.argument("topic-name", required=True, type=click.STRING, autocompletion=list_topics) @pass_state def describe_topic(state, topic_name): partitions, config = TopicController(state.cluster).get_topic(topic_name).describe() @@ -230,17 +175,13 @@ def describe_topic(state, topic_name): @get.command("offsets") -@click.argument( - "topic-name", required=False, type=click.STRING, autocompletion=list_topics -) +@click.argument("topic-name", required=False, type=click.STRING, autocompletion=list_topics) @pass_state def get_offsets(state, topic_name): # TODO: Gathering of all offsets takes super long topics = TopicController(state.cluster).list_topics(search_string=topic_name) - offsets = { - topic.name: max([v for v in topic.get_offsets().values()]) for topic in topics - } + offsets = {topic.name: max([v for v in topic.get_offsets().values()]) for topic in topics} click.echo(pretty(offsets)) @@ -255,15 +196,11 @@ def describe_broker(state, broker_id): @describe.command("consumergroup") @click.argument("consumer-id", required=False) -@click.option( - "-v", "--verbose", help="More detailed information.", default=False, is_flag=True -) +@click.option("-v", "--verbose", help="More detailed information.", default=False, is_flag=True) @pass_state def describe_consumergroup(state, consumer_id, verbose): try: - consumer_group = ConsumerGroupController(state.cluster).get_consumergroup( - consumer_id - ) + consumer_group = ConsumerGroupController(state.cluster).get_consumergroup(consumer_id) consumer_group_desc = consumer_group.describe(verbose=verbose) click.echo(pretty(consumer_group_desc, break_lists=True)) @@ -296,47 +233,15 @@ def get_topics(state, topic, o): click.echo(topic.name) -@esque.command( - "transfer", help="Transfer messages of a topic from one environment to another." -) +@esque.command("transfer", help="Transfer messages of a topic from one environment to another.") @click.argument("topic", required=True) -@click.option( - "-f", - "--from", - "from_context", - help="Source Context", - type=click.STRING, - required=True, -) -@click.option( - "-t", - "--to", - "to_context", - help="Destination context", - type=click.STRING, - required=True, -) -@click.option( - "-n", "--numbers", help="Number of messages", type=click.INT, required=True -) +@click.option("-f", "--from", "from_context", help="Source Context", type=click.STRING, required=True) +@click.option("-t", "--to", "to_context", help="Destination context", type=click.STRING, required=True) +@click.option("-n", "--numbers", help="Number of messages", type=click.INT, required=True) @click.option("--last/--first", default=False) -@click.option( - "-a", - "--avro", - help="Set this flag if the topic contains avro data", - default=False, - is_flag=True, -) +@click.option("-a", "--avro", help="Set this flag if the topic contains avro data", default=False, is_flag=True) @pass_state -def transfer( - state: State, - topic: str, - from_context: str, - to_context: str, - numbers: int, - last: bool, - avro: bool, -): +def transfer(state: State, topic: str, from_context: str, to_context: str, numbers: int, last: bool, avro: bool): current_timestamp_milliseconds = int(round(time.time() * 1000)) temp_name = topic + "_" + str(current_timestamp_milliseconds) group_id = "group_for_" + temp_name @@ -345,24 +250,13 @@ def transfer( state.config.context_switch(from_context) with DeleteOnFinished(base_dir) as working_dir: - number_consumed_messages = _consume_to_file( - working_dir, topic, group_id, from_context, numbers, avro, last - ) + number_consumed_messages = _consume_to_file(working_dir, topic, group_id, from_context, numbers, avro, last) if number_consumed_messages == 0: - click.echo( - click.style( - "Execution stopped, because no messages consumed.", fg="red" - ) - ) + click.echo(click.style("Execution stopped, because no messages consumed.", fg="red")) return - click.echo( - "\nReady to produce to context " - + blue_bold(to_context) - + " and target topic " - + blue_bold(topic) - ) + click.echo("\nReady to produce to context " + blue_bold(to_context) + " and target topic " + blue_bold(topic)) if not ensure_approval("Do you want to proceed?\n", no_verify=state.no_verify): return @@ -371,19 +265,12 @@ def transfer( _produce_from_file(topic, to_context, working_dir, avro) -def _produce_from_file( - topic: str, to_context: str, working_dir: pathlib.Path, avro: bool -): +def _produce_from_file(topic: str, to_context: str, working_dir: pathlib.Path, avro: bool): if avro: producer = AvroFileProducer.create(working_dir) else: producer = FileProducer.create(working_dir) - click.echo( - "\nStart producing to topic " - + blue_bold(topic) - + " in target context " - + blue_bold(to_context) - ) + click.echo("\nStart producing to topic " + blue_bold(topic) + " in target context " + blue_bold(to_context)) number_produced_messages = producer.produce_from_file(topic) click.echo( green_bold(str(number_produced_messages)) @@ -396,24 +283,13 @@ def _produce_from_file( def _consume_to_file( - working_dir: pathlib.Path, - topic: str, - group_id: str, - from_context: str, - numbers: int, - avro: bool, - last: bool, + working_dir: pathlib.Path, topic: str, group_id: str, from_context: str, numbers: int, avro: bool, last: bool ) -> int: if avro: consumer = AvroFileConsumer.create(group_id, topic, working_dir, last) else: consumer = FileConsumer.create(group_id, topic, working_dir, last) - click.echo( - "\nStart consuming from topic " - + blue_bold(topic) - + " in source context " - + blue_bold(from_context) - ) + click.echo("\nStart consuming from topic " + blue_bold(topic) + " in source context " + blue_bold(from_context)) number_consumed_messages = consumer.consume_to_file(int(numbers)) click.echo(blue_bold(str(number_consumed_messages)) + " messages consumed.") @@ -450,6 +326,4 @@ def ping(state, times, wait): topic_controller.delete_topic(topic_controller.get_topic(PING_TOPIC)) click.echo("--- statistics ---") click.echo(f"{len(deltas)} messages sent/received") - click.echo( - f"min/avg/max = {min(deltas):.2f}/{(sum(deltas)/len(deltas)):.2f}/{max(deltas):.2f} ms" - ) + click.echo(f"min/avg/max = {min(deltas):.2f}/{(sum(deltas)/len(deltas)):.2f}/{max(deltas):.2f} ms") diff --git a/esque/cli/options.py b/esque/cli/options.py index d631c238..a2636901 100644 --- a/esque/cli/options.py +++ b/esque/cli/options.py @@ -34,9 +34,7 @@ def cluster(self): self._cluster = Cluster() return self._cluster except NoBrokersAvailableError: - raise ClickException( - f"Could not reach Kafka Brokers {self.config.bootstrap_servers}" - ) + raise ClickException(f"Could not reach Kafka Brokers {self.config.bootstrap_servers}") pass_state = make_pass_decorator(State, ensure=True) diff --git a/esque/cli/output.py b/esque/cli/output.py index 31c1e5c3..529793d2 100644 --- a/esque/cli/output.py +++ b/esque/cli/output.py @@ -39,9 +39,7 @@ def pretty_list(l: List[Any], *, break_lists=False, list_separator: str = ", ") break_lists = True if break_lists: - sub_elements = ( - "\n ".join(elem.splitlines(keepends=False)) for elem in list_output - ) + sub_elements = ("\n ".join(elem.splitlines(keepends=False)) for elem in list_output) return "- " + "\n- ".join(sub_elements) else: return list_separator.join(list_output) @@ -124,21 +122,13 @@ def pretty_duration(value: Any, *, multiplier: int = 1) -> str: return pendulum.duration(milliseconds=value).in_words() -def pretty_topic_diffs( - topics_config_diff: Dict[str, Dict[str, Tuple[str, str]]] -) -> str: +def pretty_topic_diffs(topics_config_diff: Dict[str, Dict[str, Tuple[str, str]]]) -> str: output = [] for name, diff in topics_config_diff.items(): config_diff_attributes = {} for attribute, value in diff.items(): config_diff_attributes[attribute] = value[0] + " -> " + value[1] - output.append( - { - click.style(name, bold=True, fg="yellow"): { - "Config Diff": config_diff_attributes - } - } - ) + output.append({click.style(name, bold=True, fg="yellow"): {"Config Diff": config_diff_attributes}}) return pretty({"Topics to change": output}) @@ -151,9 +141,7 @@ def get_output_new_topics(new_topics: List[Topic]) -> str: "replication_factor: ": topic.replication_factor, "config": topic.config, } - new_topic_configs.append( - {click.style(topic.name, bold=True, fg="green"): new_topic_config} - ) + new_topic_configs.append({click.style(topic.name, bold=True, fg="green"): new_topic_config}) return pretty({"New topics to create": new_topic_configs}) diff --git a/esque/clients.py b/esque/clients.py index 273ee85f..ad0d4f9c 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -12,13 +12,7 @@ from esque.config import Config from esque.errors import raise_for_kafka_error, raise_for_message, KafkaException from esque.helpers import delivery_callback, delta_t -from esque.message import ( - KafkaMessage, - PlainTextFileReader, - PlainTextFileWriter, - FileReader, - FileWriter, -) +from esque.message import KafkaMessage, PlainTextFileReader, PlainTextFileWriter, FileReader, FileWriter from esque.schemaregistry import SchemaRegistryClient DEFAULT_RETENTION_MS = 7 * 24 * 60 * 60 * 1000 @@ -63,31 +57,20 @@ def consume_ping(self) -> Optional[Tuple[str, int]]: class FileConsumer(Consumer): - def __init__( - self, - group_id: str, - topic_name: str, - working_dir: pathlib.Path, - file_writer: FileWriter, - last: bool, - ): + def __init__(self, group_id: str, topic_name: str, working_dir: pathlib.Path, file_writer: FileWriter, last: bool): super().__init__(group_id, topic_name, last) self.working_dir = working_dir offset_reset = "earliest" if last: offset_reset = "latest" - self._config.update( - {"default.topic.config": {"auto.offset.reset": offset_reset}} - ) + self._config.update({"default.topic.config": {"auto.offset.reset": offset_reset}}) self._consumer = confluent_kafka.Consumer(self._config) self._assign_exact_partitions(topic_name) self.file_writer = file_writer @classmethod - def create( - cls, group_id: str, topic_name: str, working_dir: pathlib.Path, last: bool - ): + def create(cls, group_id: str, topic_name: str, working_dir: pathlib.Path, last: bool): return cls(group_id, topic_name, working_dir, PlainTextFileWriter(), last) def consume_to_file(self, amount: int) -> int: @@ -126,21 +109,12 @@ def _consume_single_message(self) -> Optional[Message]: class AvroFileConsumer(FileConsumer): - def __init__( - self, - group_id: str, - topic_name: str, - working_dir: pathlib.Path, - file_writer: FileWriter, - last: bool, - ): + def __init__(self, group_id: str, topic_name: str, working_dir: pathlib.Path, file_writer: FileWriter, last: bool): super().__init__(group_id, topic_name, working_dir, file_writer, last) self.writer = file_writer @classmethod - def create( - cls, group_id: str, topic_name: str, working_dir: pathlib.Path, last: bool - ): + def create(cls, group_id: str, topic_name: str, working_dir: pathlib.Path, last: bool): schema_registry_client = SchemaRegistryClient(Config().schema_registry) file_writer = AvroFileWriter(working_dir, schema_registry_client) return cls(group_id, topic_name, working_dir, file_writer, last) @@ -149,32 +123,24 @@ def create( class PingProducer(object): def __init__(self): self._config = Config().create_confluent_config() - self._config.update( - {"on_delivery": delivery_callback, "error_cb": raise_for_kafka_error} - ) + self._config.update({"on_delivery": delivery_callback, "error_cb": raise_for_kafka_error}) self._producer = confluent_kafka.Producer(self._config) def produce_ping(self, topic_name: str): start = pendulum.now() - self._producer.produce( - topic=topic_name, key=str(0), value=str(pendulum.now().timestamp()) - ) + self._producer.produce(topic=topic_name, key=str(0), value=str(pendulum.now().timestamp())) while True: left_messages = self._producer.flush(1) if left_messages == 0: break - click.echo( - f"{delta_t(start)} | Still {left_messages} messages left, flushing..." - ) + click.echo(f"{delta_t(start)} | Still {left_messages} messages left, flushing...") class FileProducer(object): def __init__(self, working_dir: pathlib.Path, file_reader: FileReader): self._config = Config().create_confluent_config() - self._config.update( - {"on_delivery": delivery_callback, "error_cb": raise_for_kafka_error} - ) + self._config.update({"on_delivery": delivery_callback, "error_cb": raise_for_kafka_error}) self._producer = confluent_kafka.Producer(self._config) self.working_dir = working_dir diff --git a/esque/cluster.py b/esque/cluster.py index 26a7ba10..22dc44d2 100644 --- a/esque/cluster.py +++ b/esque/cluster.py @@ -24,10 +24,7 @@ def bootstrap_servers(self): def brokers(self): metadata = self.confluent_client.list_topics(timeout=1) return sorted( - [ - {"id": broker.id, "host": broker.host, "port": broker.port} - for broker in metadata.brokers.values() - ], + [{"id": broker.id, "host": broker.host, "port": broker.port} for broker in metadata.brokers.values()], key=operator.itemgetter("id"), ) diff --git a/esque/config.py b/esque/config.py index c2ef0ead..42bde480 100644 --- a/esque/config.py +++ b/esque/config.py @@ -42,13 +42,7 @@ def __init__(self): @property def available_contexts(self): - return sorted( - [ - key.split(".")[1] - for key in self._cfg.keys() - if key.startswith("Context.") - ] - ) + return sorted([key.split(".")[1] for key in self._cfg.keys() if key.startswith("Context.")]) @property def current_context(self): @@ -61,8 +55,7 @@ def _current_section(self): @property def current_context_dict(self) -> Dict[str, Any]: return { - option: self._cfg.get(self._current_section, option) - for option in self._cfg.options(self._current_section) + option: self._cfg.get(self._current_section, option) for option in self._cfg.options(self._current_section) } @property @@ -91,20 +84,13 @@ def schema_registry(self) -> str: @property def bootstrap_servers(self): if self.bootstrap_domain: - return [ - f"{host_name}.{self.bootstrap_domain}:{self.bootstrap_port}" - for host_name in self.bootstrap_hosts - ] - return [ - f"{host_name}:{self.bootstrap_port}" for host_name in self.bootstrap_hosts - ] + return [f"{host_name}.{self.bootstrap_domain}:{self.bootstrap_port}" for host_name in self.bootstrap_hosts] + return [f"{host_name}:{self.bootstrap_port}" for host_name in self.bootstrap_hosts] def context_switch(self, context: str): click.echo((f"Switched to context: {context}")) if context not in self.available_contexts: - raise ContextNotDefinedException( - f"{context} not defined in {config_path()}" - ) + raise ContextNotDefinedException(f"{context} not defined in {config_path()}") self._update_config("Context", "current", context) def _update_config(self, section: str, key: str, value: str): @@ -116,37 +102,19 @@ def create_pykafka_config(self) -> Dict[str, str]: return {"hosts": ",".join(self.bootstrap_servers)} def create_confluent_config( - self, - *, - debug: bool = False, - ssl: bool = False, - auth: Optional[Tuple[str, str]] = None, + self, *, debug: bool = False, ssl: bool = False, auth: Optional[Tuple[str, str]] = None ) -> Dict[str, str]: - base_config = { - "bootstrap.servers": ",".join(self.bootstrap_servers), - "security.protocol": "PLAINTEXT", - } + base_config = {"bootstrap.servers": ",".join(self.bootstrap_servers), "security.protocol": "PLAINTEXT"} config = base_config.copy() if debug: config.update({"debug": "all", "log_level": "2"}) if ssl: - config.update( - { - "ssl.ca.location": "/etc/ssl/certs/GlobalSign_Root_CA.pem", - "security.protocol": "SSL", - } - ) + config.update({"ssl.ca.location": "/etc/ssl/certs/GlobalSign_Root_CA.pem", "security.protocol": "SSL"}) if auth: user, pw = auth - config.update( - { - "sasl.mechanisms": "SCRAM-SHA-512", - "sasl.username": user, - "sasl.password": pw, - } - ) + config.update({"sasl.mechanisms": "SCRAM-SHA-512", "sasl.username": user, "sasl.password": pw}) config["security.protocol"] = "SASL_" + config["security.protocol"] return config diff --git a/esque/consumergroup.py b/esque/consumergroup.py index 6831bb06..b8e363a6 100644 --- a/esque/consumergroup.py +++ b/esque/consumergroup.py @@ -24,8 +24,7 @@ def _pykafka_group_coordinator(self) -> pykafka.Broker: consumer_id = self.id.encode("UTF-8") if not self._pykafka_group_coordinator_instance: self._pykafka_group_coordinator_instance: pykafka.Broker = cast( - pykafka.Broker, - self.cluster.pykafka_client.cluster.get_group_coordinator(consumer_id), + pykafka.Broker, self.cluster.pykafka_client.cluster.get_group_coordinator(consumer_id) ) return self._pykafka_group_coordinator_instance @@ -40,10 +39,7 @@ def describe(self, *, verbose=False): topic_assignment = self._get_member_assignment(meta["members"]) consumer_offsets = self.get_consumer_offsets( - self._pykafka_group_coordinator, - consumer_id, - topic_assignment, - verbose=verbose, + self._pykafka_group_coordinator, consumer_id, topic_assignment, verbose=verbose ) return { @@ -54,9 +50,7 @@ def describe(self, *, verbose=False): } raise ConsumerGroupDoesNotExistException() - def get_consumer_offsets( - self, group_coordinator, consumer_id, topic_assignment, verbose - ): + def get_consumer_offsets(self, group_coordinator, consumer_id, topic_assignment, verbose): consumer_offsets = self._unpack_offset_response( group_coordinator.fetch_consumer_group_offsets(consumer_id, preqs=[]) ) @@ -69,8 +63,7 @@ def get_consumer_offsets( "consumer_offset": consumer_offset, "topic_low_watermark": topic_offsets[partition_id][0], "topic_high_watermark": topic_offsets[partition_id][1], - "consumer_lag": topic_offsets[partition_id][1] - - consumer_offset, + "consumer_lag": topic_offsets[partition_id][1] - consumer_offset, } return consumer_offsets for topic in consumer_offsets.keys(): @@ -84,10 +77,7 @@ def get_consumer_offsets( for partition_id, consumer_offset in consumer_offsets[topic].items(): current_offset = consumer_offset old_min, old_max = new_consumer_offsets["consumer_offset"] - new_consumer_offsets["consumer_offset"] = ( - min(old_min, current_offset), - max(old_max, current_offset), - ) + new_consumer_offsets["consumer_offset"] = (min(old_min, current_offset), max(old_max, current_offset)) old_min, old_max = new_consumer_offsets["topic_low_watermark"] new_consumer_offsets["topic_low_watermark"] = ( @@ -109,9 +99,7 @@ def get_consumer_offsets( return new_consumer_offsets - def _get_member_assignment( - self, member_assignment: Dict[str, Any] - ) -> List[PartitionOffsetFetchRequest]: + def _get_member_assignment(self, member_assignment: Dict[str, Any]) -> List[PartitionOffsetFetchRequest]: """ Creates a list of style [PartitionOffsetFetchRequest('topic', partition_id)] """ @@ -126,15 +114,12 @@ def _unpack_offset_response(self, resp: OffsetFetchResponseV1) -> Dict[str, Any] return { topic_name: { - partition_id: partition_data._asdict()["offset"] - for partition_id, partition_data in partitions.items() + partition_id: partition_data._asdict()["offset"] for partition_id, partition_data in partitions.items() } for topic_name, partitions in resp.topics.items() } - def _unpack_consumer_group_response( - self, resp: DescribeGroupResponse - ) -> Dict[str, Any]: + def _unpack_consumer_group_response(self, resp: DescribeGroupResponse) -> Dict[str, Any]: return { "group_id": resp.group_id, "protocol": resp.protocol, @@ -148,9 +133,7 @@ def _unpack_consumer_group_response( "client_host": member.client_host, "member_metadata": { # "version": member.member_metadata.version, - "subscription": [ - topic for topic in member.member_metadata.topic_names - ], + "subscription": [topic for topic in member.member_metadata.topic_names], # "client": member.member_metadata.user_data, }, "member_assignment": { @@ -174,13 +157,7 @@ def get_consumergroup(self, consumer_id) -> ConsumerGroup: return ConsumerGroup(consumer_id, self.cluster) def list_consumer_groups(self) -> List[str]: - brokers: Dict[ - int, pykafka.broker.Broker - ] = self.cluster.pykafka_client.cluster.brokers + brokers: Dict[int, pykafka.broker.Broker] = self.cluster.pykafka_client.cluster.brokers return list( - set( - group.decode("UTF-8") - for _, broker in brokers.items() - for group in broker.list_groups().groups - ) + set(group.decode("UTF-8") for _, broker in brokers.items() for group in broker.list_groups().groups) ) diff --git a/esque/errors.py b/esque/errors.py index 81a690f0..6913b47b 100644 --- a/esque/errors.py +++ b/esque/errors.py @@ -68,10 +68,7 @@ class TopicDoesNotExistException(Exception): pass -ERROR_LOOKUP: Dict[int, Type[KafkaException]] = { - 36: TopicAlreadyExistsException, - -191: EndOfPartitionReachedException, -} +ERROR_LOOKUP: Dict[int, Type[KafkaException]] = {36: TopicAlreadyExistsException, -191: EndOfPartitionReachedException} # BROKER_NOT_AVAILABLE = 8 # CLUSTER_AUTHORIZATION_FAILED = 31 diff --git a/esque/message.py b/esque/message.py index 016ed68b..f99bdb60 100644 --- a/esque/message.py +++ b/esque/message.py @@ -31,10 +31,7 @@ def read_from_file(self, file: BinaryIO) -> Iterable[KafkaMessage]: class PlainTextFileWriter(FileWriter): def write_message_to_file(self, message: Message, file: BinaryIO): decoded_message = decode_message(message) - serializable_message = { - "key": decoded_message.key, - "value": decoded_message.value, - } + serializable_message = {"key": decoded_message.key, "value": decoded_message.value} pickle.dump(serializable_message, file) diff --git a/esque/topic.py b/esque/topic.py index 9a31127a..0e245c42 100644 --- a/esque/topic.py +++ b/esque/topic.py @@ -8,11 +8,7 @@ from esque.cluster import Cluster from esque.errors import TopicDoesNotExistException, raise_for_kafka_exception -from esque.helpers import ( - ensure_kafka_futures_done, - invalidate_cache_after, - unpack_confluent_config, -) +from esque.helpers import ensure_kafka_futures_done, invalidate_cache_after, unpack_confluent_config class Topic: @@ -29,9 +25,7 @@ def __init__( self._pykafka_topic_instance = None self._confluent_topic_instance = None self.num_partitions = num_partitions if num_partitions is not None else 1 - self.replication_factor = ( - replication_factor if replication_factor is not None else 1 - ) + self.replication_factor = replication_factor if replication_factor is not None else 1 self.config = config if config is not None else {} def as_dict(self) -> Dict[str, Union[int, Dict[str, str]]]: @@ -52,9 +46,7 @@ def from_yaml(self, data) -> None: @property def _pykafka_topic(self) -> pykafka.Topic: if not self._pykafka_topic_instance: - self._pykafka_topic_instance = self.cluster.pykafka_client.cluster.topics[ - self.name - ] + self._pykafka_topic_instance = self.cluster.pykafka_client.cluster.topics[self.name] return self._pykafka_topic_instance @property @@ -141,9 +133,7 @@ def __init__(self, cluster: Cluster): self.cluster: Cluster = cluster @raise_for_kafka_exception - def list_topics( - self, *, search_string: str = None, sort=True, hide_internal=True - ) -> List[Topic]: + def list_topics(self, *, search_string: str = None, sort=True, hide_internal=True) -> List[Topic]: self.cluster.confluent_client.poll(timeout=1) topics = self.cluster.confluent_client.list_topics().topics topics = [self.get_topic(t.topic) for t in topics.values()] @@ -180,9 +170,7 @@ def create_topics(self, topics: List[Topic]): @invalidate_cache_after def alter_configs(self, topics: List[Topic]): for topic in topics: - config_resource = ConfigResource( - ConfigResource.Type.TOPIC, topic.name, topic.config - ) + config_resource = ConfigResource(ConfigResource.Type.TOPIC, topic.name, topic.config) future_list = self.cluster.confluent_client.alter_configs([config_resource]) ensure_kafka_futures_done(list(future_list.values())) @@ -199,6 +187,4 @@ def get_topic( replication_factor: int = None, config: Dict[str, str] = None, ) -> Topic: - return Topic( - topic_name, self.cluster, num_partitions, replication_factor, config - ) + return Topic(topic_name, self.cluster, num_partitions, replication_factor, config) From 80fa96bf7f8d28b5b0c3b5e68ffa6f8bc806c45c Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Wed, 10 Jul 2019 21:05:50 +0200 Subject: [PATCH 19/94] Add flag to keep message file --- esque/cli/commands.py | 22 ++++++++++++++++------ esque/cli/helpers.py | 7 ++++--- 2 files changed, 20 insertions(+), 9 deletions(-) diff --git a/esque/cli/commands.py b/esque/cli/commands.py index 8e711118..65d34e1a 100644 --- a/esque/cli/commands.py +++ b/esque/cli/commands.py @@ -9,7 +9,7 @@ from esque.__version__ import __version__ from esque.broker import Broker -from esque.cli.helpers import ensure_approval, DeleteOnFinished +from esque.cli.helpers import ensure_approval, HandleFileOnFinished from esque.cli.options import State, no_verify_option, pass_state from esque.cli.output import bold, pretty, pretty_topic_diffs, get_output_new_topics, blue_bold, green_bold from esque.clients import FileConsumer, FileProducer, AvroFileProducer, AvroFileConsumer, PingConsumer, PingProducer @@ -240,16 +240,26 @@ def get_topics(state, topic, o): @click.option("-n", "--numbers", help="Number of messages", type=click.INT, required=True) @click.option("--last/--first", default=False) @click.option("-a", "--avro", help="Set this flag if the topic contains avro data", default=False, is_flag=True) +@click.option( + "-k", + "--keep", + "keep_file", + help="Set this flag if the file with consumed messages should be kept.", + default=False, + is_flag=True, +) @pass_state -def transfer(state: State, topic: str, from_context: str, to_context: str, numbers: int, last: bool, avro: bool): +def transfer( + state: State, topic: str, from_context: str, to_context: str, numbers: int, last: bool, avro: bool, keep_file: bool +): current_timestamp_milliseconds = int(round(time.time() * 1000)) - temp_name = topic + "_" + str(current_timestamp_milliseconds) - group_id = "group_for_" + temp_name - directory_name = "temp-file_" + temp_name + unique_name = topic + "_" + str(current_timestamp_milliseconds) + group_id = "group_for_" + unique_name + directory_name = "message_" + unique_name base_dir = Path(directory_name) state.config.context_switch(from_context) - with DeleteOnFinished(base_dir) as working_dir: + with HandleFileOnFinished(base_dir, keep_file) as working_dir: number_consumed_messages = _consume_to_file(working_dir, topic, group_id, from_context, numbers, avro, last) if number_consumed_messages == 0: diff --git a/esque/cli/helpers.py b/esque/cli/helpers.py index 1971da83..bb798c7a 100644 --- a/esque/cli/helpers.py +++ b/esque/cli/helpers.py @@ -10,8 +10,9 @@ def ensure_approval(question: str, *, no_verify: bool = False) -> bool: return click.confirm(question) -class DeleteOnFinished: - def __init__(self, dir_: pathlib.Path): +class HandleFileOnFinished: + def __init__(self, dir_: pathlib.Path, keep_file: bool): + self.keep_file = keep_file self._dir = dir_ self._dir.mkdir(parents=True) @@ -19,5 +20,5 @@ def __enter__(self) -> pathlib.Path: return self._dir def __exit__(self, exc_type, exc_val, exc_tb): - if self._dir.exists(): + if not self.keep_file and self._dir.exists(): shutil.rmtree(self._dir) From 1f5578e0d16d85dee73658f3bb5b4b2a17dbed29 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Wed, 10 Jul 2019 21:10:08 +0200 Subject: [PATCH 20/94] Remove factory methods --- esque/cli/commands.py | 8 ++++---- esque/clients.py | 37 ++++++++++--------------------------- 2 files changed, 14 insertions(+), 31 deletions(-) diff --git a/esque/cli/commands.py b/esque/cli/commands.py index 65d34e1a..fa9d02ee 100644 --- a/esque/cli/commands.py +++ b/esque/cli/commands.py @@ -277,9 +277,9 @@ def transfer( def _produce_from_file(topic: str, to_context: str, working_dir: pathlib.Path, avro: bool): if avro: - producer = AvroFileProducer.create(working_dir) + producer = AvroFileProducer(working_dir) else: - producer = FileProducer.create(working_dir) + producer = FileProducer(working_dir) click.echo("\nStart producing to topic " + blue_bold(topic) + " in target context " + blue_bold(to_context)) number_produced_messages = producer.produce_from_file(topic) click.echo( @@ -296,9 +296,9 @@ def _consume_to_file( working_dir: pathlib.Path, topic: str, group_id: str, from_context: str, numbers: int, avro: bool, last: bool ) -> int: if avro: - consumer = AvroFileConsumer.create(group_id, topic, working_dir, last) + consumer = AvroFileConsumer(group_id, topic, working_dir, last) else: - consumer = FileConsumer.create(group_id, topic, working_dir, last) + consumer = FileConsumer(group_id, topic, working_dir, last) click.echo("\nStart consuming from topic " + blue_bold(topic) + " in source context " + blue_bold(from_context)) number_consumed_messages = consumer.consume_to_file(int(numbers)) click.echo(blue_bold(str(number_consumed_messages)) + " messages consumed.") diff --git a/esque/clients.py b/esque/clients.py index ad0d4f9c..019be918 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -57,7 +57,7 @@ def consume_ping(self) -> Optional[Tuple[str, int]]: class FileConsumer(Consumer): - def __init__(self, group_id: str, topic_name: str, working_dir: pathlib.Path, file_writer: FileWriter, last: bool): + def __init__(self, group_id: str, topic_name: str, working_dir: pathlib.Path, last: bool): super().__init__(group_id, topic_name, last) self.working_dir = working_dir offset_reset = "earliest" @@ -67,11 +67,7 @@ def __init__(self, group_id: str, topic_name: str, working_dir: pathlib.Path, fi self._config.update({"default.topic.config": {"auto.offset.reset": offset_reset}}) self._consumer = confluent_kafka.Consumer(self._config) self._assign_exact_partitions(topic_name) - self.file_writer = file_writer - - @classmethod - def create(cls, group_id: str, topic_name: str, working_dir: pathlib.Path, last: bool): - return cls(group_id, topic_name, working_dir, PlainTextFileWriter(), last) + self.file_writer = PlainTextFileWriter() def consume_to_file(self, amount: int) -> int: counter = 0 @@ -109,15 +105,10 @@ def _consume_single_message(self) -> Optional[Message]: class AvroFileConsumer(FileConsumer): - def __init__(self, group_id: str, topic_name: str, working_dir: pathlib.Path, file_writer: FileWriter, last: bool): - super().__init__(group_id, topic_name, working_dir, file_writer, last) - self.writer = file_writer - - @classmethod - def create(cls, group_id: str, topic_name: str, working_dir: pathlib.Path, last: bool): + def __init__(self, group_id: str, topic_name: str, working_dir: pathlib.Path, last: bool): + super().__init__(group_id, topic_name, working_dir, last) schema_registry_client = SchemaRegistryClient(Config().schema_registry) - file_writer = AvroFileWriter(working_dir, schema_registry_client) - return cls(group_id, topic_name, working_dir, file_writer, last) + self.writer = AvroFileWriter(working_dir, schema_registry_client) class PingProducer(object): @@ -138,17 +129,13 @@ def produce_ping(self, topic_name: str): class FileProducer(object): - def __init__(self, working_dir: pathlib.Path, file_reader: FileReader): + def __init__(self, working_dir: pathlib.Path): self._config = Config().create_confluent_config() self._config.update({"on_delivery": delivery_callback, "error_cb": raise_for_kafka_error}) self._producer = confluent_kafka.Producer(self._config) self.working_dir = working_dir - self.file_reader = file_reader - - @classmethod - def create(cls, working_dir: pathlib.Path): - return cls(working_dir, PlainTextFileReader()) + self.file_reader = PlainTextFileReader() def produce_from_file(self, topic_name: str) -> int: with (self.working_dir / "data").open("rb") as file: @@ -170,15 +157,11 @@ def produce(self, topic_name: str, message: KafkaMessage): class AvroFileProducer(FileProducer): - def __init__(self, working_dir: pathlib.Path, file_reader: FileReader): - super().__init__(working_dir, file_reader) + def __init__(self, working_dir: pathlib.Path): + super().__init__(working_dir) self._config.update({"schema.registry.url": Config().schema_registry}) self._producer = AvroProducer(self._config) - self.file_reader = file_reader - - @classmethod - def create(cls, working_dir: pathlib.Path): - return cls(working_dir, AvroFileReader(working_dir)) + self.file_reader = AvroFileReader(working_dir) def produce(self, topic_name: str, message: KafkaMessage): self._producer.produce( From cef7be62e4d02c3b163e6238b2073781275a762a Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Wed, 10 Jul 2019 21:30:38 +0200 Subject: [PATCH 21/94] Super producer and consumer implement consume and produce --- esque/cli/commands.py | 8 ++--- esque/clients.py | 71 ++++++++++++++++++++++++------------------- 2 files changed, 44 insertions(+), 35 deletions(-) diff --git a/esque/cli/commands.py b/esque/cli/commands.py index fa9d02ee..5d97e85d 100644 --- a/esque/cli/commands.py +++ b/esque/cli/commands.py @@ -281,7 +281,7 @@ def _produce_from_file(topic: str, to_context: str, working_dir: pathlib.Path, a else: producer = FileProducer(working_dir) click.echo("\nStart producing to topic " + blue_bold(topic) + " in target context " + blue_bold(to_context)) - number_produced_messages = producer.produce_from_file(topic) + number_produced_messages = producer.produce(topic) click.echo( green_bold(str(number_produced_messages)) + " messages successfully produced to context " @@ -300,7 +300,7 @@ def _consume_to_file( else: consumer = FileConsumer(group_id, topic, working_dir, last) click.echo("\nStart consuming from topic " + blue_bold(topic) + " in source context " + blue_bold(from_context)) - number_consumed_messages = consumer.consume_to_file(int(numbers)) + number_consumed_messages = consumer.consume(int(numbers)) click.echo(blue_bold(str(number_consumed_messages)) + " messages consumed.") return number_consumed_messages @@ -325,8 +325,8 @@ def ping(state, times, wait): click.echo(f"Ping with {state.cluster.bootstrap_servers}") for i in range(times): - producer.produce_ping(PING_TOPIC) - _, delta = consumer.consume_ping() + producer.produce(PING_TOPIC) + _, delta = consumer.consume() deltas.append(delta) click.echo(f"m_seq={i} time={delta:.2f}ms") sleep(wait) diff --git a/esque/clients.py b/esque/clients.py index 019be918..3efd90ad 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -12,7 +12,7 @@ from esque.config import Config from esque.errors import raise_for_kafka_error, raise_for_message, KafkaException from esque.helpers import delivery_callback, delta_t -from esque.message import KafkaMessage, PlainTextFileReader, PlainTextFileWriter, FileReader, FileWriter +from esque.message import KafkaMessage, PlainTextFileReader, PlainTextFileWriter from esque.schemaregistry import SchemaRegistryClient DEFAULT_RETENTION_MS = 7 * 24 * 60 * 60 * 1000 @@ -44,9 +44,33 @@ def __init__(self, group_id: str, topic_name: str, last: bool): def _assign_exact_partitions(self, topic: str) -> None: self._consumer.assign([TopicPartition(topic=topic, partition=0, offset=0)]) + def consume(self, amount: int) -> int: + pass + + def _consume_single_message(self) -> Optional[Message]: + poll_limit = 10 + counter = 0 + try: + while counter < poll_limit: + message = self._consumer.poll(timeout=10) + if message is None: + counter += 1 + continue + if message.error() is not None: + if message.error().code() == KafkaError._PARTITION_EOF: + print("\nEnd of partition reached!".format(**locals())) + break + else: + raise RuntimeError(message.error().str()) + raise_for_message(message) + return message + except KafkaException as ex: + print("An error occurred: " + ex.message) + return None + class PingConsumer(Consumer): - def consume_ping(self) -> Optional[Tuple[str, int]]: + def consume(self, amount: int) -> Optional[Tuple[str, int]]: msg = self._consumer.consume(timeout=10)[0] raise_for_message(msg) @@ -69,7 +93,7 @@ def __init__(self, group_id: str, topic_name: str, working_dir: pathlib.Path, la self._assign_exact_partitions(topic_name) self.file_writer = PlainTextFileWriter() - def consume_to_file(self, amount: int) -> int: + def consume(self, amount: int) -> int: counter = 0 with (self.working_dir / "data").open("wb") as file: while counter < amount: @@ -82,27 +106,6 @@ def consume_to_file(self, amount: int) -> int: return counter - def _consume_single_message(self) -> Optional[Message]: - poll_limit = 10 - counter = 0 - try: - while counter < poll_limit: - message = self._consumer.poll(timeout=10) - if message is None: - counter += 1 - continue - if message.error() is not None: - if message.error().code() == KafkaError._PARTITION_EOF: - print("\nEnd of partition reached!".format(**locals())) - break - else: - raise RuntimeError(message.error().str()) - raise_for_message(message) - return message - except KafkaException as ex: - print("An error occurred: " + ex.message) - return None - class AvroFileConsumer(FileConsumer): def __init__(self, group_id: str, topic_name: str, working_dir: pathlib.Path, last: bool): @@ -111,14 +114,19 @@ def __init__(self, group_id: str, topic_name: str, working_dir: pathlib.Path, la self.writer = AvroFileWriter(working_dir, schema_registry_client) -class PingProducer(object): +class Producer(object): + def produce(self, topic_name: str) -> int: + pass + + +class PingProducer(Producer): def __init__(self): self._config = Config().create_confluent_config() self._config.update({"on_delivery": delivery_callback, "error_cb": raise_for_kafka_error}) self._producer = confluent_kafka.Producer(self._config) - def produce_ping(self, topic_name: str): + def produce(self, topic_name: str) -> int: start = pendulum.now() self._producer.produce(topic=topic_name, key=str(0), value=str(pendulum.now().timestamp())) while True: @@ -126,9 +134,10 @@ def produce_ping(self, topic_name: str): if left_messages == 0: break click.echo(f"{delta_t(start)} | Still {left_messages} messages left, flushing...") + return 1 -class FileProducer(object): +class FileProducer(Producer): def __init__(self, working_dir: pathlib.Path): self._config = Config().create_confluent_config() self._config.update({"on_delivery": delivery_callback, "error_cb": raise_for_kafka_error}) @@ -137,11 +146,11 @@ def __init__(self, working_dir: pathlib.Path): self.working_dir = working_dir self.file_reader = PlainTextFileReader() - def produce_from_file(self, topic_name: str) -> int: + def produce(self, topic_name: str) -> int: with (self.working_dir / "data").open("rb") as file: counter = 0 for message in self.file_reader.read_from_file(file): - self.produce(topic_name, message) + self.produce_message(topic_name, message) counter += 1 while True: @@ -152,7 +161,7 @@ def produce_from_file(self, topic_name: str) -> int: return counter - def produce(self, topic_name: str, message: KafkaMessage): + def produce_message(self, topic_name: str, message: KafkaMessage): self._producer.produce(topic=topic_name, key=message.key, value=message.value) @@ -163,7 +172,7 @@ def __init__(self, working_dir: pathlib.Path): self._producer = AvroProducer(self._config) self.file_reader = AvroFileReader(working_dir) - def produce(self, topic_name: str, message: KafkaMessage): + def produce_message(self, topic_name: str, message: KafkaMessage): self._producer.produce( topic=topic_name, key=message.key, From 0af62e1b8d90041dab21660e7c10393bad7c71a1 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Fri, 12 Jul 2019 15:44:55 +0200 Subject: [PATCH 22/94] Update version --- esque/schemaregistry.py | 1 - setup.py | 5 +-- tests/conftest.py | 42 +++++++++++-------- tests/integration/test_clients.py | 8 ++++ .../test_consumergroup_controller.py | 8 +--- tests/integration/test_topic_controller.py | 20 +++------ tests/unit/test_config.py | 6 +-- 7 files changed, 42 insertions(+), 48 deletions(-) create mode 100644 tests/integration/test_clients.py diff --git a/esque/schemaregistry.py b/esque/schemaregistry.py index b92b9398..825a2278 100644 --- a/esque/schemaregistry.py +++ b/esque/schemaregistry.py @@ -1,5 +1,4 @@ import json -import struct from collections import namedtuple from functools import lru_cache from typing import Dict diff --git a/setup.py b/setup.py index 6215342b..e7d5936b 100644 --- a/setup.py +++ b/setup.py @@ -61,10 +61,7 @@ def run(self): python_requires=">=3.6", setup_requires=[], install_requires=required, - extras_require={ - "test": ["pytest", "pytest-mock", "pytest-cov"], - "dev": ["black", "flake8"], - }, + extras_require={"test": ["pytest", "pytest-mock", "pytest-cov"], "dev": ["black", "flake8"]}, include_package_data=True, license="MIT", classifiers=[ diff --git a/tests/conftest.py b/tests/conftest.py index 613d052d..e5d99df5 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -1,3 +1,4 @@ +import pathlib import random from concurrent.futures import Future from pathlib import Path @@ -10,6 +11,7 @@ from pykafka import Producer from pykafka.exceptions import NoBrokersAvailableError +from esque.clients import FileConsumer, AvroFileConsumer from esque.cluster import Cluster from esque.config import Config, sample_config_path from esque.consumergroup import ConsumerGroupController @@ -18,12 +20,7 @@ def pytest_addoption(parser): - parser.addoption( - "--integration", - action="store_true", - default=False, - help="run integration tests", - ) + parser.addoption("--integration", action="store_true", default=False, help="run integration tests") parser.addoption( "--local", action="store_true", @@ -108,11 +105,7 @@ def confluent_admin_client(test_config: Config) -> AdminClient: @pytest.fixture() def producer(topic_object: Topic): # Send messages synchronously so we can be sure offset has been commited in tests. - yield Producer( - topic_object.cluster.pykafka_client.cluster, - topic_object._pykafka_topic, - sync=True, - ) + yield Producer(topic_object.cluster.pykafka_client.cluster, topic_object._pykafka_topic, sync=True) @pytest.fixture() @@ -121,9 +114,7 @@ def consumergroup_controller(cluster: Cluster): @pytest.fixture() -def consumergroup_instance( - partly_read_consumer_group: str, consumergroup_controller: ConsumerGroupController -): +def consumergroup_instance(partly_read_consumer_group: str, consumergroup_controller: ConsumerGroupController): yield consumergroup_controller.get_consumergroup(partly_read_consumer_group) @@ -161,9 +152,26 @@ def filled_topic(producer, topic_object): @pytest.fixture() -def partly_read_consumer_group( - consumer: confluent_kafka.Consumer, filled_topic, consumer_group -): +def working_dir(): + yield Path("test_directory") + + +@pytest.fixture() +def file_consumer(mocker, consumer_group, topic: str, working_dir: pathlib.Path): + file_consumer = FileConsumer(consumer_group, topic, working_dir, False) + file_writer = mocker.patch("esque.message.FileWriter") + file_writer.write_message_to_file.call_count = 10 + file_consumer.file_writer = file_consumer + yield file_consumer + + +@pytest.fixture() +def avro_file_consumer(consumer_group, topic: str, working_dir: pathlib.Path): + yield AvroFileConsumer(consumer_group, topic, working_dir, False) + + +@pytest.fixture() +def partly_read_consumer_group(consumer: confluent_kafka.Consumer, filled_topic, consumer_group): for i in range(5): msg = consumer.consume(timeout=10)[0] consumer.commit(msg, asynchronous=False) diff --git a/tests/integration/test_clients.py b/tests/integration/test_clients.py new file mode 100644 index 00000000..ea1cb8e3 --- /dev/null +++ b/tests/integration/test_clients.py @@ -0,0 +1,8 @@ +import pytest + +from esque.clients import FileConsumer + + +@pytest.mark.integration +def test_consume_to_file(file_consumer: FileConsumer): + file_consumer.consume(10) diff --git a/tests/integration/test_consumergroup_controller.py b/tests/integration/test_consumergroup_controller.py index 3729b5d6..26536db1 100644 --- a/tests/integration/test_consumergroup_controller.py +++ b/tests/integration/test_consumergroup_controller.py @@ -4,16 +4,12 @@ @pytest.mark.integration -def test_get_consumer_group( - partly_read_consumer_group: str, consumergroup_controller: ConsumerGroupController -): +def test_get_consumer_group(partly_read_consumer_group: str, consumergroup_controller: ConsumerGroupController): instance = consumergroup_controller.get_consumergroup(partly_read_consumer_group) assert isinstance(instance, ConsumerGroup) @pytest.mark.integration -def test_list_consumer_groups( - partly_read_consumer_group: str, consumergroup_controller: ConsumerGroupController -): +def test_list_consumer_groups(partly_read_consumer_group: str, consumergroup_controller: ConsumerGroupController): groups = consumergroup_controller.list_consumer_groups() assert partly_read_consumer_group in groups diff --git a/tests/integration/test_topic_controller.py b/tests/integration/test_topic_controller.py index 46f2cf76..0fc02452 100644 --- a/tests/integration/test_topic_controller.py +++ b/tests/integration/test_topic_controller.py @@ -11,15 +11,11 @@ def topic_controller(cluster): @pytest.mark.integration def test_topic_creation_works( - topic_controller: TopicController, - confluent_admin_client: confluent_kafka.admin.AdminClient, - topic_id: str, + topic_controller: TopicController, confluent_admin_client: confluent_kafka.admin.AdminClient, topic_id: str ): topics = confluent_admin_client.list_topics(timeout=5).topics.keys() assert topic_id not in topics - topic_controller.create_topics( - [topic_controller.get_topic(topic_id, replication_factor=1)] - ) + topic_controller.create_topics([topic_controller.get_topic(topic_id, replication_factor=1)]) # invalidate cache confluent_admin_client.poll(timeout=1) topics = confluent_admin_client.list_topics(timeout=5).topics.keys() @@ -28,15 +24,11 @@ def test_topic_creation_works( @pytest.mark.integration def test_alter_topic_config_works(topic_controller: TopicController, topic_id: str): - initial_topic = topic_controller.get_topic( - topic_id, config={"cleanup.policy": "delete"} - ) + initial_topic = topic_controller.get_topic(topic_id, config={"cleanup.policy": "delete"}) topic_controller.create_topics([initial_topic]) replicas, config = initial_topic.describe() assert config.get("Config").get("cleanup.policy") == "delete" - change_topic = topic_controller.get_topic( - topic_id, config={"cleanup.policy": "compact"} - ) + change_topic = topic_controller.get_topic(topic_id, config={"cleanup.policy": "compact"}) topic_controller.alter_configs([change_topic]) after_changes_applied_topic = topic_controller.get_topic(topic_id) replicas, final_config = after_changes_applied_topic.describe() @@ -45,9 +37,7 @@ def test_alter_topic_config_works(topic_controller: TopicController, topic_id: s @pytest.mark.integration def test_topic_deletion_works( - topic_controller: TopicController, - confluent_admin_client: confluent_kafka.admin.AdminClient, - topic: str, + topic_controller: TopicController, confluent_admin_client: confluent_kafka.admin.AdminClient, topic: str ): topics = confluent_admin_client.list_topics(timeout=5).topics.keys() assert topic in topics diff --git a/tests/unit/test_config.py b/tests/unit/test_config.py index 3a8b82f0..41e94e4d 100644 --- a/tests/unit/test_config.py +++ b/tests/unit/test_config.py @@ -55,11 +55,7 @@ def test_current_context(config: Config): def test_current_context_dict(config: Config): - expected = { - "bootstrap_hosts": "localhost", - "bootstrap_port": "9091", - "security_protocol": "PLAINTEXT", - } + expected = {"bootstrap_hosts": "localhost", "bootstrap_port": "9091", "security_protocol": "PLAINTEXT"} assert config.current_context_dict == expected From e632360c9e30e6fc81f2d5e0aae27c7972eae5e9 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Fri, 12 Jul 2019 16:12:31 +0200 Subject: [PATCH 23/94] Update Pipfile.lock --- Pipfile.lock | 116 +++++++++++++++++++++++++++++++++++++++------------ 1 file changed, 90 insertions(+), 26 deletions(-) diff --git a/Pipfile.lock b/Pipfile.lock index 3b05e3dd..dcc7dcea 100644 --- a/Pipfile.lock +++ b/Pipfile.lock @@ -1,7 +1,7 @@ { "_meta": { "hash": { - "sha256": "39bdc1457e421be937c9db60e2b5bf93f09a746d17a48e1630febae254b35cb3" + "sha256": "072d6fb167dafbc1b8fbc572ff61d3995ab6bfb48ba5f6ca0331af6f84910842" }, "pipfile-spec": 6, "requires": { @@ -15,7 +15,15 @@ } ] }, - "default": {}, + "default": { + "avro-python3": { + "hashes": [ + "sha256:6163c1507621f5bdafab3370df7f82166c3c7ba53763a5637313639dcef02ae9" + ], + "index": "pypi", + "version": "==1.9.0" + } + }, "develop": { "appdirs": { "hashes": [ @@ -45,6 +53,20 @@ ], "version": "==19.3b0" }, + "certifi": { + "hashes": [ + "sha256:046832c04d4e752f37383b628bc601a7ea7211496b4638f6514d0e5b9acc4939", + "sha256:945e3ba63a0b9f577b1395204e13c3a231f9bc0223888be653286534e5873695" + ], + "version": "==2019.6.16" + }, + "chardet": { + "hashes": [ + "sha256:84ab92ed1c4d4f16916e05906b6b75a6c0fb5db821cc65e70cbd64a3e2a5eaae", + "sha256:fc323ffcaeaed0e0a02bf4d117757b98aed530d9ed4531e3e15460124c106691" + ], + "version": "==3.0.4" + }, "click": { "hashes": [ "sha256:2335065e6395b9e67ca716de5f7526736bfa6ceead690adf616d925bdc622b13", @@ -122,12 +144,42 @@ ], "version": "==0.3" }, + "fastavro": { + "hashes": [ + "sha256:0930500497586037f1ca4bcb0e929d388d8cd86d1f5c43a72ac46bfa1003c785", + "sha256:0cf083c14b2a3c72704660e89480571ed2d4956cc5b79845905956d38d6eb7b6", + "sha256:3dd785459a7cd3c17012374c8929c73f385274dbbeed784ceda7879a4681c70f", + "sha256:49e671bcacc81b7b49f2c9f6f3db4b6201d5cf5809c321b33da3d2890c0f8481", + "sha256:52fd6153c7fbce93cef4e1c169543357582815a7665c142ebd97f29b7fabfc79", + "sha256:59b4a838c4f7b2646bf4d79fe3a0c4d546acf7e0bb65eda57eaa5ac76dc8dea9", + "sha256:61bc6d2c575b76e69b3ed16ee67827d67d166baeea255151cace550801bc5040", + "sha256:688e6fd5458ba673c1e1fff8ed25a27996e7ef5bcd73a50c6127d8c511703624", + "sha256:6c39c04e8ea5a85c512335a68bf08f922901817d49c113dd74de3dfebe38701f", + "sha256:76558659cce8d73f5fbd6084e362c25fcceae4e195e722b30d185f26416b0486", + "sha256:783f8600af4fc94c799e67f9deb0aedc7432e440937921b7f743dfc0753864bd", + "sha256:a94ca58353b972bc906713e02e8c5a173250070b5cfbf0271e347a8c810e32cf", + "sha256:aaeac57f4c4249d7d7b4e3ca1a43444e61a5aa382c83fb7580059e90e502485f", + "sha256:ca0eb9ff026490eb5c4ae83f48326932d6c206c4d9325e469ab79e4ad1956ac7", + "sha256:db387ce862b9dc71ecd8760e05ac0ee30b55b557cc2e2f2b33df0f3cf6f12a05", + "sha256:e9648a7dfe9358e550d093d546ae330d2a9e3246cbe5d7425a4ca17521ee7a72", + "sha256:eaa08fc8d416314bab328e9fe26df562b28ced24346109dbbae4a81151269d0f", + "sha256:fb492b325c9a373d62e3ea41b9b868ea276d0964e98adfa26c181f748a962bf3" + ], + "version": "==0.22.2" + }, "flake8": { "hashes": [ - "sha256:859996073f341f2670741b51ec1e67a01da142831aa1fdc6242dbf88dffbe661", - "sha256:a796a115208f5c03b18f332f7c11729812c8c3ded6c46319c59b53efd3819da8" + "sha256:19241c1cbc971b9962473e4438a2ca19749a7dd002dd1a946eaba171b4114548", + "sha256:8e9dfa3cecb2400b3738a42c54c3043e821682b9c840b0448c0503f781130696" ], - "version": "==3.7.7" + "version": "==3.7.8" + }, + "idna": { + "hashes": [ + "sha256:c357b3f628cf53ae2c4c05627ecc484553142ca23264e593d327bcde5e9c3407", + "sha256:ea8b7f6188e6fa117537c3df7da9fc686d485087abf6ac197f9c46432f7e4a3c" + ], + "version": "==2.8" }, "importlib-metadata": { "hashes": [ @@ -160,11 +212,10 @@ }, "more-itertools": { "hashes": [ - "sha256:2112d2ca570bb7c3e53ea1a35cd5df42bb0fd10c45f0fb97178679c3c03d64c7", - "sha256:c3e4748ba1aad8dba30a4886b0b1a2004f9a863837b8654e7059eebf727afa5a" + "sha256:3ad685ff8512bf6dc5a8b82ebf73543999b657eded8c11803d9ba6b648986f4d", + "sha256:8bb43d1f51ecef60d81854af61a3a880555a14643691cc4b64a6ee269c78f09a" ], - "markers": "python_version > '2.7'", - "version": "==7.0.0" + "version": "==7.1.0" }, "packaging": { "hashes": [ @@ -175,15 +226,14 @@ }, "pendulum": { "hashes": [ - "sha256:0f43d963b27e92b04047ce8352e4c277db99f20d0b513df7d0ceafe674a2f727", - "sha256:14e60d26d7400980123dbb6e3f2a90b70d7c18c63742ffe5bd6d6a643f8c6ef1", - "sha256:5035a4e17504814a679f138374269cc7cc514aeac7ba6d9dc020abc224f25dbc", - "sha256:8c0b3d655c1e9205d4dacf42fffc929cde3b19b5fb544a7f7561e6896eb8a000", - "sha256:bfc7b33ae193a204ec0bec12ad0d2d3300cd7e51d91d992da525ba3b28f0d265", - "sha256:cd70b75800439794e1ad8dbfa24838845e171918df81fa98b68d0d5a6f9b8bf2", - "sha256:cf535d36c063575d4752af36df928882b2e0e31541b4482c97d63752785f9fcb" + "sha256:1cde6e3c6310fb882c98f373795f807cb2bd6af01f34d2857e6e283b5ee91e09", + "sha256:485aef2089defee88607d37d5bc238934d0b90993d7bf9ceb36e481af41e9c66", + "sha256:57801754e05f30e8a7e4d24734c9fad82c6c3ec489151555f0fc66bb32ba6d6d", + "sha256:7ee344bc87cb425b04717b90d14ffde14c1dd64eaa73060b3772edcf57f3e866", + "sha256:c460f4d8dc41ec3c4377ac1807678cd72fe5e973cc2943c104ffdeaac32dacb7", + "sha256:d3078e007315a959989c41cee5cfd63cfeeca21dd3d8295f4bc24199489e9b6c" ], - "version": "==2.0.4" + "version": "==2.0.5" }, "pluggy": { "hashes": [ @@ -228,10 +278,10 @@ }, "pytest": { "hashes": [ - "sha256:4a784f1d4f2ef198fe9b7aef793e9fa1a3b2f84e822d9b3a64a181293a572d45", - "sha256:926855726d8ae8371803f7b2e6ec0a69953d9c6311fa7c3b6c1b929ff92d27da" + "sha256:6ef6d06de77ce2961156013e9dff62f1b2688aa04d0dc244299fe7d67e09370d", + "sha256:a736fed91c12681a7b34617c8fcefe39ea04599ca72c608751c31d89579a3f77" ], - "version": "==4.6.3" + "version": "==5.0.1" }, "pytest-cov": { "hashes": [ @@ -256,10 +306,10 @@ }, "pytzdata": { "hashes": [ - "sha256:778db26940e38cf6547d6574f49375570f7d697970461de531c56cf8400958a3", - "sha256:f0469062f799c66480fcc7eae69a8270dc83f0e6522c0e70db882d6bd708d378" + "sha256:c0c8316eaf6c25ba45816390a1a45c39790767069b3275c5f7de3ddf773eb810", + "sha256:e8a91952afd853642a49f0713caac3e15a5306855ff4a47af4ddec5b7dd23a09" ], - "version": "==2019.1" + "version": "==2019.2" }, "pyyaml": { "hashes": [ @@ -277,6 +327,13 @@ ], "version": "==5.1.1" }, + "requests": { + "hashes": [ + "sha256:11e007a8a2aa0323f5a921e9e6a2d7e4e67d9877e85773fba9ba6419025cbeb4", + "sha256:9cf5292fcd0f598c671cfc1e0d7d1a7f13bb8085e9a590f48c010551dc6c4b31" + ], + "version": "==2.22.0" + }, "six": { "hashes": [ "sha256:3350809f0555b11f552448330d0b52d5f24c91a322ea4a15ef22629740f3761c", @@ -297,6 +354,13 @@ ], "version": "==0.10.0" }, + "urllib3": { + "hashes": [ + "sha256:b246607a25ac80bedac05c6f282e3cdaf3afb65420fd024ac94435cabe6e18d1", + "sha256:dbe59173209418ae49d485b87d1681aefa36252ee85884c31346debd19463232" + ], + "version": "==1.25.3" + }, "virtualenv": { "hashes": [ "sha256:b7335cddd9260a3dd214b73a2521ffc09647bde3e9457fcca31dc3be3999d04a", @@ -313,10 +377,10 @@ }, "zipp": { "hashes": [ - "sha256:8c1019c6aad13642199fbe458275ad6a84907634cc9f0989877ccc4a2840139d", - "sha256:ca943a7e809cc12257001ccfb99e3563da9af99d52f261725e96dfe0f9275bc3" + "sha256:4970c3758f4e89a7857a973b1e2a5d75bcdc47794442f2e2dd4fe8e0466e809a", + "sha256:8a5712cfd3bb4248015eb3b0b3c54a5f6ee3f2425963ef2a0125b8bc40aafaec" ], - "version": "==0.5.1" + "version": "==0.5.2" } } } From 2fa4884152578b8fd5cc7fdf59491ecfee46c892 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Fri, 12 Jul 2019 16:21:10 +0200 Subject: [PATCH 24/94] Update pip files --- Pipfile | 1 + Pipfile.lock | 177 +++++++++++++++++++++++++++++++++++++++++++++++++-- 2 files changed, 173 insertions(+), 5 deletions(-) diff --git a/Pipfile b/Pipfile index aa42257b..fab1b1af 100644 --- a/Pipfile +++ b/Pipfile @@ -7,6 +7,7 @@ verify_ssl = true kafkaesque = {path = ".",editable = true,extras = ["test", "dev"]} [packages] +esque = {editable = true,path = "."} [requires] python_version = "3.6" diff --git a/Pipfile.lock b/Pipfile.lock index dcc7dcea..2b9692ef 100644 --- a/Pipfile.lock +++ b/Pipfile.lock @@ -1,7 +1,7 @@ { "_meta": { "hash": { - "sha256": "072d6fb167dafbc1b8fbc572ff61d3995ab6bfb48ba5f6ca0331af6f84910842" + "sha256": "5eb10ba1263766fe6fe002f6c48d58c7c9ae648a6c25125893bd0346c1b11d18" }, "pipfile-spec": 6, "requires": { @@ -16,12 +16,179 @@ ] }, "default": { - "avro-python3": { + "certifi": { + "hashes": [ + "sha256:046832c04d4e752f37383b628bc601a7ea7211496b4638f6514d0e5b9acc4939", + "sha256:945e3ba63a0b9f577b1395204e13c3a231f9bc0223888be653286534e5873695" + ], + "version": "==2019.6.16" + }, + "chardet": { + "hashes": [ + "sha256:84ab92ed1c4d4f16916e05906b6b75a6c0fb5db821cc65e70cbd64a3e2a5eaae", + "sha256:fc323ffcaeaed0e0a02bf4d117757b98aed530d9ed4531e3e15460124c106691" + ], + "version": "==3.0.4" + }, + "click": { + "hashes": [ + "sha256:2335065e6395b9e67ca716de5f7526736bfa6ceead690adf616d925bdc622b13", + "sha256:5b94b49521f6456670fdb30cd82a4eca9412788a93fa6dd6df72c94d5a8ff2d7" + ], + "version": "==7.0" + }, + "confluent-kafka": { + "hashes": [ + "sha256:01737c71c4596ef588b60ee1a1c56b64d8fd04169e3d5545ed31e08a530e00df", + "sha256:16736eb9a65c2ca5ddb7200254d7a97c2b59eb3ef4f985ed319e583cb7abc0e6", + "sha256:1c84038b2c515b27b14ab2d44b3f5c6ea7eeb848bfe755f5083c58edc60867af", + "sha256:228722c35efa27cafe30bb7236c8acf699b9a07deb8df011f7d8e975715b77f6", + "sha256:342299cd49c0af395bfaf0da5acfc1509081064429f809deb77754999ade8757", + "sha256:36f3942527100646b8dcc32b59ae03d34b36df96a2fa66b3d91a10e8fcf3002f", + "sha256:442186d9d6eca9c7a9d019a782645aeb210f1eb0dc85235ba60cd2f15ef89292", + "sha256:4dcb171ebed381f2f8b3dd0827d10398097dba346b5170a42501065bee075444", + "sha256:50d2300dad99f354b316660595eb461567e253fbb314848974542c58c04a7666", + "sha256:5505d42ad10fde01f0a7254c6e7e2348b0b9dc26be95bea7ad203bcea3ece201", + "sha256:6f9a99de81028fa687499f2f335374b0ee2366a970bc0664637ab346a1b77a11", + "sha256:76e1922ada8da7ee654892f2364e9b30730fba9154868cec1017d791134bedf7", + "sha256:790decee83bd0a15dc6b65a87703c156d63fde51b6b5c1c086cca13ee9300e8f", + "sha256:7cd66710c8552e83d09b60cf699309d47f749289b10379ad3ed01d1b5114b461", + "sha256:7eae7288fc28a8cd9ce5b758530ab2690c6ba15141c9192bc65774a2a7148432", + "sha256:88fbbf7c27db4770a2d5b26795608cf2a6b672811cbe337f81ca82fe7011586d", + "sha256:9857da6a911e7beae5e11acc8a494e95dabd4348c78a26b32b9ac98142b368da", + "sha256:98ba297342d33c87775abbb9403f4759b9d48c34f84020973ddb8a8b785bfde1", + "sha256:a2634c2de4b8cfdce9963723023658190edf7d8fddd3f4aa2eda448941654bd6", + "sha256:ab5ddd32c4bcf7d5fecf71a455b5525825d907b86cdab9594bad33a1e342360d", + "sha256:b25399b3e8bada06134b5a38f24ca70acf1524e70a6290c34a709b28b29ec284", + "sha256:b809a9ddd35492b1c24487925dbc4b3cf4f392bf8856584d49b6b5a674fd0776", + "sha256:bcb2da981e8713c900ab9903226167878f5731fcf33a3551ddb0614db88d1fcf", + "sha256:dd003e54b251e8e7300e808175d21c8ab9039c9633a6fd87fb99301d94f7d2bc", + "sha256:ddbd51a8429815ea5194d1d72881604836c6ea037cba79bf7440b2d4098ac7b0", + "sha256:e809dbe2cd3b934247d2c1e22736798150769e51415ea65d28ba496fcec22a7d" + ], + "version": "==1.0.1" + }, + "esque": { + "editable": true, + "path": "." + }, + "fastavro": { + "hashes": [ + "sha256:0930500497586037f1ca4bcb0e929d388d8cd86d1f5c43a72ac46bfa1003c785", + "sha256:0cf083c14b2a3c72704660e89480571ed2d4956cc5b79845905956d38d6eb7b6", + "sha256:3dd785459a7cd3c17012374c8929c73f385274dbbeed784ceda7879a4681c70f", + "sha256:49e671bcacc81b7b49f2c9f6f3db4b6201d5cf5809c321b33da3d2890c0f8481", + "sha256:52fd6153c7fbce93cef4e1c169543357582815a7665c142ebd97f29b7fabfc79", + "sha256:59b4a838c4f7b2646bf4d79fe3a0c4d546acf7e0bb65eda57eaa5ac76dc8dea9", + "sha256:61bc6d2c575b76e69b3ed16ee67827d67d166baeea255151cace550801bc5040", + "sha256:688e6fd5458ba673c1e1fff8ed25a27996e7ef5bcd73a50c6127d8c511703624", + "sha256:6c39c04e8ea5a85c512335a68bf08f922901817d49c113dd74de3dfebe38701f", + "sha256:76558659cce8d73f5fbd6084e362c25fcceae4e195e722b30d185f26416b0486", + "sha256:783f8600af4fc94c799e67f9deb0aedc7432e440937921b7f743dfc0753864bd", + "sha256:a94ca58353b972bc906713e02e8c5a173250070b5cfbf0271e347a8c810e32cf", + "sha256:aaeac57f4c4249d7d7b4e3ca1a43444e61a5aa382c83fb7580059e90e502485f", + "sha256:ca0eb9ff026490eb5c4ae83f48326932d6c206c4d9325e469ab79e4ad1956ac7", + "sha256:db387ce862b9dc71ecd8760e05ac0ee30b55b557cc2e2f2b33df0f3cf6f12a05", + "sha256:e9648a7dfe9358e550d093d546ae330d2a9e3246cbe5d7425a4ca17521ee7a72", + "sha256:eaa08fc8d416314bab328e9fe26df562b28ced24346109dbbae4a81151269d0f", + "sha256:fb492b325c9a373d62e3ea41b9b868ea276d0964e98adfa26c181f748a962bf3" + ], + "version": "==0.22.2" + }, + "idna": { + "hashes": [ + "sha256:c357b3f628cf53ae2c4c05627ecc484553142ca23264e593d327bcde5e9c3407", + "sha256:ea8b7f6188e6fa117537c3df7da9fc686d485087abf6ac197f9c46432f7e4a3c" + ], + "version": "==2.8" + }, + "kazoo": { + "hashes": [ + "sha256:8db774f7bdece7d0dc7decb21539ff0852e42c2ffe1c28d7f1ff6f9292a1c3a4", + "sha256:a5fa2e400c5068cfee9e86b35cf0dab8232b574152d8e3590d823b3e2426ab5e" + ], + "version": "==2.5.0" + }, + "pendulum": { + "hashes": [ + "sha256:1cde6e3c6310fb882c98f373795f807cb2bd6af01f34d2857e6e283b5ee91e09", + "sha256:485aef2089defee88607d37d5bc238934d0b90993d7bf9ceb36e481af41e9c66", + "sha256:57801754e05f30e8a7e4d24734c9fad82c6c3ec489151555f0fc66bb32ba6d6d", + "sha256:7ee344bc87cb425b04717b90d14ffde14c1dd64eaa73060b3772edcf57f3e866", + "sha256:c460f4d8dc41ec3c4377ac1807678cd72fe5e973cc2943c104ffdeaac32dacb7", + "sha256:d3078e007315a959989c41cee5cfd63cfeeca21dd3d8295f4bc24199489e9b6c" + ], + "version": "==2.0.5" + }, + "pykafka": { "hashes": [ - "sha256:6163c1507621f5bdafab3370df7f82166c3c7ba53763a5637313639dcef02ae9" + "sha256:9b45c60ba40550f5b460a5a1441154255666e5935eae88c3cfa3aaba1d541690" ], - "index": "pypi", - "version": "==1.9.0" + "version": "==2.8.1.dev1" + }, + "python-dateutil": { + "hashes": [ + "sha256:7e6584c74aeed623791615e26efd690f29817a27c73085b78e4bad02493df2fb", + "sha256:c89805f6f4d64db21ed966fda138f8a5ed7a4fdbc1a8ee329ce1b74e3c74da9e" + ], + "version": "==2.8.0" + }, + "pytzdata": { + "hashes": [ + "sha256:c0c8316eaf6c25ba45816390a1a45c39790767069b3275c5f7de3ddf773eb810", + "sha256:e8a91952afd853642a49f0713caac3e15a5306855ff4a47af4ddec5b7dd23a09" + ], + "version": "==2019.2" + }, + "pyyaml": { + "hashes": [ + "sha256:57acc1d8533cbe51f6662a55434f0dbecfa2b9eaf115bede8f6fd00115a0c0d3", + "sha256:588c94b3d16b76cfed8e0be54932e5729cc185caffaa5a451e7ad2f7ed8b4043", + "sha256:68c8dd247f29f9a0d09375c9c6b8fdc64b60810ebf07ba4cdd64ceee3a58c7b7", + "sha256:70d9818f1c9cd5c48bb87804f2efc8692f1023dac7f1a1a5c61d454043c1d265", + "sha256:86a93cccd50f8c125286e637328ff4eef108400dd7089b46a7be3445eecfa391", + "sha256:a0f329125a926876f647c9fa0ef32801587a12328b4a3c741270464e3e4fa778", + "sha256:a3c252ab0fa1bb0d5a3f6449a4826732f3eb6c0270925548cac342bc9b22c225", + "sha256:b4bb4d3f5e232425e25dda21c070ce05168a786ac9eda43768ab7f3ac2770955", + "sha256:cd0618c5ba5bda5f4039b9398bb7fb6a317bb8298218c3de25c47c4740e4b95e", + "sha256:ceacb9e5f8474dcf45b940578591c7f3d960e82f926c707788a570b51ba59190", + "sha256:fe6a88094b64132c4bb3b631412e90032e8cfe9745a58370462240b8cb7553cd" + ], + "version": "==5.1.1" + }, + "requests": { + "hashes": [ + "sha256:11e007a8a2aa0323f5a921e9e6a2d7e4e67d9877e85773fba9ba6419025cbeb4", + "sha256:9cf5292fcd0f598c671cfc1e0d7d1a7f13bb8085e9a590f48c010551dc6c4b31" + ], + "version": "==2.22.0" + }, + "six": { + "hashes": [ + "sha256:3350809f0555b11f552448330d0b52d5f24c91a322ea4a15ef22629740f3761c", + "sha256:d16a0141ec1a18405cd4ce8b4613101da75da0e9a7aec5bdd4fa804d0e0eba73" + ], + "version": "==1.12.0" + }, + "tabulate": { + "hashes": [ + "sha256:8af07a39377cee1103a5c8b3330a421c2d99b9141e9cc5ddd2e3263fea416943" + ], + "version": "==0.8.3" + }, + "urllib3": { + "hashes": [ + "sha256:b246607a25ac80bedac05c6f282e3cdaf3afb65420fd024ac94435cabe6e18d1", + "sha256:dbe59173209418ae49d485b87d1681aefa36252ee85884c31346debd19463232" + ], + "version": "==1.25.3" + }, + "virtualenv": { + "hashes": [ + "sha256:b7335cddd9260a3dd214b73a2521ffc09647bde3e9457fcca31dc3be3999d04a", + "sha256:d28ca64c0f3f125f59cabf13e0a150e1c68e5eea60983cc4395d88c584495783" + ], + "version": "==16.6.1" } }, "develop": { From 87f37888b1929383f96a9d6671897ffff1f6664c Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Fri, 12 Jul 2019 16:28:07 +0200 Subject: [PATCH 25/94] Add avro-python3 --- Pipfile.lock | 12 ++++++++++++ setup.py | 1 + 2 files changed, 13 insertions(+) diff --git a/Pipfile.lock b/Pipfile.lock index 2b9692ef..e31366ee 100644 --- a/Pipfile.lock +++ b/Pipfile.lock @@ -16,6 +16,12 @@ ] }, "default": { + "avro-python3": { + "hashes": [ + "sha256:6163c1507621f5bdafab3370df7f82166c3c7ba53763a5637313639dcef02ae9" + ], + "version": "==1.9.0" + }, "certifi": { "hashes": [ "sha256:046832c04d4e752f37383b628bc601a7ea7211496b4638f6514d0e5b9acc4939", @@ -213,6 +219,12 @@ ], "version": "==19.1.0" }, + "avro-python3": { + "hashes": [ + "sha256:6163c1507621f5bdafab3370df7f82166c3c7ba53763a5637313639dcef02ae9" + ], + "version": "==1.9.0" + }, "black": { "hashes": [ "sha256:09a9dcb7c46ed496a9850b76e4e825d6049ecd38b611f1224857a79bd985a8cf", diff --git a/setup.py b/setup.py index e7d5936b..1030a428 100644 --- a/setup.py +++ b/setup.py @@ -33,6 +33,7 @@ "pyyaml", "requests", "fastavro>=0.21.8", + "avro-python3", ] From e4002e9a649440752bbd3154fd3f7450d7204921 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Fri, 12 Jul 2019 16:30:25 +0200 Subject: [PATCH 26/94] Subscribe to topic instead of assign to partition 0 --- esque/clients.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/esque/clients.py b/esque/clients.py index 3efd90ad..53d5a0a7 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -39,10 +39,10 @@ def __init__(self, group_id: str, topic_name: str, last: bool): } ) self._consumer = confluent_kafka.Consumer(self._config) - self._assign_exact_partitions(topic_name) + self._subscribe(topic_name) - def _assign_exact_partitions(self, topic: str) -> None: - self._consumer.assign([TopicPartition(topic=topic, partition=0, offset=0)]) + def _subscribe(self, topic: str) -> None: + self._consumer.subscribe(topic) def consume(self, amount: int) -> int: pass @@ -90,7 +90,7 @@ def __init__(self, group_id: str, topic_name: str, working_dir: pathlib.Path, la self._config.update({"default.topic.config": {"auto.offset.reset": offset_reset}}) self._consumer = confluent_kafka.Consumer(self._config) - self._assign_exact_partitions(topic_name) + self._subscribe(topic_name) self.file_writer = PlainTextFileWriter() def consume(self, amount: int) -> int: From cb022ff023a0592d1c59b453f2cbc6a4af6da652 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Fri, 12 Jul 2019 16:49:50 +0200 Subject: [PATCH 27/94] Change to list of topics --- esque/clients.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/esque/clients.py b/esque/clients.py index 53d5a0a7..5a0b255f 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -42,7 +42,7 @@ def __init__(self, group_id: str, topic_name: str, last: bool): self._subscribe(topic_name) def _subscribe(self, topic: str) -> None: - self._consumer.subscribe(topic) + self._consumer.subscribe([topic]) def consume(self, amount: int) -> int: pass From f7c08d40197e26fab201c34cef9f0b269d131978 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Fri, 12 Jul 2019 17:42:36 +0200 Subject: [PATCH 28/94] Make FileWriter and Reader handleable for with --- esque/avromessage.py | 20 +++++++++++++++----- esque/clients.py | 10 +++++----- esque/message.py | 40 ++++++++++++++++++++++++++++++++++++---- 3 files changed, 56 insertions(+), 14 deletions(-) diff --git a/esque/avromessage.py b/esque/avromessage.py index 45bda51e..3ed94fe8 100644 --- a/esque/avromessage.py +++ b/esque/avromessage.py @@ -24,6 +24,7 @@ def __init__(self, key: Optional[Dict], value: Optional[Dict], key_schema_id: in class AvroFileWriter(FileWriter): def __init__(self, working_dir: pathlib.Path, schema_registry_client: SchemaRegistryClient): + super().__init__(working_dir) self.working_dir = working_dir self.schema_registry_client = schema_registry_client self.current_key_schema_id = None @@ -31,7 +32,7 @@ def __init__(self, working_dir: pathlib.Path, schema_registry_client: SchemaRegi self.schema_dir_name = None self.schema_version = it.count(1) - def write_message_to_file(self, message: Message, file: BinaryIO): + def write_message_to_file(self, message: Message): key_schema_id, decoded_key = self.decode_bytes(message.key()) value_schema_id, decoded_value = self.decode_bytes(message.value()) decoded_message = DecodedAvroMessage(decoded_key, decoded_value, key_schema_id, value_schema_id) @@ -47,7 +48,7 @@ def write_message_to_file(self, message: Message, file: BinaryIO): "value": decoded_value, "schema_directory_name": self.schema_dir_name, } - pickle.dump(serializable_message, file) + pickle.dump(serializable_message, self.file) def _dump_schemata(self, key_schema_id, value_schema_id): directory = self.working_dir / self.schema_dir_name @@ -74,11 +75,14 @@ def schema_changed(self, decoded_message: DecodedAvroMessage) -> bool: self.current_value_schema_id != decoded_message.value_schema_id and decoded_message.value is not None ) or self.current_key_schema_id != decoded_message.key_schema_id + def __enter__(self): + self.file = (self.working_dir / "data").open("wb+") + + def __exit__(self, exc_type, exc_val, exc_tb): + self.file.close() -class AvroFileReader(FileReader): - def __init__(self, working_dir: pathlib.Path): - self.working_dir = working_dir +class AvroFileReader(FileReader): def read_from_file(self, file: BinaryIO) -> Iterable[KafkaMessage]: while True: try: @@ -93,6 +97,12 @@ def read_from_file(self, file: BinaryIO) -> Iterable[KafkaMessage]: yield KafkaMessage(record["key"], record["value"], key_schema, value_schema) + def __enter__(self): + self.file = (self.working_dir / "data").open("rb") + + def __exit__(self, exc_type, exc_val, exc_tb): + self.file.close() + def extract_schema_id(message: bytes) -> int: _, schema_id = struct.unpack(">bI", message[:5]) diff --git a/esque/clients.py b/esque/clients.py index 5a0b255f..b900c97e 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -91,17 +91,17 @@ def __init__(self, group_id: str, topic_name: str, working_dir: pathlib.Path, la self._config.update({"default.topic.config": {"auto.offset.reset": offset_reset}}) self._consumer = confluent_kafka.Consumer(self._config) self._subscribe(topic_name) - self.file_writer = PlainTextFileWriter() + self.file_writer = PlainTextFileWriter(working_dir) def consume(self, amount: int) -> int: counter = 0 - with (self.working_dir / "data").open("wb") as file: + with self.file_writer: while counter < amount: message = self._consume_single_message() if message is None: return counter - self.file_writer.write_message_to_file(message, file) + self.file_writer.write_message_to_file(message) counter += 1 return counter @@ -111,7 +111,7 @@ class AvroFileConsumer(FileConsumer): def __init__(self, group_id: str, topic_name: str, working_dir: pathlib.Path, last: bool): super().__init__(group_id, topic_name, working_dir, last) schema_registry_client = SchemaRegistryClient(Config().schema_registry) - self.writer = AvroFileWriter(working_dir, schema_registry_client) + self.file_writer = AvroFileWriter(working_dir, schema_registry_client) class Producer(object): @@ -147,7 +147,7 @@ def __init__(self, working_dir: pathlib.Path): self.file_reader = PlainTextFileReader() def produce(self, topic_name: str) -> int: - with (self.working_dir / "data").open("rb") as file: + with self.file_reader as file: counter = 0 for message in self.file_reader.read_from_file(file): self.produce_message(topic_name, message) diff --git a/esque/message.py b/esque/message.py index f99bdb60..ff31508c 100644 --- a/esque/message.py +++ b/esque/message.py @@ -1,5 +1,6 @@ +import pathlib import pickle -from typing import BinaryIO, Iterable +from typing import BinaryIO, Iterable, IO from confluent_kafka.cimpl import Message @@ -19,20 +20,45 @@ def __init__(self, key: str, value: str, key_schema=None, value_schema=None): class FileWriter(object): - def write_message_to_file(self, message: Message, file: BinaryIO): + def __init__(self, file_path: pathlib.Path): + self.working_dir = file_path + + def write_message_to_file(self, message: Message): + pass + + def __enter__(self): + pass + + def __exit__(self, exc_type, exc_val, exc_tb): pass class FileReader(object): + def __init__(self, file_path: pathlib.Path): + self.working_dir = file_path + def read_from_file(self, file: BinaryIO) -> Iterable[KafkaMessage]: pass + def __enter__(self): + pass + + def __exit__(self, exc_type, exc_val, exc_tb): + pass + class PlainTextFileWriter(FileWriter): - def write_message_to_file(self, message: Message, file: BinaryIO): + def write_message_to_file(self, message: Message): decoded_message = decode_message(message) serializable_message = {"key": decoded_message.key, "value": decoded_message.value} - pickle.dump(serializable_message, file) + print(serializable_message) + pickle.dump(serializable_message, self.file) + + def __enter__(self): + self.file = (self.working_dir / "data").open("w+") + + def __exit__(self, exc_type, exc_val, exc_tb): + self.file.close() class PlainTextFileReader(FileReader): @@ -45,6 +71,12 @@ def read_from_file(self, file: BinaryIO) -> Iterable[KafkaMessage]: yield KafkaMessage(record["key"], record["value"]) + def __enter__(self): + self.file = (self.working_dir / "data").open("r") + + def __exit__(self, exc_type, exc_val, exc_tb): + self.file.close() + def decode_message(message: Message) -> DecodedMessage: decoded_key = message.key().decode("utf-8") From 040ce590088a02ca5226181f15b7c6d66bd2e2fa Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sat, 13 Jul 2019 13:45:39 +0200 Subject: [PATCH 29/94] Make FileWriter and FileReader usable with "with". --- esque/avromessage.py | 33 +++++++++++---------------- esque/clients.py | 38 ++++++++++++++----------------- esque/message.py | 54 ++++++++++++++++++-------------------------- tests/conftest.py | 2 ++ 4 files changed, 54 insertions(+), 73 deletions(-) diff --git a/esque/avromessage.py b/esque/avromessage.py index 3ed94fe8..b7594269 100644 --- a/esque/avromessage.py +++ b/esque/avromessage.py @@ -3,7 +3,7 @@ import pickle import struct from io import BytesIO -from typing import Optional, Tuple, Dict, BinaryIO, Iterable +from typing import Optional, Tuple, Dict, Iterable import itertools as it import fastavro @@ -23,14 +23,15 @@ def __init__(self, key: Optional[Dict], value: Optional[Dict], key_schema_id: in class AvroFileWriter(FileWriter): - def __init__(self, working_dir: pathlib.Path, schema_registry_client: SchemaRegistryClient): - super().__init__(working_dir) - self.working_dir = working_dir + def __init__(self, directory: pathlib.Path, schema_registry_client: SchemaRegistryClient): + super().__init__(directory) + self.directory = directory self.schema_registry_client = schema_registry_client self.current_key_schema_id = None self.current_value_schema_id = None self.schema_dir_name = None self.schema_version = it.count(1) + self.open_mode = "wb+" def write_message_to_file(self, message: Message): key_schema_id, decoded_key = self.decode_bytes(message.key()) @@ -51,7 +52,7 @@ def write_message_to_file(self, message: Message): pickle.dump(serializable_message, self.file) def _dump_schemata(self, key_schema_id, value_schema_id): - directory = self.working_dir / self.schema_dir_name + directory = self.directory / self.schema_dir_name directory.mkdir() (directory / "key_schema.avsc").write_text( json.dumps(self.schema_registry_client.get_schema_from_id(key_schema_id).original_schema) @@ -75,34 +76,26 @@ def schema_changed(self, decoded_message: DecodedAvroMessage) -> bool: self.current_value_schema_id != decoded_message.value_schema_id and decoded_message.value is not None ) or self.current_key_schema_id != decoded_message.key_schema_id - def __enter__(self): - self.file = (self.working_dir / "data").open("wb+") - - def __exit__(self, exc_type, exc_val, exc_tb): - self.file.close() - class AvroFileReader(FileReader): - def read_from_file(self, file: BinaryIO) -> Iterable[KafkaMessage]: + def __init__(self, directory: pathlib.Path): + super().__init__(directory) + self.open_mode = "rb" + + def read_from_file(self) -> Iterable[KafkaMessage]: while True: try: - record = pickle.load(file) + record = pickle.load(self.file) except EOFError: return - schema_directory = self.working_dir / record["schema_directory_name"] + schema_directory = self.directory / record["schema_directory_name"] key_schema = load_schema((schema_directory / "key_schema.avsc").read_text()) value_schema = load_schema((schema_directory / "value_schema.avsc").read_text()) yield KafkaMessage(record["key"], record["value"], key_schema, value_schema) - def __enter__(self): - self.file = (self.working_dir / "data").open("rb") - - def __exit__(self, exc_type, exc_val, exc_tb): - self.file.close() - def extract_schema_id(message: bytes) -> int: _, schema_id = struct.unpack(">bI", message[:5]) diff --git a/esque/clients.py b/esque/clients.py index b900c97e..98419778 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -4,13 +4,13 @@ import click import confluent_kafka import pendulum -from confluent_kafka import TopicPartition, Message +from confluent_kafka import Message from confluent_kafka.avro import AvroProducer from confluent_kafka.cimpl import KafkaError from esque.avromessage import AvroFileReader, AvroFileWriter from esque.config import Config -from esque.errors import raise_for_kafka_error, raise_for_message, KafkaException +from esque.errors import raise_for_kafka_error, raise_for_message from esque.helpers import delivery_callback, delta_t from esque.message import KafkaMessage, PlainTextFileReader, PlainTextFileWriter from esque.schemaregistry import SchemaRegistryClient @@ -50,23 +50,19 @@ def consume(self, amount: int) -> int: def _consume_single_message(self) -> Optional[Message]: poll_limit = 10 counter = 0 - try: - while counter < poll_limit: - message = self._consumer.poll(timeout=10) - if message is None: - counter += 1 - continue - if message.error() is not None: - if message.error().code() == KafkaError._PARTITION_EOF: - print("\nEnd of partition reached!".format(**locals())) - break - else: - raise RuntimeError(message.error().str()) - raise_for_message(message) - return message - except KafkaException as ex: - print("An error occurred: " + ex.message) - return None + while counter < poll_limit: + message = self._consumer.poll(timeout=10) + if message is None: + counter += 1 + continue + if message.error() is not None: + if message.error().code() == KafkaError._PARTITION_EOF: + print("\nEnd of partition reached!".format(**locals())) + break + else: + raise RuntimeError(message.error().str()) + raise_for_message(message) + return message class PingConsumer(Consumer): @@ -147,9 +143,9 @@ def __init__(self, working_dir: pathlib.Path): self.file_reader = PlainTextFileReader() def produce(self, topic_name: str) -> int: - with self.file_reader as file: + with self.file_reader: counter = 0 - for message in self.file_reader.read_from_file(file): + for message in self.file_reader.read_from_file(): self.produce_message(topic_name, message) counter += 1 diff --git a/esque/message.py b/esque/message.py index ff31508c..90e25af5 100644 --- a/esque/message.py +++ b/esque/message.py @@ -1,6 +1,6 @@ import pathlib import pickle -from typing import BinaryIO, Iterable, IO +from typing import Iterable from confluent_kafka.cimpl import Message @@ -19,64 +19,54 @@ def __init__(self, key: str, value: str, key_schema=None, value_schema=None): self.value_schema = value_schema -class FileWriter(object): - def __init__(self, file_path: pathlib.Path): - self.working_dir = file_path - - def write_message_to_file(self, message: Message): - pass +class IOHandler: + def __init__(self, directory: pathlib.Path): + self.directory = directory + self.file_name = "data" + self.open_mode = "w+" def __enter__(self): - pass + self.file = (self.directory / self.file_name).open(self.open_mode) def __exit__(self, exc_type, exc_val, exc_tb): - pass - + self.file.close() -class FileReader(object): - def __init__(self, file_path: pathlib.Path): - self.working_dir = file_path - def read_from_file(self, file: BinaryIO) -> Iterable[KafkaMessage]: +class FileWriter(IOHandler): + def write_message_to_file(self, message: Message): pass - def __enter__(self): - pass - def __exit__(self, exc_type, exc_val, exc_tb): +class FileReader(IOHandler): + def read_from_file(self) -> Iterable[KafkaMessage]: pass class PlainTextFileWriter(FileWriter): + def __init__(self, directory: pathlib.Path): + super().__init__(directory) + self.open_mode = "w+" + def write_message_to_file(self, message: Message): decoded_message = decode_message(message) serializable_message = {"key": decoded_message.key, "value": decoded_message.value} - print(serializable_message) pickle.dump(serializable_message, self.file) - def __enter__(self): - self.file = (self.working_dir / "data").open("w+") - - def __exit__(self, exc_type, exc_val, exc_tb): - self.file.close() - class PlainTextFileReader(FileReader): - def read_from_file(self, file: BinaryIO) -> Iterable[KafkaMessage]: + def __init__(self, directory: pathlib.Path): + super().__init__(directory) + self.open_mode = "r" + + def read_from_file(self) -> Iterable[KafkaMessage]: while True: try: - record = pickle.load(file) + record = pickle.load(self.file) except EOFError: return yield KafkaMessage(record["key"], record["value"]) - def __enter__(self): - self.file = (self.working_dir / "data").open("r") - - def __exit__(self, exc_type, exc_val, exc_tb): - self.file.close() - def decode_message(message: Message) -> DecodedMessage: decoded_key = message.key().decode("utf-8") diff --git a/tests/conftest.py b/tests/conftest.py index e5d99df5..2c2192c2 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -160,6 +160,8 @@ def working_dir(): def file_consumer(mocker, consumer_group, topic: str, working_dir: pathlib.Path): file_consumer = FileConsumer(consumer_group, topic, working_dir, False) file_writer = mocker.patch("esque.message.FileWriter") + file_writer.__enter__.assert_called_once() + file_writer.__exit__.assert_called_once() file_writer.write_message_to_file.call_count = 10 file_consumer.file_writer = file_consumer yield file_consumer From a605ad20e6e237bdab868511c929d94764818414 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sat, 13 Jul 2019 14:01:06 +0200 Subject: [PATCH 30/94] Use abc library --- Pipfile | 2 +- Pipfile.lock | 229 ++++++----------------------------------------- esque/clients.py | 13 +-- 3 files changed, 34 insertions(+), 210 deletions(-) diff --git a/Pipfile b/Pipfile index fab1b1af..7ee085e2 100644 --- a/Pipfile +++ b/Pipfile @@ -5,9 +5,9 @@ verify_ssl = true [dev-packages] kafkaesque = {path = ".",editable = true,extras = ["test", "dev"]} +esque = {editable = true,path = "."} [packages] -esque = {editable = true,path = "."} [requires] python_version = "3.6" diff --git a/Pipfile.lock b/Pipfile.lock index e31366ee..f7cbe882 100644 --- a/Pipfile.lock +++ b/Pipfile.lock @@ -1,7 +1,7 @@ { "_meta": { "hash": { - "sha256": "5eb10ba1263766fe6fe002f6c48d58c7c9ae648a6c25125893bd0346c1b11d18" + "sha256": "70ffd0673a6261321aa7425d25088d686c8c8bd414a857a68321b13a7ae7cd33" }, "pipfile-spec": 6, "requires": { @@ -15,188 +15,7 @@ } ] }, - "default": { - "avro-python3": { - "hashes": [ - "sha256:6163c1507621f5bdafab3370df7f82166c3c7ba53763a5637313639dcef02ae9" - ], - "version": "==1.9.0" - }, - "certifi": { - "hashes": [ - "sha256:046832c04d4e752f37383b628bc601a7ea7211496b4638f6514d0e5b9acc4939", - "sha256:945e3ba63a0b9f577b1395204e13c3a231f9bc0223888be653286534e5873695" - ], - "version": "==2019.6.16" - }, - "chardet": { - "hashes": [ - "sha256:84ab92ed1c4d4f16916e05906b6b75a6c0fb5db821cc65e70cbd64a3e2a5eaae", - "sha256:fc323ffcaeaed0e0a02bf4d117757b98aed530d9ed4531e3e15460124c106691" - ], - "version": "==3.0.4" - }, - "click": { - "hashes": [ - "sha256:2335065e6395b9e67ca716de5f7526736bfa6ceead690adf616d925bdc622b13", - "sha256:5b94b49521f6456670fdb30cd82a4eca9412788a93fa6dd6df72c94d5a8ff2d7" - ], - "version": "==7.0" - }, - "confluent-kafka": { - "hashes": [ - "sha256:01737c71c4596ef588b60ee1a1c56b64d8fd04169e3d5545ed31e08a530e00df", - "sha256:16736eb9a65c2ca5ddb7200254d7a97c2b59eb3ef4f985ed319e583cb7abc0e6", - "sha256:1c84038b2c515b27b14ab2d44b3f5c6ea7eeb848bfe755f5083c58edc60867af", - "sha256:228722c35efa27cafe30bb7236c8acf699b9a07deb8df011f7d8e975715b77f6", - "sha256:342299cd49c0af395bfaf0da5acfc1509081064429f809deb77754999ade8757", - "sha256:36f3942527100646b8dcc32b59ae03d34b36df96a2fa66b3d91a10e8fcf3002f", - "sha256:442186d9d6eca9c7a9d019a782645aeb210f1eb0dc85235ba60cd2f15ef89292", - "sha256:4dcb171ebed381f2f8b3dd0827d10398097dba346b5170a42501065bee075444", - "sha256:50d2300dad99f354b316660595eb461567e253fbb314848974542c58c04a7666", - "sha256:5505d42ad10fde01f0a7254c6e7e2348b0b9dc26be95bea7ad203bcea3ece201", - "sha256:6f9a99de81028fa687499f2f335374b0ee2366a970bc0664637ab346a1b77a11", - "sha256:76e1922ada8da7ee654892f2364e9b30730fba9154868cec1017d791134bedf7", - "sha256:790decee83bd0a15dc6b65a87703c156d63fde51b6b5c1c086cca13ee9300e8f", - "sha256:7cd66710c8552e83d09b60cf699309d47f749289b10379ad3ed01d1b5114b461", - "sha256:7eae7288fc28a8cd9ce5b758530ab2690c6ba15141c9192bc65774a2a7148432", - "sha256:88fbbf7c27db4770a2d5b26795608cf2a6b672811cbe337f81ca82fe7011586d", - "sha256:9857da6a911e7beae5e11acc8a494e95dabd4348c78a26b32b9ac98142b368da", - "sha256:98ba297342d33c87775abbb9403f4759b9d48c34f84020973ddb8a8b785bfde1", - "sha256:a2634c2de4b8cfdce9963723023658190edf7d8fddd3f4aa2eda448941654bd6", - "sha256:ab5ddd32c4bcf7d5fecf71a455b5525825d907b86cdab9594bad33a1e342360d", - "sha256:b25399b3e8bada06134b5a38f24ca70acf1524e70a6290c34a709b28b29ec284", - "sha256:b809a9ddd35492b1c24487925dbc4b3cf4f392bf8856584d49b6b5a674fd0776", - "sha256:bcb2da981e8713c900ab9903226167878f5731fcf33a3551ddb0614db88d1fcf", - "sha256:dd003e54b251e8e7300e808175d21c8ab9039c9633a6fd87fb99301d94f7d2bc", - "sha256:ddbd51a8429815ea5194d1d72881604836c6ea037cba79bf7440b2d4098ac7b0", - "sha256:e809dbe2cd3b934247d2c1e22736798150769e51415ea65d28ba496fcec22a7d" - ], - "version": "==1.0.1" - }, - "esque": { - "editable": true, - "path": "." - }, - "fastavro": { - "hashes": [ - "sha256:0930500497586037f1ca4bcb0e929d388d8cd86d1f5c43a72ac46bfa1003c785", - "sha256:0cf083c14b2a3c72704660e89480571ed2d4956cc5b79845905956d38d6eb7b6", - "sha256:3dd785459a7cd3c17012374c8929c73f385274dbbeed784ceda7879a4681c70f", - "sha256:49e671bcacc81b7b49f2c9f6f3db4b6201d5cf5809c321b33da3d2890c0f8481", - "sha256:52fd6153c7fbce93cef4e1c169543357582815a7665c142ebd97f29b7fabfc79", - "sha256:59b4a838c4f7b2646bf4d79fe3a0c4d546acf7e0bb65eda57eaa5ac76dc8dea9", - "sha256:61bc6d2c575b76e69b3ed16ee67827d67d166baeea255151cace550801bc5040", - "sha256:688e6fd5458ba673c1e1fff8ed25a27996e7ef5bcd73a50c6127d8c511703624", - "sha256:6c39c04e8ea5a85c512335a68bf08f922901817d49c113dd74de3dfebe38701f", - "sha256:76558659cce8d73f5fbd6084e362c25fcceae4e195e722b30d185f26416b0486", - "sha256:783f8600af4fc94c799e67f9deb0aedc7432e440937921b7f743dfc0753864bd", - "sha256:a94ca58353b972bc906713e02e8c5a173250070b5cfbf0271e347a8c810e32cf", - "sha256:aaeac57f4c4249d7d7b4e3ca1a43444e61a5aa382c83fb7580059e90e502485f", - "sha256:ca0eb9ff026490eb5c4ae83f48326932d6c206c4d9325e469ab79e4ad1956ac7", - "sha256:db387ce862b9dc71ecd8760e05ac0ee30b55b557cc2e2f2b33df0f3cf6f12a05", - "sha256:e9648a7dfe9358e550d093d546ae330d2a9e3246cbe5d7425a4ca17521ee7a72", - "sha256:eaa08fc8d416314bab328e9fe26df562b28ced24346109dbbae4a81151269d0f", - "sha256:fb492b325c9a373d62e3ea41b9b868ea276d0964e98adfa26c181f748a962bf3" - ], - "version": "==0.22.2" - }, - "idna": { - "hashes": [ - "sha256:c357b3f628cf53ae2c4c05627ecc484553142ca23264e593d327bcde5e9c3407", - "sha256:ea8b7f6188e6fa117537c3df7da9fc686d485087abf6ac197f9c46432f7e4a3c" - ], - "version": "==2.8" - }, - "kazoo": { - "hashes": [ - "sha256:8db774f7bdece7d0dc7decb21539ff0852e42c2ffe1c28d7f1ff6f9292a1c3a4", - "sha256:a5fa2e400c5068cfee9e86b35cf0dab8232b574152d8e3590d823b3e2426ab5e" - ], - "version": "==2.5.0" - }, - "pendulum": { - "hashes": [ - "sha256:1cde6e3c6310fb882c98f373795f807cb2bd6af01f34d2857e6e283b5ee91e09", - "sha256:485aef2089defee88607d37d5bc238934d0b90993d7bf9ceb36e481af41e9c66", - "sha256:57801754e05f30e8a7e4d24734c9fad82c6c3ec489151555f0fc66bb32ba6d6d", - "sha256:7ee344bc87cb425b04717b90d14ffde14c1dd64eaa73060b3772edcf57f3e866", - "sha256:c460f4d8dc41ec3c4377ac1807678cd72fe5e973cc2943c104ffdeaac32dacb7", - "sha256:d3078e007315a959989c41cee5cfd63cfeeca21dd3d8295f4bc24199489e9b6c" - ], - "version": "==2.0.5" - }, - "pykafka": { - "hashes": [ - "sha256:9b45c60ba40550f5b460a5a1441154255666e5935eae88c3cfa3aaba1d541690" - ], - "version": "==2.8.1.dev1" - }, - "python-dateutil": { - "hashes": [ - "sha256:7e6584c74aeed623791615e26efd690f29817a27c73085b78e4bad02493df2fb", - "sha256:c89805f6f4d64db21ed966fda138f8a5ed7a4fdbc1a8ee329ce1b74e3c74da9e" - ], - "version": "==2.8.0" - }, - "pytzdata": { - "hashes": [ - "sha256:c0c8316eaf6c25ba45816390a1a45c39790767069b3275c5f7de3ddf773eb810", - "sha256:e8a91952afd853642a49f0713caac3e15a5306855ff4a47af4ddec5b7dd23a09" - ], - "version": "==2019.2" - }, - "pyyaml": { - "hashes": [ - "sha256:57acc1d8533cbe51f6662a55434f0dbecfa2b9eaf115bede8f6fd00115a0c0d3", - "sha256:588c94b3d16b76cfed8e0be54932e5729cc185caffaa5a451e7ad2f7ed8b4043", - "sha256:68c8dd247f29f9a0d09375c9c6b8fdc64b60810ebf07ba4cdd64ceee3a58c7b7", - "sha256:70d9818f1c9cd5c48bb87804f2efc8692f1023dac7f1a1a5c61d454043c1d265", - "sha256:86a93cccd50f8c125286e637328ff4eef108400dd7089b46a7be3445eecfa391", - "sha256:a0f329125a926876f647c9fa0ef32801587a12328b4a3c741270464e3e4fa778", - "sha256:a3c252ab0fa1bb0d5a3f6449a4826732f3eb6c0270925548cac342bc9b22c225", - "sha256:b4bb4d3f5e232425e25dda21c070ce05168a786ac9eda43768ab7f3ac2770955", - "sha256:cd0618c5ba5bda5f4039b9398bb7fb6a317bb8298218c3de25c47c4740e4b95e", - "sha256:ceacb9e5f8474dcf45b940578591c7f3d960e82f926c707788a570b51ba59190", - "sha256:fe6a88094b64132c4bb3b631412e90032e8cfe9745a58370462240b8cb7553cd" - ], - "version": "==5.1.1" - }, - "requests": { - "hashes": [ - "sha256:11e007a8a2aa0323f5a921e9e6a2d7e4e67d9877e85773fba9ba6419025cbeb4", - "sha256:9cf5292fcd0f598c671cfc1e0d7d1a7f13bb8085e9a590f48c010551dc6c4b31" - ], - "version": "==2.22.0" - }, - "six": { - "hashes": [ - "sha256:3350809f0555b11f552448330d0b52d5f24c91a322ea4a15ef22629740f3761c", - "sha256:d16a0141ec1a18405cd4ce8b4613101da75da0e9a7aec5bdd4fa804d0e0eba73" - ], - "version": "==1.12.0" - }, - "tabulate": { - "hashes": [ - "sha256:8af07a39377cee1103a5c8b3330a421c2d99b9141e9cc5ddd2e3263fea416943" - ], - "version": "==0.8.3" - }, - "urllib3": { - "hashes": [ - "sha256:b246607a25ac80bedac05c6f282e3cdaf3afb65420fd024ac94435cabe6e18d1", - "sha256:dbe59173209418ae49d485b87d1681aefa36252ee85884c31346debd19463232" - ], - "version": "==1.25.3" - }, - "virtualenv": { - "hashes": [ - "sha256:b7335cddd9260a3dd214b73a2521ffc09647bde3e9457fcca31dc3be3999d04a", - "sha256:d28ca64c0f3f125f59cabf13e0a150e1c68e5eea60983cc4395d88c584495783" - ], - "version": "==16.6.1" - } - }, + "default": {}, "develop": { "appdirs": { "hashes": [ @@ -323,28 +142,32 @@ ], "version": "==0.3" }, + "esque": { + "editable": true, + "path": "." + }, "fastavro": { "hashes": [ - "sha256:0930500497586037f1ca4bcb0e929d388d8cd86d1f5c43a72ac46bfa1003c785", - "sha256:0cf083c14b2a3c72704660e89480571ed2d4956cc5b79845905956d38d6eb7b6", - "sha256:3dd785459a7cd3c17012374c8929c73f385274dbbeed784ceda7879a4681c70f", - "sha256:49e671bcacc81b7b49f2c9f6f3db4b6201d5cf5809c321b33da3d2890c0f8481", - "sha256:52fd6153c7fbce93cef4e1c169543357582815a7665c142ebd97f29b7fabfc79", - "sha256:59b4a838c4f7b2646bf4d79fe3a0c4d546acf7e0bb65eda57eaa5ac76dc8dea9", - "sha256:61bc6d2c575b76e69b3ed16ee67827d67d166baeea255151cace550801bc5040", - "sha256:688e6fd5458ba673c1e1fff8ed25a27996e7ef5bcd73a50c6127d8c511703624", - "sha256:6c39c04e8ea5a85c512335a68bf08f922901817d49c113dd74de3dfebe38701f", - "sha256:76558659cce8d73f5fbd6084e362c25fcceae4e195e722b30d185f26416b0486", - "sha256:783f8600af4fc94c799e67f9deb0aedc7432e440937921b7f743dfc0753864bd", - "sha256:a94ca58353b972bc906713e02e8c5a173250070b5cfbf0271e347a8c810e32cf", - "sha256:aaeac57f4c4249d7d7b4e3ca1a43444e61a5aa382c83fb7580059e90e502485f", - "sha256:ca0eb9ff026490eb5c4ae83f48326932d6c206c4d9325e469ab79e4ad1956ac7", - "sha256:db387ce862b9dc71ecd8760e05ac0ee30b55b557cc2e2f2b33df0f3cf6f12a05", - "sha256:e9648a7dfe9358e550d093d546ae330d2a9e3246cbe5d7425a4ca17521ee7a72", - "sha256:eaa08fc8d416314bab328e9fe26df562b28ced24346109dbbae4a81151269d0f", - "sha256:fb492b325c9a373d62e3ea41b9b868ea276d0964e98adfa26c181f748a962bf3" - ], - "version": "==0.22.2" + "sha256:02e5e2a1a6cd8b94703071a3afa1968ddb8f0dadc889ef56160773ac6029e066", + "sha256:14b35cf46ff86d26ecb998afe98c984c3520a2191f5db67e624a2aab7dc22808", + "sha256:231332c5ef48c06e44bef6e993abea76182efb88134cfdad72cb02b0a01bc75f", + "sha256:3bb0d71087a3e52c76deb179bb6c4363ed6f8c4e671b445e605bf7652f551b32", + "sha256:6888f376183a5f28ad9acbd1ed57ba260c9e4d10eef5c29aa90a89c9ad88fe9d", + "sha256:6b6d9c338ff95e113ac53661dcc005cf65641336c259125bb448c3ccd5b33017", + "sha256:88bc5bfe8cb2c39d79805b9af06165d6ecccbd70fb27753ab65ecc77b65bbc08", + "sha256:8bd421c541c196f16053beffbd5e494bdbc6425fe3de9920ddc4208083e86552", + "sha256:9d015ff0dc109daa65f0f5735ed49cf699ce5e68ba99651aab4d1f3351971d0a", + "sha256:a0ad23c8e9b213169d7d4cfac5485d3844f2f4c43a7ad950e46fd3c39972d0e1", + "sha256:a862dcb93f81bd14d03b54506139f04066a96956d06c75c9fd46059443d44f31", + "sha256:abb3cdb8e395d31e90e304090df1f32ca81ed7652eb72eea4730aea0c7d6da5f", + "sha256:b94bb7aa7d0901b6a672b34dcec2eec4b83d3df1a3f09802b16d3d933e0caefd", + "sha256:cd70878dd240a4e1c127e489a707de60afc8ffa8602f994a109ed16f077a30af", + "sha256:d5d716b8f01f206313d2fbb9c82e1e533eeb8cbeb0c8a98db44a971a5faa8143", + "sha256:d6a307c2af45900a06917b41222fe89af6d39dc20d1dff88295095273786de53", + "sha256:dfd0bafd563593e22e285eea63894b8f0d0d7a47b3743ec76d36b0de5d0e4c09", + "sha256:f8af6f955c05ae7f73e409cb73e504e6af3a247219f94520dc3ddbcb3d97e484" + ], + "version": "==0.22.3" }, "flake8": { "hashes": [ diff --git a/esque/clients.py b/esque/clients.py index 98419778..865c6348 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -14,11 +14,10 @@ from esque.helpers import delivery_callback, delta_t from esque.message import KafkaMessage, PlainTextFileReader, PlainTextFileWriter from esque.schemaregistry import SchemaRegistryClient +from abc import ABC, abstractmethod -DEFAULT_RETENTION_MS = 7 * 24 * 60 * 60 * 1000 - -class Consumer: +class AbstractConsumer(ABC): def __init__(self, group_id: str, topic_name: str, last: bool): offset_reset = "earliest" if last: @@ -44,6 +43,7 @@ def __init__(self, group_id: str, topic_name: str, last: bool): def _subscribe(self, topic: str) -> None: self._consumer.subscribe([topic]) + @abstractmethod def consume(self, amount: int) -> int: pass @@ -65,7 +65,7 @@ def _consume_single_message(self) -> Optional[Message]: return message -class PingConsumer(Consumer): +class PingConsumer(AbstractConsumer): def consume(self, amount: int) -> Optional[Tuple[str, int]]: msg = self._consumer.consume(timeout=10)[0] @@ -76,7 +76,7 @@ def consume(self, amount: int) -> Optional[Tuple[str, int]]: return msg.key(), delta_sent.microseconds / 1000 -class FileConsumer(Consumer): +class FileConsumer(AbstractConsumer): def __init__(self, group_id: str, topic_name: str, working_dir: pathlib.Path, last: bool): super().__init__(group_id, topic_name, last) self.working_dir = working_dir @@ -110,7 +110,8 @@ def __init__(self, group_id: str, topic_name: str, working_dir: pathlib.Path, la self.file_writer = AvroFileWriter(working_dir, schema_registry_client) -class Producer(object): +class Producer(ABC): + @abstractmethod def produce(self, topic_name: str) -> int: pass From 58e5c95514f229c385cdc608764cfac43a4dc478 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sat, 13 Jul 2019 14:17:05 +0200 Subject: [PATCH 31/94] Update travis.yml and setup.py --- .travis.yml | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index 8f494963..0bc892b4 100644 --- a/.travis.yml +++ b/.travis.yml @@ -28,7 +28,7 @@ matrix: - name: black python: 3.7 env: - - TEST_CMD="black --check --verbose ." + - TEST_CMD="black --check --verbose --line-length=119 ." - name: flake8 python: 3.7 env: diff --git a/setup.py b/setup.py index 1030a428..2f8904ea 100644 --- a/setup.py +++ b/setup.py @@ -32,7 +32,7 @@ "pendulum", "pyyaml", "requests", - "fastavro>=0.21.8", + "fastavro>=0.22.3", "avro-python3", ] From 9adc0a1f8271b500866a4a0e1c4a487a10cc2ad3 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sat, 13 Jul 2019 14:57:05 +0200 Subject: [PATCH 32/94] Remove assertions for methods --- tests/conftest.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/conftest.py b/tests/conftest.py index 2c2192c2..e5d99df5 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -160,8 +160,6 @@ def working_dir(): def file_consumer(mocker, consumer_group, topic: str, working_dir: pathlib.Path): file_consumer = FileConsumer(consumer_group, topic, working_dir, False) file_writer = mocker.patch("esque.message.FileWriter") - file_writer.__enter__.assert_called_once() - file_writer.__exit__.assert_called_once() file_writer.write_message_to_file.call_count = 10 file_consumer.file_writer = file_consumer yield file_consumer From 2bac25f44ae1821b3324aed5cd6d4987f5243703 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sat, 13 Jul 2019 15:07:15 +0200 Subject: [PATCH 33/94] Update fixture file_consumer --- tests/conftest.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/conftest.py b/tests/conftest.py index e5d99df5..6d80ff78 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -160,6 +160,8 @@ def working_dir(): def file_consumer(mocker, consumer_group, topic: str, working_dir: pathlib.Path): file_consumer = FileConsumer(consumer_group, topic, working_dir, False) file_writer = mocker.patch("esque.message.FileWriter") + file_writer.__enter__ = mocker.Mock(return_value=None) + file_writer.__exit__ = mocker.Mock(return_value=None) file_writer.write_message_to_file.call_count = 10 file_consumer.file_writer = file_consumer yield file_consumer From 87e15da2256ae243ffc50cd32be65e330c417782 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sat, 13 Jul 2019 15:29:57 +0200 Subject: [PATCH 34/94] Update tests --- tests/conftest.py | 9 ++------- tests/integration/test_clients.py | 12 +++++++++++- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/tests/conftest.py b/tests/conftest.py index 6d80ff78..db618ec2 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -140,7 +140,7 @@ def consumer(topic_object: Topic, consumer_group): } ) _consumer = confluent_kafka.Consumer(_config) - _consumer.assign([TopicPartition(topic=topic_object.name, partition=0, offset=0)]) + _consumer.subscribe([topic_object.name]) yield _consumer @@ -157,13 +157,8 @@ def working_dir(): @pytest.fixture() -def file_consumer(mocker, consumer_group, topic: str, working_dir: pathlib.Path): +def file_consumer(consumer_group, topic: str, working_dir: pathlib.Path): file_consumer = FileConsumer(consumer_group, topic, working_dir, False) - file_writer = mocker.patch("esque.message.FileWriter") - file_writer.__enter__ = mocker.Mock(return_value=None) - file_writer.__exit__ = mocker.Mock(return_value=None) - file_writer.write_message_to_file.call_count = 10 - file_consumer.file_writer = file_consumer yield file_consumer diff --git a/tests/integration/test_clients.py b/tests/integration/test_clients.py index ea1cb8e3..c6903d7c 100644 --- a/tests/integration/test_clients.py +++ b/tests/integration/test_clients.py @@ -1,8 +1,18 @@ +from unittest import mock + import pytest +from confluent_kafka.cimpl import Consumer from esque.clients import FileConsumer @pytest.mark.integration -def test_consume_to_file(file_consumer: FileConsumer): +def test_consume_to_file(mocker, file_consumer: FileConsumer, consumer: Consumer): + messages = consumer.consume(10) + messages_call_list = [mock.call([message] for message in messages)] + file_writer = mocker.patch("esque.message.FileWriter") + file_writer.__enter__ = mocker.Mock(return_value=None) + file_writer.__exit__ = mocker.Mock(return_value=None) + file_writer.write_message_to_file.assert_has_calls(messages_call_list) + file_consumer.file_writer = file_writer file_consumer.consume(10) From c55c0f2ad9c538a058686bdfdb51d1df298b5a52 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sat, 13 Jul 2019 16:25:35 +0200 Subject: [PATCH 35/94] Update tests for consume and produce to and from file --- tests/conftest.py | 11 +++++---- tests/integration/test_clients.py | 39 ++++++++++++++++++++++++------- 2 files changed, 36 insertions(+), 14 deletions(-) diff --git a/tests/conftest.py b/tests/conftest.py index db618ec2..8905b687 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -11,7 +11,7 @@ from pykafka import Producer from pykafka.exceptions import NoBrokersAvailableError -from esque.clients import FileConsumer, AvroFileConsumer +from esque.clients import FileConsumer, AvroFileConsumer, FileProducer from esque.cluster import Cluster from esque.config import Config, sample_config_path from esque.consumergroup import ConsumerGroupController @@ -157,14 +157,15 @@ def working_dir(): @pytest.fixture() -def file_consumer(consumer_group, topic: str, working_dir: pathlib.Path): - file_consumer = FileConsumer(consumer_group, topic, working_dir, False) +def file_consumer(consumer_group, filled_topic: Topic, working_dir: pathlib.Path): + file_consumer = FileConsumer(consumer_group, filled_topic.name, working_dir, False) yield file_consumer @pytest.fixture() -def avro_file_consumer(consumer_group, topic: str, working_dir: pathlib.Path): - yield AvroFileConsumer(consumer_group, topic, working_dir, False) +def file_producer(working_dir: pathlib.Path): + file_producer = FileProducer(working_dir) + yield file_producer @pytest.fixture() diff --git a/tests/integration/test_clients.py b/tests/integration/test_clients.py index c6903d7c..fde11226 100644 --- a/tests/integration/test_clients.py +++ b/tests/integration/test_clients.py @@ -1,18 +1,39 @@ -from unittest import mock - import pytest -from confluent_kafka.cimpl import Consumer -from esque.clients import FileConsumer +from esque.clients import FileConsumer, FileProducer +from esque.message import KafkaMessage @pytest.mark.integration -def test_consume_to_file(mocker, file_consumer: FileConsumer, consumer: Consumer): - messages = consumer.consume(10) - messages_call_list = [mock.call([message] for message in messages)] +def test_consume_to_file(mocker, file_consumer: FileConsumer): file_writer = mocker.patch("esque.message.FileWriter") file_writer.__enter__ = mocker.Mock(return_value=None) file_writer.__exit__ = mocker.Mock(return_value=None) - file_writer.write_message_to_file.assert_has_calls(messages_call_list) + file_writer.write_message_to_file = mocker.Mock() file_consumer.file_writer = file_writer - file_consumer.consume(10) + number_of_consumer_messages = file_consumer.consume(10) + + assert number_of_consumer_messages == 10 + assert file_writer.write_message_to_file.call_count == 10 + + +@pytest.mark.integration +def test_produce_from_file(mocker, file_producer: FileProducer, topic: str): + file_reader = mocker.patch("esque.message.FileReader") + file_reader.__enter__ = mocker.Mock(return_value=None) + file_reader.__exit__ = mocker.Mock(return_value=None) + file_reader.read_from_file = mocker.Mock() + file_reader.read_from_file.return_value = [KafkaMessage("key", "value") for _ in range(10)] + file_producer.file_reader = file_reader + + producer = mocker.Mock() + producer.produce_message = mocker.Mock() + file_producer._producer = producer + + file_producer.produce(topic) + + assert producer.produce_message.call_count == 10 + + + + From 1bc83e7d989887d6d7ee3c6ce6411281bde08aea Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sat, 13 Jul 2019 16:29:46 +0200 Subject: [PATCH 36/94] Fix styling --- tests/integration/test_clients.py | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tests/integration/test_clients.py b/tests/integration/test_clients.py index fde11226..31e3ee55 100644 --- a/tests/integration/test_clients.py +++ b/tests/integration/test_clients.py @@ -33,7 +33,3 @@ def test_produce_from_file(mocker, file_producer: FileProducer, topic: str): file_producer.produce(topic) assert producer.produce_message.call_count == 10 - - - - From 583b36975a0338dae20f78db4ddcbff12651fd73 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sun, 14 Jul 2019 16:56:01 +0200 Subject: [PATCH 37/94] Check written files in tests and bug fixes --- config/sample_config.cfg | 2 +- docker-compose.yml | 37 +++------------------- esque/avromessage.py | 9 +++--- esque/clients.py | 14 ++++----- esque/message.py | 8 ++--- tests/conftest.py | 52 +++++++++++++++++++------------ tests/integration/test_clients.py | 47 +++++++++++++++------------- tests/test_samples/key_schema | 8 +++++ tests/test_samples/value_schema | 9 ++++++ 9 files changed, 95 insertions(+), 91 deletions(-) create mode 100644 tests/test_samples/key_schema create mode 100644 tests/test_samples/value_schema diff --git a/config/sample_config.cfg b/config/sample_config.cfg index bcbe2d8d..68ce021b 100644 --- a/config/sample_config.cfg +++ b/config/sample_config.cfg @@ -11,4 +11,4 @@ schema_registry = localhost:8081 bootstrap_hosts = kafka bootstrap_port = 9093 security_protocol = PLAINTEXT -schema_registry = schema-registry:8081 +schema_registry = http://schema_registry:8081 diff --git a/docker-compose.yml b/docker-compose.yml index c87c3a32..f9bc714b 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -37,45 +37,16 @@ services: KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 KAFKA_GROUP_INITIAL_REBALANCE_DELAY_MS: 0 - - schema_registry_source: + schema_registry: image: confluentinc/cp-schema-registry:5.2.2 - container_name: schema_registry_source + container_name: schema_registry environment: - SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL=zookeeper:2181 - SCHEMA_REGISTRY_HOST_NAME=schema-registry - SCHEMA_REGISTRY_LISTENERS=http://0.0.0.0:8081 - - SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS=PLAINTEXT://kafka:9092 + - SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS=PLAINTEXT://kafka:9093 links: - zookeeper - kafka ports: - - 8081:8081 - - schema_registry_target: - image: confluentinc/cp-schema-registry:5.2.2 - container_name: schema_registry_target - environment: - - SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL=zookeeper:2181 - - SCHEMA_REGISTRY_HOST_NAME=schema-registry - - SCHEMA_REGISTRY_LISTENERS=http://0.0.0.0:8082 - - SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS=PLAINTEXT://kafka:9092 - links: - - zookeeper - - kafka - ports: - - 8082:8081 - - esque: - build: . - volumes: - - .:/esque - environment: - ESQUE_TEST_ENV: "ci" - depends_on: - - kafka - command: > - -c - "(until (kafkacat -b kafka:9093 -X debug=all -L); do sleep 5s; done) \ - && pytest tests/ --integration" - + - 8081:8081 \ No newline at end of file diff --git a/esque/avromessage.py b/esque/avromessage.py index b7594269..fa5533be 100644 --- a/esque/avromessage.py +++ b/esque/avromessage.py @@ -55,10 +55,11 @@ def _dump_schemata(self, key_schema_id, value_schema_id): directory = self.directory / self.schema_dir_name directory.mkdir() (directory / "key_schema.avsc").write_text( - json.dumps(self.schema_registry_client.get_schema_from_id(key_schema_id).original_schema) + json.dumps(self.schema_registry_client.get_schema_from_id(key_schema_id).original_schema), encoding="utf-8" ) (directory / "value_schema.avsc").write_text( - json.dumps(self.schema_registry_client.get_schema_from_id(value_schema_id).original_schema) + json.dumps(self.schema_registry_client.get_schema_from_id(value_schema_id).original_schema), + encoding="utf-8", ) def decode_bytes(self, raw_data: Optional[bytes]) -> Tuple[int, Optional[Dict]]: @@ -91,8 +92,8 @@ def read_from_file(self) -> Iterable[KafkaMessage]: schema_directory = self.directory / record["schema_directory_name"] - key_schema = load_schema((schema_directory / "key_schema.avsc").read_text()) - value_schema = load_schema((schema_directory / "value_schema.avsc").read_text()) + key_schema = load_schema((schema_directory / "key_schema.avsc").read_text(encoding="utf-8")) + value_schema = load_schema((schema_directory / "value_schema.avsc").read_text(encoding="utf-8")) yield KafkaMessage(record["key"], record["value"], key_schema, value_schema) diff --git a/esque/clients.py b/esque/clients.py index 865c6348..be65e5e9 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -111,6 +111,10 @@ def __init__(self, group_id: str, topic_name: str, working_dir: pathlib.Path, la class Producer(ABC): + def __init__(self): + self._config = Config().create_confluent_config() + self._config.update({"on_delivery": delivery_callback, "error_cb": raise_for_kafka_error}) + @abstractmethod def produce(self, topic_name: str) -> int: pass @@ -118,9 +122,7 @@ def produce(self, topic_name: str) -> int: class PingProducer(Producer): def __init__(self): - self._config = Config().create_confluent_config() - self._config.update({"on_delivery": delivery_callback, "error_cb": raise_for_kafka_error}) - + super().__init__() self._producer = confluent_kafka.Producer(self._config) def produce(self, topic_name: str) -> int: @@ -136,12 +138,10 @@ def produce(self, topic_name: str) -> int: class FileProducer(Producer): def __init__(self, working_dir: pathlib.Path): - self._config = Config().create_confluent_config() - self._config.update({"on_delivery": delivery_callback, "error_cb": raise_for_kafka_error}) - + super().__init__() self._producer = confluent_kafka.Producer(self._config) self.working_dir = working_dir - self.file_reader = PlainTextFileReader() + self.file_reader = PlainTextFileReader(working_dir) def produce(self, topic_name: str) -> int: with self.file_reader: diff --git a/esque/message.py b/esque/message.py index 90e25af5..c0a91d97 100644 --- a/esque/message.py +++ b/esque/message.py @@ -1,5 +1,5 @@ +import json import pathlib -import pickle from typing import Iterable from confluent_kafka.cimpl import Message @@ -50,7 +50,7 @@ def __init__(self, directory: pathlib.Path): def write_message_to_file(self, message: Message): decoded_message = decode_message(message) serializable_message = {"key": decoded_message.key, "value": decoded_message.value} - pickle.dump(serializable_message, self.file) + self.file.write(json.dumps(serializable_message) + "\n") class PlainTextFileReader(FileReader): @@ -59,9 +59,9 @@ def __init__(self, directory: pathlib.Path): self.open_mode = "r" def read_from_file(self) -> Iterable[KafkaMessage]: - while True: + for line in self.file: try: - record = pickle.load(self.file) + record = json.loads(line) except EOFError: return diff --git a/tests/conftest.py b/tests/conftest.py index 8905b687..c3911494 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -1,4 +1,4 @@ -import pathlib +import json import random from concurrent.futures import Future from pathlib import Path @@ -7,11 +7,11 @@ import confluent_kafka import pytest from confluent_kafka.admin import AdminClient, NewTopic -from confluent_kafka.cimpl import TopicPartition -from pykafka import Producer +from confluent_kafka.avro import AvroProducer +from confluent_kafka.cimpl import TopicPartition, Producer from pykafka.exceptions import NoBrokersAvailableError +from confluent_kafka.avro import loads as load_schema -from esque.clients import FileConsumer, AvroFileConsumer, FileProducer from esque.cluster import Cluster from esque.config import Config, sample_config_path from esque.consumergroup import ConsumerGroupController @@ -103,9 +103,15 @@ def confluent_admin_client(test_config: Config) -> AdminClient: @pytest.fixture() -def producer(topic_object: Topic): - # Send messages synchronously so we can be sure offset has been commited in tests. - yield Producer(topic_object.cluster.pykafka_client.cluster, topic_object._pykafka_topic, sync=True) +def producer(test_config: Config): + yield Producer(test_config.create_confluent_config()) + + +@pytest.fixture() +def avro_producer(test_config: Config): + producer_config = test_config.create_confluent_config() + producer_config.update({"schema.registry.url": Config().schema_registry}) + yield AvroProducer(producer_config) @pytest.fixture() @@ -140,32 +146,38 @@ def consumer(topic_object: Topic, consumer_group): } ) _consumer = confluent_kafka.Consumer(_config) - _consumer.subscribe([topic_object.name]) + _consumer.assign([TopicPartition(topic=topic_object.name, partition=0, offset=0)]) yield _consumer @pytest.fixture() def filled_topic(producer, topic_object): for _ in range(10): - producer.produce("".join(random.choices(ascii_letters, k=5)).encode("utf-8")) + random_value = "".join(random.choices(ascii_letters, k=5)).encode("utf-8") + producer.produce(topic=topic_object.name, key=random_value, value=random_value) + producer.flush() yield topic_object @pytest.fixture() -def working_dir(): - yield Path("test_directory") - - -@pytest.fixture() -def file_consumer(consumer_group, filled_topic: Topic, working_dir: pathlib.Path): - file_consumer = FileConsumer(consumer_group, filled_topic.name, working_dir, False) - yield file_consumer +def filled_avro_topic(avro_producer: AvroProducer, topic_object): + with open("tests/test_samples/key_schema", "r") as file: + key_schema = load_schema(file.read()) + with open("tests/test_samples/value_schema", "r") as file: + value_schema = load_schema(file.read()) + for i in range(10): + key = {"id": str(i)} + value = {"first": "Firstname", "last": "Lastname"} + avro_producer.produce( + topic=topic_object.name, key=key, value=value, key_schema=key_schema, value_schema=value_schema + ) + avro_producer.flush() + yield topic_object @pytest.fixture() -def file_producer(working_dir: pathlib.Path): - file_producer = FileProducer(working_dir) - yield file_producer +def working_dir(tmpdir_factory): + yield tmpdir_factory.mktemp("working_directory") @pytest.fixture() diff --git a/tests/integration/test_clients.py b/tests/integration/test_clients.py index 31e3ee55..78556bc9 100644 --- a/tests/integration/test_clients.py +++ b/tests/integration/test_clients.py @@ -1,35 +1,38 @@ +import pathlib + import pytest -from esque.clients import FileConsumer, FileProducer -from esque.message import KafkaMessage +from esque.avromessage import AvroFileReader +from esque.clients import FileConsumer, AvroFileConsumer +from esque.message import PlainTextFileReader +from esque.topic import Topic @pytest.mark.integration -def test_consume_to_file(mocker, file_consumer: FileConsumer): - file_writer = mocker.patch("esque.message.FileWriter") - file_writer.__enter__ = mocker.Mock(return_value=None) - file_writer.__exit__ = mocker.Mock(return_value=None) - file_writer.write_message_to_file = mocker.Mock() - file_consumer.file_writer = file_writer +def test_plaint_text_consume_to_file(consumer_group, filled_topic: Topic, working_dir: pathlib.Path): + file_consumer = FileConsumer(consumer_group, filled_topic.name, working_dir, False) number_of_consumer_messages = file_consumer.consume(10) + messages = [] + file_reader = PlainTextFileReader(working_dir) + with file_reader: + for message in file_reader.read_from_file(): + messages.append({"key": message.key, "value": message.value}) + assert number_of_consumer_messages == 10 - assert file_writer.write_message_to_file.call_count == 10 + assert len(messages) == 10 @pytest.mark.integration -def test_produce_from_file(mocker, file_producer: FileProducer, topic: str): - file_reader = mocker.patch("esque.message.FileReader") - file_reader.__enter__ = mocker.Mock(return_value=None) - file_reader.__exit__ = mocker.Mock(return_value=None) - file_reader.read_from_file = mocker.Mock() - file_reader.read_from_file.return_value = [KafkaMessage("key", "value") for _ in range(10)] - file_producer.file_reader = file_reader - - producer = mocker.Mock() - producer.produce_message = mocker.Mock() - file_producer._producer = producer +def test_avro_consume_to_file(consumer_group, filled_avro_topic: Topic, working_dir: pathlib.Path): + file_consumer = AvroFileConsumer(consumer_group, filled_avro_topic.name, working_dir, False) + number_of_consumer_messages = file_consumer.consume(10) - file_producer.produce(topic) + messages = [] + file_reader = AvroFileReader(working_dir) + with file_reader: + for message in file_reader.read_from_file(): + messages.append({"key": message.key, "value": message.value}) - assert producer.produce_message.call_count == 10 + assert number_of_consumer_messages == 10 + assert len(messages) == 10 diff --git a/tests/test_samples/key_schema b/tests/test_samples/key_schema new file mode 100644 index 00000000..ea5741f1 --- /dev/null +++ b/tests/test_samples/key_schema @@ -0,0 +1,8 @@ +{ + "type": "record", + "namespace": "com.example", + "name": "Identifier", + "fields": [ + { "name": "id", "type": "string" } + ] +} \ No newline at end of file diff --git a/tests/test_samples/value_schema b/tests/test_samples/value_schema new file mode 100644 index 00000000..94d6320d --- /dev/null +++ b/tests/test_samples/value_schema @@ -0,0 +1,9 @@ +{ + "type": "record", + "namespace": "com.example", + "name": "FullName", + "fields": [ + { "name": "first", "type": "string" }, + { "name": "last", "type": "string" } + ] +} \ No newline at end of file From f14e1ec1645ad7194a5c1a56fcb6f3c80e01c777 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sun, 14 Jul 2019 17:09:35 +0200 Subject: [PATCH 38/94] Fix schema_registry uri --- config/sample_config.cfg | 2 +- docker-compose.test.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/config/sample_config.cfg b/config/sample_config.cfg index 68ce021b..d9244248 100644 --- a/config/sample_config.cfg +++ b/config/sample_config.cfg @@ -5,7 +5,7 @@ current = docker bootstrap_hosts = localhost bootstrap_port = 9092 security_protocol = LOCAL -schema_registry = localhost:8081 +schema_registry = http://localhost:8081 [Context.docker] bootstrap_hosts = kafka diff --git a/docker-compose.test.yml b/docker-compose.test.yml index b7e4631f..0bd56a84 100644 --- a/docker-compose.test.yml +++ b/docker-compose.test.yml @@ -11,4 +11,4 @@ services: command: > -c "(until (kafkacat -b kafka:9093 -X debug=all -L); do sleep 5s; done) \ - && pytest tests/ --integration" \ No newline at end of file + && pytest tests/integration/test_clients.py --integration" \ No newline at end of file From 207033321946d5eb9caa33542e1943f679831b5a Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sun, 14 Jul 2019 18:55:13 +0200 Subject: [PATCH 39/94] Add schema registry to travis --- .travis.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.travis.yml b/.travis.yml index 0bc892b4..9215d7c4 100644 --- a/.travis.yml +++ b/.travis.yml @@ -8,8 +8,10 @@ before_install: - mkdir -p kafka && tar xzf kafka.tgz -C kafka --strip-components 1 - nohup bash -c "cd kafka && bin/zookeeper-server-start.sh config/zookeeper.properties &" - nohup bash -c "cd kafka && bin/kafka-server-start.sh config/server.properties &" +- nohup bash -c "cd kafka && bin/schema-registry-start config/schema-registry.properties &" - scripts/wait-for-it.sh localhost:9092 - scripts/wait-for-it.sh localhost:2181 +- scripts/wait-for-it.sh localhost:8081 install: - pip install coverage coveralls flake8 pipenv - pipenv install --system --dev --deploy From ee03928488c23b26be8a0fd3139623c58007e709 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sun, 14 Jul 2019 19:13:09 +0200 Subject: [PATCH 40/94] Use confluent package with schema registry for travis --- .travis.yml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/.travis.yml b/.travis.yml index 9215d7c4..78a4173b 100644 --- a/.travis.yml +++ b/.travis.yml @@ -4,11 +4,11 @@ cache: pip env: - TEST_CMD="pytest tests/ --integration --cov=esque --local" before_install: -- wget https://mirror.netcologne.de/apache.org/kafka/2.2.0/kafka_2.12-2.2.0.tgz -O kafka.tgz +- wget http://packages.confluent.io/archive/3.0/confluent-3.0.0-2.11.tar.gz -O kafka.tgz - mkdir -p kafka && tar xzf kafka.tgz -C kafka --strip-components 1 -- nohup bash -c "cd kafka && bin/zookeeper-server-start.sh config/zookeeper.properties &" -- nohup bash -c "cd kafka && bin/kafka-server-start.sh config/server.properties &" -- nohup bash -c "cd kafka && bin/schema-registry-start config/schema-registry.properties &" +- nohup bash -c "cd kafka && bin/zookeeper-server-start.sh etc/kafka/zookeeper.properties &" +- nohup bash -c "cd kafka && bin/kafka-server-start.sh etc/kafka/server.properties &" +- nohup bash -c "cd kafka && bin/schema-registry-start etc/schema-registry/schema-registry.properties &" - scripts/wait-for-it.sh localhost:9092 - scripts/wait-for-it.sh localhost:2181 - scripts/wait-for-it.sh localhost:8081 From f22a5854920bc8692e6b9bb8d2666246da340217 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sun, 14 Jul 2019 19:24:58 +0200 Subject: [PATCH 41/94] Fix travis.yml --- .travis.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.travis.yml b/.travis.yml index 78a4173b..472b14eb 100644 --- a/.travis.yml +++ b/.travis.yml @@ -6,9 +6,9 @@ env: before_install: - wget http://packages.confluent.io/archive/3.0/confluent-3.0.0-2.11.tar.gz -O kafka.tgz - mkdir -p kafka && tar xzf kafka.tgz -C kafka --strip-components 1 -- nohup bash -c "cd kafka && bin/zookeeper-server-start.sh etc/kafka/zookeeper.properties &" -- nohup bash -c "cd kafka && bin/kafka-server-start.sh etc/kafka/server.properties &" -- nohup bash -c "cd kafka && bin/schema-registry-start etc/schema-registry/schema-registry.properties &" +- nohup bash -c "cd kafka && ./bin/zookeeper-server-start etc/kafka/zookeeper.properties &" +- nohup bash -c "cd kafka && ./bin/kafka-server-start etc/kafka/server.properties &" +- nohup bash -c "cd kafka && ./bin/schema-registry-start etc/schema-registry/schema-registry.properties &" - scripts/wait-for-it.sh localhost:9092 - scripts/wait-for-it.sh localhost:2181 - scripts/wait-for-it.sh localhost:8081 From accb8cefd6c2c5c1240596ca175eb9913814bda8 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sun, 14 Jul 2019 19:34:38 +0200 Subject: [PATCH 42/94] Fix travis.yml --- .travis.yml | 6 +++--- scripts/wait-for-it.sh | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.travis.yml b/.travis.yml index 472b14eb..b2470799 100644 --- a/.travis.yml +++ b/.travis.yml @@ -6,9 +6,9 @@ env: before_install: - wget http://packages.confluent.io/archive/3.0/confluent-3.0.0-2.11.tar.gz -O kafka.tgz - mkdir -p kafka && tar xzf kafka.tgz -C kafka --strip-components 1 -- nohup bash -c "cd kafka && ./bin/zookeeper-server-start etc/kafka/zookeeper.properties &" -- nohup bash -c "cd kafka && ./bin/kafka-server-start etc/kafka/server.properties &" -- nohup bash -c "cd kafka && ./bin/schema-registry-start etc/schema-registry/schema-registry.properties &" +- nohup bash -c "cd kafka && bin/zookeeper-server-start etc/kafka/zookeeper.properties &" +- nohup bash -c "cd kafka && bin/kafka-server-start etc/kafka/server.properties &" +- nohup bash -c "cd kafka && bin/schema-registry-start etc/schema-registry/schema-registry.properties &" - scripts/wait-for-it.sh localhost:9092 - scripts/wait-for-it.sh localhost:2181 - scripts/wait-for-it.sh localhost:8081 diff --git a/scripts/wait-for-it.sh b/scripts/wait-for-it.sh index 6bf80c51..83932f1d 100755 --- a/scripts/wait-for-it.sh +++ b/scripts/wait-for-it.sh @@ -137,7 +137,7 @@ if [[ "$WAITFORIT_HOST" == "" || "$WAITFORIT_PORT" == "" ]]; then usage fi -WAITFORIT_TIMEOUT=${WAITFORIT_TIMEOUT:-15} +WAITFORIT_TIMEOUT=${WAITFORIT_TIMEOUT:-30} WAITFORIT_STRICT=${WAITFORIT_STRICT:-0} WAITFORIT_CHILD=${WAITFORIT_CHILD:-0} WAITFORIT_QUIET=${WAITFORIT_QUIET:-0} From dbf6698acd8a8051533fd755379fc3ea9f86b7f8 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sun, 14 Jul 2019 19:40:09 +0200 Subject: [PATCH 43/94] Fix travis.yml --- .travis.yml | 2 +- scripts/wait-for-it.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index b2470799..d5a68380 100644 --- a/.travis.yml +++ b/.travis.yml @@ -9,8 +9,8 @@ before_install: - nohup bash -c "cd kafka && bin/zookeeper-server-start etc/kafka/zookeeper.properties &" - nohup bash -c "cd kafka && bin/kafka-server-start etc/kafka/server.properties &" - nohup bash -c "cd kafka && bin/schema-registry-start etc/schema-registry/schema-registry.properties &" -- scripts/wait-for-it.sh localhost:9092 - scripts/wait-for-it.sh localhost:2181 +- scripts/wait-for-it.sh localhost:9092 - scripts/wait-for-it.sh localhost:8081 install: - pip install coverage coveralls flake8 pipenv diff --git a/scripts/wait-for-it.sh b/scripts/wait-for-it.sh index 83932f1d..6bf80c51 100755 --- a/scripts/wait-for-it.sh +++ b/scripts/wait-for-it.sh @@ -137,7 +137,7 @@ if [[ "$WAITFORIT_HOST" == "" || "$WAITFORIT_PORT" == "" ]]; then usage fi -WAITFORIT_TIMEOUT=${WAITFORIT_TIMEOUT:-30} +WAITFORIT_TIMEOUT=${WAITFORIT_TIMEOUT:-15} WAITFORIT_STRICT=${WAITFORIT_STRICT:-0} WAITFORIT_CHILD=${WAITFORIT_CHILD:-0} WAITFORIT_QUIET=${WAITFORIT_QUIET:-0} From 739dc3aff12bff7a501d5cb8ae8a39941fd520c3 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sun, 14 Jul 2019 19:42:52 +0200 Subject: [PATCH 44/94] Use newer version of confluent package --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index d5a68380..64b6256a 100644 --- a/.travis.yml +++ b/.travis.yml @@ -4,7 +4,7 @@ cache: pip env: - TEST_CMD="pytest tests/ --integration --cov=esque --local" before_install: -- wget http://packages.confluent.io/archive/3.0/confluent-3.0.0-2.11.tar.gz -O kafka.tgz +- wget http://packages.confluent.io/archive/5.2/confluent-5.2.2-2.12.tar.gz -O kafka.tgz - mkdir -p kafka && tar xzf kafka.tgz -C kafka --strip-components 1 - nohup bash -c "cd kafka && bin/zookeeper-server-start etc/kafka/zookeeper.properties &" - nohup bash -c "cd kafka && bin/kafka-server-start etc/kafka/server.properties &" From 849b6f360d0078554abf89d9fb97f849fa9167f2 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sun, 14 Jul 2019 19:44:55 +0200 Subject: [PATCH 45/94] Use higher time out for kafka broker start --- scripts/wait-for-it.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scripts/wait-for-it.sh b/scripts/wait-for-it.sh index 6bf80c51..83932f1d 100755 --- a/scripts/wait-for-it.sh +++ b/scripts/wait-for-it.sh @@ -137,7 +137,7 @@ if [[ "$WAITFORIT_HOST" == "" || "$WAITFORIT_PORT" == "" ]]; then usage fi -WAITFORIT_TIMEOUT=${WAITFORIT_TIMEOUT:-15} +WAITFORIT_TIMEOUT=${WAITFORIT_TIMEOUT:-30} WAITFORIT_STRICT=${WAITFORIT_STRICT:-0} WAITFORIT_CHILD=${WAITFORIT_CHILD:-0} WAITFORIT_QUIET=${WAITFORIT_QUIET:-0} From 16f031ecfb44c3c0cad369b94f734f90eb10f7c6 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sun, 14 Jul 2019 19:54:18 +0200 Subject: [PATCH 46/94] Fix ci --- .travis.yml | 2 -- 1 file changed, 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index 64b6256a..216b0432 100644 --- a/.travis.yml +++ b/.travis.yml @@ -8,10 +8,8 @@ before_install: - mkdir -p kafka && tar xzf kafka.tgz -C kafka --strip-components 1 - nohup bash -c "cd kafka && bin/zookeeper-server-start etc/kafka/zookeeper.properties &" - nohup bash -c "cd kafka && bin/kafka-server-start etc/kafka/server.properties &" -- nohup bash -c "cd kafka && bin/schema-registry-start etc/schema-registry/schema-registry.properties &" - scripts/wait-for-it.sh localhost:2181 - scripts/wait-for-it.sh localhost:9092 -- scripts/wait-for-it.sh localhost:8081 install: - pip install coverage coveralls flake8 pipenv - pipenv install --system --dev --deploy From 9ce200af3dc13e6fe64ac7c083a85bc56df3cb2d Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sun, 14 Jul 2019 19:55:59 +0200 Subject: [PATCH 47/94] Fix ci --- .travis.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.travis.yml b/.travis.yml index 216b0432..8195063d 100644 --- a/.travis.yml +++ b/.travis.yml @@ -8,8 +8,10 @@ before_install: - mkdir -p kafka && tar xzf kafka.tgz -C kafka --strip-components 1 - nohup bash -c "cd kafka && bin/zookeeper-server-start etc/kafka/zookeeper.properties &" - nohup bash -c "cd kafka && bin/kafka-server-start etc/kafka/server.properties &" +- nohup bash -c "cd kafka && bin/schema-registry-start etc/schema-registry/schema-registry.properties &" - scripts/wait-for-it.sh localhost:2181 - scripts/wait-for-it.sh localhost:9092 +- scripts/wait-for-it.sh http://localhost:8081 install: - pip install coverage coveralls flake8 pipenv - pipenv install --system --dev --deploy From 6b7412fb26b831acda95481edaecbae2842a2816 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sun, 14 Jul 2019 20:00:58 +0200 Subject: [PATCH 48/94] Fix ci --- .travis.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.travis.yml b/.travis.yml index 8195063d..66a4e973 100644 --- a/.travis.yml +++ b/.travis.yml @@ -6,11 +6,11 @@ env: before_install: - wget http://packages.confluent.io/archive/5.2/confluent-5.2.2-2.12.tar.gz -O kafka.tgz - mkdir -p kafka && tar xzf kafka.tgz -C kafka --strip-components 1 -- nohup bash -c "cd kafka && bin/zookeeper-server-start etc/kafka/zookeeper.properties &" -- nohup bash -c "cd kafka && bin/kafka-server-start etc/kafka/server.properties &" -- nohup bash -c "cd kafka && bin/schema-registry-start etc/schema-registry/schema-registry.properties &" +- nohup bash -c "cd kafka && bin/zookeeper-server-start etc/kafka/zookeeper.properties" - scripts/wait-for-it.sh localhost:2181 +- nohup bash -c "cd kafka && bin/kafka-server-start etc/kafka/server.properties" - scripts/wait-for-it.sh localhost:9092 +- nohup bash -c "cd kafka && bin/schema-registry-start etc/schema-registry/schema-registry.properties" - scripts/wait-for-it.sh http://localhost:8081 install: - pip install coverage coveralls flake8 pipenv From 5daee5a774ef4a5857eadda8d4ef9a6d234d8d95 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sun, 14 Jul 2019 20:03:39 +0200 Subject: [PATCH 49/94] Fix ci --- .travis.yml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/.travis.yml b/.travis.yml index 66a4e973..458b1cc8 100644 --- a/.travis.yml +++ b/.travis.yml @@ -6,12 +6,12 @@ env: before_install: - wget http://packages.confluent.io/archive/5.2/confluent-5.2.2-2.12.tar.gz -O kafka.tgz - mkdir -p kafka && tar xzf kafka.tgz -C kafka --strip-components 1 -- nohup bash -c "cd kafka && bin/zookeeper-server-start etc/kafka/zookeeper.properties" +- nohup bash -c "cd kafka && bin/zookeeper-server-start etc/kafka/zookeeper.properties &" +- nohup bash -c "cd kafka && bin/kafka-server-start etc/kafka/server.properties &" - scripts/wait-for-it.sh localhost:2181 -- nohup bash -c "cd kafka && bin/kafka-server-start etc/kafka/server.properties" - scripts/wait-for-it.sh localhost:9092 -- nohup bash -c "cd kafka && bin/schema-registry-start etc/schema-registry/schema-registry.properties" -- scripts/wait-for-it.sh http://localhost:8081 +- nohup bash -c "cd kafka && bin/schema-registry-start etc/schema-registry/schema-registry.properties &" +- scripts/wait-for-it.sh localhost:8081 install: - pip install coverage coveralls flake8 pipenv - pipenv install --system --dev --deploy From 523701f8062af27fb5b9495fdb50919bcac6b43d Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sun, 14 Jul 2019 20:09:27 +0200 Subject: [PATCH 50/94] Update Pipfiles --- Pipfile | 1 - Pipfile.lock | 6 +----- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/Pipfile b/Pipfile index 7ee085e2..aa42257b 100644 --- a/Pipfile +++ b/Pipfile @@ -5,7 +5,6 @@ verify_ssl = true [dev-packages] kafkaesque = {path = ".",editable = true,extras = ["test", "dev"]} -esque = {editable = true,path = "."} [packages] diff --git a/Pipfile.lock b/Pipfile.lock index f7cbe882..5d06845f 100644 --- a/Pipfile.lock +++ b/Pipfile.lock @@ -1,7 +1,7 @@ { "_meta": { "hash": { - "sha256": "70ffd0673a6261321aa7425d25088d686c8c8bd414a857a68321b13a7ae7cd33" + "sha256": "39bdc1457e421be937c9db60e2b5bf93f09a746d17a48e1630febae254b35cb3" }, "pipfile-spec": 6, "requires": { @@ -142,10 +142,6 @@ ], "version": "==0.3" }, - "esque": { - "editable": true, - "path": "." - }, "fastavro": { "hashes": [ "sha256:02e5e2a1a6cd8b94703071a3afa1968ddb8f0dadc889ef56160773ac6029e066", From b620dd068dc54c8e757933972c1b13a04635bc38 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sun, 14 Jul 2019 20:10:58 +0200 Subject: [PATCH 51/94] Fix docker-compose.test.yml --- docker-compose.test.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker-compose.test.yml b/docker-compose.test.yml index 0bd56a84..b7e4631f 100644 --- a/docker-compose.test.yml +++ b/docker-compose.test.yml @@ -11,4 +11,4 @@ services: command: > -c "(until (kafkacat -b kafka:9093 -X debug=all -L); do sleep 5s; done) \ - && pytest tests/integration/test_clients.py --integration" \ No newline at end of file + && pytest tests/ --integration" \ No newline at end of file From 85052e842aeb48de6bc91b2a7769d47426dbf344 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sun, 14 Jul 2019 21:05:35 +0200 Subject: [PATCH 52/94] Add tests and update ci config --- .travis.yml | 2 +- docker-compose.test.yml | 1 + docker-compose.yml | 3 +-- scripts/wait-for-it.sh | 2 +- tests/integration/test_clients.py | 28 ++++++++++++++++++++++++++-- 5 files changed, 30 insertions(+), 6 deletions(-) diff --git a/.travis.yml b/.travis.yml index 458b1cc8..8e237f77 100644 --- a/.travis.yml +++ b/.travis.yml @@ -4,7 +4,7 @@ cache: pip env: - TEST_CMD="pytest tests/ --integration --cov=esque --local" before_install: -- wget http://packages.confluent.io/archive/5.2/confluent-5.2.2-2.12.tar.gz -O kafka.tgz +- wget https://packages.confluent.io/archive/5.2/confluent-5.2.2-2.12.tar.gz -O kafka.tgz - mkdir -p kafka && tar xzf kafka.tgz -C kafka --strip-components 1 - nohup bash -c "cd kafka && bin/zookeeper-server-start etc/kafka/zookeeper.properties &" - nohup bash -c "cd kafka && bin/kafka-server-start etc/kafka/server.properties &" diff --git a/docker-compose.test.yml b/docker-compose.test.yml index b7e4631f..8bd6ac22 100644 --- a/docker-compose.test.yml +++ b/docker-compose.test.yml @@ -8,6 +8,7 @@ services: ESQUE_TEST_ENV: "ci" depends_on: - kafka + - schema_registry command: > -c "(until (kafkacat -b kafka:9093 -X debug=all -L); do sleep 5s; done) \ diff --git a/docker-compose.yml b/docker-compose.yml index f9bc714b..4d92cc6e 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -45,8 +45,7 @@ services: - SCHEMA_REGISTRY_HOST_NAME=schema-registry - SCHEMA_REGISTRY_LISTENERS=http://0.0.0.0:8081 - SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS=PLAINTEXT://kafka:9093 - links: - - zookeeper + depends_on: - kafka ports: - 8081:8081 \ No newline at end of file diff --git a/scripts/wait-for-it.sh b/scripts/wait-for-it.sh index 83932f1d..e7907968 100755 --- a/scripts/wait-for-it.sh +++ b/scripts/wait-for-it.sh @@ -137,7 +137,7 @@ if [[ "$WAITFORIT_HOST" == "" || "$WAITFORIT_PORT" == "" ]]; then usage fi -WAITFORIT_TIMEOUT=${WAITFORIT_TIMEOUT:-30} +WAITFORIT_TIMEOUT=${WAITFORIT_TIMEOUT:-40} WAITFORIT_STRICT=${WAITFORIT_STRICT:-0} WAITFORIT_CHILD=${WAITFORIT_CHILD:-0} WAITFORIT_QUIET=${WAITFORIT_QUIET:-0} diff --git a/tests/integration/test_clients.py b/tests/integration/test_clients.py index 78556bc9..6fa90a26 100644 --- a/tests/integration/test_clients.py +++ b/tests/integration/test_clients.py @@ -3,13 +3,13 @@ import pytest from esque.avromessage import AvroFileReader -from esque.clients import FileConsumer, AvroFileConsumer +from esque.clients import FileConsumer, AvroFileConsumer, FileProducer, AvroFileProducer from esque.message import PlainTextFileReader from esque.topic import Topic @pytest.mark.integration -def test_plaint_text_consume_to_file(consumer_group, filled_topic: Topic, working_dir: pathlib.Path): +def test_plain_text_consume_to_file(consumer_group, filled_topic: Topic, working_dir: pathlib.Path): file_consumer = FileConsumer(consumer_group, filled_topic.name, working_dir, False) number_of_consumer_messages = file_consumer.consume(10) @@ -36,3 +36,27 @@ def test_avro_consume_to_file(consumer_group, filled_avro_topic: Topic, working_ assert number_of_consumer_messages == 10 assert len(messages) == 10 + + +@pytest.mark.integration +def test_plain_text_consume_and_produce(consumer_group, filled_topic: Topic, topic: str, working_dir: pathlib.Path): + file_consumer = FileConsumer(consumer_group, filled_topic.name, working_dir, False) + number_of_consumer_messages = file_consumer.consume(10) + + producer = FileProducer(working_dir) + number_of_produced_messages = producer.produce(topic) + + assert number_of_consumer_messages == 10 + assert number_of_produced_messages == 10 + + +@pytest.mark.integration +def test_avro_consume_and_produce(consumer_group, filled_avro_topic: Topic, topic: str, working_dir: pathlib.Path): + file_consumer = AvroFileConsumer(consumer_group, filled_avro_topic.name, working_dir, False) + number_of_consumer_messages = file_consumer.consume(10) + + producer = AvroFileProducer(working_dir) + number_of_produced_messages = producer.produce(topic) + + assert number_of_consumer_messages == 10 + assert number_of_produced_messages == 10 From fa310b9f921ce5dd762c407a2b5b573265160611 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sun, 14 Jul 2019 21:13:50 +0200 Subject: [PATCH 53/94] Try alternative ci config --- .travis.yml | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/.travis.yml b/.travis.yml index 8e237f77..f5ce431f 100644 --- a/.travis.yml +++ b/.travis.yml @@ -4,12 +4,13 @@ cache: pip env: - TEST_CMD="pytest tests/ --integration --cov=esque --local" before_install: -- wget https://packages.confluent.io/archive/5.2/confluent-5.2.2-2.12.tar.gz -O kafka.tgz +- wget https://mirror.netcologne.de/apache.org/kafka/2.2.0/kafka_2.12-2.2.0.tgz -O kafka.tgz - mkdir -p kafka && tar xzf kafka.tgz -C kafka --strip-components 1 -- nohup bash -c "cd kafka && bin/zookeeper-server-start etc/kafka/zookeeper.properties &" -- nohup bash -c "cd kafka && bin/kafka-server-start etc/kafka/server.properties &" -- scripts/wait-for-it.sh localhost:2181 +- nohup bash -c "cd kafka && bin/zookeeper-server-start.sh config/zookeeper.properties &" +- nohup bash -c "cd kafka && bin/kafka-server-start.sh config/server.properties &" +- wget https://packages.confluent.io/archive/5.2/confluent-5.2.2-2.12.tar.gz -O kafka.tgz - scripts/wait-for-it.sh localhost:9092 +- scripts/wait-for-it.sh localhost:2181 - nohup bash -c "cd kafka && bin/schema-registry-start etc/schema-registry/schema-registry.properties &" - scripts/wait-for-it.sh localhost:8081 install: From b9f5931ba897fd11fbc839d612c73feea5023356 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sun, 14 Jul 2019 21:16:17 +0200 Subject: [PATCH 54/94] Revert "Try alternative ci config" This reverts commit fa310b9f921ce5dd762c407a2b5b573265160611. --- .travis.yml | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/.travis.yml b/.travis.yml index f5ce431f..8e237f77 100644 --- a/.travis.yml +++ b/.travis.yml @@ -4,13 +4,12 @@ cache: pip env: - TEST_CMD="pytest tests/ --integration --cov=esque --local" before_install: -- wget https://mirror.netcologne.de/apache.org/kafka/2.2.0/kafka_2.12-2.2.0.tgz -O kafka.tgz -- mkdir -p kafka && tar xzf kafka.tgz -C kafka --strip-components 1 -- nohup bash -c "cd kafka && bin/zookeeper-server-start.sh config/zookeeper.properties &" -- nohup bash -c "cd kafka && bin/kafka-server-start.sh config/server.properties &" - wget https://packages.confluent.io/archive/5.2/confluent-5.2.2-2.12.tar.gz -O kafka.tgz -- scripts/wait-for-it.sh localhost:9092 +- mkdir -p kafka && tar xzf kafka.tgz -C kafka --strip-components 1 +- nohup bash -c "cd kafka && bin/zookeeper-server-start etc/kafka/zookeeper.properties &" +- nohup bash -c "cd kafka && bin/kafka-server-start etc/kafka/server.properties &" - scripts/wait-for-it.sh localhost:2181 +- scripts/wait-for-it.sh localhost:9092 - nohup bash -c "cd kafka && bin/schema-registry-start etc/schema-registry/schema-registry.properties &" - scripts/wait-for-it.sh localhost:8081 install: From 7d01ea84a2e798ac08e87903df5a0894f1df8f8f Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Sun, 14 Jul 2019 21:22:53 +0200 Subject: [PATCH 55/94] Try alternative ci config --- .travis.yml | 2 +- ci_config/kafka/server.properties | 170 ++++++++++++++++++++++++++++++ 2 files changed, 171 insertions(+), 1 deletion(-) create mode 100644 ci_config/kafka/server.properties diff --git a/.travis.yml b/.travis.yml index 8e237f77..df71d0cd 100644 --- a/.travis.yml +++ b/.travis.yml @@ -7,7 +7,7 @@ before_install: - wget https://packages.confluent.io/archive/5.2/confluent-5.2.2-2.12.tar.gz -O kafka.tgz - mkdir -p kafka && tar xzf kafka.tgz -C kafka --strip-components 1 - nohup bash -c "cd kafka && bin/zookeeper-server-start etc/kafka/zookeeper.properties &" -- nohup bash -c "cd kafka && bin/kafka-server-start etc/kafka/server.properties &" +- nohup bash -c "cd kafka && bin/kafka-server-start ../ci_config/kafka/server.properties &" - scripts/wait-for-it.sh localhost:2181 - scripts/wait-for-it.sh localhost:9092 - nohup bash -c "cd kafka && bin/schema-registry-start etc/schema-registry/schema-registry.properties &" diff --git a/ci_config/kafka/server.properties b/ci_config/kafka/server.properties new file mode 100644 index 00000000..943eecd8 --- /dev/null +++ b/ci_config/kafka/server.properties @@ -0,0 +1,170 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# see kafka.server.KafkaConfig for additional details and defaults + +############################# Server Basics ############################# + +# The id of the broker. This must be set to a unique integer for each broker. +broker.id=0 + +############################# Socket Server Settings ############################# + +# The address the socket server listens on. It will get the value returned from +# java.net.InetAddress.getCanonicalHostName() if not configured. +# FORMAT: +# listeners = listener_name://host_name:port +# EXAMPLE: +# listeners = PLAINTEXT://your.host.name:9092 +#listeners=PLAINTEXT://:9092 + +# Hostname and port the broker will advertise to producers and consumers. If not set, +# it uses the value for "listeners" if configured. Otherwise, it will use the value +# returned from java.net.InetAddress.getCanonicalHostName(). +#advertised.listeners=PLAINTEXT://your.host.name:9092 + +# Maps listener names to security protocols, the default is for them to be the same. See the config documentation for more details +#listener.security.protocol.map=PLAINTEXT:PLAINTEXT,SSL:SSL,SASL_PLAINTEXT:SASL_PLAINTEXT,SASL_SSL:SASL_SSL + +# The number of threads that the server uses for receiving requests from the network and sending responses to the network +num.network.threads=3 + +# The number of threads that the server uses for processing requests, which may include disk I/O +num.io.threads=8 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=102400 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=102400 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# A comma separated list of directories under which to store log files +log.dirs=/tmp/kafka-logs + +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. +num.partitions=1 + +# The number of threads per data directory to be used for log recovery at startup and flushing at shutdown. +# This value is recommended to be increased for installations with data dirs located in RAID array. +num.recovery.threads.per.data.dir=1 + +############################# Internal Topic Settings ############################# +# The replication factor for the group metadata internal topics "__consumer_offsets" and "__transaction_state" +# For anything other than development testing, a value greater than 1 is recommended for to ensure availability such as 3. +offsets.topic.replication.factor=1 +transaction.state.log.replication.factor=1 +transaction.state.log.min.isr=1 + +############################# Log Flush Policy ############################# + +# Messages are immediately written to the filesystem but by default we only fsync() to sync +# the OS cache lazily. The following configurations control the flush of data to disk. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data may be lost if you are not using replication. +# 2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as there will be a lot of data to flush. +# 3. Throughput: The flush is generally the most expensive operation, and a small flush interval may lead to excessive seeks. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +#log.flush.interval.messages=10000 + +# The maximum amount of time a message can sit in a log before we force a flush +#log.flush.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion due to age +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log unless the remaining +# segments drop below log.retention.bytes. Functions independently of log.retention.hours. +#log.retention.bytes=1073741824 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +log.segment.bytes=1073741824 + +# The interval at which log segments are checked to see if they can be deleted according +# to the retention policies +log.retention.check.interval.ms=300000 + +############################# Zookeeper ############################# + +# Zookeeper connection string (see zookeeper docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +# You can also append an optional chroot string to the urls to specify the +# root directory for all kafka znodes. +zookeeper.connect=localhost:2181 + +# Timeout in ms for connecting to zookeeper +zookeeper.connection.timeout.ms=6000 + +##################### Confluent Metrics Reporter ####################### +# Confluent Control Center and Confluent Auto Data Balancer integration +# +# Uncomment the following lines to publish monitoring data for +# Confluent Control Center and Confluent Auto Data Balancer +# If you are using a dedicated metrics cluster, also adjust the settings +# to point to your metrics kakfa cluster. +#metric.reporters=io.confluent.metrics.reporter.ConfluentMetricsReporter +#confluent.metrics.reporter.bootstrap.servers=localhost:9092 +# +# Uncomment the following line if the metrics cluster has a single broker +#confluent.metrics.reporter.topic.replicas=1 + +##################### Confluent Proactive Support ###################### +# If set to true, and confluent-support-metrics package is installed +# then the feature to collect and report support metrics +# ("Metrics") is enabled. If set to false, the feature is disabled. +# +confluent.support.metrics.enable=false + + +# The customer ID under which support metrics will be collected and +# reported. +# +# When the customer ID is set to "anonymous" (the default), then only a +# reduced set of metrics is being collected and reported. +# +# Confluent customers +# ------------------- +# If you are a Confluent customer, then you should replace the default +# value with your actual Confluent customer ID. Doing so will ensure +# that additional support metrics will be collected and reported. +# +confluent.support.customer.id=anonymous + +############################# Group Coordinator Settings ############################# + +# The following configuration specifies the time, in milliseconds, that the GroupCoordinator will delay the initial consumer rebalance. +# The rebalance will be further delayed by the value of group.initial.rebalance.delay.ms as new members join the group, up to a maximum of max.poll.interval.ms. +# The default value for this is 3 seconds. +# We override this to 0 here as it makes for a better out-of-the-box experience for development and testing. +# However, in production environments the default value of 3 seconds is more suitable as this will help to avoid unnecessary, and potentially expensive, rebalances during application startup. +group.initial.rebalance.delay.ms=0 \ No newline at end of file From 8b9de1866daa57000d55efea0ecc7de4e3a14d26 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 15 Jul 2019 08:36:47 +0200 Subject: [PATCH 56/94] Increase waiting time for ci environment startup --- scripts/wait-for-it.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scripts/wait-for-it.sh b/scripts/wait-for-it.sh index e7907968..fcd761e8 100755 --- a/scripts/wait-for-it.sh +++ b/scripts/wait-for-it.sh @@ -137,7 +137,7 @@ if [[ "$WAITFORIT_HOST" == "" || "$WAITFORIT_PORT" == "" ]]; then usage fi -WAITFORIT_TIMEOUT=${WAITFORIT_TIMEOUT:-40} +WAITFORIT_TIMEOUT=${WAITFORIT_TIMEOUT:-60} WAITFORIT_STRICT=${WAITFORIT_STRICT:-0} WAITFORIT_CHILD=${WAITFORIT_CHILD:-0} WAITFORIT_QUIET=${WAITFORIT_QUIET:-0} From f146ee7edd725ccb9aed04337de32cae1a103326 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 15 Jul 2019 09:01:21 +0200 Subject: [PATCH 57/94] Update schema registry config for ci --- .travis.yml | 2 +- .../schema_registry.properties | 43 +++++++++++++++++++ 2 files changed, 44 insertions(+), 1 deletion(-) create mode 100644 ci_config/schema_registry/schema_registry.properties diff --git a/.travis.yml b/.travis.yml index df71d0cd..d238cac0 100644 --- a/.travis.yml +++ b/.travis.yml @@ -10,7 +10,7 @@ before_install: - nohup bash -c "cd kafka && bin/kafka-server-start ../ci_config/kafka/server.properties &" - scripts/wait-for-it.sh localhost:2181 - scripts/wait-for-it.sh localhost:9092 -- nohup bash -c "cd kafka && bin/schema-registry-start etc/schema-registry/schema-registry.properties &" +- nohup bash -c "cd kafka && bin/schema-registry-start ../ci_config/schema_registry/schema_registry.properties &" - scripts/wait-for-it.sh localhost:8081 install: - pip install coverage coveralls flake8 pipenv diff --git a/ci_config/schema_registry/schema_registry.properties b/ci_config/schema_registry/schema_registry.properties new file mode 100644 index 00000000..26c80abd --- /dev/null +++ b/ci_config/schema_registry/schema_registry.properties @@ -0,0 +1,43 @@ +# +# Copyright 2018 Confluent Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# The address the socket server listens on. +# FORMAT: +# listeners = listener_name://host_name:port +# EXAMPLE: +# listeners = PLAINTEXT://your.host.name:9092 +listeners=http://0.0.0.0:8081 + +# Zookeeper connection string for the Zookeeper cluster used by your Kafka cluster +# (see zookeeper docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +kafkastore.connection.url=localhost:2181 + +# Alternatively, Schema Registry can now operate without Zookeeper, handling all coordination via +# Kafka brokers. Use this setting to specify the bootstrap servers for your Kafka cluster and it +# will be used both for selecting the master schema registry instance and for storing the data for +# registered schemas. +# (Note that you cannot mix the two modes; use this mode only on new deployments or by shutting down +# all instances, switching to the new configuration, and then starting the schema registry +# instances again.) +kafkastore.bootstrap.servers=PLAINTEXT://localhost:9092 + +# The name of the topic to store schemas in +kafkastore.topic=_schemas + +# If true, API requests that fail will include extra debugging information, including stack traces +debug=false From 782c7e1674336025f4ca3de5ecd6052387796258 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 15 Jul 2019 09:24:50 +0200 Subject: [PATCH 58/94] Revert "Update schema registry config for ci" This reverts commit f146ee7edd725ccb9aed04337de32cae1a103326. --- .travis.yml | 2 +- .../schema_registry.properties | 43 ------------------- 2 files changed, 1 insertion(+), 44 deletions(-) delete mode 100644 ci_config/schema_registry/schema_registry.properties diff --git a/.travis.yml b/.travis.yml index d238cac0..df71d0cd 100644 --- a/.travis.yml +++ b/.travis.yml @@ -10,7 +10,7 @@ before_install: - nohup bash -c "cd kafka && bin/kafka-server-start ../ci_config/kafka/server.properties &" - scripts/wait-for-it.sh localhost:2181 - scripts/wait-for-it.sh localhost:9092 -- nohup bash -c "cd kafka && bin/schema-registry-start ../ci_config/schema_registry/schema_registry.properties &" +- nohup bash -c "cd kafka && bin/schema-registry-start etc/schema-registry/schema-registry.properties &" - scripts/wait-for-it.sh localhost:8081 install: - pip install coverage coveralls flake8 pipenv diff --git a/ci_config/schema_registry/schema_registry.properties b/ci_config/schema_registry/schema_registry.properties deleted file mode 100644 index 26c80abd..00000000 --- a/ci_config/schema_registry/schema_registry.properties +++ /dev/null @@ -1,43 +0,0 @@ -# -# Copyright 2018 Confluent Inc. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# The address the socket server listens on. -# FORMAT: -# listeners = listener_name://host_name:port -# EXAMPLE: -# listeners = PLAINTEXT://your.host.name:9092 -listeners=http://0.0.0.0:8081 - -# Zookeeper connection string for the Zookeeper cluster used by your Kafka cluster -# (see zookeeper docs for details). -# This is a comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". -kafkastore.connection.url=localhost:2181 - -# Alternatively, Schema Registry can now operate without Zookeeper, handling all coordination via -# Kafka brokers. Use this setting to specify the bootstrap servers for your Kafka cluster and it -# will be used both for selecting the master schema registry instance and for storing the data for -# registered schemas. -# (Note that you cannot mix the two modes; use this mode only on new deployments or by shutting down -# all instances, switching to the new configuration, and then starting the schema registry -# instances again.) -kafkastore.bootstrap.servers=PLAINTEXT://localhost:9092 - -# The name of the topic to store schemas in -kafkastore.topic=_schemas - -# If true, API requests that fail will include extra debugging information, including stack traces -debug=false From a5a3c077e14d8102993fad3d94c64ebfe79c547d Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 15 Jul 2019 11:54:34 +0200 Subject: [PATCH 59/94] Update tests to check content of messages --- esque/avromessage.py | 2 +- esque/clients.py | 5 ++- tests/conftest.py | 1 - tests/integration/test_clients.py | 68 ++++++++++++++++++++++++++----- 4 files changed, 61 insertions(+), 15 deletions(-) diff --git a/esque/avromessage.py b/esque/avromessage.py index fa5533be..8a0db53e 100644 --- a/esque/avromessage.py +++ b/esque/avromessage.py @@ -95,7 +95,7 @@ def read_from_file(self) -> Iterable[KafkaMessage]: key_schema = load_schema((schema_directory / "key_schema.avsc").read_text(encoding="utf-8")) value_schema = load_schema((schema_directory / "value_schema.avsc").read_text(encoding="utf-8")) - yield KafkaMessage(record["key"], record["value"], key_schema, value_schema) + yield KafkaMessage(json.dumps(record["key"]), json.dumps(record["value"]), key_schema, value_schema) def extract_schema_id(message: bytes) -> int: diff --git a/esque/clients.py b/esque/clients.py index be65e5e9..e52dd4c0 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -1,3 +1,4 @@ +import json import pathlib from typing import Optional, Tuple @@ -172,8 +173,8 @@ def __init__(self, working_dir: pathlib.Path): def produce_message(self, topic_name: str, message: KafkaMessage): self._producer.produce( topic=topic_name, - key=message.key, - value=message.value, + key=json.loads(message.key), + value=json.loads(message.value), key_schema=message.key_schema, value_schema=message.value_schema, ) diff --git a/tests/conftest.py b/tests/conftest.py index c3911494..e427fb7c 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -1,4 +1,3 @@ -import json import random from concurrent.futures import Future from pathlib import Path diff --git a/tests/integration/test_clients.py b/tests/integration/test_clients.py index 6fa90a26..92e5cd29 100644 --- a/tests/integration/test_clients.py +++ b/tests/integration/test_clients.py @@ -1,41 +1,59 @@ +import json import pathlib +import random +from typing import Iterable +from string import ascii_letters import pytest +from confluent_kafka.cimpl import Producer as ConfluenceProducer from esque.avromessage import AvroFileReader from esque.clients import FileConsumer, AvroFileConsumer, FileProducer, AvroFileProducer -from esque.message import PlainTextFileReader +from esque.message import PlainTextFileReader, KafkaMessage from esque.topic import Topic +from confluent_kafka.avro import loads as load_schema, AvroProducer @pytest.mark.integration -def test_plain_text_consume_to_file(consumer_group, filled_topic: Topic, working_dir: pathlib.Path): - file_consumer = FileConsumer(consumer_group, filled_topic.name, working_dir, False) +def test_plain_text_consume_to_file(consumer_group, producer: ConfluenceProducer, topic: str, working_dir: pathlib.Path): + produced_messages = produce_test_messages(producer, topic) + file_consumer = FileConsumer(consumer_group, topic, working_dir, False) number_of_consumer_messages = file_consumer.consume(10) - messages = [] + consumed_messages = [] file_reader = PlainTextFileReader(working_dir) with file_reader: for message in file_reader.read_from_file(): - messages.append({"key": message.key, "value": message.value}) + consumed_messages.append(message) assert number_of_consumer_messages == 10 - assert len(messages) == 10 + assert len(consumed_messages) == 10 + assert all([ + produced_message.key == consumed_message.key and + produced_message.value == consumed_message.value + for produced_message, consumed_message in zip(produced_messages, consumed_messages) + ]) @pytest.mark.integration -def test_avro_consume_to_file(consumer_group, filled_avro_topic: Topic, working_dir: pathlib.Path): - file_consumer = AvroFileConsumer(consumer_group, filled_avro_topic.name, working_dir, False) +def test_avro_consume_to_file(consumer_group, avro_producer: AvroProducer, topic: str, working_dir: pathlib.Path): + produced_messages = produce_test_messages_with_avro(avro_producer, topic) + file_consumer = AvroFileConsumer(consumer_group, topic, working_dir, False) number_of_consumer_messages = file_consumer.consume(10) - messages = [] + consumed_messages = [] file_reader = AvroFileReader(working_dir) with file_reader: for message in file_reader.read_from_file(): - messages.append({"key": message.key, "value": message.value}) + consumed_messages.append(message) assert number_of_consumer_messages == 10 - assert len(messages) == 10 + assert len(consumed_messages) == 10 + assert all([ + produced_message.key == consumed_message.key and + produced_message.value == consumed_message.value + for produced_message, consumed_message in zip(produced_messages, consumed_messages) + ]) @pytest.mark.integration @@ -60,3 +78,31 @@ def test_avro_consume_and_produce(consumer_group, filled_avro_topic: Topic, topi assert number_of_consumer_messages == 10 assert number_of_produced_messages == 10 + + +def produce_test_messages(producer: ConfluenceProducer, topic: str) -> Iterable[KafkaMessage]: + messages = [] + for i in range(10): + random_value = "".join(random.choices(ascii_letters, k=5)) + message = KafkaMessage(str(i), random_value) + messages.append(message) + producer.produce(topic=topic, key=message.key, value=message.value) + producer.flush() + return messages + + +def produce_test_messages_with_avro(avro_producer: AvroProducer, topic: str) -> Iterable[KafkaMessage]: + with open("tests/test_samples/key_schema", "r") as file: + key_schema = load_schema(file.read()) + with open("tests/test_samples/value_schema", "r") as file: + value_schema = load_schema(file.read()) + messages = [] + for i in range(10): + key = {"id": str(i)} + value = {"first": "Firstname", "last": "Lastname"} + messages.append(KafkaMessage(json.dumps(key), json.dumps(value), key_schema, value_schema)) + avro_producer.produce( + topic=topic, key=key, value=value, key_schema=key_schema, value_schema=value_schema + ) + avro_producer.flush() + return messages From 1d7912f9d822b4b70f67abd74c37c74ac499409a Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 15 Jul 2019 12:34:07 +0200 Subject: [PATCH 60/94] Merge methods --- esque/schemaregistry.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/esque/schemaregistry.py b/esque/schemaregistry.py index 825a2278..4902b519 100644 --- a/esque/schemaregistry.py +++ b/esque/schemaregistry.py @@ -15,10 +15,7 @@ def __init__(self, schema_registry_uri: str): @lru_cache(maxsize=100) def get_schema_from_id(self, schema_id: int) -> SchemaPair: - return self.get_schema_from_server(schema_id, self.schema_registry_uri) - - def get_schema_from_server(self, schema_id: int, server_url: str) -> SchemaPair: - url = "{server_url}/schemas/ids/{schema_id}" + url = f"{self.schema_registry_uri}/schemas/ids/{schema_id}" response = requests.get(url.format(**locals())) response.raise_for_status() schema: Dict = json.loads(response.json()["schema"]) From ad54d1f1cd98e2d8a32a3df04a21ea527d05af7d Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 15 Jul 2019 12:39:37 +0200 Subject: [PATCH 61/94] Update extract_schema_id --- esque/avromessage.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/esque/avromessage.py b/esque/avromessage.py index 8a0db53e..e5fc51c9 100644 --- a/esque/avromessage.py +++ b/esque/avromessage.py @@ -99,5 +99,6 @@ def read_from_file(self) -> Iterable[KafkaMessage]: def extract_schema_id(message: bytes) -> int: - _, schema_id = struct.unpack(">bI", message[:5]) + magic_byte, schema_id = struct.unpack(">bI", message[:5]) + assert magic_byte == 0, f"Wrong magic byte ({magic_byte}), no AVRO message." return schema_id From deff96cc304dad27ed09cdf4e9a279323fdccc99 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 15 Jul 2019 12:45:05 +0200 Subject: [PATCH 62/94] Use named tuples --- esque/avromessage.py | 13 ++++++------- esque/message.py | 20 +++++++++----------- 2 files changed, 15 insertions(+), 18 deletions(-) diff --git a/esque/avromessage.py b/esque/avromessage.py index e5fc51c9..a4b36194 100644 --- a/esque/avromessage.py +++ b/esque/avromessage.py @@ -3,7 +3,7 @@ import pickle import struct from io import BytesIO -from typing import Optional, Tuple, Dict, Iterable +from typing import Optional, Tuple, Dict, Iterable, NamedTuple import itertools as it import fastavro @@ -14,12 +14,11 @@ from esque.schemaregistry import SchemaRegistryClient -class DecodedAvroMessage: - def __init__(self, key: Optional[Dict], value: Optional[Dict], key_schema_id: int, value_schema_id: int): - self.key = key - self.value = value - self.key_schema_id = key_schema_id - self.value_schema_id = value_schema_id +class DecodedAvroMessage(NamedTuple): + key: Optional[Dict] + value: Optional[Dict] + key_schema_id: int + value_schema_id: int class AvroFileWriter(FileWriter): diff --git a/esque/message.py b/esque/message.py index c0a91d97..918d6c8d 100644 --- a/esque/message.py +++ b/esque/message.py @@ -1,22 +1,20 @@ import json import pathlib -from typing import Iterable +from typing import Iterable, NamedTuple from confluent_kafka.cimpl import Message -class DecodedMessage: - def __init__(self, key: str, value: str): - self.key = key - self.value = value +class DecodedMessage(NamedTuple): + key: str + value: str -class KafkaMessage: - def __init__(self, key: str, value: str, key_schema=None, value_schema=None): - self.key = key - self.value = value - self.key_schema = key_schema - self.value_schema = value_schema +class KafkaMessage(NamedTuple): + key: str + value: str + key_schema: str = None + value_schema: str = None class IOHandler: From 0653c857e0b8e09cae24a91cad25621689f20dc4 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 15 Jul 2019 12:50:28 +0200 Subject: [PATCH 63/94] Content of DecodedAvroMessage could be anything --- esque/avromessage.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/esque/avromessage.py b/esque/avromessage.py index a4b36194..0a0d3b78 100644 --- a/esque/avromessage.py +++ b/esque/avromessage.py @@ -3,7 +3,7 @@ import pickle import struct from io import BytesIO -from typing import Optional, Tuple, Dict, Iterable, NamedTuple +from typing import Optional, Tuple, Dict, Iterable, NamedTuple, Any import itertools as it import fastavro @@ -15,8 +15,8 @@ class DecodedAvroMessage(NamedTuple): - key: Optional[Dict] - value: Optional[Dict] + key: Any + value: Any key_schema_id: int value_schema_id: int From c02b3f2cde9a8aa45c07011390952a6451473280 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 15 Jul 2019 12:50:58 +0200 Subject: [PATCH 64/94] Update travis ci config --- .travis.yml | 19 ++-- ci_config/kafka/server.properties | 170 ------------------------------ 2 files changed, 11 insertions(+), 178 deletions(-) delete mode 100644 ci_config/kafka/server.properties diff --git a/.travis.yml b/.travis.yml index df71d0cd..1be6977d 100644 --- a/.travis.yml +++ b/.travis.yml @@ -3,15 +3,18 @@ language: python cache: pip env: - TEST_CMD="pytest tests/ --integration --cov=esque --local" +- BROKER_URL="localhost:9092" +- ZOOKEEPER_URL="localhost:2181" +- SCHEMA_REGISTRY_URL="localhost:8081" before_install: -- wget https://packages.confluent.io/archive/5.2/confluent-5.2.2-2.12.tar.gz -O kafka.tgz -- mkdir -p kafka && tar xzf kafka.tgz -C kafka --strip-components 1 -- nohup bash -c "cd kafka && bin/zookeeper-server-start etc/kafka/zookeeper.properties &" -- nohup bash -c "cd kafka && bin/kafka-server-start ../ci_config/kafka/server.properties &" -- scripts/wait-for-it.sh localhost:2181 -- scripts/wait-for-it.sh localhost:9092 -- nohup bash -c "cd kafka && bin/schema-registry-start etc/schema-registry/schema-registry.properties &" -- scripts/wait-for-it.sh localhost:8081 +- wget http://packages.confluent.io/archive/5.2/confluent-community-5.2.1-2.12.tar.gz -O confluent-community.tgz +- mkdir -p confluent-community && tar xzf confluent-community.tgz -C confluent-community --strip-components 1 +- confluent-community/bin/zookeeper-server-start -daemon confluent-community/etc/kafka/zookeeper.properties +- bash -c "scripts/wait-for-it.sh ${ZOOKEEPER_URL} -t 60" +- confluent-community/bin/kafka-server-start -daemon confluent-community/etc/kafka/server.properties +- bash -c "scripts/wait-for-it.sh ${BROKER_URL} -t 60" +- confluent-community/bin/schema-registry-start -daemon confluent-community/etc/schema-registry/schema-registry.properties +- bash -c "scripts/wait-for-it.sh ${SCHEMA_REGISTRY_URL} -t 60" install: - pip install coverage coveralls flake8 pipenv - pipenv install --system --dev --deploy diff --git a/ci_config/kafka/server.properties b/ci_config/kafka/server.properties deleted file mode 100644 index 943eecd8..00000000 --- a/ci_config/kafka/server.properties +++ /dev/null @@ -1,170 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -# see kafka.server.KafkaConfig for additional details and defaults - -############################# Server Basics ############################# - -# The id of the broker. This must be set to a unique integer for each broker. -broker.id=0 - -############################# Socket Server Settings ############################# - -# The address the socket server listens on. It will get the value returned from -# java.net.InetAddress.getCanonicalHostName() if not configured. -# FORMAT: -# listeners = listener_name://host_name:port -# EXAMPLE: -# listeners = PLAINTEXT://your.host.name:9092 -#listeners=PLAINTEXT://:9092 - -# Hostname and port the broker will advertise to producers and consumers. If not set, -# it uses the value for "listeners" if configured. Otherwise, it will use the value -# returned from java.net.InetAddress.getCanonicalHostName(). -#advertised.listeners=PLAINTEXT://your.host.name:9092 - -# Maps listener names to security protocols, the default is for them to be the same. See the config documentation for more details -#listener.security.protocol.map=PLAINTEXT:PLAINTEXT,SSL:SSL,SASL_PLAINTEXT:SASL_PLAINTEXT,SASL_SSL:SASL_SSL - -# The number of threads that the server uses for receiving requests from the network and sending responses to the network -num.network.threads=3 - -# The number of threads that the server uses for processing requests, which may include disk I/O -num.io.threads=8 - -# The send buffer (SO_SNDBUF) used by the socket server -socket.send.buffer.bytes=102400 - -# The receive buffer (SO_RCVBUF) used by the socket server -socket.receive.buffer.bytes=102400 - -# The maximum size of a request that the socket server will accept (protection against OOM) -socket.request.max.bytes=104857600 - - -############################# Log Basics ############################# - -# A comma separated list of directories under which to store log files -log.dirs=/tmp/kafka-logs - -# The default number of log partitions per topic. More partitions allow greater -# parallelism for consumption, but this will also result in more files across -# the brokers. -num.partitions=1 - -# The number of threads per data directory to be used for log recovery at startup and flushing at shutdown. -# This value is recommended to be increased for installations with data dirs located in RAID array. -num.recovery.threads.per.data.dir=1 - -############################# Internal Topic Settings ############################# -# The replication factor for the group metadata internal topics "__consumer_offsets" and "__transaction_state" -# For anything other than development testing, a value greater than 1 is recommended for to ensure availability such as 3. -offsets.topic.replication.factor=1 -transaction.state.log.replication.factor=1 -transaction.state.log.min.isr=1 - -############################# Log Flush Policy ############################# - -# Messages are immediately written to the filesystem but by default we only fsync() to sync -# the OS cache lazily. The following configurations control the flush of data to disk. -# There are a few important trade-offs here: -# 1. Durability: Unflushed data may be lost if you are not using replication. -# 2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as there will be a lot of data to flush. -# 3. Throughput: The flush is generally the most expensive operation, and a small flush interval may lead to excessive seeks. -# The settings below allow one to configure the flush policy to flush data after a period of time or -# every N messages (or both). This can be done globally and overridden on a per-topic basis. - -# The number of messages to accept before forcing a flush of data to disk -#log.flush.interval.messages=10000 - -# The maximum amount of time a message can sit in a log before we force a flush -#log.flush.interval.ms=1000 - -############################# Log Retention Policy ############################# - -# The following configurations control the disposal of log segments. The policy can -# be set to delete segments after a period of time, or after a given size has accumulated. -# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens -# from the end of the log. - -# The minimum age of a log file to be eligible for deletion due to age -log.retention.hours=168 - -# A size-based retention policy for logs. Segments are pruned from the log unless the remaining -# segments drop below log.retention.bytes. Functions independently of log.retention.hours. -#log.retention.bytes=1073741824 - -# The maximum size of a log segment file. When this size is reached a new log segment will be created. -log.segment.bytes=1073741824 - -# The interval at which log segments are checked to see if they can be deleted according -# to the retention policies -log.retention.check.interval.ms=300000 - -############################# Zookeeper ############################# - -# Zookeeper connection string (see zookeeper docs for details). -# This is a comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". -# You can also append an optional chroot string to the urls to specify the -# root directory for all kafka znodes. -zookeeper.connect=localhost:2181 - -# Timeout in ms for connecting to zookeeper -zookeeper.connection.timeout.ms=6000 - -##################### Confluent Metrics Reporter ####################### -# Confluent Control Center and Confluent Auto Data Balancer integration -# -# Uncomment the following lines to publish monitoring data for -# Confluent Control Center and Confluent Auto Data Balancer -# If you are using a dedicated metrics cluster, also adjust the settings -# to point to your metrics kakfa cluster. -#metric.reporters=io.confluent.metrics.reporter.ConfluentMetricsReporter -#confluent.metrics.reporter.bootstrap.servers=localhost:9092 -# -# Uncomment the following line if the metrics cluster has a single broker -#confluent.metrics.reporter.topic.replicas=1 - -##################### Confluent Proactive Support ###################### -# If set to true, and confluent-support-metrics package is installed -# then the feature to collect and report support metrics -# ("Metrics") is enabled. If set to false, the feature is disabled. -# -confluent.support.metrics.enable=false - - -# The customer ID under which support metrics will be collected and -# reported. -# -# When the customer ID is set to "anonymous" (the default), then only a -# reduced set of metrics is being collected and reported. -# -# Confluent customers -# ------------------- -# If you are a Confluent customer, then you should replace the default -# value with your actual Confluent customer ID. Doing so will ensure -# that additional support metrics will be collected and reported. -# -confluent.support.customer.id=anonymous - -############################# Group Coordinator Settings ############################# - -# The following configuration specifies the time, in milliseconds, that the GroupCoordinator will delay the initial consumer rebalance. -# The rebalance will be further delayed by the value of group.initial.rebalance.delay.ms as new members join the group, up to a maximum of max.poll.interval.ms. -# The default value for this is 3 seconds. -# We override this to 0 here as it makes for a better out-of-the-box experience for development and testing. -# However, in production environments the default value of 3 seconds is more suitable as this will help to avoid unnecessary, and potentially expensive, rebalances during application startup. -group.initial.rebalance.delay.ms=0 \ No newline at end of file From 49746557338cf54b99e192801f7efec2070ec3fc Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 15 Jul 2019 12:53:06 +0200 Subject: [PATCH 65/94] Use .avsc ending for avro files --- tests/integration/test_clients.py | 4 ++-- tests/test_samples/{key_schema => key_schema.avsc} | 0 tests/test_samples/{value_schema => value_schema.avsc} | 0 3 files changed, 2 insertions(+), 2 deletions(-) rename tests/test_samples/{key_schema => key_schema.avsc} (100%) rename tests/test_samples/{value_schema => value_schema.avsc} (100%) diff --git a/tests/integration/test_clients.py b/tests/integration/test_clients.py index 92e5cd29..d859d1f6 100644 --- a/tests/integration/test_clients.py +++ b/tests/integration/test_clients.py @@ -92,9 +92,9 @@ def produce_test_messages(producer: ConfluenceProducer, topic: str) -> Iterable[ def produce_test_messages_with_avro(avro_producer: AvroProducer, topic: str) -> Iterable[KafkaMessage]: - with open("tests/test_samples/key_schema", "r") as file: + with open("tests/test_samples/key_schema.avsc", "r") as file: key_schema = load_schema(file.read()) - with open("tests/test_samples/value_schema", "r") as file: + with open("tests/test_samples/value_schema.avsc", "r") as file: value_schema = load_schema(file.read()) messages = [] for i in range(10): diff --git a/tests/test_samples/key_schema b/tests/test_samples/key_schema.avsc similarity index 100% rename from tests/test_samples/key_schema rename to tests/test_samples/key_schema.avsc diff --git a/tests/test_samples/value_schema b/tests/test_samples/value_schema.avsc similarity index 100% rename from tests/test_samples/value_schema rename to tests/test_samples/value_schema.avsc From 51f7935a01bf5de0716c4c7827651254b036ef6c Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 15 Jul 2019 15:08:11 +0200 Subject: [PATCH 66/94] Let KafkaMessage do not rely on str key and value --- esque/message.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/esque/message.py b/esque/message.py index 918d6c8d..d8fceb51 100644 --- a/esque/message.py +++ b/esque/message.py @@ -1,6 +1,6 @@ import json import pathlib -from typing import Iterable, NamedTuple +from typing import Iterable, NamedTuple, Any from confluent_kafka.cimpl import Message @@ -11,8 +11,8 @@ class DecodedMessage(NamedTuple): class KafkaMessage(NamedTuple): - key: str - value: str + key: Any + value: Any key_schema: str = None value_schema: str = None From 95d9994f88bf141ab38237d443ed573de30d5cd0 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 15 Jul 2019 15:08:54 +0200 Subject: [PATCH 67/94] Update test to check consumed and produced content --- tests/conftest.py | 32 +++++--------- tests/integration/test_clients.py | 72 ++++++++++++++++++++++--------- 2 files changed, 62 insertions(+), 42 deletions(-) diff --git a/tests/conftest.py b/tests/conftest.py index e427fb7c..b17f9177 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -9,7 +9,6 @@ from confluent_kafka.avro import AvroProducer from confluent_kafka.cimpl import TopicPartition, Producer from pykafka.exceptions import NoBrokersAvailableError -from confluent_kafka.avro import loads as load_schema from esque.cluster import Cluster from esque.config import Config, sample_config_path @@ -72,6 +71,16 @@ def changed_topic_object(cluster, topic): yield TopicController(cluster).get_topic(topic, 1, 3, {"cleanup.policy": "compact"}) +@pytest.fixture() +def source_topic(topic: str) -> str: + yield topic + + +@pytest.fixture() +def target_topic(topic: str) -> str: + yield topic + + @pytest.fixture() def topic(confluent_admin_client: AdminClient, topic_id: str) -> str: """ @@ -158,27 +167,6 @@ def filled_topic(producer, topic_object): yield topic_object -@pytest.fixture() -def filled_avro_topic(avro_producer: AvroProducer, topic_object): - with open("tests/test_samples/key_schema", "r") as file: - key_schema = load_schema(file.read()) - with open("tests/test_samples/value_schema", "r") as file: - value_schema = load_schema(file.read()) - for i in range(10): - key = {"id": str(i)} - value = {"first": "Firstname", "last": "Lastname"} - avro_producer.produce( - topic=topic_object.name, key=key, value=value, key_schema=key_schema, value_schema=value_schema - ) - avro_producer.flush() - yield topic_object - - -@pytest.fixture() -def working_dir(tmpdir_factory): - yield tmpdir_factory.mktemp("working_directory") - - @pytest.fixture() def partly_read_consumer_group(consumer: confluent_kafka.Consumer, filled_topic, consumer_group): for i in range(5): diff --git a/tests/integration/test_clients.py b/tests/integration/test_clients.py index d859d1f6..898270d7 100644 --- a/tests/integration/test_clients.py +++ b/tests/integration/test_clients.py @@ -1,5 +1,4 @@ import json -import pathlib import random from typing import Iterable from string import ascii_letters @@ -10,14 +9,14 @@ from esque.avromessage import AvroFileReader from esque.clients import FileConsumer, AvroFileConsumer, FileProducer, AvroFileProducer from esque.message import PlainTextFileReader, KafkaMessage -from esque.topic import Topic from confluent_kafka.avro import loads as load_schema, AvroProducer @pytest.mark.integration -def test_plain_text_consume_to_file(consumer_group, producer: ConfluenceProducer, topic: str, working_dir: pathlib.Path): - produced_messages = produce_test_messages(producer, topic) - file_consumer = FileConsumer(consumer_group, topic, working_dir, False) +def test_plain_text_consume_to_file(consumer_group, producer: ConfluenceProducer, source_topic: str, tmpdir_factory): + working_dir = tmpdir_factory.mktemp("working_directory") + produced_messages = produce_test_messages(producer, source_topic) + file_consumer = FileConsumer(consumer_group, source_topic, working_dir, False) number_of_consumer_messages = file_consumer.consume(10) consumed_messages = [] @@ -36,9 +35,10 @@ def test_plain_text_consume_to_file(consumer_group, producer: ConfluenceProducer @pytest.mark.integration -def test_avro_consume_to_file(consumer_group, avro_producer: AvroProducer, topic: str, working_dir: pathlib.Path): - produced_messages = produce_test_messages_with_avro(avro_producer, topic) - file_consumer = AvroFileConsumer(consumer_group, topic, working_dir, False) +def test_avro_consume_to_file(consumer_group, avro_producer: AvroProducer, source_topic: str, tmpdir_factory): + working_dir = tmpdir_factory.mktemp("working_directory") + produced_messages = produce_test_messages_with_avro(avro_producer, source_topic) + file_consumer = AvroFileConsumer(consumer_group, source_topic, working_dir, False) number_of_consumer_messages = file_consumer.consume(10) consumed_messages = [] @@ -57,27 +57,59 @@ def test_avro_consume_to_file(consumer_group, avro_producer: AvroProducer, topic @pytest.mark.integration -def test_plain_text_consume_and_produce(consumer_group, filled_topic: Topic, topic: str, working_dir: pathlib.Path): - file_consumer = FileConsumer(consumer_group, filled_topic.name, working_dir, False) - number_of_consumer_messages = file_consumer.consume(10) +def test_plain_text_consume_and_produce(consumer_group, producer: ConfluenceProducer, source_topic: str, target_topic: str, tmpdir_factory): + working_dir = tmpdir_factory.mktemp("working_directory") + produced_messages = produce_test_messages(producer, source_topic) + file_consumer = FileConsumer(consumer_group, source_topic, working_dir, False) + file_consumer.consume(10) producer = FileProducer(working_dir) - number_of_produced_messages = producer.produce(topic) + producer.produce(target_topic) - assert number_of_consumer_messages == 10 - assert number_of_produced_messages == 10 + # Check assertions: + assertion_check_directory = tmpdir_factory.mktemp("assertion_check_directory") + file_consumer = FileConsumer((consumer_group + "assertion_check"), target_topic, assertion_check_directory, False) + file_consumer.consume(10) + + consumed_messages = [] + file_reader = PlainTextFileReader(assertion_check_directory) + with file_reader: + for message in file_reader.read_from_file(): + consumed_messages.append(message) + + assert all([ + produced_message.key == consumed_message.key and + produced_message.value == consumed_message.value + for produced_message, consumed_message in zip(produced_messages, consumed_messages) + ]) @pytest.mark.integration -def test_avro_consume_and_produce(consumer_group, filled_avro_topic: Topic, topic: str, working_dir: pathlib.Path): - file_consumer = AvroFileConsumer(consumer_group, filled_avro_topic.name, working_dir, False) - number_of_consumer_messages = file_consumer.consume(10) +def test_avro_consume_and_produce(consumer_group, avro_producer: AvroProducer, source_topic: str, target_topic: str, tmpdir_factory): + working_dir = tmpdir_factory.mktemp("working_directory") + produced_messages = produce_test_messages_with_avro(avro_producer, source_topic) + file_consumer = AvroFileConsumer(consumer_group, source_topic, working_dir, False) + file_consumer.consume(10) producer = AvroFileProducer(working_dir) - number_of_produced_messages = producer.produce(topic) + producer.produce(target_topic) - assert number_of_consumer_messages == 10 - assert number_of_produced_messages == 10 + # Check assertions: + assertion_check_directory = tmpdir_factory.mktemp("assertion_check_directory") + file_consumer = AvroFileConsumer((consumer_group + "assertion_check"), target_topic, assertion_check_directory, False) + file_consumer.consume(10) + + consumed_messages = [] + file_reader = AvroFileReader(assertion_check_directory) + with file_reader: + for message in file_reader.read_from_file(): + consumed_messages.append(message) + + assert all([ + produced_message.key == consumed_message.key and + produced_message.value == consumed_message.value + for produced_message, consumed_message in zip(produced_messages, consumed_messages) + ]) def produce_test_messages(producer: ConfluenceProducer, topic: str) -> Iterable[KafkaMessage]: From adaa8ae4492421e4962019ee105e7712284279d2 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 15 Jul 2019 15:16:38 +0200 Subject: [PATCH 68/94] Update travis.yml --- .travis.yml | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/.travis.yml b/.travis.yml index 1be6977d..543bb990 100644 --- a/.travis.yml +++ b/.travis.yml @@ -2,10 +2,11 @@ dist: xenial language: python cache: pip env: -- TEST_CMD="pytest tests/ --integration --cov=esque --local" -- BROKER_URL="localhost:9092" -- ZOOKEEPER_URL="localhost:2181" -- SCHEMA_REGISTRY_URL="localhost:8081" +- > + TEST_CMD="pytest tests/ --integration --cov=esque --local" + BROKER_URL="localhost:9092" + ZOOKEEPER_URL="localhost:2181" + SCHEMA_REGISTRY_URL="localhost:8081" before_install: - wget http://packages.confluent.io/archive/5.2/confluent-community-5.2.1-2.12.tar.gz -O confluent-community.tgz - mkdir -p confluent-community && tar xzf confluent-community.tgz -C confluent-community --strip-components 1 From 146bdd6e0209ff966e64a9c00b0ac6c564b9b0fb Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 15 Jul 2019 15:25:22 +0200 Subject: [PATCH 69/94] Update travis.yml --- .travis.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.travis.yml b/.travis.yml index 543bb990..34b053dc 100644 --- a/.travis.yml +++ b/.travis.yml @@ -32,11 +32,13 @@ notifications: matrix: include: - name: black + before_install: [] python: 3.7 env: - TEST_CMD="black --check --verbose --line-length=119 ." - name: flake8 python: 3.7 + before_install: [] env: - TEST_CMD="flake8 esque/" - name: '3.6' From a33b18b39c0fd6ebe80d4f13f0081f37f20dbef5 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 15 Jul 2019 15:27:34 +0200 Subject: [PATCH 70/94] Fix coding style --- tests/integration/test_clients.py | 60 +++++++++++++++++-------------- 1 file changed, 34 insertions(+), 26 deletions(-) diff --git a/tests/integration/test_clients.py b/tests/integration/test_clients.py index 898270d7..29201dd5 100644 --- a/tests/integration/test_clients.py +++ b/tests/integration/test_clients.py @@ -27,11 +27,12 @@ def test_plain_text_consume_to_file(consumer_group, producer: ConfluenceProducer assert number_of_consumer_messages == 10 assert len(consumed_messages) == 10 - assert all([ - produced_message.key == consumed_message.key and - produced_message.value == consumed_message.value - for produced_message, consumed_message in zip(produced_messages, consumed_messages) - ]) + assert all( + [ + produced_message.key == consumed_message.key and produced_message.value == consumed_message.value + for produced_message, consumed_message in zip(produced_messages, consumed_messages) + ] + ) @pytest.mark.integration @@ -49,15 +50,18 @@ def test_avro_consume_to_file(consumer_group, avro_producer: AvroProducer, sourc assert number_of_consumer_messages == 10 assert len(consumed_messages) == 10 - assert all([ - produced_message.key == consumed_message.key and - produced_message.value == consumed_message.value - for produced_message, consumed_message in zip(produced_messages, consumed_messages) - ]) + assert all( + [ + produced_message.key == consumed_message.key and produced_message.value == consumed_message.value + for produced_message, consumed_message in zip(produced_messages, consumed_messages) + ] + ) @pytest.mark.integration -def test_plain_text_consume_and_produce(consumer_group, producer: ConfluenceProducer, source_topic: str, target_topic: str, tmpdir_factory): +def test_plain_text_consume_and_produce( + consumer_group, producer: ConfluenceProducer, source_topic: str, target_topic: str, tmpdir_factory +): working_dir = tmpdir_factory.mktemp("working_directory") produced_messages = produce_test_messages(producer, source_topic) file_consumer = FileConsumer(consumer_group, source_topic, working_dir, False) @@ -77,15 +81,18 @@ def test_plain_text_consume_and_produce(consumer_group, producer: ConfluenceProd for message in file_reader.read_from_file(): consumed_messages.append(message) - assert all([ - produced_message.key == consumed_message.key and - produced_message.value == consumed_message.value - for produced_message, consumed_message in zip(produced_messages, consumed_messages) - ]) + assert all( + [ + produced_message.key == consumed_message.key and produced_message.value == consumed_message.value + for produced_message, consumed_message in zip(produced_messages, consumed_messages) + ] + ) @pytest.mark.integration -def test_avro_consume_and_produce(consumer_group, avro_producer: AvroProducer, source_topic: str, target_topic: str, tmpdir_factory): +def test_avro_consume_and_produce( + consumer_group, avro_producer: AvroProducer, source_topic: str, target_topic: str, tmpdir_factory +): working_dir = tmpdir_factory.mktemp("working_directory") produced_messages = produce_test_messages_with_avro(avro_producer, source_topic) file_consumer = AvroFileConsumer(consumer_group, source_topic, working_dir, False) @@ -96,7 +103,9 @@ def test_avro_consume_and_produce(consumer_group, avro_producer: AvroProducer, s # Check assertions: assertion_check_directory = tmpdir_factory.mktemp("assertion_check_directory") - file_consumer = AvroFileConsumer((consumer_group + "assertion_check"), target_topic, assertion_check_directory, False) + file_consumer = AvroFileConsumer( + (consumer_group + "assertion_check"), target_topic, assertion_check_directory, False + ) file_consumer.consume(10) consumed_messages = [] @@ -105,11 +114,12 @@ def test_avro_consume_and_produce(consumer_group, avro_producer: AvroProducer, s for message in file_reader.read_from_file(): consumed_messages.append(message) - assert all([ - produced_message.key == consumed_message.key and - produced_message.value == consumed_message.value - for produced_message, consumed_message in zip(produced_messages, consumed_messages) - ]) + assert all( + [ + produced_message.key == consumed_message.key and produced_message.value == consumed_message.value + for produced_message, consumed_message in zip(produced_messages, consumed_messages) + ] + ) def produce_test_messages(producer: ConfluenceProducer, topic: str) -> Iterable[KafkaMessage]: @@ -133,8 +143,6 @@ def produce_test_messages_with_avro(avro_producer: AvroProducer, topic: str) -> key = {"id": str(i)} value = {"first": "Firstname", "last": "Lastname"} messages.append(KafkaMessage(json.dumps(key), json.dumps(value), key_schema, value_schema)) - avro_producer.produce( - topic=topic, key=key, value=value, key_schema=key_schema, value_schema=value_schema - ) + avro_producer.produce(topic=topic, key=key, value=value, key_schema=key_schema, value_schema=value_schema) avro_producer.flush() return messages From ce7acc66c4cb1033ac76c3b14b5b2e5c7522c0f5 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 15 Jul 2019 15:59:39 +0200 Subject: [PATCH 71/94] Optimize tests --- tests/integration/test_clients.py | 21 +++------------------ 1 file changed, 3 insertions(+), 18 deletions(-) diff --git a/tests/integration/test_clients.py b/tests/integration/test_clients.py index 29201dd5..093f3122 100644 --- a/tests/integration/test_clients.py +++ b/tests/integration/test_clients.py @@ -27,12 +27,7 @@ def test_plain_text_consume_to_file(consumer_group, producer: ConfluenceProducer assert number_of_consumer_messages == 10 assert len(consumed_messages) == 10 - assert all( - [ - produced_message.key == consumed_message.key and produced_message.value == consumed_message.value - for produced_message, consumed_message in zip(produced_messages, consumed_messages) - ] - ) + assert produced_messages == consumed_messages @pytest.mark.integration @@ -81,12 +76,7 @@ def test_plain_text_consume_and_produce( for message in file_reader.read_from_file(): consumed_messages.append(message) - assert all( - [ - produced_message.key == consumed_message.key and produced_message.value == consumed_message.value - for produced_message, consumed_message in zip(produced_messages, consumed_messages) - ] - ) + assert produced_messages == consumed_messages @pytest.mark.integration @@ -114,12 +104,7 @@ def test_avro_consume_and_produce( for message in file_reader.read_from_file(): consumed_messages.append(message) - assert all( - [ - produced_message.key == consumed_message.key and produced_message.value == consumed_message.value - for produced_message, consumed_message in zip(produced_messages, consumed_messages) - ] - ) + assert produced_messages == consumed_messages def produce_test_messages(producer: ConfluenceProducer, topic: str) -> Iterable[KafkaMessage]: From 4697e50c92a4b4661483a0c6196c7905b2a15fdc Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Tue, 16 Jul 2019 10:21:58 +0200 Subject: [PATCH 72/94] Enable partition wise storing of messages --- esque/avromessage.py | 14 +++++-- esque/clients.py | 52 ++++++++++++++++++-------- esque/message.py | 14 +++++-- tests/integration/test_clients.py | 61 +++++++++++++++---------------- 4 files changed, 88 insertions(+), 53 deletions(-) diff --git a/esque/avromessage.py b/esque/avromessage.py index 0a0d3b78..fb9f9198 100644 --- a/esque/avromessage.py +++ b/esque/avromessage.py @@ -17,6 +17,7 @@ class DecodedAvroMessage(NamedTuple): key: Any value: Any + partition: int key_schema_id: int value_schema_id: int @@ -35,7 +36,9 @@ def __init__(self, directory: pathlib.Path, schema_registry_client: SchemaRegist def write_message_to_file(self, message: Message): key_schema_id, decoded_key = self.decode_bytes(message.key()) value_schema_id, decoded_value = self.decode_bytes(message.value()) - decoded_message = DecodedAvroMessage(decoded_key, decoded_value, key_schema_id, value_schema_id) + decoded_message = DecodedAvroMessage( + decoded_key, decoded_value, message.partition(), key_schema_id, value_schema_id + ) if self.schema_changed(decoded_message) or self.schema_dir_name is None: self.schema_dir_name = f"{next(self.schema_version):04}_{key_schema_id}_{value_schema_id}" @@ -44,8 +47,9 @@ def write_message_to_file(self, message: Message): self._dump_schemata(key_schema_id, value_schema_id) serializable_message = { - "key": decoded_key, - "value": decoded_value, + "key": decoded_message.key, + "value": decoded_message.value, + "partition": decoded_message.partition, "schema_directory_name": self.schema_dir_name, } pickle.dump(serializable_message, self.file) @@ -94,7 +98,9 @@ def read_from_file(self) -> Iterable[KafkaMessage]: key_schema = load_schema((schema_directory / "key_schema.avsc").read_text(encoding="utf-8")) value_schema = load_schema((schema_directory / "value_schema.avsc").read_text(encoding="utf-8")) - yield KafkaMessage(json.dumps(record["key"]), json.dumps(record["value"]), key_schema, value_schema) + yield KafkaMessage( + json.dumps(record["key"]), json.dumps(record["value"]), record["partition"], key_schema, value_schema + ) def extract_schema_id(message: bytes) -> int: diff --git a/esque/clients.py b/esque/clients.py index e52dd4c0..a71a6b8b 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -1,5 +1,7 @@ import json import pathlib +from contextlib import ExitStack +from glob import glob from typing import Optional, Tuple import click @@ -13,7 +15,7 @@ from esque.config import Config from esque.errors import raise_for_kafka_error, raise_for_message from esque.helpers import delivery_callback, delta_t -from esque.message import KafkaMessage, PlainTextFileReader, PlainTextFileWriter +from esque.message import KafkaMessage, PlainTextFileReader, PlainTextFileWriter, FileReader, FileWriter from esque.schemaregistry import SchemaRegistryClient from abc import ABC, abstractmethod @@ -88,27 +90,38 @@ def __init__(self, group_id: str, topic_name: str, working_dir: pathlib.Path, la self._config.update({"default.topic.config": {"auto.offset.reset": offset_reset}}) self._consumer = confluent_kafka.Consumer(self._config) self._subscribe(topic_name) - self.file_writer = PlainTextFileWriter(working_dir) def consume(self, amount: int) -> int: counter = 0 - with self.file_writer: + file_writers = {} + with ExitStack() as stack: while counter < amount: message = self._consume_single_message() if message is None: return counter - self.file_writer.write_message_to_file(message) + if message.partition() not in file_writers.keys(): + file_writer = self.get_file_writer((self.working_dir / f"partition_{message.partition()}")) + stack.enter_context(file_writer) + file_writers[message.partition()] = file_writer + + file_writer = file_writers[message.partition()] + file_writer.write_message_to_file(message) counter += 1 return counter + def get_file_writer(self, directory: pathlib.Path) -> FileWriter: + return PlainTextFileWriter(directory) + class AvroFileConsumer(FileConsumer): def __init__(self, group_id: str, topic_name: str, working_dir: pathlib.Path, last: bool): super().__init__(group_id, topic_name, working_dir, last) - schema_registry_client = SchemaRegistryClient(Config().schema_registry) - self.file_writer = AvroFileWriter(working_dir, schema_registry_client) + self.schema_registry_client = SchemaRegistryClient(Config().schema_registry) + + def get_file_writer(self, directory: pathlib.Path) -> FileWriter: + return AvroFileWriter(directory, self.schema_registry_client) class Producer(ABC): @@ -142,25 +155,31 @@ def __init__(self, working_dir: pathlib.Path): super().__init__() self._producer = confluent_kafka.Producer(self._config) self.working_dir = working_dir - self.file_reader = PlainTextFileReader(working_dir) def produce(self, topic_name: str) -> int: - with self.file_reader: - counter = 0 - for message in self.file_reader.read_from_file(): - self.produce_message(topic_name, message) - counter += 1 + path_list = glob(str(self.working_dir / "partition_*")) + counter = 0 + with ExitStack() as stack: + for partition_path in path_list: + file_reader = self.get_file_reader(pathlib.Path(partition_path)) + stack.enter_context(file_reader) + for message in file_reader.read_from_file(): + self.produce_message(topic_name, message) + counter += 1 while True: left_messages = self._producer.flush(1) if left_messages == 0: break - click.echo("Still {left_messages} messages left, flushing...") + click.echo(f"Still {left_messages} messages left, flushing...") return counter + def get_file_reader(self, directory: pathlib.Path) -> FileReader: + return PlainTextFileReader(directory) + def produce_message(self, topic_name: str, message: KafkaMessage): - self._producer.produce(topic=topic_name, key=message.key, value=message.value) + self._producer.produce(topic=topic_name, key=message.key, value=message.value, partition=message.partition) class AvroFileProducer(FileProducer): @@ -168,7 +187,9 @@ def __init__(self, working_dir: pathlib.Path): super().__init__(working_dir) self._config.update({"schema.registry.url": Config().schema_registry}) self._producer = AvroProducer(self._config) - self.file_reader = AvroFileReader(working_dir) + + def get_file_reader(self, directory: pathlib.Path) -> FileReader: + return AvroFileReader(directory) def produce_message(self, topic_name: str, message: KafkaMessage): self._producer.produce( @@ -177,4 +198,5 @@ def produce_message(self, topic_name: str, message: KafkaMessage): value=json.loads(message.value), key_schema=message.key_schema, value_schema=message.value_schema, + partition=message.partition, ) diff --git a/esque/message.py b/esque/message.py index d8fceb51..b0a48e5e 100644 --- a/esque/message.py +++ b/esque/message.py @@ -8,11 +8,13 @@ class DecodedMessage(NamedTuple): key: str value: str + partition: int class KafkaMessage(NamedTuple): key: Any value: Any + partition: int key_schema: str = None value_schema: str = None @@ -24,6 +26,8 @@ def __init__(self, directory: pathlib.Path): self.open_mode = "w+" def __enter__(self): + if not self.directory.exists(): + self.directory.mkdir() self.file = (self.directory / self.file_name).open(self.open_mode) def __exit__(self, exc_type, exc_val, exc_tb): @@ -47,7 +51,11 @@ def __init__(self, directory: pathlib.Path): def write_message_to_file(self, message: Message): decoded_message = decode_message(message) - serializable_message = {"key": decoded_message.key, "value": decoded_message.value} + serializable_message = { + "key": decoded_message.key, + "value": decoded_message.value, + "partition": decoded_message.partition, + } self.file.write(json.dumps(serializable_message) + "\n") @@ -63,11 +71,11 @@ def read_from_file(self) -> Iterable[KafkaMessage]: except EOFError: return - yield KafkaMessage(record["key"], record["value"]) + yield KafkaMessage(record["key"], record["value"], record["partition"]) def decode_message(message: Message) -> DecodedMessage: decoded_key = message.key().decode("utf-8") decoded_value = message.value().decode("utf-8") - return DecodedMessage(decoded_key, decoded_value) + return DecodedMessage(decoded_key, decoded_value, message.partition()) diff --git a/tests/integration/test_clients.py b/tests/integration/test_clients.py index 093f3122..a7b3148f 100644 --- a/tests/integration/test_clients.py +++ b/tests/integration/test_clients.py @@ -1,6 +1,9 @@ import json +import pathlib import random -from typing import Iterable +from contextlib import ExitStack +from glob import glob +from typing import Iterable, List from string import ascii_letters import pytest @@ -19,11 +22,7 @@ def test_plain_text_consume_to_file(consumer_group, producer: ConfluenceProducer file_consumer = FileConsumer(consumer_group, source_topic, working_dir, False) number_of_consumer_messages = file_consumer.consume(10) - consumed_messages = [] - file_reader = PlainTextFileReader(working_dir) - with file_reader: - for message in file_reader.read_from_file(): - consumed_messages.append(message) + consumed_messages = get_consumed_messages(working_dir, False) assert number_of_consumer_messages == 10 assert len(consumed_messages) == 10 @@ -37,20 +36,11 @@ def test_avro_consume_to_file(consumer_group, avro_producer: AvroProducer, sourc file_consumer = AvroFileConsumer(consumer_group, source_topic, working_dir, False) number_of_consumer_messages = file_consumer.consume(10) - consumed_messages = [] - file_reader = AvroFileReader(working_dir) - with file_reader: - for message in file_reader.read_from_file(): - consumed_messages.append(message) + consumed_messages = get_consumed_messages(working_dir, True) assert number_of_consumer_messages == 10 assert len(consumed_messages) == 10 - assert all( - [ - produced_message.key == consumed_message.key and produced_message.value == consumed_message.value - for produced_message, consumed_message in zip(produced_messages, consumed_messages) - ] - ) + assert produced_messages == consumed_messages @pytest.mark.integration @@ -70,11 +60,7 @@ def test_plain_text_consume_and_produce( file_consumer = FileConsumer((consumer_group + "assertion_check"), target_topic, assertion_check_directory, False) file_consumer.consume(10) - consumed_messages = [] - file_reader = PlainTextFileReader(assertion_check_directory) - with file_reader: - for message in file_reader.read_from_file(): - consumed_messages.append(message) + consumed_messages = get_consumed_messages(assertion_check_directory, False) assert produced_messages == consumed_messages @@ -98,11 +84,7 @@ def test_avro_consume_and_produce( ) file_consumer.consume(10) - consumed_messages = [] - file_reader = AvroFileReader(assertion_check_directory) - with file_reader: - for message in file_reader.read_from_file(): - consumed_messages.append(message) + consumed_messages = get_consumed_messages(assertion_check_directory, True) assert produced_messages == consumed_messages @@ -111,9 +93,9 @@ def produce_test_messages(producer: ConfluenceProducer, topic: str) -> Iterable[ messages = [] for i in range(10): random_value = "".join(random.choices(ascii_letters, k=5)) - message = KafkaMessage(str(i), random_value) + message = KafkaMessage(str(i), random_value, 0) messages.append(message) - producer.produce(topic=topic, key=message.key, value=message.value) + producer.produce(topic=topic, key=message.key, value=message.value, partition=0) producer.flush() return messages @@ -127,7 +109,24 @@ def produce_test_messages_with_avro(avro_producer: AvroProducer, topic: str) -> for i in range(10): key = {"id": str(i)} value = {"first": "Firstname", "last": "Lastname"} - messages.append(KafkaMessage(json.dumps(key), json.dumps(value), key_schema, value_schema)) - avro_producer.produce(topic=topic, key=key, value=value, key_schema=key_schema, value_schema=value_schema) + messages.append(KafkaMessage(json.dumps(key), json.dumps(value), 0, key_schema, value_schema)) + avro_producer.produce( + topic=topic, key=key, value=value, key_schema=key_schema, value_schema=value_schema, partition=0 + ) avro_producer.flush() return messages + + +def get_consumed_messages(directory, avro: bool) -> List[KafkaMessage]: + consumed_messages = [] + path_list = glob(str(directory / "partition_*")) + with ExitStack() as stack: + for partition_path in path_list: + if avro: + file_reader = AvroFileReader(pathlib.Path(partition_path)) + else: + file_reader = PlainTextFileReader(pathlib.Path(partition_path)) + stack.enter_context(file_reader) + for message in file_reader.read_from_file(): + consumed_messages.append(message) + return consumed_messages From d03e98f76c4058c97fc039864c778a72b5263da8 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Tue, 16 Jul 2019 13:59:42 +0200 Subject: [PATCH 73/94] Add tests for topics with multiple partitions --- esque/cli/commands.py | 2 +- tests/conftest.py | 34 +++++++++++-------- tests/integration/test_clients.py | 54 +++++++++++++++++++++---------- 3 files changed, 58 insertions(+), 32 deletions(-) diff --git a/esque/cli/commands.py b/esque/cli/commands.py index 5d97e85d..f6199f67 100644 --- a/esque/cli/commands.py +++ b/esque/cli/commands.py @@ -227,7 +227,7 @@ def get_consumergroups(state): @get.command("topics") @click.argument("topic", required=False, type=click.STRING, autocompletion=list_topics) @pass_state -def get_topics(state, topic, o): +def get_topics(state, topic): topics = TopicController(state.cluster).list_topics(search_string=topic) for topic in topics: click.echo(topic.name) diff --git a/tests/conftest.py b/tests/conftest.py index b17f9177..a47007f8 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -2,6 +2,7 @@ from concurrent.futures import Future from pathlib import Path from string import ascii_letters +from typing import Iterable, Tuple import confluent_kafka import pytest @@ -62,41 +63,46 @@ def topic_id(confluent_admin_client) -> str: @pytest.fixture() -def topic_object(cluster, topic): - yield TopicController(cluster).get_topic(topic) +def topic_object(cluster, topic: Iterable[Tuple[str, int]]): + topic_id, _ = topic + yield TopicController(cluster).get_topic(topic_id) @pytest.fixture() -def changed_topic_object(cluster, topic): - yield TopicController(cluster).get_topic(topic, 1, 3, {"cleanup.policy": "compact"}) +def changed_topic_object(cluster, topic: Iterable[Tuple[str, int]]): + topic_id, _ = topic + yield TopicController(cluster).get_topic(topic_id, 1, 3, {"cleanup.policy": "compact"}) @pytest.fixture() -def source_topic(topic: str) -> str: +def source_topic(topic: Iterable[Tuple[str, int]]) -> Iterable[Tuple[str, int]]: yield topic @pytest.fixture() -def target_topic(topic: str) -> str: +def target_topic(topic: Iterable[Tuple[str, int]]) -> Iterable[Tuple[str, int]]: yield topic -@pytest.fixture() -def topic(confluent_admin_client: AdminClient, topic_id: str) -> str: +@pytest.fixture(params=[1, 5, 10], ids=["num_partitions=1", "num_partitions=5", "num_partitions=10"]) +def topic(request, confluent_admin_client: AdminClient, topic_id: str) -> Iterable[Tuple[str, int]]: """ - Creates a kafka topic consisting of a random 5 character string. + Creates a kafka topic consisting of a random 5 character string and being partition into 1, 2 or 4 partitions. + Then it yields the tuple (topic, n_partitions). - :return: Topic (str) + Prints topic information before and after topic was used by a test. + :return: Topic and number of partitions within it. """ - future: Future = confluent_admin_client.create_topics( - [NewTopic(topic_id, num_partitions=1, replication_factor=1)] - )[topic_id] + partitions = request.param + future: Future = confluent_admin_client.create_topics([NewTopic(topic_id, partitions, replication_factor=1)])[ + topic_id + ] while not future.done() or future.cancelled(): if future.result(): raise RuntimeError confluent_admin_client.poll(timeout=1) - yield topic_id + yield (topic_id, partitions) topics = confluent_admin_client.list_topics(timeout=5).topics.keys() if topic_id in topics: diff --git a/tests/integration/test_clients.py b/tests/integration/test_clients.py index a7b3148f..1cefd561 100644 --- a/tests/integration/test_clients.py +++ b/tests/integration/test_clients.py @@ -3,7 +3,7 @@ import random from contextlib import ExitStack from glob import glob -from typing import Iterable, List +from typing import Iterable, List, Tuple from string import ascii_letters import pytest @@ -16,10 +16,13 @@ @pytest.mark.integration -def test_plain_text_consume_to_file(consumer_group, producer: ConfluenceProducer, source_topic: str, tmpdir_factory): +def test_plain_text_consume_to_file( + consumer_group, producer: ConfluenceProducer, source_topic: Iterable[Tuple[str, int]], tmpdir_factory +): + source_topic_id, _ = source_topic working_dir = tmpdir_factory.mktemp("working_directory") - produced_messages = produce_test_messages(producer, source_topic) - file_consumer = FileConsumer(consumer_group, source_topic, working_dir, False) + produced_messages = produce_test_messages(producer, source_topic_id) + file_consumer = FileConsumer(consumer_group, source_topic_id, working_dir, False) number_of_consumer_messages = file_consumer.consume(10) consumed_messages = get_consumed_messages(working_dir, False) @@ -30,10 +33,13 @@ def test_plain_text_consume_to_file(consumer_group, producer: ConfluenceProducer @pytest.mark.integration -def test_avro_consume_to_file(consumer_group, avro_producer: AvroProducer, source_topic: str, tmpdir_factory): +def test_avro_consume_to_file( + consumer_group, avro_producer: AvroProducer, source_topic: Iterable[Tuple[str, int]], tmpdir_factory +): + source_topic_id, _ = source_topic working_dir = tmpdir_factory.mktemp("working_directory") - produced_messages = produce_test_messages_with_avro(avro_producer, source_topic) - file_consumer = AvroFileConsumer(consumer_group, source_topic, working_dir, False) + produced_messages = produce_test_messages_with_avro(avro_producer, source_topic_id) + file_consumer = AvroFileConsumer(consumer_group, source_topic_id, working_dir, False) number_of_consumer_messages = file_consumer.consume(10) consumed_messages = get_consumed_messages(working_dir, True) @@ -45,19 +51,27 @@ def test_avro_consume_to_file(consumer_group, avro_producer: AvroProducer, sourc @pytest.mark.integration def test_plain_text_consume_and_produce( - consumer_group, producer: ConfluenceProducer, source_topic: str, target_topic: str, tmpdir_factory + consumer_group, + producer: ConfluenceProducer, + source_topic: Iterable[Tuple[str, int]], + target_topic: Iterable[Tuple[str, int]], + tmpdir_factory, ): + source_topic_id, _ = source_topic + target_topic_id, _ = target_topic working_dir = tmpdir_factory.mktemp("working_directory") - produced_messages = produce_test_messages(producer, source_topic) - file_consumer = FileConsumer(consumer_group, source_topic, working_dir, False) + produced_messages = produce_test_messages(producer, source_topic_id) + file_consumer = FileConsumer(consumer_group, source_topic_id, working_dir, False) file_consumer.consume(10) producer = FileProducer(working_dir) - producer.produce(target_topic) + producer.produce(source_topic_id) # Check assertions: assertion_check_directory = tmpdir_factory.mktemp("assertion_check_directory") - file_consumer = FileConsumer((consumer_group + "assertion_check"), target_topic, assertion_check_directory, False) + file_consumer = FileConsumer( + (consumer_group + "assertion_check"), target_topic_id, assertion_check_directory, False + ) file_consumer.consume(10) consumed_messages = get_consumed_messages(assertion_check_directory, False) @@ -67,20 +81,26 @@ def test_plain_text_consume_and_produce( @pytest.mark.integration def test_avro_consume_and_produce( - consumer_group, avro_producer: AvroProducer, source_topic: str, target_topic: str, tmpdir_factory + consumer_group, + avro_producer: AvroProducer, + source_topic: Iterable[Tuple[str, int]], + target_topic: Iterable[Tuple[str, int]], + tmpdir_factory, ): + source_topic_id, _ = source_topic + target_topic_id, _ = target_topic working_dir = tmpdir_factory.mktemp("working_directory") - produced_messages = produce_test_messages_with_avro(avro_producer, source_topic) - file_consumer = AvroFileConsumer(consumer_group, source_topic, working_dir, False) + produced_messages = produce_test_messages_with_avro(avro_producer, source_topic_id) + file_consumer = AvroFileConsumer(consumer_group, source_topic_id, working_dir, False) file_consumer.consume(10) producer = AvroFileProducer(working_dir) - producer.produce(target_topic) + producer.produce(target_topic_id) # Check assertions: assertion_check_directory = tmpdir_factory.mktemp("assertion_check_directory") file_consumer = AvroFileConsumer( - (consumer_group + "assertion_check"), target_topic, assertion_check_directory, False + (consumer_group + "assertion_check"), target_topic_id, assertion_check_directory, False ) file_consumer.consume(10) From 081281f8497e9f20a83c5dc7103290adf1845eee Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Tue, 16 Jul 2019 17:12:58 +0200 Subject: [PATCH 74/94] Update topic fixtures --- tests/conftest.py | 75 ++++++++++++---------- tests/integration/test_clients.py | 12 ++-- tests/integration/test_topic_controller.py | 8 +-- 3 files changed, 50 insertions(+), 45 deletions(-) diff --git a/tests/conftest.py b/tests/conftest.py index a47007f8..99dfe987 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -2,7 +2,7 @@ from concurrent.futures import Future from pathlib import Path from string import ascii_letters -from typing import Iterable, Tuple +from typing import Iterable, Tuple, Callable import confluent_kafka import pytest @@ -55,58 +55,63 @@ def test_config(test_config_path, request): @pytest.fixture() -def topic_id(confluent_admin_client) -> str: - yield "".join(random.choices(ascii_letters, k=5)) - topics = confluent_admin_client.list_topics(timeout=5).topics.keys() - if topic_id in topics: - confluent_admin_client.delete_topics([topic_id]).popitem() +def topic_object(cluster, topic: str): + yield TopicController(cluster).get_topic(topic) @pytest.fixture() -def topic_object(cluster, topic: Iterable[Tuple[str, int]]): - topic_id, _ = topic - yield TopicController(cluster).get_topic(topic_id) +def changed_topic_object(cluster, topic: str): + yield TopicController(cluster).get_topic(topic, 1, 3, {"cleanup.policy": "compact"}) @pytest.fixture() -def changed_topic_object(cluster, topic: Iterable[Tuple[str, int]]): - topic_id, _ = topic - yield TopicController(cluster).get_topic(topic_id, 1, 3, {"cleanup.policy": "compact"}) +def topic(confluent_admin_client: AdminClient) -> str: + topic_id = "".join(random.choices(ascii_letters, k=5)) + yield topic_factory(1, topic_id)[0] -@pytest.fixture() -def source_topic(topic: Iterable[Tuple[str, int]]) -> Iterable[Tuple[str, int]]: - yield topic +@pytest.fixture(params=[1, 5, 10], ids=["1", "5", "10"]) +def source_topic( + request, topic_factory: Callable[[int, str], Tuple[str, int]] +) -> Iterable[Tuple[str, int]]: + topic_id = "".join(random.choices(ascii_letters, k=5)) + yield from topic_factory(request.param, topic_id) -@pytest.fixture() -def target_topic(topic: Iterable[Tuple[str, int]]) -> Iterable[Tuple[str, int]]: - yield topic +@pytest.fixture(params=[1, 5, 10], ids=["1", "5", "10"]) +def target_topic( + request, topic_factory: Callable[[int, str], Tuple[str, int]] +) -> Iterable[Tuple[str, int]]: + topic_id = "".join(random.choices(ascii_letters, k=5)) + yield from topic_factory(request.param, topic_id) -@pytest.fixture(params=[1, 5, 10], ids=["num_partitions=1", "num_partitions=5", "num_partitions=10"]) -def topic(request, confluent_admin_client: AdminClient, topic_id: str) -> Iterable[Tuple[str, int]]: +@pytest.fixture() +def topic_factory(confluent_admin_client: AdminClient) -> Callable[[int, str], Tuple[str, int]]: """ - Creates a kafka topic consisting of a random 5 character string and being partition into 1, 2 or 4 partitions. + Creates a kafka topic consisting of a random 5 character string and being partition into 1, 5 or 10 partitions. Then it yields the tuple (topic, n_partitions). Prints topic information before and after topic was used by a test. :return: Topic and number of partitions within it. """ - partitions = request.param - future: Future = confluent_admin_client.create_topics([NewTopic(topic_id, partitions, replication_factor=1)])[ - topic_id - ] - while not future.done() or future.cancelled(): - if future.result(): - raise RuntimeError - confluent_admin_client.poll(timeout=1) - - yield (topic_id, partitions) - - topics = confluent_admin_client.list_topics(timeout=5).topics.keys() - if topic_id in topics: - confluent_admin_client.delete_topics([topic_id]).popitem() + + def factory(partitions: int, topic_id: str) -> Tuple[str, int]: + future: Future = confluent_admin_client.create_topics( + [NewTopic(topic_id, num_partitions=partitions, replication_factor=1)] + )[topic_id] + while not future.done() or future.cancelled(): + if future.result(): + raise RuntimeError + confluent_admin_client.poll(timeout=1) + + yield (topic_id, partitions) + + topics = confluent_admin_client.list_topics(timeout=5).topics.keys() + if topic_id in topics: + confluent_admin_client.delete_topics([topic_id]).popitem() + + return factory @pytest.fixture() diff --git a/tests/integration/test_clients.py b/tests/integration/test_clients.py index 1cefd561..a6c65126 100644 --- a/tests/integration/test_clients.py +++ b/tests/integration/test_clients.py @@ -17,7 +17,7 @@ @pytest.mark.integration def test_plain_text_consume_to_file( - consumer_group, producer: ConfluenceProducer, source_topic: Iterable[Tuple[str, int]], tmpdir_factory + consumer_group, producer: ConfluenceProducer, source_topic: Tuple[str, int], tmpdir_factory ): source_topic_id, _ = source_topic working_dir = tmpdir_factory.mktemp("working_directory") @@ -34,7 +34,7 @@ def test_plain_text_consume_to_file( @pytest.mark.integration def test_avro_consume_to_file( - consumer_group, avro_producer: AvroProducer, source_topic: Iterable[Tuple[str, int]], tmpdir_factory + consumer_group, avro_producer: AvroProducer, source_topic: Tuple[str, int], tmpdir_factory ): source_topic_id, _ = source_topic working_dir = tmpdir_factory.mktemp("working_directory") @@ -53,8 +53,8 @@ def test_avro_consume_to_file( def test_plain_text_consume_and_produce( consumer_group, producer: ConfluenceProducer, - source_topic: Iterable[Tuple[str, int]], - target_topic: Iterable[Tuple[str, int]], + source_topic: Tuple[str, int], + target_topic: Tuple[str, int], tmpdir_factory, ): source_topic_id, _ = source_topic @@ -83,8 +83,8 @@ def test_plain_text_consume_and_produce( def test_avro_consume_and_produce( consumer_group, avro_producer: AvroProducer, - source_topic: Iterable[Tuple[str, int]], - target_topic: Iterable[Tuple[str, int]], + source_topic: Tuple[str, int], + target_topic: Tuple[str, int], tmpdir_factory, ): source_topic_id, _ = source_topic diff --git a/tests/integration/test_topic_controller.py b/tests/integration/test_topic_controller.py index 0fc02452..3030172e 100644 --- a/tests/integration/test_topic_controller.py +++ b/tests/integration/test_topic_controller.py @@ -23,14 +23,14 @@ def test_topic_creation_works( @pytest.mark.integration -def test_alter_topic_config_works(topic_controller: TopicController, topic_id: str): - initial_topic = topic_controller.get_topic(topic_id, config={"cleanup.policy": "delete"}) +def test_alter_topic_config_works(topic_controller: TopicController, topic: str): + initial_topic = topic_controller.get_topic(topic, config={"cleanup.policy": "delete"}) topic_controller.create_topics([initial_topic]) replicas, config = initial_topic.describe() assert config.get("Config").get("cleanup.policy") == "delete" - change_topic = topic_controller.get_topic(topic_id, config={"cleanup.policy": "compact"}) + change_topic = topic_controller.get_topic(topic, config={"cleanup.policy": "compact"}) topic_controller.alter_configs([change_topic]) - after_changes_applied_topic = topic_controller.get_topic(topic_id) + after_changes_applied_topic = topic_controller.get_topic(topic) replicas, final_config = after_changes_applied_topic.describe() assert final_config.get("Config").get("cleanup.policy") == "compact" From dbce4d3b4972bb25e381f72dde060d8990afbf89 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Tue, 16 Jul 2019 17:24:18 +0200 Subject: [PATCH 75/94] Update conftest.py --- tests/conftest.py | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/tests/conftest.py b/tests/conftest.py index 99dfe987..f709152b 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -70,18 +70,14 @@ def topic(confluent_admin_client: AdminClient) -> str: yield topic_factory(1, topic_id)[0] -@pytest.fixture(params=[1, 5, 10], ids=["1", "5", "10"]) -def source_topic( - request, topic_factory: Callable[[int, str], Tuple[str, int]] -) -> Iterable[Tuple[str, int]]: +@pytest.fixture(params=[1, 5], ids=["1", "5"]) +def source_topic(request, topic_factory: Callable[[int, str], Tuple[str, int]]) -> Iterable[Tuple[str, int]]: topic_id = "".join(random.choices(ascii_letters, k=5)) yield from topic_factory(request.param, topic_id) -@pytest.fixture(params=[1, 5, 10], ids=["1", "5", "10"]) -def target_topic( - request, topic_factory: Callable[[int, str], Tuple[str, int]] -) -> Iterable[Tuple[str, int]]: +@pytest.fixture(params=[1, 5], ids=["1", "5"]) +def target_topic(request, topic_factory: Callable[[int, str], Tuple[str, int]]) -> Iterable[Tuple[str, int]]: topic_id = "".join(random.choices(ascii_letters, k=5)) yield from topic_factory(request.param, topic_id) From 7e37a28dc91d125a63ae0f2166ca88330acb6c69 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Tue, 16 Jul 2019 17:29:43 +0200 Subject: [PATCH 76/94] Randomly produce messages in partitions --- tests/conftest.py | 4 ++-- tests/integration/test_clients.py | 12 ++++++++---- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/tests/conftest.py b/tests/conftest.py index f709152b..38a441cc 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -70,13 +70,13 @@ def topic(confluent_admin_client: AdminClient) -> str: yield topic_factory(1, topic_id)[0] -@pytest.fixture(params=[1, 5], ids=["1", "5"]) +@pytest.fixture(params=[1, 10], ids=["1", "10"]) def source_topic(request, topic_factory: Callable[[int, str], Tuple[str, int]]) -> Iterable[Tuple[str, int]]: topic_id = "".join(random.choices(ascii_letters, k=5)) yield from topic_factory(request.param, topic_id) -@pytest.fixture(params=[1, 5], ids=["1", "5"]) +@pytest.fixture(params=[1, 10], ids=["1", "10"]) def target_topic(request, topic_factory: Callable[[int, str], Tuple[str, int]]) -> Iterable[Tuple[str, int]]: topic_id = "".join(random.choices(ascii_letters, k=5)) yield from topic_factory(request.param, topic_id) diff --git a/tests/integration/test_clients.py b/tests/integration/test_clients.py index a6c65126..86b2b65c 100644 --- a/tests/integration/test_clients.py +++ b/tests/integration/test_clients.py @@ -109,29 +109,33 @@ def test_avro_consume_and_produce( assert produced_messages == consumed_messages -def produce_test_messages(producer: ConfluenceProducer, topic: str) -> Iterable[KafkaMessage]: +def produce_test_messages(producer: ConfluenceProducer, topic: Tuple[str, int]) -> Iterable[KafkaMessage]: + topic_name, num_partitions = topic messages = [] for i in range(10): + partition = random.randint(0, num_partitions) random_value = "".join(random.choices(ascii_letters, k=5)) message = KafkaMessage(str(i), random_value, 0) messages.append(message) - producer.produce(topic=topic, key=message.key, value=message.value, partition=0) + producer.produce(topic=topic_name, key=message.key, value=message.value, partition=partition) producer.flush() return messages -def produce_test_messages_with_avro(avro_producer: AvroProducer, topic: str) -> Iterable[KafkaMessage]: +def produce_test_messages_with_avro(avro_producer: AvroProducer, topic: Tuple[str, int]) -> Iterable[KafkaMessage]: + topic_name, num_partitions = topic with open("tests/test_samples/key_schema.avsc", "r") as file: key_schema = load_schema(file.read()) with open("tests/test_samples/value_schema.avsc", "r") as file: value_schema = load_schema(file.read()) messages = [] for i in range(10): + partition = random.randint(0, num_partitions) key = {"id": str(i)} value = {"first": "Firstname", "last": "Lastname"} messages.append(KafkaMessage(json.dumps(key), json.dumps(value), 0, key_schema, value_schema)) avro_producer.produce( - topic=topic, key=key, value=value, key_schema=key_schema, value_schema=value_schema, partition=0 + topic=topic, key=key, value=value, key_schema=key_schema, value_schema=value_schema, partition=partition ) avro_producer.flush() return messages From 37bc8c7ebcdcd3ce7359d51802dba0a4e6567950 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Tue, 16 Jul 2019 17:32:51 +0200 Subject: [PATCH 77/94] Fix tests --- tests/integration/test_clients.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_clients.py b/tests/integration/test_clients.py index 86b2b65c..3566be27 100644 --- a/tests/integration/test_clients.py +++ b/tests/integration/test_clients.py @@ -21,7 +21,7 @@ def test_plain_text_consume_to_file( ): source_topic_id, _ = source_topic working_dir = tmpdir_factory.mktemp("working_directory") - produced_messages = produce_test_messages(producer, source_topic_id) + produced_messages = produce_test_messages(producer, source_topic) file_consumer = FileConsumer(consumer_group, source_topic_id, working_dir, False) number_of_consumer_messages = file_consumer.consume(10) @@ -38,7 +38,7 @@ def test_avro_consume_to_file( ): source_topic_id, _ = source_topic working_dir = tmpdir_factory.mktemp("working_directory") - produced_messages = produce_test_messages_with_avro(avro_producer, source_topic_id) + produced_messages = produce_test_messages_with_avro(avro_producer, source_topic) file_consumer = AvroFileConsumer(consumer_group, source_topic_id, working_dir, False) number_of_consumer_messages = file_consumer.consume(10) @@ -60,7 +60,7 @@ def test_plain_text_consume_and_produce( source_topic_id, _ = source_topic target_topic_id, _ = target_topic working_dir = tmpdir_factory.mktemp("working_directory") - produced_messages = produce_test_messages(producer, source_topic_id) + produced_messages = produce_test_messages(producer, source_topic) file_consumer = FileConsumer(consumer_group, source_topic_id, working_dir, False) file_consumer.consume(10) @@ -90,7 +90,7 @@ def test_avro_consume_and_produce( source_topic_id, _ = source_topic target_topic_id, _ = target_topic working_dir = tmpdir_factory.mktemp("working_directory") - produced_messages = produce_test_messages_with_avro(avro_producer, source_topic_id) + produced_messages = produce_test_messages_with_avro(avro_producer, source_topic) file_consumer = AvroFileConsumer(consumer_group, source_topic_id, working_dir, False) file_consumer.consume(10) From ffdcc1892cd92e0a897481e2d521db8227cce06a Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Tue, 16 Jul 2019 17:36:47 +0200 Subject: [PATCH 78/94] Remove useless assertion --- tests/integration/test_clients.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/integration/test_clients.py b/tests/integration/test_clients.py index 3566be27..b1467826 100644 --- a/tests/integration/test_clients.py +++ b/tests/integration/test_clients.py @@ -28,7 +28,6 @@ def test_plain_text_consume_to_file( consumed_messages = get_consumed_messages(working_dir, False) assert number_of_consumer_messages == 10 - assert len(consumed_messages) == 10 assert produced_messages == consumed_messages @@ -45,7 +44,6 @@ def test_avro_consume_to_file( consumed_messages = get_consumed_messages(working_dir, True) assert number_of_consumer_messages == 10 - assert len(consumed_messages) == 10 assert produced_messages == consumed_messages From 905da8298d60056a1e02b2be019ac96a9f3f7bed Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Tue, 16 Jul 2019 17:52:08 +0200 Subject: [PATCH 79/94] Fix tests --- tests/integration/test_clients.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_clients.py b/tests/integration/test_clients.py index b1467826..0b57a901 100644 --- a/tests/integration/test_clients.py +++ b/tests/integration/test_clients.py @@ -133,7 +133,7 @@ def produce_test_messages_with_avro(avro_producer: AvroProducer, topic: Tuple[st value = {"first": "Firstname", "last": "Lastname"} messages.append(KafkaMessage(json.dumps(key), json.dumps(value), 0, key_schema, value_schema)) avro_producer.produce( - topic=topic, key=key, value=value, key_schema=key_schema, value_schema=value_schema, partition=partition + topic=topic_name, key=key, value=value, key_schema=key_schema, value_schema=value_schema, partition=partition ) avro_producer.flush() return messages From 74fd5a8c80e5fbfaf546ce494d011cb3179fca74 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Tue, 16 Jul 2019 18:13:28 +0200 Subject: [PATCH 80/94] Fix tests --- tests/integration/test_clients.py | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_clients.py b/tests/integration/test_clients.py index 0b57a901..adf2e770 100644 --- a/tests/integration/test_clients.py +++ b/tests/integration/test_clients.py @@ -111,7 +111,7 @@ def produce_test_messages(producer: ConfluenceProducer, topic: Tuple[str, int]) topic_name, num_partitions = topic messages = [] for i in range(10): - partition = random.randint(0, num_partitions) + partition = random.randrange(0, num_partitions) random_value = "".join(random.choices(ascii_letters, k=5)) message = KafkaMessage(str(i), random_value, 0) messages.append(message) @@ -128,12 +128,17 @@ def produce_test_messages_with_avro(avro_producer: AvroProducer, topic: Tuple[st value_schema = load_schema(file.read()) messages = [] for i in range(10): - partition = random.randint(0, num_partitions) + partition = random.randrange(0, num_partitions) key = {"id": str(i)} value = {"first": "Firstname", "last": "Lastname"} messages.append(KafkaMessage(json.dumps(key), json.dumps(value), 0, key_schema, value_schema)) avro_producer.produce( - topic=topic_name, key=key, value=value, key_schema=key_schema, value_schema=value_schema, partition=partition + topic=topic_name, + key=key, + value=value, + key_schema=key_schema, + value_schema=value_schema, + partition=partition, ) avro_producer.flush() return messages From 489184943f6687436e3e63d03a3d5bc7f4b8ea8f Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Tue, 16 Jul 2019 19:00:55 +0200 Subject: [PATCH 81/94] Fix tests --- docker-compose.test.yml | 2 +- tests/conftest.py | 36 ++++++++++++++----------------- tests/integration/test_clients.py | 10 ++++----- 3 files changed, 22 insertions(+), 26 deletions(-) diff --git a/docker-compose.test.yml b/docker-compose.test.yml index 8bd6ac22..4d3ebc44 100644 --- a/docker-compose.test.yml +++ b/docker-compose.test.yml @@ -12,4 +12,4 @@ services: command: > -c "(until (kafkacat -b kafka:9093 -X debug=all -L); do sleep 5s; done) \ - && pytest tests/ --integration" \ No newline at end of file + && python3 -u -m pytest -v -x tests/ --integration" \ No newline at end of file diff --git a/tests/conftest.py b/tests/conftest.py index 38a441cc..d1fe054d 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -65,34 +65,32 @@ def changed_topic_object(cluster, topic: str): @pytest.fixture() -def topic(confluent_admin_client: AdminClient) -> str: +def topic(topic_factory: Callable[[int, str], Tuple[str, int]]) -> Iterable[str]: topic_id = "".join(random.choices(ascii_letters, k=5)) - yield topic_factory(1, topic_id)[0] + for topic, _ in topic_factory(1, topic_id): + yield topic -@pytest.fixture(params=[1, 10], ids=["1", "10"]) -def source_topic(request, topic_factory: Callable[[int, str], Tuple[str, int]]) -> Iterable[Tuple[str, int]]: +@pytest.fixture() +def source_topic(num_partitions: int, topic_factory: Callable[[int, str], Tuple[str, int]]) -> Iterable[Tuple[str, int]]: topic_id = "".join(random.choices(ascii_letters, k=5)) - yield from topic_factory(request.param, topic_id) + yield from topic_factory(num_partitions, topic_id) -@pytest.fixture(params=[1, 10], ids=["1", "10"]) -def target_topic(request, topic_factory: Callable[[int, str], Tuple[str, int]]) -> Iterable[Tuple[str, int]]: +@pytest.fixture() +def target_topic(num_partitions: int, topic_factory: Callable[[int, str], Tuple[str, int]]) -> Iterable[Tuple[str, int]]: topic_id = "".join(random.choices(ascii_letters, k=5)) - yield from topic_factory(request.param, topic_id) + yield from topic_factory(num_partitions, topic_id) -@pytest.fixture() -def topic_factory(confluent_admin_client: AdminClient) -> Callable[[int, str], Tuple[str, int]]: - """ - Creates a kafka topic consisting of a random 5 character string and being partition into 1, 5 or 10 partitions. - Then it yields the tuple (topic, n_partitions). +@pytest.fixture(params=[1, 10], ids=["num_partitions=1", "num_partitions=10"]) +def num_partitions(request) -> int: + return request.param - Prints topic information before and after topic was used by a test. - :return: Topic and number of partitions within it. - """ - def factory(partitions: int, topic_id: str) -> Tuple[str, int]: +@pytest.fixture() +def topic_factory(confluent_admin_client: AdminClient) -> Callable[[int, str], Iterable[Tuple[str, int]]]: + def factory(partitions: int, topic_id: str) -> Iterable[Tuple[str, int]]: future: Future = confluent_admin_client.create_topics( [NewTopic(topic_id, num_partitions=partitions, replication_factor=1)] )[topic_id] @@ -103,9 +101,7 @@ def factory(partitions: int, topic_id: str) -> Tuple[str, int]: yield (topic_id, partitions) - topics = confluent_admin_client.list_topics(timeout=5).topics.keys() - if topic_id in topics: - confluent_admin_client.delete_topics([topic_id]).popitem() + confluent_admin_client.delete_topics([topic_id]).popitem() return factory diff --git a/tests/integration/test_clients.py b/tests/integration/test_clients.py index adf2e770..c9af4537 100644 --- a/tests/integration/test_clients.py +++ b/tests/integration/test_clients.py @@ -63,7 +63,7 @@ def test_plain_text_consume_and_produce( file_consumer.consume(10) producer = FileProducer(working_dir) - producer.produce(source_topic_id) + producer.produce(target_topic_id) # Check assertions: assertion_check_directory = tmpdir_factory.mktemp("assertion_check_directory") @@ -113,9 +113,9 @@ def produce_test_messages(producer: ConfluenceProducer, topic: Tuple[str, int]) for i in range(10): partition = random.randrange(0, num_partitions) random_value = "".join(random.choices(ascii_letters, k=5)) - message = KafkaMessage(str(i), random_value, 0) + message = KafkaMessage(str(i), random_value, partition) messages.append(message) - producer.produce(topic=topic_name, key=message.key, value=message.value, partition=partition) + producer.produce(topic=topic_name, key=message.key, value=message.value, partition=message.partition) producer.flush() return messages @@ -131,7 +131,7 @@ def produce_test_messages_with_avro(avro_producer: AvroProducer, topic: Tuple[st partition = random.randrange(0, num_partitions) key = {"id": str(i)} value = {"first": "Firstname", "last": "Lastname"} - messages.append(KafkaMessage(json.dumps(key), json.dumps(value), 0, key_schema, value_schema)) + messages.append(KafkaMessage(json.dumps(key), json.dumps(value), partition, key_schema, value_schema)) avro_producer.produce( topic=topic_name, key=key, @@ -156,4 +156,4 @@ def get_consumed_messages(directory, avro: bool) -> List[KafkaMessage]: stack.enter_context(file_reader) for message in file_reader.read_from_file(): consumed_messages.append(message) - return consumed_messages + return sorted(consumed_messages, key=(lambda msg: msg.key)) From 60f47d9a74ceed00984d1a38e6a48f8f0e86f7c2 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Tue, 16 Jul 2019 19:29:31 +0200 Subject: [PATCH 82/94] Update repo config and fix test --- README.md | 11 +++++++++++ pyproject.toml | 4 ++++ tests/conftest.py | 8 ++++++-- tests/integration/test_topic_controller.py | 8 ++++---- tests/test_samples/value_schema.avsc | 2 +- 5 files changed, 26 insertions(+), 7 deletions(-) create mode 100644 pyproject.toml diff --git a/README.md b/README.md index ee7ecb97..db2e13f9 100644 --- a/README.md +++ b/README.md @@ -134,6 +134,17 @@ While this `docker-compose` stack is up, you can run the tests from the CLI via Alternatively, you can also run the entire test suite, without needing to setup the development environment, in docker compose via `docker-compose -f docker-compose.yml -f docker-compose.test.yml` +### Pre Commit Hooks + +To install pre commit hooks run: + +``` +pip install pre-commit +pre-commit install +pre-commit install-hooks` +`` + + ## Alternatives - [LinkedIn KafkaTools](https://github.com/linkedin/kafka-tools) diff --git a/pyproject.toml b/pyproject.toml new file mode 100644 index 00000000..3af9d6c6 --- /dev/null +++ b/pyproject.toml @@ -0,0 +1,4 @@ +[tool.black] +line-length = 119 +target_version = ['py36'] +include = '\.pyi?$' \ No newline at end of file diff --git a/tests/conftest.py b/tests/conftest.py index d1fe054d..692c83aa 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -72,13 +72,17 @@ def topic(topic_factory: Callable[[int, str], Tuple[str, int]]) -> Iterable[str] @pytest.fixture() -def source_topic(num_partitions: int, topic_factory: Callable[[int, str], Tuple[str, int]]) -> Iterable[Tuple[str, int]]: +def source_topic( + num_partitions: int, topic_factory: Callable[[int, str], Tuple[str, int]] +) -> Iterable[Tuple[str, int]]: topic_id = "".join(random.choices(ascii_letters, k=5)) yield from topic_factory(num_partitions, topic_id) @pytest.fixture() -def target_topic(num_partitions: int, topic_factory: Callable[[int, str], Tuple[str, int]]) -> Iterable[Tuple[str, int]]: +def target_topic( + num_partitions: int, topic_factory: Callable[[int, str], Tuple[str, int]] +) -> Iterable[Tuple[str, int]]: topic_id = "".join(random.choices(ascii_letters, k=5)) yield from topic_factory(num_partitions, topic_id) diff --git a/tests/integration/test_topic_controller.py b/tests/integration/test_topic_controller.py index 3030172e..846f1fdb 100644 --- a/tests/integration/test_topic_controller.py +++ b/tests/integration/test_topic_controller.py @@ -11,15 +11,15 @@ def topic_controller(cluster): @pytest.mark.integration def test_topic_creation_works( - topic_controller: TopicController, confluent_admin_client: confluent_kafka.admin.AdminClient, topic_id: str + topic_controller: TopicController, confluent_admin_client: confluent_kafka.admin.AdminClient, topic: str ): topics = confluent_admin_client.list_topics(timeout=5).topics.keys() - assert topic_id not in topics - topic_controller.create_topics([topic_controller.get_topic(topic_id, replication_factor=1)]) + assert topic not in topics + topic_controller.create_topics([topic_controller.get_topic(topic, replication_factor=1)]) # invalidate cache confluent_admin_client.poll(timeout=1) topics = confluent_admin_client.list_topics(timeout=5).topics.keys() - assert topic_id in topics + assert topic in topics @pytest.mark.integration diff --git a/tests/test_samples/value_schema.avsc b/tests/test_samples/value_schema.avsc index 94d6320d..16567e60 100644 --- a/tests/test_samples/value_schema.avsc +++ b/tests/test_samples/value_schema.avsc @@ -6,4 +6,4 @@ { "name": "first", "type": "string" }, { "name": "last", "type": "string" } ] -} \ No newline at end of file +} \ No newline at end of file From a9fa37bfe1d10630ff6290afe01213afd020d824 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Wed, 17 Jul 2019 17:04:43 +0200 Subject: [PATCH 83/94] Use topic_id fixture --- tests/conftest.py | 8 ++++++++ tests/integration/test_topic_controller.py | 8 ++++---- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/tests/conftest.py b/tests/conftest.py index 692c83aa..b054e191 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -54,6 +54,14 @@ def test_config(test_config_path, request): yield esque_config +@pytest.fixture() +def topic_id(confluent_admin_client) -> str: + yield "".join(random.choices(ascii_letters, k=5)) + topics = confluent_admin_client.list_topics(timeout=5).topics.keys() + if topic_id in topics: + confluent_admin_client.delete_topics([topic_id]).popitem() + + @pytest.fixture() def topic_object(cluster, topic: str): yield TopicController(cluster).get_topic(topic) diff --git a/tests/integration/test_topic_controller.py b/tests/integration/test_topic_controller.py index 846f1fdb..3030172e 100644 --- a/tests/integration/test_topic_controller.py +++ b/tests/integration/test_topic_controller.py @@ -11,15 +11,15 @@ def topic_controller(cluster): @pytest.mark.integration def test_topic_creation_works( - topic_controller: TopicController, confluent_admin_client: confluent_kafka.admin.AdminClient, topic: str + topic_controller: TopicController, confluent_admin_client: confluent_kafka.admin.AdminClient, topic_id: str ): topics = confluent_admin_client.list_topics(timeout=5).topics.keys() - assert topic not in topics - topic_controller.create_topics([topic_controller.get_topic(topic, replication_factor=1)]) + assert topic_id not in topics + topic_controller.create_topics([topic_controller.get_topic(topic_id, replication_factor=1)]) # invalidate cache confluent_admin_client.poll(timeout=1) topics = confluent_admin_client.list_topics(timeout=5).topics.keys() - assert topic in topics + assert topic_id in topics @pytest.mark.integration From e334200bf1c2d7b29a2ab0019d0d5f26753546e1 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Fri, 19 Jul 2019 19:29:25 +0200 Subject: [PATCH 84/94] Remove unnecessary usage of keys() --- esque/clients.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/esque/clients.py b/esque/clients.py index a71a6b8b..bde7a0a7 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -100,7 +100,7 @@ def consume(self, amount: int) -> int: if message is None: return counter - if message.partition() not in file_writers.keys(): + if message.partition() not in file_writers: file_writer = self.get_file_writer((self.working_dir / f"partition_{message.partition()}")) stack.enter_context(file_writer) file_writers[message.partition()] = file_writer From 2f7d8d3a7a256e3b20d25d10219251ad3826c251 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Fri, 19 Jul 2019 19:36:57 +0200 Subject: [PATCH 85/94] Refactor clients.py --- esque/clients.py | 34 +++++++++++++++++----------------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/esque/clients.py b/esque/clients.py index bde7a0a7..49b8ac0a 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -101,18 +101,19 @@ def consume(self, amount: int) -> int: return counter if message.partition() not in file_writers: - file_writer = self.get_file_writer((self.working_dir / f"partition_{message.partition()}")) + partition = message.partition() + file_writer = self.get_file_writer(partition) stack.enter_context(file_writer) - file_writers[message.partition()] = file_writer + file_writers[partition] = file_writer - file_writer = file_writers[message.partition()] + file_writer = file_writers[partition] file_writer.write_message_to_file(message) counter += 1 return counter - def get_file_writer(self, directory: pathlib.Path) -> FileWriter: - return PlainTextFileWriter(directory) + def get_file_writer(self, partition: int) -> FileWriter: + return PlainTextFileWriter((self.working_dir / f"partition_{partition}")) class AvroFileConsumer(FileConsumer): @@ -120,8 +121,8 @@ def __init__(self, group_id: str, topic_name: str, working_dir: pathlib.Path, la super().__init__(group_id, topic_name, working_dir, last) self.schema_registry_client = SchemaRegistryClient(Config().schema_registry) - def get_file_writer(self, directory: pathlib.Path) -> FileWriter: - return AvroFileWriter(directory, self.schema_registry_client) + def get_file_writer(self, partition: int) -> FileWriter: + return AvroFileWriter((self.working_dir / f"partition_{partition}"), self.schema_registry_client) class Producer(ABC): @@ -159,21 +160,20 @@ def __init__(self, working_dir: pathlib.Path): def produce(self, topic_name: str) -> int: path_list = glob(str(self.working_dir / "partition_*")) counter = 0 - with ExitStack() as stack: - for partition_path in path_list: - file_reader = self.get_file_reader(pathlib.Path(partition_path)) - stack.enter_context(file_reader) + for partition_path in path_list: + file_reader = self.get_file_reader(pathlib.Path(partition_path)) + with file_reader: for message in file_reader.read_from_file(): self.produce_message(topic_name, message) counter += 1 - while True: - left_messages = self._producer.flush(1) - if left_messages == 0: - break - click.echo(f"Still {left_messages} messages left, flushing...") + while True: + left_messages = self._producer.flush(1) + if left_messages == 0: + break + click.echo(f"Still {left_messages} messages left, flushing...") - return counter + return counter def get_file_reader(self, directory: pathlib.Path) -> FileReader: return PlainTextFileReader(directory) From 426888f30eb7bf96f43b79432fe93a989e42eb47 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Fri, 19 Jul 2019 19:37:14 +0200 Subject: [PATCH 86/94] Remove line-length in travis.yml for black --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 34b053dc..9976c395 100644 --- a/.travis.yml +++ b/.travis.yml @@ -35,7 +35,7 @@ matrix: before_install: [] python: 3.7 env: - - TEST_CMD="black --check --verbose --line-length=119 ." + - TEST_CMD="black --check --verbose ." - name: flake8 python: 3.7 before_install: [] From 4ebcfe8df561a4f832ea2f31dbee9abf5c77fe1c Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Fri, 19 Jul 2019 19:37:41 +0200 Subject: [PATCH 87/94] Fix Readme --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index db2e13f9..2f8c15dd 100644 --- a/README.md +++ b/README.md @@ -141,8 +141,8 @@ To install pre commit hooks run: ``` pip install pre-commit pre-commit install -pre-commit install-hooks` -`` +pre-commit install-hooks +``` ## Alternatives From 2e5adf13fc2f33177267f8364faf0571998e33f7 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Fri, 19 Jul 2019 19:57:38 +0200 Subject: [PATCH 88/94] Simplify consume_single_message --- esque/clients.py | 28 +++++++--------------------- 1 file changed, 7 insertions(+), 21 deletions(-) diff --git a/esque/clients.py b/esque/clients.py index 49b8ac0a..9d2c0c9a 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -50,33 +50,19 @@ def _subscribe(self, topic: str) -> None: def consume(self, amount: int) -> int: pass - def _consume_single_message(self) -> Optional[Message]: - poll_limit = 10 - counter = 0 - while counter < poll_limit: - message = self._consumer.poll(timeout=10) - if message is None: - counter += 1 - continue - if message.error() is not None: - if message.error().code() == KafkaError._PARTITION_EOF: - print("\nEnd of partition reached!".format(**locals())) - break - else: - raise RuntimeError(message.error().str()) - raise_for_message(message) - return message + def _consume_single_message(self, timeout=10) -> Optional[Message]: + message = self._consumer.poll(timeout=timeout) + raise_for_message(message) + return message class PingConsumer(AbstractConsumer): def consume(self, amount: int) -> Optional[Tuple[str, int]]: - msg = self._consumer.consume(timeout=10)[0] - - raise_for_message(msg) + message = self._consume_single_message() - msg_sent_at = pendulum.from_timestamp(float(msg.value())) + msg_sent_at = pendulum.from_timestamp(float(message.value())) delta_sent = pendulum.now() - msg_sent_at - return msg.key(), delta_sent.microseconds / 1000 + return message.key(), delta_sent.microseconds / 1000 class FileConsumer(AbstractConsumer): From e1134419ba50b41ed52c66cf465ab94a38421782 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Fri, 19 Jul 2019 19:57:59 +0200 Subject: [PATCH 89/94] Simplify IoHandler --- esque/clients.py | 3 +-- esque/message.py | 18 ++++++++++-------- 2 files changed, 11 insertions(+), 10 deletions(-) diff --git a/esque/clients.py b/esque/clients.py index 9d2c0c9a..d4603b7c 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -147,8 +147,7 @@ def produce(self, topic_name: str) -> int: path_list = glob(str(self.working_dir / "partition_*")) counter = 0 for partition_path in path_list: - file_reader = self.get_file_reader(pathlib.Path(partition_path)) - with file_reader: + with self.get_file_reader(pathlib.Path(partition_path)) as file_reader: for message in file_reader.read_from_file(): self.produce_message(topic_name, message) counter += 1 diff --git a/esque/message.py b/esque/message.py index b0a48e5e..8d0503ab 100644 --- a/esque/message.py +++ b/esque/message.py @@ -30,25 +30,31 @@ def __enter__(self): self.directory.mkdir() self.file = (self.directory / self.file_name).open(self.open_mode) + return self + def __exit__(self, exc_type, exc_val, exc_tb): self.file.close() class FileWriter(IOHandler): + def __init__(self, directory: pathlib.Path): + super().__init__(directory) + self.open_mode = "w+" + def write_message_to_file(self, message: Message): pass class FileReader(IOHandler): + def __init__(self, directory: pathlib.Path): + super().__init__(directory) + self.open_mode = "r" + def read_from_file(self) -> Iterable[KafkaMessage]: pass class PlainTextFileWriter(FileWriter): - def __init__(self, directory: pathlib.Path): - super().__init__(directory) - self.open_mode = "w+" - def write_message_to_file(self, message: Message): decoded_message = decode_message(message) serializable_message = { @@ -60,10 +66,6 @@ def write_message_to_file(self, message: Message): class PlainTextFileReader(FileReader): - def __init__(self, directory: pathlib.Path): - super().__init__(directory) - self.open_mode = "r" - def read_from_file(self) -> Iterable[KafkaMessage]: for line in self.file: try: From ee4fb43b89b5a304ee87ec3fc1e5916a9689b9ec Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Fri, 19 Jul 2019 19:58:21 +0200 Subject: [PATCH 90/94] Remove unnecessary format --- esque/schemaregistry.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/esque/schemaregistry.py b/esque/schemaregistry.py index 4902b519..9f0f5933 100644 --- a/esque/schemaregistry.py +++ b/esque/schemaregistry.py @@ -16,7 +16,7 @@ def __init__(self, schema_registry_uri: str): @lru_cache(maxsize=100) def get_schema_from_id(self, schema_id: int) -> SchemaPair: url = f"{self.schema_registry_uri}/schemas/ids/{schema_id}" - response = requests.get(url.format(**locals())) + response = requests.get(url) response.raise_for_status() schema: Dict = json.loads(response.json()["schema"]) return SchemaPair(schema, fastavro.schema.parse_schema(schema)) From 97fa771d5d25507d877bc631f4a7b34912e4a7b5 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Fri, 19 Jul 2019 20:11:39 +0200 Subject: [PATCH 91/94] Remove unused import --- esque/clients.py | 1 - 1 file changed, 1 deletion(-) diff --git a/esque/clients.py b/esque/clients.py index d4603b7c..649c857d 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -9,7 +9,6 @@ import pendulum from confluent_kafka import Message from confluent_kafka.avro import AvroProducer -from confluent_kafka.cimpl import KafkaError from esque.avromessage import AvroFileReader, AvroFileWriter from esque.config import Config From 2b857aafa240cbf72a17a950e866a054c86193cc Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Fri, 19 Jul 2019 23:21:54 +0200 Subject: [PATCH 92/94] Add help description for --last/--first --- esque/cli/commands.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/esque/cli/commands.py b/esque/cli/commands.py index f6199f67..6136e7ec 100644 --- a/esque/cli/commands.py +++ b/esque/cli/commands.py @@ -238,7 +238,7 @@ def get_topics(state, topic): @click.option("-f", "--from", "from_context", help="Source Context", type=click.STRING, required=True) @click.option("-t", "--to", "to_context", help="Destination context", type=click.STRING, required=True) @click.option("-n", "--numbers", help="Number of messages", type=click.INT, required=True) -@click.option("--last/--first", default=False) +@click.option("--last/--first", help="Start consuming from the earliest or latest offset in the topic.", default=False) @click.option("-a", "--avro", help="Set this flag if the topic contains avro data", default=False, is_flag=True) @click.option( "-k", From a3dfbb94ed0fd5a6974ed2c0ce772f9ce9b71865 Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 22 Jul 2019 14:45:58 +0200 Subject: [PATCH 93/94] Re-add retry mechanism for consume_single_message --- esque/clients.py | 13 ++++++++++--- esque/errors.py | 2 +- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/esque/clients.py b/esque/clients.py index 649c857d..93edaceb 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -50,9 +50,16 @@ def consume(self, amount: int) -> int: pass def _consume_single_message(self, timeout=10) -> Optional[Message]: - message = self._consumer.poll(timeout=timeout) - raise_for_message(message) - return message + poll_limit = 10 + counter = 0 + while counter < poll_limit: + message = self._consumer.poll(timeout=timeout) + if message is None: + counter += 1 + continue + if message.error() is not None: + raise_for_message(message) + return message class PingConsumer(AbstractConsumer): diff --git a/esque/errors.py b/esque/errors.py index 6913b47b..4d4def52 100644 --- a/esque/errors.py +++ b/esque/errors.py @@ -25,7 +25,7 @@ def raise_for_kafka_error(err: KafkaError): def raise_for_message(message: Message): if message is None: - raise MessageEmptyException + raise MessageEmptyException(-1, message) elif message.error() is not None: raise_for_kafka_error(message.error()) From 1bd1c779d2726cff7e51e73ad43e368f6c87df1b Mon Sep 17 00:00:00 2001 From: Kevin Kreitner Date: Mon, 22 Jul 2019 15:17:03 +0200 Subject: [PATCH 94/94] Catch MessageEmptyException and return counter for Consumer --- esque/clients.py | 20 +++++++------------- esque/errors.py | 5 +++-- 2 files changed, 10 insertions(+), 15 deletions(-) diff --git a/esque/clients.py b/esque/clients.py index 93edaceb..a0295ace 100644 --- a/esque/clients.py +++ b/esque/clients.py @@ -12,7 +12,7 @@ from esque.avromessage import AvroFileReader, AvroFileWriter from esque.config import Config -from esque.errors import raise_for_kafka_error, raise_for_message +from esque.errors import raise_for_kafka_error, raise_for_message, MessageEmptyException from esque.helpers import delivery_callback, delta_t from esque.message import KafkaMessage, PlainTextFileReader, PlainTextFileWriter, FileReader, FileWriter from esque.schemaregistry import SchemaRegistryClient @@ -50,16 +50,9 @@ def consume(self, amount: int) -> int: pass def _consume_single_message(self, timeout=10) -> Optional[Message]: - poll_limit = 10 - counter = 0 - while counter < poll_limit: - message = self._consumer.poll(timeout=timeout) - if message is None: - counter += 1 - continue - if message.error() is not None: - raise_for_message(message) - return message + message = self._consumer.poll(timeout=timeout) + raise_for_message(message) + return message class PingConsumer(AbstractConsumer): @@ -88,8 +81,9 @@ def consume(self, amount: int) -> int: file_writers = {} with ExitStack() as stack: while counter < amount: - message = self._consume_single_message() - if message is None: + try: + message = self._consume_single_message() + except MessageEmptyException: return counter if message.partition() not in file_writers: diff --git a/esque/errors.py b/esque/errors.py index 4d4def52..0d90b427 100644 --- a/esque/errors.py +++ b/esque/errors.py @@ -25,7 +25,7 @@ def raise_for_kafka_error(err: KafkaError): def raise_for_message(message: Message): if message is None: - raise MessageEmptyException(-1, message) + raise MessageEmptyException elif message.error() is not None: raise_for_kafka_error(message.error()) @@ -49,7 +49,8 @@ class ContextNotDefinedException(Exception): class MessageEmptyException(KafkaException): - pass + def __init__(self): + super().__init__(-185, None) class TopicAlreadyExistsException(KafkaException):