From 155e40d3db502c092fd3c042b3da0c5b771a25bf Mon Sep 17 00:00:00 2001 From: Richard Hundt Date: Wed, 14 Jun 2023 21:32:35 +0200 Subject: [PATCH 01/54] Ensure callback map is rebuilt after reply_to topics are created (#510) * Ensure callback map is rebuilt after reply_to topics are created * Guard changes to client_only mode * register reply_to topic with the conductor in the ReplyConsumer * remove additional logging * undo refactoring of _tp_index construction * use MagicMock to make topics iterable * sort test imports * run maybe_declare before adding topic to ReplyConsumer --------- Co-authored-by: Richard Hundt --- faust/agents/replies.py | 1 + faust/transport/conductor.py | 2 +- tests/unit/agents/test_replies.py | 4 ++-- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/faust/agents/replies.py b/faust/agents/replies.py index 7ce915c2d..69e87c953 100644 --- a/faust/agents/replies.py +++ b/faust/agents/replies.py @@ -175,6 +175,7 @@ async def _start_fetcher(self, topic_name: str) -> None: # declare the topic topic = self._reply_topic(topic_name) await topic.maybe_declare() + self.app.topics.add(topic) await self.sleep(3.0) # then create the future self._fetchers[topic_name] = self.add_future(self._drain_replies(topic)) diff --git a/faust/transport/conductor.py b/faust/transport/conductor.py index 45a147ceb..cda7858f1 100644 --- a/faust/transport/conductor.py +++ b/faust/transport/conductor.py @@ -413,7 +413,7 @@ def add(self, topic: TopicT) -> None: def _topic_contain_unsubscribed_topics(self, topic: TopicT) -> bool: index = self._topic_name_index - return bool(index and any(t not in index for t in topic.topics)) + return bool(any(t not in index for t in topic.topics)) def discard(self, topic: Any) -> None: """Unregister topic from conductor.""" diff --git a/tests/unit/agents/test_replies.py b/tests/unit/agents/test_replies.py index 8745c5fd8..11912ec58 100644 --- a/tests/unit/agents/test_replies.py +++ b/tests/unit/agents/test_replies.py @@ -1,6 +1,6 @@ import asyncio import json -from unittest.mock import Mock +from unittest.mock import MagicMock, Mock import pytest @@ -192,7 +192,7 @@ async def test_add(self, *, c): async def test_start_fetcher(self, *, c): c._drain_replies = Mock() c._reply_topic = Mock( - return_value=Mock( + return_value=MagicMock( maybe_declare=AsyncMock(), ), ) From 05317154e1236d9222f05db55614f99021959c66 Mon Sep 17 00:00:00 2001 From: Richard Hundt Date: Thu, 22 Jun 2023 16:03:02 +0200 Subject: [PATCH 02/54] Fix set operations in `ChangeloggedSet` (#521) * Fixes AttributeError: 'list' object has no attribute 'difference' in _modify_set * Make sure sets are properly constructed from list objects during recovery and passed to mode's set collection --------- Co-authored-by: Richard Hundt Co-authored-by: William Barnhart --- faust/tables/sets.py | 10 +++++----- tests/unit/tables/test_sets.py | 26 +++++++++++++------------- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/faust/tables/sets.py b/faust/tables/sets.py index 1cca3679d..8da417a62 100644 --- a/faust/tables/sets.py +++ b/faust/tables/sets.py @@ -84,7 +84,7 @@ def on_change(self, added: Set[VT], removed: Set[VT]) -> None: self.manager.send_changelog_event(self.key, OPERATION_UPDATE, [added, removed]) def sync_from_storage(self, value: Any) -> None: - self.data = cast(Set, value) + self.data = set(value) def as_stored_value(self) -> Any: return self.data @@ -204,19 +204,19 @@ async def symmetric_difference_update(self, key: KT, members: Iterable[VT]) -> N await self._send_operation(SetAction.SYMDIFF, key, members) def _update(self, key: KT, members: List[VT]) -> None: - self.set_table[key].update(members) + self.set_table[key].update(set(members)) def _difference_update(self, key: KT, members: List[VT]) -> None: - self.set_table[key].difference_update(members) + self.set_table[key].difference_update(set(members)) def _clear(self, key: KT, members: List[VT]) -> None: self.set_table[key].clear() def _intersection_update(self, key: KT, members: List[VT]) -> None: - self.set_table[key].intersection_update(members) + self.set_table[key].intersection_update(set(members)) def _symmetric_difference_update(self, key: KT, members: List[VT]) -> None: - self.set_table[key].symmetric_difference_update(members) + self.set_table[key].symmetric_difference_update(set(members)) async def _send_operation( self, action: SetAction, key: KT, members: Iterable[VT] diff --git a/tests/unit/tables/test_sets.py b/tests/unit/tables/test_sets.py index f60e319f5..164e7f16b 100644 --- a/tests/unit/tables/test_sets.py +++ b/tests/unit/tables/test_sets.py @@ -248,12 +248,12 @@ async def test_symmetric_difference_update(self, *, man): def test__update(self, *, man): man.set_table = {"a": Mock(name="a"), "b": Mock(name="b")} man._update("a", ["v1"]) - man.set_table["a"].update.assert_called_once_with(["v1"]) + man.set_table["a"].update.assert_called_once_with({"v1"}) def test__difference_update(self, *, man): man.set_table = {"a": Mock(name="a"), "b": Mock(name="b")} man._difference_update("a", ["v1"]) - man.set_table["a"].difference_update.assert_called_once_with(["v1"]) + man.set_table["a"].difference_update.assert_called_once_with({"v1"}) def test__clear(self, *, man): man.set_table = {"a": Mock(name="a"), "b": Mock(name="b")} @@ -264,14 +264,14 @@ def test__intersection_update(self, *, man): man.set_table = {"a": Mock(name="a"), "b": Mock(name="b")} man._intersection_update("a", ["v1", "v2", "v3"]) man.set_table["a"].intersection_update.assert_called_once_with( - ["v1", "v2", "v3"], + {"v1", "v2", "v3"}, ) def test__symmetric_difference_update(self, *, man): man.set_table = {"a": Mock(name="a"), "b": Mock(name="b")} man._symmetric_difference_update("a", ["v1", "v2", "v3"]) man.set_table["a"].symmetric_difference_update.assert_called_once_with( - ["v1", "v2", "v3"], + {"v1", "v2", "v3"}, ) @pytest.mark.asyncio @@ -396,29 +396,29 @@ async def stream_items(): await man._modify_set(stream) - man.set_table["k1"].update.assert_called_with(["v"]) - man.set_table["k2"].difference_update.assert_called_with(["v2"]) - man.set_table["k3"].difference_update.assert_called_with([X(10, 30)]) + man.set_table["k1"].update.assert_called_with({"v"}) + man.set_table["k2"].difference_update.assert_called_with({"v2"}) + man.set_table["k3"].difference_update.assert_called_with({X(10, 30)}) man.set_table["k5"].update.assert_called_with( - [ + { X(10, 30), X(20, 40), "v3", - ] + } ) man.set_table["k6"].intersection_update.assert_called_with( - [ + { X(10, 30), X(20, 40), "v3", - ] + } ) man.set_table["k7"].symmetric_difference_update.assert_called_with( - [ + { X(10, 30), X(20, 40), "v3", - ] + } ) man.set_table["k8"].clear.assert_called_once_with() From ada6429dd38c805570f1a20a7f37cecccc4cebfd Mon Sep 17 00:00:00 2001 From: Chris Coutinho <12901868+cbcoutinho@users.noreply.github.com> Date: Sun, 25 Jun 2023 18:03:39 +0200 Subject: [PATCH 03/54] Implement OAuthbearer auth mechanism (#525) * initial commit * isort * Update type signature * Add to missing imports * Remove unused imports * Remove duplicate docstring * Add test * Fix SSL configuration, add test * Remove client_id and update repr * linting * Remove client_id from test * Add test case to aiokafka transport * Add documentation on OAuthCredentials --------- Co-authored-by: Chris Coutinho --- docs/includes/settingref.txt | 35 ++++++++++++++ faust/__init__.py | 9 +++- faust/auth.py | 30 ++++++++++++ faust/transport/drivers/aiokafka.py | 14 +++++- faust/types/auth.py | 1 + tests/unit/test_auth.py | 47 ++++++++++++++++++- tests/unit/transport/drivers/test_aiokafka.py | 10 ++++ 7 files changed, 143 insertions(+), 3 deletions(-) diff --git a/docs/includes/settingref.txt b/docs/includes/settingref.txt index cf65e6e5d..91bec6c02 100644 --- a/docs/includes/settingref.txt +++ b/docs/includes/settingref.txt @@ -439,6 +439,41 @@ SASL Authentication password=BROKER_PASSWORD, )) +OAuth2 Authentication + You can enable SASL authentication via OAuth2 Bearer tokens: + + .. sourcecode:: python + + import faust + from asyncio import get_running_loop + from aiokafka.helpers import create_ssl_context + from aiokafka.conn import AbstractTokenProvider + + class TokenProvider(AbstractTokenProvider): + async def token(self): + return await get_running_loop().run_in_executor( + None, self.get_token) + + def get_token(self): + return 'token' + + app = faust.App( + broker_credentials=faust.OAuthCredentials( + oauth_cb=TokenProvider() + ssl_context=create_ssl_context() + ) + ) + + .. info:: + + The implementation should ensure token reuse so that multiple + calls at connect time do not create multiple tokens. + The implementation should also periodically refresh the token in order to + guarantee that each call returns an unexpired token. + + Token Providers MUST implement the :meth:`token` method + + GSSAPI Authentication GSSAPI authentication over plain text: diff --git a/faust/__init__.py b/faust/__init__.py index e43771d90..c20b05903 100644 --- a/faust/__init__.py +++ b/faust/__init__.py @@ -133,7 +133,12 @@ def _extract_arg_from_argv( # pragma: no cover from .agents import Agent # noqa: E402 from .app import App # noqa: E402 - from .auth import GSSAPICredentials, SASLCredentials, SSLCredentials # noqa: E402 + from .auth import ( # noqa: E402 + GSSAPICredentials, + OAuthCredentials, + SASLCredentials, + SSLCredentials, + ) from .channels import Channel, ChannelT # noqa: E402 from .events import Event, EventT # noqa: E402 from .models import Model, ModelOptions, Record # noqa: E402 @@ -184,6 +189,7 @@ def _extract_arg_from_argv( # pragma: no cover "TopicT", "GSSAPICredentials", "SASLCredentials", + "OAuthCredentials", "SSLCredentials", "Settings", "HoppingWindow", @@ -219,6 +225,7 @@ def _extract_arg_from_argv( # pragma: no cover "GSSAPICredentials", "SASLCredentials", "SSLCredentials", + "OAuthCredentials", ], "faust.types.settings": ["Settings"], "faust.windows": [ diff --git a/faust/auth.py b/faust/auth.py index 22c4e859f..0c97e6a60 100644 --- a/faust/auth.py +++ b/faust/auth.py @@ -2,11 +2,14 @@ import ssl from typing import Any, Optional, Union +from aiokafka.conn import AbstractTokenProvider + from faust.types.auth import AuthProtocol, CredentialsT, SASLMechanism __all__ = [ "Credentials", "SASLCredentials", + "OAuthCredentials", "GSSAPICredentials", "SSLCredentials", ] @@ -49,6 +52,33 @@ def __repr__(self) -> str: return f"<{type(self).__name__}: username={self.username}>" +class OAuthCredentials(Credentials): + """Describe OAuth Bearer credentials over SASL""" + + protocol = AuthProtocol.SASL_PLAINTEXT + mechanism: SASLMechanism = SASLMechanism.OAUTHBEARER + + ssl_context: Optional[ssl.SSLContext] + + def __init__( + self, + *, + oauth_cb: AbstractTokenProvider, + ssl_context: Optional[ssl.SSLContext] = None, + ): + self.oauth_cb = oauth_cb + self.ssl_context = ssl_context + + if ssl_context is not None: + self.protocol = AuthProtocol.SASL_SSL + + def __repr__(self) -> str: + return "<{0}: oauth credentials {1} SSL support".format( + type(self).__name__, + "with" if self.protocol == AuthProtocol.SASL_SSL else "without", + ) + + class GSSAPICredentials(Credentials): """Describe GSSAPI credentials over SASL.""" diff --git a/faust/transport/drivers/aiokafka.py b/faust/transport/drivers/aiokafka.py index a0642a281..032e79f8e 100644 --- a/faust/transport/drivers/aiokafka.py +++ b/faust/transport/drivers/aiokafka.py @@ -55,7 +55,12 @@ from opentracing.ext import tags from yarl import URL -from faust.auth import GSSAPICredentials, SASLCredentials, SSLCredentials +from faust.auth import ( + GSSAPICredentials, + OAuthCredentials, + SASLCredentials, + SSLCredentials, +) from faust.exceptions import ( ConsumerNotStarted, ImproperlyConfigured, @@ -1584,6 +1589,13 @@ def credentials_to_aiokafka_auth( "security_protocol": credentials.protocol.value, "ssl_context": credentials.context, } + elif isinstance(credentials, OAuthCredentials): + return { + "security_protocol": credentials.protocol.value, + "sasl_mechanism": credentials.mechanism.value, + "sasl_oauth_token_provider": credentials.oauth_cb, + "ssl_context": credentials.ssl_context, + } elif isinstance(credentials, SASLCredentials): return { "security_protocol": credentials.protocol.value, diff --git a/faust/types/auth.py b/faust/types/auth.py index 4a78f5bdd..30b99a56b 100644 --- a/faust/types/auth.py +++ b/faust/types/auth.py @@ -25,6 +25,7 @@ class SASLMechanism(Enum): GSSAPI = "GSSAPI" SCRAM_SHA_256 = "SCRAM-SHA-256" SCRAM_SHA_512 = "SCRAM-SHA-512" + OAUTHBEARER = "OAUTHBEARER" AUTH_PROTOCOLS_SSL = {AuthProtocol.SSL, AuthProtocol.SASL_SSL} diff --git a/tests/unit/test_auth.py b/tests/unit/test_auth.py index 232e5c875..d1ad9fc13 100644 --- a/tests/unit/test_auth.py +++ b/tests/unit/test_auth.py @@ -1,9 +1,16 @@ +import asyncio import ssl from unittest.mock import Mock, patch import pytest +from aiokafka.conn import AbstractTokenProvider -from faust.auth import GSSAPICredentials, SASLCredentials, SSLCredentials +from faust.auth import ( + GSSAPICredentials, + OAuthCredentials, + SASLCredentials, + SSLCredentials, +) from faust.types.auth import AuthProtocol, SASLMechanism @@ -140,3 +147,41 @@ def test_having_context(self): context = Mock(name="context") c = SSLCredentials(context) assert c.context is context + + +class Test_OAuthCredentials: + class TokenProvider(AbstractTokenProvider): + async def token(self): + return await asyncio.get_running_loop().run_in_executor(None, self._token) + + def _token(self): + return "token" + + @pytest.mark.parametrize( + "reason,credentials,expected_fields", + [ + pytest.param( + "retains arguments", + OAuthCredentials(oauth_cb=TokenProvider()), + { + "mechanism": SASLMechanism.OAUTHBEARER, + "protocol": AuthProtocol.SASL_PLAINTEXT, + }, + ), + pytest.param( + "retains arguments", + OAuthCredentials( + oauth_cb=TokenProvider(), + ssl_context={"xxx": "yyy"}, + ), + { + "mechanism": SASLMechanism.OAUTHBEARER, + "protocol": AuthProtocol.SASL_SSL, + }, + ), + ], + ) + def test_constructor(self, credentials, expected_fields, reason): + assert repr(credentials) + for field, value in expected_fields.items(): + assert getattr(credentials, field) == value, reason diff --git a/tests/unit/transport/drivers/test_aiokafka.py b/tests/unit/transport/drivers/test_aiokafka.py index ff6109b8e..bfa405fd8 100644 --- a/tests/unit/transport/drivers/test_aiokafka.py +++ b/tests/unit/transport/drivers/test_aiokafka.py @@ -1992,6 +1992,16 @@ async def test__create_topic__raises(self, *, transport, loop): "ssl_context": None, }, ), + ( + auth.OAuthCredentials(oauth_cb="token"), + None, + { + "security_protocol": "SASL_PLAINTEXT", + "sasl_oauth_token_provider": "token", + "sasl_mechanism": "OAUTHBEARER", + "ssl_context": None, + }, + ), ], ) def test_credentials_to_aiokafka(credentials, ssl_context, expected): From f79cc1692ddfb7b273a78c46fa9ae5bced542236 Mon Sep 17 00:00:00 2001 From: Richard Hundt Date: Mon, 26 Jun 2023 18:20:45 +0200 Subject: [PATCH 04/54] Fix for current_event not in context during periodic flush (#522) Co-authored-by: Richard Hundt Co-authored-by: William Barnhart --- faust/tables/objects.py | 19 +------------------ tests/unit/tables/test_objects.py | 17 ++--------------- 2 files changed, 3 insertions(+), 33 deletions(-) diff --git a/faust/tables/objects.py b/faust/tables/objects.py index 1cef7ea13..3fb7d92a2 100644 --- a/faust/tables/objects.py +++ b/faust/tables/objects.py @@ -65,20 +65,18 @@ class ChangeloggedObjectManager(Store): data: MutableMapping _storage: Optional[StoreT] = None - _dirty: Set def __init__(self, table: Table, **kwargs: Any) -> None: self.table = table self.table_name = self.table.name self.data = {} - self._dirty = set() Service.__init__(self, loop=table.loop, **kwargs) def send_changelog_event(self, key: Any, operation: int, value: Any) -> None: """Send changelog event to the tables changelog topic.""" event = current_event() - self._dirty.add(key) self.table._send_changelog(event, (operation, key), value) + self.storage[key] = self[key].as_stored_value() def __getitem__(self, key: Any) -> ChangeloggedObject: if key in self.data: @@ -100,10 +98,6 @@ async def on_start(self) -> None: """Call when the changelogged object manager starts.""" await self.add_runtime_dependency(self.storage) - async def on_stop(self) -> None: - """Call when the changelogged object manager stops.""" - self.flush_to_storage() - def persisted_offset(self, tp: TP) -> Optional[int]: """Get the last persisted offset for changelog topic partition.""" return self.storage.persisted_offset(tp) @@ -133,17 +127,6 @@ def sync_from_storage(self) -> None: for key, value in self.storage.items(): self[key].sync_from_storage(value) - def flush_to_storage(self) -> None: - """Flush set contents to storage.""" - for key in self._dirty: - self.storage[key] = self.data[key].as_stored_value() - self._dirty.clear() - - @Service.task - async def _periodic_flush(self) -> None: # pragma: no cover - async for sleep_time in self.itertimer(2.0, name="SetManager.flush"): - self.flush_to_storage() - def reset_state(self) -> None: """Reset table local state.""" # delegate to underlying RocksDB store. diff --git a/tests/unit/tables/test_objects.py b/tests/unit/tables/test_objects.py index cf6f6e174..ad3b425ff 100644 --- a/tests/unit/tables/test_objects.py +++ b/tests/unit/tables/test_objects.py @@ -54,6 +54,7 @@ class Test_ChangeloggedObjectManager: def man(self, *, table): man = ChangeloggedObjectManager(table) man.ValueType = ValueType + man.storage.__setitem__ = Mock() return man @pytest.fixture() @@ -62,7 +63,7 @@ def storage(self, *, table): def test_send_changelog_event(self, *, man, table, key, current_event): man.send_changelog_event(key, 3, "value") - assert key in man._dirty + assert man.storage.__setitem__.called_once_with(key, "value") table._send_changelog.assert_called_once_with( current_event(), (3, key), @@ -98,12 +99,6 @@ async def test_on_start(self, *, man): await man.on_start() man.add_runtime_dependency.assert_called_once_with(man.storage) - @pytest.mark.asyncio - async def test_on_stop(self, *, man): - man.flush_to_storage = Mock() - await man.on_stop() - man.flush_to_storage.assert_called_once_with() - def test_persisted_offset(self, *, man, storage): ret = man.persisted_offset(TP1) storage.persisted_offset.assert_called_once_with(TP1) @@ -135,14 +130,6 @@ def test_sync_from_storage(self, *, man, storage): assert 1 in man["foo"].synced assert 2 in man["bar"].synced - def test_flush_to_storage(self, *, man): - man._storage = {} - man._dirty = {"foo", "bar"} - assert man["foo"] - assert man["bar"] - man.flush_to_storage() - assert man._storage["foo"] == "foo-stored" - def test_reset_state(self, *, man, storage): man.reset_state() storage.reset_state.assert_called_once_with() From 600d0ae28b74c040350522764de7475af5c475aa Mon Sep 17 00:00:00 2001 From: William Barnhart Date: Mon, 26 Jun 2023 16:57:49 -0400 Subject: [PATCH 05/54] Update CI/CD workflows, setup config files, and build wheels for Python >= 3.8 (#524) * Update CI/CD workflows * disable cython for 3.12 and pypy3.9 * forgot to disable tests for 3.7 * properly disable cython * syntax is annoying * we should only test with orjson if we have cython installed * allow experimentals to fail * a test should take no longer than ten minutes * i guess we can't use continue-on-error that way then * test cibuildwheel * revert dist.yml changes * disable testing for 3.12 until properly supported in August or so * disable scripts/coverage call in scripts/tests because we already do it in a separate job * remove old tests and disable timeout * dsiable integ * try skipping meticulous tests * disable pypy tests for now * fix typo * extend timeout * switch over setup.cfg to pyproject.toml * fix linting * flake8 is impossible to please apparently * update gh-pages.yml * fix all yamls * fix requirements seutp * cleanup * should continue on false * fix continue on error placement * check linting only * Update python-package.yml to use experimental * Update python-package.yml * Update tests to run meticulous again * Update python-package.yml to allow 3.12 to fail * just disable 3.12 for now * Update python-package.yml to not test pypy until we figure this out * Update python-package.yml to extend job timeout to 10m * consolidate dist.yml into python-package.yml * update pyproject.toml with proper cibuildwheel config * forgot fetch-depth for wheel builds * reformat * skip builds for musllinux_x86_64 * enable check requirement * revert drastic changes for linting * pull in new changes from master --- .github/workflows/codeql-analysis.yml | 36 ++-- .github/workflows/dist.yml | 73 -------- .github/workflows/gh-pages.yml | 50 +++--- .github/workflows/python-package.yml | 157 ++++++++++++++---- faust/contrib/sentry.py | 6 +- faust/sensors/monitor.py | 8 +- faust/types/agents.py | 18 +- faust/types/channels.py | 6 +- faust/types/models.py | 4 +- faust/types/sensors.py | 2 +- faust/types/serializers.py | 16 +- faust/types/stores.py | 2 +- faust/types/streams.py | 6 +- faust/types/tables.py | 4 +- faust/types/topics.py | 4 +- faust/types/transports.py | 14 +- faust/types/web.py | 6 +- faust/utils/terminal/tables.py | 6 +- faust/utils/venusian.py | 2 +- faust/web/exceptions.py | 2 +- pyproject.toml | 64 +++++++ requirements/test.txt | 1 - scripts/tests | 6 +- setup.cfg | 20 --- setup.py | 12 +- tests/functional/test_models.py | 29 +++- tests/functional/web/test_cache.py | 5 +- tests/integration/cli/test_agents.py | 6 +- tests/integration/cli/test_model.py | 6 +- tests/integration/cli/test_models.py | 5 +- tests/old/__init__.py | 0 tests/old/app.py | 12 -- tests/old/test_simple.py | 31 ---- tests/stress/app.py | 2 +- tests/stress/tests/forwarder/app.py | 4 +- tests/stress/tests/tables/app.py | 4 +- tests/unit/agents/test_agent.py | 15 +- tests/unit/livecheck/test_case.py | 11 +- tests/unit/sensors/test_datadog.py | 27 ++- tests/unit/sensors/test_monitor.py | 10 +- tests/unit/sensors/test_prometheus.py | 9 +- tests/unit/sensors/test_statsd.py | 4 +- tests/unit/serializers/test_codecs.py | 6 +- tests/unit/stores/test_aerospike.py | 6 +- tests/unit/tables/test_recovery.py | 8 +- tests/unit/tables/test_table.py | 8 +- tests/unit/test_auth.py | 11 +- tests/unit/transport/drivers/test_aiokafka.py | 43 ++++- tests/unit/transport/test_consumer.py | 7 +- tests/unit/utils/terminal/test_tables.py | 16 +- tests/unit/web/drivers/test_aiohttp.py | 6 +- 51 files changed, 497 insertions(+), 319 deletions(-) delete mode 100644 .github/workflows/dist.yml create mode 100644 pyproject.toml delete mode 100644 tests/old/__init__.py delete mode 100644 tests/old/app.py delete mode 100644 tests/old/test_simple.py diff --git a/.github/workflows/codeql-analysis.yml b/.github/workflows/codeql-analysis.yml index 6d667e86f..43427fab9 100644 --- a/.github/workflows/codeql-analysis.yml +++ b/.github/workflows/codeql-analysis.yml @@ -1,3 +1,4 @@ +--- # For most projects, this workflow file will not need changing; you simply need # to commit it to your repository. # @@ -9,17 +10,15 @@ # the `language` matrix defined below to confirm you have the correct set of # supported CodeQL languages. # -name: "CodeQL" - +name: CodeQL on: push: - branches: [ master ] + branches: [master] pull_request: # The branches below must be a subset of the branches above - branches: [ master ] + branches: [master] schedule: - - cron: '19 10 * * 6' - + - cron: 19 10 * * 6 jobs: analyze: name: Analyze @@ -28,24 +27,22 @@ jobs: actions: read contents: read security-events: write - strategy: fail-fast: false matrix: - language: [ 'python' ] + language: [python] # CodeQL supports [ 'cpp', 'csharp', 'go', 'java', 'javascript', 'python' ] # Learn more: # https://docs.github.com/en/free-pro-team@latest/github/finding-security-vulnerabilities-and-errors-in-your-code/configuring-code-scanning#changing-the-languages-that-are-analyzed - steps: - - name: Checkout repository - uses: actions/checkout@v3 + - name: Checkout repository + uses: actions/checkout@v3 # Initializes the CodeQL tools for scanning. - - name: Initialize CodeQL - uses: github/codeql-action/init@v2 - with: - languages: ${{ matrix.language }} + - name: Initialize CodeQL + uses: github/codeql-action/init@v2 + with: + languages: ${{ matrix.language }} # If you wish to specify custom queries, you can do so here or in a config file. # By default, queries listed here will override any specified in a config file. # Prefix the list here with "+" to use these queries and those in the config file. @@ -53,8 +50,8 @@ jobs: # Autobuild attempts to build any compiled languages (C/C++, C#, or Java). # If this step fails, then you should remove it and run the build manually (see below) - - name: Autobuild - uses: github/codeql-action/autobuild@v2 + - name: Autobuild + uses: github/codeql-action/autobuild@v2 # ℹī¸ Command-line programs to run using the OS shell. # 📚 https://git.io/JvXDl @@ -66,6 +63,5 @@ jobs: #- run: | # make bootstrap # make release - - - name: Perform CodeQL Analysis - uses: github/codeql-action/analyze@v2 + - name: Perform CodeQL Analysis + uses: github/codeql-action/analyze@v2 diff --git a/.github/workflows/dist.yml b/.github/workflows/dist.yml deleted file mode 100644 index 4c6005943..000000000 --- a/.github/workflows/dist.yml +++ /dev/null @@ -1,73 +0,0 @@ -# vim:ts=2:sw=2:et:ai:sts=2 -name: 'Build distribution' - -on: - # Only run when release is created in the master branch - release: - types: [created] - branches: - - 'master' - -jobs: - build_wheels: - name: Build wheels on ${{ matrix.os }} - runs-on: ${{ matrix.os }} - strategy: - matrix: - os: [ubuntu-20.04, macos-11] - - steps: - - uses: actions/checkout@v3 - - - name: Build wheels - uses: pypa/cibuildwheel@v2.10.1 - env: - CIBW_MANYLINUX_X86_64_IMAGE: 'manylinux2014' - CIBW_ARCHS: auto64 - CIBW_BUILD: 'cp3*' - CIBW_SKIP: '*p36*' - CIBW_BEFORE_BUILD: pip3 install Cython - - - uses: actions/upload-artifact@v3 - with: - path: ./wheelhouse/*.whl - - build_sdist: - name: 'Build source distribution' - runs-on: 'ubuntu-latest' - steps: - - uses: actions/checkout@v3 - name: 'Checkout source repository' - with: - # You need to include this or setuptools_scm in GitHub runners won't detect the version - fetch-depth: 0 - - - uses: actions/setup-python@v4 - - - name: 'Build sdist' - run: > - pip3 install pkgconfig cython --upgrade && - python3 setup.py sdist - - - uses: actions/upload-artifact@v3 - name: 'Upload build artifacts' - with: - path: 'dist/*.tar.gz' - - upload_pypi: - name: 'Upload packages' - needs: ['build_wheels', 'build_sdist'] - runs-on: 'ubuntu-latest' - if: github.event_name == 'release' && github.event.action == 'created' - steps: - - uses: actions/download-artifact@v3 - name: 'Download artifacts' - with: - name: 'artifact' - path: 'dist' - - - uses: pypa/gh-action-pypi-publish@release/v1 - name: "Publish package to PyPI" - with: - user: '__token__' - password: '${{ secrets.PYPI_API_TOKEN }}' diff --git a/.github/workflows/gh-pages.yml b/.github/workflows/gh-pages.yml index 2f02d7286..fda7e9ddc 100644 --- a/.github/workflows/gh-pages.yml +++ b/.github/workflows/gh-pages.yml @@ -1,47 +1,47 @@ +--- name: Pages - on: push: - branches: ["master"] + branches: [master] pull_request: - branches: ["master"] + branches: [master] release: types: [created] - branches: - - 'master' - + branches: [master] jobs: build: - name: "Build docs" + name: Build docs runs-on: ubuntu-latest steps: - - uses: actions/setup-python@v4 - - uses: actions/checkout@v3 - with: - fetch-depth: 0 # otherwise, you will failed to push refs to dest repo - - name: "Install runtime dependencies" - run: "scripts/install" - - name: "Install doc build deps and build with Sphinx" - run: make docs - - name: "Upload artifacts" - uses: actions/upload-pages-artifact@v1 - with: + - uses: actions/setup-python@v4 + - uses: actions/checkout@v3 + with: + fetch-depth: 0 # otherwise, you will failed to push refs to dest repo + - name: Install runtime dependencies + run: | + pip install . + pip install -r requirements/docs.txt + - name: Install doc build deps and build with Sphinx + run: make docs + - name: Upload artifacts + uses: actions/upload-pages-artifact@v1 + with: # Upload built docs - path: "./Documentation" + path: ./Documentation deploy: - name: "Deploy docs" + name: Deploy docs if: github.event_name == 'release' && github.event.action == 'created' needs: build runs-on: ubuntu-latest # Grant GITHUB_TOKEN the permissions required to make a Pages deployment permissions: - pages: write # to deploy to Pages - id-token: write # to verify the deployment originates from an appropriate source + pages: write # to deploy to Pages + id-token: write # to verify the deployment originates from an appropriate source # Deploy to the github-pages environment environment: name: github-pages url: ${{ steps.deployment.outputs.page_url }} steps: - - uses: actions/deploy-pages@v2 - id: deployment - name: "Deploy to GitHub Pages" + - uses: actions/deploy-pages@v2 + id: deployment + name: Deploy to GitHub Pages diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 8f8379154..5670989c2 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -1,44 +1,143 @@ -# This workflow will install Python dependencies, run tests and lint with a variety of Python versions -# For more information see: https://help.github.com/actions/language-and-framework-guides/using-python-with-github-actions - -name: Python package - +--- +name: Test Python library on: push: - branches: ["master"] + branches: [master] pull_request: - branches: ["master"] + branches: [master] + release: + types: [created] + branches: [master] +env: + FORCE_COLOR: '1' # Make tools pretty. + PIP_DISABLE_PIP_VERSION_CHECK: '1' + PIP_NO_PYTHON_VERSION_WARNING: '1' + PYTHON_LATEST: '3.11' + # For re-actors/checkout-python-sdist + sdist-artifact: python-package-distributions jobs: - tests: - name: "Python ${{ matrix.python-version }}/Cython: ${{ matrix.use-cython }}" - runs-on: "ubuntu-latest" - + lint: + name: Check linting + runs-on: ubuntu-latest + steps: + - name: Checkout project + uses: actions/checkout@v3 + with: + fetch-depth: 0 + - name: Set up Python + uses: actions/setup-python@v4 + with: + python-version: ${{ env.PYTHON_LATEST }} + cache: pip + - run: | + python -m pip install build + python -m pip install -r requirements/test.txt + name: Install core libraries for build and install + - name: Run linting checks + run: scripts/check + test-pytest: + name: 'Python ${{ matrix.python-version }}/Cython: ${{ matrix.use-cython }}' + runs-on: ubuntu-latest + timeout-minutes: 10 # Maybe we should remove this someday but the PyPy tests are acting strange strategy: # Complete all jobs even if one fails, allows us to see # for example if a test fails only when Cython is enabled fail-fast: false matrix: - python-version: ["3.8", "3.9", "3.10", "3.11"] - use-cython: ["true", "false"] + python-version: ['3.8', '3.9', '3.10', '3.11'] + use-cython: ['true', 'false'] + experimental: [false] +# include: +# - python-version: pypy3.9 +# experimental: true +# use-cython: 'false' +# - python-version: ~3.12.0-0 +# experimental: true +# use-cython: 'false' env: USE_CYTHON: ${{ matrix.use-cython }} - + continue-on-error: ${{ matrix.experimental }} steps: - - uses: "actions/checkout@v3" - with: - # You need to include this or setuptools_scm in GitHub runners won't detect the version - fetch-depth: 0 - - uses: "actions/setup-python@v4" - with: - python-version: "${{ matrix.python-version }}" - - name: "Install dependencies" - run: "scripts/install" - - name: "Run linting checks" - run: "scripts/check" - - name: "Run tests" - run: "scripts/tests" - - name: "Enforce coverage" + - uses: actions/checkout@v3 + with: + fetch-depth: 0 + - uses: actions/setup-python@v4 + with: + python-version: ${{ matrix.python-version }} + cache: pip + cache-dependency-path: requirements/test.txt + - name: Install dependencies + run: | + pip install -r requirements/test.txt + pip install . + - name: Run tests + run: scripts/tests + - name: Enforce coverage uses: codecov/codecov-action@v3 with: - token: ${{ secrets.CODECOV_TOKEN }} + token: ${{ secrets.CODECOV_TOKEN }} + check: # This job does nothing and is only used for the branch protection + name: ✅ Ensure the required checks passing + if: always() + needs: [lint, test-pytest] + runs-on: ubuntu-latest + steps: + - name: Decide whether the needed jobs succeeded or failed + uses: re-actors/alls-green@release/v1 + with: + jobs: ${{ toJSON(needs) }} + build_wheels: + name: đŸ“Ļ Build wheels on ${{ matrix.os }} + runs-on: ${{ matrix.os }} + needs: check + if: github.event_name == 'release' && github.event.action == 'created' + strategy: + matrix: + os: [ubuntu-20.04, macos-11] + steps: + - uses: actions/checkout@v3 + with: + fetch-depth: 0 + - name: Build wheels + uses: pypa/cibuildwheel@v2.10.1 + - uses: actions/upload-artifact@v3 + with: + path: ./wheelhouse/*.whl + build_sdist: + name: đŸ“Ļ Build the source distribution + runs-on: ubuntu-latest + needs: check + if: github.event_name == 'release' && github.event.action == 'created' + steps: + - uses: actions/checkout@v3 + name: Checkout source repository + with: + fetch-depth: 0 + - uses: actions/setup-python@v4 + - name: Build sdist + run: > + pip3 install pkgconfig cython --upgrade && + python3 setup.py sdist + - uses: actions/upload-artifact@v3 + name: Upload build artifacts + with: + path: dist/*.tar.gz + publish: + name: đŸ“Ļ Publish to PyPI + runs-on: ubuntu-latest + needs: [build_wheels, build_sdist] + permissions: + id-token: write + environment: pypi + if: github.event_name == 'release' && github.event.action == 'created' + steps: + - name: Download the sdist artifact + uses: actions/download-artifact@v3 + with: + name: artifact + path: dist + - name: Publish package to PyPI + uses: pypa/gh-action-pypi-publish@release/v1 + with: + password: ${{ secrets.PYPI_API_TOKEN }} diff --git a/faust/contrib/sentry.py b/faust/contrib/sentry.py index c22921e34..04d260329 100644 --- a/faust/contrib/sentry.py +++ b/faust/contrib/sentry.py @@ -96,7 +96,7 @@ def handler_from_dsn( include_paths: Iterable[str] = None, loglevel: Optional[int] = None, qsize: int = 1000, - **kwargs: Any + **kwargs: Any, ) -> Optional[logging.Handler]: if raven is None: raise ImproperlyConfigured("faust.contrib.sentry requires the `raven` library.") @@ -115,7 +115,7 @@ def handler_from_dsn( qsize=qsize, ), disable_existing_loggers=False, - **kwargs + **kwargs, ) handler = _build_sentry_handler()(client) handler.setLevel(level) @@ -130,7 +130,7 @@ def setup( workers: int = 4, max_queue_size: int = 1000, loglevel: Optional[int] = None, - **kwargs + **kwargs, ) -> None: sentry_handler = handler_from_dsn( dsn=dsn, workers=workers, qsize=max_queue_size, loglevel=loglevel, **kwargs diff --git a/faust/sensors/monitor.py b/faust/sensors/monitor.py index d02782a90..efacca358 100644 --- a/faust/sensors/monitor.py +++ b/faust/sensors/monitor.py @@ -68,7 +68,7 @@ def __init__( *, keys_retrieved: int = 0, keys_updated: int = 0, - keys_deleted: int = 0 + keys_deleted: int = 0, ) -> None: self.table: CollectionT = table self.keys_retrieved = keys_retrieved @@ -252,7 +252,7 @@ def __init__( http_response_codes: Counter[HTTPStatus] = None, http_response_latency: Deque[float] = None, http_response_latency_avg: float = 0.0, - **kwargs: Any + **kwargs: Any, ) -> None: if max_avg_history is not None: self.max_avg_history = max_avg_history @@ -636,7 +636,7 @@ def on_web_request_end( response: Optional[web.Response], state: Dict, *, - view: web.View = None + view: web.View = None, ) -> None: """Web server finished working on request.""" status_code = HTTPStatus(response.status if response is not None else 500) @@ -659,7 +659,7 @@ def _normalize( name: str, *, pattern: Pattern = RE_NORMALIZE, - substitution: str = RE_NORMALIZE_SUBSTITUTION + substitution: str = RE_NORMALIZE_SUBSTITUTION, ) -> str: return pattern.sub(substitution, name) diff --git a/faust/types/agents.py b/faust/types/agents.py index 102e05919..4c533c9e7 100644 --- a/faust/types/agents.py +++ b/faust/types/agents.py @@ -88,7 +88,7 @@ def __init__( stream: StreamT, it: _T, active_partitions: Optional[Set[TP]] = None, - **kwargs: Any + **kwargs: Any, ) -> None: ... @@ -145,7 +145,7 @@ def __init__( key_type: ModelArg = None, value_type: ModelArg = None, isolated_partitions: bool = False, - **kwargs: Any + **kwargs: Any, ) -> None: self.fun: AgentFun = fun @@ -160,7 +160,7 @@ def __call__( index: Optional[int] = None, active_partitions: Optional[Set[TP]] = None, stream: Optional[StreamT] = None, - channel: Optional[ChannelT] = None + channel: Optional[ChannelT] = None, ) -> ActorRefT: ... @@ -169,7 +169,7 @@ def test_context( self, channel: Optional[ChannelT] = None, supervisor_strategy: SupervisorStrategyT = None, - **kwargs: Any + **kwargs: Any, ) -> "AgentTestWrapperT": ... @@ -197,7 +197,7 @@ async def cast( key: K = None, partition: Optional[int] = None, timestamp: Optional[float] = None, - headers: HeadersArg = None + headers: HeadersArg = None, ) -> None: ... @@ -211,7 +211,7 @@ async def ask( timestamp: Optional[float] = None, headers: HeadersArg = None, reply_to: ReplyToArg = None, - correlation_id: Optional[str] = None + correlation_id: Optional[str] = None, ) -> Any: ... @@ -227,7 +227,7 @@ async def send( key_serializer: CodecArg = None, value_serializer: CodecArg = None, reply_to: ReplyToArg = None, - correlation_id: Optional[str] = None + correlation_id: Optional[str] = None, ) -> Awaitable[RecordMetadata]: ... @@ -348,7 +348,7 @@ async def put( *, reply_to: ReplyToArg = None, correlation_id: Optional[str] = None, - wait: bool = True + wait: bool = True, ) -> EventT: ... @@ -362,7 +362,7 @@ def to_message( offset: int = 0, timestamp: Optional[float] = None, timestamp_type: int = 0, - headers: HeadersArg = None + headers: HeadersArg = None, ) -> Message: ... diff --git a/faust/types/channels.py b/faust/types/channels.py index f8468f692..553d4f394 100644 --- a/faust/types/channels.py +++ b/faust/types/channels.py @@ -60,7 +60,7 @@ def __init__( maxsize: Optional[int] = None, root: "ChannelT" = None, active_partitions: Optional[Set[TP]] = None, - loop: Optional[asyncio.AbstractEventLoop] = None + loop: Optional[asyncio.AbstractEventLoop] = None, ) -> None: ... @@ -95,7 +95,7 @@ async def send( key_serializer: CodecArg = None, value_serializer: CodecArg = None, callback: Optional[MessageSentCallback] = None, - force: bool = False + force: bool = False, ) -> Awaitable[RecordMetadata]: ... @@ -113,7 +113,7 @@ def send_soon( value_serializer: CodecArg = None, callback: Optional[MessageSentCallback] = None, force: bool = False, - eager_partitioning: bool = False + eager_partitioning: bool = False, ) -> FutureMessage: ... diff --git a/faust/types/models.py b/faust/types/models.py index db9ab3f44..b0a432270 100644 --- a/faust/types/models.py +++ b/faust/types/models.py @@ -155,7 +155,7 @@ def loads( s: bytes, *, default_serializer: CodecArg = None, # XXX use serializer - serializer: CodecArg = None + serializer: CodecArg = None, ) -> "ModelT": ... @@ -218,7 +218,7 @@ def __init__( parent: "FieldDescriptorT" = None, exclude: Optional[bool] = None, date_parser: Callable[[Any], datetime] = None, - **kwargs: Any + **kwargs: Any, ) -> None: # we have to do this in __init__ or mypy will think # this is a method diff --git a/faust/types/sensors.py b/faust/types/sensors.py index 50975c99c..47a08f675 100644 --- a/faust/types/sensors.py +++ b/faust/types/sensors.py @@ -133,7 +133,7 @@ def on_web_request_end( response: Optional[web.Response], state: Dict, *, - view: web.View = None + view: web.View = None, ) -> None: ... diff --git a/faust/types/serializers.py b/faust/types/serializers.py index 849a6a387..79163045a 100644 --- a/faust/types/serializers.py +++ b/faust/types/serializers.py @@ -43,7 +43,7 @@ def loads_key( typ: Optional[_ModelArg], key: Optional[bytes], *, - serializer: CodecArg = None + serializer: CodecArg = None, ) -> K: ... @@ -53,7 +53,7 @@ def loads_value( typ: Optional[_ModelArg], value: Optional[bytes], *, - serializer: CodecArg = None + serializer: CodecArg = None, ) -> Any: ... @@ -86,7 +86,7 @@ def __init__( value_type: _ModelArg = None, key_serializer: CodecArg = None, value_serializer: CodecArg = None, - allow_empty: Optional[bool] = None + allow_empty: Optional[bool] = None, ) -> None: ... @@ -98,7 +98,7 @@ def update( value_type: _ModelArg = None, key_serializer: CodecArg = None, value_serializer: CodecArg = None, - allow_empty: Optional[bool] = None + allow_empty: Optional[bool] = None, ) -> None: ... @@ -109,7 +109,7 @@ def loads_key( message: _Message, *, loads: Optional[Callable] = None, - serializer: CodecArg = None + serializer: CodecArg = None, ) -> KT: ... @@ -120,7 +120,7 @@ def loads_value( message: _Message, *, loads: Optional[Callable] = None, - serializer: CodecArg = None + serializer: CodecArg = None, ) -> VT: ... @@ -131,7 +131,7 @@ def dumps_key( key: K, *, serializer: CodecArg = None, - headers: OpenHeadersArg + headers: OpenHeadersArg, ) -> Tuple[Any, OpenHeadersArg]: ... @@ -142,7 +142,7 @@ def dumps_value( value: V, *, serializer: CodecArg = None, - headers: OpenHeadersArg + headers: OpenHeadersArg, ) -> Tuple[Any, OpenHeadersArg]: ... diff --git a/faust/types/stores.py b/faust/types/stores.py index 051203111..85da70f62 100644 --- a/faust/types/stores.py +++ b/faust/types/stores.py @@ -56,7 +56,7 @@ def __init__( key_serializer: CodecArg = "", value_serializer: CodecArg = "", options: Optional[Mapping[str, Any]] = None, - **kwargs: Any + **kwargs: Any, ) -> None: ... diff --git a/faust/types/streams.py b/faust/types/streams.py index 7fef2704d..ebc255f72 100644 --- a/faust/types/streams.py +++ b/faust/types/streams.py @@ -145,7 +145,7 @@ def __init__( active_partitions: Optional[Set[TP]] = None, enable_acks: bool = True, prefix: str = "", - loop: Optional[asyncio.AbstractEventLoop] = None + loop: Optional[asyncio.AbstractEventLoop] = None, ) -> None: ... @@ -198,7 +198,7 @@ def group_by( key: GroupByKeyArg, *, name: Optional[str] = None, - topic: Optional[TopicT] = None + topic: Optional[TopicT] = None, ) -> "StreamT": ... @@ -211,7 +211,7 @@ def derive_topic( key_type: ModelArg = None, value_type: ModelArg = None, prefix: str = "", - suffix: str = "" + suffix: str = "", ) -> TopicT: ... diff --git a/faust/types/tables.py b/faust/types/tables.py index 6d16e03eb..d5eef08e3 100644 --- a/faust/types/tables.py +++ b/faust/types/tables.py @@ -125,7 +125,7 @@ def __init__( options: Optional[Mapping[str, Any]] = None, use_partitioner: bool = False, on_window_close: Optional[WindowCloseCallback] = None, - **kwargs: Any + **kwargs: Any, ) -> None: ... @@ -464,7 +464,7 @@ def __init__( *, relative_to: RelativeArg = None, key_index: bool = False, - key_index_table: Optional[TableT] = None + key_index_table: Optional[TableT] = None, ) -> None: ... diff --git a/faust/types/topics.py b/faust/types/topics.py index 2f5a67206..9250747eb 100644 --- a/faust/types/topics.py +++ b/faust/types/topics.py @@ -88,7 +88,7 @@ def __init__( active_partitions: Optional[Set[TP]] = None, allow_empty: bool = False, has_prefix: bool = False, - loop: Optional[asyncio.AbstractEventLoop] = None + loop: Optional[asyncio.AbstractEventLoop] = None, ) -> None: ... @@ -130,6 +130,6 @@ def derive_topic( config: Optional[Mapping[str, Any]] = None, prefix: str = "", suffix: str = "", - **kwargs: Any + **kwargs: Any, ) -> "TopicT": ... diff --git a/faust/types/transports.py b/faust/types/transports.py index 68a2e8111..db2030bc9 100644 --- a/faust/types/transports.py +++ b/faust/types/transports.py @@ -125,7 +125,7 @@ def __init__( self, transport: "TransportT", loop: Optional[asyncio.AbstractEventLoop] = None, - **kwargs: Any + **kwargs: Any, ) -> None: ... @@ -139,7 +139,7 @@ async def send( timestamp: Optional[float], headers: Optional[HeadersArg], *, - transactional_id: Optional[str] = None + transactional_id: Optional[str] = None, ) -> Awaitable[RecordMetadata]: ... @@ -157,7 +157,7 @@ async def send_and_wait( timestamp: Optional[float], headers: Optional[HeadersArg], *, - transactional_id: Optional[str] = None + transactional_id: Optional[str] = None, ) -> RecordMetadata: ... @@ -173,7 +173,7 @@ async def create_topic( retention: Optional[Seconds] = None, compacting: Optional[bool] = None, deleting: Optional[bool] = None, - ensure_created: bool = False + ensure_created: bool = False, ) -> None: ... @@ -232,7 +232,7 @@ def __init__( *, consumer: "ConsumerT", producer: "ProducerT", - **kwargs: Any + **kwargs: Any, ) -> None: ... @@ -315,7 +315,7 @@ def __init__( *, commit_interval: Optional[float] = None, loop: Optional[asyncio.AbstractEventLoop] = None, - **kwargs: Any + **kwargs: Any, ) -> None: self._on_partitions_revoked: PartitionsRevokedCallback self._on_partitions_assigned: PartitionsAssignedCallback @@ -332,7 +332,7 @@ async def create_topic( retention: Optional[Seconds] = None, compacting: Optional[bool] = None, deleting: Optional[bool] = None, - ensure_created: bool = False + ensure_created: bool = False, ) -> None: ... diff --git a/faust/types/web.py b/faust/types/web.py index 31e998a09..6027a4aa1 100644 --- a/faust/types/web.py +++ b/faust/types/web.py @@ -131,7 +131,7 @@ def __init__( timeout: Optional[Seconds] = None, key_prefix: Optional[str] = None, backend: Union[Type[CacheBackendT], str] = None, - **kwargs: Any + **kwargs: Any, ) -> None: ... @@ -141,7 +141,7 @@ def view( timeout: Optional[Seconds] = None, include_headers: bool = False, key_prefix: Optional[str] = None, - **kwargs: Any + **kwargs: Any, ) -> Callable[[Callable], Callable]: ... @@ -172,7 +172,7 @@ def static( uri: str, file_or_directory: Union[str, Path], *, - name: Optional[str] = None + name: Optional[str] = None, ) -> None: ... diff --git a/faust/utils/terminal/tables.py b/faust/utils/terminal/tables.py index 7acf7ef79..e37b98f3a 100644 --- a/faust/utils/terminal/tables.py +++ b/faust/utils/terminal/tables.py @@ -30,7 +30,7 @@ def table( title: str, target: IO = None, tty: Optional[bool] = None, - **kwargs: Any + **kwargs: Any, ) -> Table: """Create suitable :pypi:`terminaltables` table for target. @@ -59,7 +59,7 @@ def logtable( target: IO = None, tty: Optional[bool] = None, headers: Optional[Sequence[str]] = None, - **kwargs: Any + **kwargs: Any, ) -> str: """Prepare table for logging. @@ -87,7 +87,7 @@ def dict_as_ansitable( sort: bool = False, sortkey: Callable[[Any], Any] = DEFAULT_SORT_KEY, target: IO = sys.stdout, - title: Optional[str] = None + title: Optional[str] = None, ) -> str: header = [text.title(key), text.title(value)] data = cast(Iterable[List[str]], d.items()) diff --git a/faust/utils/venusian.py b/faust/utils/venusian.py index 5e7aebddb..e8f7e25b2 100644 --- a/faust/utils/venusian.py +++ b/faust/utils/venusian.py @@ -16,7 +16,7 @@ def attach( category: str, *, callback: Callable[[Scanner, str, Any], None] = None, - **kwargs: Any + **kwargs: Any, ) -> None: """Shortcut for :func:`venusian.attach`. diff --git a/faust/web/exceptions.py b/faust/web/exceptions.py index db78319f6..e103d3695 100644 --- a/faust/web/exceptions.py +++ b/faust/web/exceptions.py @@ -38,7 +38,7 @@ def __init__( detail: Optional[str] = None, *, code: Optional[int] = None, - **extra_context: Any + **extra_context: Any, ) -> None: if detail: self.detail = detail diff --git a/pyproject.toml b/pyproject.toml new file mode 100644 index 000000000..1c86fac64 --- /dev/null +++ b/pyproject.toml @@ -0,0 +1,64 @@ +[project] +name = "faust-streaming" +requires-python = ">=3.8" +dynamic = ["version"] + +[build-system] +requires = [ + "setuptools", + "setuptools_scm[toml]", + "wheel", + "cython>=0.29; implementation_name == 'cpython'", + "cython>=3.0.0b3; implementation_name == 'cpython' and python_version >= '3.12'", +] +build-backend = "setuptools.build_meta" + +[tool.setuptools_scm] +# enables setuptools_scm to provide the dynamic version + +[tool.flake8] +ignore = [ + "W503", + "E203", + "B305", +] +max-line-length = 88 + +[tool.isort] +profile = "black" +combine_as_imports = true + +[tool.mypy] +ignore_missing_imports = true +disallow_untyped_defs = true + +[tool.pytest.ini_options] +minversion = "6.0" +addopts = [ + "--cov-report=term-missing:skip-covered", + "--cov=faust", + "-rxXs", + "--strict-config", + "--strict-markers", +] +filterwarnings = [ + "error::ResourceWarning", +] +xfail_strict = true + +testpaths = [ + "tests", + "integration", +] + + +[tool.cibuildwheel] +build = "cp3*" + +archs = ["auto64"] +skip = ["*musllinux*"] + +before-build = "pip install Cython" + +manylinux-x86_64-image = "manylinux2014" +manylinux-pypy_x86_64-image = "manylinux2014" diff --git a/requirements/test.txt b/requirements/test.txt index af9628679..9ada8e540 100644 --- a/requirements/test.txt +++ b/requirements/test.txt @@ -28,4 +28,3 @@ intervaltree -r extras/statsd.txt -r extras/yaml.txt -r extras/prometheus.txt --r extras/orjson.txt diff --git a/scripts/tests b/scripts/tests index c95762d75..c6661a54d 100755 --- a/scripts/tests +++ b/scripts/tests @@ -14,6 +14,6 @@ fi ${PREFIX}pytest tests/unit tests/functional tests/integration tests/meticulous/ tests/regression $@ ${PREFIX}bandit -b extra/bandit/baseline.json -c extra/bandit/config.yaml -r faust -if [ -z $GITHUB_ACTIONS ]; then - scripts/coverage -fi +#if [ -z $GITHUB_ACTIONS ]; then +# scripts/coverage +#fi diff --git a/setup.cfg b/setup.cfg index a2e537bd6..ac623699e 100644 --- a/setup.cfg +++ b/setup.cfg @@ -2,25 +2,5 @@ ignore = W503, E203, B305 max-line-length = 88 -[mypy] -disallow_untyped_defs = True -ignore_missing_imports = True - [mypy-tests.*] disallow_untyped_defs = False - -[tool:isort] -profile = black -combine_as_imports = True - -[tool:pytest] -addopts = - --cov-report=term-missing:skip-covered - --cov=faust - -rxXs - --strict-config - --strict-markers -filterwarnings = - error::ResourceWarning -xfail_strict=True - diff --git a/setup.py b/setup.py index d74e12d4c..3c8a1a3ab 100644 --- a/setup.py +++ b/setup.py @@ -46,7 +46,7 @@ LIBRARIES = [] E_UNSUPPORTED_PYTHON = NAME + " 1.0 requires Python %%s or later!" -if sys.version_info < (3, 7): +if sys.version_info < (3, 8): raise Exception(E_UNSUPPORTED_PYTHON % ("3.8",)) # NOQA from pathlib import Path # noqa @@ -110,7 +110,12 @@ def run(self): def build_extension(self, ext): try: build_ext.build_extension(self, ext) - except (CCompilerError, DistutilsExecError, DistutilsPlatformError, ValueError): + except ( + CCompilerError, + DistutilsExecError, + DistutilsPlatformError, + ValueError, + ): raise BuildFailed() @@ -233,7 +238,6 @@ def do_setup(**kwargs): "Programming Language :: Python :: 3.10", "Programming Language :: Python :: 3.11", "Programming Language :: Python :: Implementation :: CPython", - "Programming Language :: Python :: Implementation :: PyPy", "Operating System :: POSIX", "Operating System :: POSIX :: Linux", "Operating System :: MacOS :: MacOS X", @@ -242,7 +246,7 @@ def do_setup(**kwargs): "Topic :: System :: Networking", "Topic :: System :: Distributed Computing", ], - **kwargs + **kwargs, ) diff --git a/tests/functional/test_models.py b/tests/functional/test_models.py index 2c71f4a89..97a655a28 100644 --- a/tests/functional/test_models.py +++ b/tests/functional/test_models.py @@ -169,7 +169,10 @@ class OptionalListOfDate2(Record, coerce=True): assert OptionalDate.loads(OptionalDate(date=n1).dumps()).date == n1 assert OptionalDate.loads(OptionalDate(date=None).dumps()).date is None n2 = datetime.utcnow() - assert ListOfDate.loads(ListOfDate(dates=[n1, n2]).dumps()).dates == [n1, n2] + assert ListOfDate.loads(ListOfDate(dates=[n1, n2]).dumps()).dates == [ + n1, + n2, + ] assert ( OptionalListOfDate.loads(OptionalListOfDate(dates=None).dumps()).dates is None ) @@ -182,7 +185,10 @@ class OptionalListOfDate2(Record, coerce=True): assert OptionalListOfDate2.loads( OptionalListOfDate2(dates=[n1, n2]).dumps() ).dates == [n1, n2] - assert TupleOfDate.loads(TupleOfDate(dates=(n1, n2)).dumps()).dates == (n1, n2) + assert TupleOfDate.loads(TupleOfDate(dates=(n1, n2)).dumps()).dates == ( + n1, + n2, + ) assert TupleOfDate.loads(TupleOfDate(dates=(n2,)).dumps()).dates == (n2,) assert SetOfDate.loads(SetOfDate(dates={n1, n2}).dumps()).dates == {n1, n2} assert MapOfDate.loads(MapOfDate(dates={"A": n1, "B": n2}).dumps()).dates == { @@ -220,7 +226,10 @@ class OptionalListOfDate2(Record, coerce=False, isodates=True): n1 = datetime.utcnow() assert Date.loads(Date(date=n1).dumps()).date == n1 n2 = datetime.utcnow() - assert ListOfDate.loads(ListOfDate(dates=[n1, n2]).dumps()).dates == [n1, n2] + assert ListOfDate.loads(ListOfDate(dates=[n1, n2]).dumps()).dates == [ + n1, + n2, + ] assert ( OptionalListOfDate.loads(OptionalListOfDate(dates=None).dumps()).dates is None ) @@ -233,7 +242,10 @@ class OptionalListOfDate2(Record, coerce=False, isodates=True): assert OptionalListOfDate2.loads( OptionalListOfDate2(dates=[n1, n2]).dumps() ).dates == [n1, n2] - assert TupleOfDate.loads(TupleOfDate(dates=(n1, n2)).dumps()).dates == (n1, n2) + assert TupleOfDate.loads(TupleOfDate(dates=(n1, n2)).dumps()).dates == ( + n1, + n2, + ) assert TupleOfDate.loads(TupleOfDate(dates=(n2,)).dumps()).dates == (n2,) assert SetOfDate.loads(SetOfDate(dates={n1, n2}).dumps()).dates == {n1, n2} assert MapOfDate.loads(MapOfDate(dates={"A": n1, "B": n2}).dumps()).dates == { @@ -646,7 +658,9 @@ def test_getattr(self): Account(id="123", name="123"), Account(id="123", name="123", active=False), User( - id="123", username="foo", account=Account(id="123", name="Foo", active=True) + id="123", + username="foo", + account=Account(id="123", name="Foo", active=True), ), User(id="123", username="foo", account=None), User(id=None, username=None, account=None), @@ -1450,7 +1464,10 @@ def test_payload_with_reserved_keyword(): class X(Record): location: str = StringField(input_name="in") foo: str = StringField( - required=False, default="BAR", input_name="bar", output_name="foobar" + required=False, + default="BAR", + input_name="bar", + output_name="foobar", ) with pytest.raises(TypeError): diff --git a/tests/functional/web/test_cache.py b/tests/functional/web/test_cache.py index 63fedfd0b..92044312a 100644 --- a/tests/functional/web/test_cache.py +++ b/tests/functional/web/test_cache.py @@ -471,4 +471,7 @@ def test_cluster_client(self, app): client = backend._new_client() assert isinstance(client, aredis.StrictRedisCluster) pool = client.connection_pool - assert {"host": backend.url.host, "port": 3636} in pool.nodes.startup_nodes + assert { + "host": backend.url.host, + "port": 3636, + } in pool.nodes.startup_nodes diff --git a/tests/integration/cli/test_agents.py b/tests/integration/cli/test_agents.py index e25cacec4..227e3c1f9 100644 --- a/tests/integration/cli/test_agents.py +++ b/tests/integration/cli/test_agents.py @@ -8,7 +8,11 @@ def test_json(faust_json): "help": "Foo agent help.", } in agents assert {"name": "@app.add", "topic": "add-topic", "help": ""} in agents - assert {"name": "@app.internal", "topic": "", "help": ""} in agents + assert { + "name": "@app.internal", + "topic": "", + "help": "", + } in agents names = [agent["name"] for agent in agents] assert ( diff --git a/tests/integration/cli/test_model.py b/tests/integration/cli/test_model.py index 7f2e4b037..01d1fd583 100644 --- a/tests/integration/cli/test_model.py +++ b/tests/integration/cli/test_model.py @@ -4,7 +4,11 @@ def test_json(self, faust_json): assert not exitcode assert model == [ - {"field": "points", "type": "typing.List[__main__.Point]", "default": "*"}, + { + "field": "points", + "type": "typing.List[__main__.Point]", + "default": "*", + }, {"field": "timestamp", "type": "float", "default": "None"}, ] diff --git a/tests/integration/cli/test_models.py b/tests/integration/cli/test_models.py index 69c9c412b..495140589 100644 --- a/tests/integration/cli/test_models.py +++ b/tests/integration/cli/test_models.py @@ -4,7 +4,10 @@ def test_json(faust_json): assert {"name": "app.Arena", "help": ""} in models assert {"name": "app.Point", "help": ""} in models - assert {"name": "@ReqRepResponse", "help": "Request-Reply response."} in models + assert { + "name": "@ReqRepResponse", + "help": "Request-Reply response.", + } in models names = [model["name"] for model in models] assert ( diff --git a/tests/old/__init__.py b/tests/old/__init__.py deleted file mode 100644 index e69de29bb..000000000 diff --git a/tests/old/app.py b/tests/old/app.py deleted file mode 100644 index 3f5d57a48..000000000 --- a/tests/old/app.py +++ /dev/null @@ -1,12 +0,0 @@ -from typing import Any, AsyncIterator - -import faust - -app = faust.App("faust.stress") -rpc_topic = app.topic("faust.stress.rpc") - - -@app.agent(rpc_topic, concurrency=10) -async def simple(it: AsyncIterator[Any]) -> AsyncIterator[Any]: - async for value in it: - yield value diff --git a/tests/old/test_simple.py b/tests/old/test_simple.py deleted file mode 100644 index 03d158c82..000000000 --- a/tests/old/test_simple.py +++ /dev/null @@ -1,31 +0,0 @@ -from typing import Mapping - -import pytest -from mode import setup_logging - -from .app import simple - -setup_logging(loglevel="INFO") - - -def _build_data(i: int) -> Mapping: - return {"A": {"the": {"quick": {"brown": {"fox": i}}}}} - - -@pytest.mark.asyncio -async def test_simple_ask() -> None: - for i in range(100): - value = _build_data(i) - assert await simple.ask(value=value) == value - - -@pytest.mark.asyncio -async def test_simple_map() -> None: - values = [_build_data(i) for i in range(100)] - check = set(range(100)) - replies = set() - async for reply in simple.map(values): - v = reply["A"]["the"]["quick"]["brown"]["fox"] - assert v in check - replies.add(v) - assert replies == check diff --git a/tests/stress/app.py b/tests/stress/app.py index d81d8fbb6..e5e24dd78 100644 --- a/tests/stress/app.py +++ b/tests/stress/app.py @@ -68,7 +68,7 @@ def create_app(name, origin, base=faust.App, **kwargs: Any) -> faust.App: topic_partitions=config.topic_partitions, loghandlers=config.loghandlers(), autodiscover=True, - **kwargs + **kwargs, ) diff --git a/tests/stress/tests/forwarder/app.py b/tests/stress/tests/forwarder/app.py index c8e763a32..3396fca41 100644 --- a/tests/stress/tests/forwarder/app.py +++ b/tests/stress/tests/forwarder/app.py @@ -96,7 +96,9 @@ async def check(forwarded_numbers: Stream[int]) -> None: # number should be larger than the previous number. # if that's not true it means we have a duplicate. app.log.error( - "Found duplicate number in %r: %r", event.message.tp, number + "Found duplicate number in %r: %r", + event.message.tp, + number, ) found_duplicates += 1 else: diff --git a/tests/stress/tests/tables/app.py b/tests/stress/tests/tables/app.py index e533301b1..4876389ff 100644 --- a/tests/stress/tests/tables/app.py +++ b/tests/stress/tests/tables/app.py @@ -137,7 +137,9 @@ async def process(numbers: Stream[int]) -> None: # WITH THE OFFSET after all!) # if the number is less than we have a problem. app.log.error( - "Found duplicate number in %r: %r", event.message.tp, number + "Found duplicate number in %r: %r", + event.message.tp, + number, ) found_duplicates += 1 else: diff --git a/tests/unit/agents/test_agent.py b/tests/unit/agents/test_agent.py index 3498b3176..bb31b75b8 100644 --- a/tests/unit/agents/test_agent.py +++ b/tests/unit/agents/test_agent.py @@ -740,7 +740,10 @@ def test_create_req(self, *, agent): with patch("faust.agents.agent.uuid4") as uuid4: uuid4.return_value = "vvv" reqrep = agent._create_req( - key=b"key", value=b"value", reply_to="reply_to", headers={"k": "v"} + key=b"key", + value=b"value", + reply_to="reply_to", + headers={"k": "v"}, )[0] agent._get_strtopic.assert_called_once_with("reply_to") @@ -755,7 +758,10 @@ def test_create_req__use_reply_headers(self, *, agent): with patch("faust.agents.agent.uuid4") as uuid4: uuid4.return_value = "vvv" value, h = agent._create_req( - key=b"key", value=b"value", reply_to="reply_to", headers={"k": "v"} + key=b"key", + value=b"value", + reply_to="reply_to", + headers={"k": "v"}, ) agent._get_strtopic.assert_called_once_with("reply_to") @@ -771,7 +777,10 @@ def test_create_req__model(self, *, agent): uuid4.return_value = "vvv" value = Word("foo") reqrep = agent._create_req( - key=b"key", value=value, reply_to="reply_to", headers={"h1": "h2"} + key=b"key", + value=value, + reply_to="reply_to", + headers={"h1": "h2"}, )[0] assert isinstance(reqrep, ReqRepRequest) diff --git a/tests/unit/livecheck/test_case.py b/tests/unit/livecheck/test_case.py index ab45970cf..983d1d1db 100644 --- a/tests/unit/livecheck/test_case.py +++ b/tests/unit/livecheck/test_case.py @@ -224,7 +224,16 @@ async def test__set_test_error_state(self, state, failures, fail_suite, *, case) ], ) async def test_on_suite_pass( - self, initial_state, ts, now, failed, expected_state, *, case, runner, execution + self, + initial_state, + ts, + now, + failed, + expected_state, + *, + case, + runner, + execution, ): assert runner.test is execution runner.test.timestamp = Mock() diff --git a/tests/unit/sensors/test_datadog.py b/tests/unit/sensors/test_datadog.py index 78a103bea..731e74612 100644 --- a/tests/unit/sensors/test_datadog.py +++ b/tests/unit/sensors/test_datadog.py @@ -241,7 +241,10 @@ def test_on_send_initiated_completed(self, *, mon): client.increment.assert_has_calls( [ call( - "messages_send_failed", sample_rate=mon.rate, tags=None, value=1.0 + "messages_send_failed", + sample_rate=mon.rate, + tags=None, + value=1.0, ), ] ) @@ -265,7 +268,10 @@ def test_on_assignment_start_completed(self, *, mon): client.increment.assert_has_calls( [ call( - "assignments_complete", sample_rate=mon.rate, tags=None, value=1.0 + "assignments_complete", + sample_rate=mon.rate, + tags=None, + value=1.0, ), ] ) @@ -284,7 +290,12 @@ def test_on_assignment_start_error(self, *, mon): client = mon.client.client client.increment.assert_has_calls( [ - call("assignments_error", sample_rate=mon.rate, tags=None, value=1.0), + call( + "assignments_error", + sample_rate=mon.rate, + tags=None, + value=1.0, + ), ] ) client.timing.assert_called_once_with( @@ -344,7 +355,10 @@ def test_on_rebalance(self, *, mon): [ call("rebalances", sample_rate=mon.rate, tags=None, value=1.0), call( - "rebalances_recovering", sample_rate=mon.rate, tags=None, value=1.0 + "rebalances_recovering", + sample_rate=mon.rate, + tags=None, + value=1.0, ), ] ) @@ -367,7 +381,10 @@ def test_on_rebalance(self, *, mon): [ call("rebalances", sample_rate=mon.rate, tags=None, value=1.0), call( - "rebalances_recovering", sample_rate=mon.rate, tags=None, value=1.0 + "rebalances_recovering", + sample_rate=mon.rate, + tags=None, + value=1.0, ), ] ) diff --git a/tests/unit/sensors/test_monitor.py b/tests/unit/sensors/test_monitor.py index 169184615..7eefcfe82 100644 --- a/tests/unit/sensors/test_monitor.py +++ b/tests/unit/sensors/test_monitor.py @@ -67,7 +67,7 @@ def create_populated_monitor( commit_latency=[1.03, 2.33, 16.33], # noqa send_latency=[0.01, 0.04, 0.06, 0.010], # noqa topic_buffer_full={"topic": 808}, # noqa - **kwargs + **kwargs, ): return self.create_monitor( messages_active=messages_active, @@ -84,7 +84,7 @@ def create_populated_monitor( commit_latency=commit_latency, send_latency=send_latency, topic_buffer_full=topic_buffer_full, - **kwargs + **kwargs, ) def test_init_max_avg_history(self): @@ -266,7 +266,11 @@ def test_on_commit_completed(self, *, mon, time): def test_on_send_initiated(self, *, mon, time): for i in range(1, 11): state = mon.on_send_initiated( - Mock(name="producer", autospec=Producer), "topic", "message", 2, 4 + Mock(name="producer", autospec=Producer), + "topic", + "message", + 2, + 4, ) assert mon.messages_sent == i assert mon.messages_sent_by_topic["topic"] == i diff --git a/tests/unit/sensors/test_prometheus.py b/tests/unit/sensors/test_prometheus.py index 3d729df55..0df2f41b9 100644 --- a/tests/unit/sensors/test_prometheus.py +++ b/tests/unit/sensors/test_prometheus.py @@ -150,7 +150,9 @@ def test_on_stream_event_out_does_not_measure_latency_without_state( monitor.on_stream_event_out(TP1, 401, stream, event, state=None) self.assert_doesnt_have_sample_values( - metrics.events_runtime_latency, "test_events_runtime_latency_total", {} + metrics.events_runtime_latency, + "test_events_runtime_latency_total", + {}, ) def test_on_table_get( @@ -310,7 +312,10 @@ def test_on_rebalance_return( monitor.on_rebalance_return(app, state) self.assert_has_sample_value( - metrics.total_rebalances, "test_total_rebalances", {}, n_rebalances - 1 + metrics.total_rebalances, + "test_total_rebalances", + {}, + n_rebalances - 1, ) self.assert_has_sample_value( metrics.total_rebalances_recovering, diff --git a/tests/unit/sensors/test_statsd.py b/tests/unit/sensors/test_statsd.py index fe72be611..c72625615 100644 --- a/tests/unit/sensors/test_statsd.py +++ b/tests/unit/sensors/test_statsd.py @@ -153,7 +153,9 @@ def test_on_send_initiated_completed(self, *, mon): mon.client.timing.assert_has_calls( [ call( - "send_latency_for_error", mon.ms_since(float(state)), rate=mon.rate + "send_latency_for_error", + mon.ms_since(float(state)), + rate=mon.rate, ), ] ) diff --git a/tests/unit/serializers/test_codecs.py b/tests/unit/serializers/test_codecs.py index 494c4b3eb..6abf80538 100644 --- a/tests/unit/serializers/test_codecs.py +++ b/tests/unit/serializers/test_codecs.py @@ -36,7 +36,11 @@ def test_interface(): def test_json_subset(codec: str) -> None: if codec == "json": # special exception for json since integers can be serialized - assert loads(codec, dumps(codec, DATA)) == {"a": 1, "b": "string", "1": 2} + assert loads(codec, dumps(codec, DATA)) == { + "a": 1, + "b": "string", + "1": 2, + } else: assert loads(codec, dumps(codec, DATA)) == DATA diff --git a/tests/unit/stores/test_aerospike.py b/tests/unit/stores/test_aerospike.py index 7c6fcb6a1..ebb0ee866 100644 --- a/tests/unit/stores/test_aerospike.py +++ b/tests/unit/stores/test_aerospike.py @@ -170,7 +170,11 @@ def test_itervalues_success(self, store): scan = MagicMock() store.client.scan = MagicMock(return_value=scan) scan_result = [ - (MagicMock(), {"ttl": 4294967295, "gen": 4}, {"value_key": "value1"}), + ( + MagicMock(), + {"ttl": 4294967295, "gen": 4}, + {"value_key": "value1"}, + ), (MagicMock(), {"ttl": 4294967295, "gen": 4}, None), ] scan.results = MagicMock(return_value=scan_result) diff --git a/tests/unit/tables/test_recovery.py b/tests/unit/tables/test_recovery.py index b60b36b15..81fcad8d5 100644 --- a/tests/unit/tables/test_recovery.py +++ b/tests/unit/tables/test_recovery.py @@ -392,7 +392,13 @@ def test__is_changelog_tp(self, *, recovery, tables): [ ({TP1: 0, TP2: -1}, {TP1: -1, TP2: -1}, True, 1, {TP1: 1, TP2: 0}), ({TP1: -1, TP2: -1}, {TP1: -1, TP2: -1}, False, 0, {TP1: 0, TP2: 0}), - ({TP1: 100, TP2: -1}, {TP1: -1, TP2: -1}, True, 101, {TP1: 101, TP2: 0}), + ( + {TP1: 100, TP2: -1}, + {TP1: -1, TP2: -1}, + True, + 101, + {TP1: 101, TP2: 0}, + ), ], ) def test_recovery_from_offset_0( diff --git a/tests/unit/tables/test_table.py b/tests/unit/tables/test_table.py index c10bb5c3d..29e0419c2 100644 --- a/tests/unit/tables/test_table.py +++ b/tests/unit/tables/test_table.py @@ -55,7 +55,13 @@ def strict_table(self, *, app): return self.create_table(app, name="strict") def create_table( - self, app, *, name="foo", key_type=TableKey, value_type=TableValue, **kwargs + self, + app, + *, + name="foo", + key_type=TableKey, + value_type=TableValue, + **kwargs, ): return app.Table(name, key_type=key_type, value_type=value_type, **kwargs) diff --git a/tests/unit/test_auth.py b/tests/unit/test_auth.py index d1ad9fc13..57ecd84d0 100644 --- a/tests/unit/test_auth.py +++ b/tests/unit/test_auth.py @@ -36,14 +36,18 @@ class Test_SASLCredentials: pytest.param( "mechanism from enum", SASLCredentials( - username="george", password="pw1", mechanism=SASLMechanism.GSSAPI + username="george", + password="pw1", + mechanism=SASLMechanism.GSSAPI, ), {"mechanism": SASLMechanism.GSSAPI}, ), pytest.param( "ssl context gives SASL_SSL", SASLCredentials( - username="george", password="pw1", ssl_context={"xxx": "yyy"} + username="george", + password="pw1", + ssl_context={"xxx": "yyy"}, ), { "username": "george", @@ -76,7 +80,8 @@ class Test_GSSAPICredentials: pytest.param( "retains arguments", GSSAPICredentials( - kerberos_service_name="george", kerberos_domain_name="domain" + kerberos_service_name="george", + kerberos_domain_name="domain", ), { "kerberos_service_name": "george", diff --git a/tests/unit/transport/drivers/test_aiokafka.py b/tests/unit/transport/drivers/test_aiokafka.py index bfa405fd8..76f28c2df 100644 --- a/tests/unit/transport/drivers/test_aiokafka.py +++ b/tests/unit/transport/drivers/test_aiokafka.py @@ -137,7 +137,13 @@ def thread(self): @pytest.fixture() def consumer( - self, *, thread, app, callback, on_partitions_revoked, on_partitions_assigned + self, + *, + thread, + app, + callback, + on_partitions_revoked, + on_partitions_assigned, ): consumer = Consumer( app.transport, @@ -1420,7 +1426,10 @@ async def test_commit_transactions(self, *, producer, _producer_call): def test__settings_extra(self, *, producer, app): app.in_transaction = True - assert producer._settings_extra() == {"acks": "all", "enable_idempotence": True} + assert producer._settings_extra() == { + "acks": "all", + "enable_idempotence": True, + } app.in_transaction = False assert producer._settings_extra() == {} @@ -1442,14 +1451,16 @@ def test__new_producer(self, *, app): marks=pytest.mark.conf(broker="kafka://a:9092;b:9092"), ), pytest.param( - {"client_id": "foo"}, marks=pytest.mark.conf(broker_client_id="foo") + {"client_id": "foo"}, + marks=pytest.mark.conf(broker_client_id="foo"), ), pytest.param( {"compression_type": "snappy"}, marks=pytest.mark.conf(producer_compression_type="snappy"), ), pytest.param( - {"linger_ms": 9345}, marks=pytest.mark.conf(producer_linger=9.345) + {"linger_ms": 9345}, + marks=pytest.mark.conf(producer_linger=9.345), ), pytest.param( {"max_batch_size": 41223}, @@ -1776,7 +1787,11 @@ def mocked_producer(self, *, new_producer_mock: Mock): @pytest.mark.asyncio async def test_on_start( - self, *, threaded_producer: ThreadedProducer, mocked_producer: Mock, loop + self, + *, + threaded_producer: ThreadedProducer, + mocked_producer: Mock, + loop, ): await threaded_producer.on_start() try: @@ -1790,7 +1805,11 @@ async def test_on_start( @pytest.mark.skip("Needs fixing") @pytest.mark.asyncio async def test_on_thread_stop( - self, *, threaded_producer: ThreadedProducer, mocked_producer: Mock, loop + self, + *, + threaded_producer: ThreadedProducer, + mocked_producer: Mock, + loop, ): await threaded_producer.start() await threaded_producer.on_thread_stop() @@ -1803,7 +1822,11 @@ async def test_on_thread_stop( @pytest.mark.asyncio async def test_publish_message( - self, *, threaded_producer: ThreadedProducer, mocked_producer: Mock, loop + self, + *, + threaded_producer: ThreadedProducer, + mocked_producer: Mock, + loop, ): await threaded_producer.start() try: @@ -1828,7 +1851,11 @@ async def test_publish_message( @pytest.mark.asyncio async def test_publish_message_with_wait( - self, *, threaded_producer: ThreadedProducer, mocked_producer: Mock, loop + self, + *, + threaded_producer: ThreadedProducer, + mocked_producer: Mock, + loop, ): await threaded_producer.start() try: diff --git a/tests/unit/transport/test_consumer.py b/tests/unit/transport/test_consumer.py index 008ea06b9..89c33ae21 100644 --- a/tests/unit/transport/test_consumer.py +++ b/tests/unit/transport/test_consumer.py @@ -1095,7 +1095,12 @@ def test_new_offset(self, tp, acked, expected_offset, expected_acked, *, consume [ (TP1, [], [], None), (TP1, [1, 2, 3, 4, 5, 6, 7, 8, 9, 10], IntervalTree(), 11), - (TP1, [1, 2, 3, 4, 5, 6, 7, 8, 10], IntervalTree([Interval(9, 10)]), 11), + ( + TP1, + [1, 2, 3, 4, 5, 6, 7, 8, 10], + IntervalTree([Interval(9, 10)]), + 11, + ), (TP1, [1, 2, 3, 4, 6, 7, 8, 10], IntervalTree([Interval(5, 6)]), 9), ( TP1, diff --git a/tests/unit/utils/terminal/test_tables.py b/tests/unit/utils/terminal/test_tables.py index e903b4d81..f607a7a80 100644 --- a/tests/unit/utils/terminal/test_tables.py +++ b/tests/unit/utils/terminal/test_tables.py @@ -59,9 +59,21 @@ def test_table(target, contexts, kwargs, expected_tty): "tty,contexts,headers,expected_tty,expected_data", [ (None, [mock_logging(isatty=True)], None, True, TABLE_DATA), - (None, [mock_logging(isatty=True)], ["foo"], True, [["foo"]] + TABLE_DATA), + ( + None, + [mock_logging(isatty=True)], + ["foo"], + True, + [["foo"]] + TABLE_DATA, + ), (None, [mock_logging(isatty=False)], None, False, TABLE_DATA), - (None, [mock_logging(isatty=False)], ["f"], False, [["f"]] + TABLE_DATA), + ( + None, + [mock_logging(isatty=False)], + ["f"], + False, + [["f"]] + TABLE_DATA, + ), (True, [], None, True, TABLE_DATA), (True, [], ["foo"], True, [["foo"]] + TABLE_DATA), (False, [], None, False, TABLE_DATA), diff --git a/tests/unit/web/drivers/test_aiohttp.py b/tests/unit/web/drivers/test_aiohttp.py index a9adb1d9c..1748f1363 100644 --- a/tests/unit/web/drivers/test_aiohttp.py +++ b/tests/unit/web/drivers/test_aiohttp.py @@ -294,7 +294,8 @@ def test_add_static(self, *, web): def test_route__with_cors_options(self, *, web): handler = Mock() handler.get_methods = Mock( - name="get_methods", return_value=set({"GET", "PUT", "POST", "DELETE"}) + name="get_methods", + return_value=set({"GET", "PUT", "POST", "DELETE"}), ) cors_options = { "http://example.com": ResourceOptions( @@ -321,7 +322,8 @@ def test_route__with_cors_options(self, *, web): web._cors.add.assert_has_calls( [ call( - web.web_app.router.add_route(), _prepare_cors_options(cors_options) + web.web_app.router.add_route(), + _prepare_cors_options(cors_options), ) for _ in NON_OPTIONS_METHODS ], From 7eea54a77d2c01a1c16d4ffaa5cba53dccaec91c Mon Sep 17 00:00:00 2001 From: William Barnhart Date: Thu, 29 Jun 2023 19:17:57 -0400 Subject: [PATCH 06/54] Enable PyPy testing (#530) * Skip failing tests on PyPy for now * use move pypy3.9 into main tests * never mind pypy is experimental then * skip meticulous tests on pypy * no cython w/pypy for now * skip janky tests on pypy * lint * rename workflow * add asyncio_mode = "auto" to pyproject.toml * undo disabled tests in functional test_streams.py * enable regular stream testing for unit tests --- .github/workflows/python-package.yml | 16 ++++++++-------- pyproject.toml | 1 + .../assignor/test_copartitioned_assignor.py | 11 +++++++++++ 3 files changed, 20 insertions(+), 8 deletions(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 5670989c2..2a358b591 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -1,5 +1,5 @@ --- -name: Test Python library +name: CI/CD on: push: branches: [master] @@ -48,13 +48,13 @@ jobs: python-version: ['3.8', '3.9', '3.10', '3.11'] use-cython: ['true', 'false'] experimental: [false] -# include: -# - python-version: pypy3.9 -# experimental: true -# use-cython: 'false' -# - python-version: ~3.12.0-0 -# experimental: true -# use-cython: 'false' + include: + - python-version: 'pypy3.9' + use-cython: false + experimental: true +# - python-version: ~3.12.0-0 +# experimental: true +# use-cython: 'false' env: USE_CYTHON: ${{ matrix.use-cython }} continue-on-error: ${{ matrix.experimental }} diff --git a/pyproject.toml b/pyproject.toml index 1c86fac64..043c42765 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -33,6 +33,7 @@ ignore_missing_imports = true disallow_untyped_defs = true [tool.pytest.ini_options] +asyncio_mode = "auto" minversion = "6.0" addopts = [ "--cov-report=term-missing:skip-covered", diff --git a/tests/meticulous/assignor/test_copartitioned_assignor.py b/tests/meticulous/assignor/test_copartitioned_assignor.py index 8179012e1..05c2f0ce8 100644 --- a/tests/meticulous/assignor/test_copartitioned_assignor.py +++ b/tests/meticulous/assignor/test_copartitioned_assignor.py @@ -1,7 +1,9 @@ import copy +import platform from collections import Counter from typing import MutableMapping +import pytest from hypothesis import assume, given, settings from hypothesis.strategies import integers @@ -77,6 +79,9 @@ def client_removal_sticky( return True +@pytest.mark.skipif( + platform.python_implementation() == "PyPy", reason="Not yet supported on PyPy" +) @given( partitions=integers(min_value=0, max_value=256), replicas=integers(min_value=0, max_value=64), @@ -95,6 +100,9 @@ def test_fresh_assignment(partitions, replicas, num_clients): assert is_valid(new_assignments, partitions, replicas) +@pytest.mark.skipif( + platform.python_implementation() == "PyPy", reason="Not yet supported on PyPy" +) @given( partitions=integers(min_value=0, max_value=256), replicas=integers(min_value=0, max_value=64), @@ -131,6 +139,9 @@ def test_add_new_clients(partitions, replicas, num_clients, num_additional_clien assert clients_balanced(new_assignments) +@pytest.mark.skipif( + platform.python_implementation() == "PyPy", reason="Not yet supported on PyPy" +) @given( partitions=integers(min_value=0, max_value=256), replicas=integers(min_value=0, max_value=64), From 236516d29c7202d116939fe9887e24cb112bf009 Mon Sep 17 00:00:00 2001 From: Oleksandr Stetsenko Date: Fri, 30 Jun 2023 19:11:10 +0300 Subject: [PATCH 07/54] Topic: Remove topic with active_partitions from Conductor after (#531) Co-authored-by: o.stetsenko --- faust/topics.py | 10 ++++ tests/functional/agents/helpers.py | 15 +++-- .../agents/test_isolated_partitions.py | 59 ++++++++++++++++++- 3 files changed, 77 insertions(+), 7 deletions(-) diff --git a/faust/topics.py b/faust/topics.py index e01cd0dfa..90b5e1bd6 100644 --- a/faust/topics.py +++ b/faust/topics.py @@ -505,6 +505,16 @@ async def declare(self) -> None: retention=self.retention, ) + def on_stop_iteration(self) -> None: + """Signal that iteration over this channel was stopped. + Tip: + Remember to call ``super`` when overriding this method. + """ + super().on_stop_iteration() + if self.active_partitions is not None: + # Remove topics for isolated partitions from the Conductor. + self.app.topics.discard(cast(TopicT, self)) + def __aiter__(self) -> ChannelT: if self.is_iterator: return self diff --git a/tests/functional/agents/helpers.py b/tests/functional/agents/helpers.py index 583066bca..ca70c6955 100644 --- a/tests/functional/agents/helpers.py +++ b/tests/functional/agents/helpers.py @@ -59,9 +59,7 @@ def __init__( self.isolated_partitions = isolated_partitions self.topic_name = self.name - self.tps = [TP(self.topic_name, p) for p in self.partitions] - self.next_tp = cycle(self.tps) - self.expected_tp = cycle(self.tps) + self._set_tps_from_partitions() self.seen_offsets = set() self.processed_total = 0 @@ -72,6 +70,11 @@ def __init__( self.agent_stopped_processing = asyncio.Event() self.finished = asyncio.Event() + def _set_tps_from_partitions(self): + self.tps = [TP(self.topic_name, p) for p in self.partitions] + self.next_tp = cycle(self.tps) + self.expected_tp = cycle(self.tps) + async def on_start(self) -> None: app = self.app topic = app.topic(self.topic_name, value_serializer=self.value_serializer) @@ -153,8 +156,10 @@ async def _send(self) -> None: self.finished.set() - async def conductor_setup(self, assigned: Set[TP]) -> None: - await self.app.agents.on_rebalance(set(), assigned) + async def conductor_setup( + self, assigned: Set[TP], revoked: Optional[Set[TP]] = None + ) -> None: + await self.app.agents.on_rebalance(revoked or set(), assigned) await self.app.topics._update_indices() await self.app.topics.on_partitions_assigned(assigned) diff --git a/tests/functional/agents/test_isolated_partitions.py b/tests/functional/agents/test_isolated_partitions.py index 33e71f5f1..45d6fb0f7 100644 --- a/tests/functional/agents/test_isolated_partitions.py +++ b/tests/functional/agents/test_isolated_partitions.py @@ -1,11 +1,11 @@ from collections import Counter from pprint import pformat -from typing import List, NamedTuple +from typing import Any, List, Mapping, NamedTuple import pytest from faust.exceptions import ImproperlyConfigured -from faust.types import EventT, StreamT +from faust.types import AppT, EventT, Message as MessageT, StreamT from .helpers import AgentCase @@ -51,6 +51,25 @@ async def test_agent_isolated_partitions__concurrency(*, app, logging): ) +@pytest.mark.asyncio +async def test_agent_isolated_partitions_rebalancing(*, app, logging): + await AgentIsolatedRebalanceCase.run_test( + app=app, + num_messages=100, + concurrency=1, + partitions=[0, 1, 2, 3], + reassign_partitions={ + 10: [0], + 20: [1], + 30: [0, 1], + 40: [2, 3], + 50: [0, 1, 2, 3], + 60: [4, 5, 6, 7], + }, + isolated_partitions=True, + ) + + class AgentIsolatedCase(AgentCase): name = "test_agent_isolated_partitions" @@ -89,3 +108,39 @@ async def assert_success(self) -> None: if max_ is None: max_ = total assert total == max_ + + +class AgentIsolatedRebalanceCase(AgentCase): + name = "test_agent_isolated_partitions_rebalancing" + + @classmethod + async def run_test( + cls, app: AppT, *, reassign_partitions: Mapping[int, List[int]], **kwargs: Any + ) -> "AgentCase": + return await super().run_test( + app, reassign_partitions=reassign_partitions, **kwargs + ) + + def __init__( + self, app: AppT, *, reassign_partitions: Mapping[int, List[int]], **kwargs: Any + ) -> None: + super().__init__(app, **kwargs) + self.reassign_partitions = reassign_partitions + + async def put(self, key: bytes, value: bytes, **kwargs: Any) -> MessageT: + message = await super().put(key, value, **kwargs) + + new_partitions = self.reassign_partitions.get(int(message.key)) + if new_partitions is not None: + await self.simulate_rebalance(new_partitions) + + return message + + async def simulate_rebalance(self, partitions: List[int]): + await self.sleep(0.1) + self.partitions = sorted(partitions) + current_tps = set(self.tps) + self._set_tps_from_partitions() + assigned = set(self.tps) + revoked = current_tps - assigned + await self.conductor_setup(assigned=assigned, revoked=revoked) From ddf27f59fc95f5857b2597490a127e8e23c79e7c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 11 Jul 2023 14:02:53 -0400 Subject: [PATCH 08/54] Bump actions/upload-pages-artifact from 1 to 2 (#533) Bumps [actions/upload-pages-artifact](https://github.com/actions/upload-pages-artifact) from 1 to 2. - [Release notes](https://github.com/actions/upload-pages-artifact/releases) - [Commits](https://github.com/actions/upload-pages-artifact/compare/v1...v2) --- updated-dependencies: - dependency-name: actions/upload-pages-artifact dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/gh-pages.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/gh-pages.yml b/.github/workflows/gh-pages.yml index fda7e9ddc..6e2401aeb 100644 --- a/.github/workflows/gh-pages.yml +++ b/.github/workflows/gh-pages.yml @@ -24,7 +24,7 @@ jobs: - name: Install doc build deps and build with Sphinx run: make docs - name: Upload artifacts - uses: actions/upload-pages-artifact@v1 + uses: actions/upload-pages-artifact@v2 with: # Upload built docs path: ./Documentation From 4a234204ee26bb28472a6640ccab286b807a3681 Mon Sep 17 00:00:00 2001 From: Chris Tam Date: Thu, 27 Jul 2023 13:56:37 -0400 Subject: [PATCH 09/54] Use rocksdict version >= v0.3.11 (#538) * Bump rocksdict to v0.3.13 * Use rocksdict>=0.3.11,<4.0 --- requirements/extras/rocksdict.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/requirements/extras/rocksdict.txt b/requirements/extras/rocksdict.txt index 5a2a31e40..48647c157 100644 --- a/requirements/extras/rocksdict.txt +++ b/requirements/extras/rocksdict.txt @@ -1 +1 @@ -rocksdict==0.3.11 +rocksdict>=0.3.11,<4.0 From 87a80a968f73220d5ac6190fb7df70b85427bdae Mon Sep 17 00:00:00 2001 From: William Barnhart Date: Tue, 8 Aug 2023 21:53:41 -0400 Subject: [PATCH 10/54] Update python-package.yml to test for Python 3.12 (#540) --- .github/workflows/python-package.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 2a358b591..b1aa54e5c 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -52,9 +52,9 @@ jobs: - python-version: 'pypy3.9' use-cython: false experimental: true -# - python-version: ~3.12.0-0 -# experimental: true -# use-cython: 'false' + - python-version: ~3.12.0-0 + experimental: true + use-cython: false env: USE_CYTHON: ${{ matrix.use-cython }} continue-on-error: ${{ matrix.experimental }} From b5db8a0d28bd486c987a1f38599831432f4aa77e Mon Sep 17 00:00:00 2001 From: Daniel Gellert Date: Mon, 14 Aug 2023 18:26:43 +0200 Subject: [PATCH 11/54] fix: ensure aiokafka commit is called with kafka.structs.TopicPartition (#539) (#541) --- faust/transport/drivers/aiokafka.py | 24 +++++++++++++++---- tests/unit/transport/drivers/test_aiokafka.py | 7 ++++++ 2 files changed, 27 insertions(+), 4 deletions(-) diff --git a/faust/transport/drivers/aiokafka.py b/faust/transport/drivers/aiokafka.py index 032e79f8e..4a0c9ebb2 100644 --- a/faust/transport/drivers/aiokafka.py +++ b/faust/transport/drivers/aiokafka.py @@ -711,13 +711,15 @@ async def commit(self, offsets: Mapping[TP, int]) -> bool: async def _commit(self, offsets: Mapping[TP, int]) -> bool: consumer = self._ensure_consumer() now = monotonic() + commitable_offsets = { + tp: offset for tp, offset in offsets.items() if tp in self.assignment() + } try: aiokafka_offsets = { - tp: OffsetAndMetadata(offset, "") - for tp, offset in offsets.items() - if tp in self.assignment() + ensure_aiokafka_TP(tp): OffsetAndMetadata(offset, "") + for tp, offset in commitable_offsets.items() } - self.tp_last_committed_at.update({tp: now for tp in aiokafka_offsets}) + self.tp_last_committed_at.update({tp: now for tp in commitable_offsets}) await consumer.commit(aiokafka_offsets) except CommitFailedError as exc: if "already rebalanced" in str(exc): @@ -1621,3 +1623,17 @@ def credentials_to_aiokafka_auth( } else: return {"security_protocol": "PLAINTEXT"} + + +def ensure_aiokafka_TP(tp: TP) -> _TopicPartition: + """Convert Faust ``TP`` to aiokafka ``TopicPartition``.""" + return ( + tp + if isinstance(tp, _TopicPartition) + else _TopicPartition(tp.topic, tp.partition) + ) + + +def ensure_aiokafka_TPset(tps: Iterable[TP]) -> Set[_TopicPartition]: + """Convert set of Faust ``TP`` to aiokafka ``TopicPartition``.""" + return {ensure_aiokafka_TP(tp) for tp in tps} diff --git a/tests/unit/transport/drivers/test_aiokafka.py b/tests/unit/transport/drivers/test_aiokafka.py index 76f28c2df..1130f2cb8 100644 --- a/tests/unit/transport/drivers/test_aiokafka.py +++ b/tests/unit/transport/drivers/test_aiokafka.py @@ -34,6 +34,7 @@ ThreadedProducer, Transport, credentials_to_aiokafka_auth, + ensure_aiokafka_TPset, server_list, ) from faust.types import TP @@ -2038,3 +2039,9 @@ def test_credentials_to_aiokafka(credentials, ssl_context, expected): def test_credentials_to_aiokafka__invalid(): with pytest.raises(ImproperlyConfigured): credentials_to_aiokafka_auth(object()) + + +def test_ensure_aiokafka_TPset(): + actual = ensure_aiokafka_TPset({TP(topic="foo", partition=0)}) + assert actual == {TopicPartition("foo", 0)} + assert all(isinstance(tp, TopicPartition) for tp in actual) From e900e548d7261f4fc2b18244ed943fc4beec87c9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 5 Sep 2023 13:49:54 -0400 Subject: [PATCH 12/54] Bump actions/checkout from 3 to 4 (#546) Bumps [actions/checkout](https://github.com/actions/checkout) from 3 to 4. - [Release notes](https://github.com/actions/checkout/releases) - [Changelog](https://github.com/actions/checkout/blob/main/CHANGELOG.md) - [Commits](https://github.com/actions/checkout/compare/v3...v4) --- updated-dependencies: - dependency-name: actions/checkout dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/codeql-analysis.yml | 2 +- .github/workflows/gh-pages.yml | 2 +- .github/workflows/python-package.yml | 8 ++++---- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/.github/workflows/codeql-analysis.yml b/.github/workflows/codeql-analysis.yml index 43427fab9..0d5078b39 100644 --- a/.github/workflows/codeql-analysis.yml +++ b/.github/workflows/codeql-analysis.yml @@ -36,7 +36,7 @@ jobs: # https://docs.github.com/en/free-pro-team@latest/github/finding-security-vulnerabilities-and-errors-in-your-code/configuring-code-scanning#changing-the-languages-that-are-analyzed steps: - name: Checkout repository - uses: actions/checkout@v3 + uses: actions/checkout@v4 # Initializes the CodeQL tools for scanning. - name: Initialize CodeQL diff --git a/.github/workflows/gh-pages.yml b/.github/workflows/gh-pages.yml index 6e2401aeb..e56a79297 100644 --- a/.github/workflows/gh-pages.yml +++ b/.github/workflows/gh-pages.yml @@ -14,7 +14,7 @@ jobs: runs-on: ubuntu-latest steps: - uses: actions/setup-python@v4 - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 with: fetch-depth: 0 # otherwise, you will failed to push refs to dest repo - name: Install runtime dependencies diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index b1aa54e5c..34b13ea76 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -22,7 +22,7 @@ jobs: runs-on: ubuntu-latest steps: - name: Checkout project - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: fetch-depth: 0 - name: Set up Python @@ -59,7 +59,7 @@ jobs: USE_CYTHON: ${{ matrix.use-cython }} continue-on-error: ${{ matrix.experimental }} steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 with: fetch-depth: 0 - uses: actions/setup-python@v4 @@ -96,7 +96,7 @@ jobs: matrix: os: [ubuntu-20.04, macos-11] steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 with: fetch-depth: 0 - name: Build wheels @@ -110,7 +110,7 @@ jobs: needs: check if: github.event_name == 'release' && github.event.action == 'created' steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 name: Checkout source repository with: fetch-depth: 0 From 648d506f4a0e5fbebac7f80d01606277580c5856 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 14 Sep 2023 17:00:03 -0400 Subject: [PATCH 13/54] Bump codecov/codecov-action from 3 to 4 (#547) Bumps [codecov/codecov-action](https://github.com/codecov/codecov-action) from 3 to 4. - [Release notes](https://github.com/codecov/codecov-action/releases) - [Changelog](https://github.com/codecov/codecov-action/blob/main/CHANGELOG.md) - [Commits](https://github.com/codecov/codecov-action/compare/v3...v4) --- updated-dependencies: - dependency-name: codecov/codecov-action dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/python-package.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 34b13ea76..e217b2d53 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -74,7 +74,7 @@ jobs: - name: Run tests run: scripts/tests - name: Enforce coverage - uses: codecov/codecov-action@v3 + uses: codecov/codecov-action@v4 with: token: ${{ secrets.CODECOV_TOKEN }} check: # This job does nothing and is only used for the branch protection From da90c7cdd08487f480cc2603716c4bedb0ffd7c9 Mon Sep 17 00:00:00 2001 From: William Barnhart Date: Fri, 29 Sep 2023 11:14:29 -0400 Subject: [PATCH 14/54] Revert to codecov/codecov-action@v3 in python-package.yml --- .github/workflows/python-package.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index e217b2d53..34b13ea76 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -74,7 +74,7 @@ jobs: - name: Run tests run: scripts/tests - name: Enforce coverage - uses: codecov/codecov-action@v4 + uses: codecov/codecov-action@v3 with: token: ${{ secrets.CODECOV_TOKEN }} check: # This job does nothing and is only used for the branch protection From 3fb3180c38554aa883f886444fc50d23f5437fbc Mon Sep 17 00:00:00 2001 From: William Barnhart Date: Sun, 5 Nov 2023 12:25:12 -0500 Subject: [PATCH 15/54] Add Python ~3.13 for testing in python-package.yml (#561) --- .github/workflows/python-package.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 34b13ea76..5ebb42c43 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -55,6 +55,9 @@ jobs: - python-version: ~3.12.0-0 experimental: true use-cython: false + - python-version: ~3.13.0-0 + experimental: true + use-cython: false env: USE_CYTHON: ${{ matrix.use-cython }} continue-on-error: ${{ matrix.experimental }} From 5e3ceccb31df9ff974c49bdab69e22941a78c789 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 7 Nov 2023 11:11:18 -0500 Subject: [PATCH 16/54] Bump venusian from 3.0.0 to 3.1.0 in /requirements (#562) Bumps [venusian](https://github.com/Pylons/venusian) from 3.0.0 to 3.1.0. - [Changelog](https://github.com/Pylons/venusian/blob/main/CHANGES.rst) - [Commits](https://github.com/Pylons/venusian/compare/3.0.0...3.1.0) --- updated-dependencies: - dependency-name: venusian dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- requirements/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/requirements/requirements.txt b/requirements/requirements.txt index 0449f8be3..bf057344e 100644 --- a/requirements/requirements.txt +++ b/requirements/requirements.txt @@ -8,6 +8,6 @@ terminaltables>=3.1,<4.0 yarl>=1.0,<2.0 croniter>=0.3.16 mypy_extensions -venusian==3.0.0 +venusian==3.1.0 intervaltree six From ccc062fa28f6f7332fdbbeb5f0c5d0c2c6c68098 Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Thu, 16 Nov 2023 16:14:48 +0100 Subject: [PATCH 17/54] Add a TTL for RocksDB store for removing outdated WAL files (#567) --- faust/stores/rocksdb.py | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/faust/stores/rocksdb.py b/faust/stores/rocksdb.py index 527ef46ba..e07579d06 100644 --- a/faust/stores/rocksdb.py +++ b/faust/stores/rocksdb.py @@ -113,6 +113,7 @@ def __init__( block_cache_compressed_size: Optional[int] = None, bloom_filter_size: Optional[int] = None, use_rocksdict: Optional[bool] = None, + ttl: Optional[int] = None, **kwargs: Any, ) -> None: if max_open_files is not None: @@ -131,6 +132,7 @@ def __init__( self.bloom_filter_size = bloom_filter_size if use_rocksdict is not None: self.use_rocksdict = use_rocksdict + self.ttl = ttl self.extra_options = kwargs def open(self, path: Path, *, read_only: bool = False) -> DB: @@ -140,7 +142,12 @@ def open(self, path: Path, *, read_only: bool = False) -> DB: db_options.set_db_paths( [rocksdict.DBPath(str(path), self.target_file_size_base)] ) - db = DB(str(path), options=self.as_options()) + db_access_type = ( + rocksdict.AccessType.ReadWrite + if self.ttl is None + else rocksdict.AccessType.with_ttl(self.ttl) + ) + db = DB(str(path), options=self.as_options(), access_type=db_access_type) db.set_read_options(rocksdict.ReadOptions()) return db else: @@ -181,6 +188,7 @@ def as_options(self) -> Options: self.block_cache_compressed_size ), ), + wal_ttl_seconds=self.ttl if self.ttl is not None else 0, **self.extra_options, ) @@ -200,6 +208,13 @@ class Store(base.SerializedStore): app.GlobalTable(..., options={'driver': 'rocksdict'}) app.GlobalTable(..., options={'driver': 'python-rocksdb'}) + If you wish to remove the WAL files after a certain amount of + time, you can set a TTL this way:: + + app.GlobalTable(..., options={'ttl': 60 * 60 * 24}) # 1 day + + Note that the TTL is in seconds. + .. warning:: Note that rocksdict uses RocksDB 8. You won't be able to return to using python-rocksdb, which uses RocksDB 6. From e31974f07ac3b1acfbfacbf24ce79876c5b23943 Mon Sep 17 00:00:00 2001 From: Ibrahim Bagalwa <67462827+IbrahimBagalwa@users.noreply.github.com> Date: Mon, 20 Nov 2023 16:24:58 +0200 Subject: [PATCH 18/54] fix: docs Autodetect for schemas (#563) Co-authored-by: William Barnhart --- docs/userguide/models.rst | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/userguide/models.rst b/docs/userguide/models.rst index 7945764df..5831a74b5 100644 --- a/docs/userguide/models.rst +++ b/docs/userguide/models.rst @@ -186,13 +186,13 @@ a schema that reads message key and value type from Kafka headers: serializer=None): if loads is None: loads = app.serializers.loads_value - # try to get key_type and serializer from Kafka headers + # try to get value_type and serializer from Kafka headers headers = dict(message.headers) value_type_name = headers.get('ValueType') serializer = serializer or headers.get('ValueSerializer') if value_type_name: - value_type = registry[value_type] - return loads(value_type, message.key, + value_type = registry[value_type_name] + return loads(value_type, message.value, serializer=serializer) else: return super().loads_value( From 448a7a47a0c121a9f74a9d98c009a87b063d6c1c Mon Sep 17 00:00:00 2001 From: Chris Tam Date: Mon, 20 Nov 2023 20:00:33 -0500 Subject: [PATCH 19/54] Fix potential NoneType access in aiokafka driver (#569) --- faust/transport/drivers/aiokafka.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/faust/transport/drivers/aiokafka.py b/faust/transport/drivers/aiokafka.py index 4a0c9ebb2..1630cd2c3 100644 --- a/faust/transport/drivers/aiokafka.py +++ b/faust/transport/drivers/aiokafka.py @@ -836,7 +836,7 @@ def _verify_aiokafka_event_path(self, now: float, tp: TP) -> bool: secs_since_started = now - self.time_started aiotp = TopicPartition(tp.topic, tp.partition) assignment = consumer._fetcher._subscriptions.subscription.assignment - if not assignment and not assignment.active: + if not assignment or not assignment.active: self.log.error(f"No active partitions for {tp}") return True poll_at = None From c629e7221fc9492899a7597f91483f266af2e1c9 Mon Sep 17 00:00:00 2001 From: Tomek Miodek <4627919+somnam@users.noreply.github.com> Date: Tue, 21 Nov 2023 15:56:26 +0100 Subject: [PATCH 20/54] Temporarily fix breaking build with setuptools v69 (#571) --- pyproject.toml | 2 +- requirements/dist.txt | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pyproject.toml b/pyproject.toml index 043c42765..0180e0888 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -5,7 +5,7 @@ dynamic = ["version"] [build-system] requires = [ - "setuptools", + "setuptools>=30.3.0,<69.0.0", "setuptools_scm[toml]", "wheel", "cython>=0.29; implementation_name == 'cpython'", diff --git a/requirements/dist.txt b/requirements/dist.txt index b976f83f0..0e086ab03 100644 --- a/requirements/dist.txt +++ b/requirements/dist.txt @@ -3,7 +3,7 @@ packaging pre-commit pydocstyle pytest-sugar -setuptools>=30.3.0 +setuptools>=30.3.0,<69.0.0 sphinx-autobuild sphinx2rst>=1.0 tox>=2.3.1 From ad8dd0e68bc9b8d9b1f725bf2eb37a7fc4cbc202 Mon Sep 17 00:00:00 2001 From: William Barnhart Date: Mon, 27 Nov 2023 10:03:59 -0500 Subject: [PATCH 21/54] Fix AccessType assigninment in rocksdb.py (#575) Closes #573. --- faust/stores/rocksdb.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/faust/stores/rocksdb.py b/faust/stores/rocksdb.py index e07579d06..d614b32c4 100644 --- a/faust/stores/rocksdb.py +++ b/faust/stores/rocksdb.py @@ -143,7 +143,7 @@ def open(self, path: Path, *, read_only: bool = False) -> DB: [rocksdict.DBPath(str(path), self.target_file_size_base)] ) db_access_type = ( - rocksdict.AccessType.ReadWrite + rocksdict.AccessType.read_write() if self.ttl is None else rocksdict.AccessType.with_ttl(self.ttl) ) From 58b18a10e11bd32139ee14ec453f6853a6a26915 Mon Sep 17 00:00:00 2001 From: Chris Tam Date: Mon, 27 Nov 2023 11:24:32 -0500 Subject: [PATCH 22/54] Fix races in conductor with dynamic agents (#568) * Fix races in conductor with dynamic agents * Add documentation for `_resubscribe_sleep_lock_seconds` --------- Co-authored-by: William Barnhart --- faust/transport/conductor.py | 68 ++++++++++++++++++++++++++++-------- 1 file changed, 53 insertions(+), 15 deletions(-) diff --git a/faust/transport/conductor.py b/faust/transport/conductor.py index cda7858f1..ff7d3d99f 100644 --- a/faust/transport/conductor.py +++ b/faust/transport/conductor.py @@ -214,6 +214,12 @@ class Conductor(ConductorT, Service): #: to call here. _tp_to_callback: MutableMapping[TP, ConsumerCallback] + #: Lock used to synchronize access to _tp_to_callback. + #: Resubscriptions and updates to the indices may modify the mapping, and + #: while that is happening, the mapping should not be accessed by message + #: handlers. + _tp_to_callback_lock: asyncio.Lock + #: Whenever a change is made, i.e. a Topic is added/removed, we notify #: the background task responsible for resubscribing. _subscription_changed: Optional[asyncio.Event] @@ -224,8 +230,21 @@ class Conductor(ConductorT, Service): _compiler: ConductorCompiler - #: We wait for 45 seconds after a resubscription request, to make - #: sure any later requests are handled at the same time. + # `_resubscribe_sleep_lock_seconds` trades off between the latency of + # receiving messages for newly added topics and the cost of resubscribing + # to topics. Note that this resubscription flow only occurs when the topic + # list has changed (see the `_subscription_changed` event). This mechanism + # attempts to coalesce topic list changes that happen in quick succession + # and prevents the framework from constantly resubscribing to topics after + # every change. + # + # If the value is set too low and an agent is adding topics very + # frequently, then resubscription will happen very often and will issue + # unnecessary work on the async loop. + # If the value is set too high, it will take a long time for a newly added + # agent to start receiving messages; this time is bounded by the value of + # `_resubscribe_sleep_lock_seconds`, barring something hogging the async + # loop. _resubscribe_sleep_lock_seconds: float = 45.0 def __init__(self, app: AppT, **kwargs: Any) -> None: @@ -235,6 +254,7 @@ def __init__(self, app: AppT, **kwargs: Any) -> None: self._topic_name_index = defaultdict(set) self._tp_index = defaultdict(set) self._tp_to_callback = {} + self._tp_to_callback_lock = asyncio.Lock() self._acking_topics = set() self._subscription_changed = None self._subscription_done = None @@ -266,12 +286,18 @@ def _compile_message_handler(self) -> ConsumerCallback: async def on_message(message: Message) -> None: tp = TP(topic=message.topic, partition=0) - return await get_callback_for_tp(tp)(message) + async with self._tp_to_callback_lock: + callback = get_callback_for_tp(tp) + + return await callback(message) else: async def on_message(message: Message) -> None: - return await get_callback_for_tp(message.tp)(message) + async with self._tp_to_callback_lock: + callback = get_callback_for_tp(message.tp) + + return await callback(message) return on_message @@ -309,11 +335,14 @@ async def _subscriber(self) -> None: # pragma: no cover # further subscription requests will happen during the same # rebalance. await self.sleep(self._resubscribe_sleep_lock_seconds) + + # Clear the event before updating indices. This way, new events + # that get triggered during the update will be handled the next + # time around. + ev.clear() subscribed_topics = await self._update_indices() await self.app.consumer.subscribe(subscribed_topics) - # clear the subscription_changed flag, so we can wait on it again. - ev.clear() # wake-up anything waiting for the subscription to be done. notify(self._subscription_done) @@ -328,15 +357,23 @@ async def maybe_wait_for_subscriptions(self) -> None: await self._subscription_done async def _update_indices(self) -> Iterable[str]: - self._topic_name_index.clear() - self._tp_to_callback.clear() - for channel in self._topics: - if channel.internal: - await channel.maybe_declare() - for topic in channel.topics: - if channel.acks: - self._acking_topics.add(topic) - self._topic_name_index[topic].add(channel) + async with self._tp_to_callback_lock: + self._topic_name_index.clear() + self._tp_to_callback.clear() + + # Make a (shallow) copy of the topics, so new additions to the set + # won't poison the iterator. Additions can come in while this + # function yields during an await. + topics = list(self._topics) + for channel in topics: + if channel.internal: + await channel.maybe_declare() + for topic in channel.topics: + if channel.acks: + self._acking_topics.add(topic) + self._topic_name_index[topic].add(channel) + + self._update_callback_map() return self._topic_name_index @@ -418,6 +455,7 @@ def _topic_contain_unsubscribed_topics(self, topic: TopicT) -> bool: def discard(self, topic: Any) -> None: """Unregister topic from conductor.""" self._topics.discard(topic) + self._flag_changes() def _flag_changes(self) -> None: if self._subscription_changed is not None: From ddef30efaae7c75cd5e3c9bbe1eceeb6789b8782 Mon Sep 17 00:00:00 2001 From: William Barnhart Date: Mon, 27 Nov 2023 11:33:10 -0500 Subject: [PATCH 23/54] Consolidate dynamic attributes for setuptools v69.0.0 and move some metadata definitions into pyproject.toml (#574) * Consolidate dynamic attributes for setuptools v69.0.0 * commit functioning build * forgot to lint setup.py lol * Update authors and maintainers metadata * remove redundant name metadata definition * Revert "remove redundant name metadata definition" This reverts commit 0263fa60e9369201da6883900bc83798d1759c8c. * bring back long_description in setup.py --- pyproject.toml | 58 +++++++++++++++++++++++++++++++++++++++++++++++--- setup.py | 41 ----------------------------------- 2 files changed, 55 insertions(+), 44 deletions(-) diff --git a/pyproject.toml b/pyproject.toml index 0180e0888..48d6e50d1 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -1,15 +1,67 @@ [project] name = "faust-streaming" requires-python = ">=3.8" -dynamic = ["version"] +dynamic = [ + "version", + "optional-dependencies", + "dependencies", + +] +readme = "README.md" +license = { file = "LICENSE" } +keywords = [ + "stream", + "processing", + "asyncio", + "distributed", + "queue", + "kafka", +] +classifiers = [ + "Framework :: AsyncIO", + "Development Status :: 5 - Production/Stable", + "Intended Audience :: Developers", + "Natural Language :: English", + "License :: OSI Approved :: BSD License", + "Programming Language :: Python", + "Programming Language :: Python :: 3 :: Only", + "Programming Language :: Python :: 3.8", + "Programming Language :: Python :: 3.9", + "Programming Language :: Python :: 3.10", + "Programming Language :: Python :: 3.11", + "Programming Language :: Python :: Implementation :: CPython", + "Operating System :: POSIX", + "Operating System :: POSIX :: Linux", + "Operating System :: MacOS :: MacOS X", + "Operating System :: POSIX :: BSD", + "Operating System :: Microsoft :: Windows", + "Topic :: System :: Networking", + "Topic :: System :: Distributed Computing", +] + +authors = [ + { name = "Ask Solem Hoel", email= "ask@robinhood.com" }, + { name = "Vineet Goel", email= "vineet@robinhood.com" }, +] + +maintainers = [ + { name = "Vikram Patki", email = "vpatki@wayfair.com" }, + { name = "William Barnhart", email = "williambbarnhart@gmail.com" }, +] + + +[project.urls] +Documentation = "https://faust-streaming.github.io/faust/" +Source = "https://github.com/faust-streaming/faust" +Changes = "https://github.com/faust-streaming/faust/releases" [build-system] requires = [ - "setuptools>=30.3.0,<69.0.0", + "setuptools>=30.3.0", "setuptools_scm[toml]", "wheel", "cython>=0.29; implementation_name == 'cpython'", - "cython>=3.0.0b3; implementation_name == 'cpython' and python_version >= '3.12'", + "cython>=3.0.0; implementation_name == 'cpython' and python_version >= '3.12'", ] build-backend = "setuptools.build_meta" diff --git a/setup.py b/setup.py index 3c8a1a3ab..d6ea6d4e9 100644 --- a/setup.py +++ b/setup.py @@ -188,16 +188,9 @@ def extras_require(): def do_setup(**kwargs): setup( name="faust-streaming", - use_scm_version=True, - setup_requires=["setuptools_scm"], description=meta["doc"], long_description=long_description, long_description_content_type="text/markdown", - author=meta["author"], - author_email=meta["contact"], - url=meta["homepage"], - platforms=["any"], - license="BSD 3-Clause", packages=find_packages(exclude=["examples", "ez_setup", "tests", "tests.*"]), # PEP-561: https://www.python.org/dev/peps/pep-0561/ package_data={"faust": ["py.typed"]}, @@ -212,40 +205,6 @@ def do_setup(**kwargs): "faust = faust.cli.faust:cli", ], }, - project_urls={ - "Bug Reports": "https://github.com/faust-streaming/faust/issues", - "Source": "https://github.com/faust-streaming/faust", - "Documentation": "https://faust-streaming.github.io/faust", - }, - keywords=[ - "stream", - "processing", - "asyncio", - "distributed", - "queue", - "kafka", - ], - classifiers=[ - "Framework :: AsyncIO", - "Development Status :: 5 - Production/Stable", - "Intended Audience :: Developers", - "Natural Language :: English", - "License :: OSI Approved :: BSD License", - "Programming Language :: Python", - "Programming Language :: Python :: 3 :: Only", - "Programming Language :: Python :: 3.8", - "Programming Language :: Python :: 3.9", - "Programming Language :: Python :: 3.10", - "Programming Language :: Python :: 3.11", - "Programming Language :: Python :: Implementation :: CPython", - "Operating System :: POSIX", - "Operating System :: POSIX :: Linux", - "Operating System :: MacOS :: MacOS X", - "Operating System :: POSIX :: BSD", - "Operating System :: Microsoft :: Windows", - "Topic :: System :: Networking", - "Topic :: System :: Distributed Computing", - ], **kwargs, ) From 2eba6908f0b6f906bffda2c880b87e5031673edf Mon Sep 17 00:00:00 2001 From: William Barnhart Date: Wed, 29 Nov 2023 11:39:00 -0500 Subject: [PATCH 24/54] Fix sdist builds (#577) * Update setup.py to use setuptools_scm * Update setup.py --- setup.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/setup.py b/setup.py index d6ea6d4e9..e8047d085 100644 --- a/setup.py +++ b/setup.py @@ -188,6 +188,8 @@ def extras_require(): def do_setup(**kwargs): setup( name="faust-streaming", + use_scm_version=True, + setup_requires=["setuptools_scm"], description=meta["doc"], long_description=long_description, long_description_content_type="text/markdown", From 6714156400b2eaef300c78fd7afbfdf24cb57dd8 Mon Sep 17 00:00:00 2001 From: William Barnhart Date: Wed, 29 Nov 2023 18:20:21 -0500 Subject: [PATCH 25/54] Add project.scripts to fix faust CLI usage (#580) --- pyproject.toml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pyproject.toml b/pyproject.toml index 48d6e50d1..de7ed77fe 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -49,6 +49,8 @@ maintainers = [ { name = "William Barnhart", email = "williambbarnhart@gmail.com" }, ] +[project.scripts] +faust = "faust.cli.faust:cli" [project.urls] Documentation = "https://faust-streaming.github.io/faust/" From 310eb661378b9645cd0d129d575fb1aead20ddeb Mon Sep 17 00:00:00 2001 From: William Barnhart Date: Wed, 29 Nov 2023 18:20:58 -0500 Subject: [PATCH 26/54] Add description field to pyproject.toml --- pyproject.toml | 1 + 1 file changed, 1 insertion(+) diff --git a/pyproject.toml b/pyproject.toml index de7ed77fe..639e43f86 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -1,5 +1,6 @@ [project] name = "faust-streaming" +description = "Python Stream Processing. A Faust fork" requires-python = ">=3.8" dynamic = [ "version", From 223b114b9040c2b14b9051c531e7c3814ff6a197 Mon Sep 17 00:00:00 2001 From: William Barnhart Date: Wed, 29 Nov 2023 23:31:31 -0500 Subject: [PATCH 27/54] Add FastAPI example (#581) * add fastapi dir and change dep to rocksdict * commit inital example * update example * use lifespan * reorganize to have an endpoint trigger a producer * reorganize timer to be above fast api decorator --- examples/django/requirements/default.txt | 2 +- examples/fastapi_example.py | 72 ++++++++++++++++++++++++ 2 files changed, 73 insertions(+), 1 deletion(-) create mode 100755 examples/fastapi_example.py diff --git a/examples/django/requirements/default.txt b/examples/django/requirements/default.txt index 042b4125b..e5aed2ea6 100644 --- a/examples/django/requirements/default.txt +++ b/examples/django/requirements/default.txt @@ -1,4 +1,4 @@ django -faust[rocksdb] +faust[rocksdict] eventlet faust-aioeventlet diff --git a/examples/fastapi_example.py b/examples/fastapi_example.py new file mode 100755 index 000000000..3e666fcda --- /dev/null +++ b/examples/fastapi_example.py @@ -0,0 +1,72 @@ +#!/usr/bin/env python +import asyncio +from contextlib import asynccontextmanager +from typing import Union + +from fastapi import FastAPI + +import faust + + +# This is just hello_world.py integrated with a FastAPI application + + +def fake_answer_to_everything_ml_model(x: float): + return x * 42 + + +ml_models = {} + + +# You MUST have "app" defined in order for Faust to discover the app +# if you're using "faust" on CLI, but this doesn't work yet +faust_app = faust.App( + 'hello-world-fastapi', + broker='kafka://localhost:9092', + web_enabled=False, +) +# app = faust_app + +greetings_topic = faust_app.topic('greetings', value_type=str) + + +@asynccontextmanager +async def lifespan(app: FastAPI): + # Load the ML model + ml_models["answer_to_everything"] = fake_answer_to_everything_ml_model + await faust_app.start() + yield + # Clean up the ML models and release the resources + ml_models.clear() + await faust_app.stop() + + +app = fastapi_app = FastAPI( + lifespan=lifespan, +) +# For now, run via "uvicorn fastapi_example:app" +# then visit http://127.0.0.1:8000/docs + + +@fastapi_app.get("/") +def read_root(): + return {"Hello": "World"} + + +@fastapi_app.get("/items/{item_id}") +def read_item(item_id: int, q: Union[str, None] = None): + return {"item_id": item_id, "q": q} + + +@faust_app.agent(greetings_topic) +async def print_greetings(greetings): + async for greeting in greetings: + print(greeting) + + +@faust_app.timer(5) # make sure you *always* add the timer above if you're using one +@fastapi_app.get("/produce") +async def produce(): + for i in range(100): + await greetings_topic.send(value=f'hello {i}') + return {"success": True} From 2f594423aa070f7e33c8cfaf13e4ff4e3b3a55bd Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 5 Dec 2023 13:36:39 -0500 Subject: [PATCH 28/54] Bump actions/deploy-pages from 2 to 3 (#582) Bumps [actions/deploy-pages](https://github.com/actions/deploy-pages) from 2 to 3. - [Release notes](https://github.com/actions/deploy-pages/releases) - [Commits](https://github.com/actions/deploy-pages/compare/v2...v3) --- updated-dependencies: - dependency-name: actions/deploy-pages dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/gh-pages.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/gh-pages.yml b/.github/workflows/gh-pages.yml index e56a79297..8853fea35 100644 --- a/.github/workflows/gh-pages.yml +++ b/.github/workflows/gh-pages.yml @@ -42,6 +42,6 @@ jobs: name: github-pages url: ${{ steps.deployment.outputs.page_url }} steps: - - uses: actions/deploy-pages@v2 + - uses: actions/deploy-pages@v3 id: deployment name: Deploy to GitHub Pages From 8a5576e9121d9d1e58a82b104920ebe4d77549d0 Mon Sep 17 00:00:00 2001 From: Anton Melser Date: Wed, 6 Dec 2023 07:12:00 +0800 Subject: [PATCH 29/54] Rename kafka to aiokafka (#565) * Rename kafka to aiokafka kafka-python is no longer to be a dependency of aiokafka Following https://github.com/aio-libs/aiokafka/discussions/915 Signed-off-by: Anton Melser * Force aiokafka version >=0.9.0 Signed-off-by: Anton Melser --------- Signed-off-by: Anton Melser Co-authored-by: William Barnhart --- examples/kubernetes/producer/producer.py | 26 ++++++++++++------------ faust/assignor/partition_assignor.py | 6 +++--- faust/tables/recovery.py | 2 +- faust/transport/drivers/aiokafka.py | 12 +++++------ faust/utils/kafka/protocol/admin.py | 2 +- faust/utils/kafka/protocol/api.py | 4 ++-- requirements/requirements.txt | 2 +- tests/consistency/consistency_checker.py | 4 ++-- 8 files changed, 29 insertions(+), 29 deletions(-) diff --git a/examples/kubernetes/producer/producer.py b/examples/kubernetes/producer/producer.py index 317485c06..94fa5dbbb 100644 --- a/examples/kubernetes/producer/producer.py +++ b/examples/kubernetes/producer/producer.py @@ -1,20 +1,20 @@ import json from random import random -from kafka import KafkaProducer +from aiokafka import KafkaProducer -TOPIC = 'test' -KEY = 'score' +TOPIC = "test" +KEY = "score" def publish_message(producer_instance, topic_name, key, value): try: - key_bytes = bytes(key, encoding='utf-8') - value_bytes = bytes(value, encoding='utf-8') + key_bytes = bytes(key, encoding="utf-8") + value_bytes = bytes(value, encoding="utf-8") producer_instance.send(topic_name, key=key_bytes, value=value_bytes) producer_instance.flush() - print('Message published successfully.') + print("Message published successfully.") except Exception as ex: - print('Exception in publishing message') + print("Exception in publishing message") print(ex) @@ -26,20 +26,20 @@ def connect_kafka_producer(): # Don't use in production, this only works with Docker for Mac in # development _producer = KafkaProducer( - bootstrap_servers=['host.docker.internal:9092'], - api_version=(0, 10)) + bootstrap_servers=["host.docker.internal:9092"], api_version=(0, 10) + ) except Exception as ex: - print('Exception while connecting Kafka') + print("Exception while connecting Kafka") print(ex) return _producer -if __name__ == '__main__': +if __name__ == "__main__": kafka_producer = connect_kafka_producer() for index in range(0, 10000): message = { - 'index': index, - 'value': round(random(), 2), + "index": index, + "value": round(random(), 2), } publish_message(kafka_producer, TOPIC, KEY, json.dumps(message)) if kafka_producer is not None: diff --git a/faust/assignor/partition_assignor.py b/faust/assignor/partition_assignor.py index 6711bbd01..705079620 100644 --- a/faust/assignor/partition_assignor.py +++ b/faust/assignor/partition_assignor.py @@ -4,9 +4,9 @@ from collections import defaultdict from typing import Iterable, List, Mapping, MutableMapping, Sequence, Set, cast -from kafka.cluster import ClusterMetadata -from kafka.coordinator.assignors.abstract import AbstractPartitionAssignor -from kafka.coordinator.protocol import ( +from aiokafka.cluster import ClusterMetadata +from aiokafka.coordinator.assignors.abstract import AbstractPartitionAssignor +from aiokafka.coordinator.protocol import ( ConsumerProtocolMemberAssignment, ConsumerProtocolMemberMetadata, ) diff --git a/faust/tables/recovery.py b/faust/tables/recovery.py index 9924880af..51ec3cb9c 100644 --- a/faust/tables/recovery.py +++ b/faust/tables/recovery.py @@ -18,7 +18,7 @@ ) import opentracing -from kafka.errors import IllegalStateError +from aiokafka.errors import IllegalStateError from mode import Service, get_logger from mode.services import WaitArgT from mode.utils.times import humanize_seconds, humanize_seconds_ago diff --git a/faust/transport/drivers/aiokafka.py b/faust/transport/drivers/aiokafka.py index 1630cd2c3..24a9de977 100644 --- a/faust/transport/drivers/aiokafka.py +++ b/faust/transport/drivers/aiokafka.py @@ -35,16 +35,16 @@ ) from aiokafka.structs import OffsetAndMetadata, TopicPartition as _TopicPartition from aiokafka.util import parse_kafka_version -from kafka import TopicPartition -from kafka.coordinator.assignors.roundrobin import RoundRobinPartitionAssignor -from kafka.errors import ( +from aiokafka import TopicPartition +from aiokafka.coordinator.assignors.roundrobin import RoundRobinPartitionAssignor +from aiokafka.errors import ( NotControllerError, TopicAlreadyExistsError as TopicExistsError, for_code, ) -from kafka.partitioner import murmur2 -from kafka.partitioner.default import DefaultPartitioner -from kafka.protocol.metadata import MetadataRequest_v1 +from aiokafka.partitioner import murmur2 +from aiokafka.partitioner import DefaultPartitioner +from aiokafka.protocol.metadata import MetadataRequest_v1 from mode import Service, get_logger from mode.threads import ServiceThread, WorkerThread from mode.utils import text diff --git a/faust/utils/kafka/protocol/admin.py b/faust/utils/kafka/protocol/admin.py index b7d7a778f..3e418e960 100644 --- a/faust/utils/kafka/protocol/admin.py +++ b/faust/utils/kafka/protocol/admin.py @@ -1,5 +1,5 @@ """Admin related Kafka protocol extensions.""" -from kafka.protocol import types +from aiokafka.protocol import types from .api import Request, Response diff --git a/faust/utils/kafka/protocol/api.py b/faust/utils/kafka/protocol/api.py index 8e33bed57..523d9ef0f 100644 --- a/faust/utils/kafka/protocol/api.py +++ b/faust/utils/kafka/protocol/api.py @@ -3,8 +3,8 @@ import abc from typing import Type -from kafka.protocol.struct import Struct -from kafka.protocol.types import Schema +from aiokafka.protocol.struct import Struct +from aiokafka.protocol.types import Schema class Response(Struct, metaclass=abc.ABCMeta): # type: ignore diff --git a/requirements/requirements.txt b/requirements/requirements.txt index bf057344e..cdd50fedd 100644 --- a/requirements/requirements.txt +++ b/requirements/requirements.txt @@ -1,6 +1,6 @@ aiohttp>=3.8.0,<4.0 aiohttp_cors>=0.7,<2.0 -aiokafka>=0.8.0,<0.9.0 +aiokafka>=0.9.0 click>=6.7,<8.2 mode-streaming>=0.3.0 opentracing>=1.3.0,<=2.4.0 diff --git a/tests/consistency/consistency_checker.py b/tests/consistency/consistency_checker.py index d9b0f073c..76f73ea17 100644 --- a/tests/consistency/consistency_checker.py +++ b/tests/consistency/consistency_checker.py @@ -2,8 +2,8 @@ from collections import defaultdict from aiokafka import AIOKafkaClient, AIOKafkaConsumer -from kafka.protocol.commit import GroupCoordinatorRequest_v0, OffsetFetchRequest_v1 -from kafka.structs import TopicPartition +from aiokafka.protocol.commit import GroupCoordinatorRequest_v0, OffsetFetchRequest_v1 +from aiokafka.structs import TopicPartition from faust.utils import json From 73be0a2ed1189a65954eadf7a5dcf02647a9fb73 Mon Sep 17 00:00:00 2001 From: William Barnhart Date: Thu, 7 Dec 2023 12:46:25 -0500 Subject: [PATCH 30/54] Lint aiokafka driver --- faust/transport/drivers/aiokafka.py | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/faust/transport/drivers/aiokafka.py b/faust/transport/drivers/aiokafka.py index 24a9de977..8838f557c 100644 --- a/faust/transport/drivers/aiokafka.py +++ b/faust/transport/drivers/aiokafka.py @@ -25,26 +25,23 @@ import aiokafka import aiokafka.abc import opentracing +from aiokafka import TopicPartition from aiokafka.consumer.group_coordinator import OffsetCommitRequest +from aiokafka.coordinator.assignors.roundrobin import RoundRobinPartitionAssignor from aiokafka.errors import ( CommitFailedError, ConsumerStoppedError, IllegalStateError, KafkaError, - ProducerFenced, -) -from aiokafka.structs import OffsetAndMetadata, TopicPartition as _TopicPartition -from aiokafka.util import parse_kafka_version -from aiokafka import TopicPartition -from aiokafka.coordinator.assignors.roundrobin import RoundRobinPartitionAssignor -from aiokafka.errors import ( NotControllerError, + ProducerFenced, TopicAlreadyExistsError as TopicExistsError, for_code, ) -from aiokafka.partitioner import murmur2 -from aiokafka.partitioner import DefaultPartitioner +from aiokafka.partitioner import DefaultPartitioner, murmur2 from aiokafka.protocol.metadata import MetadataRequest_v1 +from aiokafka.structs import OffsetAndMetadata, TopicPartition as _TopicPartition +from aiokafka.util import parse_kafka_version from mode import Service, get_logger from mode.threads import ServiceThread, WorkerThread from mode.utils import text From 53dfd18125c08e0b7c39ff1a146e9cea5d63e9fb Mon Sep 17 00:00:00 2001 From: William Barnhart Date: Fri, 8 Dec 2023 11:42:22 -0500 Subject: [PATCH 31/54] Replace called_once_with test_objects.py for 3.12 support (#586) * Replace called_once_with test_objects.py for 3.12 support * just replace with assert_called_once --- tests/unit/tables/test_objects.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/unit/tables/test_objects.py b/tests/unit/tables/test_objects.py index ad3b425ff..8c891dcf3 100644 --- a/tests/unit/tables/test_objects.py +++ b/tests/unit/tables/test_objects.py @@ -1,4 +1,4 @@ -from unittest.mock import Mock, patch +from unittest.mock import MagicMock, Mock, patch import pytest @@ -54,7 +54,7 @@ class Test_ChangeloggedObjectManager: def man(self, *, table): man = ChangeloggedObjectManager(table) man.ValueType = ValueType - man.storage.__setitem__ = Mock() + man.storage.__setitem__ = MagicMock() return man @pytest.fixture() @@ -63,7 +63,7 @@ def storage(self, *, table): def test_send_changelog_event(self, *, man, table, key, current_event): man.send_changelog_event(key, 3, "value") - assert man.storage.__setitem__.called_once_with(key, "value") + man.storage.__setitem__.assert_called_once() table._send_changelog.assert_called_once_with( current_event(), (3, key), From 3554266e666df073d0d39a525aa61104655a33b3 Mon Sep 17 00:00:00 2001 From: William Barnhart Date: Fri, 8 Dec 2023 19:51:48 -0500 Subject: [PATCH 32/54] Add official support for Python 3.12 (#587) * Add official support for Python 3.12 * commit remaining lint changes --- .github/workflows/python-package.yml | 7 ++----- faust/agents/agent.py | 2 +- faust/cli/model.py | 2 +- faust/sensors/datadog.py | 2 +- faust/transport/producer.py | 2 +- pyproject.toml | 3 +++ setup.cfg | 2 +- tests/unit/agents/test_replies.py | 2 +- tox.ini | 3 ++- 9 files changed, 13 insertions(+), 12 deletions(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 5ebb42c43..6fc6154e1 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -12,7 +12,7 @@ env: FORCE_COLOR: '1' # Make tools pretty. PIP_DISABLE_PIP_VERSION_CHECK: '1' PIP_NO_PYTHON_VERSION_WARNING: '1' - PYTHON_LATEST: '3.11' + PYTHON_LATEST: '3.12' # For re-actors/checkout-python-sdist sdist-artifact: python-package-distributions @@ -45,16 +45,13 @@ jobs: # for example if a test fails only when Cython is enabled fail-fast: false matrix: - python-version: ['3.8', '3.9', '3.10', '3.11'] + python-version: ['3.8', '3.9', '3.10', '3.11', '3.12'] use-cython: ['true', 'false'] experimental: [false] include: - python-version: 'pypy3.9' use-cython: false experimental: true - - python-version: ~3.12.0-0 - experimental: true - use-cython: false - python-version: ~3.13.0-0 experimental: true use-cython: false diff --git a/faust/agents/agent.py b/faust/agents/agent.py index 41bebfe6b..1f7060a11 100644 --- a/faust/agents/agent.py +++ b/faust/agents/agent.py @@ -509,7 +509,7 @@ def _prepare_channel( has_prefix=has_prefix, **kwargs, ) - raise TypeError(f"Channel must be channel, topic, or str; not {type(channel)}") + raise TypeError(f"Channel must be channel, topic, or str, not {type(channel)}") def __call__( self, diff --git a/faust/cli/model.py b/faust/cli/model.py index fde0fe530..67a1ecd9c 100644 --- a/faust/cli/model.py +++ b/faust/cli/model.py @@ -58,7 +58,7 @@ def _unknown_model( alt = text.didyoumean( registry, lookup, - fmt_none=f"Please run `{self.prog_name} models` for a list.", + fmt_none=f'Please run "{self.prog_name} models" for a list.', ) return click.UsageError(f"No model {name!r}. {alt}") diff --git a/faust/sensors/datadog.py b/faust/sensors/datadog.py index ed8bf57b9..3c79174c4 100644 --- a/faust/sensors/datadog.py +++ b/faust/sensors/datadog.py @@ -146,7 +146,7 @@ def __init__( self.rate = rate if datadog is None: raise ImproperlyConfigured( - f"{type(self).__name__} requires `pip install datadog`." + f'{type(self).__name__} requires "pip install datadog".' ) super().__init__(**kwargs) diff --git a/faust/transport/producer.py b/faust/transport/producer.py index dbcab255d..e36803a52 100644 --- a/faust/transport/producer.py +++ b/faust/transport/producer.py @@ -91,7 +91,7 @@ async def wait_until_ebb(self) -> None: start_time = time.time() await self.flush_atmost(self.max_messages) end_time = time.time() - logger.info(f"producer flush took {end_time-start_time}") + logger.info(f"producer flush took {end_time - start_time}") @Service.task async def _handle_pending(self) -> None: diff --git a/pyproject.toml b/pyproject.toml index 639e43f86..3748de325 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -30,6 +30,7 @@ classifiers = [ "Programming Language :: Python :: 3.9", "Programming Language :: Python :: 3.10", "Programming Language :: Python :: 3.11", + "Programming Language :: Python :: 3.12", "Programming Language :: Python :: Implementation :: CPython", "Operating System :: POSIX", "Operating System :: POSIX :: Linux", @@ -75,6 +76,8 @@ build-backend = "setuptools.build_meta" ignore = [ "W503", "E203", + "E231", + "E225", "B305", ] max-line-length = 88 diff --git a/setup.cfg b/setup.cfg index ac623699e..bb412d374 100644 --- a/setup.cfg +++ b/setup.cfg @@ -1,5 +1,5 @@ [flake8] -ignore = W503, E203, B305 +ignore = W503, E203, E231, E225, B305 max-line-length = 88 [mypy-tests.*] diff --git a/tests/unit/agents/test_replies.py b/tests/unit/agents/test_replies.py index 11912ec58..36628d9c0 100644 --- a/tests/unit/agents/test_replies.py +++ b/tests/unit/agents/test_replies.py @@ -86,7 +86,7 @@ async def test_parallel_iterate(self): if pending: raise Exception( - f"Test did not return in 5s:\n" + f"Test did not return within 5s:\n" f" DONE_TASKS={done}\n" f" PENDING_TASKS={pending}\n" f" size={p.size}\n" diff --git a/tox.ini b/tox.ini index 93dc2bdb8..59aff06e1 100644 --- a/tox.ini +++ b/tox.ini @@ -1,5 +1,5 @@ [tox] -envlist = 3.11,3.10,3.9,3.8,flake8,apicheck,configcheck,typecheck,docstyle,bandit,spell +envlist = 3.12,3.11,3.10,3.9,3.8,flake8,apicheck,configcheck,typecheck,docstyle,bandit,spell [testenv] deps= @@ -17,6 +17,7 @@ recreate = False commands = py.test --random-order --open-files -xvv --cov=faust tests/unit tests/functional tests/integration tests/meticulous/ tests/regression basepython = + 3.12,flake8,apicheck,linkcheck,configcheck,typecheck,docstyle,bandit,spell: python3.12 3.11,flake8,apicheck,linkcheck,configcheck,typecheck,docstyle,bandit,spell: python3.11 3.10,flake8,apicheck,linkcheck,configcheck,typecheck,docstyle,bandit,spell: python3.10 3.9,flake8,apicheck,linkcheck,configcheck,typecheck,docstyle,bandit,spell: python3.9 From 1f68fa2ce5f719e4640f1d8cf52a0323cf272cec Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 8 Dec 2023 20:27:15 -0500 Subject: [PATCH 33/54] Update aiokafka requirement from <0.9.0,>=0.8.0 to >=0.8.0,<0.10.0 in /requirements (#583) * Update aiokafka requirement in /requirements Updates the requirements on [aiokafka](https://github.com/aio-libs/aiokafka) to permit the latest version. - [Release notes](https://github.com/aio-libs/aiokafka/releases) - [Changelog](https://github.com/aio-libs/aiokafka/blob/master/CHANGES.rst) - [Commits](https://github.com/aio-libs/aiokafka/compare/v0.8.0...v0.9.0) --- updated-dependencies: - dependency-name: aiokafka dependency-type: direct:production ... Signed-off-by: dependabot[bot] * Update requirements.txt --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: William Barnhart From a0de3738b3f33f8679dede2a53113d59af36f837 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 8 Dec 2023 20:27:38 -0500 Subject: [PATCH 34/54] Bump actions/setup-python from 4 to 5 (#585) Bumps [actions/setup-python](https://github.com/actions/setup-python) from 4 to 5. - [Release notes](https://github.com/actions/setup-python/releases) - [Commits](https://github.com/actions/setup-python/compare/v4...v5) --- updated-dependencies: - dependency-name: actions/setup-python dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/gh-pages.yml | 2 +- .github/workflows/python-package.yml | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/gh-pages.yml b/.github/workflows/gh-pages.yml index 8853fea35..833cd0ec0 100644 --- a/.github/workflows/gh-pages.yml +++ b/.github/workflows/gh-pages.yml @@ -13,7 +13,7 @@ jobs: name: Build docs runs-on: ubuntu-latest steps: - - uses: actions/setup-python@v4 + - uses: actions/setup-python@v5 - uses: actions/checkout@v4 with: fetch-depth: 0 # otherwise, you will failed to push refs to dest repo diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 6fc6154e1..714209c7d 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -26,7 +26,7 @@ jobs: with: fetch-depth: 0 - name: Set up Python - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: ${{ env.PYTHON_LATEST }} cache: pip @@ -62,7 +62,7 @@ jobs: - uses: actions/checkout@v4 with: fetch-depth: 0 - - uses: actions/setup-python@v4 + - uses: actions/setup-python@v5 with: python-version: ${{ matrix.python-version }} cache: pip @@ -114,7 +114,7 @@ jobs: name: Checkout source repository with: fetch-depth: 0 - - uses: actions/setup-python@v4 + - uses: actions/setup-python@v5 - name: Build sdist run: > pip3 install pkgconfig cython --upgrade && From df20a17715ef3f381f455b2c6f570300baa51665 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 13 Dec 2023 19:08:03 -0500 Subject: [PATCH 35/54] Bump github/codeql-action from 2 to 3 (#588) Bumps [github/codeql-action](https://github.com/github/codeql-action) from 2 to 3. - [Release notes](https://github.com/github/codeql-action/releases) - [Changelog](https://github.com/github/codeql-action/blob/main/CHANGELOG.md) - [Commits](https://github.com/github/codeql-action/compare/v2...v3) --- updated-dependencies: - dependency-name: github/codeql-action dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/codeql-analysis.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/codeql-analysis.yml b/.github/workflows/codeql-analysis.yml index 0d5078b39..4f6360b71 100644 --- a/.github/workflows/codeql-analysis.yml +++ b/.github/workflows/codeql-analysis.yml @@ -40,7 +40,7 @@ jobs: # Initializes the CodeQL tools for scanning. - name: Initialize CodeQL - uses: github/codeql-action/init@v2 + uses: github/codeql-action/init@v3 with: languages: ${{ matrix.language }} # If you wish to specify custom queries, you can do so here or in a config file. @@ -51,7 +51,7 @@ jobs: # Autobuild attempts to build any compiled languages (C/C++, C#, or Java). # If this step fails, then you should remove it and run the build manually (see below) - name: Autobuild - uses: github/codeql-action/autobuild@v2 + uses: github/codeql-action/autobuild@v3 # ℹī¸ Command-line programs to run using the OS shell. # 📚 https://git.io/JvXDl @@ -64,4 +64,4 @@ jobs: # make bootstrap # make release - name: Perform CodeQL Analysis - uses: github/codeql-action/analyze@v2 + uses: github/codeql-action/analyze@v3 From 0628117ae03da7a687cc45f5b7917d04ce96d0b7 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 14 Dec 2023 14:30:21 -0500 Subject: [PATCH 36/54] Bump actions/download-artifact from 3 to 4 (#589) Bumps [actions/download-artifact](https://github.com/actions/download-artifact) from 3 to 4. - [Release notes](https://github.com/actions/download-artifact/releases) - [Commits](https://github.com/actions/download-artifact/compare/v3...v4) --- updated-dependencies: - dependency-name: actions/download-artifact dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/python-package.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 714209c7d..522782e62 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -133,7 +133,7 @@ jobs: if: github.event_name == 'release' && github.event.action == 'created' steps: - name: Download the sdist artifact - uses: actions/download-artifact@v3 + uses: actions/download-artifact@v4 with: name: artifact path: dist From 73fe3e1ceeb8e123fed1531af225f85124e815d8 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 14 Dec 2023 14:39:37 -0500 Subject: [PATCH 37/54] Bump actions/upload-artifact from 3 to 4 (#590) Bumps [actions/upload-artifact](https://github.com/actions/upload-artifact) from 3 to 4. - [Release notes](https://github.com/actions/upload-artifact/releases) - [Commits](https://github.com/actions/upload-artifact/compare/v3...v4) --- updated-dependencies: - dependency-name: actions/upload-artifact dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: William Barnhart --- .github/workflows/python-package.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 522782e62..ac980db6e 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -101,7 +101,7 @@ jobs: fetch-depth: 0 - name: Build wheels uses: pypa/cibuildwheel@v2.10.1 - - uses: actions/upload-artifact@v3 + - uses: actions/upload-artifact@v4 with: path: ./wheelhouse/*.whl build_sdist: @@ -119,7 +119,7 @@ jobs: run: > pip3 install pkgconfig cython --upgrade && python3 setup.py sdist - - uses: actions/upload-artifact@v3 + - uses: actions/upload-artifact@v4 name: Upload build artifacts with: path: dist/*.tar.gz From 66fd6ba8b949e434285cfea53a26b03a6a50dcf4 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 19 Dec 2023 13:53:28 -0500 Subject: [PATCH 38/54] Bump actions/deploy-pages from 3 to 4 (#592) Bumps [actions/deploy-pages](https://github.com/actions/deploy-pages) from 3 to 4. - [Release notes](https://github.com/actions/deploy-pages/releases) - [Commits](https://github.com/actions/deploy-pages/compare/v3...v4) --- updated-dependencies: - dependency-name: actions/deploy-pages dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/gh-pages.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/gh-pages.yml b/.github/workflows/gh-pages.yml index 833cd0ec0..e48a8f328 100644 --- a/.github/workflows/gh-pages.yml +++ b/.github/workflows/gh-pages.yml @@ -42,6 +42,6 @@ jobs: name: github-pages url: ${{ steps.deployment.outputs.page_url }} steps: - - uses: actions/deploy-pages@v3 + - uses: actions/deploy-pages@v4 id: deployment name: Deploy to GitHub Pages From 38121584fa0fe971c36fa53fd537b3c73a970cc2 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 19 Dec 2023 14:01:06 -0500 Subject: [PATCH 39/54] Bump actions/upload-pages-artifact from 2 to 3 (#593) Bumps [actions/upload-pages-artifact](https://github.com/actions/upload-pages-artifact) from 2 to 3. - [Release notes](https://github.com/actions/upload-pages-artifact/releases) - [Commits](https://github.com/actions/upload-pages-artifact/compare/v2...v3) --- updated-dependencies: - dependency-name: actions/upload-pages-artifact dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/gh-pages.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/gh-pages.yml b/.github/workflows/gh-pages.yml index e48a8f328..8ac306d81 100644 --- a/.github/workflows/gh-pages.yml +++ b/.github/workflows/gh-pages.yml @@ -24,7 +24,7 @@ jobs: - name: Install doc build deps and build with Sphinx run: make docs - name: Upload artifacts - uses: actions/upload-pages-artifact@v2 + uses: actions/upload-pages-artifact@v3 with: # Upload built docs path: ./Documentation From 6588a9774babd6aee61f2ce69572b167850717b4 Mon Sep 17 00:00:00 2001 From: duduru <50397689+hgalytoby@users.noreply.github.com> Date: Thu, 28 Dec 2023 12:10:33 +0800 Subject: [PATCH 40/54] Add example for medium/large projects in FastAPI (#595) --- examples/fastapi/__init__.py | 0 examples/fastapi/api/__init__.py | 7 ++++ examples/fastapi/api/my_api.py | 16 ++++++++ examples/fastapi/main.py | 42 +++++++++++++++++++++ examples/fastapi/my_faust/__init__.py | 0 examples/fastapi/my_faust/agent/__init__.py | 0 examples/fastapi/my_faust/agent/my_agent.py | 10 +++++ examples/fastapi/my_faust/app.py | 18 +++++++++ examples/fastapi/my_faust/table/__init__.py | 0 examples/fastapi/my_faust/table/my_table.py | 8 ++++ examples/fastapi/my_faust/timer/__init__.py | 6 +++ examples/fastapi/my_faust/timer/my_timer.py | 14 +++++++ examples/fastapi/my_faust/topic/__init__.py | 0 examples/fastapi/my_faust/topic/my_topic.py | 3 ++ 14 files changed, 124 insertions(+) create mode 100644 examples/fastapi/__init__.py create mode 100644 examples/fastapi/api/__init__.py create mode 100644 examples/fastapi/api/my_api.py create mode 100644 examples/fastapi/main.py create mode 100644 examples/fastapi/my_faust/__init__.py create mode 100644 examples/fastapi/my_faust/agent/__init__.py create mode 100644 examples/fastapi/my_faust/agent/my_agent.py create mode 100644 examples/fastapi/my_faust/app.py create mode 100644 examples/fastapi/my_faust/table/__init__.py create mode 100644 examples/fastapi/my_faust/table/my_table.py create mode 100644 examples/fastapi/my_faust/timer/__init__.py create mode 100644 examples/fastapi/my_faust/timer/my_timer.py create mode 100644 examples/fastapi/my_faust/topic/__init__.py create mode 100644 examples/fastapi/my_faust/topic/my_topic.py diff --git a/examples/fastapi/__init__.py b/examples/fastapi/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/examples/fastapi/api/__init__.py b/examples/fastapi/api/__init__.py new file mode 100644 index 000000000..251300510 --- /dev/null +++ b/examples/fastapi/api/__init__.py @@ -0,0 +1,7 @@ +from fastapi import APIRouter + +from api.my_api import router as my_api_router + +router = APIRouter() + +router.include_router(my_api_router) diff --git a/examples/fastapi/api/my_api.py b/examples/fastapi/api/my_api.py new file mode 100644 index 000000000..e03859834 --- /dev/null +++ b/examples/fastapi/api/my_api.py @@ -0,0 +1,16 @@ +from typing import Union +from fastapi import APIRouter + +from my_faust.table.my_table import greetings_table + +router = APIRouter() + + +@router.get("/items/{item_id}") +def read_item(item_id: int, q: Union[str, None] = None): + return {"item_id": item_id, "q": q} + + +@router.get("/table") +def read_table(): + return [{k: v} for k, v in greetings_table.items()] diff --git a/examples/fastapi/main.py b/examples/fastapi/main.py new file mode 100644 index 000000000..82e3c6b2c --- /dev/null +++ b/examples/fastapi/main.py @@ -0,0 +1,42 @@ +from contextlib import asynccontextmanager +from fastapi import FastAPI +from api import router as api_router + +from my_faust.timer import router as timer_router +from my_faust.app import faust_app + + +# This is just hello_world.py integrated with a FastAPI application + + +def fake_answer_to_everything_ml_model(x: float): + return x * 42 + + +ml_models = {} + + +@asynccontextmanager +async def lifespan(app: FastAPI): + faust_app.discover() + await faust_app.start() + yield + await faust_app.stop() + + +# You MUST have "app" defined in order for Faust to discover the app +# if you're using "faust" on CLI, but this doesn't work yet +app = fastapi_app = FastAPI( + lifespan=lifespan, +) + +# For now, run via "uvicorn fastapi_example:app" +# then visit http://127.0.0.1:8000/docs + +app.include_router(router=api_router) +app.include_router(router=timer_router) + + +@app.get("/") +def read_root(): + return {"Hello": "World"} diff --git a/examples/fastapi/my_faust/__init__.py b/examples/fastapi/my_faust/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/examples/fastapi/my_faust/agent/__init__.py b/examples/fastapi/my_faust/agent/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/examples/fastapi/my_faust/agent/my_agent.py b/examples/fastapi/my_faust/agent/my_agent.py new file mode 100644 index 000000000..ee4ccb6af --- /dev/null +++ b/examples/fastapi/my_faust/agent/my_agent.py @@ -0,0 +1,10 @@ +from my_faust.app import faust_app +from my_faust.table.my_table import greetings_table +from my_faust.topic.my_topic import greetings_topic + + +@faust_app.agent(greetings_topic) +async def print_greetings(greetings): + async for greeting in greetings: + print(f"greeting: {greeting}") + greetings_table[greeting] = {"hello world"} diff --git a/examples/fastapi/my_faust/app.py b/examples/fastapi/my_faust/app.py new file mode 100644 index 000000000..8a5b5f79b --- /dev/null +++ b/examples/fastapi/my_faust/app.py @@ -0,0 +1,18 @@ +import faust + + +def get_all_packages_to_scan(): + return ["my_faust"] + + +# You MUST have "app" defined in order for Faust to discover the app +# if you're using "faust" on CLI, but this doesn't work yet +# autodiscover https://faust-streaming.github.io/faust/userguide/settings.html#autodiscover +app = faust_app = faust.App( + 'hello-world-fastapi', + broker='kafka://localhost:9092', + web_enabled=False, + autodiscover=get_all_packages_to_scan, +) + +# For now, run via "faust -A my_faust.app worker -l info" diff --git a/examples/fastapi/my_faust/table/__init__.py b/examples/fastapi/my_faust/table/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/examples/fastapi/my_faust/table/my_table.py b/examples/fastapi/my_faust/table/my_table.py new file mode 100644 index 000000000..fda9b0fc3 --- /dev/null +++ b/examples/fastapi/my_faust/table/my_table.py @@ -0,0 +1,8 @@ +from my_faust.app import faust_app + +greetings_table = faust_app.GlobalTable( + name="greetings_table", + default=dict, + partitions=1, + recovery_buffer_size=1, +) diff --git a/examples/fastapi/my_faust/timer/__init__.py b/examples/fastapi/my_faust/timer/__init__.py new file mode 100644 index 000000000..d27e8c112 --- /dev/null +++ b/examples/fastapi/my_faust/timer/__init__.py @@ -0,0 +1,6 @@ +from fastapi import APIRouter +from my_faust.timer.my_timer import router as my_timer_router + +router = APIRouter() + +router.include_router(my_timer_router) diff --git a/examples/fastapi/my_faust/timer/my_timer.py b/examples/fastapi/my_faust/timer/my_timer.py new file mode 100644 index 000000000..57261aafd --- /dev/null +++ b/examples/fastapi/my_faust/timer/my_timer.py @@ -0,0 +1,14 @@ +from uuid import uuid4 +from fastapi import APIRouter + +from my_faust.app import faust_app +from my_faust.topic.my_topic import greetings_topic + +router = APIRouter() + + +@faust_app.timer(5) # make sure you *always* add the timer above if you're using one +@router.get("/produce") +async def produce(): + await greetings_topic.send(value=uuid4().hex) + return {"success": True} diff --git a/examples/fastapi/my_faust/topic/__init__.py b/examples/fastapi/my_faust/topic/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/examples/fastapi/my_faust/topic/my_topic.py b/examples/fastapi/my_faust/topic/my_topic.py new file mode 100644 index 000000000..b5e21a288 --- /dev/null +++ b/examples/fastapi/my_faust/topic/my_topic.py @@ -0,0 +1,3 @@ +from my_faust.app import faust_app + +greetings_topic = faust_app.topic("greetings", value_type=str) From 1f1f7595c3d1601e61c46a2c05430f1c68660630 Mon Sep 17 00:00:00 2001 From: Sam Friedman Date: Wed, 3 Jan 2024 11:06:14 -0600 Subject: [PATCH 41/54] Use `loop.create_task()` for agent tasks (#598) * Update agent.py to call loop.create_task rather than asyncio.Task Credit goes to @samtx for finding this solution for https://github.com/faust-streaming/faust/issues/175#issuecomment-1177730361 * Update agent.py Co-authored-by: Sam Friedman * Fix linting in agent.py * Update unit test to verify slurp is awaited * Update unit test to check awaitable task creation * Linting and formatting --------- Co-authored-by: William Barnhart Co-authored-by: Sam Friedman --- faust/agents/agent.py | 4 ++- tests/unit/agents/test_agent.py | 61 +++++++++++++++++---------------- 2 files changed, 35 insertions(+), 30 deletions(-) diff --git a/faust/agents/agent.py b/faust/agents/agent.py index 1f7060a11..9ebd896af 100644 --- a/faust/agents/agent.py +++ b/faust/agents/agent.py @@ -661,7 +661,9 @@ async def _prepare_actor(self, aref: ActorRefT, beacon: NodeT) -> ActorRefT: else: # agent yields and is an AsyncIterator so we have to consume it. coro = self._slurp(aref, aiter(aref)) - task = asyncio.Task(self._execute_actor(coro, aref), loop=self.loop) + # Calling asyncio.Task is not proper usage of asyncio, + # we need to create the task directly from the loop + task = self.loop.create_task(self._execute_actor(coro, aref)) task._beacon = beacon # type: ignore aref.actor_task = task self._actors.add(aref) diff --git a/tests/unit/agents/test_agent.py b/tests/unit/agents/test_agent.py index bb31b75b8..e58d876ee 100644 --- a/tests/unit/agents/test_agent.py +++ b/tests/unit/agents/test_agent.py @@ -392,22 +392,23 @@ async def test_start_task(self, *, agent): assert ret is agent._prepare_actor.return_value @pytest.mark.asyncio - async def test_prepare_actor__AsyncIterable(self, *, agent): + async def test_prepare_actor__AsyncIterable(self, *, agent, monkeypatch): + async def mock_execute_actor(coro, aref): + await coro + + mock_beacon = Mock(name="beacon", autospec=Node) + mock_slurp = AsyncMock(name="slurp") + monkeypatch.setattr(agent, "_slurp", mock_slurp) + monkeypatch.setattr(agent, "_execute_actor", mock_execute_actor) aref = agent(index=0, active_partitions=None) - with patch("asyncio.Task") as Task: - agent._slurp = Mock(name="_slurp") - agent._execute_actor = Mock(name="_execute_actor") - beacon = Mock(name="beacon", autospec=Node) - ret = await agent._prepare_actor(aref, beacon) - agent._slurp.assert_called() - coro = agent._slurp() - agent._execute_actor.assert_called_once_with(coro, aref) - Task.assert_called_once_with(agent._execute_actor(), loop=agent.loop) - task = Task() - assert task._beacon is beacon - assert aref.actor_task is task - assert aref in agent._actors - assert ret is aref + ret = await agent._prepare_actor(aref, mock_beacon) + task = aref.actor_task + await task + mock_slurp.assert_awaited() + assert mock_slurp.await_args.args[0] is aref + assert task._beacon is mock_beacon + assert aref in agent._actors + assert ret is aref @pytest.mark.asyncio async def test_prepare_actor__Awaitable(self, *, agent2): @@ -428,22 +429,24 @@ async def test_prepare_actor__Awaitable(self, *, agent2): assert ret is aref @pytest.mark.asyncio - async def test_prepare_actor__Awaitable_with_multiple_topics(self, *, agent2): + async def test_prepare_actor__Awaitable_with_multiple_topics( + self, *, agent2, monkeypatch + ): aref = agent2(index=0, active_partitions=None) - asyncio.ensure_future(aref.it).cancel() # silence warning agent2.channel.topics = ["foo", "bar"] - with patch("asyncio.Task") as Task: - agent2._execute_actor = Mock(name="_execute_actor") - beacon = Mock(name="beacon", autospec=Node) - ret = await agent2._prepare_actor(aref, beacon) - coro = aref - agent2._execute_actor.assert_called_once_with(coro, aref) - Task.assert_called_once_with(agent2._execute_actor(), loop=agent2.loop) - task = Task() - assert task._beacon is beacon - assert aref.actor_task is task - assert aref in agent2._actors - assert ret is aref + mock_beacon = Mock(name="beacon", autospec=Node) + mock_slurp = AsyncMock(name="slurp") + mock_execute_actor = AsyncMock(name="execute_actor") + monkeypatch.setattr(agent2, "_slurp", mock_slurp) + monkeypatch.setattr(agent2, "_execute_actor", mock_execute_actor) + ret = await agent2._prepare_actor(aref, mock_beacon) + task = aref.actor_task + mock_slurp.assert_not_called() + mock_slurp.assert_not_awaited() + mock_execute_actor.assert_called_with(aref, aref) + assert task._beacon is mock_beacon + assert aref in agent2._actors + assert ret is aref @pytest.mark.asyncio async def test_prepare_actor__Awaitable_cannot_have_sinks(self, *, agent2): From e91c55fea51f77306da2b2a0e1e3718c34081aa3 Mon Sep 17 00:00:00 2001 From: William Barnhart Date: Wed, 3 Jan 2024 16:53:05 -0500 Subject: [PATCH 42/54] Revert to upload-artifact v3 in python-package.yml --- .github/workflows/python-package.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index ac980db6e..03d5642b1 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -101,7 +101,7 @@ jobs: fetch-depth: 0 - name: Build wheels uses: pypa/cibuildwheel@v2.10.1 - - uses: actions/upload-artifact@v4 + - uses: actions/upload-artifact@v3 with: path: ./wheelhouse/*.whl build_sdist: From 4a09533d3cb54801238545b4fa17fa74ac130138 Mon Sep 17 00:00:00 2001 From: Marco Moser <63156265+Mopsgeschwindigkeit@users.noreply.github.com> Date: Thu, 11 Jan 2024 15:05:08 +0100 Subject: [PATCH 43/54] enable kafka static-partition-assignment (KIP-345) (#600) --- faust/transport/drivers/aiokafka.py | 2 +- tests/unit/transport/drivers/test_aiokafka.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/faust/transport/drivers/aiokafka.py b/faust/transport/drivers/aiokafka.py index 8838f557c..2bfc84078 100644 --- a/faust/transport/drivers/aiokafka.py +++ b/faust/transport/drivers/aiokafka.py @@ -525,7 +525,7 @@ def _create_worker_consumer( api_version=conf.consumer_api_version, client_id=conf.broker_client_id, group_id=conf.id, - # group_instance_id=conf.consumer_group_instance_id, + group_instance_id=conf.consumer_group_instance_id, bootstrap_servers=server_list(transport.url, transport.default_port), partition_assignment_strategy=[self._assignor], enable_auto_commit=False, diff --git a/tests/unit/transport/drivers/test_aiokafka.py b/tests/unit/transport/drivers/test_aiokafka.py index 1130f2cb8..b524aac23 100644 --- a/tests/unit/transport/drivers/test_aiokafka.py +++ b/tests/unit/transport/drivers/test_aiokafka.py @@ -787,7 +787,7 @@ def assert_create_worker_consumer( api_version=app.conf.consumer_api_version, client_id=conf.broker_client_id, group_id=conf.id, - # group_instance_id=conf.consumer_group_instance_id, + group_instance_id=conf.consumer_group_instance_id, bootstrap_servers=server_list(transport.url, transport.default_port), partition_assignment_strategy=[cthread._assignor], enable_auto_commit=False, From 6d248bf5e0ab7ac5961bad76839149ac03f7bb86 Mon Sep 17 00:00:00 2001 From: Timothy Thomas Date: Fri, 16 Feb 2024 14:32:41 -0600 Subject: [PATCH 44/54] Run black and fix setup.cfg to add E701 and E704 to flake8 ignore (#609) --- faust/__main__.py | 1 + faust/agents/__init__.py | 1 + faust/agents/actor.py | 1 + faust/agents/agent.py | 4 +- faust/agents/manager.py | 1 + faust/agents/models.py | 1 + faust/agents/replies.py | 4 +- faust/app/__init__.py | 1 + faust/app/_attached.py | 4 +- faust/app/base.py | 13 +- faust/app/router.py | 1 + faust/assignor/client_assignment.py | 1 + faust/assignor/cluster_assignment.py | 1 + faust/assignor/copartitioned_assignor.py | 1 + faust/assignor/leader_assignor.py | 1 + faust/assignor/partition_assignor.py | 1 + faust/auth.py | 1 + faust/channels.py | 1 + faust/cli/__init__.py | 1 + faust/cli/agents.py | 1 + faust/cli/base.py | 4 +- faust/cli/clean_versions.py | 1 + faust/cli/completion.py | 1 + faust/cli/livecheck.py | 1 + faust/cli/model.py | 1 + faust/cli/models.py | 1 + faust/cli/params.py | 1 + faust/cli/reset.py | 1 + faust/cli/send.py | 1 + faust/cli/tables.py | 1 + faust/cli/worker.py | 1 + faust/contrib/sentry.py | 3 +- faust/events.py | 4 +- faust/exceptions.py | 4 +- faust/fixups/__init__.py | 1 + faust/fixups/base.py | 1 + faust/fixups/django.py | 7 +- faust/joins.py | 1 + faust/livecheck/__init__.py | 1 + faust/livecheck/app.py | 1 + faust/livecheck/case.py | 7 +- faust/livecheck/locals.py | 4 +- faust/livecheck/models.py | 1 + faust/livecheck/patches/__init__.py | 1 + faust/livecheck/patches/aiohttp.py | 4 +- faust/livecheck/runners.py | 4 +- faust/livecheck/signals.py | 4 +- faust/models/__init__.py | 1 + faust/models/base.py | 1 + faust/models/record.py | 1 + faust/models/tags.py | 21 +- faust/models/typing.py | 7 +- faust/sensors/__init__.py | 1 + faust/sensors/base.py | 4 +- faust/sensors/datadog.py | 4 +- faust/sensors/monitor.py | 1 + faust/sensors/prometheus.py | 1 + faust/sensors/statsd.py | 4 +- faust/serializers/__init__.py | 1 + faust/serializers/codecs.py | 1 + faust/serializers/registry.py | 1 + faust/serializers/schemas.py | 3 +- faust/stores/__init__.py | 1 + faust/stores/aerospike.py | 1 + faust/stores/base.py | 1 + faust/stores/memory.py | 1 + faust/stores/rocksdb.py | 1 + faust/streams.py | 1 + faust/tables/__init__.py | 1 + faust/tables/base.py | 1 + faust/tables/manager.py | 4 +- faust/tables/objects.py | 1 + faust/tables/recovery.py | 7 +- faust/tables/sets.py | 1 + faust/tables/table.py | 1 + faust/tables/wrappers.py | 4 +- faust/topics.py | 4 +- faust/transport/__init__.py | 1 + faust/transport/base.py | 1 + faust/transport/conductor.py | 4 +- faust/transport/consumer.py | 28 +- faust/transport/drivers/__init__.py | 1 + faust/transport/drivers/aiokafka.py | 1 + faust/transport/producer.py | 1 + faust/transport/utils.py | 1 + faust/types/__init__.py | 1 + faust/types/_env.py | 1 + faust/types/agents.py | 108 +++----- faust/types/app.py | 219 +++++---------- faust/types/assignor.py | 36 +-- faust/types/channels.py | 104 +++----- faust/types/codecs.py | 17 +- faust/types/core.py | 3 +- faust/types/events.py | 21 +- faust/types/fixups.py | 15 +- faust/types/joins.py | 6 +- faust/types/models.py | 71 ++--- faust/types/router.py | 28 +- faust/types/sensors.py | 78 ++---- faust/types/serializers.py | 48 ++-- faust/types/settings/base.py | 12 +- faust/types/settings/params.py | 9 +- faust/types/settings/sections.py | 3 +- faust/types/settings/settings.py | 6 +- faust/types/stores.py | 39 +-- faust/types/streams.py | 95 +++---- faust/types/tables.py | 252 ++++++------------ faust/types/topics.py | 30 +-- faust/types/transports.py | 194 +++++--------- faust/types/tuples.py | 6 +- faust/types/web.py | 51 ++-- faust/types/windows.py | 16 +- faust/utils/__init__.py | 1 + faust/utils/_iso8601_python.py | 1 + faust/utils/agent_stopper.py | 1 + faust/utils/codegen.py | 1 + faust/utils/cron.py | 1 + faust/utils/functional.py | 1 + faust/utils/iso8601.py | 1 + faust/utils/json.py | 1 + faust/utils/kafka/protocol/admin.py | 1 + faust/utils/kafka/protocol/api.py | 1 + faust/utils/platforms.py | 1 + faust/utils/terminal/__init__.py | 1 + faust/utils/terminal/spinners.py | 1 + faust/utils/terminal/tables.py | 1 + faust/utils/tracing.py | 1 + faust/utils/urls.py | 1 + faust/utils/venusian.py | 4 +- faust/web/__init__.py | 1 + faust/web/apps/graph.py | 1 + faust/web/apps/production_index.py | 1 + faust/web/apps/router.py | 1 + faust/web/apps/stats.py | 1 + faust/web/apps/tables.py | 1 + faust/web/base.py | 1 + faust/web/blueprints.py | 1 + faust/web/cache/__init__.py | 1 + faust/web/cache/backends/__init__.py | 1 + faust/web/cache/backends/base.py | 10 +- faust/web/cache/backends/memory.py | 1 + faust/web/cache/backends/redis.py | 4 +- faust/web/cache/cache.py | 1 + faust/web/drivers/__init__.py | 1 + faust/web/drivers/aiohttp.py | 1 + faust/web/exceptions.py | 1 + faust/web/views.py | 1 + faust/windows.py | 1 + faust/worker.py | 4 +- setup.cfg | 2 +- tests/conftest.py | 3 +- tests/functional/agents/helpers.py | 3 +- tests/functional/test_app.py | 3 +- tests/functional/test_models.py | 8 +- tests/functional/web/test_cache.py | 3 +- tests/unit/agents/test_agent.py | 12 +- tests/unit/app/test_base.py | 45 ++-- tests/unit/app/test_service.py | 3 +- tests/unit/cli/test_base.py | 6 +- tests/unit/serializers/test_codecs.py | 3 +- tests/unit/stores/test_aerospike.py | 3 +- tests/unit/stores/test_base.py | 36 +-- tests/unit/transport/drivers/test_aiokafka.py | 3 +- tests/unit/transport/test_consumer.py | 66 ++--- tests/unit/web/test_base.py | 30 +-- 165 files changed, 749 insertions(+), 1227 deletions(-) diff --git a/faust/__main__.py b/faust/__main__.py index 6d3aa93b6..f68f971a2 100644 --- a/faust/__main__.py +++ b/faust/__main__.py @@ -1,4 +1,5 @@ """Command-line entry point.""" + # pragma: no cover from faust.cli.faust import cli diff --git a/faust/agents/__init__.py b/faust/agents/__init__.py index ed16385bd..aa720f3b8 100644 --- a/faust/agents/__init__.py +++ b/faust/agents/__init__.py @@ -1,4 +1,5 @@ """Agents.""" + from .agent import Agent, AgentFun, AgentT, SinkT, current_agent from .manager import AgentManager, AgentManagerT from .replies import ReplyConsumer diff --git a/faust/agents/actor.py b/faust/agents/actor.py index c8d743233..34c0f9d70 100644 --- a/faust/agents/actor.py +++ b/faust/agents/actor.py @@ -1,4 +1,5 @@ """Actor - Individual Agent instances.""" + from typing import Any, AsyncGenerator, AsyncIterator, Coroutine, Optional, Set, cast from mode import Service diff --git a/faust/agents/agent.py b/faust/agents/agent.py index 9ebd896af..036b0c46d 100644 --- a/faust/agents/agent.py +++ b/faust/agents/agent.py @@ -1,4 +1,5 @@ """Agent implementation.""" + import asyncio import typing from contextlib import suppress @@ -79,8 +80,7 @@ from faust.app.base import App as _App else: - class _App: - ... # noqa + class _App: ... # noqa __all__ = ["Agent"] diff --git a/faust/agents/manager.py b/faust/agents/manager.py index 8ab0e3618..b1e164383 100644 --- a/faust/agents/manager.py +++ b/faust/agents/manager.py @@ -1,4 +1,5 @@ """Agent manager.""" + import asyncio from collections import defaultdict from typing import Any, Dict, List, Mapping, MutableMapping, MutableSet, Set diff --git a/faust/agents/models.py b/faust/agents/models.py index 4fec41a39..f3ca72930 100644 --- a/faust/agents/models.py +++ b/faust/agents/models.py @@ -1,4 +1,5 @@ """Models used by agents internally.""" + from typing import Any from faust.models import Record diff --git a/faust/agents/replies.py b/faust/agents/replies.py index 69e87c953..b4e4247c8 100644 --- a/faust/agents/replies.py +++ b/faust/agents/replies.py @@ -1,4 +1,5 @@ """Agent replies: waiting for replies, sending them, etc.""" + import asyncio from collections import defaultdict from typing import Any, AsyncIterator, MutableMapping, MutableSet, NamedTuple, Optional @@ -33,8 +34,7 @@ def __init__(self, reply_to: str, correlation_id: str = "", **kwargs: Any) -> No self.__post_init__() super().__init__(**kwargs) - def __post_init__(self) -> None: - ... + def __post_init__(self) -> None: ... def _verify_correlation_id(self, correlation_id: str) -> None: if not correlation_id: diff --git a/faust/app/__init__.py b/faust/app/__init__.py index 1a6fc5109..db178690d 100644 --- a/faust/app/__init__.py +++ b/faust/app/__init__.py @@ -1,4 +1,5 @@ """Application.""" + from .base import App, BootStrategy __all__ = ["App", "BootStrategy"] diff --git a/faust/app/_attached.py b/faust/app/_attached.py index da696727a..2295f6afa 100644 --- a/faust/app/_attached.py +++ b/faust/app/_attached.py @@ -2,6 +2,7 @@ Attachments were used before transactions support. """ + import asyncio import typing from collections import defaultdict @@ -29,8 +30,7 @@ from faust.events import Event as _Event else: - class _Event: - ... # noqa + class _Event: ... # noqa __all__ = ["Attachment", "Attachments"] diff --git a/faust/app/base.py b/faust/app/base.py index bc99f99a7..5205e3968 100644 --- a/faust/app/base.py +++ b/faust/app/base.py @@ -4,6 +4,7 @@ Everything starts here. """ + import asyncio import importlib import inspect @@ -113,17 +114,13 @@ from faust.worker import Worker as _Worker else: - class _AppCommand: - ... # noqa + class _AppCommand: ... # noqa - class _LiveCheck: - ... # noqa + class _LiveCheck: ... # noqa - class _Fetcher: - ... # noqa + class _Fetcher: ... # noqa - class _Worker: - ... # noqa + class _Worker: ... # noqa __all__ = ["App", "BootStrategy"] diff --git a/faust/app/router.py b/faust/app/router.py index 2b5164db6..e15f6efb7 100644 --- a/faust/app/router.py +++ b/faust/app/router.py @@ -1,4 +1,5 @@ """Route messages to Faust nodes by partitioning.""" + from typing import Tuple from yarl import URL diff --git a/faust/assignor/client_assignment.py b/faust/assignor/client_assignment.py index 218ed528f..36069b127 100644 --- a/faust/assignor/client_assignment.py +++ b/faust/assignor/client_assignment.py @@ -1,4 +1,5 @@ """Client Assignment.""" + import copy from typing import List, Mapping, MutableMapping, Sequence, Set, Tuple, cast diff --git a/faust/assignor/cluster_assignment.py b/faust/assignor/cluster_assignment.py index d9ff6b66d..433359746 100644 --- a/faust/assignor/cluster_assignment.py +++ b/faust/assignor/cluster_assignment.py @@ -1,4 +1,5 @@ """Cluster assignment.""" + from typing import List, MutableMapping, Sequence, Set, cast from faust.models import Record diff --git a/faust/assignor/copartitioned_assignor.py b/faust/assignor/copartitioned_assignor.py index a06c8d8f4..093781afd 100644 --- a/faust/assignor/copartitioned_assignor.py +++ b/faust/assignor/copartitioned_assignor.py @@ -1,4 +1,5 @@ """Copartitioned Assignor.""" + from itertools import cycle from math import ceil, floor from typing import Iterable, Iterator, MutableMapping, Optional, Sequence, Set diff --git a/faust/assignor/leader_assignor.py b/faust/assignor/leader_assignor.py index 535667250..9e6ba3366 100644 --- a/faust/assignor/leader_assignor.py +++ b/faust/assignor/leader_assignor.py @@ -1,4 +1,5 @@ """Leader assignor.""" + from typing import Any from mode import Service diff --git a/faust/assignor/partition_assignor.py b/faust/assignor/partition_assignor.py index 705079620..d8988b5c6 100644 --- a/faust/assignor/partition_assignor.py +++ b/faust/assignor/partition_assignor.py @@ -1,4 +1,5 @@ """Partition assignor.""" + import socket import zlib from collections import defaultdict diff --git a/faust/auth.py b/faust/auth.py index 0c97e6a60..57e6919dd 100644 --- a/faust/auth.py +++ b/faust/auth.py @@ -1,4 +1,5 @@ """Authentication Credentials.""" + import ssl from typing import Any, Optional, Union diff --git a/faust/channels.py b/faust/channels.py index a338831b1..4ef013cfe 100644 --- a/faust/channels.py +++ b/faust/channels.py @@ -5,6 +5,7 @@ The stream will iterate over incoming events in the channel. """ + import asyncio from typing import ( Any, diff --git a/faust/cli/__init__.py b/faust/cli/__init__.py index f3ce191b8..71a737ac6 100644 --- a/faust/cli/__init__.py +++ b/faust/cli/__init__.py @@ -1,4 +1,5 @@ """Command-line Interface.""" + from .base import AppCommand, Command, argument, call_command, option __all__ = ["AppCommand", "Command", "argument", "call_command", "option"] diff --git a/faust/cli/agents.py b/faust/cli/agents.py index b5c08503a..cb021e49e 100644 --- a/faust/cli/agents.py +++ b/faust/cli/agents.py @@ -1,4 +1,5 @@ """Program ``faust agents`` used to list agents.""" + from operator import attrgetter from typing import Any, Callable, Optional, Sequence, Type, cast diff --git a/faust/cli/base.py b/faust/cli/base.py index cc5ef4f8b..5715176e8 100644 --- a/faust/cli/base.py +++ b/faust/cli/base.py @@ -1,4 +1,5 @@ """Command-line programs using :pypi:`click`.""" + import abc import asyncio import inspect @@ -49,8 +50,7 @@ from faust.app import App as _App else: - class _App: - ... # noqa + class _App: ... # noqa try: diff --git a/faust/cli/clean_versions.py b/faust/cli/clean_versions.py index 8c1e8d396..785598efb 100644 --- a/faust/cli/clean_versions.py +++ b/faust/cli/clean_versions.py @@ -1,4 +1,5 @@ """Program ``faust reset`` used to delete local table state.""" + from shutil import rmtree from .base import AppCommand diff --git a/faust/cli/completion.py b/faust/cli/completion.py index d9f18b126..afc8ac109 100644 --- a/faust/cli/completion.py +++ b/faust/cli/completion.py @@ -2,6 +2,7 @@ Supports ``bash``, ``ksh``, ``zsh``, etc. """ + import os from pathlib import Path diff --git a/faust/cli/livecheck.py b/faust/cli/livecheck.py index 5bf263754..514b796c1 100644 --- a/faust/cli/livecheck.py +++ b/faust/cli/livecheck.py @@ -1,4 +1,5 @@ """Program ``faust worker`` used to start application from console.""" + from typing import Any from .worker import worker diff --git a/faust/cli/model.py b/faust/cli/model.py index 67a1ecd9c..8000d68b3 100644 --- a/faust/cli/model.py +++ b/faust/cli/model.py @@ -1,4 +1,5 @@ """Program ``faust model`` used to list details about a model.""" + from datetime import datetime from typing import Any, Optional, Sequence, Type diff --git a/faust/cli/models.py b/faust/cli/models.py index 03b94cc63..6aa8c2b02 100644 --- a/faust/cli/models.py +++ b/faust/cli/models.py @@ -1,4 +1,5 @@ """Program ``faust models`` used to list models available.""" + from operator import attrgetter from typing import Any, Callable, Sequence, Type, cast diff --git a/faust/cli/params.py b/faust/cli/params.py index ed67cf058..ec62d5f23 100644 --- a/faust/cli/params.py +++ b/faust/cli/params.py @@ -1,4 +1,5 @@ """Python :pypi:`click` parameter types.""" + from typing import Any, Iterable, Optional import click diff --git a/faust/cli/reset.py b/faust/cli/reset.py index 76fa72ad5..d0ffe78e2 100644 --- a/faust/cli/reset.py +++ b/faust/cli/reset.py @@ -1,4 +1,5 @@ """Program ``faust reset`` used to delete local table state.""" + from .base import AppCommand __all__ = ["reset"] diff --git a/faust/cli/send.py b/faust/cli/send.py index cc996f589..ab1f4eacd 100644 --- a/faust/cli/send.py +++ b/faust/cli/send.py @@ -1,4 +1,5 @@ """Program ``faust send`` used to send events to agents and topics.""" + import asyncio import random from typing import Any, Optional diff --git a/faust/cli/tables.py b/faust/cli/tables.py index 1b7c558f2..6801180d8 100644 --- a/faust/cli/tables.py +++ b/faust/cli/tables.py @@ -1,4 +1,5 @@ """Program ``faust tables`` used to list tables.""" + from .base import AppCommand DEFAULT_TABLE_HELP = 'Missing description: use Table(.., help="str")' diff --git a/faust/cli/worker.py b/faust/cli/worker.py index ae22b6980..9c6ad1e96 100644 --- a/faust/cli/worker.py +++ b/faust/cli/worker.py @@ -1,4 +1,5 @@ """Program ``faust worker`` used to start application from console.""" + import asyncio import os import platform diff --git a/faust/contrib/sentry.py b/faust/contrib/sentry.py index 04d260329..6a527ac29 100644 --- a/faust/contrib/sentry.py +++ b/faust/contrib/sentry.py @@ -31,8 +31,7 @@ from raven.handlers.logging import SentryHandler as _SentryHandler else: - class _SentryHandler: - ... # noqa: E701 + class _SentryHandler: ... # noqa: E701 __all__ = ["handler_from_dsn", "setup"] diff --git a/faust/events.py b/faust/events.py index 3e218a16c..e45dd4fe1 100644 --- a/faust/events.py +++ b/faust/events.py @@ -1,4 +1,5 @@ """Events received in streams.""" + import typing from types import TracebackType from typing import Any, Awaitable, Optional, Type, Union, cast @@ -21,8 +22,7 @@ from .app.base import App as _App else: - class _App: - ... # noqa + class _App: ... # noqa USE_EXISTING_KEY = object() diff --git a/faust/exceptions.py b/faust/exceptions.py index b2ea924a0..dfe4a5e48 100644 --- a/faust/exceptions.py +++ b/faust/exceptions.py @@ -1,4 +1,5 @@ """Faust exceptions.""" + import typing __all__ = [ @@ -25,8 +26,7 @@ from .types.models import FieldDescriptorT as _FieldDescriptorT else: - class _FieldDescriptorT: - ... # noqa + class _FieldDescriptorT: ... # noqa class FaustError(Exception): diff --git a/faust/fixups/__init__.py b/faust/fixups/__init__.py index f6466d7b1..2887f93b1 100644 --- a/faust/fixups/__init__.py +++ b/faust/fixups/__init__.py @@ -1,4 +1,5 @@ """Transport registry.""" + from typing import Iterator, Type from mode.utils.imports import FactoryMapping diff --git a/faust/fixups/base.py b/faust/fixups/base.py index e0a8fe8f6..6be82f2d7 100644 --- a/faust/fixups/base.py +++ b/faust/fixups/base.py @@ -1,4 +1,5 @@ """Fixups - Base implementation.""" + from typing import Iterable from faust.types import AppT, FixupT diff --git a/faust/fixups/django.py b/faust/fixups/django.py index 804d9018a..dddc69bf3 100644 --- a/faust/fixups/django.py +++ b/faust/fixups/django.py @@ -1,4 +1,5 @@ """Django Fixups - Integration with Django.""" + import os import typing import warnings @@ -14,11 +15,9 @@ from django.settings import Settings as _Settings else: - class _Apps: - ... # noqa + class _Apps: ... # noqa - class _Settings: - ... # noqa + class _Settings: ... # noqa __all__ = ["Fixup"] diff --git a/faust/joins.py b/faust/joins.py index 761b411cc..8767df13e 100644 --- a/faust/joins.py +++ b/faust/joins.py @@ -1,4 +1,5 @@ """Join strategies.""" + from typing import Any, Optional, Tuple from .types import EventT, FieldDescriptorT, JoinableT, JoinT diff --git a/faust/livecheck/__init__.py b/faust/livecheck/__init__.py index 661457a80..847423a11 100644 --- a/faust/livecheck/__init__.py +++ b/faust/livecheck/__init__.py @@ -1,4 +1,5 @@ """LiveCheck - End-to-end testing of asynchronous systems.""" + from .app import LiveCheck from .case import Case from .locals import current_test diff --git a/faust/livecheck/app.py b/faust/livecheck/app.py index 35126a0ad..b2fab3ae8 100644 --- a/faust/livecheck/app.py +++ b/faust/livecheck/app.py @@ -1,4 +1,5 @@ """LiveCheck - Faust Application.""" + import asyncio from datetime import timedelta from typing import ( diff --git a/faust/livecheck/case.py b/faust/livecheck/case.py index 38061a729..5c8cc4d68 100644 --- a/faust/livecheck/case.py +++ b/faust/livecheck/case.py @@ -1,4 +1,5 @@ """LiveCheck - Test cases.""" + import traceback import typing from collections import deque @@ -29,8 +30,7 @@ from .app import LiveCheck as _LiveCheck else: - class _LiveCheck: - ... # noqa + class _LiveCheck: ... # noqa __all__ = ["Case"] @@ -318,8 +318,7 @@ async def _send_frequency(self) -> None: if self.app.is_leader(): await self.make_fake_request() - async def make_fake_request(self) -> None: - ... + async def make_fake_request(self) -> None: ... @Service.task async def _check_frequency(self) -> None: diff --git a/faust/livecheck/locals.py b/faust/livecheck/locals.py index 8a940d732..a7e53e727 100644 --- a/faust/livecheck/locals.py +++ b/faust/livecheck/locals.py @@ -1,4 +1,5 @@ """Locals - Current test & execution context.""" + import typing from typing import Optional @@ -10,8 +11,7 @@ from .runners import TestRunner as _TestRunner else: # pragma: no cover - class _TestRunner: - ... # noqa + class _TestRunner: ... # noqa __all__ = [ diff --git a/faust/livecheck/models.py b/faust/livecheck/models.py index f2ce41f68..a49c1299a 100644 --- a/faust/livecheck/models.py +++ b/faust/livecheck/models.py @@ -1,4 +1,5 @@ """LiveCheck - Models.""" + from datetime import datetime, timezone from enum import Enum from typing import Any, Dict, List, Mapping, Optional diff --git a/faust/livecheck/patches/__init__.py b/faust/livecheck/patches/__init__.py index ecbc6e1af..3ee96a513 100644 --- a/faust/livecheck/patches/__init__.py +++ b/faust/livecheck/patches/__init__.py @@ -1,4 +1,5 @@ """Patches - LiveCheck integration with other frameworks/libraries.""" + from . import aiohttp __all__ = ["aiohttp", "patch_all"] diff --git a/faust/livecheck/patches/aiohttp.py b/faust/livecheck/patches/aiohttp.py index 023248436..29aa3b39f 100644 --- a/faust/livecheck/patches/aiohttp.py +++ b/faust/livecheck/patches/aiohttp.py @@ -1,4 +1,5 @@ """LiveCheck :pypi:`aiohttp` integration.""" + from contextlib import ExitStack from types import SimpleNamespace from typing import Any, List, Optional, no_type_check @@ -26,8 +27,7 @@ def patch_aiohttp_session() -> None: from aiohttp import TraceConfig, client # monkeypatch to remove ridiculous "do not subclass" warning. - def __init_subclass__() -> None: - ... + def __init_subclass__() -> None: ... client.ClientSession.__init_subclass__ = __init_subclass__ # type: ignore diff --git a/faust/livecheck/runners.py b/faust/livecheck/runners.py index 6a9306395..a1d26fb1d 100644 --- a/faust/livecheck/runners.py +++ b/faust/livecheck/runners.py @@ -1,4 +1,5 @@ """LiveCheck - Test runner.""" + import asyncio import logging import traceback @@ -27,8 +28,7 @@ from .case import Case as _Case else: - class _Case: - ... # noqa + class _Case: ... # noqa __all__ = ["TestRunner"] diff --git a/faust/livecheck/signals.py b/faust/livecheck/signals.py index d7aa9b974..020767da0 100644 --- a/faust/livecheck/signals.py +++ b/faust/livecheck/signals.py @@ -1,4 +1,5 @@ """LiveCheck Signals - Test communication and synchronization.""" + import asyncio import typing from time import monotonic @@ -16,8 +17,7 @@ from .case import Case as _Case else: - class _Case: - ... # noqa + class _Case: ... # noqa __all__ = ["BaseSignal", "Signal"] diff --git a/faust/models/__init__.py b/faust/models/__init__.py index 2534d5279..1c14af735 100644 --- a/faust/models/__init__.py +++ b/faust/models/__init__.py @@ -1,4 +1,5 @@ """Models.""" + from .base import Model, ModelOptions, maybe_model, registry from .fields import FieldDescriptor, StringField from .record import Record diff --git a/faust/models/base.py b/faust/models/base.py index 6193f115c..6b7c552a6 100644 --- a/faust/models/base.py +++ b/faust/models/base.py @@ -27,6 +27,7 @@ Models are mainly used for describing the data in messages: both keys and values can be described as models. """ + import abc import warnings from datetime import datetime diff --git a/faust/models/record.py b/faust/models/record.py index 1d8fd3a54..dbfd65a5e 100644 --- a/faust/models/record.py +++ b/faust/models/record.py @@ -1,4 +1,5 @@ """Record - Dictionary Model.""" + from datetime import datetime from decimal import Decimal from itertools import chain diff --git a/faust/models/tags.py b/faust/models/tags.py index ed0a3cc7c..528287f68 100644 --- a/faust/models/tags.py +++ b/faust/models/tags.py @@ -47,12 +47,10 @@ def __repr__(self) -> str: return f"<{self._name}: {self.field}@{id(self):#x}>" @abc.abstractmethod - def __str__(self) -> str: - ... + def __str__(self) -> str: ... @abc.abstractmethod - def __format__(self, format_spec: str) -> str: - ... + def __format__(self, format_spec: str) -> str: ... @property def _name(self) -> str: @@ -190,21 +188,16 @@ def __class_getitem__(self, params: Any) -> Any: return sup(params) -class _PIIstr(str): - ... +class _PIIstr(str): ... -class _PIIbytes(bytes): - ... +class _PIIbytes(bytes): ... -class _PIIint(int): - ... +class _PIIint(int): ... -class _PIIfloat(float): - ... +class _PIIfloat(float): ... -class _PIIDecimal(Decimal): - ... +class _PIIDecimal(Decimal): ... diff --git a/faust/models/typing.py b/faust/models/typing.py index 4c44deb75..dc9ff0d88 100644 --- a/faust/models/typing.py +++ b/faust/models/typing.py @@ -6,6 +6,7 @@ to deserialize such a structure. """ + import abc import os import random @@ -280,8 +281,7 @@ def random_identifier(self, n: int = 8) -> str: ) @abc.abstractmethod - def build(self, var: Variable, *args: Type) -> str: - ... + def build(self, var: Variable, *args: Type) -> str: ... def __repr__(self) -> str: return f"<{type(self).__name__}: {self.expr!r}>" @@ -604,8 +604,7 @@ class RootNode(Node): found_types: Dict[NodeType, Set[Type]] @classmethod - def _register(cls) -> None: - ... # we do not register root nodes. + def _register(cls) -> None: ... # we do not register root nodes. def add_closure(self, local_name: str, global_name: str, obj: Any) -> None: self.globals[global_name] = obj diff --git a/faust/sensors/__init__.py b/faust/sensors/__init__.py index 6a48926b7..6c7dc41cd 100644 --- a/faust/sensors/__init__.py +++ b/faust/sensors/__init__.py @@ -1,4 +1,5 @@ """Sensors.""" + from .base import Sensor, SensorDelegate from .monitor import Monitor, TableState diff --git a/faust/sensors/base.py b/faust/sensors/base.py index f81fac758..46f679e67 100644 --- a/faust/sensors/base.py +++ b/faust/sensors/base.py @@ -1,4 +1,5 @@ """Base-interface for sensors.""" + from time import monotonic from typing import Any, Dict, Iterator, Mapping, Optional, Set @@ -143,8 +144,7 @@ def on_web_request_end( """Web server finished working on request.""" ... - def on_threaded_producer_buffer_processed(self, app: AppT, size: int) -> None: - ... + def on_threaded_producer_buffer_processed(self, app: AppT, size: int) -> None: ... def asdict(self) -> Mapping: """Convert sensor state to dictionary.""" diff --git a/faust/sensors/datadog.py b/faust/sensors/datadog.py index 3c79174c4..5db905957 100644 --- a/faust/sensors/datadog.py +++ b/faust/sensors/datadog.py @@ -1,4 +1,5 @@ """Monitor using datadog.""" + import re from typing import Any, Dict, List, Optional, cast @@ -26,8 +27,7 @@ except ImportError: # pragma: no cover datadog = None # type: ignore - class DogStatsD: - ... # noqa + class DogStatsD: ... # noqa __all__ = ["DatadogMonitor"] diff --git a/faust/sensors/monitor.py b/faust/sensors/monitor.py index efacca358..44d7da1e1 100644 --- a/faust/sensors/monitor.py +++ b/faust/sensors/monitor.py @@ -1,4 +1,5 @@ """Monitor - sensor tracking metrics.""" + import asyncio import re import weakref diff --git a/faust/sensors/prometheus.py b/faust/sensors/prometheus.py index dca750752..022dea146 100644 --- a/faust/sensors/prometheus.py +++ b/faust/sensors/prometheus.py @@ -1,4 +1,5 @@ """Monitor using Prometheus.""" + import typing from typing import Any, NamedTuple, Optional, cast diff --git a/faust/sensors/statsd.py b/faust/sensors/statsd.py index 26b24fc9a..a9ec1d055 100644 --- a/faust/sensors/statsd.py +++ b/faust/sensors/statsd.py @@ -1,4 +1,5 @@ """Monitor using Statsd.""" + import typing from typing import Any, Dict, Optional, cast @@ -30,8 +31,7 @@ from statsd import StatsClient else: - class StatsClient: - ... # noqa + class StatsClient: ... # noqa __all__ = ["StatsdMonitor"] diff --git a/faust/serializers/__init__.py b/faust/serializers/__init__.py index 1c9b5e327..6720ef950 100644 --- a/faust/serializers/__init__.py +++ b/faust/serializers/__init__.py @@ -1,4 +1,5 @@ """Serializers and Codecs.""" + from .codecs import Codec from .registry import Registry from .schemas import Schema diff --git a/faust/serializers/codecs.py b/faust/serializers/codecs.py index e642f3be4..1d0a15570 100644 --- a/faust/serializers/codecs.py +++ b/faust/serializers/codecs.py @@ -159,6 +159,7 @@ def msgpack() -> codecs.Codec: At this point may want to publish this on PyPI to share the extension with other Faust users. """ + import pickle as _pickle # nosec B403 from base64 import b64decode, b64encode from types import ModuleType diff --git a/faust/serializers/registry.py b/faust/serializers/registry.py index be56984e8..5f2f46e43 100644 --- a/faust/serializers/registry.py +++ b/faust/serializers/registry.py @@ -1,4 +1,5 @@ """Registry of supported codecs (serializers, compressors, etc.).""" + import sys from decimal import Decimal from typing import Any, Optional, Tuple, Type, cast diff --git a/faust/serializers/schemas.py b/faust/serializers/schemas.py index 8192cd2b1..aad969e2f 100644 --- a/faust/serializers/schemas.py +++ b/faust/serializers/schemas.py @@ -27,8 +27,7 @@ OnValueDecodeErrorFun = Callable[[Exception, Message], Awaitable[None]] -async def _noop_decode_error(exc: Exception, message: Message) -> None: - ... +async def _noop_decode_error(exc: Exception, message: Message) -> None: ... class Schema(SchemaT): diff --git a/faust/stores/__init__.py b/faust/stores/__init__.py index 3f536427b..56cc0dc61 100644 --- a/faust/stores/__init__.py +++ b/faust/stores/__init__.py @@ -1,4 +1,5 @@ """Storage registry.""" + from typing import Type from mode.utils.imports import FactoryMapping diff --git a/faust/stores/aerospike.py b/faust/stores/aerospike.py index e713b4068..291ccbe9f 100644 --- a/faust/stores/aerospike.py +++ b/faust/stores/aerospike.py @@ -1,4 +1,5 @@ """Aerospike storage.""" + import time import typing from typing import Any, Dict, Iterator, Optional, Tuple, Union diff --git a/faust/stores/base.py b/faust/stores/base.py index 62b55fda1..011259825 100644 --- a/faust/stores/base.py +++ b/faust/stores/base.py @@ -1,4 +1,5 @@ """Base class for table storage drivers.""" + import abc from collections.abc import ItemsView, KeysView, ValuesView from typing import ( diff --git a/faust/stores/memory.py b/faust/stores/memory.py index 484517628..3966ded60 100644 --- a/faust/stores/memory.py +++ b/faust/stores/memory.py @@ -1,4 +1,5 @@ """In-memory table storage.""" + from typing import Any, Callable, Iterable, MutableMapping, Optional, Set, Tuple, Union from faust.types import TP, EventT diff --git a/faust/stores/rocksdb.py b/faust/stores/rocksdb.py index d614b32c4..d17ba750e 100644 --- a/faust/stores/rocksdb.py +++ b/faust/stores/rocksdb.py @@ -1,4 +1,5 @@ """RocksDB storage.""" + import asyncio import gc import math diff --git a/faust/streams.py b/faust/streams.py index f23d874ad..d2faf482a 100644 --- a/faust/streams.py +++ b/faust/streams.py @@ -1,4 +1,5 @@ """Streams.""" + import asyncio import os import reprlib diff --git a/faust/tables/__init__.py b/faust/tables/__init__.py index d86876531..3e7486ec0 100644 --- a/faust/tables/__init__.py +++ b/faust/tables/__init__.py @@ -1,4 +1,5 @@ """Tables: Distributed object K/V-store.""" + from .base import Collection, CollectionT from .globaltable import GlobalTable, GlobalTableT from .manager import TableManager, TableManagerT diff --git a/faust/tables/base.py b/faust/tables/base.py index 7f90672fa..e3574ee21 100644 --- a/faust/tables/base.py +++ b/faust/tables/base.py @@ -1,4 +1,5 @@ """Base class Collection for Table and future data structures.""" + import abc import time from collections import defaultdict diff --git a/faust/tables/manager.py b/faust/tables/manager.py index c26622319..9b4952555 100644 --- a/faust/tables/manager.py +++ b/faust/tables/manager.py @@ -1,4 +1,5 @@ """Tables (changelog stream).""" + import asyncio import typing from typing import Any, MutableMapping, Optional, Set, Tuple, cast @@ -16,8 +17,7 @@ from faust.app import App as _App else: - class _App: - ... # noqa + class _App: ... # noqa __all__ = [ diff --git a/faust/tables/objects.py b/faust/tables/objects.py index 3fb7d92a2..a9c0de12d 100644 --- a/faust/tables/objects.py +++ b/faust/tables/objects.py @@ -3,6 +3,7 @@ This is also used to store data structures such as sets/lists. """ + import abc from typing import ( Any, diff --git a/faust/tables/recovery.py b/faust/tables/recovery.py index 51ec3cb9c..41bb20c18 100644 --- a/faust/tables/recovery.py +++ b/faust/tables/recovery.py @@ -1,4 +1,5 @@ """Table recovery after rebalancing.""" + import asyncio import statistics import typing @@ -39,11 +40,9 @@ from .manager import TableManager as _TableManager else: - class _App: - ... # noqa + class _App: ... # noqa - class _TableManager: - ... # noqa + class _TableManager: ... # noqa E_PERSISTED_OFFSET = """\ diff --git a/faust/tables/sets.py b/faust/tables/sets.py index 8da417a62..7876b5da8 100644 --- a/faust/tables/sets.py +++ b/faust/tables/sets.py @@ -1,4 +1,5 @@ """Storing sets in tables.""" + from enum import Enum from typing import ( Any, diff --git a/faust/tables/table.py b/faust/tables/table.py index a38d32859..a9d2f48f5 100644 --- a/faust/tables/table.py +++ b/faust/tables/table.py @@ -1,4 +1,5 @@ """Table (key/value changelog stream).""" + from typing import Any, ClassVar, Optional, Type from mode import Seconds diff --git a/faust/tables/wrappers.py b/faust/tables/wrappers.py index 5d400d3bd..0650c59e6 100644 --- a/faust/tables/wrappers.py +++ b/faust/tables/wrappers.py @@ -1,4 +1,5 @@ """Wrappers for windowed tables.""" + import operator import typing from datetime import datetime @@ -40,8 +41,7 @@ from .table import Table as _Table else: - class _Table: - ... # noqa + class _Table: ... # noqa __all__ = [ diff --git a/faust/topics.py b/faust/topics.py index 90b5e1bd6..b09a8d22d 100644 --- a/faust/topics.py +++ b/faust/topics.py @@ -1,4 +1,5 @@ """Topic - Named channel using Kafka.""" + import asyncio import re import typing @@ -46,8 +47,7 @@ from .app import App as _App else: - class _App: - ... # noqa + class _App: ... # noqa __all__ = ["Topic"] diff --git a/faust/transport/__init__.py b/faust/transport/__init__.py index 7346eb0cd..2d862c19c 100644 --- a/faust/transport/__init__.py +++ b/faust/transport/__init__.py @@ -1,4 +1,5 @@ """Transports.""" + from .drivers import by_name, by_url __all__ = ["by_name", "by_url"] diff --git a/faust/transport/base.py b/faust/transport/base.py index ac70afb3e..b3e478379 100644 --- a/faust/transport/base.py +++ b/faust/transport/base.py @@ -8,6 +8,7 @@ To see a reference transport implementation go to: :file:`faust/transport/drivers/aiokafka.py` """ + import asyncio from typing import Any, ClassVar, List, Optional, Type diff --git a/faust/transport/conductor.py b/faust/transport/conductor.py index ff7d3d99f..0bda03b02 100644 --- a/faust/transport/conductor.py +++ b/faust/transport/conductor.py @@ -1,4 +1,5 @@ """The conductor delegates messages from the consumer to the streams.""" + import asyncio import os import typing @@ -30,8 +31,7 @@ from faust.topics import Topic as _Topic else: - class _Topic: - ... # noqa + class _Topic: ... # noqa NO_CYTHON = bool(os.environ.get("NO_CYTHON", False)) diff --git a/faust/transport/consumer.py b/faust/transport/consumer.py index 0feba69c0..d5a82b1b8 100644 --- a/faust/transport/consumer.py +++ b/faust/transport/consumer.py @@ -43,6 +43,7 @@ _new_offset). """ + import abc import asyncio import gc @@ -101,8 +102,7 @@ from faust.app import App as _App else: - class _App: - ... # noqa: E701 + class _App: ... # noqa: E701 __all__ = ["Consumer", "Fetcher"] @@ -580,8 +580,7 @@ async def seek(self, partition: TP, offset: int) -> None: self._read_offset[ensure_TP(partition)] = offset if offset else None @abc.abstractmethod - async def _seek(self, partition: TP, offset: int) -> None: - ... + async def _seek(self, partition: TP, offset: int) -> None: ... def stop_flow(self) -> None: """Block consumer from processing any more messages.""" @@ -676,8 +675,7 @@ async def on_partitions_assigned( @abc.abstractmethod async def _getmany( self, active_partitions: Optional[Set[TP]], timeout: float - ) -> RecordMap: - ... + ) -> RecordMap: ... async def getmany(self, timeout: float) -> AsyncIterator[Tuple[TP, Message]]: """Fetch batch of messages from server.""" @@ -797,8 +795,7 @@ async def _wait_next_records( self.not_waiting_next_records.set() @abc.abstractmethod - def _to_message(self, tp: TP, record: Any) -> ConsumerMessage: - ... + def _to_message(self, tp: TP, record: Any) -> ConsumerMessage: ... def track_message(self, message: Message) -> None: """Track message and mark it as pending ack.""" @@ -908,11 +905,9 @@ async def verify_all_partitions_active(self) -> None: if not self.should_stop: self.verify_event_path(now, tp) - def verify_event_path(self, now: float, tp: TP) -> None: - ... + def verify_event_path(self, now: float, tp: TP) -> None: ... - def verify_recovery_event_path(self, now: float, tp: TP) -> None: - ... + def verify_recovery_event_path(self, now: float, tp: TP) -> None: ... async def commit( self, topics: TPorTopicSet = None, start_new_transaction: bool = True @@ -1302,8 +1297,7 @@ def topic_partitions(self, topic: str) -> Optional[int]: ... @abc.abstractmethod - def close(self) -> None: - ... + def close(self) -> None: ... @abc.abstractmethod async def earliest_offsets(self, *partitions: TP) -> Mapping[TP, int]: @@ -1358,8 +1352,7 @@ def key_partition( """Hash key to determine partition number.""" ... - def verify_recovery_event_path(self, now: float, tp: TP) -> None: - ... + def verify_recovery_event_path(self, now: float, tp: TP) -> None: ... class ThreadDelegateConsumer(Consumer): @@ -1381,8 +1374,7 @@ def __init__(self, *args: Any, **kwargs: Any) -> None: self.add_dependency(self._thread) @abc.abstractmethod - def _new_consumer_thread(self) -> ConsumerThread: - ... + def _new_consumer_thread(self) -> ConsumerThread: ... async def threadsafe_partitions_revoked( self, receiver_loop: asyncio.AbstractEventLoop, revoked: Set[TP] diff --git a/faust/transport/drivers/__init__.py b/faust/transport/drivers/__init__.py index 203a8907c..8deec0196 100644 --- a/faust/transport/drivers/__init__.py +++ b/faust/transport/drivers/__init__.py @@ -1,4 +1,5 @@ """Transport registry.""" + from yarl import URL from .aiokafka import Transport as AIOKafkaTransport diff --git a/faust/transport/drivers/aiokafka.py b/faust/transport/drivers/aiokafka.py index 2bfc84078..f69c532db 100644 --- a/faust/transport/drivers/aiokafka.py +++ b/faust/transport/drivers/aiokafka.py @@ -1,4 +1,5 @@ """Message transport using :pypi:`aiokafka`.""" + import asyncio import typing from asyncio import Lock, QueueEmpty diff --git a/faust/transport/producer.py b/faust/transport/producer.py index e36803a52..d70eb0aa6 100644 --- a/faust/transport/producer.py +++ b/faust/transport/producer.py @@ -6,6 +6,7 @@ - ... and the app via ``self.transport.app``. - Sending messages. """ + import asyncio import time from typing import Any, Awaitable, Mapping, Optional, cast diff --git a/faust/transport/utils.py b/faust/transport/utils.py index d09bf9c24..0b0c7b213 100644 --- a/faust/transport/utils.py +++ b/faust/transport/utils.py @@ -1,4 +1,5 @@ """Transport utils - scheduling.""" + from typing import ( Any, Dict, diff --git a/faust/types/__init__.py b/faust/types/__init__.py index 3727f36d6..d705b1aa2 100644 --- a/faust/types/__init__.py +++ b/faust/types/__init__.py @@ -1,4 +1,5 @@ """Abstract types for static typing.""" + from mode import ServiceT from .agents import AgentManagerT, AgentT diff --git a/faust/types/_env.py b/faust/types/_env.py index edabffd85..0f9e10dfc 100644 --- a/faust/types/_env.py +++ b/faust/types/_env.py @@ -1,4 +1,5 @@ """Faust environment variables.""" + import os from typing import Any, Sequence diff --git a/faust/types/agents.py b/faust/types/agents.py index 4c533c9e7..70491c20f 100644 --- a/faust/types/agents.py +++ b/faust/types/agents.py @@ -38,8 +38,7 @@ from .app import AppT as _AppT else: - class _AppT: - ... # noqa + class _AppT: ... # noqa __all__ = [ @@ -89,24 +88,19 @@ def __init__( it: _T, active_partitions: Optional[Set[TP]] = None, **kwargs: Any, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - def cancel(self) -> None: - ... + def cancel(self) -> None: ... @abc.abstractmethod - async def on_isolated_partition_revoked(self, tp: TP) -> None: - ... + async def on_isolated_partition_revoked(self, tp: TP) -> None: ... @abc.abstractmethod - async def on_isolated_partition_assigned(self, tp: TP) -> None: - ... + async def on_isolated_partition_assigned(self, tp: TP) -> None: ... @abc.abstractmethod - def traceback(self) -> str: - ... + def traceback(self) -> str: ... class AsyncIterableActorT(ActorT[AsyncIterable], AsyncIterable): @@ -150,8 +144,7 @@ def __init__( self.fun: AgentFun = fun @abc.abstractmethod - def actor_tracebacks(self) -> List[str]: - ... + def actor_tracebacks(self) -> List[str]: ... @abc.abstractmethod def __call__( @@ -161,8 +154,7 @@ def __call__( active_partitions: Optional[Set[TP]] = None, stream: Optional[StreamT] = None, channel: Optional[ChannelT] = None, - ) -> ActorRefT: - ... + ) -> ActorRefT: ... @abc.abstractmethod def test_context( @@ -170,24 +162,19 @@ def test_context( channel: Optional[ChannelT] = None, supervisor_strategy: SupervisorStrategyT = None, **kwargs: Any, - ) -> "AgentTestWrapperT": - ... + ) -> "AgentTestWrapperT": ... @abc.abstractmethod - def add_sink(self, sink: SinkT) -> None: - ... + def add_sink(self, sink: SinkT) -> None: ... @abc.abstractmethod - def stream(self, **kwargs: Any) -> StreamT: - ... + def stream(self, **kwargs: Any) -> StreamT: ... @abc.abstractmethod - async def on_partitions_assigned(self, assigned: Set[TP]) -> None: - ... + async def on_partitions_assigned(self, assigned: Set[TP]) -> None: ... @abc.abstractmethod - async def on_partitions_revoked(self, revoked: Set[TP]) -> None: - ... + async def on_partitions_revoked(self, revoked: Set[TP]) -> None: ... @abc.abstractmethod async def cast( @@ -198,8 +185,7 @@ async def cast( partition: Optional[int] = None, timestamp: Optional[float] = None, headers: HeadersArg = None, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod async def ask( @@ -212,8 +198,7 @@ async def ask( headers: HeadersArg = None, reply_to: ReplyToArg = None, correlation_id: Optional[str] = None, - ) -> Any: - ... + ) -> Any: ... @abc.abstractmethod async def send( @@ -228,8 +213,7 @@ async def send( value_serializer: CodecArg = None, reply_to: ReplyToArg = None, correlation_id: Optional[str] = None, - ) -> Awaitable[RecordMetadata]: - ... + ) -> Awaitable[RecordMetadata]: ... @abc.abstractmethod @no_type_check # XXX mypy bugs out on this @@ -238,8 +222,7 @@ async def map( values: Union[AsyncIterable, Iterable], key: K = None, reply_to: ReplyToArg = None, - ) -> AsyncIterator: - ... + ) -> AsyncIterator: ... @abc.abstractmethod @no_type_check # XXX mypy bugs out on this @@ -247,8 +230,7 @@ async def kvmap( self, items: Union[AsyncIterable[Tuple[K, V]], Iterable[Tuple[K, V]]], reply_to: ReplyToArg = None, - ) -> AsyncIterator[str]: - ... + ) -> AsyncIterator[str]: ... @abc.abstractmethod async def join( @@ -256,70 +238,56 @@ async def join( values: Union[AsyncIterable[V], Iterable[V]], key: K = None, reply_to: ReplyToArg = None, - ) -> List[Any]: - ... + ) -> List[Any]: ... @abc.abstractmethod async def kvjoin( self, items: Union[AsyncIterable[Tuple[K, V]], Iterable[Tuple[K, V]]], reply_to: ReplyToArg = None, - ) -> List[Any]: - ... + ) -> List[Any]: ... @abc.abstractmethod - def info(self) -> Mapping: - ... + def info(self) -> Mapping: ... @abc.abstractmethod - def clone(self, *, cls: Type["AgentT"] = None, **kwargs: Any) -> "AgentT": - ... + def clone(self, *, cls: Type["AgentT"] = None, **kwargs: Any) -> "AgentT": ... @abc.abstractmethod - def get_topic_names(self) -> Iterable[str]: - ... + def get_topic_names(self) -> Iterable[str]: ... @property @abc.abstractmethod - def channel(self) -> ChannelT: - ... + def channel(self) -> ChannelT: ... @channel.setter - def channel(self, channel: ChannelT) -> None: - ... + def channel(self, channel: ChannelT) -> None: ... @property @abc.abstractmethod - def channel_iterator(self) -> AsyncIterator: - ... + def channel_iterator(self) -> AsyncIterator: ... @channel_iterator.setter - def channel_iterator(self, channel: AsyncIterator) -> None: - ... + def channel_iterator(self, channel: AsyncIterator) -> None: ... @abc.abstractmethod - def _agent_label(self, name_suffix: str = "") -> str: - ... + def _agent_label(self, name_suffix: str = "") -> str: ... class AgentManagerT(ServiceT, ManagedUserDict[str, AgentT]): app: _AppT @abc.abstractmethod - async def wait_until_agents_started(self) -> None: - ... + async def wait_until_agents_started(self) -> None: ... @abc.abstractmethod - async def on_rebalance(self, revoked: Set[TP], newly_assigned: Set[TP]) -> None: - ... + async def on_rebalance(self, revoked: Set[TP], newly_assigned: Set[TP]) -> None: ... @abc.abstractmethod - def actor_tracebacks(self) -> Mapping[str, List[str]]: - ... + def actor_tracebacks(self) -> Mapping[str, List[str]]: ... @abc.abstractmethod - def human_tracebacks(self) -> str: - ... + def human_tracebacks(self) -> str: ... class AgentTestWrapperT(AgentT, AsyncIterable): @@ -332,8 +300,7 @@ class AgentTestWrapperT(AgentT, AsyncIterable): @abc.abstractmethod def __init__( self, *args: Any, original_channel: Optional[ChannelT] = None, **kwargs: Any - ) -> None: - ... + ) -> None: ... @abc.abstractmethod async def put( @@ -349,8 +316,7 @@ async def put( reply_to: ReplyToArg = None, correlation_id: Optional[str] = None, wait: bool = True, - ) -> EventT: - ... + ) -> EventT: ... @abc.abstractmethod def to_message( @@ -363,9 +329,7 @@ def to_message( timestamp: Optional[float] = None, timestamp_type: int = 0, headers: HeadersArg = None, - ) -> Message: - ... + ) -> Message: ... @abc.abstractmethod - async def throw(self, exc: BaseException) -> None: - ... + async def throw(self, exc: BaseException) -> None: ... diff --git a/faust/types/app.py b/faust/types/app.py index da9fdf407..c62858dcb 100644 --- a/faust/types/app.py +++ b/faust/types/app.py @@ -66,29 +66,21 @@ from .settings import Settings as _Settings else: - class _AppCommand: - ... # noqa + class _AppCommand: ... # noqa - class _SchemaT: - ... # noqa + class _SchemaT: ... # noqa - class _LiveCheck: - ... # noqa + class _LiveCheck: ... # noqa - class _Monitor: - ... # noqa + class _Monitor: ... # noqa - class _Worker: - ... # noqa + class _Worker: ... # noqa - class _EventT: - ... # noqa + class _EventT: ... # noqa - class _ModelArg: - ... # noqa + class _ModelArg: ... # noqa - class _Settings: - ... # noqa + class _Settings: ... # noqa __all__ = [ @@ -103,18 +95,15 @@ class _Settings: class TracerT(abc.ABC): @property @abc.abstractmethod - def default_tracer(self) -> opentracing.Tracer: - ... + def default_tracer(self) -> opentracing.Tracer: ... @abc.abstractmethod def trace( self, name: str, sample_rate: Optional[float] = None, **extra_context: Any - ) -> ContextManager: - ... + ) -> ContextManager: ... @abc.abstractmethod - def get_tracer(self, service_name: str) -> opentracing.Tracer: - ... + def get_tracer(self, service_name: str) -> opentracing.Tracer: ... class BootStrategyT: @@ -142,20 +131,16 @@ def __init__( enable_kafka_producer: Optional[bool] = None, enable_kafka_consumer: Optional[bool] = None, enable_sensors: bool = True, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - def server(self) -> Iterable[ServiceT]: - ... + def server(self) -> Iterable[ServiceT]: ... @abc.abstractmethod - def client_only(self) -> Iterable[ServiceT]: - ... + def client_only(self) -> Iterable[ServiceT]: ... @abc.abstractmethod - def producer_only(self) -> Iterable[ServiceT]: - ... + def producer_only(self) -> Iterable[ServiceT]: ... class AppT(ServiceT): @@ -224,24 +209,19 @@ def __init__( @abc.abstractmethod def config_from_object( self, obj: Any, *, silent: bool = False, force: bool = False - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - def finalize(self) -> None: - ... + def finalize(self) -> None: ... @abc.abstractmethod - def main(self) -> NoReturn: - ... + def main(self) -> NoReturn: ... @abc.abstractmethod - def worker_init(self) -> None: - ... + def worker_init(self) -> None: ... @abc.abstractmethod - def worker_init_post_autodiscover(self) -> None: - ... + def worker_init_post_autodiscover(self) -> None: ... @abc.abstractmethod def discover( @@ -249,8 +229,7 @@ def discover( *extra_modules: str, categories: Iterable[str] = ("a", "b", "c"), ignore: Iterable[Any] = ("foo", "bar"), - ) -> None: - ... + ) -> None: ... @abc.abstractmethod def topic( @@ -274,8 +253,7 @@ def topic( allow_empty: bool = False, has_prefix: bool = False, loop: Optional[asyncio.AbstractEventLoop] = None, - ) -> TopicT: - ... + ) -> TopicT: ... @abc.abstractmethod def channel( @@ -286,8 +264,7 @@ def channel( value_type: _ModelArg = None, maxsize: Optional[int] = None, loop: Optional[asyncio.AbstractEventLoop] = None, - ) -> ChannelT: - ... + ) -> ChannelT: ... @abc.abstractmethod def agent( @@ -301,15 +278,13 @@ def agent( isolated_partitions: bool = False, use_reply_headers: bool = True, **kwargs: Any, - ) -> Callable[[AgentFun[_T]], AgentT[_T]]: - ... + ) -> Callable[[AgentFun[_T]], AgentT[_T]]: ... @abc.abstractmethod @no_type_check def task( self, fun: TaskArg, *, on_leader: bool = False, traced: bool = True - ) -> Callable: - ... + ) -> Callable: ... @abc.abstractmethod def timer( @@ -319,8 +294,7 @@ def timer( traced: bool = True, name: Optional[str] = None, max_drift_correction: float = 0.1, - ) -> Callable: - ... + ) -> Callable: ... @abc.abstractmethod def crontab( @@ -330,18 +304,15 @@ def crontab( timezone: tzinfo = None, on_leader: bool = False, traced: bool = True, - ) -> Callable: - ... + ) -> Callable: ... @abc.abstractmethod - def service(self, cls: Type[ServiceT]) -> Type[ServiceT]: - ... + def service(self, cls: Type[ServiceT]) -> Type[ServiceT]: ... @abc.abstractmethod def stream( self, channel: AsyncIterable, beacon: Optional[NodeT] = None, **kwargs: Any - ) -> StreamT: - ... + ) -> StreamT: ... @abc.abstractmethod def Table( @@ -353,8 +324,7 @@ def Table( partitions: Optional[int] = None, help: Optional[str] = None, **kwargs: Any, - ) -> TableT: - ... + ) -> TableT: ... @abc.abstractmethod def GlobalTable( @@ -366,8 +336,7 @@ def GlobalTable( partitions: Optional[int] = None, help: Optional[str] = None, **kwargs: Any, - ) -> TableT: - ... + ) -> TableT: ... @abc.abstractmethod def SetTable( @@ -379,8 +348,7 @@ def SetTable( start_manager: bool = False, help: Optional[str] = None, **kwargs: Any, - ) -> TableT: - ... + ) -> TableT: ... @abc.abstractmethod def SetGlobalTable( @@ -392,8 +360,7 @@ def SetGlobalTable( start_manager: bool = False, help: Optional[str] = None, **kwargs: Any, - ) -> TableT: - ... + ) -> TableT: ... @abc.abstractmethod def page( @@ -403,8 +370,7 @@ def page( base: Type[View] = View, cors_options: Mapping[str, ResourceOptions] = None, name: Optional[str] = None, - ) -> Callable[[PageArg], Type[View]]: - ... + ) -> Callable[[PageArg], Type[View]]: ... @abc.abstractmethod def table_route( @@ -415,34 +381,28 @@ def table_route( query_param: Optional[str] = None, match_info: Optional[str] = None, exact_key: Optional[str] = None, - ) -> ViewDecorator: - ... + ) -> ViewDecorator: ... @abc.abstractmethod def command( self, *options: Any, base: Type[_AppCommand] = None, **kwargs: Any - ) -> Callable[[Callable], Type[_AppCommand]]: - ... + ) -> Callable[[Callable], Type[_AppCommand]]: ... @abc.abstractmethod def create_event( self, key: K, value: V, headers: HeadersArg, message: Message - ) -> _EventT: - ... + ) -> _EventT: ... @abc.abstractmethod - async def start_client(self) -> None: - ... + async def start_client(self) -> None: ... @abc.abstractmethod - async def maybe_start_client(self) -> None: - ... + async def maybe_start_client(self) -> None: ... @abc.abstractmethod def trace( self, name: str, trace_enabled: bool = True, **extra_context: Any - ) -> ContextManager: - ... + ) -> ContextManager: ... @abc.abstractmethod async def send( @@ -457,21 +417,17 @@ async def send( key_serializer: CodecArg = None, value_serializer: CodecArg = None, callback: Optional[MessageSentCallback] = None, - ) -> Awaitable[RecordMetadata]: - ... + ) -> Awaitable[RecordMetadata]: ... @abc.abstractmethod - def LiveCheck(self, **kwargs: Any) -> _LiveCheck: - ... + def LiveCheck(self, **kwargs: Any) -> _LiveCheck: ... @stampede @abc.abstractmethod - async def maybe_start_producer(self) -> ProducerT: - ... + async def maybe_start_producer(self) -> ProducerT: ... @abc.abstractmethod - def is_leader(self) -> bool: - ... + def is_leader(self) -> bool: ... @abc.abstractmethod def FlowControlQueue( @@ -479,92 +435,72 @@ def FlowControlQueue( maxsize: Optional[int] = None, *, clear_on_resume: bool = False, - ) -> ThrowableQueue: - ... + ) -> ThrowableQueue: ... @abc.abstractmethod - def Worker(self, **kwargs: Any) -> _Worker: - ... + def Worker(self, **kwargs: Any) -> _Worker: ... @abc.abstractmethod - def on_webserver_init(self, web: Web) -> None: - ... + def on_webserver_init(self, web: Web) -> None: ... @abc.abstractmethod - def on_rebalance_start(self) -> None: - ... + def on_rebalance_start(self) -> None: ... @abc.abstractmethod - def on_rebalance_return(self) -> None: - ... + def on_rebalance_return(self) -> None: ... @abc.abstractmethod - def on_rebalance_end(self) -> None: - ... + def on_rebalance_end(self) -> None: ... @property - def conf(self) -> _Settings: - ... + def conf(self) -> _Settings: ... @conf.setter - def conf(self, settings: _Settings) -> None: - ... + def conf(self, settings: _Settings) -> None: ... @property @abc.abstractmethod - def transport(self) -> TransportT: - ... + def transport(self) -> TransportT: ... @transport.setter - def transport(self, transport: TransportT) -> None: - ... + def transport(self, transport: TransportT) -> None: ... @property @abc.abstractmethod - def producer_transport(self) -> TransportT: - ... + def producer_transport(self) -> TransportT: ... @producer_transport.setter - def producer_transport(self, transport: TransportT) -> None: - ... + def producer_transport(self, transport: TransportT) -> None: ... @property @abc.abstractmethod - def cache(self) -> CacheBackendT: - ... + def cache(self) -> CacheBackendT: ... @cache.setter - def cache(self, cache: CacheBackendT) -> None: - ... + def cache(self, cache: CacheBackendT) -> None: ... @property @abc.abstractmethod - def producer(self) -> ProducerT: - ... + def producer(self) -> ProducerT: ... @property @abc.abstractmethod - def consumer(self) -> ConsumerT: - ... + def consumer(self) -> ConsumerT: ... @cached_property @abc.abstractmethod - def tables(self) -> TableManagerT: - ... + def tables(self) -> TableManagerT: ... @cached_property @abc.abstractmethod - def topics(self) -> ConductorT: - ... + def topics(self) -> ConductorT: ... @property @abc.abstractmethod - def monitor(self) -> _Monitor: - ... + def monitor(self) -> _Monitor: ... @monitor.setter - def monitor(self, value: _Monitor) -> None: - ... + def monitor(self, value: _Monitor) -> None: ... @cached_property @abc.abstractmethod @@ -573,42 +509,33 @@ def flow_control(self) -> FlowControlEvent: @property @abc.abstractmethod - def http_client(self) -> HttpClientT: - ... + def http_client(self) -> HttpClientT: ... @http_client.setter - def http_client(self, client: HttpClientT) -> None: - ... + def http_client(self, client: HttpClientT) -> None: ... @cached_property @abc.abstractmethod - def assignor(self) -> PartitionAssignorT: - ... + def assignor(self) -> PartitionAssignorT: ... @cached_property @abc.abstractmethod - def router(self) -> RouterT: - ... + def router(self) -> RouterT: ... @cached_property @abc.abstractmethod - def serializers(self) -> RegistryT: - ... + def serializers(self) -> RegistryT: ... @cached_property @abc.abstractmethod - def web(self) -> Web: - ... + def web(self) -> Web: ... @cached_property @abc.abstractmethod - def in_transaction(self) -> bool: - ... + def in_transaction(self) -> bool: ... @abc.abstractmethod - def _span_add_default_tags(self, span: opentracing.Span) -> None: - ... + def _span_add_default_tags(self, span: opentracing.Span) -> None: ... @abc.abstractmethod - def _start_span_from_rebalancing(self, name: str) -> opentracing.Span: - ... + def _start_span_from_rebalancing(self, name: str) -> opentracing.Span: ... diff --git a/faust/types/assignor.py b/faust/types/assignor.py index d00ffd412..b32a09da6 100644 --- a/faust/types/assignor.py +++ b/faust/types/assignor.py @@ -11,8 +11,7 @@ from .app import AppT as _AppT else: - class _AppT: - ... # noqa + class _AppT: ... # noqa __all__ = [ @@ -31,49 +30,38 @@ class PartitionAssignorT(abc.ABC): app: _AppT @abc.abstractmethod - def __init__(self, app: _AppT, replicas: int = 0) -> None: - ... + def __init__(self, app: _AppT, replicas: int = 0) -> None: ... @abc.abstractmethod - def group_for_topic(self, topic: str) -> int: - ... + def group_for_topic(self, topic: str) -> int: ... @abc.abstractmethod - def assigned_standbys(self) -> Set[TP]: - ... + def assigned_standbys(self) -> Set[TP]: ... @abc.abstractmethod - def assigned_actives(self) -> Set[TP]: - ... + def assigned_actives(self) -> Set[TP]: ... @abc.abstractmethod - def is_active(self, tp: TP) -> bool: - ... + def is_active(self, tp: TP) -> bool: ... @abc.abstractmethod - def is_standby(self, tp: TP) -> bool: - ... + def is_standby(self, tp: TP) -> bool: ... @abc.abstractmethod - def key_store(self, topic: str, key: bytes) -> URL: - ... + def key_store(self, topic: str, key: bytes) -> URL: ... @abc.abstractmethod - def table_metadata(self, topic: str) -> HostToPartitionMap: - ... + def table_metadata(self, topic: str) -> HostToPartitionMap: ... @abc.abstractmethod - def tables_metadata(self) -> HostToPartitionMap: - ... + def tables_metadata(self) -> HostToPartitionMap: ... class LeaderAssignorT(ServiceT): app: _AppT @abc.abstractmethod - def __init__(self, app: _AppT, **kwargs: Any) -> None: - ... + def __init__(self, app: _AppT, **kwargs: Any) -> None: ... @abc.abstractmethod - def is_leader(self) -> bool: - ... + def is_leader(self) -> bool: ... diff --git a/faust/types/channels.py b/faust/types/channels.py index 553d4f394..b717c1a68 100644 --- a/faust/types/channels.py +++ b/faust/types/channels.py @@ -22,20 +22,15 @@ from .streams import StreamT as _StreamT else: - class _AppT: - ... # noqa + class _AppT: ... # noqa - class _EventT(Generic[_T]): - ... # noqa + class _EventT(Generic[_T]): ... # noqa - class _ModelArg: - ... # noqa + class _ModelArg: ... # noqa - class _SchemaT: - ... # noqa + class _SchemaT: ... # noqa - class _StreamT: - ... # noqa + class _StreamT: ... # noqa class ChannelT(AsyncIterator[_EventT[_T]]): @@ -61,26 +56,21 @@ def __init__( root: "ChannelT" = None, active_partitions: Optional[Set[TP]] = None, loop: Optional[asyncio.AbstractEventLoop] = None, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod def clone( self, *, is_iterator: Optional[bool] = None, **kwargs: Any - ) -> "ChannelT[_T]": - ... + ) -> "ChannelT[_T]": ... @abc.abstractmethod - def clone_using_queue(self, queue: asyncio.Queue) -> "ChannelT[_T]": - ... + def clone_using_queue(self, queue: asyncio.Queue) -> "ChannelT[_T]": ... @abc.abstractmethod - def stream(self, **kwargs: Any) -> "_StreamT[_T]": - ... + def stream(self, **kwargs: Any) -> "_StreamT[_T]": ... @abc.abstractmethod - def get_topic_name(self) -> str: - ... + def get_topic_name(self) -> str: ... @abc.abstractmethod async def send( @@ -96,8 +86,7 @@ async def send( value_serializer: CodecArg = None, callback: Optional[MessageSentCallback] = None, force: bool = False, - ) -> Awaitable[RecordMetadata]: - ... + ) -> Awaitable[RecordMetadata]: ... @abc.abstractmethod def send_soon( @@ -114,8 +103,7 @@ def send_soon( callback: Optional[MessageSentCallback] = None, force: bool = False, eager_partitioning: bool = False, - ) -> FutureMessage: - ... + ) -> FutureMessage: ... @abc.abstractmethod def as_future_message( @@ -130,98 +118,78 @@ def as_future_message( value_serializer: CodecArg = None, callback: Optional[MessageSentCallback] = None, eager_partitioning: bool = False, - ) -> FutureMessage: - ... + ) -> FutureMessage: ... @abc.abstractmethod async def publish_message( self, fut: FutureMessage, wait: bool = True - ) -> Awaitable[RecordMetadata]: - ... + ) -> Awaitable[RecordMetadata]: ... @stampede @abc.abstractmethod - async def maybe_declare(self) -> None: - ... + async def maybe_declare(self) -> None: ... @abc.abstractmethod - async def declare(self) -> None: - ... + async def declare(self) -> None: ... @abc.abstractmethod def prepare_key( self, key: K, key_serializer: CodecArg, schema: Optional[_SchemaT] = None - ) -> Any: - ... + ) -> Any: ... @abc.abstractmethod def prepare_value( self, value: V, value_serializer: CodecArg, schema: Optional[_SchemaT] = None - ) -> Any: - ... + ) -> Any: ... @abc.abstractmethod - async def decode(self, message: Message, *, propagate: bool = False) -> _EventT[_T]: - ... + async def decode( + self, message: Message, *, propagate: bool = False + ) -> _EventT[_T]: ... @abc.abstractmethod - async def deliver(self, message: Message) -> None: - ... + async def deliver(self, message: Message) -> None: ... @abc.abstractmethod - async def put(self, value: _EventT[_T]) -> None: - ... + async def put(self, value: _EventT[_T]) -> None: ... @abc.abstractmethod - async def get(self, *, timeout: Optional[Seconds] = None) -> _EventT[_T]: - ... + async def get(self, *, timeout: Optional[Seconds] = None) -> _EventT[_T]: ... @abc.abstractmethod - def empty(self) -> bool: - ... + def empty(self) -> bool: ... @abc.abstractmethod - async def on_key_decode_error(self, exc: Exception, message: Message) -> None: - ... + async def on_key_decode_error(self, exc: Exception, message: Message) -> None: ... @abc.abstractmethod - async def on_value_decode_error(self, exc: Exception, message: Message) -> None: - ... + async def on_value_decode_error(self, exc: Exception, message: Message) -> None: ... @abc.abstractmethod - async def on_decode_error(self, exc: Exception, message: Message) -> None: - ... + async def on_decode_error(self, exc: Exception, message: Message) -> None: ... @abc.abstractmethod - def on_stop_iteration(self) -> None: - ... + def on_stop_iteration(self) -> None: ... @abc.abstractmethod - def __aiter__(self) -> "ChannelT": - ... + def __aiter__(self) -> "ChannelT": ... @abc.abstractmethod - def __anext__(self) -> Awaitable[_EventT[_T]]: - ... + def __anext__(self) -> Awaitable[_EventT[_T]]: ... @abc.abstractmethod - async def throw(self, exc: BaseException) -> None: - ... + async def throw(self, exc: BaseException) -> None: ... @abc.abstractmethod - def _throw(self, exc: BaseException) -> None: - ... + def _throw(self, exc: BaseException) -> None: ... @abc.abstractmethod - def derive(self, **kwargs: Any) -> "ChannelT": - ... + def derive(self, **kwargs: Any) -> "ChannelT": ... @property @abc.abstractmethod - def subscriber_count(self) -> int: - ... + def subscriber_count(self) -> int: ... @property @abc.abstractmethod - def queue(self) -> ThrowableQueue: - ... + def queue(self) -> ThrowableQueue: ... diff --git a/faust/types/codecs.py b/faust/types/codecs.py index e7f1bfb8f..32efe9cf2 100644 --- a/faust/types/codecs.py +++ b/faust/types/codecs.py @@ -12,24 +12,21 @@ class CodecT(metaclass=abc.ABCMeta): """ @abc.abstractmethod - def __init__(self, children: Tuple["CodecT", ...] = None, **kwargs: Any) -> None: - ... + def __init__( + self, children: Tuple["CodecT", ...] = None, **kwargs: Any + ) -> None: ... @abc.abstractmethod - def dumps(self, obj: Any) -> bytes: - ... + def dumps(self, obj: Any) -> bytes: ... @abc.abstractmethod - def loads(self, s: bytes) -> Any: - ... + def loads(self, s: bytes) -> Any: ... @abc.abstractmethod - def clone(self, *children: "CodecT") -> "CodecT": - ... + def clone(self, *children: "CodecT") -> "CodecT": ... @abc.abstractmethod - def __or__(self, other: Any) -> Any: - ... + def __or__(self, other: Any) -> Any: ... # `serializer` argument can be str or Codec instance. diff --git a/faust/types/core.py b/faust/types/core.py index ae3ea2103..3a79e7369 100644 --- a/faust/types/core.py +++ b/faust/types/core.py @@ -18,8 +18,7 @@ from .models import ModelT as _ModelT else: - class _ModelT: - ... # noqa + class _ModelT: ... # noqa __all__ = ["K", "V"] diff --git a/faust/types/events.py b/faust/types/events.py index 96f566ba7..11061e07b 100644 --- a/faust/types/events.py +++ b/faust/types/events.py @@ -14,14 +14,11 @@ from .serializers import SchemaT as _SchemaT else: - class _AppT: - ... # noqa + class _AppT: ... # noqa - class _ChannelT: - ... # noqa + class _ChannelT: ... # noqa - class _SchemaT: - ... # noqa + class _SchemaT: ... # noqa T = TypeVar("T") @@ -45,8 +42,7 @@ def __init__( value: V, headers: Optional[HeadersArg], message: Message, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod async def send( @@ -62,8 +58,7 @@ async def send( value_serializer: CodecArg = None, callback: Optional[MessageSentCallback] = None, force: bool = False, - ) -> Awaitable[RecordMetadata]: - ... + ) -> Awaitable[RecordMetadata]: ... @abc.abstractmethod async def forward( @@ -79,9 +74,7 @@ async def forward( value_serializer: CodecArg = None, callback: Optional[MessageSentCallback] = None, force: bool = False, - ) -> Awaitable[RecordMetadata]: - ... + ) -> Awaitable[RecordMetadata]: ... @abc.abstractmethod - def ack(self) -> bool: - ... + def ack(self) -> bool: ... diff --git a/faust/types/fixups.py b/faust/types/fixups.py index 1cbb5e3a2..fabff849d 100644 --- a/faust/types/fixups.py +++ b/faust/types/fixups.py @@ -6,8 +6,7 @@ from .app import AppT as _AppT else: - class _AppT: - ... # noqa + class _AppT: ... # noqa __all__ = ["FixupT"] @@ -17,17 +16,13 @@ class FixupT(abc.ABC): app: _AppT @abc.abstractmethod - def __init__(self, app: _AppT) -> None: - ... + def __init__(self, app: _AppT) -> None: ... @abc.abstractmethod - def enabled(self) -> bool: - ... + def enabled(self) -> bool: ... @abc.abstractmethod - def autodiscover_modules(self) -> Iterable[str]: - ... + def autodiscover_modules(self) -> Iterable[str]: ... @abc.abstractmethod - def on_worker_init(self) -> None: - ... + def on_worker_init(self) -> None: ... diff --git a/faust/types/joins.py b/faust/types/joins.py index aacd3c6ec..1aced9c47 100644 --- a/faust/types/joins.py +++ b/faust/types/joins.py @@ -15,9 +15,7 @@ class JoinT(abc.ABC): @abc.abstractmethod def __init__( self, *, stream: JoinableT, fields: Tuple[FieldDescriptorT, ...] - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - async def process(self, event: EventT) -> Optional[EventT]: - ... + async def process(self, event: EventT) -> Optional[EventT]: ... diff --git a/faust/types/models.py b/faust/types/models.py index b0a432270..f0710e42a 100644 --- a/faust/types/models.py +++ b/faust/types/models.py @@ -145,8 +145,9 @@ class ModelT(base): # type: ignore @classmethod @abc.abstractmethod - def from_data(cls, data: Any, *, preferred_type: Type["ModelT"] = None) -> "ModelT": - ... + def from_data( + cls, data: Any, *, preferred_type: Type["ModelT"] = None + ) -> "ModelT": ... @classmethod @abc.abstractmethod @@ -156,41 +157,32 @@ def loads( *, default_serializer: CodecArg = None, # XXX use serializer serializer: CodecArg = None, - ) -> "ModelT": - ... + ) -> "ModelT": ... @abc.abstractmethod - def __init__(self, *args: Any, **kwargs: Any) -> None: - ... + def __init__(self, *args: Any, **kwargs: Any) -> None: ... @abc.abstractmethod - def dumps(self, *, serializer: CodecArg = None) -> bytes: - ... + def dumps(self, *, serializer: CodecArg = None) -> bytes: ... @abc.abstractmethod - def derive(self, *objects: "ModelT", **fields: Any) -> "ModelT": - ... + def derive(self, *objects: "ModelT", **fields: Any) -> "ModelT": ... @abc.abstractmethod - def to_representation(self) -> Any: - ... + def to_representation(self) -> Any: ... @abc.abstractmethod - def is_valid(self) -> bool: - ... + def is_valid(self) -> bool: ... @abc.abstractmethod - def validate(self) -> List[ValidationError]: - ... + def validate(self) -> List[ValidationError]: ... @abc.abstractmethod - def validate_or_raise(self) -> None: - ... + def validate_or_raise(self) -> None: ... @property @abc.abstractmethod - def validation_errors(self) -> List[ValidationError]: - ... + def validation_errors(self) -> List[ValidationError]: ... class FieldDescriptorT(Generic[T]): @@ -227,59 +219,46 @@ def __init__( ) @abc.abstractmethod - def on_model_attached(self) -> None: - ... + def on_model_attached(self) -> None: ... @abc.abstractmethod - def clone(self, **kwargs: Any) -> "FieldDescriptorT": - ... + def clone(self, **kwargs: Any) -> "FieldDescriptorT": ... @abc.abstractmethod - def as_dict(self) -> Mapping[str, Any]: - ... + def as_dict(self) -> Mapping[str, Any]: ... @abc.abstractmethod - def validate_all(self, value: Any) -> Iterable[ValidationError]: - ... + def validate_all(self, value: Any) -> Iterable[ValidationError]: ... @abc.abstractmethod - def validate(self, value: T) -> Iterable[ValidationError]: - ... + def validate(self, value: T) -> Iterable[ValidationError]: ... @abc.abstractmethod - def to_python(self, value: Any) -> Optional[T]: - ... + def to_python(self, value: Any) -> Optional[T]: ... @abc.abstractmethod - def prepare_value(self, value: Any) -> Optional[T]: - ... + def prepare_value(self, value: Any) -> Optional[T]: ... @abc.abstractmethod - def should_coerce(self, value: Any) -> bool: - ... + def should_coerce(self, value: Any) -> bool: ... @abc.abstractmethod - def getattr(self, obj: ModelT) -> T: - ... + def getattr(self, obj: ModelT) -> T: ... @abc.abstractmethod - def validation_error(self, reason: str) -> ValidationError: - ... + def validation_error(self, reason: str) -> ValidationError: ... @property @abc.abstractmethod - def ident(self) -> str: - ... + def ident(self) -> str: ... @cached_property @abc.abstractmethod - def related_models(self) -> Set[Type[ModelT]]: - ... + def related_models(self) -> Set[Type[ModelT]]: ... @cached_property @abc.abstractmethod - def lazy_coercion(self) -> bool: - ... + def lazy_coercion(self) -> bool: ... # XXX See top of module! We redefine with actual ModelT for Sphinx, diff --git a/faust/types/router.py b/faust/types/router.py index 011027875..fad8b5bb0 100644 --- a/faust/types/router.py +++ b/faust/types/router.py @@ -1,4 +1,5 @@ """Types for module :mod:`faust.router`.""" + import abc import typing @@ -13,8 +14,7 @@ from .app import AppT as _AppT else: - class _AppT: - ... # noqa + class _AppT: ... # noqa class RouterT(abc.ABC): @@ -23,37 +23,29 @@ class RouterT(abc.ABC): app: _AppT @abc.abstractmethod - def __init__(self, app: _AppT) -> None: - ... + def __init__(self, app: _AppT) -> None: ... @abc.abstractmethod - def key_store(self, table_name: str, key: K) -> URL: - ... + def key_store(self, table_name: str, key: K) -> URL: ... @abc.abstractmethod - def external_topic_key_store(self, topic: TopicT, key: K) -> URL: - ... + def external_topic_key_store(self, topic: TopicT, key: K) -> URL: ... @abc.abstractmethod - def table_metadata(self, table_name: str) -> HostToPartitionMap: - ... + def table_metadata(self, table_name: str) -> HostToPartitionMap: ... @abc.abstractmethod - def tables_metadata(self) -> HostToPartitionMap: - ... + def tables_metadata(self) -> HostToPartitionMap: ... @abc.abstractmethod - def external_topics_metadata(self) -> HostToPartitionMap: - ... + def external_topics_metadata(self) -> HostToPartitionMap: ... @abc.abstractmethod async def route_req( self, table_name: str, key: K, web: web.Web, request: web.Request - ) -> web.Response: - ... + ) -> web.Response: ... @abc.abstractmethod async def route_topic_req( self, topic: TopicT, key: K, web: web.Web, request: web.Request - ) -> web.Response: - ... + ) -> web.Response: ... diff --git a/faust/types/sensors.py b/faust/types/sensors.py index 47a08f675..a55d39e52 100644 --- a/faust/types/sensors.py +++ b/faust/types/sensors.py @@ -16,8 +16,7 @@ from .app import AppT as _AppT else: - class _AppT: - ... # noqa + class _AppT: ... # noqa __all__ = ["SensorInterfaceT", "SensorT", "SensorDelegateT"] @@ -25,48 +24,38 @@ class _AppT: class SensorInterfaceT(abc.ABC): @abc.abstractmethod - def on_message_in(self, tp: TP, offset: int, message: Message) -> None: - ... + def on_message_in(self, tp: TP, offset: int, message: Message) -> None: ... @abc.abstractmethod def on_stream_event_in( self, tp: TP, offset: int, stream: StreamT, event: EventT - ) -> Optional[Dict]: - ... + ) -> Optional[Dict]: ... @abc.abstractmethod def on_stream_event_out( self, tp: TP, offset: int, stream: StreamT, event: EventT, state: Dict = None - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - def on_topic_buffer_full(self, tp: TP) -> None: - ... + def on_topic_buffer_full(self, tp: TP) -> None: ... @abc.abstractmethod - def on_message_out(self, tp: TP, offset: int, message: Message) -> None: - ... + def on_message_out(self, tp: TP, offset: int, message: Message) -> None: ... @abc.abstractmethod - def on_table_get(self, table: CollectionT, key: Any) -> None: - ... + def on_table_get(self, table: CollectionT, key: Any) -> None: ... @abc.abstractmethod - def on_table_set(self, table: CollectionT, key: Any, value: Any) -> None: - ... + def on_table_set(self, table: CollectionT, key: Any, value: Any) -> None: ... @abc.abstractmethod - def on_table_del(self, table: CollectionT, key: Any) -> None: - ... + def on_table_del(self, table: CollectionT, key: Any) -> None: ... @abc.abstractmethod - def on_commit_initiated(self, consumer: ConsumerT) -> Any: - ... + def on_commit_initiated(self, consumer: ConsumerT) -> Any: ... @abc.abstractmethod - def on_commit_completed(self, consumer: ConsumerT, state: Any) -> None: - ... + def on_commit_completed(self, consumer: ConsumerT, state: Any) -> None: ... @abc.abstractmethod def on_send_initiated( @@ -76,54 +65,44 @@ def on_send_initiated( message: PendingMessage, keysize: int, valsize: int, - ) -> Any: - ... + ) -> Any: ... @abc.abstractmethod def on_send_completed( self, producer: ProducerT, state: Any, metadata: RecordMetadata - ) -> None: - ... + ) -> None: ... @abc.abstractmethod def on_send_error( self, producer: ProducerT, exc: BaseException, state: Any - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - def on_assignment_start(self, assignor: PartitionAssignorT) -> Dict: - ... + def on_assignment_start(self, assignor: PartitionAssignorT) -> Dict: ... @abc.abstractmethod def on_assignment_error( self, assignor: PartitionAssignorT, state: Dict, exc: BaseException - ) -> None: - ... + ) -> None: ... @abc.abstractmethod def on_assignment_completed( self, assignor: PartitionAssignorT, state: Dict - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - def on_rebalance_start(self, app: _AppT) -> Dict: - ... + def on_rebalance_start(self, app: _AppT) -> Dict: ... @abc.abstractmethod - def on_rebalance_return(self, app: _AppT, state: Dict) -> None: - ... + def on_rebalance_return(self, app: _AppT, state: Dict) -> None: ... @abc.abstractmethod - def on_rebalance_end(self, app: _AppT, state: Dict) -> None: - ... + def on_rebalance_end(self, app: _AppT, state: Dict) -> None: ... @abc.abstractmethod def on_web_request_start( self, app: _AppT, request: web.Request, *, view: web.View = None - ) -> Dict: - ... + ) -> Dict: ... @abc.abstractmethod def on_web_request_end( @@ -134,25 +113,20 @@ def on_web_request_end( state: Dict, *, view: web.View = None, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - def on_threaded_producer_buffer_processed(self, app: _AppT, size: int) -> None: - ... + def on_threaded_producer_buffer_processed(self, app: _AppT, size: int) -> None: ... -class SensorT(SensorInterfaceT, ServiceT): - ... +class SensorT(SensorInterfaceT, ServiceT): ... class SensorDelegateT(SensorInterfaceT, Iterable): # Delegate calls to many sensors. @abc.abstractmethod - def add(self, sensor: SensorT) -> None: - ... + def add(self, sensor: SensorT) -> None: ... @abc.abstractmethod - def remove(self, sensor: SensorT) -> None: - ... + def remove(self, sensor: SensorT) -> None: ... diff --git a/faust/types/serializers.py b/faust/types/serializers.py index 79163045a..dd8b93756 100644 --- a/faust/types/serializers.py +++ b/faust/types/serializers.py @@ -11,14 +11,11 @@ from .tuples import Message as _Message else: - class _AppT: - ... # noqa + class _AppT: ... # noqa - class _ModelArg: - ... # noqa + class _ModelArg: ... # noqa - class _Message: - ... # noqa + class _Message: ... # noqa __all__ = ["RegistryT", "SchemaT"] @@ -34,8 +31,7 @@ class RegistryT(abc.ABC): @abc.abstractmethod def __init__( self, key_serializer: CodecArg = None, value_serializer: CodecArg = "json" - ) -> None: - ... + ) -> None: ... @abc.abstractmethod def loads_key( @@ -44,8 +40,7 @@ def loads_key( key: Optional[bytes], *, serializer: CodecArg = None, - ) -> K: - ... + ) -> K: ... @abc.abstractmethod def loads_value( @@ -54,20 +49,17 @@ def loads_value( value: Optional[bytes], *, serializer: CodecArg = None, - ) -> Any: - ... + ) -> Any: ... @abc.abstractmethod def dumps_key( self, typ: Optional[_ModelArg], key: K, *, serializer: CodecArg = None - ) -> Optional[bytes]: - ... + ) -> Optional[bytes]: ... @abc.abstractmethod def dumps_value( self, typ: Optional[_ModelArg], value: V, *, serializer: CodecArg = None - ) -> Optional[bytes]: - ... + ) -> Optional[bytes]: ... class SchemaT(Generic[KT, VT]): @@ -87,8 +79,7 @@ def __init__( key_serializer: CodecArg = None, value_serializer: CodecArg = None, allow_empty: Optional[bool] = None, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod def update( @@ -99,8 +90,7 @@ def update( key_serializer: CodecArg = None, value_serializer: CodecArg = None, allow_empty: Optional[bool] = None, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod def loads_key( @@ -110,8 +100,7 @@ def loads_key( *, loads: Optional[Callable] = None, serializer: CodecArg = None, - ) -> KT: - ... + ) -> KT: ... @abc.abstractmethod def loads_value( @@ -121,8 +110,7 @@ def loads_value( *, loads: Optional[Callable] = None, serializer: CodecArg = None, - ) -> VT: - ... + ) -> VT: ... @abc.abstractmethod def dumps_key( @@ -132,8 +120,7 @@ def dumps_key( *, serializer: CodecArg = None, headers: OpenHeadersArg, - ) -> Tuple[Any, OpenHeadersArg]: - ... + ) -> Tuple[Any, OpenHeadersArg]: ... @abc.abstractmethod def dumps_value( @@ -143,17 +130,14 @@ def dumps_value( *, serializer: CodecArg = None, headers: OpenHeadersArg, - ) -> Tuple[Any, OpenHeadersArg]: - ... + ) -> Tuple[Any, OpenHeadersArg]: ... @abc.abstractmethod def on_dumps_key_prepare_headers( self, key: V, headers: OpenHeadersArg - ) -> OpenHeadersArg: - ... + ) -> OpenHeadersArg: ... @abc.abstractmethod def on_dumps_value_prepare_headers( self, value: V, headers: OpenHeadersArg - ) -> OpenHeadersArg: - ... + ) -> OpenHeadersArg: ... diff --git a/faust/types/settings/base.py b/faust/types/settings/base.py index b87b67e72..92f09eb1c 100644 --- a/faust/types/settings/base.py +++ b/faust/types/settings/base.py @@ -13,11 +13,9 @@ from .settings import Settings as _Settings else: - class _Section: - ... # noqa: E701 + class _Section: ... # noqa: E701 - class _Settings: - ... # noqa: E701 + class _Settings: ... # noqa: E701 W_ALREADY_CONFIGURED = """\ @@ -132,12 +130,10 @@ def __init_subclass__(self) -> None: self._init_subclass_settings() @abc.abstractmethod - def on_init(self, id: str, **kwargs: Any) -> None: - ... + def on_init(self, id: str, **kwargs: Any) -> None: ... @abc.abstractmethod - def getenv(self, env_name: str) -> Any: - ... + def getenv(self, env_name: str) -> Any: ... def _init_entrypoint(self, *args: Any, **kwargs: Any) -> None: self._on_before_init() diff --git a/faust/types/settings/params.py b/faust/types/settings/params.py index 7e7e1014d..c55267cd7 100644 --- a/faust/types/settings/params.py +++ b/faust/types/settings/params.py @@ -37,11 +37,9 @@ from .settings import Settings as _Settings else: - class _Section: - ... # noqa + class _Section: ... # noqa - class _Settings: - ... # noqa + class _Settings: ... # noqa __all__ = [ @@ -534,8 +532,7 @@ def _init_options( self.number_aliases = number_aliases or {} @abc.abstractmethod - def convert(self, conf: _Settings, value: IT) -> OT: - ... + def convert(self, conf: _Settings, value: IT) -> OT: ... def to_python(self, conf: _Settings, value: IT) -> OT: """Convert given value to number.""" diff --git a/faust/types/settings/sections.py b/faust/types/settings/sections.py index 4c5cd3cae..7edbd0395 100644 --- a/faust/types/settings/sections.py +++ b/faust/types/settings/sections.py @@ -27,8 +27,7 @@ from .params import Param as _Param else: - class _Param(Generic[IT, OT]): - ... # noqa: E701 + class _Param(Generic[IT, OT]): ... # noqa: E701 class SectionType(Enum): diff --git a/faust/types/settings/settings.py b/faust/types/settings/settings.py index f2203fff5..2fbc84d17 100644 --- a/faust/types/settings/settings.py +++ b/faust/types/settings/settings.py @@ -47,8 +47,7 @@ from faust.types.worker import Worker as _WorkerT else: - class _WorkerT: - ... # noqa + class _WorkerT: ... # noqa # XXX mypy borks if we do `from faust import __version__` @@ -193,8 +192,7 @@ def __init__( stream_ack_exceptions: Optional[bool] = None, url: URLArg = None, **kwargs: Any, - ) -> None: - ... # replaced by __init_subclass__ in BaseSettings + ) -> None: ... # replaced by __init_subclass__ in BaseSettings def on_init(self, id: str, **kwargs: Any) -> None: # version is required for the id diff --git a/faust/types/stores.py b/faust/types/stores.py index 85da70f62..8d89a383d 100644 --- a/faust/types/stores.py +++ b/faust/types/stores.py @@ -16,14 +16,11 @@ from .tables import CollectionT as _CollectionT else: - class _AppT: - ... # noqa + class _AppT: ... # noqa - class _ModelArg: - ... # noqa + class _ModelArg: ... # noqa - class _CollectionT: - ... # noqa + class _CollectionT: ... # noqa __all__ = ["StoreT"] @@ -57,20 +54,16 @@ def __init__( value_serializer: CodecArg = "", options: Optional[Mapping[str, Any]] = None, **kwargs: Any, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - def persisted_offset(self, tp: TP) -> Optional[int]: - ... + def persisted_offset(self, tp: TP) -> Optional[int]: ... @abc.abstractmethod - def set_persisted_offset(self, tp: TP, offset: int) -> None: - ... + def set_persisted_offset(self, tp: TP, offset: int) -> None: ... @abc.abstractmethod - async def need_active_standby_for(self, tp: TP) -> bool: - ... + async def need_active_standby_for(self, tp: TP) -> bool: ... @abc.abstractmethod def apply_changelog_batch( @@ -78,12 +71,10 @@ def apply_changelog_batch( batch: Iterable[EventT], to_key: Callable[[Any], KT], to_value: Callable[[Any], VT], - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - def reset_state(self) -> None: - ... + def reset_state(self) -> None: ... @abc.abstractmethod async def on_rebalance( @@ -92,20 +83,17 @@ async def on_rebalance( revoked: Set[TP], newly_assigned: Set[TP], generation_id: int = 0, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod async def on_recovery_completed( self, active_tps: Set[TP], standby_tps: Set[TP] - ) -> None: - ... + ) -> None: ... @abc.abstractmethod async def backup_partition( self, tp: Union[TP, int], flush: bool = True, purge: bool = False, keep: int = 1 - ) -> None: - ... + ) -> None: ... @abc.abstractmethod def restore_backup( @@ -113,5 +101,4 @@ def restore_backup( tp: Union[TP, int], latest: bool = True, backup_id: int = 0, - ) -> None: - ... + ) -> None: ... diff --git a/faust/types/streams.py b/faust/types/streams.py index ebc255f72..b5ab4b1e6 100644 --- a/faust/types/streams.py +++ b/faust/types/streams.py @@ -35,14 +35,11 @@ from .serializers import SchemaT as _SchemaT else: - class _AppT: - ... # noqa + class _AppT: ... # noqa - class _JoinT: - ... # noqa + class _JoinT: ... # noqa - class _SchemaT: - ... # noqa + class _SchemaT: ... # noqa __all__ = [ @@ -67,40 +64,31 @@ class _SchemaT: class JoinableT(abc.ABC): @abc.abstractmethod - def combine(self, *nodes: "JoinableT", **kwargs: Any) -> "StreamT": - ... + def combine(self, *nodes: "JoinableT", **kwargs: Any) -> "StreamT": ... @abc.abstractmethod - def join(self, *fields: FieldDescriptorT) -> "StreamT": - ... + def join(self, *fields: FieldDescriptorT) -> "StreamT": ... @abc.abstractmethod - def left_join(self, *fields: FieldDescriptorT) -> "StreamT": - ... + def left_join(self, *fields: FieldDescriptorT) -> "StreamT": ... @abc.abstractmethod - def inner_join(self, *fields: FieldDescriptorT) -> "StreamT": - ... + def inner_join(self, *fields: FieldDescriptorT) -> "StreamT": ... @abc.abstractmethod - def outer_join(self, *fields: FieldDescriptorT) -> "StreamT": - ... + def outer_join(self, *fields: FieldDescriptorT) -> "StreamT": ... @abc.abstractmethod - def __and__(self, other: Any) -> Any: - ... + def __and__(self, other: Any) -> Any: ... @abc.abstractmethod - def contribute_to_stream(self, active: "StreamT") -> None: - ... + def contribute_to_stream(self, active: "StreamT") -> None: ... @abc.abstractmethod - async def remove_from_stream(self, stream: "StreamT") -> None: - ... + async def remove_from_stream(self, stream: "StreamT") -> None: ... @abc.abstractmethod - def _human_channel(self) -> str: - ... + def _human_channel(self) -> str: ... class StreamT(AsyncIterable[T_co], JoinableT, ServiceT): @@ -146,51 +134,42 @@ def __init__( enable_acks: bool = True, prefix: str = "", loop: Optional[asyncio.AbstractEventLoop] = None, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - def get_active_stream(self) -> "StreamT": - ... + def get_active_stream(self) -> "StreamT": ... @abc.abstractmethod - def add_processor(self, processor: Processor[T]) -> None: - ... + def add_processor(self, processor: Processor[T]) -> None: ... @abc.abstractmethod - def info(self) -> Mapping[str, Any]: - ... + def info(self) -> Mapping[str, Any]: ... @abc.abstractmethod - def clone(self, **kwargs: Any) -> "StreamT": - ... + def clone(self, **kwargs: Any) -> "StreamT": ... @abc.abstractmethod @no_type_check - async def items(self) -> AsyncIterator[Tuple[K, T_co]]: - ... + async def items(self) -> AsyncIterator[Tuple[K, T_co]]: ... @abc.abstractmethod @no_type_check - async def events(self) -> AsyncIterable[EventT]: - ... + async def events(self) -> AsyncIterable[EventT]: ... @abc.abstractmethod @no_type_check - async def take(self, max_: int, within: Seconds) -> AsyncIterable[Sequence[T_co]]: - ... + async def take( + self, max_: int, within: Seconds + ) -> AsyncIterable[Sequence[T_co]]: ... @abc.abstractmethod - def enumerate(self, start: int = 0) -> AsyncIterable[Tuple[int, T_co]]: - ... + def enumerate(self, start: int = 0) -> AsyncIterable[Tuple[int, T_co]]: ... @abc.abstractmethod - def through(self, channel: Union[str, ChannelT]) -> "StreamT": - ... + def through(self, channel: Union[str, ChannelT]) -> "StreamT": ... @abc.abstractmethod - def echo(self, *channels: Union[str, ChannelT]) -> "StreamT": - ... + def echo(self, *channels: Union[str, ChannelT]) -> "StreamT": ... @abc.abstractmethod def group_by( @@ -199,8 +178,7 @@ def group_by( *, name: Optional[str] = None, topic: Optional[TopicT] = None, - ) -> "StreamT": - ... + ) -> "StreamT": ... @abc.abstractmethod def derive_topic( @@ -212,29 +190,22 @@ def derive_topic( value_type: ModelArg = None, prefix: str = "", suffix: str = "", - ) -> TopicT: - ... + ) -> TopicT: ... @abc.abstractmethod - async def throw(self, exc: BaseException) -> None: - ... + async def throw(self, exc: BaseException) -> None: ... @abc.abstractmethod - def __copy__(self) -> "StreamT": - ... + def __copy__(self) -> "StreamT": ... @abc.abstractmethod - def __iter__(self) -> Any: - ... + def __iter__(self) -> Any: ... @abc.abstractmethod - def __next__(self) -> T: - ... + def __next__(self) -> T: ... @abc.abstractmethod - def __aiter__(self) -> AsyncIterator[T_co]: - ... + def __aiter__(self) -> AsyncIterator[T_co]: ... @abc.abstractmethod - async def ack(self, event: EventT) -> bool: - ... + async def ack(self, event: EventT) -> bool: ... diff --git a/faust/types/tables.py b/faust/types/tables.py index d5eef08e3..ffde15400 100644 --- a/faust/types/tables.py +++ b/faust/types/tables.py @@ -37,17 +37,13 @@ from .serializers import SchemaT as _SchemaT else: - class _AppT: - ... # noqa + class _AppT: ... # noqa - class _FieldDescriptorT: - ... # noqa + class _FieldDescriptorT: ... # noqa - class _ModelArg: - ... # noqa + class _ModelArg: ... # noqa - class _SchemaT: - ... # noqa + class _SchemaT: ... # noqa __all__ = [ @@ -126,41 +122,32 @@ def __init__( use_partitioner: bool = False, on_window_close: Optional[WindowCloseCallback] = None, **kwargs: Any, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - def clone(self, **kwargs: Any) -> Any: - ... + def clone(self, **kwargs: Any) -> Any: ... @property @abc.abstractmethod - def changelog_topic(self) -> TopicT: - ... + def changelog_topic(self) -> TopicT: ... @changelog_topic.setter - def changelog_topic(self, topic: TopicT) -> None: - ... + def changelog_topic(self, topic: TopicT) -> None: ... @abc.abstractmethod - def _changelog_topic_name(self) -> str: - ... + def _changelog_topic_name(self) -> str: ... @abc.abstractmethod - def apply_changelog_batch(self, batch: Iterable[EventT]) -> None: - ... + def apply_changelog_batch(self, batch: Iterable[EventT]) -> None: ... @abc.abstractmethod - def persisted_offset(self, tp: TP) -> Optional[int]: - ... + def persisted_offset(self, tp: TP) -> Optional[int]: ... @abc.abstractmethod - async def need_active_standby_for(self, tp: TP) -> bool: - ... + async def need_active_standby_for(self, tp: TP) -> bool: ... @abc.abstractmethod - def reset_state(self) -> None: - ... + def reset_state(self) -> None: ... @abc.abstractmethod def send_changelog( @@ -170,16 +157,13 @@ def send_changelog( value: Any, key_serializer: CodecArg = None, value_serializer: CodecArg = None, - ) -> FutureMessage: - ... + ) -> FutureMessage: ... @abc.abstractmethod - def partition_for_key(self, key: Any) -> Optional[int]: - ... + def partition_for_key(self, key: Any) -> Optional[int]: ... @abc.abstractmethod - async def on_window_close(self, key: Any, value: Any) -> None: - ... + async def on_window_close(self, key: Any, value: Any) -> None: ... @abc.abstractmethod async def on_rebalance( @@ -188,32 +172,26 @@ async def on_rebalance( revoked: Set[TP], newly_assigned: Set[TP], generation_id: int = 0, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - async def on_changelog_event(self, event: EventT) -> None: - ... + async def on_changelog_event(self, event: EventT) -> None: ... @abc.abstractmethod - def on_recover(self, fun: RecoverCallback) -> RecoverCallback: - ... + def on_recover(self, fun: RecoverCallback) -> RecoverCallback: ... @abc.abstractmethod async def on_recovery_completed( self, active_tps: Set[TP], standby_tps: Set[TP] - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - async def call_recover_callbacks(self) -> None: - ... + async def call_recover_callbacks(self) -> None: ... @abc.abstractmethod def using_window( self, window: WindowT, *, key_index: bool = False - ) -> "WindowWrapperT": - ... + ) -> "WindowWrapperT": ... @abc.abstractmethod def hopping( @@ -222,46 +200,36 @@ def hopping( step: Seconds, expires: Optional[Seconds] = None, key_index: bool = False, - ) -> "WindowWrapperT": - ... + ) -> "WindowWrapperT": ... @abc.abstractmethod def tumbling( self, size: Seconds, expires: Optional[Seconds] = None, key_index: bool = False - ) -> "WindowWrapperT": - ... + ) -> "WindowWrapperT": ... @abc.abstractmethod - def as_ansitable(self, **kwargs: Any) -> str: - ... + def as_ansitable(self, **kwargs: Any) -> str: ... @abc.abstractmethod - def _relative_now(self, event: Optional[EventT] = None) -> float: - ... + def _relative_now(self, event: Optional[EventT] = None) -> float: ... @abc.abstractmethod - def _relative_event(self, event: Optional[EventT] = None) -> float: - ... + def _relative_event(self, event: Optional[EventT] = None) -> float: ... @abc.abstractmethod - def _relative_field(self, field: _FieldDescriptorT) -> RelativeHandler: - ... + def _relative_field(self, field: _FieldDescriptorT) -> RelativeHandler: ... @abc.abstractmethod - def _relative_timestamp(self, timestamp: float) -> RelativeHandler: - ... + def _relative_timestamp(self, timestamp: float) -> RelativeHandler: ... @abc.abstractmethod - def _windowed_contains(self, key: Any, timestamp: float) -> bool: - ... + def _windowed_contains(self, key: Any, timestamp: float) -> bool: ... -class TableT(CollectionT, ManagedUserDict[KT, VT]): - ... +class TableT(CollectionT, ManagedUserDict[KT, VT]): ... -class GlobalTableT(TableT): - ... +class GlobalTableT(TableT): ... class TableManagerT(ServiceT, FastUserDict[str, CollectionT]): @@ -271,20 +239,16 @@ class TableManagerT(ServiceT, FastUserDict[str, CollectionT]): standbys_ready: bool @abc.abstractmethod - def __init__(self, app: _AppT, **kwargs: Any) -> None: - ... + def __init__(self, app: _AppT, **kwargs: Any) -> None: ... @abc.abstractmethod - def add(self, table: CollectionT) -> CollectionT: - ... + def add(self, table: CollectionT) -> CollectionT: ... @abc.abstractmethod - def persist_offset_on_commit(self, store: StoreT, tp: TP, offset: int) -> None: - ... + def persist_offset_on_commit(self, store: StoreT, tp: TP, offset: int) -> None: ... @abc.abstractmethod - def on_commit(self, offsets: MutableMapping[TP, int]) -> None: - ... + def on_commit(self, offsets: MutableMapping[TP, int]) -> None: ... @abc.abstractmethod async def on_rebalance( @@ -293,29 +257,23 @@ async def on_rebalance( revoked: Set[TP], newly_assigned: Set[TP], generation_id: int = 0, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - def on_partitions_revoked(self, revoked: Set[TP]) -> None: - ... + def on_partitions_revoked(self, revoked: Set[TP]) -> None: ... @abc.abstractmethod - def on_rebalance_start(self) -> None: - ... + def on_rebalance_start(self) -> None: ... @abc.abstractmethod - async def wait_until_tables_registered(self) -> None: - ... + async def wait_until_tables_registered(self) -> None: ... @abc.abstractmethod - async def wait_until_recovery_completed(self) -> bool: - ... + async def wait_until_recovery_completed(self) -> bool: ... @property @abc.abstractmethod - def changelog_topics(self) -> Set[str]: - ... + def changelog_topics(self) -> Set[str]: ... class WindowSetT(FastUserDict[KT, VT]): @@ -330,128 +288,100 @@ def __init__( table: TableT, wrapper: "WindowWrapperT", event: Optional[EventT] = None, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod def apply( self, op: Callable[[VT, VT], VT], value: VT, event: Optional[EventT] = None - ) -> "WindowSetT": - ... + ) -> "WindowSetT": ... @abc.abstractmethod - def value(self, event: Optional[EventT] = None) -> VT: - ... + def value(self, event: Optional[EventT] = None) -> VT: ... @abc.abstractmethod - def current(self, event: Optional[EventT] = None) -> VT: - ... + def current(self, event: Optional[EventT] = None) -> VT: ... @abc.abstractmethod - def now(self) -> VT: - ... + def now(self) -> VT: ... @abc.abstractmethod - def delta(self, d: Seconds, event: Optional[EventT] = None) -> VT: - ... + def delta(self, d: Seconds, event: Optional[EventT] = None) -> VT: ... @abc.abstractmethod - def __iadd__(self, other: VT) -> "WindowSetT": - ... + def __iadd__(self, other: VT) -> "WindowSetT": ... @abc.abstractmethod - def __isub__(self, other: VT) -> "WindowSetT": - ... + def __isub__(self, other: VT) -> "WindowSetT": ... @abc.abstractmethod - def __imul__(self, other: VT) -> "WindowSetT": - ... + def __imul__(self, other: VT) -> "WindowSetT": ... @abc.abstractmethod - def __itruediv__(self, other: VT) -> "WindowSetT": - ... + def __itruediv__(self, other: VT) -> "WindowSetT": ... @abc.abstractmethod - def __ifloordiv__(self, other: VT) -> "WindowSetT": - ... + def __ifloordiv__(self, other: VT) -> "WindowSetT": ... @abc.abstractmethod - def __imod__(self, other: VT) -> "WindowSetT": - ... + def __imod__(self, other: VT) -> "WindowSetT": ... @abc.abstractmethod - def __ipow__(self, other: VT) -> "WindowSetT": - ... + def __ipow__(self, other: VT) -> "WindowSetT": ... @abc.abstractmethod - def __ilshift__(self, other: VT) -> "WindowSetT": - ... + def __ilshift__(self, other: VT) -> "WindowSetT": ... @abc.abstractmethod - def __irshift__(self, other: VT) -> "WindowSetT": - ... + def __irshift__(self, other: VT) -> "WindowSetT": ... @abc.abstractmethod - def __iand__(self, other: VT) -> "WindowSetT": - ... + def __iand__(self, other: VT) -> "WindowSetT": ... @abc.abstractmethod - def __ixor__(self, other: VT) -> "WindowSetT": - ... + def __ixor__(self, other: VT) -> "WindowSetT": ... @abc.abstractmethod - def __ior__(self, other: VT) -> "WindowSetT": - ... + def __ior__(self, other: VT) -> "WindowSetT": ... class WindowedItemsViewT(ItemsView): @abc.abstractmethod def __init__( self, mapping: "WindowWrapperT", event: Optional[EventT] = None - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - def __iter__(self) -> Iterator[Tuple[Any, Any]]: - ... + def __iter__(self) -> Iterator[Tuple[Any, Any]]: ... @abc.abstractmethod - def now(self) -> Iterator[Tuple[Any, Any]]: - ... + def now(self) -> Iterator[Tuple[Any, Any]]: ... @abc.abstractmethod - def current(self, event: Optional[EventT] = None) -> Iterator[Tuple[Any, Any]]: - ... + def current(self, event: Optional[EventT] = None) -> Iterator[Tuple[Any, Any]]: ... @abc.abstractmethod def delta( self, d: Seconds, event: Optional[EventT] = None - ) -> Iterator[Tuple[Any, Any]]: - ... + ) -> Iterator[Tuple[Any, Any]]: ... class WindowedValuesViewT(ValuesView): @abc.abstractmethod def __init__( self, mapping: "WindowWrapperT", event: Optional[EventT] = None - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - def __iter__(self) -> Iterator[Any]: - ... + def __iter__(self) -> Iterator[Any]: ... @abc.abstractmethod - def now(self) -> Iterator[Any]: - ... + def now(self) -> Iterator[Any]: ... @abc.abstractmethod - def current(self, event: Optional[EventT] = None) -> Iterator[Any]: - ... + def current(self, event: Optional[EventT] = None) -> Iterator[Any]: ... @abc.abstractmethod - def delta(self, d: Seconds, event: Optional[EventT] = None) -> Iterator[Any]: - ... + def delta(self, d: Seconds, event: Optional[EventT] = None) -> Iterator[Any]: ... class WindowWrapperT(MutableMapping): @@ -465,58 +395,44 @@ def __init__( relative_to: RelativeArg = None, key_index: bool = False, key_index_table: Optional[TableT] = None, - ) -> None: - ... + ) -> None: ... @property @abc.abstractmethod - def name(self) -> str: - ... + def name(self) -> str: ... @abc.abstractmethod - def clone(self, relative_to: RelativeArg) -> "WindowWrapperT": - ... + def clone(self, relative_to: RelativeArg) -> "WindowWrapperT": ... @abc.abstractmethod - def relative_to_now(self) -> "WindowWrapperT": - ... + def relative_to_now(self) -> "WindowWrapperT": ... @abc.abstractmethod - def relative_to_field(self, field: _FieldDescriptorT) -> "WindowWrapperT": - ... + def relative_to_field(self, field: _FieldDescriptorT) -> "WindowWrapperT": ... @abc.abstractmethod - def relative_to_stream(self) -> "WindowWrapperT": - ... + def relative_to_stream(self) -> "WindowWrapperT": ... @abc.abstractmethod - def get_timestamp(self, event: Optional[EventT] = None) -> float: - ... + def get_timestamp(self, event: Optional[EventT] = None) -> float: ... @abc.abstractmethod - def __getitem__(self, key: Any) -> WindowSetT: - ... + def __getitem__(self, key: Any) -> WindowSetT: ... @abc.abstractmethod - def keys(self) -> KeysView: - ... + def keys(self) -> KeysView: ... @abc.abstractmethod - def on_set_key(self, key: Any, value: Any) -> None: - ... + def on_set_key(self, key: Any, value: Any) -> None: ... @abc.abstractmethod - def on_del_key(self, key: Any) -> None: - ... + def on_del_key(self, key: Any) -> None: ... @abc.abstractmethod - def as_ansitable(self, **kwargs: Any) -> str: - ... + def as_ansitable(self, **kwargs: Any) -> str: ... @property - def get_relative_timestamp(self) -> Optional[RelativeHandler]: - ... + def get_relative_timestamp(self) -> Optional[RelativeHandler]: ... @get_relative_timestamp.setter - def get_relative_timestamp(self, relative_to: RelativeArg) -> None: - ... + def get_relative_timestamp(self, relative_to: RelativeArg) -> None: ... diff --git a/faust/types/topics.py b/faust/types/topics.py index 9250747eb..09d52d563 100644 --- a/faust/types/topics.py +++ b/faust/types/topics.py @@ -16,14 +16,11 @@ from .serializers import SchemaT as _SchemaT else: - class _AppT: - ... # noqa + class _AppT: ... # noqa - class _ModelArg: - ... # noqa + class _ModelArg: ... # noqa - class _SchemaT: - ... # noqa + class _SchemaT: ... # noqa __all__ = ["TopicT"] @@ -89,30 +86,24 @@ def __init__( allow_empty: bool = False, has_prefix: bool = False, loop: Optional[asyncio.AbstractEventLoop] = None, - ) -> None: - ... + ) -> None: ... @property @abc.abstractmethod - def pattern(self) -> Optional[Pattern]: - ... + def pattern(self) -> Optional[Pattern]: ... @pattern.setter - def pattern(self, pattern: Union[str, Pattern]) -> None: - ... + def pattern(self, pattern: Union[str, Pattern]) -> None: ... @property @abc.abstractmethod - def partitions(self) -> Optional[int]: - ... + def partitions(self) -> Optional[int]: ... @partitions.setter - def partitions(self, partitions: int) -> None: - ... + def partitions(self, partitions: int) -> None: ... @abc.abstractmethod - def derive(self, **kwargs: Any) -> ChannelT: - ... + def derive(self, **kwargs: Any) -> ChannelT: ... @abc.abstractmethod def derive_topic( @@ -131,5 +122,4 @@ def derive_topic( prefix: str = "", suffix: str = "", **kwargs: Any, - ) -> "TopicT": - ... + ) -> "TopicT": ... diff --git a/faust/types/transports.py b/faust/types/transports.py index db2030bc9..205c56e0f 100644 --- a/faust/types/transports.py +++ b/faust/types/transports.py @@ -35,8 +35,7 @@ from .app import AppT as _AppT else: - class _AppT: - ... # noqa + class _AppT: ... # noqa __all__ = [ @@ -83,25 +82,20 @@ class ProducerBufferT(ServiceT): pending: asyncio.Queue @abc.abstractmethod - def put(self, fut: FutureMessage) -> None: - ... + def put(self, fut: FutureMessage) -> None: ... @abc.abstractmethod - async def flush(self) -> None: - ... + async def flush(self) -> None: ... @abc.abstractmethod - async def flush_atmost(self, n: int) -> int: - ... + async def flush_atmost(self, n: int) -> int: ... @abc.abstractmethod - async def wait_until_ebb(self) -> None: - ... + async def wait_until_ebb(self) -> None: ... @property @abc.abstractmethod - def size(self) -> int: - ... + def size(self) -> int: ... class ProducerT(ServiceT): @@ -126,8 +120,7 @@ def __init__( transport: "TransportT", loop: Optional[asyncio.AbstractEventLoop] = None, **kwargs: Any, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod async def send( @@ -140,12 +133,10 @@ async def send( headers: Optional[HeadersArg], *, transactional_id: Optional[str] = None, - ) -> Awaitable[RecordMetadata]: - ... + ) -> Awaitable[RecordMetadata]: ... @abc.abstractmethod - def send_soon(self, fut: FutureMessage) -> None: - ... + def send_soon(self, fut: FutureMessage) -> None: ... @abc.abstractmethod async def send_and_wait( @@ -158,8 +149,7 @@ async def send_and_wait( headers: Optional[HeadersArg], *, transactional_id: Optional[str] = None, - ) -> RecordMetadata: - ... + ) -> RecordMetadata: ... @abc.abstractmethod async def create_topic( @@ -174,36 +164,28 @@ async def create_topic( compacting: Optional[bool] = None, deleting: Optional[bool] = None, ensure_created: bool = False, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - def key_partition(self, topic: str, key: bytes) -> TP: - ... + def key_partition(self, topic: str, key: bytes) -> TP: ... @abc.abstractmethod - async def flush(self) -> None: - ... + async def flush(self) -> None: ... @abc.abstractmethod - async def begin_transaction(self, transactional_id: str) -> None: - ... + async def begin_transaction(self, transactional_id: str) -> None: ... @abc.abstractmethod - async def commit_transaction(self, transactional_id: str) -> None: - ... + async def commit_transaction(self, transactional_id: str) -> None: ... @abc.abstractmethod - async def abort_transaction(self, transactional_id: str) -> None: - ... + async def abort_transaction(self, transactional_id: str) -> None: ... @abc.abstractmethod - async def stop_transaction(self, transactional_id: str) -> None: - ... + async def stop_transaction(self, transactional_id: str) -> None: ... @abc.abstractmethod - async def maybe_begin_transaction(self, transactional_id: str) -> None: - ... + async def maybe_begin_transaction(self, transactional_id: str) -> None: ... @abc.abstractmethod async def commit_transactions( @@ -211,12 +193,10 @@ async def commit_transactions( tid_to_offset_map: Mapping[str, Mapping[TP, int]], group_id: str, start_new_transaction: bool = True, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - def supports_headers(self) -> bool: - ... + def supports_headers(self) -> bool: ... # Probably this has to go because Transactions are supported in aiokafka @@ -233,24 +213,20 @@ def __init__( consumer: "ConsumerT", producer: "ProducerT", **kwargs: Any, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - async def on_partitions_revoked(self, revoked: Set[TP]) -> None: - ... + async def on_partitions_revoked(self, revoked: Set[TP]) -> None: ... @abc.abstractmethod async def on_rebalance( self, assigned: Set[TP], revoked: Set[TP], newly_assigned: Set[TP] - ) -> None: - ... + ) -> None: ... @abc.abstractmethod async def commit( self, offsets: Mapping[TP, int], start_new_transaction: bool = True - ) -> bool: - ... + ) -> bool: ... async def begin_transaction(self, transactional_id: str) -> None: raise NotImplementedError() @@ -278,12 +254,10 @@ async def commit_transactions( class SchedulingStrategyT: @abc.abstractmethod - def __init__(self) -> None: - ... + def __init__(self) -> None: ... @abc.abstractmethod - def iterate(self, records: Mapping[TP, List]) -> Iterator[Tuple[TP, Any]]: - ... + def iterate(self, records: Mapping[TP, List]) -> Iterator[Tuple[TP, Any]]: ... class ConsumerT(ServiceT): @@ -333,118 +307,91 @@ async def create_topic( compacting: Optional[bool] = None, deleting: Optional[bool] = None, ensure_created: bool = False, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - async def subscribe(self, topics: Iterable[str]) -> None: - ... + async def subscribe(self, topics: Iterable[str]) -> None: ... @abc.abstractmethod @no_type_check - async def getmany(self, timeout: float) -> AsyncIterator[Tuple[TP, Message]]: - ... + async def getmany(self, timeout: float) -> AsyncIterator[Tuple[TP, Message]]: ... @abc.abstractmethod - def track_message(self, message: Message) -> None: - ... + def track_message(self, message: Message) -> None: ... @abc.abstractmethod - async def perform_seek(self) -> None: - ... + async def perform_seek(self) -> None: ... @abc.abstractmethod - def ack(self, message: Message) -> bool: - ... + def ack(self, message: Message) -> bool: ... @abc.abstractmethod - async def wait_empty(self) -> None: - ... + async def wait_empty(self) -> None: ... @abc.abstractmethod - def assignment(self) -> Set[TP]: - ... + def assignment(self) -> Set[TP]: ... @abc.abstractmethod - def highwater(self, tp: TP) -> int: - ... + def highwater(self, tp: TP) -> int: ... @abc.abstractmethod - def stop_flow(self) -> None: - ... + def stop_flow(self) -> None: ... @abc.abstractmethod - def resume_flow(self) -> None: - ... + def resume_flow(self) -> None: ... @abc.abstractmethod - def pause_partitions(self, tps: Iterable[TP]) -> None: - ... + def pause_partitions(self, tps: Iterable[TP]) -> None: ... @abc.abstractmethod - def resume_partitions(self, tps: Iterable[TP]) -> None: - ... + def resume_partitions(self, tps: Iterable[TP]) -> None: ... @abc.abstractmethod - async def position(self, tp: TP) -> Optional[int]: - ... + async def position(self, tp: TP) -> Optional[int]: ... @abc.abstractmethod - async def seek(self, partition: TP, offset: int) -> None: - ... + async def seek(self, partition: TP, offset: int) -> None: ... @abc.abstractmethod - async def seek_wait(self, partitions: Mapping[TP, int]) -> None: - ... + async def seek_wait(self, partitions: Mapping[TP, int]) -> None: ... @abc.abstractmethod async def commit( self, topics: TPorTopicSet = None, start_new_transaction: bool = True - ) -> bool: - ... + ) -> bool: ... @abc.abstractmethod - async def on_task_error(self, exc: BaseException) -> None: - ... + async def on_task_error(self, exc: BaseException) -> None: ... @abc.abstractmethod - async def earliest_offsets(self, *partitions: TP) -> Mapping[TP, int]: - ... + async def earliest_offsets(self, *partitions: TP) -> Mapping[TP, int]: ... @abc.abstractmethod - async def highwaters(self, *partitions: TP) -> Mapping[TP, int]: - ... + async def highwaters(self, *partitions: TP) -> Mapping[TP, int]: ... @abc.abstractmethod - def topic_partitions(self, topic: str) -> Optional[int]: - ... + def topic_partitions(self, topic: str) -> Optional[int]: ... @abc.abstractmethod def key_partition( self, topic: str, key: Optional[bytes], partition: Optional[int] = None - ) -> Optional[int]: - ... + ) -> Optional[int]: ... @abc.abstractmethod - def close(self) -> None: - ... + def close(self) -> None: ... @abc.abstractmethod - def verify_recovery_event_path(self, now: float, tp: TP) -> None: - ... + def verify_recovery_event_path(self, now: float, tp: TP) -> None: ... @property @abc.abstractmethod - def unacked(self) -> Set[Message]: - ... + def unacked(self) -> Set[Message]: ... @abc.abstractmethod - def on_buffer_full(self, tp: TP) -> None: - ... + def on_buffer_full(self, tp: TP) -> None: ... @abc.abstractmethod - def on_buffer_drop(self, tp: TP) -> None: - ... + def on_buffer_drop(self, tp: TP) -> None: ... class ConductorT(ServiceT, MutableSet[TopicT]): @@ -458,33 +405,26 @@ def __init__(self, app: _AppT, **kwargs: Any) -> None: self.on_message: ConsumerCallback @abc.abstractmethod - async def on_client_only_start(self) -> None: - ... + async def on_client_only_start(self) -> None: ... @abc.abstractmethod - def acks_enabled_for(self, topic: str) -> bool: - ... + def acks_enabled_for(self, topic: str) -> bool: ... @abc.abstractmethod - async def commit(self, topics: TPorTopicSet) -> bool: - ... + async def commit(self, topics: TPorTopicSet) -> bool: ... @abc.abstractmethod - async def wait_for_subscriptions(self) -> None: - ... + async def wait_for_subscriptions(self) -> None: ... @abc.abstractmethod - async def maybe_wait_for_subscriptions(self) -> None: - ... + async def maybe_wait_for_subscriptions(self) -> None: ... @abc.abstractmethod - async def on_partitions_assigned(self, assigned: Set[TP]) -> None: - ... + async def on_partitions_assigned(self, assigned: Set[TP]) -> None: ... @property @abc.abstractmethod - def acking_topics(self) -> Set[str]: - ... + def acking_topics(self) -> Set[str]: ... class TransportT(abc.ABC): @@ -521,17 +461,15 @@ def __init__( url: List[URL], app: _AppT, loop: Optional[asyncio.AbstractEventLoop] = None, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - def create_consumer(self, callback: ConsumerCallback, **kwargs: Any) -> ConsumerT: - ... + def create_consumer( + self, callback: ConsumerCallback, **kwargs: Any + ) -> ConsumerT: ... @abc.abstractmethod - def create_producer(self, **kwargs: Any) -> ProducerT: - ... + def create_producer(self, **kwargs: Any) -> ProducerT: ... @abc.abstractmethod - def create_conductor(self, **kwargs: Any) -> ConductorT: - ... + def create_conductor(self, **kwargs: Any) -> ConductorT: ... diff --git a/faust/types/tuples.py b/faust/types/tuples.py index 73142370f..ba8fb940e 100644 --- a/faust/types/tuples.py +++ b/faust/types/tuples.py @@ -22,11 +22,9 @@ from .transports import ConsumerT as _ConsumerT else: - class _ChannelT: - ... # noqa + class _ChannelT: ... # noqa - class _ConsumerT: - ... # noqa + class _ConsumerT: ... # noqa __all__ = [ diff --git a/faust/types/web.py b/faust/types/web.py index 6027a4aa1..62bc32a42 100644 --- a/faust/types/web.py +++ b/faust/types/web.py @@ -24,20 +24,15 @@ from faust.web.views import View else: - class _AppT: - ... # noqa + class _AppT: ... # noqa - class Request: - ... # noqa + class Request: ... # noqa - class Response: - ... # noqa + class Response: ... # noqa - class Web: - ... # noqa + class Web: ... # noqa - class View: - ... # noqa + class View: ... # noqa __all__ = [ @@ -101,22 +96,18 @@ class CacheBackendT(ServiceT): @abc.abstractmethod def __init__( self, app: _AppT, url: Union[URL, str] = "memory://", **kwargs: Any - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - async def get(self, key: str) -> Optional[bytes]: - ... + async def get(self, key: str) -> Optional[bytes]: ... @abc.abstractmethod async def set( self, key: str, value: bytes, timeout: Optional[float] = None - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - async def delete(self, key: str) -> None: - ... + async def delete(self, key: str) -> None: ... class CacheT(abc.ABC): @@ -132,8 +123,7 @@ def __init__( key_prefix: Optional[str] = None, backend: Union[Type[CacheBackendT], str] = None, **kwargs: Any, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod def view( @@ -142,8 +132,7 @@ def view( include_headers: bool = False, key_prefix: Optional[str] = None, **kwargs: Any, - ) -> Callable[[Callable], Callable]: - ... + ) -> Callable[[Callable], Callable]: ... class BlueprintT(abc.ABC): @@ -157,14 +146,12 @@ def cache( include_headers: bool = False, key_prefix: Optional[str] = None, backend: Union[Type[CacheBackendT], str] = None, - ) -> CacheT: - ... + ) -> CacheT: ... @abc.abstractmethod def route( self, uri: str, *, name: Optional[str] = None, base: Type[View] = View - ) -> RouteDecoratorRet: - ... + ) -> RouteDecoratorRet: ... @abc.abstractmethod def static( @@ -173,17 +160,13 @@ def static( file_or_directory: Union[str, Path], *, name: Optional[str] = None, - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - def register(self, app: _AppT, *, url_prefix: Optional[str] = None) -> None: - ... + def register(self, app: _AppT, *, url_prefix: Optional[str] = None) -> None: ... @abc.abstractmethod - def init_webserver(self, web: Web) -> None: - ... + def init_webserver(self, web: Web) -> None: ... @abc.abstractmethod - def on_webserver_init(self, web: Web) -> None: - ... + def on_webserver_init(self, web: Web) -> None: ... diff --git a/faust/types/windows.py b/faust/types/windows.py index 94b9f7bb1..5f5457f44 100644 --- a/faust/types/windows.py +++ b/faust/types/windows.py @@ -1,4 +1,5 @@ """Types related to windowing.""" + import abc from datetime import timezone from typing import List, Optional, Tuple @@ -24,21 +25,16 @@ class WindowT(abc.ABC): tz: Optional[timezone] = None @abc.abstractmethod - def ranges(self, timestamp: float) -> List[WindowRange]: - ... + def ranges(self, timestamp: float) -> List[WindowRange]: ... @abc.abstractmethod - def stale(self, timestamp: float, latest_timestamp: float) -> bool: - ... + def stale(self, timestamp: float, latest_timestamp: float) -> bool: ... @abc.abstractmethod - def current(self, timestamp: float) -> WindowRange: - ... + def current(self, timestamp: float) -> WindowRange: ... @abc.abstractmethod - def earliest(self, timestamp: float) -> WindowRange: - ... + def earliest(self, timestamp: float) -> WindowRange: ... @abc.abstractmethod - def delta(self, timestamp: float, d: Seconds) -> WindowRange: - ... + def delta(self, timestamp: float, d: Seconds) -> WindowRange: ... diff --git a/faust/utils/__init__.py b/faust/utils/__init__.py index 132e8edfa..e376096f1 100644 --- a/faust/utils/__init__.py +++ b/faust/utils/__init__.py @@ -1,4 +1,5 @@ """Utilities.""" + from uuid import uuid4 __all__ = ["uuid"] diff --git a/faust/utils/_iso8601_python.py b/faust/utils/_iso8601_python.py index 3732abd2b..c63390bd3 100644 --- a/faust/utils/_iso8601_python.py +++ b/faust/utils/_iso8601_python.py @@ -32,6 +32,7 @@ SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. """ + import re from datetime import datetime, timedelta, timezone, tzinfo from typing import Mapping, Match, Optional, Pattern, cast diff --git a/faust/utils/agent_stopper.py b/faust/utils/agent_stopper.py index 820ae4d20..e5db7322b 100644 --- a/faust/utils/agent_stopper.py +++ b/faust/utils/agent_stopper.py @@ -1,4 +1,5 @@ """agent_stopper for Faust when app fails""" + import logging import traceback diff --git a/faust/utils/codegen.py b/faust/utils/codegen.py index 1999eb777..ef3c9f93f 100644 --- a/faust/utils/codegen.py +++ b/faust/utils/codegen.py @@ -1,4 +1,5 @@ """Utilities for generating code at runtime.""" + from typing import Any, Callable, Dict, List, Mapping, Tuple, cast __all__ = [ diff --git a/faust/utils/cron.py b/faust/utils/cron.py index f1b583981..ed96191f2 100644 --- a/faust/utils/cron.py +++ b/faust/utils/cron.py @@ -1,4 +1,5 @@ """Crontab Utilities.""" + import time from datetime import datetime, tzinfo diff --git a/faust/utils/functional.py b/faust/utils/functional.py index 729630ea3..98d4ad764 100644 --- a/faust/utils/functional.py +++ b/faust/utils/functional.py @@ -1,4 +1,5 @@ """Functional utilities.""" + from functools import reduce from itertools import groupby from typing import Iterable, Iterator, Mapping, Sequence, Tuple, TypeVar diff --git a/faust/utils/iso8601.py b/faust/utils/iso8601.py index 69560714f..b8fe293a0 100644 --- a/faust/utils/iso8601.py +++ b/faust/utils/iso8601.py @@ -1,4 +1,5 @@ """Parsing ISO-8601 string and converting to :class:`~datetime.datetime`.""" + try: # pragma: no cover import ciso8601 except ImportError: # pragma: no cover diff --git a/faust/utils/json.py b/faust/utils/json.py index 66eb5a0ca..8fb13a386 100644 --- a/faust/utils/json.py +++ b/faust/utils/json.py @@ -1,4 +1,5 @@ """JSON utilities.""" + import datetime import enum import typing diff --git a/faust/utils/kafka/protocol/admin.py b/faust/utils/kafka/protocol/admin.py index 3e418e960..f2be80d9c 100644 --- a/faust/utils/kafka/protocol/admin.py +++ b/faust/utils/kafka/protocol/admin.py @@ -1,4 +1,5 @@ """Admin related Kafka protocol extensions.""" + from aiokafka.protocol import types from .api import Request, Response diff --git a/faust/utils/kafka/protocol/api.py b/faust/utils/kafka/protocol/api.py index 523d9ef0f..74af0f466 100644 --- a/faust/utils/kafka/protocol/api.py +++ b/faust/utils/kafka/protocol/api.py @@ -1,4 +1,5 @@ """Kafka protocol extensions.""" + # pragma: no cover import abc from typing import Type diff --git a/faust/utils/platforms.py b/faust/utils/platforms.py index a1d0ec51e..506d583b5 100644 --- a/faust/utils/platforms.py +++ b/faust/utils/platforms.py @@ -1,4 +1,5 @@ """Platform/OS utilities.""" + import platform import subprocess # nosec: B404 from typing import Optional diff --git a/faust/utils/terminal/__init__.py b/faust/utils/terminal/__init__.py index ea02226bf..7dd4ad205 100644 --- a/faust/utils/terminal/__init__.py +++ b/faust/utils/terminal/__init__.py @@ -1,4 +1,5 @@ """Terminal utilities.""" + from mode.utils.compat import isatty from .spinners import Spinner, SpinnerHandler diff --git a/faust/utils/terminal/spinners.py b/faust/utils/terminal/spinners.py index 9a49c73b4..f210659f0 100644 --- a/faust/utils/terminal/spinners.py +++ b/faust/utils/terminal/spinners.py @@ -1,4 +1,5 @@ """Terminal progress bar spinners.""" + import atexit import logging import random diff --git a/faust/utils/terminal/tables.py b/faust/utils/terminal/tables.py index e37b98f3a..dc0dacf26 100644 --- a/faust/utils/terminal/tables.py +++ b/faust/utils/terminal/tables.py @@ -1,4 +1,5 @@ """Using :pypi:`terminaltables` to draw ANSI tables.""" + import sys from operator import itemgetter from typing import ( diff --git a/faust/utils/tracing.py b/faust/utils/tracing.py index f942f6ba3..9546aef9f 100644 --- a/faust/utils/tracing.py +++ b/faust/utils/tracing.py @@ -1,4 +1,5 @@ """OpenTracing utilities.""" + import asyncio import sys import typing diff --git a/faust/utils/urls.py b/faust/utils/urls.py index cad37c863..131a6c3f0 100644 --- a/faust/utils/urls.py +++ b/faust/utils/urls.py @@ -1,4 +1,5 @@ """URL utilities - Working with URLs.""" + from typing import List, Optional, Union from yarl import URL diff --git a/faust/utils/venusian.py b/faust/utils/venusian.py index e8f7e25b2..38397c4a8 100644 --- a/faust/utils/venusian.py +++ b/faust/utils/venusian.py @@ -3,6 +3,7 @@ We define our own interface so we don't have to specify the callback argument. """ + from typing import Any, Callable import venusian @@ -26,5 +27,4 @@ def attach( return _attach(fun, callback, category=category, **kwargs) -def _on_found(scanner: venusian.Scanner, name: str, obj: Any) -> None: - ... +def _on_found(scanner: venusian.Scanner, name: str, obj: Any) -> None: ... diff --git a/faust/web/__init__.py b/faust/web/__init__.py index 3d6569027..9e4902307 100644 --- a/faust/web/__init__.py +++ b/faust/web/__init__.py @@ -1,4 +1,5 @@ """Faust Web.""" + from faust.types.web import ResourceOptions from .base import Request, Response, Web diff --git a/faust/web/apps/graph.py b/faust/web/apps/graph.py index 01949bb17..4527101a2 100644 --- a/faust/web/apps/graph.py +++ b/faust/web/apps/graph.py @@ -1,4 +1,5 @@ """Web endpoint showing graph of running :pypi:`mode` services.""" + import io from faust import web diff --git a/faust/web/apps/production_index.py b/faust/web/apps/production_index.py index 7d67ef6ba..4c27be29f 100644 --- a/faust/web/apps/production_index.py +++ b/faust/web/apps/production_index.py @@ -1,4 +1,5 @@ """HTTP endpoint showing statistics from the Faust monitor.""" + from typing import List, MutableMapping from faust import web diff --git a/faust/web/apps/router.py b/faust/web/apps/router.py index d1897cbd2..a9899b0d2 100644 --- a/faust/web/apps/router.py +++ b/faust/web/apps/router.py @@ -1,4 +1,5 @@ """HTTP endpoint showing partition routing destinations.""" + from faust import web from faust.web.exceptions import ServiceUnavailable diff --git a/faust/web/apps/stats.py b/faust/web/apps/stats.py index b178912f0..a7974ab13 100644 --- a/faust/web/apps/stats.py +++ b/faust/web/apps/stats.py @@ -1,4 +1,5 @@ """HTTP endpoint showing statistics from the Faust monitor.""" + from collections import defaultdict from typing import List, MutableMapping, Set diff --git a/faust/web/apps/tables.py b/faust/web/apps/tables.py index 654ecc6dd..32344ddcb 100644 --- a/faust/web/apps/tables.py +++ b/faust/web/apps/tables.py @@ -1,4 +1,5 @@ """HTTP endpoint showing partition routing destinations.""" + from typing import Any, Mapping, cast from faust import web diff --git a/faust/web/base.py b/faust/web/base.py index 809b1a065..f62fb2955 100644 --- a/faust/web/base.py +++ b/faust/web/base.py @@ -1,4 +1,5 @@ """Base interface for Web server and views.""" + import abc import socket from datetime import datetime diff --git a/faust/web/blueprints.py b/faust/web/blueprints.py index ed24689d9..7b69c58ec 100644 --- a/faust/web/blueprints.py +++ b/faust/web/blueprints.py @@ -51,6 +51,7 @@ async def get(self, Blueprints can be registered to multiple apps at the same time. """ + from pathlib import Path from typing import List, Mapping, NamedTuple, Optional, Type, Union diff --git a/faust/web/cache/__init__.py b/faust/web/cache/__init__.py index e23ab3e66..864aa56da 100644 --- a/faust/web/cache/__init__.py +++ b/faust/web/cache/__init__.py @@ -1,4 +1,5 @@ """Caching.""" + from .cache import Cache __all__ = ["Cache"] diff --git a/faust/web/cache/backends/__init__.py b/faust/web/cache/backends/__init__.py index 3d5260977..a12e92915 100644 --- a/faust/web/cache/backends/__init__.py +++ b/faust/web/cache/backends/__init__.py @@ -1,4 +1,5 @@ """Cache backend registry.""" + from typing import Type from mode.utils.imports import FactoryMapping diff --git a/faust/web/cache/backends/base.py b/faust/web/cache/backends/base.py index 5cd18195d..06b87ec8e 100644 --- a/faust/web/cache/backends/base.py +++ b/faust/web/cache/backends/base.py @@ -1,4 +1,5 @@ """Cache backend - base implementation.""" + import abc from typing import Any, ClassVar, Optional, Tuple, Type, Union @@ -39,18 +40,15 @@ def __init__( Service.__init__(self, **kwargs) @abc.abstractmethod - async def _get(self, key: str) -> Optional[bytes]: - ... + async def _get(self, key: str) -> Optional[bytes]: ... @abc.abstractmethod async def _set( self, key: str, value: bytes, timeout: Optional[float] = None - ) -> None: - ... + ) -> None: ... @abc.abstractmethod - async def _delete(self, key: str) -> None: - ... + async def _delete(self, key: str) -> None: ... async def get(self, key: str) -> Optional[bytes]: """Get cached-value by key.""" diff --git a/faust/web/cache/backends/memory.py b/faust/web/cache/backends/memory.py index 94b8be391..4c34f0588 100644 --- a/faust/web/cache/backends/memory.py +++ b/faust/web/cache/backends/memory.py @@ -1,4 +1,5 @@ """In-memory cache backend.""" + import sys import time from contextlib import suppress diff --git a/faust/web/cache/backends/redis.py b/faust/web/cache/backends/redis.py index 3e664084c..3c27c1f26 100644 --- a/faust/web/cache/backends/redis.py +++ b/faust/web/cache/backends/redis.py @@ -1,4 +1,5 @@ """Redis cache backend.""" + import socket import typing from enum import Enum @@ -23,8 +24,7 @@ from aredis import StrictRedis as _RedisClientT else: - class _RedisClientT: - ... # noqa + class _RedisClientT: ... # noqa class RedisScheme(Enum): diff --git a/faust/web/cache/cache.py b/faust/web/cache/cache.py index 00aaacfdf..c2de50b75 100644 --- a/faust/web/cache/cache.py +++ b/faust/web/cache/cache.py @@ -1,4 +1,5 @@ """Cache interface.""" + import hashlib from contextlib import suppress from functools import wraps diff --git a/faust/web/drivers/__init__.py b/faust/web/drivers/__init__.py index 0086a7253..b3e2e2e4a 100644 --- a/faust/web/drivers/__init__.py +++ b/faust/web/drivers/__init__.py @@ -1,4 +1,5 @@ """Web server driver registry.""" + from typing import Type from mode.utils.imports import FactoryMapping diff --git a/faust/web/drivers/aiohttp.py b/faust/web/drivers/aiohttp.py index 389637d75..16c0d4fde 100644 --- a/faust/web/drivers/aiohttp.py +++ b/faust/web/drivers/aiohttp.py @@ -1,4 +1,5 @@ """Web driver using :pypi:`aiohttp`.""" + from pathlib import Path from typing import Any, Callable, Mapping, MutableMapping, Optional, Union, cast diff --git a/faust/web/exceptions.py b/faust/web/exceptions.py index e103d3695..b8f0db86a 100644 --- a/faust/web/exceptions.py +++ b/faust/web/exceptions.py @@ -1,4 +1,5 @@ """HTTP and related errors.""" + import http from typing import Any, Dict, Optional, cast diff --git a/faust/web/views.py b/faust/web/views.py index cad55413d..c4651a492 100644 --- a/faust/web/views.py +++ b/faust/web/views.py @@ -1,4 +1,5 @@ """Class-based views.""" + from functools import wraps from typing import ( Any, diff --git a/faust/windows.py b/faust/windows.py index 6e956f34e..d0944d0eb 100644 --- a/faust/windows.py +++ b/faust/windows.py @@ -1,4 +1,5 @@ """Window Types.""" + import os import typing from math import floor diff --git a/faust/worker.py b/faust/worker.py index 1cdf65444..06533b77e 100644 --- a/faust/worker.py +++ b/faust/worker.py @@ -5,6 +5,7 @@ See Also: :ref:`app-starting`: for more information. """ + import asyncio import logging import os @@ -29,8 +30,7 @@ from setproctitle import setproctitle except ImportError: # pragma: no cover - def setproctitle(title: str) -> None: - ... # noqa + def setproctitle(title: str) -> None: ... # noqa __all__ = ["Worker"] diff --git a/setup.cfg b/setup.cfg index bb412d374..f80025682 100644 --- a/setup.cfg +++ b/setup.cfg @@ -1,5 +1,5 @@ [flake8] -ignore = W503, E203, E231, E225, B305 +ignore = W503, E203, E231, E225, B305, E701, E704 max-line-length = 88 [mypy-tests.*] diff --git a/tests/conftest.py b/tests/conftest.py index 0cd0c4b36..4ce891806 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -16,8 +16,7 @@ sentinel = object() -class DirtyTest(Exception): - ... +class DirtyTest(Exception): ... @pytest.fixture() diff --git a/tests/functional/agents/helpers.py b/tests/functional/agents/helpers.py index ca70c6955..8106873b9 100644 --- a/tests/functional/agents/helpers.py +++ b/tests/functional/agents/helpers.py @@ -97,8 +97,7 @@ async def _wait_for_success(self) -> None: async def assert_success(self) -> None: assert self.processed_total == self.num_messages - async def on_agent_event(self, stream: StreamT, event: EventT) -> None: - ... + async def on_agent_event(self, stream: StreamT, event: EventT) -> None: ... async def process(self, stream: StreamT[bytes]) -> None: self.agent_started.set() diff --git a/tests/functional/test_app.py b/tests/functional/test_app.py index 6c1b7ad57..94ebe8b86 100644 --- a/tests/functional/test_app.py +++ b/tests/functional/test_app.py @@ -35,8 +35,7 @@ TABLEDIR = Path("/var/faust/") -class OtherSchedulingStrategy(DefaultSchedulingStrategy): - ... +class OtherSchedulingStrategy(DefaultSchedulingStrategy): ... def _dummy_partitioner(a, b, c): diff --git a/tests/functional/test_models.py b/tests/functional/test_models.py index 97a655a28..816cb2abd 100644 --- a/tests/functional/test_models.py +++ b/tests/functional/test_models.py @@ -406,8 +406,9 @@ def __json__(self): def __repr__(self): return f"<{type(self).__name__}: {self.value}>" - class CanFooModel(Record, abstract=True, coercions={Foo: Foo}, serializer="json"): - ... + class CanFooModel( + Record, abstract=True, coercions={Foo: Foo}, serializer="json" + ): ... class IsFoo(CanFooModel, serializer="json"): foo: Foo @@ -1118,8 +1119,7 @@ class BaseX(Record): def test_abstract_model_repr(): - class MyBase(faust.Record, abstract=True): - ... + class MyBase(faust.Record, abstract=True): ... assert MyBase.__is_abstract__ with pytest.raises(NotImplementedError): diff --git a/tests/functional/web/test_cache.py b/tests/functional/web/test_cache.py index 92044312a..f91665af8 100644 --- a/tests/functional/web/test_cache.py +++ b/tests/functional/web/test_cache.py @@ -49,8 +49,7 @@ async def get(self, request): @blueprint.route("/C", name="c") -class CCachedView(ACachedView): - ... +class CCachedView(ACachedView): ... @blueprint.route("/D/", name="d") diff --git a/tests/unit/agents/test_agent.py b/tests/unit/agents/test_agent.py index e58d876ee..54f881f68 100644 --- a/tests/unit/agents/test_agent.py +++ b/tests/unit/agents/test_agent.py @@ -203,29 +203,25 @@ def test_init_schema_and_channel(self, *, app): with pytest.raises(AssertionError): @app.agent(app.topic("foo"), schema=faust.Schema(key_type=bytes)) - async def foo(): - ... + async def foo(): ... def test_init_key_type_and_channel(self, *, app): with pytest.raises(AssertionError): @app.agent(app.topic("foo"), key_type=bytes) - async def foo(): - ... + async def foo(): ... def test_init_value_type_and_channel(self, *, app): with pytest.raises(AssertionError): @app.agent(app.topic("foo"), value_type=bytes) - async def foo(): - ... + async def foo(): ... def test_isolated_partitions_cannot_have_concurrency(self, *, app): with pytest.raises(ImproperlyConfigured): @app.agent(isolated_partitions=True, concurrency=100) - async def foo(): - ... + async def foo(): ... def test_agent_call_reuse_stream(self, *, agent, app): stream = app.stream("foo") diff --git a/tests/unit/app/test_base.py b/tests/unit/app/test_base.py index 057668f55..c5405543b 100644 --- a/tests/unit/app/test_base.py +++ b/tests/unit/app/test_base.py @@ -603,8 +603,7 @@ def test_agent(self, *, app): with patch("faust.app.base.venusian") as venusian: @app.agent() - async def foo(stream): - ... + async def foo(stream): ... assert foo.name assert app.agents[foo.name] is foo @@ -639,8 +638,7 @@ def test_task(self, *, app): with patch("faust.app.base.venusian") as venusian: @app.task - async def foo(): - ... + async def foo(): ... venusian.attach.assert_called_once_with(foo, category=SCAN_TASK) assert foo in app._app_tasks @@ -796,8 +794,7 @@ def on_sleep(seconds, **kwargs): def test_service(self, *, app): @app.service - class Foo(Service): - ... + class Foo(Service): ... assert Foo in app._extra_services @@ -831,8 +828,7 @@ def test_page(self, *, app): with patch("faust.app.base.venusian") as venusian: @app.page("/foo") - async def view(self, request): - ... + async def view(self, request): ... assert "/foo" in app.web.views @@ -853,8 +849,7 @@ def test_page__with_cors_options(self, *, app): ), }, ) - async def view(self, request): - ... + async def view(self, request): ... assert "/foo" in app.web.views @@ -864,8 +859,7 @@ def test_page__view_class_but_not_view(self, *, app): with pytest.raises(TypeError): @app.page("/foo") - class Foo: - ... + class Foo: ... @pytest.mark.asyncio async def test_table_route__query_param(self, *, app): @@ -930,8 +924,7 @@ def test_table_route__compat_shard_param(self, *, app): with pytest.warns(DeprecationWarning): @app.table_route(table, shard_param="x") - async def view(self, request): - ... + async def view(self, request): ... def test_table_route__query_param_and_shard_param(self, *, app): table = app.Table("foo") @@ -939,16 +932,14 @@ def test_table_route__query_param_and_shard_param(self, *, app): with pytest.raises(TypeError): @app.table_route(table, query_param="q", shard_param="x") - async def view(self, request): - ... + async def view(self, request): ... def test_table_route__missing_param(self, *, app): table = app.Table("foo") with pytest.raises(TypeError): @app.table_route(table) - async def view(self, request): - ... + async def view(self, request): ... @pytest.mark.asyncio async def test_topic_route__query_param(self, *, app): @@ -1013,8 +1004,7 @@ def test_topic_route__compat_shard_param(self, *, app): with pytest.warns(DeprecationWarning): @app.topic_route(topic, shard_param="x") - async def view(self, request): - ... + async def view(self, request): ... def test_topic_route__query_param_and_shard_param(self, *, app): topic = app.topic("foo") @@ -1022,29 +1012,24 @@ def test_topic_route__query_param_and_shard_param(self, *, app): with pytest.raises(TypeError): @app.topic_route(topic, query_param="q", shard_param="x") - async def view(self, request): - ... + async def view(self, request): ... def test_topic_route__missing_param(self, *, app): topic = app.topic("foo") with pytest.raises(TypeError): @app.topic_route(topic) - async def view(self, request): - ... + async def view(self, request): ... def test_command(self, *, app): @app.command() - async def foo(): - ... + async def foo(): ... def test_command__with_base(self, *, app): - class MyBase(AppCommand): - ... + class MyBase(AppCommand): ... @app.command(base=MyBase) - async def foo(): - ... + async def foo(): ... assert issubclass(foo, MyBase) diff --git a/tests/unit/app/test_service.py b/tests/unit/app/test_service.py index 377e372e3..7348b5756 100644 --- a/tests/unit/app/test_service.py +++ b/tests/unit/app/test_service.py @@ -6,8 +6,7 @@ from tests.helpers import AsyncMock -class OtherService(Service): - ... +class OtherService(Service): ... class Test_AppService: diff --git a/tests/unit/cli/test_base.py b/tests/unit/cli/test_base.py index d2c1ac9d9..5fd3aeab0 100644 --- a/tests/unit/cli/test_base.py +++ b/tests/unit/cli/test_base.py @@ -650,9 +650,7 @@ def test_abbreviate_fqdn__no_origin(self, *, command, ctx): def test_from_handler_no_params(self, *, command): @command.from_handler() - async def takes_no_args(): - ... + async def takes_no_args(): ... @command.from_handler() - async def takes_self_arg(self): - ... + async def takes_self_arg(self): ... diff --git a/tests/unit/serializers/test_codecs.py b/tests/unit/serializers/test_codecs.py index 6abf80538..9e9aa5731 100644 --- a/tests/unit/serializers/test_codecs.py +++ b/tests/unit/serializers/test_codecs.py @@ -81,8 +81,7 @@ def test_get_codec(): def test_register(): try: - class MyCodec(Codec): - ... + class MyCodec(Codec): ... register("mine", MyCodec) assert get_codec("mine") is MyCodec diff --git a/tests/unit/stores/test_aerospike.py b/tests/unit/stores/test_aerospike.py index ebb0ee866..7f74d44c0 100644 --- a/tests/unit/stores/test_aerospike.py +++ b/tests/unit/stores/test_aerospike.py @@ -10,8 +10,7 @@ from aerospike.exception import RecordNotFound except ImportError: - class RecordNotFound(Exception): - ... + class RecordNotFound(Exception): ... m1 = MagicMock() m2 = MagicMock() diff --git a/tests/unit/stores/test_base.py b/tests/unit/stores/test_base.py index 160b19952..5f47fcb44 100644 --- a/tests/unit/stores/test_base.py +++ b/tests/unit/stores/test_base.py @@ -9,39 +9,30 @@ class MyStore(Store): - def __getitem__(self, key): - ... + def __getitem__(self, key): ... - def __setitem__(self, key, value): - ... + def __setitem__(self, key, value): ... - def __delitem__(self, key): - ... + def __delitem__(self, key): ... - def __iter__(self): - ... + def __iter__(self): ... - def __len__(self): - ... + def __len__(self): ... - def apply_changelog_batch(self, *args, **kwargs): - ... + def apply_changelog_batch(self, *args, **kwargs): ... - def reset_state(self): - ... + def reset_state(self): ... async def backup_partition( self, tp, flush: bool = True, purge: bool = False, keep: int = 1 - ) -> None: - ... + ) -> None: ... def restore_backup( self, tp, latest: bool = True, backup_id: int = 0, - ) -> None: - ... + ) -> None: ... class Test_Store: @@ -131,21 +122,18 @@ def _contains(self, key): def _clear(self): self.keep.clear() - def reset_state(self): - ... + def reset_state(self): ... async def backup_partition( self, tp, flush: bool = True, purge: bool = False, keep: int = 1 - ) -> None: - ... + ) -> None: ... def restore_backup( self, tp, latest: bool = True, backup_id: int = 0, - ) -> None: - ... + ) -> None: ... class Test_SerializedStore: diff --git a/tests/unit/transport/drivers/test_aiokafka.py b/tests/unit/transport/drivers/test_aiokafka.py index b524aac23..63a55d5d2 100644 --- a/tests/unit/transport/drivers/test_aiokafka.py +++ b/tests/unit/transport/drivers/test_aiokafka.py @@ -1288,8 +1288,7 @@ def assert_calls_thread(self, cthread, _consumer, method, *args, **kwargs): cthread.call_thread.assert_called_once_with(method, *args, **kwargs) -class MyPartitioner: - ... +class MyPartitioner: ... my_partitioner = MyPartitioner() diff --git a/tests/unit/transport/test_consumer.py b/tests/unit/transport/test_consumer.py index 89c33ae21..070066984 100644 --- a/tests/unit/transport/test_consumer.py +++ b/tests/unit/transport/test_consumer.py @@ -378,53 +378,39 @@ class MockedConsumerAbstractMethods: def assignment(self): return self.current_assignment - def position(self, *args, **kwargs): - ... + def position(self, *args, **kwargs): ... - async def create_topic(self, *args, **kwargs): - ... + async def create_topic(self, *args, **kwargs): ... - def earliest_offsets(self, *args, **kwargs): - ... + def earliest_offsets(self, *args, **kwargs): ... - def highwater(self, *args, **kwargs): - ... + def highwater(self, *args, **kwargs): ... - def highwaters(self, *args, **kwargs): - ... + def highwaters(self, *args, **kwargs): ... - async def _getmany(self, *args, **kwargs): - ... + async def _getmany(self, *args, **kwargs): ... - async def _seek(self, *args, **kwargs): - ... + async def _seek(self, *args, **kwargs): ... - def _to_message(self, *args, **kwargs): - ... + def _to_message(self, *args, **kwargs): ... async def seek_to_committed(self, *args, **kwargs): return {} - async def seek_wait(self, *args, **kwargs): - ... + async def seek_wait(self, *args, **kwargs): ... - async def subscribe(self, *args, **kwargs): - ... + async def subscribe(self, *args, **kwargs): ... - async def seek_to_beginning(self, *args, **kwargs): - ... + async def seek_to_beginning(self, *args, **kwargs): ... - async def _commit(self, offsets) -> bool: - ... + async def _commit(self, offsets) -> bool: ... - def topic_partitions(self, topic): - ... + def topic_partitions(self, topic): ... def _new_topicpartition(self, topic, partition) -> TP: return TP(topic, partition) - def key_partition(self, *args, **kwargs): - ... + def key_partition(self, *args, **kwargs): ... class MyConsumer(MockedConsumerAbstractMethods, Consumer): @@ -1168,32 +1154,24 @@ def test_close(self, *, consumer): class Test_ConsumerThread: class MyConsumerThread(MockedConsumerAbstractMethods, ConsumerThread): - def close(self): - ... + def close(self): ... async def getmany(self, *args, **kwargs): yield None, None - def pause_partitions(self, *args, **kwargs): - ... + def pause_partitions(self, *args, **kwargs): ... - def resume_partitions(self, *args, **kwargs): - ... + def resume_partitions(self, *args, **kwargs): ... - def stop_flow(self, *args, **kwargs): - ... + def stop_flow(self, *args, **kwargs): ... - def resume_flow(self, *args, **kwargs): - ... + def resume_flow(self, *args, **kwargs): ... - async def commit(self, *args, **kwargs): - ... + async def commit(self, *args, **kwargs): ... - async def perform_seek(self, *args, **kwargs): - ... + async def perform_seek(self, *args, **kwargs): ... - async def seek(self, *args, **kwargs): - ... + async def seek(self, *args, **kwargs): ... @pytest.fixture def consumer(self): diff --git a/tests/unit/web/test_base.py b/tests/unit/web/test_base.py index b38588440..fda2c595c 100644 --- a/tests/unit/web/test_base.py +++ b/tests/unit/web/test_base.py @@ -54,35 +54,25 @@ def test_apply(self, *, manager): class MyWeb(Web): - def text(self, *args, **kwargs): - ... + def text(self, *args, **kwargs): ... - def html(self, *args, **kwargs): - ... + def html(self, *args, **kwargs): ... - def json(self, *args, **kwargs): - ... + def json(self, *args, **kwargs): ... - def bytes(self, *args, **kwargs): - ... + def bytes(self, *args, **kwargs): ... - def bytes_to_response(self, *args, **kwargs): - ... + def bytes_to_response(self, *args, **kwargs): ... - def response_to_bytes(self, *args, **kwargs): - ... + def response_to_bytes(self, *args, **kwargs): ... - def route(self, *args, **kwargs): - ... + def route(self, *args, **kwargs): ... - def add_static(self, *args, **kwargs): - ... + def add_static(self, *args, **kwargs): ... - async def read_request_content(self, *args, **kwargs): - ... + async def read_request_content(self, *args, **kwargs): ... - async def wsgi(self, *args, **kwargs): - ... + async def wsgi(self, *args, **kwargs): ... class Test_Web: From ebf66ae031c3eb462ade320c73e84d1c4cb7a32f Mon Sep 17 00:00:00 2001 From: Timothy Thomas Date: Fri, 16 Feb 2024 14:43:56 -0600 Subject: [PATCH 45/54] bugfix: Fix cython stream_event_in to match python impl (#608) Co-authored-by: William Barnhart --- faust/_cython/streams.pyx | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/faust/_cython/streams.pyx b/faust/_cython/streams.pyx index 42736c6f6..42e06bc28 100644 --- a/faust/_cython/streams.pyx +++ b/faust/_cython/streams.pyx @@ -179,8 +179,8 @@ cdef class StreamIterator: self.add_unacked(message) self.on_message_in(tp, offset, message) - stream_state = self.on_stream_event_in( - tp, offset, self.stream, event) + stream_state = self.on_stream_event_in( + tp, offset, self.stream, event) self.stream._set_current_event(event) return (event, event.value, stream_state) else: From 5330c45c7fb9855b43b2f711ca9aa0ce2bdbbb06 Mon Sep 17 00:00:00 2001 From: Zev Isert Date: Fri, 1 Mar 2024 09:19:59 -0800 Subject: [PATCH 46/54] fix: serialize (named)tuple when using orjson (#612) --- faust/utils/json.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/faust/utils/json.py b/faust/utils/json.py index 8fb13a386..29db0d7d1 100644 --- a/faust/utils/json.py +++ b/faust/utils/json.py @@ -69,7 +69,7 @@ DECIMAL_MAXLEN = 1000 #: Types that we convert to lists. -SEQUENCE_TYPES: TypeTuple[Iterable] = (set, frozenset, deque) +SEQUENCE_TYPES: TypeTuple[Iterable] = (set, frozenset, deque, tuple) DateTypeTuple = Tuple[Union[Type[datetime.date], Type[datetime.time]], ...] DatetimeTypeTuple = Tuple[ From 34ae8ec7ee942ea92166428b5c358d2e6a388f1c Mon Sep 17 00:00:00 2001 From: Krzysztof Sulejczak <59609133+krzysieksulejczak@users.noreply.github.com> Date: Mon, 4 Mar 2024 16:13:23 +0100 Subject: [PATCH 47/54] Fix prometheus registry (#613) * Use registry passed to `setup_prometheus_sensors` to generate metric view * refactor: import `web` once --- faust/sensors/prometheus.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/faust/sensors/prometheus.py b/faust/sensors/prometheus.py index 022dea146..f3c497697 100644 --- a/faust/sensors/prometheus.py +++ b/faust/sensors/prometheus.py @@ -5,7 +5,7 @@ from aiohttp.web import Response -from faust import web, web as _web +from faust import web from faust.exceptions import ImproperlyConfigured from faust.types import ( TP, @@ -68,12 +68,12 @@ def setup_prometheus_sensors( app.monitor = PrometheusMonitor(metrics=faust_metrics) @app.page(pattern) - async def metrics_handler(self: _web.View, request: _web.Request) -> _web.Response: + async def metrics_handler(self: web.View, request: web.Request) -> web.Response: headers = {"Content-Type": CONTENT_TYPE_LATEST} return cast( - _web.Response, - Response(body=generate_latest(REGISTRY), headers=headers, status=200), + web.Response, + Response(body=generate_latest(registry), headers=headers, status=200), ) From 11505bca28dc5b7d274c2c8e73ea3c0b8cba6289 Mon Sep 17 00:00:00 2001 From: William Barnhart Date: Sat, 16 Mar 2024 20:52:18 -0400 Subject: [PATCH 48/54] Rollback upload and download artifacts to v3 in python-package.yml --- .github/workflows/python-package.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 03d5642b1..714209c7d 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -119,7 +119,7 @@ jobs: run: > pip3 install pkgconfig cython --upgrade && python3 setup.py sdist - - uses: actions/upload-artifact@v4 + - uses: actions/upload-artifact@v3 name: Upload build artifacts with: path: dist/*.tar.gz @@ -133,7 +133,7 @@ jobs: if: github.event_name == 'release' && github.event.action == 'created' steps: - name: Download the sdist artifact - uses: actions/download-artifact@v4 + uses: actions/download-artifact@v3 with: name: artifact path: dist From ed85356874680727695b8c0ccc8130789906a1dc Mon Sep 17 00:00:00 2001 From: William Barnhart Date: Fri, 29 Mar 2024 19:22:28 -0400 Subject: [PATCH 49/54] Fix imports from updates in mode-streaming~=0.4.0 (#618) * Fix imports from updates in mode-streaming * Require new version of mode-streaming --- docs/userguide/testing.rst | 10 +++++----- faust/agents/manager.py | 3 +-- faust/app/base.py | 4 ++-- faust/assignor/copartitioned_assignor.py | 4 +--- faust/cli/base.py | 2 +- faust/livecheck/case.py | 18 ++++++++++++++---- faust/livecheck/runners.py | 3 +-- faust/models/record.py | 2 +- faust/models/typing.py | 2 +- faust/sensors/monitor.py | 3 ++- faust/streams.py | 2 +- faust/tables/recovery.py | 3 ++- faust/tables/wrappers.py | 2 +- faust/transport/drivers/aiokafka.py | 2 +- faust/transport/utils.py | 3 +-- faust/types/app.py | 2 +- faust/types/events.py | 13 ++++++++++--- faust/web/cache/backends/base.py | 5 ++--- requirements/requirements.txt | 2 +- tests/unit/test_streams.py | 2 +- 20 files changed, 50 insertions(+), 37 deletions(-) diff --git a/docs/userguide/testing.rst b/docs/userguide/testing.rst index 05bb27c9f..ea0ed4baa 100644 --- a/docs/userguide/testing.rst +++ b/docs/userguide/testing.rst @@ -129,7 +129,7 @@ first test ``foo`` with ``bar`` mocked, then in a different test do ``bar``: @pytest.mark.asyncio() async def test_foo(test_app): with patch(__name__ + '.bar') as mocked_bar: - mocked_bar.send = mock_coro() + mocked_bar.send = mock_coro() async with foo.test_context() as agent: await agent.put('hey') mocked_bar.send.assert_called_with('hey') @@ -145,8 +145,8 @@ first test ``foo`` with ``bar`` mocked, then in a different test do ``bar``: async with bar.test_context() as agent: event = await agent.put('hey') assert agent.results[event.message.offset] == 'heyYOLO' - - + + You can put the `test_app` fixture into a [`conftest.py` file](https://docs.pytest.org/en/6.2.x/fixture.html#scope-sharing-fixtures-across-classes-modules-packages-or-session). If the fixture is not in the same file as the app's definition (which should be the case) you must import the app the fixture definition: .. sourcecode:: python @@ -155,9 +155,9 @@ You can put the `test_app` fixture into a [`conftest.py` file](https://docs.pyte @pytest.fixture(scope="function") def test_app(event_loop): """passing in event_loop helps avoid 'attached to a different loop' error""" - + from example import app - + app.loop = event_loop app.finalize() app.conf.store = 'memory://' diff --git a/faust/agents/manager.py b/faust/agents/manager.py index b1e164383..1336be522 100644 --- a/faust/agents/manager.py +++ b/faust/agents/manager.py @@ -1,13 +1,12 @@ """Agent manager.""" import asyncio -from collections import defaultdict +from collections import OrderedDict, defaultdict from typing import Any, Dict, List, Mapping, MutableMapping, MutableSet, Set from weakref import WeakSet from mode import Service from mode.utils.collections import ManagedUserDict -from mode.utils.compat import OrderedDict from mode.utils.locks import Event from faust.types import AgentManagerT, AgentT, AppT diff --git a/faust/app/base.py b/faust/app/base.py index 5205e3968..9bb15a18d 100644 --- a/faust/app/base.py +++ b/faust/app/base.py @@ -12,6 +12,7 @@ import sys import typing import warnings +from contextlib import nullcontext from datetime import tzinfo from functools import wraps from itertools import chain @@ -29,6 +30,7 @@ Mapping, MutableMapping, MutableSequence, + NoReturn, Optional, Pattern, Set, @@ -44,14 +46,12 @@ from mode import Seconds, Service, ServiceT, SupervisorStrategyT, want_seconds from mode.utils.aiter import aiter from mode.utils.collections import force_mapping -from mode.utils.contexts import nullcontext from mode.utils.futures import stampede from mode.utils.imports import import_from_cwd, smart_import from mode.utils.logging import flight_recorder, get_logger from mode.utils.objects import cached_property, qualname, shortlabel from mode.utils.queues import FlowControlEvent, ThrowableQueue from mode.utils.types.trees import NodeT -from mode.utils.typing import NoReturn from faust import transport from faust.agents import AgentFun, AgentManager, AgentT, ReplyConsumer, SinkT diff --git a/faust/assignor/copartitioned_assignor.py b/faust/assignor/copartitioned_assignor.py index 093781afd..ada471793 100644 --- a/faust/assignor/copartitioned_assignor.py +++ b/faust/assignor/copartitioned_assignor.py @@ -2,9 +2,7 @@ from itertools import cycle from math import ceil, floor -from typing import Iterable, Iterator, MutableMapping, Optional, Sequence, Set - -from mode.utils.typing import Counter +from typing import Counter, Iterable, Iterator, MutableMapping, Optional, Sequence, Set from .client_assignment import CopartitionedAssignment diff --git a/faust/cli/base.py b/faust/cli/base.py index 5715176e8..8e57b6485 100644 --- a/faust/cli/base.py +++ b/faust/cli/base.py @@ -21,6 +21,7 @@ List, Mapping, MutableSequence, + NoReturn, Optional, Sequence, Tuple, @@ -36,7 +37,6 @@ from mode.utils import text from mode.utils.compat import want_bytes from mode.utils.imports import import_from_cwd, symbol_by_name -from mode.utils.typing import NoReturn from mode.worker import exiting from faust.types import AppT, CodecArg, ModelT diff --git a/faust/livecheck/case.py b/faust/livecheck/case.py index 5c8cc4d68..771d89050 100644 --- a/faust/livecheck/case.py +++ b/faust/livecheck/case.py @@ -3,19 +3,29 @@ import traceback import typing from collections import deque -from contextlib import ExitStack +from contextlib import ExitStack, asynccontextmanager from datetime import datetime, timedelta, timezone from itertools import count from random import uniform from statistics import median from time import monotonic -from typing import Any, ClassVar, Dict, Iterable, Optional, Type, Union, cast +from typing import ( + Any, + AsyncGenerator, + ClassVar, + Counter, + Deque, + Dict, + Iterable, + Optional, + Type, + Union, + cast, +) from aiohttp import ClientError, ClientTimeout from mode import Seconds, Service, want_seconds -from mode.utils.contexts import asynccontextmanager from mode.utils.times import humanize_seconds -from mode.utils.typing import AsyncGenerator, Counter, Deque from yarl import URL from faust.utils import uuid diff --git a/faust/livecheck/runners.py b/faust/livecheck/runners.py index a1d26fb1d..2e6594e1a 100644 --- a/faust/livecheck/runners.py +++ b/faust/livecheck/runners.py @@ -5,11 +5,10 @@ import traceback import typing from time import monotonic -from typing import Any, Dict, Iterable, List, Mapping, Optional, Tuple +from typing import Any, Dict, Iterable, List, Mapping, NoReturn, Optional, Tuple from mode.utils.logging import CompositeLogger from mode.utils.times import humanize_seconds -from mode.utils.typing import NoReturn from faust.models import maybe_model diff --git a/faust/models/record.py b/faust/models/record.py index dbfd65a5e..032abd607 100644 --- a/faust/models/record.py +++ b/faust/models/record.py @@ -1,5 +1,6 @@ """Record - Dictionary Model.""" +from collections import OrderedDict from datetime import datetime from decimal import Decimal from itertools import chain @@ -18,7 +19,6 @@ cast, ) -from mode.utils.compat import OrderedDict from mode.utils.objects import annotations, is_optional, remove_optional from mode.utils.text import pluralize diff --git a/faust/models/typing.py b/faust/models/typing.py index dc9ff0d88..030d5ecb1 100644 --- a/faust/models/typing.py +++ b/faust/models/typing.py @@ -23,6 +23,7 @@ Any, Callable, ClassVar, + Counter, Dict, Iterator, List, @@ -45,7 +46,6 @@ is_union, qualname, ) -from mode.utils.typing import Counter from faust.types.models import CoercionHandler, CoercionMapping, IsInstanceArgT, ModelT from faust.utils import codegen diff --git a/faust/sensors/monitor.py b/faust/sensors/monitor.py index 44d7da1e1..b37a0f8a4 100644 --- a/faust/sensors/monitor.py +++ b/faust/sensors/monitor.py @@ -10,6 +10,8 @@ from typing import ( Any, Callable, + Counter, + Deque, Dict, Mapping, MutableMapping, @@ -21,7 +23,6 @@ from mode import Service, label from mode.utils.objects import KeywordReduce -from mode.utils.typing import Counter, Deque from faust import web from faust.types import AppT, CollectionT, EventT, StreamT diff --git a/faust/streams.py b/faust/streams.py index d2faf482a..9f4286a3f 100644 --- a/faust/streams.py +++ b/faust/streams.py @@ -12,6 +12,7 @@ AsyncIterable, AsyncIterator, Callable, + Deque, Dict, Iterable, Iterator, @@ -32,7 +33,6 @@ from mode.utils.futures import current_task, maybe_async, notify from mode.utils.queues import ThrowableQueue from mode.utils.types.trees import NodeT -from mode.utils.typing import Deque from . import joins from .exceptions import ImproperlyConfigured, Skip diff --git a/faust/tables/recovery.py b/faust/tables/recovery.py index 41bb20c18..11737aef0 100644 --- a/faust/tables/recovery.py +++ b/faust/tables/recovery.py @@ -8,6 +8,8 @@ from time import monotonic from typing import ( Any, + Counter, + Deque, Iterator, List, Mapping, @@ -23,7 +25,6 @@ from mode import Service, get_logger from mode.services import WaitArgT from mode.utils.times import humanize_seconds, humanize_seconds_ago -from mode.utils.typing import Counter, Deque from yarl import URL from faust.exceptions import ConsistencyError diff --git a/faust/tables/wrappers.py b/faust/tables/wrappers.py index 0650c59e6..bab3d72a0 100644 --- a/faust/tables/wrappers.py +++ b/faust/tables/wrappers.py @@ -10,6 +10,7 @@ ItemsView, Iterator, KeysView, + NoReturn, Optional, Tuple, Type, @@ -18,7 +19,6 @@ ) from mode import Seconds -from mode.utils.typing import NoReturn from faust.exceptions import ImproperlyConfigured from faust.streams import current_event diff --git a/faust/transport/drivers/aiokafka.py b/faust/transport/drivers/aiokafka.py index f69c532db..fe189d9d3 100644 --- a/faust/transport/drivers/aiokafka.py +++ b/faust/transport/drivers/aiokafka.py @@ -11,6 +11,7 @@ Awaitable, Callable, ClassVar, + Deque, Iterable, List, Mapping, @@ -49,7 +50,6 @@ from mode.utils.futures import StampedeWrapper from mode.utils.objects import cached_property from mode.utils.times import Seconds, humanize_seconds_ago, want_seconds -from mode.utils.typing import Deque from opentracing.ext import tags from yarl import URL diff --git a/faust/transport/utils.py b/faust/transport/utils.py index 0b0c7b213..dc0b8d7d0 100644 --- a/faust/transport/utils.py +++ b/faust/transport/utils.py @@ -1,5 +1,6 @@ """Transport utils - scheduling.""" +from collections import OrderedDict from typing import ( Any, Dict, @@ -12,8 +13,6 @@ Tuple, ) -from mode.utils.compat import OrderedDict - from faust.types import TP from faust.types.transports import SchedulingStrategyT diff --git a/faust/types/app.py b/faust/types/app.py index c62858dcb..adb336035 100644 --- a/faust/types/app.py +++ b/faust/types/app.py @@ -12,6 +12,7 @@ Iterable, Mapping, MutableSequence, + NoReturn, Optional, Pattern, Set, @@ -28,7 +29,6 @@ from mode.utils.objects import cached_property from mode.utils.queues import FlowControlEvent, ThrowableQueue from mode.utils.types.trees import NodeT -from mode.utils.typing import NoReturn from .agents import AgentFun, AgentManagerT, AgentT, SinkT from .assignor import PartitionAssignorT diff --git a/faust/types/events.py b/faust/types/events.py index 11061e07b..9b1498609 100644 --- a/faust/types/events.py +++ b/faust/types/events.py @@ -1,8 +1,15 @@ import abc import typing -from typing import Any, Awaitable, Generic, Mapping, Optional, TypeVar, Union - -from mode.utils.typing import AsyncContextManager +from typing import ( + Any, + AsyncContextManager, + Awaitable, + Generic, + Mapping, + Optional, + TypeVar, + Union, +) from .codecs import CodecArg from .core import HeadersArg, K, V diff --git a/faust/web/cache/backends/base.py b/faust/web/cache/backends/base.py index 06b87ec8e..a0cbf2ddf 100644 --- a/faust/web/cache/backends/base.py +++ b/faust/web/cache/backends/base.py @@ -1,12 +1,11 @@ """Cache backend - base implementation.""" import abc -from typing import Any, ClassVar, Optional, Tuple, Type, Union +from contextlib import asynccontextmanager +from typing import Any, AsyncGenerator, ClassVar, Optional, Tuple, Type, Union from mode import Service -from mode.utils.contexts import asynccontextmanager from mode.utils.logging import get_logger -from mode.utils.typing import AsyncGenerator from yarl import URL from faust.types import AppT diff --git a/requirements/requirements.txt b/requirements/requirements.txt index cdd50fedd..5b7f71c9b 100644 --- a/requirements/requirements.txt +++ b/requirements/requirements.txt @@ -2,7 +2,7 @@ aiohttp>=3.8.0,<4.0 aiohttp_cors>=0.7,<2.0 aiokafka>=0.9.0 click>=6.7,<8.2 -mode-streaming>=0.3.0 +mode-streaming>=0.4.0 opentracing>=1.3.0,<=2.4.0 terminaltables>=3.1,<4.0 yarl>=1.0,<2.0 diff --git a/tests/unit/test_streams.py b/tests/unit/test_streams.py index 1a9c03035..4b7af428d 100644 --- a/tests/unit/test_streams.py +++ b/tests/unit/test_streams.py @@ -1,9 +1,9 @@ import asyncio from collections import defaultdict +from contextlib import ExitStack from unittest.mock import Mock, patch import pytest -from mode.utils.contexts import ExitStack import faust from faust import joins From adcbc81880441fd15dacd7f9ebd751f74584ce8d Mon Sep 17 00:00:00 2001 From: William Barnhart Date: Fri, 29 Mar 2024 20:13:20 -0400 Subject: [PATCH 50/54] Pull in hopping window table fix (#412) * robinhood issues 514 * add hopping example * fix: 'Nonetype' object is not iterable problem * lint * remove unused var * add mock ranges to table tests * Pull in changes by @thomas-chauvet * save the popped value as a backup for now * cleanup and add more tests * test for ranges when full and empty * add linting * remove MagicMock import --------- Co-authored-by: Don Wong --- faust/tables/base.py | 33 ++++++- tests/unit/tables/test_base.py | 157 +++++++++++++++++++++++++++++++++ 2 files changed, 187 insertions(+), 3 deletions(-) diff --git a/faust/tables/base.py b/faust/tables/base.py index e3574ee21..ca26c0f11 100644 --- a/faust/tables/base.py +++ b/faust/tables/base.py @@ -382,13 +382,40 @@ async def _del_old_keys(self) -> None: for partition, timestamps in self._partition_timestamps.items(): while timestamps and window.stale(timestamps[0], time.time()): timestamp = heappop(timestamps) + triggered_windows = [ + self._partition_timestamp_keys.get( + (partition, window_range) + ) # noqa + for window_range in self._window_ranges(timestamp) + ] keys_to_remove = self._partition_timestamp_keys.pop( (partition, timestamp), None ) + window_data = {} if keys_to_remove: - for key in keys_to_remove: - value = self.data.pop(key, None) - await self.on_window_close(key, value) + for windows in triggered_windows: + if windows: + for processed_window in windows: + # we use set to avoid duplicate element in window's data + # window[0] is the window's key + # it is not related to window's timestamp + # windows are in format: + # (key, (window_start, window_end)) + window_data.setdefault(processed_window[0], []).extend( + self.data.get(processed_window, []) + ) + + for key_to_remove in keys_to_remove: + value = self.data.pop(key_to_remove, None) + if key_to_remove[1][0] > self.last_closed_window: + await self.on_window_close( + key_to_remove, + ( + window_data[key_to_remove[0]] + if key_to_remove[0] in window_data + else value + ), + ) self.last_closed_window = max( self.last_closed_window, max(key[1][0] for key in keys_to_remove), diff --git a/tests/unit/tables/test_base.py b/tests/unit/tables/test_base.py index 84e800083..bcd6e5fd1 100644 --- a/tests/unit/tables/test_base.py +++ b/tests/unit/tables/test_base.py @@ -191,6 +191,45 @@ async def test_last_closed_window(self, *, table): assert table.last_closed_window == 0.0 table.window = Mock(name="window") + self.mock_no_ranges(table) + table._data = { + ("boo", (1.1, 1.4)): "BOO", + ("moo", (1.4, 1.6)): "MOO", + ("faa", (1.9, 2.0)): "FAA", + ("bar", (4.1, 4.2)): "BAR", + } + table._partition_timestamps = { + TP1: [1.0, 2.0, 3.0, 4.0, 5.0, 6.0, 7.0], + } + table._partition_timestamp_keys = { + (TP1, 2.0): [ + ("boo", (1.1, 1.4)), + ("moo", (1.4, 1.6)), + ("faa", (1.9, 2.0)), + ], + (TP1, 5.0): [ + ("bar", (4.1, 4.2)), + ], + } + + def get_stale(limit): + def is_stale(timestamp, latest_timestamp): + return timestamp < limit + + return is_stale + + table.window.stale.side_effect = get_stale(4.0) + + await table._del_old_keys() + + assert table.last_closed_window == 1.9 + + @pytest.mark.asyncio + async def test_last_closed_window__mock_ranges(self, *, table): + assert table.last_closed_window == 0.0 + + table.window = Mock(name="window") + self.mock_ranges(table) table._data = { ("boo", (1.1, 1.4)): "BOO", ("moo", (1.4, 1.6)): "MOO", @@ -233,6 +272,64 @@ async def test_del_old_keys(self, *, table): on_window_close = table._on_window_close = AsyncMock(name="on_window_close") table.window = Mock(name="window") + self.mock_no_ranges(table) + table._data = { + ("boo", (1.1, 1.4)): "BOO", + ("moo", (1.4, 1.6)): "MOO", + ("faa", (1.9, 2.0)): "FAA", + ("bar", (4.1, 4.2)): "BAR", + } + table._partition_timestamps = { + TP1: [1.0, 2.0, 3.0, 4.0, 5.0, 6.0, 7.0], + } + table._partition_timestamp_keys = { + (TP1, 2.0): [ + ("boo", (1.1, 1.4)), + ("moo", (1.4, 1.6)), + ("faa", (1.9, 2.0)), + ], + (TP1, 5.0): [ + ("bar", (4.1, 4.2)), + ], + } + + def get_stale(limit): + def is_stale(timestamp, latest_timestamp): + return timestamp < limit + + return is_stale + + table.window.stale.side_effect = get_stale(4.0) + + await table._del_old_keys() + + assert table._partition_timestamps[TP1] == [4.0, 5.0, 6.0, 7.0] + assert table.data == {("bar", (4.1, 4.2)): "BAR"} + + on_window_close.assert_has_calls( + [ + call.__bool__(), + call(("boo", (1.1, 1.4)), "BOO"), + call.__bool__(), + call(("moo", (1.4, 1.6)), "MOO"), + call.__bool__(), + call(("faa", (1.9, 2.0)), "FAA"), + ] + ) + + table.last_closed_window = 8.0 + table.window.stale.side_effect = get_stale(6.0) + + await table._del_old_keys() + + assert not table.data + + @pytest.mark.asyncio + async def test_del_old_keys__mock_ranges(self, *, table): + on_window_close = table._on_window_close = AsyncMock(name="on_window_close") + + table.window = Mock(name="window") + self.mock_ranges(table) table._data = { ("boo", (1.1, 1.4)): "BOO", ("moo", (1.4, 1.6)): "MOO", @@ -289,6 +386,61 @@ async def test_del_old_keys_non_async_cb(self, *, table): on_window_close = table._on_window_close = Mock(name="on_window_close") table.window = Mock(name="window") + self.mock_no_ranges(table) + table._data = { + ("boo", (1.1, 1.4)): "BOO", + ("moo", (1.4, 1.6)): "MOO", + ("faa", (1.9, 2.0)): "FAA", + ("bar", (4.1, 4.2)): "BAR", + } + table._partition_timestamps = { + TP1: [1.0, 2.0, 3.0, 4.0, 5.0, 6.0, 7.0], + } + table._partition_timestamp_keys = { + (TP1, 2.0): [ + ("boo", (1.1, 1.4)), + ("moo", (1.4, 1.6)), + ("faa", (1.9, 2.0)), + ], + (TP1, 5.0): [ + ("bar", (4.1, 4.2)), + ], + } + + def get_stale(limit): + def is_stale(timestamp, latest_timestamp): + return timestamp < limit + + return is_stale + + table.window.stale.side_effect = get_stale(4.0) + + await table._del_old_keys() + + assert table._partition_timestamps[TP1] == [4.0, 5.0, 6.0, 7.0] + assert table.data == {("bar", (4.1, 4.2)): "BAR"} + + on_window_close.assert_has_calls( + [ + call(("boo", (1.1, 1.4)), "BOO"), + call(("moo", (1.4, 1.6)), "MOO"), + call(("faa", (1.9, 2.0)), "FAA"), + ] + ) + + table.last_closed_window = 8.0 + table.window.stale.side_effect = get_stale(6.0) + + await table._del_old_keys() + + assert not table.data + + @pytest.mark.asyncio + async def test_del_old_keys_non_async_cb__mock_ranges(self, *, table): + on_window_close = table._on_window_close = Mock(name="on_window_close") + + table.window = Mock(name="window") + self.mock_ranges(table) table._data = { ("boo", (1.1, 1.4)): "BOO", ("moo", (1.4, 1.6)): "MOO", @@ -527,6 +679,11 @@ def mock_ranges(self, table, ranges=[1.1, 1.2, 1.3]): # noqa table._window_ranges.return_value = ranges return ranges + def mock_no_ranges(self, table, ranges=[]): # noqa + table._window_ranges = Mock(name="_window_ranges") + table._window_ranges.return_value = ranges + return ranges + def test_relative_now(self, *, table): event = Mock(name="event", autospec=Event) table._partition_latest_timestamp[event.message.partition] = 30.3 From 3d737c754936130733103648a5af11f521a04818 Mon Sep 17 00:00:00 2001 From: William Barnhart Date: Mon, 1 Apr 2024 11:08:27 -0400 Subject: [PATCH 51/54] remove humanization conversion --- faust/transport/drivers/aiokafka.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/faust/transport/drivers/aiokafka.py b/faust/transport/drivers/aiokafka.py index fe189d9d3..3da4d192e 100644 --- a/faust/transport/drivers/aiokafka.py +++ b/faust/transport/drivers/aiokafka.py @@ -847,7 +847,7 @@ def _verify_aiokafka_event_path(self, now: float, tp: TP) -> bool: self.log.error( SLOW_PROCESSING_NO_FETCH_SINCE_START, tp, - humanize_seconds_ago(secs_since_started), + secs_since_started, ) return True @@ -857,7 +857,7 @@ def _verify_aiokafka_event_path(self, now: float, tp: TP) -> bool: self.log.error( SLOW_PROCESSING_NO_RECENT_FETCH, tp, - humanize_seconds_ago(secs_since_request), + secs_since_request, ) return True From 13e7ef1c8d60764cbe93913b9c3ce3d4a3f76483 Mon Sep 17 00:00:00 2001 From: William Barnhart Date: Mon, 1 Apr 2024 12:12:44 -0400 Subject: [PATCH 52/54] Fix disabled aiokafka unit tests (#610) * mess around this unit tests for aiokafka statuses * apparently highwater had a bug??? * fix formatting * wow has this been broken this entire time??? * fix more tests * lint * fix yet another test * fix yet another test * remove unneeded import * re-enable another test * fix linting * fix another test * Update aiokafka.py * revert changes --- faust/transport/drivers/aiokafka.py | 6 +- tests/unit/transport/drivers/test_aiokafka.py | 97 +++++++++++++------ 2 files changed, 71 insertions(+), 32 deletions(-) diff --git a/faust/transport/drivers/aiokafka.py b/faust/transport/drivers/aiokafka.py index 3da4d192e..72e30d1e8 100644 --- a/faust/transport/drivers/aiokafka.py +++ b/faust/transport/drivers/aiokafka.py @@ -840,14 +840,14 @@ def _verify_aiokafka_event_path(self, now: float, tp: TP) -> bool: poll_at = None aiotp_state = assignment.state_value(aiotp) if aiotp_state and aiotp_state.timestamp: - poll_at = aiotp_state.timestamp / 1000 + poll_at = aiotp_state.timestamp if poll_at is None: if secs_since_started >= self.tp_fetch_request_timeout_secs: # NO FETCH REQUEST SENT AT ALL SINCE WORKER START self.log.error( SLOW_PROCESSING_NO_FETCH_SINCE_START, tp, - secs_since_started, + humanize_seconds_ago(secs_since_started), ) return True @@ -857,7 +857,7 @@ def _verify_aiokafka_event_path(self, now: float, tp: TP) -> bool: self.log.error( SLOW_PROCESSING_NO_RECENT_FETCH, tp, - secs_since_request, + humanize_seconds_ago(secs_since_request), ) return True diff --git a/tests/unit/transport/drivers/test_aiokafka.py b/tests/unit/transport/drivers/test_aiokafka.py index 63a55d5d2..d25ac8aaa 100644 --- a/tests/unit/transport/drivers/test_aiokafka.py +++ b/tests/unit/transport/drivers/test_aiokafka.py @@ -11,6 +11,7 @@ from aiokafka.structs import OffsetAndMetadata, TopicPartition from mode.utils import text from mode.utils.futures import done_future +from mode.utils.times import humanize_seconds_ago from opentracing.ext import tags import faust @@ -234,7 +235,7 @@ def mock_record( serialized_value_size=40, **kwargs, ): - return Mock( + return MagicMock( name="record", topic=topic, partition=partition, @@ -281,8 +282,8 @@ def start_span(operation_name=None, **kwargs): return tracer @pytest.fixture() - def _consumer(self): - return Mock( + def _consumer(self, now, cthread, tp): + _consumer = Mock( name="AIOKafkaConsumer", autospec=aiokafka.AIOKafkaConsumer, start=AsyncMock(), @@ -293,6 +294,17 @@ def _consumer(self): _client=Mock(name="Client", close=AsyncMock()), _coordinator=Mock(name="Coordinator", close=AsyncMock()), ) + _consumer.assignment.return_value = {tp} + + ( + _consumer._fetcher._subscriptions.subscription.assignment.state_value + ).return_value = MagicMock( + assignment={tp}, + timestamp=now, + highwater=1, + position=0, + ) + return _consumer @pytest.fixture() def now(self): @@ -465,7 +477,6 @@ def test_timed_out(self, *, cthread, _consumer, now, tp, logger): ) -@pytest.mark.skip("Needs fixing") class Test_VEP_no_recent_fetch(Test_verify_event_path_base): def test_recent_fetch(self, *, cthread, now, tp, logger): self._set_last_response(now - 30.0) @@ -473,10 +484,15 @@ def test_recent_fetch(self, *, cthread, now, tp, logger): assert cthread.verify_event_path(now, tp) is None logger.error.assert_not_called() - def test_timed_out(self, *, cthread, now, tp, logger): + def test_timed_out(self, *, cthread, now, tp, logger, _consumer): self._set_last_response(now - 30.0) self._set_last_request(now - cthread.tp_fetch_request_timeout_secs * 2) - assert cthread.verify_event_path(now, tp) is None + assert ( + cthread.verify_event_path( + now + cthread.tp_fetch_request_timeout_secs * 2, tp + ) + is None + ) logger.error.assert_called_with( mod.SLOW_PROCESSING_NO_RECENT_FETCH, ANY, @@ -503,11 +519,10 @@ def test_timed_out(self, *, cthread, now, tp, logger): ) -@pytest.mark.skip("Needs fixing") class Test_VEP_no_highwater_since_start(Test_verify_event_path_base): highwater = None - def test_no_monitor(self, *, app, cthread, now, tp, logger): + def test_no_monitor(self, *, app, cthread, now, tp, logger, _consumer): self._set_last_request(now - 10.0) self._set_last_response(now - 5.0) self._set_started(now) @@ -515,17 +530,35 @@ def test_no_monitor(self, *, app, cthread, now, tp, logger): assert cthread.verify_event_path(now, tp) is None logger.error.assert_not_called() - def test_just_started(self, *, cthread, now, tp, logger): + def test_just_started(self, *, cthread, now, tp, logger, _consumer): self._set_last_request(now - 10.0) self._set_last_response(now - 5.0) self._set_started(now) assert cthread.verify_event_path(now, tp) is None logger.error.assert_not_called() - def test_timed_out(self, *, cthread, now, tp, logger): + def test_timed_out(self, *, cthread, now, tp, logger, _consumer): self._set_last_request(now - 10.0) self._set_last_response(now - 5.0) self._set_started(now - cthread.tp_stream_timeout_secs * 2) + _consumer.assignment.return_value = {tp} + assignment = cthread.assignment() + + assert assignment == {tp} + fetcher = _consumer._fetcher + (fetcher._subscriptions.subscription.assignment.state_value).return_value = ( + MagicMock( + assignment=assignment, + timestamp=now, + highwater=None, + tp_stream_timeout_secs=cthread.tp_stream_timeout_secs, + tp_fetch_request_timeout_secs=cthread.tp_fetch_request_timeout_secs, + ) + ) + ( + fetcher._subscriptions.subscription.assignment.state_value.timestamp + ).return_value = now + assert cthread.verify_event_path(now, tp) is None logger.error.assert_called_with( mod.SLOW_PROCESSING_NO_HIGHWATER_SINCE_START, @@ -534,7 +567,6 @@ def test_timed_out(self, *, cthread, now, tp, logger): ) -@pytest.mark.skip("Needs fixing") class Test_VEP_stream_idle_no_highwater(Test_verify_event_path_base): highwater = 10 committed_offset = 10 @@ -547,7 +579,6 @@ def test_highwater_same_as_offset(self, *, cthread, now, tp, logger): logger.error.assert_not_called() -@pytest.mark.skip("Needs fixing") class Test_VEP_stream_idle_highwater_no_acks(Test_verify_event_path_base): acks_enabled = False @@ -559,7 +590,6 @@ def test_no_acks(self, *, cthread, now, tp, logger): logger.error.assert_not_called() -@pytest.mark.skip("Needs fixing") class Test_VEP_stream_idle_highwater_same_has_acks_everything_OK( Test_verify_event_path_base ): @@ -572,6 +602,7 @@ def test_main(self, *, cthread, now, tp, logger): self._set_last_request(now - 10.0) self._set_last_response(now - 5.0) self._set_started(now) + assert cthread.verify_event_path(now, tp) is None logger.error.assert_not_called() @@ -636,7 +667,6 @@ def test_inbound_timed_out(self, *, app, cthread, now, tp, logger): ) -@pytest.mark.skip("Needs fixing") class Test_VEP_no_commit(Test_verify_event_path_base): highwater = 20 committed_offset = 10 @@ -664,13 +694,13 @@ def test_timed_out_since_start(self, *, app, cthread, now, tp, logger): expected_message = cthread._make_slow_processing_error( mod.SLOW_PROCESSING_NO_COMMIT_SINCE_START, [mod.SLOW_PROCESSING_CAUSE_COMMIT], + setting="broker_commit_livelock_soft_timeout", + current_value=app.conf.broker_commit_livelock_soft_timeout, ) logger.error.assert_called_once_with( expected_message, tp, - ANY, - setting="broker_commit_livelock_soft_timeout", - current_value=app.conf.broker_commit_livelock_soft_timeout, + humanize_seconds_ago(cthread.tp_commit_timeout_secs * 2), ) def test_timed_out_since_last(self, *, app, cthread, now, tp, logger): @@ -681,13 +711,13 @@ def test_timed_out_since_last(self, *, app, cthread, now, tp, logger): expected_message = cthread._make_slow_processing_error( mod.SLOW_PROCESSING_NO_RECENT_COMMIT, [mod.SLOW_PROCESSING_CAUSE_COMMIT], + setting="broker_commit_livelock_soft_timeout", + current_value=app.conf.broker_commit_livelock_soft_timeout, ) logger.error.assert_called_once_with( expected_message, tp, - ANY, - setting="broker_commit_livelock_soft_timeout", - current_value=app.conf.broker_commit_livelock_soft_timeout, + humanize_seconds_ago(now - cthread.tp_commit_timeout_secs * 4), ) def test_committing_fine(self, *, app, cthread, now, tp, logger): @@ -1344,6 +1374,8 @@ def assert_new_producer( max_batch_size=16384, max_request_size=1000000, request_timeout_ms=1200000, + metadata_max_age_ms=300000, + connections_max_idle_ms=540000, security_protocol="PLAINTEXT", **kwargs, ): @@ -1351,19 +1383,21 @@ def assert_new_producer( p = producer._new_producer() assert p is AIOKafkaProducer.return_value AIOKafkaProducer.assert_called_once_with( - acks=acks, - api_version=api_version, bootstrap_servers=bootstrap_servers, client_id=client_id, - compression_type=compression_type, + acks=acks, linger_ms=linger_ms, max_batch_size=max_batch_size, max_request_size=max_request_size, - request_timeout_ms=request_timeout_ms, + compression_type=compression_type, security_protocol=security_protocol, loop=producer.loop, partitioner=producer.partitioner, transactional_id=None, + api_version=api_version, + metadata_max_age_ms=metadata_max_age_ms, + connections_max_idle_ms=connections_max_idle_ms, + request_timeout_ms=request_timeout_ms, **kwargs, ) @@ -1433,7 +1467,6 @@ def test__settings_extra(self, *, producer, app): app.in_transaction = False assert producer._settings_extra() == {} - @pytest.mark.skip("fix me") def test__new_producer(self, *, app): producer = Producer(app.transport) self.assert_new_producer(producer) @@ -1442,8 +1475,8 @@ def test__new_producer(self, *, app): "expected_args", [ pytest.param( - {"api_version": "0.10"}, - marks=pytest.mark.conf(producer_api_version="0.10"), + {"api_version": "auto"}, + marks=pytest.mark.conf(producer_api_version="auto"), ), pytest.param({"acks": -1}, marks=pytest.mark.conf(producer_acks="all")), pytest.param( @@ -1474,6 +1507,14 @@ def test__new_producer(self, *, app): {"request_timeout_ms": 1234134000}, marks=pytest.mark.conf(producer_request_timeout=1234134), ), + pytest.param( + {"metadata_max_age_ms": 300000}, + marks=pytest.mark.conf(metadata_max_age_ms=300000), + ), + pytest.param( + {"connections_max_idle_ms": 540000}, + marks=pytest.mark.conf(connections_max_idle_ms=540000), + ), pytest.param( { "security_protocol": "SASL_PLAINTEXT", @@ -1490,7 +1531,6 @@ def test__new_producer(self, *, app): ), ], ) - @pytest.mark.skip("fix me") def test__new_producer__using_settings(self, expected_args, *, app): producer = Producer(app.transport) self.assert_new_producer(producer, **expected_args) @@ -1802,7 +1842,6 @@ async def test_on_start( await threaded_producer.start() await threaded_producer.stop() - @pytest.mark.skip("Needs fixing") @pytest.mark.asyncio async def test_on_thread_stop( self, From 2c4c8bbea56d56f3010831d1298848dba6d083e6 Mon Sep 17 00:00:00 2001 From: William Barnhart Date: Mon, 1 Apr 2024 13:09:27 -0400 Subject: [PATCH 53/54] Pin pytest<8 to fix Python 3.11 tests (#619) * Check if pytest<8.1 could fix * Update test.txt --- requirements/test.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/requirements/test.txt b/requirements/test.txt index 9ada8e540..8dfe2cdfa 100644 --- a/requirements/test.txt +++ b/requirements/test.txt @@ -15,7 +15,7 @@ pytest-forked pytest-picked pytest-cov pytest-random-order>=0.5.4 -pytest>=5.4.0 +pytest<8 python-dateutil>=2.8 pytz>=2018.7 bandit From a489db3bbee22583b6cbbcc0d840952981e24661 Mon Sep 17 00:00:00 2001 From: William Barnhart Date: Tue, 2 Apr 2024 15:31:10 -0400 Subject: [PATCH 54/54] Skip some PyPy tests and revert breaking poll_at timestamp change (#621) * skip consistency test since it's broken anyway * skip specific unit tests if pypy * conditionally disable functional streams tests on pypy * update lock acquisition for logging in newer python versions * revert breaking change with timestamp * fix remaining test * lint files --- faust/transport/drivers/aiokafka.py | 2 +- tests/consistency/test_consistency.py | 3 ++ tests/functional/conftest.py | 42 ++++++++++++----- tests/functional/test_streams.py | 46 +++++++++++++++++++ tests/unit/agents/test_agent.py | 4 ++ tests/unit/test_streams.py | 7 +++ tests/unit/transport/drivers/test_aiokafka.py | 4 +- 7 files changed, 93 insertions(+), 15 deletions(-) diff --git a/faust/transport/drivers/aiokafka.py b/faust/transport/drivers/aiokafka.py index 72e30d1e8..b52be7b16 100644 --- a/faust/transport/drivers/aiokafka.py +++ b/faust/transport/drivers/aiokafka.py @@ -840,7 +840,7 @@ def _verify_aiokafka_event_path(self, now: float, tp: TP) -> bool: poll_at = None aiotp_state = assignment.state_value(aiotp) if aiotp_state and aiotp_state.timestamp: - poll_at = aiotp_state.timestamp + poll_at = aiotp_state.timestamp / 1000 # milliseconds if poll_at is None: if secs_since_started >= self.tp_fetch_request_timeout_secs: # NO FETCH REQUEST SENT AT ALL SINCE WORKER START diff --git a/tests/consistency/test_consistency.py b/tests/consistency/test_consistency.py index 113953415..7086f0797 100644 --- a/tests/consistency/test_consistency.py +++ b/tests/consistency/test_consistency.py @@ -4,6 +4,8 @@ import subprocess import sys +import pytest + from tests.consistency.consistency_checker import ConsistencyChecker @@ -176,6 +178,7 @@ async def _stop_process(self, proc): await proc.wait() +@pytest.mark.skip(reason="Needs fixing") async def test_consistency(loop): stresser = Stresser(num_workers=4, num_producers=4, loop=loop) checker = ConsistencyChecker( diff --git a/tests/functional/conftest.py b/tests/functional/conftest.py index 4d09169dc..30e70b89e 100644 --- a/tests/functional/conftest.py +++ b/tests/functional/conftest.py @@ -1,5 +1,6 @@ import logging as _logging import os +import sys from copy import copy from typing import IO, Dict, NamedTuple, Union from unittest.mock import Mock @@ -71,26 +72,43 @@ def logging(request): **((marks.kwargs or {}) if marks else {}), } ) - _logging._acquireLock() - try: - prev_state = copy(_logging.Logger.manager.loggerDict) - prev_handlers = copy(_logging.root.handlers) - finally: - _logging._releaseLock() - try: + # acquireLock() is removed in Python 3.13 + if sys.version_info < (3, 13): + _logging._acquireLock() + try: + prev_state = copy(_logging.Logger.manager.loggerDict) + prev_handlers = copy(_logging.root.handlers) + finally: + _logging._releaseLock() + try: + setup_logging( + logfile=options.logfile, + loglevel=options.loglevel, + logging_config=options.logging_config, + ) + yield + finally: + _logging._acquireLock() + try: + _logging.Logger.manager.loggerDict = prev_state + _logging.root.handlers = prev_handlers + finally: + _logging._releaseLock() + else: + with _logging._lock: + prev_state = copy(_logging.Logger.manager.loggerDict) + prev_handlers = copy(_logging.root.handlers) + setup_logging( logfile=options.logfile, loglevel=options.loglevel, logging_config=options.logging_config, ) yield - finally: - _logging._acquireLock() - try: + + with _logging._lock: _logging.Logger.manager.loggerDict = prev_state _logging.root.handlers = prev_handlers - finally: - _logging._releaseLock() @pytest.fixture() diff --git a/tests/functional/test_streams.py b/tests/functional/test_streams.py index 95185cbeb..851594155 100644 --- a/tests/functional/test_streams.py +++ b/tests/functional/test_streams.py @@ -1,4 +1,5 @@ import asyncio +import platform from copy import copy from unittest.mock import Mock, patch @@ -39,6 +40,9 @@ def _prepare_app(app): return app +@pytest.mark.skipif( + platform.python_implementation() == "PyPy", reason="Not yet supported on PyPy" +) @pytest.mark.asyncio @pytest.mark.allow_lingering_tasks(count=1) async def test_simple(app, loop): @@ -50,6 +54,9 @@ async def test_simple(app, loop): assert await channel_empty(stream.channel) +@pytest.mark.skipif( + platform.python_implementation() == "PyPy", reason="Not yet supported on PyPy" +) @pytest.mark.asyncio async def test_async_iterator(app): async with new_stream(app) as stream: @@ -64,6 +71,9 @@ async def test_async_iterator(app): assert await channel_empty(stream.channel) +@pytest.mark.skipif( + platform.python_implementation() == "PyPy", reason="Not yet supported on PyPy" +) @pytest.mark.asyncio async def test_throw(app): async with new_stream(app) as stream: @@ -75,6 +85,9 @@ async def test_throw(app): await anext(streamit) +@pytest.mark.skipif( + platform.python_implementation() == "PyPy", reason="Not yet supported on PyPy" +) @pytest.mark.asyncio async def test_enumerate(app): async with new_stream(app) as stream: @@ -89,6 +102,9 @@ async def test_enumerate(app): assert await channel_empty(stream.channel) +@pytest.mark.skipif( + platform.python_implementation() == "PyPy", reason="Not yet supported on PyPy" +) @pytest.mark.asyncio async def test_items(app): async with new_stream(app) as stream: @@ -104,6 +120,9 @@ async def test_items(app): assert await channel_empty(stream.channel) +@pytest.mark.skipif( + platform.python_implementation() == "PyPy", reason="Not yet supported on PyPy" +) @pytest.mark.asyncio async def test_through(app): app._attachments.enabled = False @@ -236,6 +255,9 @@ async def test_stream_filter_acks_filtered_out_messages(app, event_loop): assert len(app.consumer.unacked) == 0 +@pytest.mark.skipif( + platform.python_implementation() == "PyPy", reason="Not yet supported on PyPy" +) @pytest.mark.asyncio async def test_acks_filtered_out_messages_when_using_take(app, event_loop): """ @@ -260,6 +282,9 @@ async def test_acks_filtered_out_messages_when_using_take(app, event_loop): assert len(acked) == len(initial_values) +@pytest.mark.skipif( + platform.python_implementation() == "PyPy", reason="Not yet supported on PyPy" +) @pytest.mark.asyncio async def test_events(app): async with new_stream(app) as stream: @@ -296,6 +321,9 @@ def assert_events_acked(events): raise +@pytest.mark.skipif( + platform.python_implementation() == "PyPy", reason="Not yet supported on PyPy" +) class Test_chained_streams: def _chain(self, app): root = new_stream(app) @@ -399,6 +427,9 @@ async def assert_was_stopped(self, leader, followers): assert node._stopped.is_set() +@pytest.mark.skipif( + platform.python_implementation() == "PyPy", reason="Not yet supported on PyPy" +) @pytest.mark.asyncio async def test_start_and_stop_Stream(app): s = new_topic_stream(app) @@ -414,6 +445,9 @@ async def _start_stop_stream(stream): await stream.stop() +@pytest.mark.skipif( + platform.python_implementation() == "PyPy", reason="Not yet supported on PyPy" +) @pytest.mark.asyncio async def test_ack(app): async with new_stream(app) as s: @@ -439,6 +473,9 @@ async def test_ack(app): assert not event.message.refcount +@pytest.mark.skipif( + platform.python_implementation() == "PyPy", reason="Not yet supported on PyPy" +) @pytest.mark.asyncio async def test_noack(app): async with new_stream(app) as s: @@ -459,6 +496,9 @@ async def test_noack(app): event.ack.assert_not_called() +@pytest.mark.skipif( + platform.python_implementation() == "PyPy", reason="Not yet supported on PyPy" +) @pytest.mark.asyncio async def test_acked_when_raising(app): async with new_stream(app) as s: @@ -496,6 +536,9 @@ async def test_acked_when_raising(app): assert not event2.message.refcount +@pytest.mark.skipif( + platform.python_implementation() == "PyPy", reason="Not yet supported on PyPy" +) @pytest.mark.asyncio @pytest.mark.allow_lingering_tasks(count=1) async def test_maybe_forward__when_event(app): @@ -508,6 +551,9 @@ async def test_maybe_forward__when_event(app): s.channel.send.assert_not_called() +@pytest.mark.skipif( + platform.python_implementation() == "PyPy", reason="Not yet supported on PyPy" +) @pytest.mark.asyncio async def test_maybe_forward__when_concrete_value(app): s = new_stream(app) diff --git a/tests/unit/agents/test_agent.py b/tests/unit/agents/test_agent.py index 54f881f68..25f033ac6 100644 --- a/tests/unit/agents/test_agent.py +++ b/tests/unit/agents/test_agent.py @@ -1,4 +1,5 @@ import asyncio +import platform from unittest.mock import ANY, call, patch import pytest @@ -953,6 +954,9 @@ def test_channel_iterator(self, *, agent): def test_label(self, *, agent): assert label(agent) + @pytest.mark.skipif( + platform.python_implementation() == "PyPy", reason="Not yet supported on PyPy" + ) async def test_context_calls_sink(self, *, agent): class SinkCalledException(Exception): pass diff --git a/tests/unit/test_streams.py b/tests/unit/test_streams.py index 4b7af428d..4a6cab481 100644 --- a/tests/unit/test_streams.py +++ b/tests/unit/test_streams.py @@ -1,4 +1,5 @@ import asyncio +import platform from collections import defaultdict from contextlib import ExitStack from unittest.mock import Mock, patch @@ -122,6 +123,9 @@ async def test_echo(self, *, stream, app): await echoing("val") channel.send.assert_called_once_with(value="val") + @pytest.mark.skipif( + platform.python_implementation() == "PyPy", reason="Not yet supported on PyPy" + ) @pytest.mark.asyncio @pytest.mark.allow_lingering_tasks(count=1) async def test_aiter_tracked(self, *, stream, app): @@ -137,6 +141,9 @@ async def test_aiter_tracked(self, *, stream, app): else: event.ack.assert_called_once_with() + @pytest.mark.skipif( + platform.python_implementation() == "PyPy", reason="Not yet supported on PyPy" + ) @pytest.mark.asyncio @pytest.mark.allow_lingering_tasks(count=1) async def test_aiter_tracked__CancelledError(self, *, stream, app): diff --git a/tests/unit/transport/drivers/test_aiokafka.py b/tests/unit/transport/drivers/test_aiokafka.py index d25ac8aaa..04d5d910f 100644 --- a/tests/unit/transport/drivers/test_aiokafka.py +++ b/tests/unit/transport/drivers/test_aiokafka.py @@ -300,7 +300,7 @@ def _consumer(self, now, cthread, tp): _consumer._fetcher._subscriptions.subscription.assignment.state_value ).return_value = MagicMock( assignment={tp}, - timestamp=now, + timestamp=now * 1000.0, highwater=1, position=0, ) @@ -549,7 +549,7 @@ def test_timed_out(self, *, cthread, now, tp, logger, _consumer): (fetcher._subscriptions.subscription.assignment.state_value).return_value = ( MagicMock( assignment=assignment, - timestamp=now, + timestamp=now * 1000.0, highwater=None, tp_stream_timeout_secs=cthread.tp_stream_timeout_secs, tp_fetch_request_timeout_secs=cthread.tp_fetch_request_timeout_secs,