diff --git a/README.rst b/README.rst index 0a66eb1c7..56c0ae89a 100644 --- a/README.rst +++ b/README.rst @@ -461,7 +461,15 @@ Keys to take special care are the ones needed to configure Kafka and advertised_ * - ``master_election_strategy`` - ``lowest`` - Decides on what basis the Karapace cluster master is chosen (only relevant in a multi node setup) - + * - ``metrics_extended`` + - ``true`` + - Enable extended metrics. Extended metrics: connections_active, [request|response]_size + * - ``statsd_host`` + - ``127.0.0.1`` + - Host of statsd server + * - ``statsd_port`` + - ``8125`` + - Port of statsd server Authentication and authorization of Karapace Schema Registry REST API ===================================================================== diff --git a/karapace.config.json b/karapace.config.json index 55303ff4d..f07af6710 100644 --- a/karapace.config.json +++ b/karapace.config.json @@ -27,5 +27,12 @@ "registry_authfile": null, "topic_name": "_schemas", "protobuf_runtime_directory": "runtime", - "session_timeout_ms": 10000 + "session_timeout_ms": 10000, + "stats_service": "statsd", + "metrics_extended": true, + "statsd_host": "127.0.0.1", + "statsd_port": 8125, + "prometheus_host": "127.0.0.1", + "prometheus_port": 8005, + } diff --git a/karapace/base_stats.py b/karapace/base_stats.py new file mode 100644 index 000000000..b6fe87a0c --- /dev/null +++ b/karapace/base_stats.py @@ -0,0 +1,57 @@ +""" +karapace - basestats + +Supports base class for statsd and prometheus protocols: + +Copyright (c) 2023 Aiven Ltd +See LICENSE for details +""" +from __future__ import annotations + +from abc import ABC, abstractmethod +from contextlib import contextmanager +from karapace.config import Config +from karapace.sentry import get_sentry_client +from typing import Final, Iterator + +import time + + +class StatsClient(ABC): + @abstractmethod + def __init__( + self, + config: Config, + ) -> None: + self.sentry_client: Final = get_sentry_client(sentry_config=config.get("sentry", None)) + + @contextmanager + def timing_manager(self, metric: str, tags: dict | None = None) -> Iterator[None]: + start_time = time.monotonic() + yield + self.timing(metric, time.monotonic() - start_time, tags) + + @abstractmethod + def gauge(self, metric: str, value: float, tags: dict | None = None) -> None: + pass + + @abstractmethod + def increase(self, metric: str, inc_value: int = 1, tags: dict | None = None) -> None: + pass + + @abstractmethod + def timing(self, metric: str, value: float, tags: dict | None = None) -> None: + pass + + def unexpected_exception(self, ex: Exception, where: str, tags: dict | None = None) -> None: + all_tags = { + "exception": ex.__class__.__name__, + "where": where, + } + all_tags.update(tags or {}) + self.increase("exception", tags=all_tags) + scope_args = {**(tags or {}), "where": where} + self.sentry_client.unexpected_exception(error=ex, where=where, tags=scope_args) + + def close(self) -> None: + self.sentry_client.close() diff --git a/karapace/config.py b/karapace/config.py index f426dfcc5..b3016eb07 100644 --- a/karapace/config.py +++ b/karapace/config.py @@ -75,6 +75,12 @@ class Config(TypedDict): karapace_registry: bool master_election_strategy: str protobuf_runtime_directory: str + stats_service: str + metrics_extended: bool + statsd_host: str + statsd_port: int + prometheus_host: str | None + prometheus_port: int | None sentry: NotRequired[Mapping[str, object]] tags: NotRequired[Mapping[str, object]] @@ -144,6 +150,12 @@ class ConfigDefaults(Config, total=False): "karapace_registry": False, "master_election_strategy": "lowest", "protobuf_runtime_directory": "runtime", + "stats_service": "statsd", + "metrics_extended": True, + "statsd_host": "127.0.0.1", + "statsd_port": 8125, + "prometheus_host": "127.0.0.1", + "prometheus_port": 8005, } SECRET_CONFIG_OPTIONS = [SASL_PLAIN_PASSWORD] diff --git a/karapace/metrics.py b/karapace/metrics.py new file mode 100644 index 000000000..03ee39718 --- /dev/null +++ b/karapace/metrics.py @@ -0,0 +1,127 @@ +""" +karapace - metrics +Supports collection of system metrics +list of supported metrics: +connections-active - The number of active HTTP(S) connections to server. + Data collected inside aiohttp request handler. + +Copyright (c) 2023 Aiven Ltd +See LICENSE for details +""" +from __future__ import annotations + +from karapace.base_stats import StatsClient +from karapace.config import Config +from karapace.prometheus import PrometheusClient +from karapace.statsd import StatsdClient + +import os +import psutil +import schedule +import threading +import time + + +class MetricsException(Exception): + pass + + +class Singleton(type): + _instance: Singleton | None = None + + def __call__(cls, *args: str, **kwargs: int) -> Singleton: + if cls._instance is None: + instance = super().__call__(*args, **kwargs) + cls._instance = instance + return cls._instance + + +class Metrics(metaclass=Singleton): + def __init__(self) -> None: + self.stats_client: StatsClient + self.is_ready = False + self.stop_event = threading.Event() + self.worker_thread = threading.Thread(target=self.worker) + self.lock = threading.Lock() + + def setup(self, config: Config) -> None: + stats_service = config.get("stats_service") + if not config.get("metrics_extended"): + return + if stats_service == "statsd": + self.stats_client = StatsdClient(config=config) + elif stats_service == "prometheus": + self.stats_client = PrometheusClient(config=config) + else: + raise MetricsException('Config variable "stats_service" is not defined') + with self.lock: + if self.is_ready: + return + self.is_ready = True + + schedule.every(10).seconds.do(self.connections) + self.worker_thread.start() + + def request(self, size: int) -> None: + if not self.is_ready or self.stats_client is None: + return + if not isinstance(self.stats_client, StatsClient): + raise RuntimeError("no StatsClient available") + self.stats_client.gauge("request-size", size) + + def response(self, size: int) -> None: + if not self.is_ready or self.stats_client is None: + return + if not isinstance(self.stats_client, StatsClient): + raise RuntimeError("no StatsClient available") + self.stats_client.gauge("response-size", size) + + def are_we_master(self, is_master: bool) -> None: + if not self.is_ready or self.stats_client is None: + return + if not isinstance(self.stats_client, StatsClient): + raise RuntimeError("no StatsClient available") + self.stats_client.gauge("master-slave-role", int(is_master)) + + def latency(self, latency_ms: float) -> None: + if not self.is_ready or self.stats_client is None: + return + if not isinstance(self.stats_client, StatsClient): + raise RuntimeError("no StatsClient available") + self.stats_client.timing("latency_ms", latency_ms) + + def error(self) -> None: + if not self.is_ready or self.stats_client is None: + return + if not isinstance(self.stats_client, StatsClient): + raise RuntimeError("no StatsClient available") + self.stats_client.increase("error_total", 1) + + def connections(self) -> None: + if not self.is_ready or self.stats_client is None: + return + if not isinstance(self.stats_client, StatsClient): + raise RuntimeError("no StatsClient available") + connections = 0 + karapace_proc = psutil.Process(os.getpid()) + + for conn in karapace_proc.connections(kind="tcp"): + if conn.laddr and conn.status == "ESTABLISHED": + connections += 1 + self.stats_client.gauge("connections-active", connections) + + def worker(self) -> None: + while True: + if self.stop_event.is_set(): + break + schedule.run_pending() + time.sleep(1) + + def cleanup(self) -> None: + if self.stats_client: + self.stats_client.close() + if not self.is_ready: + return + self.stop_event.set() + if self.worker_thread.is_alive(): + self.worker_thread.join() diff --git a/karapace/prometheus.py b/karapace/prometheus.py new file mode 100644 index 000000000..2ecd78c37 --- /dev/null +++ b/karapace/prometheus.py @@ -0,0 +1,69 @@ +""" +karapace - prometheus + +Supports telegraf's statsd protocol extension for 'key=value' tags: + + https://github.com/influxdata/telegraf/tree/master/plugins/inputs/statsd + +Copyright (c) 2023 Aiven Ltd +See LICENSE for details +""" +from __future__ import annotations + +from karapace.base_stats import StatsClient +from karapace.config import Config +from prometheus_client import Counter, Gauge, start_http_server, Summary +from typing import Final + +import logging + +LOG = logging.getLogger(__name__) +HOST: Final = "127.0.0.1" +PORT: Final = 8005 + + +class PrometheusException(Exception): + pass + + +class PrometheusClient(StatsClient): + server_is_active = False + + def __init__(self, config: Config, host: str = HOST, port: int = PORT) -> None: + super().__init__(config) + + _host = config.get("prometheus_host") if "prometheus_host" in config else host + _port = config.get("prometheus_port") if "prometheus_port" in config else port + if _host is None: + raise PrometheusException("prometheus_host host is undefined") + if _port is None: + raise PrometheusException("prometheus_host port is undefined") + if not self.server_is_active: + start_http_server(_port, _host) + self.server_is_active = True + else: + raise PrometheusException("Double instance of Prometheus interface") + self._gauge: dict[str, Gauge] = dict() + self._summary: dict[str, Summary] = dict() + self._counter: dict[str, Counter] = dict() + + def gauge(self, metric: str, value: float, tags: dict | None = None) -> None: + m = self._gauge.get(metric) + if m is None: + m = Gauge(metric, metric) + self._gauge[metric] = m + m.set(value) + + def increase(self, metric: str, inc_value: int = 1, tags: dict | None = None) -> None: + m = self._counter.get(metric) + if m is None: + m = Counter(metric, metric) + self._counter[metric] = m + m.inc(inc_value) + + def timing(self, metric: str, value: float, tags: dict | None = None) -> None: + m = self._summary.get(metric) + if m is None: + m = Summary(metric, metric) + self._summary[metric] = m + m.observe(value) diff --git a/karapace/rapu.py b/karapace/rapu.py index 2b9decf12..6ab3b2419 100644 --- a/karapace/rapu.py +++ b/karapace/rapu.py @@ -9,7 +9,7 @@ from accept_types import get_best_match from http import HTTPStatus from karapace.config import Config, create_server_ssl_context -from karapace.statsd import StatsClient +from karapace.metrics import Metrics from karapace.utils import json_decode, json_encode from karapace.version import __version__ from typing import Callable, Dict, NoReturn, Optional, overload, Union @@ -134,6 +134,8 @@ def __init__( if content_type: self.headers["Content-Type"] = content_type super().__init__(f"HTTPResponse {status.value}") + if not is_success(status): + Metrics().error() def ok(self) -> bool: """True if resposne has a 2xx status_code""" @@ -166,7 +168,8 @@ def __init__( self.app_request_metric = f"{app_name}_request" self.app = self._create_aiohttp_application(config=config) self.log = logging.getLogger(self.app_name) - self.stats = StatsClient(config=config) + Metrics().setup(config) + self.stats = Metrics().stats_client self.app.on_cleanup.append(self.close_by_app) self.not_ready_handler = not_ready_handler @@ -183,7 +186,7 @@ async def close(self) -> None: set as hook because the awaitables have to run inside the event loop created by the aiohttp library. """ - self.stats.close() + Metrics().cleanup() @staticmethod def cors_and_server_headers_for_request(*, request, origin="*"): # pylint: disable=unused-argument @@ -269,15 +272,21 @@ async def _handle_request( url=request.url, path_for_stats=path_for_stats, ) + try: if request.method == "OPTIONS": origin = request.headers.get("Origin") if not origin: raise HTTPResponse(body="OPTIONS missing Origin", status=HTTPStatus.BAD_REQUEST) headers = self.cors_and_server_headers_for_request(request=rapu_request, origin=origin) + raise HTTPResponse(body=b"", status=HTTPStatus.OK, headers=headers) body = await request.read() + if body: + Metrics().request(len(body)) + else: + Metrics().request(0) if json_request: if not body: raise HTTPResponse(body="Missing request JSON body", status=HTTPStatus.BAD_REQUEST) @@ -385,6 +394,7 @@ async def _handle_request( ) headers = {"Content-Type": "application/json"} resp = aiohttp.web.Response(body=body, status=status.value, headers=headers) + except asyncio.CancelledError: self.log.debug("Client closed connection") raise @@ -393,6 +403,8 @@ async def _handle_request( self.log.exception("Unexpected error handling user request: %s %s", request.method, request.url) resp = aiohttp.web.Response(text="Internal Server Error", status=HTTPStatus.INTERNAL_SERVER_ERROR.value) finally: + Metrics().response(resp.content_length) + Metrics().latency((time.monotonic() - start_time) * 1000) self.stats.timing( self.app_request_metric, time.monotonic() - start_time, diff --git a/karapace/schema_reader.py b/karapace/schema_reader.py index 3dec4a887..8b060a37b 100644 --- a/karapace/schema_reader.py +++ b/karapace/schema_reader.py @@ -26,11 +26,11 @@ from karapace.in_memory_database import InMemoryDatabase from karapace.key_format import is_key_in_canonical_format, KeyFormatter, KeyMode from karapace.master_coordinator import MasterCoordinator +from karapace.metrics import Metrics from karapace.offset_watcher import OffsetWatcher from karapace.protobuf.schema import ProtobufSchema from karapace.schema_models import parse_protobuf_schema_definition, SchemaType, TypedSchema, ValidatedTypedSchema from karapace.schema_references import LatestVersionReference, Reference, reference_from_mapping, Referents -from karapace.statsd import StatsClient from karapace.typing import JsonObject, ResolvedVersion, SchemaId, Subject from karapace.utils import json_decode, JSONDecodeError, KarapaceKafkaClient from threading import Event, Thread @@ -127,7 +127,7 @@ def __init__( self.topic_replication_factor = self.config["replication_factor"] self.consumer: KafkaConsumer | None = None self._offset_watcher = offset_watcher - self.stats = StatsClient(config=config) + self.stats = Metrics().stats_client # Thread synchronization objects # - offset is used by the REST API to wait until this thread has diff --git a/karapace/schema_registry.py b/karapace/schema_registry.py index 867eeb633..e29e167c7 100644 --- a/karapace/schema_registry.py +++ b/karapace/schema_registry.py @@ -25,6 +25,7 @@ from karapace.key_format import KeyFormatter from karapace.master_coordinator import MasterCoordinator from karapace.messaging import KarapaceProducer +from karapace.metrics import Metrics from karapace.offset_watcher import OffsetWatcher from karapace.schema_models import ParsedTypedSchema, SchemaType, SchemaVersion, TypedSchema, ValidatedTypedSchema from karapace.schema_reader import KafkaSchemaReader @@ -123,6 +124,7 @@ async def get_master(self, ignore_readiness: bool = False) -> tuple[bool, str | elif not ignore_readiness and self.schema_reader.ready is False: LOG.info("Schema reader isn't ready yet: %r", self.schema_reader.ready) else: + Metrics().are_we_master(are_we_master) return are_we_master, master_url await asyncio.sleep(1.0) diff --git a/karapace/statsd.py b/karapace/statsd.py index be37d92ef..36fdc24a6 100644 --- a/karapace/statsd.py +++ b/karapace/statsd.py @@ -10,38 +10,32 @@ """ from __future__ import annotations -from contextlib import contextmanager +from karapace.base_stats import StatsClient from karapace.config import Config -from karapace.sentry import get_sentry_client -from typing import Any, Final, Iterator +from typing import Any, Final import datetime import logging import socket -import time STATSD_HOST: Final = "127.0.0.1" STATSD_PORT: Final = 8125 LOG = logging.getLogger(__name__) -class StatsClient: +class StatsdClient(StatsClient): def __init__( self, config: Config, host: str = STATSD_HOST, port: int = STATSD_PORT, ) -> None: - self._dest_addr: Final = (host, port) - self._socket: Final = socket.socket(socket.AF_INET, socket.SOCK_DGRAM) + super().__init__(config) self._tags: Final = config.get("tags", {}) - self.sentry_client: Final = get_sentry_client(sentry_config=config.get("sentry", None)) - - @contextmanager - def timing_manager(self, metric: str, tags: dict | None = None) -> Iterator[None]: - start_time = time.monotonic() - yield - self.timing(metric, time.monotonic() - start_time, tags) + _host = config.get("statsd_host") if "statsd_host" in config else host + _port = config.get("statsd_port") if "statsd_port" in config else port + self._dest_addr: Final = (_host, _port) + self._socket: Final = socket.socket(socket.AF_INET, socket.SOCK_DGRAM) def gauge(self, metric: str, value: float, tags: dict | None = None) -> None: self._send(metric, b"g", value, tags) @@ -52,16 +46,6 @@ def increase(self, metric: str, inc_value: int = 1, tags: dict | None = None) -> def timing(self, metric: str, value: float, tags: dict | None = None) -> None: self._send(metric, b"ms", value, tags) - def unexpected_exception(self, ex: Exception, where: str, tags: dict | None = None) -> None: - all_tags = { - "exception": ex.__class__.__name__, - "where": where, - } - all_tags.update(tags or {}) - self.increase("exception", tags=all_tags) - scope_args = {**(tags or {}), "where": where} - self.sentry_client.unexpected_exception(error=ex, where=where, tags=scope_args) - def _send(self, metric: str, metric_type: bytes, value: Any, tags: dict | None) -> None: if None in self._dest_addr: # stats sending is disabled @@ -93,4 +77,4 @@ def _send(self, metric: str, metric_type: bytes, value: Any, tags: dict | None) def close(self) -> None: self._socket.close() - self.sentry_client.close() + super().close() diff --git a/requirements/requirements-dev.txt b/requirements/requirements-dev.txt index 3b2822d6b..f023fcc5b 100644 --- a/requirements/requirements-dev.txt +++ b/requirements/requirements-dev.txt @@ -14,11 +14,11 @@ aiosignal==1.3.1 # via # -r requirements.txt # aiohttp -anyio==3.7.0 +anyio==4.0.0 # via # -r requirements.txt # watchfiles -async-timeout==4.0.2 +async-timeout==4.0.3 # via # -r requirements.txt # aiohttp @@ -43,26 +43,32 @@ certifi==2023.7.22 # geventhttpclient # requests # sentry-sdk -charset-normalizer==3.1.0 +charset-normalizer==3.2.0 # via # -r requirements.txt # aiohttp # requests -click==8.1.3 +click==8.1.7 # via flask commonmark==0.9.1 # via # -r requirements.txt # rich -configargparse==1.5.3 +configargparse==1.7 # via locust -execnet==1.9.0 +exceptiongroup==1.1.3 + # via + # -r requirements.txt + # anyio + # hypothesis + # pytest +execnet==2.0.2 # via pytest-xdist fancycompleter==0.9.1 # via pdbpp -filelock==3.12.2 +filelock==3.12.3 # via -r requirements-dev.in -flask==2.3.2 +flask==2.3.3 # via # flask-basicauth # flask-cors @@ -71,20 +77,20 @@ flask-basicauth==0.2.0 # via locust flask-cors==4.0.0 # via locust -frozenlist==1.3.3 +frozenlist==1.4.0 # via # -r requirements.txt # aiohttp # aiosignal -gevent==22.10.2 +gevent==23.9.0.post1 # via # geventhttpclient # locust -geventhttpclient==2.0.9 +geventhttpclient==2.0.10 # via locust greenlet==2.0.2 # via gevent -hypothesis==6.82.7 +hypothesis==6.84.1 # via -r requirements-dev.in idna==3.4 # via @@ -92,6 +98,13 @@ idna==3.4 # anyio # requests # yarl +importlib-metadata==6.8.0 + # via flask +importlib-resources==6.0.1 + # via + # -r requirements.txt + # jsonschema + # jsonschema-specifications iniconfig==2.0.0 # via pytest isodate==0.6.1 @@ -100,7 +113,7 @@ itsdangerous==2.1.2 # via flask jinja2==3.1.2 # via flask -jsonschema==4.18.4 +jsonschema==4.19.0 # via -r requirements.txt jsonschema-specifications==2023.7.1 # via @@ -132,23 +145,30 @@ packaging==23.1 # pytest pdbpp==0.10.3 # via -r requirements-dev.in -pluggy==1.2.0 +pkgutil-resolve-name==1.3.10 + # via + # -r requirements.txt + # jsonschema +pluggy==1.3.0 # via pytest +prometheus-client==0.17.1 + # via -r requirements.txt protobuf==3.20.3 # via -r requirements.txt psutil==5.9.5 # via # -r requirements-dev.in + # -r requirements.txt # locust # pytest-xdist -pygments==2.15.1 +pygments==2.16.1 # via # -r requirements.txt # pdbpp # rich pyrepl==0.9.0 # via fancycompleter -pytest==7.4.0 +pytest==7.4.1 # via # -r requirements-dev.in # pytest-timeout @@ -159,9 +179,9 @@ pytest-xdist[psutil]==3.3.1 # via -r requirements-dev.in python-dateutil==2.8.2 # via -r requirements.txt -pyzmq==25.1.0 +pyzmq==25.1.1 # via locust -referencing==0.30.0 +referencing==0.30.2 # via # -r requirements.txt # jsonschema @@ -174,11 +194,13 @@ rich==12.6.0 # via -r requirements.txt roundrobin==0.0.4 # via locust -rpds-py==0.9.2 +rpds-py==0.10.2 # via # -r requirements.txt # jsonschema # referencing +schedule==1.2.0 + # via -r requirements.txt sentry-sdk==1.30.0 # via -r requirements-dev.in six==1.16.0 @@ -193,21 +215,25 @@ sniffio==1.3.0 # anyio sortedcontainers==2.4.0 # via hypothesis -tenacity==8.2.2 +tenacity==8.2.3 # via -r requirements.txt -typing-extensions==4.6.3 +tomli==2.0.1 + # via pytest +typing-extensions==4.7.1 # via # -r requirements.txt + # filelock # locust + # rich ujson==5.8.0 # via -r requirements.txt -urllib3==2.0.3 +urllib3==2.0.4 # via # requests # sentry-sdk watchfiles==0.20.0 # via -r requirements.txt -werkzeug==2.3.6 +werkzeug==2.3.7 # via # flask # locust @@ -219,6 +245,11 @@ yarl==1.9.2 # via # -r requirements.txt # aiohttp +zipp==3.16.2 + # via + # -r requirements.txt + # importlib-metadata + # importlib-resources zope-event==5.0 # via gevent zope-interface==6.0 diff --git a/requirements/requirements-typing.in b/requirements/requirements-typing.in index c55f35548..1d11d3198 100644 --- a/requirements/requirements-typing.in +++ b/requirements/requirements-typing.in @@ -5,3 +5,4 @@ mypy types-jsonschema sentry-sdk types-cachetools +types-psutil diff --git a/requirements/requirements-typing.txt b/requirements/requirements-typing.txt index 95e3defdb..f035ad784 100644 --- a/requirements/requirements-typing.txt +++ b/requirements/requirements-typing.txt @@ -8,7 +8,7 @@ certifi==2023.7.22 # via # -c requirements-dev.txt # sentry-sdk -mypy==1.4.1 +mypy==1.5.1 # via -r requirements-typing.in mypy-extensions==1.0.0 # via mypy @@ -16,16 +16,22 @@ sentry-sdk==1.30.0 # via # -c requirements-dev.txt # -r requirements-typing.in +tomli==2.0.1 + # via + # -c requirements-dev.txt + # mypy types-cachetools==5.3.0.6 # via -r requirements-typing.in types-jsonschema==4.17.0.10 # via -r requirements-typing.in -typing-extensions==4.6.3 +types-psutil==5.9.5.16 + # via -r requirements-typing.in +typing-extensions==4.7.1 # via # -c requirements-dev.txt # -c requirements.txt # mypy -urllib3==2.0.3 +urllib3==2.0.4 # via # -c requirements-dev.txt # sentry-sdk diff --git a/requirements/requirements.in b/requirements/requirements.in index 65b90c0ba..f4e95c895 100644 --- a/requirements/requirements.in +++ b/requirements/requirements.in @@ -11,10 +11,14 @@ tenacity<9 typing-extensions ujson<6 watchfiles<1 +schedule +psutil +prometheus-client xxhash~=3.3 rich~=12.6.0 cachetools==5.3.1 + # Patched dependencies # # Note: It is important to use commits to reference patched dependencies. This diff --git a/requirements/requirements.txt b/requirements/requirements.txt index 7112a7d12..3651ae066 100644 --- a/requirements/requirements.txt +++ b/requirements/requirements.txt @@ -12,9 +12,9 @@ aiokafka==0.8.1 # via -r requirements.in aiosignal==1.3.1 # via aiohttp -anyio==3.7.0 +anyio==4.0.0 # via watchfiles -async-timeout==4.0.2 +async-timeout==4.0.3 # via # aiohttp # aiokafka @@ -27,11 +27,13 @@ avro @ https://github.com/aiven/avro/archive/5a82d57f2a650fd87c819a30e433f1abb2c # via -r requirements.in cachetools==5.3.1 # via -r requirements.in -charset-normalizer==3.1.0 +charset-normalizer==3.2.0 # via aiohttp commonmark==0.9.1 # via rich -frozenlist==1.3.3 +exceptiongroup==1.1.3 + # via anyio +frozenlist==1.4.0 # via # aiohttp # aiosignal @@ -39,9 +41,13 @@ idna==3.4 # via # anyio # yarl +importlib-resources==6.0.1 + # via + # jsonschema + # jsonschema-specifications isodate==0.6.1 # via -r requirements.in -jsonschema==4.18.4 +jsonschema==4.19.0 # via -r requirements.in jsonschema-specifications==2023.7.1 # via jsonschema @@ -57,32 +63,42 @@ networkx==2.8.8 # via -r requirements.in packaging==23.1 # via aiokafka +pkgutil-resolve-name==1.3.10 + # via jsonschema +prometheus-client==0.17.1 + # via -r requirements.in protobuf==3.20.3 # via -r requirements.in -pygments==2.15.1 +psutil==5.9.5 + # via -r requirements.in +pygments==2.16.1 # via rich python-dateutil==2.8.2 # via -r requirements.in -referencing==0.30.0 +referencing==0.30.2 # via # jsonschema # jsonschema-specifications rich==12.6.0 # via -r requirements.in -rpds-py==0.9.2 +rpds-py==0.10.2 # via # jsonschema # referencing +schedule==1.2.0 + # via -r requirements.in six==1.16.0 # via # isodate # python-dateutil sniffio==1.3.0 # via anyio -tenacity==8.2.2 - # via -r requirements.in -typing-extensions==4.6.3 +tenacity==8.2.3 # via -r requirements.in +typing-extensions==4.7.1 + # via + # -r requirements.in + # rich ujson==5.8.0 # via -r requirements.in watchfiles==0.20.0 @@ -91,3 +107,5 @@ xxhash==3.3.0 # via -r requirements.in yarl==1.9.2 # via aiohttp +zipp==3.16.2 + # via importlib-resources