diff --git a/Makefile b/Makefile index 942c28953..526cc45af 100644 --- a/Makefile +++ b/Makefile @@ -121,7 +121,7 @@ flakecheck: $(FLAKE8) "$(PROJ)" "$(TESTDIR)" examples/ docstylecheck: - $(PYDOCSTYLE) --match-dir '(?!types)' "$(PROJ)" + $(PYDOCSTYLE) --match-dir '(?!types|assignor)' "$(PROJ)" vulture: $(VULTURE) "$(PROJ)" "$(TESTDIR)" "$(EXAMPLESDIR)" \ diff --git a/docs/includes/blurb.txt b/docs/includes/blurb.txt index 2c44a129e..20ea152ea 100644 --- a/docs/includes/blurb.txt +++ b/docs/includes/blurb.txt @@ -6,7 +6,7 @@ It's inspired by tools such as `Kafka Streams`_, `Apache Spark`_, a radically more straightforward approach to stream processing. Modern web applications are increasingly built as a collection -of microservices and even before this, it has been difficult to write +of micro services and even before this, it has been difficult to write data reporting operations at scale. In a reactive stream based system, you don't have to strain your database with costly queries. In Faust, a streaming data pipeline updates information as events happen diff --git a/docs/reference/index.rst b/docs/reference/index.rst index 4277b059a..b0290c8ac 100644 --- a/docs/reference/index.rst +++ b/docs/reference/index.rst @@ -57,7 +57,7 @@ Fixups faust.fixups.base faust.fixups.django -Livecheck +LiveCheck ========= .. toctree:: diff --git a/docs/spelling_wordlist.txt b/docs/spelling_wordlist.txt index 072f9c1e1..3185886b4 100644 --- a/docs/spelling_wordlist.txt +++ b/docs/spelling_wordlist.txt @@ -54,6 +54,7 @@ Bryson Buckens Bujniewicz Buttu +CLI CPython Carvalho Cassandra @@ -88,6 +89,7 @@ Dein Delalande Demir Deo +Deque Django Dmitry Dubus @@ -496,6 +498,7 @@ dburi de deprecated deprecations +deque der deserialization deserialize @@ -563,6 +566,7 @@ memoize memoized metadata metavar +middleware misconfiguration misconfigure misconfigured diff --git a/docs/userguide/livecheck.rst b/docs/userguide/livecheck.rst index 23569c410..ad498abd5 100644 --- a/docs/userguide/livecheck.rst +++ b/docs/userguide/livecheck.rst @@ -1,5 +1,5 @@ ================================================================ - LiveCheck: End-to-end test for prodiction/staging. + LiveCheck: End-to-end test for production/staging. ================================================================ What is the problem with unit tests? @@ -10,7 +10,7 @@ A staging environment is still desirable. Enables you to: - - track requests as they travel through your microservice architecture. + - track requests as they travel through your micro service architecture. - define contracts that should be met at every step all by writing a class that looks like a regular unit test. @@ -20,9 +20,9 @@ subsystems are down. Tests are executed based on probability, so you can run tests for every requests, or for just 30%, 50%, or even 0.1% of your requests. -This is not just for microservice architectures, it's for any asynchronous +This is not just for micro service architectures, it's for any asynchronous system. A monolith sending celery tasks is a good example, you could -track and babysit at every step of a workflow to make sure things +track and babysit at every step of a work flow to make sure things progress as they should. Every stage of your production pipeline could be tested for diff --git a/faust/_cython/__init__.py b/faust/_cython/__init__.py index e69de29bb..159392636 100644 --- a/faust/_cython/__init__.py +++ b/faust/_cython/__init__.py @@ -0,0 +1 @@ +"""Cython optimized Faust features.""" diff --git a/faust/agents/__init__.py b/faust/agents/__init__.py index 78f610868..64033628f 100644 --- a/faust/agents/__init__.py +++ b/faust/agents/__init__.py @@ -1,3 +1,4 @@ +"""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 9b8aad921..5f10163a6 100644 --- a/faust/agents/actor.py +++ b/faust/agents/actor.py @@ -36,13 +36,16 @@ def __init__(self, Service.__init__(self, **kwargs) async def on_start(self) -> None: + """Call when actor is starting.""" assert self.actor_task self.add_future(self.actor_task) async def on_stop(self) -> None: + """Call when actor is being stopped.""" self.cancel() async def on_isolated_partition_revoked(self, tp: TP) -> None: + """Call when an isolated partition is being revoked.""" self.log.debug('Cancelling current task in actor for partition %r', tp) self.cancel() self.log.info('Stopping actor for revoked partition %r...', tp) @@ -50,9 +53,11 @@ async def on_isolated_partition_revoked(self, tp: TP) -> None: self.log.debug('Actor for revoked partition %r stopped') async def on_isolated_partition_assigned(self, tp: TP) -> None: + """Call when an isolated partition is being assigned.""" self.log.dev('Actor was assigned to %r', tp) def cancel(self) -> None: + """Tell actor to stop reading from the stream.""" cast(ChannelT, self.stream.channel)._throw(StopAsyncIteration()) def __repr__(self) -> str: @@ -60,6 +65,7 @@ def __repr__(self) -> str: @property def label(self) -> str: + """Return human readable description of actor.""" s = self.agent._agent_label(name_suffix='*') if self.stream.active_partitions: partitions = { diff --git a/faust/agents/agent.py b/faust/agents/agent.py index e36c1651f..0e4635256 100644 --- a/faust/agents/agent.py +++ b/faust/agents/agent.py @@ -224,6 +224,7 @@ def __init__(self, Service.__init__(self) def on_init_dependencies(self) -> Iterable[ServiceT]: + """Return list of services dependencies required to start agent.""" # Agent service is now a child of app. self.beacon.reattach(self.app.beacon) return [] @@ -267,6 +268,7 @@ async def _start_for_partitions(self, return await self._start_one_supervised(None, active_partitions) async def on_start(self) -> None: + """Call when an agent starts.""" self.supervisor = self._new_supervisor() await self._on_start_supervisor() @@ -322,6 +324,7 @@ def _get_active_partitions(self) -> Optional[Set[TP]]: return active_partitions async def on_stop(self) -> None: + """Call when an agent stops.""" # Agents iterate over infinite streams, so we cannot wait for it # to stop. # Instead we cancel it and this forces the stream to ack the @@ -340,10 +343,12 @@ async def _stop_supervisor(self) -> None: self.supervisor = None def cancel(self) -> None: + """Cancel agent and its actor instances running in this process.""" for aref in self._actors: aref.cancel() async def on_partitions_revoked(self, revoked: Set[TP]) -> None: + """Call when partitions are revoked.""" T = traced_from_parent_span() if self.isolated_partitions: # isolated: start/stop actors for each partition @@ -352,6 +357,7 @@ async def on_partitions_revoked(self, revoked: Set[TP]) -> None: await T(self.on_shared_partitions_revoked)(revoked) async def on_partitions_assigned(self, assigned: Set[TP]) -> None: + """Call when partitions are assigned.""" T = traced_from_parent_span() if self.isolated_partitions: await T(self.on_isolated_partitions_assigned)(assigned) @@ -359,6 +365,7 @@ async def on_partitions_assigned(self, assigned: Set[TP]) -> None: await T(self.on_shared_partitions_assigned)(assigned) async def on_isolated_partitions_revoked(self, revoked: Set[TP]) -> None: + """Call when isolated partitions are revoked.""" self.log.dev('Partitions revoked') T = traced_from_parent_span() for tp in revoked: @@ -367,6 +374,7 @@ async def on_isolated_partitions_revoked(self, revoked: Set[TP]) -> None: await T(aref.on_isolated_partition_revoked)(tp) async def on_isolated_partitions_assigned(self, assigned: Set[TP]) -> None: + """Call when isolated partitions are assigned.""" T = traced_from_parent_span() for tp in sorted(assigned): await T(self._assign_isolated_partition)(tp) @@ -402,12 +410,15 @@ async def _start_isolated(self, tp: TP) -> ActorT: return await self._start_for_partitions({tp}) async def on_shared_partitions_revoked(self, revoked: Set[TP]) -> None: + """Call when non-isolated partitions are revoked.""" ... async def on_shared_partitions_assigned(self, assigned: Set[TP]) -> None: + """Call when non-isolated partitions are assigned.""" ... def info(self) -> Mapping: + """Return agent attributes as a dictionary.""" return { 'app': self.app, 'fun': self.fun, @@ -422,6 +433,11 @@ def info(self) -> Mapping: } def clone(self, *, cls: Type[AgentT] = None, **kwargs: Any) -> AgentT: + """Create clone of this agent object. + + Keyword arguments can be passed to override any argument + supported by :class:`Agent.__init__ `. + """ return (cls or type(self))(**{**self.info(), **kwargs}) def test_context(self, @@ -429,6 +445,7 @@ def test_context(self, supervisor_strategy: SupervisorStrategyT = None, on_error: AgentErrorHandler = None, **kwargs: Any) -> AgentTestWrapperT: # pragma: no cover + """Create new unit-testing wrapper for this agent.""" # flow control into channel queues are disabled at startup, # so need to resume that. self.app.flow_control.resume() @@ -471,6 +488,7 @@ def __call__(self, *, active_partitions: Set[TP] = None, stream: StreamT = None, channel: ChannelT = None) -> ActorRefT: + """Create new actor instance for this agent.""" # The agent function can be reused by other agents/tasks. # For example: # @@ -499,6 +517,7 @@ def actor_from_stream(self, index: int = None, active_partitions: Set[TP] = None, channel: ChannelT = None) -> ActorRefT: + """Create new actor from stream.""" we_created_stream = False actual_stream: StreamT if stream is None: @@ -532,6 +551,7 @@ def actor_from_stream(self, ) def add_sink(self, sink: SinkT) -> None: + """Add new sink to further handle results from this agent.""" if sink not in self._sinks: self._sinks.append(sink) @@ -539,6 +559,7 @@ def stream(self, channel: ChannelT = None, active_partitions: Set[TP] = None, **kwargs: Any) -> StreamT: + """Create underlying stream used by this agent.""" if channel is None: channel = cast(TopicT, self.channel_iterator).clone( is_iterator=False, @@ -672,6 +693,11 @@ async def cast(self, partition: int = None, timestamp: float = None, headers: HeadersArg = None) -> None: + """RPC operation: like :meth:`ask` but do not expect reply. + + Cast here is like "casting a spell", and will not expect + a reply back from the agent. + """ await self.send( key=key, value=value, @@ -689,6 +715,11 @@ async def ask(self, headers: HeadersArg = None, reply_to: ReplyToArg = None, correlation_id: str = None) -> Any: + """RPC operation: ask agent for result of processing value. + + This version will wait until the result is available + and return the processed value. + """ p = await self.ask_nowait( value, key=key, @@ -714,6 +745,11 @@ async def ask_nowait(self, reply_to: ReplyToArg = None, correlation_id: str = None, force: bool = False) -> ReplyPromise: + """RPC operation: ask agent for result of processing value. + + This version does not wait for the result to arrive, + but instead returns a promise of future evaluation. + """ if reply_to is None: raise TypeError('Missing reply_to argument') reply_to = self._get_strtopic(reply_to) @@ -805,6 +841,11 @@ async def map(self, key: K = None, reply_to: ReplyToArg = None, ) -> AsyncIterator: # pragma: no cover + """RPC map operation on a list of values. + + A map operation iterates over results as they arrive. + See :meth:`join` and :meth:`kvjoin` if you want them in order. + """ # Map takes only values, but can provide one key that is used for all. async for value in self.kvmap( ((key, v) async for v in aiter(values)), reply_to): @@ -815,6 +856,11 @@ async def kvmap( items: Union[AsyncIterable[Tuple[K, V]], Iterable[Tuple[K, V]]], reply_to: ReplyToArg = None, ) -> AsyncIterator[str]: # pragma: no cover + """RPC map operation on a list of ``(key, value)`` pairs. + + A map operation iterates over results as they arrive. + See :meth:`join` and :meth:`kvjoin` if you want them in order. + """ # kvmap takes (key, value) pairs. reply_to = self._get_strtopic(reply_to or self.app.conf.reply_to) @@ -843,6 +889,11 @@ async def join(self, key: K = None, reply_to: ReplyToArg = None, ) -> List[Any]: # pragma: no cover + """RPC map operation on a list of values. + + A join returns the results in order, and only returns once + all values have been processed. + """ return await self.kvjoin( ((key, value) async for value in aiter(values)), reply_to=reply_to, @@ -852,6 +903,11 @@ async def kvjoin( self, items: Union[AsyncIterable[Tuple[K, V]], Iterable[Tuple[K, V]]], reply_to: ReplyToArg = None) -> List[Any]: # pragma: no cover + """RPC map operation on list of ``(key, value)`` pairs. + + A join returns the results in order, and only returns once + all values have been processed. + """ reply_to = self._get_strtopic(reply_to or self.app.conf.reply_to) barrier = BarrierState(reply_to) @@ -901,6 +957,7 @@ def _repr_info(self) -> str: return shorten_fqdn(self.name) def get_topic_names(self) -> Iterable[str]: + """Return list of topic names this agent subscribes to.""" channel = self.channel if isinstance(channel, TopicT): return channel.topics @@ -908,6 +965,7 @@ def get_topic_names(self) -> Iterable[str]: @property def channel(self) -> ChannelT: + """Return channel used by agent.""" if self._channel is None: self._channel = self._prepare_channel( self._channel_arg, @@ -923,6 +981,7 @@ def channel(self, channel: ChannelT) -> None: @property def channel_iterator(self) -> AsyncIterator: + """Return channel agent iterates over.""" # The channel is "memoized" here, so subsequent access to # instance.channel_iterator will return the same value. if self._channel_iterator is None: @@ -937,6 +996,7 @@ def channel_iterator(self, it: AsyncIterator) -> None: @property def label(self) -> str: + """Return human-readable description of agent.""" return self._agent_label() def _agent_label(self, name_suffix: str = '') -> str: @@ -946,6 +1006,7 @@ def _agent_label(self, name_suffix: str = '') -> str: @property def shortlabel(self) -> str: + """Return short description of agent.""" return self._agent_label() diff --git a/faust/agents/manager.py b/faust/agents/manager.py index 6d8b5228f..0fbf01ec2 100644 --- a/faust/agents/manager.py +++ b/faust/agents/manager.py @@ -24,15 +24,18 @@ def __init__(self, app: AppT, Service.__init__(self, **kwargs) async def on_start(self) -> None: + """Call when agents are being started.""" self.update_topic_index() for agent in self.values(): await agent.maybe_start() def service_reset(self) -> None: + """Reset service state on restart.""" [agent.service_reset() for agent in self.values()] super().service_reset() async def on_stop(self) -> None: + """Call when agents are being stopped.""" for agent in self.values(): try: await asyncio.shield(agent.stop()) @@ -40,15 +43,18 @@ async def on_stop(self) -> None: pass async def stop(self) -> None: + """Stop all running agents.""" # Cancel first so _execute_task sees we are not stopped. self.cancel() # Then stop the agents await super().stop() def cancel(self) -> None: + """Cancel all running agents.""" [agent.cancel() for agent in self.values()] def update_topic_index(self) -> None: + """Update indices.""" # keep mapping from topic name to set of agents. by_topic_index = self._by_topic for agent in self.values(): @@ -58,6 +64,7 @@ def update_topic_index(self) -> None: async def on_rebalance(self, revoked: Set[TP], newly_assigned: Set[TP]) -> None: + """Call when a rebalance is needed.""" T = traced_from_parent_span() # for isolated_partitions agents we stop agents for revoked # partitions. diff --git a/faust/agents/replies.py b/faust/agents/replies.py index 57abbeb8d..6725b5547 100644 --- a/faust/agents/replies.py +++ b/faust/agents/replies.py @@ -35,6 +35,7 @@ def __init__(self, reply_to: str, correlation_id: str, super().__init__(**kwargs) def fulfill(self, correlation_id: str, value: Any) -> None: + """Fulfill promise: a reply was received.""" # If it wasn't for BarrierState we would just use .set_result() # directly, but BarrierState.fulfill requires the correlation_id # to be sent with it. That way it can mark that part of the map @@ -77,10 +78,21 @@ def __init__(self, reply_to: str, **kwargs: Any) -> None: self._results = asyncio.Queue(maxsize=1000, loop=loop) def add(self, p: ReplyPromise) -> None: + """Add promise to barrier. + + Note: + You can only add promises before the barrier is finalized + using :meth:`finalize`. + """ self.pending.add(p) self.size += 1 def finalize(self) -> None: + """Finalize this barrier. + + After finalization you can not grow or shrink the size + of the barrier. + """ self.total = self.size # The barrier may have been filled up already at this point, if self.fulfilled >= self.total: @@ -88,6 +100,11 @@ def finalize(self) -> None: self._results.put_nowait(None) # always wake-up .iterate() def fulfill(self, correlation_id: str, value: Any) -> None: + """Fulfill one of the promises in this barrier. + + Once all promises in this barrier is fulfilled, the barrier + will be ready. + """ # ReplyConsumer calls this whenever a new reply is received. self._results.put_nowait(ReplyTuple(correlation_id, value)) self.fulfilled += 1 @@ -105,7 +122,7 @@ def get_nowait(self) -> ReplyTuple: raise asyncio.QueueEmpty() async def iterate(self) -> AsyncIterator[ReplyTuple]: - """Iterate over results as arrive.""" + """Iterate over results as they arrive.""" get = self._results.get get_nowait = self._results.get_nowait is_done = self.done @@ -136,10 +153,12 @@ def __init__(self, app: AppT, **kwargs: Any) -> None: super().__init__(**kwargs) async def on_start(self) -> None: + """Call when reply consumer starts.""" if self.app.conf.reply_create_topic: await self._start_fetcher(self.app.conf.reply_to) async def add(self, correlation_id: str, promise: ReplyPromise) -> None: + """Register promise to start tracking when it arrives.""" reply_topic = promise.reply_to if reply_topic not in self._fetchers: await self._start_fetcher(reply_topic) diff --git a/faust/app/__init__.py b/faust/app/__init__.py index c8ee33d75..7509dbf71 100644 --- a/faust/app/__init__.py +++ b/faust/app/__init__.py @@ -1,3 +1,4 @@ +"""Application.""" from .base import App, BootStrategy __all__ = ['App', 'BootStrategy'] diff --git a/faust/app/_attached.py b/faust/app/_attached.py index 9c54e5b7c..3bdd4179f 100644 --- a/faust/app/_attached.py +++ b/faust/app/_attached.py @@ -74,6 +74,7 @@ def __init__(self, app: AppT) -> None: @cached_property def enabled(self) -> bool: + """Return :const:`True` if attachments are enabled.""" return self.app.conf.stream_publish_on_commit async def maybe_put(self, @@ -87,6 +88,11 @@ async def maybe_put(self, value_serializer: CodecArg = None, callback: MessageSentCallback = None, force: bool = False) -> Awaitable[RecordMetadata]: + """Attach message to source topic offset. + + This will send the message immediately if attachments + are disabled. + """ # XXX The concept of attaching should be deprecated when we # have Kafka transaction support (:kip:`KIP-98`). # This is why the interface related to attaching is private. @@ -130,6 +136,7 @@ def put(self, key_serializer: CodecArg = None, value_serializer: CodecArg = None, callback: MessageSentCallback = None) -> Awaitable[RecordMetadata]: + """Attach message to source topic offset.""" # This attaches message to be published when source message' is # acknowledged. To be replaced by transactions in :kip:`KIP-98`. @@ -149,6 +156,7 @@ def put(self, return fut async def commit(self, tp: TP, offset: int) -> None: + """Publish all messaged attached to topic partition and offset.""" await asyncio.wait( await self.publish_for_tp_offset(tp, offset), return_when=asyncio.ALL_COMPLETED, @@ -157,6 +165,7 @@ async def commit(self, tp: TP, offset: int) -> None: async def publish_for_tp_offset( self, tp: TP, offset: int) -> List[Awaitable[RecordMetadata]]: + """Publish messages attached to topic partition and offset.""" # publish pending messages attached to this TP+offset # make shallow copy to allow concurrent modifications (append) diff --git a/faust/app/base.py b/faust/app/base.py index bd8d59d3f..6d5b49b54 100644 --- a/faust/app/base.py +++ b/faust/app/base.py @@ -248,6 +248,7 @@ def __init__(self, app: AppT, *, self.enable_sensors = enable_sensors def server(self) -> Iterable[ServiceT]: + """Return services to start when app is in default mode.""" return self._chain( # Sensors (Sensor): always start first and stop last. self.sensors(), @@ -266,6 +267,7 @@ def server(self) -> Iterable[ServiceT]: ) def client_only(self) -> Iterable[ServiceT]: + """Return services to start when app is in client_only mode.""" return self._chain( self.kafka_producer(), self.kafka_client_consumer(), @@ -274,6 +276,7 @@ def client_only(self) -> Iterable[ServiceT]: ) def producer_only(self) -> Iterable[ServiceT]: + """Return services to start when app is in producer_only mode.""" return self._chain( self.web_server(), self.kafka_producer(), @@ -283,11 +286,13 @@ def _chain(self, *arguments: Iterable[ServiceT]) -> Iterable[ServiceT]: return cast(Iterable[ServiceT], chain(*arguments)) def sensors(self) -> Iterable[ServiceT]: + """Return list of services required to start sensors.""" if self.enable_sensors: return self.app.sensors return [] def kafka_producer(self) -> Iterable[ServiceT]: + """Return list of services required to start Kafka producer.""" if self._should_enable_kafka_producer(): return [self.app.producer] return [] @@ -298,6 +303,7 @@ def _should_enable_kafka_producer(self) -> bool: return self.enable_kafka_producer def kafka_consumer(self) -> Iterable[ServiceT]: + """Return list of services required to start Kafka consumer.""" if self._should_enable_kafka_consumer(): return [ self.app.consumer, @@ -314,20 +320,24 @@ def _should_enable_kafka_consumer(self) -> bool: return self.enable_kafka_consumer def kafka_client_consumer(self) -> Iterable[ServiceT]: + """Return list of services required to start Kafka client consumer.""" return [ self.app.consumer, self.app._reply_consumer, ] def agents(self) -> Iterable[ServiceT]: + """Return list of services required to start agents.""" return [self.app.agents] def kafka_conductor(self) -> Iterable[ServiceT]: + """Return list of services required to start Kafka conductor.""" if self._should_enable_kafka_consumer(): return [self.app.topics] return [] def web_server(self) -> Iterable[ServiceT]: + """Return list of web-server services.""" if self._should_enable_web(): return list(self.web_components()) + [self.app.web] return [] @@ -338,9 +348,11 @@ def _should_enable_web(self) -> bool: return self.enable_web def web_components(self) -> Iterable[ServiceT]: + """Return list of web-related services (excluding web server).""" return [self.app.cache] def tables(self) -> Iterable[ServiceT]: + """Return list of table-related services.""" return [self.app.tables] @@ -510,6 +522,11 @@ def _init_fixups(self) -> MutableSequence[FixupT]: return list(fixups(self)) def on_init_dependencies(self) -> Iterable[ServiceT]: + """Return list of additional service dependencies. + + The services returned will be started with the + app when the app starts. + """ # Add the main Monitor sensor. # The beacon is also reattached in case the monitor # was created by the user. @@ -525,12 +542,15 @@ def on_init_dependencies(self) -> Iterable[ServiceT]: return self.boot_strategy.server() async def on_first_start(self) -> None: + """Call first time app starts in this process.""" self._create_directories() async def on_start(self) -> None: + """Call every time app start/restarts.""" self.finalize() async def on_started(self) -> None: + """Call when app is fully started.""" # Wait for table recovery to complete (returns True if app stopped) if not await self._wait_for_table_recovery_completed(): # Add all asyncio.Tasks, like timers, etc. @@ -552,10 +572,12 @@ async def _wait_for_table_recovery_completed(self) -> bool: return False async def on_started_init_extra_tasks(self) -> None: + """Call when started to start additional tasks.""" for task in self._tasks: self.add_future(task()) async def on_started_init_extra_services(self) -> None: + """Call when initializing extra services at startup.""" if self._extra_service_instances is None: # instantiate the services added using the @app.service decorator. self._extra_service_instances = [ @@ -565,6 +587,7 @@ async def on_started_init_extra_services(self) -> None: async def on_init_extra_service( self, service: Union[ServiceT, Type[ServiceT]]) -> ServiceT: + """Call when adding user services to this app.""" s: ServiceT = self._prepare_subservice(service) # start the service now, or when the app is started. await self.add_runtime_dependency(s) @@ -604,6 +627,7 @@ def config_from_object(self, self._configure(silent=silent) def finalize(self) -> None: + """Finalize app configuration.""" # Finalization signals that the application have been configured # and is ready to use. @@ -622,6 +646,7 @@ async def _maybe_close_http_client(self) -> None: await self._http_client.close() def worker_init(self) -> None: + """Init worker/CLI commands.""" # This init is called by the `faust worker` command. for fixup in self.fixups: fixup.on_worker_init() @@ -632,6 +657,7 @@ def discover(self, *extra_modules: str, categories: Iterable[str] = SCAN_CATEGORIES, ignore: Iterable[Any] = SCAN_IGNORE) -> None: + """Discover decorators in packages.""" # based on autodiscovery in Django, # but finds @app.agent decorators, and so on. modules = set(self._discovery_modules()) @@ -986,6 +1012,7 @@ class Foo(Service): return cls def is_leader(self) -> bool: + """Return :const:`True` if we are in leader worker process.""" return self._leader_assignor.is_leader() def stream(self, @@ -1055,6 +1082,7 @@ def SetTable(self, start_manager: bool = False, help: str = None, **kwargs: Any) -> TableT: + """Table of sets.""" table = self.tables.add( self.conf.SetTable( self, @@ -1070,6 +1098,7 @@ def page(self, path: str, *, base: Type[View] = View, cors_options: Mapping[str, ResourceOptions] = None, name: str = None) -> Callable[[PageArg], Type[View]]: + """Decorate view to be included in the web server.""" view_base: Type[View] = base if base is not None else View def _decorator(fun: PageArg) -> Type[View]: @@ -1095,6 +1124,7 @@ def table_route(self, table: CollectionT, query_param: str = None, match_info: str = None, exact_key: str = None) -> ViewDecorator: + """Decorate view method to route request to table key destination.""" def _decorator(fun: ViewHandlerFun) -> ViewHandlerFun: _query_param = query_param if shard_param is not None: @@ -1135,6 +1165,7 @@ def command(self, *options: Any, base: Optional[Type[_AppCommand]] = None, **kwargs: Any) -> Callable[[Callable], Type[_AppCommand]]: + """Decorate ``async def`` function to be used as CLI command.""" _base: Type[_AppCommand] if base is None: from faust.cli import base as cli_base @@ -1167,6 +1198,7 @@ def trace(self, name: str, trace_enabled: bool = True, **extra_context: Any) -> ContextManager: + """Return new trace context to trace operation using OpenTracing.""" if self.tracer is None or not trace_enabled: return nullcontext() else: @@ -1178,6 +1210,7 @@ def traced(self, fun: Callable, name: str = None, sample_rate: float = 1.0, **context: Any) -> Callable: + """Decorate function to be traced using the OpenTracing API.""" assert fun operation: str = name or operation_name_from_fun(fun) @@ -1253,12 +1286,14 @@ async def send( @cached_property def in_transaction(self) -> bool: + """Return :const:`True` if stream is using transactions.""" return ( self.in_worker and self.conf.processing_guarantee == ProcessingGuarantee.EXACTLY_ONCE ) def LiveCheck(self, **kwargs: Any) -> _LiveCheck: + """Return new LiveCheck instance testing features for this app.""" from faust.livecheck import LiveCheck return LiveCheck.for_app(self, **kwargs) @@ -1285,6 +1320,11 @@ async def commit(self, topics: TPorTopicSet) -> bool: return await self.topics.commit(topics) async def on_stop(self) -> None: + """Call when application stops. + + Tip: + Remember to call ``super`` if you override this method. + """ await self._stop_consumer() # send shutdown signal await self.on_before_shutdown.send() @@ -1320,6 +1360,7 @@ async def _consumer_wait_empty( await consumer.wait_empty() def on_rebalance_start(self) -> None: + """Call when rebalancing starts.""" self.rebalancing = True self.rebalancing_count += 1 if self.tracer: @@ -1331,6 +1372,7 @@ def on_rebalance_start(self) -> None: self.tables.on_rebalance_start() def on_rebalance_end(self) -> None: + """Call when rebalancing is done.""" self.rebalancing = False if self._rebalancing_span: self._rebalancing_span.finish() @@ -1499,9 +1541,11 @@ def FlowControlQueue( ) def Worker(self, **kwargs: Any) -> _Worker: + """Return application worker instance.""" return self.conf.Worker(self, **kwargs) def on_webserver_init(self, web: Web) -> None: + """Call when the Web server is initializing.""" ... def _create_directories(self) -> None: @@ -1577,6 +1621,7 @@ def _load_settings_from_source(self, source: Any, *, @property def conf(self) -> _Settings: + """Application configuration.""" if not self.finalized and STRICT: raise ImproperlyConfigured( 'App configuration accessed before app.finalize()') @@ -1590,6 +1635,7 @@ def conf(self, settings: _Settings) -> None: @property def producer(self) -> ProducerT: + """Message producer.""" if self._producer is None: self._producer = self._new_producer() return self._producer @@ -1600,6 +1646,7 @@ def producer(self, producer: ProducerT) -> None: @property def consumer(self) -> ConsumerT: + """Message consumer.""" if self._consumer is None: self._consumer = self._new_consumer() return self._consumer @@ -1621,6 +1668,7 @@ def transport(self, transport: TransportT) -> None: @property def cache(self) -> CacheBackendT: + """Cache backend.""" if self._cache is None: self._cache = self._new_cache_backend() return self._cache @@ -1685,7 +1733,7 @@ def flow_control(self) -> FlowControlEvent: @property def http_client(self) -> HttpClientT: - """HTTP Client Session.""" + """HTTP client Session.""" if self._http_client is None: self._http_client = self.conf.HttpClient() return self._http_client @@ -1729,6 +1777,7 @@ def router(self) -> RouterT: @cached_property def web(self) -> Web: + """Web driver.""" return self._new_web() def _new_web(self) -> Web: @@ -1736,7 +1785,7 @@ def _new_web(self) -> Web: @cached_property def serializers(self) -> RegistryT: - # Serializer registry. + """Return serializer registry.""" # Many things such as key_serializer/value_serializer configures # the serializer by name (e.g. "json"). The serializer registry # lets you extend Faust with support for additional @@ -1749,8 +1798,10 @@ def serializers(self) -> RegistryT: @property def label(self) -> str: + """Return human readable description of application.""" return f'{self.shortlabel}: {self.conf.id}@{self.conf.broker}' @property def shortlabel(self) -> str: + """Return short description of application.""" return type(self).__name__ diff --git a/faust/app/router.py b/faust/app/router.py index efd227175..eb99725fd 100644 --- a/faust/app/router.py +++ b/faust/app/router.py @@ -21,17 +21,20 @@ def __init__(self, app: AppT) -> None: self._assignor = self.app.assignor def key_store(self, table_name: str, key: K) -> URL: + """Return the URL of web server that hosts key in table.""" table = self._get_table(table_name) topic = self._get_table_topic(table) k = self._get_serialized_key(table, key) return self._assignor.key_store(topic, k) def table_metadata(self, table_name: str) -> HostToPartitionMap: + """Return metadata stored for table in the partition assignor.""" table = self._get_table(table_name) topic = self._get_table_topic(table) return self._assignor.table_metadata(topic) def tables_metadata(self) -> HostToPartitionMap: + """Return metadata stored for all tables in the partition assignor.""" return self._assignor.tables_metadata() @classmethod @@ -47,6 +50,14 @@ def _get_table(self, name: str) -> CollectionT: async def route_req(self, table_name: str, key: K, web: Web, request: Request) -> Response: + """Route request to worker having key in table. + + Arguments: + table_name: Name of the table. + key: The key that we want. + web: The currently sued web driver, + request: The web request currently being served. + """ app = self.app try: dest_url: URL = app.router.key_store(table_name, key) diff --git a/faust/channels.py b/faust/channels.py index 24fce75e5..8873a2401 100644 --- a/faust/channels.py +++ b/faust/channels.py @@ -243,12 +243,12 @@ async def publish_message(self, fut: FutureMessage, wait: bool = True) -> Awaitable[RecordMetadata]: """Publish message to channel. - This is the interace used by ``topic.send()``, etc. + This is the interface used by ``topic.send()``, etc. to actually publish the message on the channel after being buffered up or similar. It takes a :class:`~faust.types.FutureMessage` object, - wich contains all the information required to send + which contains all the information required to send the message, and acts as a promise that is resolved once the message has been fully transmitted. """ @@ -404,7 +404,7 @@ def on_stop_iteration(self) -> None: """Signal that iteration over this channel was stopped. Tip: - Remember to call :keyword:`super` when overriding this method. + Remember to call ``super`` when overriding this method. """ ... diff --git a/faust/cli/__init__.py b/faust/cli/__init__.py index 77a4469cc..89221e532 100644 --- a/faust/cli/__init__.py +++ b/faust/cli/__init__.py @@ -1,3 +1,4 @@ +"""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 fce18b242..0c32c961d 100644 --- a/faust/cli/agents.py +++ b/faust/cli/agents.py @@ -18,6 +18,7 @@ class agents(AppCommand): ] async def run(self, local: bool) -> None: + """Dump list of available agents in this application.""" self.say( self.tabulate( [ @@ -29,6 +30,7 @@ async def run(self, local: bool) -> None: )) def agents(self, *, local: bool = False) -> Sequence[AgentT]: + """Convert list of agents to terminal table rows.""" sortkey = cast(Callable[[Type[AgentT]], Any], self.sortkey) return [ agent @@ -37,6 +39,7 @@ def agents(self, *, local: bool = False) -> Sequence[AgentT]: ] def agent_to_row(self, agent: AgentT) -> Sequence[str]: + """Convert agent fields to terminal table row.""" return [ self.bold_tail(self._name(agent)), self._topic(agent), diff --git a/faust/cli/base.py b/faust/cli/base.py index 2b44ab678..fb4e0e503 100644 --- a/faust/cli/base.py +++ b/faust/cli/base.py @@ -485,6 +485,7 @@ class Command(abc.ABC): @classmethod def as_click_command(cls) -> Callable: # pragma: no cover + """Convert command into :pypi:`click` command.""" # This is what actually registers the commands into the # :pypi:`click` command-line interface (the ``def cli`` main above). # __init_subclass__ calls this for the side effect of being @@ -554,24 +555,33 @@ def __init__(self, ctx: click.Context, *args: Any, **kwargs: Any) -> None: @no_type_check # Subclasses can omit *args, **kwargs in signature. async def run(self, *args: Any, **kwargs: Any) -> Any: + """Override this method to define what your command does.""" # NOTE: If you override __call__ below, you have a non-async command. # This is used by .worker to call the # Worker.execute_from_commandline() method. ... async def execute(self, *args: Any, **kwargs: Any) -> Any: + """Execute command.""" try: await self.run(*args, **kwargs) finally: await self.on_stop() async def on_stop(self) -> None: + """Call after command executed.""" ... def __call__(self, *args: Any, **kwargs: Any) -> NoReturn: + """Call command-line command. + + This will raise :exc:`SystemExit` before returning, + and the exit code will be set accordingly. + """ self.run_using_worker(*args, **kwargs) def run_using_worker(self, *args: Any, **kwargs: Any) -> NoReturn: + """Execute command using :class:`faust.Worker`.""" loop = asyncio.get_event_loop() args = self.args + args kwargs = {**self.kwargs, **kwargs} @@ -581,11 +591,13 @@ def run_using_worker(self, *args: Any, **kwargs: Any) -> NoReturn: raise worker.execute_from_commandline() def on_worker_created(self, worker: Worker) -> None: + """Call when creating :class:`faust.Worker` to execute this command.""" ... def as_service(self, loop: asyncio.AbstractEventLoop, *args: Any, **kwargs: Any) -> Service: + """Wrap command in a :class:`mode.Service` object.""" return Service.from_awaitable( self.execute(*args, **kwargs), name=type(self).__name__, @@ -594,6 +606,7 @@ def as_service(self, def worker_for_service(self, service: ServiceT, loop: asyncio.AbstractEventLoop = None) -> Worker: + """Create :class:`faust.Worker` instance for this command.""" return self._Worker( service, debug=self.debug, @@ -716,10 +729,12 @@ def carp(self, s: Any, **kwargs: Any) -> None: self.say(f'#-- {s}', **kwargs) def dumps(self, obj: Any) -> str: + """Serialize object using JSON.""" return json.dumps(obj) @property def loglevel(self) -> str: + """Return the log level used for this command.""" return self._loglevel or DEFAULT_LOGLEVEL @loglevel.setter @@ -728,6 +743,7 @@ def loglevel(self, level: str) -> None: @property def blocking_timeout(self) -> float: + """Return the blocking timeout used for this command.""" return self._blocking_timeout or BLOCKING_TIMEOUT @blocking_timeout.setter @@ -736,6 +752,7 @@ def blocking_timeout(self, timeout: float) -> None: @property def console_port(self) -> int: + """Return the :pypi:`aiomonitor` console port.""" return self._console_port or CONSOLE_PORT @console_port.setter @@ -765,6 +782,7 @@ def from_handler( cls, *options: Any, **kwargs: Any) -> Callable[[Callable], Type['AppCommand']]: + """Decorate ``async def`` command to create command class.""" def _inner(fun: Callable[..., Awaitable[Any]]) -> Type['AppCommand']: target: Any = fun if not inspect.signature(fun).parameters: @@ -841,6 +859,8 @@ def _detect_main_package(self, argv: List[str]) -> str: # pragma: no cover return package async def on_stop(self) -> None: + """Call after command executed.""" + await super().on_stop() app = self.app # If command started the producer, we should also stop that # - this will flush any buffers before exiting. diff --git a/faust/cli/clean_versions.py b/faust/cli/clean_versions.py index 985d578a1..de1d8dc52 100644 --- a/faust/cli/clean_versions.py +++ b/faust/cli/clean_versions.py @@ -15,9 +15,11 @@ class clean_versions(AppCommand): """ async def run(self) -> None: + """Execute command.""" self.remove_old_versiondirs() def remove_old_versiondirs(self) -> None: + """Remove data from old application versions from data directory.""" for dir in self.app.conf.find_old_versiondirs(): self.say(f'Removing old version directory {dir}...') rmtree(str(dir)) diff --git a/faust/cli/completion.py b/faust/cli/completion.py index 6fc9d2039..49e98d09c 100644 --- a/faust/cli/completion.py +++ b/faust/cli/completion.py @@ -20,6 +20,7 @@ class completion(AppCommand): require_app = False async def run(self) -> None: + """Dump click completion script for Faust CLI.""" if click_completion is None: raise self.UsageError( 'Missing required dependency, but this is easy to fix.\n' @@ -28,5 +29,6 @@ async def run(self) -> None: self.say(click_completion.get_code(shell=self.shell())) def shell(self) -> str: + """Return the current shell used in this environment.""" shell_path = Path(os.environ.get('SHELL', 'auto')) return shell_path.stem diff --git a/faust/cli/model.py b/faust/cli/model.py index 162e99cac..ee00b087e 100644 --- a/faust/cli/model.py +++ b/faust/cli/model.py @@ -28,6 +28,7 @@ class model(AppCommand): ] async def run(self, name: str) -> None: + """Dump list of registered models to terminal.""" try: model = registry[name] except KeyError: @@ -59,9 +60,11 @@ def _unknown_model(self, name: str, *, return click.UsageError(f'No model {name!r}. {alt}') def model_fields(self, model: Type[ModelT]) -> terminal.TableDataT: + """Convert model fields to terminal table rows.""" return [self.field(getattr(model, k)) for k in model._options.fields] def field(self, field: FieldDescriptorT) -> Sequence[str]: + """Convert model field model to terminal table columns.""" return [ field.field, self._type(field.type), @@ -72,6 +75,7 @@ def _type(self, typ: Any) -> str: return typ.__name__ if typ in BUILTIN_TYPES else repr(typ) def model_to_row(self, model: Type[ModelT]) -> Sequence[str]: + """Convert model to terminal table row.""" return [ self.bold_tail(self._name(model)), self.dark(self._help(model)), diff --git a/faust/cli/models.py b/faust/cli/models.py index 019dd89a4..6d5786a3b 100644 --- a/faust/cli/models.py +++ b/faust/cli/models.py @@ -20,6 +20,7 @@ class models(AppCommand): ] async def run(self, *, builtins: bool) -> None: + """Dump list of available models in this application.""" self.say( self.tabulate( [self.model_to_row(model) for model in self.models(builtins)], @@ -28,6 +29,7 @@ async def run(self, *, builtins: bool) -> None: )) def models(self, builtins: bool) -> Sequence[Type[ModelT]]: + """Convert list of models to terminal table rows.""" sortkey = cast(Callable[[Type[ModelT]], Any], self.sortkey) return [ model for model in sorted(registry.values(), key=sortkey) @@ -35,6 +37,7 @@ def models(self, builtins: bool) -> Sequence[Type[ModelT]]: ] def model_to_row(self, model: Type[ModelT]) -> Sequence[str]: + """Convert model fields to terminal table columns.""" return [ self.bold_tail(self._name(model)), self.dark(self._help(model)), diff --git a/faust/cli/params.py b/faust/cli/params.py index afd3ca482..8f8a1ff46 100644 --- a/faust/cli/params.py +++ b/faust/cli/params.py @@ -40,6 +40,7 @@ def convert(self, value: str, param: Optional[click.Parameter], ctx: Optional[click.Context]) -> Any: + """Convert string to case-insensitive choice.""" if value.lower() in self.choices: return value return super().convert(value, param, ctx) @@ -71,6 +72,7 @@ def convert(self, value: str, param: Optional[click.Parameter], ctx: Optional[click.Context]) -> URL: + """Convert :class:`str` argument to :class:`yarl.URL`.""" text_value = self._string_param.convert(value, param, ctx) return URL(text_value) diff --git a/faust/cli/reset.py b/faust/cli/reset.py index 383609c72..f3d5acae3 100644 --- a/faust/cli/reset.py +++ b/faust/cli/reset.py @@ -23,9 +23,11 @@ class reset(AppCommand): """ async def run(self) -> None: + """Execute command.""" await self.reset_tables() async def reset_tables(self) -> None: + """Reset local state for all tables.""" for table in self.app.tables.values(): self.say(f'Removing database for table {table.name}...') table.reset_state() diff --git a/faust/cli/send.py b/faust/cli/send.py index 5ba2985a9..f525fa5e2 100644 --- a/faust/cli/send.py +++ b/faust/cli/send.py @@ -59,6 +59,7 @@ async def run(self, min_latency: float = 0.0, max_latency: float = 0.0, **kwargs: Any) -> Any: + """Send message to topic/agent/channel.""" if key is not None: key = self.to_key(key_type, key) if value is not None: diff --git a/faust/cli/tables.py b/faust/cli/tables.py index de448e569..11cb49494 100644 --- a/faust/cli/tables.py +++ b/faust/cli/tables.py @@ -10,6 +10,7 @@ class tables(AppCommand): title = 'Tables' async def run(self) -> None: + """Dump list of application tables to terminal.""" self.say( self.tabulate( [(self.bold(table.name), diff --git a/faust/cli/worker.py b/faust/cli/worker.py index b89041bbc..1bb2dea98 100644 --- a/faust/cli/worker.py +++ b/faust/cli/worker.py @@ -51,10 +51,18 @@ class worker(AppCommand): cast(List, now_builtin_worker_options)) def on_worker_created(self, worker: Worker) -> None: + """Print banner when worker starts.""" self.say(self.banner(worker)) def as_service(self, loop: asyncio.AbstractEventLoop, *args: Any, **kwargs: Any) -> ServiceT: + """Return the service this command should execute. + + For the worker we simply start the application itself. + + Note: + The application will be started using a :class:`faust.Worker`. + """ self._init_worker_options(*args, **kwargs) return self.app @@ -133,9 +141,11 @@ def _driver_versions(self, app: AppT) -> List[str]: ] def faust_ident(self) -> str: + """Return Faust version information as ANSI string.""" return self.color('hiblue', f'{FAUST} v{faust_version}') def platform(self) -> str: + """Return platform identifier as ANSI string.""" return '{py_imp} {py_version} ({system} {machine})'.format( py_imp=platform.python_implementation(), py_version=platform.python_version(), diff --git a/faust/events.py b/faust/events.py index 43de508fc..4680e6f54 100644 --- a/faust/events.py +++ b/faust/events.py @@ -239,7 +239,7 @@ def ack(self) -> bool: When the last stream processor acks the message, the offset in the source topic will be marked as safe-to-commit, - and the worker will commit and advance the commited offset. + and the worker will commit and advance the committed offset. """ return self.message.ack(self.app.consumer) diff --git a/faust/fixups/base.py b/faust/fixups/base.py index 4ef0bd3eb..c024d1a40 100644 --- a/faust/fixups/base.py +++ b/faust/fixups/base.py @@ -16,10 +16,13 @@ def __init__(self, app: AppT) -> None: self.app = app def enabled(self) -> bool: + """Return if fixup should be enabled in this environment.""" return False def autodiscover_modules(self) -> Iterable[str]: + """Return list of additional autodiscover modules.""" return [] def on_worker_init(self) -> None: + """Call when initializing worker/CLI commands.""" ... diff --git a/faust/fixups/django.py b/faust/fixups/django.py index 597e26b3e..f41c85656 100644 --- a/faust/fixups/django.py +++ b/faust/fixups/django.py @@ -53,6 +53,7 @@ class Fixup(base.Fixup): """ def enabled(self) -> bool: + """Return :const:`True` if Django is used in this environment.""" if os.environ.get('DJANGO_SETTINGS_MODULE'): try: import django # noqa @@ -63,9 +64,16 @@ def enabled(self) -> bool: return False def autodiscover_modules(self) -> Iterable[str]: + """Return list of additional autodiscover modules. + + For Django we run autodiscovery in all packages + listed in the ``INSTALLED_APPS`` setting (with support for + custom app configurations). + """ return [config.name for config in self.apps.get_app_configs()] def on_worker_init(self) -> None: + """Initialize Django before worker/CLI command starts.""" import django from django.core.checks import run_checks django.setup() @@ -75,8 +83,10 @@ def on_worker_init(self) -> None: @cached_property def apps(self) -> _Apps: + """Return the Django app registry.""" return symbol_by_name('django.apps:apps') @cached_property def settings(self) -> _Settings: + """Return the Django settings object.""" return symbol_by_name('django.conf:settings') diff --git a/faust/livecheck/__init__.py b/faust/livecheck/__init__.py index ac3256a7b..49e0b69e5 100644 --- a/faust/livecheck/__init__.py +++ b/faust/livecheck/__init__.py @@ -1,6 +1,8 @@ +"""LiveCheck - End-to-end testing of asynchronous systems.""" from .app import LiveCheck from .case import Case from .locals import current_test +from .runners import TestRunner from .signals import Signal -__all__ = ['LiveCheck', 'Case', 'Signal', 'current_test'] +__all__ = ['LiveCheck', 'Case', 'Signal', 'TestRunner', 'current_test'] diff --git a/faust/livecheck/app.py b/faust/livecheck/app.py index ca4f18eab..ed848ab84 100644 --- a/faust/livecheck/app.py +++ b/faust/livecheck/app.py @@ -43,6 +43,7 @@ def on_stream_event_in(self, offset: int, stream: StreamT, event: EventT) -> None: + """Call when stream starts processing event.""" test = TestExecution.from_headers(event.headers) if test is not None: stream.current_test = test @@ -53,6 +54,7 @@ def on_stream_event_out(self, offset: int, stream: StreamT, event: EventT) -> None: + """Call when stream is finished handling event.""" has_active_test = getattr(stream, 'current_test', None) if has_active_test: stream.current_test = None @@ -98,6 +100,10 @@ def for_app(cls, app: AppT, *, test_concurrency: int = None, send_reports: bool = None, **kwargs: Any) -> 'LiveCheck': + """Create LiveCheck application targeting specific app. + + The target app will be used to configure the LiveCheck app. + """ app_id, passed_kwargs = app._default_options livecheck_id = f'{prefix}{app_id}' override = { @@ -155,6 +161,7 @@ def __init__(self, @property def current_test(self) -> Optional[TestExecution]: + """Return the current test context (if any).""" return current_test() @cached_property @@ -176,6 +183,7 @@ def on_produce_attach_test_headers( timestamp: float = None, headers: List[Tuple[str, bytes]] = None, **kwargs: Any) -> None: + """Attach test headers to Kafka produce requests.""" test = current_test() if test is not None: if headers is None: @@ -200,6 +208,11 @@ def case(self, *, url_error_delay_backoff: float = None, url_error_delay_max: float = None, base: Type[_Case] = Case) -> Callable[[Type], _Case]: + """Decorate class to be used as a test case. + + Returns: + :class:`faust.livecheck.Case`. + """ base_case = base def _inner(cls: Type) -> _Case: @@ -261,21 +274,25 @@ def _extract_signals(self, pass def add_case(self, case: _Case) -> _Case: + """Add and register new test case.""" self.cases[case.name] = case return case async def post_report(self, report: TestReport) -> None: + """Publish test report to reporting topic.""" key = None if report.test is not None: key = report.test.id await self.reports.send(key=key, value=report) async def on_start(self) -> None: + """Call when LiveCheck application starts.""" await super().on_start() self._install_bus_agent() self._install_test_execution_agent() async def on_started(self) -> None: + """Call when LiveCheck application is fully started.""" await super().on_started() for case in self.cases.values(): await self.add_runtime_dependency(case) @@ -316,6 +333,7 @@ def _prepare_case_name(self, name: str) -> str: @cached_property def bus(self) -> TopicT: + """Topic used for signal communication.""" return self.topic( self.bus_topic_name, key_type=str, @@ -324,6 +342,7 @@ def bus(self) -> TopicT: @cached_property def pending_tests(self) -> TopicT: + """Topic used to keep pending test executions.""" return self.topic( self.test_topic_name, key_type=str, @@ -332,6 +351,7 @@ def pending_tests(self) -> TopicT: @cached_property def reports(self) -> TopicT: + """Topic used to log test reports.""" return self.topic( self.report_topic_name, key_type=str, diff --git a/faust/livecheck/case.py b/faust/livecheck/case.py index f9b505faf..78b5b9545 100644 --- a/faust/livecheck/case.py +++ b/faust/livecheck/case.py @@ -75,7 +75,7 @@ class Case(Service): #: The warn_stalled_after timer uses this to keep track of #: either when a test was last received, or the last time the timer - #: tiemd out. + #: timed out. last_test_received: Optional[float] = None #: Timestamp of when the suite last failed. @@ -202,6 +202,7 @@ async def maybe_trigger( self, id: str = None, *args: Any, **kwargs: Any) -> AsyncGenerator[Optional[TestExecution], None]: + """Schedule test execution, or not, based on probability setting.""" execution: Optional[TestExecution] = None with ExitStack() as exit_stack: if uniform(0, 1) < self.probability: @@ -212,6 +213,7 @@ async def maybe_trigger( async def trigger(self, id: str = None, *args: Any, **kwargs: Any) -> TestExecution: + """Schedule test execution ASAP.""" id = id or uuid() execution = TestExecution( id=id, @@ -228,12 +230,15 @@ def _now(self) -> datetime: return datetime.utcnow().astimezone(timezone.utc) async def run(self, *test_args: Any, **test_kwargs: Any) -> None: + """Override this to define your test case.""" raise NotImplementedError('Case class must implement run') async def resolve_signal(self, key: str, event: SignalEvent) -> None: + """Mark test execution signal as resolved.""" await self.signals[event.signal_name].resolve(key, event) async def execute(self, test: TestExecution) -> None: + """Execute test using :class:`TestRunner`.""" t_start = monotonic() runner = self.Runner(self, test, started=t_start) with current_execution_stack.push(runner): @@ -241,6 +246,7 @@ async def execute(self, test: TestExecution) -> None: await runner.execute() async def on_test_start(self, runner: TestRunner) -> None: + """Call when a test starts executing.""" started = runner.started t_prev, self.last_test_received = self.last_test_received, started if t_prev: @@ -254,6 +260,7 @@ async def on_test_start(self, runner: TestRunner) -> None: self.frequency_history, time_since, self.max_history) async def on_test_skipped(self, runner: TestRunner) -> None: + """Call when a test is skipped.""" # wait until we have fast forwarded before raising errors # XXX should we use seek, or warn somehow if this # takes too long? @@ -262,16 +269,19 @@ async def on_test_skipped(self, runner: TestRunner) -> None: async def on_test_failed(self, runner: TestRunner, exc: BaseException) -> None: + """Call when invariant in test execution fails.""" await self._set_test_error_state(State.FAIL) async def on_test_error(self, runner: TestRunner, exc: BaseException) -> None: + """Call when a test execution raises an exception.""" await self._set_test_error_state(State.ERROR) async def on_test_timeout(self, runner: TestRunner, exc: BaseException) -> None: + """Call when a test execution times out.""" await self._set_test_error_state(State.TIMEOUT) async def _set_test_error_state(self, state: State) -> None: @@ -295,6 +305,7 @@ def _set_pass_state(self, state: State) -> None: self.total_by_state[state] += 1 async def on_test_pass(self, runner: TestRunner) -> None: + """Call when a test execution passes.""" test = runner.test runtime = runner.runtime deque_pushpopmax(self.runtime_history, runtime, self.max_history) @@ -304,6 +315,7 @@ async def on_test_pass(self, runner: TestRunner) -> None: self._maybe_recover_from_failed_state() async def post_report(self, report: TestReport) -> None: + """Publish test report.""" await self.app.post_report(report) @Service.task @@ -348,6 +360,7 @@ async def _check_frequency(self) -> None: async def on_suite_fail(self, exc: SuiteFailed, new_state: State = State.FAIL) -> None: + """Call when the suite fails.""" assert isinstance(exc, SuiteFailed) delay = self.state_transition_delay if self.state.is_ok() or self._failed_longer_than(delay): @@ -381,19 +394,23 @@ def _failed_longer_than(self, secs: float) -> bool: @property def seconds_since_last_fail(self) -> Optional[float]: + """Return number of seconds since any test failed.""" last_fail = self.last_fail return monotonic() - last_fail if last_fail else None async def get_url(self, url: Union[str, URL], **kwargs: Any) -> Optional[bytes]: + """Perform GET request using HTTP client.""" return await self.url_request('get', url, **kwargs) async def post_url(self, url: Union[str, URL], **kwargs: Any) -> Optional[bytes]: + """Perform POST request using HTTP client.""" return await self.url_request('post', url, **kwargs) async def url_request(self, method: str, url: Union[str, URL], **kwargs: Any) -> Optional[bytes]: + """Perform URL request using HTTP client.""" timeout = ClientTimeout( total=self.url_timeout_total, connect=self.url_timeout_connect, @@ -431,12 +448,15 @@ async def url_request(self, method: str, url: Union[str, URL], @property def current_test(self) -> Optional[TestExecution]: + """Return the currently active test in this task (if any).""" return current_test_stack.top @property def current_execution(self) -> Optional[TestRunner]: + """Return the currently executing :class:`TestRunner` in this task.""" return current_execution_stack.top @property def label(self) -> str: + """Return human-readable label for this test case.""" return f'{type(self).__name__}: {self.name}' diff --git a/faust/livecheck/exceptions.py b/faust/livecheck/exceptions.py index 3c8983845..6a980444a 100644 --- a/faust/livecheck/exceptions.py +++ b/faust/livecheck/exceptions.py @@ -13,7 +13,7 @@ class LiveCheckError(Exception): - """Generic base class for livecheck test errors.""" + """Generic base class for LiveCheck test errors.""" class SuiteFailed(LiveCheckError): diff --git a/faust/livecheck/locals.py b/faust/livecheck/locals.py index 793a1e057..9dd035686 100644 --- a/faust/livecheck/locals.py +++ b/faust/livecheck/locals.py @@ -1,3 +1,4 @@ +"""Locals - Current test & execution context.""" import typing from typing import Optional from mode.locals import LocalStack @@ -8,7 +9,12 @@ else: # pragma: no cover class _TestRunner: ... # noqa -__all__ = ['current_execution_stack', 'current_test', 'current_test_stack'] +__all__ = [ + 'current_execution', + 'current_execution_stack', + 'current_test', + 'current_test_stack', +] current_test_stack: LocalStack[TestExecution] current_test_stack = LocalStack() @@ -17,6 +23,11 @@ class _TestRunner: ... # noqa current_execution_stack = LocalStack() +def current_execution() -> Optional[_TestRunner]: + """Return the current :class:`~faust.livecheck.TestRunner`.""" + return current_execution_stack.top + + def current_test() -> Optional[TestExecution]: """Return information about the current test (if any).""" return current_test_stack.top diff --git a/faust/livecheck/models.py b/faust/livecheck/models.py index 3cf790c68..eb680b786 100644 --- a/faust/livecheck/models.py +++ b/faust/livecheck/models.py @@ -19,6 +19,8 @@ class State(Enum): + """Test execution status.""" + INIT = 'INIT' PASS = 'PASS' FAIL = 'FAIL' @@ -28,6 +30,7 @@ class State(Enum): SKIP = 'SKIP' def is_ok(self) -> bool: + """Return :const:`True` if this is considered an OK state.""" return self in OK_STATES @@ -44,7 +47,7 @@ class SignalEvent(Record): class TestExecution(Record, isodates=True): - """Requested test excecution.""" + """Requested test execution.""" id: str case_name: str @@ -55,6 +58,7 @@ class TestExecution(Record, isodates=True): @classmethod def from_headers(cls, headers: Mapping) -> Optional['TestExecution']: + """Create instance from mapping of HTTP/Kafka headers.""" try: test_id = want_str(headers[HEADER_TEST_ID]) except KeyError: @@ -74,6 +78,7 @@ def from_headers(cls, headers: Mapping) -> Optional['TestExecution']: ) def as_headers(self) -> Mapping: + """Return test metadata as mapping of HTTP/Kafka headers.""" return { HEADER_TEST_ID: self.id, HEADER_TEST_NAME: self.case_name, @@ -83,10 +88,12 @@ def as_headers(self) -> Mapping: @cached_property def ident(self) -> str: + """Return long identifier for this test used in logs.""" return self._build_ident(self.case_name, self.id) @cached_property def shortident(self) -> str: + """Return short identifier for this test used in logs.""" return self._build_ident( self.short_case_name, abbr(self.id, max=15, suffix='[...]'), @@ -100,6 +107,7 @@ def _now(self) -> datetime: @cached_property def human_date(self) -> str: + """Return human-readable description of test timestamp.""" if self.was_issued_today: return f'''Today {self.timestamp.strftime('%H:%M:%S')}''' else: @@ -107,14 +115,17 @@ def human_date(self) -> str: @cached_property def was_issued_today(self) -> bool: + """Return :const:`True` if test was issued on todays date.""" return self.timestamp.date() == self._now().date() @cached_property def is_expired(self) -> bool: + """Return :const:`True` if this test already expired.""" return self._now() >= self.expires @cached_property def short_case_name(self) -> str: + """Return abbreviated case name.""" return self.case_name.split('.')[-1] diff --git a/faust/livecheck/patches/__init__.py b/faust/livecheck/patches/__init__.py index 89f64e79f..9a1488ad1 100644 --- a/faust/livecheck/patches/__init__.py +++ b/faust/livecheck/patches/__init__.py @@ -1,7 +1,9 @@ +"""Patches - LiveCheck integration with other frameworks/libraries.""" from . import aiohttp __all__ = ['aiohttp', 'patch_all'] def patch_all() -> None: + """Apply all LiveCheck monkey patches.""" aiohttp.patch_all() diff --git a/faust/livecheck/patches/aiohttp.py b/faust/livecheck/patches/aiohttp.py index 61d7c3c1c..f8d5ee94b 100644 --- a/faust/livecheck/patches/aiohttp.py +++ b/faust/livecheck/patches/aiohttp.py @@ -1,3 +1,4 @@ +"""LiveCheck :pypi:`aiohttp` integration.""" from contextlib import ExitStack from types import SimpleNamespace from typing import Any, List, Optional @@ -10,10 +11,16 @@ def patch_all() -> None: + """Patch all :pypi:`aiohttp` functions to integrate with LiveCheck.""" patch_aiohttp_session() def patch_aiohttp_session() -> None: + """Patch :class:`aiohttp.ClientSession` to integrate with LiveCheck. + + If there is any currently active test, we will + use that to forward LiveCheck HTTP headers to the new HTTP request. + """ from aiohttp import TraceConfig from aiohttp import client @@ -48,8 +55,17 @@ def __init__(self, @web.middleware class LiveCheckMiddleware: + """LiveCheck support for :pypi:`aiohttp` web servers. + + This middleware is applied to all incoming web requests, + and is used to extract LiveCheck HTTP headers. + + If the web request is configured with the correct set of LiveCheck + headers, we will use that to set the "current test" context. + """ async def __call__(self, request: web.Request, handler: Any) -> Any: + """Call to handle new web request.""" related_test = TestExecution.from_headers(request.headers) with ExitStack() as stack: if related_test: diff --git a/faust/livecheck/runners.py b/faust/livecheck/runners.py index f2c2a4acc..2bdb32834 100644 --- a/faust/livecheck/runners.py +++ b/faust/livecheck/runners.py @@ -40,6 +40,7 @@ class _Case: ... # noqa class TestRunner: + """Execute and keep track of test instance.""" case: _Case @@ -73,6 +74,7 @@ def __init__(self, self.signal_latency = {} async def execute(self) -> None: + """Execute this test.""" test = self.test with current_test_stack.push(test): if not self.case.active: @@ -105,6 +107,7 @@ async def execute(self) -> None: await self.on_pass() async def skip(self, reason: str) -> NoReturn: + """Skip this test execution.""" exc = TestSkipped(f'Test {self.test.ident} skipped: {reason}') try: raise exc @@ -134,17 +137,20 @@ def _format_log(self, severity: int, msg: str, return f'[{self.test.shortident}] {msg}' async def on_skipped(self, exc: TestSkipped) -> None: + """Call when a test execution was skipped.""" self.state = State.SKIP self.log.info('Skipped expired test: %s expires=%s', self.test.ident, self.test.expires) await self.case.on_test_skipped(self) async def on_start(self) -> None: + """Call when a test starts executing.""" self.log_info('≈≈≈ Test %s executing... (issued %s) ≈≈≈', self.case.name, self.test.human_date) await self.case.on_test_start(self) async def on_signal_wait(self, signal: BaseSignal, timeout: float) -> None: + """Call when the test is waiting for a signal.""" self.log_info('∆ %r/%r %s (%rs)...', signal.index, self.case.total_signals, @@ -155,10 +161,12 @@ async def on_signal_received(self, signal: BaseSignal, time_start: float, time_end: float) -> None: + """Call when a signal related to this test is received.""" latency = time_end - time_start self.signal_latency[signal.name] = latency async def on_failed(self, exc: BaseException) -> None: + """Call when an invariant in the test has failed.""" self.end() self.error = exc self.state = State.FAIL @@ -167,6 +175,7 @@ async def on_failed(self, exc: BaseException) -> None: await self._finalize_report() async def on_error(self, exc: BaseException) -> None: + """Call when test execution raises error.""" self.end() self.state = State.ERROR self.error = exc @@ -175,6 +184,7 @@ async def on_error(self, exc: BaseException) -> None: await self._finalize_report() async def on_timeout(self, exc: BaseException) -> None: + """Call when test execution times out.""" self.end() self.error = exc self.state = State.TIMEOUT @@ -183,6 +193,7 @@ async def on_timeout(self, exc: BaseException) -> None: await self._finalize_report() async def on_pass(self) -> None: + """Call when test execution returns successfully.""" self.end() self.error = None self.state = State.PASS @@ -213,12 +224,14 @@ async def _finalize_report(self) -> None: await self.case.post_report(self.report) def log_info(self, msg: str, *args: Any) -> None: + """Log information related to the current execution.""" if self.case.realtime_logs: self.log.info(msg, *args) else: self.logs.append((msg, args)) def end(self) -> None: + """End test execution.""" self.ended = monotonic() self.runtime = self.ended - self.started diff --git a/faust/livecheck/signals.py b/faust/livecheck/signals.py index 5ba8b96fb..38db805f8 100644 --- a/faust/livecheck/signals.py +++ b/faust/livecheck/signals.py @@ -1,3 +1,4 @@ +"""LiveCheck Signals - Test communication and synchronization.""" import asyncio import typing from time import monotonic @@ -38,14 +39,17 @@ def __init__(self, async def send(self, value: VT = None, *, key: Any = None, force: bool = False) -> None: + """Notify test that this signal is now complete.""" raise NotImplementedError() async def wait(self, *, key: Any = None, timeout: Seconds = None) -> VT: + """Wait for signal to be completed.""" raise NotImplementedError() async def resolve(self, key: Any, event: SignalEvent) -> None: + """Resolve signal with value.""" self._set_current_value(key, event) self._wakeup_resolvers() @@ -71,6 +75,7 @@ def _set_current_value(self, key: Any, event: SignalEvent) -> None: self.case.app._resolved_signals[self._index_key(key)] = event def clone(self, **kwargs: Any) -> 'BaseSignal': + """Clone this signal using keyword arguments.""" return type(self)(**{**self._asdict(), **kwargs}) def _asdict(self, **kwargs: Any) -> Dict: @@ -94,6 +99,7 @@ class Signal(BaseSignal[VT]): async def send(self, value: VT = None, *, key: Any = None, force: bool = False) -> None: + """Notify test that this signal is now complete.""" current_test = current_test_stack.top if current_test is None: if not force: @@ -114,6 +120,7 @@ async def send(self, value: VT = None, *, async def wait(self, *, key: Any = None, timeout: Seconds = None) -> VT: + """Wait for signal to be completed.""" # wait for key to arrive in consumer runner = self.case.current_execution if runner is None: diff --git a/faust/models/__init__.py b/faust/models/__init__.py index 40111a1f6..0e1ba06c4 100644 --- a/faust/models/__init__.py +++ b/faust/models/__init__.py @@ -1,3 +1,4 @@ +"""Models.""" from .base import FieldDescriptor, Model, ModelOptions, maybe_model, registry from .record import Record diff --git a/faust/models/base.py b/faust/models/base.py index 27cc3ac17..3e59ec31c 100644 --- a/faust/models/base.py +++ b/faust/models/base.py @@ -384,6 +384,7 @@ def _humanize(self) -> str: # pragma: no cover ... def derive(self, *objects: ModelT, **fields: Any) -> ModelT: + """Derive new model with certain fields changed.""" return self._derive(*objects, **fields) @abc.abstractmethod # pragma: no cover @@ -474,6 +475,10 @@ def __get__(self, instance: Any, owner: Type) -> Any: return instance.__dict__[self.field] def getattr(self, obj: ModelT) -> Any: + """Get attribute from model recursively. + + Supports recursive lookups e.g. ``model.getattr('x.y.z')``. + """ return attrgetter('.'.join(reversed(list(self._parents_path()))))(obj) def _parents_path(self) -> Iterable[str]: @@ -492,4 +497,5 @@ def __repr__(self) -> str: @property def ident(self) -> str: + """Return the fields identifier.""" return f'{self.model.__name__}.{self.field}' diff --git a/faust/models/record.py b/faust/models/record.py index 3784b6320..3581af04a 100644 --- a/faust/models/record.py +++ b/faust/models/record.py @@ -315,6 +315,7 @@ def _contribute_field_descriptors(cls, @classmethod def from_data(cls, data: Mapping, *, preferred_type: Type[ModelT] = None) -> 'Record': + """Create model object from Python dictionary.""" # check for blessed key to see if another model should be used. if hasattr(data, '__is_model__'): return cast(Record, data) @@ -491,6 +492,10 @@ def _derive(self, *objects: ModelT, **fields: Any) -> ModelT: return type(self)(**{**data, **fields}) def to_representation(self) -> Mapping[str, Any]: + """Convert model to its Python generic counterpart. + + Records will be converted to dictionary. + """ # Convert known fields to mapping of ``{field: value}``. payload = self.asdict() if self._options.include_metadata: @@ -498,6 +503,7 @@ def to_representation(self) -> Mapping[str, Any]: return payload def asdict(self) -> Dict[str, Any]: # pragma: no cover + """Convert record to Python dictionary.""" ... # generated by _BUILD_asdict # Used to disallow overriding this method asdict.faust_generated = True # type: ignore diff --git a/faust/sensors/__init__.py b/faust/sensors/__init__.py index 32e83da51..0bacc66a5 100644 --- a/faust/sensors/__init__.py +++ b/faust/sensors/__init__.py @@ -1,3 +1,4 @@ +"""Sensors.""" from .base import Sensor, SensorDelegate from .monitor import Monitor, TableState diff --git a/faust/sensors/base.py b/faust/sensors/base.py index 9fd305230..f212ca03c 100644 --- a/faust/sensors/base.py +++ b/faust/sensors/base.py @@ -97,20 +97,24 @@ def on_send_error(self, def on_assignment_start(self, assignor: PartitionAssignorT) -> Mapping: + """Partition assignor is starting to assign partitions.""" return {'time_start': monotonic()} def on_assignment_error(self, assignor: PartitionAssignorT, state: Mapping, exc: BaseException) -> None: + """Partition assignor did not complete assignor due to error.""" ... def on_assignment_completed(self, assignor: PartitionAssignorT, state: Mapping) -> None: + """Partition assignor completed assignment.""" ... def asdict(self) -> Mapping: + """Convert sensor state to dictionary.""" return {} @@ -124,31 +128,37 @@ def __init__(self, app: AppT) -> None: self._sensors = set() def add(self, sensor: SensorT) -> None: + """Add sensor.""" # connect beacons sensor.beacon = self.app.beacon.new(sensor) self._sensors.add(sensor) def remove(self, sensor: SensorT) -> None: + """Remove sensor.""" self._sensors.remove(sensor) def __iter__(self) -> Iterator: return iter(self._sensors) def on_message_in(self, tp: TP, offset: int, message: Message) -> None: + """Call before message is delegated to streams.""" for sensor in self._sensors: sensor.on_message_in(tp, offset, message) def on_stream_event_in(self, tp: TP, offset: int, stream: StreamT, event: EventT) -> None: + """Call when stream starts processing an event.""" for sensor in self._sensors: sensor.on_stream_event_in(tp, offset, stream, event) def on_stream_event_out(self, tp: TP, offset: int, stream: StreamT, event: EventT) -> None: + """Call when stream is done processing an event.""" for sensor in self._sensors: sensor.on_stream_event_out(tp, offset, stream, event) def on_topic_buffer_full(self, topic: TopicT) -> None: + """Call when conductor topic buffer is full and has to wait.""" for sensor in self._sensors: sensor.on_topic_buffer_full(topic) @@ -156,22 +166,27 @@ def on_message_out(self, tp: TP, offset: int, message: Message) -> None: + """Call when message is fully acknowledged and can be committed.""" for sensor in self._sensors: sensor.on_message_out(tp, offset, message) def on_table_get(self, table: CollectionT, key: Any) -> None: + """Call when value in table is retrieved.""" for sensor in self._sensors: sensor.on_table_get(table, key) def on_table_set(self, table: CollectionT, key: Any, value: Any) -> None: + """Call when new value for key in table is set.""" for sensor in self._sensors: sensor.on_table_set(table, key, value) def on_table_del(self, table: CollectionT, key: Any) -> None: + """Call when key in a table is deleted.""" for sensor in self._sensors: sensor.on_table_del(table, key) def on_commit_initiated(self, consumer: ConsumerT) -> Any: + """Call when consumer commit offset operation starts.""" # This returns arbitrary state, so we return a map from sensor->state. return { sensor: sensor.on_commit_initiated(consumer) @@ -179,6 +194,7 @@ def on_commit_initiated(self, consumer: ConsumerT) -> Any: } def on_commit_completed(self, consumer: ConsumerT, state: Any) -> None: + """Call when consumer commit offset operation completed.""" # state is now a mapping from sensor->state, so # make sure to correct the correct state to each sensor. for sensor in self._sensors: @@ -187,6 +203,7 @@ def on_commit_completed(self, consumer: ConsumerT, state: Any) -> None: def on_send_initiated(self, producer: ProducerT, topic: str, message: PendingMessage, keysize: int, valsize: int) -> Any: + """Call when message added to producer buffer.""" return { sensor: sensor.on_send_initiated( producer, topic, message, keysize, valsize) @@ -197,6 +214,7 @@ def on_send_completed(self, producer: ProducerT, state: Any, metadata: RecordMetadata) -> None: + """Call when producer finished sending message.""" for sensor in self._sensors: sensor.on_send_completed(producer, state[sensor], metadata) @@ -204,11 +222,13 @@ def on_send_error(self, producer: ProducerT, exc: BaseException, state: Any) -> None: + """Call when producer was unable to publish message.""" for sensor in self._sensors: sensor.on_send_error(producer, exc, state[sensor]) def on_assignment_start(self, assignor: PartitionAssignorT) -> Mapping: + """Partition assignor is starting to assign partitions.""" return { sensor: sensor.on_assignment_start(assignor) for sensor in self._sensors @@ -218,12 +238,14 @@ def on_assignment_error(self, assignor: PartitionAssignorT, state: Mapping, exc: BaseException) -> None: + """Partition assignor did not complete assignor due to error.""" for sensor in self._sensors: sensor.on_assignment_error(assignor, state[sensor], exc) def on_assignment_completed(self, assignor: PartitionAssignorT, state: Mapping) -> None: + """Partition assignor completed assignment.""" for sensor in self._sensors: sensor.on_assignment_completed(assignor, state[sensor]) diff --git a/faust/sensors/datadog.py b/faust/sensors/datadog.py index d793da13e..528a1a6af 100644 --- a/faust/sensors/datadog.py +++ b/faust/sensors/datadog.py @@ -161,6 +161,7 @@ def _new_datadog_stats_client(self) -> DatadogStatsClient: host=self.host, port=self.port, prefix=self.prefix, rate=self.rate) def on_message_in(self, tp: TP, offset: int, message: Message) -> None: + """Call before message is delegated to streams.""" super().on_message_in(tp, offset, message) labels = self._format_label(tp) self.client.increment('messages_received', labels=labels) @@ -169,6 +170,7 @@ def on_message_in(self, tp: TP, offset: int, message: Message) -> None: def on_stream_event_in(self, tp: TP, offset: int, stream: StreamT, event: EventT) -> None: + """Call when stream starts processing an event.""" super().on_stream_event_in(tp, offset, stream, event) labels = self._format_label(tp, stream) self.client.increment('events', labels=labels) @@ -176,6 +178,7 @@ def on_stream_event_in(self, tp: TP, offset: int, stream: StreamT, def on_stream_event_out(self, tp: TP, offset: int, stream: StreamT, event: EventT) -> None: + """Call when stream is done processing an event.""" super().on_stream_event_out(tp, offset, stream, event) labels = self._format_label(tp, stream) self.client.decrement('events_active', labels=labels) @@ -189,11 +192,13 @@ def on_message_out(self, tp: TP, offset: int, message: Message) -> None: + """Call when message is fully acknowledged and can be committed.""" super().on_message_out(tp, offset, message) self.client.decrement('messages_active', labels=self._format_label(tp)) def on_table_get(self, table: CollectionT, key: Any) -> None: + """Call when value in table is retrieved.""" super().on_table_get(table, key) self.client.increment( 'table_keys_retrieved', @@ -201,6 +206,7 @@ def on_table_get(self, table: CollectionT, key: Any) -> None: ) def on_table_set(self, table: CollectionT, key: Any, value: Any) -> None: + """Call when new value for key in table is set.""" super().on_table_set(table, key, value) self.client.increment( 'table_keys_updated', @@ -208,6 +214,7 @@ def on_table_set(self, table: CollectionT, key: Any, value: Any) -> None: ) def on_table_del(self, table: CollectionT, key: Any) -> None: + """Call when key in a table is deleted.""" super().on_table_del(table, key) self.client.increment( 'table_keys_deleted', @@ -215,6 +222,7 @@ def on_table_del(self, table: CollectionT, key: Any) -> None: ) def on_commit_completed(self, consumer: ConsumerT, state: Any) -> None: + """Call when consumer commit offset operation completed.""" super().on_commit_completed(consumer, state) self.client.timing( 'commit_latency', @@ -224,6 +232,7 @@ def on_commit_completed(self, consumer: ConsumerT, state: Any) -> None: def on_send_initiated(self, producer: ProducerT, topic: str, message: PendingMessage, keysize: int, valsize: int) -> Any: + """Call when message added to producer buffer.""" self.client.increment( 'topic_messages_sent', labels={'topic': topic}, @@ -235,6 +244,7 @@ def on_send_completed(self, producer: ProducerT, state: Any, metadata: RecordMetadata) -> None: + """Call when producer finished sending message.""" super().on_send_completed(producer, state, metadata) self.client.increment('messages_sent') self.client.timing( @@ -246,6 +256,7 @@ def on_send_error(self, producer: ProducerT, exc: BaseException, state: Any) -> None: + """Call when producer was unable to publish message.""" super().on_send_error(producer, exc, state) self.client.increment('messages_send_failed') self.client.timing( @@ -257,6 +268,7 @@ def on_assignment_error(self, assignor: PartitionAssignorT, state: Mapping, exc: BaseException) -> None: + """Partition assignor did not complete assignor due to error.""" super().on_assignment_error(assignor, state, exc) self.client.increment('assignments_error') self.client.timing( @@ -267,6 +279,7 @@ def on_assignment_error(self, def on_assignment_completed(self, assignor: PartitionAssignorT, state: Mapping) -> None: + """Partition assignor completed assignment.""" super().on_assignment_completed(assignor, state) self.client.increment('assignments_complete') self.client.timing( @@ -275,16 +288,19 @@ def on_assignment_completed(self, ) def count(self, metric_name: str, count: int = 1) -> None: + """Count metric by name.""" super().count(metric_name, count=count) self.client.increment(metric_name, value=count) def on_tp_commit(self, tp_offsets: TPOffsetMapping) -> None: + """Call when offset in topic partition is committed.""" super().on_tp_commit(tp_offsets) for tp, offset in tp_offsets.items(): self.client.gauge('committed_offset', offset, labels=self._format_label(tp)) def track_tp_end_offset(self, tp: TP, offset: int) -> None: + """Track new topic partition end offset for monitoring lags.""" super().track_tp_end_offset(tp, offset) self.client.gauge('end_offset', offset, labels=self._format_label(tp)) @@ -325,4 +341,5 @@ def _format_table_label(self, table: CollectionT) -> Dict: @cached_property def client(self) -> DatadogStatsClient: + """Return the datadog client.""" return self._new_datadog_stats_client() diff --git a/faust/sensors/monitor.py b/faust/sensors/monitor.py index fc5af471b..0587cb9ee 100644 --- a/faust/sensors/monitor.py +++ b/faust/sensors/monitor.py @@ -58,6 +58,7 @@ def __init__(self, self.keys_deleted = keys_deleted def asdict(self) -> Mapping: + """Return table state as dictionary.""" return { 'keys_retrieved': self.keys_retrieved, 'keys_updated': self.keys_updated, @@ -259,6 +260,7 @@ def _sample(self, return prev_event_total, prev_message_total def asdict(self) -> Mapping: + """Return monitor state as dictionary.""" return { 'messages_active': self.messages_active, 'messages_received_total': self.messages_received_total, @@ -322,6 +324,7 @@ def _tp_offsets_as_dict(cls, tp_offsets: TPOffsetMapping) -> TPOffsetDict: return topic_partition_offsets def on_message_in(self, tp: TP, offset: int, message: Message) -> None: + """Call before message is delegated to streams.""" # WARNING: Sensors must never keep a reference to the Message, # as this means the message won't go out of scope! self.messages_received_total += 1 @@ -332,6 +335,7 @@ def on_message_in(self, tp: TP, offset: int, message: Message) -> None: def on_stream_event_in(self, tp: TP, offset: int, stream: StreamT, event: EventT) -> None: + """Call when stream starts processing an event.""" self.events_total += 1 self.events_by_stream[stream] += 1 self.events_by_task[stream.task_owner] += 1 @@ -344,6 +348,7 @@ def on_stream_event_in(self, tp: TP, offset: int, stream: StreamT, def on_stream_event_out(self, tp: TP, offset: int, stream: StreamT, event: EventT) -> None: + """Call when stream is done processing an event.""" time_out = self.time() state = event.message.stream_meta[id(stream)] time_in = state['time_in'] @@ -356,12 +361,14 @@ def on_stream_event_out(self, tp: TP, offset: int, stream: StreamT, deque_pushpopmax(self.events_runtime, time_total, self.max_avg_history) def on_topic_buffer_full(self, topic: TopicT) -> None: + """Call when conductor topic buffer is full and has to wait.""" self.topic_buffer_full[topic] += 1 def on_message_out(self, tp: TP, offset: int, message: Message) -> None: + """Call when message is fully acknowledged and can be committed.""" self.messages_active -= 1 time_out = message.time_out = self.time() time_in = message.time_in @@ -369,12 +376,15 @@ def on_message_out(self, message.time_total = time_out - time_in def on_table_get(self, table: CollectionT, key: Any) -> None: + """Call when value in table is retrieved.""" self._table_or_create(table).keys_retrieved += 1 def on_table_set(self, table: CollectionT, key: Any, value: Any) -> None: + """Call when new value for key in table is set.""" self._table_or_create(table).keys_updated += 1 def on_table_del(self, table: CollectionT, key: Any) -> None: + """Call when key in a table is deleted.""" self._table_or_create(table).keys_deleted += 1 def _table_or_create(self, table: CollectionT) -> TableState: @@ -385,9 +395,11 @@ def _table_or_create(self, table: CollectionT) -> TableState: return state def on_commit_initiated(self, consumer: ConsumerT) -> Any: + """Consumer is about to commit topic offset.""" return self.time() def on_commit_completed(self, consumer: ConsumerT, state: Any) -> None: + """Call when consumer commit offset operation completed.""" latency = self.time() - cast(float, state) deque_pushpopmax( self.commit_latency, @@ -398,6 +410,7 @@ def on_commit_completed(self, consumer: ConsumerT, state: Any) -> None: def on_send_initiated(self, producer: ProducerT, topic: str, message: PendingMessage, keysize: int, valsize: int) -> Any: + """Call when message added to producer buffer.""" self.messages_sent += 1 self.messages_sent_by_topic[topic] += 1 return self.time() @@ -406,6 +419,7 @@ def on_send_completed(self, producer: ProducerT, state: Any, metadata: RecordMetadata) -> None: + """Call when producer finished sending message.""" latency = self.time() - cast(float, state) deque_pushpopmax( self.send_latency, latency, self.max_send_latency_history) @@ -414,25 +428,31 @@ def on_send_error(self, producer: ProducerT, exc: BaseException, state: Any) -> None: + """Call when producer was unable to publish message.""" self.send_errors += 1 def count(self, metric_name: str, count: int = 1) -> None: + """Count metric by name.""" self.metric_counts[metric_name] += count def on_tp_commit(self, tp_offsets: TPOffsetMapping) -> None: + """Call when offset in topic partition is committed.""" self.tp_committed_offsets.update(tp_offsets) def track_tp_end_offset(self, tp: TP, offset: int) -> None: + """Track new topic partition end offset for monitoring lags.""" self.tp_end_offsets[tp] = offset def on_assignment_start(self, assignor: PartitionAssignorT) -> Mapping: + """Partition assignor is starting to assign partitions.""" return {'time_start': self.time()} def on_assignment_error(self, assignor: PartitionAssignorT, state: Mapping, exc: BaseException) -> None: + """Partition assignor did not complete assignor due to error.""" time_total = self.time() - state['time_start'] deque_pushpopmax( self.assignment_latency, time_total, @@ -442,6 +462,7 @@ def on_assignment_error(self, def on_assignment_completed(self, assignor: PartitionAssignorT, state: Mapping) -> None: + """Partition assignor completed assignment.""" time_total = self.time() - state['time_start'] deque_pushpopmax( self.assignment_latency, time_total, diff --git a/faust/sensors/statsd.py b/faust/sensors/statsd.py index 10789c53f..ddccfdacf 100644 --- a/faust/sensors/statsd.py +++ b/faust/sensors/statsd.py @@ -74,6 +74,7 @@ def _new_statsd_client(self) -> StatsClient: host=self.host, port=self.port, prefix=self.prefix) def on_message_in(self, tp: TP, offset: int, message: Message) -> None: + """Call before message is delegated to streams.""" super().on_message_in(tp, offset, message) self.client.incr('messages_received', rate=self.rate) @@ -83,6 +84,7 @@ def on_message_in(self, tp: TP, offset: int, message: Message) -> None: def on_stream_event_in(self, tp: TP, offset: int, stream: StreamT, event: EventT) -> None: + """Call when stream starts processing an event.""" super().on_stream_event_in(tp, offset, stream, event) self.client.incr('events', rate=self.rate) self.client.incr( @@ -98,6 +100,7 @@ def _stream_label(self, stream: StreamT) -> str: def on_stream_event_out(self, tp: TP, offset: int, stream: StreamT, event: EventT) -> None: + """Call when stream is done processing an event.""" super().on_stream_event_out(tp, offset, stream, event) self.client.decr('events_active', rate=self.rate) self.client.timing( @@ -109,22 +112,27 @@ def on_message_out(self, tp: TP, offset: int, message: Message) -> None: + """Call when message is fully acknowledged and can be committed.""" super().on_message_out(tp, offset, message) self.client.decr('messages_active', rate=self.rate) def on_table_get(self, table: CollectionT, key: Any) -> None: + """Call when value in table is retrieved.""" super().on_table_get(table, key) self.client.incr(f'table.{table.name}.keys_retrieved', rate=self.rate) def on_table_set(self, table: CollectionT, key: Any, value: Any) -> None: + """Call when new value for key in table is set.""" super().on_table_set(table, key, value) self.client.incr(f'table.{table.name}.keys_updated', rate=self.rate) def on_table_del(self, table: CollectionT, key: Any) -> None: + """Call when key in a table is deleted.""" super().on_table_del(table, key) self.client.incr(f'table.{table.name}.keys_deleted', rate=self.rate) def on_commit_completed(self, consumer: ConsumerT, state: Any) -> None: + """Call when consumer commit offset operation completed.""" super().on_commit_completed(consumer, state) self.client.timing( 'commit_latency', @@ -134,6 +142,7 @@ def on_commit_completed(self, consumer: ConsumerT, state: Any) -> None: def on_send_initiated(self, producer: ProducerT, topic: str, message: PendingMessage, keysize: int, valsize: int) -> Any: + """Call when message added to producer buffer.""" self.client.incr(f'topic.{topic}.messages_sent', rate=self.rate) return super().on_send_initiated( producer, topic, message, keysize, valsize) @@ -142,6 +151,7 @@ def on_send_completed(self, producer: ProducerT, state: Any, metadata: RecordMetadata) -> None: + """Call when producer finished sending message.""" super().on_send_completed(producer, state, metadata) self.client.incr('messages_sent', rate=self.rate) self.client.timing( @@ -153,6 +163,7 @@ def on_send_error(self, producer: ProducerT, exc: BaseException, state: Any) -> None: + """Call when producer was unable to publish message.""" super().on_send_error(producer, exc, state) self.client.incr('messages_sent_error', rate=self.rate) self.client.timing( @@ -164,6 +175,7 @@ def on_assignment_error(self, assignor: PartitionAssignorT, state: Mapping, exc: BaseException) -> None: + """Partition assignor did not complete assignor due to error.""" super().on_assignment_error(assignor, state, exc) self.client.incr('assignments_error', rate=self.rate) self.client.timing( @@ -174,6 +186,7 @@ def on_assignment_error(self, def on_assignment_completed(self, assignor: PartitionAssignorT, state: Mapping) -> None: + """Partition assignor completed assignment.""" super().on_assignment_completed(assignor, state) self.client.incr('assignments_complete', rate=self.rate) self.client.timing( @@ -182,16 +195,19 @@ def on_assignment_completed(self, rate=self.rate) def count(self, metric_name: str, count: int = 1) -> None: + """Count metric by name.""" super().count(metric_name, count=count) self.client.incr(metric_name, count=count, rate=self.rate) def on_tp_commit(self, tp_offsets: TPOffsetMapping) -> None: + """Call when offset in topic partition is committed.""" super().on_tp_commit(tp_offsets) for tp, offset in tp_offsets.items(): metric_name = f'committed_offset.{tp.topic}.{tp.partition}' self.client.gauge(metric_name, offset) def track_tp_end_offset(self, tp: TP, offset: int) -> None: + """Track new topic partition end offset for monitoring lags.""" super().track_tp_end_offset(tp, offset) metric_name = f'end_offset.{tp.topic}.{tp.partition}' self.client.gauge(metric_name, offset) @@ -207,4 +223,5 @@ def _time(self, time: float) -> float: @cached_property def client(self) -> StatsClient: + """Return statsd client.""" return self._new_statsd_client() diff --git a/faust/serializers/__init__.py b/faust/serializers/__init__.py index 67c0c04ce..74f440a67 100644 --- a/faust/serializers/__init__.py +++ b/faust/serializers/__init__.py @@ -1,3 +1,4 @@ +"""Serializers and Codecs.""" from .codecs import Codec from .registry import Registry diff --git a/faust/serializers/registry.py b/faust/serializers/registry.py index be4a61b0d..2edda0150 100644 --- a/faust/serializers/registry.py +++ b/faust/serializers/registry.py @@ -179,5 +179,6 @@ def dumps_value(self, @cached_property def Model(self) -> Type[ModelT]: + """Return the :class:`faust.Model` class used by this serializer.""" from faust.models.base import Model return Model diff --git a/faust/stores/memory.py b/faust/stores/memory.py index e60305c71..7c2987ca1 100644 --- a/faust/stores/memory.py +++ b/faust/stores/memory.py @@ -57,7 +57,7 @@ def persisted_offset(self, tp: TP) -> Optional[int]: return None def reset_state(self) -> None: - """Remove local filesystem state. + """Remove local file system state. This does nothing when using the in-memory store. diff --git a/faust/tables/__init__.py b/faust/tables/__init__.py index d24ac5df9..00eadb20b 100644 --- a/faust/tables/__init__.py +++ b/faust/tables/__init__.py @@ -1,3 +1,4 @@ +"""Tables: Distributed object K/V-store.""" from .base import Collection, CollectionT from .manager import TableManager, TableManagerT from .table import Table, TableT diff --git a/faust/tables/base.py b/faust/tables/base.py index 9d8e05130..6e7cf8dd6 100644 --- a/faust/tables/base.py +++ b/faust/tables/base.py @@ -180,11 +180,13 @@ def _new_store_by_url(self, url: Union[str, URL]) -> StoreT: @property # type: ignore @no_type_check # XXX https://github.com/python/mypy/issues/4125 def data(self) -> StoreT: + """Underlying table storage.""" if self._data is None: self._data = self._new_store() return self._data async def on_start(self) -> None: + """Call when table starts.""" await self.add_runtime_dependency(self.data) await self.changelog_topic.maybe_declare() @@ -195,6 +197,7 @@ def on_recover(self, fun: RecoverCallback) -> RecoverCallback: return fun def info(self) -> Mapping[str, Any]: + """Return table attributes as dictionary.""" # Used to recreate object in .clone() return { 'app': self.app, @@ -208,12 +211,15 @@ def info(self) -> Mapping[str, Any]: } def persisted_offset(self, tp: TP) -> Optional[int]: + """Return the last persisted offset for topic partition.""" return self.data.persisted_offset(tp) async def need_active_standby_for(self, tp: TP) -> bool: + """Return :const:`False` if we have access to partition data.""" return await self.data.need_active_standby_for(tp) def reset_state(self) -> None: + """Reset local state.""" self.data.reset_state() def _send_changelog(self, @@ -329,15 +335,19 @@ def _changelog_topic_name(self) -> str: return f'{self.app.conf.id}-{self.name}-changelog' def join(self, *fields: FieldDescriptorT) -> StreamT: + """Right join of this table and another stream/table.""" return self._join(joins.RightJoin(stream=self, fields=fields)) def left_join(self, *fields: FieldDescriptorT) -> StreamT: + """Left join of this table and another stream/table.""" return self._join(joins.LeftJoin(stream=self, fields=fields)) def inner_join(self, *fields: FieldDescriptorT) -> StreamT: + """Inner join of this table and another stream/table.""" return self._join(joins.InnerJoin(stream=self, fields=fields)) def outer_join(self, *fields: FieldDescriptorT) -> StreamT: + """Outer join of this table and another stream/table.""" return self._join(joins.OuterJoin(stream=self, fields=fields)) def _join(self, join_strategy: JoinT) -> StreamT: @@ -345,18 +355,22 @@ def _join(self, join_strategy: JoinT) -> StreamT: raise NotImplementedError('TODO') def clone(self, **kwargs: Any) -> Any: + """Clone table instance.""" return self.__class__(**{**self.info(), **kwargs}) def combine(self, *nodes: JoinableT, **kwargs: Any) -> StreamT: + """Combine tables and streams.""" # TODO raise NotImplementedError('TODO') def contribute_to_stream(self, active: StreamT) -> None: + """Contribute table to stream join.""" # TODO See Stream.contribute_to_stream() # Should probably connect to Table changelog. ... async def remove_from_stream(self, stream: StreamT) -> None: + """Remove table from stream join after stream stopped.""" # TODO See Stream.remove_from_stream() # Should stop any services started to support joining this table # with one or more streams. @@ -470,32 +484,39 @@ async def on_rebalance(self, assigned: Set[TP], revoked: Set[TP], newly_assigned: Set[TP]) -> None: + """Call when cluster is rebalancing.""" await self.data.on_rebalance(self, assigned, revoked, newly_assigned) async def on_recovery_completed(self, active_tps: Set[TP], standby_tps: Set[TP]) -> None: + """Call when recovery has completed after rebalancing.""" await self.data.on_recovery_completed(active_tps, standby_tps) await self.call_recover_callbacks() async def call_recover_callbacks(self) -> None: + """Call any configured recovery callbacks after rebalancing.""" for fun in self._recover_callbacks: await fun() async def on_changelog_event(self, event: EventT) -> None: + """Call when a new changelog event is received.""" if self._on_changelog_event: await self._on_changelog_event(event) @property def label(self) -> str: + """Return human-readable label used to represent this table.""" return f'{self.shortlabel}@{self._store}' @property def shortlabel(self) -> str: + """Return short label used to represent this table in logs.""" return f'{type(self).__name__}: {self.name}' @property def changelog_topic(self) -> TopicT: + """Return the changelog topic used by this table.""" if self._changelog_topic is None: self._changelog_topic = self._new_changelog_topic() return self._changelog_topic @@ -505,6 +526,7 @@ def changelog_topic(self, topic: TopicT) -> None: self._changelog_topic = topic def apply_changelog_batch(self, batch: Iterable[EventT]) -> None: + """Apply batch of events from changelog topic local table storage.""" self.data.apply_changelog_batch( batch, to_key=self._to_key, diff --git a/faust/tables/manager.py b/faust/tables/manager.py index 435587570..02372c43a 100644 --- a/faust/tables/manager.py +++ b/faust/tables/manager.py @@ -58,25 +58,30 @@ def persist_offset_on_commit(self, self._pending_persisted_offsets[tp] = (store, offset) def on_commit(self, offsets: MutableMapping[TP, int]) -> None: + """Call when committing source topic partitions.""" # flush any pending persisted offsets added by # persist_offset_on_commit for tp in offsets: self.on_commit_tp(tp) def on_commit_tp(self, tp: TP) -> None: + """Call when committing source topic partition used by this table.""" entry = self._pending_persisted_offsets.get(tp) if entry is not None: store, offset = entry store.set_persisted_offset(tp, offset) def on_rebalance_start(self) -> None: + """Call when a new rebalancing operation starts.""" self.actives_ready = False self.standbys_ready = False def on_actives_ready(self) -> None: + """Call when actives are fully up-to-date.""" self.actives_ready = True def on_standbys_ready(self) -> None: + """Call when standbys are fully up-to-date and ready for failover.""" self.standbys_ready = True def __hash__(self) -> int: @@ -84,10 +89,12 @@ def __hash__(self) -> int: @property def changelog_topics(self) -> Set[str]: + """Return the set of known changelog topics.""" return set(self._changelogs.keys()) @property def changelog_queue(self) -> ThrowableQueue: + """Queue used to buffer changelog events.""" if self._changelog_queue is None: self._changelog_queue = self.app.FlowControlQueue( maxsize=self.app.conf.stream_buffer_maxsize, @@ -98,12 +105,14 @@ def changelog_queue(self) -> ThrowableQueue: @property def recovery(self) -> Recovery: + """Recovery service used by this table manager.""" if self._recovery is None: self._recovery = Recovery( self.app, self, beacon=self.beacon, loop=self.loop) return self._recovery def add(self, table: CollectionT) -> CollectionT: + """Add table to be managed by this table manager.""" if self._recovery_started.is_set(): raise RuntimeError('Too late to add tables at this point') assert table.name is not None @@ -114,6 +123,7 @@ def add(self, table: CollectionT) -> CollectionT: return table async def on_start(self) -> None: + """Call when table manager is starting.""" await self.sleep(1.0) if not self.should_stop: await self._update_channels() @@ -133,6 +143,7 @@ async def _update_channels(self) -> None: }) async def on_stop(self) -> None: + """Call when table manager is stopping.""" await self.app._fetcher.stop() if self._recovery: await self._recovery.stop() @@ -140,6 +151,7 @@ async def on_stop(self) -> None: await table.stop() def on_partitions_revoked(self, revoked: Set[TP]) -> None: + """Call when cluster is rebalancing and partitions revoked.""" T = traced_from_parent_span() T(self.recovery.on_partitions_revoked)(revoked) @@ -147,6 +159,7 @@ async def on_rebalance(self, assigned: Set[TP], revoked: Set[TP], newly_assigned: Set[TP]) -> None: + """Call when the cluster is rebalancing.""" self._recovery_started.set() # cannot add more tables. T = traced_from_parent_span() for table in self.values(): diff --git a/faust/tables/objects.py b/faust/tables/objects.py index 40c3864bf..97ddb28d8 100644 --- a/faust/tables/objects.py +++ b/faust/tables/objects.py @@ -42,14 +42,17 @@ def __post_init__(self) -> None: # pragma: no cover @abc.abstractmethod def sync_from_storage(self, value: Any) -> None: + """Sync value from storage.""" ... @abc.abstractmethod def as_stored_value(self) -> Any: + """Return value as represented in storage.""" ... @abc.abstractmethod def apply_changelog_event(self, operation: int, value: Any) -> None: + """Apply event in changelog topic to local table state.""" ... @@ -75,6 +78,7 @@ 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) @@ -96,15 +100,19 @@ def _table_type_name(self) -> str: return f'{type(self.table).__name__}' 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) def set_persisted_offset(self, tp: TP, offset: int) -> None: + """Set the last persisted offset for changelog topic partition.""" self.storage.set_persisted_offset(tp, offset) async def on_rebalance(self, @@ -112,19 +120,23 @@ async def on_rebalance(self, assigned: Set[TP], revoked: Set[TP], newly_assigned: Set[TP]) -> None: + """Call when cluster is rebalancing.""" await self.storage.on_rebalance( table, assigned, revoked, newly_assigned) async def on_recovery_completed(self, active_tps: Set[TP], standby_tps: Set[TP]) -> None: + """Call when table recovery is completed after rebalancing.""" self.sync_from_storage() def sync_from_storage(self) -> None: + """Sync set contents from storage.""" 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() @@ -134,11 +146,13 @@ async def _periodic_flush(self) -> None: # pragma: no cover self.flush_to_storage() def reset_state(self) -> None: + """Reset table local state.""" # delegate to underlying RocksDB store. self.storage.reset_state() @property def storage(self) -> StoreT: + """Return underlying storage used by this set table.""" if self._storage is None: self._storage = self.table._new_store_by_url( self.table._store or self.table.app.conf.store) @@ -148,6 +162,7 @@ def apply_changelog_batch(self, batch: Iterable[EventT], to_key: Callable[[Any], Any], to_value: Callable[[Any], Any]) -> None: + """Apply batch of changelog events to local state.""" tp_offsets: Dict[TP, int] = {} for event in batch: tp, offset = event.message.tp, event.message.offset diff --git a/faust/tables/recovery.py b/faust/tables/recovery.py index 8ffe5d126..47253c612 100644 --- a/faust/tables/recovery.py +++ b/faust/tables/recovery.py @@ -125,32 +125,38 @@ def __init__(self, @property def signal_recovery_start(self) -> Event: + """Event used to signal that recovery has started.""" if self._signal_recovery_start is None: self._signal_recovery_start = Event(loop=self.loop) return self._signal_recovery_start @property def signal_recovery_end(self) -> Event: + """Event used to signal that recovery has ended.""" if self._signal_recovery_end is None: self._signal_recovery_end = Event(loop=self.loop) return self._signal_recovery_end @property def signal_recovery_reset(self) -> Event: + """Event used to signal that recovery is restarting.""" if self._signal_recovery_reset is None: self._signal_recovery_reset = Event(loop=self.loop) return self._signal_recovery_reset async def on_stop(self) -> None: + """Call when recovery service stops.""" # Flush buffers when stopping. self.flush_buffers() def add_active(self, table: CollectionT, tp: TP) -> None: + """Add changelog partition to be used for active recovery.""" self.active_tps.add(tp) self.actives_for_table[table].add(tp) self._add(table, tp, self.active_offsets) def add_standby(self, table: CollectionT, tp: TP) -> None: + """Add changelog partition to be used for standby recovery.""" self.standby_tps.add(tp) self.standbys_for_table[table].add(tp) self._add(table, tp, self.standby_offsets) @@ -163,12 +169,14 @@ def _add(self, table: CollectionT, tp: TP, offsets: Counter[TP]) -> None: offsets.setdefault(tp, None) def revoke(self, tp: TP) -> None: + """Revoke assignment of table changelog partition.""" self.standby_offsets.pop(tp, None) self.standby_highwaters.pop(tp, None) self.active_offsets.pop(tp, None) self.active_highwaters.pop(tp, None) def on_partitions_revoked(self, revoked: Set[TP]) -> None: + """Call when rebalancing and partitions are revoked.""" T = traced_from_parent_span() T(self.flush_buffers)() self.signal_recovery_reset.set() @@ -177,6 +185,7 @@ async def on_rebalance(self, assigned: Set[TP], revoked: Set[TP], newly_assigned: Set[TP]) -> None: + """Call when cluster is rebalancing.""" app = self.app assigned_standbys = app.assignor.assigned_standbys() assigned_actives = app.assignor.assigned_actives() @@ -435,6 +444,7 @@ async def _wait(self, coro: WaitArgT) -> None: return None async def on_recovery_completed(self) -> None: + """Call when active table recovery is completed.""" consumer = self.app.consumer self.log.info('Restore complete!') await self.app.on_rebalance_complete.send() @@ -588,14 +598,17 @@ async def _slurp_changelogs(self) -> None: self.tables.on_standbys_ready() def flush_buffers(self) -> None: + """Flush changelog buffers.""" for table, buffer in self.buffers.items(): table.apply_changelog_batch(buffer) buffer.clear() def need_recovery(self) -> bool: + """Return :const:`True` if recovery is required.""" return any(v for v in self.active_remaining().values()) def active_remaining(self) -> Counter[TP]: + """Return counter of remaining changes by active partition.""" highwaters = self.active_highwaters offsets = self.active_offsets return Counter({ @@ -605,6 +618,7 @@ def active_remaining(self) -> Counter[TP]: }) def standby_remaining(self) -> Counter[TP]: + """Return counter of remaining changes by standby partition.""" highwaters = self.standby_highwaters offsets = self.standby_offsets return Counter({ @@ -614,12 +628,15 @@ def standby_remaining(self) -> Counter[TP]: }) def active_remaining_total(self) -> int: + """Return number of changes remaining for actives to be up-to-date.""" return sum(self.active_remaining().values()) def standby_remaining_total(self) -> int: + """Return number of changes remaining for standbys to be up-to-date.""" return sum(self.standby_remaining().values()) def active_stats(self) -> MutableMapping[TP, Tuple[int, int, int]]: + """Return current active recovery statistics.""" offsets = self.active_offsets return { tp: (highwater, offsets[tp], highwater - offsets[tp]) @@ -628,6 +645,7 @@ def active_stats(self) -> MutableMapping[TP, Tuple[int, int, int]]: } def standby_stats(self) -> MutableMapping[TP, Tuple[int, int, int]]: + """Return current standby recovery statistics.""" offsets = self.standby_offsets return { tp: (highwater, offsets[tp], highwater - offsets[tp]) diff --git a/faust/tables/sets.py b/faust/tables/sets.py index 32e68b600..f93cc6a09 100644 --- a/faust/tables/sets.py +++ b/faust/tables/sets.py @@ -127,6 +127,25 @@ class SetManagerOperation(Record, Generic[VT], class SetTableManager(Service, Generic[KT, VT]): + """Manager used to perform operations on :class:`SetTable`. + + Used when set table is configured with ``SetTable('name', + start_manager=True)``. + + The workers will start an additional agent used to process + incoming set operations, and you can communicate with this + agent to modify your sets. + + Examples: + >>> set_table = SetTable('sets', start_manager=True) + + >>> await set_table.manager.add('active_orders', Order) + >>> await set_table.manager.discard('active_orders', Order) + + The manager methods can be used from HTTP views and other agents + to safely route set operations to the correct worker. + """ + app: AppT set_table: 'SetTable[KT, VT]' enabled: bool @@ -148,9 +167,11 @@ def __init__(self, set_table: 'SetTable[KT, VT]', self._enable() async def add(self, key: KT, member: VT) -> None: + """Add member to set table using key.""" await self._send_operation(SetAction.ADD, key, member) async def discard(self, key: KT, member: VT) -> None: + """Discard member from set table using key.""" await self._send_operation(SetAction.DISCARD, key, member) def _add(self, key: KT, member: VT) -> None: @@ -190,6 +211,7 @@ async def _modify_set(self, stream: StreamT[SetManagerOperation]) -> None: @cached_property def topic(self) -> TopicT: + """Return topic used by set table manager.""" return self.app.topic(self.set_table.manager_topic_name, key_type=str, value_type=SetManagerOperation) @@ -197,6 +219,7 @@ def topic(self) -> TopicT: class SetTable(Table[KT, VT]): """Table that maintains a dictionary of sets.""" + Manager: ClassVar[Type[SetTableManager]] = SetTableManager start_manager: bool manager_topic_name: str @@ -223,6 +246,7 @@ def __init__(self, app: AppT, *, self.manager = self.Manager(self, loop=self.loop, beacon=self.beacon) async def on_start(self) -> None: + """Call when set table starts.""" await self.add_runtime_dependency(self.manager) await super().on_start() diff --git a/faust/tables/table.py b/faust/tables/table.py index f45aabd02..fc212eb14 100644 --- a/faust/tables/table.py +++ b/faust/tables/table.py @@ -22,6 +22,7 @@ class Table(TableT[KT, VT], Collection): def using_window(self, window: WindowT, *, key_index: bool = False) -> WindowWrapperT: + """Wrap table using a specific window type.""" self.window = window self._changelog_compacting = True self._changelog_deleting = True @@ -31,6 +32,7 @@ def using_window(self, window: WindowT, *, def hopping(self, size: Seconds, step: Seconds, expires: Seconds = None, key_index: bool = False) -> WindowWrapperT: + """Wrap table in a hopping window.""" return self.using_window( windows.HoppingWindow(size, step, expires), key_index=key_index, @@ -39,6 +41,7 @@ def hopping(self, size: Seconds, step: Seconds, def tumbling(self, size: Seconds, expires: Seconds = None, key_index: bool = False) -> WindowWrapperT: + """Wrap table in a tumbling window.""" return self.using_window( windows.TumblingWindow(size, expires), key_index=key_index, @@ -62,9 +65,11 @@ def _del_key(self, key: KT) -> None: del self[key] def on_key_get(self, key: KT) -> None: + """Call when the value for a key in this table is retrieved.""" self._sensor_on_get(self, key) def on_key_set(self, key: KT, value: VT) -> None: + """Call when the value for a key in this table is set.""" event = current_event() if event is None: raise TypeError( @@ -75,6 +80,7 @@ def on_key_set(self, key: KT, value: VT) -> None: self._sensor_on_set(self, key, value) def on_key_del(self, key: KT) -> None: + """Call when a key in this table is removed.""" event = current_event() if event is None: raise TypeError( @@ -86,6 +92,7 @@ def on_key_del(self, key: KT) -> None: def as_ansitable(self, title: str = '{table.name}', **kwargs: Any) -> str: + """Draw table as a a terminal ANSI table.""" return dict_as_ansitable( self, title=title.format(table=self), diff --git a/faust/tables/wrappers.py b/faust/tables/wrappers.py index ef548337f..1f5dbe663 100644 --- a/faust/tables/wrappers.py +++ b/faust/tables/wrappers.py @@ -63,6 +63,10 @@ def __init__(self, self.event = event def __iter__(self) -> Iterator: + """Iterate over keys. + + The window is chosen based on the tables time-relativity setting. + """ wrapper = cast(WindowWrapper, self._mapping) for key, _ in wrapper._items(self.event): yield key @@ -71,16 +75,19 @@ def __len__(self) -> int: return len(self._mapping) def now(self) -> Iterator[Any]: + """Return all keys present in window closest to system time.""" wrapper = cast(WindowWrapper, self._mapping) for key, _ in wrapper._items_now(): yield key def current(self, event: EventT = None) -> Iterator[Any]: + """Return all keys present in window closest to stream time.""" wrapper = cast(WindowWrapper, self._mapping) for key, _ in wrapper._items_current(event or self.event): yield key def delta(self, d: Seconds, event: EventT = None) -> Iterator[Any]: + """Return all keys present in window ±n seconds ago.""" wrapper = cast(WindowWrapper, self._mapping) for key, _ in wrapper._items_delta(d, event or self.event): yield key @@ -96,20 +103,27 @@ def __init__(self, self.event = event def __iter__(self) -> Iterator[Tuple[Any, Any]]: + """Iterate over items. + + The window is chosen based on the tables time-relativity setting. + """ wrapper = cast(WindowWrapper, self._mapping) return wrapper._items(self.event) def now(self) -> Iterator[Tuple[Any, Any]]: + """Return all items present in window closest to system time.""" wrapper = cast(WindowWrapper, self._mapping) return wrapper._items_now() def current(self, event: EventT = None) -> Iterator[Tuple[Any, Any]]: + """Return all items present in window closest to stream time.""" wrapper = cast(WindowWrapper, self._mapping) return wrapper._items_current(event or self.event) def delta(self, d: Seconds, event: EventT = None) -> Iterator[Tuple[Any, Any]]: + """Return all items present in window ±n seconds ago.""" wrapper = cast(WindowWrapper, self._mapping) return wrapper._items_delta(d, event or self.event) @@ -124,21 +138,28 @@ def __init__(self, self.event = event def __iter__(self) -> Iterator[Any]: + """Iterate over values. + + The window is chosen based on the tables time-relativity setting. + """ wrapper = cast(WindowWrapper, self._mapping) for _, value in wrapper._items(self.event): yield value def now(self) -> Iterator[Any]: + """Return all values present in window closest to system time.""" wrapper = cast(WindowWrapper, self._mapping) for _, value in wrapper._items_now(): yield value def current(self, event: EventT = None) -> Iterator[Any]: + """Return all values present in window closest to stream time.""" wrapper = cast(WindowWrapper, self._mapping) for _, value in wrapper._items_current(event or self.event): yield value def delta(self, d: Seconds, event: EventT = None) -> Iterator[Any]: + """Return all values present in window ±n seconds ago.""" wrapper = cast(WindowWrapper, self._mapping) for _, value in wrapper._items_delta(d, event or self.event): yield value @@ -184,6 +205,7 @@ def apply(self, op: Callable[[VT, VT], VT], value: VT, event: EventT = None) -> WindowSetT[KT, VT]: + """Apply operation to all affected windows.""" table = cast(_Table, self.table) wrapper = cast(WindowWrapper, self.wrapper) timestamp = wrapper.get_timestamp(event or self.event) @@ -192,18 +214,27 @@ def apply(self, return self def value(self, event: EventT = None) -> VT: + """Return current value. + + The selected window depends on the current time-relativity + setting used (:meth:`relative_to_now`, :meth:`relative_to_stream`, + :meth:`relative_to_field`, etc.) + """ return cast(_Table, self.table)._windowed_timestamp( self.key, self.wrapper.get_timestamp(event or self.event)) def now(self) -> VT: + """Return current value, using the current system time.""" return cast(_Table, self.table)._windowed_now(self.key) def current(self, event: EventT = None) -> VT: + """Return current value, using stream time-relativity.""" t = cast(_Table, self.table) return t._windowed_timestamp( self.key, t._relative_event(event or self.event)) def delta(self, d: Seconds, event: EventT = None) -> VT: + """Return value as it was ±n seconds ago.""" table = cast(_Table, self.table) return table._windowed_delta(self.key, d, event or self.event) @@ -318,6 +349,7 @@ def __init__(self, table: TableT, *, self._get_relative_timestamp = self._relative_handler(relative_to) def clone(self, relative_to: RelativeArg) -> WindowWrapperT: + """Clone this table using a new time-relativity configuration.""" return type(self)( table=self.table, relative_to=relative_to or self._get_relative_timestamp, @@ -327,21 +359,49 @@ def clone(self, relative_to: RelativeArg) -> WindowWrapperT: @property def name(self) -> str: + """Return the name of this table.""" return self.table.name def relative_to(self, ts: RelativeArg) -> WindowWrapperT: + """Configure the time-relativity of this windowed table.""" return self.clone(relative_to=ts) def relative_to_now(self) -> WindowWrapperT: + """Configure table to be time-relative to the system clock.""" return self.clone(relative_to=self.table._relative_now) def relative_to_field(self, field: FieldDescriptorT) -> WindowWrapperT: + """Configure table to be time-relative to a field in the stream. + + This means the window will use the timestamp + from the event currently being processed in the stream. + + Further it will not use the timestamp of the Kafka message, + but a field in the value of the event. + + For example a model field: + + .. sourcecode:: python + + class Account(faust.Record): + created: float + + table = app.Table('foo').hopping( + ..., + ).relative_to_field(Account.created) + """ return self.clone(relative_to=self.table._relative_field(field)) def relative_to_stream(self) -> WindowWrapperT: + """Configure table to be time-relative to the stream. + + This means the window will use the timestamp + from the event currently being processed in the stream. + """ return self.clone(relative_to=self.table._relative_event) def get_timestamp(self, event: EventT = None) -> float: + """Get timestamp from event.""" event = event or current_event() get_relative_timestamp = self.get_relative_timestamp if get_relative_timestamp: @@ -354,6 +414,7 @@ def get_timestamp(self, event: EventT = None) -> float: return event.message.timestamp def on_recover(self, fun: RecoverCallback) -> RecoverCallback: + """Call after table recovery.""" return self.table.on_recover(fun) def __contains__(self, key: Any) -> bool: @@ -369,12 +430,14 @@ def __setitem__(self, key: Any, value: Any) -> None: table._set_windowed(key, value, self.get_timestamp()) def on_set_key(self, key: Any, value: Any) -> None: + """Call when the value for a key in this table is set.""" key_index_table = self.key_index_table if key_index_table is not None: if key not in key_index_table: key_index_table[key] = 1 def on_del_key(self, key: Any) -> None: + """Call when a key is deleted from this table.""" key_index_table = self.key_index_table if key_index_table is not None: key_index_table.pop(key, None) @@ -408,6 +471,7 @@ def __iter__(self) -> Iterator: return self._keys() def keys(self) -> KeysView: + """Return table keys view: iterate over keys found in this table.""" return WindowedKeysView(self) def _keys(self) -> Iterator: @@ -421,9 +485,11 @@ def _keys(self) -> Iterator: 'support .keys/.items/.values') def values(self, event: EventT = None) -> ValuesView: + """Return table values view: iterate over values in this table.""" return WindowedValuesView(self, event or current_event()) def items(self, event: EventT = None) -> ItemsView: + """Return table items view: iterate over ``(key, value)`` pairs.""" return WindowedItemsView(self, event or current_event()) def _items(self, event: EventT = None) -> Iterator[Tuple[Any, Any]]: @@ -464,6 +530,7 @@ def _items_delta(self, d: Seconds, def as_ansitable(self, title: str = '{table.name}', **kwargs: Any) -> str: + """Draw table as a terminal ANSI table.""" return dict_as_ansitable( self, title=title.format(table=self.table), @@ -471,6 +538,7 @@ def as_ansitable(self, title: str = '{table.name}', @property def get_relative_timestamp(self) -> Optional[RelativeHandler]: + """Return the current handler for extracting event timestamp.""" return self._get_relative_timestamp @get_relative_timestamp.setter diff --git a/faust/topics.py b/faust/topics.py index d651d0c1a..929ab14b1 100644 --- a/faust/topics.py +++ b/faust/topics.py @@ -271,7 +271,6 @@ def partitions(self) -> Optional[int]: """Return the number of configured partitions for this topic. Notes: - This is only active for internal topics, fully owned and managed by Faust itself. diff --git a/faust/transport/__init__.py b/faust/transport/__init__.py index 7b5826675..c058785fc 100644 --- a/faust/transport/__init__.py +++ b/faust/transport/__init__.py @@ -1,3 +1,4 @@ +"""Transports.""" from .drivers import by_name, by_url __all__ = ['by_name', 'by_url'] diff --git a/faust/transport/_cython/__init__.py b/faust/transport/_cython/__init__.py index e69de29bb..9857fc1c2 100644 --- a/faust/transport/_cython/__init__.py +++ b/faust/transport/_cython/__init__.py @@ -0,0 +1 @@ +"""Cython optimized transport components.""" diff --git a/faust/transport/base.py b/faust/transport/base.py index 7c918aba1..e73ff0705 100644 --- a/faust/transport/base.py +++ b/faust/transport/base.py @@ -63,17 +63,20 @@ def __init__(self, def create_consumer(self, callback: ConsumerCallback, **kwargs: Any) -> ConsumerT: + """Create new consumer.""" return self.Consumer(self, callback=callback, loop=self.loop, **kwargs) def create_producer(self, **kwargs: Any) -> ProducerT: + """Create new producer.""" return self.Producer(self, **kwargs) def create_transaction_manager(self, consumer: ConsumerT, producer: ProducerT, **kwargs: Any) -> TransactionManagerT: + """Create new transaction manager.""" return self.TransactionManager( self, consumer=consumer, @@ -82,4 +85,5 @@ def create_transaction_manager(self, ) def create_conductor(self, **kwargs: Any) -> ConductorT: + """Create new consumer conductor.""" return self.Conductor(app=self.app, loop=self.loop, **kwargs) diff --git a/faust/transport/conductor.py b/faust/transport/conductor.py index 2696eef31..39d3dd65b 100644 --- a/faust/transport/conductor.py +++ b/faust/transport/conductor.py @@ -53,6 +53,7 @@ def build(self, conductor: 'Conductor', tp: TP, channels: MutableSet[_Topic]) -> ConsumerCallback: + """Generate closure used to deliver messages.""" # This method localizes variables and attribute access # for better performance. This is part of the inner loop # of a Faust worker, so tiny improvements here has big impact. @@ -192,9 +193,11 @@ def __init__(self, app: AppT, **kwargs: Any) -> None: self.on_message = self._compile_message_handler() async def commit(self, topics: TPorTopicSet) -> bool: + """Commit offsets in topics.""" return await self.app.consumer.commit(topics) def acks_enabled_for(self, topic: str) -> bool: + """Return :const:`True` if acks are enabled for topic by name.""" return topic in self._acking_topics def _compile_message_handler(self) -> ConsumerCallback: @@ -246,6 +249,7 @@ async def _subscriber(self) -> None: # pragma: no cover notify(self._subscription_done) async def wait_for_subscriptions(self) -> None: + """Wait for consumer to be subscribed.""" if self._subscription_done is not None: await self._subscription_done @@ -263,6 +267,7 @@ async def _update_indices(self) -> Iterable[str]: return self._topic_name_index async def on_partitions_assigned(self, assigned: Set[TP]) -> None: + """Call when cluster is rebalancing and partitions are assigned.""" T = traced_from_parent_span() self._tp_index.clear() T(self._update_tp_index)(assigned) @@ -300,6 +305,7 @@ def _build_handler(self, return self._compiler.build(self, tp, channels) def clear(self) -> None: + """Clear all subscriptions.""" self._topics.clear() self._topic_name_index.clear() self._tp_index.clear() @@ -319,6 +325,7 @@ def __hash__(self) -> int: return object.__hash__(self) def add(self, topic: Any) -> None: + """Register topic to be subscribed.""" if topic not in self._topics: self._topics.add(topic) if self._topic_contain_unsubscribed_topics(topic): @@ -329,6 +336,7 @@ def _topic_contain_unsubscribed_topics(self, topic: TopicT) -> bool: return bool(index and any(t not in index for t in topic.topics)) def discard(self, topic: Any) -> None: + """Unregister topic from conductor.""" self._topics.discard(topic) def _flag_changes(self) -> None: @@ -339,8 +347,10 @@ def _flag_changes(self) -> None: @property def label(self) -> str: + """Return label for use in logs.""" return f'{type(self).__name__}({len(self._topics)})' @property def shortlabel(self) -> str: + """Return short label for use in logs.""" return type(self).__name__ diff --git a/faust/transport/consumer.py b/faust/transport/consumer.py index ef2702ebc..a79232497 100644 --- a/faust/transport/consumer.py +++ b/faust/transport/consumer.py @@ -115,16 +115,20 @@ class _App: ... # noqa: E701 class TopicPartitionGroup(NamedTuple): + """Tuple of ``(topic, partition, group)``.""" + topic: str partition: int group: int def ensure_TP(tp: Any) -> TP: + """Convert aiokafka ``TopicPartition`` to Faust ``TP``.""" return tp if isinstance(tp, TP) else TP(tp.topic, tp.partition) def ensure_TPset(tps: Iterable[Any]) -> Set[TP]: + """Convert set of aiokafka ``TopicPartition`` to Faust ``TP``.""" return {ensure_TP(tp) for tp in tps} @@ -141,6 +145,7 @@ def __init__(self, app: AppT, **kwargs: Any) -> None: super().__init__(**kwargs) async def on_stop(self) -> None: + """Call when the fetcher is stopping.""" if self._drainer is not None and not self._drainer.done(): self._drainer.cancel() while True: @@ -172,6 +177,8 @@ async def _fetcher(self) -> None: class TransactionManager(Service, TransactionManagerT): + """Manage producer transactions.""" + app: AppT transactional_id_format = '{tpg.group}-{tpg.partition}' @@ -188,15 +195,18 @@ def __init__(self, transport: TransportT, super().__init__(**kwargs) async def flush(self) -> None: + """Wait for producer to transmit all pending messages.""" await self.producer.flush() async def on_partitions_revoked(self, revoked: Set[TP]) -> None: + """Call when the cluster is rebalancing and partitions are revoked.""" await traced_from_parent_span()(self.flush)() async def on_rebalance(self, assigned: Set[TP], revoked: Set[TP], newly_assigned: Set[TP]) -> None: + """Call when the cluster is rebalancing.""" T = traced_from_parent_span() # Stop producers for revoked partitions. revoked_tids = list(sorted(self._tps_to_transactional_ids(revoked))) @@ -258,6 +268,7 @@ async def send(self, topic: str, key: Optional[bytes], headers: Optional[HeadersArg], *, transactional_id: str = None) -> Awaitable[RecordMetadata]: + """Schedule message to be sent by producer.""" p: int = self.consumer.key_partition(topic, key, partition) group = self.app.assignor.group_for_topic(topic) transactional_id = f'{group}-{p}' @@ -273,11 +284,13 @@ async def send_and_wait(self, topic: str, key: Optional[bytes], headers: Optional[HeadersArg], *, transactional_id: str = None) -> RecordMetadata: + """Send message and wait for it to be transmitted.""" fut = await self.send(topic, key, value, partition, timestamp, headers) return await fut async def commit(self, offsets: Mapping[TP, int], start_new_transaction: bool = True) -> bool: + """Commit offsets for partitions.""" producer = self.producer group_id = self.app.conf.id by_transactional_id: MutableMapping[str, MutableMapping[TP, int]] @@ -309,6 +322,7 @@ async def create_topic(self, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) -> None: + """Create/declare topic on server.""" return await self.producer.create_topic( topic, partitions, replication, config=config, @@ -320,6 +334,7 @@ async def create_topic(self, ) def supports_headers(self) -> bool: + """Return :const:`True` if the Kafka server supports headers.""" return self.producer.supports_headers() @@ -343,7 +358,7 @@ class Consumer(Service, ConsumerT): #: Keeps track of the currently read offset in each TP _read_offset: MutableMapping[TP, Optional[int]] - #: Keeps track of the currently commited offset in each TP. + #: Keeps track of the currently committed offset in each TP. _committed_offset: MutableMapping[TP, Optional[int]] #: The consumer.wait_empty() method will set this to be notified @@ -427,6 +442,7 @@ def __init__(self, ) def on_init_dependencies(self) -> Iterable[ServiceT]: + """Return list of services this consumer depends on.""" # We start the TransactionManager only if # processing_guarantee='exactly_once' if self.in_transaction: @@ -442,6 +458,7 @@ def _reset_state(self) -> None: self._time_start = monotonic() async def on_restart(self) -> None: + """Call when the consumer is restarted.""" self._reset_state() self.on_init() @@ -464,6 +481,7 @@ async def _commit( ... async def perform_seek(self) -> None: + """Seek all partitions to their current committed position.""" read_offset = self._read_offset _committed_offsets = await self.seek_to_committed() read_offset.update({ @@ -479,9 +497,11 @@ async def perform_seek(self) -> None: @abc.abstractmethod async def seek_to_committed(self) -> Mapping[TP, int]: + """Seek all partitions to their committed offsets.""" ... async def seek(self, partition: TP, offset: int) -> None: + """Seek partition to specific offset.""" self.log.dev('SEEK %r -> %r', partition, offset) # reset livelock detection self._last_batch = None @@ -494,19 +514,23 @@ async def _seek(self, partition: TP, offset: int) -> None: ... def stop_flow(self) -> None: + """Block consumer from processing any more messages.""" self.flow_active = False self.can_resume_flow.clear() def resume_flow(self) -> None: + """Allow consumer to process messages.""" self.flow_active = True self.can_resume_flow.set() def pause_partitions(self, tps: Iterable[TP]) -> None: + """Pause fetching from partitions.""" tpset = ensure_TPset(tps) self._get_active_partitions().difference_update(tpset) self._paused_partitions.update(tpset) def resume_partitions(self, tps: Iterable[TP]) -> None: + """Resume fetching from partitions.""" tpset = ensure_TPset(tps) self._get_active_partitions().update(tps) self._paused_partitions.difference_update(tpset) @@ -521,6 +545,7 @@ def _is_changelog_tp(self, tp: TP) -> bool: @Service.transitions_to(CONSUMER_PARTITIONS_REVOKED) async def on_partitions_revoked(self, revoked: Set[TP]) -> None: + """Call during rebalancing when partitions are being revoked.""" self.app.on_rebalance_start() span = self.app._start_span_from_rebalancing('on_partitions_revoked') T = traced_from_parent_span(span) @@ -535,6 +560,7 @@ async def on_partitions_revoked(self, revoked: Set[TP]) -> None: @Service.transitions_to(CONSUMER_PARTITIONS_ASSIGNED) async def on_partitions_assigned(self, assigned: Set[TP]) -> None: + """Call during rebalancing when partitions are being assigned.""" span = self.app._start_span_from_rebalancing('on_partitions_assigned') T = traced_from_parent_span(span) with span: @@ -557,6 +583,7 @@ async def _getmany(self, async def getmany(self, timeout: float) -> AsyncIterator[Tuple[TP, Message]]: + """Fetch batch of messages from server.""" # records' contain mapping from TP to list of messages. # if there are two agents, consuming from topics t1 and t2, # normal order of iteration would be to process each @@ -642,6 +669,7 @@ def _to_message(self, tp: TP, record: Any) -> ConsumerMessage: ... def track_message(self, message: Message) -> None: + """Track message and mark it as pending ack.""" # add to set of pending messages that must be acked for graceful # shutdown. This is called by transport.Conductor, # before delivering messages to streams. @@ -650,6 +678,7 @@ def track_message(self, message: Message) -> None: self._on_message_in(message.tp, message.offset, message) def ack(self, message: Message) -> bool: + """Mark message as being acknowledged by stream.""" if not message.acked: message.acked = True tp = message.tp @@ -705,9 +734,11 @@ async def wait_empty(self) -> None: await T(self.commit_and_end_transactions)() async def commit_and_end_transactions(self) -> None: + """Commit all safe offsets and end transaction.""" await self.commit(start_new_transaction=False) async def on_stop(self) -> None: + """Call when consumer is stopping.""" if self.app.conf.stream_wait_empty: await self.wait_empty() else: @@ -759,6 +790,7 @@ async def commit(self, topics: TPorTopicSet = None, notify(fut) async def maybe_wait_for_commit_to_finish(self) -> bool: + """Wait for any existing commit operation to finish.""" # Only one coroutine allowed to commit at a time, # and other coroutines should wait for the original commit to finish # then do nothing. @@ -777,6 +809,7 @@ async def maybe_wait_for_commit_to_finish(self) -> bool: async def force_commit(self, topics: TPorTopicSet = None, start_new_transaction: bool = True) -> bool: + """Force offset commit.""" sensor_state = self.app.sensors.on_commit_initiated(self) # Go over the ack list in each topic/partition @@ -914,6 +947,7 @@ def _new_offset(self, tp: TP) -> Optional[int]: return None async def on_task_error(self, exc: BaseException) -> None: + """Call when processing a message failed.""" await self.commit() async def _drain_messages( @@ -976,14 +1010,18 @@ async def _drain_messages( unset_flag(flag_consumer_fetching) def close(self) -> None: + """Close consumer for graceful shutdown.""" ... @property def unacked(self) -> Set[Message]: + """Return the set of currently unacknowledged messages.""" return cast(Set[Message], self._unacked_messages) class ConsumerThread(QueueServiceThread): + """Consumer running in a dedicated thread.""" + app: AppT consumer: 'ThreadDelegateConsumer' transport: TransportT @@ -996,56 +1034,69 @@ def __init__(self, consumer: ConsumerT, **kwargs: Any) -> None: @abc.abstractmethod async def subscribe(self, topics: Iterable[str]) -> None: + """Reset subscription (requires rebalance).""" ... @abc.abstractmethod async def seek_to_committed(self) -> Mapping[TP, int]: + """Seek all partitions to their committed offsets.""" ... @abc.abstractmethod async def commit(self, tps: Mapping[TP, int]) -> bool: + """Commit offsets in topic partitions.""" ... @abc.abstractmethod async def position(self, tp: TP) -> Optional[int]: + """Return the current offset for partition.""" ... @abc.abstractmethod async def seek_to_beginning(self, *partitions: TP) -> None: + """Seek to the earliest offsets available for partitions.""" ... @abc.abstractmethod async def seek_wait(self, partitions: Mapping[TP, int]) -> None: + """Seek partitions to specific offsets and wait.""" ... @abc.abstractmethod def seek(self, partition: TP, offset: int) -> None: + """Seek partition to specific offset.""" ... @abc.abstractmethod def assignment(self) -> Set[TP]: + """Return the current assignment.""" ... @abc.abstractmethod def highwater(self, tp: TP) -> int: + """Return the last available offset in partition.""" ... @abc.abstractmethod def topic_partitions(self, topic: str) -> Optional[int]: + """Return number of configured partitions for topic by name.""" ... @abc.abstractmethod async def earliest_offsets(self, *partitions: TP) -> Mapping[TP, int]: + """Return the earliest available offset for list of partitions.""" ... @abc.abstractmethod async def highwaters(self, *partitions: TP) -> Mapping[TP, int]: + """Return the last available offset for list of partitions.""" ... @abc.abstractmethod async def getmany(self, active_partitions: Set[TP], timeout: float) -> RecordMap: + """Fetch batch of messages from server.""" ... @abc.abstractmethod @@ -1060,15 +1111,18 @@ async def create_topic(self, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) -> None: + """Create/declare topic on server.""" ... async def on_partitions_revoked( self, revoked: Set[TP]) -> None: + """Call on rebalance when partitions are being revoked.""" await self.consumer.threadsafe_partitions_revoked( self.thread_loop, revoked) async def on_partitions_assigned( self, assigned: Set[TP]) -> None: + """Call on rebalance when partitions are being assigned.""" await self.consumer.threadsafe_partitions_assigned( self.thread_loop, assigned) @@ -1077,6 +1131,7 @@ def key_partition(self, topic: str, key: Optional[bytes], partition: int = None) -> int: + """Hash key to determine partition number.""" ... @@ -1107,6 +1162,7 @@ async def threadsafe_partitions_revoked( self, receiver_loop: asyncio.AbstractEventLoop, revoked: Set[TP]) -> None: + """Call rebalancing callback in a thread-safe manner.""" promise = await self._method_queue.call( receiver_loop.create_future(), self.on_partitions_revoked, @@ -1119,6 +1175,7 @@ async def threadsafe_partitions_assigned( self, receiver_loop: asyncio.AbstractEventLoop, assigned: Set[TP]) -> None: + """Call rebalancing callback in a thread-safe manner.""" promise = await self._method_queue.call( receiver_loop.create_future(), self.on_partitions_assigned, @@ -1133,43 +1190,54 @@ async def _getmany(self, return await self._thread.getmany(active_partitions, timeout) async def subscribe(self, topics: Iterable[str]) -> None: + """Reset subscription (requires rebalance).""" await self._thread.subscribe(topics=topics) async def seek_to_committed(self) -> Mapping[TP, int]: + """Seek all partitions to the committed offset.""" return await self._thread.seek_to_committed() async def position(self, tp: TP) -> Optional[int]: + """Return the current position for partition.""" return await self._thread.position(tp) async def seek_wait(self, partitions: Mapping[TP, int]) -> None: + """Seek partitions to specific offsets and wait.""" return await self._thread.seek_wait(partitions) async def _seek(self, partition: TP, offset: int) -> None: self._thread.seek(partition, offset) def assignment(self) -> Set[TP]: + """Return the current assignment.""" return self._thread.assignment() def highwater(self, tp: TP) -> int: + """Return the last available offset for specific partition.""" return self._thread.highwater(tp) def topic_partitions(self, topic: str) -> Optional[int]: + """Return the number of partitions configured for topic by name.""" return self._thread.topic_partitions(topic) async def earliest_offsets(self, *partitions: TP) -> Mapping[TP, int]: + """Return the earliest offsets for a list of partitions.""" return await self._thread.earliest_offsets(*partitions) async def highwaters(self, *partitions: TP) -> Mapping[TP, int]: + """Return the last offset for a list of partitions.""" return await self._thread.highwaters(*partitions) async def _commit(self, offsets: Mapping[TP, int]) -> bool: return await self._thread.commit(offsets) def close(self) -> None: + """Close consumer for graceful shutdown.""" self._thread.close() def key_partition(self, topic: str, key: Optional[bytes], partition: int = None) -> int: + """Hash key to determine partition number.""" return self._thread.key_partition(topic, key, partition=partition) diff --git a/faust/transport/drivers/aiokafka.py b/faust/transport/drivers/aiokafka.py index b8bf4b1e2..d6ac87bdb 100644 --- a/faust/transport/drivers/aiokafka.py +++ b/faust/transport/drivers/aiokafka.py @@ -76,6 +76,7 @@ def server_list(urls: List[URL], default_port: int) -> List[str]: + """Convert list of urls to list of servers accepted by :pypi:`aiokafka`.""" default_host = '127.0.0.1' return [f'{u.host or default_host}:{u.port or default_port}' for u in urls] @@ -88,10 +89,12 @@ def __init__(self, thread: ConsumerThread) -> None: async def on_partitions_revoked( self, revoked: Iterable[_TopicPartition]) -> None: + """Call when partitions are being revoked.""" await self._thread.on_partitions_revoked(ensure_TPset(revoked)) async def on_partitions_assigned( self, assigned: Iterable[_TopicPartition]) -> None: + """Call when partitions are being assigned.""" await self._thread.on_partitions_assigned(ensure_TPset(assigned)) @@ -121,6 +124,7 @@ async def create_topic(self, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) -> None: + """Create/declare topic on server.""" await self._thread.create_topic( topic, partitions, @@ -157,6 +161,7 @@ def _to_message(self, tp: TP, record: Any) -> ConsumerMessage: ) async def on_stop(self) -> None: + """Call when consumer is stopping.""" await super().on_stop() transport = cast(Transport, self.transport) transport._topic_waiters.clear() @@ -165,16 +170,18 @@ async def on_stop(self) -> None: class AIOKafkaConsumerThread(ConsumerThread): _consumer: Optional[aiokafka.AIOKafkaConsumer] = None - def on_init(self) -> None: + def __post_init__(self) -> None: self._partitioner: PartitionerT = ( self.app.conf.producer_partitioner or DefaultPartitioner()) self._rebalance_listener = self.consumer.RebalanceListener(self) async def on_start(self) -> None: + """Call when consumer starts.""" self._consumer = self._create_consumer(loop=self.thread_loop) await self._consumer.start() async def on_thread_stop(self) -> None: + """Call when consumer thread is stopping.""" if self._consumer is not None: await self._consumer.stop() @@ -239,11 +246,13 @@ def _create_client_consumer( ) def close(self) -> None: + """Close consumer for graceful shutdown.""" if self._consumer is not None: self._consumer.set_close() self._consumer._coordinator.set_close() async def subscribe(self, topics: Iterable[str]) -> None: + """Reset subscription (requires rebalance).""" # XXX pattern does not work :/ await self.call_thread( self._ensure_consumer().subscribe, @@ -252,10 +261,12 @@ async def subscribe(self, topics: Iterable[str]) -> None: ) async def seek_to_committed(self) -> Mapping[TP, int]: + """Seek partitions to the last committed offset.""" return await self.call_thread( self._ensure_consumer().seek_to_committed) async def commit(self, offsets: Mapping[TP, int]) -> bool: + """Commit topic offsets.""" return await self.call_thread(self._commit, offsets) async def _commit(self, offsets: Mapping[TP, int]) -> bool: @@ -281,14 +292,17 @@ async def _commit(self, offsets: Mapping[TP, int]) -> bool: return True async def position(self, tp: TP) -> Optional[int]: + """Return the current position for topic partition.""" return await self.call_thread( self._ensure_consumer().position, tp) async def seek_to_beginning(self, *partitions: _TopicPartition) -> None: + """Seek list of offsets to the first available offset.""" await self.call_thread( self._ensure_consumer().seek_to_beginning, *partitions) async def seek_wait(self, partitions: Mapping[TP, int]) -> None: + """Seek partitions to specific offset and wait for operation.""" consumer = self._ensure_consumer() await self.call_thread(self._seek_wait, consumer, partitions) @@ -305,28 +319,34 @@ async def _seek_wait(self, ]) def seek(self, partition: TP, offset: int) -> None: + """Seek partition to specific offset.""" self._ensure_consumer().seek(partition, offset) def assignment(self) -> Set[TP]: + """Return the current assignment.""" return ensure_TPset(self._ensure_consumer().assignment()) def highwater(self, tp: TP) -> int: + """Return the last offset in a specific partition.""" if self.consumer.in_transaction: return self._ensure_consumer().last_stable_offset(tp) else: return self._ensure_consumer().highwater(tp) def topic_partitions(self, topic: str) -> Optional[int]: + """Return the number of partitions configured for topic by name.""" if self._consumer is not None: return self._consumer._coordinator._metadata_snapshot.get(topic) return None async def earliest_offsets(self, *partitions: TP) -> Mapping[TP, int]: + """Return the earliest offsets for a list of partitions.""" return await self.call_thread( self._ensure_consumer().beginning_offsets, partitions) async def highwaters(self, *partitions: TP) -> Mapping[TP, int]: + """Return the last offsets for a list of partitions.""" return await self.call_thread(self._highwaters, partitions) async def _highwaters(self, partitions: List[TP]) -> Mapping[TP, int]: @@ -348,6 +368,7 @@ def _ensure_consumer(self) -> aiokafka.AIOKafkaConsumer: async def getmany(self, active_partitions: Set[TP], timeout: float) -> RecordMap: + """Fetch batch of messages from server.""" # Implementation for the Fetcher service. _consumer = self._ensure_consumer() # NOTE: Since we are enqueing the fetch request, @@ -390,6 +411,7 @@ async def create_topic(self, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) -> None: + """Create/declare topic on server.""" transport = cast(Transport, self.consumer.transport) _consumer = self._ensure_consumer() _retention = (int(want_seconds(retention) * 1000.0) @@ -413,6 +435,7 @@ def key_partition(self, topic: str, key: Optional[bytes], partition: int = None) -> int: + """Hash key to determine partition destination.""" consumer = self._ensure_consumer() metadata = consumer._client.cluster if partition is not None: @@ -433,7 +456,7 @@ class Producer(base.Producer): _producer: Optional[aiokafka.AIOKafkaProducer] = None - def on_init(self) -> None: + def __post_init__(self) -> None: self._send_on_produce_message = self.app.on_produce_message.send def _settings_default(self) -> Mapping[str, Any]: @@ -458,18 +481,23 @@ def _settings_auth(self) -> Mapping[str, Any]: self.credentials, self.ssl_context) async def begin_transaction(self, transactional_id: str) -> None: + """Begin transaction by id.""" await self._ensure_producer().begin_transaction(transactional_id) async def commit_transaction(self, transactional_id: str) -> None: + """Commit transaction by id.""" await self._ensure_producer().commit_transaction(transactional_id) async def abort_transaction(self, transactional_id: str) -> None: + """Abort and rollback transaction by id.""" await self._ensure_producer().abort_transaction(transactional_id) async def stop_transaction(self, transactional_id: str) -> None: + """Stop transaction by id.""" await self._ensure_producer().stop_transaction(transactional_id) async def maybe_begin_transaction(self, transactional_id: str) -> None: + """Begin transaction (if one does not already exist).""" await self._ensure_producer().maybe_begin_transaction(transactional_id) async def commit_transactions( @@ -477,6 +505,7 @@ async def commit_transactions( tid_to_offset_map: Mapping[str, Mapping[TP, int]], group_id: str, start_new_transaction: bool = True) -> None: + """Commit transactions.""" await self._ensure_producer().commit( tid_to_offset_map, group_id, start_new_transaction=start_new_transaction, @@ -519,6 +548,7 @@ async def create_topic(self, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) -> None: + """Create/declare topic on server.""" _retention = (int(want_seconds(retention) * 1000.0) if retention else None) producer = self._ensure_producer() @@ -542,12 +572,14 @@ def _ensure_producer(self) -> aiokafka.BaseProducer: return self._producer async def on_start(self) -> None: + """Call when producer starts.""" producer = self._producer = self._new_producer() self.beacon.add(producer) self._last_batch = None await producer.start() async def on_stop(self) -> None: + """Call when producer stops.""" cast(Transport, self.transport)._topic_waiters.clear() self._last_batch = None producer, self._producer = self._producer, None @@ -561,6 +593,7 @@ async def send(self, topic: str, key: Optional[bytes], headers: Optional[HeadersArg], *, transactional_id: str = None) -> Awaitable[RecordMetadata]: + """Schedule message to be transmitted by producer.""" producer = self._ensure_producer() if headers is not None and isinstance(headers, Mapping): headers = list(headers.items()) @@ -590,6 +623,7 @@ async def send_and_wait(self, topic: str, key: Optional[bytes], headers: Optional[HeadersArg], *, transactional_id: str = None) -> RecordMetadata: + """Send message and wait for it to be transmitted.""" fut = await self.send( topic, key=key, @@ -602,10 +636,12 @@ async def send_and_wait(self, topic: str, key: Optional[bytes], return await fut async def flush(self) -> None: + """Wait for producer to finish transmitting all buffered messages.""" if self._producer is not None: await self._producer.flush() def key_partition(self, topic: str, key: bytes) -> TP: + """Hash key to determine partition destination.""" producer = self._ensure_producer() partition = producer._partition( topic, @@ -618,6 +654,7 @@ def key_partition(self, topic: str, key: bytes) -> TP: return TP(topic, partition) def supports_headers(self) -> bool: + """Return :const:`True` if message headers are supported.""" producer = self._ensure_producer() client = producer.client if client is None: diff --git a/faust/transport/drivers/confluent.py b/faust/transport/drivers/confluent.py index 105f16dea..0e5da6a2b 100644 --- a/faust/transport/drivers/confluent.py +++ b/faust/transport/drivers/confluent.py @@ -80,6 +80,7 @@ async def create_topic(self, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) -> None: + """Create topic on broker.""" return # XXX await self._thread.create_topic( topic, @@ -443,7 +444,7 @@ class Producer(base.Producer): _producer_thread: ProducerThread _quick_produce: Any = None - def on_init(self) -> None: + def __post_init__(self) -> None: self._producer_thread = ProducerThread( self, loop=self.loop, beacon=self.beacon) self._quick_produce = self._producer_thread.produce @@ -455,6 +456,7 @@ async def _on_irrecoverable_error(self, exc: BaseException) -> None: await self.crash(exc) async def on_restart(self) -> None: + """Call when producer is restarting.""" self.on_init() async def create_topic(self, @@ -468,6 +470,7 @@ async def create_topic(self, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) -> None: + """Create topic on broker.""" return # XXX _retention = (int(want_seconds(retention) * 1000.0) if retention else None) @@ -486,11 +489,13 @@ async def create_topic(self, ) async def on_start(self) -> None: + """Call when producer is starting.""" await self._producer_thread.start() await self.sleep(0.5) # cannot remember why, necessary? [ask] self._last_batch = None async def on_stop(self) -> None: + """Call when producer is stopping.""" self._last_batch = None await self._producer_thread.stop() @@ -501,6 +506,7 @@ async def send(self, topic: str, key: Optional[bytes], headers: Optional[HeadersArg], *, transactional_id: str = None) -> Awaitable[RecordMetadata]: + """Send message for future delivery.""" fut = ProducerProduceFuture(loop=self.loop) self._quick_produce( topic, value, key, partition, @@ -521,15 +527,22 @@ async def send_and_wait(self, topic: str, key: Optional[bytes], headers: Optional[HeadersArg], *, transactional_id: str = None) -> RecordMetadata: + """Send message and wait for it to be delivered to broker(s).""" fut = await self.send( topic, key, value, partition, timestamp, headers, ) return await fut async def flush(self) -> None: + """Flush producer buffer. + + This will wait until the producer has written + all buffered up messages to any connected brokers. + """ await self._producer_thread.flush() def key_partition(self, topic: str, key: bytes) -> TP: + """Return topic and partition destination for key.""" raise NotImplementedError() diff --git a/faust/transport/drivers/memory.py b/faust/transport/drivers/memory.py index 0a9c39c23..919fff772 100644 --- a/faust/transport/drivers/memory.py +++ b/faust/transport/drivers/memory.py @@ -55,13 +55,16 @@ async def create_topic(self, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) -> None: + """Create/declare topic.""" ... async def subscribe(self, topics: Iterable[str]) -> None: + """Subscribe to topics.""" await cast(Transport, self.transport).subscribe(topics) async def getmany(self, timeout: float) -> AsyncIterator[Tuple[TP, Message]]: + """Fetch batch of messages.""" transport = cast(Transport, self.transport) max_per_partition = 100 partitions = tuple(self.assignment()) @@ -88,43 +91,54 @@ def _new_topicpartition(self, topic: str, partition: int) -> TP: return TP(topic, partition) async def perform_seek(self) -> None: + """Seek all partitions to their current committed position.""" ... async def _commit(self, offsets: Mapping[TP, int]) -> bool: return True def pause_partitions(self, tps: Iterable[TP]) -> None: + """Pause fetching from partitions.""" ... async def position(self, tp: TP) -> Optional[int]: + """Return the current offset position for partition.""" return 0 def resume_partitions(self, partitions: Iterable[TP]) -> None: + """Resume fetching from partitions.""" ... async def seek_to_latest(self, *partitions: TP) -> None: + """Seek partition to last available offset.""" ... async def seek_to_beginning(self, *partitions: TP) -> None: + """Seek partition to first available offset.""" ... async def seek(self, partition: TP, offset: int) -> None: + """Seek partition to particular offset.""" ... def assignment(self) -> Set[TP]: + """Return the current assignment.""" return { TP(t, 0) for t in cast(Transport, self.transport)._subscription } def highwater(self, tp: TP) -> int: + """Return the last offset in a partition.""" return 0 async def earliest_offsets(self, *partitions: TP) -> MutableMapping[TP, int]: + """Return the earliest available offsets for a list of partitions.""" return {tp: 0 for tp in partitions} async def highwaters(self, *partitions: TP) -> MutableMapping[TP, int]: + """Return highwater (last offset) for a list of partitions.""" return {tp: 0 for tp in partitions} @@ -142,6 +156,7 @@ async def create_topic(self, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) -> None: + """Create topic on broker.""" ... async def send(self, topic: str, key: Optional[bytes], @@ -151,6 +166,7 @@ async def send(self, topic: str, key: Optional[bytes], headers: Optional[HeadersArg], *, transactional_id: str = None) -> Awaitable[RecordMetadata]: + """Schedule message to be sent.""" res = await self.send_and_wait( topic, key, value, partition, timestamp, headers) return cast(Awaitable[RecordMetadata], done_future(res)) @@ -162,6 +178,7 @@ async def send_and_wait(self, topic: str, key: Optional[bytes], headers: Optional[HeadersArg], *, transactional_id: str = None) -> RecordMetadata: + """Send message and wait for it to be fully delivered.""" return await cast(Transport, self.transport).send( topic, value, key, partition, timestamp, headers) @@ -188,6 +205,7 @@ def __init__(self, *args: Any, **kwargs: Any) -> None: self._subscription_ready = asyncio.Event(loop=self.loop) async def subscribe(self, topics: Iterable[str]) -> None: + """Subscribe to new set of topics.""" self._subscription_ready.clear() self._subscription.clear() self._subscription.update(topics) @@ -198,6 +216,7 @@ async def send(self, topic: str, key: Optional[bytes], partition: Optional[int], timestamp: Optional[float], headers: Optional[HeadersArg]) -> RecordMetadata: + """Deliver message to consumer.""" if partition is None: partition = 0 message = Message( diff --git a/faust/transport/producer.py b/faust/transport/producer.py index fa90f0f1f..83c6b25f2 100644 --- a/faust/transport/producer.py +++ b/faust/transport/producer.py @@ -46,6 +46,7 @@ async def send(self, topic: str, key: Optional[bytes], headers: Optional[HeadersArg], *, transactional_id: str = None) -> Awaitable[RecordMetadata]: + """Schedule message to be sent by producer.""" raise NotImplementedError() async def send_and_wait(self, topic: str, key: Optional[bytes], @@ -55,9 +56,11 @@ async def send_and_wait(self, topic: str, key: Optional[bytes], headers: Optional[HeadersArg], *, transactional_id: str = None) -> RecordMetadata: + """Send message and wait for it to be transmitted.""" raise NotImplementedError() async def flush(self) -> None: + """Flush all in-flight messages.""" ... async def create_topic(self, @@ -71,24 +74,31 @@ async def create_topic(self, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) -> None: + """Create/declare topic on server.""" raise NotImplementedError() def key_partition(self, topic: str, key: bytes) -> TP: + """Hash key to determine partition.""" raise NotImplementedError() async def begin_transaction(self, transactional_id: str) -> None: + """Begin transaction by id.""" raise NotImplementedError() async def commit_transaction(self, transactional_id: str) -> None: + """Commit transaction by id.""" raise NotImplementedError() async def abort_transaction(self, transactional_id: str) -> None: + """Abort and rollback transaction by id.""" raise NotImplementedError() async def stop_transaction(self, transactional_id: str) -> None: + """Stop transaction by id.""" raise NotImplementedError() async def maybe_begin_transaction(self, transactional_id: str) -> None: + """Begin transaction by id, if not already started.""" raise NotImplementedError() async def commit_transactions( @@ -96,7 +106,9 @@ async def commit_transactions( tid_to_offset_map: Mapping[str, Mapping[TP, int]], group_id: str, start_new_transaction: bool = True) -> None: + """Commit transactions.""" raise NotImplementedError() def supports_headers(self) -> bool: + """Return :const:`True` if headers are supported by this transport.""" return False diff --git a/faust/transport/utils.py b/faust/transport/utils.py index c73c8bfe6..2089020e4 100644 --- a/faust/transport/utils.py +++ b/faust/transport/utils.py @@ -36,6 +36,7 @@ class DefaultSchedulingStrategy(SchedulingStrategyT): @classmethod def map_from_records(cls, records: Mapping[TP, List]) -> TopicIndexMap: + """Convert records to topic index map.""" topic_index: TopicIndexMap = {} for tp, messages in records.items(): try: @@ -46,10 +47,12 @@ def map_from_records(cls, records: Mapping[TP, List]) -> TopicIndexMap: return topic_index def iterate(self, records: Mapping[TP, List]) -> Iterator[Tuple[TP, Any]]: + """Iterate over records in round-robin order.""" return self.records_iterator(self.map_from_records(records)) def records_iterator(self, index: TopicIndexMap) -> Iterator[Tuple[TP, Any]]: + """Iterate over topic index map in round-robin order.""" to_remove: Set[str] = set() sentinel = object() _next = next @@ -83,6 +86,7 @@ def __init__(self) -> None: self._it = None def add(self, tp: TP, buffer: List) -> None: + """Add topic partition buffer to the cycle.""" assert tp not in self._buffers self._buffers[tp] = iter(buffer) diff --git a/faust/utils/functional.py b/faust/utils/functional.py index 26c0238b8..205760ae2 100644 --- a/faust/utils/functional.py +++ b/faust/utils/functional.py @@ -28,6 +28,6 @@ def deque_prune(l: Deque[T], max: int = None) -> Optional[T]: def deque_pushpopmax(l: Deque[T], item: T, max: int = None) -> Optional[T]: - """Append to deque and remove oldest element if size exceeeds ``max``.""" + """Append to deque and remove oldest element if size exceeds ``max``.""" l.append(item) return deque_prune(l, max) diff --git a/faust/utils/json.py b/faust/utils/json.py index 7e6f889a5..38b40f2ab 100644 --- a/faust/utils/json.py +++ b/faust/utils/json.py @@ -147,6 +147,7 @@ class JSONEncoder(json.JSONEncoder): def default(self, o: Any, *, callback: Callable[[Any], Any] = on_default) -> Any: + """Try to convert non-built-in json type to json.""" return callback(o) diff --git a/faust/utils/kafka/__init__.py b/faust/utils/kafka/__init__.py index e69de29bb..6d83e4241 100644 --- a/faust/utils/kafka/__init__.py +++ b/faust/utils/kafka/__init__.py @@ -0,0 +1 @@ +"""Kafka utilities.""" diff --git a/faust/utils/kafka/protocol/__init__.py b/faust/utils/kafka/protocol/__init__.py index e69de29bb..560567092 100644 --- a/faust/utils/kafka/protocol/__init__.py +++ b/faust/utils/kafka/protocol/__init__.py @@ -0,0 +1 @@ +"""Kafka protocol definitions.""" diff --git a/faust/utils/terminal/__init__.py b/faust/utils/terminal/__init__.py index c993007b5..777c86938 100644 --- a/faust/utils/terminal/__init__.py +++ b/faust/utils/terminal/__init__.py @@ -1,3 +1,4 @@ +"""Terminal utilities.""" from mode.utils.compat import isatty from .spinners import Spinner, SpinnerHandler from .tables import Table, TableDataT, logtable, table diff --git a/faust/utils/terminal/spinners.py b/faust/utils/terminal/spinners.py index 9da9d32c5..d12deb2ae 100644 --- a/faust/utils/terminal/spinners.py +++ b/faust/utils/terminal/spinners.py @@ -51,6 +51,7 @@ def __init__(self, file: IO = sys.stderr) -> None: self.stopped = False def update(self) -> None: + """Draw spinner, single iteration.""" if not self.stopped: if not self.count: self.begin() @@ -59,13 +60,16 @@ def update(self) -> None: self.write(self.sprites[i]) def stop(self) -> None: + """Stop spinner from being emitted.""" self.stopped = True def reset(self) -> None: + """Reset state or allow restart.""" self.stopped = False self.count = 0 def write(self, s: str) -> None: + """Write spinner character to terminal.""" if self.file.isatty(): self._print(f'{self.bell * self.width}{s.ljust(self.width)}') self.width = max(self.width, len(s)) @@ -75,10 +79,12 @@ def _print(self, s: str) -> None: self.file.flush() def begin(self) -> None: + """Prepare terminal for spinner starting.""" atexit.register(type(self)._finish, self.file, at_exit=True) self._print(self.cursor_hide) def finish(self) -> None: + """Finish spinner and reset terminal.""" print(f'{self.bell * (self.width + 1)}', end='', file=self.file) self._finish(self.file) self.stop() @@ -101,6 +107,7 @@ def __init__(self, spinner: Spinner, **kwargs: Any) -> None: super().__init__(**kwargs) def emit(self, _record: logging.LogRecord) -> None: + """Emit the next spinner character.""" # the spinner is only in effect with WARN level and below. if self.spinner and not self.spinner.stopped: self.spinner.update() diff --git a/faust/web/__init__.py b/faust/web/__init__.py index b0390a836..9edea6dae 100644 --- a/faust/web/__init__.py +++ b/faust/web/__init__.py @@ -1,3 +1,4 @@ +"""Faust Web.""" from .base import Request, Response, Web from .blueprints import Blueprint from .views import View, gives_model, takes_model diff --git a/faust/web/apps/__init__.py b/faust/web/apps/__init__.py index e69de29bb..301324b4a 100644 --- a/faust/web/apps/__init__.py +++ b/faust/web/apps/__init__.py @@ -0,0 +1 @@ +"""Built-in Web Apps.""" diff --git a/faust/web/apps/graph.py b/faust/web/apps/graph.py index 500b03d95..d46ab469d 100644 --- a/faust/web/apps/graph.py +++ b/faust/web/apps/graph.py @@ -12,6 +12,7 @@ class Graph(web.View): """Render image from graph of running services.""" async def get(self, request: web.Request) -> web.Response: + """Draw image of the services running in this worker.""" try: import pydot except ImportError: diff --git a/faust/web/apps/router.py b/faust/web/apps/router.py index 8f0aec329..7ddf7f28e 100644 --- a/faust/web/apps/router.py +++ b/faust/web/apps/router.py @@ -12,6 +12,7 @@ class TableList(web.View): """List routes for all tables.""" async def get(self, request: web.Request) -> web.Response: + """Return JSON response with list of all table routes.""" router = self.app.router return self.json(router.tables_metadata()) @@ -21,6 +22,7 @@ class TableDetail(web.View): """List route for specific table.""" async def get(self, request: web.Request, name: str) -> web.Response: + """Return JSON response with table metadata.""" router = self.app.router return self.json(router.table_metadata(name)) @@ -33,6 +35,16 @@ async def get(self, request: web.Request, name: str, key: str) -> web.Response: + """Return JSON response after looking up the route of a table key. + + Arguments: + name: Name of table. + key: Key to look up node for. + + Raises: + faust.web.exceptions.ServiceUnavailable: if broker metadata + has not yet been received. + """ router = self.app.router try: dest_url = router.key_store(name, key) diff --git a/faust/web/apps/stats.py b/faust/web/apps/stats.py index c00e9ee3f..e0833a088 100644 --- a/faust/web/apps/stats.py +++ b/faust/web/apps/stats.py @@ -17,6 +17,7 @@ class Stats(web.View): """Monitor statistics.""" async def get(self, request: web.Request) -> web.Response: + """Return JSON response with sensor information.""" return self.json( {f'Sensor{i}': s.asdict() for i, s in enumerate(self.app.sensors)}) @@ -34,6 +35,7 @@ def _topic_grouped(cls, assignment: Set[TP]) -> TPMap: return dict(tps) async def get(self, request: web.Request) -> web.Response: + """Return current assignment as a JSON response.""" assignor = self.app.assignor return self.json({ 'actives': self._topic_grouped(assignor.assigned_actives()), diff --git a/faust/web/apps/tables.py b/faust/web/apps/tables.py index 528f5ff3c..8ac7cff1f 100644 --- a/faust/web/apps/tables.py +++ b/faust/web/apps/tables.py @@ -26,15 +26,18 @@ class TableView(web.View): """Base class for table related views.""" def table_json(self, table: TableT, **kwargs: Any) -> Mapping: + """Return table info as JSON serializable object.""" return TableInfo(table.name, table.help).asdict() def get_table_or_404(self, name: str) -> TableT: + """Find table by name, or raise NotFound if not found.""" try: return self.app.tables[name] except KeyError: raise self.NotFound('unknown table', name=name) def get_table_value_or_404(self, table: TableT, key: K) -> V: + """Get value from table by key, or raise NotFound if not found.""" try: return table[key] except KeyError: @@ -46,6 +49,7 @@ class TableList(TableView): """List available table names.""" async def get(self, request: web.Request) -> web.Response: + """Return JSON response with a list of available table names.""" return self.json( [self.table_json(table) for table in self.app.tables.values()]) @@ -55,6 +59,7 @@ class TableDetail(TableView): """Get details for table by name.""" async def get(self, request: web.Request, name: str) -> web.Response: + """Return JSON response with table information.""" table = self.get_table_or_404(name) return self.json(self.table_json(table)) @@ -70,6 +75,7 @@ async def get(self, request: web.Request, name: str, key: str) -> web.Response: + """Look up value in table by key.""" router = self.app.router try: return await router.route_req(name, key, self.web, request) diff --git a/faust/web/base.py b/faust/web/base.py index cb72dd86d..f2ee7acab 100644 --- a/faust/web/base.py +++ b/faust/web/base.py @@ -55,51 +55,61 @@ class Response: @property @abc.abstractmethod def status(self) -> int: + """Return the response status code.""" ... @property @abc.abstractmethod def body(self) -> _bytes: + """Return the response body as bytes.""" ... @property @abc.abstractmethod def headers(self) -> MutableMapping: + """Return mapping of response HTTP headers.""" ... @property @abc.abstractmethod def content_length(self) -> Optional[int]: + """Return the size of the response body.""" ... @property @abc.abstractmethod def content_type(self) -> str: + """Return the response content type.""" ... @property @abc.abstractmethod def charset(self) -> Optional[str]: + """Return the response character set.""" ... @property @abc.abstractmethod def chunked(self) -> bool: + """Return :const:`True` if response is chunked.""" ... @property @abc.abstractmethod def compression(self) -> bool: + """Return :const:`True` if the response body is compressed.""" ... @property @abc.abstractmethod def keep_alive(self) -> Optional[bool]: + """Return :const:`True` if HTTP keep-alive enabled.""" ... @property @abc.abstractmethod def body_length(self) -> int: + """Size of HTTP response body.""" ... @@ -117,11 +127,13 @@ def __init__(self, initial: _BPList = None) -> None: self._active = {} def add(self, prefix: str, blueprint: SymbolArg[Type[BlueprintT]]) -> None: + """Register blueprint with this app.""" if self.applied: raise RuntimeError('Cannot add blueprints after server started') self._enabled.append((prefix, blueprint)) def apply(self, web: 'Web') -> None: + """Apply all blueprints.""" if not self.applied: self.applied = True for prefix, blueprint in self._enabled: @@ -167,6 +179,7 @@ def text(self, value: str, *, status: int = 200, reason: str = None, headers: MutableMapping = None) -> Response: + """Create text response, using "text/plain" content-type.""" ... @abc.abstractmethod @@ -175,6 +188,7 @@ def html(self, value: str, *, status: int = 200, reason: str = None, headers: MutableMapping = None) -> Response: + """Create HTML response from string, ``text/html`` content-type.""" ... @abc.abstractmethod @@ -183,6 +197,13 @@ def json(self, value: Any, *, status: int = 200, reason: str = None, headers: MutableMapping = None) -> Response: + """Create new JSON response. + + Accepts any JSON-serializable value and will automatically + serialize it for you. + + The content-type is set to "application/json". + """ ... @abc.abstractmethod @@ -193,10 +214,12 @@ def bytes(self, status: int = 200, reason: str = None, headers: MutableMapping = None) -> Response: + """Create new ``bytes`` response - for binary data.""" ... @abc.abstractmethod def bytes_to_response(self, s: _bytes) -> Response: + """Deserialize HTTP response from byte string.""" ... def _bytes_to_response( @@ -216,6 +239,7 @@ def _splitheader(self, header: _bytes) -> Tuple[str, str]: @abc.abstractmethod def response_to_bytes(self, response: Response) -> _bytes: + """Serialize HTTP response into byte string.""" ... def _response_to_bytes( @@ -242,6 +266,7 @@ def route(self, pattern: str, handler: Callable, cors_options: Mapping[str, ResourceOptions] = None) -> None: + """Add route for handler.""" ... @abc.abstractmethod @@ -249,19 +274,23 @@ def add_static(self, prefix: str, path: Union[Path, str], **kwargs: Any) -> None: + """Add static route.""" ... @abc.abstractmethod async def read_request_content(self, request: 'Request') -> _bytes: + """Read HTTP body as bytes.""" ... @abc.abstractmethod async def wsgi(self) -> Any: + """WSGI entry point.""" ... def add_view(self, view_cls: Type[View], *, prefix: str = '', cors_options: Mapping[str, ResourceOptions] = None) -> View: + """Add route for view.""" view: View = view_cls(self.app, self) path = prefix.rstrip('/') + '/' + view.view_path.lstrip('/') self.route(path, view, cors_options) @@ -288,11 +317,13 @@ def _quote_for_url(self, value: str) -> str: return quote(value, safe='') # disable '/' being safe by default def init_server(self) -> None: + """Initialize and setup web server.""" self.blueprints.apply(self) self.app.on_webserver_init(self) @property def url(self) -> URL: + """Return the canonical URL to this worker (including port).""" canon = self.app.conf.canonical_url if canon.host == socket.gethostname(): return URL(f'http://localhost:{self.app.conf.web_port}/') @@ -317,35 +348,43 @@ class Request(abc.ABC): @abc.abstractmethod def can_read_body(self) -> bool: + """Return :const:`True` if the request has a body.""" ... @abc.abstractmethod async def read(self) -> bytes: + """Read post data as bytes.""" ... @abc.abstractmethod async def text(self) -> str: + """Read post data as text.""" ... @abc.abstractmethod async def json(self) -> Any: + """Read post data and deserialize as JSON.""" ... @abc.abstractmethod async def post(self) -> Mapping[str, str]: + """Read post data.""" ... @property @abc.abstractmethod def match_info(self) -> Mapping[str, str]: + """Return match info from URL route as a mapping.""" ... @property @abc.abstractmethod def query(self) -> Mapping[str, str]: + """Return HTTP query parameters as a mapping.""" ... @property @abc.abstractmethod def cookies(self) -> Mapping[str, Any]: + """Return cookies as a mapping.""" ... diff --git a/faust/web/blueprints.py b/faust/web/blueprints.py index 272c6322d..6c87589a0 100644 --- a/faust/web/blueprints.py +++ b/faust/web/blueprints.py @@ -112,6 +112,7 @@ def cache(self, timeout: Seconds = None, key_prefix: str = None, backend: Union[Type[CacheBackendT], str] = None) -> CacheT: + """Cache API.""" if key_prefix is None: key_prefix = self.name return Cache(timeout, key_prefix, backend) @@ -122,6 +123,7 @@ def route(self, name: Optional[str] = None, cors_options: Mapping[str, ResourceOptions] = None, base: Type[View] = View) -> RouteDecoratorRet: + """Create route by decorating handler or view class.""" def _inner(handler: PageArg) -> PageArg: route = FutureRoute( uri=uri, @@ -139,6 +141,7 @@ def static(self, file_or_directory: Union[str, Path], *, name: Optional[str] = None) -> None: + """Add static route.""" _name: str = name or 'static' if not _name.startswith(self.name + '.'): _name = f'{self.name}.{name}' @@ -148,6 +151,7 @@ def static(self, def register(self, app: AppT, *, url_prefix: Optional[str] = None) -> None: + """Register blueprint with app.""" url_prefix = url_prefix or self.url_prefix # Apply routes @@ -175,9 +179,11 @@ def _view_name(self, name: str) -> str: return self.view_name_separator.join([self.name, name]) def init_webserver(self, web: Web) -> None: + """Init blueprint for web server start.""" self.on_webserver_init(web) def on_webserver_init(self, web: Web) -> None: + """Call when web server starts.""" ... def _url_with_prefix(self, url: str, prefix: str = None) -> str: diff --git a/faust/web/cache/__init__.py b/faust/web/cache/__init__.py index 8cee6b577..8ddb465b5 100644 --- a/faust/web/cache/__init__.py +++ b/faust/web/cache/__init__.py @@ -1,3 +1,4 @@ +"""Caching.""" from .cache import Cache __all__ = ['Cache'] diff --git a/faust/web/cache/backends/base.py b/faust/web/cache/backends/base.py index 2db1baa55..f5cef2e5f 100644 --- a/faust/web/cache/backends/base.py +++ b/faust/web/cache/backends/base.py @@ -52,15 +52,18 @@ async def _delete(self, key: str) -> None: ... async def get(self, key: str) -> Optional[bytes]: + """Get cached-value by key.""" async with self._recovery_context(key): return await self._get(key) async def set(self, key: str, value: bytes, timeout: float) -> None: + """Set cached-value by key.""" assert timeout is not None async with self._recovery_context(key): await self._set(key, value, timeout) async def delete(self, key: str) -> None: + """Forget value for cache key.""" async with self._recovery_context(key): await self._delete(key) diff --git a/faust/web/cache/backends/memory.py b/faust/web/cache/backends/memory.py index 3e338d6c5..21277461b 100644 --- a/faust/web/cache/backends/memory.py +++ b/faust/web/cache/backends/memory.py @@ -27,6 +27,7 @@ def __init__(self) -> None: self._expires: Dict[KT, float] = {} def get(self, key: KT) -> Optional[VT]: + """Get value for key, or :const:`None` if missing.""" with suppress(KeyError): expires = self._expires[key] now = TIME_MONOTONIC() @@ -39,20 +40,25 @@ def get(self, key: KT) -> Optional[VT]: return self._data[key] def last_set_ttl(self, key: KT) -> Optional[float]: + """Return the last set TTL for key, or :const:`None` if missing.""" return self._expires.get(key) def expire(self, key: KT) -> None: + """Expire value for key immediately.""" self._time_index[key] -= self._expires[key] def set(self, key: KT, value: VT) -> None: + """Set value for key.""" self._data[key] = value def setex(self, key: KT, timeout: float, value: VT) -> None: + """Set value & set timeout for key.""" self._expires[key] = timeout self._time_index[key] = TIME_MONOTONIC() self.set(key, value) def ttl(self, key: KT) -> Optional[float]: + """Return the remaining TTL for key.""" try: return ( self._expires[key] - TIME_MONOTONIC() - self._time_index[key]) @@ -60,11 +66,13 @@ def ttl(self, key: KT) -> Optional[float]: return None def delete(self, key: KT) -> None: + """Delete value for key.""" self._expires.pop(key, None) self._data.pop(key, None) # type: ignore self._time_index.pop(key, None) def clear(self) -> None: + """Clear all data.""" self._expires.clear() self._data.clear() self._time_index.clear() @@ -73,7 +81,7 @@ def clear(self) -> None: class CacheBackend(base.CacheBackend): """In-memory backend for cache operations.""" - def on_init(self) -> None: + def __post_init__(self) -> None: # we reuse this in t/conftest to mock a Redis server :D self.storage: CacheStorage[str, bytes] = CacheStorage() diff --git a/faust/web/cache/backends/redis.py b/faust/web/cache/backends/redis.py index 536c3a060..02f6cbf7e 100644 --- a/faust/web/cache/backends/redis.py +++ b/faust/web/cache/backends/redis.py @@ -99,12 +99,14 @@ async def _delete(self, key: str) -> None: await self.client.delete(key) async def on_start(self) -> None: + """Call when Redis backend starts.""" if aredis is None: raise ImproperlyConfigured( 'Redis cache backend requires `pip install aredis`') await self.connect() async def connect(self) -> None: + """Connect to Redis/Redis Cluster server.""" if self._client is None: self._client = self._new_client() await self.client.ping() @@ -171,6 +173,7 @@ def _db_from_path(self, path: str) -> int: @cached_property def client(self) -> _RedisClientT: + """Return Redis client instance.""" if self._client is None: raise RuntimeError('Cache backend not started') return self._client diff --git a/faust/web/cache/cache.py b/faust/web/cache/cache.py index 281c47fb0..2aed80c99 100644 --- a/faust/web/cache/cache.py +++ b/faust/web/cache/cache.py @@ -79,6 +79,7 @@ async def cached(view: View, request: Request, async def get_view(self, key: str, view: View) -> Optional[Response]: + """Get cached value for HTTP view request.""" backend = self._view_backend(view) with suppress(backend.Unavailable): payload = await backend.get(key) @@ -94,6 +95,7 @@ async def set_view(self, view: View, response: Response, timeout: Seconds) -> None: + """Set cached value for HTTP view request.""" backend = self._view_backend(view) with suppress(backend.Unavailable): return await backend.set( @@ -103,17 +105,20 @@ async def set_view(self, ) def can_cache_request(self, request: Request) -> bool: + """Return :const:`True` if we can cache this type of HTTP request.""" return True def can_cache_response(self, request: Request, response: Response) -> bool: + """Return :const:`True` for HTTP status codes we CAN cache.""" return response.status == 200 def key_for_request(self, request: Request, prefix: str = None, method: str = None) -> str: + """Return a cache key created from web request.""" actual_method: str = request.method if method is None else method if prefix is None: prefix = self.key_prefix @@ -124,6 +129,7 @@ def build_key(self, method: str, prefix: str, headers: List[str]) -> str: + """Build cache key from web request and environment.""" context = hashlib.md5( b''.join(v.encode() for v in headers if v is not None)).hexdigest() url = hashlib.md5( diff --git a/faust/web/drivers/aiohttp.py b/faust/web/drivers/aiohttp.py index e176cf562..d0ea7902d 100644 --- a/faust/web/drivers/aiohttp.py +++ b/faust/web/drivers/aiohttp.py @@ -49,29 +49,35 @@ def _faust_to_aiohttp_options( class ServerThread(ServiceThread): + """A web server running in a dedicated thread.""" def __init__(self, web: 'Web', **kwargs: Any) -> None: self.web = web super().__init__(**kwargs) async def on_start(self) -> None: + """Call in parent thread when the service thread is starting.""" await self.web.start_server() async def on_thread_stop(self) -> None: + """Call in thread when the service stops.""" # on_stop() executes in parent thread, on_thread_stop in the thread. await self.web.stop_server() class Server(Service): + """Web server service.""" def __init__(self, web: 'Web', **kwargs: Any) -> None: self.web = web super().__init__(**kwargs) async def on_start(self) -> None: + """Call when the web server starts.""" await self.web.start_server() async def on_stop(self) -> None: + """Call when the web server stops.""" await self.web.stop_server() @@ -101,12 +107,17 @@ def __init__(self, app: AppT, **kwargs: Any) -> None: @property def cors(self) -> CorsConfig: + """Return CORS config object.""" if self._cors is None: self._cors = aiohttp_cors.setup( self.web_app, defaults=self.cors_options) return self._cors async def on_start(self) -> None: + """Call when the embedded web server starts. + + Only used for `faust worker`, not when using :meth:`wsgi`. + """ cors = self.cors assert cors self.init_server() @@ -115,6 +126,11 @@ async def on_start(self) -> None: self.add_dependency(self._thread) async def wsgi(self) -> Any: + """Call WSGI handler. + + Used by :pypi:`gunicorn` and other WSGI compatible hosts + to access the Faust web entry point. + """ self.init_server() return self.web_app @@ -123,6 +139,7 @@ def text(self, value: str, *, status: int = 200, reason: str = None, headers: MutableMapping = None) -> base.Response: + """Create text response, using "text/plain" content-type.""" response = Response( text=value, content_type=content_type, @@ -137,6 +154,7 @@ def html(self, value: str, *, status: int = 200, reason: str = None, headers: MutableMapping = None) -> base.Response: + """Create HTML response from string, ``text/html`` content-type.""" return self.text( value, status=status, @@ -150,6 +168,13 @@ def json(self, value: Any, *, status: int = 200, reason: str = None, headers: MutableMapping = None) -> Any: + """Create new JSON response. + + Accepts any JSON-serializable value and will automatically + serialize it for you. + + The content-type is set to "application/json". + """ ctype = content_type or 'application/json' payload: Any = _json.dumps(value) # normal json returns str, orjson returns bytes @@ -177,6 +202,7 @@ def bytes(self, status: int = 200, reason: str = None, headers: MutableMapping = None) -> base.Response: + """Create new ``bytes`` response - for binary data.""" response = Response( body=value, content_type=content_type, @@ -187,12 +213,14 @@ def bytes(self, return cast(base.Response, response) async def read_request_content(self, request: base.Request) -> _bytes: + """Return the request body as bytes.""" return await cast(Request, request).content.read() def route(self, pattern: str, handler: Callable, cors_options: Mapping[str, ResourceOptions] = None) -> None: + """Add route for web view or handler.""" if cors_options or self.cors_options: async_handler = self._wrap_into_asyncdef(handler) for method in NON_OPTIONS_METHODS: @@ -217,9 +245,11 @@ def add_static(self, prefix: str, path: Union[Path, str], **kwargs: Any) -> None: + """Add route for static assets.""" self.web_app.router.add_static(prefix, str(path), **kwargs) def bytes_to_response(self, s: _bytes) -> base.Response: + """Deserialize byte string back into a response object.""" status, headers, body = self._bytes_to_response(s) response = Response( body=body, @@ -229,6 +259,11 @@ def bytes_to_response(self, s: _bytes) -> base.Response: return cast(base.Response, response) def response_to_bytes(self, response: base.Response) -> _bytes: + """Convert response to serializable byte string. + + The result is a byte string that can be deserialized + using :meth:`bytes_to_response`. + """ resp = cast(Response, response) return self._response_to_bytes( resp.status, @@ -256,11 +291,13 @@ def _new_transport_unix(self) -> BaseSite: ) async def start_server(self) -> None: + """Start the web server.""" await self._runner.setup() site = self._create_site() await site.start() async def stop_server(self) -> None: + """Stop the web server.""" if self._runner: await self._runner.cleanup() await self._cleanup_app() diff --git a/faust/web/views.py b/faust/web/views.py index fc5fb5e4d..f7c3b9b52 100644 --- a/faust/web/views.py +++ b/faust/web/views.py @@ -43,6 +43,7 @@ class View: @classmethod def from_handler(cls, fun: ViewHandlerFun) -> Type['View']: + """Decorate ``async def`` handler function to create view.""" if not callable(fun): raise TypeError(f'View handler must be callable, not {fun!r}') return type(fun.__name__, (cls,), { @@ -66,12 +67,15 @@ def __init__(self, app: AppT, web: Web) -> None: self.__post_init__() def __post_init__(self) -> None: + """Override this to add custom initialization to your view.""" ... async def __call__(self, request: Any) -> Any: + """Perform HTTP request.""" return await self.dispatch(request) async def dispatch(self, request: Any) -> Any: + """Dispatch the request and perform any callbacks/cleanup.""" method = request.method.lower() kwargs = request.match_info or {} # XXX Aiohttp specific @@ -87,15 +91,26 @@ async def dispatch(self, request: Any) -> Any: async def on_request_error(self, request: Request, exc: WebError) -> Response: + """Call when a request raises an exception.""" return self.error(exc.code, exc.detail, **exc.extra_context) def path_for(self, view_name: str, **kwargs: Any) -> str: + """Return the URL path for view by name. + + Supports match keyword arguments. + """ return self.web.url_for(view_name, **kwargs) def url_for(self, view_name: str, _base_url: Union[str, URL] = None, **kwargs: Any) -> URL: + """Return the canonical URL for view by name. + + Supports match keyword arguments. + Can take optional base name, which if not set will + be the canonical URL of the app. + """ if _base_url is None: _base_url = self.app.conf.canonical_url return URL('/'.join([ @@ -105,30 +120,37 @@ def url_for(self, @no_type_check async def head(self, request: Request, **kwargs: Any) -> Any: + """Override ``head`` to define the HTTP HEAD handler.""" return await self.get(request, **kwargs) @no_type_check # subclasses change signature based on route match_info async def get(self, request: Request, **kwargs: Any) -> Any: + """Override ``get`` to define the HTTP GET handler.""" raise exceptions.MethodNotAllowed('Method GET not allowed.') @no_type_check # subclasses change signature based on route match_info async def post(self, request: Request, **kwargs: Any) -> Any: + """Override ``post`` to define the HTTP POST handler.""" raise exceptions.MethodNotAllowed('Method POST not allowed.') @no_type_check # subclasses change signature based on route match_info async def put(self, request: Request, **kwargs: Any) -> Any: + """Override ``put`` to define the HTTP PUT handler.""" raise exceptions.MethodNotAllowed('Method PUT not allowed.') @no_type_check # subclasses change signature based on route match_info async def patch(self, request: Request, **kwargs: Any) -> Any: + """Override ``patch`` to define the HTTP PATCH handler.""" raise exceptions.MethodNotAllowed('Method PATCH not allowed.') @no_type_check # subclasses change signature based on route match_info async def delete(self, request: Request, **kwargs: Any) -> Any: + """Override ``delete`` to define the HTTP DELETE handler.""" raise exceptions.MethodNotAllowed('Method DELETE not allowed.') @no_type_check # subclasses change signature based on route match_info async def options(self, request: Request, **kwargs: Any) -> Any: + """Override ``options`` to define the HTTP OPTIONS handler.""" raise exceptions.MethodNotAllowed('Method OPTIONS not allowed.') def text(self, value: str, *, @@ -136,6 +158,7 @@ def text(self, value: str, *, status: int = 200, reason: str = None, headers: MutableMapping = None) -> Response: + """Create text response, using "text/plain" content-type.""" return self.web.text( value, content_type=content_type, @@ -149,6 +172,7 @@ def html(self, value: str, *, status: int = 200, reason: str = None, headers: MutableMapping = None) -> Response: + """Create HTML response from string, ``text/html`` content-type.""" return self.web.html( value, content_type=content_type, @@ -162,6 +186,13 @@ def json(self, value: Any, *, status: int = 200, reason: str = None, headers: MutableMapping = None) -> Response: + """Create new JSON response. + + Accepts any JSON-serializable value and will automatically + serialize it for you. + + The content-type is set to "application/json". + """ return self.web.json( value, content_type=content_type, @@ -177,6 +208,7 @@ def bytes(self, status: int = 200, reason: str = None, headers: MutableMapping = None) -> Response: + """Create new ``bytes`` response - for binary data.""" return self.web.bytes( value, content_type=content_type, @@ -186,23 +218,35 @@ def bytes(self, ) async def read_request_content(self, request: Request) -> _bytes: + """Return the request body as bytes.""" return await self.web.read_request_content(request) def bytes_to_response(self, s: _bytes) -> Response: + """Deserialize byte string back into a response object.""" return self.web.bytes_to_response(s) def response_to_bytes(self, response: Response) -> _bytes: + """Convert response to serializable byte string. + + The result is a byte string that can be deserialized + using :meth:`bytes_to_response`. + """ return self.web.response_to_bytes(response) def route(self, pattern: str, handler: Callable) -> Any: + """Create new route from pattern and handler.""" self.web.route(pattern, handler) return handler def notfound(self, reason: str = 'Not Found', **kwargs: Any) -> Response: - # Deprecated: Use raise NotFound() instead. + """Create not found error response. + + Deprecated: Use ``raise self.NotFound()`` instead. + """ return self.error(404, reason, **kwargs) def error(self, status: int, reason: str, **kwargs: Any) -> Response: + """Create error JSON response.""" return self.json({'error': reason, **kwargs}, status=status) diff --git a/setup.cfg b/setup.cfg index c79a62e4c..225a537d6 100644 --- a/setup.cfg +++ b/setup.cfg @@ -33,7 +33,7 @@ docstring-quotes = """ enable-extensions = G [pep257] -ignore = D102,D104,D107,D203,D105,D213,D406,D407,D413 +ignore = D107,D203,D105,D213,D406,D407,D413 [mypy] python_version = 3.6 diff --git a/tox.ini b/tox.ini index 675cd7cab..936b6132a 100644 --- a/tox.ini +++ b/tox.ini @@ -49,7 +49,7 @@ commands = [testenv:docstyle] commands = - pydocstyle --match-dir '(?!types)' faust + pydocstyle --match-dir '(?!types|assignor)' faust [testenv:bandit] commands =